diff --git a/client.go b/client.go index a263cb42..f430899e 100644 --- a/client.go +++ b/client.go @@ -282,6 +282,13 @@ type Config struct { // Defaults to DefaultRetryPolicy. RetryPolicy ClientRetryPolicy + // schema is a non-standard schema where River tables are located. All table + // references in database queries will use this value as a prefix. + // + // Defaults to empty, which causes the client to look for tables using the + // setting of Postgres `search_path`. + schema string + // SkipUnknownJobCheck is a flag to control whether the client should skip // checking to see if a registered worker exists in the client's worker bundle // for a job arg prior to insertion. @@ -376,6 +383,7 @@ func (c *Config) WithDefaults() *Config { ReindexerSchedule: c.ReindexerSchedule, RescueStuckJobsAfter: valutil.ValOrDefault(c.RescueStuckJobsAfter, rescueAfter), RetryPolicy: retryPolicy, + schema: c.schema, SkipUnknownJobCheck: c.SkipUnknownJobCheck, Test: c.Test, TestOnly: c.TestOnly, @@ -668,7 +676,7 @@ func NewClient[TTx any](driver riverdriver.Driver[TTx], config *Config) (*Client // uses listen/notify. Instead, each service polls for changes it's // interested in. e.g. Elector polls to see if leader has expired. if !config.PollOnly { - client.notifier = notifier.New(archetype, driver.GetListener()) + client.notifier = notifier.New(archetype, driver.GetListener(config.schema)) client.services = append(client.services, client.notifier) } } else { @@ -705,6 +713,7 @@ func NewClient[TTx any](driver riverdriver.Driver[TTx], config *Config) (*Client CancelledJobRetentionPeriod: config.CancelledJobRetentionPeriod, CompletedJobRetentionPeriod: config.CompletedJobRetentionPeriod, DiscardedJobRetentionPeriod: config.DiscardedJobRetentionPeriod, + Schema: config.schema, Timeout: config.JobCleanerTimeout, }, driver.GetExecutor()) maintenanceServices = append(maintenanceServices, jobCleaner) @@ -715,6 +724,7 @@ func NewClient[TTx any](driver riverdriver.Driver[TTx], config *Config) (*Client jobRescuer := maintenance.NewRescuer(archetype, &maintenance.JobRescuerConfig{ ClientRetryPolicy: config.RetryPolicy, RescueAfter: config.RescueStuckJobsAfter, + Schema: config.schema, WorkUnitFactoryFunc: func(kind string) workunit.WorkUnitFactory { if workerInfo, ok := config.Workers.workersMap[kind]; ok { return workerInfo.workUnitFactory @@ -730,6 +740,7 @@ func NewClient[TTx any](driver riverdriver.Driver[TTx], config *Config) (*Client jobScheduler := maintenance.NewJobScheduler(archetype, &maintenance.JobSchedulerConfig{ Interval: config.schedulerInterval, NotifyInsert: client.maybeNotifyInsertForQueues, + Schema: config.schema, }, driver.GetExecutor()) maintenanceServices = append(maintenanceServices, jobScheduler) client.testSignals.jobScheduler = &jobScheduler.TestSignals @@ -750,6 +761,7 @@ func NewClient[TTx any](driver riverdriver.Driver[TTx], config *Config) (*Client { queueCleaner := maintenance.NewQueueCleaner(archetype, &maintenance.QueueCleanerConfig{ RetentionPeriod: maintenance.QueueRetentionPeriodDefault, + Schema: config.schema, }, driver.GetExecutor()) maintenanceServices = append(maintenanceServices, queueCleaner) client.testSignals.queueCleaner = &queueCleaner.TestSignals @@ -761,7 +773,10 @@ func NewClient[TTx any](driver riverdriver.Driver[TTx], config *Config) (*Client scheduleFunc = config.ReindexerSchedule.Next } - reindexer := maintenance.NewReindexer(archetype, &maintenance.ReindexerConfig{ScheduleFunc: scheduleFunc}, driver.GetExecutor()) + reindexer := maintenance.NewReindexer(archetype, &maintenance.ReindexerConfig{ + ScheduleFunc: scheduleFunc, + Schema: config.schema, + }, driver.GetExecutor()) maintenanceServices = append(maintenanceServices, reindexer) client.testSignals.reindexer = &reindexer.TestSignals } @@ -1236,6 +1251,7 @@ func (c *Client[TTx]) jobCancel(ctx context.Context, exec riverdriver.Executor, ID: jobID, CancelAttemptedAt: c.baseService.Time.NowUTC(), ControlTopic: string(notifier.NotificationTopicControl), + Schema: c.config.schema, }) } @@ -1243,7 +1259,10 @@ func (c *Client[TTx]) jobCancel(ctx context.Context, exec riverdriver.Executor, // deleted row if it was deleted. Jobs in the running state are not deleted, // instead returning rivertype.ErrJobRunning. func (c *Client[TTx]) JobDelete(ctx context.Context, id int64) (*rivertype.JobRow, error) { - return c.driver.GetExecutor().JobDelete(ctx, id) + return c.driver.GetExecutor().JobDelete(ctx, &riverdriver.JobDeleteParams{ + ID: id, + Schema: c.config.schema, + }) } // JobDelete deletes the job with the given ID from the database, returning the @@ -1253,20 +1272,29 @@ func (c *Client[TTx]) JobDelete(ctx context.Context, id int64) (*rivertype.JobRo // until the transaction commits, and if the transaction rolls back, so too is // the deleted job. func (c *Client[TTx]) JobDeleteTx(ctx context.Context, tx TTx, id int64) (*rivertype.JobRow, error) { - return c.driver.UnwrapExecutor(tx).JobDelete(ctx, id) + return c.driver.UnwrapExecutor(tx).JobDelete(ctx, &riverdriver.JobDeleteParams{ + ID: id, + Schema: c.config.schema, + }) } // JobGet fetches a single job by its ID. Returns the up-to-date JobRow for the // specified jobID if it exists. Returns ErrNotFound if the job doesn't exist. func (c *Client[TTx]) JobGet(ctx context.Context, id int64) (*rivertype.JobRow, error) { - return c.driver.GetExecutor().JobGetByID(ctx, id) + return c.driver.GetExecutor().JobGetByID(ctx, &riverdriver.JobGetByIDParams{ + ID: id, + Schema: c.config.schema, + }) } // JobGetTx fetches a single job by its ID, within a transaction. Returns the // up-to-date JobRow for the specified jobID if it exists. Returns ErrNotFound // if the job doesn't exist. func (c *Client[TTx]) JobGetTx(ctx context.Context, tx TTx, id int64) (*rivertype.JobRow, error) { - return c.driver.UnwrapExecutor(tx).JobGetByID(ctx, id) + return c.driver.UnwrapExecutor(tx).JobGetByID(ctx, &riverdriver.JobGetByIDParams{ + ID: id, + Schema: c.config.schema, + }) } // JobRetry updates the job with the given ID to make it immediately available @@ -1278,7 +1306,10 @@ func (c *Client[TTx]) JobGetTx(ctx context.Context, tx TTx, id int64) (*rivertyp // MaxAttempts is also incremented by one if the job has already exhausted its // max attempts. func (c *Client[TTx]) JobRetry(ctx context.Context, id int64) (*rivertype.JobRow, error) { - return c.driver.GetExecutor().JobRetry(ctx, id) + return c.driver.GetExecutor().JobRetry(ctx, &riverdriver.JobRetryParams{ + ID: id, + Schema: c.config.schema, + }) } // JobRetryTx updates the job with the given ID to make it immediately available @@ -1295,7 +1326,10 @@ func (c *Client[TTx]) JobRetry(ctx context.Context, id int64) (*rivertype.JobRow // MaxAttempts is also incremented by one if the job has already exhausted its // max attempts. func (c *Client[TTx]) JobRetryTx(ctx context.Context, tx TTx, id int64) (*rivertype.JobRow, error) { - return c.driver.UnwrapExecutor(tx).JobRetry(ctx, id) + return c.driver.UnwrapExecutor(tx).JobRetry(ctx, &riverdriver.JobRetryParams{ + ID: id, + Schema: c.config.schema, + }) } // ID returns the unique ID of this client as set in its config or @@ -1561,7 +1595,10 @@ func (c *Client[TTx]) validateParamsAndInsertMany(ctx context.Context, tx riverd // by the PeriodicJobEnqueuer. func (c *Client[TTx]) insertMany(ctx context.Context, tx riverdriver.ExecutorTx, insertParams []*rivertype.JobInsertParams) ([]*rivertype.JobInsertResult, error) { return c.insertManyShared(ctx, tx, insertParams, func(ctx context.Context, insertParams []*riverdriver.JobInsertFastParams) ([]*rivertype.JobInsertResult, error) { - results, err := c.pilot.JobInsertMany(ctx, tx, insertParams) + results, err := c.pilot.JobInsertMany(ctx, tx, &riverdriver.JobInsertFastManyParams{ + Jobs: insertParams, + Schema: c.config.schema, + }) if err != nil { return nil, err } @@ -1731,7 +1768,10 @@ func (c *Client[TTx]) insertManyFast(ctx context.Context, tx riverdriver.Executo } results, err := c.insertManyShared(ctx, tx, insertParams, func(ctx context.Context, insertParams []*riverdriver.JobInsertFastParams) ([]*rivertype.JobInsertResult, error) { - count, err := tx.JobInsertFastManyNoReturning(ctx, insertParams) + count, err := tx.JobInsertFastManyNoReturning(ctx, &riverdriver.JobInsertFastManyParams{ + Jobs: insertParams, + Schema: c.config.schema, + }) if err != nil { return nil, err } @@ -1773,8 +1813,9 @@ func (c *Client[TTx]) maybeNotifyInsertForQueues(ctx context.Context, tx riverdr } err := tx.NotifyMany(ctx, &riverdriver.NotifyManyParams{ - Topic: string(notifier.NotificationTopicInsert), Payload: payloads, + Schema: c.config.schema, + Topic: string(notifier.NotificationTopicInsert), }) if err != nil { c.baseService.Logger.ErrorContext( @@ -1804,6 +1845,7 @@ func (c *Client[TTx]) notifyQueuePauseOrResume(ctx context.Context, tx riverdriv err = tx.NotifyMany(ctx, &riverdriver.NotifyManyParams{ Payload: []string{string(payload)}, + Schema: c.config.schema, Topic: string(notifier.NotificationTopicControl), }) if err != nil { @@ -1850,6 +1892,7 @@ func (c *Client[TTx]) addProducer(queueName string, queueConfig QueueConfig) *pr QueueEventCallback: c.subscriptionManager.distributeQueueEvent, RetryPolicy: c.config.RetryPolicy, SchedulerInterval: c.config.schedulerInterval, + Schema: c.config.schema, StaleProducerRetentionPeriod: 5 * time.Minute, Workers: c.config.Workers, }) @@ -1966,7 +2009,10 @@ func (c *Client[TTx]) Queues() *QueueBundle { return c.queues } // The provided context is used for the underlying Postgres query and can be // used to cancel the operation or apply a timeout. func (c *Client[TTx]) QueueGet(ctx context.Context, name string) (*rivertype.Queue, error) { - return c.driver.GetExecutor().QueueGet(ctx, name) + return c.driver.GetExecutor().QueueGet(ctx, &riverdriver.QueueGetParams{ + Name: name, + Schema: c.config.schema, + }) } // QueueGetTx returns the queue with the given name. If the queue has not recently @@ -1975,7 +2021,10 @@ func (c *Client[TTx]) QueueGet(ctx context.Context, name string) (*rivertype.Que // The provided context is used for the underlying Postgres query and can be // used to cancel the operation or apply a timeout. func (c *Client[TTx]) QueueGetTx(ctx context.Context, tx TTx, name string) (*rivertype.Queue, error) { - return c.driver.UnwrapExecutor(tx).QueueGet(ctx, name) + return c.driver.UnwrapExecutor(tx).QueueGet(ctx, &riverdriver.QueueGetParams{ + Name: name, + Schema: c.config.schema, + }) } // QueueListResult is the result of a job list operation. It contains a list of @@ -2001,7 +2050,10 @@ func (c *Client[TTx]) QueueList(ctx context.Context, params *QueueListParams) (* params = NewQueueListParams() } - queues, err := c.driver.GetExecutor().QueueList(ctx, int(params.paginationCount)) + queues, err := c.driver.GetExecutor().QueueList(ctx, &riverdriver.QueueListParams{ + Limit: int(params.paginationCount), + Schema: c.config.schema, + }) if err != nil { return nil, err } @@ -2025,7 +2077,10 @@ func (c *Client[TTx]) QueueListTx(ctx context.Context, tx TTx, params *QueueList params = NewQueueListParams() } - queues, err := c.driver.UnwrapExecutor(tx).QueueList(ctx, int(params.paginationCount)) + queues, err := c.driver.UnwrapExecutor(tx).QueueList(ctx, &riverdriver.QueueListParams{ + Limit: int(params.paginationCount), + Schema: c.config.schema, + }) if err != nil { return nil, err } @@ -2051,7 +2106,10 @@ func (c *Client[TTx]) QueuePause(ctx context.Context, name string, opts *QueuePa } defer tx.Rollback(ctx) - if err := tx.QueuePause(ctx, name); err != nil { + if err := tx.QueuePause(ctx, &riverdriver.QueuePauseParams{ + Name: name, + Schema: c.config.schema, + }); err != nil { return err } @@ -2076,7 +2134,10 @@ func (c *Client[TTx]) QueuePause(ctx context.Context, name string, opts *QueuePa func (c *Client[TTx]) QueuePauseTx(ctx context.Context, tx TTx, name string, opts *QueuePauseOpts) error { executorTx := c.driver.UnwrapExecutor(tx) - if err := executorTx.QueuePause(ctx, name); err != nil { + if err := executorTx.QueuePause(ctx, &riverdriver.QueuePauseParams{ + Name: name, + Schema: c.config.schema, + }); err != nil { return err } @@ -2106,7 +2167,10 @@ func (c *Client[TTx]) QueueResume(ctx context.Context, name string, opts *QueueP } defer tx.Rollback(ctx) - if err := tx.QueueResume(ctx, name); err != nil { + if err := tx.QueueResume(ctx, &riverdriver.QueueResumeParams{ + Name: name, + Schema: c.config.schema, + }); err != nil { return err } @@ -2132,7 +2196,10 @@ func (c *Client[TTx]) QueueResume(ctx context.Context, name string, opts *QueueP func (c *Client[TTx]) QueueResumeTx(ctx context.Context, tx TTx, name string, opts *QueuePauseOpts) error { executorTx := c.driver.UnwrapExecutor(tx) - if err := executorTx.QueueResume(ctx, name); err != nil { + if err := executorTx.QueueResume(ctx, &riverdriver.QueueResumeParams{ + Name: name, + Schema: c.config.schema, + }); err != nil { return err } @@ -2202,8 +2269,9 @@ func (c *Client[TTx]) queueUpdate(ctx context.Context, executorTx riverdriver.Ex } if err := executorTx.NotifyMany(ctx, &riverdriver.NotifyManyParams{ - Topic: string(notifier.NotificationTopicControl), Payload: []string{string(payload)}, + Schema: c.config.schema, + Topic: string(notifier.NotificationTopicControl), }); err != nil { return nil, err } diff --git a/client_test.go b/client_test.go index 2ea25b97..dc6ab2b4 100644 --- a/client_test.go +++ b/client_test.go @@ -585,7 +585,7 @@ func Test_Client(t *testing.T) { // Cancel an unknown job ID, within a transaction: err = dbutil.WithTx(ctx, client.driver.GetExecutor(), func(ctx context.Context, exec riverdriver.ExecutorTx) error { - jobAfter, err := exec.JobCancel(ctx, &riverdriver.JobCancelParams{ID: 0}) + jobAfter, err := exec.JobCancel(ctx, &riverdriver.JobCancelParams{ID: 0, Schema: client.config.schema}) require.ErrorIs(t, err, ErrNotFound) require.Nil(t, jobAfter) return nil @@ -2020,7 +2020,11 @@ func Test_Client_InsertManyFast(t *testing.T) { require.NoError(t, err) require.Equal(t, 2, count) - jobs, err := client.driver.GetExecutor().JobGetByKindMany(ctx, []string{(noOpArgs{}).Kind()}) + jobs, err := client.driver.GetExecutor().JobGetByKindMany(ctx, &riverdriver.JobGetByKindManyParams{ + Kind: []string{(noOpArgs{}).Kind()}, + Schema: client.config.schema, + }) + require.NoError(t, err) require.Len(t, jobs, 2, "Expected to find exactly two jobs of kind: "+(noOpArgs{}).Kind()) }) @@ -2122,7 +2126,11 @@ func Test_Client_InsertManyFast(t *testing.T) { require.NoError(t, err) require.Equal(t, 1, count) - jobs, err := client.driver.GetExecutor().JobGetByKindMany(ctx, []string{(noOpArgs{}).Kind()}) + jobs, err := client.driver.GetExecutor().JobGetByKindMany(ctx, &riverdriver.JobGetByKindManyParams{ + Kind: []string{(noOpArgs{}).Kind()}, + Schema: client.config.schema, + }) + require.NoError(t, err) require.Len(t, jobs, 1, "Expected to find exactly one job of kind: "+(noOpArgs{}).Kind()) jobRow := jobs[0] @@ -2262,14 +2270,21 @@ func Test_Client_InsertManyFastTx(t *testing.T) { require.NoError(t, err) require.Equal(t, 2, count) - jobs, err := client.driver.UnwrapExecutor(bundle.tx).JobGetByKindMany(ctx, []string{(noOpArgs{}).Kind()}) + jobs, err := client.driver.UnwrapExecutor(bundle.tx).JobGetByKindMany(ctx, &riverdriver.JobGetByKindManyParams{ + Kind: []string{(noOpArgs{}).Kind()}, + Schema: client.config.schema, + }) + require.NoError(t, err) require.Len(t, jobs, 2, "Expected to find exactly two jobs of kind: "+(noOpArgs{}).Kind()) require.NoError(t, bundle.tx.Commit(ctx)) // Ensure the jobs are visible outside the transaction: - jobs, err = client.driver.GetExecutor().JobGetByKindMany(ctx, []string{(noOpArgs{}).Kind()}) + jobs, err = client.driver.GetExecutor().JobGetByKindMany(ctx, &riverdriver.JobGetByKindManyParams{ + Kind: []string{(noOpArgs{}).Kind()}, + Schema: client.config.schema, + }) require.NoError(t, err) require.Len(t, jobs, 2, "Expected to find exactly two jobs of kind: "+(noOpArgs{}).Kind()) }) @@ -2282,7 +2297,10 @@ func Test_Client_InsertManyFastTx(t *testing.T) { _, err := client.InsertManyFastTx(ctx, bundle.tx, []InsertManyParams{{noOpArgs{}, nil}}) require.NoError(t, err) - insertedJobs, err := client.driver.UnwrapExecutor(bundle.tx).JobGetByKindMany(ctx, []string{(noOpArgs{}).Kind()}) + insertedJobs, err := client.driver.UnwrapExecutor(bundle.tx).JobGetByKindMany(ctx, &riverdriver.JobGetByKindManyParams{ + Kind: []string{(noOpArgs{}).Kind()}, + Schema: client.config.schema, + }) require.NoError(t, err) require.Len(t, insertedJobs, 1) require.Equal(t, rivertype.JobStateAvailable, insertedJobs[0].State) @@ -2300,7 +2318,10 @@ func Test_Client_InsertManyFastTx(t *testing.T) { require.NoError(t, err) require.Equal(t, 1, count) - insertedJobs, err := client.driver.UnwrapExecutor(bundle.tx).JobGetByKindMany(ctx, []string{(noOpArgs{}).Kind()}) + insertedJobs, err := client.driver.UnwrapExecutor(bundle.tx).JobGetByKindMany(ctx, &riverdriver.JobGetByKindManyParams{ + Kind: []string{(noOpArgs{}).Kind()}, + Schema: client.config.schema, + }) require.NoError(t, err) require.Len(t, insertedJobs, 1) require.Equal(t, rivertype.JobStateScheduled, insertedJobs[0].State) @@ -2466,7 +2487,10 @@ func Test_Client_InsertMany(t *testing.T) { require.NotEqual(t, results[0].Job.ID, results[1].Job.ID) - jobs, err := client.driver.GetExecutor().JobGetByKindMany(ctx, []string{(noOpArgs{}).Kind()}) + jobs, err := client.driver.GetExecutor().JobGetByKindMany(ctx, &riverdriver.JobGetByKindManyParams{ + Kind: []string{(noOpArgs{}).Kind()}, + Schema: client.config.schema, + }) require.NoError(t, err) require.Len(t, jobs, 2, "Expected to find exactly two jobs of kind: "+(noOpArgs{}).Kind()) }) @@ -2568,7 +2592,10 @@ func Test_Client_InsertMany(t *testing.T) { require.NoError(t, err) require.Len(t, results, 1) - jobs, err := client.driver.GetExecutor().JobGetByKindMany(ctx, []string{(noOpArgs{}).Kind()}) + jobs, err := client.driver.GetExecutor().JobGetByKindMany(ctx, &riverdriver.JobGetByKindManyParams{ + Kind: []string{(noOpArgs{}).Kind()}, + Schema: client.config.schema, + }) require.NoError(t, err) require.Len(t, jobs, 1, "Expected to find exactly one job of kind: "+(noOpArgs{}).Kind()) jobRow := jobs[0] @@ -2752,7 +2779,10 @@ func Test_Client_InsertManyTx(t *testing.T) { require.NotEqual(t, results[0].Job.ID, results[1].Job.ID) - jobs, err := client.driver.UnwrapExecutor(bundle.tx).JobGetByKindMany(ctx, []string{(noOpArgs{}).Kind()}) + jobs, err := client.driver.UnwrapExecutor(bundle.tx).JobGetByKindMany(ctx, &riverdriver.JobGetByKindManyParams{ + Kind: []string{(noOpArgs{}).Kind()}, + Schema: client.config.schema, + }) require.NoError(t, err) require.Len(t, jobs, 2, "Expected to find exactly two jobs of kind: "+(noOpArgs{}).Kind()) }) @@ -2769,7 +2799,10 @@ func Test_Client_InsertManyTx(t *testing.T) { require.Equal(t, rivertype.JobStateAvailable, results[0].Job.State) require.WithinDuration(t, time.Now(), results[0].Job.ScheduledAt, 2*time.Second) - insertedJobs, err := client.driver.UnwrapExecutor(bundle.tx).JobGetByKindMany(ctx, []string{(noOpArgs{}).Kind()}) + insertedJobs, err := client.driver.UnwrapExecutor(bundle.tx).JobGetByKindMany(ctx, &riverdriver.JobGetByKindManyParams{ + Kind: []string{(noOpArgs{}).Kind()}, + Schema: client.config.schema, + }) require.NoError(t, err) require.Len(t, insertedJobs, 1) require.Equal(t, rivertype.JobStateAvailable, insertedJobs[0].State) @@ -3444,7 +3477,10 @@ func Test_Client_ErrorHandler(t *testing.T) { // unknown job. insertParams, err := insertParamsFromConfigArgsAndOptions(&client.baseService.Archetype, config, unregisteredJobArgs{}, nil) require.NoError(t, err) - _, err = client.driver.GetExecutor().JobInsertFastMany(ctx, []*riverdriver.JobInsertFastParams{(*riverdriver.JobInsertFastParams)(insertParams)}) + _, err = client.driver.GetExecutor().JobInsertFastMany(ctx, &riverdriver.JobInsertFastManyParams{ + Jobs: []*riverdriver.JobInsertFastParams{(*riverdriver.JobInsertFastParams)(insertParams)}, + Schema: client.config.schema, + }) require.NoError(t, err) riversharedtest.WaitOrTimeout(t, bundle.SubscribeChan) @@ -3605,7 +3641,7 @@ func Test_Client_Maintenance(t *testing.T) { requireJobHasState := func(jobID int64, state rivertype.JobState) { t.Helper() - job, err := bundle.exec.JobGetByID(ctx, jobID) + job, err := bundle.exec.JobGetByID(ctx, &riverdriver.JobGetByIDParams{ID: jobID, Schema: ""}) require.NoError(t, err) require.Equal(t, state, job.State) } @@ -3702,7 +3738,10 @@ func Test_Client_Maintenance(t *testing.T) { svc := maintenance.GetService[*maintenance.PeriodicJobEnqueuer](client.queueMaintainer) svc.TestSignals.InsertedJobs.WaitOrTimeout() - jobs, err := bundle.exec.JobGetByKindMany(ctx, []string{(periodicJobArgs{}).Kind()}) + jobs, err := bundle.exec.JobGetByKindMany(ctx, &riverdriver.JobGetByKindManyParams{ + Kind: []string{(periodicJobArgs{}).Kind()}, + Schema: client.config.schema, + }) require.NoError(t, err) require.Len(t, jobs, 1, "Expected to find exactly one job of kind: "+(periodicJobArgs{}).Kind()) }) @@ -3728,7 +3767,10 @@ func Test_Client_Maintenance(t *testing.T) { svc.TestSignals.EnteredLoop.WaitOrTimeout() // No jobs yet because the RunOnStart option was not specified. - jobs, err := bundle.exec.JobGetByKindMany(ctx, []string{(periodicJobArgs{}).Kind()}) + jobs, err := bundle.exec.JobGetByKindMany(ctx, &riverdriver.JobGetByKindManyParams{ + Kind: []string{(periodicJobArgs{}).Kind()}, + Schema: client.config.schema, + }) require.NoError(t, err) require.Empty(t, jobs) }) @@ -3755,7 +3797,10 @@ func Test_Client_Maintenance(t *testing.T) { svc := maintenance.GetService[*maintenance.PeriodicJobEnqueuer](client.queueMaintainer) svc.TestSignals.SkippedJob.WaitOrTimeout() - jobs, err := bundle.exec.JobGetByKindMany(ctx, []string{(periodicJobArgs{}).Kind()}) + jobs, err := bundle.exec.JobGetByKindMany(ctx, &riverdriver.JobGetByKindManyParams{ + Kind: []string{(periodicJobArgs{}).Kind()}, + Schema: client.config.schema, + }) require.NoError(t, err) require.Empty(t, jobs, "Expected to find zero jobs of kind: "+(periodicJobArgs{}).Kind()) }) @@ -3789,7 +3834,10 @@ func Test_Client_Maintenance(t *testing.T) { svc.TestSignals.InsertedJobs.WaitOrTimeout() // We get a queued job because RunOnStart was specified. - jobs, err := exec.JobGetByKindMany(ctx, []string{(periodicJobArgs{}).Kind()}) + jobs, err := exec.JobGetByKindMany(ctx, &riverdriver.JobGetByKindManyParams{ + Kind: []string{(periodicJobArgs{}).Kind()}, + Schema: client.config.schema, + }) require.NoError(t, err) require.Len(t, jobs, 1) }) @@ -3839,12 +3887,18 @@ func Test_Client_Maintenance(t *testing.T) { // periodic job was inserted also due to RunOnStart, but only after the // first was removed. { - jobs, err := exec.JobGetByKindMany(ctx, []string{(periodicJobArgs{}).Kind()}) + jobs, err := exec.JobGetByKindMany(ctx, &riverdriver.JobGetByKindManyParams{ + Kind: []string{(periodicJobArgs{}).Kind()}, + Schema: client.config.schema, + }) require.NoError(t, err) require.Len(t, jobs, 1) } { - jobs, err := exec.JobGetByKindMany(ctx, []string{(OtherPeriodicArgs{}).Kind()}) + jobs, err := exec.JobGetByKindMany(ctx, &riverdriver.JobGetByKindManyParams{ + Kind: []string{(OtherPeriodicArgs{}).Kind()}, + Schema: client.config.schema, + }) require.NoError(t, err) require.Len(t, jobs, 1) } @@ -3878,7 +3932,10 @@ func Test_Client_Maintenance(t *testing.T) { svc := maintenance.GetService[*maintenance.PeriodicJobEnqueuer](client.queueMaintainer) svc.TestSignals.InsertedJobs.WaitOrTimeout() - jobs, err := bundle.exec.JobGetByKindMany(ctx, []string{(periodicJobArgs{}).Kind()}) + jobs, err := bundle.exec.JobGetByKindMany(ctx, &riverdriver.JobGetByKindManyParams{ + Kind: []string{(periodicJobArgs{}).Kind()}, + Schema: client.config.schema, + }) require.NoError(t, err) require.Len(t, jobs, 1, "Expected to find exactly one job of kind: "+(periodicJobArgs{}).Kind()) }) @@ -4371,8 +4428,10 @@ func Test_Client_RetryPolicy(t *testing.T) { _ = riversharedtest.WaitOrTimeout(t, subscribeChan) } - finishedJobs, err := client.driver.GetExecutor().JobGetByIDMany(ctx, - sliceutil.Map(originalJobs, func(m *rivertype.JobRow) int64 { return m.ID })) + finishedJobs, err := client.driver.GetExecutor().JobGetByIDMany(ctx, &riverdriver.JobGetByIDManyParams{ + ID: sliceutil.Map(originalJobs, func(m *rivertype.JobRow) int64 { return m.ID }), + Schema: "", + }) require.NoError(t, err) // Jobs aren't guaranteed to come back out of the queue in the same @@ -4799,7 +4858,10 @@ func Test_Client_SubscribeConfig(t *testing.T) { } } - _, err := client.driver.GetExecutor().JobInsertFastMany(ctx, insertParams) + _, err := client.driver.GetExecutor().JobInsertFastMany(ctx, &riverdriver.JobInsertFastManyParams{ + Jobs: insertParams, + Schema: "", + }) require.NoError(t, err) // Need to start waiting on events before running the client or the @@ -5232,7 +5294,10 @@ func Test_Client_UnknownJobKindErrorsTheJob(t *testing.T) { insertParams, err := insertParamsFromConfigArgsAndOptions(&client.baseService.Archetype, config, unregisteredJobArgs{}, nil) require.NoError(err) - insertedResults, err := client.driver.GetExecutor().JobInsertFastMany(ctx, []*riverdriver.JobInsertFastParams{(*riverdriver.JobInsertFastParams)(insertParams)}) + insertedResults, err := client.driver.GetExecutor().JobInsertFastMany(ctx, &riverdriver.JobInsertFastManyParams{ + Jobs: []*riverdriver.JobInsertFastParams{(*riverdriver.JobInsertFastParams)(insertParams)}, + Schema: client.config.schema, + }) require.NoError(err) insertedResult := insertedResults[0] diff --git a/cmd/river/riverbench/river_bench.go b/cmd/river/riverbench/river_bench.go index 85776799..f68264e0 100644 --- a/cmd/river/riverbench/river_bench.go +++ b/cmd/river/riverbench/river_bench.go @@ -20,13 +20,15 @@ type Benchmarker[TTx any] struct { driver riverdriver.Driver[TTx] // database pool wrapped in River driver logger *slog.Logger // logger, also injected to client name string // name of the service for logging purposes + schema string // custom schema where River tables are located } -func NewBenchmarker[TTx any](driver riverdriver.Driver[TTx], logger *slog.Logger) *Benchmarker[TTx] { +func NewBenchmarker[TTx any](driver riverdriver.Driver[TTx], logger *slog.Logger, schema string) *Benchmarker[TTx] { return &Benchmarker[TTx]{ driver: driver, logger: logger, name: "Benchmarker", + schema: schema, } } @@ -190,7 +192,10 @@ func (b *Benchmarker[TTx]) Run(ctx context.Context, duration time.Duration, numT return case <-ticker.C: - numJobs, err := b.driver.GetExecutor().JobCountByState(ctx, rivertype.JobStateAvailable) + numJobs, err := b.driver.GetExecutor().JobCountByState(ctx, &riverdriver.JobCountByStateParams{ + Schema: b.schema, + State: rivertype.JobStateAvailable, + }) if err != nil { b.logger.ErrorContext(ctx, "Error counting jobs", "err", err) continue diff --git a/cmd/river/rivercli/command.go b/cmd/river/rivercli/command.go index 520b0e22..cce9b44b 100644 --- a/cmd/river/rivercli/command.go +++ b/cmd/river/rivercli/command.go @@ -55,6 +55,7 @@ type CommandBase struct { DriverProcurer DriverProcurer Logger *slog.Logger Out io.Writer + Schema string GetBenchmarker func() BenchmarkerInterface GetMigrator func(config *rivermigrate.Config) (MigratorInterface, error) @@ -75,6 +76,7 @@ type RunCommandBundle struct { DriverProcurer DriverProcurer Logger *slog.Logger OutStd io.Writer + Schema string } // RunCommand bootstraps and runs a River CLI subcommand. @@ -88,6 +90,7 @@ func RunCommand[TOpts CommandOpts](ctx context.Context, bundle *RunCommandBundle DriverProcurer: bundle.DriverProcurer, Logger: bundle.Logger, Out: bundle.OutStd, + Schema: bundle.Schema, } var databaseURL *string @@ -124,7 +127,9 @@ func RunCommand[TOpts CommandOpts](ctx context.Context, bundle *RunCommandBundle driver := bundle.DriverProcurer.ProcurePgxV5(dbPool) - commandBase.GetBenchmarker = func() BenchmarkerInterface { return riverbench.NewBenchmarker(driver, commandBase.Logger) } + commandBase.GetBenchmarker = func() BenchmarkerInterface { + return riverbench.NewBenchmarker(driver, commandBase.Logger, commandBase.Schema) + } commandBase.GetMigrator = func(config *rivermigrate.Config) (MigratorInterface, error) { return rivermigrate.New(driver, config) } } diff --git a/cmd/river/rivercli/river_cli.go b/cmd/river/rivercli/river_cli.go index 78ea3b56..c07d6b8b 100644 --- a/cmd/river/rivercli/river_cli.go +++ b/cmd/river/rivercli/river_cli.go @@ -81,12 +81,13 @@ func (c *CLI) BaseCommandSet() *cobra.Command { } // Make a bundle for RunCommand. Takes a database URL pointer because not every command is required to take a database URL. - makeCommandBundle := func(databaseURL *string) *RunCommandBundle { + makeCommandBundle := func(databaseURL *string, schema string) *RunCommandBundle { return &RunCommandBundle{ DatabaseURL: databaseURL, DriverProcurer: c.driverProcurer, Logger: makeLogger(), OutStd: c.out, + Schema: schema, } } @@ -110,7 +111,7 @@ PG* vars if it's been specified. `), RunE: func(cmd *cobra.Command, args []string) error { if rootOpts.Version { - return RunCommand(ctx, makeCommandBundle(nil), &version{}, &versionOpts{Name: c.name}) + return RunCommand(ctx, makeCommandBundle(nil, ""), &version{}, &versionOpts{Name: c.name}) } _ = cmd.Usage() @@ -132,6 +133,9 @@ PG* vars if it's been specified. addLineFlag := func(cmd *cobra.Command, line *string) { cmd.Flags().StringVar(line, "line", "", "migration line to operate on (default: main)") } + addSchemaFlag := func(cmd *cobra.Command, schema *string) { + cmd.Flags().StringVar(schema, "schema", "", "name of non-default database schema where River tables are located") + } // bench { @@ -154,10 +158,11 @@ The database in --database-url will have its jobs table truncated, so make sure to use a development database only. `), RunE: func(cmd *cobra.Command, args []string) error { - return RunCommand(ctx, makeCommandBundle(&opts.DatabaseURL), &bench{}, &opts) + return RunCommand(ctx, makeCommandBundle(&opts.DatabaseURL, opts.Schema), &bench{}, &opts) }, } addDatabaseURLFlag(cmd, &opts.DatabaseURL) + addSchemaFlag(cmd, &opts.Schema) cmd.Flags().DurationVar(&opts.Duration, "duration", 0, "duration after which to stop benchmark, accepting Go-style durations like 1m, 5m30s") cmd.Flags().IntVarP(&opts.NumTotalJobs, "num-total-jobs", "n", 0, "number of jobs to insert before starting and which are worked down until finish") rootCmd.AddCommand(cmd) @@ -168,6 +173,7 @@ to use a development database only. // consistent. addMigrateFlags := func(cmd *cobra.Command, opts *migrateOpts) { addDatabaseURLFlag(cmd, &opts.DatabaseURL) + addSchemaFlag(cmd, &opts.Schema) cmd.Flags().BoolVar(&opts.DryRun, "dry-run", false, "print information on migrations, but don't apply them") cmd.Flags().StringVar(&opts.Line, "line", "", "migration line to operate on (default: main)") cmd.Flags().IntVar(&opts.MaxSteps, "max-steps", 0, "maximum number of steps to migrate") @@ -193,7 +199,7 @@ operations can be prevented with --dry-run. Combine --show-sql and --dry-run to dump prospective migrations that would be applied to stdout. `), RunE: func(cmd *cobra.Command, args []string) error { - return RunCommand(ctx, makeCommandBundle(&opts.DatabaseURL), &migrateDown{}, &opts) + return RunCommand(ctx, makeCommandBundle(&opts.DatabaseURL, opts.Schema), &migrateDown{}, &opts) }, } addMigrateFlags(cmd, &opts) @@ -231,7 +237,7 @@ framework, which aren't necessary if using an external framework: river migrate-get --all --exclude-version 1 --down > river_all.down.sql `), RunE: func(cmd *cobra.Command, args []string) error { - return RunCommand(ctx, makeCommandBundle(nil), &migrateGet{}, &opts) + return RunCommand(ctx, makeCommandBundle(nil, ""), &migrateGet{}, &opts) }, } cmd.Flags().BoolVar(&opts.All, "all", false, "print all migrations; down migrations are printed in descending order") @@ -258,10 +264,11 @@ framework, which aren't necessary if using an external framework: TODO `), RunE: func(cmd *cobra.Command, args []string) error { - return RunCommand(ctx, makeCommandBundle(&opts.DatabaseURL), &migrateList{}, &opts) + return RunCommand(ctx, makeCommandBundle(&opts.DatabaseURL, opts.Schema), &migrateList{}, &opts) }, } addDatabaseURLFlag(cmd, &opts.DatabaseURL) + addSchemaFlag(cmd, &opts.Schema) cmd.Flags().StringVar(&opts.Line, "line", "", "migration line to operate on (default: main)") rootCmd.AddCommand(cmd) } @@ -284,7 +291,7 @@ operations can be prevented with --dry-run. Combine --show-sql and --dry-run to dump prospective migrations that would be applied to stdout. `), RunE: func(cmd *cobra.Command, args []string) error { - return RunCommand(ctx, makeCommandBundle(&opts.DatabaseURL), &migrateUp{}, &opts) + return RunCommand(ctx, makeCommandBundle(&opts.DatabaseURL, opts.Schema), &migrateUp{}, &opts) }, } addMigrateFlags(cmd, &opts) @@ -306,10 +313,11 @@ Can be paired with river migrate-up --dry-run --show-sql to dump information on migrations that need to be run, but without running them. `), RunE: func(cmd *cobra.Command, args []string) error { - return RunCommand(ctx, makeCommandBundle(&opts.DatabaseURL), &validate{}, &opts) + return RunCommand(ctx, makeCommandBundle(&opts.DatabaseURL, opts.Schema), &validate{}, &opts) }, } addDatabaseURLFlag(cmd, &opts.DatabaseURL) + addSchemaFlag(cmd, &opts.Schema) cmd.Flags().StringVar(&opts.Line, "line", "", "migration line to operate on (default: main)") rootCmd.AddCommand(cmd) } @@ -323,7 +331,7 @@ migrations that need to be run, but without running them. Print River and Go version information. `), RunE: func(cmd *cobra.Command, args []string) error { - return RunCommand(ctx, makeCommandBundle(nil), &version{}, &versionOpts{Name: c.name}) + return RunCommand(ctx, makeCommandBundle(nil, ""), &version{}, &versionOpts{Name: c.name}) }, } rootCmd.AddCommand(cmd) @@ -340,6 +348,7 @@ type benchOpts struct { Debug bool Duration time.Duration NumTotalJobs int + Schema string Verbose bool } @@ -368,6 +377,7 @@ type migrateOpts struct { Line string ShowSQL bool MaxSteps int + Schema string TargetVersion int } @@ -530,6 +540,7 @@ func (c *migrateGet) Run(_ context.Context, opts *migrateGetOpts) (bool, error) type migrateListOpts struct { DatabaseURL string Line string + Schema string } func (o *migrateListOpts) Validate() error { return nil } @@ -598,6 +609,7 @@ func (c *migrateUp) Run(ctx context.Context, opts *migrateOpts) (bool, error) { type validateOpts struct { DatabaseURL string Line string + Schema string } func (o *validateOpts) Validate() error { diff --git a/driver_test.go b/driver_test.go index 3caabed8..eae1810c 100644 --- a/driver_test.go +++ b/driver_test.go @@ -98,17 +98,20 @@ func BenchmarkDriverRiverPgxV5_Executor(b *testing.B) { return driver.UnwrapExecutor(tx), &testBundle{} } - makeInsertParams := func() []*riverdriver.JobInsertFastParams { - return []*riverdriver.JobInsertFastParams{{ - EncodedArgs: []byte(`{}`), - Kind: "fake_job", - MaxAttempts: rivercommon.MaxAttemptsDefault, - Metadata: []byte(`{}`), - Priority: rivercommon.PriorityDefault, - Queue: rivercommon.QueueDefault, - ScheduledAt: nil, - State: rivertype.JobStateAvailable, - }} + makeInsertParams := func() *riverdriver.JobInsertFastManyParams { + return &riverdriver.JobInsertFastManyParams{ + Jobs: []*riverdriver.JobInsertFastParams{{ + EncodedArgs: []byte(`{}`), + Kind: "fake_job", + MaxAttempts: rivercommon.MaxAttemptsDefault, + Metadata: []byte(`{}`), + Priority: rivercommon.PriorityDefault, + Queue: rivercommon.QueueDefault, + ScheduledAt: nil, + State: rivertype.JobStateAvailable, + }}, + Schema: "", + } } b.Run("JobInsert_Sequential", func(b *testing.B) { @@ -222,14 +225,16 @@ func BenchmarkDriverRiverPgxV5Insert(b *testing.B) { _, bundle := setup(b) for range b.N { - _, err := bundle.exec.JobInsertFastMany(ctx, []*riverdriver.JobInsertFastParams{{ - EncodedArgs: []byte(`{"encoded": "args"}`), - Kind: "test_kind", - MaxAttempts: rivercommon.MaxAttemptsDefault, - Priority: rivercommon.PriorityDefault, - Queue: rivercommon.QueueDefault, - State: rivertype.JobStateAvailable, - }}) + _, err := bundle.exec.JobInsertFastMany(ctx, &riverdriver.JobInsertFastManyParams{ + Jobs: []*riverdriver.JobInsertFastParams{{ + EncodedArgs: []byte(`{"encoded": "args"}`), + Kind: "test_kind", + MaxAttempts: rivercommon.MaxAttemptsDefault, + Priority: rivercommon.PriorityDefault, + Queue: rivercommon.QueueDefault, + State: rivertype.JobStateAvailable, + }}, + }) require.NoError(b, err) } }) @@ -238,16 +243,18 @@ func BenchmarkDriverRiverPgxV5Insert(b *testing.B) { _, bundle := setup(b) for i := range b.N { - _, err := bundle.exec.JobInsertFastMany(ctx, []*riverdriver.JobInsertFastParams{{ - EncodedArgs: []byte(`{"encoded": "args"}`), - Kind: "test_kind", - MaxAttempts: rivercommon.MaxAttemptsDefault, - Priority: rivercommon.PriorityDefault, - Queue: rivercommon.QueueDefault, - State: rivertype.JobStateAvailable, - UniqueKey: []byte("test_unique_key_" + strconv.Itoa(i)), - UniqueStates: 0xFB, - }}) + _, err := bundle.exec.JobInsertFastMany(ctx, &riverdriver.JobInsertFastManyParams{ + Jobs: []*riverdriver.JobInsertFastParams{{ + EncodedArgs: []byte(`{"encoded": "args"}`), + Kind: "test_kind", + MaxAttempts: rivercommon.MaxAttemptsDefault, + Priority: rivercommon.PriorityDefault, + Queue: rivercommon.QueueDefault, + State: rivertype.JobStateAvailable, + UniqueKey: []byte("test_unique_key_" + strconv.Itoa(i)), + UniqueStates: 0xFB, + }}, + }) require.NoError(b, err) } }) diff --git a/internal/jobcompleter/job_completer_test.go b/internal/jobcompleter/job_completer_test.go index 77557291..b0426e6b 100644 --- a/internal/jobcompleter/job_completer_test.go +++ b/internal/jobcompleter/job_completer_test.go @@ -519,7 +519,7 @@ func testCompleter[TCompleter JobCompleter]( requireJob := func(t *testing.T, exec riverdriver.Executor, jobID int64) *rivertype.JobRow { t.Helper() - job, err := exec.JobGetByID(ctx, jobID) + job, err := exec.JobGetByID(ctx, &riverdriver.JobGetByIDParams{ID: jobID, Schema: ""}) require.NoError(t, err) return job } @@ -590,10 +590,16 @@ func testCompleter[TCompleter JobCompleter]( } } - _, err := bundle.exec.JobInsertFastMany(ctx, insertParams) + _, err := bundle.exec.JobInsertFastMany(ctx, &riverdriver.JobInsertFastManyParams{ + Jobs: insertParams, + Schema: "", + }) require.NoError(t, err) - jobs, err := bundle.exec.JobGetByKindMany(ctx, []string{kind}) + jobs, err := bundle.exec.JobGetByKindMany(ctx, &riverdriver.JobGetByKindManyParams{ + Kind: []string{kind}, + Schema: "", + }) require.NoError(t, err) t.Cleanup(riverinternaltest.DiscardContinuously(bundle.subscribeCh)) @@ -604,7 +610,10 @@ func testCompleter[TCompleter JobCompleter]( completer.Stop() - updatedJobs, err := bundle.exec.JobGetByKindMany(ctx, []string{kind}) + updatedJobs, err := bundle.exec.JobGetByKindMany(ctx, &riverdriver.JobGetByKindManyParams{ + Kind: []string{kind}, + Schema: "", + }) require.NoError(t, err) for i := range updatedJobs { require.Equal(t, rivertype.JobStateCompleted, updatedJobs[i].State) @@ -633,7 +642,9 @@ func testCompleter[TCompleter JobCompleter]( require.Positive(t, numInserted) - numCompleted, err := bundle.exec.JobCountByState(ctx, rivertype.JobStateCompleted) + numCompleted, err := bundle.exec.JobCountByState(ctx, &riverdriver.JobCountByStateParams{ + State: rivertype.JobStateCompleted, + }) require.NoError(t, err) t.Logf("Counted %d jobs as completed", numCompleted) require.Positive(t, numCompleted) @@ -658,7 +669,9 @@ func testCompleter[TCompleter JobCompleter]( require.Positive(t, numInserted) - numCompleted, err := bundle.exec.JobCountByState(ctx, rivertype.JobStateCompleted) + numCompleted, err := bundle.exec.JobCountByState(ctx, &riverdriver.JobCountByStateParams{ + State: rivertype.JobStateCompleted, + }) require.NoError(t, err) t.Logf("Counted %d jobs as completed", numCompleted) require.Positive(t, numCompleted) @@ -943,10 +956,16 @@ func benchmarkCompleter( } } - _, err := exec.JobInsertFastMany(ctx, insertParams) + _, err := exec.JobInsertFastMany(ctx, &riverdriver.JobInsertFastManyParams{ + Jobs: insertParams, + Schema: "", + }) require.NoError(b, err) - jobs, err := exec.JobGetByKindMany(ctx, []string{"benchmark_kind"}) + jobs, err := exec.JobGetByKindMany(ctx, &riverdriver.JobGetByKindManyParams{ + Kind: []string{"benchmark_kind"}, + Schema: "", + }) require.NoError(b, err) return completer, &testBundle{ diff --git a/internal/jobexecutor/job_executor.go b/internal/jobexecutor/job_executor.go index e07b8ef8..529cd18c 100644 --- a/internal/jobexecutor/job_executor.go +++ b/internal/jobexecutor/job_executor.go @@ -116,6 +116,8 @@ type JobExecutor struct { JobRow *rivertype.JobRow MiddlewareLookupGlobal middlewarelookup.MiddlewareLookupInterface SchedulerInterval time.Duration + Schema string + WorkerMiddleware []rivertype.WorkerMiddleware WorkUnit workunit.WorkUnit // Meant to be used from within the job executor only. diff --git a/internal/jobexecutor/job_executor_test.go b/internal/jobexecutor/job_executor_test.go index 3c6e58f4..483d5f84 100644 --- a/internal/jobexecutor/job_executor_test.go +++ b/internal/jobexecutor/job_executor_test.go @@ -138,18 +138,22 @@ func TestJobExecutor_Execute(t *testing.T) { workUnitFactory := newWorkUnitFactoryWithCustomRetry(func() error { return nil }, nil) now := time.Now().UTC() - results, err := exec.JobInsertFastMany(ctx, []*riverdriver.JobInsertFastParams{{ - EncodedArgs: []byte("{}"), - Kind: "jobexecutor_test", - MaxAttempts: rivercommon.MaxAttemptsDefault, - Priority: rivercommon.PriorityDefault, - Queue: rivercommon.QueueDefault, - // Needs to be explicitly set to a "now" horizon that's aligned with the - // JobGetAvailable call. InsertMany applies a default scheduled_at in Go - // so it can't pick up the Postgres-level `now()` default. - ScheduledAt: ptrutil.Ptr(now), - State: rivertype.JobStateAvailable, - }}) + results, err := exec.JobInsertFastMany(ctx, &riverdriver.JobInsertFastManyParams{ + Jobs: []*riverdriver.JobInsertFastParams{ + { + EncodedArgs: []byte("{}"), + Kind: "jobexecutor_test", + MaxAttempts: rivercommon.MaxAttemptsDefault, + Priority: rivercommon.PriorityDefault, + Queue: rivercommon.QueueDefault, + // Needs to be explicitly set to a "now" horizon that's aligned with the + // JobGetAvailable call. InsertMany applies a default scheduled_at in Go + // so it can't pick up the Postgres-level `now()` default. + ScheduledAt: ptrutil.Ptr(now), + State: rivertype.JobStateAvailable, + }, + }, + }) require.NoError(t, err) // Fetch the job to make sure it's marked as running: @@ -210,7 +214,10 @@ func TestJobExecutor_Execute(t *testing.T) { executor.Execute(ctx) jobUpdates := riversharedtest.WaitOrTimeout(t, bundle.updateCh) - job, err := bundle.exec.JobGetByID(ctx, bundle.jobRow.ID) + job, err := bundle.exec.JobGetByID(ctx, &riverdriver.JobGetByIDParams{ + ID: bundle.jobRow.ID, + Schema: "", + }) require.NoError(t, err) require.Equal(t, rivertype.JobStateCompleted, job.State) @@ -241,7 +248,10 @@ func TestJobExecutor_Execute(t *testing.T) { executor.Execute(ctx) riversharedtest.WaitOrTimeout(t, bundle.updateCh) - job, err := bundle.exec.JobGetByID(ctx, bundle.jobRow.ID) + job, err := bundle.exec.JobGetByID(ctx, &riverdriver.JobGetByIDParams{ + ID: bundle.jobRow.ID, + Schema: "", + }) require.NoError(t, err) require.WithinDuration(t, executor.ClientRetryPolicy.NextRetry(bundle.jobRow), job.ScheduledAt, 1*time.Second) require.Equal(t, rivertype.JobStateRetryable, job.State) @@ -265,7 +275,10 @@ func TestJobExecutor_Execute(t *testing.T) { executor.Execute(ctx) riversharedtest.WaitOrTimeout(t, bundle.updateCh) - job, err := bundle.exec.JobGetByID(ctx, bundle.jobRow.ID) + job, err := bundle.exec.JobGetByID(ctx, &riverdriver.JobGetByIDParams{ + ID: bundle.jobRow.ID, + Schema: "", + }) require.NoError(t, err) require.WithinDuration(t, executor.ClientRetryPolicy.NextRetry(bundle.jobRow), job.ScheduledAt, 1*time.Second) require.Equal(t, rivertype.JobStateRetryable, job.State) @@ -285,7 +298,10 @@ func TestJobExecutor_Execute(t *testing.T) { executor.Execute(ctx) riversharedtest.WaitOrTimeout(t, bundle.updateCh) - job, err := bundle.exec.JobGetByID(ctx, bundle.jobRow.ID) + job, err := bundle.exec.JobGetByID(ctx, &riverdriver.JobGetByIDParams{ + ID: bundle.jobRow.ID, + Schema: "", + }) require.NoError(t, err) require.WithinDuration(t, executor.ClientRetryPolicy.NextRetry(bundle.jobRow), job.ScheduledAt, 1*time.Second) require.Equal(t, rivertype.JobStateAvailable, job.State) @@ -304,7 +320,10 @@ func TestJobExecutor_Execute(t *testing.T) { executor.Execute(ctx) riversharedtest.WaitOrTimeout(t, bundle.updateCh) - job, err := bundle.exec.JobGetByID(ctx, bundle.jobRow.ID) + job, err := bundle.exec.JobGetByID(ctx, &riverdriver.JobGetByIDParams{ + ID: bundle.jobRow.ID, + Schema: "", + }) require.NoError(t, err) require.WithinDuration(t, executor.ClientRetryPolicy.NextRetry(bundle.jobRow), job.ScheduledAt, 16*time.Second) require.Equal(t, rivertype.JobStateRetryable, job.State) @@ -324,7 +343,10 @@ func TestJobExecutor_Execute(t *testing.T) { executor.Execute(ctx) riversharedtest.WaitOrTimeout(t, bundle.updateCh) - job, err := bundle.exec.JobGetByID(ctx, bundle.jobRow.ID) + job, err := bundle.exec.JobGetByID(ctx, &riverdriver.JobGetByIDParams{ + ID: bundle.jobRow.ID, + Schema: "", + }) require.NoError(t, err) require.WithinDuration(t, time.Now(), *job.FinalizedAt, 1*time.Second) require.Equal(t, rivertype.JobStateDiscarded, job.State) @@ -352,7 +374,10 @@ func TestJobExecutor_Execute(t *testing.T) { executor.Execute(ctx) riversharedtest.WaitOrTimeout(t, bundle.updateCh) - job, err := bundle.exec.JobGetByID(ctx, bundle.jobRow.ID) + job, err := bundle.exec.JobGetByID(ctx, &riverdriver.JobGetByIDParams{ + ID: bundle.jobRow.ID, + Schema: "", + }) require.NoError(t, err) require.WithinDuration(t, time.Now(), *job.FinalizedAt, 2*time.Second) require.Equal(t, rivertype.JobStateCancelled, job.State) @@ -376,7 +401,10 @@ func TestJobExecutor_Execute(t *testing.T) { executor.Execute(ctx) riversharedtest.WaitOrTimeout(t, bundle.updateCh) - job, err := bundle.exec.JobGetByID(ctx, bundle.jobRow.ID) + job, err := bundle.exec.JobGetByID(ctx, &riverdriver.JobGetByIDParams{ + ID: bundle.jobRow.ID, + Schema: "", + }) require.NoError(t, err) require.Equal(t, rivertype.JobStateScheduled, job.State) require.WithinDuration(t, time.Now().Add(30*time.Minute), job.ScheduledAt, 2*time.Second) @@ -396,7 +424,10 @@ func TestJobExecutor_Execute(t *testing.T) { executor.Execute(ctx) riversharedtest.WaitOrTimeout(t, bundle.updateCh) - job, err := bundle.exec.JobGetByID(ctx, bundle.jobRow.ID) + job, err := bundle.exec.JobGetByID(ctx, &riverdriver.JobGetByIDParams{ + ID: bundle.jobRow.ID, + Schema: "", + }) require.NoError(t, err) require.Equal(t, rivertype.JobStateAvailable, job.State) require.WithinDuration(t, time.Now(), job.ScheduledAt, 2*time.Second) @@ -416,7 +447,10 @@ func TestJobExecutor_Execute(t *testing.T) { executor.Execute(ctx) riversharedtest.WaitOrTimeout(t, bundle.updateCh) - job, err := bundle.exec.JobGetByID(ctx, bundle.jobRow.ID) + job, err := bundle.exec.JobGetByID(ctx, &riverdriver.JobGetByIDParams{ + ID: bundle.jobRow.ID, + Schema: "", + }) require.NoError(t, err) require.WithinDuration(t, executor.ClientRetryPolicy.NextRetry(bundle.jobRow), job.ScheduledAt, 1*time.Second) require.Equal(t, rivertype.JobStateRetryable, job.State) @@ -436,7 +470,10 @@ func TestJobExecutor_Execute(t *testing.T) { executor.Execute(ctx) riversharedtest.WaitOrTimeout(t, bundle.updateCh) - job, err := bundle.exec.JobGetByID(ctx, bundle.jobRow.ID) + job, err := bundle.exec.JobGetByID(ctx, &riverdriver.JobGetByIDParams{ + ID: bundle.jobRow.ID, + Schema: "", + }) require.NoError(t, err) require.Equal(t, rivertype.JobStateRetryable, job.State) require.WithinDuration(t, nextRetryAt, job.ScheduledAt, time.Microsecond) @@ -454,7 +491,10 @@ func TestJobExecutor_Execute(t *testing.T) { executor.Execute(ctx) riversharedtest.WaitOrTimeout(t, bundle.updateCh) - job, err := bundle.exec.JobGetByID(ctx, bundle.jobRow.ID) + job, err := bundle.exec.JobGetByID(ctx, &riverdriver.JobGetByIDParams{ + ID: bundle.jobRow.ID, + Schema: "", + }) require.NoError(t, err) require.WithinDuration(t, executor.DefaultClientRetryPolicy.NextRetry(bundle.jobRow), job.ScheduledAt, 1*time.Second) require.Equal(t, rivertype.JobStateRetryable, job.State) @@ -475,7 +515,10 @@ func TestJobExecutor_Execute(t *testing.T) { executor.Execute(ctx) riversharedtest.WaitOrTimeout(t, bundle.updateCh) - job, err := bundle.exec.JobGetByID(ctx, bundle.jobRow.ID) + job, err := bundle.exec.JobGetByID(ctx, &riverdriver.JobGetByIDParams{ + ID: bundle.jobRow.ID, + Schema: "", + }) require.NoError(t, err) require.Equal(t, rivertype.JobStateRetryable, job.State) @@ -496,7 +539,10 @@ func TestJobExecutor_Execute(t *testing.T) { executor.Execute(ctx) riversharedtest.WaitOrTimeout(t, bundle.updateCh) - job, err := bundle.exec.JobGetByID(ctx, bundle.jobRow.ID) + job, err := bundle.exec.JobGetByID(ctx, &riverdriver.JobGetByIDParams{ + ID: bundle.jobRow.ID, + Schema: "", + }) require.NoError(t, err) require.Equal(t, rivertype.JobStateCancelled, job.State) @@ -517,7 +563,10 @@ func TestJobExecutor_Execute(t *testing.T) { executor.Execute(ctx) riversharedtest.WaitOrTimeout(t, bundle.updateCh) - job, err := bundle.exec.JobGetByID(ctx, bundle.jobRow.ID) + job, err := bundle.exec.JobGetByID(ctx, &riverdriver.JobGetByIDParams{ + ID: bundle.jobRow.ID, + Schema: "", + }) require.NoError(t, err) require.Equal(t, rivertype.JobStateRetryable, job.State) @@ -533,7 +582,10 @@ func TestJobExecutor_Execute(t *testing.T) { executor.Execute(ctx) riversharedtest.WaitOrTimeout(t, bundle.updateCh) - job, err := bundle.exec.JobGetByID(ctx, bundle.jobRow.ID) + job, err := bundle.exec.JobGetByID(ctx, &riverdriver.JobGetByIDParams{ + ID: bundle.jobRow.ID, + Schema: "", + }) require.NoError(t, err) require.WithinDuration(t, executor.ClientRetryPolicy.NextRetry(bundle.jobRow), job.ScheduledAt, 1*time.Second) require.Equal(t, rivertype.JobStateRetryable, job.State) @@ -554,7 +606,10 @@ func TestJobExecutor_Execute(t *testing.T) { executor.Execute(ctx) riversharedtest.WaitOrTimeout(t, bundle.updateCh) - job, err := bundle.exec.JobGetByID(ctx, bundle.jobRow.ID) + job, err := bundle.exec.JobGetByID(ctx, &riverdriver.JobGetByIDParams{ + ID: bundle.jobRow.ID, + Schema: "", + }) require.NoError(t, err) require.WithinDuration(t, executor.ClientRetryPolicy.NextRetry(bundle.jobRow), job.ScheduledAt, 1*time.Second) require.Equal(t, rivertype.JobStateRetryable, job.State) @@ -572,7 +627,10 @@ func TestJobExecutor_Execute(t *testing.T) { executor.Execute(ctx) riversharedtest.WaitOrTimeout(t, bundle.updateCh) - job, err := bundle.exec.JobGetByID(ctx, bundle.jobRow.ID) + job, err := bundle.exec.JobGetByID(ctx, &riverdriver.JobGetByIDParams{ + ID: bundle.jobRow.ID, + Schema: "", + }) require.NoError(t, err) require.WithinDuration(t, time.Now(), *job.FinalizedAt, 1*time.Second) require.Equal(t, rivertype.JobStateDiscarded, job.State) @@ -614,7 +672,10 @@ func TestJobExecutor_Execute(t *testing.T) { executor.Execute(ctx) riversharedtest.WaitOrTimeout(t, bundle.updateCh) - job, err := bundle.exec.JobGetByID(ctx, bundle.jobRow.ID) + job, err := bundle.exec.JobGetByID(ctx, &riverdriver.JobGetByIDParams{ + ID: bundle.jobRow.ID, + Schema: "", + }) require.NoError(t, err) require.Equal(t, rivertype.JobStateRetryable, job.State) @@ -634,7 +695,10 @@ func TestJobExecutor_Execute(t *testing.T) { executor.Execute(ctx) riversharedtest.WaitOrTimeout(t, bundle.updateCh) - job, err := bundle.exec.JobGetByID(ctx, bundle.jobRow.ID) + job, err := bundle.exec.JobGetByID(ctx, &riverdriver.JobGetByIDParams{ + ID: bundle.jobRow.ID, + Schema: "", + }) require.NoError(t, err) require.Equal(t, rivertype.JobStateCancelled, job.State) @@ -654,7 +718,10 @@ func TestJobExecutor_Execute(t *testing.T) { executor.Execute(ctx) riversharedtest.WaitOrTimeout(t, bundle.updateCh) - job, err := bundle.exec.JobGetByID(ctx, bundle.jobRow.ID) + job, err := bundle.exec.JobGetByID(ctx, &riverdriver.JobGetByIDParams{ + ID: bundle.jobRow.ID, + Schema: "", + }) require.NoError(t, err) require.Equal(t, rivertype.JobStateRetryable, job.State) @@ -704,7 +771,10 @@ func TestJobExecutor_Execute(t *testing.T) { executor.Execute(workCtx) riversharedtest.WaitOrTimeout(t, bundle.updateCh) - jobRow, err := bundle.exec.JobGetByID(ctx, bundle.jobRow.ID) + jobRow, err := bundle.exec.JobGetByID(ctx, &riverdriver.JobGetByIDParams{ + ID: bundle.jobRow.ID, + Schema: "", + }) require.NoError(t, err) return jobRow } diff --git a/internal/leadership/elector.go b/internal/leadership/elector.go index 917940ec..4b307386 100644 --- a/internal/leadership/elector.go +++ b/internal/leadership/elector.go @@ -75,6 +75,7 @@ type Config struct { ClientID string ElectInterval time.Duration // period on which each elector attempts elect even without having received a resignation notification ElectIntervalJitter time.Duration + Schema string } func (c *Config) mustValidate() *Config { @@ -330,6 +331,7 @@ func (e *Elector) keepLeadershipLoop(ctx context.Context) error { reelected, err := attemptElectOrReelect(ctx, e.exec, true, &riverdriver.LeaderElectParams{ LeaderID: e.config.ClientID, + Schema: e.config.Schema, TTL: e.leaderTTL(), }) if err != nil { @@ -509,7 +511,9 @@ func attemptElectOrReelect(ctx context.Context, exec riverdriver.Executor, alrea defer cancel() return dbutil.WithTxV(ctx, exec, func(ctx context.Context, exec riverdriver.ExecutorTx) (bool, error) { - if _, err := exec.LeaderDeleteExpired(ctx); err != nil { + if _, err := exec.LeaderDeleteExpired(ctx, &riverdriver.LeaderDeleteExpiredParams{ + Schema: params.Schema, + }); err != nil { return false, err } diff --git a/internal/leadership/elector_test.go b/internal/leadership/elector_test.go index 96a79cc5..85f6d922 100644 --- a/internal/leadership/elector_test.go +++ b/internal/leadership/elector_test.go @@ -80,9 +80,10 @@ func TestElector_WithNotifier(t *testing.T) { archetype = riversharedtest.BaseServiceArchetype(t) dbPool = riverinternaltest.TestDB(ctx, t) driver = riverpgxv5.New(dbPool) + schema = "" // try to make tests schema-based rather than database-based in the future ) - notifier := notifier.New(archetype, driver.GetListener()) + notifier := notifier.New(archetype, driver.GetListener(schema)) { require.NoError(t, notifier.Start(ctx)) t.Cleanup(notifier.Stop) @@ -152,7 +153,9 @@ func testElector[TElectorBundle any]( elector.testSignals.GainedLeadership.WaitOrTimeout() - leader, err := bundle.exec.LeaderGetElectedLeader(ctx) + leader, err := bundle.exec.LeaderGetElectedLeader(ctx, &riverdriver.LeaderGetElectedLeaderParams{ + Schema: elector.config.Schema, + }) require.NoError(t, err) require.Equal(t, elector.config.ClientID, leader.LeaderID) @@ -160,7 +163,9 @@ func testElector[TElectorBundle any]( elector.testSignals.ResignedLeadership.WaitOrTimeout() - _, err = bundle.exec.LeaderGetElectedLeader(ctx) + _, err = bundle.exec.LeaderGetElectedLeader(ctx, &riverdriver.LeaderGetElectedLeaderParams{ + Schema: elector.config.Schema, + }) require.ErrorIs(t, err, rivertype.ErrNotFound) }) @@ -263,7 +268,9 @@ func testElector[TElectorBundle any]( t.Logf("Waiting for %s to gain leadership", elector1.config.ClientID) elector1.testSignals.GainedLeadership.WaitOrTimeout() - leader, err := bundle.exec.LeaderGetElectedLeader(ctx) + leader, err := bundle.exec.LeaderGetElectedLeader(ctx, &riverdriver.LeaderGetElectedLeaderParams{ + Schema: elector1.config.Schema, + }) require.NoError(t, err) require.Equal(t, elector1.config.ClientID, leader.LeaderID) } @@ -297,7 +304,9 @@ func testElector[TElectorBundle any]( elector2.testSignals.ResignedLeadership.WaitOrTimeout() } - _, err := bundle.exec.LeaderGetElectedLeader(ctx) + _, err := bundle.exec.LeaderGetElectedLeader(ctx, &riverdriver.LeaderGetElectedLeaderParams{ + Schema: elector1.config.Schema, + }) require.ErrorIs(t, err, rivertype.ErrNotFound) }) @@ -347,11 +356,14 @@ func TestAttemptElectOrReelect(t *testing.T) { elected, err := attemptElectOrReelect(ctx, bundle.exec, false, &riverdriver.LeaderElectParams{ LeaderID: clientID, TTL: leaderTTL, + Schema: "", }) require.NoError(t, err) require.True(t, elected) // won election - leader, err := bundle.exec.LeaderGetElectedLeader(ctx) + leader, err := bundle.exec.LeaderGetElectedLeader(ctx, &riverdriver.LeaderGetElectedLeaderParams{ + Schema: "", + }) require.NoError(t, err) require.WithinDuration(t, time.Now(), leader.ElectedAt, 100*time.Millisecond) require.WithinDuration(t, time.Now().Add(leaderTTL), leader.ExpiresAt, 100*time.Millisecond) @@ -372,13 +384,16 @@ func TestAttemptElectOrReelect(t *testing.T) { elected, err := attemptElectOrReelect(ctx, bundle.exec, true, &riverdriver.LeaderElectParams{ LeaderID: clientID, TTL: 30 * time.Second, + Schema: "", }) require.NoError(t, err) require.True(t, elected) // won re-election // expires_at should be incremented because this is the same leader that won // previously and we specified that we're already elected: - updatedLeader, err := bundle.exec.LeaderGetElectedLeader(ctx) + updatedLeader, err := bundle.exec.LeaderGetElectedLeader(ctx, &riverdriver.LeaderGetElectedLeaderParams{ + Schema: "", + }) require.NoError(t, err) require.Greater(t, updatedLeader.ExpiresAt, leader.ExpiresAt) }) @@ -395,6 +410,7 @@ func TestAttemptElectOrReelect(t *testing.T) { elected, err := attemptElectOrReelect(ctx, bundle.exec, true, &riverdriver.LeaderElectParams{ LeaderID: "different-client-id", TTL: leaderTTL, + Schema: "", }) require.NoError(t, err) require.False(t, elected) // lost election @@ -402,7 +418,9 @@ func TestAttemptElectOrReelect(t *testing.T) { // The time should not have changed because we specified that we were not // already elected, and the elect query is a no-op if there's already a // updatedLeader: - updatedLeader, err := bundle.exec.LeaderGetElectedLeader(ctx) + updatedLeader, err := bundle.exec.LeaderGetElectedLeader(ctx, &riverdriver.LeaderGetElectedLeaderParams{ + Schema: "", + }) require.NoError(t, err) require.Equal(t, leader.ExpiresAt, updatedLeader.ExpiresAt) }) diff --git a/internal/maintenance/job_cleaner.go b/internal/maintenance/job_cleaner.go index cd6a1fd5..eb83f59c 100644 --- a/internal/maintenance/job_cleaner.go +++ b/internal/maintenance/job_cleaner.go @@ -50,6 +50,10 @@ type JobCleanerConfig struct { // Interval is the amount of time to wait between runs of the cleaner. Interval time.Duration + // Schema where River tables are located. Empty string omits schema, causing + // Postgres to default to `search_path`. + Schema string + // Timeout of the individual queries in the job cleaner. Timeout time.Duration } @@ -163,6 +167,7 @@ func (s *JobCleaner) runOnce(ctx context.Context) (*jobCleanerRunOnceResult, err CompletedFinalizedAtHorizon: time.Now().Add(-s.Config.CompletedJobRetentionPeriod), DiscardedFinalizedAtHorizon: time.Now().Add(-s.Config.DiscardedJobRetentionPeriod), Max: s.batchSize, + Schema: s.Config.Schema, }) if err != nil { return 0, fmt.Errorf("error deleting completed jobs: %w", err) diff --git a/internal/maintenance/job_cleaner_test.go b/internal/maintenance/job_cleaner_test.go index c3a1a48d..57849bc6 100644 --- a/internal/maintenance/job_cleaner_test.go +++ b/internal/maintenance/job_cleaner_test.go @@ -105,32 +105,32 @@ func TestJobCleaner(t *testing.T) { cleaner.TestSignals.DeletedBatch.WaitOrTimeout() var err error - _, err = bundle.exec.JobGetByID(ctx, job1.ID) + _, err = bundle.exec.JobGetByID(ctx, &riverdriver.JobGetByIDParams{ID: job1.ID, Schema: cleaner.Config.Schema}) require.NotErrorIs(t, err, rivertype.ErrNotFound) // still there - _, err = bundle.exec.JobGetByID(ctx, job2.ID) + _, err = bundle.exec.JobGetByID(ctx, &riverdriver.JobGetByIDParams{ID: job2.ID, Schema: cleaner.Config.Schema}) require.NotErrorIs(t, err, rivertype.ErrNotFound) // still there - _, err = bundle.exec.JobGetByID(ctx, job3.ID) + _, err = bundle.exec.JobGetByID(ctx, &riverdriver.JobGetByIDParams{ID: job3.ID, Schema: cleaner.Config.Schema}) require.NotErrorIs(t, err, rivertype.ErrNotFound) // still there - _, err = bundle.exec.JobGetByID(ctx, cancelledJob1.ID) + _, err = bundle.exec.JobGetByID(ctx, &riverdriver.JobGetByIDParams{ID: cancelledJob1.ID, Schema: cleaner.Config.Schema}) require.ErrorIs(t, err, rivertype.ErrNotFound) - _, err = bundle.exec.JobGetByID(ctx, cancelledJob2.ID) + _, err = bundle.exec.JobGetByID(ctx, &riverdriver.JobGetByIDParams{ID: cancelledJob2.ID, Schema: cleaner.Config.Schema}) require.ErrorIs(t, err, rivertype.ErrNotFound) - _, err = bundle.exec.JobGetByID(ctx, cancelledJob3.ID) + _, err = bundle.exec.JobGetByID(ctx, &riverdriver.JobGetByIDParams{ID: cancelledJob3.ID, Schema: cleaner.Config.Schema}) require.NotErrorIs(t, err, rivertype.ErrNotFound) // still there - _, err = bundle.exec.JobGetByID(ctx, completedJob1.ID) + _, err = bundle.exec.JobGetByID(ctx, &riverdriver.JobGetByIDParams{ID: completedJob1.ID, Schema: cleaner.Config.Schema}) require.ErrorIs(t, err, rivertype.ErrNotFound) - _, err = bundle.exec.JobGetByID(ctx, completedJob2.ID) + _, err = bundle.exec.JobGetByID(ctx, &riverdriver.JobGetByIDParams{ID: completedJob2.ID, Schema: cleaner.Config.Schema}) require.ErrorIs(t, err, rivertype.ErrNotFound) - _, err = bundle.exec.JobGetByID(ctx, completedJob3.ID) + _, err = bundle.exec.JobGetByID(ctx, &riverdriver.JobGetByIDParams{ID: completedJob3.ID, Schema: cleaner.Config.Schema}) require.NotErrorIs(t, err, rivertype.ErrNotFound) // still there - _, err = bundle.exec.JobGetByID(ctx, discardedJob1.ID) + _, err = bundle.exec.JobGetByID(ctx, &riverdriver.JobGetByIDParams{ID: discardedJob1.ID, Schema: cleaner.Config.Schema}) require.ErrorIs(t, err, rivertype.ErrNotFound) - _, err = bundle.exec.JobGetByID(ctx, discardedJob2.ID) + _, err = bundle.exec.JobGetByID(ctx, &riverdriver.JobGetByIDParams{ID: discardedJob2.ID, Schema: cleaner.Config.Schema}) require.ErrorIs(t, err, rivertype.ErrNotFound) - _, err = bundle.exec.JobGetByID(ctx, discardedJob3.ID) + _, err = bundle.exec.JobGetByID(ctx, &riverdriver.JobGetByIDParams{ID: discardedJob3.ID, Schema: cleaner.Config.Schema}) require.NotErrorIs(t, err, rivertype.ErrNotFound) // still there }) @@ -158,7 +158,7 @@ func TestJobCleaner(t *testing.T) { cleaner.TestSignals.DeletedBatch.WaitOrTimeout() for _, job := range jobs { - _, err := bundle.exec.JobGetByID(ctx, job.ID) + _, err := bundle.exec.JobGetByID(ctx, &riverdriver.JobGetByIDParams{ID: job.ID, Schema: cleaner.Config.Schema}) require.ErrorIs(t, err, rivertype.ErrNotFound) } }) @@ -228,9 +228,9 @@ func TestJobCleaner(t *testing.T) { cleaner.TestSignals.DeletedBatch.WaitOrTimeout() var err error - _, err = bundle.exec.JobGetByID(ctx, job1.ID) + _, err = bundle.exec.JobGetByID(ctx, &riverdriver.JobGetByIDParams{ID: job1.ID, Schema: cleaner.Config.Schema}) require.ErrorIs(t, err, rivertype.ErrNotFound) - _, err = bundle.exec.JobGetByID(ctx, job2.ID) + _, err = bundle.exec.JobGetByID(ctx, &riverdriver.JobGetByIDParams{ID: job2.ID, Schema: cleaner.Config.Schema}) require.ErrorIs(t, err, rivertype.ErrNotFound) }) } diff --git a/internal/maintenance/job_rescuer.go b/internal/maintenance/job_rescuer.go index 00ce2f37..b5844431 100644 --- a/internal/maintenance/job_rescuer.go +++ b/internal/maintenance/job_rescuer.go @@ -50,6 +50,10 @@ type JobRescuerConfig struct { // considered stuck and should be rescued. RescueAfter time.Duration + // Schema where River tables are located. Empty string omits schema, causing + // Postgres to default to `search_path`. + Schema string + WorkUnitFactoryFunc func(kind string) workunit.WorkUnitFactory } @@ -169,6 +173,7 @@ func (s *JobRescuer) runOnce(ctx context.Context) (*rescuerRunOnceResult, error) Error: make([][]byte, 0, len(stuckJobs)), FinalizedAt: make([]time.Time, 0, len(stuckJobs)), ScheduledAt: make([]time.Time, 0, len(stuckJobs)), + Schema: s.Config.Schema, State: make([]string, 0, len(stuckJobs)), } @@ -247,6 +252,7 @@ func (s *JobRescuer) getStuckJobs(ctx context.Context) ([]*rivertype.JobRow, err return s.exec.JobGetStuck(ctx, &riverdriver.JobGetStuckParams{ Max: s.batchSize, + Schema: s.Config.Schema, StuckHorizon: stuckHorizon, }) } diff --git a/internal/maintenance/job_rescuer_test.go b/internal/maintenance/job_rescuer_test.go index 0390276d..cb7f66d3 100644 --- a/internal/maintenance/job_rescuer_test.go +++ b/internal/maintenance/job_rescuer_test.go @@ -136,7 +136,7 @@ func TestJobRescuer(t *testing.T) { t.Parallel() require := require.New(t) - cleaner, bundle := setup(t) + rescuer, bundle := setup(t) stuckToRetryJob1 := testfactory.Job(ctx, t, bundle.exec, &testfactory.JobOpts{Kind: ptrutil.Ptr(rescuerJobKind), State: ptrutil.Ptr(rivertype.JobStateRunning), AttemptedAt: ptrutil.Ptr(bundle.rescueHorizon.Add(-1 * time.Hour)), MaxAttempts: ptrutil.Ptr(5)}) stuckToRetryJob2 := testfactory.Job(ctx, t, bundle.exec, &testfactory.JobOpts{Kind: ptrutil.Ptr(rescuerJobKind), State: ptrutil.Ptr(rivertype.JobStateRunning), AttemptedAt: ptrutil.Ptr(bundle.rescueHorizon.Add(-1 * time.Minute)), MaxAttempts: ptrutil.Ptr(5)}) @@ -163,13 +163,13 @@ func TestJobRescuer(t *testing.T) { longTimeOutJob1 := testfactory.Job(ctx, t, bundle.exec, &testfactory.JobOpts{Kind: ptrutil.Ptr(rescuerJobKindLongTimeout), State: ptrutil.Ptr(rivertype.JobStateRunning), AttemptedAt: ptrutil.Ptr(bundle.rescueHorizon.Add(-1 * time.Minute)), MaxAttempts: ptrutil.Ptr(5)}) longTimeOutJob2 := testfactory.Job(ctx, t, bundle.exec, &testfactory.JobOpts{Kind: ptrutil.Ptr(rescuerJobKindLongTimeout), State: ptrutil.Ptr(rivertype.JobStateRunning), AttemptedAt: ptrutil.Ptr(bundle.rescueHorizon.Add(-6 * time.Minute)), MaxAttempts: ptrutil.Ptr(5)}) - require.NoError(cleaner.Start(ctx)) + require.NoError(rescuer.Start(ctx)) - cleaner.TestSignals.FetchedBatch.WaitOrTimeout() - cleaner.TestSignals.UpdatedBatch.WaitOrTimeout() + rescuer.TestSignals.FetchedBatch.WaitOrTimeout() + rescuer.TestSignals.UpdatedBatch.WaitOrTimeout() confirmRetried := func(jobBefore *rivertype.JobRow) { - jobAfter, err := bundle.exec.JobGetByID(ctx, jobBefore.ID) + jobAfter, err := bundle.exec.JobGetByID(ctx, &riverdriver.JobGetByIDParams{ID: jobBefore.ID, Schema: rescuer.Config.Schema}) require.NoError(err) require.Equal(rivertype.JobStateRetryable, jobAfter.State) } @@ -177,46 +177,47 @@ func TestJobRescuer(t *testing.T) { var err error confirmRetried(stuckToRetryJob1) confirmRetried(stuckToRetryJob2) - job3After, err := bundle.exec.JobGetByID(ctx, stuckToRetryJob3.ID) + + job3After, err := bundle.exec.JobGetByID(ctx, &riverdriver.JobGetByIDParams{ID: stuckToRetryJob3.ID, Schema: rescuer.Config.Schema}) require.NoError(err) require.Equal(stuckToRetryJob3.State, job3After.State) // not rescued - discardJob1After, err := bundle.exec.JobGetByID(ctx, stuckToDiscardJob1.ID) + discardJob1After, err := bundle.exec.JobGetByID(ctx, &riverdriver.JobGetByIDParams{ID: stuckToDiscardJob1.ID, Schema: rescuer.Config.Schema}) require.NoError(err) require.Equal(rivertype.JobStateDiscarded, discardJob1After.State) require.WithinDuration(time.Now(), *discardJob1After.FinalizedAt, 5*time.Second) require.Len(discardJob1After.Errors, 1) - discardJob2After, err := bundle.exec.JobGetByID(ctx, stuckToDiscardJob2.ID) + discardJob2After, err := bundle.exec.JobGetByID(ctx, &riverdriver.JobGetByIDParams{ID: stuckToDiscardJob2.ID, Schema: rescuer.Config.Schema}) require.NoError(err) require.Equal(rivertype.JobStateRunning, discardJob2After.State) require.Nil(discardJob2After.FinalizedAt) - cancelJob1After, err := bundle.exec.JobGetByID(ctx, stuckToCancelJob1.ID) + cancelJob1After, err := bundle.exec.JobGetByID(ctx, &riverdriver.JobGetByIDParams{ID: stuckToCancelJob1.ID, Schema: rescuer.Config.Schema}) require.NoError(err) require.Equal(rivertype.JobStateCancelled, cancelJob1After.State) require.WithinDuration(time.Now(), *cancelJob1After.FinalizedAt, 5*time.Second) require.Len(cancelJob1After.Errors, 1) - cancelJob2After, err := bundle.exec.JobGetByID(ctx, stuckToCancelJob2.ID) + cancelJob2After, err := bundle.exec.JobGetByID(ctx, &riverdriver.JobGetByIDParams{ID: stuckToCancelJob2.ID, Schema: rescuer.Config.Schema}) require.NoError(err) require.Equal(rivertype.JobStateRunning, cancelJob2After.State) require.Nil(cancelJob2After.FinalizedAt) - notRunningJob1After, err := bundle.exec.JobGetByID(ctx, notRunningJob1.ID) + notRunningJob1After, err := bundle.exec.JobGetByID(ctx, &riverdriver.JobGetByIDParams{ID: notRunningJob1.ID, Schema: rescuer.Config.Schema}) require.NoError(err) require.Equal(notRunningJob1.State, notRunningJob1After.State) - notRunningJob2After, err := bundle.exec.JobGetByID(ctx, notRunningJob2.ID) + notRunningJob2After, err := bundle.exec.JobGetByID(ctx, &riverdriver.JobGetByIDParams{ID: notRunningJob2.ID, Schema: rescuer.Config.Schema}) require.NoError(err) require.Equal(notRunningJob2.State, notRunningJob2After.State) - notRunningJob3After, err := bundle.exec.JobGetByID(ctx, notRunningJob3.ID) + notRunningJob3After, err := bundle.exec.JobGetByID(ctx, &riverdriver.JobGetByIDParams{ID: notRunningJob3.ID, Schema: rescuer.Config.Schema}) require.NoError(err) require.Equal(notRunningJob3.State, notRunningJob3After.State) - notTimedOutJob1After, err := bundle.exec.JobGetByID(ctx, longTimeOutJob1.ID) + notTimedOutJob1After, err := bundle.exec.JobGetByID(ctx, &riverdriver.JobGetByIDParams{ID: longTimeOutJob1.ID, Schema: rescuer.Config.Schema}) require.NoError(err) require.Equal(rivertype.JobStateRunning, notTimedOutJob1After.State) - notTimedOutJob2After, err := bundle.exec.JobGetByID(ctx, longTimeOutJob2.ID) + notTimedOutJob2After, err := bundle.exec.JobGetByID(ctx, &riverdriver.JobGetByIDParams{ID: longTimeOutJob2.ID, Schema: rescuer.Config.Schema}) require.NoError(err) require.Equal(rivertype.JobStateRetryable, notTimedOutJob2After.State) }) @@ -224,12 +225,12 @@ func TestJobRescuer(t *testing.T) { t.Run("RescuesInBatches", func(t *testing.T) { t.Parallel() - cleaner, bundle := setup(t) - cleaner.batchSize = 10 // reduced size for test speed + rescuer, bundle := setup(t) + rescuer.batchSize = 10 // reduced size for test speed // Add one to our chosen batch size to get one extra job and therefore // one extra batch, ensuring that we've tested working multiple. - numJobs := cleaner.batchSize + 1 + numJobs := rescuer.batchSize + 1 jobs := make([]*rivertype.JobRow, numJobs) @@ -238,16 +239,16 @@ func TestJobRescuer(t *testing.T) { jobs[i] = job } - require.NoError(t, cleaner.Start(ctx)) + require.NoError(t, rescuer.Start(ctx)) // See comment above. Exactly two batches are expected. - cleaner.TestSignals.FetchedBatch.WaitOrTimeout() - cleaner.TestSignals.UpdatedBatch.WaitOrTimeout() - cleaner.TestSignals.FetchedBatch.WaitOrTimeout() - cleaner.TestSignals.UpdatedBatch.WaitOrTimeout() // need to wait until after this for the conn to be free + rescuer.TestSignals.FetchedBatch.WaitOrTimeout() + rescuer.TestSignals.UpdatedBatch.WaitOrTimeout() + rescuer.TestSignals.FetchedBatch.WaitOrTimeout() + rescuer.TestSignals.UpdatedBatch.WaitOrTimeout() // need to wait until after this for the conn to be free for _, job := range jobs { - jobUpdated, err := bundle.exec.JobGetByID(ctx, job.ID) + jobUpdated, err := bundle.exec.JobGetByID(ctx, &riverdriver.JobGetByIDParams{ID: job.ID, Schema: rescuer.Config.Schema}) require.NoError(t, err) require.Equal(t, rivertype.JobStateRetryable, jobUpdated.State) } @@ -319,10 +320,10 @@ func TestJobRescuer(t *testing.T) { rescuer.TestSignals.FetchedBatch.WaitOrTimeout() rescuer.TestSignals.UpdatedBatch.WaitOrTimeout() - job1After, err := bundle.exec.JobGetByID(ctx, job1.ID) + job1After, err := bundle.exec.JobGetByID(ctx, &riverdriver.JobGetByIDParams{ID: job1.ID, Schema: rescuer.Config.Schema}) require.NoError(t, err) require.Equal(t, rivertype.JobStateDiscarded, job1After.State) - job2After, err := bundle.exec.JobGetByID(ctx, job2.ID) + job2After, err := bundle.exec.JobGetByID(ctx, &riverdriver.JobGetByIDParams{ID: job2.ID, Schema: rescuer.Config.Schema}) require.NoError(t, err) require.Equal(t, rivertype.JobStateDiscarded, job2After.State) }) diff --git a/internal/maintenance/job_scheduler.go b/internal/maintenance/job_scheduler.go index 39eff3b7..be643b87 100644 --- a/internal/maintenance/job_scheduler.go +++ b/internal/maintenance/job_scheduler.go @@ -52,6 +52,10 @@ type JobSchedulerConfig struct { // NotifyInsert is a function to call to emit notifications for queues // where jobs were scheduled. NotifyInsert NotifyInsertFunc + + // Schema where River tables are located. Empty string omits schema, causing + // Postgres to default to `search_path`. + Schema string } func (c *JobSchedulerConfig) mustValidate() *JobSchedulerConfig { @@ -155,8 +159,9 @@ func (s *JobScheduler) runOnce(ctx context.Context) (*schedulerRunOnceResult, er nowWithLookAhead := now.Add(s.config.Interval) scheduledJobResults, err := tx.JobSchedule(ctx, &riverdriver.JobScheduleParams{ - Max: s.config.Limit, - Now: nowWithLookAhead, + Max: s.config.Limit, + Now: nowWithLookAhead, + Schema: s.config.Schema, }) if err != nil { return 0, fmt.Errorf("error scheduling jobs: %w", err) diff --git a/internal/maintenance/job_scheduler_test.go b/internal/maintenance/job_scheduler_test.go index f82271d2..f04eec8f 100644 --- a/internal/maintenance/job_scheduler_test.go +++ b/internal/maintenance/job_scheduler_test.go @@ -65,23 +65,23 @@ func TestJobScheduler(t *testing.T) { return setup(t, riverpgxv5.New(nil).UnwrapExecutor(tx)) } - requireJobStateUnchanged := func(t *testing.T, exec riverdriver.Executor, job *rivertype.JobRow) *rivertype.JobRow { + requireJobStateUnchanged := func(t *testing.T, scheduler *JobScheduler, exec riverdriver.Executor, job *rivertype.JobRow) *rivertype.JobRow { t.Helper() - newJob, err := exec.JobGetByID(ctx, job.ID) + newJob, err := exec.JobGetByID(ctx, &riverdriver.JobGetByIDParams{ID: job.ID, Schema: scheduler.config.Schema}) require.NoError(t, err) require.Equal(t, job.State, newJob.State) return newJob } - requireJobStateAvailable := func(t *testing.T, exec riverdriver.Executor, job *rivertype.JobRow) *rivertype.JobRow { + requireJobStateAvailable := func(t *testing.T, scheduler *JobScheduler, exec riverdriver.Executor, job *rivertype.JobRow) *rivertype.JobRow { t.Helper() - newJob, err := exec.JobGetByID(ctx, job.ID) + newJob, err := exec.JobGetByID(ctx, &riverdriver.JobGetByIDParams{ID: job.ID, Schema: scheduler.config.Schema}) require.NoError(t, err) require.Equal(t, rivertype.JobStateAvailable, newJob.State) return newJob } - requireJobStateDiscardedWithMeta := func(t *testing.T, exec riverdriver.Executor, job *rivertype.JobRow) *rivertype.JobRow { + requireJobStateDiscardedWithMeta := func(t *testing.T, scheduler *JobScheduler, exec riverdriver.Executor, job *rivertype.JobRow) *rivertype.JobRow { t.Helper() - newJob, err := exec.JobGetByID(ctx, job.ID) + newJob, err := exec.JobGetByID(ctx, &riverdriver.JobGetByIDParams{ID: job.ID, Schema: scheduler.config.Schema}) require.NoError(t, err) require.Equal(t, rivertype.JobStateDiscarded, newJob.State) require.NotNil(t, newJob.FinalizedAt) @@ -134,20 +134,20 @@ func TestJobScheduler(t *testing.T) { scheduler.TestSignals.ScheduledBatch.WaitOrTimeout() - requireJobStateUnchanged(t, bundle.exec, job1) - requireJobStateUnchanged(t, bundle.exec, job2) - requireJobStateUnchanged(t, bundle.exec, job3) - requireJobStateUnchanged(t, bundle.exec, job4) - requireJobStateUnchanged(t, bundle.exec, job5) + requireJobStateUnchanged(t, scheduler, bundle.exec, job1) + requireJobStateUnchanged(t, scheduler, bundle.exec, job2) + requireJobStateUnchanged(t, scheduler, bundle.exec, job3) + requireJobStateUnchanged(t, scheduler, bundle.exec, job4) + requireJobStateUnchanged(t, scheduler, bundle.exec, job5) - requireJobStateAvailable(t, bundle.exec, scheduledJob1) - requireJobStateAvailable(t, bundle.exec, scheduledJob2) - requireJobStateAvailable(t, bundle.exec, scheduledJob3) - requireJobStateUnchanged(t, bundle.exec, scheduledJob4) // still scheduled + requireJobStateAvailable(t, scheduler, bundle.exec, scheduledJob1) + requireJobStateAvailable(t, scheduler, bundle.exec, scheduledJob2) + requireJobStateAvailable(t, scheduler, bundle.exec, scheduledJob3) + requireJobStateUnchanged(t, scheduler, bundle.exec, scheduledJob4) // still scheduled - requireJobStateAvailable(t, bundle.exec, retryableJob1) - requireJobStateAvailable(t, bundle.exec, retryableJob2) - requireJobStateUnchanged(t, bundle.exec, retryableJob3) // still retryable + requireJobStateAvailable(t, scheduler, bundle.exec, retryableJob1) + requireJobStateAvailable(t, scheduler, bundle.exec, retryableJob2) + requireJobStateUnchanged(t, scheduler, bundle.exec, retryableJob3) // still retryable }) t.Run("MovesUniqueKeyConflictingJobsToDiscarded", func(t *testing.T) { @@ -185,13 +185,13 @@ func TestJobScheduler(t *testing.T) { scheduler.TestSignals.ScheduledBatch.WaitOrTimeout() - requireJobStateAvailable(t, bundle.exec, retryableJob1) - requireJobStateAvailable(t, bundle.exec, retryableJob2) - requireJobStateDiscardedWithMeta(t, bundle.exec, retryableJob3) - requireJobStateDiscardedWithMeta(t, bundle.exec, retryableJob4) - requireJobStateDiscardedWithMeta(t, bundle.exec, retryableJob5) - requireJobStateDiscardedWithMeta(t, bundle.exec, retryableJob6) - requireJobStateDiscardedWithMeta(t, bundle.exec, retryableJob7) + requireJobStateAvailable(t, scheduler, bundle.exec, retryableJob1) + requireJobStateAvailable(t, scheduler, bundle.exec, retryableJob2) + requireJobStateDiscardedWithMeta(t, scheduler, bundle.exec, retryableJob3) + requireJobStateDiscardedWithMeta(t, scheduler, bundle.exec, retryableJob4) + requireJobStateDiscardedWithMeta(t, scheduler, bundle.exec, retryableJob5) + requireJobStateDiscardedWithMeta(t, scheduler, bundle.exec, retryableJob6) + requireJobStateDiscardedWithMeta(t, scheduler, bundle.exec, retryableJob7) }) t.Run("SchedulesInBatches", func(t *testing.T) { @@ -228,7 +228,7 @@ func TestJobScheduler(t *testing.T) { scheduler.TestSignals.ScheduledBatch.WaitOrTimeout() for _, job := range jobs { - requireJobStateAvailable(t, bundle.exec, job) + requireJobStateAvailable(t, scheduler, bundle.exec, job) } }) @@ -297,8 +297,8 @@ func TestJobScheduler(t *testing.T) { scheduler.TestSignals.ScheduledBatch.WaitOrTimeout() - requireJobStateAvailable(t, bundle.exec, job1) - requireJobStateAvailable(t, bundle.exec, job2) + requireJobStateAvailable(t, scheduler, bundle.exec, job1) + requireJobStateAvailable(t, scheduler, bundle.exec, job2) }) t.Run("TriggersNotificationsOnEachQueueWithNewlyAvailableJobs", func(t *testing.T) { diff --git a/internal/maintenance/periodic_job_enqueuer_test.go b/internal/maintenance/periodic_job_enqueuer_test.go index ac80c8cf..5a76a802 100644 --- a/internal/maintenance/periodic_job_enqueuer_test.go +++ b/internal/maintenance/periodic_job_enqueuer_test.go @@ -82,7 +82,10 @@ func TestPeriodicJobEnqueuer(t *testing.T) { finalInsertParams := sliceutil.Map(insertParams, func(params *rivertype.JobInsertParams) *riverdriver.JobInsertFastParams { return (*riverdriver.JobInsertFastParams)(params) }) - results, err := tx.JobInsertFastMany(ctx, finalInsertParams) + results, err := tx.JobInsertFastMany(ctx, &riverdriver.JobInsertFastManyParams{ + Jobs: finalInsertParams, + Schema: "", + }) if err != nil { return nil, err } @@ -109,12 +112,15 @@ func TestPeriodicJobEnqueuer(t *testing.T) { return svc, bundle } - requireNJobs := func(t *testing.T, exec riverdriver.Executor, kind string, n int) []*rivertype.JobRow { + requireNJobs := func(t *testing.T, exec riverdriver.Executor, kind string, expectedNumJobs int) []*rivertype.JobRow { t.Helper() - jobs, err := exec.JobGetByKindMany(ctx, []string{kind}) + jobs, err := exec.JobGetByKindMany(ctx, &riverdriver.JobGetByKindManyParams{ + Kind: []string{kind}, + Schema: "", + }) require.NoError(t, err) - require.Len(t, jobs, n, "Expected to find exactly %d job(s) of kind: %s, but found %d", n, kind, len(jobs)) + require.Len(t, jobs, expectedNumJobs, "Expected to find exactly %d job(s) of kind: %s, but found %d", expectedNumJobs, kind, len(jobs)) return jobs } diff --git a/internal/maintenance/queue_cleaner.go b/internal/maintenance/queue_cleaner.go index a15072a1..90b6e235 100644 --- a/internal/maintenance/queue_cleaner.go +++ b/internal/maintenance/queue_cleaner.go @@ -35,9 +35,14 @@ func (ts *QueueCleanerTestSignals) Init() { type QueueCleanerConfig struct { // Interval is the amount of time to wait between runs of the cleaner. Interval time.Duration + // RetentionPeriod is the amount of time to keep queues around before they're // removed. RetentionPeriod time.Duration + + // Schema where River tables are located. Empty string omits schema, causing + // Postgres to default to `search_path`. + Schema string } func (c *QueueCleanerConfig) mustValidate() *QueueCleanerConfig { @@ -134,6 +139,7 @@ func (s *QueueCleaner) runOnce(ctx context.Context) (*queueCleanerRunOnceResult, queuesDeleted, err := s.exec.QueueDeleteExpired(ctx, &riverdriver.QueueDeleteExpiredParams{ Max: s.batchSize, + Schema: s.Config.Schema, UpdatedAtHorizon: time.Now().Add(-s.Config.RetentionPeriod), }) if err != nil { diff --git a/internal/maintenance/queue_cleaner_test.go b/internal/maintenance/queue_cleaner_test.go index 43955a97..fff1bf93 100644 --- a/internal/maintenance/queue_cleaner_test.go +++ b/internal/maintenance/queue_cleaner_test.go @@ -90,16 +90,31 @@ func TestQueueCleaner(t *testing.T) { cleaner.TestSignals.DeletedBatch.WaitOrTimeout() var err error - _, err = bundle.exec.QueueGet(ctx, queue1.Name) + _, err = bundle.exec.QueueGet(ctx, &riverdriver.QueueGetParams{ + Name: queue1.Name, + Schema: cleaner.Config.Schema, + }) require.NoError(t, err) // still there - _, err = bundle.exec.QueueGet(ctx, queue2.Name) + _, err = bundle.exec.QueueGet(ctx, &riverdriver.QueueGetParams{ + Name: queue2.Name, + Schema: cleaner.Config.Schema, + }) require.NoError(t, err) // still there - _, err = bundle.exec.QueueGet(ctx, queue3.Name) + _, err = bundle.exec.QueueGet(ctx, &riverdriver.QueueGetParams{ + Name: queue3.Name, + Schema: cleaner.Config.Schema, + }) require.ErrorIs(t, err, rivertype.ErrNotFound) // still there - _, err = bundle.exec.QueueGet(ctx, queue4.Name) + _, err = bundle.exec.QueueGet(ctx, &riverdriver.QueueGetParams{ + Name: queue4.Name, + Schema: cleaner.Config.Schema, + }) require.ErrorIs(t, err, rivertype.ErrNotFound) // still there - _, err = bundle.exec.QueueGet(ctx, queue5.Name) + _, err = bundle.exec.QueueGet(ctx, &riverdriver.QueueGetParams{ + Name: queue5.Name, + Schema: cleaner.Config.Schema, + }) require.ErrorIs(t, err, rivertype.ErrNotFound) // still there }) @@ -130,7 +145,10 @@ func TestQueueCleaner(t *testing.T) { cleaner.TestSignals.DeletedBatch.WaitOrTimeout() for _, queue := range queues { - _, err := bundle.exec.QueueGet(ctx, queue.Name) + _, err := bundle.exec.QueueGet(ctx, &riverdriver.QueueGetParams{ + Name: queue.Name, + Schema: cleaner.Config.Schema, + }) require.ErrorIs(t, err, rivertype.ErrNotFound) } }) @@ -200,9 +218,15 @@ func TestQueueCleaner(t *testing.T) { cleaner.TestSignals.DeletedBatch.WaitOrTimeout() var err error - _, err = bundle.exec.QueueGet(ctx, queue1.Name) + _, err = bundle.exec.QueueGet(ctx, &riverdriver.QueueGetParams{ + Name: queue1.Name, + Schema: cleaner.Config.Schema, + }) require.ErrorIs(t, err, rivertype.ErrNotFound) - _, err = bundle.exec.QueueGet(ctx, queue2.Name) + _, err = bundle.exec.QueueGet(ctx, &riverdriver.QueueGetParams{ + Name: queue2.Name, + Schema: cleaner.Config.Schema, + }) require.ErrorIs(t, err, rivertype.ErrNotFound) }) } diff --git a/internal/maintenance/reindexer.go b/internal/maintenance/reindexer.go index e69b6d3f..5d37f430 100644 --- a/internal/maintenance/reindexer.go +++ b/internal/maintenance/reindexer.go @@ -37,6 +37,10 @@ type ReindexerConfig struct { // current time. ScheduleFunc func(time.Time) time.Time + // Schema where River tables are located. Empty string omits schema, causing + // Postgres to default to `search_path`. + Schema string + // Timeout is the amount of time to wait for a single reindex query to return. Timeout time.Duration } @@ -156,7 +160,12 @@ func (s *Reindexer) reindexOne(ctx context.Context, indexName string) error { ctx, cancel := context.WithTimeout(ctx, s.Config.Timeout) defer cancel() - _, err := s.exec.Exec(ctx, "REINDEX INDEX CONCURRENTLY "+indexName) + var maybeSchema string + if s.Config.Schema != "" { + maybeSchema = "." + s.Config.Schema + } + + _, err := s.exec.Exec(ctx, "REINDEX INDEX CONCURRENTLY "+maybeSchema+indexName) if err != nil { return err } diff --git a/internal/notifier/notifier_test.go b/internal/notifier/notifier_test.go index d44e8d4d..55b86d24 100644 --- a/internal/notifier/notifier_test.go +++ b/internal/notifier/notifier_test.go @@ -43,7 +43,8 @@ func TestNotifier(t *testing.T) { var ( dbPool = riverinternaltest.TestDB(ctx, t) driver = riverpgxv5.New(dbPool) - listener = driver.GetListener() + schema = "" // try to make tests schema-based rather than database-based in the future + listener = driver.GetListener(schema) ) notifier := New(riversharedtest.BaseServiceArchetype(t), listener) @@ -599,5 +600,5 @@ func sendNotification(ctx context.Context, t *testing.T, exec riverdriver.Execut t.Helper() t.Logf("Sending notification on %q: %s", topic, payload) - require.NoError(t, exec.NotifyMany(ctx, &riverdriver.NotifyManyParams{Payload: []string{payload}, Topic: topic})) + require.NoError(t, exec.NotifyMany(ctx, &riverdriver.NotifyManyParams{Payload: []string{payload}, Schema: "", Topic: topic})) } diff --git a/internal/riverinternaltest/riverdrivertest/riverdrivertest.go b/internal/riverinternaltest/riverdrivertest/riverdrivertest.go index 2fcff39a..749467ff 100644 --- a/internal/riverinternaltest/riverdrivertest/riverdrivertest.go +++ b/internal/riverinternaltest/riverdrivertest/riverdrivertest.go @@ -10,6 +10,8 @@ import ( "testing" "time" + "github.com/jackc/pgerrcode" + "github.com/jackc/pgx/v5/pgconn" "github.com/stretchr/testify/require" "github.com/tidwall/gjson" "golang.org/x/text/cases" @@ -83,12 +85,12 @@ func Exercise[TTx any](ctx context.Context, t *testing.T, { job := testfactory.Job(ctx, t, tx, &testfactory.JobOpts{}) - _, err = tx.JobGetByID(ctx, job.ID) + _, err := tx.JobGetByID(ctx, &riverdriver.JobGetByIDParams{ID: job.ID, Schema: ""}) require.NoError(t, err) require.NoError(t, tx.Rollback(ctx)) - _, err = exec.JobGetByID(ctx, job.ID) + _, err = exec.JobGetByID(ctx, &riverdriver.JobGetByIDParams{ID: job.ID, Schema: ""}) require.ErrorIs(t, err, rivertype.ErrNotFound) } }) @@ -115,16 +117,16 @@ func Exercise[TTx any](ctx context.Context, t *testing.T, { job2 := testfactory.Job(ctx, t, tx2, &testfactory.JobOpts{}) - _, err = tx2.JobGetByID(ctx, job2.ID) + _, err := tx2.JobGetByID(ctx, &riverdriver.JobGetByIDParams{ID: job2.ID, Schema: ""}) require.NoError(t, err) require.NoError(t, tx2.Rollback(ctx)) - _, err = tx1.JobGetByID(ctx, job2.ID) + _, err = tx1.JobGetByID(ctx, &riverdriver.JobGetByIDParams{ID: job2.ID, Schema: ""}) require.ErrorIs(t, err, rivertype.ErrNotFound) } - _, err = tx1.JobGetByID(ctx, job1.ID) + _, err = tx1.JobGetByID(ctx, &riverdriver.JobGetByIDParams{ID: job1.ID, Schema: ""}) require.NoError(t, err) } @@ -138,22 +140,22 @@ func Exercise[TTx any](ctx context.Context, t *testing.T, { job2 := testfactory.Job(ctx, t, tx2, &testfactory.JobOpts{}) - _, err = tx2.JobGetByID(ctx, job2.ID) + _, err = tx2.JobGetByID(ctx, &riverdriver.JobGetByIDParams{ID: job2.ID, Schema: ""}) require.NoError(t, err) require.NoError(t, tx2.Rollback(ctx)) - _, err = tx1.JobGetByID(ctx, job2.ID) + _, err = tx1.JobGetByID(ctx, &riverdriver.JobGetByIDParams{ID: job2.ID, Schema: ""}) require.ErrorIs(t, err, rivertype.ErrNotFound) } - _, err = tx1.JobGetByID(ctx, job1.ID) + _, err = tx1.JobGetByID(ctx, &riverdriver.JobGetByIDParams{ID: job1.ID, Schema: ""}) require.NoError(t, err) } require.NoError(t, tx1.Rollback(ctx)) - _, err = exec.JobGetByID(ctx, job1.ID) + _, err = exec.JobGetByID(ctx, &riverdriver.JobGetByIDParams{ID: job1.ID, Schema: ""}) require.ErrorIs(t, err, rivertype.ErrNotFound) } }) @@ -178,7 +180,7 @@ func Exercise[TTx any](ctx context.Context, t *testing.T, // Despite rollback being called after commit, the job is still // visible from the outer transaction. - _, err = tx1.JobGetByID(ctx, job.ID) + _, err = tx1.JobGetByID(ctx, &riverdriver.JobGetByIDParams{ID: job.ID, Schema: ""}) require.NoError(t, err) }) }) @@ -188,15 +190,24 @@ func Exercise[TTx any](ctx context.Context, t *testing.T, exec, _ := setup(ctx, t) - exists, err := exec.ColumnExists(ctx, "river_job", "id") + exists, err := exec.ColumnExists(ctx, &riverdriver.ColumnExistsParams{ + Column: "id", + Table: "river_job", + }) require.NoError(t, err) require.True(t, exists) - exists, err = exec.ColumnExists(ctx, "river_job", "does_not_exist") + exists, err = exec.ColumnExists(ctx, &riverdriver.ColumnExistsParams{ + Column: "does_not_exist", + Table: "river_job", + }) require.NoError(t, err) require.False(t, exists) - exists, err = exec.ColumnExists(ctx, "does_not_exist", "id") + exists, err = exec.ColumnExists(ctx, &riverdriver.ColumnExistsParams{ + Column: "id", + Table: "does_not_exist", + }) require.NoError(t, err) require.False(t, exists) @@ -212,7 +223,10 @@ func Exercise[TTx any](ctx context.Context, t *testing.T, _, err = exec.Exec(ctx, "CREATE TABLE river_job (another_id bigint)") require.NoError(t, err) - exists, err = exec.ColumnExists(ctx, "river_job", "id") + exists, err = exec.ColumnExists(ctx, &riverdriver.ColumnExistsParams{ + Column: "id", + Table: "river_job", + }) require.NoError(t, err) require.False(t, exists) }) @@ -334,21 +348,39 @@ func Exercise[TTx any](ctx context.Context, t *testing.T, t.Run("JobCountByState", func(t *testing.T) { t.Parallel() - exec, _ := setup(ctx, t) + t.Run("CountsJobsByState", func(t *testing.T) { + t.Parallel() - // Included because they're the queried state. - _ = testfactory.Job(ctx, t, exec, &testfactory.JobOpts{State: ptrutil.Ptr(rivertype.JobStateAvailable)}) - _ = testfactory.Job(ctx, t, exec, &testfactory.JobOpts{State: ptrutil.Ptr(rivertype.JobStateAvailable)}) + exec, _ := setup(ctx, t) - // Excluded because they're not. - finalizedAt := ptrutil.Ptr(time.Now()) - _ = testfactory.Job(ctx, t, exec, &testfactory.JobOpts{FinalizedAt: finalizedAt, State: ptrutil.Ptr(rivertype.JobStateCancelled)}) - _ = testfactory.Job(ctx, t, exec, &testfactory.JobOpts{FinalizedAt: finalizedAt, State: ptrutil.Ptr(rivertype.JobStateCompleted)}) - _ = testfactory.Job(ctx, t, exec, &testfactory.JobOpts{FinalizedAt: finalizedAt, State: ptrutil.Ptr(rivertype.JobStateDiscarded)}) + // Included because they're the queried state. + _ = testfactory.Job(ctx, t, exec, &testfactory.JobOpts{State: ptrutil.Ptr(rivertype.JobStateAvailable)}) + _ = testfactory.Job(ctx, t, exec, &testfactory.JobOpts{State: ptrutil.Ptr(rivertype.JobStateAvailable)}) - numJobs, err := exec.JobCountByState(ctx, rivertype.JobStateAvailable) - require.NoError(t, err) - require.Equal(t, 2, numJobs) + // Excluded because they're not. + finalizedAt := ptrutil.Ptr(time.Now()) + _ = testfactory.Job(ctx, t, exec, &testfactory.JobOpts{FinalizedAt: finalizedAt, State: ptrutil.Ptr(rivertype.JobStateCancelled)}) + _ = testfactory.Job(ctx, t, exec, &testfactory.JobOpts{FinalizedAt: finalizedAt, State: ptrutil.Ptr(rivertype.JobStateCompleted)}) + _ = testfactory.Job(ctx, t, exec, &testfactory.JobOpts{FinalizedAt: finalizedAt, State: ptrutil.Ptr(rivertype.JobStateDiscarded)}) + + numJobs, err := exec.JobCountByState(ctx, &riverdriver.JobCountByStateParams{ + State: rivertype.JobStateAvailable, + }) + require.NoError(t, err) + require.Equal(t, 2, numJobs) + }) + + t.Run("AlternateSchema", func(t *testing.T) { + t.Parallel() + + exec, _ := setup(ctx, t) + + _, err := exec.JobCountByState(ctx, &riverdriver.JobCountByStateParams{ + Schema: "custom_schema", + State: rivertype.JobStateAvailable, + }) + requireMissingRelation(t, err, "custom_schema.river_job") + }) }) t.Run("JobDelete", func(t *testing.T) { @@ -363,11 +395,13 @@ func Exercise[TTx any](ctx context.Context, t *testing.T, State: ptrutil.Ptr(rivertype.JobStateRunning), }) - jobAfter, err := exec.JobDelete(ctx, job.ID) + jobAfter, err := exec.JobDelete(ctx, &riverdriver.JobDeleteParams{ + ID: job.ID, + }) require.ErrorIs(t, err, rivertype.ErrJobRunning) require.Nil(t, jobAfter) - jobUpdated, err := exec.JobGetByID(ctx, job.ID) + jobUpdated, err := exec.JobGetByID(ctx, &riverdriver.JobGetByIDParams{ID: job.ID, Schema: ""}) require.NoError(t, err) require.Equal(t, rivertype.JobStateRunning, jobUpdated.State) }) @@ -405,13 +439,16 @@ func Exercise[TTx any](ctx context.Context, t *testing.T, State: &state, }) - jobAfter, err := exec.JobDelete(ctx, job.ID) + jobAfter, err := exec.JobDelete(ctx, &riverdriver.JobDeleteParams{ + ID: job.ID, + Schema: "", + }) require.NoError(t, err) require.NotNil(t, jobAfter) require.Equal(t, job.ID, jobAfter.ID) require.Equal(t, state, jobAfter.State) - _, err = exec.JobGetByID(ctx, job.ID) + _, err = exec.JobGetByID(ctx, &riverdriver.JobGetByIDParams{ID: job.ID, Schema: ""}) require.ErrorIs(t, err, rivertype.ErrNotFound) }) } @@ -421,10 +458,26 @@ func Exercise[TTx any](ctx context.Context, t *testing.T, exec, _ := setup(ctx, t) - jobAfter, err := exec.JobDelete(ctx, 1234567890) + jobAfter, err := exec.JobDelete(ctx, &riverdriver.JobDeleteParams{ + ID: 1234567890, + }) require.ErrorIs(t, err, rivertype.ErrNotFound) require.Nil(t, jobAfter) }) + + t.Run("AlternateSchema", func(t *testing.T) { + t.Parallel() + + exec, _ := setup(ctx, t) + + job := testfactory.Job(ctx, t, exec, &testfactory.JobOpts{}) + + _, err := exec.JobDelete(ctx, &riverdriver.JobDeleteParams{ + ID: job.ID, + Schema: "custom_schema", + }) + requireMissingRelation(t, err, "custom_schema.river_job") + }) }) t.Run("JobDeleteBefore", func(t *testing.T) { @@ -470,19 +523,19 @@ func Exercise[TTx any](ctx context.Context, t *testing.T, require.Equal(t, 1, numDeleted) // All deleted. - _, err = exec.JobGetByID(ctx, deletedJob1.ID) + _, err = exec.JobGetByID(ctx, &riverdriver.JobGetByIDParams{ID: deletedJob1.ID, Schema: ""}) require.ErrorIs(t, err, rivertype.ErrNotFound) - _, err = exec.JobGetByID(ctx, deletedJob2.ID) + _, err = exec.JobGetByID(ctx, &riverdriver.JobGetByIDParams{ID: deletedJob2.ID, Schema: ""}) require.ErrorIs(t, err, rivertype.ErrNotFound) - _, err = exec.JobGetByID(ctx, deletedJob3.ID) + _, err = exec.JobGetByID(ctx, &riverdriver.JobGetByIDParams{ID: deletedJob3.ID, Schema: ""}) require.ErrorIs(t, err, rivertype.ErrNotFound) // Not deleted - _, err = exec.JobGetByID(ctx, notDeletedJob1.ID) + _, err = exec.JobGetByID(ctx, &riverdriver.JobGetByIDParams{ID: notDeletedJob1.ID, Schema: ""}) require.NoError(t, err) - _, err = exec.JobGetByID(ctx, notDeletedJob2.ID) + _, err = exec.JobGetByID(ctx, &riverdriver.JobGetByIDParams{ID: notDeletedJob2.ID, Schema: ""}) require.NoError(t, err) - _, err = exec.JobGetByID(ctx, notDeletedJob3.ID) + _, err = exec.JobGetByID(ctx, &riverdriver.JobGetByIDParams{ID: notDeletedJob3.ID, Schema: ""}) require.NoError(t, err) }) @@ -645,7 +698,7 @@ func Exercise[TTx any](ctx context.Context, t *testing.T, job := testfactory.Job(ctx, t, exec, &testfactory.JobOpts{}) - fetchedJob, err := exec.JobGetByID(ctx, job.ID) + fetchedJob, err := exec.JobGetByID(ctx, &riverdriver.JobGetByIDParams{ID: job.ID, Schema: ""}) require.NoError(t, err) require.NotNil(t, fetchedJob) @@ -660,7 +713,7 @@ func Exercise[TTx any](ctx context.Context, t *testing.T, exec, _ := setup(ctx, t) - job, err := exec.JobGetByID(ctx, 0) + job, err := exec.JobGetByID(ctx, &riverdriver.JobGetByIDParams{ID: 0, Schema: ""}) require.Error(t, err) require.ErrorIs(t, err, rivertype.ErrNotFound) require.Nil(t, job) @@ -678,144 +731,14 @@ func Exercise[TTx any](ctx context.Context, t *testing.T, // Not returned. _ = testfactory.Job(ctx, t, exec, &testfactory.JobOpts{}) - jobs, err := exec.JobGetByIDMany(ctx, []int64{job1.ID, job2.ID}) + jobs, err := exec.JobGetByIDMany(ctx, &riverdriver.JobGetByIDManyParams{ + ID: []int64{job1.ID, job2.ID}, + }) require.NoError(t, err) require.Equal(t, []int64{job1.ID, job2.ID}, sliceutil.Map(jobs, func(j *rivertype.JobRow) int64 { return j.ID })) }) - t.Run("JobGetByKindAndUniqueProperties", func(t *testing.T) { - t.Parallel() - - const uniqueJobKind = "unique_job_kind" - - t.Run("NoOptions", func(t *testing.T) { - t.Parallel() - - exec, _ := setup(ctx, t) - - job := testfactory.Job(ctx, t, exec, &testfactory.JobOpts{Kind: ptrutil.Ptr(uniqueJobKind)}) - _ = testfactory.Job(ctx, t, exec, &testfactory.JobOpts{Kind: ptrutil.Ptr("other_kind")}) - - fetchedJob, err := exec.JobGetByKindAndUniqueProperties(ctx, &riverdriver.JobGetByKindAndUniquePropertiesParams{ - Kind: uniqueJobKind, - }) - require.NoError(t, err) - require.Equal(t, job.ID, fetchedJob.ID) - - _, err = exec.JobGetByKindAndUniqueProperties(ctx, &riverdriver.JobGetByKindAndUniquePropertiesParams{ - Kind: "does_not_exist", - }) - require.ErrorIs(t, err, rivertype.ErrNotFound) - }) - - t.Run("ByArgs", func(t *testing.T) { - t.Parallel() - - exec, _ := setup(ctx, t) - - args := []byte(`{"unique": "args"}`) - - job := testfactory.Job(ctx, t, exec, &testfactory.JobOpts{Kind: ptrutil.Ptr(uniqueJobKind), EncodedArgs: args}) - _ = testfactory.Job(ctx, t, exec, &testfactory.JobOpts{Kind: ptrutil.Ptr(uniqueJobKind), EncodedArgs: []byte(`{"other": "args"}`)}) - - fetchedJob, err := exec.JobGetByKindAndUniqueProperties(ctx, &riverdriver.JobGetByKindAndUniquePropertiesParams{ - Kind: uniqueJobKind, - ByArgs: true, - Args: args, - }) - require.NoError(t, err) - require.Equal(t, job.ID, fetchedJob.ID) - - _, err = exec.JobGetByKindAndUniqueProperties(ctx, &riverdriver.JobGetByKindAndUniquePropertiesParams{ - Kind: uniqueJobKind, - ByArgs: true, - Args: []byte(`{"does_not_exist": "args"}`), - }) - require.ErrorIs(t, err, rivertype.ErrNotFound) - }) - - t.Run("ByCreatedAt", func(t *testing.T) { - t.Parallel() - - exec, _ := setup(ctx, t) - - createdAt := time.Now().UTC() - - job := testfactory.Job(ctx, t, exec, &testfactory.JobOpts{Kind: ptrutil.Ptr(uniqueJobKind), CreatedAt: &createdAt}) - _ = testfactory.Job(ctx, t, exec, &testfactory.JobOpts{Kind: ptrutil.Ptr(uniqueJobKind), CreatedAt: ptrutil.Ptr(createdAt.Add(10 * time.Minute))}) - - fetchedJob, err := exec.JobGetByKindAndUniqueProperties(ctx, &riverdriver.JobGetByKindAndUniquePropertiesParams{ - Kind: uniqueJobKind, - ByCreatedAt: true, - CreatedAtBegin: createdAt.Add(-5 * time.Minute), - CreatedAtEnd: createdAt.Add(5 * time.Minute), - }) - require.NoError(t, err) - require.Equal(t, job.ID, fetchedJob.ID) - - _, err = exec.JobGetByKindAndUniqueProperties(ctx, &riverdriver.JobGetByKindAndUniquePropertiesParams{ - Kind: uniqueJobKind, - ByCreatedAt: true, - CreatedAtBegin: createdAt.Add(-15 * time.Minute), - CreatedAtEnd: createdAt.Add(-5 * time.Minute), - }) - require.ErrorIs(t, err, rivertype.ErrNotFound) - }) - - t.Run("ByQueue", func(t *testing.T) { - t.Parallel() - - exec, _ := setup(ctx, t) - - const queue = "unique_queue" - - job := testfactory.Job(ctx, t, exec, &testfactory.JobOpts{Kind: ptrutil.Ptr(uniqueJobKind), Queue: ptrutil.Ptr(queue)}) - _ = testfactory.Job(ctx, t, exec, &testfactory.JobOpts{Kind: ptrutil.Ptr(uniqueJobKind), Queue: ptrutil.Ptr("other_queue")}) - - fetchedJob, err := exec.JobGetByKindAndUniqueProperties(ctx, &riverdriver.JobGetByKindAndUniquePropertiesParams{ - Kind: uniqueJobKind, - ByQueue: true, - Queue: queue, - }) - require.NoError(t, err) - require.Equal(t, job.ID, fetchedJob.ID) - - _, err = exec.JobGetByKindAndUniqueProperties(ctx, &riverdriver.JobGetByKindAndUniquePropertiesParams{ - Kind: uniqueJobKind, - ByQueue: true, - Queue: "does_not_exist", - }) - require.ErrorIs(t, err, rivertype.ErrNotFound) - }) - - t.Run("ByState", func(t *testing.T) { - t.Parallel() - - exec, _ := setup(ctx, t) - - const state = rivertype.JobStateCompleted - - job := testfactory.Job(ctx, t, exec, &testfactory.JobOpts{Kind: ptrutil.Ptr(uniqueJobKind), FinalizedAt: ptrutil.Ptr(time.Now()), State: ptrutil.Ptr(state)}) - _ = testfactory.Job(ctx, t, exec, &testfactory.JobOpts{Kind: ptrutil.Ptr(uniqueJobKind), State: ptrutil.Ptr(rivertype.JobStateRetryable)}) - - fetchedJob, err := exec.JobGetByKindAndUniqueProperties(ctx, &riverdriver.JobGetByKindAndUniquePropertiesParams{ - Kind: uniqueJobKind, - ByState: true, - State: []string{string(state)}, - }) - require.NoError(t, err) - require.Equal(t, job.ID, fetchedJob.ID) - - _, err = exec.JobGetByKindAndUniqueProperties(ctx, &riverdriver.JobGetByKindAndUniquePropertiesParams{ - Kind: uniqueJobKind, - ByState: true, - State: []string{string(rivertype.JobStateScheduled)}, - }) - require.ErrorIs(t, err, rivertype.ErrNotFound) - }) - }) - t.Run("JobGetByKindMany", func(t *testing.T) { t.Parallel() @@ -827,7 +750,10 @@ func Exercise[TTx any](ctx context.Context, t *testing.T, // Not returned. _ = testfactory.Job(ctx, t, exec, &testfactory.JobOpts{Kind: ptrutil.Ptr("kind3")}) - jobs, err := exec.JobGetByKindMany(ctx, []string{job1.Kind, job2.Kind}) + jobs, err := exec.JobGetByKindMany(ctx, &riverdriver.JobGetByKindManyParams{ + Kind: []string{job1.Kind, job2.Kind}, + Schema: "", + }) require.NoError(t, err) require.Equal(t, []int64{job1.ID, job2.ID}, sliceutil.Map(jobs, func(j *rivertype.JobRow) int64 { return j.ID })) @@ -894,7 +820,9 @@ func Exercise[TTx any](ctx context.Context, t *testing.T, } } - resultRows, err := exec.JobInsertFastMany(ctx, insertParams) + resultRows, err := exec.JobInsertFastMany(ctx, &riverdriver.JobInsertFastManyParams{ + Jobs: insertParams, + }) require.NoError(t, err) require.Len(t, resultRows, len(insertParams)) @@ -943,11 +871,17 @@ func Exercise[TTx any](ctx context.Context, t *testing.T, } } - results, err := exec.JobInsertFastMany(ctx, insertParams) + results, err := exec.JobInsertFastMany(ctx, &riverdriver.JobInsertFastManyParams{ + Jobs: insertParams, + Schema: "", + }) require.NoError(t, err) require.Len(t, results, len(insertParams)) - jobsAfter, err := exec.JobGetByKindMany(ctx, []string{"test_kind"}) + jobsAfter, err := exec.JobGetByKindMany(ctx, &riverdriver.JobGetByKindManyParams{ + Kind: []string{"test_kind"}, + Schema: "", + }) require.NoError(t, err) require.Len(t, jobsAfter, len(insertParams)) for _, job := range jobsAfter { @@ -968,24 +902,31 @@ func Exercise[TTx any](ctx context.Context, t *testing.T, exec, _ := setup(ctx, t) uniqueKey := []byte{0x00, 0x01, 0x02} - results, err := exec.JobInsertFastMany(ctx, []*riverdriver.JobInsertFastParams{{ - EncodedArgs: []byte(`{"encoded": "args"}`), - Kind: "test_kind", - MaxAttempts: rivercommon.MaxAttemptsDefault, - Metadata: []byte(`{"meta": "data"}`), - Priority: rivercommon.PriorityDefault, - Queue: rivercommon.QueueDefault, - ScheduledAt: nil, // explicit nil - State: rivertype.JobStateAvailable, - Tags: []string{"tag"}, - UniqueKey: uniqueKey, - UniqueStates: 0xff, - }}) + results, err := exec.JobInsertFastMany(ctx, &riverdriver.JobInsertFastManyParams{ + Jobs: []*riverdriver.JobInsertFastParams{ + { + EncodedArgs: []byte(`{"encoded": "args"}`), + Kind: "test_kind", + MaxAttempts: rivercommon.MaxAttemptsDefault, + Metadata: []byte(`{"meta": "data"}`), + Priority: rivercommon.PriorityDefault, + Queue: rivercommon.QueueDefault, + ScheduledAt: nil, // explicit nil + State: rivertype.JobStateAvailable, + Tags: []string{"tag"}, + UniqueKey: uniqueKey, + UniqueStates: 0xff, + }, + }, + }) require.NoError(t, err) require.Len(t, results, 1) require.Equal(t, uniqueKey, results[0].Job.UniqueKey) - jobs, err := exec.JobGetByKindMany(ctx, []string{"test_kind"}) + jobs, err := exec.JobGetByKindMany(ctx, &riverdriver.JobGetByKindManyParams{ + Kind: []string{"test_kind"}, + Schema: "", + }) require.NoError(t, err) require.Equal(t, uniqueKey, jobs[0].UniqueKey) }) @@ -1021,11 +962,17 @@ func Exercise[TTx any](ctx context.Context, t *testing.T, } } - count, err := exec.JobInsertFastManyNoReturning(ctx, insertParams) + count, err := exec.JobInsertFastManyNoReturning(ctx, &riverdriver.JobInsertFastManyParams{ + Jobs: insertParams, + Schema: "", + }) require.NoError(t, err) require.Len(t, insertParams, count) - jobsAfter, err := exec.JobGetByKindMany(ctx, []string{"test_kind"}) + jobsAfter, err := exec.JobGetByKindMany(ctx, &riverdriver.JobGetByKindManyParams{ + Kind: []string{"test_kind"}, + Schema: "", + }) require.NoError(t, err) require.Len(t, jobsAfter, len(insertParams)) for i, job := range jobsAfter { @@ -1066,11 +1013,17 @@ func Exercise[TTx any](ctx context.Context, t *testing.T, } } - count, err := exec.JobInsertFastManyNoReturning(ctx, insertParams) + count, err := exec.JobInsertFastManyNoReturning(ctx, &riverdriver.JobInsertFastManyParams{ + Jobs: insertParams, + Schema: "", + }) require.NoError(t, err) require.Len(t, insertParams, count) - jobsAfter, err := exec.JobGetByKindMany(ctx, []string{"test_kind"}) + jobsAfter, err := exec.JobGetByKindMany(ctx, &riverdriver.JobGetByKindManyParams{ + Kind: []string{"test_kind"}, + Schema: "", + }) require.NoError(t, err) require.Len(t, jobsAfter, len(insertParams)) for _, job := range jobsAfter { @@ -1096,11 +1049,17 @@ func Exercise[TTx any](ctx context.Context, t *testing.T, } } - count, err := exec.JobInsertFastManyNoReturning(ctx, insertParams) + count, err := exec.JobInsertFastManyNoReturning(ctx, &riverdriver.JobInsertFastManyParams{ + Jobs: insertParams, + Schema: "", + }) require.NoError(t, err) require.Len(t, insertParams, count) - jobsAfter, err := exec.JobGetByKindMany(ctx, []string{"test_kind"}) + jobsAfter, err := exec.JobGetByKindMany(ctx, &riverdriver.JobGetByKindManyParams{ + Kind: []string{"test_kind"}, + Schema: "", + }) require.NoError(t, err) require.Len(t, jobsAfter, len(insertParams)) for _, job := range jobsAfter { @@ -1378,14 +1337,14 @@ func Exercise[TTx any](ctx context.Context, t *testing.T, }) require.NoError(t, err) - updatedJob1, err := exec.JobGetByID(ctx, job1.ID) + updatedJob1, err := exec.JobGetByID(ctx, &riverdriver.JobGetByIDParams{ID: job1.ID, Schema: ""}) require.NoError(t, err) require.Equal(t, "message1", updatedJob1.Errors[0].Error) require.Nil(t, updatedJob1.FinalizedAt) requireEqualTime(t, now, updatedJob1.ScheduledAt) require.Equal(t, rivertype.JobStateAvailable, updatedJob1.State) - updatedJob2, err := exec.JobGetByID(ctx, job2.ID) + updatedJob2, err := exec.JobGetByID(ctx, &riverdriver.JobGetByIDParams{ID: job2.ID, Schema: ""}) require.NoError(t, err) require.Equal(t, "message2", updatedJob2.Errors[0].Error) requireEqualTime(t, now, *updatedJob2.FinalizedAt) @@ -1405,12 +1364,15 @@ func Exercise[TTx any](ctx context.Context, t *testing.T, State: ptrutil.Ptr(rivertype.JobStateRunning), }) - jobAfter, err := exec.JobRetry(ctx, job.ID) + jobAfter, err := exec.JobRetry(ctx, &riverdriver.JobRetryParams{ + ID: job.ID, + Schema: "", + }) require.NoError(t, err) require.Equal(t, rivertype.JobStateRunning, jobAfter.State) require.WithinDuration(t, job.ScheduledAt, jobAfter.ScheduledAt, time.Microsecond) - jobUpdated, err := exec.JobGetByID(ctx, job.ID) + jobUpdated, err := exec.JobGetByID(ctx, &riverdriver.JobGetByIDParams{ID: job.ID, Schema: ""}) require.NoError(t, err) require.Equal(t, rivertype.JobStateRunning, jobUpdated.State) }) @@ -1448,12 +1410,15 @@ func Exercise[TTx any](ctx context.Context, t *testing.T, State: &state, }) - jobAfter, err := exec.JobRetry(ctx, job.ID) + jobAfter, err := exec.JobRetry(ctx, &riverdriver.JobRetryParams{ + ID: job.ID, + Schema: "", + }) require.NoError(t, err) require.Equal(t, rivertype.JobStateAvailable, jobAfter.State) require.WithinDuration(t, time.Now().UTC(), jobAfter.ScheduledAt, 100*time.Millisecond) - jobUpdated, err := exec.JobGetByID(ctx, job.ID) + jobUpdated, err := exec.JobGetByID(ctx, &riverdriver.JobGetByIDParams{ID: job.ID, Schema: ""}) require.NoError(t, err) require.Equal(t, rivertype.JobStateAvailable, jobUpdated.State) require.Nil(t, jobUpdated.FinalizedAt) @@ -1478,7 +1443,10 @@ func Exercise[TTx any](ctx context.Context, t *testing.T, State: ptrutil.Ptr(rivertype.JobStateCompleted), }) - jobAfter, err := exec.JobRetry(ctx, job.ID) + jobAfter, err := exec.JobRetry(ctx, &riverdriver.JobRetryParams{ + ID: job.ID, + Schema: "", + }) require.NoError(t, err) require.Equal(t, rivertype.JobStateAvailable, jobAfter.State) require.WithinDuration(t, now, jobAfter.ScheduledAt, 5*time.Second) @@ -1497,12 +1465,15 @@ func Exercise[TTx any](ctx context.Context, t *testing.T, ScheduledAt: ptrutil.Ptr(now.Add(-1 * time.Hour)), }) - jobAfter, err := exec.JobRetry(ctx, job.ID) + jobAfter, err := exec.JobRetry(ctx, &riverdriver.JobRetryParams{ + ID: job.ID, + Schema: "", + }) require.NoError(t, err) require.Equal(t, rivertype.JobStateAvailable, jobAfter.State) require.WithinDuration(t, job.ScheduledAt, jobAfter.ScheduledAt, time.Microsecond) - jobUpdated, err := exec.JobGetByID(ctx, job.ID) + jobUpdated, err := exec.JobGetByID(ctx, &riverdriver.JobGetByIDParams{ID: job.ID, Schema: ""}) require.NoError(t, err) require.Equal(t, rivertype.JobStateAvailable, jobUpdated.State) }) @@ -1512,7 +1483,10 @@ func Exercise[TTx any](ctx context.Context, t *testing.T, exec, _ := setup(ctx, t) - _, err := exec.JobRetry(ctx, 0) + _, err := exec.JobRetry(ctx, &riverdriver.JobRetryParams{ + ID: 0, + Schema: "", + }) require.Error(t, err) require.ErrorIs(t, err, rivertype.ErrNotFound) }) @@ -1559,15 +1533,15 @@ func Exercise[TTx any](ctx context.Context, t *testing.T, require.NoError(t, err) require.Len(t, result, 1) - updatedJob1, err := exec.JobGetByID(ctx, job1.ID) + updatedJob1, err := exec.JobGetByID(ctx, &riverdriver.JobGetByIDParams{ID: job1.ID, Schema: ""}) require.NoError(t, err) require.Equal(t, rivertype.JobStateAvailable, updatedJob1.State) - updatedJob2, err := exec.JobGetByID(ctx, job2.ID) + updatedJob2, err := exec.JobGetByID(ctx, &riverdriver.JobGetByIDParams{ID: job2.ID, Schema: ""}) require.NoError(t, err) require.Equal(t, rivertype.JobStateAvailable, updatedJob2.State) - updatedJob3, err := exec.JobGetByID(ctx, job3.ID) + updatedJob3, err := exec.JobGetByID(ctx, &riverdriver.JobGetByIDParams{ID: job3.ID, Schema: ""}) require.NoError(t, err) require.Equal(t, rivertype.JobStateAvailable, updatedJob3.State) }) @@ -1642,17 +1616,17 @@ func Exercise[TTx any](ctx context.Context, t *testing.T, require.NoError(t, err) require.Len(t, result, 3) - updatedJob1, err := exec.JobGetByID(ctx, job1.ID) + updatedJob1, err := exec.JobGetByID(ctx, &riverdriver.JobGetByIDParams{ID: job1.ID, Schema: ""}) require.NoError(t, err) require.Equal(t, rivertype.JobStateDiscarded, updatedJob1.State) require.Equal(t, "scheduler_discarded", gjson.GetBytes(updatedJob1.Metadata, "unique_key_conflict").String()) - updatedJob2, err := exec.JobGetByID(ctx, job2.ID) + updatedJob2, err := exec.JobGetByID(ctx, &riverdriver.JobGetByIDParams{ID: job2.ID, Schema: ""}) require.NoError(t, err) require.Equal(t, rivertype.JobStateAvailable, updatedJob2.State) require.False(t, gjson.GetBytes(updatedJob2.Metadata, "unique_key_conflict").Exists()) - updatedJob3, err := exec.JobGetByID(ctx, job3.ID) + updatedJob3, err := exec.JobGetByID(ctx, &riverdriver.JobGetByIDParams{ID: job3.ID, Schema: ""}) require.NoError(t, err) require.Equal(t, rivertype.JobStateAvailable, updatedJob3.State) require.False(t, gjson.GetBytes(updatedJob3.Metadata, "unique_key_conflict").Exists()) @@ -1696,12 +1670,12 @@ func Exercise[TTx any](ctx context.Context, t *testing.T, require.NoError(t, err) require.Len(t, result, 2) - updatedJob1, err := exec.JobGetByID(ctx, job1.ID) + updatedJob1, err := exec.JobGetByID(ctx, &riverdriver.JobGetByIDParams{ID: job1.ID, Schema: ""}) require.NoError(t, err) require.Equal(t, rivertype.JobStateAvailable, updatedJob1.State) require.False(t, gjson.GetBytes(updatedJob1.Metadata, "unique_key_conflict").Exists()) - updatedJob2, err := exec.JobGetByID(ctx, job2.ID) + updatedJob2, err := exec.JobGetByID(ctx, &riverdriver.JobGetByIDParams{ID: job2.ID, Schema: ""}) require.NoError(t, err) require.Equal(t, rivertype.JobStateDiscarded, updatedJob2.State) require.Equal(t, "scheduler_discarded", gjson.GetBytes(updatedJob2.Metadata, "unique_key_conflict").String()) @@ -1774,7 +1748,7 @@ func Exercise[TTx any](ctx context.Context, t *testing.T, require.Equal(t, rivertype.JobStateCompleted, jobAfter.State) require.WithinDuration(t, now, *jobAfter.FinalizedAt, time.Microsecond) - jobUpdated, err := exec.JobGetByID(ctx, job.ID) + jobUpdated, err := exec.JobGetByID(ctx, &riverdriver.JobGetByIDParams{ID: job.ID, Schema: ""}) require.NoError(t, err) require.Equal(t, rivertype.JobStateCompleted, jobUpdated.State) require.Equal(t, "unique-key", string(jobUpdated.UniqueKey)) @@ -1798,7 +1772,7 @@ func Exercise[TTx any](ctx context.Context, t *testing.T, require.Equal(t, rivertype.JobStateRetryable, jobAfter.State) require.Nil(t, jobAfter.FinalizedAt) - jobUpdated, err := exec.JobGetByID(ctx, job.ID) + jobUpdated, err := exec.JobGetByID(ctx, &riverdriver.JobGetByIDParams{ID: job.ID, Schema: ""}) require.NoError(t, err) require.Equal(t, rivertype.JobStateRetryable, jobUpdated.State) require.Equal(t, "unique-key", string(jobUpdated.UniqueKey)) @@ -1846,7 +1820,7 @@ func Exercise[TTx any](ctx context.Context, t *testing.T, require.Equal(t, rivertype.JobStateRetryable, jobAfter.State) require.WithinDuration(t, now, jobAfter.ScheduledAt, time.Microsecond) - jobUpdated, err := exec.JobGetByID(ctx, job.ID) + jobUpdated, err := exec.JobGetByID(ctx, &riverdriver.JobGetByIDParams{ID: job.ID, Schema: ""}) require.NoError(t, err) require.Equal(t, rivertype.JobStateRetryable, jobUpdated.State) require.Equal(t, "unique-key", string(jobUpdated.UniqueKey)) @@ -1877,7 +1851,7 @@ func Exercise[TTx any](ctx context.Context, t *testing.T, require.Equal(t, rivertype.JobStateRetryable, jobAfter.State) require.WithinDuration(t, job.ScheduledAt, jobAfter.ScheduledAt, time.Microsecond) - jobUpdated, err := exec.JobGetByID(ctx, job.ID) + jobUpdated, err := exec.JobGetByID(ctx, &riverdriver.JobGetByIDParams{ID: job.ID, Schema: ""}) require.NoError(t, err) require.Equal(t, rivertype.JobStateRetryable, jobUpdated.State) require.WithinDuration(t, job.ScheduledAt, jobAfter.ScheduledAt, time.Microsecond) @@ -1906,7 +1880,7 @@ func Exercise[TTx any](ctx context.Context, t *testing.T, require.Empty(t, jobAfter.Errors) require.Equal(t, job1.ScheduledAt, jobAfter.ScheduledAt) - jobUpdated, err := exec.JobGetByID(ctx, job1.ID) + jobUpdated, err := exec.JobGetByID(ctx, &riverdriver.JobGetByIDParams{ID: job1.ID, Schema: ""}) require.NoError(t, err) require.Equal(t, rivertype.JobStateRetryable, jobUpdated.State) require.JSONEq(t, `{"baz":"qux", "foo":"1", "output":{"a":"b"}}`, string(jobUpdated.Metadata)) @@ -1948,7 +1922,7 @@ func Exercise[TTx any](ctx context.Context, t *testing.T, require.Len(t, jobAfter.Errors, 1) require.Contains(t, jobAfter.Errors[0].Error, "fake error") - jobUpdated, err := exec.JobGetByID(ctx, job.ID) + jobUpdated, err := exec.JobGetByID(ctx, &riverdriver.JobGetByIDParams{ID: job.ID, Schema: ""}) require.NoError(t, err) require.Equal(t, rivertype.JobStateCancelled, jobUpdated.State) require.WithinDuration(t, job.ScheduledAt, jobAfter.ScheduledAt, time.Microsecond) @@ -1977,7 +1951,7 @@ func Exercise[TTx any](ctx context.Context, t *testing.T, require.Equal(t, rivertype.JobStateCancelled, jobAfter.State) require.WithinDuration(t, now, *jobAfter.FinalizedAt, time.Microsecond) - jobUpdated, err := exec.JobGetByID(ctx, job.ID) + jobUpdated, err := exec.JobGetByID(ctx, &riverdriver.JobGetByIDParams{ID: job.ID, Schema: ""}) require.NoError(t, err) require.Equal(t, rivertype.JobStateCancelled, jobUpdated.State) require.Equal(t, "unique-key", string(jobUpdated.UniqueKey)) @@ -2008,7 +1982,7 @@ func Exercise[TTx any](ctx context.Context, t *testing.T, require.Equal(t, "unique-key", string(jobAfter.UniqueKey)) require.Equal(t, rivertype.JobStates(), jobAfter.UniqueStates) - jobUpdated, err := exec.JobGetByID(ctx, job.ID) + jobUpdated, err := exec.JobGetByID(ctx, &riverdriver.JobGetByIDParams{ID: job.ID, Schema: ""}) require.NoError(t, err) require.Equal(t, rivertype.JobStateDiscarded, jobUpdated.State) }) @@ -2040,7 +2014,7 @@ func Exercise[TTx any](ctx context.Context, t *testing.T, require.Equal(t, rivertype.JobStateScheduled, jobAfter.State) require.WithinDuration(t, snoozeUntil, jobAfter.ScheduledAt, time.Microsecond) - jobUpdated, err := exec.JobGetByID(ctx, job.ID) + jobUpdated, err := exec.JobGetByID(ctx, &riverdriver.JobGetByIDParams{ID: job.ID, Schema: ""}) require.NoError(t, err) require.Equal(t, 4, jobUpdated.Attempt) require.Equal(t, job.MaxAttempts, jobUpdated.MaxAttempts) @@ -2073,7 +2047,7 @@ func Exercise[TTx any](ctx context.Context, t *testing.T, require.Equal(t, rivertype.JobStateScheduled, jobAfter.State) require.WithinDuration(t, snoozeUntil, jobAfter.ScheduledAt, time.Microsecond) - jobUpdated, err := exec.JobGetByID(ctx, job.ID) + jobUpdated, err := exec.JobGetByID(ctx, &riverdriver.JobGetByIDParams{ID: job.ID, Schema: ""}) require.NoError(t, err) require.Equal(t, 4, jobUpdated.Attempt) require.Equal(t, job.MaxAttempts, jobUpdated.MaxAttempts) @@ -2164,7 +2138,9 @@ func Exercise[TTx any](ctx context.Context, t *testing.T, now := time.Now().UTC() { - numDeleted, err := exec.LeaderDeleteExpired(ctx) + numDeleted, err := exec.LeaderDeleteExpired(ctx, &riverdriver.LeaderDeleteExpiredParams{ + Schema: "", + }) require.NoError(t, err) require.Zero(t, numDeleted) } @@ -2176,7 +2152,9 @@ func Exercise[TTx any](ctx context.Context, t *testing.T, }) { - numDeleted, err := exec.LeaderDeleteExpired(ctx) + numDeleted, err := exec.LeaderDeleteExpired(ctx, &riverdriver.LeaderDeleteExpiredParams{ + Schema: "", + }) require.NoError(t, err) require.Equal(t, 1, numDeleted) } @@ -2197,7 +2175,9 @@ func Exercise[TTx any](ctx context.Context, t *testing.T, require.NoError(t, err) require.True(t, elected) // won election - leader, err := exec.LeaderGetElectedLeader(ctx) + leader, err := exec.LeaderGetElectedLeader(ctx, &riverdriver.LeaderGetElectedLeaderParams{ + Schema: "", + }) require.NoError(t, err) require.WithinDuration(t, time.Now(), leader.ElectedAt, 100*time.Millisecond) require.WithinDuration(t, time.Now().Add(leaderTTL), leader.ExpiresAt, 100*time.Millisecond) @@ -2222,7 +2202,9 @@ func Exercise[TTx any](ctx context.Context, t *testing.T, // The time should not have changed because we specified that we were not // already elected, and the elect query is a no-op if there's already a // updatedLeader: - updatedLeader, err := exec.LeaderGetElectedLeader(ctx) + updatedLeader, err := exec.LeaderGetElectedLeader(ctx, &riverdriver.LeaderGetElectedLeaderParams{ + Schema: "", + }) require.NoError(t, err) require.Equal(t, leader.ExpiresAt, updatedLeader.ExpiresAt) }) @@ -2243,7 +2225,9 @@ func Exercise[TTx any](ctx context.Context, t *testing.T, require.NoError(t, err) require.True(t, elected) // won election - leader, err := exec.LeaderGetElectedLeader(ctx) + leader, err := exec.LeaderGetElectedLeader(ctx, &riverdriver.LeaderGetElectedLeaderParams{ + Schema: "", + }) require.NoError(t, err) require.WithinDuration(t, time.Now(), leader.ElectedAt, 100*time.Millisecond) require.WithinDuration(t, time.Now().Add(leaderTTL), leader.ExpiresAt, 100*time.Millisecond) @@ -2270,7 +2254,9 @@ func Exercise[TTx any](ctx context.Context, t *testing.T, // expires_at should be incremented because this is the same leader that won // previously and we specified that we're already elected: - updatedLeader, err := exec.LeaderGetElectedLeader(ctx) + updatedLeader, err := exec.LeaderGetElectedLeader(ctx, &riverdriver.LeaderGetElectedLeaderParams{ + Schema: "", + }) require.NoError(t, err) require.Greater(t, updatedLeader.ExpiresAt, leader.ExpiresAt) }) @@ -2300,7 +2286,9 @@ func Exercise[TTx any](ctx context.Context, t *testing.T, LeaderID: ptrutil.Ptr(clientID), }) - leader, err := exec.LeaderGetElectedLeader(ctx) + leader, err := exec.LeaderGetElectedLeader(ctx, &riverdriver.LeaderGetElectedLeaderParams{ + Schema: "", + }) require.NoError(t, err) require.WithinDuration(t, time.Now(), leader.ElectedAt, 500*time.Millisecond) require.WithinDuration(t, time.Now().Add(leaderTTL), leader.ExpiresAt, 500*time.Millisecond) @@ -2382,9 +2370,12 @@ func Exercise[TTx any](ctx context.Context, t *testing.T, _, err := exec.Exec(ctx, "ALTER TABLE river_migration DROP COLUMN line") require.NoError(t, err) - migrations, err := exec.MigrationDeleteAssumingMainMany(ctx, []int{ - migration1.Version, - migration2.Version, + migrations, err := exec.MigrationDeleteAssumingMainMany(ctx, &riverdriver.MigrationDeleteAssumingMainManyParams{ + Schema: "", + Versions: []int{ + migration1.Version, + migration2.Version, + }, }) require.NoError(t, err) require.Len(t, migrations, 2) @@ -2408,9 +2399,13 @@ func Exercise[TTx any](ctx context.Context, t *testing.T, migration1 := testfactory.Migration(ctx, t, exec, &testfactory.MigrationOpts{Line: ptrutil.Ptr("alternate")}) migration2 := testfactory.Migration(ctx, t, exec, &testfactory.MigrationOpts{Line: ptrutil.Ptr("alternate")}) - migrations, err := exec.MigrationDeleteByLineAndVersionMany(ctx, "alternate", []int{ - migration1.Version, - migration2.Version, + migrations, err := exec.MigrationDeleteByLineAndVersionMany(ctx, &riverdriver.MigrationDeleteByLineAndVersionManyParams{ + Line: "alternate", + Schema: "", + Versions: []int{ + migration1.Version, + migration2.Version, + }, }) require.NoError(t, err) require.Len(t, migrations, 2) @@ -2438,7 +2433,9 @@ func Exercise[TTx any](ctx context.Context, t *testing.T, _, err := exec.Exec(ctx, "ALTER TABLE river_migration DROP COLUMN line") require.NoError(t, err) - migrations, err := exec.MigrationGetAllAssumingMain(ctx) + migrations, err := exec.MigrationGetAllAssumingMain(ctx, &riverdriver.MigrationGetAllAssumingMainParams{ + Schema: "", + }) require.NoError(t, err) require.Len(t, migrations, 2) require.Equal(t, migration1.Version, migrations[0].Version) @@ -2464,7 +2461,10 @@ func Exercise[TTx any](ctx context.Context, t *testing.T, migration1 := testfactory.Migration(ctx, t, exec, &testfactory.MigrationOpts{Line: ptrutil.Ptr("alternate")}) migration2 := testfactory.Migration(ctx, t, exec, &testfactory.MigrationOpts{Line: ptrutil.Ptr("alternate")}) - migrations, err := exec.MigrationGetByLine(ctx, "alternate") + migrations, err := exec.MigrationGetByLine(ctx, &riverdriver.MigrationGetByLineParams{ + Line: "alternate", + Schema: "", + }) require.NoError(t, err) require.Len(t, migrations, 2) require.Equal(t, migration1.Version, migrations[0].Version) @@ -2484,7 +2484,10 @@ func Exercise[TTx any](ctx context.Context, t *testing.T, truncateMigrations(ctx, t, exec) - migrations, err := exec.MigrationInsertMany(ctx, "alternate", []int{1, 2}) + migrations, err := exec.MigrationInsertMany(ctx, &riverdriver.MigrationInsertManyParams{ + Line: "alternate", + Versions: []int{1, 2}, + }) require.NoError(t, err) require.Len(t, migrations, 2) require.Equal(t, "alternate", migrations[0].Line) @@ -2507,7 +2510,11 @@ func Exercise[TTx any](ctx context.Context, t *testing.T, _, err := exec.Exec(ctx, "ALTER TABLE river_migration DROP COLUMN line") require.NoError(t, err) - migrations, err := exec.MigrationInsertManyAssumingMain(ctx, []int{1, 2}) + migrations, err := exec.MigrationInsertManyAssumingMain(ctx, &riverdriver.MigrationInsertManyAssumingMainParams{ + Schema: "", + Versions: []int{1, 2}, + }) + require.NoError(t, err) require.Len(t, migrations, 2) require.Equal(t, riverdriver.MigrationLineMain, migrations[0].Line) @@ -2521,11 +2528,17 @@ func Exercise[TTx any](ctx context.Context, t *testing.T, exec, _ := setup(ctx, t) - exists, err := exec.TableExists(ctx, "river_job") + exists, err := exec.TableExists(ctx, &riverdriver.TableExistsParams{ + Schema: "", + Table: "river_job", + }) require.NoError(t, err) require.True(t, exists) - exists, err = exec.TableExists(ctx, "does_not_exist") + exists, err = exec.TableExists(ctx, &riverdriver.TableExistsParams{ + Schema: "", + Table: "does_not_exist", + }) require.NoError(t, err) require.False(t, exists) @@ -2536,7 +2549,10 @@ func Exercise[TTx any](ctx context.Context, t *testing.T, _, err = exec.Exec(ctx, "SET search_path = another_schema_123") require.NoError(t, err) - exists, err = exec.TableExists(ctx, "river_job") + exists, err = exec.TableExists(ctx, &riverdriver.TableExistsParams{ + Schema: "", + Table: "river_job", + }) require.NoError(t, err) require.False(t, exists) }) @@ -2684,7 +2700,10 @@ func Exercise[TTx any](ctx context.Context, t *testing.T, queue := testfactory.Queue(ctx, t, exec, &testfactory.QueueOpts{Metadata: []byte(`{"foo": "bar"}`)}) - queueFetched, err := exec.QueueGet(ctx, queue.Name) + queueFetched, err := exec.QueueGet(ctx, &riverdriver.QueueGetParams{ + Name: queue.Name, + Schema: "", + }) require.NoError(t, err) require.WithinDuration(t, queue.CreatedAt, queueFetched.CreatedAt, time.Millisecond) @@ -2693,7 +2712,10 @@ func Exercise[TTx any](ctx context.Context, t *testing.T, require.Nil(t, queueFetched.PausedAt) require.WithinDuration(t, queue.UpdatedAt, queueFetched.UpdatedAt, time.Millisecond) - queueFetched, err = exec.QueueGet(ctx, "nonexistent-queue") + queueFetched, err = exec.QueueGet(ctx, &riverdriver.QueueGetParams{ + Name: "nonexistent-queue", + Schema: "", + }) require.ErrorIs(t, err, rivertype.ErrNotFound) require.Nil(t, queueFetched) }) @@ -2716,7 +2738,10 @@ func Exercise[TTx any](ctx context.Context, t *testing.T, } } - queues, err := exec.QueueList(ctx, 10) + queues, err := exec.QueueList(ctx, &riverdriver.QueueListParams{ + Limit: 10, + Schema: "", + }) require.NoError(t, err) require.Empty(t, queues) @@ -2727,14 +2752,20 @@ func Exercise[TTx any](ctx context.Context, t *testing.T, queue2 := testfactory.Queue(ctx, t, exec, nil) queue3 := testfactory.Queue(ctx, t, exec, nil) - queues, err = exec.QueueList(ctx, 2) + queues, err = exec.QueueList(ctx, &riverdriver.QueueListParams{ + Limit: 2, + Schema: "", + }) require.NoError(t, err) require.Len(t, queues, 2) requireQueuesEqual(t, queue1, queues[0]) requireQueuesEqual(t, queue2, queues[1]) - queues, err = exec.QueueList(ctx, 3) + queues, err = exec.QueueList(ctx, &riverdriver.QueueListParams{ + Limit: 3, + Schema: "", + }) require.NoError(t, err) require.Len(t, queues, 3) @@ -2753,9 +2784,14 @@ func Exercise[TTx any](ctx context.Context, t *testing.T, PausedAt: ptrutil.Ptr(time.Now()), }) - require.NoError(t, exec.QueuePause(ctx, queue.Name)) - - queueFetched, err := exec.QueueGet(ctx, queue.Name) + require.NoError(t, exec.QueuePause(ctx, &riverdriver.QueuePauseParams{ + Name: queue.Name, + Schema: "", + })) + queueFetched, err := exec.QueueGet(ctx, &riverdriver.QueueGetParams{ + Name: queue.Name, + Schema: "", + }) require.NoError(t, err) require.NotNil(t, queueFetched.PausedAt) requireEqualTime(t, *queue.PausedAt, *queueFetched.PausedAt) // paused_at stays unchanged @@ -2770,9 +2806,15 @@ func Exercise[TTx any](ctx context.Context, t *testing.T, queue := testfactory.Queue(ctx, t, exec, nil) require.Nil(t, queue.PausedAt) - require.NoError(t, exec.QueuePause(ctx, queue.Name)) + require.NoError(t, exec.QueuePause(ctx, &riverdriver.QueuePauseParams{ + Name: queue.Name, + Schema: "", + })) - queueFetched, err := exec.QueueGet(ctx, queue.Name) + queueFetched, err := exec.QueueGet(ctx, &riverdriver.QueueGetParams{ + Name: queue.Name, + Schema: "", + }) require.NoError(t, err) require.NotNil(t, queueFetched.PausedAt) require.WithinDuration(t, time.Now(), *(queueFetched.PausedAt), 500*time.Millisecond) @@ -2783,7 +2825,10 @@ func Exercise[TTx any](ctx context.Context, t *testing.T, exec, _ := setup(ctx, t) - err := exec.QueuePause(ctx, "queue1") + err := exec.QueuePause(ctx, &riverdriver.QueuePauseParams{ + Name: "queue1", + Schema: "", + }) require.ErrorIs(t, err, rivertype.ErrNotFound) }) @@ -2797,16 +2842,25 @@ func Exercise[TTx any](ctx context.Context, t *testing.T, queue2 := testfactory.Queue(ctx, t, exec, nil) require.Nil(t, queue2.PausedAt) - require.NoError(t, exec.QueuePause(ctx, rivercommon.AllQueuesString)) + require.NoError(t, exec.QueuePause(ctx, &riverdriver.QueuePauseParams{ + Name: rivercommon.AllQueuesString, + Schema: "", + })) now := time.Now() - queue1Fetched, err := exec.QueueGet(ctx, queue1.Name) + queue1Fetched, err := exec.QueueGet(ctx, &riverdriver.QueueGetParams{ + Name: queue1.Name, + Schema: "", + }) require.NoError(t, err) require.NotNil(t, queue1Fetched.PausedAt) require.WithinDuration(t, now, *(queue1Fetched.PausedAt), 500*time.Millisecond) - queue2Fetched, err := exec.QueueGet(ctx, queue2.Name) + queue2Fetched, err := exec.QueueGet(ctx, &riverdriver.QueueGetParams{ + Name: queue2.Name, + Schema: "", + }) require.NoError(t, err) require.NotNil(t, queue2Fetched.PausedAt) require.WithinDuration(t, now, *(queue2Fetched.PausedAt), 500*time.Millisecond) @@ -2817,7 +2871,10 @@ func Exercise[TTx any](ctx context.Context, t *testing.T, exec, _ := setup(ctx, t) - require.NoError(t, exec.QueuePause(ctx, rivercommon.AllQueuesString)) + require.NoError(t, exec.QueuePause(ctx, &riverdriver.QueuePauseParams{ + Name: rivercommon.AllQueuesString, + Schema: "", + })) }) }) @@ -2833,9 +2890,15 @@ func Exercise[TTx any](ctx context.Context, t *testing.T, PausedAt: ptrutil.Ptr(time.Now()), }) - require.NoError(t, exec.QueueResume(ctx, queue.Name)) + require.NoError(t, exec.QueueResume(ctx, &riverdriver.QueueResumeParams{ + Name: queue.Name, + Schema: "", + })) - queueFetched, err := exec.QueueGet(ctx, queue.Name) + queueFetched, err := exec.QueueGet(ctx, &riverdriver.QueueGetParams{ + Name: queue.Name, + Schema: "", + }) require.NoError(t, err) require.Nil(t, queueFetched.PausedAt) }) @@ -2847,9 +2910,15 @@ func Exercise[TTx any](ctx context.Context, t *testing.T, queue := testfactory.Queue(ctx, t, exec, nil) - require.NoError(t, exec.QueueResume(ctx, queue.Name)) + require.NoError(t, exec.QueueResume(ctx, &riverdriver.QueueResumeParams{ + Name: queue.Name, + Schema: "", + })) - queueFetched, err := exec.QueueGet(ctx, queue.Name) + queueFetched, err := exec.QueueGet(ctx, &riverdriver.QueueGetParams{ + Name: queue.Name, + Schema: "", + }) require.NoError(t, err) require.Nil(t, queueFetched.PausedAt) requireEqualTime(t, queue.UpdatedAt, queueFetched.UpdatedAt) // updated_at stays unchanged @@ -2860,7 +2929,10 @@ func Exercise[TTx any](ctx context.Context, t *testing.T, exec, _ := setup(ctx, t) - err := exec.QueueResume(ctx, "queue1") + err := exec.QueueResume(ctx, &riverdriver.QueueResumeParams{ + Name: "queue1", + Schema: "", + }) require.ErrorIs(t, err, rivertype.ErrNotFound) }) @@ -2874,14 +2946,26 @@ func Exercise[TTx any](ctx context.Context, t *testing.T, queue2 := testfactory.Queue(ctx, t, exec, nil) require.Nil(t, queue2.PausedAt) - require.NoError(t, exec.QueuePause(ctx, rivercommon.AllQueuesString)) - require.NoError(t, exec.QueueResume(ctx, rivercommon.AllQueuesString)) - - queue1Fetched, err := exec.QueueGet(ctx, queue1.Name) + require.NoError(t, exec.QueuePause(ctx, &riverdriver.QueuePauseParams{ + Name: rivercommon.AllQueuesString, + Schema: "", + })) + require.NoError(t, exec.QueueResume(ctx, &riverdriver.QueueResumeParams{ + Name: rivercommon.AllQueuesString, + Schema: "", + })) + + queue1Fetched, err := exec.QueueGet(ctx, &riverdriver.QueueGetParams{ + Name: queue1.Name, + Schema: "", + }) require.NoError(t, err) require.Nil(t, queue1Fetched.PausedAt) - queue2Fetched, err := exec.QueueGet(ctx, queue2.Name) + queue2Fetched, err := exec.QueueGet(ctx, &riverdriver.QueueGetParams{ + Name: queue2.Name, + Schema: "", + }) require.NoError(t, err) require.Nil(t, queue2Fetched.PausedAt) }) @@ -2891,7 +2975,10 @@ func Exercise[TTx any](ctx context.Context, t *testing.T, exec, _ := setup(ctx, t) - require.NoError(t, exec.QueueResume(ctx, rivercommon.AllQueuesString)) + require.NoError(t, exec.QueueResume(ctx, &riverdriver.QueueResumeParams{ + Name: rivercommon.AllQueuesString, + Schema: "", + })) }) }) @@ -2941,9 +3028,10 @@ type testListenerBundle[TTx any] struct { func setupListener[TTx any](ctx context.Context, t *testing.T, getDriverWithPool func(ctx context.Context, t *testing.T) riverdriver.Driver[TTx]) (riverdriver.Listener, *testListenerBundle[TTx]) { t.Helper() - driver := getDriverWithPool(ctx, t) - - listener := driver.GetListener() + var ( + driver = getDriverWithPool(ctx, t) + listener = driver.GetListener("") + ) return listener, &testListenerBundle[TTx]{ driver: driver, @@ -3004,8 +3092,8 @@ func exerciseListener[TTx any](ctx context.Context, t *testing.T, driverWithPool require.NoError(t, listener.Ping(ctx)) // still alive { - require.NoError(t, bundle.exec.NotifyMany(ctx, &riverdriver.NotifyManyParams{Topic: "topic1", Payload: []string{"payload1_1"}})) - require.NoError(t, bundle.exec.NotifyMany(ctx, &riverdriver.NotifyManyParams{Topic: "topic2", Payload: []string{"payload2_1"}})) + require.NoError(t, bundle.exec.NotifyMany(ctx, &riverdriver.NotifyManyParams{Topic: "topic1", Payload: []string{"payload1_1"}, Schema: ""})) + require.NoError(t, bundle.exec.NotifyMany(ctx, &riverdriver.NotifyManyParams{Topic: "topic2", Payload: []string{"payload2_1"}, Schema: ""})) notification := waitForNotification(ctx, t, listener) require.Equal(t, &riverdriver.Notification{Topic: "topic1", Payload: "payload1_1"}, notification) @@ -3016,8 +3104,8 @@ func exerciseListener[TTx any](ctx context.Context, t *testing.T, driverWithPool require.NoError(t, listener.Unlisten(ctx, "topic2")) { - require.NoError(t, bundle.exec.NotifyMany(ctx, &riverdriver.NotifyManyParams{Topic: "topic1", Payload: []string{"payload1_2"}})) - require.NoError(t, bundle.exec.NotifyMany(ctx, &riverdriver.NotifyManyParams{Topic: "topic2", Payload: []string{"payload2_2"}})) + require.NoError(t, bundle.exec.NotifyMany(ctx, &riverdriver.NotifyManyParams{Topic: "topic1", Payload: []string{"payload1_2"}, Schema: ""})) + require.NoError(t, bundle.exec.NotifyMany(ctx, &riverdriver.NotifyManyParams{Topic: "topic2", Payload: []string{"payload2_2"}, Schema: ""})) notification := waitForNotification(ctx, t, listener) require.Equal(t, &riverdriver.Notification{Topic: "topic1", Payload: "payload1_2"}, notification) @@ -3030,6 +3118,25 @@ func exerciseListener[TTx any](ctx context.Context, t *testing.T, driverWithPool require.NoError(t, listener.Close(ctx)) }) + t.Run("SchemaFromParameter", func(t *testing.T) { + t.Parallel() + + var ( + driver = driverWithPool(ctx, t) + listener = driver.GetListener("my_custom_schema") + ) + + require.Equal(t, "my_custom_schema", listener.Schema()) + }) + + t.Run("SchemaFromSearchPath", func(t *testing.T) { + t.Parallel() + + listener, _ := setupListener(ctx, t, driverWithPool) + connectListener(ctx, t, listener) + require.Equal(t, "public", listener.Schema()) + }) + t.Run("TransactionGated", func(t *testing.T) { t.Parallel() @@ -3042,7 +3149,7 @@ func exerciseListener[TTx any](ctx context.Context, t *testing.T, driverWithPool tx, err := bundle.exec.Begin(ctx) require.NoError(t, err) - require.NoError(t, tx.NotifyMany(ctx, &riverdriver.NotifyManyParams{Topic: "topic1", Payload: []string{"payload1"}})) + require.NoError(t, tx.NotifyMany(ctx, &riverdriver.NotifyManyParams{Topic: "topic1", Payload: []string{"payload1"}, Schema: ""})) // No notification because the transaction hasn't committed yet. requireNoNotification(ctx, t, listener) @@ -3088,3 +3195,12 @@ func requireEqualTime(t *testing.T, expected, actual time.Time) { actual.Format(rfc3339Micro), ) } + +func requireMissingRelation(t *testing.T, err error, missingRelation string) { + t.Helper() + + var pgErr *pgconn.PgError + require.ErrorAs(t, err, &pgErr) + require.Equal(t, pgerrcode.UndefinedTable, pgErr.Code) + require.Equal(t, fmt.Sprintf(`relation "%s" does not exist`, missingRelation), pgErr.Message) +} diff --git a/job_complete_tx_test.go b/job_complete_tx_test.go index b1fbfe1b..692f1045 100644 --- a/job_complete_tx_test.go +++ b/job_complete_tx_test.go @@ -66,7 +66,10 @@ func TestJobCompleteTx(t *testing.T) { require.Equal(t, rivertype.JobStateCompleted, completedJob.State) require.WithinDuration(t, time.Now(), *completedJob.FinalizedAt, 2*time.Second) - updatedJob, err := bundle.exec.JobGetByID(ctx, job.ID) + updatedJob, err := bundle.exec.JobGetByID(ctx, &riverdriver.JobGetByIDParams{ + ID: job.ID, + Schema: "", + }) require.NoError(t, err) require.Equal(t, rivertype.JobStateCompleted, updatedJob.State) }) @@ -88,7 +91,10 @@ func TestJobCompleteTx(t *testing.T) { require.NoError(t, err) require.Equal(t, rivertype.JobStateCompleted, completedJob.State) - updatedJob, err := bundle.exec.JobGetByID(ctx, job.ID) + updatedJob, err := bundle.exec.JobGetByID(ctx, &riverdriver.JobGetByIDParams{ + ID: job.ID, + Schema: "", + }) require.NoError(t, err) require.Equal(t, rivertype.JobStateCompleted, updatedJob.State) }) @@ -130,7 +136,7 @@ func TestJobCompleteTx(t *testing.T) { }) // delete the job - _, err := bundle.exec.JobDelete(ctx, job.ID) + _, err := bundle.exec.JobDelete(ctx, &riverdriver.JobDeleteParams{ID: job.ID}) require.NoError(t, err) // fake the job's state to work around the check: diff --git a/producer.go b/producer.go index 27b73a5f..26e2ce9e 100644 --- a/producer.go +++ b/producer.go @@ -103,6 +103,7 @@ type producerConfig struct { QueueReportInterval time.Duration RetryPolicy ClientRetryPolicy SchedulerInterval time.Duration + Schema string StaleProducerRetentionPeriod time.Duration Workers *Workers } @@ -306,6 +307,7 @@ func (p *producer) StartWorkContext(fetchCtx, workCtx context.Context) error { ProducerID: id, Queue: p.config.Queue, QueueMetadata: initialMetadata, + Schema: p.config.Schema, }) if err != nil { stopped() @@ -684,6 +686,7 @@ func (p *producer) dispatchWork(workCtx context.Context, count int, fetchResultC Max: count, Queue: p.config.Queue, ProducerID: p.id.Load(), + Schema: p.config.Schema, }) if err != nil { p.Logger.Error(p.Name+": Error fetching jobs", slog.String("err", err.Error()), slog.String("queue", p.config.Queue)) @@ -749,6 +752,7 @@ func (p *producer) startNewExecutors(workCtx context.Context, jobs []*rivertype. InformProducerDoneFunc: p.handleWorkerDone, JobRow: job, SchedulerInterval: p.config.SchedulerInterval, + Schema: p.config.Schema, WorkUnit: workUnit, }) p.addActiveJob(job.ID, executor) @@ -840,7 +844,10 @@ func (p *producer) fetchQueueSettings(ctx context.Context) (*rivertype.Queue, er ctx, cancel := context.WithTimeout(ctx, 10*time.Second) defer cancel() - return p.exec.QueueGet(ctx, p.config.Queue) + return p.exec.QueueGet(ctx, &riverdriver.QueueGetParams{ + Name: p.config.Queue, + Schema: p.config.Schema, + }) } func (p *producer) reportProducerStatusLoop(ctx context.Context, wg *sync.WaitGroup) { @@ -866,6 +873,7 @@ func (p *producer) reportProducerStatusOnce(ctx context.Context) { err := p.pilot.ProducerKeepAlive(ctx, p.exec, &riverdriver.ProducerKeepAliveParams{ ID: p.id.Load(), QueueName: p.config.Queue, + Schema: p.config.Schema, StaleUpdatedAtHorizon: p.Time.NowUTC().Add(-p.config.StaleProducerRetentionPeriod), }) if err != nil && errors.Is(context.Cause(ctx), startstop.ErrStop) { diff --git a/producer_test.go b/producer_test.go index 36bf2814..af981c48 100644 --- a/producer_test.go +++ b/producer_test.go @@ -57,7 +57,8 @@ func Test_Producer_CanSafelyCompleteJobsWhileFetchingNewOnes(t *testing.T) { config := newTestConfig(t, nil) dbDriver := riverpgxv5.New(dbPool) exec := dbDriver.GetExecutor() - listener := dbDriver.GetListener() + schema := "" // try to make tests schema-based rather than database-based in the future + listener := dbDriver.GetListener(schema) pilot := &riverpilot.StandardPilot{} subscribeCh := make(chan []jobcompleter.CompleterJobUpdated, 100) @@ -105,6 +106,7 @@ func Test_Producer_CanSafelyCompleteJobsWhileFetchingNewOnes(t *testing.T) { QueueReportInterval: queueReportIntervalDefault, RetryPolicy: &DefaultClientRetryPolicy{}, SchedulerInterval: maintenance.JobSchedulerIntervalDefault, + Schema: schema, StaleProducerRetentionPeriod: time.Minute, Workers: workers, }) @@ -136,7 +138,10 @@ func Test_Producer_CanSafelyCompleteJobsWhileFetchingNewOnes(t *testing.T) { } }() - _, err := exec.JobInsertFastMany(ctx, params) + _, err := exec.JobInsertFastMany(ctx, &riverdriver.JobInsertFastManyParams{ + Jobs: params, + Schema: producer.config.Schema, + }) require.NoError(err) require.NoError(producer.StartWorkContext(ctx, ctx)) @@ -162,6 +167,7 @@ func TestProducer_PollOnly(t *testing.T) { driver = riverpgxv5.New(nil) pilot = &riverpilot.StandardPilot{} queueName = fmt.Sprintf("test-producer-poll-only-%05d", randutil.IntBetween(1, 100_000)) + schema = "" // try to make tests schema-based rather than database-based in the future tx = riverinternaltest.TestTx(ctx, t) ) @@ -197,6 +203,7 @@ func TestProducer_PollOnly(t *testing.T) { QueueReportInterval: queueReportIntervalDefault, RetryPolicy: &DefaultClientRetryPolicy{}, SchedulerInterval: riverinternaltest.SchedulerShortInterval, + Schema: schema, StaleProducerRetentionPeriod: time.Minute, Workers: NewWorkers(), }), jobUpdates @@ -215,7 +222,8 @@ func TestProducer_WithNotifier(t *testing.T) { driver = riverpgxv5.New(dbPool) exec = driver.GetExecutor() jobUpdates = make(chan []jobcompleter.CompleterJobUpdated, 10) - listener = driver.GetListener() + schema = "" // try to make tests schema-based rather than database-based in the future + listener = driver.GetListener(schema) pilot = &riverpilot.StandardPilot{} queueName = fmt.Sprintf("test-producer-with-notifier-%05d", randutil.IntBetween(1, 100_000)) ) @@ -249,6 +257,7 @@ func TestProducer_WithNotifier(t *testing.T) { QueueReportInterval: queueReportIntervalDefault, RetryPolicy: &DefaultClientRetryPolicy{}, SchedulerInterval: riverinternaltest.SchedulerShortInterval, + Schema: schema, StaleProducerRetentionPeriod: time.Minute, Workers: NewWorkers(), }), jobUpdates @@ -301,7 +310,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) { + mustInsert := func(ctx context.Context, t *testing.T, producer *producer, bundle *testBundle, args JobArgs) { t.Helper() insertParams, err := insertParamsFromConfigArgsAndOptions(bundle.archetype, bundle.config, args, &InsertOpts{ @@ -320,7 +329,10 @@ func testProducer(t *testing.T, makeProducer func(ctx context.Context, t *testin insertParams.ScheduledAt = &bundle.timeBeforeStart } - _, err = bundle.exec.JobInsertFastMany(ctx, []*riverdriver.JobInsertFastParams{(*riverdriver.JobInsertFastParams)(insertParams)}) + _, err = bundle.exec.JobInsertFastMany(ctx, &riverdriver.JobInsertFastManyParams{ + Jobs: []*riverdriver.JobInsertFastParams{(*riverdriver.JobInsertFastParams)(insertParams)}, + Schema: producer.config.Schema, + }) require.NoError(t, err) } @@ -345,7 +357,7 @@ func testProducer(t *testing.T, makeProducer func(ctx context.Context, t *testin producer, bundle := setup(t) AddWorker(bundle.workers, &noOpWorker{}) - mustInsert(ctx, t, bundle, &noOpArgs{}) + mustInsert(ctx, t, producer, bundle, &noOpArgs{}) startProducer(t, ctx, ctx, producer) @@ -362,7 +374,10 @@ func testProducer(t *testing.T, makeProducer func(ctx context.Context, t *testin now := time.Now().UTC() startProducer(t, ctx, ctx, producer) - queue, err := bundle.exec.QueueGet(ctx, producer.config.Queue) + queue, err := bundle.exec.QueueGet(ctx, &riverdriver.QueueGetParams{ + Name: producer.config.Queue, + Schema: producer.config.Schema, + }) require.NoError(t, err) require.WithinDuration(t, now, queue.CreatedAt, 2*time.Second) require.Equal(t, []byte("{}"), queue.Metadata) @@ -380,8 +395,8 @@ func testProducer(t *testing.T, makeProducer func(ctx context.Context, t *testin producer, bundle := setup(t) AddWorker(bundle.workers, &noOpWorker{}) - mustInsert(ctx, t, bundle, &noOpArgs{}) - mustInsert(ctx, t, bundle, &callbackArgs{}) // not registered + mustInsert(ctx, t, producer, bundle, &noOpArgs{}) + mustInsert(ctx, t, producer, bundle, &callbackArgs{}) // not registered startProducer(t, ctx, ctx, producer) @@ -430,7 +445,7 @@ func testProducer(t *testing.T, makeProducer func(ctx context.Context, t *testin workCtx, workCancel := context.WithCancel(ctx) defer workCancel() - mustInsert(ctx, t, bundle, &JobArgs{}) + mustInsert(ctx, t, producer, bundle, &JobArgs{}) startProducer(t, ctx, workCtx, producer) @@ -466,7 +481,7 @@ func testProducer(t *testing.T, makeProducer func(ctx context.Context, t *testin })) for range numJobs { - mustInsert(ctx, t, bundle, &JobArgs{}) + mustInsert(ctx, t, producer, bundle, &JobArgs{}) } startProducer(t, ctx, ctx, producer) @@ -474,7 +489,10 @@ func testProducer(t *testing.T, makeProducer func(ctx context.Context, t *testin producer.testSignals.StartedExecutors.WaitOrTimeout() // Jobs are still paused as we fetch updated job states. - updatedJobs, err := bundle.exec.JobGetByKindMany(ctx, []string{(&JobArgs{}).Kind()}) + updatedJobs, err := bundle.exec.JobGetByKindMany(ctx, &riverdriver.JobGetByKindManyParams{ + Kind: []string{(&JobArgs{}).Kind()}, + Schema: producer.config.Schema, + }) require.NoError(t, err) jobStateCounts := make(map[rivertype.JobState]int) @@ -511,7 +529,7 @@ func testProducer(t *testing.T, makeProducer func(ctx context.Context, t *testin PausedAt: ptrutil.Ptr(time.Now()), }) - mustInsert(ctx, t, bundle, &noOpArgs{}) + mustInsert(ctx, t, producer, bundle, &noOpArgs{}) startProducer(t, ctx, ctx, producer) @@ -530,7 +548,7 @@ func testProducer(t *testing.T, makeProducer func(ctx context.Context, t *testin producer.config.QueuePollInterval = 50 * time.Millisecond AddWorker(bundle.workers, &noOpWorker{}) - mustInsert(ctx, t, bundle, &noOpArgs{}) + mustInsert(ctx, t, producer, bundle, &noOpArgs{}) startProducer(t, ctx, ctx, producer) @@ -543,7 +561,10 @@ func testProducer(t *testing.T, makeProducer func(ctx context.Context, t *testin if pauseAll { queueNameToPause = rivercommon.AllQueuesString } - require.NoError(t, bundle.exec.QueuePause(ctx, queueNameToPause)) + require.NoError(t, bundle.exec.QueuePause(ctx, &riverdriver.QueuePauseParams{ + Name: queueNameToPause, + Schema: producer.config.Schema, + })) if producer.config.Notifier != nil { // also emit notification: emitQueueNotification(t, ctx, bundle.exec, queueNameToPause, "pause", nil) @@ -551,7 +572,7 @@ func testProducer(t *testing.T, makeProducer func(ctx context.Context, t *testin producer.testSignals.Paused.WaitOrTimeout() // Job should not be executed while paused: - mustInsert(ctx, t, bundle, &noOpArgs{}) + mustInsert(ctx, t, producer, bundle, &noOpArgs{}) select { case update := <-bundle.jobUpdates: @@ -560,7 +581,10 @@ func testProducer(t *testing.T, makeProducer func(ctx context.Context, t *testin } // Resume the queue and wait for confirmation: - require.NoError(t, bundle.exec.QueueResume(ctx, queueNameToPause)) + require.NoError(t, bundle.exec.QueueResume(ctx, &riverdriver.QueueResumeParams{ + Name: queueNameToPause, + Schema: producer.config.Schema, + })) if producer.config.Notifier != nil { // also emit notification: emitQueueNotification(t, ctx, bundle.exec, queueNameToPause, "resume", nil) @@ -677,6 +701,7 @@ func emitQueueNotification(t *testing.T, ctx context.Context, exec riverdriver.E err = exec.NotifyMany(ctx, &riverdriver.NotifyManyParams{ Topic: string(notifier.NotificationTopicControl), Payload: []string{string(payloadBytes)}, + Schema: "", }) require.NoError(t, err) } diff --git a/riverdriver/river_driver_interface.go b/riverdriver/river_driver_interface.go index 35d23c2b..ba5b03bc 100644 --- a/riverdriver/river_driver_interface.go +++ b/riverdriver/river_driver_interface.go @@ -53,7 +53,7 @@ type Driver[TTx any] interface { // GetListener gets a listener for purposes of receiving notifications. // // API is not stable. DO NOT USE. - GetListener() Listener + GetListener(schema string) Listener // GetMigrationFS gets a filesystem containing migrations for the driver. // @@ -100,76 +100,75 @@ type Executor interface { // ColumnExists checks whether a column for a particular table exists for // the schema in the current search schema. - ColumnExists(ctx context.Context, tableName, columnName string) (bool, error) + ColumnExists(ctx context.Context, params *ColumnExistsParams) (bool, error) // Exec executes raw SQL. Used for migrations. Exec(ctx context.Context, sql string) (struct{}, error) JobCancel(ctx context.Context, params *JobCancelParams) (*rivertype.JobRow, error) - JobCountByState(ctx context.Context, state rivertype.JobState) (int, error) - JobDelete(ctx context.Context, id int64) (*rivertype.JobRow, error) + JobCountByState(ctx context.Context, params *JobCountByStateParams) (int, error) + JobDelete(ctx context.Context, params *JobDeleteParams) (*rivertype.JobRow, error) JobDeleteBefore(ctx context.Context, params *JobDeleteBeforeParams) (int, error) JobGetAvailable(ctx context.Context, params *JobGetAvailableParams) ([]*rivertype.JobRow, error) - JobGetByID(ctx context.Context, id int64) (*rivertype.JobRow, error) - JobGetByIDMany(ctx context.Context, id []int64) ([]*rivertype.JobRow, error) - JobGetByKindAndUniqueProperties(ctx context.Context, params *JobGetByKindAndUniquePropertiesParams) (*rivertype.JobRow, error) - JobGetByKindMany(ctx context.Context, kind []string) ([]*rivertype.JobRow, error) + JobGetByID(ctx context.Context, params *JobGetByIDParams) (*rivertype.JobRow, error) + JobGetByIDMany(ctx context.Context, params *JobGetByIDManyParams) ([]*rivertype.JobRow, error) + JobGetByKindMany(ctx context.Context, params *JobGetByKindManyParams) ([]*rivertype.JobRow, error) JobGetStuck(ctx context.Context, params *JobGetStuckParams) ([]*rivertype.JobRow, error) - JobInsertFastMany(ctx context.Context, params []*JobInsertFastParams) ([]*JobInsertFastResult, error) - JobInsertFastManyNoReturning(ctx context.Context, params []*JobInsertFastParams) (int, error) + JobInsertFastMany(ctx context.Context, params *JobInsertFastManyParams) ([]*JobInsertFastResult, error) + JobInsertFastManyNoReturning(ctx context.Context, params *JobInsertFastManyParams) (int, error) JobInsertFull(ctx context.Context, params *JobInsertFullParams) (*rivertype.JobRow, error) JobList(ctx context.Context, params *JobListParams) ([]*rivertype.JobRow, error) JobRescueMany(ctx context.Context, params *JobRescueManyParams) (*struct{}, error) - JobRetry(ctx context.Context, id int64) (*rivertype.JobRow, error) + JobRetry(ctx context.Context, params *JobRetryParams) (*rivertype.JobRow, error) JobSchedule(ctx context.Context, params *JobScheduleParams) ([]*JobScheduleResult, error) JobSetStateIfRunningMany(ctx context.Context, params *JobSetStateIfRunningManyParams) ([]*rivertype.JobRow, error) JobUpdate(ctx context.Context, params *JobUpdateParams) (*rivertype.JobRow, error) LeaderAttemptElect(ctx context.Context, params *LeaderElectParams) (bool, error) LeaderAttemptReelect(ctx context.Context, params *LeaderElectParams) (bool, error) - LeaderDeleteExpired(ctx context.Context) (int, error) - LeaderGetElectedLeader(ctx context.Context) (*Leader, error) + LeaderDeleteExpired(ctx context.Context, params *LeaderDeleteExpiredParams) (int, error) + LeaderGetElectedLeader(ctx context.Context, params *LeaderGetElectedLeaderParams) (*Leader, error) LeaderInsert(ctx context.Context, params *LeaderInsertParams) (*Leader, error) LeaderResign(ctx context.Context, params *LeaderResignParams) (bool, error) // MigrationDeleteAssumingMainMany deletes many migrations assuming // everything is on the main line. This is suitable for use in databases on // a version before the `line` column exists. - MigrationDeleteAssumingMainMany(ctx context.Context, versions []int) ([]*Migration, error) + MigrationDeleteAssumingMainMany(ctx context.Context, params *MigrationDeleteAssumingMainManyParams) ([]*Migration, error) // MigrationDeleteByLineAndVersionMany deletes many migration versions on a // particular line. - MigrationDeleteByLineAndVersionMany(ctx context.Context, line string, versions []int) ([]*Migration, error) + MigrationDeleteByLineAndVersionMany(ctx context.Context, params *MigrationDeleteByLineAndVersionManyParams) ([]*Migration, error) // MigrationGetAllAssumingMain gets all migrations assuming everything is on // the main line. This is suitable for use in databases on a version before // the `line` column exists. - MigrationGetAllAssumingMain(ctx context.Context) ([]*Migration, error) + MigrationGetAllAssumingMain(ctx context.Context, params *MigrationGetAllAssumingMainParams) ([]*Migration, error) // MigrationGetByLine gets all currently applied migrations. - MigrationGetByLine(ctx context.Context, line string) ([]*Migration, error) + MigrationGetByLine(ctx context.Context, params *MigrationGetByLineParams) ([]*Migration, error) // MigrationInsertMany inserts many migration versions. - MigrationInsertMany(ctx context.Context, line string, versions []int) ([]*Migration, error) + MigrationInsertMany(ctx context.Context, params *MigrationInsertManyParams) ([]*Migration, error) // MigrationInsertManyAssumingMain inserts many migrations, assuming they're // on the main line. This operation is necessary for compatibility before // the `line` column was added to the migrations table. - MigrationInsertManyAssumingMain(ctx context.Context, versions []int) ([]*Migration, error) + MigrationInsertManyAssumingMain(ctx context.Context, params *MigrationInsertManyAssumingMainParams) ([]*Migration, error) NotifyMany(ctx context.Context, params *NotifyManyParams) error PGAdvisoryXactLock(ctx context.Context, key int64) (*struct{}, error) QueueCreateOrSetUpdatedAt(ctx context.Context, params *QueueCreateOrSetUpdatedAtParams) (*rivertype.Queue, error) QueueDeleteExpired(ctx context.Context, params *QueueDeleteExpiredParams) ([]string, error) - QueueGet(ctx context.Context, name string) (*rivertype.Queue, error) - QueueList(ctx context.Context, limit int) ([]*rivertype.Queue, error) - QueuePause(ctx context.Context, name string) error - QueueResume(ctx context.Context, name string) error + QueueGet(ctx context.Context, params *QueueGetParams) (*rivertype.Queue, error) + QueueList(ctx context.Context, params *QueueListParams) ([]*rivertype.Queue, error) + QueuePause(ctx context.Context, params *QueuePauseParams) error + QueueResume(ctx context.Context, params *QueueResumeParams) error QueueUpdate(ctx context.Context, params *QueueUpdateParams) (*rivertype.Queue, error) // TableExists checks whether a table exists for the schema in the current // search schema. - TableExists(ctx context.Context, tableName string) (bool, error) + TableExists(ctx context.Context, params *TableExistsParams) (bool, error) } // ExecutorTx is an executor which is a transaction. In addition to standard @@ -199,6 +198,7 @@ type Listener interface { Connect(ctx context.Context) error Listen(ctx context.Context, topic string) error Ping(ctx context.Context) error + Schema() string Unlisten(ctx context.Context, topic string) error WaitForNotification(ctx context.Context) (*Notification, error) } @@ -208,10 +208,27 @@ type Notification struct { Topic string } +type ColumnExistsParams struct { + Column string + Schema string + Table string +} + type JobCancelParams struct { + ID int64 CancelAttemptedAt time.Time ControlTopic string - ID int64 + Schema string +} + +type JobCountByStateParams struct { + Schema string + State rivertype.JobState +} + +type JobDeleteParams struct { + ID int64 + Schema string } type JobDeleteBeforeParams struct { @@ -219,31 +236,36 @@ type JobDeleteBeforeParams struct { CompletedFinalizedAtHorizon time.Time DiscardedFinalizedAtHorizon time.Time Max int + Schema string } type JobGetAvailableParams struct { ClientID string Max int Now *time.Time - Queue string ProducerID int64 + Queue string + Schema string +} + +type JobGetByIDParams struct { + ID int64 + Schema string +} + +type JobGetByIDManyParams struct { + ID []int64 + Schema string } -type JobGetByKindAndUniquePropertiesParams struct { - Kind string - ByArgs bool - Args []byte - ByCreatedAt bool - CreatedAtBegin time.Time - CreatedAtEnd time.Time - ByQueue bool - Queue string - ByState bool - State []string +type JobGetByKindManyParams struct { + Kind []string + Schema string } type JobGetStuckParams struct { Max int + Schema string StuckHorizon time.Time } @@ -266,6 +288,11 @@ type JobInsertFastParams struct { UniqueStates byte } +type JobInsertFastManyParams struct { + Jobs []*JobInsertFastParams + Schema string +} + type JobInsertFastResult struct { Job *rivertype.JobRow UniqueSkippedAsDuplicate bool @@ -285,6 +312,7 @@ type JobInsertFullParams struct { Priority int Queue string ScheduledAt *time.Time + Schema string State rivertype.JobState Tags []string UniqueKey []byte @@ -295,6 +323,7 @@ type JobListParams struct { Max int32 NamedArgs map[string]any OrderByClause string + Schema string WhereClause string } @@ -303,12 +332,19 @@ type JobRescueManyParams struct { Error [][]byte FinalizedAt []time.Time ScheduledAt []time.Time + Schema string State []string } +type JobRetryParams struct { + ID int64 + Schema string +} + type JobScheduleParams struct { - Max int - Now time.Time + Max int + Now time.Time + Schema string } type JobScheduleResult struct { @@ -417,6 +453,7 @@ type JobSetStateIfRunningManyParams struct { MetadataDoMerge []bool MetadataUpdates [][]byte ScheduledAt []*time.Time + Schema string State []rivertype.JobState } @@ -432,6 +469,7 @@ type JobUpdateParams struct { Errors [][]byte FinalizedAtDoUpdate bool FinalizedAt *time.Time + Schema string StateDoUpdate bool State rivertype.JobState // Deprecated and will be removed when advisory lock unique path is removed. @@ -449,21 +487,32 @@ type Leader struct { LeaderID string } +type LeaderDeleteExpiredParams struct { + Schema string +} + +type LeaderGetElectedLeaderParams struct { + Schema string +} + type LeaderInsertParams struct { ElectedAt *time.Time ExpiresAt *time.Time LeaderID string + Schema string TTL time.Duration } type LeaderElectParams struct { LeaderID string + Schema string TTL time.Duration } type LeaderResignParams struct { LeaderID string LeadershipTopic string + Schema string } // Migration represents a River migration. @@ -486,16 +535,49 @@ type Migration struct { Version int } +type MigrationDeleteAssumingMainManyParams struct { + Schema string + Versions []int +} + +type MigrationDeleteByLineAndVersionManyParams struct { + Line string + Schema string + Versions []int +} + +type MigrationGetAllAssumingMainParams struct { + Schema string +} + +type MigrationGetByLineParams struct { + Line string + Schema string +} + +type MigrationInsertManyParams struct { + Line string + Schema string + Versions []int +} + +type MigrationInsertManyAssumingMainParams struct { + Schema string + Versions []int +} + // NotifyManyParams are parameters to issue many pubsub notifications all at // once for a single topic. type NotifyManyParams struct { Payload []string Topic string + Schema string } type ProducerKeepAliveParams struct { ID int64 QueueName string + Schema string StaleUpdatedAtHorizon time.Time } @@ -503,16 +585,44 @@ type QueueCreateOrSetUpdatedAtParams struct { Metadata []byte Name string PausedAt *time.Time + Schema string UpdatedAt *time.Time } type QueueDeleteExpiredParams struct { Max int + Schema string UpdatedAtHorizon time.Time } +type QueueGetParams struct { + Name string + Schema string +} + +type QueueListParams struct { + Limit int + Schema string +} + +type QueuePauseParams struct { + Name string + Schema string +} + +type QueueResumeParams struct { + Name string + Schema string +} + type QueueUpdateParams struct { Metadata []byte MetadataDoUpdate bool Name string + Schema string +} + +type TableExistsParams struct { + Schema string + Table string } diff --git a/riverdriver/riverdatabasesql/internal/dbsqlc/pg_misc.sql.go b/riverdriver/riverdatabasesql/internal/dbsqlc/pg_misc.sql.go index 693147b8..a7097015 100644 --- a/riverdriver/riverdatabasesql/internal/dbsqlc/pg_misc.sql.go +++ b/riverdriver/riverdatabasesql/internal/dbsqlc/pg_misc.sql.go @@ -7,6 +7,7 @@ package dbsqlc import ( "context" + "database/sql" "github.com/lib/pq" ) @@ -23,8 +24,8 @@ func (q *Queries) PGAdvisoryXactLock(ctx context.Context, db DBTX, key int64) er const pGNotifyMany = `-- name: PGNotifyMany :exec WITH topic_to_notify AS ( SELECT - concat(current_schema(), '.', $1::text) AS topic, - unnest($2::text[]) AS payload + concat(coalesce($1::text, current_schema()), '.', $2::text) AS topic, + unnest($3::text[]) AS payload ) SELECT pg_notify( topic_to_notify.topic, @@ -34,11 +35,12 @@ FROM topic_to_notify ` type PGNotifyManyParams struct { + Schema sql.NullString Topic string Payload []string } func (q *Queries) PGNotifyMany(ctx context.Context, db DBTX, arg *PGNotifyManyParams) error { - _, err := db.ExecContext(ctx, pGNotifyMany, arg.Topic, pq.Array(arg.Payload)) + _, err := db.ExecContext(ctx, pGNotifyMany, arg.Schema, arg.Topic, pq.Array(arg.Payload)) return err } diff --git a/riverdriver/riverdatabasesql/internal/dbsqlc/river_client.sql.go b/riverdriver/riverdatabasesql/internal/dbsqlc/river_client.sql.go index bc383b86..a5620621 100644 --- a/riverdriver/riverdatabasesql/internal/dbsqlc/river_client.sql.go +++ b/riverdriver/riverdatabasesql/internal/dbsqlc/river_client.sql.go @@ -11,7 +11,7 @@ import ( ) const clientCreateOrSetUpdatedAt = `-- name: ClientCreateOrSetUpdatedAt :one -INSERT INTO river_client ( +INSERT INTO /* TEMPLATE: schema */river_client ( id, metadata, paused_at, diff --git a/riverdriver/riverdatabasesql/internal/dbsqlc/river_client_queue.sql.go b/riverdriver/riverdatabasesql/internal/dbsqlc/river_client_queue.sql.go index d029d195..ee68082b 100644 --- a/riverdriver/riverdatabasesql/internal/dbsqlc/river_client_queue.sql.go +++ b/riverdriver/riverdatabasesql/internal/dbsqlc/river_client_queue.sql.go @@ -13,7 +13,7 @@ import ( ) const clientQueueCreateOrSetUpdatedAtMany = `-- name: ClientQueueCreateOrSetUpdatedAtMany :one -INSERT INTO river_client_queue ( +INSERT INTO /* TEMPLATE: schema */river_client_queue ( metadata, name, paused_at, diff --git a/riverdriver/riverdatabasesql/internal/dbsqlc/river_job.sql.go b/riverdriver/riverdatabasesql/internal/dbsqlc/river_job.sql.go index dc430083..6037a0dd 100644 --- a/riverdriver/riverdatabasesql/internal/dbsqlc/river_job.sql.go +++ b/riverdriver/riverdatabasesql/internal/dbsqlc/river_job.sql.go @@ -7,6 +7,7 @@ package dbsqlc import ( "context" + "database/sql" "time" "github.com/lib/pq" @@ -17,7 +18,7 @@ const jobCancel = `-- name: JobCancel :one WITH locked_job AS ( SELECT id, queue, state, finalized_at - FROM river_job + FROM /* TEMPLATE: schema */river_job WHERE river_job.id = $1 FOR UPDATE ), @@ -25,7 +26,7 @@ notification AS ( SELECT id, pg_notify( - concat(current_schema(), '.', $2::text), + concat(coalesce($2::text, current_schema()), '.', $3::text), json_build_object('action', 'cancel', 'job_id', id, 'queue', queue)::text ) FROM @@ -35,7 +36,7 @@ notification AS ( AND finalized_at IS NULL ), updated_job AS ( - UPDATE river_job + UPDATE /* TEMPLATE: schema */river_job SET -- If the job is actively running, we want to let its current client and -- producer handle the cancellation. Otherwise, immediately cancel it. @@ -43,13 +44,13 @@ updated_job AS ( finalized_at = CASE WHEN state = 'running' THEN finalized_at ELSE now() END, -- Mark the job as cancelled by query so that the rescuer knows not to -- rescue it, even if it gets stuck in the running state: - metadata = jsonb_set(metadata, '{cancel_attempted_at}'::text[], $3::jsonb, true) + metadata = jsonb_set(metadata, '{cancel_attempted_at}'::text[], $4::jsonb, true) 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, 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, unique_states -FROM river_job +FROM /* TEMPLATE: schema */river_job WHERE id = $1::bigint AND id NOT IN (SELECT id FROM updated_job) UNION @@ -59,12 +60,18 @@ FROM updated_job type JobCancelParams struct { ID int64 + Schema sql.NullString ControlTopic string CancelAttemptedAt string } func (q *Queries) JobCancel(ctx context.Context, db DBTX, arg *JobCancelParams) (*RiverJob, error) { - row := db.QueryRowContext(ctx, jobCancel, arg.ID, arg.ControlTopic, arg.CancelAttemptedAt) + row := db.QueryRowContext(ctx, jobCancel, + arg.ID, + arg.Schema, + arg.ControlTopic, + arg.CancelAttemptedAt, + ) var i RiverJob err := row.Scan( &i.ID, @@ -91,7 +98,7 @@ func (q *Queries) JobCancel(ctx context.Context, db DBTX, arg *JobCancelParams) const jobCountByState = `-- name: JobCountByState :one SELECT count(*) -FROM river_job +FROM /* TEMPLATE: schema */river_job WHERE state = $1 ` @@ -105,13 +112,13 @@ func (q *Queries) JobCountByState(ctx context.Context, db DBTX, state RiverJobSt const jobDelete = `-- name: JobDelete :one WITH job_to_delete AS ( SELECT id - FROM river_job + FROM /* TEMPLATE: schema */river_job WHERE river_job.id = $1 FOR UPDATE ), deleted_job AS ( DELETE - FROM river_job + FROM /* TEMPLATE: schema */river_job USING job_to_delete WHERE river_job.id = job_to_delete.id -- Do not touch running jobs: @@ -119,7 +126,7 @@ deleted_job AS ( 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, unique_states -FROM river_job +FROM /* TEMPLATE: schema */river_job WHERE id = $1::bigint AND id NOT IN (SELECT id FROM deleted_job) UNION @@ -155,10 +162,10 @@ func (q *Queries) JobDelete(ctx context.Context, db DBTX, id int64) (*RiverJob, const jobDeleteBefore = `-- name: JobDeleteBefore :one WITH deleted_jobs AS ( - DELETE FROM river_job + DELETE FROM /* TEMPLATE: schema */river_job WHERE id IN ( SELECT id - FROM river_job + FROM /* TEMPLATE: schema */river_job WHERE (state = 'cancelled' AND finalized_at < $1::timestamptz) OR (state = 'completed' AND finalized_at < $2::timestamptz) OR @@ -196,7 +203,7 @@ 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, unique_states FROM - river_job + /* TEMPLATE: schema */river_job WHERE state = 'available' AND queue = $2::text @@ -210,7 +217,7 @@ WITH locked_jobs AS ( SKIP LOCKED ) UPDATE - river_job + /* TEMPLATE: schema */river_job SET state = 'running', attempt = river_job.attempt + 1, @@ -280,7 +287,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, unique_states -FROM river_job +FROM /* TEMPLATE: schema */river_job WHERE id = $1 LIMIT 1 ` @@ -313,7 +320,7 @@ func (q *Queries) JobGetByID(ctx context.Context, db DBTX, id int64) (*RiverJob, 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, unique_states -FROM river_job +FROM /* TEMPLATE: schema */river_job WHERE id = any($1::bigint[]) ORDER BY id ` @@ -362,7 +369,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, unique_states -FROM river_job +FROM /* TEMPLATE: schema */river_job WHERE kind = $1 AND CASE WHEN $2::boolean THEN args = $3 ELSE true END AND CASE WHEN $4::boolean THEN tstzrange($5::timestamptz, $6::timestamptz, '[)') @> created_at ELSE true END @@ -422,7 +429,7 @@ func (q *Queries) JobGetByKindAndUniqueProperties(ctx context.Context, db DBTX, 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, unique_states -FROM river_job +FROM /* TEMPLATE: schema */river_job WHERE kind = any($1::text[]) ORDER BY id ` @@ -471,7 +478,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, unique_states -FROM river_job +FROM /* TEMPLATE: schema */river_job WHERE state = 'running' AND attempted_at < $1::timestamptz ORDER BY id @@ -526,7 +533,7 @@ func (q *Queries) JobGetStuck(ctx context.Context, db DBTX, arg *JobGetStuckPara } const jobInsertFastMany = `-- name: JobInsertFastMany :many -INSERT INTO river_job( +INSERT INTO /* TEMPLATE: schema */river_job( args, created_at, kind, @@ -645,7 +652,7 @@ func (q *Queries) JobInsertFastMany(ctx context.Context, db DBTX, arg *JobInsert } const jobInsertFastManyNoReturning = `-- name: JobInsertFastManyNoReturning :execrows -INSERT INTO river_job( +INSERT INTO /* TEMPLATE: schema */river_job( args, created_at, kind, @@ -722,7 +729,7 @@ func (q *Queries) JobInsertFastManyNoReturning(ctx context.Context, db DBTX, arg } const jobInsertFull = `-- name: JobInsertFull :one -INSERT INTO river_job( +INSERT INTO /* TEMPLATE: schema */river_job( args, attempt, attempted_at, @@ -827,7 +834,7 @@ func (q *Queries) JobInsertFull(ctx context.Context, db DBTX, arg *JobInsertFull const jobList = `-- name: JobList :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, unique_states -FROM river_job +FROM /* TEMPLATE: schema */river_job WHERE /* TEMPLATE_BEGIN: where_clause */ 1 /* TEMPLATE_END */ ORDER BY /* TEMPLATE_BEGIN: order_by_clause */ id /* TEMPLATE_END */ LIMIT $1::int @@ -876,7 +883,7 @@ func (q *Queries) JobList(ctx context.Context, db DBTX, max int32) ([]*RiverJob, } const jobRescueMany = `-- name: JobRescueMany :exec -UPDATE river_job +UPDATE /* TEMPLATE: schema */river_job SET errors = array_append(errors, updated_job.error), finalized_at = updated_job.finalized_at, @@ -916,12 +923,12 @@ func (q *Queries) JobRescueMany(ctx context.Context, db DBTX, arg *JobRescueMany const jobRetry = `-- name: JobRetry :one WITH job_to_update AS ( SELECT id - FROM river_job + FROM /* TEMPLATE: schema */river_job WHERE river_job.id = $1 FOR UPDATE ), updated_job AS ( - UPDATE river_job + UPDATE /* TEMPLATE: schema */river_job SET state = 'available', scheduled_at = now(), @@ -936,7 +943,7 @@ updated_job AS ( 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, unique_states -FROM river_job +FROM /* TEMPLATE: schema */river_job WHERE id = $1::bigint AND id NOT IN (SELECT id FROM updated_job) UNION @@ -978,7 +985,7 @@ WITH jobs_to_schedule AS ( unique_states, priority, scheduled_at - FROM river_job + FROM /* TEMPLATE: schema */river_job WHERE state IN ('retryable', 'scheduled') AND queue IS NOT NULL @@ -1006,7 +1013,7 @@ jobs_with_rownum AS ( ), unique_conflicts AS ( SELECT river_job.unique_key - FROM river_job + FROM /* TEMPLATE: schema */river_job JOIN jobs_with_rownum ON river_job.unique_key = jobs_with_rownum.unique_key AND river_job.id != jobs_with_rownum.id @@ -1032,7 +1039,7 @@ job_updates AS ( LEFT JOIN unique_conflicts uc ON job.unique_key = uc.unique_key ), updated_jobs AS ( - UPDATE river_job + UPDATE /* TEMPLATE: schema */river_job SET state = job_updates.new_state, finalized_at = CASE WHEN job_updates.finalized_at_do_update THEN $1::timestamptz @@ -1048,7 +1055,7 @@ updated_jobs AS ( SELECT river_job.id, river_job.args, river_job.attempt, river_job.attempted_at, river_job.attempted_by, river_job.created_at, river_job.errors, river_job.finalized_at, river_job.kind, river_job.max_attempts, river_job.metadata, river_job.priority, river_job.queue, river_job.state, river_job.scheduled_at, river_job.tags, river_job.unique_key, river_job.unique_states, updated_jobs.conflict_discarded -FROM river_job +FROM /* TEMPLATE: schema */river_job JOIN updated_jobs ON river_job.id = updated_jobs.id ` @@ -1138,7 +1145,7 @@ job_to_update AS ( job_input.scheduled_at_do_update, (job_input.state IN ('retryable', 'scheduled') AND river_job.metadata ? 'cancel_attempted_at') AS should_cancel, job_input.state - FROM river_job + FROM /* TEMPLATE: schema */river_job JOIN job_input ON river_job.id = job_input.id WHERE river_job.state = 'running' OR job_input.metadata_do_merge FOR UPDATE @@ -1166,7 +1173,7 @@ updated_running AS ( 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 ), updated_metadata_only AS ( - UPDATE river_job + UPDATE /* TEMPLATE: schema */river_job SET metadata = river_job.metadata || job_to_update.metadata_updates FROM job_to_update WHERE river_job.id = job_to_update.id @@ -1176,7 +1183,7 @@ updated_metadata_only AS ( 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, unique_states -FROM river_job +FROM /* TEMPLATE: schema */river_job WHERE id IN (SELECT id FROM job_input) AND id NOT IN (SELECT id FROM updated_metadata_only) AND id NOT IN (SELECT id FROM updated_running) @@ -1255,7 +1262,7 @@ func (q *Queries) JobSetStateIfRunningMany(ctx context.Context, db DBTX, arg *Jo } const jobUpdate = `-- name: JobUpdate :one -UPDATE river_job +UPDATE /* TEMPLATE: schema */river_job SET attempt = CASE WHEN $1::boolean THEN $2 ELSE attempt END, attempted_at = CASE WHEN $3::boolean THEN $4 ELSE attempted_at END, diff --git a/riverdriver/riverdatabasesql/internal/dbsqlc/river_leader.sql.go b/riverdriver/riverdatabasesql/internal/dbsqlc/river_leader.sql.go index 22fb5098..f7fbc913 100644 --- a/riverdriver/riverdatabasesql/internal/dbsqlc/river_leader.sql.go +++ b/riverdriver/riverdatabasesql/internal/dbsqlc/river_leader.sql.go @@ -7,11 +7,12 @@ package dbsqlc import ( "context" + "database/sql" "time" ) const leaderAttemptElect = `-- name: LeaderAttemptElect :execrows -INSERT INTO river_leader(leader_id, elected_at, expires_at) +INSERT INTO /* TEMPLATE: schema */river_leader (leader_id, elected_at, expires_at) VALUES ($1, now(), now() + $2::interval) ON CONFLICT (name) DO NOTHING @@ -31,7 +32,7 @@ func (q *Queries) LeaderAttemptElect(ctx context.Context, db DBTX, arg *LeaderAt } const leaderAttemptReelect = `-- name: LeaderAttemptReelect :execrows -INSERT INTO river_leader(leader_id, elected_at, expires_at) +INSERT INTO /* TEMPLATE: schema */river_leader (leader_id, elected_at, expires_at) VALUES ($1, now(), now() + $2::interval) ON CONFLICT (name) DO UPDATE SET @@ -54,7 +55,7 @@ func (q *Queries) LeaderAttemptReelect(ctx context.Context, db DBTX, arg *Leader } const leaderDeleteExpired = `-- name: LeaderDeleteExpired :execrows -DELETE FROM river_leader +DELETE FROM /* TEMPLATE: schema */river_leader WHERE expires_at < now() ` @@ -68,7 +69,7 @@ func (q *Queries) LeaderDeleteExpired(ctx context.Context, db DBTX) (int64, erro const leaderGetElectedLeader = `-- name: LeaderGetElectedLeader :one SELECT elected_at, expires_at, leader_id, name -FROM river_leader +FROM /* TEMPLATE: schema */river_leader ` func (q *Queries) LeaderGetElectedLeader(ctx context.Context, db DBTX) (*RiverLeader, error) { @@ -84,7 +85,7 @@ func (q *Queries) LeaderGetElectedLeader(ctx context.Context, db DBTX) (*RiverLe } const leaderInsert = `-- name: LeaderInsert :one -INSERT INTO river_leader( +INSERT INTO /* TEMPLATE: schema */river_leader( elected_at, expires_at, leader_id @@ -122,27 +123,28 @@ func (q *Queries) LeaderInsert(ctx context.Context, db DBTX, arg *LeaderInsertPa const leaderResign = `-- name: LeaderResign :execrows WITH currently_held_leaders AS ( SELECT elected_at, expires_at, leader_id, name - FROM river_leader + FROM /* TEMPLATE: schema */river_leader WHERE leader_id = $1::text FOR UPDATE ), notified_resignations AS ( SELECT pg_notify( - concat(current_schema(), '.', $2::text), + concat(coalesce($2::text, current_schema()), '.', $3::text), json_build_object('leader_id', leader_id, 'action', 'resigned')::text ) FROM currently_held_leaders ) -DELETE FROM river_leader USING notified_resignations +DELETE FROM /* TEMPLATE: schema */river_leader USING notified_resignations ` type LeaderResignParams struct { LeaderID string + Schema sql.NullString LeadershipTopic string } func (q *Queries) LeaderResign(ctx context.Context, db DBTX, arg *LeaderResignParams) (int64, error) { - result, err := db.ExecContext(ctx, leaderResign, arg.LeaderID, arg.LeadershipTopic) + result, err := db.ExecContext(ctx, leaderResign, arg.LeaderID, arg.Schema, arg.LeadershipTopic) if err != nil { return 0, err } diff --git a/riverdriver/riverdatabasesql/internal/dbsqlc/river_migration.sql.go b/riverdriver/riverdatabasesql/internal/dbsqlc/river_migration.sql.go index b7d39c47..a196452c 100644 --- a/riverdriver/riverdatabasesql/internal/dbsqlc/river_migration.sql.go +++ b/riverdriver/riverdatabasesql/internal/dbsqlc/river_migration.sql.go @@ -17,7 +17,7 @@ SELECT EXISTS ( SELECT column_name FROM information_schema.columns WHERE table_name = $1::text - AND table_schema = CURRENT_SCHEMA + AND table_schema = /* TEMPLATE_BEGIN: schema */ CURRENT_SCHEMA /* TEMPLATE_END */ AND column_name = $2::text ) ` @@ -35,7 +35,7 @@ func (q *Queries) ColumnExists(ctx context.Context, db DBTX, arg *ColumnExistsPa } const riverMigrationDeleteAssumingMainMany = `-- name: RiverMigrationDeleteAssumingMainMany :many -DELETE FROM river_migration +DELETE FROM /* TEMPLATE: schema */river_migration WHERE version = any($1::bigint[]) RETURNING created_at, @@ -71,7 +71,7 @@ func (q *Queries) RiverMigrationDeleteAssumingMainMany(ctx context.Context, db D } const riverMigrationDeleteByLineAndVersionMany = `-- name: RiverMigrationDeleteByLineAndVersionMany :many -DELETE FROM river_migration +DELETE FROM /* TEMPLATE: schema */river_migration WHERE line = $1 AND version = any($2::bigint[]) RETURNING line, version, created_at @@ -109,7 +109,7 @@ const riverMigrationGetAllAssumingMain = `-- name: RiverMigrationGetAllAssumingM SELECT created_at, version -FROM river_migration +FROM /* TEMPLATE: schema */river_migration ORDER BY version ` @@ -148,7 +148,7 @@ func (q *Queries) RiverMigrationGetAllAssumingMain(ctx context.Context, db DBTX) const riverMigrationGetByLine = `-- name: RiverMigrationGetByLine :many SELECT line, version, created_at -FROM river_migration +FROM /* TEMPLATE: schema */river_migration WHERE line = $1 ORDER BY version ` @@ -177,7 +177,7 @@ func (q *Queries) RiverMigrationGetByLine(ctx context.Context, db DBTX, line str } const riverMigrationInsert = `-- name: RiverMigrationInsert :one -INSERT INTO river_migration ( +INSERT INTO /* TEMPLATE: schema */river_migration ( line, version ) VALUES ( @@ -199,7 +199,7 @@ func (q *Queries) RiverMigrationInsert(ctx context.Context, db DBTX, arg *RiverM } const riverMigrationInsertMany = `-- name: RiverMigrationInsertMany :many -INSERT INTO river_migration ( +INSERT INTO /* TEMPLATE: schema */river_migration ( line, version ) @@ -238,7 +238,7 @@ func (q *Queries) RiverMigrationInsertMany(ctx context.Context, db DBTX, arg *Ri } const riverMigrationInsertManyAssumingMain = `-- name: RiverMigrationInsertManyAssumingMain :many -INSERT INTO river_migration ( +INSERT INTO /* TEMPLATE: schema */river_migration ( version ) SELECT @@ -281,8 +281,8 @@ SELECT CASE WHEN to_regclass($1) IS NULL THEN false ELSE true END ` -func (q *Queries) TableExists(ctx context.Context, db DBTX, tableName string) (bool, error) { - row := db.QueryRowContext(ctx, tableExists, tableName) +func (q *Queries) TableExists(ctx context.Context, db DBTX, schemaAndTable string) (bool, error) { + row := db.QueryRowContext(ctx, tableExists, schemaAndTable) var column_1 bool err := row.Scan(&column_1) return column_1, err diff --git a/riverdriver/riverdatabasesql/internal/dbsqlc/river_queue.sql.go b/riverdriver/riverdatabasesql/internal/dbsqlc/river_queue.sql.go index 9a46fc47..b022d8d4 100644 --- a/riverdriver/riverdatabasesql/internal/dbsqlc/river_queue.sql.go +++ b/riverdriver/riverdatabasesql/internal/dbsqlc/river_queue.sql.go @@ -12,7 +12,7 @@ import ( ) const queueCreateOrSetUpdatedAt = `-- name: QueueCreateOrSetUpdatedAt :one -INSERT INTO river_queue( +INSERT INTO /* TEMPLATE: schema */river_queue( created_at, metadata, name, @@ -56,10 +56,10 @@ func (q *Queries) QueueCreateOrSetUpdatedAt(ctx context.Context, db DBTX, arg *Q } const queueDeleteExpired = `-- name: QueueDeleteExpired :many -DELETE FROM river_queue +DELETE FROM /* TEMPLATE: schema */river_queue WHERE name IN ( SELECT name - FROM river_queue + FROM /* TEMPLATE: schema */river_queue WHERE updated_at < $1::timestamptz ORDER BY name ASC LIMIT $2::bigint @@ -103,7 +103,7 @@ func (q *Queries) QueueDeleteExpired(ctx context.Context, db DBTX, arg *QueueDel const queueGet = `-- name: QueueGet :one SELECT name, created_at, metadata, paused_at, updated_at -FROM river_queue +FROM /* TEMPLATE: schema */river_queue WHERE name = $1::text ` @@ -122,7 +122,7 @@ func (q *Queries) QueueGet(ctx context.Context, db DBTX, name string) (*RiverQue const queueList = `-- name: QueueList :many SELECT name, created_at, metadata, paused_at, updated_at -FROM river_queue +FROM /* TEMPLATE: schema */river_queue ORDER BY name ASC LIMIT $1::integer ` @@ -159,12 +159,12 @@ func (q *Queries) QueueList(ctx context.Context, db DBTX, limitCount int32) ([]* const queuePause = `-- name: QueuePause :execresult WITH queue_to_update AS ( SELECT name, paused_at - FROM river_queue + FROM /* TEMPLATE: schema */river_queue WHERE CASE WHEN $1::text = '*' THEN true ELSE name = $1 END FOR UPDATE ), updated_queue AS ( - UPDATE river_queue + UPDATE /* TEMPLATE: schema */river_queue SET paused_at = now(), updated_at = now() @@ -174,7 +174,7 @@ updated_queue AS ( RETURNING river_queue.name, river_queue.created_at, river_queue.metadata, river_queue.paused_at, river_queue.updated_at ) SELECT name, created_at, metadata, paused_at, updated_at -FROM river_queue +FROM /* TEMPLATE: schema */river_queue WHERE name = $1 AND name NOT IN (SELECT name FROM updated_queue) UNION @@ -189,12 +189,12 @@ func (q *Queries) QueuePause(ctx context.Context, db DBTX, name string) (sql.Res const queueResume = `-- name: QueueResume :execresult WITH queue_to_update AS ( SELECT name - FROM river_queue + FROM /* TEMPLATE: schema */river_queue WHERE CASE WHEN $1::text = '*' THEN true ELSE river_queue.name = $1::text END FOR UPDATE ), updated_queue AS ( - UPDATE river_queue + UPDATE /* TEMPLATE: schema */river_queue SET paused_at = NULL, updated_at = now() @@ -203,7 +203,7 @@ updated_queue AS ( RETURNING river_queue.name, river_queue.created_at, river_queue.metadata, river_queue.paused_at, river_queue.updated_at ) SELECT name, created_at, metadata, paused_at, updated_at -FROM river_queue +FROM /* TEMPLATE: schema */river_queue WHERE name = $1 AND name NOT IN (SELECT name FROM updated_queue) UNION @@ -216,7 +216,7 @@ func (q *Queries) QueueResume(ctx context.Context, db DBTX, name string) (sql.Re } const queueUpdate = `-- name: QueueUpdate :one -UPDATE river_queue +UPDATE /* TEMPLATE: schema */river_queue SET metadata = CASE WHEN $1::boolean THEN $2::jsonb ELSE metadata END, updated_at = now() diff --git a/riverdriver/riverdatabasesql/migration/main/001_create_river_migration.down.sql b/riverdriver/riverdatabasesql/migration/main/001_create_river_migration.down.sql index 74c1984b..8bfe8202 100644 --- a/riverdriver/riverdatabasesql/migration/main/001_create_river_migration.down.sql +++ b/riverdriver/riverdatabasesql/migration/main/001_create_river_migration.down.sql @@ -1 +1 @@ -DROP TABLE river_migration; \ No newline at end of file +DROP TABLE /* TEMPLATE: schema */river_migration; \ No newline at end of file diff --git a/riverdriver/riverdatabasesql/migration/main/001_create_river_migration.up.sql b/riverdriver/riverdatabasesql/migration/main/001_create_river_migration.up.sql index 1e8a2f39..27006d56 100644 --- a/riverdriver/riverdatabasesql/migration/main/001_create_river_migration.up.sql +++ b/riverdriver/riverdatabasesql/migration/main/001_create_river_migration.up.sql @@ -1,8 +1,8 @@ -CREATE TABLE river_migration( +CREATE TABLE /* TEMPLATE: schema */river_migration( id bigserial PRIMARY KEY, created_at timestamptz NOT NULL DEFAULT NOW(), version bigint NOT NULL, CONSTRAINT version CHECK (version >= 1) ); -CREATE UNIQUE INDEX ON river_migration USING btree(version); \ No newline at end of file +CREATE UNIQUE INDEX ON /* TEMPLATE: schema */river_migration USING btree(version); \ No newline at end of file diff --git a/riverdriver/riverdatabasesql/migration/main/002_initial_schema.down.sql b/riverdriver/riverdatabasesql/migration/main/002_initial_schema.down.sql index bafef0ba..d334d8a6 100644 --- a/riverdriver/riverdatabasesql/migration/main/002_initial_schema.down.sql +++ b/riverdriver/riverdatabasesql/migration/main/002_initial_schema.down.sql @@ -1,5 +1,5 @@ -DROP TABLE river_job; -DROP FUNCTION river_job_notify; -DROP TYPE river_job_state; +DROP TABLE /* TEMPLATE: schema */river_job; +DROP FUNCTION /* TEMPLATE: schema */river_job_notify; +DROP TYPE /* TEMPLATE: schema */river_job_state; -DROP TABLE river_leader; \ No newline at end of file +DROP TABLE /* TEMPLATE: schema */river_leader; \ No newline at end of file diff --git a/riverdriver/riverdatabasesql/migration/main/002_initial_schema.up.sql b/riverdriver/riverdatabasesql/migration/main/002_initial_schema.up.sql index 57397e31..604e82ba 100644 --- a/riverdriver/riverdatabasesql/migration/main/002_initial_schema.up.sql +++ b/riverdriver/riverdatabasesql/migration/main/002_initial_schema.up.sql @@ -1,4 +1,4 @@ -CREATE TYPE river_job_state AS ENUM( +CREATE TYPE /* TEMPLATE: schema */river_job_state AS ENUM( 'available', 'cancelled', 'completed', @@ -8,7 +8,7 @@ CREATE TYPE river_job_state AS ENUM( 'scheduled' ); -CREATE TABLE river_job( +CREATE TABLE /* TEMPLATE: schema */river_job( -- 8 bytes id bigserial PRIMARY KEY, @@ -49,17 +49,17 @@ CREATE TABLE river_job( -- We may want to consider adding another property here after `kind` if it seems -- like it'd be useful for something. -CREATE INDEX river_job_kind ON river_job USING btree(kind); +CREATE INDEX river_job_kind ON /* TEMPLATE: schema */river_job USING btree(kind); -CREATE INDEX river_job_state_and_finalized_at_index ON river_job USING btree(state, finalized_at) WHERE finalized_at IS NOT NULL; +CREATE INDEX river_job_state_and_finalized_at_index ON /* TEMPLATE: schema */river_job USING btree(state, finalized_at) WHERE finalized_at IS NOT NULL; -CREATE INDEX river_job_prioritized_fetching_index ON river_job USING btree(state, queue, priority, scheduled_at, id); +CREATE INDEX river_job_prioritized_fetching_index ON /* TEMPLATE: schema */river_job USING btree(state, queue, priority, scheduled_at, id); -CREATE INDEX river_job_args_index ON river_job USING GIN(args); +CREATE INDEX river_job_args_index ON /* TEMPLATE: schema */river_job USING GIN(args); -CREATE INDEX river_job_metadata_index ON river_job USING GIN(metadata); +CREATE INDEX river_job_metadata_index ON /* TEMPLATE: schema */river_job USING GIN(metadata); -CREATE OR REPLACE FUNCTION river_job_notify() +CREATE OR REPLACE FUNCTION /* TEMPLATE: schema */river_job_notify() RETURNS TRIGGER AS $$ DECLARE @@ -78,11 +78,11 @@ $$ LANGUAGE plpgsql; CREATE TRIGGER river_notify - AFTER INSERT ON river_job + AFTER INSERT ON /* TEMPLATE: schema */river_job FOR EACH ROW - EXECUTE PROCEDURE river_job_notify(); + EXECUTE PROCEDURE /* TEMPLATE: schema */river_job_notify(); -CREATE UNLOGGED TABLE river_leader( +CREATE UNLOGGED TABLE /* TEMPLATE: schema */river_leader( -- 8 bytes each (no alignment needed) elected_at timestamptz NOT NULL, expires_at timestamptz NOT NULL, diff --git a/riverdriver/riverdatabasesql/migration/main/003_river_job_tags_non_null.down.sql b/riverdriver/riverdatabasesql/migration/main/003_river_job_tags_non_null.down.sql index 37a09255..acef65cb 100644 --- a/riverdriver/riverdatabasesql/migration/main/003_river_job_tags_non_null.down.sql +++ b/riverdriver/riverdatabasesql/migration/main/003_river_job_tags_non_null.down.sql @@ -1,2 +1,3 @@ -ALTER TABLE river_job ALTER COLUMN tags DROP NOT NULL, - ALTER COLUMN tags DROP DEFAULT; +ALTER TABLE /* TEMPLATE: schema */river_job + ALTER COLUMN tags DROP NOT NULL, + ALTER COLUMN tags DROP DEFAULT; diff --git a/riverdriver/riverdatabasesql/migration/main/003_river_job_tags_non_null.up.sql b/riverdriver/riverdatabasesql/migration/main/003_river_job_tags_non_null.up.sql index 5849bd55..0a472dde 100644 --- a/riverdriver/riverdatabasesql/migration/main/003_river_job_tags_non_null.up.sql +++ b/riverdriver/riverdatabasesql/migration/main/003_river_job_tags_non_null.up.sql @@ -1,3 +1,3 @@ -ALTER TABLE river_job ALTER COLUMN tags SET DEFAULT '{}'; -UPDATE river_job SET tags = '{}' WHERE tags IS NULL; -ALTER TABLE river_job ALTER COLUMN tags SET NOT NULL; +ALTER TABLE /* TEMPLATE: schema */river_job ALTER COLUMN tags SET DEFAULT '{}'; +UPDATE /* TEMPLATE: schema */river_job SET tags = '{}' WHERE tags IS NULL; +ALTER TABLE /* TEMPLATE: schema */river_job ALTER COLUMN tags SET NOT NULL; diff --git a/riverdriver/riverdatabasesql/migration/main/004_pending_and_more.down.sql b/riverdriver/riverdatabasesql/migration/main/004_pending_and_more.down.sql index 22d5e998..1b7ec7e8 100644 --- a/riverdriver/riverdatabasesql/migration/main/004_pending_and_more.down.sql +++ b/riverdriver/riverdatabasesql/migration/main/004_pending_and_more.down.sql @@ -1,17 +1,17 @@ -ALTER TABLE river_job ALTER COLUMN args DROP NOT NULL; +ALTER TABLE /* TEMPLATE: schema */river_job ALTER COLUMN args DROP NOT NULL; -ALTER TABLE river_job ALTER COLUMN metadata DROP NOT NULL; -ALTER TABLE river_job ALTER COLUMN metadata DROP DEFAULT; +ALTER TABLE /* TEMPLATE: schema */river_job ALTER COLUMN metadata DROP NOT NULL; +ALTER TABLE /* TEMPLATE: schema */river_job ALTER COLUMN metadata DROP DEFAULT; -- It is not possible to safely remove 'pending' from the river_job_state enum, -- so leave it in place. -ALTER TABLE river_job DROP CONSTRAINT finalized_or_finalized_at_null; -ALTER TABLE river_job ADD CONSTRAINT finalized_or_finalized_at_null CHECK ( +ALTER TABLE /* TEMPLATE: schema */river_job DROP CONSTRAINT finalized_or_finalized_at_null; +ALTER TABLE /* TEMPLATE: schema */river_job ADD CONSTRAINT finalized_or_finalized_at_null CHECK ( (state IN ('cancelled', 'completed', 'discarded') AND finalized_at IS NOT NULL) OR finalized_at IS NULL ); -CREATE OR REPLACE FUNCTION river_job_notify() +CREATE OR REPLACE FUNCTION /* TEMPLATE: schema */river_job_notify() RETURNS TRIGGER AS $$ DECLARE @@ -30,13 +30,13 @@ $$ LANGUAGE plpgsql; CREATE TRIGGER river_notify - AFTER INSERT ON river_job + AFTER INSERT ON /* TEMPLATE: schema */river_job FOR EACH ROW - EXECUTE PROCEDURE river_job_notify(); + EXECUTE PROCEDURE /* TEMPLATE: schema */river_job_notify(); -DROP TABLE river_queue; +DROP TABLE /* TEMPLATE: schema */river_queue; -ALTER TABLE river_leader +ALTER TABLE /* TEMPLATE: schema */river_leader ALTER COLUMN name DROP DEFAULT, DROP CONSTRAINT name_length, ADD CONSTRAINT name_length CHECK (char_length(name) > 0 AND char_length(name) < 128); \ No newline at end of file diff --git a/riverdriver/riverdatabasesql/migration/main/004_pending_and_more.up.sql b/riverdriver/riverdatabasesql/migration/main/004_pending_and_more.up.sql index b5cf9a91..f98ff7d0 100644 --- a/riverdriver/riverdatabasesql/migration/main/004_pending_and_more.up.sql +++ b/riverdriver/riverdatabasesql/migration/main/004_pending_and_more.up.sql @@ -1,29 +1,29 @@ -- The args column never had a NOT NULL constraint or default value at the -- database level, though we tried to ensure one at the application level. -ALTER TABLE river_job ALTER COLUMN args SET DEFAULT '{}'; -UPDATE river_job SET args = '{}' WHERE args IS NULL; -ALTER TABLE river_job ALTER COLUMN args SET NOT NULL; -ALTER TABLE river_job ALTER COLUMN args DROP DEFAULT; +ALTER TABLE /* TEMPLATE: schema */river_job ALTER COLUMN args SET DEFAULT '{}'; +UPDATE /* TEMPLATE: schema */river_job SET args = '{}' WHERE args IS NULL; +ALTER TABLE /* TEMPLATE: schema */river_job ALTER COLUMN args SET NOT NULL; +ALTER TABLE /* TEMPLATE: schema */river_job ALTER COLUMN args DROP DEFAULT; -- The metadata column never had a NOT NULL constraint or default value at the -- database level, though we tried to ensure one at the application level. -ALTER TABLE river_job ALTER COLUMN metadata SET DEFAULT '{}'; -UPDATE river_job SET metadata = '{}' WHERE metadata IS NULL; -ALTER TABLE river_job ALTER COLUMN metadata SET NOT NULL; +ALTER TABLE /* TEMPLATE: schema */river_job ALTER COLUMN metadata SET DEFAULT '{}'; +UPDATE /* TEMPLATE: schema */river_job SET metadata = '{}' WHERE metadata IS NULL; +ALTER TABLE /* TEMPLATE: schema */river_job ALTER COLUMN metadata SET NOT NULL; -- The 'pending' job state will be used for upcoming functionality: -ALTER TYPE river_job_state ADD VALUE IF NOT EXISTS 'pending' AFTER 'discarded'; +ALTER TYPE /* TEMPLATE: schema */river_job_state ADD VALUE IF NOT EXISTS 'pending' AFTER 'discarded'; -ALTER TABLE river_job DROP CONSTRAINT finalized_or_finalized_at_null; -ALTER TABLE river_job ADD CONSTRAINT finalized_or_finalized_at_null CHECK ( +ALTER TABLE /* TEMPLATE: schema */river_job DROP CONSTRAINT finalized_or_finalized_at_null; +ALTER TABLE /* TEMPLATE: schema */river_job ADD 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')) ); -DROP TRIGGER river_notify ON river_job; -DROP FUNCTION river_job_notify; +DROP TRIGGER river_notify ON /* TEMPLATE: schema */river_job; +DROP FUNCTION /* TEMPLATE: schema */river_job_notify; -CREATE TABLE river_queue( +CREATE TABLE /* TEMPLATE: schema */river_queue( name text PRIMARY KEY NOT NULL, created_at timestamptz NOT NULL DEFAULT NOW(), metadata jsonb NOT NULL DEFAULT '{}' ::jsonb, @@ -31,7 +31,7 @@ CREATE TABLE river_queue( updated_at timestamptz NOT NULL ); -ALTER TABLE river_leader +ALTER TABLE /* TEMPLATE: schema */river_leader ALTER COLUMN name SET DEFAULT 'default', DROP CONSTRAINT name_length, ADD CONSTRAINT name_length CHECK (name = 'default'); \ No newline at end of file diff --git a/riverdriver/riverdatabasesql/migration/main/005_migration_unique_client.down.sql b/riverdriver/riverdatabasesql/migration/main/005_migration_unique_client.down.sql index c3ba58c6..b8e041d5 100644 --- a/riverdriver/riverdatabasesql/migration/main/005_migration_unique_client.down.sql +++ b/riverdriver/riverdatabasesql/migration/main/005_migration_unique_client.down.sql @@ -10,33 +10,33 @@ BEGIN -- Tolerate users who may be using their own migration system rather than -- River's. If they are, they will have skipped version 001 containing -- `CREATE TABLE river_migration`, so this table won't exist. - IF (SELECT to_regclass('river_migration') IS NOT NULL) THEN + IF (SELECT to_regclass('/* TEMPLATE: schema */river_migration') IS NOT NULL) THEN IF EXISTS ( SELECT * - FROM river_migration + FROM /* TEMPLATE: schema */river_migration WHERE line <> 'main' ) THEN RAISE EXCEPTION 'Found non-main migration lines in the database; version 005 migration is irreversible because it would result in loss of migration information.'; END IF; - ALTER TABLE river_migration + ALTER TABLE /* TEMPLATE: schema */river_migration RENAME TO river_migration_old; - CREATE TABLE river_migration( + CREATE TABLE /* TEMPLATE: schema */river_migration( id bigserial PRIMARY KEY, created_at timestamptz NOT NULL DEFAULT NOW(), version bigint NOT NULL, CONSTRAINT version CHECK (version >= 1) ); - CREATE UNIQUE INDEX ON river_migration USING btree(version); + CREATE UNIQUE INDEX ON /* TEMPLATE: schema */river_migration USING btree(version); - INSERT INTO river_migration + INSERT INTO /* TEMPLATE: schema */river_migration (created_at, version) SELECT created_at, version - FROM river_migration_old; + FROM /* TEMPLATE: schema */river_migration_old; - DROP TABLE river_migration_old; + DROP TABLE /* TEMPLATE: schema */river_migration_old; END IF; END; $body$ @@ -46,12 +46,12 @@ LANGUAGE 'plpgsql'; -- Drop `river_job.unique_key`. -- -ALTER TABLE river_job +ALTER TABLE /* TEMPLATE: schema */river_job DROP COLUMN unique_key; -- -- Drop `river_client` and derivative. -- -DROP TABLE river_client_queue; -DROP TABLE river_client; +DROP TABLE /* TEMPLATE: schema */river_client_queue; +DROP TABLE /* TEMPLATE: schema */river_client; diff --git a/riverdriver/riverdatabasesql/migration/main/005_migration_unique_client.up.sql b/riverdriver/riverdatabasesql/migration/main/005_migration_unique_client.up.sql index 50601651..ff964304 100644 --- a/riverdriver/riverdatabasesql/migration/main/005_migration_unique_client.up.sql +++ b/riverdriver/riverdatabasesql/migration/main/005_migration_unique_client.up.sql @@ -8,11 +8,11 @@ BEGIN -- Tolerate users who may be using their own migration system rather than -- River's. If they are, they will have skipped version 001 containing -- `CREATE TABLE river_migration`, so this table won't exist. - IF (SELECT to_regclass('river_migration') IS NOT NULL) THEN - ALTER TABLE river_migration + IF (SELECT to_regclass('/* TEMPLATE: schema */river_migration') IS NOT NULL) THEN + ALTER TABLE /* TEMPLATE: schema */river_migration RENAME TO river_migration_old; - CREATE TABLE river_migration( + CREATE TABLE /* TEMPLATE: schema */river_migration( line TEXT NOT NULL, version bigint NOT NULL, created_at timestamptz NOT NULL DEFAULT NOW(), @@ -21,12 +21,12 @@ BEGIN PRIMARY KEY (line, version) ); - INSERT INTO river_migration + INSERT INTO /* TEMPLATE: schema */river_migration (created_at, line, version) SELECT created_at, 'main', version - FROM river_migration_old; + FROM /* TEMPLATE: schema */river_migration_old; - DROP TABLE river_migration_old; + DROP TABLE /* TEMPLATE: schema */river_migration_old; END IF; END; $body$ @@ -39,10 +39,10 @@ LANGUAGE 'plpgsql'; -- These statements use `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. -ALTER TABLE river_job +ALTER TABLE /* TEMPLATE: schema */river_job ADD COLUMN IF NOT EXISTS unique_key bytea; -CREATE UNIQUE INDEX IF NOT EXISTS river_job_kind_unique_key_idx ON river_job (kind, unique_key) WHERE unique_key IS NOT NULL; +CREATE UNIQUE INDEX IF NOT EXISTS river_job_kind_unique_key_idx ON /* TEMPLATE: schema */river_job (kind, unique_key) WHERE unique_key IS NOT NULL; -- -- Create `river_client` and derivative. @@ -52,7 +52,7 @@ CREATE UNIQUE INDEX IF NOT EXISTS river_job_kind_unique_key_idx ON river_job (ki -- additional migration. -- -CREATE UNLOGGED TABLE river_client ( +CREATE UNLOGGED TABLE /* TEMPLATE: schema */river_client ( id text PRIMARY KEY NOT NULL, created_at timestamptz NOT NULL DEFAULT now(), metadata jsonb NOT NULL DEFAULT '{}', @@ -63,7 +63,7 @@ CREATE UNLOGGED TABLE river_client ( -- Differs from `river_queue` in that it tracks the queue state for a particular -- active client. -CREATE UNLOGGED TABLE river_client_queue ( +CREATE UNLOGGED TABLE /* TEMPLATE: schema */river_client_queue ( river_client_id text NOT NULL REFERENCES river_client (id) ON DELETE CASCADE, name text NOT NULL, created_at timestamptz NOT NULL DEFAULT now(), diff --git a/riverdriver/riverdatabasesql/migration/main/006_bulk_unique.down.sql b/riverdriver/riverdatabasesql/migration/main/006_bulk_unique.down.sql index ae65cfed..26cd8434 100644 --- a/riverdriver/riverdatabasesql/migration/main/006_bulk_unique.down.sql +++ b/riverdriver/riverdatabasesql/migration/main/006_bulk_unique.down.sql @@ -3,14 +3,14 @@ -- Drop `river_job.unique_states` and its index. -- -DROP INDEX river_job_unique_idx; +DROP INDEX /* TEMPLATE: schema */river_job_unique_idx; -ALTER TABLE river_job +ALTER TABLE /* TEMPLATE: schema */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; +CREATE UNIQUE INDEX IF NOT EXISTS river_job_kind_unique_key_idx ON /* TEMPLATE: schema */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; +DROP FUNCTION /* TEMPLATE: schema */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 index 29c1a3cb..15f0ee53 100644 --- a/riverdriver/riverdatabasesql/migration/main/006_bulk_unique.up.sql +++ b/riverdriver/riverdatabasesql/migration/main/006_bulk_unique.up.sql @@ -1,5 +1,5 @@ -CREATE OR REPLACE FUNCTION river_job_state_in_bitmask(bitmask BIT(8), state river_job_state) +CREATE OR REPLACE FUNCTION /* TEMPLATE: schema */river_job_state_in_bitmask(bitmask BIT(8), state river_job_state) RETURNS boolean LANGUAGE SQL IMMUTABLE @@ -23,12 +23,12 @@ $$; -- This column may exist already if users manually created the column and index -- as instructed in the changelog so the index could be created `CONCURRENTLY`. -- -ALTER TABLE river_job ADD COLUMN IF NOT EXISTS unique_states BIT(8); +ALTER TABLE /* TEMPLATE: schema */river_job ADD COLUMN IF NOT EXISTS unique_states BIT(8); -- This statement 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) +CREATE UNIQUE INDEX IF NOT EXISTS river_job_unique_idx ON /* TEMPLATE: schema */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); @@ -38,4 +38,4 @@ CREATE UNIQUE INDEX IF NOT EXISTS river_job_unique_idx ON river_job (unique_key) -- 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; +DROP INDEX /* TEMPLATE: schema */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 23f6aa54..f77698cc 100644 --- a/riverdriver/riverdatabasesql/river_database_sql_driver.go +++ b/riverdriver/riverdatabasesql/river_database_sql_driver.go @@ -54,7 +54,10 @@ func (d *Driver) GetExecutor() riverdriver.Executor { return &Executor{d.dbPool, templateReplaceWrapper{d.dbPool, &d.replacer}, d} } -func (d *Driver) GetListener() riverdriver.Listener { panic(riverdriver.ErrNotImplemented) } +func (d *Driver) GetListener(schema string) riverdriver.Listener { + panic(riverdriver.ErrNotImplemented) +} + func (d *Driver) GetMigrationFS(line string) fs.FS { if line == riverdriver.MigrationLineMain { return migrationFS @@ -91,10 +94,19 @@ func (e *Executor) Begin(ctx context.Context) (riverdriver.ExecutorTx, error) { return &ExecutorTx{Executor: Executor{nil, templateReplaceWrapper{tx, &e.driver.replacer}, e.driver}, tx: tx}, nil } -func (e *Executor) ColumnExists(ctx context.Context, tableName, columnName string) (bool, error) { +func (e *Executor) ColumnExists(ctx context.Context, params *riverdriver.ColumnExistsParams) (bool, error) { + // Schema injection is a bit different on this one because we're querying a table with a schema name. + schema := "CURRENT_SCHEMA" + if params.Schema != "" { + schema = "'" + params.Schema + "'" + } + ctx = sqlctemplate.WithReplacements(ctx, map[string]sqlctemplate.Replacement{ + "schema": {Value: schema}, + }, nil) + exists, err := dbsqlc.New().ColumnExists(ctx, e.dbtx, &dbsqlc.ColumnExistsParams{ - ColumnName: columnName, - TableName: tableName, + ColumnName: params.Column, + TableName: params.Table, }) return exists, interpretError(err) } @@ -110,9 +122,11 @@ func (e *Executor) JobCancel(ctx context.Context, params *riverdriver.JobCancelP return nil, err } - job, err := dbsqlc.New().JobCancel(ctx, e.dbtx, &dbsqlc.JobCancelParams{ + job, err := dbsqlc.New().JobCancel(schemaTemplateParam(ctx, params.Schema), e.dbtx, &dbsqlc.JobCancelParams{ ID: params.ID, CancelAttemptedAt: string(cancelledAt), + ControlTopic: params.ControlTopic, + Schema: sql.NullString{String: params.Schema, Valid: params.Schema != ""}, }) if err != nil { return nil, interpretError(err) @@ -120,16 +134,16 @@ func (e *Executor) JobCancel(ctx context.Context, params *riverdriver.JobCancelP return jobRowFromInternal(job) } -func (e *Executor) JobCountByState(ctx context.Context, state rivertype.JobState) (int, error) { - numJobs, err := dbsqlc.New().JobCountByState(ctx, e.dbtx, dbsqlc.RiverJobState(state)) +func (e *Executor) JobCountByState(ctx context.Context, params *riverdriver.JobCountByStateParams) (int, error) { + numJobs, err := dbsqlc.New().JobCountByState(schemaTemplateParam(ctx, params.Schema), e.dbtx, dbsqlc.RiverJobState(params.State)) if err != nil { return 0, err } return int(numJobs), nil } -func (e *Executor) JobDelete(ctx context.Context, id int64) (*rivertype.JobRow, error) { - job, err := dbsqlc.New().JobDelete(ctx, e.dbtx, id) +func (e *Executor) JobDelete(ctx context.Context, params *riverdriver.JobDeleteParams) (*rivertype.JobRow, error) { + job, err := dbsqlc.New().JobDelete(schemaTemplateParam(ctx, params.Schema), e.dbtx, params.ID) if err != nil { return nil, interpretError(err) } @@ -140,7 +154,7 @@ func (e *Executor) JobDelete(ctx context.Context, id int64) (*rivertype.JobRow, } func (e *Executor) JobDeleteBefore(ctx context.Context, params *riverdriver.JobDeleteBeforeParams) (int, error) { - numDeleted, err := dbsqlc.New().JobDeleteBefore(ctx, e.dbtx, &dbsqlc.JobDeleteBeforeParams{ + numDeleted, err := dbsqlc.New().JobDeleteBefore(schemaTemplateParam(ctx, params.Schema), e.dbtx, &dbsqlc.JobDeleteBeforeParams{ CancelledFinalizedAtHorizon: params.CancelledFinalizedAtHorizon, CompletedFinalizedAtHorizon: params.CompletedFinalizedAtHorizon, DiscardedFinalizedAtHorizon: params.DiscardedFinalizedAtHorizon, @@ -150,7 +164,7 @@ func (e *Executor) JobDeleteBefore(ctx context.Context, params *riverdriver.JobD } func (e *Executor) JobGetAvailable(ctx context.Context, params *riverdriver.JobGetAvailableParams) ([]*rivertype.JobRow, error) { - jobs, err := dbsqlc.New().JobGetAvailable(ctx, e.dbtx, &dbsqlc.JobGetAvailableParams{ + jobs, err := dbsqlc.New().JobGetAvailable(schemaTemplateParam(ctx, params.Schema), e.dbtx, &dbsqlc.JobGetAvailableParams{ AttemptedBy: params.ClientID, Max: int32(min(params.Max, math.MaxInt32)), //nolint:gosec Now: params.Now, @@ -162,43 +176,24 @@ func (e *Executor) JobGetAvailable(ctx context.Context, params *riverdriver.JobG return mapSliceError(jobs, jobRowFromInternal) } -func (e *Executor) JobGetByID(ctx context.Context, id int64) (*rivertype.JobRow, error) { - job, err := dbsqlc.New().JobGetByID(ctx, e.dbtx, id) +func (e *Executor) JobGetByID(ctx context.Context, params *riverdriver.JobGetByIDParams) (*rivertype.JobRow, error) { + job, err := dbsqlc.New().JobGetByID(schemaTemplateParam(ctx, params.Schema), e.dbtx, params.ID) if err != nil { return nil, interpretError(err) } return jobRowFromInternal(job) } -func (e *Executor) JobGetByIDMany(ctx context.Context, id []int64) ([]*rivertype.JobRow, error) { - jobs, err := dbsqlc.New().JobGetByIDMany(ctx, e.dbtx, id) +func (e *Executor) JobGetByIDMany(ctx context.Context, params *riverdriver.JobGetByIDManyParams) ([]*rivertype.JobRow, error) { + jobs, err := dbsqlc.New().JobGetByIDMany(schemaTemplateParam(ctx, params.Schema), e.dbtx, params.ID) if err != nil { return nil, interpretError(err) } return mapSliceError(jobs, jobRowFromInternal) } -func (e *Executor) JobGetByKindAndUniqueProperties(ctx context.Context, params *riverdriver.JobGetByKindAndUniquePropertiesParams) (*rivertype.JobRow, error) { - job, err := dbsqlc.New().JobGetByKindAndUniqueProperties(ctx, e.dbtx, &dbsqlc.JobGetByKindAndUniquePropertiesParams{ - Args: valutil.ValOrDefault(string(params.Args), "{}"), - ByArgs: params.ByArgs, - ByCreatedAt: params.ByCreatedAt, - ByQueue: params.ByQueue, - ByState: params.ByState, - CreatedAtBegin: params.CreatedAtBegin, - CreatedAtEnd: params.CreatedAtEnd, - Kind: params.Kind, - Queue: params.Queue, - State: params.State, - }) - if err != nil { - return nil, interpretError(err) - } - return jobRowFromInternal(job) -} - -func (e *Executor) JobGetByKindMany(ctx context.Context, kind []string) ([]*rivertype.JobRow, error) { - jobs, err := dbsqlc.New().JobGetByKindMany(ctx, e.dbtx, kind) +func (e *Executor) JobGetByKindMany(ctx context.Context, params *riverdriver.JobGetByKindManyParams) ([]*rivertype.JobRow, error) { + jobs, err := dbsqlc.New().JobGetByKindMany(schemaTemplateParam(ctx, params.Schema), e.dbtx, params.Kind) if err != nil { return nil, interpretError(err) } @@ -206,32 +201,32 @@ func (e *Executor) JobGetByKindMany(ctx context.Context, kind []string) ([]*rive } func (e *Executor) JobGetStuck(ctx context.Context, params *riverdriver.JobGetStuckParams) ([]*rivertype.JobRow, error) { - jobs, err := dbsqlc.New().JobGetStuck(ctx, e.dbtx, &dbsqlc.JobGetStuckParams{Max: int32(min(params.Max, math.MaxInt32)), StuckHorizon: params.StuckHorizon}) //nolint:gosec + jobs, err := dbsqlc.New().JobGetStuck(schemaTemplateParam(ctx, params.Schema), e.dbtx, &dbsqlc.JobGetStuckParams{Max: int32(min(params.Max, math.MaxInt32)), StuckHorizon: params.StuckHorizon}) //nolint:gosec if err != nil { return nil, interpretError(err) } return mapSliceError(jobs, jobRowFromInternal) } -func (e *Executor) JobInsertFastMany(ctx context.Context, params []*riverdriver.JobInsertFastParams) ([]*riverdriver.JobInsertFastResult, error) { +func (e *Executor) JobInsertFastMany(ctx context.Context, params *riverdriver.JobInsertFastManyParams) ([]*riverdriver.JobInsertFastResult, error) { insertJobsParams := &dbsqlc.JobInsertFastManyParams{ - Args: make([]string, len(params)), - CreatedAt: make([]time.Time, 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([]pgtypealias.NullBytea, len(params)), - UniqueStates: make([]pgtypealias.Bits, len(params)), + Args: make([]string, len(params.Jobs)), + CreatedAt: make([]time.Time, len(params.Jobs)), + Kind: make([]string, len(params.Jobs)), + MaxAttempts: make([]int16, len(params.Jobs)), + Metadata: make([]string, len(params.Jobs)), + Priority: make([]int16, len(params.Jobs)), + Queue: make([]string, len(params.Jobs)), + ScheduledAt: make([]time.Time, len(params.Jobs)), + State: make([]string, len(params.Jobs)), + Tags: make([]string, len(params.Jobs)), + UniqueKey: make([]pgtypealias.NullBytea, len(params.Jobs)), + UniqueStates: make([]pgtypealias.Bits, len(params.Jobs)), } now := time.Now().UTC() - for i := 0; i < len(params); i++ { - params := params[i] + for i := range len(params.Jobs) { + params := params.Jobs[i] createdAt := now if params.CreatedAt != nil { @@ -264,7 +259,7 @@ func (e *Executor) JobInsertFastMany(ctx context.Context, params []*riverdriver. insertJobsParams.UniqueStates[i] = pgtypealias.Bits{Bits: pgtype.Bits{Bytes: []byte{params.UniqueStates}, Len: 8, Valid: params.UniqueStates != 0}} } - items, err := dbsqlc.New().JobInsertFastMany(ctx, e.dbtx, insertJobsParams) + items, err := dbsqlc.New().JobInsertFastMany(schemaTemplateParam(ctx, params.Schema), e.dbtx, insertJobsParams) if err != nil { return nil, interpretError(err) } @@ -278,25 +273,25 @@ func (e *Executor) JobInsertFastMany(ctx context.Context, params []*riverdriver. }) } -func (e *Executor) JobInsertFastManyNoReturning(ctx context.Context, params []*riverdriver.JobInsertFastParams) (int, error) { +func (e *Executor) JobInsertFastManyNoReturning(ctx context.Context, params *riverdriver.JobInsertFastManyParams) (int, error) { insertJobsParams := &dbsqlc.JobInsertFastManyNoReturningParams{ - Args: make([]string, len(params)), - CreatedAt: make([]time.Time, 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([]pgtypealias.NullBytea, len(params)), - UniqueStates: make([]pgtypealias.Bits, len(params)), + Args: make([]string, len(params.Jobs)), + CreatedAt: make([]time.Time, len(params.Jobs)), + Kind: make([]string, len(params.Jobs)), + MaxAttempts: make([]int16, len(params.Jobs)), + Metadata: make([]string, len(params.Jobs)), + Priority: make([]int16, len(params.Jobs)), + Queue: make([]string, len(params.Jobs)), + ScheduledAt: make([]time.Time, len(params.Jobs)), + State: make([]dbsqlc.RiverJobState, len(params.Jobs)), + Tags: make([]string, len(params.Jobs)), + UniqueKey: make([]pgtypealias.NullBytea, len(params.Jobs)), + UniqueStates: make([]pgtypealias.Bits, len(params.Jobs)), } now := time.Now().UTC() - for i := 0; i < len(params); i++ { - params := params[i] + for i := range len(params.Jobs) { + params := params.Jobs[i] createdAt := now if params.CreatedAt != nil { @@ -329,7 +324,7 @@ func (e *Executor) JobInsertFastManyNoReturning(ctx context.Context, params []*r insertJobsParams.UniqueStates[i] = pgtypealias.Bits{Bits: pgtype.Bits{Bytes: []byte{params.UniqueStates}, Len: 8, Valid: params.UniqueStates != 0}} } - numInserted, err := dbsqlc.New().JobInsertFastManyNoReturning(ctx, e.dbtx, insertJobsParams) + numInserted, err := dbsqlc.New().JobInsertFastManyNoReturning(schemaTemplateParam(ctx, params.Schema), e.dbtx, insertJobsParams) if err != nil { return 0, interpretError(err) } @@ -338,7 +333,7 @@ func (e *Executor) JobInsertFastManyNoReturning(ctx context.Context, params []*r } func (e *Executor) JobInsertFull(ctx context.Context, params *riverdriver.JobInsertFullParams) (*rivertype.JobRow, error) { - job, err := dbsqlc.New().JobInsertFull(ctx, e.dbtx, &dbsqlc.JobInsertFullParams{ + job, err := dbsqlc.New().JobInsertFull(schemaTemplateParam(ctx, params.Schema), e.dbtx, &dbsqlc.JobInsertFullParams{ Attempt: int16(min(params.Attempt, math.MaxInt16)), //nolint:gosec AttemptedAt: params.AttemptedAt, AttemptedBy: params.AttemptedBy, @@ -374,7 +369,7 @@ func (e *Executor) JobList(ctx context.Context, params *riverdriver.JobListParam "where_clause": {Value: whereClause}, }, nil) // named params not passed because they've already been replaced above - jobs, err := dbsqlc.New().JobList(ctx, e.dbtx, params.Max) + jobs, err := dbsqlc.New().JobList(schemaTemplateParam(ctx, params.Schema), e.dbtx, params.Max) if err != nil { return nil, interpretError(err) } @@ -474,7 +469,7 @@ func replaceNamed(query string, namedArgs map[string]any) (string, error) { } func (e *Executor) JobRescueMany(ctx context.Context, params *riverdriver.JobRescueManyParams) (*struct{}, error) { - err := dbsqlc.New().JobRescueMany(ctx, e.dbtx, &dbsqlc.JobRescueManyParams{ + err := dbsqlc.New().JobRescueMany(schemaTemplateParam(ctx, params.Schema), e.dbtx, &dbsqlc.JobRescueManyParams{ ID: params.ID, Error: sliceutil.Map(params.Error, func(e []byte) string { return string(e) }), FinalizedAt: params.FinalizedAt, @@ -487,8 +482,8 @@ func (e *Executor) JobRescueMany(ctx context.Context, params *riverdriver.JobRes return &struct{}{}, nil } -func (e *Executor) JobRetry(ctx context.Context, id int64) (*rivertype.JobRow, error) { - job, err := dbsqlc.New().JobRetry(ctx, e.dbtx, id) +func (e *Executor) JobRetry(ctx context.Context, params *riverdriver.JobRetryParams) (*rivertype.JobRow, error) { + job, err := dbsqlc.New().JobRetry(schemaTemplateParam(ctx, params.Schema), e.dbtx, params.ID) if err != nil { return nil, interpretError(err) } @@ -496,7 +491,7 @@ func (e *Executor) JobRetry(ctx context.Context, id int64) (*rivertype.JobRow, e } func (e *Executor) JobSchedule(ctx context.Context, params *riverdriver.JobScheduleParams) ([]*riverdriver.JobScheduleResult, error) { - scheduleResults, err := dbsqlc.New().JobSchedule(ctx, e.dbtx, &dbsqlc.JobScheduleParams{ + scheduleResults, err := dbsqlc.New().JobSchedule(schemaTemplateParam(ctx, params.Schema), e.dbtx, &dbsqlc.JobScheduleParams{ Max: int64(params.Max), Now: params.Now, }) @@ -558,7 +553,7 @@ func (e *Executor) JobSetStateIfRunningMany(ctx context.Context, params *riverdr setStateParams.State[i] = string(params.State[i]) } - jobs, err := dbsqlc.New().JobSetStateIfRunningMany(ctx, e.dbtx, setStateParams) + jobs, err := dbsqlc.New().JobSetStateIfRunningMany(schemaTemplateParam(ctx, params.Schema), e.dbtx, setStateParams) if err != nil { return nil, interpretError(err) } @@ -566,7 +561,7 @@ func (e *Executor) JobSetStateIfRunningMany(ctx context.Context, params *riverdr } func (e *Executor) JobUpdate(ctx context.Context, params *riverdriver.JobUpdateParams) (*rivertype.JobRow, error) { - job, err := dbsqlc.New().JobUpdate(ctx, e.dbtx, &dbsqlc.JobUpdateParams{ + job, err := dbsqlc.New().JobUpdate(schemaTemplateParam(ctx, params.Schema), e.dbtx, &dbsqlc.JobUpdateParams{ ID: params.ID, Attempt: int16(min(params.Attempt, math.MaxInt16)), //nolint:gosec AttemptDoUpdate: params.AttemptDoUpdate, @@ -589,7 +584,7 @@ func (e *Executor) JobUpdate(ctx context.Context, params *riverdriver.JobUpdateP } func (e *Executor) LeaderAttemptElect(ctx context.Context, params *riverdriver.LeaderElectParams) (bool, error) { - numElectionsWon, err := dbsqlc.New().LeaderAttemptElect(ctx, e.dbtx, &dbsqlc.LeaderAttemptElectParams{ + numElectionsWon, err := dbsqlc.New().LeaderAttemptElect(schemaTemplateParam(ctx, params.Schema), e.dbtx, &dbsqlc.LeaderAttemptElectParams{ LeaderID: params.LeaderID, TTL: params.TTL, }) @@ -600,7 +595,7 @@ func (e *Executor) LeaderAttemptElect(ctx context.Context, params *riverdriver.L } func (e *Executor) LeaderAttemptReelect(ctx context.Context, params *riverdriver.LeaderElectParams) (bool, error) { - numElectionsWon, err := dbsqlc.New().LeaderAttemptReelect(ctx, e.dbtx, &dbsqlc.LeaderAttemptReelectParams{ + numElectionsWon, err := dbsqlc.New().LeaderAttemptReelect(schemaTemplateParam(ctx, params.Schema), e.dbtx, &dbsqlc.LeaderAttemptReelectParams{ LeaderID: params.LeaderID, TTL: params.TTL, }) @@ -610,16 +605,16 @@ func (e *Executor) LeaderAttemptReelect(ctx context.Context, params *riverdriver return numElectionsWon > 0, nil } -func (e *Executor) LeaderDeleteExpired(ctx context.Context) (int, error) { - numDeleted, err := dbsqlc.New().LeaderDeleteExpired(ctx, e.dbtx) +func (e *Executor) LeaderDeleteExpired(ctx context.Context, params *riverdriver.LeaderDeleteExpiredParams) (int, error) { + numDeleted, err := dbsqlc.New().LeaderDeleteExpired(schemaTemplateParam(ctx, params.Schema), e.dbtx) if err != nil { return 0, interpretError(err) } return int(numDeleted), nil } -func (e *Executor) LeaderGetElectedLeader(ctx context.Context) (*riverdriver.Leader, error) { - leader, err := dbsqlc.New().LeaderGetElectedLeader(ctx, e.dbtx) +func (e *Executor) LeaderGetElectedLeader(ctx context.Context, params *riverdriver.LeaderGetElectedLeaderParams) (*riverdriver.Leader, error) { + leader, err := dbsqlc.New().LeaderGetElectedLeader(schemaTemplateParam(ctx, params.Schema), e.dbtx) if err != nil { return nil, interpretError(err) } @@ -627,7 +622,7 @@ func (e *Executor) LeaderGetElectedLeader(ctx context.Context) (*riverdriver.Lea } func (e *Executor) LeaderInsert(ctx context.Context, params *riverdriver.LeaderInsertParams) (*riverdriver.Leader, error) { - leader, err := dbsqlc.New().LeaderInsert(ctx, e.dbtx, &dbsqlc.LeaderInsertParams{ + leader, err := dbsqlc.New().LeaderInsert(schemaTemplateParam(ctx, params.Schema), e.dbtx, &dbsqlc.LeaderInsertParams{ ElectedAt: params.ElectedAt, ExpiresAt: params.ExpiresAt, LeaderID: params.LeaderID, @@ -640,9 +635,10 @@ func (e *Executor) LeaderInsert(ctx context.Context, params *riverdriver.LeaderI } func (e *Executor) LeaderResign(ctx context.Context, params *riverdriver.LeaderResignParams) (bool, error) { - numResigned, err := dbsqlc.New().LeaderResign(ctx, e.dbtx, &dbsqlc.LeaderResignParams{ + numResigned, err := dbsqlc.New().LeaderResign(schemaTemplateParam(ctx, params.Schema), e.dbtx, &dbsqlc.LeaderResignParams{ LeaderID: params.LeaderID, LeadershipTopic: params.LeadershipTopic, + Schema: sql.NullString{String: params.Schema, Valid: params.Schema != ""}, }) if err != nil { return false, interpretError(err) @@ -650,9 +646,9 @@ func (e *Executor) LeaderResign(ctx context.Context, params *riverdriver.LeaderR return numResigned > 0, nil } -func (e *Executor) MigrationDeleteAssumingMainMany(ctx context.Context, versions []int) ([]*riverdriver.Migration, error) { - migrations, err := dbsqlc.New().RiverMigrationDeleteAssumingMainMany(ctx, e.dbtx, - sliceutil.Map(versions, func(v int) int64 { return int64(v) })) +func (e *Executor) MigrationDeleteAssumingMainMany(ctx context.Context, params *riverdriver.MigrationDeleteAssumingMainManyParams) ([]*riverdriver.Migration, error) { + migrations, err := dbsqlc.New().RiverMigrationDeleteAssumingMainMany(schemaTemplateParam(ctx, params.Schema), e.dbtx, + sliceutil.Map(params.Versions, func(v int) int64 { return int64(v) })) if err != nil { return nil, interpretError(err) } @@ -665,10 +661,10 @@ func (e *Executor) MigrationDeleteAssumingMainMany(ctx context.Context, versions }), nil } -func (e *Executor) MigrationDeleteByLineAndVersionMany(ctx context.Context, line string, versions []int) ([]*riverdriver.Migration, error) { - migrations, err := dbsqlc.New().RiverMigrationDeleteByLineAndVersionMany(ctx, e.dbtx, &dbsqlc.RiverMigrationDeleteByLineAndVersionManyParams{ - Line: line, - Version: sliceutil.Map(versions, func(v int) int64 { return int64(v) }), +func (e *Executor) MigrationDeleteByLineAndVersionMany(ctx context.Context, params *riverdriver.MigrationDeleteByLineAndVersionManyParams) ([]*riverdriver.Migration, error) { + migrations, err := dbsqlc.New().RiverMigrationDeleteByLineAndVersionMany(schemaTemplateParam(ctx, params.Schema), e.dbtx, &dbsqlc.RiverMigrationDeleteByLineAndVersionManyParams{ + Line: params.Line, + Version: sliceutil.Map(params.Versions, func(v int) int64 { return int64(v) }), }) if err != nil { return nil, interpretError(err) @@ -676,8 +672,8 @@ func (e *Executor) MigrationDeleteByLineAndVersionMany(ctx context.Context, line return sliceutil.Map(migrations, migrationFromInternal), nil } -func (e *Executor) MigrationGetAllAssumingMain(ctx context.Context) ([]*riverdriver.Migration, error) { - migrations, err := dbsqlc.New().RiverMigrationGetAllAssumingMain(ctx, e.dbtx) +func (e *Executor) MigrationGetAllAssumingMain(ctx context.Context, params *riverdriver.MigrationGetAllAssumingMainParams) ([]*riverdriver.Migration, error) { + migrations, err := dbsqlc.New().RiverMigrationGetAllAssumingMain(schemaTemplateParam(ctx, params.Schema), e.dbtx) if err != nil { return nil, interpretError(err) } @@ -690,18 +686,18 @@ func (e *Executor) MigrationGetAllAssumingMain(ctx context.Context) ([]*riverdri }), nil } -func (e *Executor) MigrationGetByLine(ctx context.Context, line string) ([]*riverdriver.Migration, error) { - migrations, err := dbsqlc.New().RiverMigrationGetByLine(ctx, e.dbtx, line) +func (e *Executor) MigrationGetByLine(ctx context.Context, params *riverdriver.MigrationGetByLineParams) ([]*riverdriver.Migration, error) { + migrations, err := dbsqlc.New().RiverMigrationGetByLine(schemaTemplateParam(ctx, params.Schema), e.dbtx, params.Line) if err != nil { return nil, interpretError(err) } return sliceutil.Map(migrations, migrationFromInternal), nil } -func (e *Executor) MigrationInsertMany(ctx context.Context, line string, versions []int) ([]*riverdriver.Migration, error) { - migrations, err := dbsqlc.New().RiverMigrationInsertMany(ctx, e.dbtx, &dbsqlc.RiverMigrationInsertManyParams{ - Line: line, - Version: sliceutil.Map(versions, func(v int) int64 { return int64(v) }), +func (e *Executor) MigrationInsertMany(ctx context.Context, params *riverdriver.MigrationInsertManyParams) ([]*riverdriver.Migration, error) { + migrations, err := dbsqlc.New().RiverMigrationInsertMany(schemaTemplateParam(ctx, params.Schema), e.dbtx, &dbsqlc.RiverMigrationInsertManyParams{ + Line: params.Line, + Version: sliceutil.Map(params.Versions, func(v int) int64 { return int64(v) }), }) if err != nil { return nil, interpretError(err) @@ -709,9 +705,9 @@ func (e *Executor) MigrationInsertMany(ctx context.Context, line string, version return sliceutil.Map(migrations, migrationFromInternal), nil } -func (e *Executor) MigrationInsertManyAssumingMain(ctx context.Context, versions []int) ([]*riverdriver.Migration, error) { - migrations, err := dbsqlc.New().RiverMigrationInsertManyAssumingMain(ctx, e.dbtx, - sliceutil.Map(versions, func(v int) int64 { return int64(v) }), +func (e *Executor) MigrationInsertManyAssumingMain(ctx context.Context, params *riverdriver.MigrationInsertManyAssumingMainParams) ([]*riverdriver.Migration, error) { + migrations, err := dbsqlc.New().RiverMigrationInsertManyAssumingMain(schemaTemplateParam(ctx, params.Schema), e.dbtx, + sliceutil.Map(params.Versions, func(v int) int64 { return int64(v) }), ) if err != nil { return nil, interpretError(err) @@ -728,6 +724,7 @@ func (e *Executor) MigrationInsertManyAssumingMain(ctx context.Context, versions func (e *Executor) NotifyMany(ctx context.Context, params *riverdriver.NotifyManyParams) error { return dbsqlc.New().PGNotifyMany(ctx, e.dbtx, &dbsqlc.PGNotifyManyParams{ Payload: params.Payload, + Schema: sql.NullString{String: params.Schema, Valid: params.Schema != ""}, Topic: params.Topic, }) } @@ -738,7 +735,7 @@ func (e *Executor) PGAdvisoryXactLock(ctx context.Context, key int64) (*struct{} } func (e *Executor) QueueCreateOrSetUpdatedAt(ctx context.Context, params *riverdriver.QueueCreateOrSetUpdatedAtParams) (*rivertype.Queue, error) { - queue, err := dbsqlc.New().QueueCreateOrSetUpdatedAt(ctx, e.dbtx, &dbsqlc.QueueCreateOrSetUpdatedAtParams{ + queue, err := dbsqlc.New().QueueCreateOrSetUpdatedAt(schemaTemplateParam(ctx, params.Schema), e.dbtx, &dbsqlc.QueueCreateOrSetUpdatedAtParams{ Metadata: valutil.ValOrDefault(string(params.Metadata), "{}"), Name: params.Name, PausedAt: params.PausedAt, @@ -751,7 +748,7 @@ func (e *Executor) QueueCreateOrSetUpdatedAt(ctx context.Context, params *riverd } func (e *Executor) QueueDeleteExpired(ctx context.Context, params *riverdriver.QueueDeleteExpiredParams) ([]string, error) { - queues, err := dbsqlc.New().QueueDeleteExpired(ctx, e.dbtx, &dbsqlc.QueueDeleteExpiredParams{ + queues, err := dbsqlc.New().QueueDeleteExpired(schemaTemplateParam(ctx, params.Schema), e.dbtx, &dbsqlc.QueueDeleteExpiredParams{ Max: int64(params.Max), UpdatedAtHorizon: params.UpdatedAtHorizon, }) @@ -765,16 +762,16 @@ func (e *Executor) QueueDeleteExpired(ctx context.Context, params *riverdriver.Q return queueNames, nil } -func (e *Executor) QueueGet(ctx context.Context, name string) (*rivertype.Queue, error) { - queue, err := dbsqlc.New().QueueGet(ctx, e.dbtx, name) +func (e *Executor) QueueGet(ctx context.Context, params *riverdriver.QueueGetParams) (*rivertype.Queue, error) { + queue, err := dbsqlc.New().QueueGet(schemaTemplateParam(ctx, params.Schema), e.dbtx, params.Name) if err != nil { return nil, interpretError(err) } return queueFromInternal(queue), nil } -func (e *Executor) QueueList(ctx context.Context, limit int) ([]*rivertype.Queue, error) { - internalQueues, err := dbsqlc.New().QueueList(ctx, e.dbtx, int32(min(limit, math.MaxInt32))) //nolint:gosec +func (e *Executor) QueueList(ctx context.Context, params *riverdriver.QueueListParams) ([]*rivertype.Queue, error) { + internalQueues, err := dbsqlc.New().QueueList(schemaTemplateParam(ctx, params.Schema), e.dbtx, int32(min(params.Limit, math.MaxInt32))) //nolint:gosec if err != nil { return nil, interpretError(err) } @@ -785,8 +782,8 @@ func (e *Executor) QueueList(ctx context.Context, limit int) ([]*rivertype.Queue return queues, nil } -func (e *Executor) QueuePause(ctx context.Context, name string) error { - res, err := dbsqlc.New().QueuePause(ctx, e.dbtx, name) +func (e *Executor) QueuePause(ctx context.Context, params *riverdriver.QueuePauseParams) error { + res, err := dbsqlc.New().QueuePause(schemaTemplateParam(ctx, params.Schema), e.dbtx, params.Name) if err != nil { return interpretError(err) } @@ -794,14 +791,14 @@ func (e *Executor) QueuePause(ctx context.Context, name string) error { if err != nil { return interpretError(err) } - if rowsAffected == 0 && name != riverdriver.AllQueuesString { + if rowsAffected == 0 && params.Name != riverdriver.AllQueuesString { return rivertype.ErrNotFound } return nil } -func (e *Executor) QueueResume(ctx context.Context, name string) error { - res, err := dbsqlc.New().QueueResume(ctx, e.dbtx, name) +func (e *Executor) QueueResume(ctx context.Context, params *riverdriver.QueueResumeParams) error { + res, err := dbsqlc.New().QueueResume(schemaTemplateParam(ctx, params.Schema), e.dbtx, params.Name) if err != nil { return interpretError(err) } @@ -809,14 +806,14 @@ func (e *Executor) QueueResume(ctx context.Context, name string) error { if err != nil { return interpretError(err) } - if rowsAffected == 0 && name != riverdriver.AllQueuesString { + if rowsAffected == 0 && params.Name != riverdriver.AllQueuesString { return rivertype.ErrNotFound } return nil } func (e *Executor) QueueUpdate(ctx context.Context, params *riverdriver.QueueUpdateParams) (*rivertype.Queue, error) { - queue, err := dbsqlc.New().QueueUpdate(ctx, e.dbtx, &dbsqlc.QueueUpdateParams{ + queue, err := dbsqlc.New().QueueUpdate(schemaTemplateParam(ctx, params.Schema), e.dbtx, &dbsqlc.QueueUpdateParams{ Metadata: string(params.Metadata), MetadataDoUpdate: params.MetadataDoUpdate, Name: params.Name, @@ -827,8 +824,14 @@ func (e *Executor) QueueUpdate(ctx context.Context, params *riverdriver.QueueUpd return queueFromInternal(queue), nil } -func (e *Executor) TableExists(ctx context.Context, tableName string) (bool, error) { - exists, err := dbsqlc.New().TableExists(ctx, e.dbtx, tableName) +func (e *Executor) TableExists(ctx context.Context, params *riverdriver.TableExistsParams) (bool, error) { + // Different from other operations because the schemaAndTable name is a parameter. + schemaAndTable := params.Table + if params.Schema != "" { + schemaAndTable = params.Schema + "." + schemaAndTable + } + + exists, err := dbsqlc.New().TableExists(ctx, e.dbtx, schemaAndTable) return exists, interpretError(err) } @@ -1059,3 +1062,13 @@ func queueFromInternal(internal *dbsqlc.RiverQueue) *rivertype.Queue { UpdatedAt: internal.UpdatedAt.UTC(), } } + +func schemaTemplateParam(ctx context.Context, schema string) context.Context { + if schema != "" { + schema += "." + } + + return sqlctemplate.WithReplacements(ctx, map[string]sqlctemplate.Replacement{ + "schema": {Value: schema}, + }, nil) +} diff --git a/riverdriver/riverdatabasesql/river_database_sql_driver_test.go b/riverdriver/riverdatabasesql/river_database_sql_driver_test.go index d6daf359..209b009b 100644 --- a/riverdriver/riverdatabasesql/river_database_sql_driver_test.go +++ b/riverdriver/riverdatabasesql/river_database_sql_driver_test.go @@ -1,6 +1,7 @@ package riverdatabasesql import ( + "context" "database/sql" "errors" "testing" @@ -8,6 +9,7 @@ import ( "github.com/stretchr/testify/require" "github.com/riverqueue/river/riverdriver" + "github.com/riverqueue/river/rivershared/sqlctemplate" "github.com/riverqueue/river/rivertype" ) @@ -87,3 +89,45 @@ func TestReplaceNamed(t *testing.T) { }) } } + +func TestSchemaTemplateParam(t *testing.T) { + t.Parallel() + + ctx := context.Background() + + type testBundle struct{} + + setup := func(t *testing.T) (*sqlctemplate.Replacer, *testBundle) { //nolint:unparam + t.Helper() + + return &sqlctemplate.Replacer{}, &testBundle{} + } + + t.Run("NoSchema", func(t *testing.T) { + t.Parallel() + + replacer, _ := setup(t) + + updatedSQL, _, err := replacer.RunSafely( + schemaTemplateParam(ctx, ""), + "SELECT 1 FROM /* TEMPLATE: schema */river_job", + nil, + ) + require.NoError(t, err) + require.Equal(t, "SELECT 1 FROM river_job", updatedSQL) + }) + + t.Run("WithSchema", func(t *testing.T) { + t.Parallel() + + replacer, _ := setup(t) + + updatedSQL, _, err := replacer.RunSafely( + schemaTemplateParam(ctx, "custom_schema"), + "SELECT 1 FROM /* TEMPLATE: schema */river_job", + nil, + ) + require.NoError(t, err) + require.Equal(t, "SELECT 1 FROM custom_schema.river_job", updatedSQL) + }) +} diff --git a/riverdriver/riverpgxv5/internal/dbsqlc/pg_misc.sql b/riverdriver/riverpgxv5/internal/dbsqlc/pg_misc.sql index 03a1b0d4..19a7b99f 100644 --- a/riverdriver/riverpgxv5/internal/dbsqlc/pg_misc.sql +++ b/riverdriver/riverpgxv5/internal/dbsqlc/pg_misc.sql @@ -4,7 +4,7 @@ SELECT pg_advisory_xact_lock(@key); -- name: PGNotifyMany :exec WITH topic_to_notify AS ( SELECT - concat(current_schema(), '.', @topic::text) AS topic, + concat(coalesce(sqlc.narg('schema')::text, current_schema()), '.', @topic::text) AS topic, unnest(@payload::text[]) AS payload ) SELECT pg_notify( diff --git a/riverdriver/riverpgxv5/internal/dbsqlc/pg_misc.sql.go b/riverdriver/riverpgxv5/internal/dbsqlc/pg_misc.sql.go index 3b10b419..20e84446 100644 --- a/riverdriver/riverpgxv5/internal/dbsqlc/pg_misc.sql.go +++ b/riverdriver/riverpgxv5/internal/dbsqlc/pg_misc.sql.go @@ -7,6 +7,8 @@ package dbsqlc import ( "context" + + "github.com/jackc/pgx/v5/pgtype" ) const pGAdvisoryXactLock = `-- name: PGAdvisoryXactLock :exec @@ -21,8 +23,8 @@ func (q *Queries) PGAdvisoryXactLock(ctx context.Context, db DBTX, key int64) er const pGNotifyMany = `-- name: PGNotifyMany :exec WITH topic_to_notify AS ( SELECT - concat(current_schema(), '.', $1::text) AS topic, - unnest($2::text[]) AS payload + concat(coalesce($1::text, current_schema()), '.', $2::text) AS topic, + unnest($3::text[]) AS payload ) SELECT pg_notify( topic_to_notify.topic, @@ -32,11 +34,12 @@ FROM topic_to_notify ` type PGNotifyManyParams struct { + Schema pgtype.Text Topic string Payload []string } func (q *Queries) PGNotifyMany(ctx context.Context, db DBTX, arg *PGNotifyManyParams) error { - _, err := db.Exec(ctx, pGNotifyMany, arg.Topic, arg.Payload) + _, err := db.Exec(ctx, pGNotifyMany, arg.Schema, arg.Topic, arg.Payload) return err } diff --git a/riverdriver/riverpgxv5/internal/dbsqlc/river_client.sql b/riverdriver/riverpgxv5/internal/dbsqlc/river_client.sql index 40919fcc..bdc1cf20 100644 --- a/riverdriver/riverpgxv5/internal/dbsqlc/river_client.sql +++ b/riverdriver/riverpgxv5/internal/dbsqlc/river_client.sql @@ -8,7 +8,7 @@ CREATE UNLOGGED TABLE river_client ( ); -- name: ClientCreateOrSetUpdatedAt :one -INSERT INTO river_client ( +INSERT INTO /* TEMPLATE: schema */river_client ( id, metadata, paused_at, diff --git a/riverdriver/riverpgxv5/internal/dbsqlc/river_client.sql.go b/riverdriver/riverpgxv5/internal/dbsqlc/river_client.sql.go index 106f6b15..d0facb68 100644 --- a/riverdriver/riverpgxv5/internal/dbsqlc/river_client.sql.go +++ b/riverdriver/riverpgxv5/internal/dbsqlc/river_client.sql.go @@ -11,7 +11,7 @@ import ( ) const clientCreateOrSetUpdatedAt = `-- name: ClientCreateOrSetUpdatedAt :one -INSERT INTO river_client ( +INSERT INTO /* TEMPLATE: schema */river_client ( id, metadata, paused_at, diff --git a/riverdriver/riverpgxv5/internal/dbsqlc/river_client_queue.sql b/riverdriver/riverpgxv5/internal/dbsqlc/river_client_queue.sql index 767efcd2..b1186b4c 100644 --- a/riverdriver/riverpgxv5/internal/dbsqlc/river_client_queue.sql +++ b/riverdriver/riverpgxv5/internal/dbsqlc/river_client_queue.sql @@ -14,7 +14,7 @@ CREATE UNLOGGED TABLE river_client_queue ( ); -- name: ClientQueueCreateOrSetUpdatedAtMany :one -INSERT INTO river_client_queue ( +INSERT INTO /* TEMPLATE: schema */river_client_queue ( metadata, name, paused_at, diff --git a/riverdriver/riverpgxv5/internal/dbsqlc/river_client_queue.sql.go b/riverdriver/riverpgxv5/internal/dbsqlc/river_client_queue.sql.go index 5bf20351..6569a278 100644 --- a/riverdriver/riverpgxv5/internal/dbsqlc/river_client_queue.sql.go +++ b/riverdriver/riverpgxv5/internal/dbsqlc/river_client_queue.sql.go @@ -11,7 +11,7 @@ import ( ) const clientQueueCreateOrSetUpdatedAtMany = `-- name: ClientQueueCreateOrSetUpdatedAtMany :one -INSERT INTO river_client_queue ( +INSERT INTO /* TEMPLATE: schema */river_client_queue ( metadata, name, paused_at, diff --git a/riverdriver/riverpgxv5/internal/dbsqlc/river_job.sql b/riverdriver/riverpgxv5/internal/dbsqlc/river_job.sql index bf4ac802..5ef9b8d6 100644 --- a/riverdriver/riverpgxv5/internal/dbsqlc/river_job.sql +++ b/riverdriver/riverpgxv5/internal/dbsqlc/river_job.sql @@ -41,7 +41,7 @@ CREATE TABLE river_job( WITH locked_job AS ( SELECT id, queue, state, finalized_at - FROM river_job + FROM /* TEMPLATE: schema */river_job WHERE river_job.id = @id FOR UPDATE ), @@ -49,7 +49,7 @@ notification AS ( SELECT id, pg_notify( - concat(current_schema(), '.', @control_topic::text), + concat(coalesce(sqlc.narg('schema')::text, current_schema()), '.', @control_topic::text), json_build_object('action', 'cancel', 'job_id', id, 'queue', queue)::text ) FROM @@ -59,7 +59,7 @@ notification AS ( AND finalized_at IS NULL ), updated_job AS ( - UPDATE river_job + UPDATE /* TEMPLATE: schema */river_job SET -- If the job is actively running, we want to let its current client and -- producer handle the cancellation. Otherwise, immediately cancel it. @@ -73,7 +73,7 @@ updated_job AS ( RETURNING river_job.* ) SELECT * -FROM river_job +FROM /* TEMPLATE: schema */river_job WHERE id = @id::bigint AND id NOT IN (SELECT id FROM updated_job) UNION @@ -82,19 +82,19 @@ FROM updated_job; -- name: JobCountByState :one SELECT count(*) -FROM river_job +FROM /* TEMPLATE: schema */river_job WHERE state = @state; -- name: JobDelete :one WITH job_to_delete AS ( SELECT id - FROM river_job + FROM /* TEMPLATE: schema */river_job WHERE river_job.id = @id FOR UPDATE ), deleted_job AS ( DELETE - FROM river_job + FROM /* TEMPLATE: schema */river_job USING job_to_delete WHERE river_job.id = job_to_delete.id -- Do not touch running jobs: @@ -102,7 +102,7 @@ deleted_job AS ( RETURNING river_job.* ) SELECT * -FROM river_job +FROM /* TEMPLATE: schema */river_job WHERE id = @id::bigint AND id NOT IN (SELECT id FROM deleted_job) UNION @@ -111,10 +111,10 @@ FROM deleted_job; -- name: JobDeleteBefore :one WITH deleted_jobs AS ( - DELETE FROM river_job + DELETE FROM /* TEMPLATE: schema */river_job WHERE id IN ( SELECT id - FROM river_job + FROM /* TEMPLATE: schema */river_job WHERE (state = 'cancelled' AND finalized_at < @cancelled_finalized_at_horizon::timestamptz) OR (state = 'completed' AND finalized_at < @completed_finalized_at_horizon::timestamptz) OR @@ -132,7 +132,7 @@ WITH locked_jobs AS ( SELECT * FROM - river_job + /* TEMPLATE: schema */river_job WHERE state = 'available' AND queue = @queue::text @@ -146,7 +146,7 @@ WITH locked_jobs AS ( SKIP LOCKED ) UPDATE - river_job + /* TEMPLATE: schema */river_job SET state = 'running', attempt = river_job.attempt + 1, @@ -161,7 +161,7 @@ RETURNING -- name: JobGetByKindAndUniqueProperties :one SELECT * -FROM river_job +FROM /* TEMPLATE: schema */river_job WHERE kind = @kind AND CASE WHEN @by_args::boolean THEN args = @args ELSE true END AND CASE WHEN @by_created_at::boolean THEN tstzrange(@created_at_begin::timestamptz, @created_at_end::timestamptz, '[)') @> created_at ELSE true END @@ -170,32 +170,32 @@ WHERE kind = @kind -- name: JobGetByKindMany :many SELECT * -FROM river_job +FROM /* TEMPLATE: schema */river_job WHERE kind = any(@kind::text[]) ORDER BY id; -- name: JobGetByID :one SELECT * -FROM river_job +FROM /* TEMPLATE: schema */river_job WHERE id = @id LIMIT 1; -- name: JobGetByIDMany :many SELECT * -FROM river_job +FROM /* TEMPLATE: schema */river_job WHERE id = any(@id::bigint[]) ORDER BY id; -- name: JobGetStuck :many SELECT * -FROM river_job +FROM /* TEMPLATE: schema */river_job WHERE state = 'running' AND attempted_at < @stuck_horizon::timestamptz ORDER BY id LIMIT @max; -- name: JobInsertFastMany :many -INSERT INTO river_job( +INSERT INTO /* TEMPLATE: schema */river_job( args, created_at, kind, @@ -237,7 +237,7 @@ ON CONFLICT (unique_key) RETURNING sqlc.embed(river_job), (xmax != 0) AS unique_skipped_as_duplicate; -- name: JobInsertFastManyNoReturning :execrows -INSERT INTO river_job( +INSERT INTO /* TEMPLATE: schema */river_job( args, created_at, kind, @@ -277,7 +277,7 @@ ON CONFLICT (unique_key) DO NOTHING; -- name: JobInsertFull :one -INSERT INTO river_job( +INSERT INTO /* TEMPLATE: schema */river_job( args, attempt, attempted_at, @@ -317,14 +317,14 @@ INSERT INTO river_job( -- name: JobList :many SELECT * -FROM river_job +FROM /* TEMPLATE: schema */river_job WHERE /* TEMPLATE_BEGIN: where_clause */ 1 /* TEMPLATE_END */ ORDER BY /* TEMPLATE_BEGIN: order_by_clause */ id /* TEMPLATE_END */ LIMIT @max::int; -- Run by the rescuer to queue for retry or discard depending on job state. -- name: JobRescueMany :exec -UPDATE river_job +UPDATE /* TEMPLATE: schema */river_job SET errors = array_append(errors, updated_job.error), finalized_at = updated_job.finalized_at, @@ -343,12 +343,12 @@ WHERE river_job.id = updated_job.id; -- name: JobRetry :one WITH job_to_update AS ( SELECT id - FROM river_job + FROM /* TEMPLATE: schema */river_job WHERE river_job.id = @id FOR UPDATE ), updated_job AS ( - UPDATE river_job + UPDATE /* TEMPLATE: schema */river_job SET state = 'available', scheduled_at = now(), @@ -363,7 +363,7 @@ updated_job AS ( RETURNING river_job.* ) SELECT * -FROM river_job +FROM /* TEMPLATE: schema */river_job WHERE id = @id::bigint AND id NOT IN (SELECT id FROM updated_job) UNION @@ -378,7 +378,7 @@ WITH jobs_to_schedule AS ( unique_states, priority, scheduled_at - FROM river_job + FROM /* TEMPLATE: schema */river_job WHERE state IN ('retryable', 'scheduled') AND queue IS NOT NULL @@ -406,7 +406,7 @@ jobs_with_rownum AS ( ), unique_conflicts AS ( SELECT river_job.unique_key - FROM river_job + FROM /* TEMPLATE: schema */river_job JOIN jobs_with_rownum ON river_job.unique_key = jobs_with_rownum.unique_key AND river_job.id != jobs_with_rownum.id @@ -432,7 +432,7 @@ job_updates AS ( LEFT JOIN unique_conflicts uc ON job.unique_key = uc.unique_key ), updated_jobs AS ( - UPDATE river_job + UPDATE /* TEMPLATE: schema */river_job SET state = job_updates.new_state, finalized_at = CASE WHEN job_updates.finalized_at_do_update THEN @now::timestamptz @@ -448,7 +448,7 @@ updated_jobs AS ( SELECT sqlc.embed(river_job), updated_jobs.conflict_discarded -FROM river_job +FROM /* TEMPLATE: schema */river_job JOIN updated_jobs ON river_job.id = updated_jobs.id; -- name: JobSetStateIfRunningMany :many @@ -484,7 +484,7 @@ job_to_update AS ( job_input.scheduled_at_do_update, (job_input.state IN ('retryable', 'scheduled') AND river_job.metadata ? 'cancel_attempted_at') AS should_cancel, job_input.state - FROM river_job + FROM /* TEMPLATE: schema */river_job JOIN job_input ON river_job.id = job_input.id WHERE river_job.state = 'running' OR job_input.metadata_do_merge FOR UPDATE @@ -512,7 +512,7 @@ updated_running AS ( RETURNING river_job.* ), updated_metadata_only AS ( - UPDATE river_job + UPDATE /* TEMPLATE: schema */river_job SET metadata = river_job.metadata || job_to_update.metadata_updates FROM job_to_update WHERE river_job.id = job_to_update.id @@ -522,7 +522,7 @@ updated_metadata_only AS ( RETURNING river_job.* ) SELECT * -FROM river_job +FROM /* TEMPLATE: schema */river_job WHERE id IN (SELECT id FROM job_input) AND id NOT IN (SELECT id FROM updated_metadata_only) AND id NOT IN (SELECT id FROM updated_running) @@ -532,7 +532,7 @@ UNION SELECT * FROM updated_running; -- A generalized update for any property on a job. This brings in a large number -- of parameters and therefore may be more suitable for testing than production. -- name: JobUpdate :one -UPDATE river_job +UPDATE /* TEMPLATE: schema */river_job SET attempt = CASE WHEN @attempt_do_update::boolean THEN @attempt ELSE attempt END, attempted_at = CASE WHEN @attempted_at_do_update::boolean THEN @attempted_at ELSE attempted_at END, diff --git a/riverdriver/riverpgxv5/internal/dbsqlc/river_job.sql.go b/riverdriver/riverpgxv5/internal/dbsqlc/river_job.sql.go index a4c8673e..61ef0419 100644 --- a/riverdriver/riverpgxv5/internal/dbsqlc/river_job.sql.go +++ b/riverdriver/riverpgxv5/internal/dbsqlc/river_job.sql.go @@ -16,7 +16,7 @@ const jobCancel = `-- name: JobCancel :one WITH locked_job AS ( SELECT id, queue, state, finalized_at - FROM river_job + FROM /* TEMPLATE: schema */river_job WHERE river_job.id = $1 FOR UPDATE ), @@ -24,7 +24,7 @@ notification AS ( SELECT id, pg_notify( - concat(current_schema(), '.', $2::text), + concat(coalesce($2::text, current_schema()), '.', $3::text), json_build_object('action', 'cancel', 'job_id', id, 'queue', queue)::text ) FROM @@ -34,7 +34,7 @@ notification AS ( AND finalized_at IS NULL ), updated_job AS ( - UPDATE river_job + UPDATE /* TEMPLATE: schema */river_job SET -- If the job is actively running, we want to let its current client and -- producer handle the cancellation. Otherwise, immediately cancel it. @@ -42,13 +42,13 @@ updated_job AS ( finalized_at = CASE WHEN state = 'running' THEN finalized_at ELSE now() END, -- Mark the job as cancelled by query so that the rescuer knows not to -- rescue it, even if it gets stuck in the running state: - metadata = jsonb_set(metadata, '{cancel_attempted_at}'::text[], $3::jsonb, true) + metadata = jsonb_set(metadata, '{cancel_attempted_at}'::text[], $4::jsonb, true) 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, 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, unique_states -FROM river_job +FROM /* TEMPLATE: schema */river_job WHERE id = $1::bigint AND id NOT IN (SELECT id FROM updated_job) UNION @@ -58,12 +58,18 @@ FROM updated_job type JobCancelParams struct { ID int64 + Schema pgtype.Text ControlTopic string CancelAttemptedAt []byte } func (q *Queries) JobCancel(ctx context.Context, db DBTX, arg *JobCancelParams) (*RiverJob, error) { - row := db.QueryRow(ctx, jobCancel, arg.ID, arg.ControlTopic, arg.CancelAttemptedAt) + row := db.QueryRow(ctx, jobCancel, + arg.ID, + arg.Schema, + arg.ControlTopic, + arg.CancelAttemptedAt, + ) var i RiverJob err := row.Scan( &i.ID, @@ -90,7 +96,7 @@ func (q *Queries) JobCancel(ctx context.Context, db DBTX, arg *JobCancelParams) const jobCountByState = `-- name: JobCountByState :one SELECT count(*) -FROM river_job +FROM /* TEMPLATE: schema */river_job WHERE state = $1 ` @@ -104,13 +110,13 @@ func (q *Queries) JobCountByState(ctx context.Context, db DBTX, state RiverJobSt const jobDelete = `-- name: JobDelete :one WITH job_to_delete AS ( SELECT id - FROM river_job + FROM /* TEMPLATE: schema */river_job WHERE river_job.id = $1 FOR UPDATE ), deleted_job AS ( DELETE - FROM river_job + FROM /* TEMPLATE: schema */river_job USING job_to_delete WHERE river_job.id = job_to_delete.id -- Do not touch running jobs: @@ -118,7 +124,7 @@ deleted_job AS ( 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, unique_states -FROM river_job +FROM /* TEMPLATE: schema */river_job WHERE id = $1::bigint AND id NOT IN (SELECT id FROM deleted_job) UNION @@ -154,10 +160,10 @@ func (q *Queries) JobDelete(ctx context.Context, db DBTX, id int64) (*RiverJob, const jobDeleteBefore = `-- name: JobDeleteBefore :one WITH deleted_jobs AS ( - DELETE FROM river_job + DELETE FROM /* TEMPLATE: schema */river_job WHERE id IN ( SELECT id - FROM river_job + FROM /* TEMPLATE: schema */river_job WHERE (state = 'cancelled' AND finalized_at < $1::timestamptz) OR (state = 'completed' AND finalized_at < $2::timestamptz) OR @@ -195,7 +201,7 @@ 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, unique_states FROM - river_job + /* TEMPLATE: schema */river_job WHERE state = 'available' AND queue = $2::text @@ -209,7 +215,7 @@ WITH locked_jobs AS ( SKIP LOCKED ) UPDATE - river_job + /* TEMPLATE: schema */river_job SET state = 'running', attempt = river_job.attempt + 1, @@ -276,7 +282,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, unique_states -FROM river_job +FROM /* TEMPLATE: schema */river_job WHERE id = $1 LIMIT 1 ` @@ -309,7 +315,7 @@ func (q *Queries) JobGetByID(ctx context.Context, db DBTX, id int64) (*RiverJob, 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, unique_states -FROM river_job +FROM /* TEMPLATE: schema */river_job WHERE id = any($1::bigint[]) ORDER BY id ` @@ -355,7 +361,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, unique_states -FROM river_job +FROM /* TEMPLATE: schema */river_job WHERE kind = $1 AND CASE WHEN $2::boolean THEN args = $3 ELSE true END AND CASE WHEN $4::boolean THEN tstzrange($5::timestamptz, $6::timestamptz, '[)') @> created_at ELSE true END @@ -415,7 +421,7 @@ func (q *Queries) JobGetByKindAndUniqueProperties(ctx context.Context, db DBTX, 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, unique_states -FROM river_job +FROM /* TEMPLATE: schema */river_job WHERE kind = any($1::text[]) ORDER BY id ` @@ -461,7 +467,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, unique_states -FROM river_job +FROM /* TEMPLATE: schema */river_job WHERE state = 'running' AND attempted_at < $1::timestamptz ORDER BY id @@ -513,7 +519,7 @@ func (q *Queries) JobGetStuck(ctx context.Context, db DBTX, arg *JobGetStuckPara } const jobInsertFastMany = `-- name: JobInsertFastMany :many -INSERT INTO river_job( +INSERT INTO /* TEMPLATE: schema */river_job( args, created_at, kind, @@ -629,7 +635,7 @@ func (q *Queries) JobInsertFastMany(ctx context.Context, db DBTX, arg *JobInsert } const jobInsertFastManyNoReturning = `-- name: JobInsertFastManyNoReturning :execrows -INSERT INTO river_job( +INSERT INTO /* TEMPLATE: schema */river_job( args, created_at, kind, @@ -706,7 +712,7 @@ func (q *Queries) JobInsertFastManyNoReturning(ctx context.Context, db DBTX, arg } const jobInsertFull = `-- name: JobInsertFull :one -INSERT INTO river_job( +INSERT INTO /* TEMPLATE: schema */river_job( args, attempt, attempted_at, @@ -811,7 +817,7 @@ func (q *Queries) JobInsertFull(ctx context.Context, db DBTX, arg *JobInsertFull const jobList = `-- name: JobList :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, unique_states -FROM river_job +FROM /* TEMPLATE: schema */river_job WHERE /* TEMPLATE_BEGIN: where_clause */ 1 /* TEMPLATE_END */ ORDER BY /* TEMPLATE_BEGIN: order_by_clause */ id /* TEMPLATE_END */ LIMIT $1::int @@ -857,7 +863,7 @@ func (q *Queries) JobList(ctx context.Context, db DBTX, max int32) ([]*RiverJob, } const jobRescueMany = `-- name: JobRescueMany :exec -UPDATE river_job +UPDATE /* TEMPLATE: schema */river_job SET errors = array_append(errors, updated_job.error), finalized_at = updated_job.finalized_at, @@ -897,12 +903,12 @@ func (q *Queries) JobRescueMany(ctx context.Context, db DBTX, arg *JobRescueMany const jobRetry = `-- name: JobRetry :one WITH job_to_update AS ( SELECT id - FROM river_job + FROM /* TEMPLATE: schema */river_job WHERE river_job.id = $1 FOR UPDATE ), updated_job AS ( - UPDATE river_job + UPDATE /* TEMPLATE: schema */river_job SET state = 'available', scheduled_at = now(), @@ -917,7 +923,7 @@ updated_job AS ( 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, unique_states -FROM river_job +FROM /* TEMPLATE: schema */river_job WHERE id = $1::bigint AND id NOT IN (SELECT id FROM updated_job) UNION @@ -959,7 +965,7 @@ WITH jobs_to_schedule AS ( unique_states, priority, scheduled_at - FROM river_job + FROM /* TEMPLATE: schema */river_job WHERE state IN ('retryable', 'scheduled') AND queue IS NOT NULL @@ -987,7 +993,7 @@ jobs_with_rownum AS ( ), unique_conflicts AS ( SELECT river_job.unique_key - FROM river_job + FROM /* TEMPLATE: schema */river_job JOIN jobs_with_rownum ON river_job.unique_key = jobs_with_rownum.unique_key AND river_job.id != jobs_with_rownum.id @@ -1013,7 +1019,7 @@ job_updates AS ( LEFT JOIN unique_conflicts uc ON job.unique_key = uc.unique_key ), updated_jobs AS ( - UPDATE river_job + UPDATE /* TEMPLATE: schema */river_job SET state = job_updates.new_state, finalized_at = CASE WHEN job_updates.finalized_at_do_update THEN $1::timestamptz @@ -1029,7 +1035,7 @@ updated_jobs AS ( SELECT river_job.id, river_job.args, river_job.attempt, river_job.attempted_at, river_job.attempted_by, river_job.created_at, river_job.errors, river_job.finalized_at, river_job.kind, river_job.max_attempts, river_job.metadata, river_job.priority, river_job.queue, river_job.state, river_job.scheduled_at, river_job.tags, river_job.unique_key, river_job.unique_states, updated_jobs.conflict_discarded -FROM river_job +FROM /* TEMPLATE: schema */river_job JOIN updated_jobs ON river_job.id = updated_jobs.id ` @@ -1116,7 +1122,7 @@ job_to_update AS ( job_input.scheduled_at_do_update, (job_input.state IN ('retryable', 'scheduled') AND river_job.metadata ? 'cancel_attempted_at') AS should_cancel, job_input.state - FROM river_job + FROM /* TEMPLATE: schema */river_job JOIN job_input ON river_job.id = job_input.id WHERE river_job.state = 'running' OR job_input.metadata_do_merge FOR UPDATE @@ -1144,7 +1150,7 @@ updated_running AS ( 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 ), updated_metadata_only AS ( - UPDATE river_job + UPDATE /* TEMPLATE: schema */river_job SET metadata = river_job.metadata || job_to_update.metadata_updates FROM job_to_update WHERE river_job.id = job_to_update.id @@ -1154,7 +1160,7 @@ updated_metadata_only AS ( 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, unique_states -FROM river_job +FROM /* TEMPLATE: schema */river_job WHERE id IN (SELECT id FROM job_input) AND id NOT IN (SELECT id FROM updated_metadata_only) AND id NOT IN (SELECT id FROM updated_running) @@ -1230,7 +1236,7 @@ func (q *Queries) JobSetStateIfRunningMany(ctx context.Context, db DBTX, arg *Jo } const jobUpdate = `-- name: JobUpdate :one -UPDATE river_job +UPDATE /* TEMPLATE: schema */river_job SET attempt = CASE WHEN $1::boolean THEN $2 ELSE attempt END, attempted_at = CASE WHEN $3::boolean THEN $4 ELSE attempted_at END, diff --git a/riverdriver/riverpgxv5/internal/dbsqlc/river_job_copyfrom.sql b/riverdriver/riverpgxv5/internal/dbsqlc/river_job_copyfrom.sql index 54fdbeaa..ecb1b18f 100644 --- a/riverdriver/riverpgxv5/internal/dbsqlc/river_job_copyfrom.sql +++ b/riverdriver/riverpgxv5/internal/dbsqlc/river_job_copyfrom.sql @@ -1,5 +1,5 @@ -- name: JobInsertFastManyCopyFrom :copyfrom -INSERT INTO river_job( +INSERT INTO /* TEMPLATE: schema */river_job( args, created_at, kind, diff --git a/riverdriver/riverpgxv5/internal/dbsqlc/river_leader.sql b/riverdriver/riverpgxv5/internal/dbsqlc/river_leader.sql index d0cfc15a..a2ae64e1 100644 --- a/riverdriver/riverpgxv5/internal/dbsqlc/river_leader.sql +++ b/riverdriver/riverpgxv5/internal/dbsqlc/river_leader.sql @@ -8,13 +8,13 @@ CREATE UNLOGGED TABLE river_leader( ); -- name: LeaderAttemptElect :execrows -INSERT INTO river_leader(leader_id, elected_at, expires_at) +INSERT INTO /* TEMPLATE: schema */river_leader (leader_id, elected_at, expires_at) VALUES (@leader_id, now(), now() + @ttl::interval) ON CONFLICT (name) DO NOTHING; -- name: LeaderAttemptReelect :execrows -INSERT INTO river_leader(leader_id, elected_at, expires_at) +INSERT INTO /* TEMPLATE: schema */river_leader (leader_id, elected_at, expires_at) VALUES (@leader_id, now(), now() + @ttl::interval) ON CONFLICT (name) DO UPDATE SET @@ -23,15 +23,15 @@ ON CONFLICT (name) river_leader.leader_id = @leader_id; -- name: LeaderDeleteExpired :execrows -DELETE FROM river_leader +DELETE FROM /* TEMPLATE: schema */river_leader WHERE expires_at < now(); -- name: LeaderGetElectedLeader :one SELECT * -FROM river_leader; +FROM /* TEMPLATE: schema */river_leader; -- name: LeaderInsert :one -INSERT INTO river_leader( +INSERT INTO /* TEMPLATE: schema */river_leader( elected_at, expires_at, leader_id @@ -44,15 +44,15 @@ INSERT INTO river_leader( -- name: LeaderResign :execrows WITH currently_held_leaders AS ( SELECT * - FROM river_leader + FROM /* TEMPLATE: schema */river_leader WHERE leader_id = @leader_id::text FOR UPDATE ), notified_resignations AS ( SELECT pg_notify( - concat(current_schema(), '.', @leadership_topic::text), + concat(coalesce(sqlc.narg('schema')::text, current_schema()), '.', @leadership_topic::text), json_build_object('leader_id', leader_id, 'action', 'resigned')::text ) FROM currently_held_leaders ) -DELETE FROM river_leader USING notified_resignations; \ No newline at end of file +DELETE FROM /* TEMPLATE: schema */river_leader USING notified_resignations; \ No newline at end of file diff --git a/riverdriver/riverpgxv5/internal/dbsqlc/river_leader.sql.go b/riverdriver/riverpgxv5/internal/dbsqlc/river_leader.sql.go index e818cec8..b4b5b251 100644 --- a/riverdriver/riverpgxv5/internal/dbsqlc/river_leader.sql.go +++ b/riverdriver/riverpgxv5/internal/dbsqlc/river_leader.sql.go @@ -8,10 +8,12 @@ package dbsqlc import ( "context" "time" + + "github.com/jackc/pgx/v5/pgtype" ) const leaderAttemptElect = `-- name: LeaderAttemptElect :execrows -INSERT INTO river_leader(leader_id, elected_at, expires_at) +INSERT INTO /* TEMPLATE: schema */river_leader (leader_id, elected_at, expires_at) VALUES ($1, now(), now() + $2::interval) ON CONFLICT (name) DO NOTHING @@ -31,7 +33,7 @@ func (q *Queries) LeaderAttemptElect(ctx context.Context, db DBTX, arg *LeaderAt } const leaderAttemptReelect = `-- name: LeaderAttemptReelect :execrows -INSERT INTO river_leader(leader_id, elected_at, expires_at) +INSERT INTO /* TEMPLATE: schema */river_leader (leader_id, elected_at, expires_at) VALUES ($1, now(), now() + $2::interval) ON CONFLICT (name) DO UPDATE SET @@ -54,7 +56,7 @@ func (q *Queries) LeaderAttemptReelect(ctx context.Context, db DBTX, arg *Leader } const leaderDeleteExpired = `-- name: LeaderDeleteExpired :execrows -DELETE FROM river_leader +DELETE FROM /* TEMPLATE: schema */river_leader WHERE expires_at < now() ` @@ -68,7 +70,7 @@ func (q *Queries) LeaderDeleteExpired(ctx context.Context, db DBTX) (int64, erro const leaderGetElectedLeader = `-- name: LeaderGetElectedLeader :one SELECT elected_at, expires_at, leader_id, name -FROM river_leader +FROM /* TEMPLATE: schema */river_leader ` func (q *Queries) LeaderGetElectedLeader(ctx context.Context, db DBTX) (*RiverLeader, error) { @@ -84,7 +86,7 @@ func (q *Queries) LeaderGetElectedLeader(ctx context.Context, db DBTX) (*RiverLe } const leaderInsert = `-- name: LeaderInsert :one -INSERT INTO river_leader( +INSERT INTO /* TEMPLATE: schema */river_leader( elected_at, expires_at, leader_id @@ -122,27 +124,28 @@ func (q *Queries) LeaderInsert(ctx context.Context, db DBTX, arg *LeaderInsertPa const leaderResign = `-- name: LeaderResign :execrows WITH currently_held_leaders AS ( SELECT elected_at, expires_at, leader_id, name - FROM river_leader + FROM /* TEMPLATE: schema */river_leader WHERE leader_id = $1::text FOR UPDATE ), notified_resignations AS ( SELECT pg_notify( - concat(current_schema(), '.', $2::text), + concat(coalesce($2::text, current_schema()), '.', $3::text), json_build_object('leader_id', leader_id, 'action', 'resigned')::text ) FROM currently_held_leaders ) -DELETE FROM river_leader USING notified_resignations +DELETE FROM /* TEMPLATE: schema */river_leader USING notified_resignations ` type LeaderResignParams struct { LeaderID string + Schema pgtype.Text LeadershipTopic string } func (q *Queries) LeaderResign(ctx context.Context, db DBTX, arg *LeaderResignParams) (int64, error) { - result, err := db.Exec(ctx, leaderResign, arg.LeaderID, arg.LeadershipTopic) + result, err := db.Exec(ctx, leaderResign, arg.LeaderID, arg.Schema, arg.LeadershipTopic) if err != nil { return 0, err } diff --git a/riverdriver/riverpgxv5/internal/dbsqlc/river_migration.sql b/riverdriver/riverpgxv5/internal/dbsqlc/river_migration.sql index 12915ee3..6a43a899 100644 --- a/riverdriver/riverpgxv5/internal/dbsqlc/river_migration.sql +++ b/riverdriver/riverpgxv5/internal/dbsqlc/river_migration.sql @@ -8,14 +8,14 @@ CREATE TABLE river_migration( ); -- name: RiverMigrationDeleteAssumingMainMany :many -DELETE FROM river_migration +DELETE FROM /* TEMPLATE: schema */river_migration WHERE version = any(@version::bigint[]) RETURNING created_at, version; -- name: RiverMigrationDeleteByLineAndVersionMany :many -DELETE FROM river_migration +DELETE FROM /* TEMPLATE: schema */river_migration WHERE line = @line AND version = any(@version::bigint[]) RETURNING *; @@ -30,17 +30,17 @@ RETURNING *; SELECT created_at, version -FROM river_migration +FROM /* TEMPLATE: schema */river_migration ORDER BY version; -- name: RiverMigrationGetByLine :many SELECT * -FROM river_migration +FROM /* TEMPLATE: schema */river_migration WHERE line = @line ORDER BY version; -- name: RiverMigrationInsert :one -INSERT INTO river_migration ( +INSERT INTO /* TEMPLATE: schema */river_migration ( line, version ) VALUES ( @@ -49,7 +49,7 @@ INSERT INTO river_migration ( ) RETURNING *; -- name: RiverMigrationInsertMany :many -INSERT INTO river_migration ( +INSERT INTO /* TEMPLATE: schema */river_migration ( line, version ) @@ -59,7 +59,7 @@ SELECT RETURNING *; -- name: RiverMigrationInsertManyAssumingMain :many -INSERT INTO river_migration ( +INSERT INTO /* TEMPLATE: schema */river_migration ( version ) SELECT @@ -73,10 +73,10 @@ SELECT EXISTS ( SELECT column_name FROM information_schema.columns WHERE table_name = @table_name::text - AND table_schema = CURRENT_SCHEMA + AND table_schema = /* TEMPLATE_BEGIN: schema */ CURRENT_SCHEMA /* TEMPLATE_END */ AND column_name = @column_name::text ); -- name: TableExists :one -SELECT CASE WHEN to_regclass(@table_name) IS NULL THEN false +SELECT CASE WHEN to_regclass(@schema_and_table) IS NULL THEN false ELSE true END; \ No newline at end of file diff --git a/riverdriver/riverpgxv5/internal/dbsqlc/river_migration.sql.go b/riverdriver/riverpgxv5/internal/dbsqlc/river_migration.sql.go index c189772e..3bfaf0a2 100644 --- a/riverdriver/riverpgxv5/internal/dbsqlc/river_migration.sql.go +++ b/riverdriver/riverpgxv5/internal/dbsqlc/river_migration.sql.go @@ -15,7 +15,7 @@ SELECT EXISTS ( SELECT column_name FROM information_schema.columns WHERE table_name = $1::text - AND table_schema = CURRENT_SCHEMA + AND table_schema = /* TEMPLATE_BEGIN: schema */ CURRENT_SCHEMA /* TEMPLATE_END */ AND column_name = $2::text ) ` @@ -33,7 +33,7 @@ func (q *Queries) ColumnExists(ctx context.Context, db DBTX, arg *ColumnExistsPa } const riverMigrationDeleteAssumingMainMany = `-- name: RiverMigrationDeleteAssumingMainMany :many -DELETE FROM river_migration +DELETE FROM /* TEMPLATE: schema */river_migration WHERE version = any($1::bigint[]) RETURNING created_at, @@ -66,7 +66,7 @@ func (q *Queries) RiverMigrationDeleteAssumingMainMany(ctx context.Context, db D } const riverMigrationDeleteByLineAndVersionMany = `-- name: RiverMigrationDeleteByLineAndVersionMany :many -DELETE FROM river_migration +DELETE FROM /* TEMPLATE: schema */river_migration WHERE line = $1 AND version = any($2::bigint[]) RETURNING line, version, created_at @@ -101,7 +101,7 @@ const riverMigrationGetAllAssumingMain = `-- name: RiverMigrationGetAllAssumingM SELECT created_at, version -FROM river_migration +FROM /* TEMPLATE: schema */river_migration ORDER BY version ` @@ -137,7 +137,7 @@ func (q *Queries) RiverMigrationGetAllAssumingMain(ctx context.Context, db DBTX) const riverMigrationGetByLine = `-- name: RiverMigrationGetByLine :many SELECT line, version, created_at -FROM river_migration +FROM /* TEMPLATE: schema */river_migration WHERE line = $1 ORDER BY version ` @@ -163,7 +163,7 @@ func (q *Queries) RiverMigrationGetByLine(ctx context.Context, db DBTX, line str } const riverMigrationInsert = `-- name: RiverMigrationInsert :one -INSERT INTO river_migration ( +INSERT INTO /* TEMPLATE: schema */river_migration ( line, version ) VALUES ( @@ -185,7 +185,7 @@ func (q *Queries) RiverMigrationInsert(ctx context.Context, db DBTX, arg *RiverM } const riverMigrationInsertMany = `-- name: RiverMigrationInsertMany :many -INSERT INTO river_migration ( +INSERT INTO /* TEMPLATE: schema */river_migration ( line, version ) @@ -221,7 +221,7 @@ func (q *Queries) RiverMigrationInsertMany(ctx context.Context, db DBTX, arg *Ri } const riverMigrationInsertManyAssumingMain = `-- name: RiverMigrationInsertManyAssumingMain :many -INSERT INTO river_migration ( +INSERT INTO /* TEMPLATE: schema */river_migration ( version ) SELECT @@ -261,8 +261,8 @@ SELECT CASE WHEN to_regclass($1) IS NULL THEN false ELSE true END ` -func (q *Queries) TableExists(ctx context.Context, db DBTX, tableName string) (bool, error) { - row := db.QueryRow(ctx, tableExists, tableName) +func (q *Queries) TableExists(ctx context.Context, db DBTX, schemaAndTable string) (bool, error) { + row := db.QueryRow(ctx, tableExists, schemaAndTable) var column_1 bool err := row.Scan(&column_1) return column_1, err diff --git a/riverdriver/riverpgxv5/internal/dbsqlc/river_queue.sql b/riverdriver/riverpgxv5/internal/dbsqlc/river_queue.sql index d3c83f76..9f46dde2 100644 --- a/riverdriver/riverpgxv5/internal/dbsqlc/river_queue.sql +++ b/riverdriver/riverpgxv5/internal/dbsqlc/river_queue.sql @@ -7,7 +7,7 @@ CREATE TABLE river_queue( ); -- name: QueueCreateOrSetUpdatedAt :one -INSERT INTO river_queue( +INSERT INTO /* TEMPLATE: schema */river_queue( created_at, metadata, name, @@ -25,10 +25,10 @@ SET RETURNING *; -- name: QueueDeleteExpired :many -DELETE FROM river_queue +DELETE FROM /* TEMPLATE: schema */river_queue WHERE name IN ( SELECT name - FROM river_queue + FROM /* TEMPLATE: schema */river_queue WHERE updated_at < @updated_at_horizon::timestamptz ORDER BY name ASC LIMIT @max::bigint @@ -37,24 +37,24 @@ RETURNING *; -- name: QueueGet :one SELECT * -FROM river_queue +FROM /* TEMPLATE: schema */river_queue WHERE name = @name::text; -- name: QueueList :many SELECT * -FROM river_queue +FROM /* TEMPLATE: schema */river_queue ORDER BY name ASC LIMIT @limit_count::integer; -- name: QueuePause :execresult WITH queue_to_update AS ( SELECT name, paused_at - FROM river_queue + FROM /* TEMPLATE: schema */river_queue WHERE CASE WHEN @name::text = '*' THEN true ELSE name = @name END FOR UPDATE ), updated_queue AS ( - UPDATE river_queue + UPDATE /* TEMPLATE: schema */river_queue SET paused_at = now(), updated_at = now() @@ -64,7 +64,7 @@ updated_queue AS ( RETURNING river_queue.* ) SELECT * -FROM river_queue +FROM /* TEMPLATE: schema */river_queue WHERE name = @name AND name NOT IN (SELECT name FROM updated_queue) UNION @@ -74,12 +74,12 @@ FROM updated_queue; -- name: QueueResume :execresult WITH queue_to_update AS ( SELECT name - FROM river_queue + FROM /* TEMPLATE: schema */river_queue WHERE CASE WHEN @name::text = '*' THEN true ELSE river_queue.name = @name::text END FOR UPDATE ), updated_queue AS ( - UPDATE river_queue + UPDATE /* TEMPLATE: schema */river_queue SET paused_at = NULL, updated_at = now() @@ -88,7 +88,7 @@ updated_queue AS ( RETURNING river_queue.* ) SELECT * -FROM river_queue +FROM /* TEMPLATE: schema */river_queue WHERE name = @name AND name NOT IN (SELECT name FROM updated_queue) UNION @@ -96,7 +96,7 @@ SELECT * FROM updated_queue; -- name: QueueUpdate :one -UPDATE river_queue +UPDATE /* TEMPLATE: schema */river_queue SET metadata = CASE WHEN @metadata_do_update::boolean THEN @metadata::jsonb ELSE metadata END, updated_at = now() diff --git a/riverdriver/riverpgxv5/internal/dbsqlc/river_queue.sql.go b/riverdriver/riverpgxv5/internal/dbsqlc/river_queue.sql.go index 7e66f291..bfaa29e9 100644 --- a/riverdriver/riverpgxv5/internal/dbsqlc/river_queue.sql.go +++ b/riverdriver/riverpgxv5/internal/dbsqlc/river_queue.sql.go @@ -13,7 +13,7 @@ import ( ) const queueCreateOrSetUpdatedAt = `-- name: QueueCreateOrSetUpdatedAt :one -INSERT INTO river_queue( +INSERT INTO /* TEMPLATE: schema */river_queue( created_at, metadata, name, @@ -57,10 +57,10 @@ func (q *Queries) QueueCreateOrSetUpdatedAt(ctx context.Context, db DBTX, arg *Q } const queueDeleteExpired = `-- name: QueueDeleteExpired :many -DELETE FROM river_queue +DELETE FROM /* TEMPLATE: schema */river_queue WHERE name IN ( SELECT name - FROM river_queue + FROM /* TEMPLATE: schema */river_queue WHERE updated_at < $1::timestamptz ORDER BY name ASC LIMIT $2::bigint @@ -101,7 +101,7 @@ func (q *Queries) QueueDeleteExpired(ctx context.Context, db DBTX, arg *QueueDel const queueGet = `-- name: QueueGet :one SELECT name, created_at, metadata, paused_at, updated_at -FROM river_queue +FROM /* TEMPLATE: schema */river_queue WHERE name = $1::text ` @@ -120,7 +120,7 @@ func (q *Queries) QueueGet(ctx context.Context, db DBTX, name string) (*RiverQue const queueList = `-- name: QueueList :many SELECT name, created_at, metadata, paused_at, updated_at -FROM river_queue +FROM /* TEMPLATE: schema */river_queue ORDER BY name ASC LIMIT $1::integer ` @@ -154,12 +154,12 @@ func (q *Queries) QueueList(ctx context.Context, db DBTX, limitCount int32) ([]* const queuePause = `-- name: QueuePause :execresult WITH queue_to_update AS ( SELECT name, paused_at - FROM river_queue + FROM /* TEMPLATE: schema */river_queue WHERE CASE WHEN $1::text = '*' THEN true ELSE name = $1 END FOR UPDATE ), updated_queue AS ( - UPDATE river_queue + UPDATE /* TEMPLATE: schema */river_queue SET paused_at = now(), updated_at = now() @@ -169,7 +169,7 @@ updated_queue AS ( RETURNING river_queue.name, river_queue.created_at, river_queue.metadata, river_queue.paused_at, river_queue.updated_at ) SELECT name, created_at, metadata, paused_at, updated_at -FROM river_queue +FROM /* TEMPLATE: schema */river_queue WHERE name = $1 AND name NOT IN (SELECT name FROM updated_queue) UNION @@ -184,12 +184,12 @@ func (q *Queries) QueuePause(ctx context.Context, db DBTX, name string) (pgconn. const queueResume = `-- name: QueueResume :execresult WITH queue_to_update AS ( SELECT name - FROM river_queue + FROM /* TEMPLATE: schema */river_queue WHERE CASE WHEN $1::text = '*' THEN true ELSE river_queue.name = $1::text END FOR UPDATE ), updated_queue AS ( - UPDATE river_queue + UPDATE /* TEMPLATE: schema */river_queue SET paused_at = NULL, updated_at = now() @@ -198,7 +198,7 @@ updated_queue AS ( RETURNING river_queue.name, river_queue.created_at, river_queue.metadata, river_queue.paused_at, river_queue.updated_at ) SELECT name, created_at, metadata, paused_at, updated_at -FROM river_queue +FROM /* TEMPLATE: schema */river_queue WHERE name = $1 AND name NOT IN (SELECT name FROM updated_queue) UNION @@ -211,7 +211,7 @@ func (q *Queries) QueueResume(ctx context.Context, db DBTX, name string) (pgconn } const queueUpdate = `-- name: QueueUpdate :one -UPDATE river_queue +UPDATE /* TEMPLATE: schema */river_queue SET metadata = CASE WHEN $1::boolean THEN $2::jsonb ELSE metadata END, updated_at = now() diff --git a/riverdriver/riverpgxv5/migration/main/001_create_river_migration.down.sql b/riverdriver/riverpgxv5/migration/main/001_create_river_migration.down.sql index 74c1984b..8bfe8202 100644 --- a/riverdriver/riverpgxv5/migration/main/001_create_river_migration.down.sql +++ b/riverdriver/riverpgxv5/migration/main/001_create_river_migration.down.sql @@ -1 +1 @@ -DROP TABLE river_migration; \ No newline at end of file +DROP TABLE /* TEMPLATE: schema */river_migration; \ No newline at end of file diff --git a/riverdriver/riverpgxv5/migration/main/001_create_river_migration.up.sql b/riverdriver/riverpgxv5/migration/main/001_create_river_migration.up.sql index 1e8a2f39..27006d56 100644 --- a/riverdriver/riverpgxv5/migration/main/001_create_river_migration.up.sql +++ b/riverdriver/riverpgxv5/migration/main/001_create_river_migration.up.sql @@ -1,8 +1,8 @@ -CREATE TABLE river_migration( +CREATE TABLE /* TEMPLATE: schema */river_migration( id bigserial PRIMARY KEY, created_at timestamptz NOT NULL DEFAULT NOW(), version bigint NOT NULL, CONSTRAINT version CHECK (version >= 1) ); -CREATE UNIQUE INDEX ON river_migration USING btree(version); \ No newline at end of file +CREATE UNIQUE INDEX ON /* TEMPLATE: schema */river_migration USING btree(version); \ No newline at end of file diff --git a/riverdriver/riverpgxv5/migration/main/002_initial_schema.down.sql b/riverdriver/riverpgxv5/migration/main/002_initial_schema.down.sql index bafef0ba..d334d8a6 100644 --- a/riverdriver/riverpgxv5/migration/main/002_initial_schema.down.sql +++ b/riverdriver/riverpgxv5/migration/main/002_initial_schema.down.sql @@ -1,5 +1,5 @@ -DROP TABLE river_job; -DROP FUNCTION river_job_notify; -DROP TYPE river_job_state; +DROP TABLE /* TEMPLATE: schema */river_job; +DROP FUNCTION /* TEMPLATE: schema */river_job_notify; +DROP TYPE /* TEMPLATE: schema */river_job_state; -DROP TABLE river_leader; \ No newline at end of file +DROP TABLE /* TEMPLATE: schema */river_leader; \ No newline at end of file diff --git a/riverdriver/riverpgxv5/migration/main/002_initial_schema.up.sql b/riverdriver/riverpgxv5/migration/main/002_initial_schema.up.sql index 57397e31..604e82ba 100644 --- a/riverdriver/riverpgxv5/migration/main/002_initial_schema.up.sql +++ b/riverdriver/riverpgxv5/migration/main/002_initial_schema.up.sql @@ -1,4 +1,4 @@ -CREATE TYPE river_job_state AS ENUM( +CREATE TYPE /* TEMPLATE: schema */river_job_state AS ENUM( 'available', 'cancelled', 'completed', @@ -8,7 +8,7 @@ CREATE TYPE river_job_state AS ENUM( 'scheduled' ); -CREATE TABLE river_job( +CREATE TABLE /* TEMPLATE: schema */river_job( -- 8 bytes id bigserial PRIMARY KEY, @@ -49,17 +49,17 @@ CREATE TABLE river_job( -- We may want to consider adding another property here after `kind` if it seems -- like it'd be useful for something. -CREATE INDEX river_job_kind ON river_job USING btree(kind); +CREATE INDEX river_job_kind ON /* TEMPLATE: schema */river_job USING btree(kind); -CREATE INDEX river_job_state_and_finalized_at_index ON river_job USING btree(state, finalized_at) WHERE finalized_at IS NOT NULL; +CREATE INDEX river_job_state_and_finalized_at_index ON /* TEMPLATE: schema */river_job USING btree(state, finalized_at) WHERE finalized_at IS NOT NULL; -CREATE INDEX river_job_prioritized_fetching_index ON river_job USING btree(state, queue, priority, scheduled_at, id); +CREATE INDEX river_job_prioritized_fetching_index ON /* TEMPLATE: schema */river_job USING btree(state, queue, priority, scheduled_at, id); -CREATE INDEX river_job_args_index ON river_job USING GIN(args); +CREATE INDEX river_job_args_index ON /* TEMPLATE: schema */river_job USING GIN(args); -CREATE INDEX river_job_metadata_index ON river_job USING GIN(metadata); +CREATE INDEX river_job_metadata_index ON /* TEMPLATE: schema */river_job USING GIN(metadata); -CREATE OR REPLACE FUNCTION river_job_notify() +CREATE OR REPLACE FUNCTION /* TEMPLATE: schema */river_job_notify() RETURNS TRIGGER AS $$ DECLARE @@ -78,11 +78,11 @@ $$ LANGUAGE plpgsql; CREATE TRIGGER river_notify - AFTER INSERT ON river_job + AFTER INSERT ON /* TEMPLATE: schema */river_job FOR EACH ROW - EXECUTE PROCEDURE river_job_notify(); + EXECUTE PROCEDURE /* TEMPLATE: schema */river_job_notify(); -CREATE UNLOGGED TABLE river_leader( +CREATE UNLOGGED TABLE /* TEMPLATE: schema */river_leader( -- 8 bytes each (no alignment needed) elected_at timestamptz NOT NULL, expires_at timestamptz NOT NULL, diff --git a/riverdriver/riverpgxv5/migration/main/003_river_job_tags_non_null.down.sql b/riverdriver/riverpgxv5/migration/main/003_river_job_tags_non_null.down.sql index 37a09255..acef65cb 100644 --- a/riverdriver/riverpgxv5/migration/main/003_river_job_tags_non_null.down.sql +++ b/riverdriver/riverpgxv5/migration/main/003_river_job_tags_non_null.down.sql @@ -1,2 +1,3 @@ -ALTER TABLE river_job ALTER COLUMN tags DROP NOT NULL, - ALTER COLUMN tags DROP DEFAULT; +ALTER TABLE /* TEMPLATE: schema */river_job + ALTER COLUMN tags DROP NOT NULL, + ALTER COLUMN tags DROP DEFAULT; diff --git a/riverdriver/riverpgxv5/migration/main/003_river_job_tags_non_null.up.sql b/riverdriver/riverpgxv5/migration/main/003_river_job_tags_non_null.up.sql index 5849bd55..0a472dde 100644 --- a/riverdriver/riverpgxv5/migration/main/003_river_job_tags_non_null.up.sql +++ b/riverdriver/riverpgxv5/migration/main/003_river_job_tags_non_null.up.sql @@ -1,3 +1,3 @@ -ALTER TABLE river_job ALTER COLUMN tags SET DEFAULT '{}'; -UPDATE river_job SET tags = '{}' WHERE tags IS NULL; -ALTER TABLE river_job ALTER COLUMN tags SET NOT NULL; +ALTER TABLE /* TEMPLATE: schema */river_job ALTER COLUMN tags SET DEFAULT '{}'; +UPDATE /* TEMPLATE: schema */river_job SET tags = '{}' WHERE tags IS NULL; +ALTER TABLE /* TEMPLATE: schema */river_job ALTER COLUMN tags SET NOT NULL; diff --git a/riverdriver/riverpgxv5/migration/main/004_pending_and_more.down.sql b/riverdriver/riverpgxv5/migration/main/004_pending_and_more.down.sql index 22d5e998..1b7ec7e8 100644 --- a/riverdriver/riverpgxv5/migration/main/004_pending_and_more.down.sql +++ b/riverdriver/riverpgxv5/migration/main/004_pending_and_more.down.sql @@ -1,17 +1,17 @@ -ALTER TABLE river_job ALTER COLUMN args DROP NOT NULL; +ALTER TABLE /* TEMPLATE: schema */river_job ALTER COLUMN args DROP NOT NULL; -ALTER TABLE river_job ALTER COLUMN metadata DROP NOT NULL; -ALTER TABLE river_job ALTER COLUMN metadata DROP DEFAULT; +ALTER TABLE /* TEMPLATE: schema */river_job ALTER COLUMN metadata DROP NOT NULL; +ALTER TABLE /* TEMPLATE: schema */river_job ALTER COLUMN metadata DROP DEFAULT; -- It is not possible to safely remove 'pending' from the river_job_state enum, -- so leave it in place. -ALTER TABLE river_job DROP CONSTRAINT finalized_or_finalized_at_null; -ALTER TABLE river_job ADD CONSTRAINT finalized_or_finalized_at_null CHECK ( +ALTER TABLE /* TEMPLATE: schema */river_job DROP CONSTRAINT finalized_or_finalized_at_null; +ALTER TABLE /* TEMPLATE: schema */river_job ADD CONSTRAINT finalized_or_finalized_at_null CHECK ( (state IN ('cancelled', 'completed', 'discarded') AND finalized_at IS NOT NULL) OR finalized_at IS NULL ); -CREATE OR REPLACE FUNCTION river_job_notify() +CREATE OR REPLACE FUNCTION /* TEMPLATE: schema */river_job_notify() RETURNS TRIGGER AS $$ DECLARE @@ -30,13 +30,13 @@ $$ LANGUAGE plpgsql; CREATE TRIGGER river_notify - AFTER INSERT ON river_job + AFTER INSERT ON /* TEMPLATE: schema */river_job FOR EACH ROW - EXECUTE PROCEDURE river_job_notify(); + EXECUTE PROCEDURE /* TEMPLATE: schema */river_job_notify(); -DROP TABLE river_queue; +DROP TABLE /* TEMPLATE: schema */river_queue; -ALTER TABLE river_leader +ALTER TABLE /* TEMPLATE: schema */river_leader ALTER COLUMN name DROP DEFAULT, DROP CONSTRAINT name_length, ADD CONSTRAINT name_length CHECK (char_length(name) > 0 AND char_length(name) < 128); \ No newline at end of file diff --git a/riverdriver/riverpgxv5/migration/main/004_pending_and_more.up.sql b/riverdriver/riverpgxv5/migration/main/004_pending_and_more.up.sql index b5cf9a91..f98ff7d0 100644 --- a/riverdriver/riverpgxv5/migration/main/004_pending_and_more.up.sql +++ b/riverdriver/riverpgxv5/migration/main/004_pending_and_more.up.sql @@ -1,29 +1,29 @@ -- The args column never had a NOT NULL constraint or default value at the -- database level, though we tried to ensure one at the application level. -ALTER TABLE river_job ALTER COLUMN args SET DEFAULT '{}'; -UPDATE river_job SET args = '{}' WHERE args IS NULL; -ALTER TABLE river_job ALTER COLUMN args SET NOT NULL; -ALTER TABLE river_job ALTER COLUMN args DROP DEFAULT; +ALTER TABLE /* TEMPLATE: schema */river_job ALTER COLUMN args SET DEFAULT '{}'; +UPDATE /* TEMPLATE: schema */river_job SET args = '{}' WHERE args IS NULL; +ALTER TABLE /* TEMPLATE: schema */river_job ALTER COLUMN args SET NOT NULL; +ALTER TABLE /* TEMPLATE: schema */river_job ALTER COLUMN args DROP DEFAULT; -- The metadata column never had a NOT NULL constraint or default value at the -- database level, though we tried to ensure one at the application level. -ALTER TABLE river_job ALTER COLUMN metadata SET DEFAULT '{}'; -UPDATE river_job SET metadata = '{}' WHERE metadata IS NULL; -ALTER TABLE river_job ALTER COLUMN metadata SET NOT NULL; +ALTER TABLE /* TEMPLATE: schema */river_job ALTER COLUMN metadata SET DEFAULT '{}'; +UPDATE /* TEMPLATE: schema */river_job SET metadata = '{}' WHERE metadata IS NULL; +ALTER TABLE /* TEMPLATE: schema */river_job ALTER COLUMN metadata SET NOT NULL; -- The 'pending' job state will be used for upcoming functionality: -ALTER TYPE river_job_state ADD VALUE IF NOT EXISTS 'pending' AFTER 'discarded'; +ALTER TYPE /* TEMPLATE: schema */river_job_state ADD VALUE IF NOT EXISTS 'pending' AFTER 'discarded'; -ALTER TABLE river_job DROP CONSTRAINT finalized_or_finalized_at_null; -ALTER TABLE river_job ADD CONSTRAINT finalized_or_finalized_at_null CHECK ( +ALTER TABLE /* TEMPLATE: schema */river_job DROP CONSTRAINT finalized_or_finalized_at_null; +ALTER TABLE /* TEMPLATE: schema */river_job ADD 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')) ); -DROP TRIGGER river_notify ON river_job; -DROP FUNCTION river_job_notify; +DROP TRIGGER river_notify ON /* TEMPLATE: schema */river_job; +DROP FUNCTION /* TEMPLATE: schema */river_job_notify; -CREATE TABLE river_queue( +CREATE TABLE /* TEMPLATE: schema */river_queue( name text PRIMARY KEY NOT NULL, created_at timestamptz NOT NULL DEFAULT NOW(), metadata jsonb NOT NULL DEFAULT '{}' ::jsonb, @@ -31,7 +31,7 @@ CREATE TABLE river_queue( updated_at timestamptz NOT NULL ); -ALTER TABLE river_leader +ALTER TABLE /* TEMPLATE: schema */river_leader ALTER COLUMN name SET DEFAULT 'default', DROP CONSTRAINT name_length, ADD CONSTRAINT name_length CHECK (name = 'default'); \ No newline at end of file diff --git a/riverdriver/riverpgxv5/migration/main/005_migration_unique_client.down.sql b/riverdriver/riverpgxv5/migration/main/005_migration_unique_client.down.sql index c3ba58c6..b8e041d5 100644 --- a/riverdriver/riverpgxv5/migration/main/005_migration_unique_client.down.sql +++ b/riverdriver/riverpgxv5/migration/main/005_migration_unique_client.down.sql @@ -10,33 +10,33 @@ BEGIN -- Tolerate users who may be using their own migration system rather than -- River's. If they are, they will have skipped version 001 containing -- `CREATE TABLE river_migration`, so this table won't exist. - IF (SELECT to_regclass('river_migration') IS NOT NULL) THEN + IF (SELECT to_regclass('/* TEMPLATE: schema */river_migration') IS NOT NULL) THEN IF EXISTS ( SELECT * - FROM river_migration + FROM /* TEMPLATE: schema */river_migration WHERE line <> 'main' ) THEN RAISE EXCEPTION 'Found non-main migration lines in the database; version 005 migration is irreversible because it would result in loss of migration information.'; END IF; - ALTER TABLE river_migration + ALTER TABLE /* TEMPLATE: schema */river_migration RENAME TO river_migration_old; - CREATE TABLE river_migration( + CREATE TABLE /* TEMPLATE: schema */river_migration( id bigserial PRIMARY KEY, created_at timestamptz NOT NULL DEFAULT NOW(), version bigint NOT NULL, CONSTRAINT version CHECK (version >= 1) ); - CREATE UNIQUE INDEX ON river_migration USING btree(version); + CREATE UNIQUE INDEX ON /* TEMPLATE: schema */river_migration USING btree(version); - INSERT INTO river_migration + INSERT INTO /* TEMPLATE: schema */river_migration (created_at, version) SELECT created_at, version - FROM river_migration_old; + FROM /* TEMPLATE: schema */river_migration_old; - DROP TABLE river_migration_old; + DROP TABLE /* TEMPLATE: schema */river_migration_old; END IF; END; $body$ @@ -46,12 +46,12 @@ LANGUAGE 'plpgsql'; -- Drop `river_job.unique_key`. -- -ALTER TABLE river_job +ALTER TABLE /* TEMPLATE: schema */river_job DROP COLUMN unique_key; -- -- Drop `river_client` and derivative. -- -DROP TABLE river_client_queue; -DROP TABLE river_client; +DROP TABLE /* TEMPLATE: schema */river_client_queue; +DROP TABLE /* TEMPLATE: schema */river_client; diff --git a/riverdriver/riverpgxv5/migration/main/005_migration_unique_client.up.sql b/riverdriver/riverpgxv5/migration/main/005_migration_unique_client.up.sql index 50601651..ff964304 100644 --- a/riverdriver/riverpgxv5/migration/main/005_migration_unique_client.up.sql +++ b/riverdriver/riverpgxv5/migration/main/005_migration_unique_client.up.sql @@ -8,11 +8,11 @@ BEGIN -- Tolerate users who may be using their own migration system rather than -- River's. If they are, they will have skipped version 001 containing -- `CREATE TABLE river_migration`, so this table won't exist. - IF (SELECT to_regclass('river_migration') IS NOT NULL) THEN - ALTER TABLE river_migration + IF (SELECT to_regclass('/* TEMPLATE: schema */river_migration') IS NOT NULL) THEN + ALTER TABLE /* TEMPLATE: schema */river_migration RENAME TO river_migration_old; - CREATE TABLE river_migration( + CREATE TABLE /* TEMPLATE: schema */river_migration( line TEXT NOT NULL, version bigint NOT NULL, created_at timestamptz NOT NULL DEFAULT NOW(), @@ -21,12 +21,12 @@ BEGIN PRIMARY KEY (line, version) ); - INSERT INTO river_migration + INSERT INTO /* TEMPLATE: schema */river_migration (created_at, line, version) SELECT created_at, 'main', version - FROM river_migration_old; + FROM /* TEMPLATE: schema */river_migration_old; - DROP TABLE river_migration_old; + DROP TABLE /* TEMPLATE: schema */river_migration_old; END IF; END; $body$ @@ -39,10 +39,10 @@ LANGUAGE 'plpgsql'; -- These statements use `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. -ALTER TABLE river_job +ALTER TABLE /* TEMPLATE: schema */river_job ADD COLUMN IF NOT EXISTS unique_key bytea; -CREATE UNIQUE INDEX IF NOT EXISTS river_job_kind_unique_key_idx ON river_job (kind, unique_key) WHERE unique_key IS NOT NULL; +CREATE UNIQUE INDEX IF NOT EXISTS river_job_kind_unique_key_idx ON /* TEMPLATE: schema */river_job (kind, unique_key) WHERE unique_key IS NOT NULL; -- -- Create `river_client` and derivative. @@ -52,7 +52,7 @@ CREATE UNIQUE INDEX IF NOT EXISTS river_job_kind_unique_key_idx ON river_job (ki -- additional migration. -- -CREATE UNLOGGED TABLE river_client ( +CREATE UNLOGGED TABLE /* TEMPLATE: schema */river_client ( id text PRIMARY KEY NOT NULL, created_at timestamptz NOT NULL DEFAULT now(), metadata jsonb NOT NULL DEFAULT '{}', @@ -63,7 +63,7 @@ CREATE UNLOGGED TABLE river_client ( -- Differs from `river_queue` in that it tracks the queue state for a particular -- active client. -CREATE UNLOGGED TABLE river_client_queue ( +CREATE UNLOGGED TABLE /* TEMPLATE: schema */river_client_queue ( river_client_id text NOT NULL REFERENCES river_client (id) ON DELETE CASCADE, name text NOT NULL, created_at timestamptz NOT NULL DEFAULT now(), diff --git a/riverdriver/riverpgxv5/migration/main/006_bulk_unique.down.sql b/riverdriver/riverpgxv5/migration/main/006_bulk_unique.down.sql index ae65cfed..26cd8434 100644 --- a/riverdriver/riverpgxv5/migration/main/006_bulk_unique.down.sql +++ b/riverdriver/riverpgxv5/migration/main/006_bulk_unique.down.sql @@ -3,14 +3,14 @@ -- Drop `river_job.unique_states` and its index. -- -DROP INDEX river_job_unique_idx; +DROP INDEX /* TEMPLATE: schema */river_job_unique_idx; -ALTER TABLE river_job +ALTER TABLE /* TEMPLATE: schema */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; +CREATE UNIQUE INDEX IF NOT EXISTS river_job_kind_unique_key_idx ON /* TEMPLATE: schema */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; +DROP FUNCTION /* TEMPLATE: schema */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 index 29c1a3cb..15f0ee53 100644 --- a/riverdriver/riverpgxv5/migration/main/006_bulk_unique.up.sql +++ b/riverdriver/riverpgxv5/migration/main/006_bulk_unique.up.sql @@ -1,5 +1,5 @@ -CREATE OR REPLACE FUNCTION river_job_state_in_bitmask(bitmask BIT(8), state river_job_state) +CREATE OR REPLACE FUNCTION /* TEMPLATE: schema */river_job_state_in_bitmask(bitmask BIT(8), state river_job_state) RETURNS boolean LANGUAGE SQL IMMUTABLE @@ -23,12 +23,12 @@ $$; -- This column may exist already if users manually created the column and index -- as instructed in the changelog so the index could be created `CONCURRENTLY`. -- -ALTER TABLE river_job ADD COLUMN IF NOT EXISTS unique_states BIT(8); +ALTER TABLE /* TEMPLATE: schema */river_job ADD COLUMN IF NOT EXISTS unique_states BIT(8); -- This statement 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) +CREATE UNIQUE INDEX IF NOT EXISTS river_job_unique_idx ON /* TEMPLATE: schema */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); @@ -38,4 +38,4 @@ CREATE UNIQUE INDEX IF NOT EXISTS river_job_unique_idx ON river_job (unique_key) -- 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; +DROP INDEX /* TEMPLATE: schema */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 708c914d..504dba60 100644 --- a/riverdriver/riverpgxv5/river_pgx_v5_driver.go +++ b/riverdriver/riverpgxv5/river_pgx_v5_driver.go @@ -60,7 +60,11 @@ func New(dbPool *pgxpool.Pool) *Driver { func (d *Driver) GetExecutor() riverdriver.Executor { return &Executor{templateReplaceWrapper{d.dbPool, &d.replacer}, d} } -func (d *Driver) GetListener() riverdriver.Listener { return &Listener{dbPool: d.dbPool} } + +func (d *Driver) GetListener(schema string) riverdriver.Listener { + return &Listener{dbPool: d.dbPool, schema: schema} +} + func (d *Driver) GetMigrationFS(line string) fs.FS { if line == riverdriver.MigrationLineMain { return migrationFS @@ -96,10 +100,19 @@ func (e *Executor) Begin(ctx context.Context) (riverdriver.ExecutorTx, error) { return &ExecutorTx{Executor: Executor{templateReplaceWrapper{tx, &e.driver.replacer}, e.driver}, tx: tx}, nil } -func (e *Executor) ColumnExists(ctx context.Context, tableName, columnName string) (bool, error) { +func (e *Executor) ColumnExists(ctx context.Context, params *riverdriver.ColumnExistsParams) (bool, error) { + // Schema injection is a bit different on this one because we're querying a table with a schema name. + schema := "CURRENT_SCHEMA" + if params.Schema != "" { + schema = "'" + params.Schema + "'" + } + ctx = sqlctemplate.WithReplacements(ctx, map[string]sqlctemplate.Replacement{ + "schema": {Value: schema}, + }, nil) + exists, err := dbsqlc.New().ColumnExists(ctx, e.dbtx, &dbsqlc.ColumnExistsParams{ - ColumnName: columnName, - TableName: tableName, + ColumnName: params.Column, + TableName: params.Table, }) return exists, interpretError(err) } @@ -115,10 +128,11 @@ func (e *Executor) JobCancel(ctx context.Context, params *riverdriver.JobCancelP return nil, err } - job, err := dbsqlc.New().JobCancel(ctx, e.dbtx, &dbsqlc.JobCancelParams{ + job, err := dbsqlc.New().JobCancel(schemaTemplateParam(ctx, params.Schema), e.dbtx, &dbsqlc.JobCancelParams{ ID: params.ID, CancelAttemptedAt: cancelledAt, ControlTopic: params.ControlTopic, + Schema: pgtype.Text{String: params.Schema, Valid: params.Schema != ""}, }) if err != nil { return nil, interpretError(err) @@ -126,16 +140,16 @@ func (e *Executor) JobCancel(ctx context.Context, params *riverdriver.JobCancelP return jobRowFromInternal(job) } -func (e *Executor) JobCountByState(ctx context.Context, state rivertype.JobState) (int, error) { - numJobs, err := dbsqlc.New().JobCountByState(ctx, e.dbtx, dbsqlc.RiverJobState(state)) +func (e *Executor) JobCountByState(ctx context.Context, params *riverdriver.JobCountByStateParams) (int, error) { + numJobs, err := dbsqlc.New().JobCountByState(schemaTemplateParam(ctx, params.Schema), e.dbtx, dbsqlc.RiverJobState(params.State)) if err != nil { return 0, err } return int(numJobs), nil } -func (e *Executor) JobDelete(ctx context.Context, id int64) (*rivertype.JobRow, error) { - job, err := dbsqlc.New().JobDelete(ctx, e.dbtx, id) +func (e *Executor) JobDelete(ctx context.Context, params *riverdriver.JobDeleteParams) (*rivertype.JobRow, error) { + job, err := dbsqlc.New().JobDelete(schemaTemplateParam(ctx, params.Schema), e.dbtx, params.ID) if err != nil { return nil, interpretError(err) } @@ -146,7 +160,7 @@ func (e *Executor) JobDelete(ctx context.Context, id int64) (*rivertype.JobRow, } func (e *Executor) JobDeleteBefore(ctx context.Context, params *riverdriver.JobDeleteBeforeParams) (int, error) { - numDeleted, err := dbsqlc.New().JobDeleteBefore(ctx, e.dbtx, &dbsqlc.JobDeleteBeforeParams{ + numDeleted, err := dbsqlc.New().JobDeleteBefore(schemaTemplateParam(ctx, params.Schema), e.dbtx, &dbsqlc.JobDeleteBeforeParams{ CancelledFinalizedAtHorizon: params.CancelledFinalizedAtHorizon, CompletedFinalizedAtHorizon: params.CompletedFinalizedAtHorizon, DiscardedFinalizedAtHorizon: params.DiscardedFinalizedAtHorizon, @@ -156,7 +170,7 @@ func (e *Executor) JobDeleteBefore(ctx context.Context, params *riverdriver.JobD } func (e *Executor) JobGetAvailable(ctx context.Context, params *riverdriver.JobGetAvailableParams) ([]*rivertype.JobRow, error) { - jobs, err := dbsqlc.New().JobGetAvailable(ctx, e.dbtx, &dbsqlc.JobGetAvailableParams{ + jobs, err := dbsqlc.New().JobGetAvailable(schemaTemplateParam(ctx, params.Schema), e.dbtx, &dbsqlc.JobGetAvailableParams{ AttemptedBy: params.ClientID, Max: int32(min(params.Max, math.MaxInt32)), //nolint:gosec Now: params.Now, @@ -168,32 +182,24 @@ func (e *Executor) JobGetAvailable(ctx context.Context, params *riverdriver.JobG return mapSliceError(jobs, jobRowFromInternal) } -func (e *Executor) JobGetByID(ctx context.Context, id int64) (*rivertype.JobRow, error) { - job, err := dbsqlc.New().JobGetByID(ctx, e.dbtx, id) +func (e *Executor) JobGetByID(ctx context.Context, params *riverdriver.JobGetByIDParams) (*rivertype.JobRow, error) { + job, err := dbsqlc.New().JobGetByID(schemaTemplateParam(ctx, params.Schema), e.dbtx, params.ID) if err != nil { return nil, interpretError(err) } return jobRowFromInternal(job) } -func (e *Executor) JobGetByIDMany(ctx context.Context, id []int64) ([]*rivertype.JobRow, error) { - jobs, err := dbsqlc.New().JobGetByIDMany(ctx, e.dbtx, id) +func (e *Executor) JobGetByIDMany(ctx context.Context, params *riverdriver.JobGetByIDManyParams) ([]*rivertype.JobRow, error) { + jobs, err := dbsqlc.New().JobGetByIDMany(schemaTemplateParam(ctx, params.Schema), e.dbtx, params.ID) if err != nil { return nil, interpretError(err) } return mapSliceError(jobs, jobRowFromInternal) } -func (e *Executor) JobGetByKindAndUniqueProperties(ctx context.Context, params *riverdriver.JobGetByKindAndUniquePropertiesParams) (*rivertype.JobRow, error) { - job, err := dbsqlc.New().JobGetByKindAndUniqueProperties(ctx, e.dbtx, (*dbsqlc.JobGetByKindAndUniquePropertiesParams)(params)) - if err != nil { - return nil, interpretError(err) - } - return jobRowFromInternal(job) -} - -func (e *Executor) JobGetByKindMany(ctx context.Context, kind []string) ([]*rivertype.JobRow, error) { - jobs, err := dbsqlc.New().JobGetByKindMany(ctx, e.dbtx, kind) +func (e *Executor) JobGetByKindMany(ctx context.Context, params *riverdriver.JobGetByKindManyParams) ([]*rivertype.JobRow, error) { + jobs, err := dbsqlc.New().JobGetByKindMany(schemaTemplateParam(ctx, params.Schema), e.dbtx, params.Kind) if err != nil { return nil, interpretError(err) } @@ -201,32 +207,32 @@ func (e *Executor) JobGetByKindMany(ctx context.Context, kind []string) ([]*rive } func (e *Executor) JobGetStuck(ctx context.Context, params *riverdriver.JobGetStuckParams) ([]*rivertype.JobRow, error) { - jobs, err := dbsqlc.New().JobGetStuck(ctx, e.dbtx, &dbsqlc.JobGetStuckParams{Max: int32(min(params.Max, math.MaxInt32)), StuckHorizon: params.StuckHorizon}) //nolint:gosec + jobs, err := dbsqlc.New().JobGetStuck(schemaTemplateParam(ctx, params.Schema), e.dbtx, &dbsqlc.JobGetStuckParams{Max: int32(min(params.Max, math.MaxInt32)), StuckHorizon: params.StuckHorizon}) //nolint:gosec if err != nil { return nil, interpretError(err) } return mapSliceError(jobs, jobRowFromInternal) } -func (e *Executor) JobInsertFastMany(ctx context.Context, params []*riverdriver.JobInsertFastParams) ([]*riverdriver.JobInsertFastResult, error) { +func (e *Executor) JobInsertFastMany(ctx context.Context, params *riverdriver.JobInsertFastManyParams) ([]*riverdriver.JobInsertFastResult, error) { insertJobsParams := &dbsqlc.JobInsertFastManyParams{ - Args: make([][]byte, len(params)), - CreatedAt: make([]time.Time, 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)), + Args: make([][]byte, len(params.Jobs)), + CreatedAt: make([]time.Time, len(params.Jobs)), + Kind: make([]string, len(params.Jobs)), + MaxAttempts: make([]int16, len(params.Jobs)), + Metadata: make([][]byte, len(params.Jobs)), + Priority: make([]int16, len(params.Jobs)), + Queue: make([]string, len(params.Jobs)), + ScheduledAt: make([]time.Time, len(params.Jobs)), + State: make([]string, len(params.Jobs)), + Tags: make([]string, len(params.Jobs)), + UniqueKey: make([][]byte, len(params.Jobs)), + UniqueStates: make([]pgtype.Bits, len(params.Jobs)), } now := time.Now().UTC() - for i := 0; i < len(params); i++ { - params := params[i] + for i := range len(params.Jobs) { + params := params.Jobs[i] createdAt := now if params.CreatedAt != nil { @@ -259,7 +265,7 @@ func (e *Executor) JobInsertFastMany(ctx context.Context, params []*riverdriver. insertJobsParams.UniqueStates[i] = pgtype.Bits{Bytes: []byte{params.UniqueStates}, Len: 8, Valid: params.UniqueStates != 0} } - items, err := dbsqlc.New().JobInsertFastMany(ctx, e.dbtx, insertJobsParams) + items, err := dbsqlc.New().JobInsertFastMany(schemaTemplateParam(ctx, params.Schema), e.dbtx, insertJobsParams) if err != nil { return nil, interpretError(err) } @@ -273,12 +279,12 @@ func (e *Executor) JobInsertFastMany(ctx context.Context, params []*riverdriver. }) } -func (e *Executor) JobInsertFastManyNoReturning(ctx context.Context, params []*riverdriver.JobInsertFastParams) (int, error) { - insertJobsParams := make([]*dbsqlc.JobInsertFastManyCopyFromParams, len(params)) +func (e *Executor) JobInsertFastManyNoReturning(ctx context.Context, params *riverdriver.JobInsertFastManyParams) (int, error) { + insertJobsParams := make([]*dbsqlc.JobInsertFastManyCopyFromParams, len(params.Jobs)) now := time.Now().UTC() - for i := 0; i < len(params); i++ { - params := params[i] + for i := range len(params.Jobs) { + params := params.Jobs[i] createdAt := now if params.CreatedAt != nil { @@ -316,7 +322,7 @@ func (e *Executor) JobInsertFastManyNoReturning(ctx context.Context, params []*r } } - numInserted, err := dbsqlc.New().JobInsertFastManyCopyFrom(ctx, e.dbtx, insertJobsParams) + numInserted, err := dbsqlc.New().JobInsertFastManyCopyFrom(schemaTemplateParam(ctx, params.Schema), e.dbtx, insertJobsParams) if err != nil { return 0, interpretError(err) } @@ -325,7 +331,7 @@ func (e *Executor) JobInsertFastManyNoReturning(ctx context.Context, params []*r } func (e *Executor) JobInsertFull(ctx context.Context, params *riverdriver.JobInsertFullParams) (*rivertype.JobRow, error) { - job, err := dbsqlc.New().JobInsertFull(ctx, e.dbtx, &dbsqlc.JobInsertFullParams{ + job, err := dbsqlc.New().JobInsertFull(schemaTemplateParam(ctx, params.Schema), e.dbtx, &dbsqlc.JobInsertFullParams{ Attempt: int16(min(params.Attempt, math.MaxInt16)), //nolint:gosec AttemptedAt: params.AttemptedAt, AttemptedBy: params.AttemptedBy, @@ -356,7 +362,7 @@ func (e *Executor) JobList(ctx context.Context, params *riverdriver.JobListParam "where_clause": {Value: params.WhereClause}, }, params.NamedArgs) - jobs, err := dbsqlc.New().JobList(ctx, e.dbtx, params.Max) + jobs, err := dbsqlc.New().JobList(schemaTemplateParam(ctx, params.Schema), e.dbtx, params.Max) if err != nil { return nil, interpretError(err) } @@ -364,15 +370,21 @@ func (e *Executor) JobList(ctx context.Context, params *riverdriver.JobListParam } func (e *Executor) JobRescueMany(ctx context.Context, params *riverdriver.JobRescueManyParams) (*struct{}, error) { - err := dbsqlc.New().JobRescueMany(ctx, e.dbtx, (*dbsqlc.JobRescueManyParams)(params)) + err := dbsqlc.New().JobRescueMany(schemaTemplateParam(ctx, params.Schema), e.dbtx, &dbsqlc.JobRescueManyParams{ + ID: params.ID, + Error: params.Error, + FinalizedAt: params.FinalizedAt, + ScheduledAt: params.ScheduledAt, + State: params.State, + }) if err != nil { return nil, interpretError(err) } return &struct{}{}, nil } -func (e *Executor) JobRetry(ctx context.Context, id int64) (*rivertype.JobRow, error) { - job, err := dbsqlc.New().JobRetry(ctx, e.dbtx, id) +func (e *Executor) JobRetry(ctx context.Context, params *riverdriver.JobRetryParams) (*rivertype.JobRow, error) { + job, err := dbsqlc.New().JobRetry(schemaTemplateParam(ctx, params.Schema), e.dbtx, params.ID) if err != nil { return nil, interpretError(err) } @@ -380,7 +392,7 @@ func (e *Executor) JobRetry(ctx context.Context, id int64) (*rivertype.JobRow, e } func (e *Executor) JobSchedule(ctx context.Context, params *riverdriver.JobScheduleParams) ([]*riverdriver.JobScheduleResult, error) { - scheduleResults, err := dbsqlc.New().JobSchedule(ctx, e.dbtx, &dbsqlc.JobScheduleParams{ + scheduleResults, err := dbsqlc.New().JobSchedule(schemaTemplateParam(ctx, params.Schema), e.dbtx, &dbsqlc.JobScheduleParams{ Max: int64(params.Max), Now: params.Now, }) @@ -435,7 +447,7 @@ func (e *Executor) JobSetStateIfRunningMany(ctx context.Context, params *riverdr setStateParams.State[i] = string(params.State[i]) } - jobs, err := dbsqlc.New().JobSetStateIfRunningMany(ctx, e.dbtx, setStateParams) + jobs, err := dbsqlc.New().JobSetStateIfRunningMany(schemaTemplateParam(ctx, params.Schema), e.dbtx, setStateParams) if err != nil { return nil, interpretError(err) } @@ -443,7 +455,7 @@ func (e *Executor) JobSetStateIfRunningMany(ctx context.Context, params *riverdr } func (e *Executor) JobUpdate(ctx context.Context, params *riverdriver.JobUpdateParams) (*rivertype.JobRow, error) { - job, err := dbsqlc.New().JobUpdate(ctx, e.dbtx, &dbsqlc.JobUpdateParams{ + job, err := dbsqlc.New().JobUpdate(schemaTemplateParam(ctx, params.Schema), e.dbtx, &dbsqlc.JobUpdateParams{ ID: params.ID, AttemptedAtDoUpdate: params.AttemptedAtDoUpdate, Attempt: int16(min(params.Attempt, math.MaxInt16)), //nolint:gosec @@ -466,7 +478,7 @@ func (e *Executor) JobUpdate(ctx context.Context, params *riverdriver.JobUpdateP } func (e *Executor) LeaderAttemptElect(ctx context.Context, params *riverdriver.LeaderElectParams) (bool, error) { - numElectionsWon, err := dbsqlc.New().LeaderAttemptElect(ctx, e.dbtx, &dbsqlc.LeaderAttemptElectParams{ + numElectionsWon, err := dbsqlc.New().LeaderAttemptElect(schemaTemplateParam(ctx, params.Schema), e.dbtx, &dbsqlc.LeaderAttemptElectParams{ LeaderID: params.LeaderID, TTL: params.TTL, }) @@ -477,7 +489,7 @@ func (e *Executor) LeaderAttemptElect(ctx context.Context, params *riverdriver.L } func (e *Executor) LeaderAttemptReelect(ctx context.Context, params *riverdriver.LeaderElectParams) (bool, error) { - numElectionsWon, err := dbsqlc.New().LeaderAttemptReelect(ctx, e.dbtx, &dbsqlc.LeaderAttemptReelectParams{ + numElectionsWon, err := dbsqlc.New().LeaderAttemptReelect(schemaTemplateParam(ctx, params.Schema), e.dbtx, &dbsqlc.LeaderAttemptReelectParams{ LeaderID: params.LeaderID, TTL: params.TTL, }) @@ -487,16 +499,16 @@ func (e *Executor) LeaderAttemptReelect(ctx context.Context, params *riverdriver return numElectionsWon > 0, nil } -func (e *Executor) LeaderDeleteExpired(ctx context.Context) (int, error) { - numDeleted, err := dbsqlc.New().LeaderDeleteExpired(ctx, e.dbtx) +func (e *Executor) LeaderDeleteExpired(ctx context.Context, params *riverdriver.LeaderDeleteExpiredParams) (int, error) { + numDeleted, err := dbsqlc.New().LeaderDeleteExpired(schemaTemplateParam(ctx, params.Schema), e.dbtx) if err != nil { return 0, interpretError(err) } return int(numDeleted), nil } -func (e *Executor) LeaderGetElectedLeader(ctx context.Context) (*riverdriver.Leader, error) { - leader, err := dbsqlc.New().LeaderGetElectedLeader(ctx, e.dbtx) +func (e *Executor) LeaderGetElectedLeader(ctx context.Context, params *riverdriver.LeaderGetElectedLeaderParams) (*riverdriver.Leader, error) { + leader, err := dbsqlc.New().LeaderGetElectedLeader(schemaTemplateParam(ctx, params.Schema), e.dbtx) if err != nil { return nil, interpretError(err) } @@ -504,7 +516,7 @@ func (e *Executor) LeaderGetElectedLeader(ctx context.Context) (*riverdriver.Lea } func (e *Executor) LeaderInsert(ctx context.Context, params *riverdriver.LeaderInsertParams) (*riverdriver.Leader, error) { - leader, err := dbsqlc.New().LeaderInsert(ctx, e.dbtx, &dbsqlc.LeaderInsertParams{ + leader, err := dbsqlc.New().LeaderInsert(schemaTemplateParam(ctx, params.Schema), e.dbtx, &dbsqlc.LeaderInsertParams{ ElectedAt: params.ElectedAt, ExpiresAt: params.ExpiresAt, LeaderID: params.LeaderID, @@ -517,9 +529,10 @@ func (e *Executor) LeaderInsert(ctx context.Context, params *riverdriver.LeaderI } func (e *Executor) LeaderResign(ctx context.Context, params *riverdriver.LeaderResignParams) (bool, error) { - numResigned, err := dbsqlc.New().LeaderResign(ctx, e.dbtx, &dbsqlc.LeaderResignParams{ + numResigned, err := dbsqlc.New().LeaderResign(schemaTemplateParam(ctx, params.Schema), e.dbtx, &dbsqlc.LeaderResignParams{ LeaderID: params.LeaderID, LeadershipTopic: params.LeadershipTopic, + Schema: pgtype.Text{String: params.Schema, Valid: params.Schema != ""}, }) if err != nil { return false, interpretError(err) @@ -527,9 +540,9 @@ func (e *Executor) LeaderResign(ctx context.Context, params *riverdriver.LeaderR return numResigned > 0, nil } -func (e *Executor) MigrationDeleteAssumingMainMany(ctx context.Context, versions []int) ([]*riverdriver.Migration, error) { - migrations, err := dbsqlc.New().RiverMigrationDeleteAssumingMainMany(ctx, e.dbtx, - sliceutil.Map(versions, func(v int) int64 { return int64(v) })) +func (e *Executor) MigrationDeleteAssumingMainMany(ctx context.Context, params *riverdriver.MigrationDeleteAssumingMainManyParams) ([]*riverdriver.Migration, error) { + migrations, err := dbsqlc.New().RiverMigrationDeleteAssumingMainMany(schemaTemplateParam(ctx, params.Schema), e.dbtx, + sliceutil.Map(params.Versions, func(v int) int64 { return int64(v) })) if err != nil { return nil, interpretError(err) } @@ -542,10 +555,10 @@ func (e *Executor) MigrationDeleteAssumingMainMany(ctx context.Context, versions }), nil } -func (e *Executor) MigrationDeleteByLineAndVersionMany(ctx context.Context, line string, versions []int) ([]*riverdriver.Migration, error) { - migrations, err := dbsqlc.New().RiverMigrationDeleteByLineAndVersionMany(ctx, e.dbtx, &dbsqlc.RiverMigrationDeleteByLineAndVersionManyParams{ - Line: line, - Version: sliceutil.Map(versions, func(v int) int64 { return int64(v) }), +func (e *Executor) MigrationDeleteByLineAndVersionMany(ctx context.Context, params *riverdriver.MigrationDeleteByLineAndVersionManyParams) ([]*riverdriver.Migration, error) { + migrations, err := dbsqlc.New().RiverMigrationDeleteByLineAndVersionMany(schemaTemplateParam(ctx, params.Schema), e.dbtx, &dbsqlc.RiverMigrationDeleteByLineAndVersionManyParams{ + Line: params.Line, + Version: sliceutil.Map(params.Versions, func(v int) int64 { return int64(v) }), }) if err != nil { return nil, interpretError(err) @@ -553,8 +566,8 @@ func (e *Executor) MigrationDeleteByLineAndVersionMany(ctx context.Context, line return sliceutil.Map(migrations, migrationFromInternal), nil } -func (e *Executor) MigrationGetAllAssumingMain(ctx context.Context) ([]*riverdriver.Migration, error) { - migrations, err := dbsqlc.New().RiverMigrationGetAllAssumingMain(ctx, e.dbtx) +func (e *Executor) MigrationGetAllAssumingMain(ctx context.Context, params *riverdriver.MigrationGetAllAssumingMainParams) ([]*riverdriver.Migration, error) { + migrations, err := dbsqlc.New().RiverMigrationGetAllAssumingMain(schemaTemplateParam(ctx, params.Schema), e.dbtx) if err != nil { return nil, interpretError(err) } @@ -567,18 +580,18 @@ func (e *Executor) MigrationGetAllAssumingMain(ctx context.Context) ([]*riverdri }), nil } -func (e *Executor) MigrationGetByLine(ctx context.Context, line string) ([]*riverdriver.Migration, error) { - migrations, err := dbsqlc.New().RiverMigrationGetByLine(ctx, e.dbtx, line) +func (e *Executor) MigrationGetByLine(ctx context.Context, params *riverdriver.MigrationGetByLineParams) ([]*riverdriver.Migration, error) { + migrations, err := dbsqlc.New().RiverMigrationGetByLine(schemaTemplateParam(ctx, params.Schema), e.dbtx, params.Line) if err != nil { return nil, interpretError(err) } return sliceutil.Map(migrations, migrationFromInternal), nil } -func (e *Executor) MigrationInsertMany(ctx context.Context, line string, versions []int) ([]*riverdriver.Migration, error) { - migrations, err := dbsqlc.New().RiverMigrationInsertMany(ctx, e.dbtx, &dbsqlc.RiverMigrationInsertManyParams{ - Line: line, - Version: sliceutil.Map(versions, func(v int) int64 { return int64(v) }), +func (e *Executor) MigrationInsertMany(ctx context.Context, params *riverdriver.MigrationInsertManyParams) ([]*riverdriver.Migration, error) { + migrations, err := dbsqlc.New().RiverMigrationInsertMany(schemaTemplateParam(ctx, params.Schema), e.dbtx, &dbsqlc.RiverMigrationInsertManyParams{ + Line: params.Line, + Version: sliceutil.Map(params.Versions, func(v int) int64 { return int64(v) }), }) if err != nil { return nil, interpretError(err) @@ -586,9 +599,9 @@ func (e *Executor) MigrationInsertMany(ctx context.Context, line string, version return sliceutil.Map(migrations, migrationFromInternal), nil } -func (e *Executor) MigrationInsertManyAssumingMain(ctx context.Context, versions []int) ([]*riverdriver.Migration, error) { - migrations, err := dbsqlc.New().RiverMigrationInsertManyAssumingMain(ctx, e.dbtx, - sliceutil.Map(versions, func(v int) int64 { return int64(v) }), +func (e *Executor) MigrationInsertManyAssumingMain(ctx context.Context, params *riverdriver.MigrationInsertManyAssumingMainParams) ([]*riverdriver.Migration, error) { + migrations, err := dbsqlc.New().RiverMigrationInsertManyAssumingMain(schemaTemplateParam(ctx, params.Schema), e.dbtx, + sliceutil.Map(params.Versions, func(v int) int64 { return int64(v) }), ) if err != nil { return nil, interpretError(err) @@ -605,6 +618,7 @@ func (e *Executor) MigrationInsertManyAssumingMain(ctx context.Context, versions func (e *Executor) NotifyMany(ctx context.Context, params *riverdriver.NotifyManyParams) error { return dbsqlc.New().PGNotifyMany(ctx, e.dbtx, &dbsqlc.PGNotifyManyParams{ Payload: params.Payload, + Schema: pgtype.Text{String: params.Schema, Valid: params.Schema != ""}, Topic: params.Topic, }) } @@ -615,7 +629,7 @@ func (e *Executor) PGAdvisoryXactLock(ctx context.Context, key int64) (*struct{} } func (e *Executor) QueueCreateOrSetUpdatedAt(ctx context.Context, params *riverdriver.QueueCreateOrSetUpdatedAtParams) (*rivertype.Queue, error) { - queue, err := dbsqlc.New().QueueCreateOrSetUpdatedAt(ctx, e.dbtx, &dbsqlc.QueueCreateOrSetUpdatedAtParams{ + queue, err := dbsqlc.New().QueueCreateOrSetUpdatedAt(schemaTemplateParam(ctx, params.Schema), e.dbtx, &dbsqlc.QueueCreateOrSetUpdatedAtParams{ Metadata: params.Metadata, Name: params.Name, PausedAt: params.PausedAt, @@ -628,7 +642,7 @@ func (e *Executor) QueueCreateOrSetUpdatedAt(ctx context.Context, params *riverd } func (e *Executor) QueueDeleteExpired(ctx context.Context, params *riverdriver.QueueDeleteExpiredParams) ([]string, error) { - queues, err := dbsqlc.New().QueueDeleteExpired(ctx, e.dbtx, &dbsqlc.QueueDeleteExpiredParams{ + queues, err := dbsqlc.New().QueueDeleteExpired(schemaTemplateParam(ctx, params.Schema), e.dbtx, &dbsqlc.QueueDeleteExpiredParams{ Max: int64(params.Max), UpdatedAtHorizon: params.UpdatedAtHorizon, }) @@ -642,16 +656,16 @@ func (e *Executor) QueueDeleteExpired(ctx context.Context, params *riverdriver.Q return queueNames, nil } -func (e *Executor) QueueGet(ctx context.Context, name string) (*rivertype.Queue, error) { - queue, err := dbsqlc.New().QueueGet(ctx, e.dbtx, name) +func (e *Executor) QueueGet(ctx context.Context, params *riverdriver.QueueGetParams) (*rivertype.Queue, error) { + queue, err := dbsqlc.New().QueueGet(schemaTemplateParam(ctx, params.Schema), e.dbtx, params.Name) if err != nil { return nil, interpretError(err) } return queueFromInternal(queue), nil } -func (e *Executor) QueueList(ctx context.Context, limit int) ([]*rivertype.Queue, error) { - internalQueues, err := dbsqlc.New().QueueList(ctx, e.dbtx, int32(min(limit, math.MaxInt32))) //nolint:gosec +func (e *Executor) QueueList(ctx context.Context, params *riverdriver.QueueListParams) ([]*rivertype.Queue, error) { + internalQueues, err := dbsqlc.New().QueueList(schemaTemplateParam(ctx, params.Schema), e.dbtx, int32(min(params.Limit, math.MaxInt32))) //nolint:gosec if err != nil { return nil, interpretError(err) } @@ -662,30 +676,30 @@ func (e *Executor) QueueList(ctx context.Context, limit int) ([]*rivertype.Queue return queues, nil } -func (e *Executor) QueuePause(ctx context.Context, name string) error { - res, err := dbsqlc.New().QueuePause(ctx, e.dbtx, name) +func (e *Executor) QueuePause(ctx context.Context, params *riverdriver.QueuePauseParams) error { + res, err := dbsqlc.New().QueuePause(schemaTemplateParam(ctx, params.Schema), e.dbtx, params.Name) if err != nil { return interpretError(err) } - if res.RowsAffected() == 0 && name != riverdriver.AllQueuesString { + if res.RowsAffected() == 0 && params.Name != riverdriver.AllQueuesString { return rivertype.ErrNotFound } return nil } -func (e *Executor) QueueResume(ctx context.Context, name string) error { - res, err := dbsqlc.New().QueueResume(ctx, e.dbtx, name) +func (e *Executor) QueueResume(ctx context.Context, params *riverdriver.QueueResumeParams) error { + res, err := dbsqlc.New().QueueResume(schemaTemplateParam(ctx, params.Schema), e.dbtx, params.Name) if err != nil { return interpretError(err) } - if res.RowsAffected() == 0 && name != riverdriver.AllQueuesString { + if res.RowsAffected() == 0 && params.Name != riverdriver.AllQueuesString { return rivertype.ErrNotFound } return nil } func (e *Executor) QueueUpdate(ctx context.Context, params *riverdriver.QueueUpdateParams) (*rivertype.Queue, error) { - queue, err := dbsqlc.New().QueueUpdate(ctx, e.dbtx, &dbsqlc.QueueUpdateParams{ + queue, err := dbsqlc.New().QueueUpdate(schemaTemplateParam(ctx, params.Schema), e.dbtx, &dbsqlc.QueueUpdateParams{ Metadata: params.Metadata, MetadataDoUpdate: params.MetadataDoUpdate, Name: params.Name, @@ -696,8 +710,14 @@ func (e *Executor) QueueUpdate(ctx context.Context, params *riverdriver.QueueUpd return queueFromInternal(queue), nil } -func (e *Executor) TableExists(ctx context.Context, tableName string) (bool, error) { - exists, err := dbsqlc.New().TableExists(ctx, e.dbtx, tableName) +func (e *Executor) TableExists(ctx context.Context, params *riverdriver.TableExistsParams) (bool, error) { + // Different from other operations because the schemaAndTable name is a parameter. + schemaAndTable := params.Table + if params.Schema != "" { + schemaAndTable = params.Schema + "." + schemaAndTable + } + + exists, err := dbsqlc.New().TableExists(ctx, e.dbtx, schemaAndTable) return exists, interpretError(err) } @@ -717,8 +737,9 @@ func (t *ExecutorTx) Rollback(ctx context.Context) error { type Listener struct { conn *pgx.Conn dbPool *pgxpool.Pool - prefix string + prefix string // schema with a dot on the end (very minor optimization) mu sync.Mutex + schema string } func (l *Listener) Close(ctx context.Context) error { @@ -755,10 +776,15 @@ func (l *Listener) Connect(ctx context.Context) error { return err } - var schema string - if err := poolConn.QueryRow(ctx, "SELECT current_schema();").Scan(&schema); err != nil { - poolConn.Release() - return err + // Use a configured schema if non-empty, otherwise try to select the current + // schema based on `search_path`. + schema := l.schema + if schema == "" { + if err := poolConn.QueryRow(ctx, "SELECT current_schema();").Scan(&schema); err != nil { + poolConn.Release() + return err + } + l.schema = schema } l.prefix = schema + "." @@ -784,6 +810,13 @@ func (l *Listener) Ping(ctx context.Context) error { return l.conn.Ping(ctx) } +func (l *Listener) Schema() string { + l.mu.Lock() + defer l.mu.Unlock() + + return l.schema +} + func (l *Listener) Unlisten(ctx context.Context, topic string) error { l.mu.Lock() defer l.mu.Unlock() @@ -946,3 +979,13 @@ func queueFromInternal(internal *dbsqlc.RiverQueue) *rivertype.Queue { UpdatedAt: internal.UpdatedAt.UTC(), } } + +func schemaTemplateParam(ctx context.Context, schema string) context.Context { + if schema != "" { + schema += "." + } + + return sqlctemplate.WithReplacements(ctx, map[string]sqlctemplate.Replacement{ + "schema": {Value: schema}, + }, nil) +} diff --git a/riverdriver/riverpgxv5/river_pgx_v5_driver_test.go b/riverdriver/riverpgxv5/river_pgx_v5_driver_test.go index 4372d165..979c0eb0 100644 --- a/riverdriver/riverpgxv5/river_pgx_v5_driver_test.go +++ b/riverdriver/riverpgxv5/river_pgx_v5_driver_test.go @@ -15,6 +15,7 @@ import ( "github.com/stretchr/testify/require" "github.com/riverqueue/river/riverdriver" + "github.com/riverqueue/river/rivershared/sqlctemplate" "github.com/riverqueue/river/rivertype" ) @@ -145,3 +146,45 @@ func testPool(ctx context.Context, t *testing.T, config *pgxpool.Config) *pgxpoo t.Cleanup(dbPool.Close) return dbPool } + +func TestSchemaTemplateParam(t *testing.T) { + t.Parallel() + + ctx := context.Background() + + type testBundle struct{} + + setup := func(t *testing.T) (*sqlctemplate.Replacer, *testBundle) { //nolint:unparam + t.Helper() + + return &sqlctemplate.Replacer{}, &testBundle{} + } + + t.Run("NoSchema", func(t *testing.T) { + t.Parallel() + + replacer, _ := setup(t) + + updatedSQL, _, err := replacer.RunSafely( + schemaTemplateParam(ctx, ""), + "SELECT 1 FROM /* TEMPLATE: schema */river_job", + nil, + ) + require.NoError(t, err) + require.Equal(t, "SELECT 1 FROM river_job", updatedSQL) + }) + + t.Run("WithSchema", func(t *testing.T) { + t.Parallel() + + replacer, _ := setup(t) + + updatedSQL, _, err := replacer.RunSafely( + schemaTemplateParam(ctx, "custom_schema"), + "SELECT 1 FROM /* TEMPLATE: schema */river_job", + nil, + ) + require.NoError(t, err) + require.Equal(t, "SELECT 1 FROM custom_schema.river_job", updatedSQL) + }) +} diff --git a/rivermigrate/migration/commit_required/001_first.down.sql b/rivermigrate/migration/commit_required/001_first.down.sql index 3b21ed14..ed0b74be 100644 --- a/rivermigrate/migration/commit_required/001_first.down.sql +++ b/rivermigrate/migration/commit_required/001_first.down.sql @@ -1 +1 @@ -DROP TYPE foobar; +DROP TYPE /* TEMPLATE: schema */foobar; diff --git a/rivermigrate/migration/commit_required/001_first.up.sql b/rivermigrate/migration/commit_required/001_first.up.sql index 4646c8ea..84b70977 100644 --- a/rivermigrate/migration/commit_required/001_first.up.sql +++ b/rivermigrate/migration/commit_required/001_first.up.sql @@ -1,2 +1,2 @@ -- create a foobar enum with values foo, bar: -CREATE TYPE foobar AS ENUM ('foo', 'bar'); +CREATE TYPE /* TEMPLATE: schema */foobar AS ENUM ('foo', 'bar'); diff --git a/rivermigrate/migration/commit_required/002_second.down.sql b/rivermigrate/migration/commit_required/002_second.down.sql index 8c3007ce..3027c1a4 100644 --- a/rivermigrate/migration/commit_required/002_second.down.sql +++ b/rivermigrate/migration/commit_required/002_second.down.sql @@ -1 +1,3 @@ -- not truly reversible, can't remove enum values. +-- +-- here to prevent templating system from throwing here: /* TEMPLATE: schema */ \ No newline at end of file diff --git a/rivermigrate/migration/commit_required/002_second.up.sql b/rivermigrate/migration/commit_required/002_second.up.sql index c1cd1631..2e391ce0 100644 --- a/rivermigrate/migration/commit_required/002_second.up.sql +++ b/rivermigrate/migration/commit_required/002_second.up.sql @@ -1 +1 @@ -ALTER TYPE foobar ADD VALUE 'baz' AFTER 'bar'; +ALTER TYPE /* TEMPLATE: schema */foobar ADD VALUE 'baz' AFTER 'bar'; diff --git a/rivermigrate/migration/commit_required/003_third.down.sql b/rivermigrate/migration/commit_required/003_third.down.sql index 898efaf7..1e584651 100644 --- a/rivermigrate/migration/commit_required/003_third.down.sql +++ b/rivermigrate/migration/commit_required/003_third.down.sql @@ -1 +1 @@ -DROP FUNCTION foobar_in_bitmask; +DROP FUNCTION /* TEMPLATE: schema */foobar_in_bitmask; diff --git a/rivermigrate/migration/commit_required/003_third.up.sql b/rivermigrate/migration/commit_required/003_third.up.sql index 44e9a824..33ab68ce 100644 --- a/rivermigrate/migration/commit_required/003_third.up.sql +++ b/rivermigrate/migration/commit_required/003_third.up.sql @@ -1,4 +1,4 @@ -CREATE OR REPLACE FUNCTION foobar_in_bitmask(bitmask BIT(8), val foobar) +CREATE OR REPLACE FUNCTION /* TEMPLATE: schema */foobar_in_bitmask(bitmask BIT(8), val /* TEMPLATE: schema */foobar) RETURNS boolean LANGUAGE SQL IMMUTABLE diff --git a/rivermigrate/river_migrate.go b/rivermigrate/river_migrate.go index d5574b55..68e8d0a6 100644 --- a/rivermigrate/river_migrate.go +++ b/rivermigrate/river_migrate.go @@ -20,6 +20,7 @@ import ( "github.com/riverqueue/river/riverdriver" "github.com/riverqueue/river/rivershared/baseservice" "github.com/riverqueue/river/rivershared/levenshtein" + "github.com/riverqueue/river/rivershared/sqlctemplate" "github.com/riverqueue/river/rivershared/util/maputil" "github.com/riverqueue/river/rivershared/util/sliceutil" "github.com/riverqueue/river/rivershared/util/valutil" @@ -59,6 +60,8 @@ type Config struct { // specified, logs will be emitted to STDOUT with messages at warn level // or higher. Logger *slog.Logger + + schema string } // Migrator is a database migration tool for River which can run up or down @@ -69,6 +72,8 @@ type Migrator[TTx any] struct { driver riverdriver.Driver[TTx] line string migrations map[int]Migration // allows us to inject test migrations + replacer sqlctemplate.Replacer + schema string } // New returns a new migrator with the given database driver and configuration. @@ -148,6 +153,7 @@ func New[TTx any](driver riverdriver.Driver[TTx], config *Config) (*Migrator[TTx driver: driver, line: line, migrations: validateAndInit(riverMigrations), + schema: config.schema, }), nil } @@ -412,11 +418,18 @@ func (m *Migrator[TTx]) migrateDown(ctx context.Context, exec riverdriver.Execut // 005, we can remove migrations with a line included, but otherwise we // must omit the `line` column from queries because it doesn't exist. if m.line == riverdriver.MigrationLineMain && slices.Min(versions) <= migrateVersionLineColumnAdded { - if _, err := exec.MigrationDeleteAssumingMainMany(ctx, versions); err != nil { + if _, err := exec.MigrationDeleteAssumingMainMany(ctx, &riverdriver.MigrationDeleteAssumingMainManyParams{ + Versions: versions, + Schema: m.schema, + }); err != nil { return nil, fmt.Errorf("error inserting migration rows for versions %+v assuming main: %w", res.Versions, err) } } else { - if _, err := exec.MigrationDeleteByLineAndVersionMany(ctx, m.line, versions); err != nil { + if _, err := exec.MigrationDeleteByLineAndVersionMany(ctx, &riverdriver.MigrationDeleteByLineAndVersionManyParams{ + Line: m.line, + Schema: m.schema, + Versions: versions, + }); err != nil { return nil, fmt.Errorf("error deleting migration rows for versions %+v on line %q: %w", res.Versions, m.line, err) } } @@ -453,14 +466,18 @@ func (m *Migrator[TTx]) migrateUp(ctx context.Context, exec riverdriver.Executor // 005, we can insert migrations with a line included, but otherwise we // must omit the `line` column from queries because it doesn't exist. if m.line == riverdriver.MigrationLineMain && slices.Max(versions) < migrateVersionLineColumnAdded { - if _, err := exec.MigrationInsertManyAssumingMain(ctx, versions); err != nil { + if _, err := exec.MigrationInsertManyAssumingMain(ctx, &riverdriver.MigrationInsertManyAssumingMainParams{ + Schema: m.schema, + Versions: versions, + }); err != nil { return nil, fmt.Errorf("error inserting migration rows for versions %+v assuming main: %w", res.Versions, err) } } else { - if _, err := exec.MigrationInsertMany(ctx, - m.line, - versions, - ); err != nil { + if _, err := exec.MigrationInsertMany(ctx, &riverdriver.MigrationInsertManyParams{ + Line: m.line, + Schema: m.schema, + Versions: versions, + }); err != nil { return nil, fmt.Errorf("error inserting migration rows for versions %+v on line %q: %w", res.Versions, m.line, err) } } @@ -547,6 +564,14 @@ func (m *Migrator[TTx]) applyMigrations(ctx context.Context, exec riverdriver.Ex return res, nil } + var schema string + if m.schema != "" { + schema = m.schema + "." + } + schemaReplacement := map[string]sqlctemplate.Replacement{ + "schema": {Value: schema}, + } + for _, versionBundle := range sortedTargetMigrations { var sql string switch direction { @@ -556,6 +581,13 @@ func (m *Migrator[TTx]) applyMigrations(ctx context.Context, exec riverdriver.Ex sql = versionBundle.SQLUp } + // Most migrations contain schema in their SQL by necessity, but some of + // the test ones do not because they only run trivial operations. + if strings.Contains(sql, "/* TEMPLATE: schema */") { + ctx := sqlctemplate.WithReplacements(ctx, schemaReplacement, nil) + sql, _ = m.replacer.Run(ctx, sql, nil) + } + var duration time.Duration if !opts.DryRun { @@ -598,7 +630,10 @@ func (m *Migrator[TTx]) applyMigrations(ctx context.Context, exec riverdriver.Ex // because otherwise the existing transaction would become aborted on an // unsuccessful `river_migration` check.) func (m *Migrator[TTx]) existingMigrations(ctx context.Context, exec riverdriver.Executor) ([]*riverdriver.Migration, error) { - migrateTableExists, err := exec.TableExists(ctx, "river_migration") + migrateTableExists, err := exec.TableExists(ctx, &riverdriver.TableExistsParams{ + Schema: m.schema, + Table: "river_migration", + }) if err != nil { return nil, fmt.Errorf("error checking if `%s` exists: %w", "river_migration", err) } @@ -610,7 +645,11 @@ func (m *Migrator[TTx]) existingMigrations(ctx context.Context, exec riverdriver return nil, nil } - lineColumnExists, err := exec.ColumnExists(ctx, "river_migration", "line") + lineColumnExists, err := exec.ColumnExists(ctx, &riverdriver.ColumnExistsParams{ + Column: "line", + Schema: m.schema, + Table: "river_migration", + }) if err != nil { return nil, fmt.Errorf("error checking if `%s.%s` exists: %w", "river_migration", "line", err) } @@ -620,7 +659,9 @@ func (m *Migrator[TTx]) existingMigrations(ctx context.Context, exec riverdriver return nil, errors.New("can't add a non-main migration line until `river_migration.line` is raised; fully migrate the main migration line and try again") } - migrations, err := exec.MigrationGetAllAssumingMain(ctx) + migrations, err := exec.MigrationGetAllAssumingMain(ctx, &riverdriver.MigrationGetAllAssumingMainParams{ + Schema: m.schema, + }) if err != nil { return nil, fmt.Errorf("error getting existing migrations: %w", err) } @@ -628,7 +669,10 @@ func (m *Migrator[TTx]) existingMigrations(ctx context.Context, exec riverdriver return migrations, nil } - migrations, err := exec.MigrationGetByLine(ctx, m.line) + migrations, err := exec.MigrationGetByLine(ctx, &riverdriver.MigrationGetByLineParams{ + Line: m.line, + Schema: m.schema, + }) if err != nil { return nil, fmt.Errorf("error getting existing migrations for line %q: %w", m.line, err) } diff --git a/rivermigrate/river_migrate_test.go b/rivermigrate/river_migrate_test.go index 2fb96dac..44e5721f 100644 --- a/rivermigrate/river_migrate_test.go +++ b/rivermigrate/river_migrate_test.go @@ -11,6 +11,7 @@ import ( "slices" "testing" + "github.com/davecgh/go-spew/spew" "github.com/jackc/pgerrcode" "github.com/jackc/pgx/v5" "github.com/jackc/pgx/v5/pgconn" @@ -24,6 +25,7 @@ import ( "github.com/riverqueue/river/riverdriver/riverdatabasesql" "github.com/riverqueue/river/riverdriver/riverpgxv5" "github.com/riverqueue/river/rivershared/riversharedtest" + "github.com/riverqueue/river/rivershared/sqlctemplate" "github.com/riverqueue/river/rivershared/util/randutil" "github.com/riverqueue/river/rivershared/util/sliceutil" ) @@ -74,6 +76,7 @@ func TestMigrator(t *testing.T) { dbPool *pgxpool.Pool driver *driverWithAlternateLine logger *slog.Logger + schema string } setup := func(t *testing.T) (*Migrator[pgx.Tx], *testBundle) { @@ -86,28 +89,27 @@ func TestMigrator(t *testing.T) { // // To make this easier to clean up afterward, we create a new, clean schema // for each test run and then drop it afterward. - baseDBPool := riverinternaltest.TestDB(ctx, t) - schemaName := "river_migrate_test_" + randutil.Hex(8) - _, err := baseDBPool.Exec(ctx, "CREATE SCHEMA "+schemaName) + dbPool := riverinternaltest.TestDB(ctx, t) + schema := "river_migrate_test_" + randutil.Hex(8) + _, err := dbPool.Exec(ctx, "CREATE SCHEMA "+schema) require.NoError(t, err) t.Cleanup(func() { - _, err := baseDBPool.Exec(ctx, fmt.Sprintf("DROP SCHEMA %s CASCADE", schemaName)) + _, err := dbPool.Exec(ctx, fmt.Sprintf("DROP SCHEMA %s CASCADE", schema)) require.NoError(t, err) }) - newSchemaConfig := baseDBPool.Config() - newSchemaConfig.ConnConfig.RuntimeParams["search_path"] = schemaName - newSchemaPool, err := pgxpool.NewWithConfig(ctx, newSchemaConfig) - require.NoError(t, err) - bundle := &testBundle{ - dbPool: newSchemaPool, - driver: &driverWithAlternateLine{Driver: riverpgxv5.New(newSchemaPool)}, + dbPool: dbPool, + driver: &driverWithAlternateLine{Driver: riverpgxv5.New(dbPool)}, logger: riversharedtest.Logger(t), + schema: schema, } - migrator, err := New(bundle.driver, &Config{Logger: bundle.logger}) + migrator, err := New(bundle.driver, &Config{ + Logger: bundle.logger, + schema: schema, + }) require.NoError(t, err) migrator.migrations = migrationsBundle.WithTestVersionsMap @@ -126,7 +128,10 @@ func TestMigrator(t *testing.T) { t.Cleanup(func() { require.NoError(t, tx.Rollback()) }) driver := riverdatabasesql.New(stdPool) - migrator, err := New(driver, &Config{Logger: bundle.logger}) + migrator, err := New(driver, &Config{ + Logger: bundle.logger, + schema: bundle.schema, + }) require.NoError(t, err) migrator.migrations = migrationsBundle.WithTestVersionsMap @@ -252,7 +257,7 @@ func TestMigrator(t *testing.T) { require.Equal(t, DirectionUp, res.Direction) require.Equal(t, seqOneTo(migrateVersionIncludingRiverJob), sliceutil.Map(res.Versions, migrateVersionToInt)) - err = dbExecError(ctx, bundle.driver.GetExecutor(), "SELECT * FROM river_job") + err = dbExecError(ctx, bundle.driver.GetExecutor(), fmt.Sprintf("SELECT * FROM %s.river_job", bundle.schema)) require.NoError(t, err) // Run once more to go down one more step @@ -265,7 +270,7 @@ func TestMigrator(t *testing.T) { version := res.Versions[0] require.Equal(t, "initial schema", version.Name) - err = dbExecError(ctx, bundle.driver.GetExecutor(), "SELECT * FROM river_job") + err = dbExecError(ctx, bundle.driver.GetExecutor(), fmt.Sprintf("SELECT * FROM %s.river_job", bundle.schema)) require.Error(t, err) } }) @@ -296,12 +301,15 @@ func TestMigrator(t *testing.T) { require.Equal(t, []int{migrationsBundle.WithTestVersionsMaxVersion, migrationsBundle.WithTestVersionsMaxVersion - 1}, sliceutil.Map(res.Versions, migrateVersionToInt)) - migrations, err := bundle.driver.GetExecutor().MigrationGetByLine(ctx, riverdriver.MigrationLineMain) + migrations, err := bundle.driver.GetExecutor().MigrationGetByLine(ctx, &riverdriver.MigrationGetByLineParams{ + Line: riverdriver.MigrationLineMain, + Schema: bundle.schema, + }) require.NoError(t, err) require.Equal(t, seqOneTo(migrationsBundle.WithTestVersionsMaxVersion-2), sliceutil.Map(migrations, driverMigrationToInt)) - err = dbExecError(ctx, bundle.driver.GetExecutor(), "SELECT name FROM test_table") + err = dbExecError(ctx, bundle.driver.GetExecutor(), fmt.Sprintf("SELECT name FROM %s.test_table", bundle.schema)) require.Error(t, err) }) @@ -320,7 +328,10 @@ func TestMigrator(t *testing.T) { require.NoError(t, err) require.Equal(t, []int{}, sliceutil.Map(res.Versions, migrateVersionToInt)) - migrations, err := bundle.driver.GetExecutor().MigrationGetByLine(ctx, riverdriver.MigrationLineMain) + migrations, err := bundle.driver.GetExecutor().MigrationGetByLine(ctx, &riverdriver.MigrationGetByLineParams{ + Line: riverdriver.MigrationLineMain, + Schema: bundle.schema, + }) require.NoError(t, err) require.Equal(t, seqOneTo(migrationsBundle.MaxVersion), sliceutil.Map(migrations, driverMigrationToInt)) @@ -337,9 +348,12 @@ func TestMigrator(t *testing.T) { res, err := migrator.Migrate(ctx, DirectionDown, &MigrateOpts{MaxSteps: 1}) require.NoError(t, err) + spew.Dump(res.Versions) require.Equal(t, []int{migrationsBundle.MaxVersion}, sliceutil.Map(res.Versions, migrateVersionToInt)) - migrations, err := migrator.driver.UnwrapExecutor(tx).MigrationGetAllAssumingMain(ctx) + migrations, err := migrator.driver.UnwrapExecutor(tx).MigrationGetAllAssumingMain(ctx, &riverdriver.MigrationGetAllAssumingMainParams{ + Schema: bundle.schema, + }) require.NoError(t, err) require.Equal(t, seqOneTo(migrationsBundle.MaxVersion-1), sliceutil.Map(migrations, driverMigrationToInt)) @@ -358,12 +372,14 @@ func TestMigrator(t *testing.T) { require.Equal(t, seqDownTo(migrationsBundle.WithTestVersionsMaxVersion, 5), sliceutil.Map(res.Versions, migrateVersionToInt)) - migrations, err := bundle.driver.GetExecutor().MigrationGetAllAssumingMain(ctx) + migrations, err := bundle.driver.GetExecutor().MigrationGetAllAssumingMain(ctx, &riverdriver.MigrationGetAllAssumingMainParams{ + Schema: bundle.schema, + }) require.NoError(t, err) require.Equal(t, seqOneTo(4), sliceutil.Map(migrations, driverMigrationToInt)) - err = dbExecError(ctx, bundle.driver.GetExecutor(), "SELECT name FROM test_table") + err = dbExecError(ctx, bundle.driver.GetExecutor(), fmt.Sprintf("SELECT name FROM %s.test_table", bundle.schema)) require.Error(t, err) }) @@ -380,7 +396,7 @@ func TestMigrator(t *testing.T) { require.Equal(t, seqDownTo(migrationsBundle.WithTestVersionsMaxVersion, 1), sliceutil.Map(res.Versions, migrateVersionToInt)) - err = dbExecError(ctx, bundle.driver.GetExecutor(), "SELECT name FROM river_migrate") + err = dbExecError(ctx, bundle.driver.GetExecutor(), fmt.Sprintf("SELECT name FROM %s.river_migrate", bundle.schema)) require.Error(t, err) }) @@ -417,7 +433,10 @@ func TestMigrator(t *testing.T) { // Migrate down returned a result above for a migration that was // removed, but because we're in a dry run, the database still shows // this version. - migrations, err := bundle.driver.GetExecutor().MigrationGetByLine(ctx, riverdriver.MigrationLineMain) + migrations, err := bundle.driver.GetExecutor().MigrationGetByLine(ctx, &riverdriver.MigrationGetByLineParams{ + Line: riverdriver.MigrationLineMain, + Schema: bundle.schema, + }) require.NoError(t, err) require.Equal(t, seqOneTo(migrationsBundle.WithTestVersionsMaxVersion), sliceutil.Map(migrations, driverMigrationToInt)) @@ -467,12 +486,15 @@ func TestMigrator(t *testing.T) { require.Equal(t, seqOneTo(migrationsBundle.WithTestVersionsMaxVersion), sliceutil.Map(res.Versions, migrateVersionToInt)) - migrations, err := bundle.driver.GetExecutor().MigrationGetByLine(ctx, riverdriver.MigrationLineMain) + migrations, err := bundle.driver.GetExecutor().MigrationGetByLine(ctx, &riverdriver.MigrationGetByLineParams{ + Line: riverdriver.MigrationLineMain, + Schema: bundle.schema, + }) require.NoError(t, err) require.Equal(t, seqOneTo(migrationsBundle.WithTestVersionsMaxVersion), sliceutil.Map(migrations, driverMigrationToInt)) - _, err = bundle.dbPool.Exec(ctx, "SELECT * FROM test_table") + _, err = bundle.dbPool.Exec(ctx, fmt.Sprintf("SELECT * FROM %s.test_table", bundle.schema)) require.NoError(t, err) } @@ -483,12 +505,15 @@ func TestMigrator(t *testing.T) { require.Equal(t, DirectionUp, res.Direction) require.Equal(t, []int{}, sliceutil.Map(res.Versions, migrateVersionToInt)) - migrations, err := bundle.driver.GetExecutor().MigrationGetByLine(ctx, riverdriver.MigrationLineMain) + migrations, err := bundle.driver.GetExecutor().MigrationGetByLine(ctx, &riverdriver.MigrationGetByLineParams{ + Line: riverdriver.MigrationLineMain, + Schema: bundle.schema, + }) require.NoError(t, err) require.Equal(t, seqOneTo(migrationsBundle.WithTestVersionsMaxVersion), sliceutil.Map(migrations, driverMigrationToInt)) - _, err = bundle.dbPool.Exec(ctx, "SELECT * FROM test_table") + _, err = bundle.dbPool.Exec(ctx, fmt.Sprintf("SELECT * FROM %s.test_table", bundle.schema)) require.NoError(t, err) } }) @@ -506,13 +531,16 @@ func TestMigrator(t *testing.T) { require.Equal(t, []int{migrationsBundle.WithTestVersionsMaxVersion - 1}, sliceutil.Map(res.Versions, migrateVersionToInt)) - migrations, err := bundle.driver.GetExecutor().MigrationGetByLine(ctx, riverdriver.MigrationLineMain) + migrations, err := bundle.driver.GetExecutor().MigrationGetByLine(ctx, &riverdriver.MigrationGetByLineParams{ + Line: riverdriver.MigrationLineMain, + Schema: bundle.schema, + }) require.NoError(t, err) require.Equal(t, seqOneTo(migrationsBundle.WithTestVersionsMaxVersion-1), sliceutil.Map(migrations, driverMigrationToInt)) // Column `name` is only added in the second test version. - err = dbExecError(ctx, bundle.driver.GetExecutor(), "SELECT name FROM test_table") + err = dbExecError(ctx, bundle.driver.GetExecutor(), fmt.Sprintf("SELECT name FROM %s.test_table", bundle.schema)) require.Error(t, err) var pgErr *pgconn.PgError @@ -529,7 +557,10 @@ func TestMigrator(t *testing.T) { require.NoError(t, err) require.Equal(t, seqOneTo(migrationsBundle.MaxVersion), sliceutil.Map(res.Versions, migrateVersionToInt)) - migrations, err := bundle.driver.GetExecutor().MigrationGetByLine(ctx, riverdriver.MigrationLineMain) + migrations, err := bundle.driver.GetExecutor().MigrationGetByLine(ctx, &riverdriver.MigrationGetByLineParams{ + Line: riverdriver.MigrationLineMain, + Schema: bundle.schema, + }) require.NoError(t, err) require.Equal(t, seqOneTo(migrationsBundle.MaxVersion), sliceutil.Map(migrations, driverMigrationToInt)) @@ -548,7 +579,10 @@ func TestMigrator(t *testing.T) { require.NoError(t, err) require.Equal(t, []int{migrationsBundle.MaxVersion + 1}, sliceutil.Map(res.Versions, migrateVersionToInt)) - migrations, err := migrator.driver.UnwrapExecutor(tx).MigrationGetByLine(ctx, riverdriver.MigrationLineMain) + migrations, err := migrator.driver.UnwrapExecutor(tx).MigrationGetByLine(ctx, &riverdriver.MigrationGetByLineParams{ + Line: riverdriver.MigrationLineMain, + Schema: bundle.schema, + }) require.NoError(t, err) require.Equal(t, seqOneTo(migrationsBundle.MaxVersion+1), sliceutil.Map(migrations, driverMigrationToInt)) @@ -564,7 +598,10 @@ func TestMigrator(t *testing.T) { require.Equal(t, seqOneTo(migrationsBundle.WithTestVersionsMaxVersion), sliceutil.Map(res.Versions, migrateVersionToInt)) - migrations, err := bundle.driver.GetExecutor().MigrationGetByLine(ctx, riverdriver.MigrationLineMain) + migrations, err := bundle.driver.GetExecutor().MigrationGetByLine(ctx, &riverdriver.MigrationGetByLineParams{ + Line: riverdriver.MigrationLineMain, + Schema: bundle.schema, + }) require.NoError(t, err) require.Equal(t, seqOneTo(migrationsBundle.MaxVersion+2), sliceutil.Map(migrations, driverMigrationToInt)) }) @@ -607,7 +644,10 @@ func TestMigrator(t *testing.T) { // Migrate up returned a result above for migrations that were applied, // but because we're in a dry run, the database still shows the test // migration versions not applied. - migrations, err := bundle.driver.GetExecutor().MigrationGetByLine(ctx, riverdriver.MigrationLineMain) + migrations, err := bundle.driver.GetExecutor().MigrationGetByLine(ctx, &riverdriver.MigrationGetByLineParams{ + Line: riverdriver.MigrationLineMain, + Schema: bundle.schema, + }) require.NoError(t, err) require.Equal(t, seqOneTo(migrationsBundle.MaxVersion), sliceutil.Map(migrations, driverMigrationToInt)) @@ -648,7 +688,10 @@ func TestMigrator(t *testing.T) { migrator, bundle := setup(t) requireMigrationTableExists := func(expectedExists bool) { - migrationExists, err := bundle.driver.GetExecutor().TableExists(ctx, "river_migration") + migrationExists, err := bundle.driver.GetExecutor().TableExists(ctx, &riverdriver.TableExistsParams{ + Table: "river_migration", + Schema: bundle.schema, + }) require.NoError(t, err) require.Equal(t, expectedExists, migrationExists) } @@ -675,7 +718,10 @@ func TestMigrator(t *testing.T) { require.Equal(t, seqOneTo(migrationsBundle.WithTestVersionsMaxVersion), sliceutil.Map(res.Versions, migrateVersionToInt)) - migrations, err := bundle.driver.GetExecutor().MigrationGetByLine(ctx, riverdriver.MigrationLineMain) + migrations, err := bundle.driver.GetExecutor().MigrationGetByLine(ctx, &riverdriver.MigrationGetByLineParams{ + Line: riverdriver.MigrationLineMain, + Schema: bundle.schema, + }) require.NoError(t, err) require.Equal(t, seqOneTo(migrationsBundle.WithTestVersionsMaxVersion), sliceutil.Map(migrations, driverMigrationToInt)) @@ -695,6 +741,7 @@ func TestMigrator(t *testing.T) { alternateMigrator, err := New(bundle.driver, &Config{ Line: migrationLineAlternate, Logger: bundle.logger, + schema: bundle.schema, }) require.NoError(t, err) @@ -703,7 +750,10 @@ func TestMigrator(t *testing.T) { require.Equal(t, seqOneTo(migrationLineAlternateMaxVersion), sliceutil.Map(res.Versions, migrateVersionToInt)) - migrations, err := bundle.driver.GetExecutor().MigrationGetByLine(ctx, migrationLineAlternate) + migrations, err := bundle.driver.GetExecutor().MigrationGetByLine(ctx, &riverdriver.MigrationGetByLineParams{ + Line: migrationLineAlternate, + Schema: bundle.schema, + }) require.NoError(t, err) require.Equal(t, seqOneTo(migrationLineAlternateMaxVersion), sliceutil.Map(migrations, driverMigrationToInt)) @@ -714,7 +764,10 @@ func TestMigrator(t *testing.T) { sliceutil.Map(res.Versions, migrateVersionToInt)) // The main migration line should not have been touched. - migrations, err = bundle.driver.GetExecutor().MigrationGetByLine(ctx, riverdriver.MigrationLineMain) + migrations, err = bundle.driver.GetExecutor().MigrationGetByLine(ctx, &riverdriver.MigrationGetByLineParams{ + Line: riverdriver.MigrationLineMain, + Schema: bundle.schema, + }) require.NoError(t, err) require.Equal(t, seqOneTo(migrationsBundle.MaxVersion), sliceutil.Map(migrations, driverMigrationToInt)) @@ -732,6 +785,7 @@ func TestMigrator(t *testing.T) { alternateMigrator, err := New(bundle.driver, &Config{ Line: migrationLineAlternate, Logger: bundle.logger, + schema: bundle.schema, }) require.NoError(t, err) @@ -765,19 +819,29 @@ func TestMigrator(t *testing.T) { require.Equal(t, seqOneTo(4), sliceutil.Map(res.Versions, migrateVersionToInt)) // Drop `river_migration` table as if version 001 had never originally run. - _, err = bundle.dbPool.Exec(ctx, "DROP TABLE river_migration") + _, err = bundle.dbPool.Exec(ctx, fmt.Sprintf("DROP TABLE %s.river_migration", bundle.schema)) require.NoError(t, err) - // Run version 005 to make sure it can tolerate the absence of - // `river_migration`. Note that we have to run the version's SQL - // directly because using the migrator will try to interact with - // `river_migration`, which is no longer present. - _, err = bundle.dbPool.Exec(ctx, migrationsBundle.WithTestVersionsMap[5].SQLUp) - require.NoError(t, err) + { + schemaReplacement := map[string]sqlctemplate.Replacement{ + "schema": {Value: bundle.schema + "."}, + } + + ctx := sqlctemplate.WithReplacements(ctx, schemaReplacement, nil) + + // Run version 005 to make sure it can tolerate the absence of + // `river_migration`. Note that we have to run the version's SQL + // directly because using the migrator will try to interact with + // `river_migration`, which is no longer present. + sql, _ := migrator.replacer.Run(ctx, migrationsBundle.WithTestVersionsMap[5].SQLUp, nil) + _, err = bundle.dbPool.Exec(ctx, sql) + require.NoError(t, err) - // And the version 005 down migration to verify the same. - _, err = bundle.dbPool.Exec(ctx, migrationsBundle.WithTestVersionsMap[5].SQLDown) - require.NoError(t, err) + // And the version 005 down migration to verify the same. + sql, _ = migrator.replacer.Run(ctx, migrationsBundle.WithTestVersionsMap[5].SQLDown, nil) + _, err = bundle.dbPool.Exec(ctx, sql) + require.NoError(t, err) + } }) t.Run("MigrationsWithCommitRequired", func(t *testing.T) { @@ -791,13 +855,13 @@ func TestMigrator(t *testing.T) { defer tx.Rollback(ctx) // Clean up the types we created. - _, err = tx.Exec(ctx, "DROP FUNCTION IF EXISTS foobar_in_bitmask") + _, err = tx.Exec(ctx, fmt.Sprintf("DROP FUNCTION IF EXISTS %s.foobar_in_bitmask", bundle.schema)) require.NoError(t, err) - _, err = tx.Exec(ctx, "DROP TYPE IF EXISTS foobar") + _, err = tx.Exec(ctx, fmt.Sprintf("DROP TYPE IF EXISTS %s.foobar", bundle.schema)) require.NoError(t, err) - _, err = tx.Exec(ctx, "DELETE FROM river_migration WHERE line = $1", migrationLineCommitRequired) + _, err = tx.Exec(ctx, fmt.Sprintf("DELETE FROM %s.river_migration WHERE line = $1", bundle.schema), migrationLineCommitRequired) require.NoError(t, err) require.NoError(t, tx.Commit(ctx)) @@ -811,6 +875,7 @@ func TestMigrator(t *testing.T) { commitRequiredMigrator, err := New(bundle.driver, &Config{ Line: migrationLineCommitRequired, Logger: bundle.logger, + schema: bundle.schema, }) require.NoError(t, err) @@ -888,13 +953,13 @@ func buildTestMigrationsBundle(t *testing.T) *testMigrationsBundle { testVersions := []Migration{ { Version: migrationsMaxVersion + 1, - SQLUp: "CREATE TABLE test_table(id bigserial PRIMARY KEY);", - SQLDown: "DROP TABLE test_table;", + SQLUp: "CREATE TABLE /* TEMPLATE: schema */test_table(id bigserial PRIMARY KEY);", + SQLDown: "DROP TABLE /* TEMPLATE: schema */test_table;", }, { Version: migrationsMaxVersion + 2, - SQLUp: "ALTER TABLE test_table ADD COLUMN name varchar(200); CREATE INDEX idx_test_table_name ON test_table(name);", - SQLDown: "DROP INDEX idx_test_table_name; ALTER TABLE test_table DROP COLUMN name;", + SQLUp: "ALTER TABLE /* TEMPLATE: schema */test_table ADD COLUMN name varchar(200); CREATE INDEX idx_test_table_name ON /* TEMPLATE: schema */test_table(name);", + SQLDown: "DROP INDEX /* TEMPLATE: schema */idx_test_table_name; ALTER TABLE /* TEMPLATE: schema */test_table DROP COLUMN name;", }, } diff --git a/rivershared/riverpilot/pilot.go b/rivershared/riverpilot/pilot.go index 65c5a7c4..598dcd6b 100644 --- a/rivershared/riverpilot/pilot.go +++ b/rivershared/riverpilot/pilot.go @@ -26,7 +26,7 @@ type Pilot interface { JobInsertMany( ctx context.Context, tx riverdriver.ExecutorTx, - params []*riverdriver.JobInsertFastParams, + params *riverdriver.JobInsertFastManyParams, ) ([]*riverdriver.JobInsertFastResult, error) JobSetStateIfRunningMany(ctx context.Context, tx riverdriver.ExecutorTx, params *riverdriver.JobSetStateIfRunningManyParams) ([]*rivertype.JobRow, error) @@ -54,4 +54,5 @@ type ProducerInitParams struct { ProducerID int64 Queue string QueueMetadata []byte + Schema string } diff --git a/rivershared/riverpilot/standard.go b/rivershared/riverpilot/standard.go index 5a928add..1ced1dce 100644 --- a/rivershared/riverpilot/standard.go +++ b/rivershared/riverpilot/standard.go @@ -23,7 +23,7 @@ func (p *StandardPilot) JobGetAvailable(ctx context.Context, exec riverdriver.Ex func (p *StandardPilot) JobInsertMany( ctx context.Context, tx riverdriver.ExecutorTx, - params []*riverdriver.JobInsertFastParams, + params *riverdriver.JobInsertFastManyParams, ) ([]*riverdriver.JobInsertFastResult, error) { return tx.JobInsertFastMany(ctx, params) } diff --git a/rivershared/testfactory/test_factory.go b/rivershared/testfactory/test_factory.go index 6aa080e1..c034f0bf 100644 --- a/rivershared/testfactory/test_factory.go +++ b/rivershared/testfactory/test_factory.go @@ -118,10 +118,11 @@ type MigrationOpts struct { func Migration(ctx context.Context, tb testing.TB, exec riverdriver.Executor, opts *MigrationOpts) *riverdriver.Migration { tb.Helper() - migration, err := exec.MigrationInsertMany(ctx, - ptrutil.ValOrDefault(opts.Line, riverdriver.MigrationLineMain), - []int{ptrutil.ValOrDefaultFunc(opts.Version, nextSeq)}, - ) + migration, err := exec.MigrationInsertMany(ctx, &riverdriver.MigrationInsertManyParams{ + Line: ptrutil.ValOrDefault(opts.Line, riverdriver.MigrationLineMain), + Schema: "", + Versions: []int{ptrutil.ValOrDefaultFunc(opts.Version, nextSeq)}, + }) require.NoError(tb, err) return migration[0] } diff --git a/rivertest/rivertest.go b/rivertest/rivertest.go index f3cd7c43..cad33459 100644 --- a/rivertest/rivertest.go +++ b/rivertest/rivertest.go @@ -146,7 +146,10 @@ func requireInsertedErr[TDriver riverdriver.Driver[TTx], TTx any, TArgs river.Jo t.Helper() // Returned ordered by ID. - jobRows, err := exec.JobGetByKindMany(ctx, []string{expectedJob.Kind()}) + jobRows, err := exec.JobGetByKindMany(ctx, &riverdriver.JobGetByKindManyParams{ + Kind: []string{expectedJob.Kind()}, + Schema: "", + }) if err != nil { return nil, fmt.Errorf("error querying jobs: %w", err) } @@ -242,7 +245,10 @@ func requireNotInsertedErr[TDriver riverdriver.Driver[TTx], TTx any, TArgs river t.Helper() // Returned ordered by ID. - jobRows, err := exec.JobGetByKindMany(ctx, []string{expectedJob.Kind()}) + jobRows, err := exec.JobGetByKindMany(ctx, &riverdriver.JobGetByKindManyParams{ + Kind: []string{expectedJob.Kind()}, + Schema: "", + }) if err != nil { return fmt.Errorf("error querying jobs: %w", err) } @@ -363,7 +369,10 @@ func requireManyInsertedErr[TDriver riverdriver.Driver[TTx], TTx any](ctx contex expectedArgsKinds := sliceutil.Map(expectedJobs, func(j ExpectedJob) string { return j.Args.Kind() }) // Returned ordered by ID. - jobRows, err := exec.JobGetByKindMany(ctx, expectedArgsKinds) + jobRows, err := exec.JobGetByKindMany(ctx, &riverdriver.JobGetByKindManyParams{ + Kind: expectedArgsKinds, + Schema: "", + }) if err != nil { return nil, fmt.Errorf("error querying jobs: %w", err) } diff --git a/rivertest/worker_test.go b/rivertest/worker_test.go index 3a28b6a6..2a3ff4fd 100644 --- a/rivertest/worker_test.go +++ b/rivertest/worker_test.go @@ -12,6 +12,7 @@ import ( "github.com/riverqueue/river" "github.com/riverqueue/river/internal/execution" "github.com/riverqueue/river/internal/riverinternaltest" + "github.com/riverqueue/river/riverdriver" "github.com/riverqueue/river/riverdriver/riverpgxv5" "github.com/riverqueue/river/rivershared/riversharedtest" "github.com/riverqueue/river/rivershared/testfactory" @@ -397,7 +398,7 @@ func TestWorker_WorkJob(t *testing.T) { require.NoError(t, err) bundle.workFunc = func(ctx context.Context, job *river.Job[testArgs]) error { - updatedJob, err := bundle.driver.UnwrapExecutor(bundle.tx).JobGetByID(ctx, insertRes.Job.ID) + updatedJob, err := bundle.driver.UnwrapExecutor(bundle.tx).JobGetByID(ctx, &riverdriver.JobGetByIDParams{ID: insertRes.Job.ID, Schema: ""}) require.NoError(t, err) require.Equal(t, rivertype.JobStateRunning, updatedJob.State) @@ -411,7 +412,7 @@ func TestWorker_WorkJob(t *testing.T) { require.NoError(t, err) require.Equal(t, river.EventKindJobCompleted, res.EventKind) - updatedJob, err := bundle.driver.UnwrapExecutor(bundle.tx).JobGetByID(ctx, insertRes.Job.ID) + updatedJob, err := bundle.driver.UnwrapExecutor(bundle.tx).JobGetByID(ctx, &riverdriver.JobGetByIDParams{ID: insertRes.Job.ID, Schema: ""}) require.NoError(t, err) require.Equal(t, rivertype.JobStateCompleted, updatedJob.State) })