From 1c3e89222c25e3629a6e5581cdc763f4f1506668 Mon Sep 17 00:00:00 2001 From: Blake Gentry Date: Sat, 14 Sep 2024 14:03:37 -0500 Subject: [PATCH] new unique jobs implementation that works on bulk insert --- client.go | 70 +- client_test.go | 75 +- driver_test.go | 21 +- insert_opts.go | 5 + internal/dbunique/db_unique.go | 206 +---- internal/dbunique/db_unique_test.go | 867 ++++-------------- internal/maintenance/periodic_job_enqueuer.go | 87 +- .../maintenance/periodic_job_enqueuer_test.go | 25 +- internal/maintenance/queue_maintainer_test.go | 5 +- .../riverdrivertest/riverdrivertest.go | 248 ++--- job_executor_test.go | 12 +- periodic_job.go | 5 +- periodic_job_test.go | 6 +- producer_test.go | 4 +- riverdriver/river_driver_interface.go | 71 +- .../internal/dbsqlc/models.go | 35 +- .../internal/dbsqlc/river_job.sql.go | 422 ++++----- .../internal/dbsqlc/sqlc.yaml | 6 + .../migration/main/006_bulk_unique.down.sql | 16 + .../migration/main/006_bulk_unique.up.sql | 38 + .../river_database_sql_driver.go | 118 ++- .../riverpgxv5/internal/dbsqlc/copyfrom.go | 4 +- .../riverpgxv5/internal/dbsqlc/models.go | 37 +- .../riverpgxv5/internal/dbsqlc/river_job.sql | 97 +- .../internal/dbsqlc/river_job.sql.go | 424 ++++----- .../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 | 116 ++- rivermigrate/river_migrate.go | 33 +- 31 files changed, 1310 insertions(+), 1831 deletions(-) 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/client.go b/client.go index 89509c07..a7c49eb8 100644 --- a/client.go +++ b/client.go @@ -53,24 +53,6 @@ const ( // it so River can check that inserted job kinds have a worker that can run // them. type Config struct { - // AdvisoryLockPrefix is a configurable 32-bit prefix that River will use - // when generating any key to acquire a Postgres advisory lock. All advisory - // locks share the same 64-bit number space, so this allows a calling - // application to guarantee that a River advisory lock will never conflict - // with one of its own by cordoning each type to its own prefix. - // - // If this value isn't set, River defaults to generating key hashes across - // the entire 64-bit advisory lock number space, which is large enough that - // conflicts are exceedingly unlikely. If callers don't strictly need this - // option then it's recommended to leave it unset because the prefix leaves - // 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. - AdvisoryLockPrefix int32 - // CancelledJobRetentionPeriod is the amount of time to keep cancelled jobs // around before they're removed permanently. // @@ -339,7 +321,6 @@ type Client[TTx any] struct { stopped <-chan struct{} subscriptionManager *subscriptionManager testSignals clientTestSignals - uniqueInserter *dbunique.UniqueInserter // workCancel cancels the context used for all work goroutines. Normal Stop // does not cancel that context. @@ -451,7 +432,6 @@ func NewClient[TTx any](driver riverdriver.Driver[TTx], config *Config) (*Client // original object, so everything that we care about must be initialized // here, even if it's only carrying over the original value. config = &Config{ - AdvisoryLockPrefix: config.AdvisoryLockPrefix, CancelledJobRetentionPeriod: valutil.ValOrDefault(config.CancelledJobRetentionPeriod, maintenance.CancelledJobRetentionPeriodDefault), CompletedJobRetentionPeriod: valutil.ValOrDefault(config.CompletedJobRetentionPeriod, maintenance.CompletedJobRetentionPeriodDefault), DiscardedJobRetentionPeriod: valutil.ValOrDefault(config.DiscardedJobRetentionPeriod, maintenance.DiscardedJobRetentionPeriodDefault), @@ -488,10 +468,7 @@ func NewClient[TTx any](driver riverdriver.Driver[TTx], config *Config) (*Client driver: driver, producersByQueueName: make(map[string]*producer), testSignals: clientTestSignals{}, - uniqueInserter: baseservice.Init(archetype, &dbunique.UniqueInserter{ - AdvisoryLockPrefix: config.AdvisoryLockPrefix, - }), - workCancel: func(cause error) {}, // replaced on start, but here in case StopAndCancel is called before start up + workCancel: func(cause error) {}, // replaced on start, but here in case StopAndCancel is called before start up } client.queues = &QueueBundle{addProducer: client.addProducer} @@ -589,8 +566,7 @@ func NewClient[TTx any](driver riverdriver.Driver[TTx], config *Config) (*Client { periodicJobEnqueuer := maintenance.NewPeriodicJobEnqueuer(archetype, &maintenance.PeriodicJobEnqueuerConfig{ - AdvisoryLockPrefix: config.AdvisoryLockPrefix, - NotifyInsert: client.maybeNotifyInsertForQueues, + NotifyInsert: client.maybeNotifyInsertForQueues, }, driver.GetExecutor()) maintenanceServices = append(maintenanceServices, periodicJobEnqueuer) client.testSignals.periodicJobEnqueuer = &periodicJobEnqueuer.TestSignals @@ -1150,10 +1126,10 @@ 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) (*riverdriver.JobInsertFastParams, error) { encodedArgs, err := json.Marshal(args) if err != nil { - return nil, nil, fmt.Errorf("error marshaling args to JSON: %w", err) + return nil, fmt.Errorf("error marshaling args to JSON: %w", err) } if insertOpts == nil { @@ -1175,7 +1151,7 @@ func insertParamsFromConfigArgsAndOptions(archetype *baseservice.Archetype, conf queue := valutil.FirstNonZero(insertOpts.Queue, jobInsertOpts.Queue, rivercommon.QueueDefault) if err := validateQueueName(queue); err != nil { - return nil, nil, err + return nil, err } tags := insertOpts.Tags @@ -1187,16 +1163,16 @@ func insertParamsFromConfigArgsAndOptions(archetype *baseservice.Archetype, conf } else { for _, tag := range tags { if len(tag) > 255 { - return nil, nil, errors.New("tags should be a maximum of 255 characters long") + return nil, errors.New("tags should be a maximum of 255 characters long") } if !tagRE.MatchString(tag) { - return nil, nil, errors.New("tags should match regex " + tagRE.String()) + return nil, errors.New("tags should match regex " + tagRE.String()) } } } if priority > 4 { - return nil, nil, errors.New("priority must be between 1 and 4") + return nil, errors.New("priority must be between 1 and 4") } uniqueOpts := insertOpts.UniqueOpts @@ -1204,7 +1180,7 @@ func insertParamsFromConfigArgsAndOptions(archetype *baseservice.Archetype, conf uniqueOpts = jobInsertOpts.UniqueOpts } if err := uniqueOpts.validate(); err != nil { - return nil, nil, err + return nil, err } metadata := insertOpts.Metadata @@ -1223,6 +1199,11 @@ func insertParamsFromConfigArgsAndOptions(archetype *baseservice.Archetype, conf State: rivertype.JobStateAvailable, Tags: tags, } + if !uniqueOpts.isEmpty() { + internalUniqueOpts := (*dbunique.UniqueOpts)(&uniqueOpts) + insertParams.UniqueKey = dbunique.UniqueKey(archetype.Time, internalUniqueOpts, insertParams) + insertParams.UniqueStates = internalUniqueOpts.StateBitmask() + } switch { case !insertOpts.ScheduledAt.IsZero(): @@ -1241,7 +1222,7 @@ func insertParamsFromConfigArgsAndOptions(archetype *baseservice.Archetype, conf insertParams.State = rivertype.JobStatePending } - return insertParams, (*dbunique.UniqueOpts)(&uniqueOpts), nil + return insertParams, 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") @@ -1290,7 +1271,7 @@ func (c *Client[TTx]) insert(ctx context.Context, exec riverdriver.Executor, arg return nil, err } - params, uniqueOpts, err := insertParamsFromConfigArgsAndOptions(&c.baseService.Archetype, c.config, args, opts) + params, err := insertParamsFromConfigArgsAndOptions(&c.baseService.Archetype, c.config, args, opts) if err != nil { return nil, err } @@ -1301,7 +1282,14 @@ 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) + // TODO: move into insertParamsFromConfigArgsAndOptions + // params.UniqueKey = uniqueOpts.UniqueKey() + // params.UniqueStates = uniqueOpts.UniqueStates() + + jobInsertRes, err := tx.JobInsertFast(ctx, params) + // JobInsertFastParams: params, + // UniqueKey: uniqueKeyHash[:], + // }) if err != nil { return nil, err } @@ -1313,7 +1301,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 @@ -1419,8 +1407,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 } @@ -1448,7 +1436,7 @@ func (c *Client[TTx]) insertManyParams(params []InsertManyParams) ([]*riverdrive } var err error - insertParams[i], _, err = insertParamsFromConfigArgsAndOptions(&c.baseService.Archetype, c.config, param.Args, param.InsertOpts) + insertParams[i], err = insertParamsFromConfigArgsAndOptions(&c.baseService.Archetype, c.config, param.Args, param.InsertOpts) if err != nil { return nil, err } @@ -1577,7 +1565,7 @@ func (c *Client[TTx]) insertManyFastParams(params []InsertManyParams) ([]*riverd } var err error - insertParams[i], _, err = insertParamsFromConfigArgsAndOptions(&c.baseService.Archetype, c.config, param.Args, param.InsertOpts) + insertParams[i], err = insertParamsFromConfigArgsAndOptions(&c.baseService.Archetype, c.config, param.Args, param.InsertOpts) if err != nil { return nil, err } diff --git a/client_test.go b/client_test.go index e38cbd24..8809e65d 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" @@ -2886,7 +2887,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) require.NoError(t, err) _, err = client.driver.GetExecutor().JobInsertFast(ctx, insertParams) require.NoError(t, err) @@ -4474,19 +4475,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) 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) @@ -4608,8 +4609,6 @@ func Test_NewClient_Defaults(t *testing.T) { }) require.NoError(t, err) - require.Zero(t, client.uniqueInserter.AdvisoryLockPrefix) - jobCleaner := maintenance.GetService[*maintenance.JobCleaner](client.queueMaintainer) require.Equal(t, maintenance.CancelledJobRetentionPeriodDefault, jobCleaner.Config.CancelledJobRetentionPeriod) require.Equal(t, maintenance.CompletedJobRetentionPeriodDefault, jobCleaner.Config.CompletedJobRetentionPeriod) @@ -4617,7 +4616,6 @@ func Test_NewClient_Defaults(t *testing.T) { require.False(t, jobCleaner.StaggerStartupIsDisabled()) enqueuer := maintenance.GetService[*maintenance.PeriodicJobEnqueuer](client.queueMaintainer) - require.Zero(t, enqueuer.Config.AdvisoryLockPrefix) require.False(t, enqueuer.StaggerStartupIsDisabled()) require.Nil(t, client.config.ErrorHandler) @@ -4644,7 +4642,6 @@ func Test_NewClient_Overrides(t *testing.T) { retryPolicy := &DefaultClientRetryPolicy{} client, err := NewClient(riverpgxv5.New(dbPool), &Config{ - AdvisoryLockPrefix: 123_456, CancelledJobRetentionPeriod: 1 * time.Hour, CompletedJobRetentionPeriod: 2 * time.Hour, DiscardedJobRetentionPeriod: 3 * time.Hour, @@ -4661,8 +4658,6 @@ func Test_NewClient_Overrides(t *testing.T) { }) require.NoError(t, err) - require.Equal(t, int32(123_456), client.uniqueInserter.AdvisoryLockPrefix) - jobCleaner := maintenance.GetService[*maintenance.JobCleaner](client.queueMaintainer) require.Equal(t, 1*time.Hour, jobCleaner.Config.CancelledJobRetentionPeriod) require.Equal(t, 2*time.Hour, jobCleaner.Config.CompletedJobRetentionPeriod) @@ -4670,7 +4665,6 @@ func Test_NewClient_Overrides(t *testing.T) { require.True(t, jobCleaner.StaggerStartupIsDisabled()) enqueuer := maintenance.GetService[*maintenance.PeriodicJobEnqueuer](client.queueMaintainer) - require.Equal(t, int32(123_456), enqueuer.Config.AdvisoryLockPrefix) require.True(t, enqueuer.StaggerStartupIsDisabled()) require.Equal(t, errorHandler, client.config.ErrorHandler) @@ -5075,7 +5069,7 @@ func TestInsertParamsFromJobArgsAndOptions(t *testing.T) { t.Run("Defaults", func(t *testing.T) { t.Parallel() - insertParams, uniqueOpts, err := insertParamsFromConfigArgsAndOptions(archetype, config, noOpArgs{}, nil) + insertParams, err := insertParamsFromConfigArgsAndOptions(archetype, config, noOpArgs{}, nil) require.NoError(t, err) require.Equal(t, `{"name":""}`, string(insertParams.EncodedArgs)) require.Equal(t, (noOpArgs{}).Kind(), insertParams.Kind) @@ -5085,7 +5079,8 @@ func TestInsertParamsFromJobArgsAndOptions(t *testing.T) { require.Nil(t, insertParams.ScheduledAt) require.Equal(t, []string{}, insertParams.Tags) - require.True(t, uniqueOpts.IsEmpty()) + require.Empty(t, insertParams.UniqueKey) + require.Zero(t, insertParams.UniqueStates) }) t.Run("ConfigOverrides", func(t *testing.T) { @@ -5095,7 +5090,7 @@ func TestInsertParamsFromJobArgsAndOptions(t *testing.T) { MaxAttempts: 34, } - insertParams, _, err := insertParamsFromConfigArgsAndOptions(archetype, overrideConfig, noOpArgs{}, nil) + insertParams, err := insertParamsFromConfigArgsAndOptions(archetype, overrideConfig, noOpArgs{}, nil) require.NoError(t, err) require.Equal(t, overrideConfig.MaxAttempts, insertParams.MaxAttempts) }) @@ -5110,7 +5105,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) require.NoError(t, err) require.Equal(t, 42, insertParams.MaxAttempts) require.Equal(t, 2, insertParams.Priority) @@ -5124,7 +5119,7 @@ func TestInsertParamsFromJobArgsAndOptions(t *testing.T) { nearFuture := time.Now().Add(5 * time.Minute) - insertParams, _, err := insertParamsFromConfigArgsAndOptions(archetype, config, &customInsertOptsJobArgs{ + insertParams, err := insertParamsFromConfigArgsAndOptions(archetype, config, &customInsertOptsJobArgs{ ScheduledAt: nearFuture, }, nil) require.NoError(t, err) @@ -5140,7 +5135,7 @@ func TestInsertParamsFromJobArgsAndOptions(t *testing.T) { t.Run("WorkerInsertOptsScheduledAtNotRespectedIfZero", func(t *testing.T) { t.Parallel() - insertParams, _, err := insertParamsFromConfigArgsAndOptions(archetype, config, &customInsertOptsJobArgs{ + insertParams, err := insertParamsFromConfigArgsAndOptions(archetype, config, &customInsertOptsJobArgs{ ScheduledAt: time.Time{}, }, nil) require.NoError(t, err) @@ -5151,14 +5146,14 @@ func TestInsertParamsFromJobArgsAndOptions(t *testing.T) { t.Parallel() { - _, _, err := insertParamsFromConfigArgsAndOptions(archetype, config, &customInsertOptsJobArgs{}, &InsertOpts{ + _, err := insertParamsFromConfigArgsAndOptions(archetype, config, &customInsertOptsJobArgs{}, &InsertOpts{ Tags: []string{strings.Repeat("h", 256)}, }) require.EqualError(t, err, "tags should be a maximum of 255 characters long") } { - _, _, err := insertParamsFromConfigArgsAndOptions(archetype, config, &customInsertOptsJobArgs{}, &InsertOpts{ + _, err := insertParamsFromConfigArgsAndOptions(archetype, config, &customInsertOptsJobArgs{}, &InsertOpts{ Tags: []string{"tag,with,comma"}, }) require.EqualError(t, err, "tags should match regex "+tagRE.String()) @@ -5168,25 +5163,37 @@ func TestInsertParamsFromJobArgsAndOptions(t *testing.T) { t.Run("UniqueOpts", 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, - ByState: []rivertype.JobState{rivertype.JobStateAvailable, rivertype.JobStateCompleted}, + ByArgs: true, + ByPeriod: 10 * time.Second, + ByQueue: true, + ByState: []rivertype.JobState{rivertype.JobStateAvailable, rivertype.JobStateCompleted}, + ExcludeKind: true, } - _, internalUniqueOpts, err := insertParamsFromConfigArgsAndOptions(archetype, config, noOpArgs{}, &InsertOpts{UniqueOpts: uniqueOpts}) + params, err := insertParamsFromConfigArgsAndOptions(archetype, config, noOpArgs{}, &InsertOpts{UniqueOpts: uniqueOpts}) 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) + internalUniqueOpts := &dbunique.UniqueOpts{ + ByArgs: true, + ByPeriod: 10 * time.Second, + ByQueue: true, + ByState: []rivertype.JobState{rivertype.JobStateAvailable, rivertype.JobStateCompleted}, + ExcludeKind: true, + } + + expectedKey := dbunique.UniqueKey(archetype.Time, internalUniqueOpts, params) + + require.Equal(t, expectedKey, params.UniqueKey) + require.Equal(t, internalUniqueOpts.StateBitmask(), params.UniqueStates) }) 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}) require.ErrorContains(t, err, "priority must be between 1 and 4") require.Nil(t, insertParams) }) @@ -5195,7 +5202,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) require.NoError(t, err) require.Equal(t, `{"timeout_value":3600000000000}`, string(insertParams.EncodedArgs)) }) @@ -5206,7 +5213,7 @@ 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, err := insertParamsFromConfigArgsAndOptions( archetype, config, noOpArgs{}, @@ -5397,7 +5404,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{} diff --git a/driver_test.go b/driver_test.go index 7280fc81..36ecca9a 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, - }, + _, 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(n)), + UniqueStates: 0xFB, }) require.NoError(b, err) } diff --git a/insert_opts.go b/insert_opts.go index 423c2cde..48b585c5 100644 --- a/insert_opts.go +++ b/insert_opts.go @@ -142,6 +142,11 @@ type UniqueOpts struct { // lock and performs a look up before insertion. For best performance, it's // recommended that the default set of states is used. 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. diff --git a/internal/dbunique/db_unique.go b/internal/dbunique/db_unique.go index 196d0e4b..2370afb5 100644 --- a/internal/dbunique/db_unique.go +++ b/internal/dbunique/db_unique.go @@ -1,15 +1,11 @@ package dbunique import ( - "context" "crypto/sha256" - "errors" - "fmt" "slices" "strings" "time" - "github.com/riverqueue/river/internal/util/hashutil" "github.com/riverqueue/river/riverdriver" "github.com/riverqueue/river/rivershared/baseservice" "github.com/riverqueue/river/rivershared/util/sliceutil" @@ -32,128 +28,64 @@ 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 -} - -type UniqueInserter struct { - baseservice.BaseService - AdvisoryLockPrefix int32 + o.ByState == nil && + !o.ExcludeKind } -func (i *UniqueInserter) JobInsert(ctx context.Context, exec riverdriver.Executor, params *riverdriver.JobInsertFastParams, uniqueOpts *UniqueOpts) (*rivertype.JobInsertResult, error) { - // With no unique options set, do a normal non-unique insert. - if uniqueOpts == nil || uniqueOpts.IsEmpty() { - return insertNonUnique(ctx, exec, params) - } - - // Build a unique key for use in either the `(kind, unique_key)` index or in - // an advisory lock prefix if we end up taking the slow path. - uniqueKey, doUniqueInsert := i.buildUniqueKey(params, uniqueOpts) - if !doUniqueInsert { - return insertNonUnique(ctx, exec, params) - } - - // Sort so we can more easily compare against default state list. - if uniqueOpts.ByState != nil { - slices.Sort(uniqueOpts.ByState) - } - - // Fast path: as long as uniqueness uses the default set of lifecycle states - // we can take the fast path wherein uniqueness is determined based on an - // upsert to a unique index on `(kind, unique_key)`. This works because when - // cancelling or discarding jobs the executor/completer will zero the job's - // `unique_key` field, taking it out of consideration for future inserts - // given the same unique opts. - if uniqueOpts.ByState == nil || slices.Compare(defaultUniqueStates, uniqueOpts.ByState) == 0 { - uniqueKeyHash := sha256.Sum256([]byte(uniqueKey)) - - insertRes, err := exec.JobInsertUnique(ctx, &riverdriver.JobInsertUniqueParams{ - JobInsertFastParams: params, - UniqueKey: uniqueKeyHash[:], - }) - if err != nil { - return nil, err - } - - return (*rivertype.JobInsertResult)(insertRes), nil - } - - // Slow path: open a subtransaction, take an advisory lock, check to see if - // a job with the given criteria exists, then either return an existing row - // or insert a new one. - - advisoryLockHash := hashutil.NewAdvisoryLockHash(i.AdvisoryLockPrefix) - advisoryLockHash.Write([]byte("unique_key")) - advisoryLockHash.Write([]byte("kind=" + params.Kind)) - advisoryLockHash.Write([]byte(uniqueKey)) - - getParams := i.buildGetParams(params, uniqueOpts) - - // Begin a subtransaction - subExec, err := exec.Begin(ctx) - if err != nil { - return nil, err - } - defer subExec.Rollback(ctx) - - // The wrapping transaction should maintain snapshot consistency even if we - // were to only have a SELECT + INSERT, but given that a conflict is - // possible, obtain an advisory lock based on the parameters of the unique - // job first, and have contending inserts wait for it. This is a synchronous - // lock so we rely on context timeout in case something goes wrong and it's - // blocking for too long. - if _, err := subExec.PGAdvisoryXactLock(ctx, advisoryLockHash.Key()); err != nil { - return nil, fmt.Errorf("error acquiring unique lock: %w", err) - } - - existing, err := subExec.JobGetByKindAndUniqueProperties(ctx, getParams) - if err != nil { - if !errors.Is(err, rivertype.ErrNotFound) { - return nil, fmt.Errorf("error getting unique job: %w", err) - } - } - - if existing != nil { - // Insert skipped; returns an existing row. - return &rivertype.JobInsertResult{Job: existing, UniqueSkippedAsDuplicate: true}, nil - } - - jobRow, err := subExec.JobInsertFast(ctx, params) - if err != nil { - return nil, err - } - - if err := subExec.Commit(ctx); err != nil { - return nil, err +func (o *UniqueOpts) StateBitmask() byte { + states := defaultUniqueStates + if len(o.ByState) > 0 { + states = o.ByState } + return UniqueStatesToBitmask(states) +} - return &rivertype.JobInsertResult{Job: jobRow}, nil +func UniqueKey(timeGen baseservice.TimeGenerator, uniqueOpts *UniqueOpts, params *riverdriver.JobInsertFastParams) []byte { + uniqueKeyString := buildUniqueKeyString(timeGen, uniqueOpts, params) + uniqueKeyHash := sha256.Sum256([]byte(uniqueKeyString)) + return uniqueKeyHash[:] } // 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 (i *UniqueInserter) buildUniqueKey(params *riverdriver.JobInsertFastParams, uniqueOpts *UniqueOpts) (string, bool) { +func buildUniqueKeyString(timeGen baseservice.TimeGenerator, uniqueOpts *UniqueOpts, params *riverdriver.JobInsertFastParams) string { var sb strings.Builder + if !uniqueOpts.ExcludeKind { + sb.WriteString("&kind=" + params.Kind) + } + if uniqueOpts.ByArgs { sb.WriteString("&args=") sb.Write(params.EncodedArgs) } if uniqueOpts.ByPeriod != time.Duration(0) { - lowerPeriodBound := i.Time.NowUTC().Truncate(uniqueOpts.ByPeriod) + lowerPeriodBound := timeGen.NowUTC().Truncate(uniqueOpts.ByPeriod) sb.WriteString("&period=" + lowerPeriodBound.Format(time.RFC3339)) } @@ -161,66 +93,28 @@ func (i *UniqueInserter) buildUniqueKey(params *riverdriver.JobInsertFastParams, sb.WriteString("&queue=" + params.Queue) } - { - stateSet := defaultUniqueStatesStrings - if len(uniqueOpts.ByState) > 0 { - stateSet = sliceutil.Map(uniqueOpts.ByState, func(s rivertype.JobState) string { return string(s) }) - slices.Sort(stateSet) - } - - sb.WriteString("&state=" + strings.Join(stateSet, ",")) - - if !slices.Contains(stateSet, string(params.State)) { - return "", false - } + stateSet := defaultUniqueStatesStrings + if len(uniqueOpts.ByState) > 0 { + stateSet = sliceutil.Map(uniqueOpts.ByState, func(s rivertype.JobState) string { return string(s) }) + slices.Sort(stateSet) } - return sb.String(), true -} - -// Builds get parameters suitable for looking up a unique job on the slow unique -// insertion path. -func (i *UniqueInserter) buildGetParams(params *riverdriver.JobInsertFastParams, uniqueOpts *UniqueOpts) *riverdriver.JobGetByKindAndUniquePropertiesParams { - getParams := riverdriver.JobGetByKindAndUniquePropertiesParams{ - Kind: params.Kind, - } + sb.WriteString("&state=" + strings.Join(stateSet, ",")) - if uniqueOpts.ByArgs { - getParams.Args = params.EncodedArgs - getParams.ByArgs = true - } - - if uniqueOpts.ByPeriod != time.Duration(0) { - lowerPeriodBound := i.Time.NowUTC().Truncate(uniqueOpts.ByPeriod) - - getParams.ByCreatedAt = true - getParams.CreatedAtBegin = lowerPeriodBound - getParams.CreatedAtEnd = lowerPeriodBound.Add(uniqueOpts.ByPeriod) - } + return sb.String() +} - if uniqueOpts.ByQueue { - getParams.ByQueue = true - getParams.Queue = params.Queue - } +func UniqueStatesToBitmask(states []rivertype.JobState) byte { + var b byte - { - stateSet := defaultUniqueStatesStrings - if len(uniqueOpts.ByState) > 0 { - stateSet = sliceutil.Map(uniqueOpts.ByState, func(s rivertype.JobState) string { return string(s) }) + for _, state := range states { + bitIndex, exists := jobStateBitPositions[state] + if !exists { + continue // Ignore unknown states } - - getParams.ByState = true - getParams.State = stateSet + bitPosition := 7 - (bitIndex % 8) + b |= 1 << bitPosition } - return &getParams -} - -// Shared shortcut for inserting a row without uniqueness. -func insertNonUnique(ctx context.Context, exec riverdriver.Executor, params *riverdriver.JobInsertFastParams) (*rivertype.JobInsertResult, error) { - jobRow, err := exec.JobInsertFast(ctx, params) - if err != nil { - return nil, err - } - return &rivertype.JobInsertResult{Job: jobRow}, nil + return b } diff --git a/internal/dbunique/db_unique_test.go b/internal/dbunique/db_unique_test.go index 487b5940..bb51de22 100644 --- a/internal/dbunique/db_unique_test.go +++ b/internal/dbunique/db_unique_test.go @@ -1,25 +1,16 @@ package dbunique import ( - "context" - "fmt" - "runtime" + "crypto/sha256" "slices" - "sync" "testing" "time" - "github.com/jackc/pgx/v5" "github.com/stretchr/testify/require" "github.com/riverqueue/river/internal/rivercommon" - "github.com/riverqueue/river/internal/riverinternaltest" - "github.com/riverqueue/river/internal/util/dbutil" "github.com/riverqueue/river/riverdriver" - "github.com/riverqueue/river/riverdriver/riverpgxv5" - "github.com/riverqueue/river/rivershared/baseservice" "github.com/riverqueue/river/rivershared/riversharedtest" - "github.com/riverqueue/river/rivershared/util/ptrutil" "github.com/riverqueue/river/rivertype" ) @@ -47,691 +38,235 @@ func TestDefaultUniqueStatesSorted(t *testing.T) { require.Equal(t, states, defaultUniqueStates, "Default unique states should be sorted") } -func TestUniqueInserter_JobInsert(t *testing.T) { - t.Parallel() - - ctx := context.Background() - - type testBundle struct { - baselineTime time.Time // baseline time frozen at now when setup is called - driver riverdriver.Driver[pgx.Tx] - exec riverdriver.Executor - tx pgx.Tx +func TestUniqueKey(t *testing.T) { + // Fixed time for deterministic tests + fixedTime := time.Date(2024, 9, 14, 12, 0, 0, 0, time.UTC) + // timeGen := &MockTimeGenerator{FixedTime: fixedTime} + stubSvc := &riversharedtest.TimeStub{} + stubSvc.StubNowUTC(fixedTime) + + // Define test parameters + params := &riverdriver.JobInsertFastParams{ + Kind: "email", + EncodedArgs: []byte(`{"to":"user@example.com","subject":"Test Email"}`), + Queue: "default", } - setup := func(t *testing.T) (*UniqueInserter, *testBundle) { - t.Helper() - - var ( - driver = riverpgxv5.New(nil) - tx = riverinternaltest.TestTx(ctx, t) - ) - - bundle := &testBundle{ - driver: driver, - exec: driver.UnwrapExecutor(tx), - tx: tx, - } - - inserter := baseservice.Init(riversharedtest.BaseServiceArchetype(t), &UniqueInserter{}) - - // Tests that use ByPeriod below can be sensitive to intermittency if - // the tests run at say 14:59:59.998, then it's possible to accidentally - // cross a period threshold, even if very unlikely. So here, seed mostly - // the current time, but make sure it's nicened up a little to be - // roughly in the middle of the hour and well clear of any period - // boundaries. - bundle.baselineTime = inserter.Time.StubNowUTC(time.Now().UTC().Truncate(1 * time.Hour).Add(37*time.Minute + 23*time.Second + 123*time.Millisecond)) - - return inserter, bundle + shasum := func(s string) []byte { + value := sha256.Sum256([]byte(s)) + return value[:] } - t.Run("Success", func(t *testing.T) { - t.Parallel() - - inserter, bundle := setup(t) - - insertParams := makeInsertParams(&bundle.baselineTime) - res, err := inserter.JobInsert(ctx, bundle.exec, insertParams, nil) - require.NoError(t, err) - - // Sanity check, following assertion depends on this: - require.Nil(t, insertParams.ScheduledAt) - - require.Positive(t, res.Job.ID, "expected job ID to be set, got %d", res.Job.ID) - require.JSONEq(t, string(insertParams.EncodedArgs), string(res.Job.EncodedArgs)) - require.Equal(t, 0, res.Job.Attempt) - require.Nil(t, res.Job.AttemptedAt) - require.Empty(t, res.Job.AttemptedBy) - require.Equal(t, bundle.baselineTime.Truncate(1*time.Microsecond), res.Job.CreatedAt) - require.Empty(t, res.Job.Errors) - require.Nil(t, res.Job.FinalizedAt) - require.Equal(t, insertParams.Kind, res.Job.Kind) - require.Equal(t, insertParams.MaxAttempts, res.Job.MaxAttempts) - require.Equal(t, insertParams.Metadata, res.Job.Metadata) - require.Equal(t, insertParams.Priority, res.Job.Priority) - require.Equal(t, insertParams.Queue, res.Job.Queue) - require.Equal(t, rivertype.JobStateAvailable, res.Job.State) - require.WithinDuration(t, time.Now(), res.Job.ScheduledAt, 2*time.Second) - require.Empty(t, res.Job.Tags) - }) - - t.Run("InsertAndFetch", func(t *testing.T) { - t.Parallel() - - inserter, bundle := setup(t) - - const maxJobsToFetch = 8 - - res, err := inserter.JobInsert(ctx, bundle.exec, makeInsertParams(&bundle.baselineTime), nil) - require.NoError(t, err) - require.NotEqual(t, 0, res.Job.ID, "expected job ID to be set, got %d", res.Job.ID) - require.WithinDuration(t, time.Now(), res.Job.ScheduledAt, 1*time.Second) - - jobs, err := bundle.exec.JobGetAvailable(ctx, &riverdriver.JobGetAvailableParams{ - AttemptedBy: "test-id", - Max: maxJobsToFetch, - Queue: rivercommon.QueueDefault, - }) - require.NoError(t, err) - require.Len(t, jobs, 1, - "inserted 1 job but fetched %d jobs:\n%+v", len(jobs), jobs) - require.Equal(t, rivertype.JobStateRunning, jobs[0].State, - "expected selected job to be in running state, got %q", jobs[0].State) - - for i := 1; i < 10; i++ { - _, err := inserter.JobInsert(ctx, bundle.exec, makeInsertParams(&bundle.baselineTime), nil) - require.NoError(t, err) - } - - jobs, err = bundle.exec.JobGetAvailable(ctx, &riverdriver.JobGetAvailableParams{ - AttemptedBy: "test-id", - Max: maxJobsToFetch, - Queue: rivercommon.QueueDefault, - }) - require.NoError(t, err) - require.Len(t, jobs, maxJobsToFetch, - "inserted 9 more jobs and expected to fetch max of %d jobs but fetched %d jobs:\n%+v", maxJobsToFetch, len(jobs), jobs) - for _, j := range jobs { - require.Equal(t, rivertype.JobStateRunning, j.State, - "expected selected job to be in running state, got %q", j.State) - } - - jobs, err = bundle.exec.JobGetAvailable(ctx, &riverdriver.JobGetAvailableParams{ - AttemptedBy: "test-id", - Max: maxJobsToFetch, - Queue: rivercommon.QueueDefault, - }) - require.NoError(t, err) - require.Len(t, jobs, 1, - "expected to fetch 1 remaining job but fetched %d jobs:\n%+v", len(jobs), jobs) - }) - - t.Run("UniqueJobByArgsFastPath", func(t *testing.T) { - t.Parallel() - - inserter, bundle := setup(t) - - insertParams := makeInsertParams(&bundle.baselineTime) - uniqueOpts := &UniqueOpts{ - ByArgs: true, - } - - res0, err := inserter.JobInsert(ctx, bundle.exec, insertParams, uniqueOpts) - require.NoError(t, err) - require.NotNil(t, res0.Job.UniqueKey) - require.False(t, res0.UniqueSkippedAsDuplicate) - - // Insert a second job with the same args, but expect that the same job - // ID to come back because we're still within its unique parameters. - res1, err := inserter.JobInsert(ctx, bundle.exec, insertParams, uniqueOpts) - require.NoError(t, err) - require.Equal(t, res0.Job.ID, res1.Job.ID) - require.True(t, res1.UniqueSkippedAsDuplicate) - - insertParams.EncodedArgs = []byte(`{"key":"different"}`) - - // Same operation again, except that because we've modified the unique - // dimension, another job is allowed to be queued, so the new ID is - // not the same. - res2, err := inserter.JobInsert(ctx, bundle.exec, insertParams, uniqueOpts) - require.NoError(t, err) - require.NotEqual(t, res0.Job.ID, res2.Job.ID) - require.False(t, res2.UniqueSkippedAsDuplicate) - }) - - t.Run("UniqueJobByArgsSlowPath", func(t *testing.T) { - t.Parallel() - - inserter, bundle := setup(t) - - insertParams := makeInsertParams(&bundle.baselineTime) - uniqueOpts := &UniqueOpts{ - ByArgs: true, - ByState: []rivertype.JobState{rivertype.JobStateAvailable, rivertype.JobStateCancelled}, // use of non-standard states triggers slow path - } - - res0, err := inserter.JobInsert(ctx, bundle.exec, insertParams, uniqueOpts) - require.NoError(t, err) - require.Nil(t, res0.Job.UniqueKey) - require.False(t, res0.UniqueSkippedAsDuplicate) - - // Insert a second job with the same args, but expect that the same job - // ID to come back because we're still within its unique parameters. - res1, err := inserter.JobInsert(ctx, bundle.exec, insertParams, uniqueOpts) - require.NoError(t, err) - require.Equal(t, res0.Job.ID, res1.Job.ID) - require.True(t, res1.UniqueSkippedAsDuplicate) - - insertParams.EncodedArgs = []byte(`{"key":"different"}`) - - // Same operation again, except that because we've modified the unique - // dimension, another job is allowed to be queued, so the new ID is - // not the same. - res2, err := inserter.JobInsert(ctx, bundle.exec, insertParams, uniqueOpts) - require.NoError(t, err) - require.NotEqual(t, res0.Job.ID, res2.Job.ID) - require.False(t, res2.UniqueSkippedAsDuplicate) - }) - - t.Run("UniqueJobByPeriodFastPath", func(t *testing.T) { - t.Parallel() - - inserter, bundle := setup(t) - - insertParams := makeInsertParams(&bundle.baselineTime) - uniqueOpts := &UniqueOpts{ - ByPeriod: 15 * time.Minute, - } - - res0, err := inserter.JobInsert(ctx, bundle.exec, insertParams, uniqueOpts) - require.NoError(t, err) - require.NotNil(t, res0.Job.UniqueKey) - require.False(t, res0.UniqueSkippedAsDuplicate) - - // Insert a second job with the same args, but expect that the same job - // ID to come back because we're still within its unique parameters. - res1, err := inserter.JobInsert(ctx, bundle.exec, insertParams, uniqueOpts) - require.NoError(t, err) - require.Equal(t, res0.Job.ID, res1.Job.ID) - require.True(t, res1.UniqueSkippedAsDuplicate) - - inserter.Time.StubNowUTC(bundle.baselineTime.Add(uniqueOpts.ByPeriod).Add(1 * time.Second)) - - // Same operation again, except that because we've advanced time passed - // the period within unique bounds, another job is allowed to be queued, - // so the new ID is not the same. - res2, err := inserter.JobInsert(ctx, bundle.exec, insertParams, uniqueOpts) - require.NoError(t, err) - require.NotEqual(t, res0.Job.ID, res2.Job.ID) - require.False(t, res2.UniqueSkippedAsDuplicate) - }) - - t.Run("UniqueJobByPeriodSlowPath", func(t *testing.T) { - t.Parallel() - - inserter, bundle := setup(t) - - insertParams := makeInsertParams(&bundle.baselineTime) - uniqueOpts := &UniqueOpts{ - ByPeriod: 15 * time.Minute, - ByState: []rivertype.JobState{rivertype.JobStateAvailable, rivertype.JobStateCancelled}, // use of non-standard states triggers slow path - } - - res0, err := inserter.JobInsert(ctx, bundle.exec, insertParams, uniqueOpts) - require.Nil(t, res0.Job.UniqueKey) - require.NoError(t, err) - require.False(t, res0.UniqueSkippedAsDuplicate) - - // Insert a second job with the same args, but expect that the same job - // ID to come back because we're still within its unique parameters. - res1, err := inserter.JobInsert(ctx, bundle.exec, insertParams, uniqueOpts) - require.NoError(t, err) - require.Equal(t, res0.Job.ID, res1.Job.ID) - require.True(t, res1.UniqueSkippedAsDuplicate) - - inserter.Time.StubNowUTC(bundle.baselineTime.Add(uniqueOpts.ByPeriod).Add(1 * time.Second)) - - // Same operation again, except that because we've advanced time passed - // the period within unique bounds, another job is allowed to be queued, - // so the new ID is not the same. - res2, err := inserter.JobInsert(ctx, bundle.exec, insertParams, uniqueOpts) - require.NoError(t, err) - require.NotEqual(t, res0.Job.ID, res2.Job.ID) - require.False(t, res2.UniqueSkippedAsDuplicate) - }) - - t.Run("UniqueJobByQueueFastPath", func(t *testing.T) { - t.Parallel() - - inserter, bundle := setup(t) - - insertParams := makeInsertParams(&bundle.baselineTime) - uniqueOpts := &UniqueOpts{ - ByQueue: true, - } - - res0, err := inserter.JobInsert(ctx, bundle.exec, insertParams, uniqueOpts) - require.NoError(t, err) - require.NotNil(t, res0.Job.UniqueKey) - require.False(t, res0.UniqueSkippedAsDuplicate) - - // Insert a second job with the same args, but expect that the same job - // ID to come back because we're still within its unique parameters. - res1, err := inserter.JobInsert(ctx, bundle.exec, insertParams, uniqueOpts) - require.NoError(t, err) - require.Equal(t, res0.Job.ID, res1.Job.ID) - require.True(t, res1.UniqueSkippedAsDuplicate) - - insertParams.Queue = queueAlternate - - // Same operation again, except that because we've modified the unique - // dimension, another job is allowed to be queued, so the new ID is - // not the same. - res2, err := inserter.JobInsert(ctx, bundle.exec, insertParams, uniqueOpts) - require.NoError(t, err) - require.NotEqual(t, res0.Job.ID, res2.Job.ID) - require.False(t, res2.UniqueSkippedAsDuplicate) - }) - - t.Run("UniqueJobByQueueSlowPath", func(t *testing.T) { - t.Parallel() - - inserter, bundle := setup(t) - - insertParams := makeInsertParams(&bundle.baselineTime) - uniqueOpts := &UniqueOpts{ - ByQueue: true, - ByState: []rivertype.JobState{rivertype.JobStateAvailable, rivertype.JobStateCancelled}, // use of non-standard states triggers slow path - } - - res0, err := inserter.JobInsert(ctx, bundle.exec, insertParams, uniqueOpts) - require.NoError(t, err) - require.Nil(t, res0.Job.UniqueKey) - require.False(t, res0.UniqueSkippedAsDuplicate) - - // Insert a second job with the same args, but expect that the same job - // ID to come back because we're still within its unique parameters. - res1, err := inserter.JobInsert(ctx, bundle.exec, insertParams, uniqueOpts) - require.NoError(t, err) - require.Equal(t, res0.Job.ID, res1.Job.ID) - require.True(t, res1.UniqueSkippedAsDuplicate) - - insertParams.Queue = queueAlternate - - // Same operation again, except that because we've modified the unique - // dimension, another job is allowed to be queued, so the new ID is - // not the same. - res2, err := inserter.JobInsert(ctx, bundle.exec, insertParams, uniqueOpts) - require.NoError(t, err) - require.NotEqual(t, res0.Job.ID, res2.Job.ID) - require.False(t, res2.UniqueSkippedAsDuplicate) - }) - - // Unlike other unique options, state gets a default set when it's not - // supplied. This test case checks that the default is working as expected. - t.Run("UniqueJobByDefaultStateFastPath", func(t *testing.T) { - t.Parallel() - - inserter, bundle := setup(t) - - insertParams := makeInsertParams(&bundle.baselineTime) - uniqueOpts := &UniqueOpts{ - ByQueue: true, - } - - res0, err := inserter.JobInsert(ctx, bundle.exec, insertParams, uniqueOpts) - require.NoError(t, err) - require.NotNil(t, res0.Job.UniqueKey) // fast path because states are a subset of defaults - require.False(t, res0.UniqueSkippedAsDuplicate) - - // Insert a second job with the same args, but expect that the same job - // ID to come back because we're still within its unique parameters. - res1, err := inserter.JobInsert(ctx, bundle.exec, insertParams, uniqueOpts) - require.NoError(t, err) - require.Equal(t, res0.Job.ID, res1.Job.ID) - require.True(t, res1.UniqueSkippedAsDuplicate) - - // Test all the other default unique states (see `defaultUniqueStates`) - // to make sure that in each case an inserted job still counts as a - // duplicate. The only state we don't test is `available` because that's - // already been done above. - for _, defaultState := range []rivertype.JobState{ - rivertype.JobStateCompleted, - rivertype.JobStatePending, - rivertype.JobStateRunning, - rivertype.JobStateRetryable, - rivertype.JobStateScheduled, - } { - var finalizedAt *time.Time - if defaultState == rivertype.JobStateCompleted { - finalizedAt = ptrutil.Ptr(bundle.baselineTime) - } - - _, err = bundle.exec.JobUpdate(ctx, &riverdriver.JobUpdateParams{ - ID: res0.Job.ID, - FinalizedAtDoUpdate: true, - FinalizedAt: finalizedAt, - StateDoUpdate: true, - State: defaultState, - }) - require.NoError(t, err) - - // Still counts as a duplicate. - res1, err := inserter.JobInsert(ctx, bundle.exec, insertParams, uniqueOpts) - require.NoError(t, err) - require.Equal(t, res0.Job.ID, res1.Job.ID) - require.True(t, res1.UniqueSkippedAsDuplicate) - } - - _, err = bundle.exec.JobUpdate(ctx, &riverdriver.JobUpdateParams{ - ID: res0.Job.ID, - FinalizedAtDoUpdate: true, - FinalizedAt: ptrutil.Ptr(bundle.baselineTime), - StateDoUpdate: true, - State: rivertype.JobStateDiscarded, - UniqueKeyDoUpdate: true, // `unique_key` is normally NULLed by the client or completer - UniqueKey: nil, - }) - require.NoError(t, err) - - // Uniqueness includes a default set of states, so by moving the - // original job to "discarded", we're now allowed to insert a new job - // again, despite not having explicitly set the `ByState` option. - res2, err := inserter.JobInsert(ctx, bundle.exec, insertParams, uniqueOpts) - require.NoError(t, err) - require.NotEqual(t, res0.Job.ID, res2.Job.ID) - require.False(t, res2.UniqueSkippedAsDuplicate) - }) - - t.Run("UniqueJobByStateSlowPath", func(t *testing.T) { - t.Parallel() - - inserter, bundle := setup(t) - - insertParams := makeInsertParams(&bundle.baselineTime) - uniqueOpts := &UniqueOpts{ - ByState: []rivertype.JobState{rivertype.JobStateAvailable, rivertype.JobStateCancelled}, - } - - res0, err := inserter.JobInsert(ctx, bundle.exec, insertParams, uniqueOpts) - require.NoError(t, err) - require.Nil(t, res0.Job.UniqueKey) // slow path because states are *not* a subset of defaults - require.False(t, res0.UniqueSkippedAsDuplicate) - - // Insert a second job with the same args, but expect that the same job - // ID to come back because we're still within its unique parameters. - res1, err := inserter.JobInsert(ctx, bundle.exec, insertParams, uniqueOpts) - require.NoError(t, err) - require.Equal(t, res0.Job.ID, res1.Job.ID) - require.True(t, res1.UniqueSkippedAsDuplicate) - - // A new job is allowed if we're inserting the job with a state that's - // not included in the unique state set. + tests := []struct { + name string + uniqueOpts *UniqueOpts + params *riverdriver.JobInsertFastParams + expectedKey []byte + }{ { - insertParams := *insertParams // dup - insertParams.State = rivertype.JobStateRunning - - res2, err := inserter.JobInsert(ctx, bundle.exec, &insertParams, uniqueOpts) - require.NoError(t, err) - require.NotEqual(t, res0.Job.ID, res2.Job.ID) - require.False(t, res2.UniqueSkippedAsDuplicate) - } - - // A new job is also allowed if the state of the originally inserted job - // changes to one that's not included in the unique state set. + name: "DefaultOptions", + uniqueOpts: &UniqueOpts{}, + params: params, + expectedKey: shasum( + "&kind=email&state=Available,Completed,Pending,Retryable,Running,Scheduled", + ), + }, { - _, err := bundle.exec.JobUpdate(ctx, &riverdriver.JobUpdateParams{ - ID: res0.Job.ID, - FinalizedAtDoUpdate: true, - FinalizedAt: ptrutil.Ptr(bundle.baselineTime), - StateDoUpdate: true, - State: rivertype.JobStateCompleted, - }) - require.NoError(t, err) - - res2, err := inserter.JobInsert(ctx, bundle.exec, insertParams, uniqueOpts) - require.NoError(t, err) - require.NotEqual(t, res0.Job.ID, res2.Job.ID) - require.False(t, res2.UniqueSkippedAsDuplicate) - } - }) - - t.Run("UniqueJobAllOptions", func(t *testing.T) { - t.Parallel() - - inserter, bundle := setup(t) - - insertParams := makeInsertParams(&bundle.baselineTime) - uniqueOpts := &UniqueOpts{ - ByArgs: true, - ByPeriod: 15 * time.Minute, - ByQueue: true, - ByState: []rivertype.JobState{rivertype.JobStateAvailable, rivertype.JobStateRunning}, - } - - res0, err := inserter.JobInsert(ctx, bundle.exec, insertParams, uniqueOpts) - require.NoError(t, err) - require.False(t, res0.UniqueSkippedAsDuplicate) - - // Insert a second job with the same args, but expect that the same job - // ID to come back because we're still within its unique parameters. - res1, err := inserter.JobInsert(ctx, bundle.exec, insertParams, uniqueOpts) - require.NoError(t, err) - require.Equal(t, res0.Job.ID, res1.Job.ID) - require.True(t, res1.UniqueSkippedAsDuplicate) - - // With args modified + name: "ExcludeKind", + uniqueOpts: &UniqueOpts{ + ExcludeKind: true, + }, + params: params, + expectedKey: shasum( + "&state=Available,Completed,Pending,Retryable,Running,Scheduled", + ), + }, { - insertParams := *insertParams // dup - insertParams.EncodedArgs = []byte(`{"key":"different"}`) - - // New job because a unique dimension has changed. - res2, err := inserter.JobInsert(ctx, bundle.exec, &insertParams, uniqueOpts) - require.NoError(t, err) - require.NotEqual(t, res0.Job.ID, res2.Job.ID) - require.False(t, res2.UniqueSkippedAsDuplicate) - } - - // With period modified + name: "ByArgs", + uniqueOpts: &UniqueOpts{ + ByArgs: true, + }, + params: params, + expectedKey: shasum( + "&kind=email&args={\"to\":\"user@example.com\",\"subject\":\"Test Email\"}&state=Available,Completed,Pending,Retryable,Running,Scheduled", + ), + }, { - insertParams := *insertParams // dup - inserter.Time.StubNowUTC(bundle.baselineTime.Add(uniqueOpts.ByPeriod).Add(1 * time.Second)) - - // New job because a unique dimension has changed. - res2, err := inserter.JobInsert(ctx, bundle.exec, &insertParams, uniqueOpts) - require.NoError(t, err) - require.NotEqual(t, res0.Job.ID, res2.Job.ID) - require.False(t, res2.UniqueSkippedAsDuplicate) - - // Make sure to change timeNow back - inserter.Time.StubNowUTC(bundle.baselineTime) - } - - // With queue modified + name: "ByPeriod", + uniqueOpts: &UniqueOpts{ + ByPeriod: 2 * time.Hour, + }, + params: params, + expectedKey: shasum( + "&kind=email&period=2024-09-14T12:00:00Z&state=Available,Completed,Pending,Retryable,Running,Scheduled", + ), + }, { - insertParams := *insertParams // dup - insertParams.Queue = queueAlternate - - // New job because a unique dimension has changed. - res2, err := inserter.JobInsert(ctx, bundle.exec, &insertParams, uniqueOpts) - require.NoError(t, err) - require.NotEqual(t, res0.Job.ID, res2.Job.ID) - require.False(t, res2.UniqueSkippedAsDuplicate) - } - - // With state modified + name: "ByQueue", + uniqueOpts: &UniqueOpts{ + ByQueue: true, + }, + params: params, + expectedKey: shasum("&kind=email&queue=default&state=Available,Completed,Pending,Retryable,Running,Scheduled"), + }, { - insertParams := *insertParams // dup - insertParams.State = rivertype.JobStatePending - - // New job because a unique dimension has changed. - res2, err := inserter.JobInsert(ctx, bundle.exec, &insertParams, uniqueOpts) - require.NoError(t, err) - require.NotEqual(t, res0.Job.ID, res2.Job.ID) - require.False(t, res2.UniqueSkippedAsDuplicate) - } - }) - - t.Run("UniqueJobContention", func(t *testing.T) { - t.Parallel() - - inserter, bundle := setup(t) - require.NoError(t, bundle.tx.Rollback(ctx)) - bundle.driver = riverpgxv5.New(riverinternaltest.TestDB(ctx, t)) - bundle.exec = bundle.driver.GetExecutor() - - insertParams := makeInsertParams(&bundle.baselineTime) - uniqueOpts := &UniqueOpts{ - ByPeriod: 15 * time.Minute, - } + name: "ByState", + uniqueOpts: &UniqueOpts{ + ByState: []rivertype.JobState{ + rivertype.JobStateCancelled, + rivertype.JobStateDiscarded, + }, + }, + params: params, + expectedKey: shasum("&kind=email&state=Cancelled,Discarded"), + }, + { + name: "CombinationOptions", + uniqueOpts: &UniqueOpts{ + ByArgs: true, + ByPeriod: 1 * time.Hour, + ByQueue: true, + ByState: []rivertype.JobState{rivertype.JobStateRunning, rivertype.JobStatePending}, + ExcludeKind: false, + }, + params: params, + expectedKey: shasum( + "&kind=email&args={\"to\":\"user@example.com\",\"subject\":\"Test Email\"}&period=2024-09-14T12:00:00Z&queue=default&state=Pending,Running", + ), + }, + { + name: "EmptyUniqueOpts", + uniqueOpts: &UniqueOpts{ + ByArgs: false, + ByPeriod: 0, + ByQueue: false, + ByState: nil, + }, + params: params, + expectedKey: shasum("&kind=email&state=Available,Completed,Pending,Retryable,Running,Scheduled"), + }, + { + name: "EmptyEncodedArgs", + uniqueOpts: &UniqueOpts{ + ByArgs: true, + }, + params: &riverdriver.JobInsertFastParams{ + Kind: "email", + EncodedArgs: []byte{}, + Queue: "default", + }, + expectedKey: shasum("&kind=email&args=&state=Available,Completed,Pending,Retryable,Running,Scheduled"), + }, + { + name: "SpecialCharactersInKindAndQueue", + uniqueOpts: &UniqueOpts{ + ByQueue: true, + }, + params: &riverdriver.JobInsertFastParams{ + Kind: "email¬ification", + EncodedArgs: []byte(`{"to":"user@example.com","subject":"Test Email"}`), + Queue: "default/queue", + }, + expectedKey: shasum( + "&kind=email¬ification&queue=default/queue&state=Available,Completed,Pending,Retryable,Running,Scheduled", + ), + }, + { + name: "UnknownJobState", + uniqueOpts: &UniqueOpts{ + ByState: []rivertype.JobState{ + "UnknownState", + rivertype.JobStateRunning, + }, + }, + params: params, + expectedKey: shasum("&kind=email&state=Running"), + }, + } - var ( - numContendingJobs = runtime.NumCPU() // max allowed test manager connections - insertedJobs = make([]*rivertype.JobRow, numContendingJobs) - insertedJobsMu sync.Mutex - wg sync.WaitGroup - ) + for _, tt := range tests { + tt := tt // capture range variable + t.Run(tt.name, func(t *testing.T) { + uniqueKey := UniqueKey(stubSvc, tt.uniqueOpts, tt.params) - for i := 0; i < numContendingJobs; i++ { - jobNum := i - wg.Add(1) + // Compare the generated unique key with the expected hash + require.Equal(t, tt.expectedKey[:], uniqueKey, "UniqueKey hash does not match expected value") + }) + } - 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) + // Additional tests to ensure uniqueness + t.Run("DifferentUniqueOptsProduceDifferentKeys", func(t *testing.T) { + opts1 := &UniqueOpts{ByArgs: true} + opts2 := &UniqueOpts{ByQueue: true} - insertedJobsMu.Lock() - insertedJobs[jobNum] = res.Job - insertedJobsMu.Unlock() + key1 := UniqueKey(stubSvc, opts1, params) + key2 := UniqueKey(stubSvc, opts2, params) - return struct{}{}, nil - }) - require.NoError(t, err) + require.NotEqual(t, key1, key2, "UniqueKeys should differ for different UniqueOpts") + }) - wg.Done() - }() + t.Run("SameInputsProduceSameKey", func(t *testing.T) { + opts := &UniqueOpts{ + ByArgs: true, + ByPeriod: 30 * time.Minute, + ByQueue: true, + ByState: []rivertype.JobState{rivertype.JobStateRunning}, + ExcludeKind: false, } + key1 := UniqueKey(stubSvc, opts, params) + key2 := UniqueKey(stubSvc, opts, params) - wg.Wait() - - firstJobID := insertedJobs[0].ID - for i := 1; i < numContendingJobs; i++ { - require.Equal(t, firstJobID, insertedJobs[i].ID) - } + require.Equal(t, key1, key2, "UniqueKeys should be identical for the same inputs") }) } -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) - } - }) } + +// TODO(bgentry): tests for new functions/methods in dbunique diff --git a/internal/maintenance/periodic_job_enqueuer.go b/internal/maintenance/periodic_job_enqueuer.go index 831a642d..26e71eb3 100644 --- a/internal/maintenance/periodic_job_enqueuer.go +++ b/internal/maintenance/periodic_job_enqueuer.go @@ -39,7 +39,7 @@ func (ts *PeriodicJobEnqueuerTestSignals) Init() { // river.PeriodicJobArgs, but needs a separate type because the enqueuer is in a // subpackage. type PeriodicJob struct { - ConstructorFunc func() (*riverdriver.JobInsertFastParams, *dbunique.UniqueOpts, error) + ConstructorFunc func() (*riverdriver.JobInsertFastParams, error) RunOnStart bool ScheduleFunc func(time.Time) time.Time @@ -58,8 +58,6 @@ func (j *PeriodicJob) mustValidate() *PeriodicJob { } type PeriodicJobEnqueuerConfig struct { - AdvisoryLockPrefix int32 - // NotifyInsert is a function to call to emit notifications for queues // where jobs were scheduled. NotifyInsert NotifyInsertFunc @@ -87,7 +85,6 @@ type PeriodicJobEnqueuer struct { nextHandle rivertype.PeriodicJobHandle periodicJobs map[rivertype.PeriodicJobHandle]*PeriodicJob recalculateNextRun chan struct{} - uniqueInserter *dbunique.UniqueInserter } func NewPeriodicJobEnqueuer(archetype *baseservice.Archetype, config *PeriodicJobEnqueuerConfig, exec riverdriver.Executor) *PeriodicJobEnqueuer { @@ -105,16 +102,14 @@ func NewPeriodicJobEnqueuer(archetype *baseservice.Archetype, config *PeriodicJo svc := baseservice.Init(archetype, &PeriodicJobEnqueuer{ Config: (&PeriodicJobEnqueuerConfig{ - AdvisoryLockPrefix: config.AdvisoryLockPrefix, - NotifyInsert: config.NotifyInsert, - PeriodicJobs: config.PeriodicJobs, + NotifyInsert: config.NotifyInsert, + PeriodicJobs: config.PeriodicJobs, }).mustValidate(), exec: exec, nextHandle: nextHandle, periodicJobs: periodicJobs, recalculateNextRun: make(chan struct{}, 1), - uniqueInserter: baseservice.Init(archetype, &dbunique.UniqueInserter{AdvisoryLockPrefix: config.AdvisoryLockPrefix}), }) return svc @@ -231,10 +226,8 @@ func (s *PeriodicJobEnqueuer) Start(ctx context.Context) error { defer s.mu.RUnlock() var ( - insertParamsMany []*riverdriver.JobInsertFastParams - insertParamsUnique []*insertParamsAndUniqueOpts - - now = s.Time.NowUTC() + insertParamsMany []*riverdriver.JobInsertFastParams + now = s.Time.NowUTC() ) // Handle periodic jobs in sorted order so we can correctly account @@ -257,19 +250,15 @@ func (s *PeriodicJobEnqueuer) Start(ctx context.Context) error { continue } - if insertParams, uniqueOpts, ok := s.insertParamsFromConstructor(ctx, periodicJob.ConstructorFunc, now); ok { - if !uniqueOpts.IsEmpty() { - insertParamsUnique = append(insertParamsUnique, &insertParamsAndUniqueOpts{insertParams, uniqueOpts}) - } else { - insertParamsMany = append(insertParamsMany, insertParams) - } + if insertParams, ok := s.insertParamsFromConstructor(ctx, periodicJob.ConstructorFunc, now); ok { + insertParamsMany = append(insertParamsMany, insertParams) } } - s.insertBatch(ctx, insertParamsMany, insertParamsUnique) + s.insertBatch(ctx, insertParamsMany) - if len(insertParamsMany) > 0 || len(insertParamsUnique) > 0 { - s.Logger.DebugContext(ctx, s.Name+": Inserted RunOnStart jobs", "num_jobs", len(insertParamsMany)+len(insertParamsUnique)) + if len(insertParamsMany) > 0 { + s.Logger.DebugContext(ctx, s.Name+": Inserted RunOnStart jobs", "num_jobs", len(insertParamsMany)) } } @@ -283,10 +272,7 @@ func (s *PeriodicJobEnqueuer) Start(ctx context.Context) error { for { select { case <-timerUntilNextRun.C: - var ( - insertParamsMany []*riverdriver.JobInsertFastParams - insertParamsUnique []*insertParamsAndUniqueOpts - ) + var insertParamsMany []*riverdriver.JobInsertFastParams now := s.Time.NowUTC() @@ -304,12 +290,8 @@ func (s *PeriodicJobEnqueuer) Start(ctx context.Context) error { continue } - if insertParams, uniqueOpts, ok := s.insertParamsFromConstructor(ctx, periodicJob.ConstructorFunc, periodicJob.nextRunAt); ok { - if !uniqueOpts.IsEmpty() { - insertParamsUnique = append(insertParamsUnique, &insertParamsAndUniqueOpts{insertParams, uniqueOpts}) - } else { - insertParamsMany = append(insertParamsMany, insertParams) - } + if insertParams, ok := s.insertParamsFromConstructor(ctx, periodicJob.ConstructorFunc, periodicJob.nextRunAt); ok { + insertParamsMany = append(insertParamsMany, insertParams) } // Although we may have inserted a new job a little @@ -320,7 +302,7 @@ func (s *PeriodicJobEnqueuer) Start(ctx context.Context) error { } }() - s.insertBatch(ctx, insertParamsMany, insertParamsUnique) + s.insertBatch(ctx, insertParamsMany) case <-s.recalculateNextRun: if !timerUntilNextRun.Stop() { @@ -350,8 +332,8 @@ func (s *PeriodicJobEnqueuer) Start(ctx context.Context) error { return nil } -func (s *PeriodicJobEnqueuer) insertBatch(ctx context.Context, insertParamsMany []*riverdriver.JobInsertFastParams, insertParamsUnique []*insertParamsAndUniqueOpts) { - if len(insertParamsMany) == 0 && len(insertParamsUnique) == 0 { +func (s *PeriodicJobEnqueuer) insertBatch(ctx context.Context, insertParamsMany []*riverdriver.JobInsertFastParams) { + if len(insertParamsMany) == 0 { return } @@ -362,33 +344,18 @@ func (s *PeriodicJobEnqueuer) insertBatch(ctx context.Context, insertParamsMany } defer tx.Rollback(ctx) - queues := make([]string, 0, len(insertParamsMany)+len(insertParamsUnique)) + queues := make([]string, 0, len(insertParamsMany)) 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) - } - } - - // Unique periodic jobs must be inserted one at a time because bulk insert - // doesn't respect uniqueness. Unique jobs are rare compared to non-unique, - // so we still maintain an insert many fast path above for programs that - // aren't inserting any unique jobs periodically (which we expect is most). - if len(insertParamsUnique) > 0 { - for _, params := range insertParamsUnique { - res, err := s.uniqueInserter.JobInsert(ctx, tx, params.InsertParams, params.UniqueOpts) - if err != nil { - s.Logger.ErrorContext(ctx, s.Name+": Error inserting unique periodic job", - "error", err.Error(), "kind", params.InsertParams.Kind) - continue - } - if !res.UniqueSkippedAsDuplicate { - queues = append(queues, params.InsertParams.Queue) + for _, result := range results { + if !result.UniqueSkippedAsDuplicate { + queues = append(queues, result.Job.Queue) } } } @@ -409,23 +376,23 @@ func (s *PeriodicJobEnqueuer) insertBatch(ctx context.Context, insertParamsMany s.TestSignals.InsertedJobs.Signal(struct{}{}) } -func (s *PeriodicJobEnqueuer) insertParamsFromConstructor(ctx context.Context, constructorFunc func() (*riverdriver.JobInsertFastParams, *dbunique.UniqueOpts, error), scheduledAt time.Time) (*riverdriver.JobInsertFastParams, *dbunique.UniqueOpts, bool) { - insertParams, uniqueOpts, err := constructorFunc() +func (s *PeriodicJobEnqueuer) insertParamsFromConstructor(ctx context.Context, constructorFunc func() (*riverdriver.JobInsertFastParams, error), scheduledAt time.Time) (*riverdriver.JobInsertFastParams, bool) { + insertParams, err := constructorFunc() if err != nil { if errors.Is(err, ErrNoJobToInsert) { s.Logger.InfoContext(ctx, s.Name+": nil returned from periodic job constructor, skipping") s.TestSignals.SkippedJob.Signal(struct{}{}) - return nil, nil, false + return nil, false } s.Logger.ErrorContext(ctx, s.Name+": Internal error generating periodic job", "error", err.Error()) - return nil, nil, false + return nil, false } if insertParams.ScheduledAt == nil { insertParams.ScheduledAt = &scheduledAt } - return insertParams, uniqueOpts, true + return insertParams, true } const periodicJobEnqueuerVeryLongDuration = 24 * time.Hour diff --git a/internal/maintenance/periodic_job_enqueuer_test.go b/internal/maintenance/periodic_job_enqueuer_test.go index 82f57fb4..0d4e9e24 100644 --- a/internal/maintenance/periodic_job_enqueuer_test.go +++ b/internal/maintenance/periodic_job_enqueuer_test.go @@ -33,20 +33,31 @@ func TestPeriodicJobEnqueuer(t *testing.T) { waitChan chan (struct{}) } - jobConstructorWithQueueFunc := func(name string, unique bool, queue string) func() (*riverdriver.JobInsertFastParams, *dbunique.UniqueOpts, error) { - return func() (*riverdriver.JobInsertFastParams, *dbunique.UniqueOpts, error) { - return &riverdriver.JobInsertFastParams{ + stubSvc := &riversharedtest.TimeStub{} + stubSvc.StubNowUTC(time.Now().UTC()) + + jobConstructorWithQueueFunc := func(name string, unique bool, queue string) func() (*riverdriver.JobInsertFastParams, error) { + return func() (*riverdriver.JobInsertFastParams, error) { + + params := &riverdriver.JobInsertFastParams{ 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} + params.UniqueKey = dbunique.UniqueKey(stubSvc, uniqueOpts, params) + params.UniqueStates = uniqueOpts.StateBitmask() + } + + return params, nil } } - jobConstructorFunc := func(name string, unique bool) func() (*riverdriver.JobInsertFastParams, *dbunique.UniqueOpts, error) { + jobConstructorFunc := func(name string, unique bool) func() (*riverdriver.JobInsertFastParams, error) { return jobConstructorWithQueueFunc(name, unique, rivercommon.QueueDefault) } @@ -236,8 +247,8 @@ func TestPeriodicJobEnqueuer(t *testing.T) { svc.AddMany([]*PeriodicJob{ // skip this insert when it returns nil: - {ScheduleFunc: periodicIntervalSchedule(time.Second), ConstructorFunc: func() (*riverdriver.JobInsertFastParams, *dbunique.UniqueOpts, error) { - return nil, nil, ErrNoJobToInsert + {ScheduleFunc: periodicIntervalSchedule(time.Second), ConstructorFunc: func() (*riverdriver.JobInsertFastParams, error) { + return nil, ErrNoJobToInsert }, RunOnStart: true}, }) diff --git a/internal/maintenance/queue_maintainer_test.go b/internal/maintenance/queue_maintainer_test.go index d8f94323..a68e91a8 100644 --- a/internal/maintenance/queue_maintainer_test.go +++ b/internal/maintenance/queue_maintainer_test.go @@ -8,7 +8,6 @@ import ( "github.com/robfig/cron/v3" "github.com/stretchr/testify/require" - "github.com/riverqueue/river/internal/dbunique" "github.com/riverqueue/river/internal/riverinternaltest" "github.com/riverqueue/river/internal/riverinternaltest/sharedtx" "github.com/riverqueue/river/riverdriver" @@ -108,8 +107,8 @@ func TestQueueMaintainer(t *testing.T) { NewPeriodicJobEnqueuer(archetype, &PeriodicJobEnqueuerConfig{ PeriodicJobs: []*PeriodicJob{ { - ConstructorFunc: func() (*riverdriver.JobInsertFastParams, *dbunique.UniqueOpts, error) { - return nil, nil, ErrNoJobToInsert + ConstructorFunc: func() (*riverdriver.JobInsertFastParams, error) { + return nil, ErrNoJobToInsert }, ScheduleFunc: cron.Every(15 * time.Minute).Next, }, diff --git a/internal/riverinternaltest/riverdrivertest/riverdrivertest.go b/internal/riverinternaltest/riverdrivertest/riverdrivertest.go index e8e3e76c..03506bf7 100644 --- a/internal/riverinternaltest/riverdrivertest/riverdrivertest.go +++ b/internal/riverinternaltest/riverdrivertest/riverdrivertest.go @@ -854,7 +854,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 +863,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 +887,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 +945,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,110 +1237,110 @@ 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("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() @@ -1950,7 +1958,6 @@ func Exercise[TTx any](ctx context.Context, t *testing.T, jobUpdated, err := exec.JobGetByID(ctx, job.ID) require.NoError(t, err) require.Equal(t, rivertype.JobStateCancelled, jobUpdated.State) - require.Nil(t, jobUpdated.UniqueKey) }) }) @@ -1977,7 +1984,6 @@ func Exercise[TTx any](ctx context.Context, t *testing.T, jobUpdated, err := exec.JobGetByID(ctx, job.ID) require.NoError(t, err) require.Equal(t, rivertype.JobStateDiscarded, jobUpdated.State) - require.Nil(t, jobUpdated.UniqueKey) }) }) 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..c3db82f2 100644 --- a/periodic_job.go +++ b/periodic_job.go @@ -3,7 +3,6 @@ package river import ( "time" - "github.com/riverqueue/river/internal/dbunique" "github.com/riverqueue/river/internal/maintenance" "github.com/riverqueue/river/riverdriver" "github.com/riverqueue/river/rivershared/util/sliceutil" @@ -181,10 +180,10 @@ func (b *PeriodicJobBundle) toInternal(periodicJob *PeriodicJob) *maintenance.Pe opts = periodicJob.opts } return &maintenance.PeriodicJob{ - ConstructorFunc: func() (*riverdriver.JobInsertFastParams, *dbunique.UniqueOpts, error) { + ConstructorFunc: func() (*riverdriver.JobInsertFastParams, error) { args, options := periodicJob.constructorFunc() if args == nil { - return nil, nil, maintenance.ErrNoJobToInsert + return nil, maintenance.ErrNoJobToInsert } return insertParamsFromConfigArgsAndOptions(&b.periodicJobEnqueuer.Archetype, b.clientConfig, args, options) }, diff --git a/periodic_job_test.go b/periodic_job_test.go index 745a06d8..8857eee0 100644 --- a/periodic_job_test.go +++ b/periodic_job_test.go @@ -51,11 +51,11 @@ func TestPeriodicJobBundle(t *testing.T) { internalPeriodicJob := periodicJobBundle.toInternal(periodicJob) - insertParams1, _, err := internalPeriodicJob.ConstructorFunc() + insertParams1, err := internalPeriodicJob.ConstructorFunc() require.NoError(t, err) require.Equal(t, 1, mustUnmarshalJSON[TestJobArgs](t, insertParams1.EncodedArgs).JobNum) - insertParams2, _, err := internalPeriodicJob.ConstructorFunc() + insertParams2, err := internalPeriodicJob.ConstructorFunc() require.NoError(t, err) require.Equal(t, 2, mustUnmarshalJSON[TestJobArgs](t, insertParams2.EncodedArgs).JobNum) }) @@ -76,7 +76,7 @@ func TestPeriodicJobBundle(t *testing.T) { internalPeriodicJob := periodicJobBundle.toInternal(periodicJob) - _, _, err := internalPeriodicJob.ConstructorFunc() + _, err := internalPeriodicJob.ConstructorFunc() require.ErrorIs(t, err, maintenance.ErrNoJobToInsert) }) } diff --git a/producer_test.go b/producer_test.go index bcb9d159..2a0b8c5b 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) 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) 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..05d29172 100644 --- a/riverdriver/river_driver_interface.go +++ b/riverdriver/river_driver_interface.go @@ -115,11 +115,10 @@ 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) @@ -249,44 +248,42 @@ 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 { + 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 { @@ -366,8 +363,6 @@ type JobUpdateParams struct { FinalizedAt *time.Time StateDoUpdate bool State rivertype.JobState - UniqueKeyDoUpdate bool - UniqueKey []byte } // Leader represents a River leader. diff --git a/riverdriver/riverdatabasesql/internal/dbsqlc/models.go b/riverdriver/riverdatabasesql/internal/dbsqlc/models.go index ab99f050..a2765b24 100644 --- a/riverdriver/riverdatabasesql/internal/dbsqlc/models.go +++ b/riverdriver/riverdatabasesql/internal/dbsqlc/models.go @@ -78,23 +78,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 interface{} } 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..55cd4104 100644 --- a/riverdriver/riverdatabasesql/internal/dbsqlc/river_job.sql.go +++ b/riverdriver/riverdatabasesql/internal/dbsqlc/river_job.sql.go @@ -48,14 +48,14 @@ updated_job AS ( unique_key = CASE WHEN state = 'running' 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 +86,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 +118,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 +150,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 +168,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 +196,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 +223,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 +259,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 +275,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 +302,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 +341,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 +357,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 +411,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 +450,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 +466,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 +506,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 +533,9 @@ INSERT INTO river_job( queue, scheduled_at, state, - tags + tags, + unique_key, + unique_states ) VALUES ( $1, coalesce($2::timestamptz, now()), @@ -537,25 +547,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 interface{} +} + +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.QueryRowContext(ctx, jobInsertFast, arg.Args, arg.CreatedAt, @@ -568,26 +594,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 +632,9 @@ INSERT INTO river_job( queue, scheduled_at, state, - tags + tags, + unique_key, + unique_states ) SELECT unnest($1::jsonb[]), unnest($2::text[]), @@ -617,23 +649,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 []interface{} } -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 +693,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 +747,9 @@ INSERT INTO river_job( queue, scheduled_at, state, - tags + tags, + unique_key, + unique_states ) SELECT unnest($1::jsonb[]), unnest($2::text[]), @@ -709,19 +764,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 []interface{} } func (q *Queries) JobInsertFastManyNoReturning(ctx context.Context, db DBTX, arg *JobInsertFastManyNoReturningParams) (int64, error) { @@ -735,6 +801,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 +826,8 @@ INSERT INTO river_job( scheduled_at, state, tags, - unique_key + unique_key, + unique_states ) VALUES ( $1::jsonb, coalesce($2::smallint, 0), @@ -774,26 +843,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 interface{} } func (q *Queries) JobInsertFull(ctx context.Context, db DBTX, arg *JobInsertFullParams) (*RiverJob, error) { @@ -813,6 +884,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 +905,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 +968,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,6 +1000,7 @@ 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 } @@ -1047,7 +1028,7 @@ river_job_scheduled AS ( WHERE river_job.id = jobs_to_schedule.id RETURNING river_job.id ) -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 river_job_scheduled) ` @@ -1084,6 +1065,7 @@ func (q *Queries) JobSchedule(ctx context.Context, db DBTX, arg *JobSchedulePara &i.ScheduledAt, pq.Array(&i.Tags), &i.UniqueKey, + &i.UniqueStates, ); err != nil { return nil, err } @@ -1118,13 +1100,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 +1142,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 } @@ -1196,20 +1179,18 @@ updated_job AS ( max_attempts = CASE WHEN NOT should_cancel AND $7::boolean THEN $8 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 - ELSE unique_key END + ELSE scheduled_at 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 +1239,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 } @@ -1269,10 +1251,9 @@ SET attempted_at = CASE WHEN $3::boolean THEN $4 ELSE attempted_at END, 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, - 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 + state = CASE WHEN $9::boolean THEN $10 ELSE state END +WHERE id = $11 +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 { @@ -1286,8 +1267,6 @@ type JobUpdateParams struct { FinalizedAt *time.Time StateDoUpdate bool State RiverJobState - UniqueKeyDoUpdate bool - UniqueKey []byte ID int64 } @@ -1305,8 +1284,6 @@ func (q *Queries) JobUpdate(ctx context.Context, db DBTX, arg *JobUpdateParams) arg.FinalizedAt, arg.StateDoUpdate, arg.State, - arg.UniqueKeyDoUpdate, - arg.UniqueKey, arg.ID, ) var i RiverJob @@ -1328,6 +1305,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..f5a870d0 100644 --- a/riverdriver/riverdatabasesql/internal/dbsqlc/sqlc.yaml +++ b/riverdriver/riverdatabasesql/internal/dbsqlc/sqlc.yaml @@ -31,6 +31,12 @@ sql: ttl: "TTL" overrides: + - db_type: "bit(8)" + go_type: "pgtype.Bits" + + - db_type: "bit(8)[]" + go_type: "pq.Array(byte)" + # `database/sql` really does not play nicely with json/jsonb. If it's # left as `[]byte` or `json.RawMessage`, `database/sql` will try to # encode it as binary (with a \x) which Postgres won't accept as 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..f7ed45c4 100644 --- a/riverdriver/riverdatabasesql/river_database_sql_driver.go +++ b/riverdriver/riverdatabasesql/river_database_sql_driver.go @@ -18,6 +18,7 @@ import ( "strings" "time" + "github.com/jackc/pgx/v5/pgtype" "github.com/lib/pq" "github.com/riverqueue/river/riverdriver" @@ -197,36 +198,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: 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([]interface{}, len(params)), } now := time.Now() @@ -254,6 +263,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) @@ -261,20 +272,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([]interface{}, len(params)), } now := time.Now() @@ -302,6 +321,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] = pgtype.Bits{Bytes: []byte{params.UniqueStates}, Len: 8, Valid: params.UniqueStates != 0} } numInserted, err := dbsqlc.New().JobInsertFastManyNoReturning(ctx, e.dbtx, insertJobsParams) @@ -336,35 +357,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 { @@ -586,8 +578,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) 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..2a82fbc3 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')) @@ -208,7 +209,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 +223,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 +245,9 @@ INSERT INTO river_job( queue, scheduled_at, state, - tags + tags, + unique_key, + unique_states ) SELECT unnest(@args::jsonb[]), unnest(@kind::text[]), @@ -248,8 +262,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 +285,9 @@ INSERT INTO river_job( queue, scheduled_at, state, - tags + tags, + unique_key, + unique_states ) SELECT unnest(@args::jsonb[]), unnest(@kind::text[]), @@ -276,7 +302,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 +329,8 @@ INSERT INTO river_job( scheduled_at, state, tags, - unique_key + unique_key, + unique_states ) VALUES ( @args::jsonb, coalesce(@attempt::smallint, 0), @@ -310,42 +346,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 UPDATE river_job @@ -472,9 +476,7 @@ updated_job AS ( max_attempts = CASE WHEN NOT should_cancel AND @max_attempts_update::boolean THEN @max_attempts 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 - ELSE unique_key END + ELSE scheduled_at END FROM job_to_update WHERE river_job.id = job_to_update.id AND river_job.state = 'running' @@ -497,7 +499,6 @@ SET attempted_at = CASE WHEN @attempted_at_do_update::boolean THEN @attempted_at ELSE attempted_at END, 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, - unique_key = CASE WHEN @unique_key_do_update::boolean THEN @unique_key ELSE unique_key END + state = CASE WHEN @state_do_update::boolean THEN @state ELSE state 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..a72ac24d 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 @@ -46,14 +48,14 @@ updated_job AS ( unique_key = CASE WHEN state = 'running' 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 +86,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 +118,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 +150,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 +168,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 +196,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 +223,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 +259,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 +272,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 +299,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 +338,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 +351,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 +405,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 +444,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 +457,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 +497,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 +521,9 @@ INSERT INTO river_job( queue, scheduled_at, state, - tags + tags, + unique_key, + unique_states ) VALUES ( $1, coalesce($2::timestamptz, now()), @@ -523,25 +535,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 +582,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 +620,9 @@ INSERT INTO river_job( queue, scheduled_at, state, - tags + tags, + unique_key, + unique_states ) SELECT unnest($1::jsonb[]), unnest($2::text[]), @@ -603,23 +637,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 +681,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 +732,9 @@ INSERT INTO river_job( queue, scheduled_at, state, - tags + tags, + unique_key, + unique_states ) SELECT unnest($1::jsonb[]), unnest($2::text[]), @@ -692,19 +749,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 +786,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 +811,8 @@ INSERT INTO river_job( scheduled_at, state, tags, - unique_key + unique_key, + unique_states ) VALUES ( $1::jsonb, coalesce($2::smallint, 0), @@ -757,26 +828,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 +869,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 +890,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 +953,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,6 +985,7 @@ func (q *Queries) JobRetry(ctx context.Context, db DBTX, id int64) (*RiverJob, e &i.ScheduledAt, &i.Tags, &i.UniqueKey, + &i.UniqueStates, ) return &i, err } @@ -1030,7 +1013,7 @@ river_job_scheduled AS ( WHERE river_job.id = jobs_to_schedule.id RETURNING river_job.id ) -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 river_job_scheduled) ` @@ -1067,6 +1050,7 @@ func (q *Queries) JobSchedule(ctx context.Context, db DBTX, arg *JobSchedulePara &i.ScheduledAt, &i.Tags, &i.UniqueKey, + &i.UniqueStates, ); err != nil { return nil, err } @@ -1098,13 +1082,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 +1124,7 @@ func (q *Queries) JobSetCompleteIfRunningMany(ctx context.Context, db DBTX, arg &i.ScheduledAt, &i.Tags, &i.UniqueKey, + &i.UniqueStates, ); err != nil { return nil, err } @@ -1173,20 +1158,18 @@ updated_job AS ( max_attempts = CASE WHEN NOT should_cancel AND $7::boolean THEN $8 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 - ELSE unique_key END + ELSE scheduled_at 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 +1218,7 @@ func (q *Queries) JobSetStateIfRunning(ctx context.Context, db DBTX, arg *JobSet &i.ScheduledAt, &i.Tags, &i.UniqueKey, + &i.UniqueStates, ) return &i, err } @@ -1246,10 +1230,9 @@ SET attempted_at = CASE WHEN $3::boolean THEN $4 ELSE attempted_at END, 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, - 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 + state = CASE WHEN $9::boolean THEN $10 ELSE state END +WHERE id = $11 +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 { @@ -1263,8 +1246,6 @@ type JobUpdateParams struct { FinalizedAt *time.Time StateDoUpdate bool State RiverJobState - UniqueKeyDoUpdate bool - UniqueKey []byte ID int64 } @@ -1282,8 +1263,6 @@ func (q *Queries) JobUpdate(ctx context.Context, db DBTX, arg *JobUpdateParams) arg.FinalizedAt, arg.StateDoUpdate, arg.State, - arg.UniqueKeyDoUpdate, - arg.UniqueKey, arg.ID, ) var i RiverJob @@ -1305,6 +1284,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..367db136 --- /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..7bb7787d 100644 --- a/riverdriver/riverpgxv5/river_pgx_v5_driver.go +++ b/riverdriver/riverpgxv5/river_pgx_v5_driver.go @@ -17,6 +17,7 @@ import ( "time" "github.com/jackc/pgx/v5" + "github.com/jackc/pgx/v5/pgtype" "github.com/jackc/pgx/v5/pgxpool" "github.com/jackc/puddle/v2" @@ -192,36 +193,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 +258,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 +267,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 +299,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}, } } @@ -326,35 +345,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 { @@ -473,8 +463,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) diff --git a/rivermigrate/river_migrate.go b/rivermigrate/river_migrate.go index 0e6702ce..8f3bd955 100644 --- a/rivermigrate/river_migrate.go +++ b/rivermigrate/river_migrate.go @@ -297,16 +297,15 @@ func (m *Migrator[TTx]) GetVersion(version int) (Migration, error) { // // handle error // } func (m *Migrator[TTx]) Migrate(ctx context.Context, direction Direction, opts *MigrateOpts) (*MigrateResult, error) { - return dbutil.WithTxV(ctx, m.driver.GetExecutor(), func(ctx context.Context, exec riverdriver.ExecutorTx) (*MigrateResult, error) { - switch direction { - case DirectionDown: - return m.migrateDown(ctx, exec, direction, opts) - case DirectionUp: - return m.migrateUp(ctx, exec, direction, opts) - } + exec := m.driver.GetExecutor() + switch direction { + case DirectionDown: + return m.migrateDown(ctx, exec, direction, opts) + case DirectionUp: + return m.migrateUp(ctx, exec, direction, opts) + } - panic("invalid direction: " + direction) - }) + panic("invalid direction: " + direction) } // Migrate migrates the database in the given direction (up or down). The opts @@ -560,10 +559,20 @@ func (m *Migrator[TTx]) applyMigrations(ctx context.Context, exec riverdriver.Ex if !opts.DryRun { start := time.Now() - _, err := exec.Exec(ctx, sql) + // Similar to ActiveRecord migrations, we wrap each individual migration + // in its own transaction. Without this, certain migrations that require + // a commit on a preexisting operation (such as adding an enum value to be + // used in an immutable function) cannot succeed. + err := dbutil.WithTx(ctx, exec, func(ctx context.Context, exec riverdriver.ExecutorTx) error { + _, err := exec.Exec(ctx, sql) + if err != nil { + return fmt.Errorf("error applying version %03d [%s]: %w", + versionBundle.Version, strings.ToUpper(string(direction)), err) + } + return nil + }) if err != nil { - return nil, fmt.Errorf("error applying version %03d [%s]: %w", - versionBundle.Version, strings.ToUpper(string(direction)), err) + return nil, err } duration = time.Since(start) }