From d977e10130038e2aa0eaea0601ac82f7d3ec1556 Mon Sep 17 00:00:00 2001 From: Blake Gentry Date: Sun, 22 Sep 2024 15:11:36 -0500 Subject: [PATCH] Bulk unique insertion, uniqueness with subset of args (#590) * add state machine mermaid chart * new unique jobs implementation that works on bulk insert * support uniqueness for specific args via struct tags * block bulk inserts using advisory lock uniqueness * handle unique conflicts from scheduler by discarding jobs * add changelog entry --- CHANGELOG.md | 52 + client.go | 90 +- client_test.go | 393 +++++++- docs/state_machine.md | 77 ++ driver_test.go | 23 +- go.mod | 4 + go.sum | 9 + go.work.sum | 1 + insert_opts.go | 81 +- insert_opts_test.go | 33 + internal/dbunique/db_unique.go | 187 +++- internal/dbunique/db_unique_test.go | 951 +++++------------- internal/dbunique/unique_fields.go | 127 +++ internal/maintenance/job_scheduler.go | 12 +- internal/maintenance/job_scheduler_test.go | 55 + internal/maintenance/periodic_job_enqueuer.go | 27 +- .../maintenance/periodic_job_enqueuer_test.go | 73 +- .../riverdrivertest/riverdrivertest.go | 233 ++--- job_executor_test.go | 12 +- periodic_job.go | 2 +- producer_test.go | 4 +- riverdriver/river_driver_interface.go | 85 +- riverdriver/riverdatabasesql/go.mod | 1 + riverdriver/riverdatabasesql/go.sum | 14 + .../internal/dbsqlc/models.go | 37 +- .../internal/dbsqlc/river_job.sql.go | 510 +++++----- .../internal/dbsqlc/sqlc.yaml | 20 + .../internal/pgtypealias/pgtype_alias.go | 9 + .../migration/main/006_bulk_unique.down.sql | 16 + .../migration/main/006_bulk_unique.up.sql | 38 + .../river_database_sql_driver.go | 207 ++-- .../riverpgxv5/internal/dbsqlc/copyfrom.go | 4 +- .../riverpgxv5/internal/dbsqlc/models.go | 37 +- .../riverpgxv5/internal/dbsqlc/river_job.sql | 145 ++- .../internal/dbsqlc/river_job.sql.go | 511 +++++----- .../internal/dbsqlc/river_job_copyfrom.sql | 10 +- .../internal/dbsqlc/river_job_copyfrom.sql.go | 24 +- .../migration/main/006_bulk_unique.down.sql | 16 + .../migration/main/006_bulk_unique.up.sql | 38 + riverdriver/riverpgxv5/river_pgx_v5_driver.go | 204 ++-- rivershared/testfactory/test_factory.go | 62 +- rivertype/river_type.go | 11 + 42 files changed, 2592 insertions(+), 1853 deletions(-) create mode 100644 docs/state_machine.md create mode 100644 internal/dbunique/unique_fields.go create mode 100644 riverdriver/riverdatabasesql/internal/pgtypealias/pgtype_alias.go create mode 100644 riverdriver/riverdatabasesql/migration/main/006_bulk_unique.down.sql create mode 100644 riverdriver/riverdatabasesql/migration/main/006_bulk_unique.up.sql create mode 100644 riverdriver/riverpgxv5/migration/main/006_bulk_unique.down.sql create mode 100644 riverdriver/riverpgxv5/migration/main/006_bulk_unique.up.sql diff --git a/CHANGELOG.md b/CHANGELOG.md index 1edfca1c..ccef70ef 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -7,6 +7,37 @@ and this project adheres to [Semantic Versioning](https://semver.org/spec/v2.0.0 ## [Unreleased] +⚠️ Version 0.12.0 contains a new database migration, version 6. See [documentation on running River migrations](https://riverqueue.com/docs/migrations). If migrating with the CLI, make sure to update it to its latest version: + +```shell +go install github.com/riverqueue/river/cmd/river@latest +river migrate-up --database-url "$DATABASE_URL" +``` + +If not using River's internal migration system, the raw SQL can alternatively be dumped with: + +```shell +go install github.com/riverqueue/river/cmd/river@latest +river migrate-get --version 6 --up > river6.up.sql +river migrate-get --version 6 --down > river6.down.sql +``` + +The migration **includes a new index**. Users with a very large job table may want to consider raising the index separately using `CONCURRENTLY` (which must be run outside of a transaction), then run `river migrate-up` to finalize the process (it will tolerate an index that already exists): + +```sql +ALTER TABLE river_job ADD COLUMN unique_states BIT(8); + +CREATE UNIQUE INDEX CONCURRENTLY river_job_unique_idx ON river_job (unique_key) + WHERE unique_key IS NOT NULL + AND unique_states IS NOT NULL + AND river_job_state_in_bitmask(unique_states, state); +``` + +```shell +go install github.com/riverqueue/river/cmd/river@latest +river migrate-up --database-url "$DATABASE_URL" +``` + ## Added - `rivertest.WorkContext`, a test function that can be used to initialize a context to test a `JobArgs.Work` implementation that will have a client set to context for use with `river.ClientFromContext`. [PR #526](https://github.com/riverqueue/river/pull/526). @@ -35,6 +66,27 @@ and this project adheres to [Semantic Versioning](https://semver.org/spec/v2.0.0 } ``` +- Unique jobs have been improved to allow bulk insertion of unique jobs via `InsertMany` / `InsertManyTx`, and to allow customizing the `ByState` list to add or remove certain states. This enables users to expand the set of unique states to also include `cancelled` and `discarded` jobs, or to remove `retryable` from uniqueness consideration. This updated implementation maintains the speed advantage of the newer index-backed uniqueness system, while allowing some flexibility in which job states. + + Unique jobs utilizing `ByArgs` can now also opt to have a subset of the job's arguments considered for uniqueness. For example, you could choose to consider only the `customer_id` field while ignoring the `trace_id` field: + + ```go + type MyJobArgs { + CustomerID string `json:"customer_id" river:"unique` + TraceID string `json:"trace_id"` + } + ``` + + Any fields considered in uniqueness are also sorted alphabetically in order to guarantee a consistent result, even if the encoded JSON isn't sorted consistently. For example `encoding/json` encodes struct fields in their defined order, so merely reordering struct fields would previously have been enough to cause a new job to not be considered identical to a pre-existing one with different JSON order. + + The `UniqueOpts` type also gains an `ExcludeKind` option for cases where uniqueness needs to be guaranteed across multiple job types. + + In-flight unique jobs using the previous designs will continue to be executed successfully with these changes, so there should be no need for downtime as part of the migration. However the v6 migration adds a new unique job index while also removing the old one, so users with in-flight unique jobs may also wish to avoid removing the old index until the new River release has been deployed in order to guarantee that jobs aren't duplicated by old River code once that index is removed. + + **Deprecated**: The original unique jobs implementation which relied on advisory locks has been deprecated, but not yet removed. The only way to trigger this old code path is with a single insert (`Insert`/`InsertTx`) and using `UniqueOpts.ByState` with a custom list of states that omits some of the now-required states for unique jobs. Specifically, `pending`, `scheduled`, `available`, and `running` can not be removed from the `ByState` list with the new implementation. These are included in the default list so only the places which customize this attribute need to be updated to opt into the new (much faster) unique jobs. The advisory lock unique implementation will be removed in an upcoming release. + + [PR #590](https://github.com/riverqueue/river/pull/590). + - **Deprecated**: The `MigrateTx` method of `rivermigrate` has been deprecated. It turns out there are certain combinations of schema changes which cannot be run within a single transaction, and the migrator now prefers to run each migration in its own transaction, one-at-a-time. `MigrateTx` will be removed in future version. - The migrator now produces a better error in case of a non-existent migration line including suggestions for known migration lines that are similar in name to the invalid one. [PR #558](https://github.com/riverqueue/river/pull/558). diff --git a/client.go b/client.go index eac7e294..1a4a39d9 100644 --- a/client.go +++ b/client.go @@ -66,9 +66,9 @@ type Config struct { // only 32 bits of number space for advisory lock hashes, so it makes // internally conflicting River-generated keys more likely. // - // Advisory locks are currently only used for the fallback/slow path of - // unique job insertion where finalized states are included in a ByState - // configuration. + // Advisory locks are currently only used for the deprecated fallback/slow + // path of unique job insertion when pending, scheduled, available, or running + // are omitted from a customized ByState configuration. AdvisoryLockPrefix int32 // CancelledJobRetentionPeriod is the amount of time to keep cancelled jobs @@ -346,7 +346,7 @@ type Client[TTx any] struct { stopped <-chan struct{} subscriptionManager *subscriptionManager testSignals clientTestSignals - uniqueInserter *dbunique.UniqueInserter + uniqueInserter *dbunique.UniqueInserter // deprecated fallback path for unique job insertion // workCancel cancels the context used for all work goroutines. Normal Stop // does not cancel that context. @@ -1162,7 +1162,7 @@ func (c *Client[TTx]) ID() string { return c.config.ID } -func insertParamsFromConfigArgsAndOptions(archetype *baseservice.Archetype, config *Config, args JobArgs, insertOpts *InsertOpts) (*riverdriver.JobInsertFastParams, *dbunique.UniqueOpts, error) { +func insertParamsFromConfigArgsAndOptions(archetype *baseservice.Archetype, config *Config, args JobArgs, insertOpts *InsertOpts, bulk bool) (*riverdriver.JobInsertFastParams, *dbunique.UniqueOpts, error) { encodedArgs, err := json.Marshal(args) if err != nil { return nil, nil, fmt.Errorf("error marshaling args to JSON: %w", err) @@ -1225,6 +1225,7 @@ func insertParamsFromConfigArgsAndOptions(archetype *baseservice.Archetype, conf } insertParams := &riverdriver.JobInsertFastParams{ + Args: args, CreatedAt: createdAt, EncodedArgs: json.RawMessage(encodedArgs), Kind: args.Kind(), @@ -1235,6 +1236,22 @@ func insertParamsFromConfigArgsAndOptions(archetype *baseservice.Archetype, conf State: rivertype.JobStateAvailable, Tags: tags, } + var returnUniqueOpts *dbunique.UniqueOpts + if !uniqueOpts.isEmpty() { + if uniqueOpts.isV1() { + if bulk { + return nil, nil, errors.New("bulk inserts do not support advisory lock uniqueness and cannot remove required states") + } + returnUniqueOpts = (*dbunique.UniqueOpts)(&uniqueOpts) + } else { + internalUniqueOpts := (*dbunique.UniqueOpts)(&uniqueOpts) + insertParams.UniqueKey, err = dbunique.UniqueKey(archetype.Time, internalUniqueOpts, insertParams) + if err != nil { + return nil, nil, err + } + insertParams.UniqueStates = internalUniqueOpts.StateBitmask() + } + } switch { case !insertOpts.ScheduledAt.IsZero(): @@ -1253,7 +1270,7 @@ func insertParamsFromConfigArgsAndOptions(archetype *baseservice.Archetype, conf insertParams.State = rivertype.JobStatePending } - return insertParams, (*dbunique.UniqueOpts)(&uniqueOpts), nil + return insertParams, returnUniqueOpts, nil } var errNoDriverDBPool = errors.New("driver must have non-nil database pool to use non-transactional methods like Insert and InsertMany (try InsertTx or InsertManyTx instead") @@ -1273,7 +1290,7 @@ func (c *Client[TTx]) Insert(ctx context.Context, args JobArgs, opts *InsertOpts return nil, errNoDriverDBPool } - return c.insert(ctx, c.driver.GetExecutor(), args, opts) + return c.insert(ctx, c.driver.GetExecutor(), args, opts, false) } // InsertTx inserts a new job with the provided args on the given transaction. @@ -1294,15 +1311,15 @@ func (c *Client[TTx]) Insert(ctx context.Context, args JobArgs, opts *InsertOpts // transactions, the job will not be worked until the transaction has committed, // and if the transaction rolls back, so too is the inserted job. func (c *Client[TTx]) InsertTx(ctx context.Context, tx TTx, args JobArgs, opts *InsertOpts) (*rivertype.JobInsertResult, error) { - return c.insert(ctx, c.driver.UnwrapExecutor(tx), args, opts) + return c.insert(ctx, c.driver.UnwrapExecutor(tx), args, opts, false) } -func (c *Client[TTx]) insert(ctx context.Context, exec riverdriver.Executor, args JobArgs, opts *InsertOpts) (*rivertype.JobInsertResult, error) { +func (c *Client[TTx]) insert(ctx context.Context, exec riverdriver.Executor, args JobArgs, opts *InsertOpts, bulk bool) (*rivertype.JobInsertResult, error) { if err := c.validateJobArgs(args); err != nil { return nil, err } - params, uniqueOpts, err := insertParamsFromConfigArgsAndOptions(&c.baseService.Archetype, c.config, args, opts) + params, uniqueOpts, err := insertParamsFromConfigArgsAndOptions(&c.baseService.Archetype, c.config, args, opts, bulk) if err != nil { return nil, err } @@ -1313,9 +1330,23 @@ func (c *Client[TTx]) insert(ctx context.Context, exec riverdriver.Executor, arg } defer tx.Rollback(ctx) - jobInsertRes, err := c.uniqueInserter.JobInsert(ctx, tx, params, uniqueOpts) - if err != nil { - return nil, err + // TODO: consolidate insertion paths for single + multi, remove deprecated uniqueness design + var jobInsertRes *riverdriver.JobInsertFastResult + if uniqueOpts == nil { + jobInsertRes, err = tx.JobInsertFast(ctx, params) + if err != nil { + return nil, err + } + } else { + if bulk { + return nil, errors.New("bulk inserts do not support advisory lock uniqueness") + } + // Old deprecated advisory lock route + c.baseService.Logger.WarnContext(ctx, "Using deprecated advisory lock uniqueness for job insert") + jobInsertRes, err = c.uniqueInserter.JobInsert(ctx, tx, params, uniqueOpts) + if err != nil { + return nil, err + } } if err := c.maybeNotifyInsert(ctx, tx, params.State, params.Queue); err != nil { @@ -1325,7 +1356,7 @@ func (c *Client[TTx]) insert(ctx context.Context, exec riverdriver.Executor, arg return nil, err } - return jobInsertRes, nil + return (*rivertype.JobInsertResult)(jobInsertRes), nil } // InsertManyParams encapsulates a single job combined with insert options for @@ -1431,8 +1462,8 @@ func (c *Client[TTx]) insertMany(ctx context.Context, tx riverdriver.ExecutorTx, } return sliceutil.Map(jobRows, - func(jobRow *rivertype.JobRow) *rivertype.JobInsertResult { - return &rivertype.JobInsertResult{Job: jobRow} + func(result *riverdriver.JobInsertFastResult) *rivertype.JobInsertResult { + return (*rivertype.JobInsertResult)(result) }, ), nil } @@ -1450,20 +1481,12 @@ func (c *Client[TTx]) insertManyParams(params []InsertManyParams) ([]*riverdrive return nil, err } - if param.InsertOpts != nil { - // UniqueOpts aren't supported for batch inserts because they use PG - // advisory locks to work, and taking many locks simultaneously could - // easily lead to contention and deadlocks. - if !param.InsertOpts.UniqueOpts.isEmpty() { - return nil, errors.New("UniqueOpts are not supported for batch inserts") - } - } - - var err error - insertParams[i], _, err = insertParamsFromConfigArgsAndOptions(&c.baseService.Archetype, c.config, param.Args, param.InsertOpts) + insertParamsItem, _, err := insertParamsFromConfigArgsAndOptions(&c.baseService.Archetype, c.config, param.Args, param.InsertOpts, true) if err != nil { return nil, err } + + insertParams[i] = insertParamsItem } return insertParams, nil @@ -1579,20 +1602,11 @@ func (c *Client[TTx]) insertManyFastParams(params []InsertManyParams) ([]*riverd return nil, err } - if param.InsertOpts != nil { - // UniqueOpts aren't support for batch inserts because they use PG - // advisory locks to work, and taking many locks simultaneously - // could easily lead to contention and deadlocks. - if !param.InsertOpts.UniqueOpts.isEmpty() { - return nil, errors.New("UniqueOpts are not supported for batch inserts") - } - } - - var err error - insertParams[i], _, err = insertParamsFromConfigArgsAndOptions(&c.baseService.Archetype, c.config, param.Args, param.InsertOpts) + insertParamsItem, _, err := insertParamsFromConfigArgsAndOptions(&c.baseService.Archetype, c.config, param.Args, param.InsertOpts, true) if err != nil { return nil, err } + insertParams[i] = insertParamsItem } return insertParams, nil diff --git a/client_test.go b/client_test.go index c222a33f..b7b33d9c 100644 --- a/client_test.go +++ b/client_test.go @@ -21,6 +21,7 @@ import ( "github.com/robfig/cron/v3" "github.com/stretchr/testify/require" + "github.com/riverqueue/river/internal/dbunique" "github.com/riverqueue/river/internal/maintenance" "github.com/riverqueue/river/internal/notifier" "github.com/riverqueue/river/internal/rivercommon" @@ -1417,6 +1418,26 @@ func Test_Client_Insert(t *testing.T) { require.NoError(t, client.Stop(ctx)) }) + t.Run("WithUniqueOpts", func(t *testing.T) { + t.Parallel() + + client, _ := setup(t) + + job1, err := client.Insert(ctx, noOpArgs{Name: "foo"}, &InsertOpts{UniqueOpts: UniqueOpts{ByArgs: true}}) + require.NoError(t, err) + require.NotNil(t, job1) + + // Dupe, same args: + job2, err := client.Insert(ctx, noOpArgs{Name: "foo"}, &InsertOpts{UniqueOpts: UniqueOpts{ByArgs: true}}) + require.NoError(t, err) + require.Equal(t, job1.Job.ID, job2.Job.ID) + + // Not a dupe, different args + job3, err := client.Insert(ctx, noOpArgs{Name: "bar"}, &InsertOpts{UniqueOpts: UniqueOpts{ByArgs: true}}) + require.NoError(t, err) + require.NotEqual(t, job1.Job.ID, job3.Job.ID) + }) + t.Run("ErrorsOnInvalidQueueName", func(t *testing.T) { t.Parallel() @@ -1545,6 +1566,26 @@ func Test_Client_InsertTx(t *testing.T) { require.NoError(t, err) }) + t.Run("WithUniqueOpts", func(t *testing.T) { + t.Parallel() + + client, bundle := setup(t) + + job1, err := client.InsertTx(ctx, bundle.tx, noOpArgs{Name: "foo"}, &InsertOpts{UniqueOpts: UniqueOpts{ByArgs: true}}) + require.NoError(t, err) + require.NotNil(t, job1) + + // Dupe, same args: + job2, err := client.InsertTx(ctx, bundle.tx, noOpArgs{Name: "foo"}, &InsertOpts{UniqueOpts: UniqueOpts{ByArgs: true}}) + require.NoError(t, err) + require.Equal(t, job1.Job.ID, job2.Job.ID) + + // Not a dupe, different args + job3, err := client.InsertTx(ctx, bundle.tx, noOpArgs{Name: "bar"}, &InsertOpts{UniqueOpts: UniqueOpts{ByArgs: true}}) + require.NoError(t, err) + require.NotEqual(t, job1.Job.ID, job3.Job.ID) + }) + t.Run("ErrorsOnUnknownJobKindWithWorkers", func(t *testing.T) { t.Parallel() @@ -1774,15 +1815,19 @@ func Test_Client_InsertManyFast(t *testing.T) { require.NoError(t, err) }) - t.Run("ErrorsOnInsertOptsUniqueOpts", func(t *testing.T) { + t.Run("ErrorsOnInsertOptsWithV1UniqueOpts", func(t *testing.T) { t.Parallel() client, _ := setup(t) count, err := client.InsertManyFast(ctx, []InsertManyParams{ - {Args: noOpArgs{}, InsertOpts: &InsertOpts{UniqueOpts: UniqueOpts{ByArgs: true}}}, + {Args: noOpArgs{}, InsertOpts: &InsertOpts{UniqueOpts: UniqueOpts{ + ByArgs: true, + // force the v1 unique path with a custom state list that isn't supported in v3: + ByState: []rivertype.JobState{rivertype.JobStateAvailable}, + }}}, }) - require.EqualError(t, err, "UniqueOpts are not supported for batch inserts") + require.EqualError(t, err, "bulk inserts do not support advisory lock uniqueness and cannot remove required states") require.Equal(t, 0, count) }) } @@ -1925,15 +1970,19 @@ func Test_Client_InsertManyFastTx(t *testing.T) { require.NoError(t, err) }) - t.Run("ErrorsOnInsertOptsUniqueOpts", func(t *testing.T) { + t.Run("ErrorsOnInsertOptsWithV1UniqueOpts", func(t *testing.T) { t.Parallel() client, bundle := setup(t) count, err := client.InsertManyFastTx(ctx, bundle.tx, []InsertManyParams{ - {Args: noOpArgs{}, InsertOpts: &InsertOpts{UniqueOpts: UniqueOpts{ByArgs: true}}}, + {Args: noOpArgs{}, InsertOpts: &InsertOpts{UniqueOpts: UniqueOpts{ + ByArgs: true, + // force the v1 unique path with a custom state list that isn't supported in v3: + ByState: []rivertype.JobState{rivertype.JobStateAvailable}, + }}}, }) - require.EqualError(t, err, "UniqueOpts are not supported for batch inserts") + require.EqualError(t, err, "bulk inserts do not support advisory lock uniqueness and cannot remove required states") require.Equal(t, 0, count) }) } @@ -2187,15 +2236,19 @@ func Test_Client_InsertMany(t *testing.T) { require.Len(t, results, 1) }) - t.Run("ErrorsOnInsertOptsUniqueOpts", func(t *testing.T) { + t.Run("ErrorsOnInsertOptsWithV1UniqueOpts", func(t *testing.T) { t.Parallel() client, _ := setup(t) results, err := client.InsertMany(ctx, []InsertManyParams{ - {Args: noOpArgs{}, InsertOpts: &InsertOpts{UniqueOpts: UniqueOpts{ByArgs: true}}}, + {Args: noOpArgs{}, InsertOpts: &InsertOpts{UniqueOpts: UniqueOpts{ + ByArgs: true, + // force the v1 unique path with a custom state list that isn't supported in v3: + ByState: []rivertype.JobState{rivertype.JobStateAvailable}, + }}}, }) - require.EqualError(t, err, "UniqueOpts are not supported for batch inserts") + require.EqualError(t, err, "bulk inserts do not support advisory lock uniqueness and cannot remove required states") require.Empty(t, results) }) } @@ -2337,6 +2390,29 @@ func Test_Client_InsertManyTx(t *testing.T) { require.Len(t, results, 1) }) + t.Run("WithUniqueOpts", func(t *testing.T) { + t.Parallel() + + client, bundle := setup(t) + + results, err := client.InsertManyTx(ctx, bundle.tx, []InsertManyParams{{Args: noOpArgs{Name: "foo"}, InsertOpts: &InsertOpts{UniqueOpts: UniqueOpts{ByArgs: true}}}}) + require.NoError(t, err) + require.Len(t, results, 1) + job1 := results[0] + + // Dupe, same args: + results, err = client.InsertManyTx(ctx, bundle.tx, []InsertManyParams{{Args: noOpArgs{Name: "foo"}, InsertOpts: &InsertOpts{UniqueOpts: UniqueOpts{ByArgs: true}}}}) + require.NoError(t, err) + job2 := results[0] + require.Equal(t, job1.Job.ID, job2.Job.ID) + + // Not a dupe, different args + results, err = client.InsertManyTx(ctx, bundle.tx, []InsertManyParams{{Args: noOpArgs{Name: "bar"}, InsertOpts: &InsertOpts{UniqueOpts: UniqueOpts{ByArgs: true}}}}) + require.NoError(t, err) + job3 := results[0] + require.NotEqual(t, job1.Job.ID, job3.Job.ID) + }) + t.Run("ErrorsWithZeroJobs", func(t *testing.T) { t.Parallel() @@ -2375,15 +2451,19 @@ func Test_Client_InsertManyTx(t *testing.T) { require.Len(t, results, 1) }) - t.Run("ErrorsOnInsertOptsUniqueOpts", func(t *testing.T) { + t.Run("ErrorsOnInsertOptsWithV1UniqueOpts", func(t *testing.T) { t.Parallel() client, bundle := setup(t) results, err := client.InsertManyTx(ctx, bundle.tx, []InsertManyParams{ - {Args: noOpArgs{}, InsertOpts: &InsertOpts{UniqueOpts: UniqueOpts{ByArgs: true}}}, + {Args: noOpArgs{}, InsertOpts: &InsertOpts{UniqueOpts: UniqueOpts{ + ByArgs: true, + // force the v1 unique path with a custom state list that isn't supported in v3: + ByState: []rivertype.JobState{rivertype.JobStateAvailable}, + }}}, }) - require.EqualError(t, err, "UniqueOpts are not supported for batch inserts") + require.EqualError(t, err, "bulk inserts do not support advisory lock uniqueness and cannot remove required states") require.Empty(t, results) }) } @@ -2886,7 +2966,7 @@ func Test_Client_ErrorHandler(t *testing.T) { // Bypass the normal Insert function because that will error on an // unknown job. - insertParams, _, err := insertParamsFromConfigArgsAndOptions(&client.baseService.Archetype, config, unregisteredJobArgs{}, nil) + insertParams, _, err := insertParamsFromConfigArgsAndOptions(&client.baseService.Archetype, config, unregisteredJobArgs{}, nil, false) require.NoError(t, err) _, err = client.driver.GetExecutor().JobInsertFast(ctx, insertParams) require.NoError(t, err) @@ -4047,6 +4127,7 @@ func Test_Client_SubscribeConfig(t *testing.T) { ) for i := 0; i < numJobsToInsert; i++ { insertParams[i] = &riverdriver.JobInsertFastParams{ + Args: &JobArgs{}, EncodedArgs: []byte(`{}`), Kind: kind, MaxAttempts: rivercommon.MaxAttemptsDefault, @@ -4241,6 +4322,65 @@ func Test_Client_InsertNotificationsAreDeduplicatedAndDebounced(t *testing.T) { expectImmediateNotification(t, "queue1") } +func Test_Client_CanInsertAndWorkV1UniqueJob(t *testing.T) { + t.Parallel() + + ctx := context.Background() + dbPool := riverinternaltest.TestDB(ctx, t) + startedChan := make(chan int64) + waitChan := make(chan struct{}) + config := newTestConfig(t, func(ctx context.Context, job *Job[callbackArgs]) error { + close(startedChan) + <-waitChan + return nil + }) + client := newTestClient(t, dbPool, config) + + startClient(ctx, t, client) + riversharedtest.WaitOrTimeout(t, client.baseStartStop.Started()) + + // Insert a job, wait for it to start, then insert the same job again. The + // second insert should be ignored because the job is already running. + insertRes1, err := client.Insert(ctx, callbackArgs{}, &InsertOpts{UniqueOpts: UniqueOpts{ByState: []rivertype.JobState{rivertype.JobStateAvailable, rivertype.JobStateRunning}}}) + require.NoError(t, err) + require.False(t, insertRes1.UniqueSkippedAsDuplicate) + + riversharedtest.WaitOrTimeout(t, startedChan) + + insertRes2, err := client.Insert(ctx, callbackArgs{}, &InsertOpts{UniqueOpts: UniqueOpts{ByState: []rivertype.JobState{rivertype.JobStateAvailable, rivertype.JobStateRunning}}}) + require.NoError(t, err) + require.True(t, insertRes2.UniqueSkippedAsDuplicate) + require.Equal(t, insertRes1.Job.ID, insertRes2.Job.ID) + + close(waitChan) +} + +func Test_Client_CanWorkPreexistingV2UniqueJob(t *testing.T) { + t.Parallel() + + ctx := context.Background() + dbPool := riverinternaltest.TestDB(ctx, t) + config := newTestConfig(t, nil) + client := newTestClient(t, dbPool, config) + + subscribeChan, cancel := client.Subscribe(EventKindJobCompleted) + t.Cleanup(cancel) + + startClient(ctx, t, client) + riversharedtest.WaitOrTimeout(t, client.baseStartStop.Started()) + + job := testfactory.Job(ctx, t, client.driver.GetExecutor(), &testfactory.JobOpts{ + Kind: ptrutil.Ptr((&noOpArgs{}).Kind()), + State: ptrutil.Ptr(rivertype.JobStateAvailable), + UniqueKey: []byte("v2_unique_key"), + UniqueStates: 0, // will be inserted as null + }) + + jobWorked := riversharedtest.WaitOrTimeout(t, subscribeChan) + require.Equal(t, job.ID, jobWorked.Job.ID) + require.Equal(t, rivertype.JobStateCompleted, jobWorked.Job.State) +} + func Test_Client_JobCompletion(t *testing.T) { t.Parallel() @@ -4487,19 +4627,19 @@ func Test_Client_UnknownJobKindErrorsTheJob(t *testing.T) { subscribeChan, cancel := client.Subscribe(EventKindJobFailed) t.Cleanup(cancel) - insertParams, _, err := insertParamsFromConfigArgsAndOptions(&client.baseService.Archetype, config, unregisteredJobArgs{}, nil) + insertParams, _, err := insertParamsFromConfigArgsAndOptions(&client.baseService.Archetype, config, unregisteredJobArgs{}, nil, false) require.NoError(err) - insertedJob, err := client.driver.GetExecutor().JobInsertFast(ctx, insertParams) + insertedResult, err := client.driver.GetExecutor().JobInsertFast(ctx, insertParams) require.NoError(err) event := riversharedtest.WaitOrTimeout(t, subscribeChan) - require.Equal(insertedJob.ID, event.Job.ID) - require.Equal("RandomWorkerNameThatIsNeverRegistered", insertedJob.Kind) + require.Equal(insertedResult.Job.ID, event.Job.ID) + require.Equal("RandomWorkerNameThatIsNeverRegistered", insertedResult.Job.Kind) require.Len(event.Job.Errors, 1) require.Equal((&UnknownJobKindError{Kind: "RandomWorkerNameThatIsNeverRegistered"}).Error(), event.Job.Errors[0].Error) require.Equal(rivertype.JobStateRetryable, event.Job.State) // Ensure that ScheduledAt was updated with next run time: - require.True(event.Job.ScheduledAt.After(insertedJob.ScheduledAt)) + require.True(event.Job.ScheduledAt.After(insertedResult.Job.ScheduledAt)) // It's the 1st attempt that failed. Attempt won't be incremented again until // the job gets fetched a 2nd time. require.Equal(1, event.Job.Attempt) @@ -5080,6 +5220,14 @@ func TestClient_JobTimeout(t *testing.T) { } } +type JobArgsStaticKind struct { + kind string +} + +func (a JobArgsStaticKind) Kind() string { + return a.kind +} + func TestInsertParamsFromJobArgsAndOptions(t *testing.T) { t.Parallel() @@ -5089,7 +5237,7 @@ func TestInsertParamsFromJobArgsAndOptions(t *testing.T) { t.Run("Defaults", func(t *testing.T) { t.Parallel() - insertParams, uniqueOpts, err := insertParamsFromConfigArgsAndOptions(archetype, config, noOpArgs{}, nil) + insertParams, uniqueOpts, err := insertParamsFromConfigArgsAndOptions(archetype, config, noOpArgs{}, nil, false) require.NoError(t, err) require.Equal(t, `{"name":""}`, string(insertParams.EncodedArgs)) require.Equal(t, (noOpArgs{}).Kind(), insertParams.Kind) @@ -5099,7 +5247,9 @@ func TestInsertParamsFromJobArgsAndOptions(t *testing.T) { require.Nil(t, insertParams.ScheduledAt) require.Equal(t, []string{}, insertParams.Tags) - require.True(t, uniqueOpts.IsEmpty()) + require.Nil(t, uniqueOpts) + require.Empty(t, insertParams.UniqueKey) + require.Zero(t, insertParams.UniqueStates) }) t.Run("ConfigOverrides", func(t *testing.T) { @@ -5109,7 +5259,7 @@ func TestInsertParamsFromJobArgsAndOptions(t *testing.T) { MaxAttempts: 34, } - insertParams, _, err := insertParamsFromConfigArgsAndOptions(archetype, overrideConfig, noOpArgs{}, nil) + insertParams, _, err := insertParamsFromConfigArgsAndOptions(archetype, overrideConfig, noOpArgs{}, nil, false) require.NoError(t, err) require.Equal(t, overrideConfig.MaxAttempts, insertParams.MaxAttempts) }) @@ -5124,7 +5274,7 @@ func TestInsertParamsFromJobArgsAndOptions(t *testing.T) { ScheduledAt: time.Now().Add(time.Hour), Tags: []string{"tag1", "tag2"}, } - insertParams, _, err := insertParamsFromConfigArgsAndOptions(archetype, config, noOpArgs{}, opts) + insertParams, _, err := insertParamsFromConfigArgsAndOptions(archetype, config, noOpArgs{}, opts, false) require.NoError(t, err) require.Equal(t, 42, insertParams.MaxAttempts) require.Equal(t, 2, insertParams.Priority) @@ -5140,7 +5290,7 @@ func TestInsertParamsFromJobArgsAndOptions(t *testing.T) { insertParams, _, err := insertParamsFromConfigArgsAndOptions(archetype, config, &customInsertOptsJobArgs{ ScheduledAt: nearFuture, - }, nil) + }, nil, false) require.NoError(t, err) // All these come from overrides in customInsertOptsJobArgs's definition: require.Equal(t, 42, insertParams.MaxAttempts) @@ -5156,7 +5306,7 @@ func TestInsertParamsFromJobArgsAndOptions(t *testing.T) { insertParams, _, err := insertParamsFromConfigArgsAndOptions(archetype, config, &customInsertOptsJobArgs{ ScheduledAt: time.Time{}, - }, nil) + }, nil, false) require.NoError(t, err) require.Nil(t, insertParams.ScheduledAt) }) @@ -5167,40 +5317,169 @@ func TestInsertParamsFromJobArgsAndOptions(t *testing.T) { { _, _, err := insertParamsFromConfigArgsAndOptions(archetype, config, &customInsertOptsJobArgs{}, &InsertOpts{ Tags: []string{strings.Repeat("h", 256)}, - }) + }, false) require.EqualError(t, err, "tags should be a maximum of 255 characters long") } { _, _, err := insertParamsFromConfigArgsAndOptions(archetype, config, &customInsertOptsJobArgs{}, &InsertOpts{ Tags: []string{"tag,with,comma"}, - }) + }, false) require.EqualError(t, err, "tags should match regex "+tagRE.String()) } }) - t.Run("UniqueOpts", func(t *testing.T) { + t.Run("UniqueOptsDefaultStates", func(t *testing.T) { + t.Parallel() + + archetype := riversharedtest.BaseServiceArchetype(t) + archetype.Time.StubNowUTC(time.Now().UTC()) + + uniqueOpts := UniqueOpts{ + ByArgs: true, + ByPeriod: 10 * time.Second, + ByQueue: true, + ExcludeKind: true, + } + + params, resultUniqueOpts, err := insertParamsFromConfigArgsAndOptions(archetype, config, noOpArgs{}, &InsertOpts{UniqueOpts: uniqueOpts}, false) + require.NoError(t, err) + require.Nil(t, resultUniqueOpts) + internalUniqueOpts := &dbunique.UniqueOpts{ + ByArgs: true, + ByPeriod: 10 * time.Second, + ByQueue: true, + ByState: []rivertype.JobState{rivertype.JobStateAvailable, rivertype.JobStateCompleted, rivertype.JobStatePending, rivertype.JobStateRunning, rivertype.JobStateRetryable, rivertype.JobStateScheduled}, + ExcludeKind: true, + } + + expectedKey, err := dbunique.UniqueKey(archetype.Time, internalUniqueOpts, params) + require.NoError(t, err) + + require.Equal(t, expectedKey, params.UniqueKey) + require.Equal(t, internalUniqueOpts.StateBitmask(), params.UniqueStates) + }) + + t.Run("UniqueOptsCustomStates", func(t *testing.T) { + t.Parallel() + + archetype := riversharedtest.BaseServiceArchetype(t) + archetype.Time.StubNowUTC(time.Now().UTC()) + + states := []rivertype.JobState{ + rivertype.JobStateAvailable, + rivertype.JobStatePending, + rivertype.JobStateRetryable, + rivertype.JobStateRunning, + rivertype.JobStateScheduled, + } + + uniqueOpts := UniqueOpts{ + ByPeriod: 10 * time.Second, + ByQueue: true, + ByState: states, + } + + params, resultUniqueOpts, err := insertParamsFromConfigArgsAndOptions(archetype, config, noOpArgs{}, &InsertOpts{UniqueOpts: uniqueOpts}, true) + require.NoError(t, err) + require.Nil(t, resultUniqueOpts) + internalUniqueOpts := &dbunique.UniqueOpts{ + ByPeriod: 10 * time.Second, + ByQueue: true, + ByState: states, + } + + expectedKey, err := dbunique.UniqueKey(archetype.Time, internalUniqueOpts, params) + require.NoError(t, err) + + require.Equal(t, expectedKey, params.UniqueKey) + require.Equal(t, internalUniqueOpts.StateBitmask(), params.UniqueStates) + }) + + t.Run("UniqueOptsV1", func(t *testing.T) { t.Parallel() + archetype := riversharedtest.BaseServiceArchetype(t) + archetype.Time.StubNowUTC(time.Now().UTC()) + uniqueOpts := UniqueOpts{ + ByArgs: true, + ByPeriod: 10 * time.Second, + ByQueue: true, + // This list of custom states (without pending, scheduled, running, etc.) is only valid for v1 unique opts: + ByState: []rivertype.JobState{rivertype.JobStateAvailable, rivertype.JobStateCompleted}, + } + + params, resultUniqueOpts, err := insertParamsFromConfigArgsAndOptions(archetype, config, noOpArgs{}, &InsertOpts{UniqueOpts: uniqueOpts}, false) + require.NoError(t, err) + require.NotNil(t, resultUniqueOpts) + internalUniqueOpts := &dbunique.UniqueOpts{ ByArgs: true, ByPeriod: 10 * time.Second, ByQueue: true, ByState: []rivertype.JobState{rivertype.JobStateAvailable, rivertype.JobStateCompleted}, } + require.Equal(t, internalUniqueOpts, resultUniqueOpts) + + require.Nil(t, params.UniqueKey) + require.Zero(t, params.UniqueStates) + + // In a bulk insert, this should be explicitly blocked: + params, resultUniqueOpts, err = insertParamsFromConfigArgsAndOptions(archetype, config, noOpArgs{}, &InsertOpts{UniqueOpts: uniqueOpts}, true) + require.ErrorContains(t, err, "bulk inserts do not support advisory lock uniqueness and cannot remove required states") + require.Nil(t, params) + require.Nil(t, resultUniqueOpts) + }) + + t.Run("UniqueOptsWithPartialArgs", func(t *testing.T) { + t.Parallel() + + uniqueOpts := UniqueOpts{ByArgs: true} + + type PartialArgs struct { + JobArgsStaticKind + Included bool `json:"included" river:"unique"` + Excluded bool `json:"excluded"` + } - _, internalUniqueOpts, err := insertParamsFromConfigArgsAndOptions(archetype, config, noOpArgs{}, &InsertOpts{UniqueOpts: uniqueOpts}) + args := PartialArgs{ + JobArgsStaticKind: JobArgsStaticKind{kind: "partialArgs"}, + Included: true, + Excluded: true, + } + + params, resultUniqueOpts, err := insertParamsFromConfigArgsAndOptions(archetype, config, args, &InsertOpts{UniqueOpts: uniqueOpts}, true) + require.NoError(t, err) + require.Nil(t, resultUniqueOpts) + internalUniqueOpts := &dbunique.UniqueOpts{ByArgs: true} + + expectedKey, err := dbunique.UniqueKey(archetype.Time, internalUniqueOpts, params) require.NoError(t, err) - require.Equal(t, uniqueOpts.ByArgs, internalUniqueOpts.ByArgs) - require.Equal(t, uniqueOpts.ByPeriod, internalUniqueOpts.ByPeriod) - require.Equal(t, uniqueOpts.ByQueue, internalUniqueOpts.ByQueue) - require.Equal(t, uniqueOpts.ByState, internalUniqueOpts.ByState) + require.Equal(t, expectedKey, params.UniqueKey) + require.Equal(t, internalUniqueOpts.StateBitmask(), params.UniqueStates) + + argsWithExcludedFalse := PartialArgs{ + JobArgsStaticKind: JobArgsStaticKind{kind: "partialArgs"}, + Included: true, + Excluded: false, + } + + params2, resultUniqueOpts2, err := insertParamsFromConfigArgsAndOptions(archetype, config, argsWithExcludedFalse, &InsertOpts{UniqueOpts: uniqueOpts}, true) + require.NoError(t, err) + require.Nil(t, resultUniqueOpts2) + internalUniqueOpts2 := &dbunique.UniqueOpts{ByArgs: true} + + expectedKey2, err := dbunique.UniqueKey(archetype.Time, internalUniqueOpts2, params2) + require.NoError(t, err) + require.Equal(t, expectedKey2, params2.UniqueKey) + require.Equal(t, internalUniqueOpts2.StateBitmask(), params.UniqueStates) + require.Equal(t, params.UniqueKey, params2.UniqueKey, "unique keys should be identical because included args are the same, even though others differ") }) t.Run("PriorityIsLimitedTo4", func(t *testing.T) { t.Parallel() - insertParams, _, err := insertParamsFromConfigArgsAndOptions(archetype, config, noOpArgs{}, &InsertOpts{Priority: 5}) + insertParams, _, err := insertParamsFromConfigArgsAndOptions(archetype, config, noOpArgs{}, &InsertOpts{Priority: 5}, false) require.ErrorContains(t, err, "priority must be between 1 and 4") require.Nil(t, insertParams) }) @@ -5209,7 +5488,7 @@ func TestInsertParamsFromJobArgsAndOptions(t *testing.T) { t.Parallel() args := timeoutTestArgs{TimeoutValue: time.Hour} - insertParams, _, err := insertParamsFromConfigArgsAndOptions(archetype, config, args, nil) + insertParams, _, err := insertParamsFromConfigArgsAndOptions(archetype, config, args, nil, false) require.NoError(t, err) require.Equal(t, `{"timeout_value":3600000000000}`, string(insertParams.EncodedArgs)) }) @@ -5220,15 +5499,19 @@ func TestInsertParamsFromJobArgsAndOptions(t *testing.T) { // Ensure that unique opts are validated. No need to be exhaustive here // since we already have tests elsewhere for that. Just make sure validation // is running. - insertParams, _, err := insertParamsFromConfigArgsAndOptions( + insertParams, resultUniqueOpts, err := insertParamsFromConfigArgsAndOptions( archetype, config, noOpArgs{}, &InsertOpts{UniqueOpts: UniqueOpts{ByPeriod: 1 * time.Millisecond}}, + false, ) require.EqualError(t, err, "JobUniqueOpts.ByPeriod should not be less than 1 second") require.Nil(t, insertParams) + require.Nil(t, resultUniqueOpts) }) + + // TODO NOW NEXT: validate unique opts for v1 unique opts w/ advisory lock and custom states: } func TestID(t *testing.T) { @@ -5411,7 +5694,7 @@ func TestUniqueOpts(t *testing.T) { // roughly in the middle of the hour and well clear of any period // boundaries. client.baseService.Time.StubNowUTC( - time.Now().Truncate(1 * time.Hour).Add(37*time.Minute + 23*time.Second + 123*time.Millisecond), + time.Now().Truncate(1 * time.Hour).Add(37*time.Minute + 23*time.Second + 123*time.Millisecond).UTC(), ) return client, &testBundle{} @@ -5442,7 +5725,43 @@ func TestUniqueOpts(t *testing.T) { require.Equal(t, insertRes0.Job.ID, insertRes1.Job.ID) }) - t.Run("UniqueByState", func(t *testing.T) { + t.Run("UniqueByCustomStates", func(t *testing.T) { + t.Parallel() + + client, _ := setup(t) + + uniqueOpts := UniqueOpts{ + ByPeriod: 24 * time.Hour, + ByState: rivertype.JobStates(), + ByQueue: true, + } + + insertRes0, err := client.Insert(ctx, noOpArgs{}, &InsertOpts{ + UniqueOpts: uniqueOpts, + }) + require.NoError(t, err) + + insertRes1, err := client.Insert(ctx, noOpArgs{}, &InsertOpts{ + UniqueOpts: uniqueOpts, + }) + require.NoError(t, err) + + // Expect the same job to come back because we deduplicate from the original. + require.Equal(t, insertRes0.Job.ID, insertRes1.Job.ID) + + insertRes2, err := client.Insert(ctx, noOpArgs{}, &InsertOpts{ + // Use another queue so the job can be inserted: + Queue: "other", + UniqueOpts: uniqueOpts, + }) + require.NoError(t, err) + + // This job however is _not_ the same because it's inserted as + // `scheduled` which is outside the unique constraints. + require.NotEqual(t, insertRes0.Job.ID, insertRes2.Job.ID) + }) + + t.Run("UniqueV1ByCustomStates", func(t *testing.T) { t.Parallel() client, _ := setup(t) diff --git a/docs/state_machine.md b/docs/state_machine.md new file mode 100644 index 00000000..3b56c96c --- /dev/null +++ b/docs/state_machine.md @@ -0,0 +1,77 @@ +```mermaid +flowchart LR + %% Define Styles + classDef waiting fill:#f9f,stroke:#333,stroke-width:2px; + classDef available fill:#9fc,stroke:#333,stroke-width:2px; + classDef running fill:#ff9,stroke:#333,stroke-width:2px; + classDef retryable fill:#ffcc00,stroke:#333,stroke-width:2px; + classDef final fill:#9f9,stroke:#333,stroke-width:2px,color:black; + classDef finalFailed fill:#f99,stroke:#333,stroke-width:2px,color:black; + classDef retryLine stroke-dasharray: 5 5 + + %% Apply Styles + A:::available + S:::waiting + P:::waiting + R:::running + Re:::retryable + C:::final + Ca:::finalFailed + D:::finalFailed + + %% Define Initial States + subgraph Initial_States + A["Available"] + S["Scheduled"] + P["Pending"] + end + + %% Define Intermediate States + R["Running"] + Re["Retryable"] + + %% Define Final States + subgraph Finalized + C["Completed"] + Ca["Cancelled"] + D["Discarded"] + end + + + %% Main Flow + A -- fetched --> R + R -- success --> C + R -- error --> Re + R -- too many errors --> D + + R -- cancel --> Ca + R -- discard --> D + R -- snooze --> S + + Re -- schedule --> A + + S -- schedule --> A + + P -- preconditions met, future schedule --> S + P -- preconditions met --> A + + %% Rescuer + R -- rescued --> Re + R -- rescued --> D + + %% Retry Transitions + C -- manual retry --> A + D -- manual retry --> A + Ca -- manual retry --> A + Re -- manual retry --> A + S -- manual retry --> A + P -- manual retry --> A + + %% Cancellation Transitions + A -- manual cancel --> Ca + R -- manual cancel --> Ca + S -- manual cancel --> Ca + P -- manual cancel --> Ca + Re -- manual cancel --> Ca + +``` diff --git a/driver_test.go b/driver_test.go index 7280fc81..72a42965 100644 --- a/driver_test.go +++ b/driver_test.go @@ -4,6 +4,7 @@ import ( "context" "database/sql" "runtime" + "strconv" "testing" "time" @@ -233,19 +234,19 @@ func BenchmarkDriverRiverPgxV5Insert(b *testing.B) { } }) - b.Run("InsertUnique", func(b *testing.B) { + b.Run("InsertFast_WithUnique", func(b *testing.B) { _, bundle := setup(b) - for n := 0; n < b.N; n++ { - _, err := bundle.exec.JobInsertUnique(ctx, &riverdriver.JobInsertUniqueParams{ - JobInsertFastParams: &riverdriver.JobInsertFastParams{ - EncodedArgs: []byte(`{"encoded": "args"}`), - Kind: "test_kind", - MaxAttempts: rivercommon.MaxAttemptsDefault, - Priority: rivercommon.PriorityDefault, - Queue: rivercommon.QueueDefault, - State: rivertype.JobStateAvailable, - }, + for i := 0; i < b.N; i++ { + _, err := bundle.exec.JobInsertFast(ctx, &riverdriver.JobInsertFastParams{ + EncodedArgs: []byte(`{"encoded": "args"}`), + Kind: "test_kind", + MaxAttempts: rivercommon.MaxAttemptsDefault, + Priority: rivercommon.PriorityDefault, + Queue: rivercommon.QueueDefault, + State: rivertype.JobStateAvailable, + UniqueKey: []byte("test_unique_key_" + strconv.Itoa(i)), + UniqueStates: 0xFB, }) require.NoError(b, err) } diff --git a/go.mod b/go.mod index 709ce0e4..def611e2 100644 --- a/go.mod +++ b/go.mod @@ -15,6 +15,8 @@ require ( github.com/riverqueue/river/rivertype v0.11.4 github.com/robfig/cron/v3 v3.0.1 github.com/stretchr/testify v1.9.0 + github.com/tidwall/gjson v1.17.3 + github.com/tidwall/sjson v1.2.5 go.uber.org/goleak v1.3.0 golang.org/x/sync v0.8.0 golang.org/x/text v0.18.0 @@ -26,6 +28,8 @@ require ( github.com/jackc/pgservicefile v0.0.0-20240606120523-5a60cdf6a761 // indirect github.com/lib/pq v1.10.9 // indirect github.com/pmezard/go-difflib v1.0.0 // indirect + github.com/tidwall/match v1.1.1 // indirect + github.com/tidwall/pretty v1.2.0 // indirect golang.org/x/crypto v0.27.0 // indirect gopkg.in/yaml.v3 v3.0.1 // indirect ) diff --git a/go.sum b/go.sum index b0ab3196..6df49363 100644 --- a/go.sum +++ b/go.sum @@ -36,6 +36,15 @@ github.com/stretchr/testify v1.3.0/go.mod h1:M5WIy9Dh21IEIfnGCwXGc5bZfKNJtfHm1UV github.com/stretchr/testify v1.7.0/go.mod h1:6Fq8oRcR53rry900zMqJjRRixrwX3KX962/h/Wwjteg= github.com/stretchr/testify v1.9.0 h1:HtqpIVDClZ4nwg75+f6Lvsy/wHu+3BoSGCbBAcpTsTg= github.com/stretchr/testify v1.9.0/go.mod h1:r2ic/lqez/lEtzL7wO/rwa5dbSLXVDPFyf8C91i36aY= +github.com/tidwall/gjson v1.14.2/go.mod h1:/wbyibRr2FHMks5tjHJ5F8dMZh3AcwJEMf5vlfC0lxk= +github.com/tidwall/gjson v1.17.3 h1:bwWLZU7icoKRG+C+0PNwIKC6FCJO/Q3p2pZvuP0jN94= +github.com/tidwall/gjson v1.17.3/go.mod h1:/wbyibRr2FHMks5tjHJ5F8dMZh3AcwJEMf5vlfC0lxk= +github.com/tidwall/match v1.1.1 h1:+Ho715JplO36QYgwN9PGYNhgZvoUSc9X2c80KVTi+GA= +github.com/tidwall/match v1.1.1/go.mod h1:eRSPERbgtNPcGhD8UCthc6PmLEQXEWd3PRB5JTxsfmM= +github.com/tidwall/pretty v1.2.0 h1:RWIZEg2iJ8/g6fDDYzMpobmaoGh5OLl4AXtGUGPcqCs= +github.com/tidwall/pretty v1.2.0/go.mod h1:ITEVvHYasfjBbM0u2Pg8T2nJnzm8xPwvNhhsoaGGjNU= +github.com/tidwall/sjson v1.2.5 h1:kLy8mja+1c9jlljvWTlSazM7cKDRfJuR/bOJhcY5NcY= +github.com/tidwall/sjson v1.2.5/go.mod h1:Fvgq9kS/6ociJEDnK0Fk1cpYF4FIW6ZF7LAe+6jwd28= go.uber.org/goleak v1.3.0 h1:2K3zAYmnTNqV73imy9J1T3WC+gmCePx2hEGkimedGto= go.uber.org/goleak v1.3.0/go.mod h1:CoHD4mav9JJNrW/WLlf7HGZPjdw8EucARQHekz1X6bE= golang.org/x/crypto v0.27.0 h1:GXm2NjJrPaiv/h1tb2UH8QfgC/hOf/+z0p6PT8o1w7A= diff --git a/go.work.sum b/go.work.sum index b9b39ddb..310b1797 100644 --- a/go.work.sum +++ b/go.work.sum @@ -13,6 +13,7 @@ github.com/stretchr/testify v1.8.0/go.mod h1:yNjHg4UonilssWZ8iaSj1OCr/vHnekPRkoO github.com/stretchr/testify v1.8.1/go.mod h1:w2LPCIKwWwSfY2zedu0+kehJoqGctiVI29o6fzry7u4= github.com/yuin/goldmark v1.4.13 h1:fVcFKWvrslecOb/tg+Cc05dkeYx540o0FuFt3nUVDoE= github.com/yuin/goldmark v1.4.13/go.mod h1:6yULJ656Px+3vBD8DxQVa3kxgyrAnzto9xy5taEt/CY= +golang.org/x/crypto v0.17.0/go.mod h1:gCAAfMLgwOJRpTjQ2zCCt2OcSfYMTeZVSRtQlPC7Nq4= golang.org/x/crypto v0.23.0/go.mod h1:CKFgDieR+mRhux2Lsu27y0fO304Db0wZe70UKqHu0v8= golang.org/x/mod v0.17.0 h1:zY54UmvipHiNd+pm+m0x9KhZ9hl1/7QNMyxXbc6ICqA= golang.org/x/mod v0.17.0/go.mod h1:hTbmBsO62+eylJbnUtE2MGJUyE7QWk4xUqPFrRgJ+7c= diff --git a/insert_opts.go b/insert_opts.go index 423c2cde..ffd1399c 100644 --- a/insert_opts.go +++ b/insert_opts.go @@ -93,17 +93,32 @@ type InsertOpts struct { // args or queue is changed on a new job, it's allowed to be inserted as a new // job. // -// Uniquenes is checked at insert time by taking a Postgres advisory lock, doing -// a look up for an equivalent row, and inserting only if none was found. -// There's no database-level mechanism that guarantees jobs stay unique, so if -// an equivalent row is inserted out of band (or batch inserted, where a unique -// check doesn't occur), it's conceivable that duplicates could coexist. +// Uniqueness relies on a hash of the job kind and any unique properties along +// with a database unique constraint. See the note on ByState for more details +// including about the fallback to a deprecated advisory lock method. type UniqueOpts struct { // ByArgs indicates that uniqueness should be enforced for any specific // instance of encoded args for a job. // // Default is false, meaning that as long as any other unique property is // enabled, uniqueness will be enforced for a kind regardless of input args. + // + // When set to true, the entire encoded args field will be included in the + // uniqueness hash, which requires care to ensure that no irrelevant args are + // factored into the uniqueness check. It is also possible to use a subset of + // the args by indicating on the `JobArgs` struct which fields should be + // included in the uniqueness check using struct tags: + // + // type MyJobArgs struct { + // CustomerID string `json:"customer_id" river:"unique"` + // TraceID string `json:"trace_id" + // } + // + // In this example, only the encoded `customer_id` key will be included in the + // uniqueness check and the `trace_id` key will be ignored. + // + // All keys are sorted alphabetically before hashing to ensure consistent + // results. ByArgs bool // ByPeriod defines uniqueness within a given period. On an insert time is @@ -123,25 +138,34 @@ type UniqueOpts struct { ByQueue bool // ByState indicates that uniqueness should be enforced across any of the - // states in the given set. For example, if the given states were - // `(scheduled, running)` then a new job could be inserted even if one of - // the same kind was already being worked by the queue (new jobs are - // inserted as `available`). + // states in the given set. Unlike other unique options, ByState gets a + // default when it's not set for user convenience. The default is equivalent + // to: // - // Unlike other unique options, ByState gets a default when it's not set for - // user convenience. The default is equivalent to: - // - // ByState: []rivertype.JobState{rivertype.JobStateAvailable, rivertype.JobStateCompleted, rivertype.JobStateRunning, rivertype.JobStateRetryable, rivertype.JobStateScheduled} + // ByState: []rivertype.JobState{rivertype.JobStateAvailable, rivertype.JobStateCompleted, rivertype.JobStatePending, rivertype.JobStateRunning, rivertype.JobStateRetryable, rivertype.JobStateScheduled} // // With this setting, any jobs of the same kind that have been completed or - // discarded, but not yet cleaned out by the system, won't count towards the - // uniqueness of a new insert. + // discarded, but not yet cleaned out by the system, will still prevent a + // duplicate unique job from being inserted. For example, with the default + // states, if a unique job is actively `running`, a duplicate cannot be + // inserted. Likewise, if a unique job has `completed`, you still can't + // insert a duplicate, at least not until the job cleaner maintenance process + // eventually removes the completed job from the `river_job` table. + // + // The list may be safely customized to _add_ additional states (`cancelled` + // or `discarded`), though only `retryable` may be safely _removed_ from the + // list. // - // Warning: A non-default slice of states in ByState will force the unique - // inserter to fall back to a slower insertion path that takes an advisory - // lock and performs a look up before insertion. For best performance, it's - // recommended that the default set of states is used. + // Warning: Removing any states from the default list (other than `retryable` + // forces a fallback to a slower insertion path that takes an advisory lock + // and performs a look up before insertion. This path is deprecated and should + // be avoided if possible. ByState []rivertype.JobState + + // ExcludeKind indicates that the job kind should not be included in the + // uniqueness check. This is useful when you want to enforce uniqueness + // across all jobs regardless of kind. + ExcludeKind bool } // isEmpty returns true for an empty, uninitialized options struct. @@ -157,6 +181,25 @@ func (o *UniqueOpts) isEmpty() bool { o.ByState == nil } +func (o *UniqueOpts) isV1() bool { + requiredV3states := []rivertype.JobState{ + rivertype.JobStatePending, + rivertype.JobStateScheduled, + rivertype.JobStateAvailable, + rivertype.JobStateRunning, + } + if len(o.ByState) == 0 { + return false + } + + for _, state := range requiredV3states { + if !slices.Contains(o.ByState, state) { + return true + } + } + return false +} + var jobStateAll = rivertype.JobStates() //nolint:gochecknoglobals func (o *UniqueOpts) validate() error { diff --git a/insert_opts_test.go b/insert_opts_test.go index 7693f234..e41a5642 100644 --- a/insert_opts_test.go +++ b/insert_opts_test.go @@ -46,3 +46,36 @@ func TestJobUniqueOpts_validate(t *testing.T) { require.EqualError(t, (&UniqueOpts{ByPeriod: 1 * time.Millisecond}).validate(), "JobUniqueOpts.ByPeriod should not be less than 1 second") require.EqualError(t, (&UniqueOpts{ByState: []rivertype.JobState{rivertype.JobState("invalid")}}).validate(), `JobUniqueOpts.ByState contains invalid state "invalid"`) } + +func TestJobUniqueOpts_isV1(t *testing.T) { + t.Parallel() + + // Test when ByState is empty + require.False(t, (&UniqueOpts{}).isV1()) + + // Test when ByState contains none of the required V3 states + require.True(t, (&UniqueOpts{ByState: []rivertype.JobState{rivertype.JobStateCompleted}}).isV1()) + + // Test when ByState contains some but not all required V3 states + require.True(t, (&UniqueOpts{ByState: []rivertype.JobState{rivertype.JobStatePending}}).isV1()) + require.True(t, (&UniqueOpts{ByState: []rivertype.JobState{rivertype.JobStateScheduled}}).isV1()) + require.True(t, (&UniqueOpts{ByState: []rivertype.JobState{rivertype.JobStateAvailable}}).isV1()) + require.True(t, (&UniqueOpts{ByState: []rivertype.JobState{rivertype.JobStateRunning}}).isV1()) + + // Test when ByState contains all required V3 states + require.False(t, (&UniqueOpts{ByState: []rivertype.JobState{ + rivertype.JobStatePending, + rivertype.JobStateScheduled, + rivertype.JobStateAvailable, + rivertype.JobStateRunning, + }}).isV1()) + + // Test when ByState contains more than the required V3 states + require.False(t, (&UniqueOpts{ByState: []rivertype.JobState{ + rivertype.JobStatePending, + rivertype.JobStateScheduled, + rivertype.JobStateAvailable, + rivertype.JobStateRunning, + rivertype.JobStateCompleted, + }}).isV1()) +} diff --git a/internal/dbunique/db_unique.go b/internal/dbunique/db_unique.go index 196d0e4b..a153149a 100644 --- a/internal/dbunique/db_unique.go +++ b/internal/dbunique/db_unique.go @@ -9,6 +9,9 @@ import ( "strings" "time" + "github.com/tidwall/gjson" + "github.com/tidwall/sjson" + "github.com/riverqueue/river/internal/util/hashutil" "github.com/riverqueue/river/riverdriver" "github.com/riverqueue/river/rivershared/baseservice" @@ -32,18 +35,150 @@ var defaultUniqueStates = []rivertype.JobState{ //nolint:gochecknoglobals var defaultUniqueStatesStrings = sliceutil.Map(defaultUniqueStates, func(s rivertype.JobState) string { return string(s) }) //nolint:gochecknoglobals +var jobStateBitPositions = map[rivertype.JobState]uint{ //nolint:gochecknoglobals + rivertype.JobStateAvailable: 7, + rivertype.JobStateCancelled: 6, + rivertype.JobStateCompleted: 5, + rivertype.JobStateDiscarded: 4, + rivertype.JobStatePending: 3, + rivertype.JobStateRetryable: 2, + rivertype.JobStateRunning: 1, + rivertype.JobStateScheduled: 0, +} + type UniqueOpts struct { - ByArgs bool - ByPeriod time.Duration - ByQueue bool - ByState []rivertype.JobState + ByArgs bool + ByPeriod time.Duration + ByQueue bool + ByState []rivertype.JobState + ExcludeKind bool } func (o *UniqueOpts) IsEmpty() bool { return !o.ByArgs && o.ByPeriod == time.Duration(0) && !o.ByQueue && - o.ByState == nil + o.ByState == nil && + !o.ExcludeKind +} + +func (o *UniqueOpts) StateBitmask() byte { + states := defaultUniqueStates + if len(o.ByState) > 0 { + states = o.ByState + } + return UniqueStatesToBitmask(states) +} + +func UniqueKey(timeGen baseservice.TimeGenerator, uniqueOpts *UniqueOpts, params *riverdriver.JobInsertFastParams) ([]byte, error) { + uniqueKeyString, err := buildUniqueKeyString(timeGen, uniqueOpts, params) + if err != nil { + return nil, err + } + uniqueKeyHash := sha256.Sum256([]byte(uniqueKeyString)) + return uniqueKeyHash[:], nil +} + +// Builds a unique key made up of the unique options in place. The key is hashed +// to become a value for `unique_key` in the fast insertion path, or hashed and +// used for an advisory lock on the slow insertion path. +func buildUniqueKeyString(timeGen baseservice.TimeGenerator, uniqueOpts *UniqueOpts, params *riverdriver.JobInsertFastParams) (string, error) { + var sb strings.Builder + + if !uniqueOpts.ExcludeKind { + sb.WriteString("&kind=" + params.Kind) + } + + if uniqueOpts.ByArgs { + var encodedArgsForUnique []byte + // Get unique JSON keys from the JobArgs struct: + uniqueFields, err := getSortedUniqueFieldsCached(params.Args) + if err != nil { + return "", err + } + + if len(uniqueFields) > 0 { + // Extract unique values from the EncodedArgs JSON + uniqueValues := extractUniqueValues(params.EncodedArgs, uniqueFields) + + // Assemble the JSON object using bytes.Buffer + // Better to overallocate a bit than to allocate multiple times, so just + // assume we'll cap out at the length of the full encoded args. + sortedJSONWithOnlyUniqueValues := make([]byte, 0, len(params.EncodedArgs)) + + sjsonOpts := &sjson.Options{ReplaceInPlace: true} + for i, key := range uniqueFields { + if uniqueValues[i] == "undefined" { + continue + } + sortedJSONWithOnlyUniqueValues, err = sjson.SetRawBytesOptions(sortedJSONWithOnlyUniqueValues, key, []byte(uniqueValues[i]), sjsonOpts) + if err != nil { + // Should not happen unless key was invalid + return "", err + } + } + encodedArgsForUnique = sortedJSONWithOnlyUniqueValues + } else { + // Use all keys from EncodedArgs sorted alphabetically + keys := sliceutil.Map(gjson.GetBytes(params.EncodedArgs, "@keys").Array(), func(v gjson.Result) string { return v.String() }) + slices.Sort(keys) + + sortedJSON := make([]byte, 0, len(params.EncodedArgs)) + sortedJSON = append(sortedJSON, "{}"...) + sjsonOpts := &sjson.Options{ReplaceInPlace: true} + for _, key := range keys { + sortedJSON, err = sjson.SetRawBytesOptions(sortedJSON, key, []byte(gjson.GetBytes(params.EncodedArgs, key).Raw), sjsonOpts) + if err != nil { + // Should not happen unless key was invalid + return "", err + } + } + encodedArgsForUnique = sortedJSON + } + + sb.WriteString("&args=") + sb.Write(encodedArgsForUnique) + } + + if uniqueOpts.ByPeriod != time.Duration(0) { + lowerPeriodBound := timeGen.NowUTC().Truncate(uniqueOpts.ByPeriod) + sb.WriteString("&period=" + lowerPeriodBound.Format(time.RFC3339)) + } + + if uniqueOpts.ByQueue { + sb.WriteString("&queue=" + params.Queue) + } + + return sb.String(), nil +} + +func UniqueStatesToBitmask(states []rivertype.JobState) byte { + var val byte + + for _, state := range states { + bitIndex, exists := jobStateBitPositions[state] + if !exists { + continue // Ignore unknown states + } + bitPosition := 7 - (bitIndex % 8) + val |= 1 << bitPosition + } + + return val +} + +func UniqueBitmaskToStates(mask byte) []rivertype.JobState { + var states []rivertype.JobState + + for state, bitIndex := range jobStateBitPositions { + bitPosition := 7 - (bitIndex % 8) + if mask&(1< 0 { + states = tt.uniqueOpts.ByState + } - go func() { - _, err := dbutil.WithTxV(ctx, bundle.exec, func(ctx context.Context, exec riverdriver.ExecutorTx) (struct{}, error) { - res, err := inserter.JobInsert(ctx, exec, insertParams, uniqueOpts) - require.NoError(t, err) + jobParams := &riverdriver.JobInsertFastParams{ + Args: args, + CreatedAt: &now, + EncodedArgs: encodedArgs, + Kind: args.Kind(), + Metadata: []byte(`{"source":"api"}`), + Queue: "email_queue", + ScheduledAt: &now, + State: "Pending", + Tags: []string{"notification", "email"}, + UniqueStates: UniqueStatesToBitmask(states), + } - insertedJobsMu.Lock() - insertedJobs[jobNum] = res.Job - insertedJobsMu.Unlock() + uniqueKeyPreHash, err := buildUniqueKeyString(stubSvc, &tt.uniqueOpts, jobParams) + require.NoError(t, err) + require.Equal(t, tt.expectedJSON, uniqueKeyPreHash) + expectedHash := sha256.Sum256([]byte(tt.expectedJSON)) - return struct{}{}, nil - }) - require.NoError(t, err) + uniqueKey, err := UniqueKey(stubSvc, &tt.uniqueOpts, jobParams) + require.NoError(t, err) + require.NotNil(t, uniqueKey) - wg.Done() - }() - } + require.Equal(t, expectedHash[:], uniqueKey, "UniqueKey hash does not match expected value") + }) + } +} - wg.Wait() +func TestDefaultUniqueStatesSorted(t *testing.T) { + t.Parallel() - firstJobID := insertedJobs[0].ID - for i := 1; i < numContendingJobs; i++ { - require.Equal(t, firstJobID, insertedJobs[i].ID) - } - }) + states := slices.Clone(defaultUniqueStates) + slices.Sort(states) + require.Equal(t, states, defaultUniqueStates, "Default unique states should be sorted") } -func BenchmarkUniqueInserter(b *testing.B) { - ctx := context.Background() +func TestUniqueOptsIsEmpty(t *testing.T) { + t.Parallel() - type testBundle struct { - driver riverdriver.Driver[pgx.Tx] - exec riverdriver.Executor - tx pgx.Tx + emptyOpts := &UniqueOpts{} + require.True(t, emptyOpts.IsEmpty(), "Empty unique options should be empty") + + require.False(t, (&UniqueOpts{ByArgs: true}).IsEmpty(), "Unique options with ByArgs should not be empty") + require.False(t, (&UniqueOpts{ByPeriod: time.Minute}).IsEmpty(), "Unique options with ByPeriod should not be empty") + require.False(t, (&UniqueOpts{ByQueue: true}).IsEmpty(), "Unique options with ByQueue should not be empty") + require.False(t, (&UniqueOpts{ByState: []rivertype.JobState{rivertype.JobStateAvailable}}).IsEmpty(), "Unique options with ByState should not be empty") + require.False(t, (&UniqueOpts{ExcludeKind: true}).IsEmpty(), "Unique options with ExcludeKind should not be empty") + + nonEmptyOpts := &UniqueOpts{ + ByArgs: true, + ByPeriod: time.Minute, + ByQueue: true, + ByState: []rivertype.JobState{rivertype.JobStateAvailable}, + ExcludeKind: true, } + require.False(t, nonEmptyOpts.IsEmpty(), "Non-empty unique options should not be empty") +} - setup := func(b *testing.B) (*UniqueInserter, *testBundle) { - b.Helper() - - var ( - driver = riverpgxv5.New(nil) - tx = riverinternaltest.TestTx(ctx, b) - ) - - bundle := &testBundle{ - driver: driver, - exec: driver.UnwrapExecutor(tx), - tx: tx, - } +func TestUniqueOptsStateBitmask(t *testing.T) { + t.Parallel() - inserter := baseservice.Init(riversharedtest.BaseServiceArchetype(b), &UniqueInserter{}) + emptyOpts := &UniqueOpts{} + require.Equal(t, UniqueStatesToBitmask(defaultUniqueStates), emptyOpts.StateBitmask(), "Empty unique options should have default bitmask") - return inserter, bundle + otherStates := []rivertype.JobState{rivertype.JobStateAvailable, rivertype.JobStateCompleted} + nonEmptyOpts := &UniqueOpts{ + ByState: otherStates, } + require.Equal(t, UniqueStatesToBitmask([]rivertype.JobState{rivertype.JobStateAvailable, rivertype.JobStateCompleted}), nonEmptyOpts.StateBitmask(), "Non-empty unique options should have correct bitmask") +} - // Simulates the case where many existing jobs are in the database already. - // Useful as a benchmark because the advisory lock strategy's look up get - // slow with many existing jobs. - generateManyExistingJobs := func(b *testing.B, inserter *UniqueInserter, bundle *testBundle) { - b.Helper() +func TestUniqueStatesToBitmask(t *testing.T) { + t.Parallel() - insertParams := makeInsertParams(nil) + bitmask := UniqueStatesToBitmask(defaultUniqueStates) + require.Equal(t, byte(0b11110101), bitmask, "Default unique states should be all set except cancelled and discarded") - for i := 0; i < 10_000; i++ { - _, err := inserter.JobInsert(ctx, bundle.exec, insertParams, nil) - require.NoError(b, err) - } + for state, position := range jobStateBitPositions { + bitmask = UniqueStatesToBitmask([]rivertype.JobState{state}) + // Bit shifting uses postgres bit numbering with MSB on the right, so we + // need to flip the position when shifting manually: + require.Equal(t, byte(1<<(7-position)), bitmask, "Bitmask should be set for single state %s", state) } - - b.Run("FastPathEmptyDatabase", func(b *testing.B) { - inserter, bundle := setup(b) - - insertParams := makeInsertParams(nil) - uniqueOpts := &UniqueOpts{ByArgs: true} - - b.ResetTimer() - - for n := 0; n < b.N; n++ { - insertParams.EncodedArgs = []byte(fmt.Sprintf(`{"job_num":%d}`, n%1000)) - _, err := inserter.JobInsert(ctx, bundle.exec, insertParams, uniqueOpts) - require.NoError(b, err) - } - }) - - b.Run("FastPathManyExistingJobs", func(b *testing.B) { - inserter, bundle := setup(b) - - generateManyExistingJobs(b, inserter, bundle) - - insertParams := makeInsertParams(nil) - uniqueOpts := &UniqueOpts{ByArgs: true} - - b.ResetTimer() - - for n := 0; n < b.N; n++ { - insertParams.EncodedArgs = []byte(fmt.Sprintf(`{"job_num":%d}`, n%1000)) - _, err := inserter.JobInsert(ctx, bundle.exec, insertParams, uniqueOpts) - require.NoError(b, err) - } - }) - - b.Run("SlowPathEmptyDatabase", func(b *testing.B) { - inserter, bundle := setup(b) - - insertParams := makeInsertParams(nil) - uniqueOpts := &UniqueOpts{ - ByArgs: true, - ByState: []rivertype.JobState{rivertype.JobStateAvailable, rivertype.JobStateCancelled}, // use of non-standard states triggers slow path - } - - b.ResetTimer() - - for n := 0; n < b.N; n++ { - insertParams.EncodedArgs = []byte(fmt.Sprintf(`{"job_num":%d}`, n%1000)) - _, err := inserter.JobInsert(ctx, bundle.exec, insertParams, uniqueOpts) - require.NoError(b, err) - } - }) - - b.Run("SlowPathManyExistingJobs", func(b *testing.B) { - inserter, bundle := setup(b) - - generateManyExistingJobs(b, inserter, bundle) - - insertParams := makeInsertParams(nil) - uniqueOpts := &UniqueOpts{ - ByArgs: true, - ByState: []rivertype.JobState{rivertype.JobStateAvailable, rivertype.JobStateCancelled}, // use of non-standard states triggers slow path - } - - b.ResetTimer() - - for n := 0; n < b.N; n++ { - insertParams.EncodedArgs = []byte(fmt.Sprintf(`{"job_num":%d}`, n%1000)) - _, err := inserter.JobInsert(ctx, bundle.exec, insertParams, uniqueOpts) - require.NoError(b, err) - } - }) } diff --git a/internal/dbunique/unique_fields.go b/internal/dbunique/unique_fields.go new file mode 100644 index 00000000..e8062fbf --- /dev/null +++ b/internal/dbunique/unique_fields.go @@ -0,0 +1,127 @@ +package dbunique + +import ( + "fmt" + "reflect" + "sort" + "strings" + "sync" + + "github.com/tidwall/gjson" + + "github.com/riverqueue/river/rivertype" +) + +var ( + // uniqueFieldsCache caches the unique fields for each JobArgs type. These are + // global to ensure that each struct type's tags are only extracted once. + uniqueFieldsCache = make(map[reflect.Type][]string) //nolint:gochecknoglobals + cacheMutex sync.RWMutex //nolint:gochecknoglobals +) + +// extractUniqueValues extracts the raw JSON values of the specified keys from the JSON-encoded args. +func extractUniqueValues(encodedArgs []byte, uniqueKeys []string) []string { + // Use GetManyBytes to retrieve multiple values at once + results := gjson.GetManyBytes(encodedArgs, uniqueKeys...) + + uniqueValues := make([]string, len(results)) + for i, res := range results { + if res.Exists() { + uniqueValues[i] = res.Raw // Use Raw to get the JSON-encoded value + } else { + // Handle missing keys as "undefined" (they'll be skipped when building + // the unique key). We don't want to use "null" here because the JSON may + // actually contain "null" as a value. + uniqueValues[i] = "undefined" + } + } + + return uniqueValues +} + +// getSortedUniqueFields uses reflection to retrieve the JSON keys of fields +// marked with `river:"unique"` among potentially other comma-separated values. +// The return values are the JSON keys using the same logic as the `json` struct tag. +func getSortedUniqueFields(args rivertype.JobArgs) ([]string, error) { + typ := reflect.TypeOf(args) + + // Handle pointer to struct + if typ != nil && typ.Kind() == reflect.Ptr { + typ = typ.Elem() + } + + // Ensure we're dealing with a struct + if typ == nil || typ.Kind() != reflect.Struct { + return nil, fmt.Errorf("expected struct, got %T", args) + } + + var uniqueFields []string + + // Iterate over all fields + for i := 0; i < typ.NumField(); i++ { + field := typ.Field(i) + + // Check for `river:"unique"` tag, possibly among other comma-separated values + if riverTag, ok := field.Tag.Lookup("river"); ok { + // Split riverTag by comma + tags := strings.Split(riverTag, ",") + for _, tag := range tags { + if strings.TrimSpace(tag) == "unique" { + // Get the corresponding JSON key + jsonTag := field.Tag.Get("json") + if jsonTag == "" { + // If no JSON tag, use the field name as-is + uniqueFields = append(uniqueFields, field.Name) + } else { + // Handle cases like `json:"recipient,omitempty"` + jsonKey := parseJSONTag(jsonTag) + uniqueFields = append(uniqueFields, jsonKey) + } + break // No need to check other tags once "unique" is found + } + } + } + } + + // Sort the uniqueFields alphabetically for consistent ordering + sort.Strings(uniqueFields) + + return uniqueFields, nil +} + +// getSortedUniqueFieldsCached retrieves unique fields with caching to avoid +// extracting fields from the same struct type repeatedly. +func getSortedUniqueFieldsCached(args rivertype.JobArgs) ([]string, error) { + typ := reflect.TypeOf(args) + + // Check cache first + cacheMutex.RLock() + if fields, ok := uniqueFieldsCache[typ]; ok { + cacheMutex.RUnlock() + return fields, nil + } + cacheMutex.RUnlock() + + // Not in cache; retrieve using reflection + fields, err := getSortedUniqueFields(args) + if err != nil { + return nil, err + } + + // Store in cache + cacheMutex.Lock() + uniqueFieldsCache[typ] = fields + cacheMutex.Unlock() + + return fields, nil +} + +// parseJSONTag extracts the JSON key from the struct tag. +// It handles tags with options, e.g., `json:"recipient,omitempty"`. +func parseJSONTag(tag string) string { + // Tags can be like "recipient,omitempty", so split by comma + if commaIdx := strings.Index(tag, ","); commaIdx != -1 { + return tag[:commaIdx] + } + return tag +} diff --git a/internal/maintenance/job_scheduler.go b/internal/maintenance/job_scheduler.go index e94feab1..b09bc464 100644 --- a/internal/maintenance/job_scheduler.go +++ b/internal/maintenance/job_scheduler.go @@ -151,7 +151,7 @@ func (s *JobScheduler) runOnce(ctx context.Context) (*schedulerRunOnceResult, er now := s.Time.NowUTC() nowWithLookAhead := now.Add(s.config.Interval) - scheduledJobs, err := tx.JobSchedule(ctx, &riverdriver.JobScheduleParams{ + scheduledJobResults, err := tx.JobSchedule(ctx, &riverdriver.JobScheduleParams{ Max: s.config.Limit, Now: nowWithLookAhead, }) @@ -159,7 +159,7 @@ func (s *JobScheduler) runOnce(ctx context.Context) (*schedulerRunOnceResult, er return 0, fmt.Errorf("error scheduling jobs: %w", err) } - queues := make([]string, 0, len(scheduledJobs)) + queues := make([]string, 0, len(scheduledJobResults)) // Notify about scheduled jobs with a scheduled_at in the past, or just // slightly in the future (this loop, the notify, and tx commit will take @@ -167,12 +167,12 @@ func (s *JobScheduler) runOnce(ctx context.Context) (*schedulerRunOnceResult, er // is to roughly try to guess when the clients will attempt to fetch jobs. notificationHorizon := s.Time.NowUTC().Add(5 * time.Millisecond) - for _, job := range scheduledJobs { - if job.ScheduledAt.After(notificationHorizon) { + for _, result := range scheduledJobResults { + if result.Job.ScheduledAt.After(notificationHorizon) { continue } - queues = append(queues, job.Queue) + queues = append(queues, result.Job.Queue) } if len(queues) > 0 { @@ -182,7 +182,7 @@ func (s *JobScheduler) runOnce(ctx context.Context) (*schedulerRunOnceResult, er s.TestSignals.NotifiedQueues.Signal(queues) } - return len(scheduledJobs), tx.Commit(ctx) + return len(scheduledJobResults), tx.Commit(ctx) }() if err != nil { return nil, err diff --git a/internal/maintenance/job_scheduler_test.go b/internal/maintenance/job_scheduler_test.go index e5acf3ab..d2f11f64 100644 --- a/internal/maintenance/job_scheduler_test.go +++ b/internal/maintenance/job_scheduler_test.go @@ -7,7 +7,9 @@ import ( "time" "github.com/stretchr/testify/require" + "github.com/tidwall/gjson" + "github.com/riverqueue/river/internal/dbunique" "github.com/riverqueue/river/internal/riverinternaltest" "github.com/riverqueue/river/riverdriver" "github.com/riverqueue/river/riverdriver/riverpgxv5" @@ -77,6 +79,15 @@ func TestJobScheduler(t *testing.T) { require.Equal(t, rivertype.JobStateAvailable, newJob.State) return newJob } + requireJobStateDiscardedWithMeta := func(t *testing.T, exec riverdriver.Executor, job *rivertype.JobRow) *rivertype.JobRow { + t.Helper() + newJob, err := exec.JobGetByID(ctx, job.ID) + require.NoError(t, err) + require.Equal(t, rivertype.JobStateDiscarded, newJob.State) + require.NotNil(t, newJob.FinalizedAt) + require.Equal(t, "scheduler_discarded", gjson.GetBytes(newJob.Metadata, "unique_key_conflict").String()) + return newJob + } t.Run("Defaults", func(t *testing.T) { t.Parallel() @@ -139,6 +150,50 @@ func TestJobScheduler(t *testing.T) { requireJobStateUnchanged(t, bundle.exec, retryableJob3) // still retryable }) + t.Run("MovesUniqueKeyConflictingJobsToDiscarded", func(t *testing.T) { + t.Parallel() + + scheduler, bundle := setupTx(t) + now := time.Now().UTC() + + // The list of default states, but without retryable to allow for dupes in that state: + uniqueStates := []rivertype.JobState{ + rivertype.JobStateAvailable, + rivertype.JobStateCompleted, + rivertype.JobStatePending, + rivertype.JobStateRunning, + rivertype.JobStateScheduled, + } + uniqueMap := dbunique.UniqueStatesToBitmask(uniqueStates) + + retryableJob1 := testfactory.Job(ctx, t, bundle.exec, &testfactory.JobOpts{UniqueKey: []byte("1"), UniqueStates: uniqueMap, State: ptrutil.Ptr(rivertype.JobStateRetryable), ScheduledAt: ptrutil.Ptr(now.Add(-1 * time.Hour))}) + retryableJob2 := testfactory.Job(ctx, t, bundle.exec, &testfactory.JobOpts{UniqueKey: []byte("2"), UniqueStates: uniqueMap, State: ptrutil.Ptr(rivertype.JobStateRetryable), ScheduledAt: ptrutil.Ptr(now.Add(-5 * time.Second))}) + retryableJob3 := testfactory.Job(ctx, t, bundle.exec, &testfactory.JobOpts{UniqueKey: []byte("3"), UniqueStates: uniqueMap, State: ptrutil.Ptr(rivertype.JobStateRetryable), ScheduledAt: ptrutil.Ptr(now.Add(-5 * time.Second))}) // dupe + retryableJob4 := testfactory.Job(ctx, t, bundle.exec, &testfactory.JobOpts{UniqueKey: []byte("4"), UniqueStates: uniqueMap, State: ptrutil.Ptr(rivertype.JobStateRetryable), ScheduledAt: ptrutil.Ptr(now.Add(-5 * time.Second))}) // dupe + retryableJob5 := testfactory.Job(ctx, t, bundle.exec, &testfactory.JobOpts{UniqueKey: []byte("5"), UniqueStates: uniqueMap, State: ptrutil.Ptr(rivertype.JobStateRetryable), ScheduledAt: ptrutil.Ptr(now.Add(-5 * time.Second))}) // dupe + retryableJob6 := testfactory.Job(ctx, t, bundle.exec, &testfactory.JobOpts{UniqueKey: []byte("6"), UniqueStates: uniqueMap, State: ptrutil.Ptr(rivertype.JobStateRetryable), ScheduledAt: ptrutil.Ptr(now.Add(-5 * time.Second))}) // dupe + retryableJob7 := testfactory.Job(ctx, t, bundle.exec, &testfactory.JobOpts{UniqueKey: []byte("7"), UniqueStates: uniqueMap, State: ptrutil.Ptr(rivertype.JobStateRetryable), ScheduledAt: ptrutil.Ptr(now.Add(-5 * time.Second))}) // dupe + + // Will cause conflicts with above jobs when retried: + testfactory.Job(ctx, t, bundle.exec, &testfactory.JobOpts{UniqueKey: []byte("3"), UniqueStates: uniqueMap, State: ptrutil.Ptr(rivertype.JobStateAvailable)}) + testfactory.Job(ctx, t, bundle.exec, &testfactory.JobOpts{UniqueKey: []byte("4"), UniqueStates: uniqueMap, State: ptrutil.Ptr(rivertype.JobStateCompleted)}) + testfactory.Job(ctx, t, bundle.exec, &testfactory.JobOpts{UniqueKey: []byte("5"), UniqueStates: uniqueMap, State: ptrutil.Ptr(rivertype.JobStatePending)}) + testfactory.Job(ctx, t, bundle.exec, &testfactory.JobOpts{UniqueKey: []byte("6"), UniqueStates: uniqueMap, State: ptrutil.Ptr(rivertype.JobStateRunning)}) + testfactory.Job(ctx, t, bundle.exec, &testfactory.JobOpts{UniqueKey: []byte("7"), UniqueStates: uniqueMap, State: ptrutil.Ptr(rivertype.JobStateScheduled)}) + + require.NoError(t, scheduler.Start(ctx)) + + scheduler.TestSignals.ScheduledBatch.WaitOrTimeout() + + requireJobStateAvailable(t, bundle.exec, retryableJob1) + requireJobStateAvailable(t, bundle.exec, retryableJob2) + requireJobStateDiscardedWithMeta(t, bundle.exec, retryableJob3) + requireJobStateDiscardedWithMeta(t, bundle.exec, retryableJob4) + requireJobStateDiscardedWithMeta(t, bundle.exec, retryableJob5) + requireJobStateDiscardedWithMeta(t, bundle.exec, retryableJob6) + requireJobStateDiscardedWithMeta(t, bundle.exec, retryableJob7) + }) + t.Run("SchedulesInBatches", func(t *testing.T) { t.Parallel() diff --git a/internal/maintenance/periodic_job_enqueuer.go b/internal/maintenance/periodic_job_enqueuer.go index 831a642d..6fbb31af 100644 --- a/internal/maintenance/periodic_job_enqueuer.go +++ b/internal/maintenance/periodic_job_enqueuer.go @@ -231,10 +231,10 @@ func (s *PeriodicJobEnqueuer) Start(ctx context.Context) error { defer s.mu.RUnlock() var ( - insertParamsMany []*riverdriver.JobInsertFastParams + insertParamsMany []*riverdriver.JobInsertFastParams + // only contains jobs using deprecated v1 unique options: insertParamsUnique []*insertParamsAndUniqueOpts - - now = s.Time.NowUTC() + now = s.Time.NowUTC() ) // Handle periodic jobs in sorted order so we can correctly account @@ -258,10 +258,10 @@ func (s *PeriodicJobEnqueuer) Start(ctx context.Context) error { } if insertParams, uniqueOpts, ok := s.insertParamsFromConstructor(ctx, periodicJob.ConstructorFunc, now); ok { - if !uniqueOpts.IsEmpty() { - insertParamsUnique = append(insertParamsUnique, &insertParamsAndUniqueOpts{insertParams, uniqueOpts}) - } else { + if uniqueOpts == nil || uniqueOpts.IsEmpty() { insertParamsMany = append(insertParamsMany, insertParams) + } else { + insertParamsUnique = append(insertParamsUnique, &insertParamsAndUniqueOpts{insertParams, uniqueOpts}) } } } @@ -305,10 +305,10 @@ func (s *PeriodicJobEnqueuer) Start(ctx context.Context) error { } if insertParams, uniqueOpts, ok := s.insertParamsFromConstructor(ctx, periodicJob.ConstructorFunc, periodicJob.nextRunAt); ok { - if !uniqueOpts.IsEmpty() { - insertParamsUnique = append(insertParamsUnique, &insertParamsAndUniqueOpts{insertParams, uniqueOpts}) - } else { + if uniqueOpts == nil || uniqueOpts.IsEmpty() { insertParamsMany = append(insertParamsMany, insertParams) + } else { + insertParamsUnique = append(insertParamsUnique, &insertParamsAndUniqueOpts{insertParams, uniqueOpts}) } } @@ -365,13 +365,16 @@ func (s *PeriodicJobEnqueuer) insertBatch(ctx context.Context, insertParamsMany queues := make([]string, 0, len(insertParamsMany)+len(insertParamsUnique)) if len(insertParamsMany) > 0 { - if _, err := tx.JobInsertFastMany(ctx, insertParamsMany); err != nil { + results, err := tx.JobInsertFastMany(ctx, insertParamsMany) + if err != nil { s.Logger.ErrorContext(ctx, s.Name+": Error inserting periodic jobs", "error", err.Error(), "num_jobs", len(insertParamsMany)) return } - for _, params := range insertParamsMany { - queues = append(queues, params.Queue) + for _, result := range results { + if !result.UniqueSkippedAsDuplicate { + queues = append(queues, result.Job.Queue) + } } } diff --git a/internal/maintenance/periodic_job_enqueuer_test.go b/internal/maintenance/periodic_job_enqueuer_test.go index 82f57fb4..c1cb776a 100644 --- a/internal/maintenance/periodic_job_enqueuer_test.go +++ b/internal/maintenance/periodic_job_enqueuer_test.go @@ -22,6 +22,10 @@ import ( "github.com/riverqueue/river/rivertype" ) +type noOpArgs struct{} + +func (noOpArgs) Kind() string { return "no_op" } + func TestPeriodicJobEnqueuer(t *testing.T) { t.Parallel() @@ -33,16 +37,45 @@ func TestPeriodicJobEnqueuer(t *testing.T) { waitChan chan (struct{}) } + stubSvc := &riversharedtest.TimeStub{} + stubSvc.StubNowUTC(time.Now().UTC()) + jobConstructorWithQueueFunc := func(name string, unique bool, queue string) func() (*riverdriver.JobInsertFastParams, *dbunique.UniqueOpts, error) { return func() (*riverdriver.JobInsertFastParams, *dbunique.UniqueOpts, error) { - return &riverdriver.JobInsertFastParams{ + params := &riverdriver.JobInsertFastParams{ + Args: noOpArgs{}, EncodedArgs: []byte("{}"), Kind: name, MaxAttempts: rivercommon.MaxAttemptsDefault, Priority: rivercommon.PriorityDefault, Queue: queue, State: rivertype.JobStateAvailable, - }, &dbunique.UniqueOpts{ByArgs: unique}, nil + } + if unique { + uniqueOpts := &dbunique.UniqueOpts{ByArgs: true} + var err error + params.UniqueKey, err = dbunique.UniqueKey(stubSvc, uniqueOpts, params) + if err != nil { + return nil, nil, err + } + + params.UniqueStates = uniqueOpts.StateBitmask() + } + + return params, nil, nil + } + } + + jobConstructorUniqueV1Func := func(name string) func() (*riverdriver.JobInsertFastParams, *dbunique.UniqueOpts, error) { + return func() (*riverdriver.JobInsertFastParams, *dbunique.UniqueOpts, error) { + return &riverdriver.JobInsertFastParams{ + EncodedArgs: []byte("{}"), + Kind: name, + MaxAttempts: rivercommon.MaxAttemptsDefault, + Priority: rivercommon.PriorityDefault, + Queue: rivercommon.QueueDefault, + State: rivertype.JobStateAvailable, + }, &dbunique.UniqueOpts{ByArgs: true}, nil } } @@ -208,6 +241,42 @@ func TestPeriodicJobEnqueuer(t *testing.T) { } }) + t.Run("RespectsV1JobUniqueness", func(t *testing.T) { + t.Parallel() + + svc, bundle := setup(t) + + svc.AddMany([]*PeriodicJob{ + {ScheduleFunc: periodicIntervalSchedule(500 * time.Millisecond), ConstructorFunc: jobConstructorUniqueV1Func("unique_periodic_job_500ms")}, + }) + + startService(t, svc) + + // Should be no jobs to start. + requireNJobs(t, bundle.exec, "unique_periodic_job_500ms", 0) + + svc.TestSignals.InsertedJobs.WaitOrTimeout() + requireNJobs(t, bundle.exec, "unique_periodic_job_500ms", 1) + // This initial insert should emit a notification: + svc.TestSignals.NotifiedQueues.WaitOrTimeout() + + // Another insert was attempted, but there's still only one job due to + // uniqueness conditions. + svc.TestSignals.InsertedJobs.WaitOrTimeout() + requireNJobs(t, bundle.exec, "unique_periodic_job_500ms", 1) + + svc.TestSignals.InsertedJobs.WaitOrTimeout() + requireNJobs(t, bundle.exec, "unique_periodic_job_500ms", 1) + + // Ensure that no notifications were emitted beyond the first one because no + // additional jobs were inserted: + select { + case queues := <-svc.TestSignals.NotifiedQueues.WaitC(): + t.Fatalf("Expected no notification to be emitted, but got one for queues: %v", queues) + case <-time.After(100 * time.Millisecond): + } + }) + t.Run("RunOnStart", func(t *testing.T) { t.Parallel() diff --git a/internal/riverinternaltest/riverdrivertest/riverdrivertest.go b/internal/riverinternaltest/riverdrivertest/riverdrivertest.go index e8e3e76c..ce93868c 100644 --- a/internal/riverinternaltest/riverdrivertest/riverdrivertest.go +++ b/internal/riverinternaltest/riverdrivertest/riverdrivertest.go @@ -258,7 +258,6 @@ func Exercise[TTx any](ctx context.Context, t *testing.T, require.Equal(t, rivertype.JobStateCancelled, jobAfter.State) require.WithinDuration(t, time.Now(), *jobAfter.FinalizedAt, 2*time.Second) require.JSONEq(t, fmt.Sprintf(`{"cancel_attempted_at":%q}`, nowStr), string(jobAfter.Metadata)) - require.Nil(t, jobAfter.UniqueKey) }) } @@ -854,7 +853,7 @@ func Exercise[TTx any](ctx context.Context, t *testing.T, now := time.Now().UTC() - job, err := exec.JobInsertFast(ctx, &riverdriver.JobInsertFastParams{ + result, err := exec.JobInsertFast(ctx, &riverdriver.JobInsertFastParams{ EncodedArgs: []byte(`{"encoded": "args"}`), Kind: "test_kind", MaxAttempts: rivercommon.MaxAttemptsDefault, @@ -863,6 +862,7 @@ func Exercise[TTx any](ctx context.Context, t *testing.T, State: rivertype.JobStateAvailable, }) require.NoError(t, err) + job := result.Job require.Equal(t, 0, job.Attempt) require.Nil(t, job.AttemptedAt) require.WithinDuration(t, now, job.CreatedAt, 2*time.Second) @@ -886,19 +886,24 @@ func Exercise[TTx any](ctx context.Context, t *testing.T, targetTime := time.Now().UTC().Add(-15 * time.Minute) - job, err := exec.JobInsertFast(ctx, &riverdriver.JobInsertFastParams{ - CreatedAt: &targetTime, - EncodedArgs: []byte(`{"encoded": "args"}`), - Kind: "test_kind", - MaxAttempts: 6, - Metadata: []byte(`{"meta": "data"}`), - Priority: 2, - Queue: "queue_name", - ScheduledAt: &targetTime, - State: rivertype.JobStateRunning, - Tags: []string{"tag"}, + result, err := exec.JobInsertFast(ctx, &riverdriver.JobInsertFastParams{ + CreatedAt: &targetTime, + EncodedArgs: []byte(`{"encoded": "args"}`), + Kind: "test_kind", + MaxAttempts: 6, + Metadata: []byte(`{"meta": "data"}`), + Priority: 2, + Queue: "queue_name", + ScheduledAt: &targetTime, + State: rivertype.JobStateRunning, + Tags: []string{"tag"}, + UniqueKey: []byte("unique-key"), + UniqueStates: 0xFF, }) require.NoError(t, err) + + require.False(t, result.UniqueSkippedAsDuplicate) + job := result.Job require.Equal(t, 0, job.Attempt) require.Nil(t, job.AttemptedAt) requireEqualTime(t, targetTime, job.CreatedAt) @@ -939,11 +944,13 @@ func Exercise[TTx any](ctx context.Context, t *testing.T, } } - jobRows, err := exec.JobInsertFastMany(ctx, insertParams) + resultRows, err := exec.JobInsertFastMany(ctx, insertParams) require.NoError(t, err) - require.Len(t, jobRows, len(insertParams)) + require.Len(t, resultRows, len(insertParams)) - for i, job := range jobRows { + for i, result := range resultRows { + require.False(t, result.UniqueSkippedAsDuplicate) + job := result.Job require.Equal(t, 0, job.Attempt) require.Nil(t, job.AttemptedAt) require.Empty(t, job.AttemptedBy) @@ -1229,111 +1236,6 @@ func Exercise[TTx any](ctx context.Context, t *testing.T, }) }) - t.Run("JobInsertUnique", func(t *testing.T) { - t.Parallel() - - t.Run("MinimalArgsWithDefaults", func(t *testing.T) { - t.Parallel() - - exec, _ := setup(ctx, t) - - now := time.Now().UTC() - - insertRes, err := exec.JobInsertUnique(ctx, &riverdriver.JobInsertUniqueParams{ - JobInsertFastParams: &riverdriver.JobInsertFastParams{ - EncodedArgs: []byte(`{"encoded": "args"}`), - Kind: "test_kind", - MaxAttempts: rivercommon.MaxAttemptsDefault, - Priority: rivercommon.PriorityDefault, - Queue: rivercommon.QueueDefault, - State: rivertype.JobStateAvailable, - }, - UniqueKey: []byte("unique-key"), - }) - require.NoError(t, err) - require.Equal(t, 0, insertRes.Job.Attempt) - require.Nil(t, insertRes.Job.AttemptedAt) - require.WithinDuration(t, now, insertRes.Job.CreatedAt, 2*time.Second) - require.Equal(t, []byte(`{"encoded": "args"}`), insertRes.Job.EncodedArgs) - require.Empty(t, insertRes.Job.Errors) - require.Nil(t, insertRes.Job.FinalizedAt) - require.Equal(t, "test_kind", insertRes.Job.Kind) - require.Equal(t, rivercommon.MaxAttemptsDefault, insertRes.Job.MaxAttempts) - require.Equal(t, []byte(`{}`), insertRes.Job.Metadata) - require.Equal(t, rivercommon.PriorityDefault, insertRes.Job.Priority) - require.Equal(t, rivercommon.QueueDefault, insertRes.Job.Queue) - require.WithinDuration(t, now, insertRes.Job.ScheduledAt, 2*time.Second) - require.Equal(t, rivertype.JobStateAvailable, insertRes.Job.State) - require.Equal(t, []string{}, insertRes.Job.Tags) - require.Equal(t, []byte("unique-key"), insertRes.Job.UniqueKey) - }) - - t.Run("AllArgs", func(t *testing.T) { - t.Parallel() - - exec, _ := setup(ctx, t) - - targetTime := time.Now().UTC().Add(-15 * time.Minute) - - insertRes, err := exec.JobInsertUnique(ctx, &riverdriver.JobInsertUniqueParams{ - JobInsertFastParams: &riverdriver.JobInsertFastParams{ - CreatedAt: &targetTime, - EncodedArgs: []byte(`{"encoded": "args"}`), - Kind: "test_kind", - MaxAttempts: 6, - Metadata: []byte(`{"meta": "data"}`), - Priority: 2, - Queue: "queue_name", - ScheduledAt: &targetTime, - State: rivertype.JobStateRunning, - Tags: []string{"tag"}, - }, - UniqueKey: []byte("unique-key"), - }) - require.NoError(t, err) - require.Equal(t, 0, insertRes.Job.Attempt) - require.Nil(t, insertRes.Job.AttemptedAt) - requireEqualTime(t, targetTime, insertRes.Job.CreatedAt) - require.Equal(t, []byte(`{"encoded": "args"}`), insertRes.Job.EncodedArgs) - require.Empty(t, insertRes.Job.Errors) - require.Nil(t, insertRes.Job.FinalizedAt) - require.Equal(t, "test_kind", insertRes.Job.Kind) - require.Equal(t, 6, insertRes.Job.MaxAttempts) - require.Equal(t, []byte(`{"meta": "data"}`), insertRes.Job.Metadata) - require.Equal(t, 2, insertRes.Job.Priority) - require.Equal(t, "queue_name", insertRes.Job.Queue) - requireEqualTime(t, targetTime, insertRes.Job.ScheduledAt) - require.Equal(t, rivertype.JobStateRunning, insertRes.Job.State) - require.Equal(t, []string{"tag"}, insertRes.Job.Tags) - require.Equal(t, []byte("unique-key"), insertRes.Job.UniqueKey) - }) - - t.Run("ReturnsExistingOnConflict", func(t *testing.T) { - t.Parallel() - - exec, _ := setup(ctx, t) - - params := &riverdriver.JobInsertUniqueParams{ - JobInsertFastParams: &riverdriver.JobInsertFastParams{ - EncodedArgs: []byte(`{"encoded": "args"}`), - Kind: "test_kind", - MaxAttempts: rivercommon.MaxAttemptsDefault, - Priority: rivercommon.PriorityDefault, - Queue: rivercommon.QueueDefault, - State: rivertype.JobStateAvailable, - }, - UniqueKey: []byte("unique-key"), - } - - insertRes1, err := exec.JobInsertUnique(ctx, params) - require.NoError(t, err) - - insertRes2, err := exec.JobInsertUnique(ctx, params) - require.NoError(t, err) - require.Equal(t, insertRes1.Job.ID, insertRes2.Job.ID) - }) - }) - t.Run("JobList", func(t *testing.T) { t.Parallel() @@ -1343,16 +1245,18 @@ func Exercise[TTx any](ctx context.Context, t *testing.T, now := time.Now().UTC() job := testfactory.Job(ctx, t, exec, &testfactory.JobOpts{ - Attempt: ptrutil.Ptr(3), - AttemptedAt: &now, - CreatedAt: &now, - EncodedArgs: []byte(`{"encoded": "args"}`), - Errors: [][]byte{[]byte(`{"error": "message1"}`), []byte(`{"error": "message2"}`)}, - FinalizedAt: &now, - Metadata: []byte(`{"meta": "data"}`), - ScheduledAt: &now, - State: ptrutil.Ptr(rivertype.JobStateCompleted), - Tags: []string{"tag"}, + Attempt: ptrutil.Ptr(3), + AttemptedAt: &now, + CreatedAt: &now, + EncodedArgs: []byte(`{"encoded": "args"}`), + Errors: [][]byte{[]byte(`{"error": "message1"}`), []byte(`{"error": "message2"}`)}, + FinalizedAt: &now, + Metadata: []byte(`{"meta": "data"}`), + ScheduledAt: &now, + State: ptrutil.Ptr(rivertype.JobStateCompleted), + Tags: []string{"tag"}, + UniqueKey: []byte("unique-key"), + UniqueStates: 0xFF, }) fetchedJobs, err := exec.JobList( @@ -1379,6 +1283,8 @@ func Exercise[TTx any](ctx context.Context, t *testing.T, require.Equal(t, job.ScheduledAt, fetchedJob.ScheduledAt) require.Equal(t, job.State, fetchedJob.State) require.Equal(t, job.Tags, fetchedJob.Tags) + require.Equal(t, []byte("unique-key"), fetchedJob.UniqueKey) + require.Equal(t, rivertype.JobStates(), fetchedJob.UniqueStates) }) t.Run("HandlesRequiredArgumentTypes", func(t *testing.T) { @@ -1414,7 +1320,7 @@ func Exercise[TTx any](ctx context.Context, t *testing.T, exec, _ := setup(ctx, t) - require.Equal(t, "id, args, attempt, attempted_at, attempted_by, created_at, errors, finalized_at, kind, max_attempts, metadata, priority, queue, state, scheduled_at, tags", + require.Equal(t, "id, args, attempt, attempted_at, attempted_by, created_at, errors, finalized_at, kind, max_attempts, metadata, priority, queue, state, scheduled_at, tags, unique_key, unique_states", exec.JobListFields()) }) @@ -1927,25 +1833,53 @@ func Exercise[TTx any](ctx context.Context, t *testing.T, }) }) - t.Run("JobSetStateIfRunning_JobSetStateCancelled", func(t *testing.T) { //nolint:dupl + t.Run("JobSetStateIfRunning_JobSetStateCancelled", func(t *testing.T) { t.Parallel() - t.Run("DiscardsARunningJob", func(t *testing.T) { + t.Run("CancelsARunningJob", func(t *testing.T) { t.Parallel() exec, _ := setup(ctx, t) now := time.Now().UTC() + job := testfactory.Job(ctx, t, exec, &testfactory.JobOpts{ + State: ptrutil.Ptr(rivertype.JobStateRunning), + UniqueKey: []byte("unique-key"), + UniqueStates: 0xFF, + }) + + jobAfter, err := exec.JobSetStateIfRunning(ctx, riverdriver.JobSetStateCancelled(job.ID, now, makeErrPayload(t, now))) + require.NoError(t, err) + require.Equal(t, rivertype.JobStateCancelled, jobAfter.State) + require.WithinDuration(t, now, *jobAfter.FinalizedAt, time.Microsecond) + + jobUpdated, err := exec.JobGetByID(ctx, job.ID) + require.NoError(t, err) + require.Equal(t, rivertype.JobStateCancelled, jobUpdated.State) + require.Equal(t, "unique-key", string(jobUpdated.UniqueKey)) + }) + + t.Run("CancelsARunningV2UniqueJobAndClearsUniqueKey", func(t *testing.T) { //nolint:dupl + t.Parallel() + + exec, _ := setup(ctx, t) + + now := time.Now().UTC() + // V2 unique jobs (with no UniqueStates) should not have UniqueKey cleared: job := testfactory.Job(ctx, t, exec, &testfactory.JobOpts{ State: ptrutil.Ptr(rivertype.JobStateRunning), UniqueKey: []byte("unique-key"), }) + // expclitly null out UniqueStates to simulate an old v2 job: + _, err := exec.Exec(ctx, fmt.Sprintf("UPDATE river_job SET unique_states = NULL WHERE id = %d", job.ID)) + require.NoError(t, err) jobAfter, err := exec.JobSetStateIfRunning(ctx, riverdriver.JobSetStateCancelled(job.ID, now, makeErrPayload(t, now))) require.NoError(t, err) require.Equal(t, rivertype.JobStateCancelled, jobAfter.State) require.WithinDuration(t, now, *jobAfter.FinalizedAt, time.Microsecond) + require.Nil(t, jobAfter.UniqueKey) jobUpdated, err := exec.JobGetByID(ctx, job.ID) require.NoError(t, err) @@ -1954,7 +1888,7 @@ func Exercise[TTx any](ctx context.Context, t *testing.T, }) }) - t.Run("JobSetStateIfRunning_JobSetStateDiscarded", func(t *testing.T) { //nolint:dupl + t.Run("JobSetStateIfRunning_JobSetStateDiscarded", func(t *testing.T) { t.Parallel() t.Run("DiscardsARunningJob", func(t *testing.T) { @@ -1964,15 +1898,44 @@ func Exercise[TTx any](ctx context.Context, t *testing.T, now := time.Now().UTC() + job := testfactory.Job(ctx, t, exec, &testfactory.JobOpts{ + State: ptrutil.Ptr(rivertype.JobStateRunning), + UniqueKey: []byte("unique-key"), + UniqueStates: 0xFF, + }) + + jobAfter, err := exec.JobSetStateIfRunning(ctx, riverdriver.JobSetStateDiscarded(job.ID, now, makeErrPayload(t, now))) + require.NoError(t, err) + require.Equal(t, rivertype.JobStateDiscarded, jobAfter.State) + require.WithinDuration(t, now, *jobAfter.FinalizedAt, time.Microsecond) + require.Equal(t, "unique-key", string(jobAfter.UniqueKey)) + require.Equal(t, rivertype.JobStates(), jobAfter.UniqueStates) + + jobUpdated, err := exec.JobGetByID(ctx, job.ID) + require.NoError(t, err) + require.Equal(t, rivertype.JobStateDiscarded, jobUpdated.State) + }) + + t.Run("DiscardsARunningV2UniqueJobAndClearsUniqueKey", func(t *testing.T) { //nolint:dupl + t.Parallel() + + exec, _ := setup(ctx, t) + + now := time.Now().UTC() + // V2 unique jobs (with no UniqueStates) should not have UniqueKey cleared: job := testfactory.Job(ctx, t, exec, &testfactory.JobOpts{ State: ptrutil.Ptr(rivertype.JobStateRunning), UniqueKey: []byte("unique-key"), }) + // expclitly null out UniqueStates to simulate an old v2 job: + _, err := exec.Exec(ctx, fmt.Sprintf("UPDATE river_job SET unique_states = NULL WHERE id = %d", job.ID)) + require.NoError(t, err) jobAfter, err := exec.JobSetStateIfRunning(ctx, riverdriver.JobSetStateDiscarded(job.ID, now, makeErrPayload(t, now))) require.NoError(t, err) require.Equal(t, rivertype.JobStateDiscarded, jobAfter.State) require.WithinDuration(t, now, *jobAfter.FinalizedAt, time.Microsecond) + require.Nil(t, jobAfter.UniqueKey) jobUpdated, err := exec.JobGetByID(ctx, job.ID) require.NoError(t, err) diff --git a/job_executor_test.go b/job_executor_test.go index e99de1a1..a181b99c 100644 --- a/job_executor_test.go +++ b/job_executor_test.go @@ -141,7 +141,7 @@ func TestJobExecutor_Execute(t *testing.T) { workUnitFactory := newWorkUnitFactoryWithCustomRetry(func() error { return nil }, nil) - job, err := exec.JobInsertFast(ctx, &riverdriver.JobInsertFastParams{ + result, err := exec.JobInsertFast(ctx, &riverdriver.JobInsertFastParams{ EncodedArgs: []byte("{}"), Kind: (callbackArgs{}).Kind(), MaxAttempts: rivercommon.MaxAttemptsDefault, @@ -158,8 +158,8 @@ func TestJobExecutor_Execute(t *testing.T) { }) require.NoError(t, err) require.Len(t, jobs, 1) - require.Equal(t, job.ID, jobs[0].ID) - job = jobs[0] + require.Equal(t, result.Job.ID, jobs[0].ID) + job := jobs[0] bundle := &testBundle{ completer: completer, @@ -334,10 +334,8 @@ func TestJobExecutor_Execute(t *testing.T) { // add a unique key so we can verify it's cleared var err error bundle.jobRow, err = bundle.exec.JobUpdate(ctx, &riverdriver.JobUpdateParams{ - ID: bundle.jobRow.ID, - State: rivertype.JobStateAvailable, // required for encoding but ignored - UniqueKeyDoUpdate: true, - UniqueKey: []byte("unique-key"), + ID: bundle.jobRow.ID, + State: rivertype.JobStateAvailable, // required for encoding but ignored }) require.NoError(t, err) diff --git a/periodic_job.go b/periodic_job.go index 20648e09..5d043a21 100644 --- a/periodic_job.go +++ b/periodic_job.go @@ -186,7 +186,7 @@ func (b *PeriodicJobBundle) toInternal(periodicJob *PeriodicJob) *maintenance.Pe if args == nil { return nil, nil, maintenance.ErrNoJobToInsert } - return insertParamsFromConfigArgsAndOptions(&b.periodicJobEnqueuer.Archetype, b.clientConfig, args, options) + return insertParamsFromConfigArgsAndOptions(&b.periodicJobEnqueuer.Archetype, b.clientConfig, args, options, false) }, RunOnStart: opts.RunOnStart, ScheduleFunc: periodicJob.scheduleFunc.Next, diff --git a/producer_test.go b/producer_test.go index bcb9d159..eec07da0 100644 --- a/producer_test.go +++ b/producer_test.go @@ -102,7 +102,7 @@ func Test_Producer_CanSafelyCompleteJobsWhileFetchingNewOnes(t *testing.T) { params := make([]*riverdriver.JobInsertFastParams, maxJobCount) for i := range params { - insertParams, _, err := insertParamsFromConfigArgsAndOptions(archetype, config, WithJobNumArgs{JobNum: i}, nil) + insertParams, _, err := insertParamsFromConfigArgsAndOptions(archetype, config, WithJobNumArgs{JobNum: i}, nil, true) require.NoError(err) params[i] = insertParams @@ -277,7 +277,7 @@ func testProducer(t *testing.T, makeProducer func(ctx context.Context, t *testin mustInsert := func(ctx context.Context, t *testing.T, bundle *testBundle, args JobArgs) { t.Helper() - insertParams, _, err := insertParamsFromConfigArgsAndOptions(bundle.archetype, bundle.config, args, nil) + insertParams, _, err := insertParamsFromConfigArgsAndOptions(bundle.archetype, bundle.config, args, nil, false) require.NoError(t, err) _, err = bundle.exec.JobInsertFast(ctx, insertParams) diff --git a/riverdriver/river_driver_interface.go b/riverdriver/river_driver_interface.go index e94c64a1..2cedb610 100644 --- a/riverdriver/river_driver_interface.go +++ b/riverdriver/river_driver_interface.go @@ -115,16 +115,15 @@ type Executor interface { JobGetByKindAndUniqueProperties(ctx context.Context, params *JobGetByKindAndUniquePropertiesParams) (*rivertype.JobRow, error) JobGetByKindMany(ctx context.Context, kind []string) ([]*rivertype.JobRow, error) JobGetStuck(ctx context.Context, params *JobGetStuckParams) ([]*rivertype.JobRow, error) - JobInsertFast(ctx context.Context, params *JobInsertFastParams) (*rivertype.JobRow, error) - JobInsertFastMany(ctx context.Context, params []*JobInsertFastParams) ([]*rivertype.JobRow, error) + JobInsertFast(ctx context.Context, params *JobInsertFastParams) (*JobInsertFastResult, error) + JobInsertFastMany(ctx context.Context, params []*JobInsertFastParams) ([]*JobInsertFastResult, error) JobInsertFastManyNoReturning(ctx context.Context, params []*JobInsertFastParams) (int, error) JobInsertFull(ctx context.Context, params *JobInsertFullParams) (*rivertype.JobRow, error) - JobInsertUnique(ctx context.Context, params *JobInsertUniqueParams) (*JobInsertUniqueResult, error) JobList(ctx context.Context, query string, namedArgs map[string]any) ([]*rivertype.JobRow, error) JobListFields() string JobRescueMany(ctx context.Context, params *JobRescueManyParams) (*struct{}, error) JobRetry(ctx context.Context, id int64) (*rivertype.JobRow, error) - JobSchedule(ctx context.Context, params *JobScheduleParams) ([]*rivertype.JobRow, error) + JobSchedule(ctx context.Context, params *JobScheduleParams) ([]*JobScheduleResult, error) JobSetCompleteIfRunningMany(ctx context.Context, params *JobSetCompleteIfRunningManyParams) ([]*rivertype.JobRow, error) JobSetStateIfRunning(ctx context.Context, params *JobSetStateIfRunningParams) (*rivertype.JobRow, error) JobUpdate(ctx context.Context, params *JobUpdateParams) (*rivertype.JobRow, error) @@ -249,44 +248,46 @@ type JobGetStuckParams struct { } type JobInsertFastParams struct { - CreatedAt *time.Time - EncodedArgs []byte - Kind string - MaxAttempts int - Metadata []byte - Priority int - Queue string - ScheduledAt *time.Time - State rivertype.JobState - Tags []string -} - -type JobInsertUniqueParams struct { - *JobInsertFastParams - UniqueKey []byte -} - -type JobInsertUniqueResult struct { + // Args contains the raw underlying job arguments struct. It has already been + // encoded into EncodedArgs, but the original is kept here for to leverage its + // struct tags and interfaces, such as for use in unique key generation. + Args rivertype.JobArgs + CreatedAt *time.Time + EncodedArgs []byte + Kind string + MaxAttempts int + Metadata []byte + Priority int + Queue string + ScheduledAt *time.Time + State rivertype.JobState + Tags []string + UniqueKey []byte + UniqueStates byte +} + +type JobInsertFastResult struct { Job *rivertype.JobRow UniqueSkippedAsDuplicate bool } type JobInsertFullParams struct { - Attempt int - AttemptedAt *time.Time - CreatedAt *time.Time - EncodedArgs []byte - Errors [][]byte - FinalizedAt *time.Time - Kind string - MaxAttempts int - Metadata []byte - Priority int - Queue string - ScheduledAt *time.Time - State rivertype.JobState - Tags []string - UniqueKey []byte + Attempt int + AttemptedAt *time.Time + CreatedAt *time.Time + EncodedArgs []byte + Errors [][]byte + FinalizedAt *time.Time + Kind string + MaxAttempts int + Metadata []byte + Priority int + Queue string + ScheduledAt *time.Time + State rivertype.JobState + Tags []string + UniqueKey []byte + UniqueStates byte } type JobRescueManyParams struct { @@ -303,8 +304,8 @@ type JobScheduleParams struct { } type JobScheduleResult struct { - Queue string - ScheduledAt time.Time + Job rivertype.JobRow + ConflictDiscarded bool } // JobSetCompleteIfRunningManyParams are parameters to set many running jobs to @@ -366,8 +367,10 @@ type JobUpdateParams struct { FinalizedAt *time.Time StateDoUpdate bool State rivertype.JobState - UniqueKeyDoUpdate bool - UniqueKey []byte + // Deprecated and will be removed when advisory lock unique path is removed. + UniqueKeyDoUpdate bool + // Deprecated and will be removed when advisory lock unique path is removed. + UniqueKey []byte } // Leader represents a River leader. diff --git a/riverdriver/riverdatabasesql/go.mod b/riverdriver/riverdatabasesql/go.mod index 07f1c784..6c7f784a 100644 --- a/riverdriver/riverdatabasesql/go.mod +++ b/riverdriver/riverdatabasesql/go.mod @@ -5,6 +5,7 @@ go 1.21 toolchain go1.23.0 require ( + github.com/jackc/pgx/v5 v5.7.1 github.com/lib/pq v1.10.9 github.com/riverqueue/river/riverdriver v0.11.4 github.com/riverqueue/river/rivershared v0.11.4 diff --git a/riverdriver/riverdatabasesql/go.sum b/riverdriver/riverdatabasesql/go.sum index 49610545..ef0854f9 100644 --- a/riverdriver/riverdatabasesql/go.sum +++ b/riverdriver/riverdatabasesql/go.sum @@ -1,6 +1,14 @@ github.com/creack/pty v1.1.9/go.mod h1:oKZEueFk5CKHvIhNR5MUki03XCEU+Q6VDXinZuGJ33E= github.com/davecgh/go-spew v1.1.1 h1:vj9j/u1bqnvCEfJOwUhtlOARqs3+rkHYY13jYWTU97c= github.com/davecgh/go-spew v1.1.1/go.mod h1:J7Y8YcW2NihsgmVo/mv3lAwl/skON4iLHjSsI+c5H38= +github.com/jackc/pgpassfile v1.0.0 h1:/6Hmqy13Ss2zCq62VdNG8tM1wchn8zjSGOBJ6icpsIM= +github.com/jackc/pgpassfile v1.0.0/go.mod h1:CEx0iS5ambNFdcRtxPj5JhEz+xB6uRky5eyVu/W2HEg= +github.com/jackc/pgservicefile v0.0.0-20240606120523-5a60cdf6a761 h1:iCEnooe7UlwOQYpKFhBabPMi4aNAfoODPEFNiAnClxo= +github.com/jackc/pgservicefile v0.0.0-20240606120523-5a60cdf6a761/go.mod h1:5TJZWKEWniPve33vlWYSoGYefn3gLQRzjfDlhSJ9ZKM= +github.com/jackc/pgx/v5 v5.7.1 h1:x7SYsPBYDkHDksogeSmZZ5xzThcTgRz++I5E+ePFUcs= +github.com/jackc/pgx/v5 v5.7.1/go.mod h1:e7O26IywZZ+naJtWWos6i6fvWK+29etgITqrqHLfoZA= +github.com/jackc/puddle/v2 v2.2.2 h1:PR8nw+E/1w0GLuRFSmiioY6UooMp6KJv0/61nB7icHo= +github.com/jackc/puddle/v2 v2.2.2/go.mod h1:vriiEXHvEE654aYKXXjOvZM39qJ0q+azkZFrfEOc3H4= github.com/kr/pretty v0.3.0 h1:WgNl7dwNpEZ6jJ9k1snq4pZsg7DOEN8hP9Xw0Tsjwk0= github.com/kr/pretty v0.3.0/go.mod h1:640gp4NfQd8pI5XOwp5fnNeVWj67G7CFk/SaSQn7NBk= github.com/kr/text v0.2.0 h1:5Nx0Ya0ZqY2ygV366QzturHI13Jq95ApcVaJBhpS+AY= @@ -19,6 +27,12 @@ github.com/rogpeppe/go-internal v1.12.0 h1:exVL4IDcn6na9z1rAb56Vxr+CgyK3nn3O+epU github.com/rogpeppe/go-internal v1.12.0/go.mod h1:E+RYuTGaKKdloAfM02xzb0FW3Paa99yedzYV+kq4uf4= github.com/stretchr/testify v1.9.0 h1:HtqpIVDClZ4nwg75+f6Lvsy/wHu+3BoSGCbBAcpTsTg= github.com/stretchr/testify v1.9.0/go.mod h1:r2ic/lqez/lEtzL7wO/rwa5dbSLXVDPFyf8C91i36aY= +golang.org/x/crypto v0.27.0 h1:GXm2NjJrPaiv/h1tb2UH8QfgC/hOf/+z0p6PT8o1w7A= +golang.org/x/crypto v0.27.0/go.mod h1:1Xngt8kV6Dvbssa53Ziq6Eqn0HqbZi5Z6R0ZpwQzt70= +golang.org/x/sync v0.8.0 h1:3NFvSEYkUoMifnESzZl15y791HH1qU2xm6eCJU5ZPXQ= +golang.org/x/sync v0.8.0/go.mod h1:Czt+wKu1gCyEFDUtn0jG5QVvpJ6rzVqr5aXyt9drQfk= +golang.org/x/text v0.18.0 h1:XvMDiNzPAl0jr17s6W9lcaIhGUfUORdGCNsuLmPG224= +golang.org/x/text v0.18.0/go.mod h1:BuEKDfySbSR4drPmRPG/7iBdf8hvFMuRexcpahXilzY= gopkg.in/check.v1 v0.0.0-20161208181325-20d25e280405/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0= gopkg.in/check.v1 v1.0.0-20201130134442-10cb98267c6c h1:Hei/4ADfdWqJk1ZMxUNpqntNwaWcugrBjAiHlqqRiVk= gopkg.in/check.v1 v1.0.0-20201130134442-10cb98267c6c/go.mod h1:JHkPIbrfpd72SG/EVd6muEfDQjcINNoR0C8j2r3qZ4Q= diff --git a/riverdriver/riverdatabasesql/internal/dbsqlc/models.go b/riverdriver/riverdatabasesql/internal/dbsqlc/models.go index ab99f050..866e0dbc 100644 --- a/riverdriver/riverdatabasesql/internal/dbsqlc/models.go +++ b/riverdriver/riverdatabasesql/internal/dbsqlc/models.go @@ -8,6 +8,8 @@ import ( "database/sql/driver" "fmt" "time" + + "github.com/riverqueue/river/riverdriver/riverdatabasesql/internal/pgtypealias" ) type RiverJobState string @@ -78,23 +80,24 @@ type RiverClientQueue struct { } type RiverJob struct { - ID int64 - Args string - Attempt int16 - AttemptedAt *time.Time - AttemptedBy []string - CreatedAt time.Time - Errors []string - FinalizedAt *time.Time - Kind string - MaxAttempts int16 - Metadata string - Priority int16 - Queue string - State RiverJobState - ScheduledAt time.Time - Tags []string - UniqueKey []byte + ID int64 + Args string + Attempt int16 + AttemptedAt *time.Time + AttemptedBy []string + CreatedAt time.Time + Errors []string + FinalizedAt *time.Time + Kind string + MaxAttempts int16 + Metadata string + Priority int16 + Queue string + State RiverJobState + ScheduledAt time.Time + Tags []string + UniqueKey []byte + UniqueStates pgtypealias.Bits } type RiverLeader struct { diff --git a/riverdriver/riverdatabasesql/internal/dbsqlc/river_job.sql.go b/riverdriver/riverdatabasesql/internal/dbsqlc/river_job.sql.go index dafdc2d9..77f16cda 100644 --- a/riverdriver/riverdatabasesql/internal/dbsqlc/river_job.sql.go +++ b/riverdriver/riverdatabasesql/internal/dbsqlc/river_job.sql.go @@ -10,6 +10,7 @@ import ( "time" "github.com/lib/pq" + "github.com/riverqueue/river/riverdriver/riverdatabasesql/internal/pgtypealias" ) const jobCancel = `-- name: JobCancel :one @@ -44,18 +45,23 @@ updated_job AS ( -- rescue it, even if it gets stuck in the running state: metadata = jsonb_set(metadata, '{cancel_attempted_at}'::text[], $3::jsonb, true), -- Similarly, zero a ` + "`" + `unique_key` + "`" + ` if the job is transitioning directly - -- to cancelled. Otherwise, it'll be clear the job executor. - unique_key = CASE WHEN state = 'running' THEN unique_key ELSE NULL END + -- to cancelled. Otherwise, it'll be cleared in the job executor. + -- + -- This is transition code to support existing jobs using the old v2 + -- uniqueness design. We specifically avoid clearing this value if the + -- v3 unique_states field is populated, because the v3 design never + -- involves clearing unique_key. + unique_key = CASE WHEN (state = 'running' OR unique_states IS NOT NULL) THEN unique_key ELSE NULL END FROM notification WHERE river_job.id = notification.id - RETURNING river_job.id, river_job.args, river_job.attempt, river_job.attempted_at, river_job.attempted_by, river_job.created_at, river_job.errors, river_job.finalized_at, river_job.kind, river_job.max_attempts, river_job.metadata, river_job.priority, river_job.queue, river_job.state, river_job.scheduled_at, river_job.tags, river_job.unique_key + RETURNING river_job.id, river_job.args, river_job.attempt, river_job.attempted_at, river_job.attempted_by, river_job.created_at, river_job.errors, river_job.finalized_at, river_job.kind, river_job.max_attempts, river_job.metadata, river_job.priority, river_job.queue, river_job.state, river_job.scheduled_at, river_job.tags, river_job.unique_key, river_job.unique_states ) -SELECT id, args, attempt, attempted_at, attempted_by, created_at, errors, finalized_at, kind, max_attempts, metadata, priority, queue, state, scheduled_at, tags, unique_key +SELECT id, args, attempt, attempted_at, attempted_by, created_at, errors, finalized_at, kind, max_attempts, metadata, priority, queue, state, scheduled_at, tags, unique_key, unique_states FROM river_job WHERE id = $1::bigint AND id NOT IN (SELECT id FROM updated_job) UNION -SELECT id, args, attempt, attempted_at, attempted_by, created_at, errors, finalized_at, kind, max_attempts, metadata, priority, queue, state, scheduled_at, tags, unique_key +SELECT id, args, attempt, attempted_at, attempted_by, created_at, errors, finalized_at, kind, max_attempts, metadata, priority, queue, state, scheduled_at, tags, unique_key, unique_states FROM updated_job ` @@ -86,6 +92,7 @@ func (q *Queries) JobCancel(ctx context.Context, db DBTX, arg *JobCancelParams) &i.ScheduledAt, pq.Array(&i.Tags), &i.UniqueKey, + &i.UniqueStates, ) return &i, err } @@ -117,14 +124,14 @@ deleted_job AS ( WHERE river_job.id = job_to_delete.id -- Do not touch running jobs: AND river_job.state != 'running' - RETURNING river_job.id, river_job.args, river_job.attempt, river_job.attempted_at, river_job.attempted_by, river_job.created_at, river_job.errors, river_job.finalized_at, river_job.kind, river_job.max_attempts, river_job.metadata, river_job.priority, river_job.queue, river_job.state, river_job.scheduled_at, river_job.tags, river_job.unique_key + RETURNING river_job.id, river_job.args, river_job.attempt, river_job.attempted_at, river_job.attempted_by, river_job.created_at, river_job.errors, river_job.finalized_at, river_job.kind, river_job.max_attempts, river_job.metadata, river_job.priority, river_job.queue, river_job.state, river_job.scheduled_at, river_job.tags, river_job.unique_key, river_job.unique_states ) -SELECT id, args, attempt, attempted_at, attempted_by, created_at, errors, finalized_at, kind, max_attempts, metadata, priority, queue, state, scheduled_at, tags, unique_key +SELECT id, args, attempt, attempted_at, attempted_by, created_at, errors, finalized_at, kind, max_attempts, metadata, priority, queue, state, scheduled_at, tags, unique_key, unique_states FROM river_job WHERE id = $1::bigint AND id NOT IN (SELECT id FROM deleted_job) UNION -SELECT id, args, attempt, attempted_at, attempted_by, created_at, errors, finalized_at, kind, max_attempts, metadata, priority, queue, state, scheduled_at, tags, unique_key +SELECT id, args, attempt, attempted_at, attempted_by, created_at, errors, finalized_at, kind, max_attempts, metadata, priority, queue, state, scheduled_at, tags, unique_key, unique_states FROM deleted_job ` @@ -149,6 +156,7 @@ func (q *Queries) JobDelete(ctx context.Context, db DBTX, id int64) (*RiverJob, &i.ScheduledAt, pq.Array(&i.Tags), &i.UniqueKey, + &i.UniqueStates, ) return &i, err } @@ -166,7 +174,7 @@ WITH deleted_jobs AS ( ORDER BY id LIMIT $4::bigint ) - RETURNING id, args, attempt, attempted_at, attempted_by, created_at, errors, finalized_at, kind, max_attempts, metadata, priority, queue, state, scheduled_at, tags, unique_key + RETURNING id, args, attempt, attempted_at, attempted_by, created_at, errors, finalized_at, kind, max_attempts, metadata, priority, queue, state, scheduled_at, tags, unique_key, unique_states ) SELECT count(*) FROM deleted_jobs @@ -194,7 +202,7 @@ func (q *Queries) JobDeleteBefore(ctx context.Context, db DBTX, arg *JobDeleteBe const jobGetAvailable = `-- name: JobGetAvailable :many WITH locked_jobs AS ( SELECT - id, args, attempt, attempted_at, attempted_by, created_at, errors, finalized_at, kind, max_attempts, metadata, priority, queue, state, scheduled_at, tags, unique_key + id, args, attempt, attempted_at, attempted_by, created_at, errors, finalized_at, kind, max_attempts, metadata, priority, queue, state, scheduled_at, tags, unique_key, unique_states FROM river_job WHERE @@ -221,7 +229,7 @@ FROM WHERE river_job.id = locked_jobs.id RETURNING - river_job.id, river_job.args, river_job.attempt, river_job.attempted_at, river_job.attempted_by, river_job.created_at, river_job.errors, river_job.finalized_at, river_job.kind, river_job.max_attempts, river_job.metadata, river_job.priority, river_job.queue, river_job.state, river_job.scheduled_at, river_job.tags, river_job.unique_key + river_job.id, river_job.args, river_job.attempt, river_job.attempted_at, river_job.attempted_by, river_job.created_at, river_job.errors, river_job.finalized_at, river_job.kind, river_job.max_attempts, river_job.metadata, river_job.priority, river_job.queue, river_job.state, river_job.scheduled_at, river_job.tags, river_job.unique_key, river_job.unique_states ` type JobGetAvailableParams struct { @@ -257,6 +265,7 @@ func (q *Queries) JobGetAvailable(ctx context.Context, db DBTX, arg *JobGetAvail &i.ScheduledAt, pq.Array(&i.Tags), &i.UniqueKey, + &i.UniqueStates, ); err != nil { return nil, err } @@ -272,7 +281,7 @@ func (q *Queries) JobGetAvailable(ctx context.Context, db DBTX, arg *JobGetAvail } const jobGetByID = `-- name: JobGetByID :one -SELECT id, args, attempt, attempted_at, attempted_by, created_at, errors, finalized_at, kind, max_attempts, metadata, priority, queue, state, scheduled_at, tags, unique_key +SELECT id, args, attempt, attempted_at, attempted_by, created_at, errors, finalized_at, kind, max_attempts, metadata, priority, queue, state, scheduled_at, tags, unique_key, unique_states FROM river_job WHERE id = $1 LIMIT 1 @@ -299,12 +308,13 @@ func (q *Queries) JobGetByID(ctx context.Context, db DBTX, id int64) (*RiverJob, &i.ScheduledAt, pq.Array(&i.Tags), &i.UniqueKey, + &i.UniqueStates, ) return &i, err } const jobGetByIDMany = `-- name: JobGetByIDMany :many -SELECT id, args, attempt, attempted_at, attempted_by, created_at, errors, finalized_at, kind, max_attempts, metadata, priority, queue, state, scheduled_at, tags, unique_key +SELECT id, args, attempt, attempted_at, attempted_by, created_at, errors, finalized_at, kind, max_attempts, metadata, priority, queue, state, scheduled_at, tags, unique_key, unique_states FROM river_job WHERE id = any($1::bigint[]) ORDER BY id @@ -337,6 +347,7 @@ func (q *Queries) JobGetByIDMany(ctx context.Context, db DBTX, id []int64) ([]*R &i.ScheduledAt, pq.Array(&i.Tags), &i.UniqueKey, + &i.UniqueStates, ); err != nil { return nil, err } @@ -352,7 +363,7 @@ func (q *Queries) JobGetByIDMany(ctx context.Context, db DBTX, id []int64) ([]*R } const jobGetByKindAndUniqueProperties = `-- name: JobGetByKindAndUniqueProperties :one -SELECT id, args, attempt, attempted_at, attempted_by, created_at, errors, finalized_at, kind, max_attempts, metadata, priority, queue, state, scheduled_at, tags, unique_key +SELECT id, args, attempt, attempted_at, attempted_by, created_at, errors, finalized_at, kind, max_attempts, metadata, priority, queue, state, scheduled_at, tags, unique_key, unique_states FROM river_job WHERE kind = $1 AND CASE WHEN $2::boolean THEN args = $3 ELSE true END @@ -406,12 +417,13 @@ func (q *Queries) JobGetByKindAndUniqueProperties(ctx context.Context, db DBTX, &i.ScheduledAt, pq.Array(&i.Tags), &i.UniqueKey, + &i.UniqueStates, ) return &i, err } const jobGetByKindMany = `-- name: JobGetByKindMany :many -SELECT id, args, attempt, attempted_at, attempted_by, created_at, errors, finalized_at, kind, max_attempts, metadata, priority, queue, state, scheduled_at, tags, unique_key +SELECT id, args, attempt, attempted_at, attempted_by, created_at, errors, finalized_at, kind, max_attempts, metadata, priority, queue, state, scheduled_at, tags, unique_key, unique_states FROM river_job WHERE kind = any($1::text[]) ORDER BY id @@ -444,6 +456,7 @@ func (q *Queries) JobGetByKindMany(ctx context.Context, db DBTX, kind []string) &i.ScheduledAt, pq.Array(&i.Tags), &i.UniqueKey, + &i.UniqueStates, ); err != nil { return nil, err } @@ -459,7 +472,7 @@ func (q *Queries) JobGetByKindMany(ctx context.Context, db DBTX, kind []string) } const jobGetStuck = `-- name: JobGetStuck :many -SELECT id, args, attempt, attempted_at, attempted_by, created_at, errors, finalized_at, kind, max_attempts, metadata, priority, queue, state, scheduled_at, tags, unique_key +SELECT id, args, attempt, attempted_at, attempted_by, created_at, errors, finalized_at, kind, max_attempts, metadata, priority, queue, state, scheduled_at, tags, unique_key, unique_states FROM river_job WHERE state = 'running' AND attempted_at < $1::timestamptz @@ -499,6 +512,7 @@ func (q *Queries) JobGetStuck(ctx context.Context, db DBTX, arg *JobGetStuckPara &i.ScheduledAt, pq.Array(&i.Tags), &i.UniqueKey, + &i.UniqueStates, ); err != nil { return nil, err } @@ -525,7 +539,9 @@ INSERT INTO river_job( queue, scheduled_at, state, - tags + tags, + unique_key, + unique_states ) VALUES ( $1, coalesce($2::timestamptz, now()), @@ -537,25 +553,41 @@ INSERT INTO river_job( $8, coalesce($9::timestamptz, now()), $10, - coalesce($11::varchar(255)[], '{}') -) RETURNING id, args, attempt, attempted_at, attempted_by, created_at, errors, finalized_at, kind, max_attempts, metadata, priority, queue, state, scheduled_at, tags, unique_key + coalesce($11::varchar(255)[], '{}'), + $12, + $13 +) +ON CONFLICT (unique_key) + WHERE unique_key IS NOT NULL + AND unique_states IS NOT NULL + AND river_job_state_in_bitmask(unique_states, state) + -- Something needs to be updated for a row to be returned on a conflict. + DO UPDATE SET kind = EXCLUDED.kind +RETURNING river_job.id, river_job.args, river_job.attempt, river_job.attempted_at, river_job.attempted_by, river_job.created_at, river_job.errors, river_job.finalized_at, river_job.kind, river_job.max_attempts, river_job.metadata, river_job.priority, river_job.queue, river_job.state, river_job.scheduled_at, river_job.tags, river_job.unique_key, river_job.unique_states, (xmax != 0) AS unique_skipped_as_duplicate ` type JobInsertFastParams struct { - Args string - CreatedAt *time.Time - FinalizedAt *time.Time - Kind string - MaxAttempts int16 - Metadata string - Priority int16 - Queue string - ScheduledAt *time.Time - State RiverJobState - Tags []string + Args string + CreatedAt *time.Time + FinalizedAt *time.Time + Kind string + MaxAttempts int16 + Metadata string + Priority int16 + Queue string + ScheduledAt *time.Time + State RiverJobState + Tags []string + UniqueKey []byte + UniqueStates pgtypealias.Bits } -func (q *Queries) JobInsertFast(ctx context.Context, db DBTX, arg *JobInsertFastParams) (*RiverJob, error) { +type JobInsertFastRow struct { + RiverJob RiverJob + UniqueSkippedAsDuplicate bool +} + +func (q *Queries) JobInsertFast(ctx context.Context, db DBTX, arg *JobInsertFastParams) (*JobInsertFastRow, error) { row := db.QueryRowContext(ctx, jobInsertFast, arg.Args, arg.CreatedAt, @@ -568,26 +600,30 @@ func (q *Queries) JobInsertFast(ctx context.Context, db DBTX, arg *JobInsertFast arg.ScheduledAt, arg.State, pq.Array(arg.Tags), + arg.UniqueKey, + arg.UniqueStates, ) - var i RiverJob + var i JobInsertFastRow err := row.Scan( - &i.ID, - &i.Args, - &i.Attempt, - &i.AttemptedAt, - pq.Array(&i.AttemptedBy), - &i.CreatedAt, - pq.Array(&i.Errors), - &i.FinalizedAt, - &i.Kind, - &i.MaxAttempts, - &i.Metadata, - &i.Priority, - &i.Queue, - &i.State, - &i.ScheduledAt, - pq.Array(&i.Tags), - &i.UniqueKey, + &i.RiverJob.ID, + &i.RiverJob.Args, + &i.RiverJob.Attempt, + &i.RiverJob.AttemptedAt, + pq.Array(&i.RiverJob.AttemptedBy), + &i.RiverJob.CreatedAt, + pq.Array(&i.RiverJob.Errors), + &i.RiverJob.FinalizedAt, + &i.RiverJob.Kind, + &i.RiverJob.MaxAttempts, + &i.RiverJob.Metadata, + &i.RiverJob.Priority, + &i.RiverJob.Queue, + &i.RiverJob.State, + &i.RiverJob.ScheduledAt, + pq.Array(&i.RiverJob.Tags), + &i.RiverJob.UniqueKey, + &i.RiverJob.UniqueStates, + &i.UniqueSkippedAsDuplicate, ) return &i, err } @@ -602,7 +638,9 @@ INSERT INTO river_job( queue, scheduled_at, state, - tags + tags, + unique_key, + unique_states ) SELECT unnest($1::jsonb[]), unnest($2::text[]), @@ -617,23 +655,40 @@ INSERT INTO river_job( -- Unnest on a multi-dimensional array will fully flatten the array, so we -- encode the tag list as a comma-separated string and split it in the -- query. - string_to_array(unnest($9::text[]), ',') -RETURNING id, args, attempt, attempted_at, attempted_by, created_at, errors, finalized_at, kind, max_attempts, metadata, priority, queue, state, scheduled_at, tags, unique_key + string_to_array(unnest($9::text[]), ','), + + unnest($10::bytea[]), + unnest($11::bit(8)[]) + +ON CONFLICT (unique_key) + WHERE unique_key IS NOT NULL + AND unique_states IS NOT NULL + AND river_job_state_in_bitmask(unique_states, state) + -- Something needs to be updated for a row to be returned on a conflict. + DO UPDATE SET kind = EXCLUDED.kind +RETURNING river_job.id, river_job.args, river_job.attempt, river_job.attempted_at, river_job.attempted_by, river_job.created_at, river_job.errors, river_job.finalized_at, river_job.kind, river_job.max_attempts, river_job.metadata, river_job.priority, river_job.queue, river_job.state, river_job.scheduled_at, river_job.tags, river_job.unique_key, river_job.unique_states, (xmax != 0) AS unique_skipped_as_duplicate ` type JobInsertFastManyParams struct { - Args []string - Kind []string - MaxAttempts []int16 - Metadata []string - Priority []int16 - Queue []string - ScheduledAt []time.Time - State []string - Tags []string + Args []string + Kind []string + MaxAttempts []int16 + Metadata []string + Priority []int16 + Queue []string + ScheduledAt []time.Time + State []string + Tags []string + UniqueKey [][]byte + UniqueStates []pgtypealias.Bits } -func (q *Queries) JobInsertFastMany(ctx context.Context, db DBTX, arg *JobInsertFastManyParams) ([]*RiverJob, error) { +type JobInsertFastManyRow struct { + RiverJob RiverJob + UniqueSkippedAsDuplicate bool +} + +func (q *Queries) JobInsertFastMany(ctx context.Context, db DBTX, arg *JobInsertFastManyParams) ([]*JobInsertFastManyRow, error) { rows, err := db.QueryContext(ctx, jobInsertFastMany, pq.Array(arg.Args), pq.Array(arg.Kind), @@ -644,32 +699,36 @@ func (q *Queries) JobInsertFastMany(ctx context.Context, db DBTX, arg *JobInsert pq.Array(arg.ScheduledAt), pq.Array(arg.State), pq.Array(arg.Tags), + pq.Array(arg.UniqueKey), + pq.Array(arg.UniqueStates), ) if err != nil { return nil, err } defer rows.Close() - var items []*RiverJob + var items []*JobInsertFastManyRow for rows.Next() { - var i RiverJob + var i JobInsertFastManyRow if err := rows.Scan( - &i.ID, - &i.Args, - &i.Attempt, - &i.AttemptedAt, - pq.Array(&i.AttemptedBy), - &i.CreatedAt, - pq.Array(&i.Errors), - &i.FinalizedAt, - &i.Kind, - &i.MaxAttempts, - &i.Metadata, - &i.Priority, - &i.Queue, - &i.State, - &i.ScheduledAt, - pq.Array(&i.Tags), - &i.UniqueKey, + &i.RiverJob.ID, + &i.RiverJob.Args, + &i.RiverJob.Attempt, + &i.RiverJob.AttemptedAt, + pq.Array(&i.RiverJob.AttemptedBy), + &i.RiverJob.CreatedAt, + pq.Array(&i.RiverJob.Errors), + &i.RiverJob.FinalizedAt, + &i.RiverJob.Kind, + &i.RiverJob.MaxAttempts, + &i.RiverJob.Metadata, + &i.RiverJob.Priority, + &i.RiverJob.Queue, + &i.RiverJob.State, + &i.RiverJob.ScheduledAt, + pq.Array(&i.RiverJob.Tags), + &i.RiverJob.UniqueKey, + &i.RiverJob.UniqueStates, + &i.UniqueSkippedAsDuplicate, ); err != nil { return nil, err } @@ -694,7 +753,9 @@ INSERT INTO river_job( queue, scheduled_at, state, - tags + tags, + unique_key, + unique_states ) SELECT unnest($1::jsonb[]), unnest($2::text[]), @@ -709,19 +770,30 @@ INSERT INTO river_job( -- so instead we pack each set of tags into a string, send them through, -- then unpack them here into an array to put in each row. This isn't -- necessary in the Pgx driver where copyfrom is used instead. - string_to_array(unnest($9::text[]), ',') + string_to_array(unnest($9::text[]), ','), + + unnest($10::bytea[]), + unnest($11::bit(8)[]) + +ON CONFLICT (unique_key) + WHERE unique_key IS NOT NULL + AND unique_states IS NOT NULL + AND river_job_state_in_bitmask(unique_states, state) +DO NOTHING ` type JobInsertFastManyNoReturningParams struct { - Args []string - Kind []string - MaxAttempts []int16 - Metadata []string - Priority []int16 - Queue []string - ScheduledAt []time.Time - State []RiverJobState - Tags []string + Args []string + Kind []string + MaxAttempts []int16 + Metadata []string + Priority []int16 + Queue []string + ScheduledAt []time.Time + State []RiverJobState + Tags []string + UniqueKey [][]byte + UniqueStates []pgtypealias.Bits } func (q *Queries) JobInsertFastManyNoReturning(ctx context.Context, db DBTX, arg *JobInsertFastManyNoReturningParams) (int64, error) { @@ -735,6 +807,8 @@ func (q *Queries) JobInsertFastManyNoReturning(ctx context.Context, db DBTX, arg pq.Array(arg.ScheduledAt), pq.Array(arg.State), pq.Array(arg.Tags), + pq.Array(arg.UniqueKey), + pq.Array(arg.UniqueStates), ) if err != nil { return 0, err @@ -758,7 +832,8 @@ INSERT INTO river_job( scheduled_at, state, tags, - unique_key + unique_key, + unique_states ) VALUES ( $1::jsonb, coalesce($2::smallint, 0), @@ -774,26 +849,28 @@ INSERT INTO river_job( coalesce($12::timestamptz, now()), $13, coalesce($14::varchar(255)[], '{}'), - $15 -) RETURNING id, args, attempt, attempted_at, attempted_by, created_at, errors, finalized_at, kind, max_attempts, metadata, priority, queue, state, scheduled_at, tags, unique_key + $15, + $16 +) RETURNING id, args, attempt, attempted_at, attempted_by, created_at, errors, finalized_at, kind, max_attempts, metadata, priority, queue, state, scheduled_at, tags, unique_key, unique_states ` type JobInsertFullParams struct { - Args string - Attempt int16 - AttemptedAt *time.Time - CreatedAt *time.Time - Errors []string - FinalizedAt *time.Time - Kind string - MaxAttempts int16 - Metadata string - Priority int16 - Queue string - ScheduledAt *time.Time - State RiverJobState - Tags []string - UniqueKey []byte + Args string + Attempt int16 + AttemptedAt *time.Time + CreatedAt *time.Time + Errors []string + FinalizedAt *time.Time + Kind string + MaxAttempts int16 + Metadata string + Priority int16 + Queue string + ScheduledAt *time.Time + State RiverJobState + Tags []string + UniqueKey []byte + UniqueStates pgtypealias.Bits } func (q *Queries) JobInsertFull(ctx context.Context, db DBTX, arg *JobInsertFullParams) (*RiverJob, error) { @@ -813,6 +890,7 @@ func (q *Queries) JobInsertFull(ctx context.Context, db DBTX, arg *JobInsertFull arg.State, pq.Array(arg.Tags), arg.UniqueKey, + arg.UniqueStates, ) var i RiverJob err := row.Scan( @@ -833,99 +911,7 @@ func (q *Queries) JobInsertFull(ctx context.Context, db DBTX, arg *JobInsertFull &i.ScheduledAt, pq.Array(&i.Tags), &i.UniqueKey, - ) - return &i, err -} - -const jobInsertUnique = `-- name: JobInsertUnique :one -INSERT INTO river_job( - args, - created_at, - finalized_at, - kind, - max_attempts, - metadata, - priority, - queue, - scheduled_at, - state, - tags, - unique_key -) VALUES ( - $1, - coalesce($2::timestamptz, now()), - $3, - $4, - $5, - coalesce($6::jsonb, '{}'), - $7, - $8, - coalesce($9::timestamptz, now()), - $10, - coalesce($11::varchar(255)[], '{}'), - $12 -) -ON CONFLICT (kind, unique_key) WHERE unique_key IS NOT NULL - -- Something needs to be updated for a row to be returned on a conflict. - DO UPDATE SET kind = EXCLUDED.kind -RETURNING river_job.id, river_job.args, river_job.attempt, river_job.attempted_at, river_job.attempted_by, river_job.created_at, river_job.errors, river_job.finalized_at, river_job.kind, river_job.max_attempts, river_job.metadata, river_job.priority, river_job.queue, river_job.state, river_job.scheduled_at, river_job.tags, river_job.unique_key, (xmax != 0) AS unique_skipped_as_duplicate -` - -type JobInsertUniqueParams struct { - Args string - CreatedAt *time.Time - FinalizedAt *time.Time - Kind string - MaxAttempts int16 - Metadata string - Priority int16 - Queue string - ScheduledAt *time.Time - State RiverJobState - Tags []string - UniqueKey []byte -} - -type JobInsertUniqueRow struct { - RiverJob RiverJob - UniqueSkippedAsDuplicate bool -} - -func (q *Queries) JobInsertUnique(ctx context.Context, db DBTX, arg *JobInsertUniqueParams) (*JobInsertUniqueRow, error) { - row := db.QueryRowContext(ctx, jobInsertUnique, - arg.Args, - arg.CreatedAt, - arg.FinalizedAt, - arg.Kind, - arg.MaxAttempts, - arg.Metadata, - arg.Priority, - arg.Queue, - arg.ScheduledAt, - arg.State, - pq.Array(arg.Tags), - arg.UniqueKey, - ) - var i JobInsertUniqueRow - err := row.Scan( - &i.RiverJob.ID, - &i.RiverJob.Args, - &i.RiverJob.Attempt, - &i.RiverJob.AttemptedAt, - pq.Array(&i.RiverJob.AttemptedBy), - &i.RiverJob.CreatedAt, - pq.Array(&i.RiverJob.Errors), - &i.RiverJob.FinalizedAt, - &i.RiverJob.Kind, - &i.RiverJob.MaxAttempts, - &i.RiverJob.Metadata, - &i.RiverJob.Priority, - &i.RiverJob.Queue, - &i.RiverJob.State, - &i.RiverJob.ScheduledAt, - pq.Array(&i.RiverJob.Tags), - &i.RiverJob.UniqueKey, - &i.UniqueSkippedAsDuplicate, + &i.UniqueStates, ) return &i, err } @@ -988,14 +974,14 @@ updated_job AS ( AND river_job.state != 'running' -- If the job is already available with a prior scheduled_at, leave it alone. AND NOT (river_job.state = 'available' AND river_job.scheduled_at < now()) - RETURNING river_job.id, river_job.args, river_job.attempt, river_job.attempted_at, river_job.attempted_by, river_job.created_at, river_job.errors, river_job.finalized_at, river_job.kind, river_job.max_attempts, river_job.metadata, river_job.priority, river_job.queue, river_job.state, river_job.scheduled_at, river_job.tags, river_job.unique_key + RETURNING river_job.id, river_job.args, river_job.attempt, river_job.attempted_at, river_job.attempted_by, river_job.created_at, river_job.errors, river_job.finalized_at, river_job.kind, river_job.max_attempts, river_job.metadata, river_job.priority, river_job.queue, river_job.state, river_job.scheduled_at, river_job.tags, river_job.unique_key, river_job.unique_states ) -SELECT id, args, attempt, attempted_at, attempted_by, created_at, errors, finalized_at, kind, max_attempts, metadata, priority, queue, state, scheduled_at, tags, unique_key +SELECT id, args, attempt, attempted_at, attempted_by, created_at, errors, finalized_at, kind, max_attempts, metadata, priority, queue, state, scheduled_at, tags, unique_key, unique_states FROM river_job WHERE id = $1::bigint AND id NOT IN (SELECT id FROM updated_job) UNION -SELECT id, args, attempt, attempted_at, attempted_by, created_at, errors, finalized_at, kind, max_attempts, metadata, priority, queue, state, scheduled_at, tags, unique_key +SELECT id, args, attempt, attempted_at, attempted_by, created_at, errors, finalized_at, kind, max_attempts, metadata, priority, queue, state, scheduled_at, tags, unique_key, unique_states FROM updated_job ` @@ -1020,19 +1006,20 @@ func (q *Queries) JobRetry(ctx context.Context, db DBTX, id int64) (*RiverJob, e &i.ScheduledAt, pq.Array(&i.Tags), &i.UniqueKey, + &i.UniqueStates, ) return &i, err } const jobSchedule = `-- name: JobSchedule :many WITH jobs_to_schedule AS ( - SELECT id + SELECT id, unique_key, unique_states FROM river_job WHERE state IN ('retryable', 'scheduled') AND queue IS NOT NULL AND priority >= 0 - AND river_job.scheduled_at <= $1::timestamptz + AND scheduled_at <= $1::timestamptz ORDER BY priority, scheduled_at, @@ -1040,16 +1027,38 @@ WITH jobs_to_schedule AS ( LIMIT $2::bigint FOR UPDATE ), -river_job_scheduled AS ( +conflicting_jobs AS ( + SELECT DISTINCT unique_key + FROM river_job + WHERE unique_key IN ( + SELECT unique_key + FROM jobs_to_schedule + WHERE unique_key IS NOT NULL + AND unique_states IS NOT NULL + ) + AND unique_states IS NOT NULL + AND river_job_state_in_bitmask(unique_states, state) +), +updated_jobs AS ( UPDATE river_job - SET state = 'available' - FROM jobs_to_schedule - WHERE river_job.id = jobs_to_schedule.id - RETURNING river_job.id + SET + state = CASE WHEN cj.unique_key IS NULL THEN 'available'::river_job_state + ELSE 'discarded'::river_job_state END, + finalized_at = CASE WHEN cj.unique_key IS NULL THEN finalized_at + ELSE $1::timestamptz END, + -- Purely for debugging to understand when this code path was used: + metadata = CASE WHEN cj.unique_key IS NULL THEN metadata + ELSE metadata || '{"unique_key_conflict": "scheduler_discarded"}'::jsonb END + FROM jobs_to_schedule jts + LEFT JOIN conflicting_jobs cj ON jts.unique_key = cj.unique_key + WHERE river_job.id = jts.id + RETURNING river_job.id, state = 'discarded'::river_job_state AS conflict_discarded ) -SELECT id, args, attempt, attempted_at, attempted_by, created_at, errors, finalized_at, kind, max_attempts, metadata, priority, queue, state, scheduled_at, tags, unique_key +SELECT + river_job.id, river_job.args, river_job.attempt, river_job.attempted_at, river_job.attempted_by, river_job.created_at, river_job.errors, river_job.finalized_at, river_job.kind, river_job.max_attempts, river_job.metadata, river_job.priority, river_job.queue, river_job.state, river_job.scheduled_at, river_job.tags, river_job.unique_key, river_job.unique_states, + updated_jobs.conflict_discarded FROM river_job -WHERE id IN (SELECT id FROM river_job_scheduled) +JOIN updated_jobs ON river_job.id = updated_jobs.id ` type JobScheduleParams struct { @@ -1057,33 +1066,40 @@ type JobScheduleParams struct { Max int64 } -func (q *Queries) JobSchedule(ctx context.Context, db DBTX, arg *JobScheduleParams) ([]*RiverJob, error) { +type JobScheduleRow struct { + RiverJob RiverJob + ConflictDiscarded bool +} + +func (q *Queries) JobSchedule(ctx context.Context, db DBTX, arg *JobScheduleParams) ([]*JobScheduleRow, error) { rows, err := db.QueryContext(ctx, jobSchedule, arg.Now, arg.Max) if err != nil { return nil, err } defer rows.Close() - var items []*RiverJob + var items []*JobScheduleRow for rows.Next() { - var i RiverJob + var i JobScheduleRow if err := rows.Scan( - &i.ID, - &i.Args, - &i.Attempt, - &i.AttemptedAt, - pq.Array(&i.AttemptedBy), - &i.CreatedAt, - pq.Array(&i.Errors), - &i.FinalizedAt, - &i.Kind, - &i.MaxAttempts, - &i.Metadata, - &i.Priority, - &i.Queue, - &i.State, - &i.ScheduledAt, - pq.Array(&i.Tags), - &i.UniqueKey, + &i.RiverJob.ID, + &i.RiverJob.Args, + &i.RiverJob.Attempt, + &i.RiverJob.AttemptedAt, + pq.Array(&i.RiverJob.AttemptedBy), + &i.RiverJob.CreatedAt, + pq.Array(&i.RiverJob.Errors), + &i.RiverJob.FinalizedAt, + &i.RiverJob.Kind, + &i.RiverJob.MaxAttempts, + &i.RiverJob.Metadata, + &i.RiverJob.Priority, + &i.RiverJob.Queue, + &i.RiverJob.State, + &i.RiverJob.ScheduledAt, + pq.Array(&i.RiverJob.Tags), + &i.RiverJob.UniqueKey, + &i.RiverJob.UniqueStates, + &i.ConflictDiscarded, ); err != nil { return nil, err } @@ -1118,13 +1134,13 @@ updated_job AS ( state = 'completed' FROM job_to_update WHERE river_job.id = job_to_update.id - RETURNING river_job.id, river_job.args, river_job.attempt, river_job.attempted_at, river_job.attempted_by, river_job.created_at, river_job.errors, river_job.finalized_at, river_job.kind, river_job.max_attempts, river_job.metadata, river_job.priority, river_job.queue, river_job.state, river_job.scheduled_at, river_job.tags, river_job.unique_key + RETURNING river_job.id, river_job.args, river_job.attempt, river_job.attempted_at, river_job.attempted_by, river_job.created_at, river_job.errors, river_job.finalized_at, river_job.kind, river_job.max_attempts, river_job.metadata, river_job.priority, river_job.queue, river_job.state, river_job.scheduled_at, river_job.tags, river_job.unique_key, river_job.unique_states ) -SELECT id, args, attempt, attempted_at, attempted_by, created_at, errors, finalized_at, kind, max_attempts, metadata, priority, queue, state, scheduled_at, tags, unique_key +SELECT id, args, attempt, attempted_at, attempted_by, created_at, errors, finalized_at, kind, max_attempts, metadata, priority, queue, state, scheduled_at, tags, unique_key, unique_states FROM river_job WHERE id IN (SELECT id FROM job_to_finalized_at EXCEPT SELECT id FROM updated_job) UNION -SELECT id, args, attempt, attempted_at, attempted_by, created_at, errors, finalized_at, kind, max_attempts, metadata, priority, queue, state, scheduled_at, tags, unique_key +SELECT id, args, attempt, attempted_at, attempted_by, created_at, errors, finalized_at, kind, max_attempts, metadata, priority, queue, state, scheduled_at, tags, unique_key, unique_states FROM updated_job ` @@ -1160,6 +1176,7 @@ func (q *Queries) JobSetCompleteIfRunningMany(ctx context.Context, db DBTX, arg &i.ScheduledAt, pq.Array(&i.Tags), &i.UniqueKey, + &i.UniqueStates, ); err != nil { return nil, err } @@ -1197,19 +1214,22 @@ updated_job AS ( ELSE max_attempts END, scheduled_at = CASE WHEN NOT should_cancel AND $9::boolean THEN $10::timestamptz ELSE scheduled_at END, - unique_key = CASE WHEN ($1 IN ('cancelled', 'discarded') OR should_cancel) THEN NULL + -- This is transitional code for the v2 uniqueness design. We specifically + -- avoid clearing this value if the v3 unique_states field is populated, + -- because the v3 design never involves clearing unique_key. + unique_key = CASE WHEN (unique_states IS NULL AND ($1 IN ('cancelled', 'discarded') OR should_cancel)) THEN NULL ELSE unique_key END FROM job_to_update WHERE river_job.id = job_to_update.id AND river_job.state = 'running' - RETURNING river_job.id, river_job.args, river_job.attempt, river_job.attempted_at, river_job.attempted_by, river_job.created_at, river_job.errors, river_job.finalized_at, river_job.kind, river_job.max_attempts, river_job.metadata, river_job.priority, river_job.queue, river_job.state, river_job.scheduled_at, river_job.tags, river_job.unique_key + RETURNING river_job.id, river_job.args, river_job.attempt, river_job.attempted_at, river_job.attempted_by, river_job.created_at, river_job.errors, river_job.finalized_at, river_job.kind, river_job.max_attempts, river_job.metadata, river_job.priority, river_job.queue, river_job.state, river_job.scheduled_at, river_job.tags, river_job.unique_key, river_job.unique_states ) -SELECT id, args, attempt, attempted_at, attempted_by, created_at, errors, finalized_at, kind, max_attempts, metadata, priority, queue, state, scheduled_at, tags, unique_key +SELECT id, args, attempt, attempted_at, attempted_by, created_at, errors, finalized_at, kind, max_attempts, metadata, priority, queue, state, scheduled_at, tags, unique_key, unique_states FROM river_job WHERE id = $2::bigint AND id NOT IN (SELECT id FROM updated_job) UNION -SELECT id, args, attempt, attempted_at, attempted_by, created_at, errors, finalized_at, kind, max_attempts, metadata, priority, queue, state, scheduled_at, tags, unique_key +SELECT id, args, attempt, attempted_at, attempted_by, created_at, errors, finalized_at, kind, max_attempts, metadata, priority, queue, state, scheduled_at, tags, unique_key, unique_states FROM updated_job ` @@ -1258,6 +1278,7 @@ func (q *Queries) JobSetStateIfRunning(ctx context.Context, db DBTX, arg *JobSet &i.ScheduledAt, pq.Array(&i.Tags), &i.UniqueKey, + &i.UniqueStates, ) return &i, err } @@ -1270,9 +1291,11 @@ SET errors = CASE WHEN $5::boolean THEN $6::jsonb[] ELSE errors END, finalized_at = CASE WHEN $7::boolean THEN $8 ELSE finalized_at END, state = CASE WHEN $9::boolean THEN $10 ELSE state END, + -- Transitional code to support tests for v2 uniqueness design. This field + -- is never modified in the v3 design. unique_key = CASE WHEN $11::boolean THEN $12 ELSE unique_key END WHERE id = $13 -RETURNING id, args, attempt, attempted_at, attempted_by, created_at, errors, finalized_at, kind, max_attempts, metadata, priority, queue, state, scheduled_at, tags, unique_key +RETURNING id, args, attempt, attempted_at, attempted_by, created_at, errors, finalized_at, kind, max_attempts, metadata, priority, queue, state, scheduled_at, tags, unique_key, unique_states ` type JobUpdateParams struct { @@ -1328,6 +1351,7 @@ func (q *Queries) JobUpdate(ctx context.Context, db DBTX, arg *JobUpdateParams) &i.ScheduledAt, pq.Array(&i.Tags), &i.UniqueKey, + &i.UniqueStates, ) return &i, err } diff --git a/riverdriver/riverdatabasesql/internal/dbsqlc/sqlc.yaml b/riverdriver/riverdatabasesql/internal/dbsqlc/sqlc.yaml index 9bb74f7d..0290d662 100644 --- a/riverdriver/riverdatabasesql/internal/dbsqlc/sqlc.yaml +++ b/riverdriver/riverdatabasesql/internal/dbsqlc/sqlc.yaml @@ -52,3 +52,23 @@ sql: type: "time.Time" pointer: true nullable: true + + # There doesn't appear to be a good type that's suitable for database/sql other + # than the ones in pgtype. It's not great to make the database/sql driver take + # a dependency on pgx, but the reality is most users will (or should) be using + # pgx anyway. + # + # Unfortunately due to some sqlc limitations, you can't just use the + # pgtype package directly (it tries to use the non-v5 import path and + # you end up with duplicate pgtype imports). So there's an alias + # package that exposes it indirectly. + - db_type: "pg_catalog.bit" + go_type: + import: "github.com/riverqueue/river/riverdriver/riverdatabasesql/internal/pgtypealias" + type: "Bits" + + - db_type: "pg_catalog.bit" + go_type: + import: "github.com/riverqueue/river/riverdriver/riverdatabasesql/internal/pgtypealias" + type: "Bits" + nullable: true diff --git a/riverdriver/riverdatabasesql/internal/pgtypealias/pgtype_alias.go b/riverdriver/riverdatabasesql/internal/pgtypealias/pgtype_alias.go new file mode 100644 index 00000000..52dcbb21 --- /dev/null +++ b/riverdriver/riverdatabasesql/internal/pgtypealias/pgtype_alias.go @@ -0,0 +1,9 @@ +// package pgtypealias exists to work aronud sqlc bugs with being able to +// reference v5 the pgtype package from within a dbsql package. +package pgtypealias + +import "github.com/jackc/pgx/v5/pgtype" + +type Bits struct { + pgtype.Bits +} diff --git a/riverdriver/riverdatabasesql/migration/main/006_bulk_unique.down.sql b/riverdriver/riverdatabasesql/migration/main/006_bulk_unique.down.sql new file mode 100644 index 00000000..ae65cfed --- /dev/null +++ b/riverdriver/riverdatabasesql/migration/main/006_bulk_unique.down.sql @@ -0,0 +1,16 @@ + +-- +-- Drop `river_job.unique_states` and its index. +-- + +DROP INDEX river_job_unique_idx; + +ALTER TABLE river_job + DROP COLUMN unique_states; + +CREATE UNIQUE INDEX IF NOT EXISTS river_job_kind_unique_key_idx ON river_job (kind, unique_key) WHERE unique_key IS NOT NULL; + +-- +-- Drop `river_job_state_in_bitmask` function. +-- +DROP FUNCTION river_job_state_in_bitmask; diff --git a/riverdriver/riverdatabasesql/migration/main/006_bulk_unique.up.sql b/riverdriver/riverdatabasesql/migration/main/006_bulk_unique.up.sql new file mode 100644 index 00000000..e149dfba --- /dev/null +++ b/riverdriver/riverdatabasesql/migration/main/006_bulk_unique.up.sql @@ -0,0 +1,38 @@ + +CREATE OR REPLACE FUNCTION river_job_state_in_bitmask(bitmask BIT(8), state river_job_state) +RETURNS boolean +LANGUAGE SQL +IMMUTABLE +AS $$ + SELECT CASE state + WHEN 'available' THEN get_bit(bitmask, 7) + WHEN 'cancelled' THEN get_bit(bitmask, 6) + WHEN 'completed' THEN get_bit(bitmask, 5) + WHEN 'discarded' THEN get_bit(bitmask, 4) + WHEN 'pending' THEN get_bit(bitmask, 3) + WHEN 'retryable' THEN get_bit(bitmask, 2) + WHEN 'running' THEN get_bit(bitmask, 1) + WHEN 'scheduled' THEN get_bit(bitmask, 0) + ELSE 0 + END = 1; +$$; + +-- +-- Add `river_job.unique_states` and bring up an index on it. +-- +ALTER TABLE river_job ADD COLUMN unique_states BIT(8); + +-- This statements uses `IF NOT EXISTS` to allow users with a `river_job` table +-- of non-trivial size to build the index `CONCURRENTLY` out of band of this +-- migration, then follow by completing the migration. +CREATE UNIQUE INDEX IF NOT EXISTS river_job_unique_idx ON river_job (unique_key) + WHERE unique_key IS NOT NULL + AND unique_states IS NOT NULL + AND river_job_state_in_bitmask(unique_states, state); + +-- Remove the old unique index. Users who are actively using the unique jobs +-- feature and who wish to avoid deploy downtime may want od drop this in a +-- subsequent migration once all jobs using the old unique system have been +-- completed (i.e. no more rows with non-null unique_key and null +-- unique_states). +DROP INDEX river_job_kind_unique_key_idx; diff --git a/riverdriver/riverdatabasesql/river_database_sql_driver.go b/riverdriver/riverdatabasesql/river_database_sql_driver.go index b43076cd..4e4e894e 100644 --- a/riverdriver/riverdatabasesql/river_database_sql_driver.go +++ b/riverdriver/riverdatabasesql/river_database_sql_driver.go @@ -18,10 +18,13 @@ import ( "strings" "time" + "github.com/jackc/pgx/v5/pgtype" "github.com/lib/pq" + "github.com/riverqueue/river/internal/dbunique" "github.com/riverqueue/river/riverdriver" "github.com/riverqueue/river/riverdriver/riverdatabasesql/internal/dbsqlc" + "github.com/riverqueue/river/riverdriver/riverdatabasesql/internal/pgtypealias" "github.com/riverqueue/river/rivershared/util/sliceutil" "github.com/riverqueue/river/rivershared/util/valutil" "github.com/riverqueue/river/rivertype" @@ -197,36 +200,44 @@ func (e *Executor) JobGetStuck(ctx context.Context, params *riverdriver.JobGetSt return mapSliceError(jobs, jobRowFromInternal) } -func (e *Executor) JobInsertFast(ctx context.Context, params *riverdriver.JobInsertFastParams) (*rivertype.JobRow, error) { - job, err := dbsqlc.New().JobInsertFast(ctx, e.dbtx, &dbsqlc.JobInsertFastParams{ - Args: string(params.EncodedArgs), - CreatedAt: params.CreatedAt, - Kind: params.Kind, - MaxAttempts: int16(min(params.MaxAttempts, math.MaxInt16)), //nolint:gosec - Metadata: valutil.ValOrDefault(string(params.Metadata), "{}"), - Priority: int16(min(params.Priority, math.MaxInt16)), //nolint:gosec - Queue: params.Queue, - ScheduledAt: params.ScheduledAt, - State: dbsqlc.RiverJobState(params.State), - Tags: params.Tags, +func (e *Executor) JobInsertFast(ctx context.Context, params *riverdriver.JobInsertFastParams) (*riverdriver.JobInsertFastResult, error) { + result, err := dbsqlc.New().JobInsertFast(ctx, e.dbtx, &dbsqlc.JobInsertFastParams{ + Args: string(params.EncodedArgs), + CreatedAt: params.CreatedAt, + Kind: params.Kind, + MaxAttempts: int16(min(params.MaxAttempts, math.MaxInt16)), //nolint:gosec + Metadata: valutil.ValOrDefault(string(params.Metadata), "{}"), + Priority: int16(min(params.Priority, math.MaxInt16)), //nolint:gosec + Queue: params.Queue, + ScheduledAt: params.ScheduledAt, + State: dbsqlc.RiverJobState(params.State), + Tags: params.Tags, + UniqueKey: params.UniqueKey, + UniqueStates: pgtypealias.Bits{Bits: pgtype.Bits{Bytes: []byte{params.UniqueStates}, Len: 8, Valid: params.UniqueStates != 0}}, }) if err != nil { return nil, interpretError(err) } - return jobRowFromInternal(job) + externalJob, err := jobRowFromInternal(&result.RiverJob) + if err != nil { + return nil, err + } + return &riverdriver.JobInsertFastResult{Job: externalJob, UniqueSkippedAsDuplicate: result.UniqueSkippedAsDuplicate}, nil } -func (e *Executor) JobInsertFastMany(ctx context.Context, params []*riverdriver.JobInsertFastParams) ([]*rivertype.JobRow, error) { +func (e *Executor) JobInsertFastMany(ctx context.Context, params []*riverdriver.JobInsertFastParams) ([]*riverdriver.JobInsertFastResult, error) { insertJobsParams := &dbsqlc.JobInsertFastManyParams{ - Args: make([]string, len(params)), - Kind: make([]string, len(params)), - MaxAttempts: make([]int16, len(params)), - Metadata: make([]string, len(params)), - Priority: make([]int16, len(params)), - Queue: make([]string, len(params)), - ScheduledAt: make([]time.Time, len(params)), - State: make([]string, len(params)), - Tags: make([]string, len(params)), + Args: make([]string, len(params)), + Kind: make([]string, len(params)), + MaxAttempts: make([]int16, len(params)), + Metadata: make([]string, len(params)), + Priority: make([]int16, len(params)), + Queue: make([]string, len(params)), + ScheduledAt: make([]time.Time, len(params)), + State: make([]string, len(params)), + Tags: make([]string, len(params)), + UniqueKey: make([][]byte, len(params)), + UniqueStates: make([]pgtypealias.Bits, len(params)), } now := time.Now() @@ -254,6 +265,8 @@ func (e *Executor) JobInsertFastMany(ctx context.Context, params []*riverdriver. insertJobsParams.ScheduledAt[i] = scheduledAt insertJobsParams.State[i] = string(params.State) insertJobsParams.Tags[i] = strings.Join(tags, ",") + insertJobsParams.UniqueKey[i] = sliceutil.DefaultIfEmpty(params.UniqueKey, nil) + insertJobsParams.UniqueStates[i] = pgtypealias.Bits{Bits: pgtype.Bits{Bytes: []byte{params.UniqueStates}, Len: 8, Valid: params.UniqueStates != 0}} } items, err := dbsqlc.New().JobInsertFastMany(ctx, e.dbtx, insertJobsParams) @@ -261,20 +274,28 @@ func (e *Executor) JobInsertFastMany(ctx context.Context, params []*riverdriver. return nil, interpretError(err) } - return mapSliceError(items, jobRowFromInternal) + return mapSliceError(items, func(row *dbsqlc.JobInsertFastManyRow) (*riverdriver.JobInsertFastResult, error) { + job, err := jobRowFromInternal(&row.RiverJob) + if err != nil { + return nil, err + } + return &riverdriver.JobInsertFastResult{Job: job, UniqueSkippedAsDuplicate: row.UniqueSkippedAsDuplicate}, nil + }) } func (e *Executor) JobInsertFastManyNoReturning(ctx context.Context, params []*riverdriver.JobInsertFastParams) (int, error) { insertJobsParams := &dbsqlc.JobInsertFastManyNoReturningParams{ - Args: make([]string, len(params)), - Kind: make([]string, len(params)), - MaxAttempts: make([]int16, len(params)), - Metadata: make([]string, len(params)), - Priority: make([]int16, len(params)), - Queue: make([]string, len(params)), - ScheduledAt: make([]time.Time, len(params)), - State: make([]dbsqlc.RiverJobState, len(params)), - Tags: make([]string, len(params)), + Args: make([]string, len(params)), + Kind: make([]string, len(params)), + MaxAttempts: make([]int16, len(params)), + Metadata: make([]string, len(params)), + Priority: make([]int16, len(params)), + Queue: make([]string, len(params)), + ScheduledAt: make([]time.Time, len(params)), + State: make([]dbsqlc.RiverJobState, len(params)), + Tags: make([]string, len(params)), + UniqueKey: make([][]byte, len(params)), + UniqueStates: make([]pgtypealias.Bits, len(params)), } now := time.Now() @@ -302,6 +323,8 @@ func (e *Executor) JobInsertFastManyNoReturning(ctx context.Context, params []*r insertJobsParams.ScheduledAt[i] = scheduledAt insertJobsParams.State[i] = dbsqlc.RiverJobState(params.State) insertJobsParams.Tags[i] = strings.Join(tags, ",") + insertJobsParams.UniqueKey[i] = params.UniqueKey + insertJobsParams.UniqueStates[i] = pgtypealias.Bits{Bits: pgtype.Bits{Bytes: []byte{params.UniqueStates}, Len: 8, Valid: params.UniqueStates != 0}} } numInserted, err := dbsqlc.New().JobInsertFastManyNoReturning(ctx, e.dbtx, insertJobsParams) @@ -314,21 +337,22 @@ func (e *Executor) JobInsertFastManyNoReturning(ctx context.Context, params []*r func (e *Executor) JobInsertFull(ctx context.Context, params *riverdriver.JobInsertFullParams) (*rivertype.JobRow, error) { job, err := dbsqlc.New().JobInsertFull(ctx, e.dbtx, &dbsqlc.JobInsertFullParams{ - Attempt: int16(min(params.Attempt, math.MaxInt16)), //nolint:gosec - AttemptedAt: params.AttemptedAt, - Args: string(params.EncodedArgs), - CreatedAt: params.CreatedAt, - Errors: sliceutil.Map(params.Errors, func(e []byte) string { return string(e) }), - FinalizedAt: params.FinalizedAt, - Kind: params.Kind, - MaxAttempts: int16(min(params.MaxAttempts, math.MaxInt16)), //nolint:gosec - Metadata: valutil.ValOrDefault(string(params.Metadata), "{}"), - Priority: int16(min(params.Priority, math.MaxInt16)), //nolint:gosec - Queue: params.Queue, - ScheduledAt: params.ScheduledAt, - State: dbsqlc.RiverJobState(params.State), - Tags: params.Tags, - UniqueKey: params.UniqueKey, + Attempt: int16(min(params.Attempt, math.MaxInt16)), //nolint:gosec + AttemptedAt: params.AttemptedAt, + Args: string(params.EncodedArgs), + CreatedAt: params.CreatedAt, + Errors: sliceutil.Map(params.Errors, func(e []byte) string { return string(e) }), + FinalizedAt: params.FinalizedAt, + Kind: params.Kind, + MaxAttempts: int16(min(params.MaxAttempts, math.MaxInt16)), //nolint:gosec + Metadata: valutil.ValOrDefault(string(params.Metadata), "{}"), + Priority: int16(min(params.Priority, math.MaxInt16)), //nolint:gosec + Queue: params.Queue, + ScheduledAt: params.ScheduledAt, + State: dbsqlc.RiverJobState(params.State), + Tags: params.Tags, + UniqueKey: params.UniqueKey, + UniqueStates: pgtypealias.Bits{Bits: pgtype.Bits{Bytes: []byte{params.UniqueStates}, Len: 8, Valid: params.UniqueStates != 0}}, }) if err != nil { return nil, interpretError(err) @@ -336,35 +360,6 @@ func (e *Executor) JobInsertFull(ctx context.Context, params *riverdriver.JobIns return jobRowFromInternal(job) } -func (e *Executor) JobInsertUnique(ctx context.Context, params *riverdriver.JobInsertUniqueParams) (*riverdriver.JobInsertUniqueResult, error) { - insertRes, err := dbsqlc.New().JobInsertUnique(ctx, e.dbtx, &dbsqlc.JobInsertUniqueParams{ - Args: string(params.EncodedArgs), - CreatedAt: params.CreatedAt, - Kind: params.Kind, - MaxAttempts: int16(min(params.MaxAttempts, math.MaxInt16)), //nolint:gosec - Metadata: valutil.ValOrDefault(string(params.Metadata), "{}"), - Priority: int16(min(params.Priority, math.MaxInt16)), //nolint:gosec - Queue: params.Queue, - ScheduledAt: params.ScheduledAt, - State: dbsqlc.RiverJobState(params.State), - Tags: params.Tags, - UniqueKey: params.UniqueKey, - }) - if err != nil { - return nil, interpretError(err) - } - - jobRow, err := jobRowFromInternal(&insertRes.RiverJob) - if err != nil { - return nil, err - } - - return &riverdriver.JobInsertUniqueResult{ - Job: jobRow, - UniqueSkippedAsDuplicate: insertRes.UniqueSkippedAsDuplicate, - }, nil -} - func (e *Executor) JobList(ctx context.Context, query string, namedArgs map[string]any) ([]*rivertype.JobRow, error) { query, err := replaceNamed(query, namedArgs) if err != nil { @@ -397,6 +392,8 @@ func (e *Executor) JobList(ctx context.Context, query string, namedArgs map[stri &i.State, &i.ScheduledAt, pq.Array(&i.Tags), + &i.UniqueKey, + &i.UniqueStates, ); err != nil { return nil, err } @@ -502,7 +499,7 @@ func replaceNamed(query string, namedArgs map[string]any) (string, error) { } func (e *Executor) JobListFields() string { - return "id, args, attempt, attempted_at, attempted_by, created_at, errors, finalized_at, kind, max_attempts, metadata, priority, queue, state, scheduled_at, tags" + return "id, args, attempt, attempted_at, attempted_by, created_at, errors, finalized_at, kind, max_attempts, metadata, priority, queue, state, scheduled_at, tags, unique_key, unique_states" } func (e *Executor) JobRescueMany(ctx context.Context, params *riverdriver.JobRescueManyParams) (*struct{}, error) { @@ -527,15 +524,21 @@ func (e *Executor) JobRetry(ctx context.Context, id int64) (*rivertype.JobRow, e return jobRowFromInternal(job) } -func (e *Executor) JobSchedule(ctx context.Context, params *riverdriver.JobScheduleParams) ([]*rivertype.JobRow, error) { - jobs, err := dbsqlc.New().JobSchedule(ctx, e.dbtx, &dbsqlc.JobScheduleParams{ +func (e *Executor) JobSchedule(ctx context.Context, params *riverdriver.JobScheduleParams) ([]*riverdriver.JobScheduleResult, error) { + scheduleResults, err := dbsqlc.New().JobSchedule(ctx, e.dbtx, &dbsqlc.JobScheduleParams{ Max: int64(params.Max), Now: params.Now, }) if err != nil { return nil, interpretError(err) } - return mapSliceError(jobs, jobRowFromInternal) + return mapSliceError(scheduleResults, func(result *dbsqlc.JobScheduleRow) (*riverdriver.JobScheduleResult, error) { + job, err := jobRowFromInternal(&result.RiverJob) + if err != nil { + return nil, err + } + return &riverdriver.JobScheduleResult{ConflictDiscarded: result.ConflictDiscarded, Job: *job}, nil + }) } func (e *Executor) JobSetCompleteIfRunningMany(ctx context.Context, params *riverdriver.JobSetCompleteIfRunningManyParams) ([]*rivertype.JobRow, error) { @@ -586,8 +589,6 @@ func (e *Executor) JobUpdate(ctx context.Context, params *riverdriver.JobUpdateP FinalizedAt: params.FinalizedAt, StateDoUpdate: params.StateDoUpdate, State: dbsqlc.RiverJobState(params.State), - UniqueKeyDoUpdate: params.UniqueKeyDoUpdate, - UniqueKey: params.UniqueKey, }) if err != nil { return nil, interpretError(err) @@ -952,24 +953,30 @@ func jobRowFromInternal(internal *dbsqlc.RiverJob) (*rivertype.JobRow, error) { finalizedAt = &t } + var uniqueStatesByte byte + if internal.UniqueStates.Valid && len(internal.UniqueStates.Bytes) > 0 { + uniqueStatesByte = internal.UniqueStates.Bytes[0] + } + return &rivertype.JobRow{ - ID: internal.ID, - Attempt: max(int(internal.Attempt), 0), - AttemptedAt: attemptedAt, - AttemptedBy: internal.AttemptedBy, - CreatedAt: internal.CreatedAt.UTC(), - EncodedArgs: []byte(internal.Args), - Errors: errors, - FinalizedAt: finalizedAt, - Kind: internal.Kind, - MaxAttempts: max(int(internal.MaxAttempts), 0), - Metadata: []byte(internal.Metadata), - Priority: max(int(internal.Priority), 0), - Queue: internal.Queue, - ScheduledAt: internal.ScheduledAt.UTC(), - State: rivertype.JobState(internal.State), - Tags: internal.Tags, - UniqueKey: internal.UniqueKey, + ID: internal.ID, + Attempt: max(int(internal.Attempt), 0), + AttemptedAt: attemptedAt, + AttemptedBy: internal.AttemptedBy, + CreatedAt: internal.CreatedAt.UTC(), + EncodedArgs: []byte(internal.Args), + Errors: errors, + FinalizedAt: finalizedAt, + Kind: internal.Kind, + MaxAttempts: max(int(internal.MaxAttempts), 0), + Metadata: []byte(internal.Metadata), + Priority: max(int(internal.Priority), 0), + Queue: internal.Queue, + ScheduledAt: internal.ScheduledAt.UTC(), + State: rivertype.JobState(internal.State), + Tags: internal.Tags, + UniqueKey: internal.UniqueKey, + UniqueStates: dbunique.UniqueBitmaskToStates(uniqueStatesByte), }, nil } diff --git a/riverdriver/riverpgxv5/internal/dbsqlc/copyfrom.go b/riverdriver/riverpgxv5/internal/dbsqlc/copyfrom.go index 026f223e..8cedc3f3 100644 --- a/riverdriver/riverpgxv5/internal/dbsqlc/copyfrom.go +++ b/riverdriver/riverpgxv5/internal/dbsqlc/copyfrom.go @@ -39,6 +39,8 @@ func (r iteratorForJobInsertFastManyCopyFrom) Values() ([]interface{}, error) { r.rows[0].ScheduledAt, r.rows[0].State, r.rows[0].Tags, + r.rows[0].UniqueKey, + r.rows[0].UniqueStates, }, nil } @@ -47,5 +49,5 @@ func (r iteratorForJobInsertFastManyCopyFrom) Err() error { } func (q *Queries) JobInsertFastManyCopyFrom(ctx context.Context, db DBTX, arg []*JobInsertFastManyCopyFromParams) (int64, error) { - return db.CopyFrom(ctx, []string{"river_job"}, []string{"args", "finalized_at", "kind", "max_attempts", "metadata", "priority", "queue", "scheduled_at", "state", "tags"}, &iteratorForJobInsertFastManyCopyFrom{rows: arg}) + return db.CopyFrom(ctx, []string{"river_job"}, []string{"args", "finalized_at", "kind", "max_attempts", "metadata", "priority", "queue", "scheduled_at", "state", "tags", "unique_key", "unique_states"}, &iteratorForJobInsertFastManyCopyFrom{rows: arg}) } diff --git a/riverdriver/riverpgxv5/internal/dbsqlc/models.go b/riverdriver/riverpgxv5/internal/dbsqlc/models.go index 571bb42d..1ae420aa 100644 --- a/riverdriver/riverpgxv5/internal/dbsqlc/models.go +++ b/riverdriver/riverpgxv5/internal/dbsqlc/models.go @@ -8,6 +8,8 @@ import ( "database/sql/driver" "fmt" "time" + + "github.com/jackc/pgx/v5/pgtype" ) type RiverJobState string @@ -78,23 +80,24 @@ type RiverClientQueue struct { } type RiverJob struct { - ID int64 - Args []byte - Attempt int16 - AttemptedAt *time.Time - AttemptedBy []string - CreatedAt time.Time - Errors [][]byte - FinalizedAt *time.Time - Kind string - MaxAttempts int16 - Metadata []byte - Priority int16 - Queue string - State RiverJobState - ScheduledAt time.Time - Tags []string - UniqueKey []byte + ID int64 + Args []byte + Attempt int16 + AttemptedAt *time.Time + AttemptedBy []string + CreatedAt time.Time + Errors [][]byte + FinalizedAt *time.Time + Kind string + MaxAttempts int16 + Metadata []byte + Priority int16 + Queue string + State RiverJobState + ScheduledAt time.Time + Tags []string + UniqueKey []byte + UniqueStates pgtype.Bits } type RiverLeader struct { diff --git a/riverdriver/riverpgxv5/internal/dbsqlc/river_job.sql b/riverdriver/riverpgxv5/internal/dbsqlc/river_job.sql index e62ee160..06c84c25 100644 --- a/riverdriver/riverpgxv5/internal/dbsqlc/river_job.sql +++ b/riverdriver/riverpgxv5/internal/dbsqlc/river_job.sql @@ -27,6 +27,7 @@ CREATE TABLE river_job( scheduled_at timestamptz NOT NULL DEFAULT NOW(), tags varchar(255)[] NOT NULL DEFAULT '{}', unique_key bytea, + unique_states bit(8), CONSTRAINT finalized_or_finalized_at_null CHECK ( (finalized_at IS NULL AND state NOT IN ('cancelled', 'completed', 'discarded')) OR (finalized_at IS NOT NULL AND state IN ('cancelled', 'completed', 'discarded')) @@ -68,8 +69,13 @@ updated_job AS ( -- rescue it, even if it gets stuck in the running state: metadata = jsonb_set(metadata, '{cancel_attempted_at}'::text[], @cancel_attempted_at::jsonb, true), -- Similarly, zero a `unique_key` if the job is transitioning directly - -- to cancelled. Otherwise, it'll be clear the job executor. - unique_key = CASE WHEN state = 'running' THEN unique_key ELSE NULL END + -- to cancelled. Otherwise, it'll be cleared in the job executor. + -- + -- This is transition code to support existing jobs using the old v2 + -- uniqueness design. We specifically avoid clearing this value if the + -- v3 unique_states field is populated, because the v3 design never + -- involves clearing unique_key. + unique_key = CASE WHEN (state = 'running' OR unique_states IS NOT NULL) THEN unique_key ELSE NULL END FROM notification WHERE river_job.id = notification.id RETURNING river_job.* @@ -208,7 +214,9 @@ INSERT INTO river_job( queue, scheduled_at, state, - tags + tags, + unique_key, + unique_states ) VALUES ( @args, coalesce(sqlc.narg('created_at')::timestamptz, now()), @@ -220,8 +228,17 @@ INSERT INTO river_job( @queue, coalesce(sqlc.narg('scheduled_at')::timestamptz, now()), @state, - coalesce(@tags::varchar(255)[], '{}') -) RETURNING *; + coalesce(@tags::varchar(255)[], '{}'), + @unique_key, + @unique_states +) +ON CONFLICT (unique_key) + WHERE unique_key IS NOT NULL + AND unique_states IS NOT NULL + AND river_job_state_in_bitmask(unique_states, state) + -- Something needs to be updated for a row to be returned on a conflict. + DO UPDATE SET kind = EXCLUDED.kind +RETURNING sqlc.embed(river_job), (xmax != 0) AS unique_skipped_as_duplicate; -- name: JobInsertFastMany :many INSERT INTO river_job( @@ -233,7 +250,9 @@ INSERT INTO river_job( queue, scheduled_at, state, - tags + tags, + unique_key, + unique_states ) SELECT unnest(@args::jsonb[]), unnest(@kind::text[]), @@ -248,8 +267,18 @@ INSERT INTO river_job( -- Unnest on a multi-dimensional array will fully flatten the array, so we -- encode the tag list as a comma-separated string and split it in the -- query. - string_to_array(unnest(@tags::text[]), ',') -RETURNING *; + string_to_array(unnest(@tags::text[]), ','), + + unnest(@unique_key::bytea[]), + unnest(@unique_states::bit(8)[]) + +ON CONFLICT (unique_key) + WHERE unique_key IS NOT NULL + AND unique_states IS NOT NULL + AND river_job_state_in_bitmask(unique_states, state) + -- Something needs to be updated for a row to be returned on a conflict. + DO UPDATE SET kind = EXCLUDED.kind +RETURNING sqlc.embed(river_job), (xmax != 0) AS unique_skipped_as_duplicate; -- name: JobInsertFastManyNoReturning :execrows INSERT INTO river_job( @@ -261,7 +290,9 @@ INSERT INTO river_job( queue, scheduled_at, state, - tags + tags, + unique_key, + unique_states ) SELECT unnest(@args::jsonb[]), unnest(@kind::text[]), @@ -276,7 +307,16 @@ INSERT INTO river_job( -- so instead we pack each set of tags into a string, send them through, -- then unpack them here into an array to put in each row. This isn't -- necessary in the Pgx driver where copyfrom is used instead. - string_to_array(unnest(@tags::text[]), ','); + string_to_array(unnest(@tags::text[]), ','), + + unnest(@unique_key::bytea[]), + unnest(@unique_states::bit(8)[]) + +ON CONFLICT (unique_key) + WHERE unique_key IS NOT NULL + AND unique_states IS NOT NULL + AND river_job_state_in_bitmask(unique_states, state) +DO NOTHING; -- name: JobInsertFull :one INSERT INTO river_job( @@ -294,7 +334,8 @@ INSERT INTO river_job( scheduled_at, state, tags, - unique_key + unique_key, + unique_states ) VALUES ( @args::jsonb, coalesce(@attempt::smallint, 0), @@ -310,41 +351,10 @@ INSERT INTO river_job( coalesce(sqlc.narg('scheduled_at')::timestamptz, now()), @state, coalesce(@tags::varchar(255)[], '{}'), - @unique_key + @unique_key, + @unique_states ) RETURNING *; --- name: JobInsertUnique :one -INSERT INTO river_job( - args, - created_at, - finalized_at, - kind, - max_attempts, - metadata, - priority, - queue, - scheduled_at, - state, - tags, - unique_key -) VALUES ( - @args, - coalesce(sqlc.narg('created_at')::timestamptz, now()), - @finalized_at, - @kind, - @max_attempts, - coalesce(@metadata::jsonb, '{}'), - @priority, - @queue, - coalesce(sqlc.narg('scheduled_at')::timestamptz, now()), - @state, - coalesce(@tags::varchar(255)[], '{}'), - @unique_key -) -ON CONFLICT (kind, unique_key) WHERE unique_key IS NOT NULL - -- Something needs to be updated for a row to be returned on a conflict. - DO UPDATE SET kind = EXCLUDED.kind -RETURNING sqlc.embed(river_job), (xmax != 0) AS unique_skipped_as_duplicate; -- Run by the rescuer to queue for retry or discard depending on job state. -- name: JobRescueMany :exec @@ -396,13 +406,13 @@ FROM updated_job; -- name: JobSchedule :many WITH jobs_to_schedule AS ( - SELECT id + SELECT id, unique_key, unique_states FROM river_job WHERE state IN ('retryable', 'scheduled') AND queue IS NOT NULL AND priority >= 0 - AND river_job.scheduled_at <= @now::timestamptz + AND scheduled_at <= @now::timestamptz ORDER BY priority, scheduled_at, @@ -410,16 +420,38 @@ WITH jobs_to_schedule AS ( LIMIT @max::bigint FOR UPDATE ), -river_job_scheduled AS ( +conflicting_jobs AS ( + SELECT DISTINCT unique_key + FROM river_job + WHERE unique_key IN ( + SELECT unique_key + FROM jobs_to_schedule + WHERE unique_key IS NOT NULL + AND unique_states IS NOT NULL + ) + AND unique_states IS NOT NULL + AND river_job_state_in_bitmask(unique_states, state) +), +updated_jobs AS ( UPDATE river_job - SET state = 'available' - FROM jobs_to_schedule - WHERE river_job.id = jobs_to_schedule.id - RETURNING river_job.id + SET + state = CASE WHEN cj.unique_key IS NULL THEN 'available'::river_job_state + ELSE 'discarded'::river_job_state END, + finalized_at = CASE WHEN cj.unique_key IS NULL THEN finalized_at + ELSE @now::timestamptz END, + -- Purely for debugging to understand when this code path was used: + metadata = CASE WHEN cj.unique_key IS NULL THEN metadata + ELSE metadata || '{"unique_key_conflict": "scheduler_discarded"}'::jsonb END + FROM jobs_to_schedule jts + LEFT JOIN conflicting_jobs cj ON jts.unique_key = cj.unique_key + WHERE river_job.id = jts.id + RETURNING river_job.id, state = 'discarded'::river_job_state AS conflict_discarded ) -SELECT * +SELECT + sqlc.embed(river_job), + updated_jobs.conflict_discarded FROM river_job -WHERE id IN (SELECT id FROM river_job_scheduled); +JOIN updated_jobs ON river_job.id = updated_jobs.id; -- name: JobSetCompleteIfRunningMany :many WITH job_to_finalized_at AS ( @@ -473,7 +505,10 @@ updated_job AS ( ELSE max_attempts END, scheduled_at = CASE WHEN NOT should_cancel AND @scheduled_at_do_update::boolean THEN sqlc.narg('scheduled_at')::timestamptz ELSE scheduled_at END, - unique_key = CASE WHEN (@state IN ('cancelled', 'discarded') OR should_cancel) THEN NULL + -- This is transitional code for the v2 uniqueness design. We specifically + -- avoid clearing this value if the v3 unique_states field is populated, + -- because the v3 design never involves clearing unique_key. + unique_key = CASE WHEN (unique_states IS NULL AND (@state IN ('cancelled', 'discarded') OR should_cancel)) THEN NULL ELSE unique_key END FROM job_to_update WHERE river_job.id = job_to_update.id @@ -498,6 +533,8 @@ SET errors = CASE WHEN @errors_do_update::boolean THEN @errors::jsonb[] ELSE errors END, finalized_at = CASE WHEN @finalized_at_do_update::boolean THEN @finalized_at ELSE finalized_at END, state = CASE WHEN @state_do_update::boolean THEN @state ELSE state END, + -- Transitional code to support tests for v2 uniqueness design. This field + -- is never modified in the v3 design. unique_key = CASE WHEN @unique_key_do_update::boolean THEN @unique_key ELSE unique_key END WHERE id = @id RETURNING *; diff --git a/riverdriver/riverpgxv5/internal/dbsqlc/river_job.sql.go b/riverdriver/riverpgxv5/internal/dbsqlc/river_job.sql.go index a2713bef..399300c7 100644 --- a/riverdriver/riverpgxv5/internal/dbsqlc/river_job.sql.go +++ b/riverdriver/riverpgxv5/internal/dbsqlc/river_job.sql.go @@ -8,6 +8,8 @@ package dbsqlc import ( "context" "time" + + "github.com/jackc/pgx/v5/pgtype" ) const jobCancel = `-- name: JobCancel :one @@ -42,18 +44,23 @@ updated_job AS ( -- rescue it, even if it gets stuck in the running state: metadata = jsonb_set(metadata, '{cancel_attempted_at}'::text[], $3::jsonb, true), -- Similarly, zero a ` + "`" + `unique_key` + "`" + ` if the job is transitioning directly - -- to cancelled. Otherwise, it'll be clear the job executor. - unique_key = CASE WHEN state = 'running' THEN unique_key ELSE NULL END + -- to cancelled. Otherwise, it'll be cleared in the job executor. + -- + -- This is transition code to support existing jobs using the old v2 + -- uniqueness design. We specifically avoid clearing this value if the + -- v3 unique_states field is populated, because the v3 design never + -- involves clearing unique_key. + unique_key = CASE WHEN (state = 'running' OR unique_states IS NOT NULL) THEN unique_key ELSE NULL END FROM notification WHERE river_job.id = notification.id - RETURNING river_job.id, river_job.args, river_job.attempt, river_job.attempted_at, river_job.attempted_by, river_job.created_at, river_job.errors, river_job.finalized_at, river_job.kind, river_job.max_attempts, river_job.metadata, river_job.priority, river_job.queue, river_job.state, river_job.scheduled_at, river_job.tags, river_job.unique_key + RETURNING river_job.id, river_job.args, river_job.attempt, river_job.attempted_at, river_job.attempted_by, river_job.created_at, river_job.errors, river_job.finalized_at, river_job.kind, river_job.max_attempts, river_job.metadata, river_job.priority, river_job.queue, river_job.state, river_job.scheduled_at, river_job.tags, river_job.unique_key, river_job.unique_states ) -SELECT id, args, attempt, attempted_at, attempted_by, created_at, errors, finalized_at, kind, max_attempts, metadata, priority, queue, state, scheduled_at, tags, unique_key +SELECT id, args, attempt, attempted_at, attempted_by, created_at, errors, finalized_at, kind, max_attempts, metadata, priority, queue, state, scheduled_at, tags, unique_key, unique_states FROM river_job WHERE id = $1::bigint AND id NOT IN (SELECT id FROM updated_job) UNION -SELECT id, args, attempt, attempted_at, attempted_by, created_at, errors, finalized_at, kind, max_attempts, metadata, priority, queue, state, scheduled_at, tags, unique_key +SELECT id, args, attempt, attempted_at, attempted_by, created_at, errors, finalized_at, kind, max_attempts, metadata, priority, queue, state, scheduled_at, tags, unique_key, unique_states FROM updated_job ` @@ -84,6 +91,7 @@ func (q *Queries) JobCancel(ctx context.Context, db DBTX, arg *JobCancelParams) &i.ScheduledAt, &i.Tags, &i.UniqueKey, + &i.UniqueStates, ) return &i, err } @@ -115,14 +123,14 @@ deleted_job AS ( WHERE river_job.id = job_to_delete.id -- Do not touch running jobs: AND river_job.state != 'running' - RETURNING river_job.id, river_job.args, river_job.attempt, river_job.attempted_at, river_job.attempted_by, river_job.created_at, river_job.errors, river_job.finalized_at, river_job.kind, river_job.max_attempts, river_job.metadata, river_job.priority, river_job.queue, river_job.state, river_job.scheduled_at, river_job.tags, river_job.unique_key + RETURNING river_job.id, river_job.args, river_job.attempt, river_job.attempted_at, river_job.attempted_by, river_job.created_at, river_job.errors, river_job.finalized_at, river_job.kind, river_job.max_attempts, river_job.metadata, river_job.priority, river_job.queue, river_job.state, river_job.scheduled_at, river_job.tags, river_job.unique_key, river_job.unique_states ) -SELECT id, args, attempt, attempted_at, attempted_by, created_at, errors, finalized_at, kind, max_attempts, metadata, priority, queue, state, scheduled_at, tags, unique_key +SELECT id, args, attempt, attempted_at, attempted_by, created_at, errors, finalized_at, kind, max_attempts, metadata, priority, queue, state, scheduled_at, tags, unique_key, unique_states FROM river_job WHERE id = $1::bigint AND id NOT IN (SELECT id FROM deleted_job) UNION -SELECT id, args, attempt, attempted_at, attempted_by, created_at, errors, finalized_at, kind, max_attempts, metadata, priority, queue, state, scheduled_at, tags, unique_key +SELECT id, args, attempt, attempted_at, attempted_by, created_at, errors, finalized_at, kind, max_attempts, metadata, priority, queue, state, scheduled_at, tags, unique_key, unique_states FROM deleted_job ` @@ -147,6 +155,7 @@ func (q *Queries) JobDelete(ctx context.Context, db DBTX, id int64) (*RiverJob, &i.ScheduledAt, &i.Tags, &i.UniqueKey, + &i.UniqueStates, ) return &i, err } @@ -164,7 +173,7 @@ WITH deleted_jobs AS ( ORDER BY id LIMIT $4::bigint ) - RETURNING id, args, attempt, attempted_at, attempted_by, created_at, errors, finalized_at, kind, max_attempts, metadata, priority, queue, state, scheduled_at, tags, unique_key + RETURNING id, args, attempt, attempted_at, attempted_by, created_at, errors, finalized_at, kind, max_attempts, metadata, priority, queue, state, scheduled_at, tags, unique_key, unique_states ) SELECT count(*) FROM deleted_jobs @@ -192,7 +201,7 @@ func (q *Queries) JobDeleteBefore(ctx context.Context, db DBTX, arg *JobDeleteBe const jobGetAvailable = `-- name: JobGetAvailable :many WITH locked_jobs AS ( SELECT - id, args, attempt, attempted_at, attempted_by, created_at, errors, finalized_at, kind, max_attempts, metadata, priority, queue, state, scheduled_at, tags, unique_key + id, args, attempt, attempted_at, attempted_by, created_at, errors, finalized_at, kind, max_attempts, metadata, priority, queue, state, scheduled_at, tags, unique_key, unique_states FROM river_job WHERE @@ -219,7 +228,7 @@ FROM WHERE river_job.id = locked_jobs.id RETURNING - river_job.id, river_job.args, river_job.attempt, river_job.attempted_at, river_job.attempted_by, river_job.created_at, river_job.errors, river_job.finalized_at, river_job.kind, river_job.max_attempts, river_job.metadata, river_job.priority, river_job.queue, river_job.state, river_job.scheduled_at, river_job.tags, river_job.unique_key + river_job.id, river_job.args, river_job.attempt, river_job.attempted_at, river_job.attempted_by, river_job.created_at, river_job.errors, river_job.finalized_at, river_job.kind, river_job.max_attempts, river_job.metadata, river_job.priority, river_job.queue, river_job.state, river_job.scheduled_at, river_job.tags, river_job.unique_key, river_job.unique_states ` type JobGetAvailableParams struct { @@ -255,6 +264,7 @@ func (q *Queries) JobGetAvailable(ctx context.Context, db DBTX, arg *JobGetAvail &i.ScheduledAt, &i.Tags, &i.UniqueKey, + &i.UniqueStates, ); err != nil { return nil, err } @@ -267,7 +277,7 @@ func (q *Queries) JobGetAvailable(ctx context.Context, db DBTX, arg *JobGetAvail } const jobGetByID = `-- name: JobGetByID :one -SELECT id, args, attempt, attempted_at, attempted_by, created_at, errors, finalized_at, kind, max_attempts, metadata, priority, queue, state, scheduled_at, tags, unique_key +SELECT id, args, attempt, attempted_at, attempted_by, created_at, errors, finalized_at, kind, max_attempts, metadata, priority, queue, state, scheduled_at, tags, unique_key, unique_states FROM river_job WHERE id = $1 LIMIT 1 @@ -294,12 +304,13 @@ func (q *Queries) JobGetByID(ctx context.Context, db DBTX, id int64) (*RiverJob, &i.ScheduledAt, &i.Tags, &i.UniqueKey, + &i.UniqueStates, ) return &i, err } const jobGetByIDMany = `-- name: JobGetByIDMany :many -SELECT id, args, attempt, attempted_at, attempted_by, created_at, errors, finalized_at, kind, max_attempts, metadata, priority, queue, state, scheduled_at, tags, unique_key +SELECT id, args, attempt, attempted_at, attempted_by, created_at, errors, finalized_at, kind, max_attempts, metadata, priority, queue, state, scheduled_at, tags, unique_key, unique_states FROM river_job WHERE id = any($1::bigint[]) ORDER BY id @@ -332,6 +343,7 @@ func (q *Queries) JobGetByIDMany(ctx context.Context, db DBTX, id []int64) ([]*R &i.ScheduledAt, &i.Tags, &i.UniqueKey, + &i.UniqueStates, ); err != nil { return nil, err } @@ -344,7 +356,7 @@ func (q *Queries) JobGetByIDMany(ctx context.Context, db DBTX, id []int64) ([]*R } const jobGetByKindAndUniqueProperties = `-- name: JobGetByKindAndUniqueProperties :one -SELECT id, args, attempt, attempted_at, attempted_by, created_at, errors, finalized_at, kind, max_attempts, metadata, priority, queue, state, scheduled_at, tags, unique_key +SELECT id, args, attempt, attempted_at, attempted_by, created_at, errors, finalized_at, kind, max_attempts, metadata, priority, queue, state, scheduled_at, tags, unique_key, unique_states FROM river_job WHERE kind = $1 AND CASE WHEN $2::boolean THEN args = $3 ELSE true END @@ -398,12 +410,13 @@ func (q *Queries) JobGetByKindAndUniqueProperties(ctx context.Context, db DBTX, &i.ScheduledAt, &i.Tags, &i.UniqueKey, + &i.UniqueStates, ) return &i, err } const jobGetByKindMany = `-- name: JobGetByKindMany :many -SELECT id, args, attempt, attempted_at, attempted_by, created_at, errors, finalized_at, kind, max_attempts, metadata, priority, queue, state, scheduled_at, tags, unique_key +SELECT id, args, attempt, attempted_at, attempted_by, created_at, errors, finalized_at, kind, max_attempts, metadata, priority, queue, state, scheduled_at, tags, unique_key, unique_states FROM river_job WHERE kind = any($1::text[]) ORDER BY id @@ -436,6 +449,7 @@ func (q *Queries) JobGetByKindMany(ctx context.Context, db DBTX, kind []string) &i.ScheduledAt, &i.Tags, &i.UniqueKey, + &i.UniqueStates, ); err != nil { return nil, err } @@ -448,7 +462,7 @@ func (q *Queries) JobGetByKindMany(ctx context.Context, db DBTX, kind []string) } const jobGetStuck = `-- name: JobGetStuck :many -SELECT id, args, attempt, attempted_at, attempted_by, created_at, errors, finalized_at, kind, max_attempts, metadata, priority, queue, state, scheduled_at, tags, unique_key +SELECT id, args, attempt, attempted_at, attempted_by, created_at, errors, finalized_at, kind, max_attempts, metadata, priority, queue, state, scheduled_at, tags, unique_key, unique_states FROM river_job WHERE state = 'running' AND attempted_at < $1::timestamptz @@ -488,6 +502,7 @@ func (q *Queries) JobGetStuck(ctx context.Context, db DBTX, arg *JobGetStuckPara &i.ScheduledAt, &i.Tags, &i.UniqueKey, + &i.UniqueStates, ); err != nil { return nil, err } @@ -511,7 +526,9 @@ INSERT INTO river_job( queue, scheduled_at, state, - tags + tags, + unique_key, + unique_states ) VALUES ( $1, coalesce($2::timestamptz, now()), @@ -523,25 +540,41 @@ INSERT INTO river_job( $8, coalesce($9::timestamptz, now()), $10, - coalesce($11::varchar(255)[], '{}') -) RETURNING id, args, attempt, attempted_at, attempted_by, created_at, errors, finalized_at, kind, max_attempts, metadata, priority, queue, state, scheduled_at, tags, unique_key + coalesce($11::varchar(255)[], '{}'), + $12, + $13 +) +ON CONFLICT (unique_key) + WHERE unique_key IS NOT NULL + AND unique_states IS NOT NULL + AND river_job_state_in_bitmask(unique_states, state) + -- Something needs to be updated for a row to be returned on a conflict. + DO UPDATE SET kind = EXCLUDED.kind +RETURNING river_job.id, river_job.args, river_job.attempt, river_job.attempted_at, river_job.attempted_by, river_job.created_at, river_job.errors, river_job.finalized_at, river_job.kind, river_job.max_attempts, river_job.metadata, river_job.priority, river_job.queue, river_job.state, river_job.scheduled_at, river_job.tags, river_job.unique_key, river_job.unique_states, (xmax != 0) AS unique_skipped_as_duplicate ` type JobInsertFastParams struct { - Args []byte - CreatedAt *time.Time - FinalizedAt *time.Time - Kind string - MaxAttempts int16 - Metadata []byte - Priority int16 - Queue string - ScheduledAt *time.Time - State RiverJobState - Tags []string + Args []byte + CreatedAt *time.Time + FinalizedAt *time.Time + Kind string + MaxAttempts int16 + Metadata []byte + Priority int16 + Queue string + ScheduledAt *time.Time + State RiverJobState + Tags []string + UniqueKey []byte + UniqueStates pgtype.Bits +} + +type JobInsertFastRow struct { + RiverJob RiverJob + UniqueSkippedAsDuplicate bool } -func (q *Queries) JobInsertFast(ctx context.Context, db DBTX, arg *JobInsertFastParams) (*RiverJob, error) { +func (q *Queries) JobInsertFast(ctx context.Context, db DBTX, arg *JobInsertFastParams) (*JobInsertFastRow, error) { row := db.QueryRow(ctx, jobInsertFast, arg.Args, arg.CreatedAt, @@ -554,26 +587,30 @@ func (q *Queries) JobInsertFast(ctx context.Context, db DBTX, arg *JobInsertFast arg.ScheduledAt, arg.State, arg.Tags, + arg.UniqueKey, + arg.UniqueStates, ) - var i RiverJob + var i JobInsertFastRow err := row.Scan( - &i.ID, - &i.Args, - &i.Attempt, - &i.AttemptedAt, - &i.AttemptedBy, - &i.CreatedAt, - &i.Errors, - &i.FinalizedAt, - &i.Kind, - &i.MaxAttempts, - &i.Metadata, - &i.Priority, - &i.Queue, - &i.State, - &i.ScheduledAt, - &i.Tags, - &i.UniqueKey, + &i.RiverJob.ID, + &i.RiverJob.Args, + &i.RiverJob.Attempt, + &i.RiverJob.AttemptedAt, + &i.RiverJob.AttemptedBy, + &i.RiverJob.CreatedAt, + &i.RiverJob.Errors, + &i.RiverJob.FinalizedAt, + &i.RiverJob.Kind, + &i.RiverJob.MaxAttempts, + &i.RiverJob.Metadata, + &i.RiverJob.Priority, + &i.RiverJob.Queue, + &i.RiverJob.State, + &i.RiverJob.ScheduledAt, + &i.RiverJob.Tags, + &i.RiverJob.UniqueKey, + &i.RiverJob.UniqueStates, + &i.UniqueSkippedAsDuplicate, ) return &i, err } @@ -588,7 +625,9 @@ INSERT INTO river_job( queue, scheduled_at, state, - tags + tags, + unique_key, + unique_states ) SELECT unnest($1::jsonb[]), unnest($2::text[]), @@ -603,23 +642,40 @@ INSERT INTO river_job( -- Unnest on a multi-dimensional array will fully flatten the array, so we -- encode the tag list as a comma-separated string and split it in the -- query. - string_to_array(unnest($9::text[]), ',') -RETURNING id, args, attempt, attempted_at, attempted_by, created_at, errors, finalized_at, kind, max_attempts, metadata, priority, queue, state, scheduled_at, tags, unique_key + string_to_array(unnest($9::text[]), ','), + + unnest($10::bytea[]), + unnest($11::bit(8)[]) + +ON CONFLICT (unique_key) + WHERE unique_key IS NOT NULL + AND unique_states IS NOT NULL + AND river_job_state_in_bitmask(unique_states, state) + -- Something needs to be updated for a row to be returned on a conflict. + DO UPDATE SET kind = EXCLUDED.kind +RETURNING river_job.id, river_job.args, river_job.attempt, river_job.attempted_at, river_job.attempted_by, river_job.created_at, river_job.errors, river_job.finalized_at, river_job.kind, river_job.max_attempts, river_job.metadata, river_job.priority, river_job.queue, river_job.state, river_job.scheduled_at, river_job.tags, river_job.unique_key, river_job.unique_states, (xmax != 0) AS unique_skipped_as_duplicate ` type JobInsertFastManyParams struct { - Args [][]byte - Kind []string - MaxAttempts []int16 - Metadata [][]byte - Priority []int16 - Queue []string - ScheduledAt []time.Time - State []string - Tags []string + Args [][]byte + Kind []string + MaxAttempts []int16 + Metadata [][]byte + Priority []int16 + Queue []string + ScheduledAt []time.Time + State []string + Tags []string + UniqueKey [][]byte + UniqueStates []pgtype.Bits } -func (q *Queries) JobInsertFastMany(ctx context.Context, db DBTX, arg *JobInsertFastManyParams) ([]*RiverJob, error) { +type JobInsertFastManyRow struct { + RiverJob RiverJob + UniqueSkippedAsDuplicate bool +} + +func (q *Queries) JobInsertFastMany(ctx context.Context, db DBTX, arg *JobInsertFastManyParams) ([]*JobInsertFastManyRow, error) { rows, err := db.Query(ctx, jobInsertFastMany, arg.Args, arg.Kind, @@ -630,32 +686,36 @@ func (q *Queries) JobInsertFastMany(ctx context.Context, db DBTX, arg *JobInsert arg.ScheduledAt, arg.State, arg.Tags, + arg.UniqueKey, + arg.UniqueStates, ) if err != nil { return nil, err } defer rows.Close() - var items []*RiverJob + var items []*JobInsertFastManyRow for rows.Next() { - var i RiverJob + var i JobInsertFastManyRow if err := rows.Scan( - &i.ID, - &i.Args, - &i.Attempt, - &i.AttemptedAt, - &i.AttemptedBy, - &i.CreatedAt, - &i.Errors, - &i.FinalizedAt, - &i.Kind, - &i.MaxAttempts, - &i.Metadata, - &i.Priority, - &i.Queue, - &i.State, - &i.ScheduledAt, - &i.Tags, - &i.UniqueKey, + &i.RiverJob.ID, + &i.RiverJob.Args, + &i.RiverJob.Attempt, + &i.RiverJob.AttemptedAt, + &i.RiverJob.AttemptedBy, + &i.RiverJob.CreatedAt, + &i.RiverJob.Errors, + &i.RiverJob.FinalizedAt, + &i.RiverJob.Kind, + &i.RiverJob.MaxAttempts, + &i.RiverJob.Metadata, + &i.RiverJob.Priority, + &i.RiverJob.Queue, + &i.RiverJob.State, + &i.RiverJob.ScheduledAt, + &i.RiverJob.Tags, + &i.RiverJob.UniqueKey, + &i.RiverJob.UniqueStates, + &i.UniqueSkippedAsDuplicate, ); err != nil { return nil, err } @@ -677,7 +737,9 @@ INSERT INTO river_job( queue, scheduled_at, state, - tags + tags, + unique_key, + unique_states ) SELECT unnest($1::jsonb[]), unnest($2::text[]), @@ -692,19 +754,30 @@ INSERT INTO river_job( -- so instead we pack each set of tags into a string, send them through, -- then unpack them here into an array to put in each row. This isn't -- necessary in the Pgx driver where copyfrom is used instead. - string_to_array(unnest($9::text[]), ',') + string_to_array(unnest($9::text[]), ','), + + unnest($10::bytea[]), + unnest($11::bit(8)[]) + +ON CONFLICT (unique_key) + WHERE unique_key IS NOT NULL + AND unique_states IS NOT NULL + AND river_job_state_in_bitmask(unique_states, state) +DO NOTHING ` type JobInsertFastManyNoReturningParams struct { - Args [][]byte - Kind []string - MaxAttempts []int16 - Metadata [][]byte - Priority []int16 - Queue []string - ScheduledAt []time.Time - State []RiverJobState - Tags []string + Args [][]byte + Kind []string + MaxAttempts []int16 + Metadata [][]byte + Priority []int16 + Queue []string + ScheduledAt []time.Time + State []RiverJobState + Tags []string + UniqueKey [][]byte + UniqueStates []pgtype.Bits } func (q *Queries) JobInsertFastManyNoReturning(ctx context.Context, db DBTX, arg *JobInsertFastManyNoReturningParams) (int64, error) { @@ -718,6 +791,8 @@ func (q *Queries) JobInsertFastManyNoReturning(ctx context.Context, db DBTX, arg arg.ScheduledAt, arg.State, arg.Tags, + arg.UniqueKey, + arg.UniqueStates, ) if err != nil { return 0, err @@ -741,7 +816,8 @@ INSERT INTO river_job( scheduled_at, state, tags, - unique_key + unique_key, + unique_states ) VALUES ( $1::jsonb, coalesce($2::smallint, 0), @@ -757,26 +833,28 @@ INSERT INTO river_job( coalesce($12::timestamptz, now()), $13, coalesce($14::varchar(255)[], '{}'), - $15 -) RETURNING id, args, attempt, attempted_at, attempted_by, created_at, errors, finalized_at, kind, max_attempts, metadata, priority, queue, state, scheduled_at, tags, unique_key + $15, + $16 +) RETURNING id, args, attempt, attempted_at, attempted_by, created_at, errors, finalized_at, kind, max_attempts, metadata, priority, queue, state, scheduled_at, tags, unique_key, unique_states ` type JobInsertFullParams struct { - Args []byte - Attempt int16 - AttemptedAt *time.Time - CreatedAt *time.Time - Errors [][]byte - FinalizedAt *time.Time - Kind string - MaxAttempts int16 - Metadata []byte - Priority int16 - Queue string - ScheduledAt *time.Time - State RiverJobState - Tags []string - UniqueKey []byte + Args []byte + Attempt int16 + AttemptedAt *time.Time + CreatedAt *time.Time + Errors [][]byte + FinalizedAt *time.Time + Kind string + MaxAttempts int16 + Metadata []byte + Priority int16 + Queue string + ScheduledAt *time.Time + State RiverJobState + Tags []string + UniqueKey []byte + UniqueStates pgtype.Bits } func (q *Queries) JobInsertFull(ctx context.Context, db DBTX, arg *JobInsertFullParams) (*RiverJob, error) { @@ -796,6 +874,7 @@ func (q *Queries) JobInsertFull(ctx context.Context, db DBTX, arg *JobInsertFull arg.State, arg.Tags, arg.UniqueKey, + arg.UniqueStates, ) var i RiverJob err := row.Scan( @@ -816,99 +895,7 @@ func (q *Queries) JobInsertFull(ctx context.Context, db DBTX, arg *JobInsertFull &i.ScheduledAt, &i.Tags, &i.UniqueKey, - ) - return &i, err -} - -const jobInsertUnique = `-- name: JobInsertUnique :one -INSERT INTO river_job( - args, - created_at, - finalized_at, - kind, - max_attempts, - metadata, - priority, - queue, - scheduled_at, - state, - tags, - unique_key -) VALUES ( - $1, - coalesce($2::timestamptz, now()), - $3, - $4, - $5, - coalesce($6::jsonb, '{}'), - $7, - $8, - coalesce($9::timestamptz, now()), - $10, - coalesce($11::varchar(255)[], '{}'), - $12 -) -ON CONFLICT (kind, unique_key) WHERE unique_key IS NOT NULL - -- Something needs to be updated for a row to be returned on a conflict. - DO UPDATE SET kind = EXCLUDED.kind -RETURNING river_job.id, river_job.args, river_job.attempt, river_job.attempted_at, river_job.attempted_by, river_job.created_at, river_job.errors, river_job.finalized_at, river_job.kind, river_job.max_attempts, river_job.metadata, river_job.priority, river_job.queue, river_job.state, river_job.scheduled_at, river_job.tags, river_job.unique_key, (xmax != 0) AS unique_skipped_as_duplicate -` - -type JobInsertUniqueParams struct { - Args []byte - CreatedAt *time.Time - FinalizedAt *time.Time - Kind string - MaxAttempts int16 - Metadata []byte - Priority int16 - Queue string - ScheduledAt *time.Time - State RiverJobState - Tags []string - UniqueKey []byte -} - -type JobInsertUniqueRow struct { - RiverJob RiverJob - UniqueSkippedAsDuplicate bool -} - -func (q *Queries) JobInsertUnique(ctx context.Context, db DBTX, arg *JobInsertUniqueParams) (*JobInsertUniqueRow, error) { - row := db.QueryRow(ctx, jobInsertUnique, - arg.Args, - arg.CreatedAt, - arg.FinalizedAt, - arg.Kind, - arg.MaxAttempts, - arg.Metadata, - arg.Priority, - arg.Queue, - arg.ScheduledAt, - arg.State, - arg.Tags, - arg.UniqueKey, - ) - var i JobInsertUniqueRow - err := row.Scan( - &i.RiverJob.ID, - &i.RiverJob.Args, - &i.RiverJob.Attempt, - &i.RiverJob.AttemptedAt, - &i.RiverJob.AttemptedBy, - &i.RiverJob.CreatedAt, - &i.RiverJob.Errors, - &i.RiverJob.FinalizedAt, - &i.RiverJob.Kind, - &i.RiverJob.MaxAttempts, - &i.RiverJob.Metadata, - &i.RiverJob.Priority, - &i.RiverJob.Queue, - &i.RiverJob.State, - &i.RiverJob.ScheduledAt, - &i.RiverJob.Tags, - &i.RiverJob.UniqueKey, - &i.UniqueSkippedAsDuplicate, + &i.UniqueStates, ) return &i, err } @@ -971,14 +958,14 @@ updated_job AS ( AND river_job.state != 'running' -- If the job is already available with a prior scheduled_at, leave it alone. AND NOT (river_job.state = 'available' AND river_job.scheduled_at < now()) - RETURNING river_job.id, river_job.args, river_job.attempt, river_job.attempted_at, river_job.attempted_by, river_job.created_at, river_job.errors, river_job.finalized_at, river_job.kind, river_job.max_attempts, river_job.metadata, river_job.priority, river_job.queue, river_job.state, river_job.scheduled_at, river_job.tags, river_job.unique_key + RETURNING river_job.id, river_job.args, river_job.attempt, river_job.attempted_at, river_job.attempted_by, river_job.created_at, river_job.errors, river_job.finalized_at, river_job.kind, river_job.max_attempts, river_job.metadata, river_job.priority, river_job.queue, river_job.state, river_job.scheduled_at, river_job.tags, river_job.unique_key, river_job.unique_states ) -SELECT id, args, attempt, attempted_at, attempted_by, created_at, errors, finalized_at, kind, max_attempts, metadata, priority, queue, state, scheduled_at, tags, unique_key +SELECT id, args, attempt, attempted_at, attempted_by, created_at, errors, finalized_at, kind, max_attempts, metadata, priority, queue, state, scheduled_at, tags, unique_key, unique_states FROM river_job WHERE id = $1::bigint AND id NOT IN (SELECT id FROM updated_job) UNION -SELECT id, args, attempt, attempted_at, attempted_by, created_at, errors, finalized_at, kind, max_attempts, metadata, priority, queue, state, scheduled_at, tags, unique_key +SELECT id, args, attempt, attempted_at, attempted_by, created_at, errors, finalized_at, kind, max_attempts, metadata, priority, queue, state, scheduled_at, tags, unique_key, unique_states FROM updated_job ` @@ -1003,19 +990,20 @@ func (q *Queries) JobRetry(ctx context.Context, db DBTX, id int64) (*RiverJob, e &i.ScheduledAt, &i.Tags, &i.UniqueKey, + &i.UniqueStates, ) return &i, err } const jobSchedule = `-- name: JobSchedule :many WITH jobs_to_schedule AS ( - SELECT id + SELECT id, unique_key, unique_states FROM river_job WHERE state IN ('retryable', 'scheduled') AND queue IS NOT NULL AND priority >= 0 - AND river_job.scheduled_at <= $1::timestamptz + AND scheduled_at <= $1::timestamptz ORDER BY priority, scheduled_at, @@ -1023,16 +1011,38 @@ WITH jobs_to_schedule AS ( LIMIT $2::bigint FOR UPDATE ), -river_job_scheduled AS ( +conflicting_jobs AS ( + SELECT DISTINCT unique_key + FROM river_job + WHERE unique_key IN ( + SELECT unique_key + FROM jobs_to_schedule + WHERE unique_key IS NOT NULL + AND unique_states IS NOT NULL + ) + AND unique_states IS NOT NULL + AND river_job_state_in_bitmask(unique_states, state) +), +updated_jobs AS ( UPDATE river_job - SET state = 'available' - FROM jobs_to_schedule - WHERE river_job.id = jobs_to_schedule.id - RETURNING river_job.id + SET + state = CASE WHEN cj.unique_key IS NULL THEN 'available'::river_job_state + ELSE 'discarded'::river_job_state END, + finalized_at = CASE WHEN cj.unique_key IS NULL THEN finalized_at + ELSE $1::timestamptz END, + -- Purely for debugging to understand when this code path was used: + metadata = CASE WHEN cj.unique_key IS NULL THEN metadata + ELSE metadata || '{"unique_key_conflict": "scheduler_discarded"}'::jsonb END + FROM jobs_to_schedule jts + LEFT JOIN conflicting_jobs cj ON jts.unique_key = cj.unique_key + WHERE river_job.id = jts.id + RETURNING river_job.id, state = 'discarded'::river_job_state AS conflict_discarded ) -SELECT id, args, attempt, attempted_at, attempted_by, created_at, errors, finalized_at, kind, max_attempts, metadata, priority, queue, state, scheduled_at, tags, unique_key +SELECT + river_job.id, river_job.args, river_job.attempt, river_job.attempted_at, river_job.attempted_by, river_job.created_at, river_job.errors, river_job.finalized_at, river_job.kind, river_job.max_attempts, river_job.metadata, river_job.priority, river_job.queue, river_job.state, river_job.scheduled_at, river_job.tags, river_job.unique_key, river_job.unique_states, + updated_jobs.conflict_discarded FROM river_job -WHERE id IN (SELECT id FROM river_job_scheduled) +JOIN updated_jobs ON river_job.id = updated_jobs.id ` type JobScheduleParams struct { @@ -1040,33 +1050,40 @@ type JobScheduleParams struct { Max int64 } -func (q *Queries) JobSchedule(ctx context.Context, db DBTX, arg *JobScheduleParams) ([]*RiverJob, error) { +type JobScheduleRow struct { + RiverJob RiverJob + ConflictDiscarded bool +} + +func (q *Queries) JobSchedule(ctx context.Context, db DBTX, arg *JobScheduleParams) ([]*JobScheduleRow, error) { rows, err := db.Query(ctx, jobSchedule, arg.Now, arg.Max) if err != nil { return nil, err } defer rows.Close() - var items []*RiverJob + var items []*JobScheduleRow for rows.Next() { - var i RiverJob + var i JobScheduleRow if err := rows.Scan( - &i.ID, - &i.Args, - &i.Attempt, - &i.AttemptedAt, - &i.AttemptedBy, - &i.CreatedAt, - &i.Errors, - &i.FinalizedAt, - &i.Kind, - &i.MaxAttempts, - &i.Metadata, - &i.Priority, - &i.Queue, - &i.State, - &i.ScheduledAt, - &i.Tags, - &i.UniqueKey, + &i.RiverJob.ID, + &i.RiverJob.Args, + &i.RiverJob.Attempt, + &i.RiverJob.AttemptedAt, + &i.RiverJob.AttemptedBy, + &i.RiverJob.CreatedAt, + &i.RiverJob.Errors, + &i.RiverJob.FinalizedAt, + &i.RiverJob.Kind, + &i.RiverJob.MaxAttempts, + &i.RiverJob.Metadata, + &i.RiverJob.Priority, + &i.RiverJob.Queue, + &i.RiverJob.State, + &i.RiverJob.ScheduledAt, + &i.RiverJob.Tags, + &i.RiverJob.UniqueKey, + &i.RiverJob.UniqueStates, + &i.ConflictDiscarded, ); err != nil { return nil, err } @@ -1098,13 +1115,13 @@ updated_job AS ( state = 'completed' FROM job_to_update WHERE river_job.id = job_to_update.id - RETURNING river_job.id, river_job.args, river_job.attempt, river_job.attempted_at, river_job.attempted_by, river_job.created_at, river_job.errors, river_job.finalized_at, river_job.kind, river_job.max_attempts, river_job.metadata, river_job.priority, river_job.queue, river_job.state, river_job.scheduled_at, river_job.tags, river_job.unique_key + RETURNING river_job.id, river_job.args, river_job.attempt, river_job.attempted_at, river_job.attempted_by, river_job.created_at, river_job.errors, river_job.finalized_at, river_job.kind, river_job.max_attempts, river_job.metadata, river_job.priority, river_job.queue, river_job.state, river_job.scheduled_at, river_job.tags, river_job.unique_key, river_job.unique_states ) -SELECT id, args, attempt, attempted_at, attempted_by, created_at, errors, finalized_at, kind, max_attempts, metadata, priority, queue, state, scheduled_at, tags, unique_key +SELECT id, args, attempt, attempted_at, attempted_by, created_at, errors, finalized_at, kind, max_attempts, metadata, priority, queue, state, scheduled_at, tags, unique_key, unique_states FROM river_job WHERE id IN (SELECT id FROM job_to_finalized_at EXCEPT SELECT id FROM updated_job) UNION -SELECT id, args, attempt, attempted_at, attempted_by, created_at, errors, finalized_at, kind, max_attempts, metadata, priority, queue, state, scheduled_at, tags, unique_key +SELECT id, args, attempt, attempted_at, attempted_by, created_at, errors, finalized_at, kind, max_attempts, metadata, priority, queue, state, scheduled_at, tags, unique_key, unique_states FROM updated_job ` @@ -1140,6 +1157,7 @@ func (q *Queries) JobSetCompleteIfRunningMany(ctx context.Context, db DBTX, arg &i.ScheduledAt, &i.Tags, &i.UniqueKey, + &i.UniqueStates, ); err != nil { return nil, err } @@ -1174,19 +1192,22 @@ updated_job AS ( ELSE max_attempts END, scheduled_at = CASE WHEN NOT should_cancel AND $9::boolean THEN $10::timestamptz ELSE scheduled_at END, - unique_key = CASE WHEN ($1 IN ('cancelled', 'discarded') OR should_cancel) THEN NULL + -- This is transitional code for the v2 uniqueness design. We specifically + -- avoid clearing this value if the v3 unique_states field is populated, + -- because the v3 design never involves clearing unique_key. + unique_key = CASE WHEN (unique_states IS NULL AND ($1 IN ('cancelled', 'discarded') OR should_cancel)) THEN NULL ELSE unique_key END FROM job_to_update WHERE river_job.id = job_to_update.id AND river_job.state = 'running' - RETURNING river_job.id, river_job.args, river_job.attempt, river_job.attempted_at, river_job.attempted_by, river_job.created_at, river_job.errors, river_job.finalized_at, river_job.kind, river_job.max_attempts, river_job.metadata, river_job.priority, river_job.queue, river_job.state, river_job.scheduled_at, river_job.tags, river_job.unique_key + RETURNING river_job.id, river_job.args, river_job.attempt, river_job.attempted_at, river_job.attempted_by, river_job.created_at, river_job.errors, river_job.finalized_at, river_job.kind, river_job.max_attempts, river_job.metadata, river_job.priority, river_job.queue, river_job.state, river_job.scheduled_at, river_job.tags, river_job.unique_key, river_job.unique_states ) -SELECT id, args, attempt, attempted_at, attempted_by, created_at, errors, finalized_at, kind, max_attempts, metadata, priority, queue, state, scheduled_at, tags, unique_key +SELECT id, args, attempt, attempted_at, attempted_by, created_at, errors, finalized_at, kind, max_attempts, metadata, priority, queue, state, scheduled_at, tags, unique_key, unique_states FROM river_job WHERE id = $2::bigint AND id NOT IN (SELECT id FROM updated_job) UNION -SELECT id, args, attempt, attempted_at, attempted_by, created_at, errors, finalized_at, kind, max_attempts, metadata, priority, queue, state, scheduled_at, tags, unique_key +SELECT id, args, attempt, attempted_at, attempted_by, created_at, errors, finalized_at, kind, max_attempts, metadata, priority, queue, state, scheduled_at, tags, unique_key, unique_states FROM updated_job ` @@ -1235,6 +1256,7 @@ func (q *Queries) JobSetStateIfRunning(ctx context.Context, db DBTX, arg *JobSet &i.ScheduledAt, &i.Tags, &i.UniqueKey, + &i.UniqueStates, ) return &i, err } @@ -1247,9 +1269,11 @@ SET errors = CASE WHEN $5::boolean THEN $6::jsonb[] ELSE errors END, finalized_at = CASE WHEN $7::boolean THEN $8 ELSE finalized_at END, state = CASE WHEN $9::boolean THEN $10 ELSE state END, + -- Transitional code to support tests for v2 uniqueness design. This field + -- is never modified in the v3 design. unique_key = CASE WHEN $11::boolean THEN $12 ELSE unique_key END WHERE id = $13 -RETURNING id, args, attempt, attempted_at, attempted_by, created_at, errors, finalized_at, kind, max_attempts, metadata, priority, queue, state, scheduled_at, tags, unique_key +RETURNING id, args, attempt, attempted_at, attempted_by, created_at, errors, finalized_at, kind, max_attempts, metadata, priority, queue, state, scheduled_at, tags, unique_key, unique_states ` type JobUpdateParams struct { @@ -1305,6 +1329,7 @@ func (q *Queries) JobUpdate(ctx context.Context, db DBTX, arg *JobUpdateParams) &i.ScheduledAt, &i.Tags, &i.UniqueKey, + &i.UniqueStates, ) return &i, err } diff --git a/riverdriver/riverpgxv5/internal/dbsqlc/river_job_copyfrom.sql b/riverdriver/riverpgxv5/internal/dbsqlc/river_job_copyfrom.sql index ec138fb8..3ce273d4 100644 --- a/riverdriver/riverpgxv5/internal/dbsqlc/river_job_copyfrom.sql +++ b/riverdriver/riverpgxv5/internal/dbsqlc/river_job_copyfrom.sql @@ -9,7 +9,9 @@ INSERT INTO river_job( queue, scheduled_at, state, - tags + tags, + unique_key, + unique_states ) VALUES ( @args, @finalized_at, @@ -20,5 +22,7 @@ INSERT INTO river_job( @queue, @scheduled_at, @state, - @tags -); \ No newline at end of file + @tags, + @unique_key, + @unique_states +); diff --git a/riverdriver/riverpgxv5/internal/dbsqlc/river_job_copyfrom.sql.go b/riverdriver/riverpgxv5/internal/dbsqlc/river_job_copyfrom.sql.go index bf77991e..47738d18 100644 --- a/riverdriver/riverpgxv5/internal/dbsqlc/river_job_copyfrom.sql.go +++ b/riverdriver/riverpgxv5/internal/dbsqlc/river_job_copyfrom.sql.go @@ -7,17 +7,21 @@ package dbsqlc import ( "time" + + "github.com/jackc/pgx/v5/pgtype" ) type JobInsertFastManyCopyFromParams struct { - Args []byte - FinalizedAt *time.Time - Kind string - MaxAttempts int16 - Metadata []byte - Priority int16 - Queue string - ScheduledAt time.Time - State RiverJobState - Tags []string + Args []byte + FinalizedAt *time.Time + Kind string + MaxAttempts int16 + Metadata []byte + Priority int16 + Queue string + ScheduledAt time.Time + State RiverJobState + Tags []string + UniqueKey []byte + UniqueStates pgtype.Bits } diff --git a/riverdriver/riverpgxv5/migration/main/006_bulk_unique.down.sql b/riverdriver/riverpgxv5/migration/main/006_bulk_unique.down.sql new file mode 100644 index 00000000..ae65cfed --- /dev/null +++ b/riverdriver/riverpgxv5/migration/main/006_bulk_unique.down.sql @@ -0,0 +1,16 @@ + +-- +-- Drop `river_job.unique_states` and its index. +-- + +DROP INDEX river_job_unique_idx; + +ALTER TABLE river_job + DROP COLUMN unique_states; + +CREATE UNIQUE INDEX IF NOT EXISTS river_job_kind_unique_key_idx ON river_job (kind, unique_key) WHERE unique_key IS NOT NULL; + +-- +-- Drop `river_job_state_in_bitmask` function. +-- +DROP FUNCTION river_job_state_in_bitmask; diff --git a/riverdriver/riverpgxv5/migration/main/006_bulk_unique.up.sql b/riverdriver/riverpgxv5/migration/main/006_bulk_unique.up.sql new file mode 100644 index 00000000..e149dfba --- /dev/null +++ b/riverdriver/riverpgxv5/migration/main/006_bulk_unique.up.sql @@ -0,0 +1,38 @@ + +CREATE OR REPLACE FUNCTION river_job_state_in_bitmask(bitmask BIT(8), state river_job_state) +RETURNS boolean +LANGUAGE SQL +IMMUTABLE +AS $$ + SELECT CASE state + WHEN 'available' THEN get_bit(bitmask, 7) + WHEN 'cancelled' THEN get_bit(bitmask, 6) + WHEN 'completed' THEN get_bit(bitmask, 5) + WHEN 'discarded' THEN get_bit(bitmask, 4) + WHEN 'pending' THEN get_bit(bitmask, 3) + WHEN 'retryable' THEN get_bit(bitmask, 2) + WHEN 'running' THEN get_bit(bitmask, 1) + WHEN 'scheduled' THEN get_bit(bitmask, 0) + ELSE 0 + END = 1; +$$; + +-- +-- Add `river_job.unique_states` and bring up an index on it. +-- +ALTER TABLE river_job ADD COLUMN unique_states BIT(8); + +-- This statements uses `IF NOT EXISTS` to allow users with a `river_job` table +-- of non-trivial size to build the index `CONCURRENTLY` out of band of this +-- migration, then follow by completing the migration. +CREATE UNIQUE INDEX IF NOT EXISTS river_job_unique_idx ON river_job (unique_key) + WHERE unique_key IS NOT NULL + AND unique_states IS NOT NULL + AND river_job_state_in_bitmask(unique_states, state); + +-- Remove the old unique index. Users who are actively using the unique jobs +-- feature and who wish to avoid deploy downtime may want od drop this in a +-- subsequent migration once all jobs using the old unique system have been +-- completed (i.e. no more rows with non-null unique_key and null +-- unique_states). +DROP INDEX river_job_kind_unique_key_idx; diff --git a/riverdriver/riverpgxv5/river_pgx_v5_driver.go b/riverdriver/riverpgxv5/river_pgx_v5_driver.go index 659f60d1..3107f598 100644 --- a/riverdriver/riverpgxv5/river_pgx_v5_driver.go +++ b/riverdriver/riverpgxv5/river_pgx_v5_driver.go @@ -17,9 +17,11 @@ import ( "time" "github.com/jackc/pgx/v5" + "github.com/jackc/pgx/v5/pgtype" "github.com/jackc/pgx/v5/pgxpool" "github.com/jackc/puddle/v2" + "github.com/riverqueue/river/internal/dbunique" "github.com/riverqueue/river/riverdriver" "github.com/riverqueue/river/riverdriver/riverpgxv5/internal/dbsqlc" "github.com/riverqueue/river/rivershared/util/sliceutil" @@ -192,36 +194,44 @@ func (e *Executor) JobGetStuck(ctx context.Context, params *riverdriver.JobGetSt return mapSliceError(jobs, jobRowFromInternal) } -func (e *Executor) JobInsertFast(ctx context.Context, params *riverdriver.JobInsertFastParams) (*rivertype.JobRow, error) { - job, err := dbsqlc.New().JobInsertFast(ctx, e.dbtx, &dbsqlc.JobInsertFastParams{ - Args: params.EncodedArgs, - CreatedAt: params.CreatedAt, - Kind: params.Kind, - MaxAttempts: int16(min(params.MaxAttempts, math.MaxInt16)), //nolint:gosec - Metadata: params.Metadata, - Priority: int16(min(params.Priority, math.MaxInt16)), //nolint:gosec - Queue: params.Queue, - ScheduledAt: params.ScheduledAt, - State: dbsqlc.RiverJobState(params.State), - Tags: params.Tags, +func (e *Executor) JobInsertFast(ctx context.Context, params *riverdriver.JobInsertFastParams) (*riverdriver.JobInsertFastResult, error) { + result, err := dbsqlc.New().JobInsertFast(ctx, e.dbtx, &dbsqlc.JobInsertFastParams{ + Args: params.EncodedArgs, + CreatedAt: params.CreatedAt, + Kind: params.Kind, + MaxAttempts: int16(min(params.MaxAttempts, math.MaxInt16)), //nolint:gosec + Metadata: params.Metadata, + Priority: int16(min(params.Priority, math.MaxInt16)), //nolint:gosec + Queue: params.Queue, + ScheduledAt: params.ScheduledAt, + State: dbsqlc.RiverJobState(params.State), + Tags: params.Tags, + UniqueKey: params.UniqueKey, + UniqueStates: pgtype.Bits{Bytes: []byte{params.UniqueStates}, Len: 8, Valid: params.UniqueStates != 0}, }) if err != nil { return nil, interpretError(err) } - return jobRowFromInternal(job) + externalJob, err := jobRowFromInternal(&result.RiverJob) + if err != nil { + return nil, err + } + return &riverdriver.JobInsertFastResult{Job: externalJob, UniqueSkippedAsDuplicate: result.UniqueSkippedAsDuplicate}, nil } -func (e *Executor) JobInsertFastMany(ctx context.Context, params []*riverdriver.JobInsertFastParams) ([]*rivertype.JobRow, error) { +func (e *Executor) JobInsertFastMany(ctx context.Context, params []*riverdriver.JobInsertFastParams) ([]*riverdriver.JobInsertFastResult, error) { insertJobsParams := &dbsqlc.JobInsertFastManyParams{ - Args: make([][]byte, len(params)), - Kind: make([]string, len(params)), - MaxAttempts: make([]int16, len(params)), - Metadata: make([][]byte, len(params)), - Priority: make([]int16, len(params)), - Queue: make([]string, len(params)), - ScheduledAt: make([]time.Time, len(params)), - State: make([]string, len(params)), - Tags: make([]string, len(params)), + Args: make([][]byte, len(params)), + Kind: make([]string, len(params)), + MaxAttempts: make([]int16, len(params)), + Metadata: make([][]byte, len(params)), + Priority: make([]int16, len(params)), + Queue: make([]string, len(params)), + ScheduledAt: make([]time.Time, len(params)), + State: make([]string, len(params)), + Tags: make([]string, len(params)), + UniqueKey: make([][]byte, len(params)), + UniqueStates: make([]pgtype.Bits, len(params)), } now := time.Now() @@ -249,6 +259,8 @@ func (e *Executor) JobInsertFastMany(ctx context.Context, params []*riverdriver. insertJobsParams.ScheduledAt[i] = scheduledAt insertJobsParams.State[i] = string(params.State) insertJobsParams.Tags[i] = strings.Join(tags, ",") + insertJobsParams.UniqueKey[i] = sliceutil.DefaultIfEmpty(params.UniqueKey, nil) + insertJobsParams.UniqueStates[i] = pgtype.Bits{Bytes: []byte{params.UniqueStates}, Len: 8, Valid: params.UniqueStates != 0} } items, err := dbsqlc.New().JobInsertFastMany(ctx, e.dbtx, insertJobsParams) @@ -256,7 +268,13 @@ func (e *Executor) JobInsertFastMany(ctx context.Context, params []*riverdriver. return nil, interpretError(err) } - return mapSliceError(items, jobRowFromInternal) + return mapSliceError(items, func(row *dbsqlc.JobInsertFastManyRow) (*riverdriver.JobInsertFastResult, error) { + job, err := jobRowFromInternal(&row.RiverJob) + if err != nil { + return nil, err + } + return &riverdriver.JobInsertFastResult{Job: job, UniqueSkippedAsDuplicate: row.UniqueSkippedAsDuplicate}, nil + }) } func (e *Executor) JobInsertFastManyNoReturning(ctx context.Context, params []*riverdriver.JobInsertFastParams) (int, error) { @@ -282,15 +300,17 @@ func (e *Executor) JobInsertFastManyNoReturning(ctx context.Context, params []*r } insertJobsParams[i] = &dbsqlc.JobInsertFastManyCopyFromParams{ - Args: params.EncodedArgs, - Kind: params.Kind, - MaxAttempts: int16(min(params.MaxAttempts, math.MaxInt16)), //nolint:gosec - Metadata: metadata, - Priority: int16(min(params.Priority, math.MaxInt16)), //nolint:gosec - Queue: params.Queue, - ScheduledAt: scheduledAt, - State: dbsqlc.RiverJobState(params.State), - Tags: tags, + Args: params.EncodedArgs, + Kind: params.Kind, + MaxAttempts: int16(min(params.MaxAttempts, math.MaxInt16)), //nolint:gosec + Metadata: metadata, + Priority: int16(min(params.Priority, math.MaxInt16)), //nolint:gosec + Queue: params.Queue, + ScheduledAt: scheduledAt, + State: dbsqlc.RiverJobState(params.State), + Tags: tags, + UniqueKey: params.UniqueKey, + UniqueStates: pgtype.Bits{Bytes: []byte{params.UniqueStates}, Len: 8, Valid: params.UniqueStates != 0}, } } @@ -304,21 +324,22 @@ func (e *Executor) JobInsertFastManyNoReturning(ctx context.Context, params []*r func (e *Executor) JobInsertFull(ctx context.Context, params *riverdriver.JobInsertFullParams) (*rivertype.JobRow, error) { job, err := dbsqlc.New().JobInsertFull(ctx, e.dbtx, &dbsqlc.JobInsertFullParams{ - Attempt: int16(min(params.Attempt, math.MaxInt16)), //nolint:gosec - AttemptedAt: params.AttemptedAt, - Args: params.EncodedArgs, - CreatedAt: params.CreatedAt, - Errors: params.Errors, - FinalizedAt: params.FinalizedAt, - Kind: params.Kind, - MaxAttempts: int16(min(params.MaxAttempts, math.MaxInt16)), //nolint:gosec - Metadata: params.Metadata, - Priority: int16(min(params.Priority, math.MaxInt16)), //nolint:gosec - Queue: params.Queue, - ScheduledAt: params.ScheduledAt, - State: dbsqlc.RiverJobState(params.State), - Tags: params.Tags, - UniqueKey: params.UniqueKey, + Attempt: int16(min(params.Attempt, math.MaxInt16)), //nolint:gosec + AttemptedAt: params.AttemptedAt, + Args: params.EncodedArgs, + CreatedAt: params.CreatedAt, + Errors: params.Errors, + FinalizedAt: params.FinalizedAt, + Kind: params.Kind, + MaxAttempts: int16(min(params.MaxAttempts, math.MaxInt16)), //nolint:gosec + Metadata: params.Metadata, + Priority: int16(min(params.Priority, math.MaxInt16)), //nolint:gosec + Queue: params.Queue, + ScheduledAt: params.ScheduledAt, + State: dbsqlc.RiverJobState(params.State), + Tags: params.Tags, + UniqueKey: params.UniqueKey, + UniqueStates: pgtype.Bits{Bytes: []byte{params.UniqueStates}, Len: 8, Valid: params.UniqueStates != 0}, }) if err != nil { return nil, interpretError(err) @@ -326,35 +347,6 @@ func (e *Executor) JobInsertFull(ctx context.Context, params *riverdriver.JobIns return jobRowFromInternal(job) } -func (e *Executor) JobInsertUnique(ctx context.Context, params *riverdriver.JobInsertUniqueParams) (*riverdriver.JobInsertUniqueResult, error) { - insertRes, err := dbsqlc.New().JobInsertUnique(ctx, e.dbtx, &dbsqlc.JobInsertUniqueParams{ - Args: params.EncodedArgs, - CreatedAt: params.CreatedAt, - Kind: params.Kind, - MaxAttempts: int16(min(params.MaxAttempts, math.MaxInt16)), //nolint:gosec - Metadata: params.Metadata, - Priority: int16(min(params.Priority, math.MaxInt16)), //nolint:gosec - Queue: params.Queue, - ScheduledAt: params.ScheduledAt, - State: dbsqlc.RiverJobState(params.State), - Tags: params.Tags, - UniqueKey: params.UniqueKey, - }) - if err != nil { - return nil, interpretError(err) - } - - jobRow, err := jobRowFromInternal(&insertRes.RiverJob) - if err != nil { - return nil, err - } - - return &riverdriver.JobInsertUniqueResult{ - Job: jobRow, - UniqueSkippedAsDuplicate: insertRes.UniqueSkippedAsDuplicate, - }, nil -} - func (e *Executor) JobList(ctx context.Context, query string, namedArgs map[string]any) ([]*rivertype.JobRow, error) { rows, err := e.dbtx.Query(ctx, query, pgx.NamedArgs(namedArgs)) if err != nil { @@ -382,6 +374,8 @@ func (e *Executor) JobList(ctx context.Context, query string, namedArgs map[stri &i.State, &i.ScheduledAt, &i.Tags, + &i.UniqueKey, + &i.UniqueStates, ); err != nil { return nil, err } @@ -395,7 +389,7 @@ func (e *Executor) JobList(ctx context.Context, query string, namedArgs map[stri } func (e *Executor) JobListFields() string { - return "id, args, attempt, attempted_at, attempted_by, created_at, errors, finalized_at, kind, max_attempts, metadata, priority, queue, state, scheduled_at, tags" + return "id, args, attempt, attempted_at, attempted_by, created_at, errors, finalized_at, kind, max_attempts, metadata, priority, queue, state, scheduled_at, tags, unique_key, unique_states" } func (e *Executor) JobRescueMany(ctx context.Context, params *riverdriver.JobRescueManyParams) (*struct{}, error) { @@ -414,15 +408,21 @@ func (e *Executor) JobRetry(ctx context.Context, id int64) (*rivertype.JobRow, e return jobRowFromInternal(job) } -func (e *Executor) JobSchedule(ctx context.Context, params *riverdriver.JobScheduleParams) ([]*rivertype.JobRow, error) { - jobs, err := dbsqlc.New().JobSchedule(ctx, e.dbtx, &dbsqlc.JobScheduleParams{ +func (e *Executor) JobSchedule(ctx context.Context, params *riverdriver.JobScheduleParams) ([]*riverdriver.JobScheduleResult, error) { + scheduleResults, err := dbsqlc.New().JobSchedule(ctx, e.dbtx, &dbsqlc.JobScheduleParams{ Max: int64(params.Max), Now: params.Now, }) if err != nil { return nil, interpretError(err) } - return mapSliceError(jobs, jobRowFromInternal) + return mapSliceError(scheduleResults, func(result *dbsqlc.JobScheduleRow) (*riverdriver.JobScheduleResult, error) { + job, err := jobRowFromInternal(&result.RiverJob) + if err != nil { + return nil, err + } + return &riverdriver.JobScheduleResult{ConflictDiscarded: result.ConflictDiscarded, Job: *job}, nil + }) } func (e *Executor) JobSetCompleteIfRunningMany(ctx context.Context, params *riverdriver.JobSetCompleteIfRunningManyParams) ([]*rivertype.JobRow, error) { @@ -473,8 +473,6 @@ func (e *Executor) JobUpdate(ctx context.Context, params *riverdriver.JobUpdateP FinalizedAt: params.FinalizedAt, StateDoUpdate: params.StateDoUpdate, State: dbsqlc.RiverJobState(params.State), - UniqueKeyDoUpdate: params.UniqueKeyDoUpdate, - UniqueKey: params.UniqueKey, }) if err != nil { return nil, interpretError(err) @@ -841,24 +839,30 @@ func jobRowFromInternal(internal *dbsqlc.RiverJob) (*rivertype.JobRow, error) { finalizedAt = &t } + var uniqueStatesByte byte + if internal.UniqueStates.Valid && len(internal.UniqueStates.Bytes) > 0 { + uniqueStatesByte = internal.UniqueStates.Bytes[0] + } + return &rivertype.JobRow{ - ID: internal.ID, - Attempt: max(int(internal.Attempt), 0), - AttemptedAt: attemptedAt, - AttemptedBy: internal.AttemptedBy, - CreatedAt: internal.CreatedAt.UTC(), - EncodedArgs: internal.Args, - Errors: errors, - FinalizedAt: finalizedAt, - Kind: internal.Kind, - MaxAttempts: max(int(internal.MaxAttempts), 0), - Metadata: internal.Metadata, - Priority: max(int(internal.Priority), 0), - Queue: internal.Queue, - ScheduledAt: internal.ScheduledAt.UTC(), - State: rivertype.JobState(internal.State), - Tags: internal.Tags, - UniqueKey: internal.UniqueKey, + ID: internal.ID, + Attempt: max(int(internal.Attempt), 0), + AttemptedAt: attemptedAt, + AttemptedBy: internal.AttemptedBy, + CreatedAt: internal.CreatedAt.UTC(), + EncodedArgs: internal.Args, + Errors: errors, + FinalizedAt: finalizedAt, + Kind: internal.Kind, + MaxAttempts: max(int(internal.MaxAttempts), 0), + Metadata: internal.Metadata, + Priority: max(int(internal.Priority), 0), + Queue: internal.Queue, + ScheduledAt: internal.ScheduledAt.UTC(), + State: rivertype.JobState(internal.State), + Tags: internal.Tags, + UniqueKey: internal.UniqueKey, + UniqueStates: dbunique.UniqueBitmaskToStates(uniqueStatesByte), }, nil } diff --git a/rivershared/testfactory/test_factory.go b/rivershared/testfactory/test_factory.go index 2ab5fb78..f6f3dd4c 100644 --- a/rivershared/testfactory/test_factory.go +++ b/rivershared/testfactory/test_factory.go @@ -19,21 +19,22 @@ import ( ) type JobOpts struct { - Attempt *int - AttemptedAt *time.Time - CreatedAt *time.Time - EncodedArgs []byte - Errors [][]byte - FinalizedAt *time.Time - Kind *string - MaxAttempts *int - Metadata json.RawMessage - Priority *int - Queue *string - ScheduledAt *time.Time - State *rivertype.JobState - Tags []string - UniqueKey []byte + Attempt *int + AttemptedAt *time.Time + CreatedAt *time.Time + EncodedArgs []byte + Errors [][]byte + FinalizedAt *time.Time + Kind *string + MaxAttempts *int + Metadata json.RawMessage + Priority *int + Queue *string + ScheduledAt *time.Time + State *rivertype.JobState + Tags []string + UniqueKey []byte + UniqueStates byte } func Job(ctx context.Context, tb testing.TB, exec riverdriver.Executor, opts *JobOpts) *rivertype.JobRow { @@ -70,21 +71,22 @@ func Job_Build(tb testing.TB, opts *JobOpts) *riverdriver.JobInsertFullParams { } return &riverdriver.JobInsertFullParams{ - Attempt: ptrutil.ValOrDefault(opts.Attempt, 0), - AttemptedAt: opts.AttemptedAt, - CreatedAt: opts.CreatedAt, - EncodedArgs: encodedArgs, - Errors: opts.Errors, - FinalizedAt: finalizedAt, - Kind: ptrutil.ValOrDefault(opts.Kind, "fake_job"), - MaxAttempts: ptrutil.ValOrDefault(opts.MaxAttempts, rivercommon.MaxAttemptsDefault), - Metadata: metadata, - Priority: ptrutil.ValOrDefault(opts.Priority, rivercommon.PriorityDefault), - Queue: ptrutil.ValOrDefault(opts.Queue, rivercommon.QueueDefault), - ScheduledAt: opts.ScheduledAt, - State: ptrutil.ValOrDefault(opts.State, rivertype.JobStateAvailable), - Tags: tags, - UniqueKey: opts.UniqueKey, + Attempt: ptrutil.ValOrDefault(opts.Attempt, 0), + AttemptedAt: opts.AttemptedAt, + CreatedAt: opts.CreatedAt, + EncodedArgs: encodedArgs, + Errors: opts.Errors, + FinalizedAt: finalizedAt, + Kind: ptrutil.ValOrDefault(opts.Kind, "fake_job"), + MaxAttempts: ptrutil.ValOrDefault(opts.MaxAttempts, rivercommon.MaxAttemptsDefault), + Metadata: metadata, + Priority: ptrutil.ValOrDefault(opts.Priority, rivercommon.PriorityDefault), + Queue: ptrutil.ValOrDefault(opts.Queue, rivercommon.QueueDefault), + ScheduledAt: opts.ScheduledAt, + State: ptrutil.ValOrDefault(opts.State, rivertype.JobStateAvailable), + Tags: tags, + UniqueKey: opts.UniqueKey, + UniqueStates: opts.UniqueStates, } } diff --git a/rivertype/river_type.go b/rivertype/river_type.go index 0f8f309c..1998577a 100644 --- a/rivertype/river_type.go +++ b/rivertype/river_type.go @@ -17,6 +17,15 @@ var ErrNotFound = errors.New("not found") // running. var ErrJobRunning = errors.New("running jobs cannot be deleted") +// JobArgs is an interface that should be implemented by the arguments to a job. +// This definition duplicates the JobArgs interface in the river package so that +// it can be used in other packages without creating a circular dependency. +type JobArgs interface { + // Kind returns a unique string that identifies the type of job. It's used to + // determine which worker should work the job. + Kind() string +} + // JobInsertResult is the result of a job insert, containing the inserted job // along with some other useful metadata. type JobInsertResult struct { @@ -118,6 +127,8 @@ type JobRow struct { // unique job insertions. It's generated by hashing an inserted job's unique // opts configuration. UniqueKey []byte + + UniqueStates []JobState } // JobState is the state of a job. Jobs start their lifecycle as either