diff --git a/pkg/ccl/backupccl/backup_job.go b/pkg/ccl/backupccl/backup_job.go index 67f9ff1baf91..6d54eec3991f 100644 --- a/pkg/ccl/backupccl/backup_job.go +++ b/pkg/ccl/backupccl/backup_job.go @@ -46,6 +46,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/execinfrapb" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/sessiondata" + "github.com/cockroachdb/cockroach/pkg/sql/sqlutil" "github.com/cockroachdb/cockroach/pkg/sql/stats" "github.com/cockroachdb/cockroach/pkg/util" "github.com/cockroachdb/cockroach/pkg/util/ctxgroup" @@ -370,13 +371,17 @@ func backup( } func releaseProtectedTimestamp( - ctx context.Context, txn *kv.Txn, pts protectedts.Storage, ptsID *uuid.UUID, + ctx context.Context, + txn *kv.Txn, + ie sqlutil.InternalExecutor, + pts protectedts.Storage, + ptsID *uuid.UUID, ) error { // If the job doesn't have a protected timestamp then there's nothing to do. if ptsID == nil { return nil } - err := pts.Release(ctx, txn, *ptsID) + err := pts.Release(ctx, txn, ie, *ptsID) if errors.Is(err, protectedts.ErrNotExists) { // No reason to return an error which might cause problems if it doesn't // seem to exist. @@ -495,10 +500,8 @@ func (b *backupResumer) Resume(ctx context.Context, execCtx interface{}) error { details.ProtectedTimestampRecord = &protectedtsID if details.ProtectedTimestampRecord != nil { - if err := p.ExecCfg().DB.Txn(ctx, func(ctx context.Context, txn *kv.Txn) error { - return protectTimestampForBackup( - ctx, p.ExecCfg(), txn, b.job.ID(), m, details, - ) + if err := p.ExecCfg().InternalExecutorFactory.TxnWithExecutor(ctx, p.ExecCfg().DB, nil /* sessionData */, func(ctx context.Context, txn *kv.Txn, ie sqlutil.InternalExecutor) error { + return protectTimestampForBackup(ctx, p.ExecCfg(), txn, ie, b.job.ID(), m, details) }); err != nil { return err } @@ -543,7 +546,7 @@ func (b *backupResumer) Resume(ctx context.Context, execCtx interface{}) error { // Update the job payload (non-volatile job definition) once, with the now // resolved destination, updated description, etc. If we resume again we'll // skip this whole block so this isn't an excessive update of payload. - if err := b.job.Update(ctx, nil, func(txn *kv.Txn, md jobs.JobMetadata, ju *jobs.JobUpdater) error { + if err := b.job.Update(ctx, nil /* txn */, nil /* ie */, func(_ *kv.Txn, _ sqlutil.InternalExecutor, md jobs.JobMetadata, ju *jobs.JobUpdater) error { if err := md.CheckRunningOrReverting(); err != nil { return err } @@ -689,9 +692,9 @@ func (b *backupResumer) Resume(ctx context.Context, execCtx interface{}) error { } if details.ProtectedTimestampRecord != nil && !b.testingKnobs.ignoreProtectedTimestamps { - if err := p.ExecCfg().DB.Txn(ctx, func(ctx context.Context, txn *kv.Txn) error { + if err := p.ExecCfg().InternalExecutorFactory.TxnWithExecutor(ctx, p.ExecCfg().DB, nil /* sessionData */, func(ctx context.Context, txn *kv.Txn, ie sqlutil.InternalExecutor) error { details := b.job.Details().(jobspb.BackupDetails) - return releaseProtectedTimestamp(ctx, txn, p.ExecCfg().ProtectedTimestampProvider, + return releaseProtectedTimestamp(ctx, txn, ie, p.ExecCfg().ProtectedTimestampProvider, details.ProtectedTimestampRecord) }); err != nil { log.Errorf(ctx, "failed to release protected timestamp: %v", err) @@ -1003,9 +1006,9 @@ func (b *backupResumer) OnFailOrCancel( p := execCtx.(sql.JobExecContext) cfg := p.ExecCfg() b.deleteCheckpoint(ctx, cfg, p.User()) - if err := cfg.DB.Txn(ctx, func(ctx context.Context, txn *kv.Txn) error { + if err := cfg.InternalExecutorFactory.TxnWithExecutor(ctx, cfg.DB, nil /* sessionData */, func(ctx context.Context, txn *kv.Txn, ie sqlutil.InternalExecutor) error { details := b.job.Details().(jobspb.BackupDetails) - return releaseProtectedTimestamp(ctx, txn, cfg.ProtectedTimestampProvider, + return releaseProtectedTimestamp(ctx, txn, ie, cfg.ProtectedTimestampProvider, details.ProtectedTimestampRecord) }); err != nil { return err diff --git a/pkg/ccl/backupccl/backup_planning.go b/pkg/ccl/backupccl/backup_planning.go index 6c0e1bba34c4..318760468722 100644 --- a/pkg/ccl/backupccl/backup_planning.go +++ b/pkg/ccl/backupccl/backup_planning.go @@ -48,6 +48,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/privilege" "github.com/cockroachdb/cockroach/pkg/sql/rowenc" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" + "github.com/cockroachdb/cockroach/pkg/sql/sqlutil" "github.com/cockroachdb/cockroach/pkg/sql/syntheticprivilege" "github.com/cockroachdb/cockroach/pkg/util/hlc" "github.com/cockroachdb/cockroach/pkg/util/interval" @@ -785,11 +786,14 @@ func backupPlanHook( if detached { // When running inside an explicit transaction, we simply create the job // record. We do not wait for the job to finish. - _, err := p.ExecCfg().JobRegistry.CreateAdoptableJobWithTxn( - ctx, jr, jobID, plannerTxn) - if err != nil { + if err := p.WithInternalExecutor(ctx, func(ctx context.Context, txn *kv.Txn, ie sqlutil.InternalExecutor) error { + _, err = p.ExecCfg().JobRegistry.CreateAdoptableJobWithTxn( + ctx, jr, jobID, plannerTxn, ie) + return err + }); err != nil { return err } + resultsCh <- tree.Datums{tree.NewDInt(tree.DInt(jobID))} return nil } @@ -920,7 +924,7 @@ func getScheduledBackupExecutionArgsFromSchedule( ctx context.Context, env scheduledjobs.JobSchedulerEnv, txn *kv.Txn, - ie *sql.InternalExecutor, + ie sqlutil.InternalExecutor, scheduleID int64, ) (*jobs.ScheduledJob, *backuppb.ScheduledBackupExecutionArgs, error) { // Load the schedule that has spawned this job. @@ -1164,6 +1168,7 @@ func protectTimestampForBackup( ctx context.Context, execCfg *sql.ExecutorConfig, txn *kv.Txn, + ie sqlutil.InternalExecutor, jobID jobspb.JobID, backupManifest backuppb.BackupManifest, backupDetails jobspb.BackupDetails, @@ -1184,7 +1189,7 @@ func protectTimestampForBackup( target.IgnoreIfExcludedFromBackup = true rec := jobsprotectedts.MakeRecord(*backupDetails.ProtectedTimestampRecord, int64(jobID), tsToProtect, backupManifest.Spans, jobsprotectedts.Jobs, target) - return execCfg.ProtectedTimestampProvider.Protect(ctx, txn, rec) + return execCfg.ProtectedTimestampProvider.Protect(ctx, txn, ie, rec) } // checkForNewDatabases returns an error if any new complete databases were diff --git a/pkg/ccl/backupccl/create_scheduled_backup.go b/pkg/ccl/backupccl/create_scheduled_backup.go index 066c4bd58565..add60ba1fe49 100644 --- a/pkg/ccl/backupccl/create_scheduled_backup.go +++ b/pkg/ccl/backupccl/create_scheduled_backup.go @@ -35,6 +35,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/sem/eval" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/sessiondata" + "github.com/cockroachdb/cockroach/pkg/sql/sqlutil" "github.com/cockroachdb/cockroach/pkg/sql/types" "github.com/cockroachdb/cockroach/pkg/util/log/eventpb" "github.com/cockroachdb/errors" @@ -640,7 +641,15 @@ func dryRunInvokeBackup( if err != nil { return eventpb.RecoveryEvent{}, err } - return invokeBackup(ctx, backupFn, p.ExecCfg().JobRegistry, p.Txn()) + var recoverEnv eventpb.RecoveryEvent + if err := p.WithInternalExecutor(ctx, func(ctx context.Context, txn *kv.Txn, ie sqlutil.InternalExecutor) error { + recoverEnv, err = invokeBackup(ctx, backupFn, p.ExecCfg().JobRegistry, p.Txn(), ie) + return err + }); err != nil { + return eventpb.RecoveryEvent{}, err + } + + return recoverEnv, nil } func fullyQualifyScheduledBackupTargetTables( diff --git a/pkg/ccl/backupccl/restore_job.go b/pkg/ccl/backupccl/restore_job.go index 58b7336b2539..0d9557174e04 100644 --- a/pkg/ccl/backupccl/restore_job.go +++ b/pkg/ccl/backupccl/restore_job.go @@ -963,8 +963,8 @@ func createImportingDescriptors( } if !details.PrepareCompleted { - err := sql.DescsTxn(ctx, p.ExecCfg(), func( - ctx context.Context, txn *kv.Txn, descsCol *descs.Collection, + err := p.ExecCfg().InternalExecutorFactory.DescsTxnWithExecutor(ctx, p.ExecCfg().DB, nil /* session data */, func( + ctx context.Context, txn *kv.Txn, descsCol *descs.Collection, ie sqlutil.InternalExecutor, ) error { // A couple of pieces of cleanup are required for multi-region databases. // First, we need to find all of the MULTIREGION_ENUMs types and remap the @@ -1228,7 +1228,7 @@ func createImportingDescriptors( } // Update the job once all descs have been prepared for ingestion. - err := r.job.SetDetails(ctx, txn, details) + err := r.job.SetDetails(ctx, txn, ie, details) // Emit to the event log now that the job has finished preparing descs. emitRestoreJobEvent(ctx, p, jobs.StatusRunning, r.job) @@ -1512,10 +1512,11 @@ func (r *restoreResumer) doResume(ctx context.Context, execCtx interface{}) erro // public. // TODO (lucy): Ideally we'd just create the database in the public state in // the first place, as a special case. - publishDescriptors := func(ctx context.Context, txn *kv.Txn, descsCol *descs.Collection, ie sqlutil.InternalExecutor) (err error) { + if err := p.ExecCfg().InternalExecutorFactory.DescsTxnWithExecutor(ctx, p.ExecCfg().DB, nil /* session data */, func( + ctx context.Context, txn *kv.Txn, descsCol *descs.Collection, ie sqlutil.InternalExecutor, + ) error { return r.publishDescriptors(ctx, txn, ie, p.ExecCfg(), p.User(), descsCol, details, nil) - } - if err := r.execCfg.InternalExecutorFactory.DescsTxnWithExecutor(ctx, r.execCfg.DB, nil /* sd */, publishDescriptors); err != nil { + }); err != nil { return err } p.ExecCfg().JobRegistry.NotifyToAdoptJobs() @@ -1571,7 +1572,7 @@ func (r *restoreResumer) doResume(ctx context.Context, execCtx interface{}) erro resTotal.Add(res) if details.DescriptorCoverage == tree.AllDescriptors { - if err := r.restoreSystemTables(ctx, p.ExecCfg().DB, preData.systemTables); err != nil { + if err := r.restoreSystemTables(ctx, p.ExecCfg().DB, p.ExecCfg().InternalExecutorFactory, preData.systemTables); err != nil { return err } // Reload the details as we may have updated the job. @@ -1632,7 +1633,7 @@ func (r *restoreResumer) doResume(ctx context.Context, execCtx interface{}) erro var devalidateIndexes map[descpb.ID][]descpb.IndexID if toValidate := len(details.RevalidateIndexes); toValidate > 0 { - if err := r.job.RunningStatus(ctx, nil /* txn */, func(_ context.Context, _ jobspb.Details) (jobs.RunningStatus, error) { + if err := r.job.RunningStatus(ctx, nil /* txn */, nil /* ie */, func(_ context.Context, _ jobspb.Details) (jobs.RunningStatus, error) { return jobs.RunningStatus(fmt.Sprintf("re-validating %d indexes", toValidate)), nil }); err != nil { return errors.Wrapf(err, "failed to update running status of job %d", errors.Safe(r.job.ID())) @@ -1644,10 +1645,11 @@ func (r *restoreResumer) doResume(ctx context.Context, execCtx interface{}) erro devalidateIndexes = bad } - publishDescriptors := func(ctx context.Context, txn *kv.Txn, descsCol *descs.Collection, ie sqlutil.InternalExecutor) (err error) { + if err := p.ExecCfg().InternalExecutorFactory.DescsTxnWithExecutor(ctx, p.ExecCfg().DB, nil /* session data */, func( + ctx context.Context, txn *kv.Txn, descsCol *descs.Collection, ie sqlutil.InternalExecutor, + ) error { return r.publishDescriptors(ctx, txn, ie, p.ExecCfg(), p.User(), descsCol, details, devalidateIndexes) - } - if err := r.execCfg.InternalExecutorFactory.DescsTxnWithExecutor(ctx, r.execCfg.DB, nil /* sd */, publishDescriptors); err != nil { + }); err != nil { return err } @@ -1670,7 +1672,7 @@ func (r *restoreResumer) doResume(ctx context.Context, execCtx interface{}) erro // includes the jobs that are being restored. As soon as we restore these // jobs, they become accessible to the user, and may start executing. We // need this to happen after the descriptors have been marked public. - if err := r.restoreSystemTables(ctx, p.ExecCfg().DB, mainData.systemTables); err != nil { + if err := r.restoreSystemTables(ctx, p.ExecCfg().DB, p.ExecCfg().InternalExecutorFactory, mainData.systemTables); err != nil { return err } // Reload the details as we may have updated the job. @@ -1803,6 +1805,7 @@ func revalidateIndexes( if err := sql.ValidateForwardIndexes( ctx, job, + execCfg.InternalExecutorFactory, tableDesc.MakePublic(), forward, runner, @@ -1823,6 +1826,7 @@ func revalidateIndexes( ctx, execCfg.Codec, job, + execCfg.InternalExecutorFactory, tableDesc.MakePublic(), inverted, runner, @@ -1922,7 +1926,9 @@ func insertStats( restoreStatsInsertBatchSize = len(latestStats) } - if err := execCfg.DB.Txn(ctx, func(ctx context.Context, txn *kv.Txn) error { + if err := execCfg.InternalExecutorFactory.TxnWithExecutor(ctx, execCfg.DB, nil /* session data */, func( + ctx context.Context, txn *kv.Txn, ie sqlutil.InternalExecutor, + ) error { if err := stats.InsertNewStats(ctx, execCfg.Settings, execCfg.InternalExecutor, txn, latestStats[:restoreStatsInsertBatchSize]); err != nil { return errors.Wrapf(err, "inserting stats from backup") @@ -1931,7 +1937,7 @@ func insertStats( // If this is the last batch, mark the stats insertion complete. if restoreStatsInsertBatchSize == len(latestStats) { details.StatsInserted = true - if err := job.SetDetails(ctx, txn, details); err != nil { + if err := job.SetDetails(ctx, txn, ie, details); err != nil { return errors.Wrapf(err, "updating job marking stats insertion complete") } } @@ -1995,9 +2001,7 @@ func (r *restoreResumer) publishDescriptors( // Go through the descriptors and find any declarative schema change jobs // affecting them. - if err := scbackup.CreateDeclarativeSchemaChangeJobs( - ctx, r.execCfg.JobRegistry, txn, ie, all, - ); err != nil { + if err := scbackup.CreateDeclarativeSchemaChangeJobs(ctx, r.execCfg.JobRegistry, txn, ie, all); err != nil { return err } @@ -2120,7 +2124,7 @@ func (r *restoreResumer) publishDescriptors( details.SchemaDescs = newSchemas details.DatabaseDescs = newDBs details.FunctionDescs = newFunctions - if err := r.job.SetDetails(ctx, txn, details); err != nil { + if err := r.job.SetDetails(ctx, txn, ie, details); err != nil { return errors.Wrap(err, "updating job details after publishing tables") } @@ -2795,7 +2799,10 @@ func (r *restoreResumer) restoreSystemUsers( // restoreSystemTables atomically replaces the contents of the system tables // with the data from the restored system tables. func (r *restoreResumer) restoreSystemTables( - ctx context.Context, db *kv.DB, tables []catalog.TableDescriptor, + ctx context.Context, + db *kv.DB, + ief sqlutil.InternalExecutorFactory, + tables []catalog.TableDescriptor, ) error { details := r.job.Details().(jobspb.RestoreDetails) if details.SystemTablesMigrated == nil { @@ -2834,7 +2841,7 @@ func (r *restoreResumer) restoreSystemTables( continue } - if err := db.Txn(ctx, func(ctx context.Context, txn *kv.Txn) error { + if err := ief.TxnWithExecutor(ctx, db, nil /* sessionData */, func(ctx context.Context, txn *kv.Txn, ie sqlutil.InternalExecutor) error { if err := systemTable.config.migrationFunc(ctx, r.execCfg, txn, systemTable.stagingTableName, details.DescriptorRewrites); err != nil { return err @@ -2844,7 +2851,7 @@ func (r *restoreResumer) restoreSystemTables( // restarts don't try to import data over our migrated data. This would // fail since the restored data would shadow the migrated keys. details.SystemTablesMigrated[systemTable.systemTableName] = true - return r.job.SetDetails(ctx, txn, details) + return r.job.SetDetails(ctx, txn, ie, details) }); err != nil { return err } diff --git a/pkg/ccl/backupccl/restore_planning.go b/pkg/ccl/backupccl/restore_planning.go index d195f915e85c..6f89258aecc9 100644 --- a/pkg/ccl/backupccl/restore_planning.go +++ b/pkg/ccl/backupccl/restore_planning.go @@ -1840,11 +1840,14 @@ func doRestorePlan( // When running in detached mode, we simply create the job record. // We do not wait for the job to finish. jobID := p.ExecCfg().JobRegistry.MakeJobID() - _, err := p.ExecCfg().JobRegistry.CreateAdoptableJobWithTxn( - ctx, jr, jobID, p.Txn()) - if err != nil { + if err := p.WithInternalExecutor(ctx, func(ctx context.Context, txn *kv.Txn, ie sqlutil.InternalExecutor) error { + _, err = p.ExecCfg().JobRegistry.CreateAdoptableJobWithTxn( + ctx, jr, jobID, p.Txn(), ie) + return err + }); err != nil { return err } + resultsCh <- tree.Datums{tree.NewDInt(tree.DInt(jobID))} collectRestoreTelemetry(ctx, jobID, restoreDetails, intoDB, newDBName, subdir, restoreStmt, descsByTablePattern, restoreDBs, asOfInterval, debugPauseOn, p.SessionData().ApplicationName) diff --git a/pkg/ccl/backupccl/schedule_exec.go b/pkg/ccl/backupccl/schedule_exec.go index 42f067ccf8f7..91f9abe73a5a 100644 --- a/pkg/ccl/backupccl/schedule_exec.go +++ b/pkg/ccl/backupccl/schedule_exec.go @@ -53,6 +53,7 @@ func (e *scheduledBackupExecutor) ExecuteJob( env scheduledjobs.JobSchedulerEnv, sj *jobs.ScheduledJob, txn *kv.Txn, + ie sqlutil.InternalExecutor, ) error { if err := e.executeBackup(ctx, cfg, sj, txn); err != nil { e.metrics.NumFailed.Inc(1) @@ -108,12 +109,16 @@ func (e *scheduledBackupExecutor) executeBackup( if err != nil { return err } - _, err = invokeBackup(ctx, backupFn, nil, nil) + _, err = invokeBackup(ctx, backupFn, nil /* registry */, nil /* txn */, nil /* ie */) return err } func invokeBackup( - ctx context.Context, backupFn sql.PlanHookRowFn, registry *jobs.Registry, txn *kv.Txn, + ctx context.Context, + backupFn sql.PlanHookRowFn, + registry *jobs.Registry, + txn *kv.Txn, + ie sqlutil.InternalExecutor, ) (eventpb.RecoveryEvent, error) { resultCh := make(chan tree.Datums) // No need to close g := ctxgroup.WithContext(ctx) @@ -122,7 +127,7 @@ func invokeBackup( g.GoCtx(func(ctx context.Context) error { select { case res := <-resultCh: - backupEvent = getBackupFnTelemetry(ctx, registry, txn, res) + backupEvent = getBackupFnTelemetry(ctx, registry, txn, ie, res) return nil case <-ctx.Done(): return ctx.Err() @@ -427,6 +432,7 @@ func unlinkOrDropDependentSchedule( scheduleControllerEnv scheduledjobs.ScheduleControllerEnv, env scheduledjobs.JobSchedulerEnv, txn *kv.Txn, + ie sqlutil.InternalExecutor, args *backuppb.ScheduledBackupExecutionArgs, ) (int, error) { if args.DependentScheduleID == 0 { @@ -455,7 +461,7 @@ func unlinkOrDropDependentSchedule( return 0, err } - return 1, releaseProtectedTimestamp(ctx, txn, scheduleControllerEnv.PTSProvider(), + return 1, releaseProtectedTimestamp(ctx, txn, ie, scheduleControllerEnv.PTSProvider(), dependentArgs.ProtectedTimestampRecord) } @@ -483,6 +489,7 @@ func (e *scheduledBackupExecutor) OnDrop( sj *jobs.ScheduledJob, txn *kv.Txn, descsCol *descs.Collection, + ie sqlutil.InternalExecutor, ) (int, error) { args := &backuppb.ScheduledBackupExecutionArgs{} @@ -490,19 +497,23 @@ func (e *scheduledBackupExecutor) OnDrop( return 0, errors.Wrap(err, "un-marshaling args") } - dependentRowsDropped, err := unlinkOrDropDependentSchedule(ctx, scheduleControllerEnv, env, txn, args) + dependentRowsDropped, err := unlinkOrDropDependentSchedule(ctx, scheduleControllerEnv, env, txn, ie, args) if err != nil { return dependentRowsDropped, errors.Wrap(err, "failed to unlink dependent schedule") } - return dependentRowsDropped, releaseProtectedTimestamp(ctx, txn, scheduleControllerEnv.PTSProvider(), + return dependentRowsDropped, releaseProtectedTimestamp(ctx, txn, ie, scheduleControllerEnv.PTSProvider(), args.ProtectedTimestampRecord) } // getBackupFnTelemetry collects the telemetry from the dry-run backup // corresponding to backupFnResult. func getBackupFnTelemetry( - ctx context.Context, registry *jobs.Registry, txn *kv.Txn, backupFnResult tree.Datums, + ctx context.Context, + registry *jobs.Registry, + txn *kv.Txn, + ie sqlutil.InternalExecutor, + backupFnResult tree.Datums, ) eventpb.RecoveryEvent { if registry == nil { return eventpb.RecoveryEvent{} @@ -523,7 +534,7 @@ func getBackupFnTelemetry( return jobspb.BackupDetails{}, errors.New("expected job ID as first column of result") } - job, err := registry.LoadJobWithTxn(ctx, jobspb.JobID(jobID), txn) + job, err := registry.LoadJobWithTxn(ctx, jobspb.JobID(jobID), txn, ie) if err != nil { return jobspb.BackupDetails{}, errors.Wrap(err, "failed to load dry-run backup job") } diff --git a/pkg/ccl/backupccl/schedule_pts_chaining.go b/pkg/ccl/backupccl/schedule_pts_chaining.go index 066cd9801ae1..c6930a93cadd 100644 --- a/pkg/ccl/backupccl/schedule_pts_chaining.go +++ b/pkg/ccl/backupccl/schedule_pts_chaining.go @@ -25,6 +25,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/sessiondata" + "github.com/cockroachdb/cockroach/pkg/sql/sqlutil" "github.com/cockroachdb/cockroach/pkg/util/hlc" "github.com/cockroachdb/cockroach/pkg/util/log" "github.com/cockroachdb/cockroach/pkg/util/uuid" @@ -95,10 +96,10 @@ func maybeUpdateSchedulePTSRecord( } } - return exec.DB.Txn(ctx, func(ctx context.Context, txn *kv.Txn) error { + return exec.InternalExecutorFactory.TxnWithExecutor(ctx, exec.DB, nil /* sessionData */, func(ctx context.Context, txn *kv.Txn, ie sqlutil.InternalExecutor) error { // We cannot rely on b.job containing created_by_id because on job // resumption the registry does not populate the resumers' CreatedByInfo. - datums, err := exec.InternalExecutor.QueryRowEx( + datums, err := ie.QueryRowEx( ctx, "lookup-schedule-info", txn, @@ -153,7 +154,7 @@ func maybeUpdateSchedulePTSRecord( } if err := manageIncrementalBackupPTSChaining(ctx, backupDetails.SchedulePTSChainingRecord.ProtectedTimestampRecord, - backupDetails.EndTime, exec, txn, scheduleID); err != nil { + backupDetails.EndTime, exec, txn, ie, scheduleID); err != nil { return errors.Wrap(err, "failed to manage chaining of pts record during a inc backup") } case backuppb.ScheduledBackupExecutionArgs_FULL: @@ -161,7 +162,7 @@ func maybeUpdateSchedulePTSRecord( return errors.AssertionFailedf("full backup has unexpected chaining action %d on"+ " backup job details", backupDetails.SchedulePTSChainingRecord.Action) } - if err := manageFullBackupPTSChaining(ctx, env, txn, backupDetails, exec, args); err != nil { + if err := manageFullBackupPTSChaining(ctx, env, txn, ie, backupDetails, exec, args); err != nil { return errors.Wrap(err, "failed to manage chaining of pts record during a full backup") } } @@ -175,6 +176,7 @@ func manageFullBackupPTSChaining( ctx context.Context, env scheduledjobs.JobSchedulerEnv, txn *kv.Txn, + ie sqlutil.InternalExecutor, backupDetails jobspb.BackupDetails, exec *sql.ExecutorConfig, fullScheduleArgs *backuppb.ScheduledBackupExecutionArgs, @@ -182,7 +184,7 @@ func manageFullBackupPTSChaining( // Let's resolve the dependent incremental schedule as the first step. If the // schedule has been dropped then we can avoid doing unnecessary work. incSj, incArgs, err := getScheduledBackupExecutionArgsFromSchedule(ctx, env, txn, - exec.InternalExecutor, fullScheduleArgs.DependentScheduleID) + ie, fullScheduleArgs.DependentScheduleID) if err != nil { if jobs.HasScheduledJobNotFoundError(err) { log.Warningf(ctx, "could not find dependent schedule with id %d", @@ -194,7 +196,7 @@ func manageFullBackupPTSChaining( // Resolve the target that needs to be protected on this execution of the // scheduled backup. - targetToProtect, deprecatedSpansToProtect, err := getTargetProtectedByBackup(ctx, backupDetails, txn, exec) + targetToProtect, deprecatedSpansToProtect, err := getTargetProtectedByBackup(ctx, backupDetails, txn, ie, exec) if err != nil { return errors.Wrap(err, "getting target to protect") } @@ -216,8 +218,7 @@ func manageFullBackupPTSChaining( // inc schedule ID as the records' Meta. This ensures that even if the full // schedule is dropped, the reconciliation job will not release the pts // record stored on the inc schedule, and the chaining will continue. - ptsRecord, err := protectTimestampRecordForSchedule(ctx, targetToProtect, deprecatedSpansToProtect, - backupDetails.EndTime, incSj.ScheduleID(), exec, txn) + ptsRecord, err := protectTimestampRecordForSchedule(ctx, targetToProtect, deprecatedSpansToProtect, backupDetails.EndTime, incSj.ScheduleID(), exec, txn, ie) if err != nil { return errors.Wrap(err, "protect pts record for schedule") } @@ -236,7 +237,7 @@ func manageFullBackupPTSChaining( // about to release. Already running incremental backup jobs would have // written their own pts record during planning, and should complete // successfully. - if err := releaseProtectedTimestamp(ctx, txn, exec.ProtectedTimestampProvider, + if err := releaseProtectedTimestamp(ctx, txn, ie, exec.ProtectedTimestampProvider, backupDetails.SchedulePTSChainingRecord.ProtectedTimestampRecord); err != nil { return errors.Wrap(err, "release pts record for schedule") } @@ -260,13 +261,13 @@ func manageIncrementalBackupPTSChaining( tsToProtect hlc.Timestamp, exec *sql.ExecutorConfig, txn *kv.Txn, + ie sqlutil.InternalExecutor, scheduleID int64, ) error { if ptsRecordID == nil { return errors.AssertionFailedf("unexpected nil pts record id on incremental schedule %d", scheduleID) } - err := exec.ProtectedTimestampProvider.UpdateTimestamp(ctx, txn, *ptsRecordID, - tsToProtect) + err := exec.ProtectedTimestampProvider.UpdateTimestamp(ctx, txn, ie, *ptsRecordID, tsToProtect) // If we cannot find the pts record to update it is possible that a concurrent // full backup has released the record, and written a new record on the // incremental schedule. This should only happen if this is an "overhang" @@ -283,14 +284,17 @@ func manageIncrementalBackupPTSChaining( } func getTargetProtectedByBackup( - ctx context.Context, backupDetails jobspb.BackupDetails, txn *kv.Txn, exec *sql.ExecutorConfig, + ctx context.Context, + backupDetails jobspb.BackupDetails, + txn *kv.Txn, + ie sqlutil.InternalExecutor, + exec *sql.ExecutorConfig, ) (target *ptpb.Target, deprecatedSpans []roachpb.Span, err error) { if backupDetails.ProtectedTimestampRecord == nil { return nil, nil, nil } - ptsRecord, err := exec.ProtectedTimestampProvider.GetRecord(ctx, txn, - *backupDetails.ProtectedTimestampRecord) + ptsRecord, err := exec.ProtectedTimestampProvider.GetRecord(ctx, txn, *backupDetails.ProtectedTimestampRecord, ie) if err != nil { return nil, nil, err } @@ -306,9 +310,10 @@ func protectTimestampRecordForSchedule( scheduleID int64, exec *sql.ExecutorConfig, txn *kv.Txn, + ie sqlutil.InternalExecutor, ) (uuid.UUID, error) { protectedtsID := uuid.MakeV4() rec := jobsprotectedts.MakeRecord(protectedtsID, scheduleID, tsToProtect, deprecatedSpansToProtect, jobsprotectedts.Schedules, targetToProtect) - return protectedtsID, exec.ProtectedTimestampProvider.Protect(ctx, txn, rec) + return protectedtsID, exec.ProtectedTimestampProvider.Protect(ctx, txn, ie, rec) } diff --git a/pkg/ccl/backupccl/schedule_pts_chaining_test.go b/pkg/ccl/backupccl/schedule_pts_chaining_test.go index 8dd7bbd83517..ab97c6ae6793 100644 --- a/pkg/ccl/backupccl/schedule_pts_chaining_test.go +++ b/pkg/ccl/backupccl/schedule_pts_chaining_test.go @@ -24,6 +24,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/kv/kvserver/protectedts/ptpb" "github.com/cockroachdb/cockroach/pkg/sql" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" + "github.com/cockroachdb/cockroach/pkg/sql/sqlutil" "github.com/cockroachdb/cockroach/pkg/testutils" "github.com/cockroachdb/cockroach/pkg/util/hlc" "github.com/cockroachdb/cockroach/pkg/util/leaktest" @@ -74,9 +75,10 @@ func checkPTSRecord( ) { var ptsRecord *ptpb.Record var err error - require.NoError(t, th.server.DB().Txn(ctx, func(ctx context.Context, txn *kv.Txn) error { + ief := th.server.InternalExecutorFactory().(sqlutil.InternalExecutorFactory) + require.NoError(t, ief.TxnWithExecutor(ctx, th.server.DB(), nil /* sessionData */, func(ctx context.Context, txn *kv.Txn, ie sqlutil.InternalExecutor) error { ptsRecord, err = th.server.ExecutorConfig().(sql.ExecutorConfig).ProtectedTimestampProvider. - GetRecord(context.Background(), txn, id) + GetRecord(context.Background(), txn, id, ie) require.NoError(t, err) return nil })) @@ -199,9 +201,9 @@ INSERT INTO t values (1), (10), (100); require.NotEqual(t, *ptsOnIncID, *incArgs.ProtectedTimestampRecord) // Check that the old pts record has been released. - require.NoError(t, th.cfg.DB.Txn(ctx, func(ctx context.Context, txn *kv.Txn) error { - _, err := th.server.ExecutorConfig().(sql.ExecutorConfig).ProtectedTimestampProvider.GetRecord( - ctx, txn, *ptsOnIncID) + ief := th.server.InternalExecutorFactory().(sqlutil.InternalExecutorFactory) + require.NoError(t, ief.TxnWithExecutor(ctx, th.server.DB(), nil /* sessionData */, func(ctx context.Context, txn *kv.Txn, ie sqlutil.InternalExecutor) error { + _, err := th.server.ExecutorConfig().(sql.ExecutorConfig).ProtectedTimestampProvider.GetRecord(ctx, txn, *ptsOnIncID, ie) require.True(t, errors.Is(err, protectedts.ErrNotExists)) return nil })) @@ -292,9 +294,9 @@ INSERT INTO t values (1), (10), (100); require.Error(t, err) // Check that the incremental schedule's PTS is dropped - require.NoError(t, th.cfg.DB.Txn(ctx, func(ctx context.Context, txn *kv.Txn) error { - _, err := th.server.ExecutorConfig().(sql.ExecutorConfig).ProtectedTimestampProvider.GetRecord( - ctx, txn, *ptsOnIncID) + ief := th.server.InternalExecutorFactory().(sqlutil.InternalExecutorFactory) + require.NoError(t, ief.TxnWithExecutor(ctx, th.server.DB(), nil /* sessionData */, func(ctx context.Context, txn *kv.Txn, ie sqlutil.InternalExecutor) error { + _, err := th.server.ExecutorConfig().(sql.ExecutorConfig).ProtectedTimestampProvider.GetRecord(ctx, txn, *ptsOnIncID, ie) require.True(t, errors.Is(err, protectedts.ErrNotExists)) return nil })) diff --git a/pkg/ccl/changefeedccl/BUILD.bazel b/pkg/ccl/changefeedccl/BUILD.bazel index ce5b5782aeb6..d17cdd81b2fd 100644 --- a/pkg/ccl/changefeedccl/BUILD.bazel +++ b/pkg/ccl/changefeedccl/BUILD.bazel @@ -246,6 +246,7 @@ go_test( "//pkg/sql/sessiondatapb", "//pkg/sql/sqlliveness", "//pkg/sql/sqlliveness/sqllivenesstestutils", + "//pkg/sql/sqlutil", "//pkg/sql/tests", "//pkg/sql/types", "//pkg/storage", diff --git a/pkg/ccl/changefeedccl/alter_changefeed_stmt.go b/pkg/ccl/changefeedccl/alter_changefeed_stmt.go index d3b9fc399f3c..d3f21ff37e4d 100644 --- a/pkg/ccl/changefeedccl/alter_changefeed_stmt.go +++ b/pkg/ccl/changefeedccl/alter_changefeed_stmt.go @@ -31,6 +31,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror" "github.com/cockroachdb/cockroach/pkg/sql/sem/eval" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" + "github.com/cockroachdb/cockroach/pkg/sql/sqlutil" "github.com/cockroachdb/cockroach/pkg/sql/types" "github.com/cockroachdb/cockroach/pkg/util/hlc" "github.com/cockroachdb/errors" @@ -94,8 +95,11 @@ func alterChangefeedPlanHook( } jobID := jobspb.JobID(tree.MustBeDInt(typedExpr)) - job, err := p.ExecCfg().JobRegistry.LoadJobWithTxn(ctx, jobID, p.Txn()) - if err != nil { + var job *jobs.Job + if err := p.WithInternalExecutor(ctx, func(ctx context.Context, txn *kv.Txn, ie sqlutil.InternalExecutor) error { + job, err = p.ExecCfg().JobRegistry.LoadJobWithTxn(ctx, jobID, p.Txn(), ie) + return err + }); err != nil { err = errors.Wrapf(err, `could not load job with job id %d`, jobID) return err } @@ -187,17 +191,17 @@ func alterChangefeedPlanHook( newPayload.Description = jobRecord.Description newPayload.DescriptorIDs = jobRecord.DescriptorIDs - err = p.ExecCfg().JobRegistry.UpdateJobWithTxn(ctx, jobID, p.Txn(), lockForUpdate, func( - txn *kv.Txn, md jobs.JobMetadata, ju *jobs.JobUpdater, - ) error { - ju.UpdatePayload(&newPayload) - if newProgress != nil { - ju.UpdateProgress(newProgress) - } - return nil - }) - - if err != nil { + if err := p.WithInternalExecutor(ctx, func(ctx context.Context, txn *kv.Txn, ie sqlutil.InternalExecutor) error { + return p.ExecCfg().JobRegistry.UpdateJobWithTxn(ctx, jobID, p.Txn(), ie, lockForUpdate, func( + _ *kv.Txn, _ sqlutil.InternalExecutor, md jobs.JobMetadata, ju *jobs.JobUpdater, + ) error { + ju.UpdatePayload(&newPayload) + if newProgress != nil { + ju.UpdateProgress(newProgress) + } + return nil + }) + }); err != nil { return err } diff --git a/pkg/ccl/changefeedccl/changefeed_processors.go b/pkg/ccl/changefeedccl/changefeed_processors.go index 101d1ecff4e9..890dbda33bfc 100644 --- a/pkg/ccl/changefeedccl/changefeed_processors.go +++ b/pkg/ccl/changefeedccl/changefeed_processors.go @@ -32,6 +32,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror" "github.com/cockroachdb/cockroach/pkg/sql/rowenc" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" + "github.com/cockroachdb/cockroach/pkg/sql/sqlutil" "github.com/cockroachdb/cockroach/pkg/util/hlc" "github.com/cockroachdb/cockroach/pkg/util/log" "github.com/cockroachdb/cockroach/pkg/util/log/logcrash" @@ -1224,8 +1225,8 @@ func (cf *changeFrontier) checkpointJobProgress( var updateSkipped error if cf.js.job != nil { - if err := cf.js.job.Update(cf.Ctx, nil, func( - txn *kv.Txn, md jobs.JobMetadata, ju *jobs.JobUpdater, + if err := cf.js.job.Update(cf.Ctx, nil /* txn */, nil /* ie */, func( + txn *kv.Txn, ie sqlutil.InternalExecutor, md jobs.JobMetadata, ju *jobs.JobUpdater, ) error { // If we're unable to update the job due to the job state, such as during // pause-requested, simply skip the checkpoint @@ -1249,7 +1250,7 @@ func (cf *changeFrontier) checkpointJobProgress( if !changefeedbase.ActiveProtectedTimestampsEnabled.Get(&cf.flowCtx.Cfg.Settings.SV) { timestampManager = cf.deprecatedManageProtectedTimestamps } - if err := timestampManager(cf.Ctx, txn, changefeedProgress); err != nil { + if err := timestampManager(cf.Ctx, txn, ie, changefeedProgress); err != nil { log.Warningf(cf.Ctx, "error managing protected timestamp record: %v", err) return err } @@ -1295,7 +1296,10 @@ func (cf *changeFrontier) checkpointJobProgress( // the changefeed's targets to the current highwater mark. The record is // cleared during changefeedResumer.OnFailOrCancel func (cf *changeFrontier) manageProtectedTimestamps( - ctx context.Context, txn *kv.Txn, progress *jobspb.ChangefeedProgress, + ctx context.Context, + txn *kv.Txn, + ie sqlutil.InternalExecutor, + progress *jobspb.ChangefeedProgress, ) error { ptsUpdateInterval := changefeedbase.ProtectTimestampInterval.Get(&cf.flowCtx.Cfg.Settings.SV) if timeutil.Since(cf.lastProtectedTimestampUpdate) < ptsUpdateInterval { @@ -1314,12 +1318,12 @@ func (cf *changeFrontier) manageProtectedTimestamps( recordID := progress.ProtectedTimestampRecord if recordID == uuid.Nil { ptr := createProtectedTimestampRecord(ctx, cf.flowCtx.Codec(), cf.spec.JobID, AllTargets(cf.spec.Feed), highWater, progress) - if err := pts.Protect(ctx, txn, ptr); err != nil { + if err := pts.Protect(ctx, txn, ie, ptr); err != nil { return err } } else { log.VEventf(ctx, 2, "updating protected timestamp %v at %v", recordID, highWater) - if err := pts.UpdateTimestamp(ctx, txn, recordID, highWater); err != nil { + if err := pts.UpdateTimestamp(ctx, txn, ie, recordID, highWater); err != nil { return err } } @@ -1332,10 +1336,13 @@ func (cf *changeFrontier) manageProtectedTimestamps( // sufficient degree after a backfill. This was deprecated in favor of always // maintaining a timestamp record to avoid issues with a low gcttl setting. func (cf *changeFrontier) deprecatedManageProtectedTimestamps( - ctx context.Context, txn *kv.Txn, progress *jobspb.ChangefeedProgress, + ctx context.Context, + txn *kv.Txn, + ie sqlutil.InternalExecutor, + progress *jobspb.ChangefeedProgress, ) error { pts := cf.flowCtx.Cfg.ProtectedTimestampProvider - if err := cf.deprecatedMaybeReleaseProtectedTimestamp(ctx, progress, pts, txn); err != nil { + if err := cf.deprecatedMaybeReleaseProtectedTimestamp(ctx, progress, pts, txn, ie); err != nil { return err } @@ -1344,13 +1351,17 @@ func (cf *changeFrontier) deprecatedManageProtectedTimestamps( if cf.frontier.schemaChangeBoundaryReached() && shouldProtectBoundaries { highWater := cf.frontier.Frontier() ptr := createProtectedTimestampRecord(ctx, cf.flowCtx.Codec(), cf.spec.JobID, AllTargets(cf.spec.Feed), highWater, progress) - return pts.Protect(ctx, txn, ptr) + return pts.Protect(ctx, txn, ie, ptr) } return nil } func (cf *changeFrontier) deprecatedMaybeReleaseProtectedTimestamp( - ctx context.Context, progress *jobspb.ChangefeedProgress, pts protectedts.Storage, txn *kv.Txn, + ctx context.Context, + progress *jobspb.ChangefeedProgress, + pts protectedts.Storage, + txn *kv.Txn, + ie sqlutil.InternalExecutor, ) error { if progress.ProtectedTimestampRecord == uuid.Nil { return nil @@ -1361,7 +1372,7 @@ func (cf *changeFrontier) deprecatedMaybeReleaseProtectedTimestamp( } log.VEventf(ctx, 2, "releasing protected timestamp %v", progress.ProtectedTimestampRecord) - if err := pts.Release(ctx, txn, progress.ProtectedTimestampRecord); err != nil { + if err := pts.Release(ctx, txn, ie, progress.ProtectedTimestampRecord); err != nil { return err } progress.ProtectedTimestampRecord = uuid.Nil diff --git a/pkg/ccl/changefeedccl/changefeed_stmt.go b/pkg/ccl/changefeedccl/changefeed_stmt.go index c7c8840a48b0..4f99150e4b95 100644 --- a/pkg/ccl/changefeedccl/changefeed_stmt.go +++ b/pkg/ccl/changefeedccl/changefeed_stmt.go @@ -46,6 +46,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/roleoption" "github.com/cockroachdb/cockroach/pkg/sql/sem/asof" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" + "github.com/cockroachdb/cockroach/pkg/sql/sqlutil" "github.com/cockroachdb/cockroach/pkg/sql/types" "github.com/cockroachdb/cockroach/pkg/util/hlc" "github.com/cockroachdb/cockroach/pkg/util/log" @@ -274,12 +275,12 @@ func changefeedPlanHook( jr.Progress = *progress.GetChangefeed() - if err := p.ExecCfg().DB.Txn(ctx, func(ctx context.Context, txn *kv.Txn) error { + if err := p.ExecCfg().InternalExecutorFactory.TxnWithExecutor(ctx, p.ExecCfg().DB, nil /* sessionData */, func(ctx context.Context, txn *kv.Txn, ie sqlutil.InternalExecutor) error { if err := p.ExecCfg().JobRegistry.CreateStartableJobWithTxn(ctx, &sj, jobID, txn, *jr); err != nil { return err } if ptr != nil { - return p.ExecCfg().ProtectedTimestampProvider.Protect(ctx, txn, ptr) + return p.ExecCfg().ProtectedTimestampProvider.Protect(ctx, txn, ie, ptr) } return nil }); err != nil { @@ -918,7 +919,7 @@ func (b *changefeedResumer) setJobRunningStatus( } status := jobs.RunningStatus(fmt.Sprintf(fmtOrMsg, args...)) - if err := b.job.RunningStatus(ctx, nil, + if err := b.job.RunningStatus(ctx, nil /* txn */, nil, /* ie */ func(_ context.Context, _ jobspb.Details) (jobs.RunningStatus, error) { return status, nil }, @@ -967,17 +968,20 @@ func (b *changefeedResumer) handleChangefeedError( const errorFmt = "job failed (%v) but is being paused because of %s=%s" errorMessage := fmt.Sprintf(errorFmt, changefeedErr, changefeedbase.OptOnError, changefeedbase.OptOnErrorPause) - return b.job.PauseRequested(ctx, jobExec.Txn(), func(ctx context.Context, - planHookState interface{}, txn *kv.Txn, progress *jobspb.Progress) error { - err := b.OnPauseRequest(ctx, jobExec, txn, progress) - if err != nil { - return err - } - // directly update running status to avoid the running/reverted job status check - progress.RunningStatus = errorMessage - log.Warningf(ctx, errorFmt, changefeedErr, changefeedbase.OptOnError, changefeedbase.OptOnErrorPause) - return nil - }, errorMessage) + return jobExec.WithInternalExecutor(ctx, func(ctx context.Context, txn *kv.Txn, ie sqlutil.InternalExecutor) error { + return b.job.PauseRequested(ctx, jobExec.Txn(), ie, func(ctx context.Context, + planHookState interface{}, newTxn *kv.Txn, executor sqlutil.InternalExecutor, progress *jobspb.Progress) error { + err := b.OnPauseRequest(ctx, jobExec, newTxn, executor, progress) + if err != nil { + return err + } + // directly update running status to avoid the running/reverted job status check + progress.RunningStatus = errorMessage + log.Warningf(ctx, errorFmt, changefeedErr, changefeedbase.OptOnError, changefeedbase.OptOnErrorPause) + return nil + }, errorMessage) + }) + default: return errors.Wrapf(changefeedErr, "unrecognized option value: %s=%s for handling error", changefeedbase.OptOnError, details.Opts[changefeedbase.OptOnError]) @@ -1059,7 +1063,7 @@ func (b *changefeedResumer) OnFailOrCancel( exec := jobExec.(sql.JobExecContext) execCfg := exec.ExecCfg() progress := b.job.Progress() - b.maybeCleanUpProtectedTimestamp(ctx, execCfg.DB, execCfg.ProtectedTimestampProvider, + b.maybeCleanUpProtectedTimestamp(ctx, execCfg.InternalExecutorFactory, execCfg.DB, execCfg.ProtectedTimestampProvider, progress.GetChangefeed().ProtectedTimestampRecord) // If this job has failed (not canceled), increment the counter. @@ -1077,13 +1081,17 @@ func (b *changefeedResumer) OnFailOrCancel( // Try to clean up a protected timestamp created by the changefeed. func (b *changefeedResumer) maybeCleanUpProtectedTimestamp( - ctx context.Context, db *kv.DB, pts protectedts.Storage, ptsID uuid.UUID, + ctx context.Context, + ief sqlutil.InternalExecutorFactory, + db *kv.DB, + pts protectedts.Storage, + ptsID uuid.UUID, ) { if ptsID == uuid.Nil { return } - if err := db.Txn(ctx, func(ctx context.Context, txn *kv.Txn) error { - return pts.Release(ctx, txn, ptsID) + if err := ief.TxnWithExecutor(ctx, db, nil /* sessionData */, func(ctx context.Context, txn *kv.Txn, ie sqlutil.InternalExecutor) error { + return pts.Release(ctx, txn, ie, ptsID) }); err != nil && !errors.Is(err, protectedts.ErrNotExists) { // NB: The record should get cleaned up by the reconciliation loop. // No good reason to cause more trouble by returning an error here. @@ -1097,7 +1105,11 @@ var _ jobs.PauseRequester = (*changefeedResumer)(nil) // OnPauseRequest implements jobs.PauseRequester. If this changefeed is being // paused, we may want to clear the protected timestamp record. func (b *changefeedResumer) OnPauseRequest( - ctx context.Context, jobExec interface{}, txn *kv.Txn, progress *jobspb.Progress, + ctx context.Context, + jobExec interface{}, + txn *kv.Txn, + ie sqlutil.InternalExecutor, + progress *jobspb.Progress, ) error { details := b.job.Details().(jobspb.ChangefeedDetails) @@ -1108,7 +1120,7 @@ func (b *changefeedResumer) OnPauseRequest( // Release existing pts record to avoid a single changefeed left on pause // resulting in storage issues if cp.ProtectedTimestampRecord != uuid.Nil { - if err := execCfg.ProtectedTimestampProvider.Release(ctx, txn, cp.ProtectedTimestampRecord); err != nil { + if err := execCfg.ProtectedTimestampProvider.Release(ctx, txn, ie, cp.ProtectedTimestampRecord); err != nil { log.Warningf(ctx, "failed to release protected timestamp %v: %v", cp.ProtectedTimestampRecord, err) } else { cp.ProtectedTimestampRecord = uuid.Nil @@ -1124,7 +1136,7 @@ func (b *changefeedResumer) OnPauseRequest( } pts := execCfg.ProtectedTimestampProvider ptr := createProtectedTimestampRecord(ctx, execCfg.Codec, b.job.ID(), AllTargets(details), *resolved, cp) - return pts.Protect(ctx, txn, ptr) + return pts.Protect(ctx, txn, ie, ptr) } return nil diff --git a/pkg/ccl/changefeedccl/changefeed_test.go b/pkg/ccl/changefeedccl/changefeed_test.go index e3454db4f56c..9d9ebf64ca0f 100644 --- a/pkg/ccl/changefeedccl/changefeed_test.go +++ b/pkg/ccl/changefeedccl/changefeed_test.go @@ -72,6 +72,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/sessiondatapb" "github.com/cockroachdb/cockroach/pkg/sql/sqlliveness" "github.com/cockroachdb/cockroach/pkg/sql/sqlliveness/sqllivenesstestutils" + "github.com/cockroachdb/cockroach/pkg/sql/sqlutil" "github.com/cockroachdb/cockroach/pkg/storage" "github.com/cockroachdb/cockroach/pkg/testutils" "github.com/cockroachdb/cockroach/pkg/testutils/serverutils" @@ -5058,8 +5059,9 @@ func TestChangefeedProtectedTimestampOnPause(t *testing.T) { if shouldPause { require.NotEqual(t, uuid.Nil, details.ProtectedTimestampRecord) var r *ptpb.Record - require.NoError(t, serverCfg.DB.Txn(ctx, func(ctx context.Context, txn *kv.Txn) (err error) { - r, err = pts.GetRecord(ctx, txn, details.ProtectedTimestampRecord) + ief := serverCfg.InternalExecutorFactory + require.NoError(t, ief.TxnWithExecutor(ctx, serverCfg.DB, nil /* sessionData */, func(ctx context.Context, txn *kv.Txn, ie sqlutil.InternalExecutor) error { + r, err = pts.GetRecord(ctx, txn, details.ProtectedTimestampRecord, ie) return err })) require.True(t, r.Timestamp.LessEq(*progress.GetHighWater())) @@ -5076,9 +5078,9 @@ func TestChangefeedProtectedTimestampOnPause(t *testing.T) { j, err := jr.LoadJob(ctx, feedJob.JobID()) require.NoError(t, err) details := j.Progress().Details.(*jobspb.Progress_Changefeed).Changefeed - - err = serverCfg.DB.Txn(ctx, func(ctx context.Context, txn *kv.Txn) (err error) { - r, err := pts.GetRecord(ctx, txn, details.ProtectedTimestampRecord) + ief := serverCfg.InternalExecutorFactory + err = ief.TxnWithExecutor(ctx, serverCfg.DB, nil /* sessionData */, func(ctx context.Context, txn *kv.Txn, ie sqlutil.InternalExecutor) error { + r, err := pts.GetRecord(ctx, txn, details.ProtectedTimestampRecord, ie) if err != nil || r.Timestamp.Less(resolvedTs) { return fmt.Errorf("expected protected timestamp record %v to have timestamp greater than %v", r, resolvedTs) } diff --git a/pkg/ccl/changefeedccl/testfeed_test.go b/pkg/ccl/changefeedccl/testfeed_test.go index d42f8be30c3d..2c488cae2346 100644 --- a/pkg/ccl/changefeedccl/testfeed_test.go +++ b/pkg/ccl/changefeedccl/testfeed_test.go @@ -43,6 +43,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/parser" "github.com/cockroachdb/cockroach/pkg/sql/sem/eval" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" + "github.com/cockroachdb/cockroach/pkg/sql/sqlutil" "github.com/cockroachdb/cockroach/pkg/sql/types" "github.com/cockroachdb/cockroach/pkg/testutils" "github.com/cockroachdb/cockroach/pkg/testutils/serverutils" @@ -259,9 +260,13 @@ func (r *reportErrorResumer) OnFailOrCancel( // OnPauseRequest implements PauseRequester interface. func (r *reportErrorResumer) OnPauseRequest( - ctx context.Context, execCtx interface{}, txn *kv.Txn, details *jobspb.Progress, + ctx context.Context, + execCtx interface{}, + txn *kv.Txn, + ie sqlutil.InternalExecutor, + details *jobspb.Progress, ) error { - return r.wrapped.(*changefeedResumer).OnPauseRequest(ctx, execCtx, txn, details) + return r.wrapped.(*changefeedResumer).OnPauseRequest(ctx, execCtx, txn, ie, details) } type wrapSinkFn func(sink Sink) Sink diff --git a/pkg/ccl/jobsccl/jobsprotectedtsccl/BUILD.bazel b/pkg/ccl/jobsccl/jobsprotectedtsccl/BUILD.bazel index d49b8ca79a5d..71b0b593e242 100644 --- a/pkg/ccl/jobsccl/jobsprotectedtsccl/BUILD.bazel +++ b/pkg/ccl/jobsccl/jobsprotectedtsccl/BUILD.bazel @@ -28,6 +28,7 @@ go_test( "//pkg/server", "//pkg/sql", "//pkg/sql/catalog/descpb", + "//pkg/sql/sqlutil", "//pkg/testutils", "//pkg/testutils/serverutils", "//pkg/testutils/skip", diff --git a/pkg/ccl/jobsccl/jobsprotectedtsccl/jobs_protected_ts_test.go b/pkg/ccl/jobsccl/jobsprotectedtsccl/jobs_protected_ts_test.go index 178feeaf4882..3939f5efe7b8 100644 --- a/pkg/ccl/jobsccl/jobsprotectedtsccl/jobs_protected_ts_test.go +++ b/pkg/ccl/jobsccl/jobsprotectedtsccl/jobs_protected_ts_test.go @@ -27,6 +27,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/security/username" "github.com/cockroachdb/cockroach/pkg/sql" "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" + "github.com/cockroachdb/cockroach/pkg/sql/sqlutil" "github.com/cockroachdb/cockroach/pkg/testutils" "github.com/cockroachdb/cockroach/pkg/testutils/serverutils" "github.com/cockroachdb/cockroach/pkg/testutils/skip" @@ -71,7 +72,7 @@ func testJobsProtectedTimestamp( mkJobAndRecord := func() (j *jobs.Job, rec *ptpb.Record) { ts := clock.Now() jobID := jr.MakeJobID() - require.NoError(t, execCfg.DB.Txn(ctx, func(ctx context.Context, txn *kv.Txn) (err error) { + require.NoError(t, execCfg.InternalExecutorFactory.TxnWithExecutor(ctx, execCfg.DB, nil /* sessionData */, func(ctx context.Context, txn *kv.Txn, ie sqlutil.InternalExecutor) (err error) { if j, err = jr.CreateJobWithTxn(ctx, mkJobRec(), jobID, txn); err != nil { return err } @@ -79,21 +80,21 @@ func testJobsProtectedTimestamp( targetToProtect := ptpb.MakeClusterTarget() rec = jobsprotectedts.MakeRecord(uuid.MakeV4(), int64(jobID), ts, deprecatedSpansToProtect, jobsprotectedts.Jobs, targetToProtect) - return ptp.Protect(ctx, txn, rec) + return ptp.Protect(ctx, txn, ie, rec) })) return j, rec } jMovedToFailed, recMovedToFailed := mkJobAndRecord() - require.NoError(t, execCfg.DB.Txn(ctx, func(ctx context.Context, txn *kv.Txn) error { - return jr.Failed(ctx, txn, jMovedToFailed.ID(), io.ErrUnexpectedEOF) + require.NoError(t, execCfg.InternalExecutorFactory.TxnWithExecutor(ctx, execCfg.DB, nil /* sessionData */, func(ctx context.Context, txn *kv.Txn, ie sqlutil.InternalExecutor) error { + return jr.Failed(ctx, txn, jMovedToFailed.ID(), ie, io.ErrUnexpectedEOF) })) jFinished, recFinished := mkJobAndRecord() - require.NoError(t, execCfg.DB.Txn(ctx, func(ctx context.Context, txn *kv.Txn) error { - return jr.Succeeded(ctx, txn, jFinished.ID()) + require.NoError(t, execCfg.InternalExecutorFactory.TxnWithExecutor(ctx, execCfg.DB, nil /* sessionData */, func(ctx context.Context, txn *kv.Txn, ie sqlutil.InternalExecutor) error { + return jr.Succeeded(ctx, txn, ie, jFinished.ID()) })) _, recRemains := mkJobAndRecord() - ensureNotExists := func(ctx context.Context, txn *kv.Txn, ptsID uuid.UUID) (err error) { - _, err = ptp.GetRecord(ctx, txn, ptsID) + ensureNotExists := func(ctx context.Context, txn *kv.Txn, ie sqlutil.InternalExecutor, ptsID uuid.UUID) (err error) { + _, err = ptp.GetRecord(ctx, txn, ptsID, ie) if err == nil { return errors.New("found pts record, waiting for ErrNotExists") } @@ -103,14 +104,15 @@ func testJobsProtectedTimestamp( return errors.Wrap(err, "waiting for ErrNotExists") } testutils.SucceedsSoon(t, func() (err error) { - return execCfg.DB.Txn(ctx, func(ctx context.Context, txn *kv.Txn) error { - if err := ensureNotExists(ctx, txn, recMovedToFailed.ID.GetUUID()); err != nil { + ief := execCfg.InternalExecutorFactory + return ief.TxnWithExecutor(ctx, execCfg.DB, nil /* sessionData */, func(ctx context.Context, txn *kv.Txn, ie sqlutil.InternalExecutor) error { + if err := ensureNotExists(ctx, txn, ie, recMovedToFailed.ID.GetUUID()); err != nil { return err } - if err := ensureNotExists(ctx, txn, recFinished.ID.GetUUID()); err != nil { + if err := ensureNotExists(ctx, txn, ie, recFinished.ID.GetUUID()); err != nil { return err } - _, err := ptp.GetRecord(ctx, txn, recRemains.ID.GetUUID()) + _, err := ptp.GetRecord(ctx, txn, recRemains.ID.GetUUID(), ie) require.NoError(t, err) return err }) @@ -203,14 +205,14 @@ func testSchedulesProtectedTimestamp( ts := clock.Now() var rec *ptpb.Record var sj *jobs.ScheduledJob - require.NoError(t, execCfg.DB.Txn(ctx, func(ctx context.Context, txn *kv.Txn) (err error) { + require.NoError(t, execCfg.InternalExecutorFactory.TxnWithExecutor(ctx, execCfg.DB, nil /* sessionData */, func(ctx context.Context, txn *kv.Txn, ie sqlutil.InternalExecutor) (err error) { sj = mkScheduledJobRec(scheduleLabel) require.NoError(t, sj.Create(ctx, execCfg.InternalExecutor, txn)) deprecatedSpansToProtect := roachpb.Spans{{Key: keys.MinKey, EndKey: keys.MaxKey}} targetToProtect := ptpb.MakeClusterTarget() rec = jobsprotectedts.MakeRecord(uuid.MakeV4(), sj.ScheduleID(), ts, deprecatedSpansToProtect, jobsprotectedts.Schedules, targetToProtect) - return ptp.Protect(ctx, txn, rec) + return ptp.Protect(ctx, txn, ie, rec) })) return sj, rec } @@ -219,8 +221,8 @@ func testSchedulesProtectedTimestamp( `DROP SCHEDULE $1`, sjDropped.ScheduleID()) require.NoError(t, err) _, recSchedule := mkScheduleAndRecord("do-not-drop") - ensureNotExists := func(ctx context.Context, txn *kv.Txn, ptsID uuid.UUID) (err error) { - _, err = ptp.GetRecord(ctx, txn, ptsID) + ensureNotExists := func(ctx context.Context, txn *kv.Txn, ie sqlutil.InternalExecutor, ptsID uuid.UUID) (err error) { + _, err = ptp.GetRecord(ctx, txn, ptsID, ie) if err == nil { return errors.New("found pts record, waiting for ErrNotExists") } @@ -230,11 +232,11 @@ func testSchedulesProtectedTimestamp( return errors.Wrap(err, "waiting for ErrNotExists") } testutils.SucceedsSoon(t, func() (err error) { - return execCfg.DB.Txn(ctx, func(ctx context.Context, txn *kv.Txn) error { - if err := ensureNotExists(ctx, txn, recScheduleDropped.ID.GetUUID()); err != nil { + return execCfg.InternalExecutorFactory.TxnWithExecutor(ctx, execCfg.DB, nil /* sessionData */, func(ctx context.Context, txn *kv.Txn, ie sqlutil.InternalExecutor) error { + if err := ensureNotExists(ctx, txn, ie, recScheduleDropped.ID.GetUUID()); err != nil { return err } - _, err := ptp.GetRecord(ctx, txn, recSchedule.ID.GetUUID()) + _, err := ptp.GetRecord(ctx, txn, recSchedule.ID.GetUUID(), ie) require.NoError(t, err) return err }) diff --git a/pkg/ccl/streamingccl/streamingest/BUILD.bazel b/pkg/ccl/streamingccl/streamingest/BUILD.bazel index 267220d942f0..0004cd3bb80d 100644 --- a/pkg/ccl/streamingccl/streamingest/BUILD.bazel +++ b/pkg/ccl/streamingccl/streamingest/BUILD.bazel @@ -44,6 +44,7 @@ go_library( "//pkg/sql/rowexec", "//pkg/sql/sem/eval", "//pkg/sql/sem/tree", + "//pkg/sql/sqlutil", "//pkg/sql/types", "//pkg/storage", "//pkg/storage/enginepb", @@ -109,6 +110,7 @@ go_test( "//pkg/sql/execinfrapb", "//pkg/sql/sem/eval", "//pkg/sql/sem/tree", + "//pkg/sql/sqlutil", "//pkg/storage", "//pkg/streaming", "//pkg/testutils", diff --git a/pkg/ccl/streamingccl/streamingest/stream_ingest_manager.go b/pkg/ccl/streamingccl/streamingest/stream_ingest_manager.go index 0f2e8b24ecd8..0d2a4c8bffab 100644 --- a/pkg/ccl/streamingccl/streamingest/stream_ingest_manager.go +++ b/pkg/ccl/streamingccl/streamingest/stream_ingest_manager.go @@ -19,6 +19,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgcode" "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror" "github.com/cockroachdb/cockroach/pkg/sql/sem/eval" + "github.com/cockroachdb/cockroach/pkg/sql/sqlutil" "github.com/cockroachdb/cockroach/pkg/streaming" "github.com/cockroachdb/cockroach/pkg/util/hlc" ) @@ -26,24 +27,25 @@ import ( type streamIngestManagerImpl struct { evalCtx *eval.Context txn *kv.Txn + ie sqlutil.InternalExecutor } // CompleteStreamIngestion implements streaming.StreamIngestManager interface. func (r *streamIngestManagerImpl) CompleteStreamIngestion( ctx context.Context, ingestionJobID jobspb.JobID, cutoverTimestamp hlc.Timestamp, ) error { - return completeStreamIngestion(ctx, r.evalCtx, r.txn, ingestionJobID, cutoverTimestamp) + return completeStreamIngestion(ctx, r.evalCtx, r.txn, r.ie, ingestionJobID, cutoverTimestamp) } // GetStreamIngestionStats implements streaming.StreamIngestManager interface. func (r *streamIngestManagerImpl) GetStreamIngestionStats( ctx context.Context, ingestionJobID jobspb.JobID, ) (*streampb.StreamIngestionStats, error) { - return getStreamIngestionStats(ctx, r.evalCtx, r.txn, ingestionJobID) + return getStreamIngestionStats(ctx, r.evalCtx, r.txn, r.ie, ingestionJobID) } func newStreamIngestManagerWithPrivilegesCheck( - ctx context.Context, evalCtx *eval.Context, txn *kv.Txn, + ctx context.Context, evalCtx *eval.Context, txn *kv.Txn, ie sqlutil.InternalExecutor, ) (eval.StreamIngestManager, error) { isAdmin, err := evalCtx.SessionAccessor.HasAdminRole(ctx) if err != nil { @@ -63,7 +65,7 @@ func newStreamIngestManagerWithPrivilegesCheck( pgcode.InsufficientPrivilege, "replication requires enterprise license") } - return &streamIngestManagerImpl{evalCtx: evalCtx, txn: txn}, nil + return &streamIngestManagerImpl{evalCtx: evalCtx, txn: txn, ie: ie}, nil } func init() { diff --git a/pkg/ccl/streamingccl/streamingest/stream_ingestion_frontier_processor.go b/pkg/ccl/streamingccl/streamingest/stream_ingestion_frontier_processor.go index 5c50713e0f73..f585b5b21e9f 100644 --- a/pkg/ccl/streamingccl/streamingest/stream_ingestion_frontier_processor.go +++ b/pkg/ccl/streamingccl/streamingest/stream_ingestion_frontier_processor.go @@ -26,6 +26,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/rowenc" "github.com/cockroachdb/cockroach/pkg/sql/rowexec" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" + "github.com/cockroachdb/cockroach/pkg/sql/sqlutil" "github.com/cockroachdb/cockroach/pkg/streaming" "github.com/cockroachdb/cockroach/pkg/util/ctxgroup" "github.com/cockroachdb/cockroach/pkg/util/hlc" @@ -413,8 +414,8 @@ func (sf *streamIngestionFrontier) maybeUpdatePartitionProgress() error { sf.lastPartitionUpdate = timeutil.Now() - if err := registry.UpdateJobWithTxn(ctx, jobID, nil, false, func( - txn *kv.Txn, md jobs.JobMetadata, ju *jobs.JobUpdater, + if err := registry.UpdateJobWithTxn(ctx, jobID, nil /* txn */, nil /* ie */, false, func( + _ *kv.Txn, _ sqlutil.InternalExecutor, md jobs.JobMetadata, ju *jobs.JobUpdater, ) error { if err := md.CheckRunningOrReverting(); err != nil { return err diff --git a/pkg/ccl/streamingccl/streamingest/stream_ingestion_job.go b/pkg/ccl/streamingccl/streamingest/stream_ingestion_job.go index f1f11ae74609..eb8184d32fcc 100644 --- a/pkg/ccl/streamingccl/streamingest/stream_ingestion_job.go +++ b/pkg/ccl/streamingccl/streamingest/stream_ingestion_job.go @@ -23,6 +23,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/settings/cluster" "github.com/cockroachdb/cockroach/pkg/sql" "github.com/cockroachdb/cockroach/pkg/sql/sem/eval" + "github.com/cockroachdb/cockroach/pkg/sql/sqlutil" "github.com/cockroachdb/cockroach/pkg/streaming" "github.com/cockroachdb/cockroach/pkg/util/hlc" "github.com/cockroachdb/cockroach/pkg/util/log" @@ -37,12 +38,13 @@ func completeStreamIngestion( ctx context.Context, evalCtx *eval.Context, txn *kv.Txn, + ie sqlutil.InternalExecutor, ingestionJobID jobspb.JobID, cutoverTimestamp hlc.Timestamp, ) error { jobRegistry := evalCtx.Planner.ExecutorConfig().(*sql.ExecutorConfig).JobRegistry - return jobRegistry.UpdateJobWithTxn(ctx, ingestionJobID, txn, false, /* useReadLock */ - func(txn *kv.Txn, md jobs.JobMetadata, ju *jobs.JobUpdater) error { + return jobRegistry.UpdateJobWithTxn(ctx, ingestionJobID, txn, ie, false, /* useReadLock */ + func(_ *kv.Txn, _ sqlutil.InternalExecutor, md jobs.JobMetadata, ju *jobs.JobUpdater) error { // TODO(adityamaru): This should change in the future, a user should be // allowed to correct their cutover time if the process of reverting the job // has not started. @@ -60,10 +62,14 @@ func completeStreamIngestion( } func getStreamIngestionStats( - ctx context.Context, evalCtx *eval.Context, txn *kv.Txn, ingestionJobID jobspb.JobID, + ctx context.Context, + evalCtx *eval.Context, + txn *kv.Txn, + ie sqlutil.InternalExecutor, + ingestionJobID jobspb.JobID, ) (*streampb.StreamIngestionStats, error) { registry := evalCtx.Planner.ExecutorConfig().(*sql.ExecutorConfig).JobRegistry - j, err := registry.LoadJobWithTxn(ctx, ingestionJobID, txn) + j, err := registry.LoadJobWithTxn(ctx, ingestionJobID, txn, ie) if err != nil { return nil, err } @@ -180,7 +186,7 @@ func waitUntilProducerActive( } func updateRunningStatus(ctx context.Context, ingestionJob *jobs.Job, status string) { - if err := ingestionJob.RunningStatus(ctx, nil, + if err := ingestionJob.RunningStatus(ctx, nil /* txn */, nil, /* ie */ func(ctx context.Context, details jobspb.Details) (jobs.RunningStatus, error) { return jobs.RunningStatus(status), nil }); err != nil { @@ -236,7 +242,7 @@ func ingest(ctx context.Context, execCtx sql.JobExecContext, ingestionJob *jobs. } // TODO(casper): update running status - err = ingestionJob.Update(ctx, nil, func(txn *kv.Txn, md jobs.JobMetadata, ju *jobs.JobUpdater) error { + err = ingestionJob.Update(ctx, nil /* txn */, nil /* ie */, func(_ *kv.Txn, _ sqlutil.InternalExecutor, md jobs.JobMetadata, ju *jobs.JobUpdater) error { if md.Progress.GetStreamIngest().StartTime.Less(startTime) { md.Progress.GetStreamIngest().StartTime = startTime } @@ -356,11 +362,16 @@ func (s *streamIngestionResumer) handleResumeError( // running until it times out. Users can still resume ingestion before // the producer job times out. jobExecCtx := execCtx.(sql.JobExecContext) - return s.job.PauseRequested(resumeCtx, jobExecCtx.Txn(), func(ctx context.Context, - planHookState interface{}, txn *kv.Txn, progress *jobspb.Progress) error { - progress.RunningStatus = errorMessage - return nil - }, errorMessage) + + return jobExecCtx.WithInternalExecutor(resumeCtx, func(ctx context.Context, txn *kv.Txn, ie sqlutil.InternalExecutor) error { + return s.job.PauseRequested(resumeCtx, jobExecCtx.Txn(), ie, func(_ context.Context, + _ interface{}, _ *kv.Txn, _ sqlutil.InternalExecutor, progress *jobspb.Progress) error { + progress.RunningStatus = errorMessage + return nil + }, errorMessage) + + }) + } // Resume is part of the jobs.Resumer interface. Ensure that any errors @@ -459,7 +470,7 @@ func maybeRevertToCutoverTimestamp( } } } - return true, j.SetProgress(ctx, nil /* txn */, *sp.StreamIngest) + return true, j.SetProgress(ctx, nil /* txn */, nil /* ie */, *sp.StreamIngest) } func activateTenant(ctx context.Context, execCtx interface{}, newTenantID roachpb.TenantID) error { diff --git a/pkg/ccl/streamingccl/streamingest/stream_ingestion_job_test.go b/pkg/ccl/streamingccl/streamingest/stream_ingestion_job_test.go index f0355ba9516c..cec4f4930e1e 100644 --- a/pkg/ccl/streamingccl/streamingest/stream_ingestion_job_test.go +++ b/pkg/ccl/streamingccl/streamingest/stream_ingestion_job_test.go @@ -29,6 +29,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/kv" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/security/username" + "github.com/cockroachdb/cockroach/pkg/sql/sqlutil" "github.com/cockroachdb/cockroach/pkg/testutils" "github.com/cockroachdb/cockroach/pkg/testutils/jobutils" "github.com/cockroachdb/cockroach/pkg/testutils/serverutils" @@ -277,7 +278,7 @@ func TestCutoverBuiltin(t *testing.T) { require.True(t, sp.StreamIngest.CutoverTime.IsEmpty()) var highWater time.Time - err = job.Update(ctx, nil, func(_ *kv.Txn, md jobs.JobMetadata, ju *jobs.JobUpdater) error { + err = job.Update(ctx, nil /* txn */, nil /* ie */, func(_ *kv.Txn, _ sqlutil.InternalExecutor, md jobs.JobMetadata, ju *jobs.JobUpdater) error { highWater = timeutil.Now().Round(time.Microsecond) hlcHighWater := hlc.Timestamp{WallTime: highWater.UnixNano()} return jobs.UpdateHighwaterProgressed(hlcHighWater, md, ju) diff --git a/pkg/ccl/streamingccl/streamingest/stream_ingestion_planning.go b/pkg/ccl/streamingccl/streamingest/stream_ingestion_planning.go index 7b32c2778978..74326d6eb180 100644 --- a/pkg/ccl/streamingccl/streamingest/stream_ingestion_planning.go +++ b/pkg/ccl/streamingccl/streamingest/stream_ingestion_planning.go @@ -17,6 +17,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/jobs" "github.com/cockroachdb/cockroach/pkg/jobs/jobspb" "github.com/cockroachdb/cockroach/pkg/keys" + "github.com/cockroachdb/cockroach/pkg/kv" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/settings/cluster" "github.com/cockroachdb/cockroach/pkg/sql" @@ -26,6 +27,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgcode" "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" + "github.com/cockroachdb/cockroach/pkg/sql/sqlutil" "github.com/cockroachdb/cockroach/pkg/sql/types" "github.com/cockroachdb/cockroach/pkg/util/tracing" "github.com/cockroachdb/errors" @@ -194,11 +196,17 @@ func ingestionPlanHook( } jobID := p.ExecCfg().JobRegistry.MakeJobID() - sj, err := p.ExecCfg().JobRegistry.CreateAdoptableJobWithTxn(ctx, jr, - jobID, p.Txn()) - if err != nil { + + var sj *jobs.Job + + if err := p.WithInternalExecutor(ctx, func(ctx context.Context, txn *kv.Txn, ie sqlutil.InternalExecutor) error { + sj, err = p.ExecCfg().JobRegistry.CreateAdoptableJobWithTxn(ctx, jr, + jobID, p.Txn(), ie) + return err + }); err != nil { return err } + resultsCh <- tree.Datums{tree.NewDInt(tree.DInt(sj.ID())), tree.NewDInt(tree.DInt(streamID))} return nil } diff --git a/pkg/ccl/streamingccl/streamproducer/BUILD.bazel b/pkg/ccl/streamingccl/streamproducer/BUILD.bazel index 0aa22e7c4ad9..2875c520e2fa 100644 --- a/pkg/ccl/streamingccl/streamproducer/BUILD.bazel +++ b/pkg/ccl/streamingccl/streamproducer/BUILD.bazel @@ -32,6 +32,7 @@ go_library( "//pkg/sql/pgwire/pgerror", "//pkg/sql/sem/eval", "//pkg/sql/sem/tree", + "//pkg/sql/sqlutil", "//pkg/sql/types", "//pkg/storage", "//pkg/streaming", @@ -87,6 +88,7 @@ go_test( "//pkg/sql/distsql", "//pkg/sql/sem/eval", "//pkg/sql/sessiondatapb", + "//pkg/sql/sqlutil", "//pkg/streaming", "//pkg/testutils", "//pkg/testutils/jobutils", diff --git a/pkg/ccl/streamingccl/streamproducer/producer_job.go b/pkg/ccl/streamingccl/streamproducer/producer_job.go index 04091c7ba735..bc15cb63e100 100644 --- a/pkg/ccl/streamingccl/streamproducer/producer_job.go +++ b/pkg/ccl/streamingccl/streamproducer/producer_job.go @@ -23,6 +23,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/security/username" "github.com/cockroachdb/cockroach/pkg/settings/cluster" "github.com/cockroachdb/cockroach/pkg/sql" + "github.com/cockroachdb/cockroach/pkg/sql/sqlutil" "github.com/cockroachdb/cockroach/pkg/util/timeutil" "github.com/cockroachdb/cockroach/pkg/util/uuid" "github.com/cockroachdb/errors" @@ -67,8 +68,8 @@ func (p *producerJobResumer) releaseProtectedTimestamp( ctx context.Context, executorConfig *sql.ExecutorConfig, ) error { ptr := p.job.Details().(jobspb.StreamReplicationDetails).ProtectedTimestampRecordID - return executorConfig.DB.Txn(ctx, func(ctx context.Context, txn *kv.Txn) error { - err := executorConfig.ProtectedTimestampProvider.Release(ctx, txn, ptr) + return executorConfig.InternalExecutorFactory.TxnWithExecutor(ctx, executorConfig.DB, nil /* sessionData */, func(ctx context.Context, txn *kv.Txn, ie sqlutil.InternalExecutor) error { + err := executorConfig.ProtectedTimestampProvider.Release(ctx, txn, ie, ptr) // In case that a retry happens, the record might have been released. if errors.Is(err, exec.ErrNotFound) { return nil @@ -101,7 +102,7 @@ func (p *producerJobResumer) Resume(ctx context.Context, execCtx interface{}) er case jobspb.StreamReplicationProgress_FINISHED_SUCCESSFULLY: return p.releaseProtectedTimestamp(ctx, execCfg) case jobspb.StreamReplicationProgress_FINISHED_UNSUCCESSFULLY: - return j.Update(ctx, nil, func(txn *kv.Txn, md jobs.JobMetadata, ju *jobs.JobUpdater) error { + return j.Update(ctx, nil /* txn */, nil /* ie */, func(_ *kv.Txn, _ sqlutil.InternalExecutor, md jobs.JobMetadata, ju *jobs.JobUpdater) error { ju.UpdateStatus(jobs.StatusCancelRequested) return nil }) diff --git a/pkg/ccl/streamingccl/streamproducer/producer_job_test.go b/pkg/ccl/streamingccl/streamproducer/producer_job_test.go index b1d1a0614087..dc4a85ee4ab5 100644 --- a/pkg/ccl/streamingccl/streamproducer/producer_job_test.go +++ b/pkg/ccl/streamingccl/streamproducer/producer_job_test.go @@ -25,6 +25,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/security/username" "github.com/cockroachdb/cockroach/pkg/settings/cluster" "github.com/cockroachdb/cockroach/pkg/sql/distsql" + "github.com/cockroachdb/cockroach/pkg/sql/sqlutil" "github.com/cockroachdb/cockroach/pkg/streaming" "github.com/cockroachdb/cockroach/pkg/testutils" "github.com/cockroachdb/cockroach/pkg/testutils/sqlutils" @@ -116,6 +117,7 @@ func TestStreamReplicationProducerJob(t *testing.T) { defer tc.Stopper().Stop(ctx) source := tc.Server(0) + ief := source.InternalExecutorFactory().(sqlutil.InternalExecutorFactory) sql := sqlutils.MakeSQLRunner(tc.ServerConn(0)) // Shorten the tracking frequency to make timer easy to be triggerred. sql.Exec(t, "SET CLUSTER SETTING stream_replication.stream_liveness_track_frequency = '1ms'") @@ -157,21 +159,22 @@ func TestStreamReplicationProducerJob(t *testing.T) { return record.Progress.(jobspb.StreamReplicationProgress).Expiration } runJobWithProtectedTimestamp := func(ptsID uuid.UUID, ts hlc.Timestamp, jr jobs.Record) error { - return source.DB().Txn(ctx, func(ctx context.Context, txn *kv.Txn) error { - deprecatedTenantSpan := roachpb.Spans{*makeTenantSpan(30)} - tenantTarget := ptpb.MakeTenantsTarget([]roachpb.TenantID{roachpb.MakeTenantID(30)}) - if err := ptp.Protect(ctx, txn, - jobsprotectedts.MakeRecord(ptsID, int64(jr.JobID), ts, + return source.InternalExecutorFactory().(sqlutil.InternalExecutorFactory).TxnWithExecutor( + ctx, source.DB(), nil, func(ctx context.Context, txn *kv.Txn, ie sqlutil.InternalExecutor) error { + deprecatedTenantSpan := roachpb.Spans{*makeTenantSpan(30)} + tenantTarget := ptpb.MakeTenantsTarget([]roachpb.TenantID{roachpb.MakeTenantID(30)}) + if err := ptp.Protect(ctx, txn, ie, jobsprotectedts.MakeRecord(ptsID, int64(jr.JobID), ts, deprecatedTenantSpan, jobsprotectedts.Jobs, tenantTarget)); err != nil { + return err + } + _, err := registry.CreateAdoptableJobWithTxn(ctx, jr, jr.JobID, txn, ie) return err - } - _, err := registry.CreateAdoptableJobWithTxn(ctx, jr, jr.JobID, txn) - return err - }) + }) } getPTSRecord := func(ptsID uuid.UUID) (r *ptpb.Record, err error) { - err = source.DB().Txn(ctx, func(ctx context.Context, txn *kv.Txn) error { - r, err = ptp.GetRecord(ctx, txn, ptsID) + ief := source.InternalExecutorFactory().(sqlutil.InternalExecutorFactory) + err = ief.TxnWithExecutor(ctx, source.DB(), nil /* sessionData */, func(ctx context.Context, txn *kv.Txn, ie sqlutil.InternalExecutor) error { + r, err = ptp.GetRecord(ctx, txn, ptsID, ie) return err }) return r, err @@ -200,10 +203,10 @@ func TestStreamReplicationProducerJob(t *testing.T) { require.True(t, testutils.IsError(err, "protected timestamp record does not exist"), err) var status streampb.StreamReplicationStatus - require.NoError(t, source.DB().Txn(ctx, func(ctx context.Context, txn *kv.Txn) error { + require.NoError(t, ief.TxnWithExecutor(ctx, source.DB(), nil /* sessionData */, func(ctx context.Context, txn *kv.Txn, ie sqlutil.InternalExecutor) error { status, err = updateReplicationStreamProgress( ctx, timeutil.Now(), ptp, registry, streaming.StreamID(jr.JobID), - hlc.Timestamp{WallTime: timeutil.Now().UnixNano()}, txn) + hlc.Timestamp{WallTime: timeutil.Now().UnixNano()}, txn, ie) return err })) require.Equal(t, streampb.StreamReplicationStatus_STREAM_INACTIVE, status.StreamStatus) @@ -236,9 +239,9 @@ func TestStreamReplicationProducerJob(t *testing.T) { var streamStatus streampb.StreamReplicationStatus var err error expire := expirationTime(jr).Add(10 * time.Millisecond) - require.NoError(t, source.DB().Txn(ctx, func(ctx context.Context, txn *kv.Txn) error { + require.NoError(t, ief.TxnWithExecutor(ctx, source.DB(), nil /* sessionData */, func(ctx context.Context, txn *kv.Txn, ie sqlutil.InternalExecutor) error { streamStatus, err = updateReplicationStreamProgress( - ctx, expire, ptp, registry, streaming.StreamID(jr.JobID), updatedFrontier, txn) + ctx, expire, ptp, registry, streaming.StreamID(jr.JobID), updatedFrontier, txn, ie) return err })) require.Equal(t, streampb.StreamReplicationStatus_STREAM_ACTIVE, streamStatus.StreamStatus) diff --git a/pkg/ccl/streamingccl/streamproducer/replication_manager.go b/pkg/ccl/streamingccl/streamproducer/replication_manager.go index 5ab6c379d5c8..8111a138bfc5 100644 --- a/pkg/ccl/streamingccl/streamproducer/replication_manager.go +++ b/pkg/ccl/streamingccl/streamproducer/replication_manager.go @@ -18,6 +18,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgcode" "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror" "github.com/cockroachdb/cockroach/pkg/sql/sem/eval" + "github.com/cockroachdb/cockroach/pkg/sql/sqlutil" "github.com/cockroachdb/cockroach/pkg/streaming" "github.com/cockroachdb/cockroach/pkg/util/hlc" ) @@ -25,20 +26,21 @@ import ( type replicationStreamManagerImpl struct { evalCtx *eval.Context txn *kv.Txn + ie sqlutil.InternalExecutor } // StartReplicationStream implements streaming.ReplicationStreamManager interface. func (r *replicationStreamManagerImpl) StartReplicationStream( ctx context.Context, tenantID uint64, ) (streampb.StreamID, error) { - return startReplicationStreamJob(ctx, r.evalCtx, r.txn, tenantID) + return startReplicationStreamJob(ctx, r.evalCtx, r.txn, r.ie, tenantID) } // HeartbeatReplicationStream implements streaming.ReplicationStreamManager interface. func (r *replicationStreamManagerImpl) HeartbeatReplicationStream( ctx context.Context, streamID streampb.StreamID, frontier hlc.Timestamp, ) (streampb.StreamReplicationStatus, error) { - return heartbeatReplicationStream(ctx, r.evalCtx, r.txn, streamID, frontier) + return heartbeatReplicationStream(ctx, r.evalCtx, r.txn, streamID, frontier, r.ie) } // StreamPartition implements streaming.ReplicationStreamManager interface. @@ -59,11 +61,11 @@ func (r *replicationStreamManagerImpl) GetReplicationStreamSpec( func (r *replicationStreamManagerImpl) CompleteReplicationStream( ctx context.Context, streamID streampb.StreamID, successfulIngestion bool, ) error { - return completeReplicationStream(ctx, r.evalCtx, r.txn, streamID, successfulIngestion) + return completeReplicationStream(ctx, r.evalCtx, r.txn, r.ie, streamID, successfulIngestion) } func newReplicationStreamManagerWithPrivilegesCheck( - ctx context.Context, evalCtx *eval.Context, txn *kv.Txn, + ctx context.Context, evalCtx *eval.Context, txn *kv.Txn, ie sqlutil.InternalExecutor, ) (eval.ReplicationStreamManager, error) { isAdmin, err := evalCtx.SessionAccessor.HasAdminRole(ctx) if err != nil { @@ -84,7 +86,7 @@ func newReplicationStreamManagerWithPrivilegesCheck( pgcode.InsufficientPrivilege, "replication requires enterprise license") } - return &replicationStreamManagerImpl{evalCtx: evalCtx, txn: txn}, nil + return &replicationStreamManagerImpl{evalCtx: evalCtx, txn: txn, ie: ie}, nil } func init() { diff --git a/pkg/ccl/streamingccl/streamproducer/replication_manager_test.go b/pkg/ccl/streamingccl/streamproducer/replication_manager_test.go index 2090ae5cdc1c..5051e90a11fe 100644 --- a/pkg/ccl/streamingccl/streamproducer/replication_manager_test.go +++ b/pkg/ccl/streamingccl/streamproducer/replication_manager_test.go @@ -15,10 +15,12 @@ import ( "github.com/cockroachdb/cockroach/pkg/base" "github.com/cockroachdb/cockroach/pkg/ccl/utilccl" + "github.com/cockroachdb/cockroach/pkg/kv" "github.com/cockroachdb/cockroach/pkg/security/username" "github.com/cockroachdb/cockroach/pkg/sql" "github.com/cockroachdb/cockroach/pkg/sql/sem/eval" "github.com/cockroachdb/cockroach/pkg/sql/sessiondatapb" + "github.com/cockroachdb/cockroach/pkg/sql/sqlutil" "github.com/cockroachdb/cockroach/pkg/testutils/serverutils" "github.com/cockroachdb/cockroach/pkg/testutils/sqlutils" "github.com/cockroachdb/cockroach/pkg/util/leaktest" @@ -51,7 +53,14 @@ func TestReplicationManagerRequiresAdminRole(t *testing.T) { p, cleanup := sql.NewInternalPlanner("test", txn, sqlUser, &sql.MemoryMetrics{}, &execCfg, sessionData) defer cleanup() ec := p.(interface{ EvalContext() *eval.Context }).EvalContext() - return newReplicationStreamManagerWithPrivilegesCheck(ctx, ec, txn) + var repManager eval.ReplicationStreamManager + if err := p.(sql.PlanHookState).WithInternalExecutor(ctx, func(ctx context.Context, txn *kv.Txn, ie sqlutil.InternalExecutor) error { + repManager, err = newReplicationStreamManagerWithPrivilegesCheck(ctx, ec, txn, ie) + return err + }); err != nil { + return nil, err + } + return repManager, nil } for _, tc := range []struct { diff --git a/pkg/ccl/streamingccl/streamproducer/replication_stream_test.go b/pkg/ccl/streamingccl/streamproducer/replication_stream_test.go index 1f22f14f2556..a43844030f04 100644 --- a/pkg/ccl/streamingccl/streamproducer/replication_stream_test.go +++ b/pkg/ccl/streamingccl/streamproducer/replication_stream_test.go @@ -33,6 +33,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/sql/catalog/desctestutils" "github.com/cockroachdb/cockroach/pkg/sql/distsql" + "github.com/cockroachdb/cockroach/pkg/sql/sqlutil" "github.com/cockroachdb/cockroach/pkg/testutils/jobutils" "github.com/cockroachdb/cockroach/pkg/testutils/serverutils" "github.com/cockroachdb/cockroach/pkg/testutils/skip" @@ -601,9 +602,10 @@ func TestCompleteStreamReplication(t *testing.T) { pj, err := jr.LoadJob(ctx, jobspb.JobID(streamID)) require.NoError(t, err) payload := pj.Payload() - require.ErrorIs(t, h.SysServer.DB().Txn(ctx, func(ctx context.Context, txn *kv.Txn) error { + ief := h.SysServer.InternalExecutorFactory().(sqlutil.InternalExecutorFactory) + require.ErrorIs(t, ief.TxnWithExecutor(ctx, h.SysServer.DB(), nil /* sessionData */, func(ctx context.Context, txn *kv.Txn, ie sqlutil.InternalExecutor) error { ptp := h.SysServer.DistSQLServer().(*distsql.ServerImpl).ServerConfig.ProtectedTimestampProvider - _, err = ptp.GetRecord(ctx, txn, payload.GetStreamReplication().ProtectedTimestampRecordID) + _, err = ptp.GetRecord(ctx, txn, payload.GetStreamReplication().ProtectedTimestampRecordID, ie) return err }), protectedts.ErrNotExists) } diff --git a/pkg/ccl/streamingccl/streamproducer/stream_lifetime.go b/pkg/ccl/streamingccl/streamproducer/stream_lifetime.go index 652de91ec09f..8d33bee5706d 100644 --- a/pkg/ccl/streamingccl/streamproducer/stream_lifetime.go +++ b/pkg/ccl/streamingccl/streamproducer/stream_lifetime.go @@ -23,6 +23,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/sql" "github.com/cockroachdb/cockroach/pkg/sql/sem/eval" + "github.com/cockroachdb/cockroach/pkg/sql/sqlutil" "github.com/cockroachdb/cockroach/pkg/testutils" "github.com/cockroachdb/cockroach/pkg/util/hlc" "github.com/cockroachdb/cockroach/pkg/util/timeutil" @@ -34,7 +35,11 @@ import ( // 1. Tracks the liveness of the replication stream consumption // 2. TODO(casper): Updates the protected timestamp for spans being replicated func startReplicationStreamJob( - ctx context.Context, evalCtx *eval.Context, txn *kv.Txn, tenantID uint64, + ctx context.Context, + evalCtx *eval.Context, + txn *kv.Txn, + ie sqlutil.InternalExecutor, + tenantID uint64, ) (streampb.StreamID, error) { execConfig := evalCtx.Planner.ExecutorConfig().(*sql.ExecutorConfig) hasAdminRole, err := evalCtx.SessionAccessor.HasAdminRole(ctx) @@ -51,7 +56,7 @@ func startReplicationStreamJob( timeout := streamingccl.StreamReplicationJobLivenessTimeout.Get(&evalCtx.Settings.SV) ptsID := uuid.MakeV4() jr := makeProducerJobRecord(registry, tenantID, timeout, evalCtx.SessionData().User(), ptsID) - if _, err := registry.CreateAdoptableJobWithTxn(ctx, jr, jr.JobID, txn); err != nil { + if _, err := registry.CreateAdoptableJobWithTxn(ctx, jr, jr.JobID, txn, ie); err != nil { return streampb.InvalidStreamID, err } @@ -66,7 +71,7 @@ func startReplicationStreamJob( pts := jobsprotectedts.MakeRecord(ptsID, int64(jr.JobID), statementTime, deprecatedSpansToProtect, jobsprotectedts.Jobs, targetToProtect) - if err := ptp.Protect(ctx, txn, pts); err != nil { + if err := ptp.Protect(ctx, txn, ie, pts); err != nil { return streampb.InvalidStreamID, err } return streampb.StreamID(jr.JobID), nil @@ -101,10 +106,11 @@ func updateReplicationStreamProgress( streamID streampb.StreamID, consumedTime hlc.Timestamp, txn *kv.Txn, + ie sqlutil.InternalExecutor, ) (status streampb.StreamReplicationStatus, err error) { const useReadLock = false - err = registry.UpdateJobWithTxn(ctx, jobspb.JobID(streamID), txn, useReadLock, - func(txn *kv.Txn, md jobs.JobMetadata, ju *jobs.JobUpdater) error { + err = registry.UpdateJobWithTxn(ctx, jobspb.JobID(streamID), txn, ie, useReadLock, + func(txn *kv.Txn, ie sqlutil.InternalExecutor, md jobs.JobMetadata, ju *jobs.JobUpdater) error { status.StreamStatus = convertProducerJobStatusToStreamStatus(md.Status) // Skip checking PTS record in cases that it might already be released if status.StreamStatus != streampb.StreamReplicationStatus_STREAM_ACTIVE && @@ -113,7 +119,7 @@ func updateReplicationStreamProgress( } ptsID := md.Payload.GetStreamReplication().ProtectedTimestampRecordID - ptsRecord, err := ptsProvider.GetRecord(ctx, txn, ptsID) + ptsRecord, err := ptsProvider.GetRecord(ctx, txn, ptsID, ie) if err != nil { return err } @@ -131,7 +137,7 @@ func updateReplicationStreamProgress( // ingestion using the previous ingestion high watermark, it can fall behind the // source cluster protected timestamp. if shouldUpdatePTS := ptsRecord.Timestamp.Less(consumedTime); shouldUpdatePTS { - if err = ptsProvider.UpdateTimestamp(ctx, txn, ptsID, consumedTime); err != nil { + if err = ptsProvider.UpdateTimestamp(ctx, txn, ie, ptsID, consumedTime); err != nil { return err } status.ProtectedTimestamp = &consumedTime @@ -159,6 +165,7 @@ func heartbeatReplicationStream( txn *kv.Txn, streamID streampb.StreamID, frontier hlc.Timestamp, + ie sqlutil.InternalExecutor, ) (streampb.StreamReplicationStatus, error) { execConfig := evalCtx.Planner.ExecutorConfig().(*sql.ExecutorConfig) timeout := streamingccl.StreamReplicationJobLivenessTimeout.Get(&evalCtx.Settings.SV) @@ -177,8 +184,7 @@ func heartbeatReplicationStream( } status.StreamStatus = convertProducerJobStatusToStreamStatus(pj.Status()) payload := pj.Payload() - ptsRecord, err := execConfig.ProtectedTimestampProvider.GetRecord(ctx, txn, - payload.GetStreamReplication().ProtectedTimestampRecordID) + ptsRecord, err := execConfig.ProtectedTimestampProvider.GetRecord(ctx, txn, payload.GetStreamReplication().ProtectedTimestampRecordID, ie) // Nil protected timestamp indicates it was not created or has been released. if errors.Is(err, protectedts.ErrNotExists) { return status, nil @@ -192,7 +198,7 @@ func heartbeatReplicationStream( return updateReplicationStreamProgress(ctx, expirationTime, execConfig.ProtectedTimestampProvider, execConfig.JobRegistry, - streamID, frontier, txn) + streamID, frontier, txn, ie) } // getReplicationStreamSpec gets a replication stream specification for the specified stream. @@ -256,13 +262,14 @@ func completeReplicationStream( ctx context.Context, evalCtx *eval.Context, txn *kv.Txn, + ie sqlutil.InternalExecutor, streamID streampb.StreamID, successfulIngestion bool, ) error { registry := evalCtx.Planner.ExecutorConfig().(*sql.ExecutorConfig).JobRegistry const useReadLock = false - return registry.UpdateJobWithTxn(ctx, jobspb.JobID(streamID), txn, useReadLock, - func(txn *kv.Txn, md jobs.JobMetadata, ju *jobs.JobUpdater) error { + return registry.UpdateJobWithTxn(ctx, jobspb.JobID(streamID), txn, ie, useReadLock, + func(_ *kv.Txn, _ sqlutil.InternalExecutor, md jobs.JobMetadata, ju *jobs.JobUpdater) error { // Updates the stream ingestion status, make the job resumer exit running // when picking up the new status. if (md.Status == jobs.StatusRunning || md.Status == jobs.StatusPending) && diff --git a/pkg/ccl/testccl/sqlccl/BUILD.bazel b/pkg/ccl/testccl/sqlccl/BUILD.bazel index 37167f76671f..30d506a6b36a 100644 --- a/pkg/ccl/testccl/sqlccl/BUILD.bazel +++ b/pkg/ccl/testccl/sqlccl/BUILD.bazel @@ -37,6 +37,7 @@ go_test( "//pkg/sql/sessiondatapb", "//pkg/sql/sqlliveness/slinstance", "//pkg/sql/sqltestutils", + "//pkg/sql/sqlutil", "//pkg/sql/tests", "//pkg/testutils", "//pkg/testutils/serverutils", diff --git a/pkg/ccl/testccl/sqlccl/tenant_gc_test.go b/pkg/ccl/testccl/sqlccl/tenant_gc_test.go index 86922eff04dd..c363c2caac60 100644 --- a/pkg/ccl/testccl/sqlccl/tenant_gc_test.go +++ b/pkg/ccl/testccl/sqlccl/tenant_gc_test.go @@ -28,6 +28,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" "github.com/cockroachdb/cockroach/pkg/sql/catalog/desctestutils" "github.com/cockroachdb/cockroach/pkg/sql/gcjob" + "github.com/cockroachdb/cockroach/pkg/sql/sqlutil" "github.com/cockroachdb/cockroach/pkg/testutils" "github.com/cockroachdb/cockroach/pkg/testutils/serverutils" "github.com/cockroachdb/cockroach/pkg/testutils/skip" @@ -474,8 +475,8 @@ func TestGCTenantJobWaitsForProtectedTimestamps(t *testing.T) { recordID := uuid.MakeV4() rec := jobsprotectedts.MakeRecord(recordID, int64(1), ts, nil, /* deprecatedSpans */ jobsprotectedts.Jobs, target) - require.NoError(t, execCfg.DB.Txn(ctx, func(ctx context.Context, txn *kv.Txn) error { - return ptp.Protect(ctx, txn, rec) + require.NoError(t, execCfg.InternalExecutorFactory.TxnWithExecutor(ctx, execCfg.DB, nil /* sessionData */, func(ctx context.Context, txn *kv.Txn, ie sqlutil.InternalExecutor) (err error) { + return ptp.Protect(ctx, txn, ie, rec) })) return rec } @@ -540,8 +541,8 @@ func TestGCTenantJobWaitsForProtectedTimestamps(t *testing.T) { checkGCBlockedByPTS(t, sj, tenID) // Release the record. - require.NoError(t, execCfg.DB.Txn(ctx, func(ctx context.Context, txn *kv.Txn) error { - require.NoError(t, ptp.Release(ctx, txn, rec.ID.GetUUID())) + require.NoError(t, execCfg.InternalExecutorFactory.TxnWithExecutor(ctx, execCfg.DB, nil /* sessionData */, func(ctx context.Context, txn *kv.Txn, ie sqlutil.InternalExecutor) error { + require.NoError(t, ptp.Release(ctx, txn, ie, rec.ID.GetUUID())) return nil })) @@ -575,9 +576,9 @@ func TestGCTenantJobWaitsForProtectedTimestamps(t *testing.T) { checkTenantGCed(t, sj, roachpb.MakeTenantID(tenID)) // Cleanup. - require.NoError(t, execCfg.DB.Txn(ctx, func(ctx context.Context, txn *kv.Txn) error { - require.NoError(t, ptp.Release(ctx, txn, clusterRec.ID.GetUUID())) - require.NoError(t, ptp.Release(ctx, txn, tenantRec.ID.GetUUID())) + require.NoError(t, execCfg.InternalExecutorFactory.TxnWithExecutor(ctx, execCfg.DB, nil /* sessionData */, func(ctx context.Context, txn *kv.Txn, ie sqlutil.InternalExecutor) error { + require.NoError(t, ptp.Release(ctx, txn, ie, clusterRec.ID.GetUUID())) + require.NoError(t, ptp.Release(ctx, txn, ie, tenantRec.ID.GetUUID())) return nil })) }) @@ -597,8 +598,8 @@ func TestGCTenantJobWaitsForProtectedTimestamps(t *testing.T) { rec := jobsprotectedts.MakeRecord(recordID, int64(1), hlc.Timestamp{WallTime: timeutil.Now().UnixNano()}, nil, /* deprecatedSpans */ jobsprotectedts.Jobs, clusterTarget) - require.NoError(t, execCfg.DB.Txn(ctx, func(ctx context.Context, txn *kv.Txn) error { - return tenPtp.Protect(ctx, txn, rec) + require.NoError(t, ten.ExecutorConfig().(sql.ExecutorConfig).InternalExecutorFactory.TxnWithExecutor(ctx, execCfg.DB, nil /* sessionData */, func(ctx context.Context, txn *kv.Txn, ie sqlutil.InternalExecutor) (err error) { + return tenPtp.Protect(ctx, txn, ie, rec) })) sqlDB.Exec(t, fmt.Sprintf(`SELECT crdb_internal.destroy_tenant(%d)`, @@ -634,8 +635,8 @@ func TestGCTenantJobWaitsForProtectedTimestamps(t *testing.T) { checkGCBlockedByPTS(t, sj, tenID) // Release the record. - require.NoError(t, execCfg.DB.Txn(ctx, func(ctx context.Context, txn *kv.Txn) error { - require.NoError(t, ptp.Release(ctx, txn, rec.ID.GetUUID())) + require.NoError(t, execCfg.InternalExecutorFactory.TxnWithExecutor(ctx, execCfg.DB, nil /* sessionData */, func(ctx context.Context, txn *kv.Txn, ie sqlutil.InternalExecutor) error { + require.NoError(t, ptp.Release(ctx, txn, ie, rec.ID.GetUUID())) return nil })) diff --git a/pkg/jobs/adopt.go b/pkg/jobs/adopt.go index 24557ddcada4..2360b64ee3a6 100644 --- a/pkg/jobs/adopt.go +++ b/pkg/jobs/adopt.go @@ -24,6 +24,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/sessiondata" "github.com/cockroachdb/cockroach/pkg/sql/sqlliveness" + "github.com/cockroachdb/cockroach/pkg/sql/sqlutil" "github.com/cockroachdb/cockroach/pkg/util/log" "github.com/cockroachdb/cockroach/pkg/util/tracing" "github.com/cockroachdb/cockroach/pkg/util/tracing/tracingpb" @@ -93,13 +94,13 @@ func (r *Registry) maybeDumpTrace( // claimJobs places a claim with the given SessionID to job rows that are // available. func (r *Registry) claimJobs(ctx context.Context, s sqlliveness.Session) error { - return r.db.Txn(ctx, func(ctx context.Context, txn *kv.Txn) error { + return r.internalExecutorFactory.TxnWithExecutor(ctx, r.db, nil /* sessionData */, func(ctx context.Context, txn *kv.Txn, ie sqlutil.InternalExecutor) error { // Run the claim transaction at low priority to ensure that it does not // contend with foreground reads. if err := txn.SetUserPriority(roachpb.MinUserPriority); err != nil { return errors.WithAssertionFailure(err) } - numRows, err := r.ex.Exec( + numRows, err := ie.Exec( ctx, "claim-jobs", txn, claimQuery, s.ID().UnsafeBytes(), r.ID(), maxAdoptionsPerLoop) if err != nil { @@ -164,11 +165,13 @@ func getProcessQuery( } // processClaimedJobs processes all jobs currently claimed by the registry. -func (r *Registry) processClaimedJobs(ctx context.Context, s sqlliveness.Session) error { +func (r *Registry) processClaimedJobs( + ctx context.Context, s sqlliveness.Session, ieNotBoundToTxn sqlutil.InternalExecutor, +) error { query, args := getProcessQuery(ctx, s, r) - it, err := r.ex.QueryIteratorEx( - ctx, "select-running/get-claimed-jobs", nil, + it, err := ieNotBoundToTxn.QueryIteratorEx( + ctx, "select-running/get-claimed-jobs", nil, /* txn */ sessiondata.InternalExecutorOverride{User: username.NodeUserName()}, query, args..., ) if err != nil { @@ -246,8 +249,9 @@ func (r *Registry) resumeJob(ctx context.Context, jobID jobspb.JobID, s sqlliven resumeQuery := resumeQueryWithBackoff args := []interface{}{jobID, s.ID().UnsafeBytes(), r.clock.Now().GoTime(), r.RetryInitialDelay(), r.RetryMaxDelay()} - row, err := r.ex.QueryRowEx( - ctx, "get-job-row", nil, + ieNotBoundToTxn := r.internalExecutorFactory.MakeInternalExecutorWithoutTxn() + row, err := ieNotBoundToTxn.QueryRowEx( + ctx, "get-job-row", nil, /* txn */ sessiondata.InternalExecutorOverride{User: username.NodeUserName()}, resumeQuery, args..., ) if err != nil { @@ -397,7 +401,8 @@ func (r *Registry) runJob( span.SetTag("job-id", attribute.Int64Value(int64(job.ID()))) defer span.Finish() if span.TraceID() != 0 { - if err := job.Update(ctx, nil /* txn */, func(txn *kv.Txn, md JobMetadata, + if err := job.Update(ctx, nil /* txn */, nil /* ie */, func( + _ *kv.Txn, _ sqlutil.InternalExecutor, md JobMetadata, ju *JobUpdater) error { progress := *md.Progress progress.TraceID = span.TraceID() @@ -474,7 +479,7 @@ RETURNING id, status ` func (r *Registry) servePauseAndCancelRequests(ctx context.Context, s sqlliveness.Session) error { - return r.db.Txn(ctx, func(ctx context.Context, txn *kv.Txn) error { + return r.internalExecutorFactory.TxnWithExecutor(ctx, r.db, nil /* sessionData */, func(ctx context.Context, txn *kv.Txn, ie sqlutil.InternalExecutor) error { // Run the claim transaction at low priority to ensure that it does not // contend with foreground reads. if err := txn.SetUserPriority(roachpb.MinUserPriority); err != nil { @@ -484,7 +489,7 @@ func (r *Registry) servePauseAndCancelRequests(ctx context.Context, s sqllivenes // job - because we have to make sure that the query executes without an // error (otherwise, the system.jobs table might diverge from the jobs // registry). - rows, err := r.ex.QueryBufferedEx( + rows, err := ie.QueryBufferedEx( ctx, "cancel/pause-requested", txn, sessiondata.InternalExecutorOverride{User: username.NodeUserName()}, pauseAndCancelUpdate, s.ID().UnsafeBytes(), r.ID(), ) @@ -500,7 +505,7 @@ func (r *Registry) servePauseAndCancelRequests(ctx context.Context, s sqllivenes r.cancelRegisteredJobContext(id) log.Infof(ctx, "job %d, session %s: paused", id, s.ID()) case StatusReverting: - if err := job.Update(ctx, txn, func(txn *kv.Txn, md JobMetadata, ju *JobUpdater) error { + if err := job.Update(ctx, txn, ie, func(_ *kv.Txn, _ sqlutil.InternalExecutor, md JobMetadata, ju *JobUpdater) error { r.cancelRegisteredJobContext(id) md.Payload.Error = errJobCanceled.Error() encodedErr := errors.EncodeError(ctx, errJobCanceled) diff --git a/pkg/jobs/delegate_control_test.go b/pkg/jobs/delegate_control_test.go index e6f2ceb045b5..1334425244df 100644 --- a/pkg/jobs/delegate_control_test.go +++ b/pkg/jobs/delegate_control_test.go @@ -175,7 +175,7 @@ func TestJobsControlForSchedules(t *testing.T) { // Create few jobs not started by any schedule. for i := 0; i < numJobs; i++ { _, err := registry.CreateAdoptableJobWithTxn( - context.Background(), record, registry.MakeJobID(), nil, /* txn */ + context.Background(), record, registry.MakeJobID(), nil /* txn */, nil, /* ie */ ) require.NoError(t, err) } @@ -208,7 +208,7 @@ func TestJobsControlForSchedules(t *testing.T) { } jobID := registry.MakeJobID() _, err := registry.CreateAdoptableJobWithTxn( - context.Background(), record, jobID, nil, /* txn */ + context.Background(), record, jobID, nil /* txn */, nil, /* ie */ ) require.NoError(t, err) @@ -304,7 +304,7 @@ func TestFilterJobsControlForSchedules(t *testing.T) { ID: scheduleID, } jobID := registry.MakeJobID() - _, err := registry.CreateAdoptableJobWithTxn(context.Background(), record, jobID, nil /* txn */) + _, err := registry.CreateAdoptableJobWithTxn(context.Background(), record, jobID, nil /* txn */, nil /* ie */) require.NoError(t, err) th.sqlDB.Exec(t, "UPDATE system.jobs SET status=$1 WHERE id=$2", status, jobID) } @@ -420,7 +420,7 @@ func TestJobControlByType(t *testing.T) { jobID := registry.MakeJobID() jobIDStrings = append(jobIDStrings, fmt.Sprintf("%d", jobID)) - _, err := registry.CreateAdoptableJobWithTxn(context.Background(), record, jobID, nil /* txn */) + _, err := registry.CreateAdoptableJobWithTxn(context.Background(), record, jobID, nil /* txn */, nil /* ie */) require.NoError(t, err) th.sqlDB.Exec(t, "UPDATE system.jobs SET status=$1 WHERE id=$2", status, jobID) } diff --git a/pkg/jobs/executor_impl.go b/pkg/jobs/executor_impl.go index b3effb60b76e..fa878070ce01 100644 --- a/pkg/jobs/executor_impl.go +++ b/pkg/jobs/executor_impl.go @@ -43,9 +43,10 @@ const retryFailedJobAfter = time.Minute func (e *inlineScheduledJobExecutor) ExecuteJob( ctx context.Context, cfg *scheduledjobs.JobExecutionConfig, - _ scheduledjobs.JobSchedulerEnv, + env scheduledjobs.JobSchedulerEnv, schedule *ScheduledJob, txn *kv.Txn, + ie sqlutil.InternalExecutor, ) error { sqlArgs := &jobspb.SqlStatementExecutionArg{} diff --git a/pkg/jobs/helpers_test.go b/pkg/jobs/helpers_test.go index 13c6c4ea6452..974947b33692 100644 --- a/pkg/jobs/helpers_test.go +++ b/pkg/jobs/helpers_test.go @@ -16,6 +16,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/jobs/jobspb" "github.com/cockroachdb/cockroach/pkg/kv" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" + "github.com/cockroachdb/cockroach/pkg/sql/sqlutil" "github.com/cockroachdb/errors" ) @@ -59,50 +60,56 @@ type OnPauseRequestFunc = onPauseRequestFunc var _ PauseRequester = FakeResumer{} func (d FakeResumer) OnPauseRequest( - ctx context.Context, execCtx interface{}, txn *kv.Txn, details *jobspb.Progress, + ctx context.Context, + execCtx interface{}, + txn *kv.Txn, + ie sqlutil.InternalExecutor, + details *jobspb.Progress, ) error { if d.PauseRequest == nil { return nil } - return d.PauseRequest(ctx, execCtx, txn, details) + return d.PauseRequest(ctx, execCtx, txn, ie, details) } // Started is a wrapper around the internal function that moves a job to the // started state. func (j *Job) Started(ctx context.Context) error { - return j.started(ctx, nil /* txn */) + return j.started(ctx, nil /* txn */, nil /* ie */) } // Reverted is a wrapper around the internal function that moves a job to the // reverting state. func (j *Job) Reverted(ctx context.Context, err error) error { - return j.reverted(ctx, nil /* txn */, err, nil) + return j.reverted(ctx, nil /* txn */, nil /* ie */, err, nil) } // Paused is a wrapper around the internal function that moves a job to the // paused state. func (j *Job) Paused(ctx context.Context) error { - return j.paused(ctx, nil /* txn */, nil /* fn */) + return j.paused(ctx, nil /* txn */, nil /* ie */, nil /* fn */) } // Failed is a wrapper around the internal function that moves a job to the // failed state. func (j *Job) Failed(ctx context.Context, causingErr error) error { - return j.failed(ctx, nil /* txn */, causingErr, nil /* fn */) + return j.failed(ctx, nil /* txn */, nil /* ie */, causingErr, nil /* fn */) } // Succeeded is a wrapper around the internal function that moves a job to the // succeeded state. func (j *Job) Succeeded(ctx context.Context) error { - return j.succeeded(ctx, nil /* txn */, nil /* fn */) + return j.succeeded(ctx, nil /* txn */, nil /* ie */, nil /* fn */) } // TestingCurrentStatus returns the current job status from the jobs table or error. -func (j *Job) TestingCurrentStatus(ctx context.Context, txn *kv.Txn) (Status, error) { +func (j *Job) TestingCurrentStatus( + ctx context.Context, txn *kv.Txn, ie sqlutil.InternalExecutor, +) (Status, error) { var statusString tree.DString - if err := j.runInTxn(ctx, txn, func(ctx context.Context, txn *kv.Txn) error { + if err := j.runInTxn(ctx, txn, ie, func(ctx context.Context, txn *kv.Txn, executor sqlutil.InternalExecutor) error { const selectStmt = "SELECT status FROM system.jobs WHERE id = $1" - row, err := j.registry.ex.QueryRow(ctx, "job-status", txn, selectStmt, j.ID()) + row, err := executor.QueryRow(ctx, "job-status", txn, selectStmt, j.ID()) if err != nil { return errors.Wrapf(err, "job %d: can't query system.jobs", j.ID()) } diff --git a/pkg/jobs/job_scheduler.go b/pkg/jobs/job_scheduler.go index 23fcfa48c611..6cccef1dfa30 100644 --- a/pkg/jobs/job_scheduler.go +++ b/pkg/jobs/job_scheduler.go @@ -112,13 +112,14 @@ func lookupNumRunningJobs( scheduleID int64, env scheduledjobs.JobSchedulerEnv, ie sqlutil.InternalExecutor, + txn *kv.Txn, ) (int64, error) { lookupStmt := fmt.Sprintf( "SELECT count(*) FROM %s WHERE created_by_type = '%s' AND created_by_id = %d AND status IN %s", env.SystemJobsTableName(), CreatedByScheduledJobs, scheduleID, NonTerminalStatusTupleString) row, err := ie.QueryRowEx( ctx, "lookup-num-running", - /*txn=*/ nil, + txn, sessiondata.InternalExecutorOverride{User: username.RootUserName()}, lookupStmt) if err != nil { @@ -130,7 +131,11 @@ func lookupNumRunningJobs( const recheckRunningAfter = 1 * time.Minute func (s *jobScheduler) processSchedule( - ctx context.Context, schedule *ScheduledJob, numRunning int64, txn *kv.Txn, + ctx context.Context, + schedule *ScheduledJob, + numRunning int64, + txn *kv.Txn, + ie sqlutil.InternalExecutor, ) error { if numRunning > 0 { switch schedule.ScheduleDetails().Wait { @@ -141,14 +146,14 @@ func (s *jobScheduler) processSchedule( schedule.SetNextRun(s.env.Now().Add(recheckRunningAfter)) schedule.SetScheduleStatus("delayed due to %d already running", numRunning) s.metrics.RescheduleWait.Inc(1) - return schedule.Update(ctx, s.InternalExecutor, txn) + return schedule.Update(ctx, ie, txn) case jobspb.ScheduleDetails_SKIP: if err := schedule.ScheduleNextRun(); err != nil { return err } schedule.SetScheduleStatus("rescheduled due to %d already running", numRunning) s.metrics.RescheduleSkip.Inc(1) - return schedule.Update(ctx, s.InternalExecutor, txn) + return schedule.Update(ctx, ie, txn) } } @@ -166,7 +171,7 @@ func (s *jobScheduler) processSchedule( schedule.SetNextRun(time.Time{}) } - if err := schedule.Update(ctx, s.InternalExecutor, txn); err != nil { + if err := schedule.Update(ctx, ie, txn); err != nil { return err } @@ -182,14 +187,14 @@ func (s *jobScheduler) processSchedule( schedule.ScheduledRunTime(), schedule.NextRun()) execCtx := logtags.AddTag(ctx, "schedule", schedule.ScheduleID()) - if err := executor.ExecuteJob(execCtx, s.JobExecutionConfig, s.env, schedule, txn); err != nil { + if err := executor.ExecuteJob(execCtx, s.JobExecutionConfig, s.env, schedule, txn, ie); err != nil { return errors.Wrapf(err, "executing schedule %d", schedule.ScheduleID()) } s.metrics.NumStarted.Inc(1) // Persist any mutations to the underlying schedule. - return schedule.Update(ctx, s.InternalExecutor, txn) + return schedule.Update(ctx, ie, txn) } type savePointError struct { @@ -236,9 +241,9 @@ func withSavePoint(ctx context.Context, txn *kv.Txn, fn func() error) error { // executeCandidateSchedule attempts to execute schedule. // The schedule is executed only if it's running. func (s *jobScheduler) executeCandidateSchedule( - ctx context.Context, candidate int64, txn *kv.Txn, + ctx context.Context, candidate int64, txn *kv.Txn, ie sqlutil.InternalExecutor, ) error { - schedule, err := loadCandidateScheduleForExecution(ctx, candidate, s.env, s.InternalExecutor, txn) + schedule, err := loadCandidateScheduleForExecution(ctx, candidate, s.env, ie, txn) if err != nil { if errors.Is(err, errScheduleNotRunnable) { return nil @@ -254,7 +259,7 @@ func (s *jobScheduler) executeCandidateSchedule( return nil } - numRunning, err := lookupNumRunningJobs(ctx, schedule.ScheduleID(), s.env, s.InternalExecutor) + numRunning, err := lookupNumRunningJobs(ctx, schedule.ScheduleID(), s.env, ie, txn) if err != nil { return err } @@ -265,10 +270,10 @@ func (s *jobScheduler) executeCandidateSchedule( return contextutil.RunWithTimeout( ctx, fmt.Sprintf("process-schedule-%d", schedule.ScheduleID()), timeout, func(ctx context.Context) error { - return s.processSchedule(ctx, schedule, numRunning, txn) + return s.processSchedule(ctx, schedule, numRunning, txn, ie) }) } - return s.processSchedule(ctx, schedule, numRunning, txn) + return s.processSchedule(ctx, schedule, numRunning, txn, ie) }); processErr != nil { if errors.HasType(processErr, (*savePointError)(nil)) { return errors.Wrapf(processErr, "savepoint error for schedule %d", schedule.ScheduleID()) @@ -342,8 +347,8 @@ func (s *jobScheduler) executeSchedules(ctx context.Context, maxSchedules int64) for ok, err = it.Next(ctx); ok; ok, err = it.Next(ctx) { row := it.Cur() candidateID := int64(tree.MustBeDInt(row[0])) - if err := s.DB.Txn(ctx, func(ctx context.Context, txn *kv.Txn) error { - return s.executeCandidateSchedule(ctx, candidateID, txn) + if err := s.InternalExecutorFactory.TxnWithExecutor(ctx, s.DB, nil /* sessionData */, func(ctx context.Context, txn *kv.Txn, ie sqlutil.InternalExecutor) error { + return s.executeCandidateSchedule(ctx, candidateID, txn, ie) }); err != nil { log.Errorf(ctx, "error executing candidate schedule %d: %s", candidateID, err) } diff --git a/pkg/jobs/job_scheduler_test.go b/pkg/jobs/job_scheduler_test.go index 7f6460c1f02d..8494f7105559 100644 --- a/pkg/jobs/job_scheduler_test.go +++ b/pkg/jobs/job_scheduler_test.go @@ -239,11 +239,12 @@ type recordScheduleExecutor struct { } func (n *recordScheduleExecutor) ExecuteJob( - _ context.Context, - _ *scheduledjobs.JobExecutionConfig, - _ scheduledjobs.JobSchedulerEnv, + ctx context.Context, + cfg *scheduledjobs.JobExecutionConfig, + env scheduledjobs.JobSchedulerEnv, schedule *ScheduledJob, - _ *kv.Txn, + txn *kv.Txn, + ie sqlutil.InternalExecutor, ) error { n.executed = append(n.executed, schedule.ScheduleID()) return nil @@ -469,11 +470,12 @@ type returnErrorExecutor struct { } func (e *returnErrorExecutor) ExecuteJob( - _ context.Context, - _ *scheduledjobs.JobExecutionConfig, - _ scheduledjobs.JobSchedulerEnv, + ctx context.Context, + cfg *scheduledjobs.JobExecutionConfig, + env scheduledjobs.JobSchedulerEnv, schedule *ScheduledJob, - _ *kv.Txn, + txn *kv.Txn, + ie sqlutil.InternalExecutor, ) error { e.numCalls++ return errors.Newf("error for schedule %d", schedule.ScheduleID()) @@ -635,6 +637,7 @@ func (e *txnConflictExecutor) ExecuteJob( env scheduledjobs.JobSchedulerEnv, schedule *ScheduledJob, txn *kv.Txn, + ie sqlutil.InternalExecutor, ) error { // Read number of rows -- this count will be used when updating // a single row in the table. @@ -830,6 +833,7 @@ func (e *blockUntilCancelledExecutor) ExecuteJob( env scheduledjobs.JobSchedulerEnv, schedule *ScheduledJob, txn *kv.Txn, + ie sqlutil.InternalExecutor, ) error { done := func() {} e.once.Do(func() { diff --git a/pkg/jobs/jobs.go b/pkg/jobs/jobs.go index 47af9b73eff5..c557d79963f9 100644 --- a/pkg/jobs/jobs.go +++ b/pkg/jobs/jobs.go @@ -256,8 +256,8 @@ func (j *Job) taskName() string { // Started marks the tracked job as started by updating status to running in // jobs table. -func (j *Job) started(ctx context.Context, txn *kv.Txn) error { - return j.Update(ctx, txn, func(_ *kv.Txn, md JobMetadata, ju *JobUpdater) error { +func (j *Job) started(ctx context.Context, txn *kv.Txn, ie sqlutil.InternalExecutor) error { + return j.Update(ctx, txn, ie, func(_ *kv.Txn, _ sqlutil.InternalExecutor, md JobMetadata, ju *JobUpdater) error { if md.Status != StatusPending && md.Status != StatusRunning { return errors.Errorf("job with status %s cannot be marked started", md.Status) } @@ -282,15 +282,17 @@ func (j *Job) started(ctx context.Context, txn *kv.Txn) error { // CheckStatus verifies the status of the job and returns an error if the job's // status isn't Running or Reverting. -func (j *Job) CheckStatus(ctx context.Context, txn *kv.Txn) error { - return j.Update(ctx, txn, func(_ *kv.Txn, md JobMetadata, _ *JobUpdater) error { +func (j *Job) CheckStatus(ctx context.Context, txn *kv.Txn, ie sqlutil.InternalExecutor) error { + return j.Update(ctx, txn, ie, func(_ *kv.Txn, _ sqlutil.InternalExecutor, md JobMetadata, _ *JobUpdater) error { return md.CheckRunningOrReverting() }) } // CheckTerminalStatus returns true if the job is in a terminal status. -func (j *Job) CheckTerminalStatus(ctx context.Context, txn *kv.Txn) bool { - err := j.Update(ctx, txn, func(_ *kv.Txn, md JobMetadata, _ *JobUpdater) error { +func (j *Job) CheckTerminalStatus( + ctx context.Context, txn *kv.Txn, ie sqlutil.InternalExecutor, +) bool { + err := j.Update(ctx, txn, ie, func(_ *kv.Txn, _ sqlutil.InternalExecutor, md JobMetadata, _ *JobUpdater) error { if !md.Status.Terminal() { return &InvalidStatusError{md.ID, md.Status, "checking that job status is success", md.Payload.Error} } @@ -304,9 +306,9 @@ func (j *Job) CheckTerminalStatus(ctx context.Context, txn *kv.Txn) bool { // It sets the job's RunningStatus field to the value returned by runningStatusFn // and persists runningStatusFn's modifications to the job's details, if any. func (j *Job) RunningStatus( - ctx context.Context, txn *kv.Txn, runningStatusFn RunningStatusFn, + ctx context.Context, txn *kv.Txn, ie sqlutil.InternalExecutor, runningStatusFn RunningStatusFn, ) error { - return j.Update(ctx, txn, func(_ *kv.Txn, md JobMetadata, ju *JobUpdater) error { + return j.Update(ctx, txn, ie, func(_ *kv.Txn, _ sqlutil.InternalExecutor, md JobMetadata, ju *JobUpdater) error { if err := md.CheckRunningOrReverting(); err != nil { return err } @@ -348,9 +350,9 @@ func FractionUpdater(f float32) FractionProgressedFn { // Jobs for which progress computations do not depend on their details can // use the FractionUpdater helper to construct a ProgressedFn. func (j *Job) FractionProgressed( - ctx context.Context, txn *kv.Txn, progressedFn FractionProgressedFn, + ctx context.Context, txn *kv.Txn, ie sqlutil.InternalExecutor, progressedFn FractionProgressedFn, ) error { - return j.Update(ctx, txn, func(_ *kv.Txn, md JobMetadata, ju *JobUpdater) error { + return j.Update(ctx, txn, ie, func(_ *kv.Txn, _ sqlutil.InternalExecutor, md JobMetadata, ju *JobUpdater) error { if err := md.CheckRunningOrReverting(); err != nil { return err } @@ -377,9 +379,12 @@ func (j *Job) FractionProgressed( // registry adoption loop by the node currently running a job to move it from // PauseRequested to paused. func (j *Job) paused( - ctx context.Context, txn *kv.Txn, fn func(context.Context, *kv.Txn) error, + ctx context.Context, + txn *kv.Txn, + ie sqlutil.InternalExecutor, + fn func(context.Context, *kv.Txn) error, ) error { - return j.Update(ctx, txn, func(txn *kv.Txn, md JobMetadata, ju *JobUpdater) error { + return j.Update(ctx, txn, ie, func(txn *kv.Txn, ie sqlutil.InternalExecutor, md JobMetadata, ju *JobUpdater) error { if md.Status == StatusPaused { // Already paused - do nothing. return nil @@ -401,8 +406,8 @@ func (j *Job) paused( // job is currently paused. It does not directly resume the job; rather, it // expires the job's lease so that a Registry adoption loop detects it and // resumes it. -func (j *Job) unpaused(ctx context.Context, txn *kv.Txn) error { - return j.Update(ctx, txn, func(txn *kv.Txn, md JobMetadata, ju *JobUpdater) error { +func (j *Job) unpaused(ctx context.Context, txn *kv.Txn, ie sqlutil.InternalExecutor) error { + return j.Update(ctx, txn, ie, func(txn *kv.Txn, ie sqlutil.InternalExecutor, md JobMetadata, ju *JobUpdater) error { if md.Status == StatusRunning || md.Status == StatusReverting { // Already resumed - do nothing. return nil @@ -429,9 +434,12 @@ func (j *Job) unpaused(ctx context.Context, txn *kv.Txn) error { // that it is in state StatusCancelRequested and will move it to state // StatusReverting. func (j *Job) cancelRequested( - ctx context.Context, txn *kv.Txn, fn func(context.Context, *kv.Txn) error, + ctx context.Context, + txn *kv.Txn, + ie sqlutil.InternalExecutor, + fn func(context.Context, *kv.Txn) error, ) error { - return j.Update(ctx, txn, func(txn *kv.Txn, md JobMetadata, ju *JobUpdater) error { + return j.Update(ctx, txn, ie, func(_ *kv.Txn, _ sqlutil.InternalExecutor, md JobMetadata, ju *JobUpdater) error { if md.Payload.Noncancelable { return errors.Newf("job %d: not cancelable", j.ID()) } @@ -459,7 +467,7 @@ func (j *Job) cancelRequested( // onPauseRequestFunc is a function used to perform action on behalf of a job // implementation when a pause is requested. type onPauseRequestFunc func( - ctx context.Context, planHookState interface{}, txn *kv.Txn, progress *jobspb.Progress, + ctx context.Context, planHookState interface{}, txn *kv.Txn, ie sqlutil.InternalExecutor, progress *jobspb.Progress, ) error // PauseRequested sets the status of the tracked job to pause-requested. It does @@ -467,9 +475,13 @@ type onPauseRequestFunc func( // actively cancel it when it notices that it is in state StatusPauseRequested // and will move it to state StatusPaused. func (j *Job) PauseRequested( - ctx context.Context, txn *kv.Txn, fn onPauseRequestFunc, reason string, + ctx context.Context, + txn *kv.Txn, + ie sqlutil.InternalExecutor, + fn onPauseRequestFunc, + reason string, ) error { - return j.Update(ctx, txn, func(txn *kv.Txn, md JobMetadata, ju *JobUpdater) error { + return j.Update(ctx, txn, ie, func(txn1 *kv.Txn, executor sqlutil.InternalExecutor, md JobMetadata, ju *JobUpdater) error { if md.Status == StatusPauseRequested || md.Status == StatusPaused { return nil } @@ -479,7 +491,7 @@ func (j *Job) PauseRequested( if fn != nil { execCtx, cleanup := j.registry.execCtx("pause request", j.Payload().UsernameProto.Decode()) defer cleanup() - if err := fn(ctx, execCtx, txn, md.Progress); err != nil { + if err := fn(ctx, execCtx, txn1, executor, md.Progress); err != nil { return err } ju.UpdateProgress(md.Progress) @@ -494,9 +506,13 @@ func (j *Job) PauseRequested( // reverted sets the status of the tracked job to reverted. func (j *Job) reverted( - ctx context.Context, txn *kv.Txn, err error, fn func(context.Context, *kv.Txn) error, + ctx context.Context, + txn *kv.Txn, + ie sqlutil.InternalExecutor, + err error, + fn func(context.Context, *kv.Txn) error, ) error { - return j.Update(ctx, txn, func(txn *kv.Txn, md JobMetadata, ju *JobUpdater) error { + return j.Update(ctx, txn, ie, func(txn *kv.Txn, ie sqlutil.InternalExecutor, md JobMetadata, ju *JobUpdater) error { if md.Status != StatusReverting && md.Status != StatusCancelRequested && md.Status != StatusRunning && @@ -545,9 +561,12 @@ func (j *Job) reverted( // Canceled sets the status of the tracked job to cancel. func (j *Job) canceled( - ctx context.Context, txn *kv.Txn, fn func(context.Context, *kv.Txn) error, + ctx context.Context, + txn *kv.Txn, + ie sqlutil.InternalExecutor, + fn func(context.Context, *kv.Txn) error, ) error { - return j.Update(ctx, txn, func(txn *kv.Txn, md JobMetadata, ju *JobUpdater) error { + return j.Update(ctx, txn, ie, func(_ *kv.Txn, _ sqlutil.InternalExecutor, md JobMetadata, ju *JobUpdater) error { if md.Status == StatusCanceled { return nil } @@ -568,9 +587,13 @@ func (j *Job) canceled( // Failed marks the tracked job as having failed with the given error. func (j *Job) failed( - ctx context.Context, txn *kv.Txn, err error, fn func(context.Context, *kv.Txn) error, + ctx context.Context, + txn *kv.Txn, + ie sqlutil.InternalExecutor, + err error, + fn func(context.Context, *kv.Txn) error, ) error { - return j.Update(ctx, txn, func(txn *kv.Txn, md JobMetadata, ju *JobUpdater) error { + return j.Update(ctx, txn, ie, func(txn *kv.Txn, _ sqlutil.InternalExecutor, md JobMetadata, ju *JobUpdater) error { // TODO(spaskob): should we fail if the terminal state is not StatusFailed? if md.Status.Terminal() { // Already done - do nothing. @@ -607,14 +630,18 @@ func (j *Job) failed( // RevertFailed marks the tracked job as having failed during revert with the // given error. Manual cleanup is required when the job is in this state. func (j *Job) revertFailed( - ctx context.Context, txn *kv.Txn, err error, fn func(context.Context, *kv.Txn) error, + ctx context.Context, + txn *kv.Txn, + ie sqlutil.InternalExecutor, + err error, + fn func(context.Context, *kv.Txn) error, ) error { - return j.Update(ctx, txn, func(txn *kv.Txn, md JobMetadata, ju *JobUpdater) error { + return j.Update(ctx, txn, ie, func(txn1 *kv.Txn, ie1 sqlutil.InternalExecutor, md JobMetadata, ju *JobUpdater) error { if md.Status != StatusReverting { return fmt.Errorf("job with status %s cannot fail during a revert", md.Status) } if fn != nil { - if err := fn(ctx, txn); err != nil { + if err := fn(ctx, txn1); err != nil { return err } } @@ -629,9 +656,12 @@ func (j *Job) revertFailed( // succeeded marks the tracked job as having succeeded and sets its fraction // completed to 1.0. func (j *Job) succeeded( - ctx context.Context, txn *kv.Txn, fn func(context.Context, *kv.Txn) error, + ctx context.Context, + txn *kv.Txn, + ie sqlutil.InternalExecutor, + fn func(context.Context, *kv.Txn) error, ) error { - return j.Update(ctx, txn, func(txn *kv.Txn, md JobMetadata, ju *JobUpdater) error { + return j.Update(ctx, txn, ie, func(txn *kv.Txn, ie sqlutil.InternalExecutor, md JobMetadata, ju *JobUpdater) error { if md.Status == StatusSucceeded { return nil } @@ -655,8 +685,10 @@ func (j *Job) succeeded( } // SetDetails sets the details field of the currently running tracked job. -func (j *Job) SetDetails(ctx context.Context, txn *kv.Txn, details interface{}) error { - return j.Update(ctx, txn, func(txn *kv.Txn, md JobMetadata, ju *JobUpdater) error { +func (j *Job) SetDetails( + ctx context.Context, txn *kv.Txn, ie sqlutil.InternalExecutor, details interface{}, +) error { + return j.Update(ctx, txn, ie, func(_ *kv.Txn, _ sqlutil.InternalExecutor, md JobMetadata, ju *JobUpdater) error { if err := md.CheckRunningOrReverting(); err != nil { return err } @@ -667,8 +699,10 @@ func (j *Job) SetDetails(ctx context.Context, txn *kv.Txn, details interface{}) } // SetProgress sets the details field of the currently running tracked job. -func (j *Job) SetProgress(ctx context.Context, txn *kv.Txn, details interface{}) error { - return j.Update(ctx, txn, func(txn *kv.Txn, md JobMetadata, ju *JobUpdater) error { +func (j *Job) SetProgress( + ctx context.Context, txn *kv.Txn, ie sqlutil.InternalExecutor, details interface{}, +) error { + return j.Update(ctx, txn, ie, func(_ *kv.Txn, _ sqlutil.InternalExecutor, md JobMetadata, ju *JobUpdater) error { if err := md.CheckRunningOrReverting(); err != nil { return err } @@ -735,14 +769,21 @@ func (j *Job) MarkIdle(isIdle bool) { } func (j *Job) runInTxn( - ctx context.Context, txn *kv.Txn, fn func(context.Context, *kv.Txn) error, + ctx context.Context, + txn *kv.Txn, + ie sqlutil.InternalExecutor, + fn func(context.Context, *kv.Txn, sqlutil.InternalExecutor) error, ) error { if txn != nil { + if ie == nil { + return errors.New("txn is not nil but ie is nil") + } + // TODO(janexing): ie can't be nil // Don't run fn in a retry loop because we need retryable errors to // propagate up to the transaction's properly-scoped retry loop. - return fn(ctx, txn) + return fn(ctx, txn, ie) } - return j.registry.db.Txn(ctx, fn) + return j.registry.internalExecutorFactory.TxnWithExecutor(ctx, j.registry.db, nil /* sessionData */, fn) } // JobNotFoundError is returned from load when the job does not exist. @@ -764,7 +805,7 @@ func HasJobNotFoundError(err error) bool { return errors.HasType(err, (*JobNotFoundError)(nil)) } -func (j *Job) load(ctx context.Context, txn *kv.Txn) error { +func (j *Job) load(ctx context.Context, txn *kv.Txn, ie sqlutil.InternalExecutor) error { ctx, sp := tracing.ChildSpan(ctx, "load-job") defer sp.Finish() @@ -773,7 +814,7 @@ func (j *Job) load(ctx context.Context, txn *kv.Txn) error { var createdBy *CreatedByInfo var status Status - if err := j.runInTxn(ctx, txn, func(ctx context.Context, txn *kv.Txn) error { + if err := j.runInTxn(ctx, txn, ie, func(ctx context.Context, newTxn *kv.Txn, executor sqlutil.InternalExecutor) error { const ( queryNoSessionID = "SELECT payload, progress, created_by_type, created_by_id, status FROM system.jobs WHERE id = $1" queryWithSessionID = queryNoSessionID + " AND claim_session_id = $2" @@ -783,10 +824,10 @@ func (j *Job) load(ctx context.Context, txn *kv.Txn) error { var err error var row tree.Datums if j.session == nil { - row, err = j.registry.ex.QueryRowEx(ctx, "load-job-query", txn, sess, + row, err = executor.QueryRowEx(ctx, "load-job-query", newTxn, sess, queryNoSessionID, j.ID()) } else { - row, err = j.registry.ex.QueryRowEx(ctx, "load-job-query", txn, sess, + row, err = executor.QueryRowEx(ctx, "load-job-query", newTxn, sess, queryWithSessionID, j.ID(), j.session.ID().UnsafeBytes()) } if err != nil { @@ -992,7 +1033,7 @@ func (sj *StartableJob) Cancel(ctx context.Context) error { sj.registry.unregister(sj.ID()) } }() - return sj.registry.CancelRequested(ctx, nil, sj.ID()) + return sj.registry.CancelRequested(ctx, nil /* txn */, nil /* ie */, sj.ID()) } func (sj *StartableJob) recordStart() (alreadyStarted bool) { diff --git a/pkg/jobs/jobs_test.go b/pkg/jobs/jobs_test.go index a312245fd419..3a7fa4c16da8 100644 --- a/pkg/jobs/jobs_test.go +++ b/pkg/jobs/jobs_test.go @@ -198,7 +198,11 @@ type registryTestSuite struct { } func noopPauseRequestFunc( - ctx context.Context, planHookState interface{}, txn *kv.Txn, progress *jobspb.Progress, + ctx context.Context, + planHookState interface{}, + txn *kv.Txn, + ie sqlutil.InternalExecutor, + progress *jobspb.Progress, ) error { return nil } @@ -284,7 +288,7 @@ func (rts *registryTestSuite) setUp(t *testing.T) { case err := <-rts.resumeCh: return err case <-rts.progressCh: - err := job.FractionProgressed(rts.ctx, nil /* txn */, jobs.FractionUpdater(0)) + err := job.FractionProgressed(rts.ctx, nil /* txn */, nil /* ie */, jobs.FractionUpdater(0)) if err != nil { return err } @@ -327,8 +331,8 @@ func (rts *registryTestSuite) setUp(t *testing.T) { rts.mu.a.Success = true return rts.successErr }, - PauseRequest: func(ctx context.Context, execCfg interface{}, txn *kv.Txn, progress *jobspb.Progress) error { - return rts.onPauseRequest(ctx, execCfg, txn, progress) + PauseRequest: func(ctx context.Context, execCfg interface{}, txn *kv.Txn, ie sqlutil.InternalExecutor, progress *jobspb.Progress) error { + return rts.onPauseRequest(ctx, execCfg, txn, ie, progress) }, } }, jobs.UsesTenantCostControl) @@ -357,7 +361,7 @@ func (rts *registryTestSuite) check(t *testing.T, expectedStatus jobs.Status) { if expectedStatus == "" { return nil } - st, err := rts.job.TestingCurrentStatus(rts.ctx, nil /* txn */) + st, err := rts.job.TestingCurrentStatus(rts.ctx, nil /* txn */, nil /* ie */) if err != nil { return err } @@ -912,7 +916,7 @@ func TestRegistryLifecycle(t *testing.T) { madeUpSpans := []roachpb.Span{ {Key: roachpb.Key("foo")}, } - rts.onPauseRequest = func(ctx context.Context, planHookState interface{}, txn *kv.Txn, progress *jobspb.Progress) error { + rts.onPauseRequest = func(ctx context.Context, planHookState interface{}, txn *kv.Txn, ie sqlutil.InternalExecutor, progress *jobspb.Progress) error { progress.GetImport().SpanProgress = madeUpSpans return nil } @@ -948,7 +952,7 @@ func TestRegistryLifecycle(t *testing.T) { rts.setUp(t) defer rts.tearDown() - rts.onPauseRequest = func(ctx context.Context, planHookState interface{}, txn *kv.Txn, progress *jobspb.Progress) error { + rts.onPauseRequest = func(ctx context.Context, planHookState interface{}, txn *kv.Txn, ie sqlutil.InternalExecutor, progress *jobspb.Progress) error { return errors.New("boom") } @@ -1188,7 +1192,7 @@ func TestJobLifecycle(t *testing.T) { createJob := func(record jobs.Record) (*jobs.Job, expectation) { beforeTime := timeutil.Now() - job, err := registry.CreateAdoptableJobWithTxn(ctx, record, registry.MakeJobID(), nil /* txn */) + job, err := registry.CreateAdoptableJobWithTxn(ctx, record, registry.MakeJobID(), nil /* txn */, nil /* ie */) require.NoError(t, err) payload := job.Payload() return job, expectation{ @@ -1274,7 +1278,7 @@ func TestJobLifecycle(t *testing.T) { {0.0, 0.0}, {0.5, 0.5}, {0.5, 0.5}, {0.4, 0.4}, {0.8, 0.8}, {1.0, 1.0}, } for _, f := range progresses { - if err := woodyJob.FractionProgressed(ctx, nil /* txn */, jobs.FractionUpdater(f.actual)); err != nil { + if err := woodyJob.FractionProgressed(ctx, nil /* txn */, nil /* ie */, jobs.FractionUpdater(f.actual)); err != nil { t.Fatal(err) } woodyExp.FractionCompleted = f.expected @@ -1284,7 +1288,7 @@ func TestJobLifecycle(t *testing.T) { } // Test Progressed callbacks. - if err := woodyJob.FractionProgressed(ctx, nil /* txn */, func(_ context.Context, details jobspb.ProgressDetails) float32 { + if err := woodyJob.FractionProgressed(ctx, nil /* txn */, nil /* ie */, func(_ context.Context, details jobspb.ProgressDetails) float32 { details.(*jobspb.Progress_Restore).Restore.HighWater = roachpb.Key("mariana") return 1.0 }); err != nil { @@ -1318,7 +1322,7 @@ func TestJobLifecycle(t *testing.T) { Before: timeutil.Now(), Error: "Buzz Lightyear can't fly", } - buzzJob, err := registry.CreateAdoptableJobWithTxn(ctx, buzzRecord, registry.MakeJobID(), nil /* txn */) + buzzJob, err := registry.CreateAdoptableJobWithTxn(ctx, buzzRecord, registry.MakeJobID(), nil /* txn */, nil /* txn */) require.NoError(t, err) if err := buzzExp.verify(buzzJob.ID(), jobs.StatusRunning); err != nil { t.Fatal(err) @@ -1331,7 +1335,7 @@ func TestJobLifecycle(t *testing.T) { t.Fatal(err) } - if err := buzzJob.FractionProgressed(ctx, nil /* txn */, jobs.FractionUpdater(.42)); err != nil { + if err := buzzJob.FractionProgressed(ctx, nil /* txn */, nil /* ie */, jobs.FractionUpdater(.42)); err != nil { t.Fatal(err) } buzzExp.FractionCompleted = .42 @@ -1451,7 +1455,7 @@ func TestJobLifecycle(t *testing.T) { t.Run("cancelable jobs can be paused until finished", func(t *testing.T) { job, exp := startLeasedJob(t, defaultRecord) - if err := registry.PauseRequested(ctx, nil, job.ID(), ""); err != nil { + if err := registry.PauseRequested(ctx, nil /* txn */, nil /* ie */, job.ID(), ""); err != nil { t.Fatal(err) } if err := job.Paused(ctx); err != nil { @@ -1460,11 +1464,11 @@ func TestJobLifecycle(t *testing.T) { if err := exp.verify(job.ID(), jobs.StatusPaused); err != nil { t.Fatal(err) } - if err := registry.Unpause(ctx, nil, job.ID()); err != nil { + if err := registry.Unpause(ctx, nil /* txn */, nil /* ie */, job.ID()); err != nil { t.Fatal(err) } // Resume the job again to ensure that the resumption is idempotent. - if err := registry.Unpause(ctx, nil, job.ID()); err != nil { + if err := registry.Unpause(ctx, nil /* txn */, nil /* ie */, job.ID()); err != nil { t.Fatal(err) } if err := exp.verify(job.ID(), jobs.StatusRunning); err != nil { @@ -1479,7 +1483,7 @@ func TestJobLifecycle(t *testing.T) { if err := job.Succeeded(ctx); err != nil { t.Fatal(err) } - if err := registry.PauseRequested(ctx, nil, job.ID(), ""); !testutils.IsError(err, "cannot be requested to be paused") { + if err := registry.PauseRequested(ctx, nil /* txn */, nil /* ie */, job.ID(), ""); !testutils.IsError(err, "cannot be requested to be paused") { t.Fatalf("expected 'cannot pause succeeded job', but got '%s'", err) } }) @@ -1487,7 +1491,7 @@ func TestJobLifecycle(t *testing.T) { t.Run("cancelable jobs can be canceled until finished", func(t *testing.T) { { job, exp := startLeasedJob(t, defaultRecord) - if err := registry.CancelRequested(ctx, nil, job.ID()); err != nil { + if err := registry.CancelRequested(ctx, nil /* txn */, nil /* ie */, job.ID()); err != nil { t.Fatal(err) } if err := exp.verify(job.ID(), jobs.StatusCancelRequested); err != nil { @@ -1500,7 +1504,7 @@ func TestJobLifecycle(t *testing.T) { if err := job.Started(ctx); err != nil { t.Fatal(err) } - if err := registry.CancelRequested(ctx, nil, job.ID()); err != nil { + if err := registry.CancelRequested(ctx, nil /* txn */, nil /* ie */, job.ID()); err != nil { t.Fatal(err) } if err := exp.verify(job.ID(), jobs.StatusCancelRequested); err != nil { @@ -1510,13 +1514,13 @@ func TestJobLifecycle(t *testing.T) { { job, exp := startLeasedJob(t, defaultRecord) - if err := registry.PauseRequested(ctx, nil, job.ID(), ""); err != nil { + if err := registry.PauseRequested(ctx, nil /* txn */, nil /* ie */, job.ID(), ""); err != nil { t.Fatal(err) } if err := job.Paused(ctx); err != nil { t.Fatal(err) } - if err := registry.CancelRequested(ctx, nil, job.ID()); err != nil { + if err := registry.CancelRequested(ctx, nil /* txn */, nil /* ie */, job.ID()); err != nil { t.Fatal(err) } if err := exp.verify(job.ID(), jobs.StatusCancelRequested); err != nil { @@ -1530,7 +1534,7 @@ func TestJobLifecycle(t *testing.T) { t.Fatal(err) } expectedErr := "job with status succeeded cannot be requested to be canceled" - if err := registry.CancelRequested(ctx, nil, job.ID()); !testutils.IsError(err, expectedErr) { + if err := registry.CancelRequested(ctx, nil /* txn */, nil /* ie */, job.ID()); !testutils.IsError(err, expectedErr) { t.Fatalf("expected '%s', but got '%s'", expectedErr, err) } } @@ -1539,10 +1543,10 @@ func TestJobLifecycle(t *testing.T) { t.Run("unpaused jobs cannot be resumed", func(t *testing.T) { { job, _ := startLeasedJob(t, defaultRecord) - if err := registry.CancelRequested(ctx, nil, job.ID()); err != nil { + if err := registry.CancelRequested(ctx, nil /* txn */, nil /* ie */, job.ID()); err != nil { t.Fatal(err) } - if err := registry.Unpause(ctx, nil, job.ID()); !testutils.IsError(err, "cannot be resumed") { + if err := registry.Unpause(ctx, nil /* txn */, nil /* ie */, job.ID()); !testutils.IsError(err, "cannot be resumed") { t.Errorf("got unexpected status '%v'", err) } } @@ -1553,7 +1557,7 @@ func TestJobLifecycle(t *testing.T) { t.Fatal(err) } expectedErr := fmt.Sprintf("job with status %s cannot be resumed", jobs.StatusSucceeded) - if err := registry.Unpause(ctx, nil, job.ID()); !testutils.IsError(err, expectedErr) { + if err := registry.Unpause(ctx, nil /* txn */, nil /* ie */, job.ID()); !testutils.IsError(err, expectedErr) { t.Errorf("expected '%s', but got '%v'", expectedErr, err) } } @@ -1569,17 +1573,18 @@ func TestJobLifecycle(t *testing.T) { // panic. _, _ = registry.CreateAdoptableJobWithTxn(ctx, jobs.Record{ Details: 42, - }, registry.MakeJobID(), nil /* txn */) + }, registry.MakeJobID(), nil /* txn */, nil /* txn */) }) t.Run("update before create fails", func(t *testing.T) { // Attempt to create the job but abort the transaction. var job *jobs.Job - require.Regexp(t, "boom", s.DB().Txn(ctx, func(ctx context.Context, txn *kv.Txn) error { + ief := s.InternalExecutorFactory().(sqlutil.InternalExecutorFactory) + require.Regexp(t, "boom", ief.TxnWithExecutor(ctx, s.DB(), nil /* sessionData */, func(ctx context.Context, txn *kv.Txn, ie sqlutil.InternalExecutor) error { job, _ = registry.CreateAdoptableJobWithTxn(ctx, jobs.Record{ Details: jobspb.RestoreDetails{}, Progress: jobspb.RestoreProgress{}, - }, registry.MakeJobID(), txn) + }, registry.MakeJobID(), txn, ie) return errors.New("boom") })) if err := job.Started(ctx); !testutils.IsError(err, "not found in system.jobs table") { @@ -1613,7 +1618,7 @@ func TestJobLifecycle(t *testing.T) { {WallTime: 2, Logical: 0}, } for _, ts := range highWaters { - require.NoError(t, job.Update(ctx, nil, func(_ *kv.Txn, md jobs.JobMetadata, ju *jobs.JobUpdater) error { + require.NoError(t, job.Update(ctx, nil /* txn */, nil /* txn */, func(_ *kv.Txn, _ sqlutil.InternalExecutor, md jobs.JobMetadata, ju *jobs.JobUpdater) error { return jobs.UpdateHighwaterProgressed(ts, md, ju) })) p := job.Progress() @@ -1628,13 +1633,13 @@ func TestJobLifecycle(t *testing.T) { if err := job.Started(ctx); err != nil { t.Fatal(err) } - if err := job.FractionProgressed(ctx, nil /* txn */, jobs.FractionUpdater(-0.1)); !testutils.IsError(err, "outside allowable range") { + if err := job.FractionProgressed(ctx, nil /* txn */, nil /* ie */, jobs.FractionUpdater(-0.1)); !testutils.IsError(err, "outside allowable range") { t.Fatalf("expected 'outside allowable range' error, but got %v", err) } - if err := job.FractionProgressed(ctx, nil /* txn */, jobs.FractionUpdater(1.1)); !testutils.IsError(err, "outside allowable range") { + if err := job.FractionProgressed(ctx, nil /* txn */, nil /* ie */, jobs.FractionUpdater(1.1)); !testutils.IsError(err, "outside allowable range") { t.Fatalf("expected 'outside allowable range' error, but got %v", err) } - if err := job.Update(ctx, nil, func(_ *kv.Txn, md jobs.JobMetadata, ju *jobs.JobUpdater) error { + if err := job.Update(ctx, nil /* txn */, nil /* ie */, func(_ *kv.Txn, _ sqlutil.InternalExecutor, md jobs.JobMetadata, ju *jobs.JobUpdater) error { return jobs.UpdateHighwaterProgressed(hlc.Timestamp{WallTime: -1}, md, ju) }); !testutils.IsError(err, "outside allowable range") { t.Fatalf("expected 'outside allowable range' error, but got %v", err) @@ -1646,7 +1651,7 @@ func TestJobLifecycle(t *testing.T) { if err := job.Started(ctx); err != nil { t.Fatal(err) } - if err := job.Update(ctx, nil, func(_ *kv.Txn, _ jobs.JobMetadata, ju *jobs.JobUpdater) error { + if err := job.Update(ctx, nil /* txn */, nil /* ie */, func(_ *kv.Txn, _ sqlutil.InternalExecutor, _ jobs.JobMetadata, ju *jobs.JobUpdater) error { return errors.Errorf("boom") }); !testutils.IsError(err, "boom") { t.Fatalf("expected 'boom' error, but got %v", err) @@ -1661,7 +1666,7 @@ func TestJobLifecycle(t *testing.T) { if err := job.Succeeded(ctx); err != nil { t.Fatal(err) } - if err := job.FractionProgressed(ctx, nil /* txn */, jobs.FractionUpdater(0.5)); !testutils.IsError( + if err := job.FractionProgressed(ctx, nil /* txn */, nil /* ie */, jobs.FractionUpdater(0.5)); !testutils.IsError( err, `cannot update progress on succeeded job \(id \d+\)`, ) { t.Fatalf("expected 'cannot update progress' error, but got %v", err) @@ -1670,10 +1675,10 @@ func TestJobLifecycle(t *testing.T) { t.Run("progress on paused job fails", func(t *testing.T) { job, _ := startLeasedJob(t, defaultRecord) - if err := registry.PauseRequested(ctx, nil, job.ID(), ""); err != nil { + if err := registry.PauseRequested(ctx, nil /* txn */, nil /* ie */, job.ID(), ""); err != nil { t.Fatal(err) } - if err := job.FractionProgressed(ctx, nil /* txn */, jobs.FractionUpdater(0.5)); !testutils.IsError( + if err := job.FractionProgressed(ctx, nil /* txn */, nil /* ie */, jobs.FractionUpdater(0.5)); !testutils.IsError( err, `cannot update progress on pause-requested job`, ) { t.Fatalf("expected progress error, but got %v", err) @@ -1682,10 +1687,10 @@ func TestJobLifecycle(t *testing.T) { t.Run("progress on canceled job fails", func(t *testing.T) { job, _ := startLeasedJob(t, defaultRecord) - if err := registry.CancelRequested(ctx, nil, job.ID()); err != nil { + if err := registry.CancelRequested(ctx, nil /* txn */, nil /* ie */, job.ID()); err != nil { t.Fatal(err) } - if err := job.FractionProgressed(ctx, nil /* txn */, jobs.FractionUpdater(0.5)); !testutils.IsError( + if err := job.FractionProgressed(ctx, nil /* txn */, nil /* ie */, jobs.FractionUpdater(0.5)); !testutils.IsError( err, `cannot update progress on cancel-requested job \(id \d+\)`, ) { t.Fatalf("expected progress error, but got %v", err) @@ -1697,7 +1702,7 @@ func TestJobLifecycle(t *testing.T) { if err := job.Started(ctx); err != nil { t.Fatal(err) } - if err := job.FractionProgressed(ctx, nil /* txn */, jobs.FractionUpdater(0.2)); err != nil { + if err := job.FractionProgressed(ctx, nil /* txn */, nil /* ie */, jobs.FractionUpdater(0.2)); err != nil { t.Fatal(err) } if err := job.Succeeded(ctx); err != nil { @@ -1717,14 +1722,14 @@ func TestJobLifecycle(t *testing.T) { require.NoError(t, exp.verify(job.ID(), jobs.StatusRunning)) newDetails := jobspb.ImportDetails{URIs: []string{"new"}} exp.Record.Details = newDetails - require.NoError(t, job.SetDetails(ctx, nil /* txn */, newDetails)) + require.NoError(t, job.SetDetails(ctx, nil /* txn */, nil /* ie */, newDetails)) require.NoError(t, exp.verify(job.ID(), jobs.StatusRunning)) - require.NoError(t, job.SetDetails(ctx, nil /* txn */, newDetails)) + require.NoError(t, job.SetDetails(ctx, nil /* txn */, nil /* ie */, newDetails)) // Now change job's session id and check that updates are rejected. _, err := exp.DB.Exec(updateClaimStmt, "!@#!@$!$@#", job.ID()) require.NoError(t, err) - require.Error(t, job.SetDetails(ctx, nil /* txn */, newDetails)) + require.Error(t, job.SetDetails(ctx, nil /* txn */, nil /* ie */, newDetails)) require.NoError(t, exp.verify(job.ID(), jobs.StatusRunning)) }) @@ -1733,7 +1738,7 @@ func TestJobLifecycle(t *testing.T) { require.NoError(t, exp.verify(job.ID(), jobs.StatusRunning)) _, err := exp.DB.Exec(updateStatusStmt, jobs.StatusCancelRequested, job.ID()) require.NoError(t, err) - require.Error(t, job.SetDetails(ctx, nil /* txn */, jobspb.ImportDetails{URIs: []string{"new"}})) + require.Error(t, job.SetDetails(ctx, nil /* txn */, nil /* ie */, jobspb.ImportDetails{URIs: []string{"new"}})) require.NoError(t, exp.verify(job.ID(), jobs.StatusCancelRequested)) }) @@ -1742,13 +1747,13 @@ func TestJobLifecycle(t *testing.T) { require.NoError(t, exp.verify(job.ID(), jobs.StatusRunning)) newProgress := jobspb.ImportProgress{ResumePos: []int64{42}} exp.Record.Progress = newProgress - require.NoError(t, job.SetProgress(ctx, nil /* txn */, newProgress)) + require.NoError(t, job.SetProgress(ctx, nil /* txn */, nil /* ie */, newProgress)) require.NoError(t, exp.verify(job.ID(), jobs.StatusRunning)) // Now change job's session id and check that updates are rejected. _, err := exp.DB.Exec(updateClaimStmt, "!@#!@$!$@#", job.ID()) require.NoError(t, err) - require.Error(t, job.SetDetails(ctx, nil /* txn */, newProgress)) + require.Error(t, job.SetDetails(ctx, nil /* txn */, nil /* ie */, newProgress)) require.NoError(t, exp.verify(job.ID(), jobs.StatusRunning)) }) @@ -1757,7 +1762,7 @@ func TestJobLifecycle(t *testing.T) { require.NoError(t, exp.verify(job.ID(), jobs.StatusRunning)) _, err := exp.DB.Exec(updateStatusStmt, jobs.StatusPauseRequested, job.ID()) require.NoError(t, err) - require.Error(t, job.SetProgress(ctx, nil /* txn */, jobspb.ImportProgress{ResumePos: []int64{42}})) + require.Error(t, job.SetProgress(ctx, nil /* txn */, nil /* ie */, jobspb.ImportProgress{ResumePos: []int64{42}})) require.NoError(t, exp.verify(job.ID(), jobs.StatusPauseRequested)) }) @@ -1781,7 +1786,7 @@ func TestJobLifecycle(t *testing.T) { Progress: jobspb.BackupProgress{}, CreatedBy: &jobs.CreatedByInfo{Name: createdByType, ID: 123}, } - job, err := registry.CreateAdoptableJobWithTxn(ctx, record, jobID, nil /* txn */) + job, err := registry.CreateAdoptableJobWithTxn(ctx, record, jobID, nil /* txn */, nil /* ie */) require.NoError(t, err) loadedJob, err := registry.LoadJob(ctx, jobID) @@ -2637,7 +2642,7 @@ func TestStartableJob(t *testing.T) { require.NoError(t, err) require.NoError(t, txn.Commit(ctx)) require.NoError(t, sj.Cancel(ctx)) - status, err := sj.TestingCurrentStatus(ctx, nil /* txn */) + status, err := sj.TestingCurrentStatus(ctx, nil /* txn */, nil /* ie */) require.NoError(t, err) require.Equal(t, jobs.StatusCancelRequested, status) // Start should fail since we have already called cancel on the job. @@ -2711,7 +2716,7 @@ func TestStartableJob(t *testing.T) { testutils.SucceedsSoon(t, func() error { loaded, err := jr.LoadJob(ctx, sj.ID()) require.NoError(t, err) - st, err := loaded.TestingCurrentStatus(ctx, nil /* txn */) + st, err := loaded.TestingCurrentStatus(ctx, nil /* txn */, nil /* ie */) require.NoError(t, err) if st != jobs.StatusSucceeded { return errors.Errorf("expected %s, got %s", jobs.StatusSucceeded, st) @@ -2801,7 +2806,7 @@ func TestRegistryTestingNudgeAdoptionQueue(t *testing.T) { }, jobs.UsesTenantCostControl) before := timeutil.Now() jobID := registry.MakeJobID() - _, err := registry.CreateAdoptableJobWithTxn(ctx, rec, jobID, nil /* txn */) + _, err := registry.CreateAdoptableJobWithTxn(ctx, rec, jobID, nil /* txn */, nil /* ie */) require.NoError(t, err) registry.TestingNudgeAdoptionQueue() // We want the job to be resumed very rapidly. We set this long timeout of 2s @@ -2894,7 +2899,7 @@ func TestMetrics(t *testing.T) { Details: jobspb.BackupDetails{}, Progress: jobspb.BackupProgress{}, } - _, err := registry.CreateAdoptableJobWithTxn(ctx, rec, registry.MakeJobID(), nil /* txn */) + _, err := registry.CreateAdoptableJobWithTxn(ctx, rec, registry.MakeJobID(), nil /* txn */, nil /* ie */) require.NoError(t, err) errCh := <-resuming backupMetrics := registry.MetricsStruct().JobMetrics[jobspb.TypeBackup] @@ -2913,7 +2918,7 @@ func TestMetrics(t *testing.T) { importMetrics := registry.MetricsStruct().JobMetrics[jobspb.TypeImport] jobID := registry.MakeJobID() - _, err := registry.CreateAdoptableJobWithTxn(ctx, rec, jobID, nil /* txn */) + _, err := registry.CreateAdoptableJobWithTxn(ctx, rec, jobID, nil /* txn */, nil /* ie */) require.NoError(t, err) { // Fail the Resume with a retriable error. @@ -2928,7 +2933,7 @@ func TestMetrics(t *testing.T) { // We'll pause the job this time around and make sure it stops running. <-resuming require.Equal(t, int64(1), importMetrics.CurrentlyRunning.Value()) - require.NoError(t, registry.PauseRequested(ctx, nil, jobID, "for testing")) + require.NoError(t, registry.PauseRequested(ctx, nil /* txn */, nil /* ie */, jobID, "for testing")) int64EqSoon(t, importMetrics.ResumeRetryError.Count, 2) require.Equal(t, int64(0), importMetrics.ResumeFailed.Count()) require.Equal(t, int64(0), importMetrics.ResumeCompleted.Count()) @@ -2951,7 +2956,7 @@ func TestMetrics(t *testing.T) { } { // Now resume the job and let it succeed. - require.NoError(t, registry.Unpause(ctx, nil, jobID)) + require.NoError(t, registry.Unpause(ctx, nil /* txn */, nil /* ie */, jobID)) errCh := <-resuming require.Equal(t, int64(1), importMetrics.CurrentlyRunning.Value()) errCh <- nil @@ -2968,7 +2973,7 @@ func TestMetrics(t *testing.T) { } importMetrics := registry.MetricsStruct().JobMetrics[jobspb.TypeImport] - _, err := registry.CreateAdoptableJobWithTxn(ctx, rec, registry.MakeJobID(), nil /* txn */) + _, err := registry.CreateAdoptableJobWithTxn(ctx, rec, registry.MakeJobID(), nil /* txn */, nil /* ie */) require.NoError(t, err) { // Fail the Resume with a permanent error. @@ -3004,7 +3009,7 @@ func TestMetrics(t *testing.T) { importMetrics := registry.MetricsStruct().JobMetrics[jobspb.TypeImport] jobID := registry.MakeJobID() - _, err := registry.CreateAdoptableJobWithTxn(ctx, rec, jobID, nil /* txn */) + _, err := registry.CreateAdoptableJobWithTxn(ctx, rec, jobID, nil /* txn */, nil /* ie */) require.NoError(t, err) { // Fail the Resume with a retriable error. @@ -3019,7 +3024,7 @@ func TestMetrics(t *testing.T) { // We'll pause the job this time around and make sure it stops running. <-resuming require.Equal(t, int64(1), importMetrics.CurrentlyRunning.Value()) - require.NoError(t, registry.PauseRequested(ctx, nil, jobID, "")) + require.NoError(t, registry.PauseRequested(ctx, nil /* txn */, nil /* ie */, jobID, "")) int64EqSoon(t, importMetrics.FailOrCancelRetryError.Count, 1) require.Equal(t, int64(1), importMetrics.ResumeFailed.Count()) require.Equal(t, int64(0), importMetrics.ResumeCompleted.Count()) @@ -3033,7 +3038,7 @@ func TestMetrics(t *testing.T) { } { // Now resume the job and let it succeed. - require.NoError(t, registry.Unpause(ctx, nil, jobID)) + require.NoError(t, registry.Unpause(ctx, nil /* txn */, nil /* ie */, jobID)) errCh := <-resuming require.Equal(t, int64(1), importMetrics.CurrentlyRunning.Value()) errCh <- nil @@ -3076,12 +3081,13 @@ func TestLoseLeaseDuringExecution(t *testing.T) { return jobs.FakeResumer{ OnResume: func(ctx context.Context) error { defer close(resumed) - _, err := s.InternalExecutor().(sqlutil.InternalExecutor).Exec( + ie := s.InternalExecutorFactory().(sqlutil.InternalExecutorFactory).MakeInternalExecutorWithoutTxn() + _, err := ie.Exec( ctx, "set-claim-null", nil, /* txn */ `UPDATE system.jobs SET claim_session_id = NULL WHERE id = $1`, j.ID()) assert.NoError(t, err) - err = j.Update(ctx, nil /* txn */, func(txn *kv.Txn, md jobs.JobMetadata, ju *jobs.JobUpdater) error { + err = j.Update(ctx, nil /* txn */, nil /* ie */, func(_ *kv.Txn, _ sqlutil.InternalExecutor, _ jobs.JobMetadata, _ *jobs.JobUpdater) error { return nil }) resumed <- err @@ -3183,7 +3189,7 @@ func TestPauseReason(t *testing.T) { tdb := sqlutils.MakeSQLRunner(db) jobID := registry.MakeJobID() - _, err := registry.CreateAdoptableJobWithTxn(ctx, rec, jobID, nil /* txn */) + _, err := registry.CreateAdoptableJobWithTxn(ctx, rec, jobID, nil /* txn */, nil /* ie */) require.NoError(t, err) countRowsWithClaimInfo := func() int { @@ -3239,7 +3245,7 @@ func TestPauseReason(t *testing.T) { { // Next, pause the job with a reason. Wait for pause and make sure the pause reason is set. - require.NoError(t, registry.PauseRequested(ctx, nil, jobID, "for testing")) + require.NoError(t, registry.PauseRequested(ctx, nil /* txn */, nil /* ie */, jobID, "for testing")) tdb.CheckQueryResultsRetry(t, q, [][]string{{"paused"}}) checkStatusAndPauseReason(t, jobID, "paused", "for testing") mustNotHaveClaim() @@ -3247,7 +3253,7 @@ func TestPauseReason(t *testing.T) { { // Now resume the job. Verify that the job is running now, but the pause reason is still there. - require.NoError(t, registry.Unpause(ctx, nil, jobID)) + require.NoError(t, registry.Unpause(ctx, nil /* txn */, nil /* ie */, jobID)) tdb.CheckQueryResultsRetry(t, q, [][]string{{"running"}}) checkStatusAndPauseReason(t, jobID, "running", "for testing") @@ -3256,7 +3262,7 @@ func TestPauseReason(t *testing.T) { } { // Pause the job again with a different reason. Verify that the job is paused with the reason. - require.NoError(t, registry.PauseRequested(ctx, nil, jobID, "second time")) + require.NoError(t, registry.PauseRequested(ctx, nil /* txn */, nil /* ie */, jobID, "second time")) tdb.CheckQueryResultsRetry(t, q, [][]string{{"paused"}}) checkStatusAndPauseReason(t, jobID, "paused", "second time") mustNotHaveClaim() @@ -3457,7 +3463,7 @@ func TestPausepoints(t *testing.T) { } else { require.Error(t, sj.AwaitCompletion(ctx)) } - status, err := sj.TestingCurrentStatus(ctx, nil) + status, err := sj.TestingCurrentStatus(ctx, nil /* txn */, nil /* ie */) // Map pause-requested to paused to avoid races. if status == jobs.StatusPauseRequested { status = jobs.StatusPaused diff --git a/pkg/jobs/jobsprotectedts/jobs_protected_ts.go b/pkg/jobs/jobsprotectedts/jobs_protected_ts.go index 5da0493f0651..3afb8a03a7a6 100644 --- a/pkg/jobs/jobsprotectedts/jobs_protected_ts.go +++ b/pkg/jobs/jobsprotectedts/jobs_protected_ts.go @@ -52,28 +52,26 @@ func GetMetaType(metaType MetaType) string { // MakeStatusFunc returns a function which determines whether the job or // schedule implied with this value of meta should be removed by the reconciler. -func MakeStatusFunc( - jr *jobs.Registry, ie sqlutil.InternalExecutor, metaType MetaType, -) ptreconcile.StatusFunc { +func MakeStatusFunc(jr *jobs.Registry, metaType MetaType) ptreconcile.StatusFunc { switch metaType { case Jobs: - return func(ctx context.Context, txn *kv.Txn, meta []byte) (shouldRemove bool, _ error) { + return func(ctx context.Context, txn *kv.Txn, ie sqlutil.InternalExecutor, meta []byte) (shouldRemove bool, _ error) { jobID, err := decodeID(meta) if err != nil { return false, err } - j, err := jr.LoadJobWithTxn(ctx, jobspb.JobID(jobID), txn) + j, err := jr.LoadJobWithTxn(ctx, jobspb.JobID(jobID), txn, ie) if jobs.HasJobNotFoundError(err) { return true, nil } if err != nil { return false, err } - isTerminal := j.CheckTerminalStatus(ctx, txn) + isTerminal := j.CheckTerminalStatus(ctx, txn, ie) return isTerminal, nil } case Schedules: - return func(ctx context.Context, txn *kv.Txn, meta []byte) (shouldRemove bool, _ error) { + return func(ctx context.Context, txn *kv.Txn, ie sqlutil.InternalExecutor, meta []byte) (shouldRemove bool, _ error) { scheduleID, err := decodeID(meta) if err != nil { return false, err diff --git a/pkg/jobs/jobsprotectedts/jobs_protected_ts_manager.go b/pkg/jobs/jobsprotectedts/jobs_protected_ts_manager.go index 42889f0e5b97..846c812440c5 100644 --- a/pkg/jobs/jobsprotectedts/jobs_protected_ts_manager.go +++ b/pkg/jobs/jobsprotectedts/jobs_protected_ts_manager.go @@ -23,6 +23,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/kv/kvserver/protectedts/ptpb" "github.com/cockroachdb/cockroach/pkg/sql/catalog" "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" + "github.com/cockroachdb/cockroach/pkg/sql/sqlutil" "github.com/cockroachdb/cockroach/pkg/util/ctxgroup" "github.com/cockroachdb/cockroach/pkg/util/hlc" "github.com/cockroachdb/cockroach/pkg/util/uuid" @@ -38,6 +39,7 @@ const timedProtectTimeStampGCPct = 0.8 // is hit. type Manager struct { db *kv.DB + ief sqlutil.InternalExecutorFactory codec keys.SQLCodec protectedTSProvider protectedts.Provider systemConfig config.SystemConfigProvider @@ -76,6 +78,7 @@ func getProtectedTSOnJob(details jobspb.Details) *uuid.UUID { // for jobs. func NewManager( db *kv.DB, + ief sqlutil.InternalExecutorFactory, codec keys.SQLCodec, protectedTSProvider protectedts.Provider, systemConfig config.SystemConfigProvider, @@ -83,6 +86,7 @@ func NewManager( ) *Manager { return &Manager{ db: db, + ief: ief, codec: codec, protectedTSProvider: protectedTSProvider, systemConfig: systemConfig, @@ -168,8 +172,8 @@ func (p *Manager) Protect( return nil, nil } var protectedtsID *uuid.UUID - err := p.db.Txn(ctx, func(ctx context.Context, txn *kv.Txn) error { - job, err := p.jr.LoadJobWithTxn(ctx, jobID, txn) + err := p.ief.TxnWithExecutor(ctx, p.db, nil /* sessionData */, func(ctx context.Context, txn *kv.Txn, ie sqlutil.InternalExecutor) error { + job, err := p.jr.LoadJobWithTxn(ctx, jobID, txn, ie) if err != nil { return err } @@ -181,7 +185,7 @@ func (p *Manager) Protect( newID := uuid.MakeV4() protectedtsID = &newID // Set up a new protected timestamp ID and install it on the job. - return job.Update(ctx, txn, func(txn *kv.Txn, md jobs.JobMetadata, ju *jobs.JobUpdater) error { + return job.Update(ctx, txn, ie, func(txn *kv.Txn, ie sqlutil.InternalExecutor, md jobs.JobMetadata, ju *jobs.JobUpdater) error { details = job.Details() details = setProtectedTSOnJob(details, protectedtsID) md.Payload.Details = jobspb.WrapPayloadDetails(details) @@ -190,11 +194,11 @@ func (p *Manager) Protect( target := ptpb.MakeSchemaObjectsTarget(descpb.IDs{tableDesc.GetID()}) rec := MakeRecord(*protectedtsID, int64(jobID), readAsOf, nil, Jobs, target) - return p.protectedTSProvider.Protect(ctx, txn, rec) + return p.protectedTSProvider.Protect(ctx, txn, ie, rec) }) } // Refresh the existing timestamp. - return p.protectedTSProvider.UpdateTimestamp(ctx, txn, *protectedtsID, readAsOf) + return p.protectedTSProvider.UpdateTimestamp(ctx, txn, ie, *protectedtsID, readAsOf) }) if err != nil { return nil, err @@ -208,8 +212,10 @@ func (p *Manager) Protect( // Note: This should only be used for job cleanup if is not currently, // executing. func (p *Manager) Unprotect(ctx context.Context, jobID jobspb.JobID) error { - return p.db.Txn(ctx, func(ctx context.Context, txn *kv.Txn) error { - job, err := p.jr.LoadJobWithTxn(ctx, jobID, txn) + return p.ief.TxnWithExecutor(ctx, p.db, nil /* session data */, func( + ctx context.Context, txn *kv.Txn, ie sqlutil.InternalExecutor, + ) error { + job, err := p.jr.LoadJobWithTxn(ctx, jobID, txn, ie) if err != nil { return err } @@ -221,12 +227,12 @@ func (p *Manager) Unprotect(ctx context.Context, jobID jobspb.JobID) error { } // If we do find one then we need to clean up the protected timestamp, // and remove it from the job. - return job.Update(ctx, txn, func(txn *kv.Txn, md jobs.JobMetadata, ju *jobs.JobUpdater) error { + return job.Update(ctx, txn, ie, func(txn *kv.Txn, ie sqlutil.InternalExecutor, md jobs.JobMetadata, ju *jobs.JobUpdater) error { details = md.Payload.UnwrapDetails() details = setProtectedTSOnJob(details, nil) md.Payload.Details = jobspb.WrapPayloadDetails(details) ju.UpdatePayload(md.Payload) - return p.protectedTSProvider.Release(ctx, txn, *protectedtsID) + return p.protectedTSProvider.Release(ctx, txn, ie, *protectedtsID) }) }) } diff --git a/pkg/jobs/progress.go b/pkg/jobs/progress.go index eda09c5c8639..5deda3ffd7ed 100644 --- a/pkg/jobs/progress.go +++ b/pkg/jobs/progress.go @@ -76,7 +76,7 @@ func NewChunkProgressLogger( completed: startFraction, reported: startFraction, Report: func(ctx context.Context, pct float32) error { - return j.FractionProgressed(ctx, nil /* txn */, func(ctx context.Context, details jobspb.ProgressDetails) float32 { + return j.FractionProgressed(ctx, nil /* txn */, nil /* ie */, func(ctx context.Context, details jobspb.ProgressDetails) float32 { if progressedFn != nil { progressedFn(ctx, details) } diff --git a/pkg/jobs/registry.go b/pkg/jobs/registry.go index ce7d1a8e3e41..387617d32fae 100644 --- a/pkg/jobs/registry.go +++ b/pkg/jobs/registry.go @@ -499,20 +499,20 @@ VALUES ($1, $2, $3, $4, $5, $6)`, jobID, StatusRunning, payloadBytes, progressBy // CreateAdoptableJobWithTxn creates a job which will be adopted for execution // at a later time by some node in the cluster. func (r *Registry) CreateAdoptableJobWithTxn( - ctx context.Context, record Record, jobID jobspb.JobID, txn *kv.Txn, + ctx context.Context, record Record, jobID jobspb.JobID, txn *kv.Txn, ie sqlutil.InternalExecutor, ) (*Job, error) { // TODO(sajjad): Clean up the interface - remove jobID from the params as // Record now has JobID field. record.JobID = jobID j := r.newJob(ctx, record) - if err := j.runInTxn(ctx, txn, func(ctx context.Context, txn *kv.Txn) error { + if err := j.runInTxn(ctx, txn, ie, func(ctx context.Context, txn1 *kv.Txn, executor sqlutil.InternalExecutor) error { // Note: although the following uses ReadTimestamp and // ReadTimestamp can diverge from the value of now() throughout a // transaction, this may be OK -- we merely required ModifiedMicro // to be equal *or greater* than previously inserted timestamps // computed by now(). For now ReadTimestamp can only move forward // and the assertion ReadTimestamp >= now() holds at all times. - j.mu.progress.ModifiedMicros = timeutil.ToUnixMicros(txn.ReadTimestamp().GoTime()) + j.mu.progress.ModifiedMicros = timeutil.ToUnixMicros(txn1.ReadTimestamp().GoTime()) payloadBytes, err := protoutil.Marshal(&j.mu.payload) if err != nil { return err @@ -541,7 +541,7 @@ func (r *Registry) CreateAdoptableJobWithTxn( created_by_id ) VALUES ($1, $2, $3, $4, $5, $6);` - _, err = j.registry.ex.Exec(ctx, "job-insert", txn, stmt, + _, err = executor.ExecEx(ctx, "job-insert", txn1, sessiondata.InternalExecutorOverride{User: username.RootUserName()}, stmt, jobID, StatusRunning, payloadBytes, progressBytes, createdByType, createdByID) return err }); err != nil { @@ -633,7 +633,7 @@ func (r *Registry) CreateStartableJobWithTxn( // // TODO(ssd): Remove this API and replace it with a safer API. func (r *Registry) LoadJob(ctx context.Context, jobID jobspb.JobID) (*Job, error) { - return r.LoadJobWithTxn(ctx, jobID, nil) + return r.LoadJobWithTxn(ctx, jobID, nil /* txn */, nil /* ie */) } // LoadClaimedJob loads an existing job with the given jobID from the @@ -644,7 +644,7 @@ func (r *Registry) LoadClaimedJob(ctx context.Context, jobID jobspb.JobID) (*Job if err != nil { return nil, err } - if err := j.load(ctx, nil); err != nil { + if err := j.load(ctx, nil /* txn */, nil /* ie */); err != nil { return nil, err } return j, nil @@ -654,13 +654,13 @@ func (r *Registry) LoadClaimedJob(ctx context.Context, jobID jobspb.JobID) (*Job // the txn argument. Passing a nil transaction is equivalent to calling LoadJob // in that a transaction will be automatically created. func (r *Registry) LoadJobWithTxn( - ctx context.Context, jobID jobspb.JobID, txn *kv.Txn, + ctx context.Context, jobID jobspb.JobID, txn *kv.Txn, ie sqlutil.InternalExecutor, ) (*Job, error) { j := &Job{ id: jobID, registry: r, } - if err := j.load(ctx, txn); err != nil { + if err := j.load(ctx, txn, ie); err != nil { return nil, err } return j, nil @@ -675,13 +675,18 @@ func (r *Registry) LoadJobWithTxn( // and may do extra work and thus should not do locking. Cases where the job // is used to coordinate resources from multiple nodes may benefit from locking. func (r *Registry) UpdateJobWithTxn( - ctx context.Context, jobID jobspb.JobID, txn *kv.Txn, useReadLock bool, updateFunc UpdateFn, + ctx context.Context, + jobID jobspb.JobID, + txn *kv.Txn, + ie sqlutil.InternalExecutor, + useReadLock bool, + updateFunc UpdateFn, ) error { j := &Job{ id: jobID, registry: r, } - return j.update(ctx, txn, useReadLock, updateFunc) + return j.update(ctx, txn, ie, useReadLock, updateFunc) } // TODO (sajjad): make maxAdoptionsPerLoop a cluster setting. @@ -737,14 +742,14 @@ func (r *Registry) Start(ctx context.Context, stopper *stop.Stopper) error { // removeClaimsFromDeadSessions queries the jobs table for non-terminal // jobs and nullifies their claims if the claims are owned by known dead sessions. removeClaimsFromDeadSessions := func(ctx context.Context, s sqlliveness.Session) { - if err := r.db.Txn(ctx, func(ctx context.Context, txn *kv.Txn) error { + if err := r.internalExecutorFactory.TxnWithExecutor(ctx, r.db, nil /* sessionData */, func(ctx context.Context, txn *kv.Txn, ie sqlutil.InternalExecutor) error { // Run the expiration transaction at low priority to ensure that it does // not contend with foreground reads. Note that the adoption and cancellation // queries also use low priority so they will interact nicely. if err := txn.SetUserPriority(roachpb.MinUserPriority); err != nil { return errors.WithAssertionFailure(err) } - _, err := r.ex.ExecEx( + _, err := ie.ExecEx( ctx, "expire-sessions", txn, sessiondata.InternalExecutorOverride{User: username.RootUserName()}, removeClaimsForDeadSessionsQuery, @@ -787,14 +792,14 @@ func (r *Registry) Start(ctx context.Context, stopper *stop.Stopper) error { // removeClaimsFromJobs queries the jobs table for non-terminal jobs and // nullifies their claims if the claims are owned by the current session. removeClaimsFromSession := func(ctx context.Context, s sqlliveness.Session) { - if err := r.db.Txn(ctx, func(ctx context.Context, txn *kv.Txn) error { + if err := r.internalExecutorFactory.TxnWithExecutor(ctx, r.db, nil /* sessionData */, func(ctx context.Context, txn *kv.Txn, ie sqlutil.InternalExecutor) error { // Run the expiration transaction at low priority to ensure that it does // not contend with foreground reads. Note that the adoption and cancellation // queries also use low priority so they will interact nicely. if err := txn.SetUserPriority(roachpb.MinUserPriority); err != nil { return errors.WithAssertionFailure(err) } - _, err := r.ex.ExecEx( + _, err := ie.ExecEx( ctx, "remove-claims-for-session", txn, sessiondata.InternalExecutorOverride{User: username.RootUserName()}, removeClaimsForSessionQuery, s.ID().UnsafeBytes(), @@ -820,7 +825,8 @@ func (r *Registry) Start(ctx context.Context, stopper *stop.Stopper) error { r.cancelAllAdoptedJobs() return } - if err := r.processClaimedJobs(ctx, s); err != nil { + ieNotBoundToTxn := r.internalExecutorFactory.MakeInternalExecutorWithoutTxn() + if err := r.processClaimedJobs(ctx, s, ieNotBoundToTxn); err != nil { log.Errorf(ctx, "error processing claimed jobs: %s", err) } }) @@ -948,7 +954,8 @@ const expiredJobsQuery = "SELECT id, payload, status, created FROM system.jobs " func (r *Registry) cleanupOldJobsPage( ctx context.Context, olderThan time.Time, minID jobspb.JobID, pageSize int, ) (done bool, maxID jobspb.JobID, retErr error) { - it, err := r.ex.QueryIterator(ctx, "gc-jobs", nil /* txn */, expiredJobsQuery, olderThan, minID, pageSize) + ieNotBoundToTxn := r.internalExecutorFactory.MakeInternalExecutorWithoutTxn() + it, err := ieNotBoundToTxn.QueryIterator(ctx, "gc-jobs", nil /* txn */, expiredJobsQuery, olderThan, minID, pageSize) if err != nil { return false, 0, err } @@ -988,7 +995,7 @@ func (r *Registry) cleanupOldJobsPage( log.VEventf(ctx, 2, "attempting to clean up %d expired job records", len(toDelete.Array)) const stmt = `DELETE FROM system.jobs WHERE id = ANY($1)` var nDeleted int - if nDeleted, err = r.ex.Exec( + if nDeleted, err = ieNotBoundToTxn.Exec( ctx, "gc-jobs", nil /* txn */, stmt, toDelete, ); err != nil { log.Warningf(ctx, "error cleaning up %d jobs: %v", len(toDelete.Array), err) @@ -1010,9 +1017,9 @@ func (r *Registry) cleanupOldJobsPage( // getJobFn attempts to get a resumer from the given job id. If the job id // does not have a resumer then it returns an error message suitable for users. func (r *Registry) getJobFn( - ctx context.Context, txn *kv.Txn, id jobspb.JobID, + ctx context.Context, txn *kv.Txn, ie sqlutil.InternalExecutor, id jobspb.JobID, ) (*Job, Resumer, error) { - job, err := r.LoadJobWithTxn(ctx, id, txn) + job, err := r.LoadJobWithTxn(ctx, id, txn, ie) if err != nil { return nil, nil, err } @@ -1024,8 +1031,10 @@ func (r *Registry) getJobFn( } // CancelRequested marks the job as cancel-requested using the specified txn (may be nil). -func (r *Registry) CancelRequested(ctx context.Context, txn *kv.Txn, id jobspb.JobID) error { - job, _, err := r.getJobFn(ctx, txn, id) +func (r *Registry) CancelRequested( + ctx context.Context, txn *kv.Txn, ie sqlutil.InternalExecutor, id jobspb.JobID, +) error { + job, _, err := r.getJobFn(ctx, txn, ie, id) if err != nil { // Special case schema change jobs to mark the job as canceled. if job != nil { @@ -1040,19 +1049,19 @@ func (r *Registry) CancelRequested(ctx context.Context, txn *kv.Txn, id jobspb.J // safest way for now (i.e., without a larger jobs/schema change refactor) // is to hack this up with a string comparison. if payload.Type() == jobspb.TypeSchemaChange && !strings.HasPrefix(payload.Description, "ROLL BACK") { - return job.cancelRequested(ctx, txn, nil) + return job.cancelRequested(ctx, txn, ie, nil /* fn */) } } return err } - return job.cancelRequested(ctx, txn, nil) + return job.cancelRequested(ctx, txn, ie, nil /* fn */) } // PauseRequested marks the job with id as paused-requested using the specified txn (may be nil). func (r *Registry) PauseRequested( - ctx context.Context, txn *kv.Txn, id jobspb.JobID, reason string, + ctx context.Context, txn *kv.Txn, ie sqlutil.InternalExecutor, id jobspb.JobID, reason string, ) error { - job, resumer, err := r.getJobFn(ctx, txn, id) + job, resumer, err := r.getJobFn(ctx, txn, ie, id) if err != nil { return err } @@ -1060,37 +1069,45 @@ func (r *Registry) PauseRequested( if pr, ok := resumer.(PauseRequester); ok { onPauseRequested = pr.OnPauseRequest } - return job.PauseRequested(ctx, txn, onPauseRequested, reason) + return job.PauseRequested(ctx, txn, ie, onPauseRequested, reason) } // Succeeded marks the job with id as succeeded. -func (r *Registry) Succeeded(ctx context.Context, txn *kv.Txn, id jobspb.JobID) error { - job, _, err := r.getJobFn(ctx, txn, id) +func (r *Registry) Succeeded( + ctx context.Context, txn *kv.Txn, ie sqlutil.InternalExecutor, id jobspb.JobID, +) error { + job, _, err := r.getJobFn(ctx, txn, ie, id) if err != nil { return err } - return job.succeeded(ctx, txn, nil) + return job.succeeded(ctx, txn, ie, nil) } // Failed marks the job with id as failed. func (r *Registry) Failed( - ctx context.Context, txn *kv.Txn, id jobspb.JobID, causingError error, + ctx context.Context, + txn *kv.Txn, + id jobspb.JobID, + ie sqlutil.InternalExecutor, + causingError error, ) error { - job, _, err := r.getJobFn(ctx, txn, id) + job, _, err := r.getJobFn(ctx, txn, ie, id) if err != nil { return err } - return job.failed(ctx, txn, causingError, nil) + return job.failed(ctx, txn, ie, causingError, nil /* fn */) } // Unpause changes the paused job with id to running or reverting using the // specified txn (may be nil). -func (r *Registry) Unpause(ctx context.Context, txn *kv.Txn, id jobspb.JobID) error { - job, _, err := r.getJobFn(ctx, txn, id) +func (r *Registry) Unpause( + ctx context.Context, txn *kv.Txn, ie sqlutil.InternalExecutor, id jobspb.JobID, +) error { + job, _, err := r.getJobFn(ctx, txn, ie, id) if err != nil { return err } - return job.unpaused(ctx, txn) + return job.unpaused(ctx, txn, ie) } // Resumer is a resumable job, and is associated with a Job object. Jobs can be @@ -1182,7 +1199,7 @@ type PauseRequester interface { // OnPauseRequest is called in the transaction that moves a job to PauseRequested. // If an error is returned, the pause request will fail. execCtx is a // sql.JobExecCtx. - OnPauseRequest(ctx context.Context, execCtx interface{}, txn *kv.Txn, details *jobspb.Progress) error + OnPauseRequest(ctx context.Context, execCtx interface{}, txn *kv.Txn, ie sqlutil.InternalExecutor, details *jobspb.Progress) error } // JobResultsReporter is an interface for reporting the results of the job execution. @@ -1274,7 +1291,7 @@ func (r *Registry) stepThroughStateMachine( resumeCtx := logtags.AddTag(ctx, "job", job.ID()) labels := pprof.Labels("job", fmt.Sprintf("%s id=%d", jobType, job.ID())) - if err := job.started(ctx, nil /* txn */); err != nil { + if err := job.started(ctx, nil /* txn */, nil /* ie */); err != nil { return err } @@ -1308,7 +1325,7 @@ func (r *Registry) stepThroughStateMachine( } if errors.Is(err, errPauseSelfSentinel) { - if err := r.PauseRequested(ctx, nil, job.ID(), err.Error()); err != nil { + if err := r.PauseRequested(ctx, nil /* txn */, nil /* ie */, job.ID(), err.Error()); err != nil { return err } return errors.Wrap(err, PauseRequestExplained) @@ -1341,7 +1358,7 @@ func (r *Registry) stepThroughStateMachine( return errors.NewAssertionErrorWithWrappedErrf(jobErr, "job %d: unexpected status %s provided to state machine", job.ID(), status) case StatusCanceled: - if err := job.canceled(ctx, nil /* txn */, nil /* fn */); err != nil { + if err := job.canceled(ctx, nil /* txn */, nil /* ie */, nil /* fn */); err != nil { // If we can't transactionally mark the job as canceled then it will be // restarted during the next adopt loop and reverting will be retried. return errors.WithSecondaryError( @@ -1357,7 +1374,7 @@ func (r *Registry) stepThroughStateMachine( return errors.NewAssertionErrorWithWrappedErrf(jobErr, "job %d: successful but unexpected error provided", job.ID()) } - err := job.succeeded(ctx, nil /* txn */, nil /* fn */) + err := job.succeeded(ctx, nil /* txn */, nil /* ie */, nil /* fn */) switch { case err == nil: telemetry.Inc(TelemetryMetrics[jobType].Successful) @@ -1369,7 +1386,7 @@ func (r *Registry) stepThroughStateMachine( } return err case StatusReverting: - if err := job.reverted(ctx, nil /* txn */, jobErr, nil /* fn */); err != nil { + if err := job.reverted(ctx, nil /* txn */, nil /* ie */, jobErr, nil /* fn */); err != nil { // If we can't transactionally mark the job as reverting then it will be // restarted during the next adopt loop and it will be retried. return errors.WithSecondaryError( @@ -1409,7 +1426,7 @@ func (r *Registry) stepThroughStateMachine( if jobErr == nil { return errors.AssertionFailedf("job %d: has StatusFailed but no error was provided", job.ID()) } - if err := job.failed(ctx, nil /* txn */, jobErr, nil /* fn */); err != nil { + if err := job.failed(ctx, nil /* txn */, nil /* ie */, jobErr, nil /* fn */); err != nil { // If we can't transactionally mark the job as failed then it will be // restarted during the next adopt loop and reverting will be retried. return errors.WithSecondaryError( @@ -1428,7 +1445,7 @@ func (r *Registry) stepThroughStateMachine( return errors.AssertionFailedf("job %d: has StatusRevertFailed but no error was provided", job.ID()) } - if err := job.revertFailed(ctx, nil /* txn */, jobErr, nil /* fn */); err != nil { + if err := job.revertFailed(ctx, nil /* txn */, nil /* ie */, jobErr, nil /* fn */); err != nil { // If we can't transactionally mark the job as failed then it will be // restarted during the next adopt loop and reverting will be retried. return errors.WithSecondaryError( @@ -1553,8 +1570,8 @@ func (r *Registry) maybeRecordExecutionFailure(ctx context.Context, err error, j return } - updateErr := j.Update(ctx, nil, func( - txn *kv.Txn, md JobMetadata, ju *JobUpdater, + updateErr := j.Update(ctx, nil /* txn */, nil /* ie */, func( + _ *kv.Txn, _ sqlutil.InternalExecutor, md JobMetadata, ju *JobUpdater, ) error { pl := md.Payload { // Append the entry to the log diff --git a/pkg/jobs/registry_external_test.go b/pkg/jobs/registry_external_test.go index 7cafb27eee10..aad26a1653ee 100644 --- a/pkg/jobs/registry_external_test.go +++ b/pkg/jobs/registry_external_test.go @@ -65,7 +65,7 @@ func TestRoundtripJob(t *testing.T) { Details: jobspb.RestoreDetails{}, Progress: jobspb.RestoreProgress{}, } - storedJob, err := registry.CreateAdoptableJobWithTxn(ctx, record, jobID, nil /* txn */) + storedJob, err := registry.CreateAdoptableJobWithTxn(ctx, record, jobID, nil /* txn */, nil /* ie */) require.NoError(t, err) retrievedJob, err := registry.LoadJob(ctx, jobID) if err != nil { diff --git a/pkg/jobs/registry_test.go b/pkg/jobs/registry_test.go index d8aad354bd0b..0960be2fbce3 100644 --- a/pkg/jobs/registry_test.go +++ b/pkg/jobs/registry_test.go @@ -406,13 +406,13 @@ func TestRetriesWithExponentialBackoff(t *testing.T) { // pauseOrCancelJob pauses or cancels a job. If pauseJob is true, the job is paused, // otherwise the job is canceled. pauseOrCancelJob := func( - t *testing.T, ctx context.Context, db *kv.DB, registry *Registry, jobID jobspb.JobID, pauseJob bool, + t *testing.T, ctx context.Context, db *kv.DB, ief sqlutil.InternalExecutorFactory, registry *Registry, jobID jobspb.JobID, pauseJob bool, ) { - assert.NoError(t, db.Txn(ctx, func(ctx context.Context, txn *kv.Txn) error { + assert.NoError(t, ief.TxnWithExecutor(ctx, db, nil /* sessionData */, func(ctx context.Context, txn *kv.Txn, ie sqlutil.InternalExecutor) error { if pauseJob { - return registry.PauseRequested(ctx, txn, jobID, "") + return registry.PauseRequested(ctx, txn, ie, jobID, "") } - return registry.CancelRequested(ctx, txn, jobID) + return registry.CancelRequested(ctx, txn, ie, jobID) })) } // nextDelay returns the next delay based calculated from the given retryCnt @@ -615,16 +615,18 @@ func TestRetriesWithExponentialBackoff(t *testing.T) { cleanup := testInfraSetUp(ctx, &bti) defer cleanup() + ief := bti.s.InternalExecutorFactory().(sqlutil.InternalExecutorFactory) + jobID, lastRun := createJob(ctx, bti.s, bti.registry, bti.tdb, bti.kvDB) retryCnt := 0 expectedResumed := int64(0) runTest(t, jobID, retryCnt, expectedResumed, lastRun, &bti, func(_ int64) { <-bti.resumeCh - pauseOrCancelJob(t, ctx, bti.kvDB, bti.registry, jobID, pause) + pauseOrCancelJob(t, ctx, bti.kvDB, ief, bti.registry, jobID, pause) bti.errCh <- nil <-bti.transitionCh waitUntilStatus(t, bti.tdb, jobID, StatusPaused) - require.NoError(t, bti.registry.Unpause(ctx, nil, jobID)) + require.NoError(t, bti.registry.Unpause(ctx, nil /* txn */, nil /* ie */, jobID)) }) }) @@ -662,10 +664,12 @@ func TestRetriesWithExponentialBackoff(t *testing.T) { cleanup := testInfraSetUp(ctx, &bti) defer cleanup() + ief := bti.s.InternalExecutorFactory().(sqlutil.InternalExecutorFactory) + jobID, lastRun := createJob(ctx, bti.s, bti.registry, bti.tdb, bti.kvDB) bti.clock.AdvanceTo(lastRun) <-bti.resumeCh - pauseOrCancelJob(t, ctx, bti.kvDB, bti.registry, jobID, cancel) + pauseOrCancelJob(t, ctx, bti.kvDB, ief, bti.registry, jobID, cancel) bti.errCh <- nil <-bti.failOrCancelCh bti.errCh <- MarkAsRetryJobError(errors.New("injecting error in reverting state")) @@ -692,6 +696,8 @@ func TestRetriesWithExponentialBackoff(t *testing.T) { cleanup := testInfraSetUp(ctx, &bti) defer cleanup() + ief := bti.s.InternalExecutorFactory().(sqlutil.InternalExecutorFactory) + jobID, lastRun := createJob(ctx, bti.s, bti.registry, bti.tdb, bti.kvDB) bti.clock.AdvanceTo(lastRun) <-bti.resumeCh @@ -703,7 +709,7 @@ func TestRetriesWithExponentialBackoff(t *testing.T) { retryCnt := 1 runTest(t, jobID, retryCnt, expectedResumed, lastRun, &bti, func(_ int64) { <-bti.failOrCancelCh - pauseOrCancelJob(t, ctx, bti.kvDB, bti.registry, jobID, pause) + pauseOrCancelJob(t, ctx, bti.kvDB, ief, bti.registry, jobID, pause) // We have to return error here because, otherwise, the job will be marked as // failed regardless of the fact that it is currently pause-requested in the // jobs table. This is because we currently do not check the current status @@ -711,7 +717,7 @@ func TestRetriesWithExponentialBackoff(t *testing.T) { bti.errCh <- MarkAsRetryJobError(errors.New("injecting error in reverting state to retry")) <-bti.transitionCh waitUntilStatus(t, bti.tdb, jobID, StatusPaused) - require.NoError(t, bti.registry.Unpause(ctx, nil, jobID)) + require.NoError(t, bti.registry.Unpause(ctx, nil /* txn */, nil /* ie */, jobID)) }) }) } diff --git a/pkg/jobs/scheduled_job_executor.go b/pkg/jobs/scheduled_job_executor.go index 59066cc96e37..a70c6a21606d 100644 --- a/pkg/jobs/scheduled_job_executor.go +++ b/pkg/jobs/scheduled_job_executor.go @@ -27,13 +27,7 @@ import ( type ScheduledJobExecutor interface { // Executes scheduled job; Implementation may use provided transaction. // Modifications to the ScheduledJob object will be persisted. - ExecuteJob( - ctx context.Context, - cfg *scheduledjobs.JobExecutionConfig, - env scheduledjobs.JobSchedulerEnv, - schedule *ScheduledJob, - txn *kv.Txn, - ) error + ExecuteJob(ctx context.Context, cfg *scheduledjobs.JobExecutionConfig, env scheduledjobs.JobSchedulerEnv, schedule *ScheduledJob, txn *kv.Txn, ie sqlutil.InternalExecutor) error // Notifies that the system.job started by the ScheduledJob completed. // Implementation may use provided transaction to perform any additional mutations. @@ -71,14 +65,7 @@ type ScheduledJobController interface { // OnDrop runs before the passed in `schedule` is dropped as part of a `DROP // SCHEDULE` query. OnDrop may drop the schedule's dependent schedules and will // return the number of additional schedules it drops. - OnDrop( - ctx context.Context, - scheduleControllerEnv scheduledjobs.ScheduleControllerEnv, - env scheduledjobs.JobSchedulerEnv, - schedule *ScheduledJob, - txn *kv.Txn, - descsCol *descs.Collection, - ) (int, error) + OnDrop(ctx context.Context, scheduleControllerEnv scheduledjobs.ScheduleControllerEnv, env scheduledjobs.JobSchedulerEnv, schedule *ScheduledJob, txn *kv.Txn, descsCol *descs.Collection, ie sqlutil.InternalExecutor) (int, error) } // ScheduledJobExecutorFactory is a callback to create a ScheduledJobExecutor. diff --git a/pkg/jobs/scheduled_job_executor_test.go b/pkg/jobs/scheduled_job_executor_test.go index c2b585c39a28..64dce6a0c111 100644 --- a/pkg/jobs/scheduled_job_executor_test.go +++ b/pkg/jobs/scheduled_job_executor_test.go @@ -32,11 +32,12 @@ type statusTrackingExecutor struct { } func (s *statusTrackingExecutor) ExecuteJob( - _ context.Context, - _ *scheduledjobs.JobExecutionConfig, - _ scheduledjobs.JobSchedulerEnv, - _ *ScheduledJob, - _ *kv.Txn, + ctx context.Context, + cfg *scheduledjobs.JobExecutionConfig, + env scheduledjobs.JobSchedulerEnv, + schedule *ScheduledJob, + txn *kv.Txn, + ie sqlutil.InternalExecutor, ) error { s.numExec++ return nil diff --git a/pkg/jobs/testutils_test.go b/pkg/jobs/testutils_test.go index c02ad88490f5..521c42a421ed 100644 --- a/pkg/jobs/testutils_test.go +++ b/pkg/jobs/testutils_test.go @@ -96,10 +96,11 @@ func newTestHelperForTables( env: env, server: s, cfg: &scheduledjobs.JobExecutionConfig{ - Settings: s.ClusterSettings(), - InternalExecutor: s.InternalExecutor().(sqlutil.InternalExecutor), - DB: kvDB, - TestingKnobs: knobs, + Settings: s.ClusterSettings(), + InternalExecutor: s.InternalExecutor().(sqlutil.InternalExecutor), + DB: kvDB, + InternalExecutorFactory: s.InternalExecutorFactory().(sqlutil.InternalExecutorFactory), + TestingKnobs: knobs, }, sqlDB: sqlDB, execSchedules: execSchedules, diff --git a/pkg/jobs/update.go b/pkg/jobs/update.go index 575e5cae4e7b..bb07422abab9 100644 --- a/pkg/jobs/update.go +++ b/pkg/jobs/update.go @@ -23,6 +23,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/sessiondata" "github.com/cockroachdb/cockroach/pkg/sql/sqlliveness" + "github.com/cockroachdb/cockroach/pkg/sql/sqlutil" "github.com/cockroachdb/cockroach/pkg/util/hlc" "github.com/cockroachdb/cockroach/pkg/util/log" "github.com/cockroachdb/cockroach/pkg/util/protoutil" @@ -38,7 +39,7 @@ import ( // // The function is free to modify contents of JobMetadata in place (but the // changes will be ignored unless JobUpdater is used). -type UpdateFn func(txn *kv.Txn, md JobMetadata, ju *JobUpdater) error +type UpdateFn func(txn *kv.Txn, ie sqlutil.InternalExecutor, md JobMetadata, ju *JobUpdater) error // RunStats consists of job-run statistics: num of runs and last-run timestamp. type RunStats struct { @@ -135,12 +136,20 @@ func UpdateHighwaterProgressed(highWater hlc.Timestamp, md JobMetadata, ju *JobU // // Note that there are various convenience wrappers (like FractionProgressed) // defined in jobs.go. -func (j *Job) Update(ctx context.Context, txn *kv.Txn, updateFn UpdateFn) error { +func (j *Job) Update( + ctx context.Context, txn *kv.Txn, ie sqlutil.InternalExecutor, updateFn UpdateFn, +) error { const useReadLock = false - return j.update(ctx, txn, useReadLock, updateFn) + return j.update(ctx, txn, ie, useReadLock, updateFn) } -func (j *Job) update(ctx context.Context, txn *kv.Txn, useReadLock bool, updateFn UpdateFn) error { +func (j *Job) update( + ctx context.Context, + txn *kv.Txn, + ie sqlutil.InternalExecutor, + useReadLock bool, + updateFn UpdateFn, +) error { ctx, sp := tracing.ChildSpan(ctx, "update-job") defer sp.Finish() @@ -149,11 +158,11 @@ func (j *Job) update(ctx context.Context, txn *kv.Txn, useReadLock bool, updateF var status Status var runStats *RunStats - if err := j.runInTxn(ctx, txn, func(ctx context.Context, txn *kv.Txn) error { + if err := j.runInTxn(ctx, txn, ie, func(ctx context.Context, txn *kv.Txn, executor sqlutil.InternalExecutor) error { payload, progress, runStats = nil, nil, nil var err error var row tree.Datums - row, err = j.registry.ex.QueryRowEx( + row, err = executor.QueryRowEx( ctx, "select-job", txn, sessiondata.InternalExecutorOverride{User: username.RootUserName()}, getSelectStmtForJobUpdate(j.session != nil, useReadLock), j.ID(), @@ -218,7 +227,7 @@ func (j *Job) update(ctx context.Context, txn *kv.Txn, useReadLock bool, updateF } var ju JobUpdater - if err := updateFn(txn, md, &ju); err != nil { + if err := updateFn(txn, executor, md, &ju); err != nil { return err } if j.registry.knobs.BeforeUpdate != nil { diff --git a/pkg/kv/kvserver/client_protectedts_test.go b/pkg/kv/kvserver/client_protectedts_test.go index d14d6424d28f..c85facf32fa6 100644 --- a/pkg/kv/kvserver/client_protectedts_test.go +++ b/pkg/kv/kvserver/client_protectedts_test.go @@ -22,8 +22,8 @@ import ( "github.com/cockroachdb/cockroach/pkg/kv/kvserver/protectedts/ptstorage" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/protectedts/ptutil" "github.com/cockroachdb/cockroach/pkg/roachpb" - "github.com/cockroachdb/cockroach/pkg/sql" "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" + "github.com/cockroachdb/cockroach/pkg/sql/sqlutil" "github.com/cockroachdb/cockroach/pkg/testutils" "github.com/cockroachdb/cockroach/pkg/testutils/skip" "github.com/cockroachdb/cockroach/pkg/testutils/testcluster" @@ -61,6 +61,7 @@ func TestProtectedTimestamps(t *testing.T) { tc := testcluster.StartTestCluster(t, 3, args) defer tc.Stopper().Stop(ctx) s0 := tc.Server(0) + ief := s0.InternalExecutorFactory().(sqlutil.InternalExecutorFactory) conn := tc.ServerConn(0) _, err := conn.Exec("CREATE TABLE foo (k INT PRIMARY KEY, v BYTES)") @@ -180,16 +181,16 @@ func TestProtectedTimestamps(t *testing.T) { beforeWrites := s0.Clock().Now() gcSoon() - pts := ptstorage.New(s0.ClusterSettings(), s0.InternalExecutor().(*sql.InternalExecutor), - nil /* knobs */) - ptsWithDB := ptstorage.WithDatabase(pts, s0.DB()) + pts := ptstorage.New(s0.ClusterSettings(), nil /* knobs */) + ptsWithDB := ptstorage.WithDatabase(pts, s0.DB(), ief) ptsRec := ptpb.Record{ ID: uuid.MakeV4().GetBytes(), Timestamp: s0.Clock().Now(), Mode: ptpb.PROTECT_AFTER, Target: ptpb.MakeSchemaObjectsTarget([]descpb.ID{getTableID()}), } - require.NoError(t, ptsWithDB.Protect(ctx, nil /* txn */, &ptsRec)) + ieNotBoundToTxn := ief.MakeInternalExecutorWithoutTxn() + require.NoError(t, ptsWithDB.Protect(ctx, nil /* txn */, ieNotBoundToTxn, &ptsRec)) upsertUntilBackpressure() // We need to be careful choosing a time. We're a little limited because the // ttl is defined in seconds and we need to wait for the threshold to be @@ -227,8 +228,8 @@ func TestProtectedTimestamps(t *testing.T) { failedRec.ID = uuid.MakeV4().GetBytes() failedRec.Timestamp = beforeWrites failedRec.Timestamp.Logical = 0 - require.NoError(t, ptsWithDB.Protect(ctx, nil /* txn */, &failedRec)) - _, err = ptsWithDB.GetRecord(ctx, nil /* txn */, failedRec.ID.GetUUID()) + require.NoError(t, ptsWithDB.Protect(ctx, nil /* txn */, ieNotBoundToTxn, &failedRec)) + _, err = ptsWithDB.GetRecord(ctx, nil /* txn */, failedRec.ID.GetUUID(), ieNotBoundToTxn) require.NoError(t, err) // Verify that the record did indeed make its way down into KV where the @@ -246,7 +247,7 @@ func TestProtectedTimestamps(t *testing.T) { laterRec.ID = uuid.MakeV4().GetBytes() laterRec.Timestamp = afterWrites laterRec.Timestamp.Logical = 0 - require.NoError(t, ptsWithDB.Protect(ctx, nil /* txn */, &laterRec)) + require.NoError(t, ptsWithDB.Protect(ctx, nil /* txn */, ieNotBoundToTxn, &laterRec)) require.NoError( t, ptutil.TestingVerifyProtectionTimestampExistsOnSpans( @@ -256,7 +257,7 @@ func TestProtectedTimestamps(t *testing.T) { // Release the record that had succeeded and ensure that GC eventually // happens up to the protected timestamp of the new record. - require.NoError(t, ptsWithDB.Release(ctx, nil, ptsRec.ID.GetUUID())) + require.NoError(t, ptsWithDB.Release(ctx, nil /* txn */, ieNotBoundToTxn, ptsRec.ID.GetUUID())) testutils.SucceedsSoon(t, func() error { trace, _, err = s.Enqueue(ctx, "mvccGC", repl, false /* skipShouldQueue */, false /* async */) require.NoError(t, err) @@ -272,9 +273,9 @@ func TestProtectedTimestamps(t *testing.T) { }) // Release the failed record. - require.NoError(t, ptsWithDB.Release(ctx, nil, failedRec.ID.GetUUID())) - require.NoError(t, ptsWithDB.Release(ctx, nil, laterRec.ID.GetUUID())) - state, err := ptsWithDB.GetState(ctx, nil) + require.NoError(t, ptsWithDB.Release(ctx, nil /* txn */, ieNotBoundToTxn, failedRec.ID.GetUUID())) + require.NoError(t, ptsWithDB.Release(ctx, nil /* txn */, ieNotBoundToTxn, laterRec.ID.GetUUID())) + state, err := ptsWithDB.GetState(ctx, nil /* txn */, ieNotBoundToTxn) require.NoError(t, err) require.Len(t, state.Records, 0) require.Equal(t, int(state.NumRecords), len(state.Records)) diff --git a/pkg/kv/kvserver/client_replica_test.go b/pkg/kv/kvserver/client_replica_test.go index 13a9353e8baa..a6fa0fd2ad1d 100644 --- a/pkg/kv/kvserver/client_replica_test.go +++ b/pkg/kv/kvserver/client_replica_test.go @@ -46,6 +46,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql" "github.com/cockroachdb/cockroach/pkg/sql/catalog/bootstrap" "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" + "github.com/cockroachdb/cockroach/pkg/sql/sqlutil" "github.com/cockroachdb/cockroach/pkg/storage" "github.com/cockroachdb/cockroach/pkg/testutils" "github.com/cockroachdb/cockroach/pkg/testutils/kvclientutils" @@ -4165,6 +4166,7 @@ func TestStrictGCEnforcement(t *testing.T) { var ( db = tc.Server(0).DB() + ief = tc.Server(0).InternalExecutorFactory().(sqlutil.InternalExecutorFactory) getTableID = func() (tableID uint32) { sqlDB.QueryRow(t, `SELECT table_id FROM crdb_internal.tables`+ ` WHERE name = 'foo' AND database_name = current_database()`).Scan(&tableID) @@ -4450,12 +4452,12 @@ func TestStrictGCEnforcement(t *testing.T) { // Create a protected timestamp, and make sure it's not respected since the // KVSubscriber is blocked. rec := mkRecord() - require.NoError(t, db.Txn(ctx, func(ctx context.Context, txn *kv.Txn) error { - return ptp.Protect(ctx, txn, &rec) + require.NoError(t, ief.TxnWithExecutor(ctx, db, nil /* sessionData */, func(ctx context.Context, txn *kv.Txn, ie sqlutil.InternalExecutor) error { + return ptp.Protect(ctx, txn, ie, &rec) })) defer func() { - require.NoError(t, db.Txn(ctx, func(ctx context.Context, txn *kv.Txn) error { - return ptp.Release(ctx, txn, rec.ID.GetUUID()) + require.NoError(t, ief.TxnWithExecutor(ctx, db, nil /* sessionData */, func(ctx context.Context, txn *kv.Txn, ie sqlutil.InternalExecutor) error { + return ptp.Release(ctx, txn, ie, rec.ID.GetUUID()) })) }() assertScanRejected(t) diff --git a/pkg/kv/kvserver/protectedts/BUILD.bazel b/pkg/kv/kvserver/protectedts/BUILD.bazel index d28ca25f5e44..8a7d634c49a7 100644 --- a/pkg/kv/kvserver/protectedts/BUILD.bazel +++ b/pkg/kv/kvserver/protectedts/BUILD.bazel @@ -17,6 +17,7 @@ go_library( "//pkg/roachpb", "//pkg/settings", "//pkg/spanconfig", + "//pkg/sql/sqlutil", "//pkg/util/hlc", "//pkg/util/metric", "//pkg/util/stop", diff --git a/pkg/kv/kvserver/protectedts/protectedts.go b/pkg/kv/kvserver/protectedts/protectedts.go index 2fedb889f32f..0ca636556b16 100644 --- a/pkg/kv/kvserver/protectedts/protectedts.go +++ b/pkg/kv/kvserver/protectedts/protectedts.go @@ -19,6 +19,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/kv/kvserver/protectedts/ptpb" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/spanconfig" + "github.com/cockroachdb/cockroach/pkg/sql/sqlutil" "github.com/cockroachdb/cockroach/pkg/util/hlc" "github.com/cockroachdb/cockroach/pkg/util/metric" "github.com/cockroachdb/cockroach/pkg/util/stop" @@ -67,7 +68,7 @@ type Storage interface { // // An error will be returned if the ID of the provided record already exists // so callers should be sure to generate new IDs when creating records. - Protect(context.Context, *kv.Txn, *ptpb.Record) error + Protect(context.Context, *kv.Txn, sqlutil.InternalExecutor, *ptpb.Record) error // GetRecord retreives the record with the specified UUID as well as the MVCC // timestamp at which it was written. If no corresponding record exists @@ -78,30 +79,30 @@ type Storage interface { // should be protected as well as the timestamp at which the Record providing // that protection is known to be alive. The ReadTimestamp of the Txn used in // this method can be used to provide such a timestamp. - GetRecord(context.Context, *kv.Txn, uuid.UUID) (*ptpb.Record, error) + GetRecord(context.Context, *kv.Txn, uuid.UUID, sqlutil.InternalExecutor) (*ptpb.Record, error) // MarkVerified will mark a protected timestamp as verified. // // This method is generally used by an implementation of Verifier. - MarkVerified(context.Context, *kv.Txn, uuid.UUID) error + MarkVerified(context.Context, *kv.Txn, sqlutil.InternalExecutor, uuid.UUID) error // Release allows spans which were previously protected to now be garbage // collected. // // If the specified UUID does not exist ErrNotFound is returned but the // passed txn remains safe for future use. - Release(context.Context, *kv.Txn, uuid.UUID) error + Release(context.Context, *kv.Txn, sqlutil.InternalExecutor, uuid.UUID) error // GetMetadata retrieves the metadata with the provided Txn. - GetMetadata(context.Context, *kv.Txn) (ptpb.Metadata, error) + GetMetadata(context.Context, *kv.Txn, sqlutil.InternalExecutor) (ptpb.Metadata, error) // GetState retrieves the entire state of protectedts.Storage with the // provided Txn. - GetState(context.Context, *kv.Txn) (ptpb.State, error) + GetState(context.Context, *kv.Txn, sqlutil.InternalExecutor) (ptpb.State, error) // UpdateTimestamp updates the timestamp protected by the record with the // specified UUID. - UpdateTimestamp(ctx context.Context, txn *kv.Txn, id uuid.UUID, timestamp hlc.Timestamp) error + UpdateTimestamp(ctx context.Context, txn *kv.Txn, ie sqlutil.InternalExecutor, id uuid.UUID, timestamp hlc.Timestamp) error } // Iterator iterates records in a cache until wantMore is false or all Records diff --git a/pkg/kv/kvserver/protectedts/ptcache/BUILD.bazel b/pkg/kv/kvserver/protectedts/ptcache/BUILD.bazel index df1b46288ab9..a0a3658c3396 100644 --- a/pkg/kv/kvserver/protectedts/ptcache/BUILD.bazel +++ b/pkg/kv/kvserver/protectedts/ptcache/BUILD.bazel @@ -12,6 +12,7 @@ go_library( "//pkg/kv/kvserver/protectedts/ptpb", "//pkg/roachpb", "//pkg/settings/cluster", + "//pkg/sql/sqlutil", "//pkg/util/hlc", "//pkg/util/log", "//pkg/util/stop", @@ -35,6 +36,7 @@ go_test( ":ptcache", "//pkg/base", "//pkg/keys", + "//pkg/kv", "//pkg/kv/kvserver", "//pkg/kv/kvserver/protectedts", "//pkg/kv/kvserver/protectedts/ptpb", diff --git a/pkg/kv/kvserver/protectedts/ptcache/cache.go b/pkg/kv/kvserver/protectedts/ptcache/cache.go index 6d277ffb6561..b94ceca966fe 100644 --- a/pkg/kv/kvserver/protectedts/ptcache/cache.go +++ b/pkg/kv/kvserver/protectedts/ptcache/cache.go @@ -19,6 +19,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/kv/kvserver/protectedts/ptpb" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/settings/cluster" + "github.com/cockroachdb/cockroach/pkg/sql/sqlutil" "github.com/cockroachdb/cockroach/pkg/util/hlc" "github.com/cockroachdb/cockroach/pkg/util/log" "github.com/cockroachdb/cockroach/pkg/util/stop" @@ -32,6 +33,7 @@ import ( // Cache implements protectedts.Cache. type Cache struct { db *kv.DB + ief sqlutil.InternalExecutorFactory storage protectedts.Storage stopper *stop.Stopper settings *cluster.Settings @@ -56,15 +58,17 @@ type Cache struct { // Config configures a Cache. type Config struct { - DB *kv.DB - Storage protectedts.Storage - Settings *cluster.Settings + DB *kv.DB + IeFactory sqlutil.InternalExecutorFactory + Storage protectedts.Storage + Settings *cluster.Settings } // New returns a new cache. func New(config Config) *Cache { c := &Cache{ db: config.DB, + ief: config.IeFactory, storage: config.Storage, settings: config.Settings, } @@ -226,7 +230,9 @@ func (c *Cache) doUpdate(ctx context.Context) error { state ptpb.State ts hlc.Timestamp ) - err := c.db.Txn(ctx, func(ctx context.Context, txn *kv.Txn) (err error) { + err := c.ief.TxnWithExecutor(ctx, c.db, nil /* sessionData */, func( + ctx context.Context, txn *kv.Txn, ie sqlutil.InternalExecutor, + ) (err error) { // NB: because this is a read-only transaction, the commit will be a no-op; // returning nil here means the transaction will commit and will never need // to change its read timestamp. @@ -235,14 +241,14 @@ func (c *Cache) doUpdate(ctx context.Context) error { ts = txn.ReadTimestamp() } }() - md, err := c.storage.GetMetadata(ctx, txn) + md, err := c.storage.GetMetadata(ctx, txn, ie) if err != nil { return errors.Wrap(err, "failed to fetch protectedts metadata") } if versionChanged = md.Version != prev.Version; !versionChanged { return nil } - if state, err = c.storage.GetState(ctx, txn); err != nil { + if state, err = c.storage.GetState(ctx, txn, ie); err != nil { return errors.Wrap(err, "failed to fetch protectedts state") } return nil diff --git a/pkg/kv/kvserver/protectedts/ptcache/cache_test.go b/pkg/kv/kvserver/protectedts/ptcache/cache_test.go index 64ec9b9ab781..5a2daaa41f3f 100644 --- a/pkg/kv/kvserver/protectedts/ptcache/cache_test.go +++ b/pkg/kv/kvserver/protectedts/ptcache/cache_test.go @@ -19,6 +19,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/base" "github.com/cockroachdb/cockroach/pkg/keys" + "github.com/cockroachdb/cockroach/pkg/kv" "github.com/cockroachdb/cockroach/pkg/kv/kvserver" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/protectedts" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/protectedts/ptcache" @@ -56,20 +57,21 @@ func TestCacheBasic(t *testing.T) { }) defer tc.Stopper().Stop(ctx) s := tc.Server(0) + ief := s.InternalExecutorFactory().(sqlutil.InternalExecutorFactory) p := ptstorage.WithDatabase( - ptstorage.New(s.ClusterSettings(), - s.InternalExecutor().(sqlutil.InternalExecutor), - &protectedts.TestingKnobs{DisableProtectedTimestampForMultiTenant: true}), + ptstorage.New(s.ClusterSettings(), &protectedts.TestingKnobs{DisableProtectedTimestampForMultiTenant: true}), s.DB(), + ief, ) // Set the poll interval to be very short. protectedts.PollInterval.Override(ctx, &s.ClusterSettings().SV, 500*time.Microsecond) c := ptcache.New(ptcache.Config{ - Settings: s.ClusterSettings(), - DB: s.DB(), - Storage: p, + Settings: s.ClusterSettings(), + DB: s.DB(), + IeFactory: ief, + Storage: p, }) require.NoError(t, c.Start(ctx, tc.Stopper())) @@ -98,8 +100,9 @@ func TestCacheBasic(t *testing.T) { return nil }) + ieNotBoundToTxn := ief.MakeInternalExecutorWithoutTxn() // Then release the record and make sure that that gets seen. - require.Nil(t, p.Release(ctx, nil /* txn */, r.ID.GetUUID())) + require.Nil(t, p.Release(ctx, nil /* txn */, ieNotBoundToTxn, r.ID.GetUUID())) testutils.SucceedsSoon(t, func() error { var coveredBy []*ptpb.Record _ = c.Iterate(ctx, sp.Key, sp.EndKey, @@ -133,21 +136,21 @@ func TestRefresh(t *testing.T) { }) defer tc.Stopper().Stop(ctx) s := tc.Server(0) + ief := s.InternalExecutorFactory().(sqlutil.InternalExecutorFactory) p := ptstorage.WithDatabase( - ptstorage.New( - s.ClusterSettings(), - s.InternalExecutor().(sqlutil.InternalExecutor), - ptsKnobs), + ptstorage.New(s.ClusterSettings(), ptsKnobs), s.DB(), + ief, ) // Set the poll interval to be very long. protectedts.PollInterval.Override(ctx, &s.ClusterSettings().SV, 500*time.Hour) c := ptcache.New(ptcache.Config{ - Settings: s.ClusterSettings(), - DB: s.DB(), - Storage: p, + Settings: s.ClusterSettings(), + DB: s.DB(), + IeFactory: ief, + Storage: p, }) require.NoError(t, c.Start(ctx, tc.Stopper())) t.Run("already up-to-date", func(t *testing.T) { @@ -232,7 +235,8 @@ func TestRefresh(t *testing.T) { // operation, amd then refresh after it. This will demonstrate that the // iteration call does not block concurrent refreshes. ch := <-inIterate - require.NoError(t, p.Release(ctx, nil /* txn */, rec.ID.GetUUID())) + ieNotBoundToTxn := ief.MakeInternalExecutorWithoutTxn() + require.NoError(t, p.Release(ctx, nil /* txn */, ieNotBoundToTxn, rec.ID.GetUUID())) require.NoError(t, c.Refresh(ctx, s.Clock().Now())) // Signal the Iterate loop to exit and wait for it to close the channel. close(ch) @@ -254,13 +258,15 @@ func TestStart(t *testing.T) { }, }) s := tc.Server(0) + ief := s.InternalExecutorFactory().(sqlutil.InternalExecutorFactory) p := s.ExecutorConfig().(sql.ExecutorConfig).ProtectedTimestampProvider // Set the poll interval to be very long. protectedts.PollInterval.Override(ctx, &s.ClusterSettings().SV, 500*time.Hour) c := ptcache.New(ptcache.Config{ - Settings: s.ClusterSettings(), - DB: s.DB(), - Storage: p, + Settings: s.ClusterSettings(), + DB: s.DB(), + IeFactory: ief, + Storage: p, }) return tc, c } @@ -286,19 +292,19 @@ func TestQueryRecord(t *testing.T) { tc := testcluster.StartTestCluster(t, 1, base.TestClusterArgs{}) defer tc.Stopper().Stop(ctx) s := tc.Server(0) + ief := s.InternalExecutorFactory().(sqlutil.InternalExecutorFactory) p := ptstorage.WithDatabase( - ptstorage.New( - s.ClusterSettings(), - s.InternalExecutor().(sqlutil.InternalExecutor), - &protectedts.TestingKnobs{DisableProtectedTimestampForMultiTenant: true}), + ptstorage.New(s.ClusterSettings(), &protectedts.TestingKnobs{DisableProtectedTimestampForMultiTenant: true}), s.DB(), + ief, ) // Set the poll interval to be very long. protectedts.PollInterval.Override(ctx, &s.ClusterSettings().SV, 500*time.Hour) c := ptcache.New(ptcache.Config{ - Settings: s.ClusterSettings(), - DB: s.DB(), - Storage: p, + Settings: s.ClusterSettings(), + DB: s.DB(), + IeFactory: ief, + Storage: p, }) require.NoError(t, c.Start(ctx, tc.Stopper())) @@ -325,7 +331,8 @@ func TestQueryRecord(t *testing.T) { require.True(t, exists2) require.True(t, !asOf.Less(createdAt2)) // Release 2 and then create 3. - require.NoError(t, p.Release(ctx, nil /* txn */, r2.ID.GetUUID())) + ieNotBoundToTxn := ief.MakeInternalExecutorWithoutTxn() + require.NoError(t, p.Release(ctx, nil /* txn */, ieNotBoundToTxn, r2.ID.GetUUID())) r3, createdAt3 := protect(t, s, p, s.Clock().Now(), sp42) exists2, asOf = c.QueryRecord(ctx, r2.ID.GetUUID()) require.True(t, exists2) @@ -348,20 +355,21 @@ func TestIterate(t *testing.T) { tc := testcluster.StartTestCluster(t, 1, base.TestClusterArgs{}) defer tc.Stopper().Stop(ctx) s := tc.Server(0) + ief := s.InternalExecutorFactory().(sqlutil.InternalExecutorFactory) p := ptstorage.WithDatabase( - ptstorage.New(s.ClusterSettings(), - s.InternalExecutor().(sqlutil.InternalExecutor), - &protectedts.TestingKnobs{DisableProtectedTimestampForMultiTenant: true}), + ptstorage.New(s.ClusterSettings(), &protectedts.TestingKnobs{DisableProtectedTimestampForMultiTenant: true}), s.DB(), + ief, ) // Set the poll interval to be very long. protectedts.PollInterval.Override(ctx, &s.ClusterSettings().SV, 500*time.Hour) c := ptcache.New(ptcache.Config{ - Settings: s.ClusterSettings(), - DB: s.DB(), - Storage: p, + Settings: s.ClusterSettings(), + DB: s.DB(), + IeFactory: ief, + Storage: p, }) require.NoError(t, c.Start(ctx, tc.Stopper())) @@ -426,6 +434,7 @@ func TestGetProtectionTimestamps(t *testing.T) { defer tc.Stopper().Stop(ctx) // Set the poll interval to be very long. s := tc.Server(0) + ief := s.InternalExecutorFactory().(sqlutil.InternalExecutorFactory) protectedts.PollInterval.Override(ctx, &s.ClusterSettings().SV, 500*time.Hour) ts := func(nanos int) hlc.Timestamp { @@ -498,22 +507,23 @@ func TestGetProtectionTimestamps(t *testing.T) { } { t.Run(testCase.name, func(t *testing.T) { p := ptstorage.WithDatabase( - ptstorage.New(s.ClusterSettings(), - s.InternalExecutor().(sqlutil.InternalExecutor), - &protectedts.TestingKnobs{DisableProtectedTimestampForMultiTenant: true}), + ptstorage.New(s.ClusterSettings(), &protectedts.TestingKnobs{DisableProtectedTimestampForMultiTenant: true}), s.DB(), + ief, ) c := ptcache.New(ptcache.Config{ - Settings: s.ClusterSettings(), - DB: s.DB(), - Storage: p, + Settings: s.ClusterSettings(), + DB: s.DB(), + IeFactory: ief, + Storage: p, }) require.NoError(t, c.Start(ctx, tc.Stopper())) + ieNotBoundToTxn := ief.MakeInternalExecutorWithoutTxn() testCase.test(t, p, c, func(records ...*ptpb.Record) { for _, r := range records { - require.NoError(t, p.Release(ctx, nil, r.ID.GetUUID())) + require.NoError(t, p.Release(ctx, nil /* txn */, ieNotBoundToTxn, r.ID.GetUUID())) } }) }) @@ -525,20 +535,21 @@ func TestSettingChangedLeadsToFetch(t *testing.T) { tc := testcluster.StartTestCluster(t, 1, base.TestClusterArgs{}) defer tc.Stopper().Stop(ctx) s := tc.Server(0) + ief := s.InternalExecutorFactory().(sqlutil.InternalExecutorFactory) p := ptstorage.WithDatabase( - ptstorage.New(s.ClusterSettings(), - s.InternalExecutor().(sqlutil.InternalExecutor), - &protectedts.TestingKnobs{DisableProtectedTimestampForMultiTenant: true}), + ptstorage.New(s.ClusterSettings(), &protectedts.TestingKnobs{DisableProtectedTimestampForMultiTenant: true}), s.DB(), + ief, ) // Set the poll interval to be very long. protectedts.PollInterval.Override(ctx, &s.ClusterSettings().SV, 500*time.Hour) c := ptcache.New(ptcache.Config{ - Settings: s.ClusterSettings(), - DB: s.DB(), - Storage: p, + Settings: s.ClusterSettings(), + DB: s.DB(), + IeFactory: ief, + Storage: p, }) require.NoError(t, c.Start(ctx, tc.Stopper())) @@ -589,12 +600,18 @@ func protect( DeprecatedSpans: spans, } ctx := context.Background() - txn := s.DB().NewTxn(ctx, "test") - require.NoError(t, p.Protect(ctx, txn, r)) - require.NoError(t, txn.Commit(ctx)) - _, err := p.GetRecord(ctx, nil, r.ID.GetUUID()) + + ief := s.InternalExecutorFactory().(sqlutil.InternalExecutorFactory) + err := ief.TxnWithExecutor(ctx, s.DB(), nil /* sessionData */, func(ctx context.Context, txn *kv.Txn, ie sqlutil.InternalExecutor) (err error) { + require.NoError(t, p.Protect(ctx, txn, ie, r)) + createdAt = txn.CommitTimestamp() + return nil + }) + require.NoError(t, err) + + ieNotBoundToTxn := ief.MakeInternalExecutorWithoutTxn() + _, err = p.GetRecord(ctx, nil /* txn */, r.ID.GetUUID(), ieNotBoundToTxn) require.NoError(t, err) - createdAt = txn.CommitTimestamp() return r, createdAt } diff --git a/pkg/kv/kvserver/protectedts/ptprovider/provider.go b/pkg/kv/kvserver/protectedts/ptprovider/provider.go index 377039e80c53..1bdd0e502628 100644 --- a/pkg/kv/kvserver/protectedts/ptprovider/provider.go +++ b/pkg/kv/kvserver/protectedts/ptprovider/provider.go @@ -30,12 +30,13 @@ import ( // Config configures the Provider. type Config struct { - Settings *cluster.Settings - DB *kv.DB - Stores *kvserver.Stores - ReconcileStatusFuncs ptreconcile.StatusFuncs - InternalExecutor sqlutil.InternalExecutor - Knobs *protectedts.TestingKnobs + Settings *cluster.Settings + DB *kv.DB + Stores *kvserver.Stores + ReconcileStatusFuncs ptreconcile.StatusFuncs + InternalExecutorFactory sqlutil.InternalExecutorFactory + InternalExecutor sqlutil.InternalExecutor + Knobs *protectedts.TestingKnobs } // Provider is the concrete implementation of protectedts.Provider interface. @@ -51,12 +52,13 @@ func New(cfg Config) (protectedts.Provider, error) { if err := validateConfig(cfg); err != nil { return nil, err } - storage := ptstorage.New(cfg.Settings, cfg.InternalExecutor, cfg.Knobs) - reconciler := ptreconcile.New(cfg.Settings, cfg.DB, storage, cfg.ReconcileStatusFuncs) + storage := ptstorage.New(cfg.Settings, cfg.Knobs) + reconciler := ptreconcile.New(cfg.Settings, cfg.DB, cfg.InternalExecutorFactory, storage, cfg.ReconcileStatusFuncs) cache := ptcache.New(ptcache.Config{ - DB: cfg.DB, - Storage: storage, - Settings: cfg.Settings, + DB: cfg.DB, + IeFactory: cfg.InternalExecutorFactory, + Storage: storage, + Settings: cfg.Settings, }) return &Provider{ diff --git a/pkg/kv/kvserver/protectedts/ptreconcile/BUILD.bazel b/pkg/kv/kvserver/protectedts/ptreconcile/BUILD.bazel index f990a6b58c67..880d84a58a20 100644 --- a/pkg/kv/kvserver/protectedts/ptreconcile/BUILD.bazel +++ b/pkg/kv/kvserver/protectedts/ptreconcile/BUILD.bazel @@ -15,6 +15,7 @@ go_library( "//pkg/kv/kvserver/protectedts/ptpb", "//pkg/settings", "//pkg/settings/cluster", + "//pkg/sql/sqlutil", "//pkg/util/log", "//pkg/util/metric", "//pkg/util/stop", @@ -45,6 +46,7 @@ go_test( "//pkg/server", "//pkg/settings/cluster", "//pkg/sql", + "//pkg/sql/sqlutil", "//pkg/testutils", "//pkg/testutils/serverutils", "//pkg/testutils/testcluster", diff --git a/pkg/kv/kvserver/protectedts/ptreconcile/reconciler.go b/pkg/kv/kvserver/protectedts/ptreconcile/reconciler.go index 14a522a66c15..b7a755e90c20 100644 --- a/pkg/kv/kvserver/protectedts/ptreconcile/reconciler.go +++ b/pkg/kv/kvserver/protectedts/ptreconcile/reconciler.go @@ -22,6 +22,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/kv/kvserver/protectedts/ptpb" "github.com/cockroachdb/cockroach/pkg/settings" "github.com/cockroachdb/cockroach/pkg/settings/cluster" + "github.com/cockroachdb/cockroach/pkg/sql/sqlutil" "github.com/cockroachdb/cockroach/pkg/util/log" "github.com/cockroachdb/cockroach/pkg/util/stop" "github.com/cockroachdb/cockroach/pkg/util/timeutil" @@ -41,7 +42,7 @@ var ReconcileInterval = settings.RegisterDurationSetting( // StatusFunc is used to check on the status of a Record based on its Meta // field. type StatusFunc func( - ctx context.Context, txn *kv.Txn, meta []byte, + ctx context.Context, txn *kv.Txn, ie sqlutil.InternalExecutor, meta []byte, ) (shouldRemove bool, _ error) // StatusFuncs maps from MetaType to a StatusFunc. @@ -53,6 +54,7 @@ type StatusFuncs map[string]StatusFunc type Reconciler struct { settings *cluster.Settings db *kv.DB + ief sqlutil.InternalExecutorFactory pts protectedts.Storage metrics Metrics statusFuncs StatusFuncs @@ -60,11 +62,16 @@ type Reconciler struct { // New constructs a Reconciler. func New( - st *cluster.Settings, db *kv.DB, storage protectedts.Storage, statusFuncs StatusFuncs, + st *cluster.Settings, + db *kv.DB, + ief sqlutil.InternalExecutorFactory, + storage protectedts.Storage, + statusFuncs StatusFuncs, ) *Reconciler { return &Reconciler{ settings: st, db: db, + ief: ief, pts: storage, metrics: makeMetrics(), statusFuncs: statusFuncs, @@ -119,9 +126,9 @@ func (r *Reconciler) run(ctx context.Context, stopper *stop.Stopper) { func (r *Reconciler) reconcile(ctx context.Context) { // Load protected timestamp records. var state ptpb.State - if err := r.db.Txn(ctx, func(ctx context.Context, txn *kv.Txn) error { + if err := r.ief.TxnWithExecutor(ctx, r.db, nil /* sessionData */, func(ctx context.Context, txn *kv.Txn, ie sqlutil.InternalExecutor) error { var err error - state, err = r.pts.GetState(ctx, txn) + state, err = r.pts.GetState(ctx, txn, ie) return err }); err != nil { r.metrics.ReconciliationErrors.Inc(1) @@ -135,16 +142,16 @@ func (r *Reconciler) reconcile(ctx context.Context) { continue } var didRemove bool - if err := r.db.Txn(ctx, func(ctx context.Context, txn *kv.Txn) (err error) { + if err := r.ief.TxnWithExecutor(ctx, r.db, nil /* sessionData */, func(ctx context.Context, txn *kv.Txn, ie sqlutil.InternalExecutor) (err error) { didRemove = false // reset for retries - shouldRemove, err := task(ctx, txn, rec.Meta) + shouldRemove, err := task(ctx, txn, ie, rec.Meta) if err != nil { return err } if !shouldRemove { return nil } - err = r.pts.Release(ctx, txn, rec.ID.GetUUID()) + err = r.pts.Release(ctx, txn, ie, rec.ID.GetUUID()) if err != nil && !errors.Is(err, protectedts.ErrNotExists) { return err } diff --git a/pkg/kv/kvserver/protectedts/ptreconcile/reconciler_test.go b/pkg/kv/kvserver/protectedts/ptreconcile/reconciler_test.go index ca24342b7a52..8ca1bb60406e 100644 --- a/pkg/kv/kvserver/protectedts/ptreconcile/reconciler_test.go +++ b/pkg/kv/kvserver/protectedts/ptreconcile/reconciler_test.go @@ -24,6 +24,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/settings/cluster" "github.com/cockroachdb/cockroach/pkg/sql" + "github.com/cockroachdb/cockroach/pkg/sql/sqlutil" "github.com/cockroachdb/cockroach/pkg/testutils" "github.com/cockroachdb/cockroach/pkg/testutils/testcluster" "github.com/cockroachdb/cockroach/pkg/util/leaktest" @@ -50,6 +51,7 @@ func TestReconciler(t *testing.T) { // Now I want to create some artifacts that should get reconciled away and // then make sure that they do and others which should not do not. s0 := tc.Server(0) + ief := s0.InternalExecutorFactory().(sqlutil.InternalExecutorFactory) ptp := s0.ExecutorConfig().(sql.ExecutorConfig).ProtectedTimestampProvider settings := cluster.MakeTestingClusterSettings() @@ -59,10 +61,10 @@ func TestReconciler(t *testing.T) { toRemove map[string]struct{} }{} state.toRemove = map[string]struct{}{} - r := ptreconcile.New(settings, s0.DB(), ptp, + r := ptreconcile.New(settings, s0.DB(), ief, ptp, ptreconcile.StatusFuncs{ testTaskType: func( - ctx context.Context, txn *kv.Txn, meta []byte, + ctx context.Context, txn *kv.Txn, _ sqlutil.InternalExecutor, meta []byte, ) (shouldRemove bool, err error) { state.mu.Lock() defer state.mu.Unlock() @@ -84,8 +86,8 @@ func TestReconciler(t *testing.T) { } else { rec1.Target = ptpb.MakeClusterTarget() } - require.NoError(t, s0.DB().Txn(ctx, func(ctx context.Context, txn *kv.Txn) error { - return ptp.Protect(ctx, txn, &rec1) + require.NoError(t, ief.TxnWithExecutor(ctx, s0.DB(), nil /* sessionData */, func(ctx context.Context, txn *kv.Txn, ie sqlutil.InternalExecutor) error { + return ptp.Protect(ctx, txn, ie, &rec1) })) t.Run("update settings", func(t *testing.T) { @@ -112,8 +114,8 @@ func TestReconciler(t *testing.T) { } return nil }) - require.Regexp(t, protectedts.ErrNotExists, s0.DB().Txn(ctx, func(ctx context.Context, txn *kv.Txn) error { - _, err := ptp.GetRecord(ctx, txn, rec1.ID.GetUUID()) + require.Regexp(t, protectedts.ErrNotExists, ief.TxnWithExecutor(ctx, s0.DB(), nil /* sessionData */, func(ctx context.Context, txn *kv.Txn, ie sqlutil.InternalExecutor) error { + _, err := ptp.GetRecord(ctx, txn, rec1.ID.GetUUID(), ie) return err })) }) diff --git a/pkg/kv/kvserver/protectedts/ptstorage/storage.go b/pkg/kv/kvserver/protectedts/ptstorage/storage.go index 995b58e435aa..aac08b474d72 100644 --- a/pkg/kv/kvserver/protectedts/ptstorage/storage.go +++ b/pkg/kv/kvserver/protectedts/ptstorage/storage.go @@ -44,9 +44,7 @@ import ( // storage interacts with the durable state of the protectedts subsystem. type storage struct { settings *cluster.Settings - ex sqlutil.InternalExecutor - - knobs *protectedts.TestingKnobs + knobs *protectedts.TestingKnobs } var _ protectedts.Storage = (*storage)(nil) @@ -59,21 +57,23 @@ func useDeprecatedProtectedTSStorage( } // New creates a new Storage. -func New( - settings *cluster.Settings, ex sqlutil.InternalExecutor, knobs *protectedts.TestingKnobs, -) protectedts.Storage { +func New(settings *cluster.Settings, knobs *protectedts.TestingKnobs) protectedts.Storage { if knobs == nil { knobs = &protectedts.TestingKnobs{} } - return &storage{settings: settings, ex: ex, knobs: knobs} + return &storage{settings: settings, knobs: knobs} } var errNoTxn = errors.New("must provide a non-nil transaction") func (p *storage) UpdateTimestamp( - ctx context.Context, txn *kv.Txn, id uuid.UUID, timestamp hlc.Timestamp, + ctx context.Context, + txn *kv.Txn, + ie sqlutil.InternalExecutor, + id uuid.UUID, + timestamp hlc.Timestamp, ) error { - row, err := p.ex.QueryRowEx(ctx, "protectedts-update", txn, + row, err := ie.QueryRowEx(ctx, "protectedts-update", txn, sessiondata.InternalExecutorOverride{User: username.NodeUserName()}, updateTimestampQuery, id.GetBytesMut(), timestamp.AsOfSystemTime()) if err != nil { @@ -86,14 +86,14 @@ func (p *storage) UpdateTimestamp( } func (p *storage) deprecatedProtect( - ctx context.Context, txn *kv.Txn, r *ptpb.Record, meta []byte, + ctx context.Context, txn *kv.Txn, ie sqlutil.InternalExecutor, r *ptpb.Record, meta []byte, ) error { s := makeSettings(p.settings) encodedSpans, err := protoutil.Marshal(&Spans{Spans: r.DeprecatedSpans}) if err != nil { // how can this possibly fail? return errors.Wrap(err, "failed to marshal spans") } - it, err := p.ex.QueryIteratorEx(ctx, "protectedts-deprecated-protect", txn, + it, err := ie.QueryIteratorEx(ctx, "protectedts-deprecated-protect", txn, sessiondata.InternalExecutorOverride{User: username.NodeUserName()}, protectQueryWithoutTarget, s.maxSpans, s.maxBytes, len(r.DeprecatedSpans), @@ -135,7 +135,9 @@ func (p *storage) deprecatedProtect( return nil } -func (p *storage) Protect(ctx context.Context, txn *kv.Txn, r *ptpb.Record) error { +func (p *storage) Protect( + ctx context.Context, txn *kv.Txn, ie sqlutil.InternalExecutor, r *ptpb.Record, +) error { if err := validateRecordForProtect(ctx, r, p.settings, p.knobs); err != nil { return err } @@ -159,7 +161,7 @@ func (p *storage) Protect(ctx context.Context, txn *kv.Txn, r *ptpb.Record) erro // // TODO(adityamaru): Delete in 22.2 once we exclusively protect `target`s. if useDeprecatedProtectedTSStorage(ctx, p.settings, p.knobs) { - return p.deprecatedProtect(ctx, txn, r, meta) + return p.deprecatedProtect(ctx, txn, ie, r, meta) } // Clear the `DeprecatedSpans` field even if it has been set by the caller. @@ -173,7 +175,7 @@ func (p *storage) Protect(ctx context.Context, txn *kv.Txn, r *ptpb.Record) erro if err != nil { // how can this possibly fail? return errors.Wrap(err, "failed to marshal spans") } - it, err := p.ex.QueryIteratorEx(ctx, "protectedts-protect", txn, + it, err := ie.QueryIteratorEx(ctx, "protectedts-protect", txn, sessiondata.InternalExecutorOverride{User: username.NodeUserName()}, protectQuery, s.maxSpans, s.maxBytes, len(r.DeprecatedSpans), @@ -210,9 +212,9 @@ func (p *storage) Protect(ctx context.Context, txn *kv.Txn, r *ptpb.Record) erro } func (p *storage) deprecatedGetRecord( - ctx context.Context, txn *kv.Txn, id uuid.UUID, + ctx context.Context, txn *kv.Txn, ie sqlutil.InternalExecutor, id uuid.UUID, ) (*ptpb.Record, error) { - row, err := p.ex.QueryRowEx(ctx, "protectedts-deprecated-GetRecord", txn, + row, err := ie.QueryRowEx(ctx, "protectedts-deprecated-GetRecord", txn, sessiondata.InternalExecutorOverride{User: username.NodeUserName()}, getRecordWithoutTargetQuery, id.GetBytesMut()) if err != nil { @@ -228,7 +230,9 @@ func (p *storage) deprecatedGetRecord( return &r, nil } -func (p *storage) GetRecord(ctx context.Context, txn *kv.Txn, id uuid.UUID) (*ptpb.Record, error) { +func (p *storage) GetRecord( + ctx context.Context, txn *kv.Txn, id uuid.UUID, executor sqlutil.InternalExecutor, +) (*ptpb.Record, error) { if txn == nil { return nil, errNoTxn } @@ -239,10 +243,10 @@ func (p *storage) GetRecord(ctx context.Context, txn *kv.Txn, id uuid.UUID) (*pt // // TODO(adityamaru): Delete in 22.2 once we exclusively protect `target`s. if useDeprecatedProtectedTSStorage(ctx, p.settings, p.knobs) { - return p.deprecatedGetRecord(ctx, txn, id) + return p.deprecatedGetRecord(ctx, txn, executor, id) } - row, err := p.ex.QueryRowEx(ctx, "protectedts-GetRecord", txn, + row, err := executor.QueryRowEx(ctx, "protectedts-GetRecord", txn, sessiondata.InternalExecutorOverride{User: username.NodeUserName()}, getRecordQuery, id.GetBytesMut()) if err != nil { @@ -258,11 +262,13 @@ func (p *storage) GetRecord(ctx context.Context, txn *kv.Txn, id uuid.UUID) (*pt return &r, nil } -func (p *storage) MarkVerified(ctx context.Context, txn *kv.Txn, id uuid.UUID) error { +func (p *storage) MarkVerified( + ctx context.Context, txn *kv.Txn, executor sqlutil.InternalExecutor, id uuid.UUID, +) error { if txn == nil { return errNoTxn } - numRows, err := p.ex.ExecEx(ctx, "protectedts-MarkVerified", txn, + numRows, err := executor.ExecEx(ctx, "protectedts-MarkVerified", txn, sessiondata.InternalExecutorOverride{User: username.NodeUserName()}, markVerifiedQuery, id.GetBytesMut()) if err != nil { @@ -274,11 +280,13 @@ func (p *storage) MarkVerified(ctx context.Context, txn *kv.Txn, id uuid.UUID) e return nil } -func (p *storage) Release(ctx context.Context, txn *kv.Txn, id uuid.UUID) error { +func (p *storage) Release( + ctx context.Context, txn *kv.Txn, ie sqlutil.InternalExecutor, id uuid.UUID, +) error { if txn == nil { return errNoTxn } - numRows, err := p.ex.ExecEx(ctx, "protectedts-Release", txn, + numRows, err := ie.ExecEx(ctx, "protectedts-Release", txn, sessiondata.InternalExecutorOverride{User: username.NodeUserName()}, releaseQuery, id.GetBytesMut()) if err != nil { @@ -290,11 +298,13 @@ func (p *storage) Release(ctx context.Context, txn *kv.Txn, id uuid.UUID) error return nil } -func (p *storage) GetMetadata(ctx context.Context, txn *kv.Txn) (ptpb.Metadata, error) { +func (p *storage) GetMetadata( + ctx context.Context, txn *kv.Txn, executor sqlutil.InternalExecutor, +) (ptpb.Metadata, error) { if txn == nil { return ptpb.Metadata{}, errNoTxn } - row, err := p.ex.QueryRowEx(ctx, "protectedts-GetMetadata", txn, + row, err := executor.QueryRowEx(ctx, "protectedts-GetMetadata", txn, sessiondata.InternalExecutorOverride{User: username.NodeUserName()}, getMetadataQuery) if err != nil { @@ -311,15 +321,17 @@ func (p *storage) GetMetadata(ctx context.Context, txn *kv.Txn) (ptpb.Metadata, }, nil } -func (p *storage) GetState(ctx context.Context, txn *kv.Txn) (ptpb.State, error) { +func (p *storage) GetState( + ctx context.Context, txn *kv.Txn, executor sqlutil.InternalExecutor, +) (ptpb.State, error) { if txn == nil { return ptpb.State{}, errNoTxn } - md, err := p.GetMetadata(ctx, txn) + md, err := p.GetMetadata(ctx, txn, executor) if err != nil { return ptpb.State{}, err } - records, err := p.getRecords(ctx, txn) + records, err := p.getRecords(ctx, txn, executor) if err != nil { return ptpb.State{}, err } @@ -329,8 +341,10 @@ func (p *storage) GetState(ctx context.Context, txn *kv.Txn) (ptpb.State, error) }, nil } -func (p *storage) deprecatedGetRecords(ctx context.Context, txn *kv.Txn) ([]ptpb.Record, error) { - it, err := p.ex.QueryIteratorEx(ctx, "protectedts-deprecated-GetRecords", txn, +func (p *storage) deprecatedGetRecords( + ctx context.Context, txn *kv.Txn, ie sqlutil.InternalExecutor, +) ([]ptpb.Record, error) { + it, err := ie.QueryIteratorEx(ctx, "protectedts-deprecated-GetRecords", txn, sessiondata.InternalExecutorOverride{User: username.NodeUserName()}, getRecordsWithoutTargetQuery) if err != nil { @@ -352,12 +366,14 @@ func (p *storage) deprecatedGetRecords(ctx context.Context, txn *kv.Txn) ([]ptpb return records, nil } -func (p *storage) getRecords(ctx context.Context, txn *kv.Txn) ([]ptpb.Record, error) { +func (p *storage) getRecords( + ctx context.Context, txn *kv.Txn, ie sqlutil.InternalExecutor, +) ([]ptpb.Record, error) { if useDeprecatedProtectedTSStorage(ctx, p.settings, p.knobs) { - return p.deprecatedGetRecords(ctx, txn) + return p.deprecatedGetRecords(ctx, txn, ie) } - it, err := p.ex.QueryIteratorEx(ctx, "protectedts-GetRecords", txn, + it, err := ie.QueryIteratorEx(ctx, "protectedts-GetRecords", txn, sessiondata.InternalExecutorOverride{User: username.NodeUserName()}, getRecordsQuery) if err != nil { return nil, errors.Wrap(err, "failed to read records") diff --git a/pkg/kv/kvserver/protectedts/ptstorage/storage_test.go b/pkg/kv/kvserver/protectedts/ptstorage/storage_test.go index be96b9d0aa60..3d2d553a1899 100644 --- a/pkg/kv/kvserver/protectedts/ptstorage/storage_test.go +++ b/pkg/kv/kvserver/protectedts/ptstorage/storage_test.go @@ -88,8 +88,8 @@ var testCases = []testCase{ ops: []op{ funcOp(func(ctx context.Context, t *testing.T, tCtx *testContext) { rec := newRecord(tCtx, hlc.Timestamp{}, "", nil, tableTarget(42), tableSpan(42)) - err := tCtx.db.Txn(ctx, func(ctx context.Context, txn *kv.Txn) error { - return tCtx.pts.Protect(ctx, txn, &rec) + err := tCtx.ief.TxnWithExecutor(ctx, tCtx.db, nil /* sessionData */, func(ctx context.Context, txn *kv.Txn, ie sqlutil.InternalExecutor) error { + return tCtx.pts.Protect(ctx, txn, ie, &rec) }) require.Regexp(t, "invalid zero value timestamp", err.Error()) }), @@ -102,8 +102,8 @@ var testCases = []testCase{ rec := newRecord(tCtx, tCtx.tc.Server(0).Clock().Now(), "", nil, tableTarget(42), tableSpan(42)) rec.Verified = true - err := tCtx.db.Txn(ctx, func(ctx context.Context, txn *kv.Txn) error { - return tCtx.pts.Protect(ctx, txn, &rec) + err := tCtx.ief.TxnWithExecutor(ctx, tCtx.db, nil /* sessionData */, func(ctx context.Context, txn *kv.Txn, ie sqlutil.InternalExecutor) error { + return tCtx.pts.Protect(ctx, txn, ie, &rec) }) require.Regexp(t, "cannot create a verified record", err.Error()) }), @@ -126,8 +126,8 @@ var testCases = []testCase{ funcOp(func(ctx context.Context, t *testing.T, tCtx *testContext) { rec := newRecord(tCtx, tCtx.tc.Server(0).Clock().Now(), "", nil, tableTarget(42), tableSpan(42)) rec.ID = pickOneRecord(tCtx).GetBytes() - err := tCtx.db.Txn(ctx, func(ctx context.Context, txn *kv.Txn) error { - return tCtx.pts.Protect(ctx, txn, &rec) + err := tCtx.ief.TxnWithExecutor(ctx, tCtx.db, nil /* sessionData */, func(ctx context.Context, txn *kv.Txn, ie sqlutil.InternalExecutor) error { + return tCtx.pts.Protect(ctx, txn, ie, &rec) }) require.EqualError(t, err, protectedts.ErrExists.Error()) }), @@ -240,8 +240,8 @@ var testCases = []testCase{ ops: []op{ funcOp(func(ctx context.Context, t *testing.T, tCtx *testContext) { var rec *ptpb.Record - err := tCtx.db.Txn(ctx, func(ctx context.Context, txn *kv.Txn) (err error) { - rec, err = tCtx.pts.GetRecord(ctx, txn, randomID(tCtx)) + err := tCtx.ief.TxnWithExecutor(ctx, tCtx.db, nil /* sessionData */, func(ctx context.Context, txn *kv.Txn, ie sqlutil.InternalExecutor) (err error) { + rec, err = tCtx.pts.GetRecord(ctx, txn, randomID(tCtx), ie) return err }) require.EqualError(t, err, protectedts.ErrNotExists.Error()) @@ -289,8 +289,8 @@ var testCases = []testCase{ name: "UpdateTimestamp -- does not exist", ops: []op{ funcOp(func(ctx context.Context, t *testing.T, tCtx *testContext) { - err := tCtx.db.Txn(ctx, func(ctx context.Context, txn *kv.Txn) (err error) { - return tCtx.pts.UpdateTimestamp(ctx, txn, randomID(tCtx), hlc.Timestamp{WallTime: 1}) + err := tCtx.ief.TxnWithExecutor(ctx, tCtx.db, nil /* sessionData */, func(ctx context.Context, txn *kv.Txn, ie sqlutil.InternalExecutor) (err error) { + return tCtx.pts.UpdateTimestamp(ctx, txn, ie, randomID(tCtx), hlc.Timestamp{WallTime: 1}) }) require.EqualError(t, err, protectedts.ErrNotExists.Error()) }), @@ -301,15 +301,16 @@ var testCases = []testCase{ ops: []op{ funcOp(func(ctx context.Context, t *testing.T, tCtx *testContext) { rec := newRecord(tCtx, tCtx.tc.Server(0).Clock().Now(), "", nil, tableTarget(42), tableSpan(42)) + ieNotBoundToTxn := tCtx.tc.Server(0).InternalExecutorFactory().(sqlutil.InternalExecutorFactory).MakeInternalExecutorWithoutTxn() const msg = "must provide a non-nil transaction" - require.Regexp(t, msg, tCtx.pts.Protect(ctx, nil /* txn */, &rec).Error()) - require.Regexp(t, msg, tCtx.pts.Release(ctx, nil /* txn */, uuid.MakeV4()).Error()) - require.Regexp(t, msg, tCtx.pts.MarkVerified(ctx, nil /* txn */, uuid.MakeV4()).Error()) - _, err := tCtx.pts.GetRecord(ctx, nil /* txn */, uuid.MakeV4()) + require.Regexp(t, msg, tCtx.pts.Protect(ctx, nil /* txn */, ieNotBoundToTxn, &rec).Error()) + require.Regexp(t, msg, tCtx.pts.Release(ctx, nil /* txn */, ieNotBoundToTxn, uuid.MakeV4()).Error()) + require.Regexp(t, msg, tCtx.pts.MarkVerified(ctx, nil /* txn */, ieNotBoundToTxn, uuid.MakeV4()).Error()) + _, err := tCtx.pts.GetRecord(ctx, nil /* txn */, uuid.MakeV4(), ieNotBoundToTxn) require.Regexp(t, msg, err.Error()) - _, err = tCtx.pts.GetMetadata(ctx, nil /* txn */) + _, err = tCtx.pts.GetMetadata(ctx, nil /* txn */, ieNotBoundToTxn) require.Regexp(t, msg, err.Error()) - _, err = tCtx.pts.GetState(ctx, nil /* txn */) + _, err = tCtx.pts.GetState(ctx, nil /* txn */, ieNotBoundToTxn) require.Regexp(t, msg, err.Error()) }), }, @@ -320,6 +321,7 @@ type testContext struct { pts protectedts.Storage tc *testcluster.TestCluster db *kv.DB + ief sqlutil.InternalExecutorFactory // If set to false, the test will be run with // `DisableProtectedTimestampForMultiTenant` set to true, thereby testing the @@ -346,8 +348,8 @@ type releaseOp struct { func (r releaseOp) run(ctx context.Context, t *testing.T, tCtx *testContext) { id := r.idFunc(tCtx) - err := tCtx.db.Txn(ctx, func(ctx context.Context, txn *kv.Txn) error { - return tCtx.pts.Release(ctx, txn, id) + err := tCtx.ief.TxnWithExecutor(ctx, tCtx.db, nil /* sessionData */, func(ctx context.Context, txn *kv.Txn, ie sqlutil.InternalExecutor) error { + return tCtx.pts.Release(ctx, txn, ie, id) }) if !testutils.IsError(err, r.expErr) { t.Fatalf("expected error to match %q, got %q", r.expErr, err) @@ -383,8 +385,8 @@ type markVerifiedOp struct { func (mv markVerifiedOp) run(ctx context.Context, t *testing.T, tCtx *testContext) { id := mv.idFunc(tCtx) - err := tCtx.db.Txn(ctx, func(ctx context.Context, txn *kv.Txn) error { - return tCtx.pts.MarkVerified(ctx, txn, id) + err := tCtx.ief.TxnWithExecutor(ctx, tCtx.db, nil /* sessionData */, func(ctx context.Context, txn *kv.Txn, ie sqlutil.InternalExecutor) error { + return tCtx.pts.MarkVerified(ctx, txn, ie, id) }) if !testutils.IsError(err, mv.expErr) { t.Fatalf("expected error to match %q, got %q", mv.expErr, err) @@ -411,8 +413,8 @@ func (p protectOp) run(ctx context.Context, t *testing.T, tCtx *testContext) { if p.idFunc != nil { rec.ID = p.idFunc(tCtx).GetBytes() } - err := tCtx.db.Txn(ctx, func(ctx context.Context, txn *kv.Txn) error { - return tCtx.pts.Protect(ctx, txn, &rec) + err := tCtx.ief.TxnWithExecutor(ctx, tCtx.db, nil /* sessionData */, func(ctx context.Context, txn *kv.Txn, ie sqlutil.InternalExecutor) error { + return tCtx.pts.Protect(ctx, txn, ie, &rec) }) if !testutils.IsError(err, p.expErr) { t.Fatalf("expected error to match %q, got %q", p.expErr, err) @@ -447,8 +449,8 @@ type updateTimestampOp struct { func (p updateTimestampOp) run(ctx context.Context, t *testing.T, tCtx *testContext) { id := pickOneRecord(tCtx) - err := tCtx.db.Txn(ctx, func(ctx context.Context, txn *kv.Txn) error { - return tCtx.pts.UpdateTimestamp(ctx, txn, id, p.updateTimestamp) + err := tCtx.ief.TxnWithExecutor(ctx, tCtx.db, nil /* sessionData */, func(ctx context.Context, txn *kv.Txn, ie sqlutil.InternalExecutor) error { + return tCtx.pts.UpdateTimestamp(ctx, txn, ie, id, p.updateTimestamp) }) if !testutils.IsError(err, p.expErr) { t.Fatalf("expected error to match %q, got %q", p.expErr, err) @@ -481,31 +483,33 @@ func (test testCase) run(t *testing.T) { defer tc.Stopper().Stop(ctx) s := tc.Server(0) - pts := ptstorage.New(s.ClusterSettings(), s.InternalExecutor().(*sql.InternalExecutor), ptsKnobs) + pts := ptstorage.New(s.ClusterSettings(), ptsKnobs) db := s.DB() + ief := s.InternalExecutorFactory().(sqlutil.InternalExecutorFactory) tCtx := testContext{ pts: pts, db: db, + ief: ief, tc: tc, runWithDeprecatedSpans: test.runWithDeprecatedSpans, } verify := func(t *testing.T) { var state ptpb.State - require.NoError(t, db.Txn(ctx, func(ctx context.Context, txn *kv.Txn) (err error) { - state, err = pts.GetState(ctx, txn) + require.NoError(t, ief.TxnWithExecutor(ctx, db, nil /* sessionData */, func(ctx context.Context, txn *kv.Txn, ie sqlutil.InternalExecutor) (err error) { + state, err = pts.GetState(ctx, txn, ie) return err })) var md ptpb.Metadata - require.NoError(t, db.Txn(ctx, func(ctx context.Context, txn *kv.Txn) (err error) { - md, err = pts.GetMetadata(ctx, txn) + require.NoError(t, ief.TxnWithExecutor(ctx, db, nil /* sessionData */, func(ctx context.Context, txn *kv.Txn, ie sqlutil.InternalExecutor) (err error) { + md, err = pts.GetMetadata(ctx, txn, ie) return err })) require.EqualValues(t, tCtx.state, state) require.EqualValues(t, tCtx.state.Metadata, md) for _, r := range tCtx.state.Records { var rec *ptpb.Record - require.NoError(t, db.Txn(ctx, func(ctx context.Context, txn *kv.Txn) (err error) { - rec, err = pts.GetRecord(ctx, txn, r.ID.GetUUID()) + require.NoError(t, ief.TxnWithExecutor(ctx, db, nil /* sessionData */, func(ctx context.Context, txn *kv.Txn, ie sqlutil.InternalExecutor) (err error) { + rec, err = pts.GetRecord(ctx, txn, r.ID.GetUUID(), ie) return err })) require.EqualValues(t, &r, rec) @@ -616,9 +620,10 @@ func TestCorruptData(t *testing.T) { runCorruptDataTest := func(tCtx *testContext, s serverutils.TestServerInterface, tc *testcluster.TestCluster, pts protectedts.Storage) { + ief := s.InternalExecutorFactory().(sqlutil.InternalExecutorFactory) rec := newRecord(tCtx, s.Clock().Now(), "foo", []byte("bar"), tableTarget(42), tableSpan(42)) - require.NoError(t, s.DB().Txn(ctx, func(ctx context.Context, txn *kv.Txn) error { - return pts.Protect(ctx, txn, &rec) + require.NoError(t, ief.TxnWithExecutor(ctx, s.DB(), nil /* sessionData */, func(ctx context.Context, txn *kv.Txn, ie sqlutil.InternalExecutor) error { + return pts.Protect(ctx, txn, ie, &rec) })) ie := tc.Server(0).InternalExecutor().(sqlutil.InternalExecutor) updateQuery := "UPDATE system.protected_ts_records SET target = $1 WHERE id = $2" @@ -636,13 +641,13 @@ func TestCorruptData(t *testing.T) { var got *ptpb.Record msg := regexp.MustCompile("failed to unmarshal (span|target) for " + rec.ID.String() + ": ") require.Regexp(t, msg, - s.DB().Txn(ctx, func(ctx context.Context, txn *kv.Txn) (err error) { - got, err = pts.GetRecord(ctx, txn, rec.ID.GetUUID()) + ief.TxnWithExecutor(ctx, s.DB(), nil /* sessionData */, func(ctx context.Context, txn *kv.Txn, ie sqlutil.InternalExecutor) (err error) { + got, err = pts.GetRecord(ctx, txn, rec.ID.GetUUID(), ie) return err }).Error()) require.Nil(t, got) - require.NoError(t, s.DB().Txn(ctx, func(ctx context.Context, txn *kv.Txn) (err error) { - _, err = pts.GetState(ctx, txn) + require.NoError(t, ief.TxnWithExecutor(ctx, s.DB(), nil /* sessionData */, func(ctx context.Context, txn *kv.Txn, ie sqlutil.InternalExecutor) (err error) { + _, err = pts.GetState(ctx, txn, ie) return err })) log.Flush() @@ -703,11 +708,12 @@ func TestCorruptData(t *testing.T) { defer tc.Stopper().Stop(ctx) s := tc.Server(0) + ief := s.InternalExecutorFactory().(sqlutil.InternalExecutorFactory) pts := s.ExecutorConfig().(sql.ExecutorConfig).ProtectedTimestampProvider rec := newRecord(&testContext{}, s.Clock().Now(), "foo", []byte("bar"), tableTarget(42), tableSpan(42)) - require.NoError(t, s.DB().Txn(ctx, func(ctx context.Context, txn *kv.Txn) error { - return pts.Protect(ctx, txn, &rec) + require.NoError(t, ief.TxnWithExecutor(ctx, s.DB(), nil /* sessionData */, func(ctx context.Context, txn *kv.Txn, ie sqlutil.InternalExecutor) error { + return pts.Protect(ctx, txn, ie, &rec) })) // This timestamp has too many logical digits and thus will fail parsing. @@ -726,13 +732,13 @@ func TestCorruptData(t *testing.T) { msg := regexp.MustCompile("failed to parse timestamp for " + rec.ID.String() + ": logical part has too many digits") require.Regexp(t, msg, - s.DB().Txn(ctx, func(ctx context.Context, txn *kv.Txn) (err error) { - got, err = pts.GetRecord(ctx, txn, rec.ID.GetUUID()) + ief.TxnWithExecutor(ctx, s.DB(), nil /* sessionData */, func(ctx context.Context, txn *kv.Txn, ie sqlutil.InternalExecutor) (err error) { + got, err = pts.GetRecord(ctx, txn, rec.ID.GetUUID(), ie) return err })) require.Nil(t, got) - require.NoError(t, s.DB().Txn(ctx, func(ctx context.Context, txn *kv.Txn) (err error) { - _, err = pts.GetState(ctx, txn) + require.NoError(t, ief.TxnWithExecutor(ctx, s.DB(), nil /* sessionData */, func(ctx context.Context, txn *kv.Txn, ie sqlutil.InternalExecutor) (err error) { + _, err = pts.GetState(ctx, txn, ie) return err })) log.Flush() @@ -757,48 +763,73 @@ func TestErrorsFromSQL(t *testing.T) { defer tc.Stopper().Stop(ctx) s := tc.Server(0) - ie := s.InternalExecutor().(sqlutil.InternalExecutor) - wrappedIE := &wrappedInternalExecutor{wrapped: ie} - pts := ptstorage.New(s.ClusterSettings(), wrappedIE, &protectedts.TestingKnobs{}) - wrappedIE.setErrFunc(func(string) error { - return errors.New("boom") - }) + ief := s.InternalExecutorFactory().(sqlutil.InternalExecutorFactory) + pts := ptstorage.New(s.ClusterSettings(), &protectedts.TestingKnobs{}) + rec := newRecord(&testContext{}, s.Clock().Now(), "foo", []byte("bar"), tableTarget(42), tableSpan(42)) - require.EqualError(t, s.DB().Txn(ctx, func(ctx context.Context, txn *kv.Txn) error { - return pts.Protect(ctx, txn, &rec) + require.EqualError(t, ief.TxnWithExecutor(ctx, s.DB(), nil /* sessionData */, func(ctx context.Context, txn *kv.Txn, ie sqlutil.InternalExecutor) error { + wrappedIE := &wrappedInternalExecutor{wrapped: ie} + wrappedIE.setErrFunc(func(string) error { + return errors.New("boom") + }) + return pts.Protect(ctx, txn, wrappedIE, &rec) }), fmt.Sprintf("failed to write record %v: boom", rec.ID)) - require.EqualError(t, s.DB().Txn(ctx, func(ctx context.Context, txn *kv.Txn) error { - _, err := pts.GetRecord(ctx, txn, rec.ID.GetUUID()) + require.EqualError(t, ief.TxnWithExecutor(ctx, s.DB(), nil /* sessionData */, func(ctx context.Context, txn *kv.Txn, ie sqlutil.InternalExecutor) error { + wrappedIE := &wrappedInternalExecutor{wrapped: ie} + wrappedIE.setErrFunc(func(string) error { + return errors.New("boom") + }) + _, err := pts.GetRecord(ctx, txn, rec.ID.GetUUID(), wrappedIE) return err }), fmt.Sprintf("failed to read record %v: boom", rec.ID)) - require.EqualError(t, s.DB().Txn(ctx, func(ctx context.Context, txn *kv.Txn) error { - return pts.MarkVerified(ctx, txn, rec.ID.GetUUID()) + require.EqualError(t, ief.TxnWithExecutor(ctx, s.DB(), nil /* sessionData */, func(ctx context.Context, txn *kv.Txn, ie sqlutil.InternalExecutor) error { + wrappedIE := &wrappedInternalExecutor{wrapped: ie} + wrappedIE.setErrFunc(func(string) error { + return errors.New("boom") + }) + return pts.MarkVerified(ctx, txn, wrappedIE, rec.ID.GetUUID()) }), fmt.Sprintf("failed to mark record %v as verified: boom", rec.ID)) - require.EqualError(t, s.DB().Txn(ctx, func(ctx context.Context, txn *kv.Txn) error { - return pts.Release(ctx, txn, rec.ID.GetUUID()) + require.EqualError(t, ief.TxnWithExecutor(ctx, s.DB(), nil /* sessionData */, func(ctx context.Context, txn *kv.Txn, ie sqlutil.InternalExecutor) error { + wrappedIE := &wrappedInternalExecutor{wrapped: ie} + wrappedIE.setErrFunc(func(string) error { + return errors.New("boom") + }) + return pts.Release(ctx, txn, wrappedIE, rec.ID.GetUUID()) }), fmt.Sprintf("failed to release record %v: boom", rec.ID)) - require.EqualError(t, s.DB().Txn(ctx, func(ctx context.Context, txn *kv.Txn) error { - _, err := pts.GetMetadata(ctx, txn) + require.EqualError(t, ief.TxnWithExecutor(ctx, s.DB(), nil /* sessionData */, func(ctx context.Context, txn *kv.Txn, ie sqlutil.InternalExecutor) error { + wrappedIE := &wrappedInternalExecutor{wrapped: ie} + wrappedIE.setErrFunc(func(string) error { + return errors.New("boom") + }) + _, err := pts.GetMetadata(ctx, txn, wrappedIE) return err }), "failed to read metadata: boom") - require.EqualError(t, s.DB().Txn(ctx, func(ctx context.Context, txn *kv.Txn) error { - _, err := pts.GetState(ctx, txn) + require.EqualError(t, ief.TxnWithExecutor(ctx, s.DB(), nil /* sessionData */, func(ctx context.Context, txn *kv.Txn, ie sqlutil.InternalExecutor) error { + wrappedIE := &wrappedInternalExecutor{wrapped: ie} + wrappedIE.setErrFunc(func(string) error { + return errors.New("boom") + }) + _, err := pts.GetState(ctx, txn, wrappedIE) return err }), "failed to read metadata: boom") - // Test that we get an error retrieving the records in GetState. - // The preceding call tested the error while retriving the metadata in a - // call to GetState. - var seen bool - wrappedIE.setErrFunc(func(string) error { - if !seen { - seen = true - return nil - } - return errors.New("boom") - }) - require.EqualError(t, s.DB().Txn(ctx, func(ctx context.Context, txn *kv.Txn) error { - _, err := pts.GetState(ctx, txn) + require.EqualError(t, ief.TxnWithExecutor(ctx, s.DB(), nil /* sessionData */, func(ctx context.Context, txn *kv.Txn, ie sqlutil.InternalExecutor) error { + wrappedIE := &wrappedInternalExecutor{wrapped: ie} + wrappedIE.setErrFunc(func(string) error { + return errors.New("boom") + }) + // Test that we get an error retrieving the records in GetState. + // The preceding call tested the error while retriving the metadata in a + // call to GetState. + var seen bool + wrappedIE.setErrFunc(func(string) error { + if !seen { + seen = true + return nil + } + return errors.New("boom") + }) + _, err := pts.GetState(ctx, txn, wrappedIE) return err }), "failed to read records: boom") } diff --git a/pkg/kv/kvserver/protectedts/ptstorage/storage_with_database.go b/pkg/kv/kvserver/protectedts/ptstorage/storage_with_database.go index 8eb62e7650a9..140fcaa44dc7 100644 --- a/pkg/kv/kvserver/protectedts/ptstorage/storage_with_database.go +++ b/pkg/kv/kvserver/protectedts/ptstorage/storage_with_database.go @@ -16,95 +16,109 @@ import ( "github.com/cockroachdb/cockroach/pkg/kv" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/protectedts" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/protectedts/ptpb" + "github.com/cockroachdb/cockroach/pkg/sql/sqlutil" "github.com/cockroachdb/cockroach/pkg/util/hlc" "github.com/cockroachdb/cockroach/pkg/util/uuid" ) // WithDatabase wraps s such that any calls made with a nil *Txn will be wrapped // in a call to db.Txn. This is often convenient in testing. -func WithDatabase(s protectedts.Storage, db *kv.DB) protectedts.Storage { - return &storageWithDatabase{s: s, db: db} +func WithDatabase( + s protectedts.Storage, db *kv.DB, ief sqlutil.InternalExecutorFactory, +) protectedts.Storage { + return &storageWithDatabase{s: s, db: db, ief: ief} } type storageWithDatabase struct { - db *kv.DB - s protectedts.Storage + db *kv.DB + ief sqlutil.InternalExecutorFactory + s protectedts.Storage } -func (s *storageWithDatabase) Protect(ctx context.Context, txn *kv.Txn, r *ptpb.Record) error { +func (s *storageWithDatabase) Protect( + ctx context.Context, txn *kv.Txn, executor sqlutil.InternalExecutor, r *ptpb.Record, +) error { if txn == nil { - return s.db.Txn(ctx, func(ctx context.Context, txn *kv.Txn) error { - return s.s.Protect(ctx, txn, r) + return s.ief.TxnWithExecutor(ctx, s.db, nil /* sessionData */, func(ctx context.Context, newTxn *kv.Txn, ie sqlutil.InternalExecutor) error { + return s.s.Protect(ctx, newTxn, ie, r) }) } - return s.s.Protect(ctx, txn, r) + return s.s.Protect(ctx, txn, executor, r) } func (s *storageWithDatabase) GetRecord( - ctx context.Context, txn *kv.Txn, id uuid.UUID, + ctx context.Context, txn *kv.Txn, id uuid.UUID, executor sqlutil.InternalExecutor, ) (r *ptpb.Record, err error) { if txn == nil { - err = s.db.Txn(ctx, func(ctx context.Context, txn *kv.Txn) error { - r, err = s.s.GetRecord(ctx, txn, id) + err = s.ief.TxnWithExecutor(ctx, s.db, nil /* sessionData */, func(ctx context.Context, newTxn *kv.Txn, ie sqlutil.InternalExecutor) error { + r, err = s.s.GetRecord(ctx, newTxn, id, ie) return err }) return r, err } - return s.s.GetRecord(ctx, txn, id) + return s.s.GetRecord(ctx, txn, id, executor) } -func (s *storageWithDatabase) MarkVerified(ctx context.Context, txn *kv.Txn, id uuid.UUID) error { +func (s *storageWithDatabase) MarkVerified( + ctx context.Context, txn *kv.Txn, ie sqlutil.InternalExecutor, id uuid.UUID, +) error { if txn == nil { - return s.db.Txn(ctx, func(ctx context.Context, txn *kv.Txn) error { - return s.s.Release(ctx, txn, id) + return s.ief.TxnWithExecutor(ctx, s.db, nil /* sessionData */, func(ctx context.Context, newTxn *kv.Txn, executor sqlutil.InternalExecutor) error { + return s.s.Release(ctx, newTxn, executor, id) }) } - return s.s.Release(ctx, txn, id) + return s.s.Release(ctx, txn, ie, id) } -func (s *storageWithDatabase) Release(ctx context.Context, txn *kv.Txn, id uuid.UUID) error { +func (s *storageWithDatabase) Release( + ctx context.Context, txn *kv.Txn, ie sqlutil.InternalExecutor, id uuid.UUID, +) error { if txn == nil { - return s.db.Txn(ctx, func(ctx context.Context, txn *kv.Txn) error { - return s.s.Release(ctx, txn, id) + return s.ief.TxnWithExecutor(ctx, s.db, nil /* sessionData */, func(ctx context.Context, newTxn *kv.Txn, executor sqlutil.InternalExecutor) error { + return s.s.Release(ctx, newTxn, executor, id) }) } - return s.s.Release(ctx, txn, id) + return s.s.Release(ctx, txn, ie, id) } func (s *storageWithDatabase) GetMetadata( - ctx context.Context, txn *kv.Txn, + ctx context.Context, txn *kv.Txn, executor sqlutil.InternalExecutor, ) (md ptpb.Metadata, err error) { if txn == nil { - err = s.db.Txn(ctx, func(ctx context.Context, txn *kv.Txn) error { - md, err = s.s.GetMetadata(ctx, txn) + err = s.ief.TxnWithExecutor(ctx, s.db, nil /* sessionData */, func(ctx context.Context, newTxn *kv.Txn, ie sqlutil.InternalExecutor) error { + md, err = s.s.GetMetadata(ctx, newTxn, ie) return err }) return md, err } - return s.s.GetMetadata(ctx, txn) + return s.s.GetMetadata(ctx, txn, executor) } func (s *storageWithDatabase) GetState( - ctx context.Context, txn *kv.Txn, + ctx context.Context, txn *kv.Txn, executor sqlutil.InternalExecutor, ) (state ptpb.State, err error) { if txn == nil { - err = s.db.Txn(ctx, func(ctx context.Context, txn *kv.Txn) error { - state, err = s.s.GetState(ctx, txn) + err = s.ief.TxnWithExecutor(ctx, s.db, nil /* sessionData */, func(ctx context.Context, newTxn *kv.Txn, ie sqlutil.InternalExecutor) (err error) { + state, err = s.s.GetState(ctx, newTxn, ie) return err }) return state, err } - return s.s.GetState(ctx, txn) + return s.s.GetState(ctx, txn, executor) } func (s *storageWithDatabase) UpdateTimestamp( - ctx context.Context, txn *kv.Txn, id uuid.UUID, timestamp hlc.Timestamp, + ctx context.Context, + txn *kv.Txn, + ie sqlutil.InternalExecutor, + id uuid.UUID, + timestamp hlc.Timestamp, ) (err error) { if txn == nil { - err = s.db.Txn(ctx, func(ctx context.Context, txn *kv.Txn) error { - return s.s.UpdateTimestamp(ctx, txn, id, timestamp) + err = s.ief.TxnWithExecutor(ctx, s.db, nil /* sessionData */, func(ctx context.Context, newTxn *kv.Txn, executor sqlutil.InternalExecutor) error { + return s.s.UpdateTimestamp(ctx, newTxn, executor, id, timestamp) }) return err } - return s.s.UpdateTimestamp(ctx, txn, id, timestamp) + return s.s.UpdateTimestamp(ctx, txn, ie, id, timestamp) } diff --git a/pkg/scheduledjobs/env.go b/pkg/scheduledjobs/env.go index 6e76ef16b000..c9a61d1447d0 100644 --- a/pkg/scheduledjobs/env.go +++ b/pkg/scheduledjobs/env.go @@ -43,9 +43,10 @@ type JobSchedulerEnv interface { // JobExecutionConfig encapsulates external components needed for scheduled job execution. type JobExecutionConfig struct { - Settings *cluster.Settings - InternalExecutor sqlutil.InternalExecutor - DB *kv.DB + Settings *cluster.Settings + InternalExecutor sqlutil.InternalExecutor + InternalExecutorFactory sqlutil.InternalExecutorFactory + DB *kv.DB // TestingKnobs is *jobs.TestingKnobs; however we cannot depend // on jobs package due to circular dependencies. TestingKnobs base.ModuleTestingKnobs diff --git a/pkg/server/server.go b/pkg/server/server.go index b90ab64bcbab..d02f3a1f7ed6 100644 --- a/pkg/server/server.go +++ b/pkg/server/server.go @@ -499,15 +499,14 @@ func NewServer(cfg Config, stopper *stop.Stopper) (*Server, error) { protectedtsKnobs, _ := cfg.TestingKnobs.ProtectedTS.(*protectedts.TestingKnobs) protectedtsProvider, err := ptprovider.New(ptprovider.Config{ - DB: db, - InternalExecutor: internalExecutor, - Settings: st, - Knobs: protectedtsKnobs, + DB: db, + InternalExecutor: internalExecutor, + InternalExecutorFactory: internalExecutorFactory, + Settings: st, + Knobs: protectedtsKnobs, ReconcileStatusFuncs: ptreconcile.StatusFuncs{ - jobsprotectedts.GetMetaType(jobsprotectedts.Jobs): jobsprotectedts.MakeStatusFunc( - jobRegistry, internalExecutor, jobsprotectedts.Jobs), - jobsprotectedts.GetMetaType(jobsprotectedts.Schedules): jobsprotectedts.MakeStatusFunc(jobRegistry, - internalExecutor, jobsprotectedts.Schedules), + jobsprotectedts.GetMetaType(jobsprotectedts.Jobs): jobsprotectedts.MakeStatusFunc(jobRegistry, jobsprotectedts.Jobs), + jobsprotectedts.GetMetaType(jobsprotectedts.Schedules): jobsprotectedts.MakeStatusFunc(jobRegistry, jobsprotectedts.Schedules), }, }) if err != nil { diff --git a/pkg/server/server_sql.go b/pkg/server/server_sql.go index 7aa89c6cf830..c37188a5d818 100644 --- a/pkg/server/server_sql.go +++ b/pkg/server/server_sql.go @@ -1042,6 +1042,7 @@ func newSQLServer(ctx context.Context, cfg sqlServerArgs) (*SQLServer, error) { execCfg.IndexMerger = sql.NewIndexBackfillerMergePlanner(execCfg) execCfg.ProtectedTimestampManager = jobsprotectedts.NewManager( execCfg.DB, + ieFactory, execCfg.Codec, execCfg.ProtectedTimestampProvider, execCfg.SystemConfig, @@ -1475,10 +1476,11 @@ func (s *SQLServer) preStart( stopper, s.metricsRegistry, &scheduledjobs.JobExecutionConfig{ - Settings: s.execCfg.Settings, - InternalExecutor: s.internalExecutor, - DB: s.execCfg.DB, - TestingKnobs: knobs.JobsTestingKnobs, + Settings: s.execCfg.Settings, + InternalExecutor: s.internalExecutor, + DB: s.execCfg.DB, + InternalExecutorFactory: s.internalExecutorFactory, + TestingKnobs: knobs.JobsTestingKnobs, PlanHookMaker: func(opName string, txn *kv.Txn, user username.SQLUsername) (interface{}, func()) { // This is a hack to get around a Go package dependency cycle. See comment // in sql/jobs/registry.go on planHookMaker. diff --git a/pkg/server/tenant.go b/pkg/server/tenant.go index 96cddea02c5e..880c244a0721 100644 --- a/pkg/server/tenant.go +++ b/pkg/server/tenant.go @@ -823,15 +823,14 @@ func makeTenantSQLServerArgs( var protectedTSProvider protectedts.Provider protectedtsKnobs, _ := baseCfg.TestingKnobs.ProtectedTS.(*protectedts.TestingKnobs) pp, err := ptprovider.New(ptprovider.Config{ - DB: db, - InternalExecutor: circularInternalExecutor, - Settings: st, - Knobs: protectedtsKnobs, + DB: db, + InternalExecutor: circularInternalExecutor, + InternalExecutorFactory: internalExecutorFactory, + Settings: st, + Knobs: protectedtsKnobs, ReconcileStatusFuncs: ptreconcile.StatusFuncs{ - jobsprotectedts.GetMetaType(jobsprotectedts.Jobs): jobsprotectedts.MakeStatusFunc( - circularJobRegistry, circularInternalExecutor, jobsprotectedts.Jobs), - jobsprotectedts.GetMetaType(jobsprotectedts.Schedules): jobsprotectedts.MakeStatusFunc( - circularJobRegistry, circularInternalExecutor, jobsprotectedts.Schedules), + jobsprotectedts.GetMetaType(jobsprotectedts.Jobs): jobsprotectedts.MakeStatusFunc(circularJobRegistry, jobsprotectedts.Jobs), + jobsprotectedts.GetMetaType(jobsprotectedts.Schedules): jobsprotectedts.MakeStatusFunc(circularJobRegistry, jobsprotectedts.Schedules), }, }) if err != nil { diff --git a/pkg/server/testserver.go b/pkg/server/testserver.go index de94437c4693..a53f480927bc 100644 --- a/pkg/server/testserver.go +++ b/pkg/server/testserver.go @@ -47,6 +47,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/physicalplan" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/sessiondata" + "github.com/cockroachdb/cockroach/pkg/sql/sqlutil" "github.com/cockroachdb/cockroach/pkg/storage" "github.com/cockroachdb/cockroach/pkg/testutils/serverutils" "github.com/cockroachdb/cockroach/pkg/testutils/skip" @@ -615,9 +616,9 @@ type tenantProtectedTSProvider struct { } func (d tenantProtectedTSProvider) Protect( - ctx context.Context, txn *kv.Txn, rec *ptpb.Record, + ctx context.Context, txn *kv.Txn, ie sqlutil.InternalExecutor, rec *ptpb.Record, ) error { - return d.Provider.Protect(ctx, txn, rec) + return d.Provider.Protect(ctx, txn, ie, rec) } // TestTenant is an in-memory instantiation of the SQL-only process created for diff --git a/pkg/spanconfig/spanconfigjob/BUILD.bazel b/pkg/spanconfig/spanconfigjob/BUILD.bazel index 76c75637e524..6b832bdf8c6c 100644 --- a/pkg/spanconfig/spanconfigjob/BUILD.bazel +++ b/pkg/spanconfig/spanconfigjob/BUILD.bazel @@ -14,6 +14,7 @@ go_library( "//pkg/settings/cluster", "//pkg/spanconfig", "//pkg/sql", + "//pkg/sql/sqlutil", "//pkg/util", "//pkg/util/hlc", "//pkg/util/log", diff --git a/pkg/spanconfig/spanconfigjob/job.go b/pkg/spanconfig/spanconfigjob/job.go index 7602b7ca88df..9bf1c3059656 100644 --- a/pkg/spanconfig/spanconfigjob/job.go +++ b/pkg/spanconfig/spanconfigjob/job.go @@ -21,6 +21,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/settings/cluster" "github.com/cockroachdb/cockroach/pkg/spanconfig" "github.com/cockroachdb/cockroach/pkg/sql" + "github.com/cockroachdb/cockroach/pkg/sql/sqlutil" "github.com/cockroachdb/cockroach/pkg/util" "github.com/cockroachdb/cockroach/pkg/util/hlc" "github.com/cockroachdb/cockroach/pkg/util/log" @@ -82,7 +83,7 @@ func (r *resumer) Resume(ctx context.Context, execCtxI interface{}) (jobErr erro // Note that we are doing this before the possible error return below. If // there is a problem starting the reconciler this job will aggressively // restart at the job system level with no backoff. - if err := r.job.Update(ctx, nil, func(_ *kv.Txn, md jobs.JobMetadata, ju *jobs.JobUpdater) error { + if err := r.job.Update(ctx, nil /* txn */, nil /* ie */, func(_ *kv.Txn, _ sqlutil.InternalExecutor, md jobs.JobMetadata, ju *jobs.JobUpdater) error { if md.RunStats != nil && md.RunStats.NumRuns > 1 { ju.UpdateRunStats(1, md.RunStats.LastRun) } @@ -182,7 +183,7 @@ func (r *resumer) Resume(ctx context.Context, execCtxI interface{}) (jobErr erro } lastCheckpoint = rc.Checkpoint() - return r.job.SetProgress(ctx, nil, jobspb.AutoSpanConfigReconciliationProgress{ + return r.job.SetProgress(ctx, nil /* txn */, nil /* ie */, jobspb.AutoSpanConfigReconciliationProgress{ Checkpoint: rc.Checkpoint(), }) }); err != nil { diff --git a/pkg/spanconfig/spanconfigreconciler/reconciler.go b/pkg/spanconfig/spanconfigreconciler/reconciler.go index db2c1bad9fcd..055975a86532 100644 --- a/pkg/spanconfig/spanconfigreconciler/reconciler.go +++ b/pkg/spanconfig/spanconfigreconciler/reconciler.go @@ -491,7 +491,7 @@ func (r *incrementalReconciler) reconcile( } missingProtectedTimestampTargets, err = r.filterForMissingProtectedTimestampSystemTargets( - ctx, txn, sqlUpdates, + ctx, txn, sqlUpdates, ie, ) if err != nil { return err @@ -549,7 +549,7 @@ func (r *incrementalReconciler) reconcile( // correspond to cluster or tenant target protected timestamp records that are // no longer found, because they've been released. func (r *incrementalReconciler) filterForMissingProtectedTimestampSystemTargets( - ctx context.Context, txn *kv.Txn, updates []spanconfig.SQLUpdate, + ctx context.Context, txn *kv.Txn, updates []spanconfig.SQLUpdate, ie sqlutil.InternalExecutor, ) ([]spanconfig.SystemTarget, error) { seen := make(map[spanconfig.SystemTarget]struct{}) var missingSystemTargets []spanconfig.SystemTarget @@ -568,7 +568,7 @@ func (r *incrementalReconciler) filterForMissingProtectedTimestampSystemTargets( // timestamp subsystem, and the internal limits to limit the size of this // table, there is scope for improvement in the future. One option could be // a rangefeed-backed materialized view of the system table. - ptsState, err := r.execCfg.ProtectedTimestampProvider.GetState(ctx, txn) + ptsState, err := r.execCfg.ProtectedTimestampProvider.GetState(ctx, txn, ie) if err != nil { return nil, errors.Wrap(err, "failed to get protected timestamp state") } diff --git a/pkg/spanconfig/spanconfigsqltranslator/sqltranslator.go b/pkg/spanconfig/spanconfigsqltranslator/sqltranslator.go index 894d6aa4cb61..eeed2efb4e23 100644 --- a/pkg/spanconfig/spanconfigsqltranslator/sqltranslator.go +++ b/pkg/spanconfig/spanconfigsqltranslator/sqltranslator.go @@ -42,9 +42,7 @@ var _ spanconfig.SQLTranslator = &SQLTranslator{} type txnBundle struct { txn *kv.Txn descsCol *descs.Collection - // TODO(janexing): we inject ie here is to replace the executor used in - // s.ptsProvider.GetState() in SQLTranslator.Translate(). - ie sqlutil.InternalExecutor + ie sqlutil.InternalExecutor } // SQLTranslator is the concrete implementation of spanconfig.SQLTranslator. @@ -123,7 +121,7 @@ func (s *SQLTranslator) Translate( // timestamp subsystem, and the internal limits to limit the size of this // table, there is scope for improvement in the future. One option could be // a rangefeed-backed materialized view of the system table. - ptsState, err := s.ptsProvider.GetState(ctx, s.GetTxn()) + ptsState, err := s.ptsProvider.GetState(ctx, s.GetTxn(), s.GetInternalExecutor()) if err != nil { return nil, hlc.Timestamp{}, errors.Wrap(err, "failed to get protected timestamp state") } diff --git a/pkg/spanconfig/spanconfigsqlwatcher/BUILD.bazel b/pkg/spanconfig/spanconfigsqlwatcher/BUILD.bazel index b9514caee50b..113713ee56a8 100644 --- a/pkg/spanconfig/spanconfigsqlwatcher/BUILD.bazel +++ b/pkg/spanconfig/spanconfigsqlwatcher/BUILD.bazel @@ -65,6 +65,7 @@ go_test( "//pkg/sql/catalog", "//pkg/sql/catalog/descpb", "//pkg/sql/distsql", + "//pkg/sql/sqlutil", "//pkg/testutils/serverutils", "//pkg/testutils/sqlutils", "//pkg/testutils/testcluster", diff --git a/pkg/spanconfig/spanconfigsqlwatcher/protectedtsdecoder_test.go b/pkg/spanconfig/spanconfigsqlwatcher/protectedtsdecoder_test.go index 692eefddf154..3f9f55becb06 100644 --- a/pkg/spanconfig/spanconfigsqlwatcher/protectedtsdecoder_test.go +++ b/pkg/spanconfig/spanconfigsqlwatcher/protectedtsdecoder_test.go @@ -24,6 +24,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/spanconfig/spanconfigsqlwatcher" "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" "github.com/cockroachdb/cockroach/pkg/sql/distsql" + "github.com/cockroachdb/cockroach/pkg/sql/sqlutil" "github.com/cockroachdb/cockroach/pkg/testutils/testcluster" "github.com/cockroachdb/cockroach/pkg/util/leaktest" "github.com/cockroachdb/cockroach/pkg/util/uuid" @@ -40,6 +41,7 @@ func TestProtectedTimestampDecoder(t *testing.T) { defer tc.Stopper().Stop(ctx) s0 := tc.Server(0) + ief := s0.InternalExecutorFactory().(sqlutil.InternalExecutorFactory) ptp := s0.DistSQLServer().(*distsql.ServerImpl).ServerConfig.ProtectedTimestampProvider jr := s0.JobRegistry().(*jobs.Registry) k := keys.SystemSQLCodec.TablePrefix(keys.ProtectedTimestampsRecordsTableID) @@ -66,11 +68,10 @@ func TestProtectedTimestampDecoder(t *testing.T) { var rec *ptpb.Record ts := s0.Clock().Now() jobID := jr.MakeJobID() - - require.NoError(t, s0.DB().Txn(ctx, func(ctx context.Context, txn *kv.Txn) (err error) { + require.NoError(t, ief.TxnWithExecutor(ctx, s0.DB(), nil /* sessionData */, func(ctx context.Context, txn *kv.Txn, ie sqlutil.InternalExecutor) error { rec = jobsprotectedts.MakeRecord(uuid.MakeV4(), int64(jobID), ts, nil /* deprecatedSpans */, jobsprotectedts.Jobs, testCase.target) - return ptp.Protect(ctx, txn, rec) + return ptp.Protect(ctx, txn, ie, rec) })) rows, err := tc.Server(0).DB().Scan(ctx, k, k.PrefixEnd(), 0 /* maxRows */) @@ -88,8 +89,8 @@ func TestProtectedTimestampDecoder(t *testing.T) { require.Truef(t, rec.Target.Equal(got), "expected target=%s, got target=%s", rec.Target.String(), got.String()) - require.NoError(t, s0.DB().Txn(ctx, func(ctx context.Context, txn *kv.Txn) (err error) { - return ptp.Release(ctx, txn, rec.ID.GetUUID()) + require.NoError(t, ief.TxnWithExecutor(ctx, s0.DB(), nil /* sessionData */, func(ctx context.Context, txn *kv.Txn, ie sqlutil.InternalExecutor) error { + return ptp.Release(ctx, txn, ie, rec.ID.GetUUID()) })) }) } diff --git a/pkg/spanconfig/spanconfigtestutils/spanconfigtestcluster/BUILD.bazel b/pkg/spanconfig/spanconfigtestutils/spanconfigtestcluster/BUILD.bazel index 888df890cc7d..83972c5608ef 100644 --- a/pkg/spanconfig/spanconfigtestutils/spanconfigtestcluster/BUILD.bazel +++ b/pkg/spanconfig/spanconfigtestutils/spanconfigtestcluster/BUILD.bazel @@ -30,6 +30,7 @@ go_library( "//pkg/sql/catalog/tabledesc", "//pkg/sql/distsql", "//pkg/sql/sem/tree", + "//pkg/sql/sqlutil", "//pkg/testutils", "//pkg/testutils/serverutils", "//pkg/testutils/sqlutils", diff --git a/pkg/spanconfig/spanconfigtestutils/spanconfigtestcluster/tenant_state.go b/pkg/spanconfig/spanconfigtestutils/spanconfigtestcluster/tenant_state.go index 2aa3b1a96287..0482aa4d79ed 100644 --- a/pkg/spanconfig/spanconfigtestutils/spanconfigtestcluster/tenant_state.go +++ b/pkg/spanconfig/spanconfigtestutils/spanconfigtestcluster/tenant_state.go @@ -32,6 +32,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/catalog/tabledesc" "github.com/cockroachdb/cockroach/pkg/sql/distsql" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" + "github.com/cockroachdb/cockroach/pkg/sql/sqlutil" "github.com/cockroachdb/cockroach/pkg/testutils/serverutils" "github.com/cockroachdb/cockroach/pkg/testutils/sqlutils" "github.com/cockroachdb/cockroach/pkg/util/hlc" @@ -269,8 +270,8 @@ func (s *Tenant) MakeProtectedTimestampRecordAndProtect( ctx context.Context, recordID string, protectTS int, target *ptpb.Target, ) { jobID := s.JobsRegistry().MakeJobID() - require.NoError(s.t, s.ExecCfg().DB.Txn(ctx, - func(ctx context.Context, txn *kv.Txn) (err error) { + require.NoError(s.t, s.ExecCfg().InternalExecutorFactory. + TxnWithExecutor(ctx, s.ExecCfg().DB, nil /* sessionData */, func(ctx context.Context, txn *kv.Txn, ie sqlutil.InternalExecutor) error { require.Len(s.t, recordID, 1, "datadriven test only supports single character record IDs") recID, err := uuid.FromBytes([]byte(strings.Repeat(recordID, 16))) @@ -278,20 +279,19 @@ func (s *Tenant) MakeProtectedTimestampRecordAndProtect( rec := jobsprotectedts.MakeRecord(recID, int64(jobID), hlc.Timestamp{WallTime: int64(protectTS)}, nil, /* deprecatedSpans */ jobsprotectedts.Jobs, target) - return s.ProtectedTimestampProvider().Protect(ctx, txn, rec) + return s.ProtectedTimestampProvider().Protect(ctx, txn, ie, rec) })) s.updateTimestampAfterLastSQLChange() } // ReleaseProtectedTimestampRecord will release a ptpb.Record. func (s *Tenant) ReleaseProtectedTimestampRecord(ctx context.Context, recordID string) { - require.NoError(s.t, s.ExecCfg().DB.Txn(ctx, - func(ctx context.Context, txn *kv.Txn) error { - require.Len(s.t, recordID, 1, - "datadriven test only supports single character record IDs") - recID, err := uuid.FromBytes([]byte(strings.Repeat(recordID, 16))) - require.NoError(s.t, err) - return s.ProtectedTimestampProvider().Release(ctx, txn, recID) - })) + require.NoError(s.t, s.ExecCfg().InternalExecutorFactory.TxnWithExecutor(ctx, s.ExecCfg().DB, nil /* sessionData */, func(ctx context.Context, txn *kv.Txn, ie sqlutil.InternalExecutor) error { + require.Len(s.t, recordID, 1, + "datadriven test only supports single character record IDs") + recID, err := uuid.FromBytes([]byte(strings.Repeat(recordID, 16))) + require.NoError(s.t, err) + return s.ProtectedTimestampProvider().Release(ctx, txn, ie, recID) + })) s.updateTimestampAfterLastSQLChange() } diff --git a/pkg/sql/backfill.go b/pkg/sql/backfill.go index 3a0efb0c63b5..9652da344214 100644 --- a/pkg/sql/backfill.go +++ b/pkg/sql/backfill.go @@ -938,9 +938,9 @@ func (sc *SchemaChanger) distIndexBackfill( var todoSpans []roachpb.Span var mutationIdx int - if err := DescsTxn(ctx, sc.execCfg, func(ctx context.Context, txn *kv.Txn, col *descs.Collection) (err error) { + if err := sc.txnWithExecutor(ctx, func(ctx context.Context, txn *kv.Txn, _ *sessiondata.SessionData, col *descs.Collection, ie sqlutil.InternalExecutor) (err error) { todoSpans, _, mutationIdx, err = rowexec.GetResumeSpans( - ctx, sc.jobRegistry, txn, sc.execCfg.Codec, col, sc.descID, sc.mutationID, filter) + ctx, sc.jobRegistry, txn, ie, sc.execCfg.Codec, col, sc.descID, sc.mutationID, filter) return err }); err != nil { return err @@ -954,7 +954,7 @@ func (sc *SchemaChanger) distIndexBackfill( writeAsOf := sc.job.Details().(jobspb.SchemaChangeDetails).WriteTimestamp if writeAsOf.IsEmpty() { - if err := sc.job.RunningStatus(ctx, nil /* txn */, func(_ context.Context, _ jobspb.Details) (jobs.RunningStatus, error) { + if err := sc.job.RunningStatus(ctx, nil /* txn */, nil /* ie */, func(_ context.Context, _ jobspb.Details) (jobs.RunningStatus, error) { return jobs.RunningStatus("scanning target index for in-progress transactions"), nil }); err != nil { return errors.Wrapf(err, "failed to update running status of job %d", errors.Safe(sc.job.ID())) @@ -986,14 +986,14 @@ func (sc *SchemaChanger) distIndexBackfill( return err } log.Infof(ctx, "persisting target safe write time %v...", writeAsOf) - if err := sc.db.Txn(ctx, func(ctx context.Context, txn *kv.Txn) error { + if err := sc.ieFactory.TxnWithExecutor(ctx, sc.db, nil /* sessionData */, func(ctx context.Context, txn *kv.Txn, ie sqlutil.InternalExecutor) error { details := sc.job.Details().(jobspb.SchemaChangeDetails) details.WriteTimestamp = writeAsOf - return sc.job.SetDetails(ctx, txn, details) + return sc.job.SetDetails(ctx, txn, ie, details) }); err != nil { return err } - if err := sc.job.RunningStatus(ctx, nil /* txn */, func(_ context.Context, _ jobspb.Details) (jobs.RunningStatus, error) { + if err := sc.job.RunningStatus(ctx, nil /* txn */, nil /* ie */, func(_ context.Context, _ jobspb.Details) (jobs.RunningStatus, error) { return RunningStatusBackfill, nil }); err != nil { return errors.Wrapf(err, "failed to update running status of job %d", errors.Safe(sc.job.ID())) @@ -1122,7 +1122,7 @@ func (sc *SchemaChanger) distIndexBackfill( if origNRanges == -1 { origNRanges = nRanges } - return sc.db.Txn(ctx, func(ctx context.Context, txn *kv.Txn) error { + return sc.ieFactory.TxnWithExecutor(ctx, sc.db, nil /* sessionData */, func(ctx context.Context, txn *kv.Txn, ie sqlutil.InternalExecutor) error { // No processor has returned completed spans yet. if nRanges < origNRanges { fractionRangesFinished := float32(origNRanges-nRanges) / float32(origNRanges) @@ -1130,7 +1130,7 @@ func (sc *SchemaChanger) distIndexBackfill( if err != nil { return err } - if err := sc.job.FractionProgressed(ctx, txn, + if err := sc.job.FractionProgressed(ctx, txn, ie, jobs.FractionUpdater(fractionCompleted)); err != nil { return jobs.SimplifyInvalidStatusError(err) } @@ -1146,7 +1146,7 @@ func (sc *SchemaChanger) distIndexBackfill( var updateJobMu syncutil.Mutex updateJobDetails = func() error { updatedTodoSpans := getTodoSpansForUpdate() - return sc.db.Txn(ctx, func(ctx context.Context, txn *kv.Txn) error { + return sc.ieFactory.TxnWithExecutor(ctx, sc.db, nil /* sessionData */, func(ctx context.Context, txn *kv.Txn, ie sqlutil.InternalExecutor) error { updateJobMu.Lock() defer updateJobMu.Unlock() // No processor has returned completed spans yet. @@ -1154,7 +1154,7 @@ func (sc *SchemaChanger) distIndexBackfill( return nil } log.VEventf(ctx, 2, "writing todo spans to job details: %+v", updatedTodoSpans) - return rowexec.SetResumeSpansInJob(ctx, updatedTodoSpans, mutationIdx, txn, sc.job) + return rowexec.SetResumeSpansInJob(ctx, updatedTodoSpans, mutationIdx, txn, ie, sc.job) }) } @@ -1285,15 +1285,15 @@ func (sc *SchemaChanger) distColumnBackfill( // update operation to be short and to not be coupled to any other // backfill work, which may take much longer. return sc.job.FractionProgressed( - ctx, nil /* txn */, jobs.FractionUpdater(fractionCompleted), + ctx, nil /* txn */, nil /* ie */, jobs.FractionUpdater(fractionCompleted), ) } readAsOf := sc.clock.Now() var mutationIdx int - if err := DescsTxn(ctx, sc.execCfg, func(ctx context.Context, txn *kv.Txn, col *descs.Collection) (err error) { + if err := sc.txnWithExecutor(ctx, func(ctx context.Context, txn *kv.Txn, _ *sessiondata.SessionData, col *descs.Collection, ie sqlutil.InternalExecutor) (err error) { todoSpans, _, mutationIdx, err = rowexec.GetResumeSpans( - ctx, sc.jobRegistry, txn, sc.execCfg.Codec, col, sc.descID, sc.mutationID, filter) + ctx, sc.jobRegistry, txn, ie, sc.execCfg.Codec, col, sc.descID, sc.mutationID, filter) return err }); err != nil { return err @@ -1363,8 +1363,8 @@ func (sc *SchemaChanger) distColumnBackfill( // Record what is left to do for the job. // TODO(spaskob): Execute this at a regular cadence. - if err := sc.db.Txn(ctx, func(ctx context.Context, txn *kv.Txn) error { - return rowexec.SetResumeSpansInJob(ctx, todoSpans, mutationIdx, txn, sc.job) + if err := sc.ieFactory.TxnWithExecutor(ctx, sc.db, nil /* sessionData */, func(ctx context.Context, txn *kv.Txn, ie sqlutil.InternalExecutor) error { + return rowexec.SetResumeSpansInJob(ctx, todoSpans, mutationIdx, txn, ie, sc.job) }); err != nil { return err } @@ -1381,7 +1381,7 @@ func (sc *SchemaChanger) distColumnBackfill( func (sc *SchemaChanger) updateJobRunningStatus( ctx context.Context, status jobs.RunningStatus, ) (tableDesc catalog.TableDescriptor, err error) { - err = DescsTxn(ctx, sc.execCfg, func(ctx context.Context, txn *kv.Txn, col *descs.Collection) (err error) { + err = sc.txnWithExecutor(ctx, func(ctx context.Context, txn *kv.Txn, _ *sessiondata.SessionData, col *descs.Collection, ie sqlutil.InternalExecutor) (err error) { // Read table descriptor without holding a lease. tableDesc, err = col.Direct().MustGetTableDescByID(ctx, txn, sc.descID) if err != nil { @@ -1404,7 +1404,7 @@ func (sc *SchemaChanger) updateJobRunningStatus( } } if updateJobRunningProgress && !tableDesc.Dropped() { - if err := sc.job.RunningStatus(ctx, txn, func( + if err := sc.job.RunningStatus(ctx, txn, ie, func( ctx context.Context, details jobspb.Details) (jobs.RunningStatus, error) { return status, nil }); err != nil { @@ -1479,6 +1479,7 @@ func (sc *SchemaChanger) validateIndexes(ctx context.Context) error { return ValidateForwardIndexes( ctx, sc.job, + sc.ieFactory, tableDesc, forwardIndexes, runHistoricalTxn, @@ -1495,6 +1496,7 @@ func (sc *SchemaChanger) validateIndexes(ctx context.Context) error { ctx, sc.execCfg.Codec, sc.job, + sc.ieFactory, tableDesc, invertedIndexes, runHistoricalTxn, @@ -1568,6 +1570,7 @@ func ValidateInvertedIndexes( ctx context.Context, codec keys.SQLCodec, job *jobs.Job, + ief sqlutil.InternalExecutorFactory, tableDesc catalog.TableDescriptor, indexes []catalog.Index, runHistoricalTxn descs.HistoricalInternalExecTxnRunner, @@ -1771,6 +1774,7 @@ func countExpectedRowsForInvertedIndex( func ValidateForwardIndexes( ctx context.Context, job *jobs.Job, + ief sqlutil.InternalExecutorFactory, tableDesc catalog.TableDescriptor, indexes []catalog.Index, runHistoricalTxn descs.HistoricalInternalExecTxnRunner, @@ -2243,8 +2247,8 @@ func (sc *SchemaChanger) mergeFromTemporaryIndex( // DROP and steps any MERGING indexes to WRITE_ONLY func (sc *SchemaChanger) runStateMachineAfterTempIndexMerge(ctx context.Context) error { var runStatus jobs.RunningStatus - return sc.txn(ctx, func( - ctx context.Context, txn *kv.Txn, descsCol *descs.Collection, + return sc.txnWithExecutor(ctx, func( + ctx context.Context, txn *kv.Txn, _ *sessiondata.SessionData, descsCol *descs.Collection, ie sqlutil.InternalExecutor, ) error { tbl, err := descsCol.GetMutableTableVersionByID(ctx, sc.descID, txn) if err != nil { @@ -2282,7 +2286,7 @@ func (sc *SchemaChanger) runStateMachineAfterTempIndexMerge(ctx context.Context) return err } if sc.job != nil { - if err := sc.job.RunningStatus(ctx, txn, func( + if err := sc.job.RunningStatus(ctx, txn, ie, func( ctx context.Context, details jobspb.Details, ) (jobs.RunningStatus, error) { return runStatus, nil diff --git a/pkg/sql/catalog/schematelemetry/scheduled_job_executor.go b/pkg/sql/catalog/schematelemetry/scheduled_job_executor.go index 4625e3e88939..25580beb5072 100644 --- a/pkg/sql/catalog/schematelemetry/scheduled_job_executor.go +++ b/pkg/sql/catalog/schematelemetry/scheduled_job_executor.go @@ -51,6 +51,7 @@ func (s schemaTelemetryExecutor) OnDrop( schedule *jobs.ScheduledJob, txn *kv.Txn, descsCol *descs.Collection, + ie sqlutil.InternalExecutor, ) (int, error) { return 0, errScheduleUndroppable } @@ -64,6 +65,7 @@ func (s schemaTelemetryExecutor) ExecuteJob( env scheduledjobs.JobSchedulerEnv, sj *jobs.ScheduledJob, txn *kv.Txn, + ie sqlutil.InternalExecutor, ) (err error) { defer func() { if err == nil { @@ -76,7 +78,7 @@ func (s schemaTelemetryExecutor) ExecuteJob( defer cleanup() jr := p.(sql.PlanHookState).ExecCfg().JobRegistry r := schematelemetrycontroller.CreateSchemaTelemetryJobRecord(jobs.CreatedByScheduledJobs, sj.ScheduleID()) - _, err = jr.CreateAdoptableJobWithTxn(ctx, r, jr.MakeJobID(), txn) + _, err = jr.CreateAdoptableJobWithTxn(ctx, r, jr.MakeJobID(), txn, ie) return err } diff --git a/pkg/sql/compact_sql_stats.go b/pkg/sql/compact_sql_stats.go index b9e9f7c33811..a5f0b1c1003c 100644 --- a/pkg/sql/compact_sql_stats.go +++ b/pkg/sql/compact_sql_stats.go @@ -167,6 +167,7 @@ func (e *scheduledSQLStatsCompactionExecutor) OnDrop( schedule *jobs.ScheduledJob, txn *kv.Txn, descsCol *descs.Collection, + ie sqlutil.InternalExecutor, ) (int, error) { return 0, persistedsqlstats.ErrScheduleUndroppable } @@ -178,8 +179,9 @@ func (e *scheduledSQLStatsCompactionExecutor) ExecuteJob( env scheduledjobs.JobSchedulerEnv, sj *jobs.ScheduledJob, txn *kv.Txn, + ie sqlutil.InternalExecutor, ) error { - if err := e.createSQLStatsCompactionJob(ctx, cfg, sj, txn); err != nil { + if err := e.createSQLStatsCompactionJob(ctx, cfg, sj, txn, ie); err != nil { e.metrics.NumFailed.Inc(1) } @@ -188,7 +190,11 @@ func (e *scheduledSQLStatsCompactionExecutor) ExecuteJob( } func (e *scheduledSQLStatsCompactionExecutor) createSQLStatsCompactionJob( - ctx context.Context, cfg *scheduledjobs.JobExecutionConfig, sj *jobs.ScheduledJob, txn *kv.Txn, + ctx context.Context, + cfg *scheduledjobs.JobExecutionConfig, + sj *jobs.ScheduledJob, + txn *kv.Txn, + ie sqlutil.InternalExecutor, ) error { p, cleanup := cfg.PlanHookMaker("invoke-sql-stats-compact", txn, username.NodeUserName()) defer cleanup() @@ -197,7 +203,7 @@ func (e *scheduledSQLStatsCompactionExecutor) createSQLStatsCompactionJob( persistedsqlstats.CreateCompactionJob(ctx, &jobs.CreatedByInfo{ ID: sj.ScheduleID(), Name: jobs.CreatedByScheduledJobs, - }, txn, p.(*planner).ExecCfg().JobRegistry) + }, txn, ie, p.(*planner).ExecCfg().JobRegistry) if err != nil { return err diff --git a/pkg/sql/conn_executor.go b/pkg/sql/conn_executor.go index 08d2b1de8073..3cf0f1169ea9 100644 --- a/pkg/sql/conn_executor.go +++ b/pkg/sql/conn_executor.go @@ -58,6 +58,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/sqlstats/insights" "github.com/cockroachdb/cockroach/pkg/sql/sqlstats/persistedsqlstats" "github.com/cockroachdb/cockroach/pkg/sql/sqlstats/sslocal" + "github.com/cockroachdb/cockroach/pkg/sql/sqlutil" "github.com/cockroachdb/cockroach/pkg/sql/stmtdiagnostics" "github.com/cockroachdb/cockroach/pkg/util" "github.com/cockroachdb/cockroach/pkg/util/buildutil" @@ -3338,31 +3339,38 @@ func (ex *connExecutor) runPreCommitStages(ctx context.Context) error { if len(scs.state.Targets) == 0 { return nil } - deps := newSchemaChangerTxnRunDependencies( - ctx, - ex.planner.SessionData(), - ex.planner.User(), - ex.server.cfg, - ex.planner.txn, - ex.extraTxnState.descCollection, - ex.planner.EvalContext(), - ex.planner.ExtendedEvalContext().Tracing.KVTracingEnabled(), - scs.jobID, - scs.stmts, - ) - ex.extraTxnState.descCollection.ResetSyntheticDescriptors() - after, jobID, err := scrun.RunPreCommitPhase( - ctx, ex.server.cfg.DeclarativeSchemaChangerTestingKnobs, deps, scs.state, - ) - if err != nil { + + if err := ex.planner.WithInternalExecutor(ctx, func(ctx context.Context, txn *kv.Txn, ie sqlutil.InternalExecutor) error { + deps := newSchemaChangerTxnRunDependencies( + ctx, + ex.planner.SessionData(), + ex.planner.User(), + ex.server.cfg, + ex.planner.txn, + ie, + ex.extraTxnState.descCollection, + ex.planner.EvalContext(), + ex.planner.ExtendedEvalContext().Tracing.KVTracingEnabled(), + scs.jobID, + scs.stmts, + ) + ex.extraTxnState.descCollection.ResetSyntheticDescriptors() + after, jobID, err := scrun.RunPreCommitPhase( + ctx, ex.server.cfg.DeclarativeSchemaChangerTestingKnobs, deps, scs.state, + ) + if err != nil { + return err + } + scs.state = after + scs.jobID = jobID + if jobID != jobspb.InvalidJobID { + ex.extraTxnState.jobs.add(jobID) + log.Infof(ctx, "queued new schema change job %d using the new schema changer", jobID) + } + return nil + }); err != nil { return err } - scs.state = after - scs.jobID = jobID - if jobID != jobspb.InvalidJobID { - ex.extraTxnState.jobs.add(jobID) - log.Infof(ctx, "queued new schema change job %d using the new schema changer", jobID) - } return nil } diff --git a/pkg/sql/control_jobs.go b/pkg/sql/control_jobs.go index 0c3b86dd8f05..ec6e59bb682d 100644 --- a/pkg/sql/control_jobs.go +++ b/pkg/sql/control_jobs.go @@ -15,12 +15,14 @@ import ( "github.com/cockroachdb/cockroach/pkg/jobs" "github.com/cockroachdb/cockroach/pkg/jobs/jobspb" + "github.com/cockroachdb/cockroach/pkg/kv" "github.com/cockroachdb/cockroach/pkg/server/telemetry" "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgcode" "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror" "github.com/cockroachdb/cockroach/pkg/sql/roleoption" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/sqltelemetry" + "github.com/cockroachdb/cockroach/pkg/sql/sqlutil" "github.com/cockroachdb/errors" ) @@ -88,8 +90,11 @@ func (n *controlJobsNode) startExec(params runParams) error { return errors.AssertionFailedf("%q: expected *DInt, found %T", jobIDDatum, jobIDDatum) } - job, err := reg.LoadJobWithTxn(params.ctx, jobspb.JobID(jobID), params.p.Txn()) - if err != nil { + var job *jobs.Job + if err := params.p.WithInternalExecutor(params.ctx, func(ctx context.Context, txn *kv.Txn, ie sqlutil.InternalExecutor) error { + job, err = reg.LoadJobWithTxn(params.ctx, jobspb.JobID(jobID), params.p.Txn(), ie) + return err + }); err != nil { return err } @@ -112,11 +117,17 @@ func (n *controlJobsNode) startExec(params runParams) error { switch n.desiredStatus { case jobs.StatusPaused: - err = reg.PauseRequested(params.ctx, params.p.txn, jobspb.JobID(jobID), n.reason) + err = params.p.WithInternalExecutor(params.ctx, func(ctx context.Context, txn *kv.Txn, ie sqlutil.InternalExecutor) error { + return reg.PauseRequested(params.ctx, params.p.txn, ie, jobspb.JobID(jobID), n.reason) + }) case jobs.StatusRunning: - err = reg.Unpause(params.ctx, params.p.txn, jobspb.JobID(jobID)) + err = params.p.WithInternalExecutor(params.ctx, func(ctx context.Context, txn *kv.Txn, ie sqlutil.InternalExecutor) error { + return reg.Unpause(params.ctx, params.p.txn, ie, jobspb.JobID(jobID)) + }) case jobs.StatusCanceled: - err = reg.CancelRequested(params.ctx, params.p.txn, jobspb.JobID(jobID)) + err = params.p.WithInternalExecutor(params.ctx, func(ctx context.Context, txn *kv.Txn, ie sqlutil.InternalExecutor) error { + return reg.CancelRequested(params.ctx, params.p.txn, ie, jobspb.JobID(jobID)) + }) default: err = errors.AssertionFailedf("unhandled status %v", n.desiredStatus) } diff --git a/pkg/sql/control_schedules.go b/pkg/sql/control_schedules.go index 728d87a5da4c..a4d8818ebfcb 100644 --- a/pkg/sql/control_schedules.go +++ b/pkg/sql/control_schedules.go @@ -24,6 +24,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/sessiondata" "github.com/cockroachdb/cockroach/pkg/sql/sqltelemetry" + "github.com/cockroachdb/cockroach/pkg/sql/sqlutil" "github.com/cockroachdb/errors" ) @@ -173,15 +174,11 @@ func (n *controlSchedulesNode) startExec(params runParams) error { if controller, ok := ex.(jobs.ScheduledJobController); ok { scheduleControllerEnv := scheduledjobs.MakeProdScheduleControllerEnv( params.ExecCfg().ProtectedTimestampProvider, params.ExecCfg().InternalExecutor) - additionalDroppedSchedules, err := controller.OnDrop( - params.ctx, - scheduleControllerEnv, - scheduledjobs.ProdJobSchedulerEnv, - schedule, - params.p.Txn(), - params.p.Descriptors(), - ) - if err != nil { + var additionalDroppedSchedules int + if err := params.p.WithInternalExecutor(params.ctx, func(ctx context.Context, txn *kv.Txn, ie sqlutil.InternalExecutor) error { + additionalDroppedSchedules, err = controller.OnDrop(params.ctx, scheduleControllerEnv, scheduledjobs.ProdJobSchedulerEnv, schedule, params.p.Txn(), params.p.Descriptors(), ie) + return err + }); err != nil { return errors.Wrap(err, "failed to run OnDrop") } n.numRows += additionalDroppedSchedules diff --git a/pkg/sql/create_stats.go b/pkg/sql/create_stats.go index dcc4bec68353..fabcc51e43e0 100644 --- a/pkg/sql/create_stats.go +++ b/pkg/sql/create_stats.go @@ -633,7 +633,7 @@ func (r *createStatsResumer) Resume(ctx context.Context, execCtx interface{}) er // then return the original error, otherwise return this error instead so // it can be cleaned up at a higher level. if jobErr := r.job.FractionProgressed( - ctx, nil, /* txn */ + ctx, nil /* txn */, nil, /* ie */ func(ctx context.Context, _ jobspb.ProgressDetails) float32 { // The job failed so the progress value here doesn't really matter. return 0 diff --git a/pkg/sql/drop_table.go b/pkg/sql/drop_table.go index dc50fd1c497d..2de68a1bba52 100644 --- a/pkg/sql/drop_table.go +++ b/pkg/sql/drop_table.go @@ -27,6 +27,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/sqlerrors" "github.com/cockroachdb/cockroach/pkg/sql/sqltelemetry" + "github.com/cockroachdb/cockroach/pkg/sql/sqlutil" "github.com/cockroachdb/cockroach/pkg/util/log" "github.com/cockroachdb/cockroach/pkg/util/log/eventpb" "github.com/cockroachdb/cockroach/pkg/util/timeutil" @@ -416,33 +417,41 @@ func (p *planner) markTableMutationJobsSuccessful( delete(p.ExtendedEvalContext().SchemaChangeJobRecords, tableDesc.ID) continue } - mutationJob, err := p.execCfg.JobRegistry.LoadJobWithTxn(ctx, jobID, p.txn) - if err != nil { + + var mutationJob *jobs.Job + var err error + if err := p.WithInternalExecutor(ctx, func(ctx context.Context, txn *kv.Txn, ie sqlutil.InternalExecutor) error { + mutationJob, err = p.execCfg.JobRegistry.LoadJobWithTxn(ctx, jobID, p.txn, ie) + return err + }); err != nil { if jobs.HasJobNotFoundError(err) { log.Warningf(ctx, "mutation job %d not found", jobID) continue } return err } - if err := mutationJob.Update( - ctx, p.txn, func(txn *kv.Txn, md jobs.JobMetadata, ju *jobs.JobUpdater) error { - status := md.Status - switch status { - case jobs.StatusSucceeded, jobs.StatusCanceled, jobs.StatusFailed, jobs.StatusRevertFailed: - log.Warningf(ctx, "mutation job %d in unexpected state %s", jobID, status) + + if err := p.WithInternalExecutor(ctx, func(ctx context.Context, txn *kv.Txn, ie sqlutil.InternalExecutor) error { + return mutationJob.Update( + ctx, p.txn, ie, func(txn *kv.Txn, _ sqlutil.InternalExecutor, md jobs.JobMetadata, ju *jobs.JobUpdater) error { + status := md.Status + switch status { + case jobs.StatusSucceeded, jobs.StatusCanceled, jobs.StatusFailed, jobs.StatusRevertFailed: + log.Warningf(ctx, "mutation job %d in unexpected state %s", jobID, status) + return nil + case jobs.StatusRunning, jobs.StatusPending: + status = jobs.StatusSucceeded + default: + // We shouldn't mark jobs as succeeded if they're not in a state where + // they're eligible to ever succeed, so mark them as failed. + status = jobs.StatusFailed + } + log.Infof(ctx, "marking mutation job %d for dropped table as %s", jobID, status) + ju.UpdateStatus(status) return nil - case jobs.StatusRunning, jobs.StatusPending: - status = jobs.StatusSucceeded - default: - // We shouldn't mark jobs as succeeded if they're not in a state where - // they're eligible to ever succeed, so mark them as failed. - status = jobs.StatusFailed - } - log.Infof(ctx, "marking mutation job %d for dropped table as %s", jobID, status) - ju.UpdateStatus(status) - return nil - }); err != nil { - return errors.Wrap(err, "updating mutation job for dropped table") + }) + }); err != nil { + return err } } return nil diff --git a/pkg/sql/gcjob/BUILD.bazel b/pkg/sql/gcjob/BUILD.bazel index ecf11ad5a346..66045e31a6cf 100644 --- a/pkg/sql/gcjob/BUILD.bazel +++ b/pkg/sql/gcjob/BUILD.bazel @@ -40,6 +40,7 @@ go_library( "//pkg/sql/pgwire/pgerror", "//pkg/sql/sem/tree", "//pkg/sql/sqlerrors", + "//pkg/sql/sqlutil", "//pkg/storage", "//pkg/util/admission/admissionpb", "//pkg/util/hlc", diff --git a/pkg/sql/gcjob/gc_job_utils.go b/pkg/sql/gcjob/gc_job_utils.go index 5cfdcb5471f7..3481a7695bfe 100644 --- a/pkg/sql/gcjob/gc_job_utils.go +++ b/pkg/sql/gcjob/gc_job_utils.go @@ -20,6 +20,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/kv" "github.com/cockroachdb/cockroach/pkg/sql" "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" + "github.com/cockroachdb/cockroach/pkg/sql/sqlutil" "github.com/cockroachdb/cockroach/pkg/util/log" "github.com/cockroachdb/errors" ) @@ -68,9 +69,9 @@ func initDetailsAndProgress( var details jobspb.SchemaChangeGCDetails var progress *jobspb.SchemaChangeGCProgress var job *jobs.Job - if err := execCfg.DB.Txn(ctx, func(ctx context.Context, txn *kv.Txn) error { + if err := execCfg.InternalExecutorFactory.TxnWithExecutor(ctx, execCfg.DB, nil /* sessionData */, func(ctx context.Context, txn *kv.Txn, ie sqlutil.InternalExecutor) error { var err error - job, err = execCfg.JobRegistry.LoadJobWithTxn(ctx, jobID, txn) + job, err = execCfg.JobRegistry.LoadJobWithTxn(ctx, jobID, txn, ie) if err != nil { return err } @@ -122,12 +123,12 @@ func initializeProgress( } if update { - if err := execCfg.DB.Txn(ctx, func(ctx context.Context, txn *kv.Txn) error { - job, err := execCfg.JobRegistry.LoadJobWithTxn(ctx, jobID, txn) + if err := execCfg.InternalExecutorFactory.TxnWithExecutor(ctx, execCfg.DB, nil /* sessionData */, func(ctx context.Context, txn *kv.Txn, ie sqlutil.InternalExecutor) error { + job, err := execCfg.JobRegistry.LoadJobWithTxn(ctx, jobID, txn, ie) if err != nil { return err } - return job.SetProgress(ctx, txn, *progress) + return job.SetProgress(ctx, txn, ie, *progress) }); err != nil { return err } @@ -278,16 +279,16 @@ func persistProgress( progress *jobspb.SchemaChangeGCProgress, runningStatus jobs.RunningStatus, ) { - if err := execCfg.DB.Txn(ctx, func(ctx context.Context, txn *kv.Txn) error { - job, err := execCfg.JobRegistry.LoadJobWithTxn(ctx, jobID, txn) + if err := execCfg.InternalExecutorFactory.TxnWithExecutor(ctx, execCfg.DB, nil /* sessionData */, func(ctx context.Context, txn *kv.Txn, ie sqlutil.InternalExecutor) error { + job, err := execCfg.JobRegistry.LoadJobWithTxn(ctx, jobID, txn, ie) if err != nil { return err } - if err := job.SetProgress(ctx, txn, *progress); err != nil { + if err := job.SetProgress(ctx, txn, ie, *progress); err != nil { return err } log.Infof(ctx, "updated progress payload: %+v", progress) - err = job.RunningStatus(ctx, txn, func(_ context.Context, _ jobspb.Details) (jobs.RunningStatus, error) { + err = job.RunningStatus(ctx, txn, ie, func(_ context.Context, _ jobspb.Details) (jobs.RunningStatus, error) { return runningStatus, nil }) if err != nil { diff --git a/pkg/sql/gcjob/refresh_statuses.go b/pkg/sql/gcjob/refresh_statuses.go index b4f29361de5a..ae6000d86593 100644 --- a/pkg/sql/gcjob/refresh_statuses.go +++ b/pkg/sql/gcjob/refresh_statuses.go @@ -28,6 +28,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/catalog" "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" "github.com/cockroachdb/cockroach/pkg/sql/catalog/descs" + "github.com/cockroachdb/cockroach/pkg/sql/sqlutil" "github.com/cockroachdb/cockroach/pkg/util/hlc" "github.com/cockroachdb/cockroach/pkg/util/log" "github.com/cockroachdb/cockroach/pkg/util/timeutil" @@ -410,8 +411,8 @@ func isTenantProtected( isProtected := false ptsProvider := execCfg.ProtectedTimestampProvider - if err := execCfg.DB.Txn(ctx, func(ctx context.Context, txn *kv.Txn) error { - ptsState, err := ptsProvider.GetState(ctx, txn) + if err := execCfg.InternalExecutorFactory.TxnWithExecutor(ctx, execCfg.DB, nil /* sessionData */, func(ctx context.Context, txn *kv.Txn, ie sqlutil.InternalExecutor) error { + ptsState, err := ptsProvider.GetState(ctx, txn, ie) if err != nil { return errors.Wrap(err, "failed to get protectedts State") } diff --git a/pkg/sql/importer/BUILD.bazel b/pkg/sql/importer/BUILD.bazel index 7704c2625837..9a84e34c7148 100644 --- a/pkg/sql/importer/BUILD.bazel +++ b/pkg/sql/importer/BUILD.bazel @@ -89,6 +89,7 @@ go_library( "//pkg/sql/sem/tree", "//pkg/sql/sessiondata", "//pkg/sql/sqltelemetry", + "//pkg/sql/sqlutil", "//pkg/sql/stats", "//pkg/sql/types", "//pkg/storage", diff --git a/pkg/sql/importer/import_job.go b/pkg/sql/importer/import_job.go index 4c382a95985f..afc44a28c9da 100644 --- a/pkg/sql/importer/import_job.go +++ b/pkg/sql/importer/import_job.go @@ -45,6 +45,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/sem/catid" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/sqltelemetry" + "github.com/cockroachdb/cockroach/pkg/sql/sqlutil" "github.com/cockroachdb/cockroach/pkg/sql/stats" "github.com/cockroachdb/cockroach/pkg/storage" "github.com/cockroachdb/cockroach/pkg/util/hlc" @@ -115,9 +116,8 @@ func (r *importResumer) Resume(ctx context.Context, execCtx interface{}) error { // Skip prepare stage on job resumption, if it has already been completed. if !details.PrepareComplete { var schemaMetadata *preparedSchemaMetadata - if err := sql.DescsTxn(ctx, p.ExecCfg(), func( - ctx context.Context, txn *kv.Txn, descsCol *descs.Collection, - ) error { + cfg := p.ExecCfg() + if err := cfg.InternalExecutorFactory.DescsTxnWithExecutor(ctx, cfg.DB, nil /* sessionData */, func(ctx context.Context, txn *kv.Txn, descsCol *descs.Collection, ie sqlutil.InternalExecutor) error { var preparedDetails jobspb.ImportDetails schemaMetadata = &preparedSchemaMetadata{ newSchemaIDToName: make(map[descpb.ID]string), @@ -161,8 +161,8 @@ func (r *importResumer) Resume(ctx context.Context, execCtx interface{}) error { // Update the job details now that the schemas and table descs have // been "prepared". - return r.job.Update(ctx, txn, func( - txn *kv.Txn, md jobs.JobMetadata, ju *jobs.JobUpdater, + return r.job.Update(ctx, txn, ie, func( + _ *kv.Txn, _ sqlutil.InternalExecutor, md jobs.JobMetadata, ju *jobs.JobUpdater, ) error { pl := md.Payload *pl.GetImport() = preparedDetails @@ -276,7 +276,7 @@ func (r *importResumer) Resume(ctx context.Context, execCtx interface{}) error { } } - if err := r.job.SetDetails(ctx, nil /* txn */, details); err != nil { + if err := r.job.SetDetails(ctx, nil /* txn */, nil /* ie */, details); err != nil { return err } } @@ -337,8 +337,8 @@ func (r *importResumer) Resume(ctx context.Context, execCtx interface{}) error { // IMPORT INTO was planned on the older node. // // TODO(adityamaru): Remove in 22.1. - if err := p.ExecCfg().DB.Txn(ctx, func(ctx context.Context, txn *kv.Txn) error { - return r.releaseProtectedTimestamp(ctx, txn, p.ExecCfg().ProtectedTimestampProvider) + if err := p.ExecCfg().InternalExecutorFactory.TxnWithExecutor(ctx, p.ExecCfg().DB, nil /* sessionData */, func(ctx context.Context, txn *kv.Txn, ie sqlutil.InternalExecutor) error { + return r.releaseProtectedTimestamp(ctx, txn, ie, p.ExecCfg().ProtectedTimestampProvider) }); err != nil { log.Errorf(ctx, "failed to release protected timestamp: %v", err) } @@ -781,7 +781,7 @@ func (r *importResumer) parseBundleSchemaIfNeeded(ctx context.Context, phs inter ctx, span = tracing.ChildSpan(ctx, "import-parsing-bundle-schema") defer span.Finish() - if err := r.job.RunningStatus(ctx, nil /* txn */, func(_ context.Context, _ jobspb.Details) (jobs.RunningStatus, error) { + if err := r.job.RunningStatus(ctx, nil /* txn */, nil /* ie */, func(_ context.Context, _ jobspb.Details) (jobs.RunningStatus, error) { return runningStatusImportBundleParseSchema, nil }); err != nil { return errors.Wrapf(err, "failed to update running status of job %d", errors.Safe(r.job.ID())) @@ -845,7 +845,7 @@ func (r *importResumer) parseBundleSchemaIfNeeded(ctx context.Context, phs inter // Prevent job from redoing schema parsing and table desc creation // on subsequent resumptions. details.ParseBundleSchema = false - if err := r.job.SetDetails(ctx, nil /* txn */, details); err != nil { + if err := r.job.SetDetails(ctx, nil /* txn */, nil /* ie */, details); err != nil { return err } } @@ -980,8 +980,8 @@ func (r *importResumer) publishTables( log.Event(ctx, "making tables live") - err := sql.DescsTxn(ctx, execCfg, func( - ctx context.Context, txn *kv.Txn, descsCol *descs.Collection, + err := execCfg.InternalExecutorFactory.DescsTxnWithExecutor(ctx, execCfg.DB, nil /* session data */, func( + ctx context.Context, txn *kv.Txn, descsCol *descs.Collection, ie sqlutil.InternalExecutor, ) error { b := txn.NewBatch() for _, tbl := range details.Tables { @@ -1030,7 +1030,7 @@ func (r *importResumer) publishTables( // Update job record to mark tables published state as complete. details.TablesPublished = true - err := r.job.SetDetails(ctx, txn, details) + err := r.job.SetDetails(ctx, txn, ie, details) if err != nil { return errors.Wrap(err, "updating job details after publishing tables") } @@ -1101,8 +1101,8 @@ func (r *importResumer) publishSchemas(ctx context.Context, execCfg *sql.Executo } log.Event(ctx, "making schemas live") - return sql.DescsTxn(ctx, execCfg, func( - ctx context.Context, txn *kv.Txn, descsCol *descs.Collection, + return execCfg.InternalExecutorFactory.DescsTxnWithExecutor(ctx, execCfg.DB, nil /* session data */, func( + ctx context.Context, txn *kv.Txn, descsCol *descs.Collection, ie sqlutil.InternalExecutor, ) error { b := txn.NewBatch() for _, schema := range details.Schemas { @@ -1128,7 +1128,7 @@ func (r *importResumer) publishSchemas(ctx context.Context, execCfg *sql.Executo // Update job record to mark tables published state as complete. details.SchemasPublished = true - err := r.job.SetDetails(ctx, txn, details) + err := r.job.SetDetails(ctx, txn, ie, details) if err != nil { return errors.Wrap(err, "updating job details after publishing schemas") } @@ -1146,7 +1146,7 @@ func (r *importResumer) checkVirtualConstraints( desc.SetPublic() if sql.HasVirtualUniqueConstraints(desc) { - if err := job.RunningStatus(ctx, nil /* txn */, func(_ context.Context, _ jobspb.Details) (jobs.RunningStatus, error) { + if err := job.RunningStatus(ctx, nil /* txn */, nil /* ie */, func(_ context.Context, _ jobspb.Details) (jobs.RunningStatus, error) { return jobs.RunningStatus(fmt.Sprintf("re-validating %s", desc.GetName())), nil }); err != nil { return errors.Wrapf(err, "failed to update running status of job %d", errors.Safe(job.ID())) @@ -1433,9 +1433,7 @@ func (r *importResumer) OnFailOrCancel(ctx context.Context, execCtx interface{}, addToFileFormatTelemetry(details.Format.Format.String(), "failed") cfg := execCtx.(sql.JobExecContext).ExecCfg() var jobsToRunAfterTxnCommit []jobspb.JobID - if err := sql.DescsTxn(ctx, cfg, func( - ctx context.Context, txn *kv.Txn, descsCol *descs.Collection, - ) error { + if err := cfg.InternalExecutorFactory.DescsTxnWithExecutor(ctx, cfg.DB, nil /* sessionData */, func(ctx context.Context, txn *kv.Txn, descsCol *descs.Collection, ie sqlutil.InternalExecutor) error { if err := r.dropTables(ctx, txn, descsCol, cfg); err != nil { return err } @@ -1452,7 +1450,7 @@ func (r *importResumer) OnFailOrCancel(ctx context.Context, execCtx interface{}, } // TODO(adityamaru): Remove in 22.1 since we do not write PTS records during // IMPORT INTO from 21.2+. - return r.releaseProtectedTimestamp(ctx, txn, cfg.ProtectedTimestampProvider) + return r.releaseProtectedTimestamp(ctx, txn, ie, cfg.ProtectedTimestampProvider) }); err != nil { return err } @@ -1736,7 +1734,7 @@ func (r *importResumer) dropSchemas( } func (r *importResumer) releaseProtectedTimestamp( - ctx context.Context, txn *kv.Txn, pts protectedts.Storage, + ctx context.Context, txn *kv.Txn, ie sqlutil.InternalExecutor, pts protectedts.Storage, ) error { details := r.job.Details().(jobspb.ImportDetails) ptsID := details.ProtectedTimestampRecord @@ -1744,7 +1742,7 @@ func (r *importResumer) releaseProtectedTimestamp( if ptsID == nil { return nil } - err := pts.Release(ctx, txn, *ptsID) + err := pts.Release(ctx, txn, ie, *ptsID) if errors.Is(err, protectedts.ErrNotExists) { // No reason to return an error which might cause problems if it doesn't // seem to exist. diff --git a/pkg/sql/importer/import_planning.go b/pkg/sql/importer/import_planning.go index 503b7b9b8d3a..24103c961e71 100644 --- a/pkg/sql/importer/import_planning.go +++ b/pkg/sql/importer/import_planning.go @@ -48,6 +48,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgnotice" "github.com/cockroachdb/cockroach/pkg/sql/privilege" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" + "github.com/cockroachdb/cockroach/pkg/sql/sqlutil" "github.com/cockroachdb/cockroach/pkg/util" "github.com/cockroachdb/cockroach/pkg/util/errorutil/unimplemented" "github.com/cockroachdb/cockroach/pkg/util/humanizeutil" @@ -962,9 +963,11 @@ func importPlanHook( // When running inside an explicit transaction, we simply create the job // record. We do not wait for the job to finish. jobID := p.ExecCfg().JobRegistry.MakeJobID() - _, err := p.ExecCfg().JobRegistry.CreateAdoptableJobWithTxn( - ctx, jr, jobID, p.Txn()) - if err != nil { + if err := p.WithInternalExecutor(ctx, func(ctx context.Context, txn *kv.Txn, ie sqlutil.InternalExecutor) error { + _, err = p.ExecCfg().JobRegistry.CreateAdoptableJobWithTxn( + ctx, jr, jobID, p.Txn(), ie) + return err + }); err != nil { return err } diff --git a/pkg/sql/importer/import_processor.go b/pkg/sql/importer/import_processor.go index f927026318b3..529e8c31a1e3 100644 --- a/pkg/sql/importer/import_processor.go +++ b/pkg/sql/importer/import_processor.go @@ -163,9 +163,10 @@ func newReadImportDataProcessor( // job progress within the import processor. if cp.flowCtx.Cfg.JobRegistry != nil { cp.seqChunkProvider = &row.SeqChunkProvider{ - JobID: cp.spec.Progress.JobID, - Registry: cp.flowCtx.Cfg.JobRegistry, - DB: cp.flowCtx.Cfg.DB, + JobID: cp.spec.Progress.JobID, + Registry: cp.flowCtx.Cfg.JobRegistry, + DB: cp.flowCtx.Cfg.DB, + InternalExecutorFactory: cp.flowCtx.Cfg.InternalExecutorFactory, } } diff --git a/pkg/sql/importer/import_processor_planning.go b/pkg/sql/importer/import_processor_planning.go index b4e659ae5208..40450c8ebefd 100644 --- a/pkg/sql/importer/import_processor_planning.go +++ b/pkg/sql/importer/import_processor_planning.go @@ -136,7 +136,10 @@ func distImport( importDetails := job.Progress().Details.(*jobspb.Progress_Import).Import if importDetails.ReadProgress == nil { // Initialize the progress metrics on the first attempt. - if err := job.FractionProgressed(ctx, nil, /* txn */ + if err := job.FractionProgressed( + ctx, + nil, /* txn */ + nil, /* ie */ func(ctx context.Context, details jobspb.ProgressDetails) float32 { prog := details.(*jobspb.Progress_Import).Import prog.ReadProgress = make([]float32, len(from)) @@ -159,7 +162,10 @@ func distImport( fractionProgress := make([]uint32, len(from)) updateJobProgress := func() error { - return job.FractionProgressed(ctx, nil, /* txn */ + return job.FractionProgressed( + ctx, + nil, /* txn */ + nil, /* ie */ func(ctx context.Context, details jobspb.ProgressDetails) float32 { var overall float32 prog := details.(*jobspb.Progress_Import).Import diff --git a/pkg/sql/importer/import_processor_test.go b/pkg/sql/importer/import_processor_test.go index 1f98cd2c0541..f6c483d7eabe 100644 --- a/pkg/sql/importer/import_processor_test.go +++ b/pkg/sql/importer/import_processor_test.go @@ -731,7 +731,7 @@ func TestCSVImportCanBeResumed(t *testing.T) { js := queryJobUntil(t, sqlDB.DB, jobID, func(js jobState) bool { return js.prog.ResumePos[0] > 0 }) // Pause the job; - if err := registry.PauseRequested(ctx, nil, jobID, ""); err != nil { + if err := registry.PauseRequested(ctx, nil /* txn */, nil /* ie */, jobID, ""); err != nil { t.Fatal(err) } // Send cancellation and unblock breakpoint. @@ -744,7 +744,7 @@ func TestCSVImportCanBeResumed(t *testing.T) { t.Logf("Resume pos: %v\n", js.prog.ResumePos[0]) // Unpause the job and wait for it to complete. - if err := registry.Unpause(ctx, nil, jobID); err != nil { + if err := registry.Unpause(ctx, nil /* txn */, nil /* ie */, jobID); err != nil { t.Fatal(err) } js = queryJobUntil(t, sqlDB.DB, jobID, func(js jobState) bool { return jobs.StatusSucceeded == js.status }) @@ -835,7 +835,7 @@ func TestCSVImportMarksFilesFullyProcessed(t *testing.T) { proceedImport := controllerBarrier.Enter() // Pause the job; - if err := registry.PauseRequested(ctx, nil, jobID, ""); err != nil { + if err := registry.PauseRequested(ctx, nil /* txn */, nil /* ie */, jobID, ""); err != nil { t.Fatal(err) } @@ -850,7 +850,7 @@ func TestCSVImportMarksFilesFullyProcessed(t *testing.T) { proceedImport() // Unpause the job and wait for it to complete. - if err := registry.Unpause(ctx, nil, jobID); err != nil { + if err := registry.Unpause(ctx, nil /* txn */, nil /* ie */, jobID); err != nil { t.Fatal(err) } js = queryJobUntil(t, sqlDB.DB, jobID, func(js jobState) bool { return jobs.StatusSucceeded == js.status }) diff --git a/pkg/sql/importer/import_stmt_test.go b/pkg/sql/importer/import_stmt_test.go index 4b8f5b2d0c37..7b7e1d0b2366 100644 --- a/pkg/sql/importer/import_stmt_test.go +++ b/pkg/sql/importer/import_stmt_test.go @@ -4662,7 +4662,7 @@ func TestImportDefaultWithResume(t *testing.T) { }) // Pause the job; - if err := registry.PauseRequested(ctx, nil, jobID, ""); err != nil { + if err := registry.PauseRequested(ctx, nil /* txn */, nil /* ie */, jobID, ""); err != nil { t.Fatal(err) } // Send cancellation and unblock breakpoint. @@ -4692,7 +4692,7 @@ func TestImportDefaultWithResume(t *testing.T) { sqlDB.QueryRow(t, fmt.Sprintf(`SELECT last_value FROM %s`, test.sequence)).Scan(&seqValOnPause) // Unpause the job and wait for it to complete. - if err := registry.Unpause(ctx, nil, jobID); err != nil { + if err := registry.Unpause(ctx, nil /* txn */, nil /* ie */, jobID); err != nil { t.Fatal(err) } js = queryJobUntil(t, sqlDB.DB, jobID, func(js jobState) bool { return jobs.StatusSucceeded == js.status }) diff --git a/pkg/sql/indexbackfiller_test.go b/pkg/sql/indexbackfiller_test.go index 058ea2a925c8..118a9c83e31f 100644 --- a/pkg/sql/indexbackfiller_test.go +++ b/pkg/sql/indexbackfiller_test.go @@ -37,6 +37,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/rowenc" "github.com/cockroachdb/cockroach/pkg/sql/rowinfra" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" + "github.com/cockroachdb/cockroach/pkg/sql/sqlutil" "github.com/cockroachdb/cockroach/pkg/sql/tests" "github.com/cockroachdb/cockroach/pkg/sql/types" "github.com/cockroachdb/cockroach/pkg/startupmigrations" @@ -487,8 +488,8 @@ INSERT INTO foo VALUES (1), (10), (100); jr := s0.JobRegistry().(*jobs.Registry) var j *jobs.Job var table catalog.TableDescriptor - require.NoError(t, sql.DescsTxn(ctx, &execCfg, func( - ctx context.Context, txn *kv.Txn, descriptors *descs.Collection, + require.NoError(t, execCfg.InternalExecutorFactory.DescsTxnWithExecutor(ctx, execCfg.DB, nil /* sd */, func( + ctx context.Context, txn *kv.Txn, descriptors *descs.Collection, ie sqlutil.InternalExecutor, ) (err error) { mut, err := descriptors.GetMutableTableByID(ctx, txn, tableID, tree.ObjectLookupFlags{}) if err != nil { @@ -515,7 +516,7 @@ INSERT INTO foo VALUES (1), (10), (100); ResumeSpanList: resumeSpanList, }, Progress: jobspb.SchemaChangeGCProgress{}, - }, jobID, txn) + }, jobID, txn, ie) if err != nil { return err } diff --git a/pkg/sql/internal.go b/pkg/sql/internal.go index bfd5816e1871..969a5d947ceb 100644 --- a/pkg/sql/internal.go +++ b/pkg/sql/internal.go @@ -259,7 +259,6 @@ func (ie *InternalExecutor) initConnEx( ex.executorType = executorTypeInternal return ex, nil - } // newConnExecutorWithTxn creates a connExecutor that will execute statements @@ -306,6 +305,7 @@ func (ie *InternalExecutor) newConnExecutorWithTxn( ex.extraTxnState.jobs = ie.extraTxnState.jobs ex.extraTxnState.schemaChangerState = ie.extraTxnState.schemaChangerState ex.extraTxnState.shouldResetSyntheticDescriptors = shouldResetSyntheticDescriptors + ex.planner.txn = ie.extraTxnState.txn ex.initPlanner(ctx, &ex.planner) } } @@ -1033,9 +1033,6 @@ func (ie *InternalExecutor) commitTxn(ctx context.Context) error { if err != nil { return errors.Wrap(err, "cannot create conn executor to commit txn") } - // TODO(janexing): is this correct? - ex.planner.txn = ie.extraTxnState.txn - defer ex.close(ctx, externalTxnClose) return ex.commitSQLTransactionInternal(ctx) } diff --git a/pkg/sql/job_exec_context.go b/pkg/sql/job_exec_context.go index fea0978fe65f..92a73fb33bb8 100644 --- a/pkg/sql/job_exec_context.go +++ b/pkg/sql/job_exec_context.go @@ -23,6 +23,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/sessiondata" "github.com/cockroachdb/cockroach/pkg/sql/sessiondatapb" + "github.com/cockroachdb/cockroach/pkg/sql/sqlutil" "github.com/cockroachdb/cockroach/pkg/upgrade" ) @@ -72,6 +73,13 @@ func (e *plannerJobExecContext) SpanConfigReconciler() spanconfig.Reconciler { } func (e *plannerJobExecContext) Txn() *kv.Txn { return e.p.Txn() } +func (e *plannerJobExecContext) WithInternalExecutor( + ctx context.Context, + run func(ctx context.Context, txn *kv.Txn, ie sqlutil.InternalExecutor) error, +) error { + return e.p.WithInternalExecutor(ctx, run) +} + // ConstrainPrimaryIndexSpanByExpr implements SpanConstrainer func (e *plannerJobExecContext) ConstrainPrimaryIndexSpanByExpr( ctx context.Context, @@ -107,4 +115,8 @@ type JobExecContext interface { MigrationJobDeps() upgrade.JobDeps SpanConfigReconciler() spanconfig.Reconciler Txn() *kv.Txn + WithInternalExecutor( + ctx context.Context, + run func(ctx context.Context, txn *kv.Txn, ie sqlutil.InternalExecutor) error, + ) error } diff --git a/pkg/sql/mvcc_backfiller.go b/pkg/sql/mvcc_backfiller.go index 70555ae4ce46..9e3ca53915f7 100644 --- a/pkg/sql/mvcc_backfiller.go +++ b/pkg/sql/mvcc_backfiller.go @@ -280,7 +280,7 @@ func (imt *IndexMergeTracker) FlushCheckpoint(ctx context.Context) error { details.ResumeSpanList[progress.MutationIdx[idx]].ResumeSpans = progress.TodoSpans[idx] } - return imt.jobMu.job.SetDetails(ctx, nil, details) + return imt.jobMu.job.SetDetails(ctx, nil /* txn */, nil /* ie */, details) } // FlushFractionCompleted writes out the fraction completed based on the number of total @@ -307,7 +307,7 @@ func (imt *IndexMergeTracker) FlushFractionCompleted(ctx context.Context) error imt.jobMu.Lock() defer imt.jobMu.Unlock() - if err := imt.jobMu.job.FractionProgressed(ctx, nil, + if err := imt.jobMu.job.FractionProgressed(ctx, nil /* txn */, nil, /* ie */ jobs.FractionUpdater(frac)); err != nil { return jobs.SimplifyInvalidStatusError(err) } diff --git a/pkg/sql/planhook.go b/pkg/sql/planhook.go index 1d2070ba4e5c..aabfd64ef417 100644 --- a/pkg/sql/planhook.go +++ b/pkg/sql/planhook.go @@ -29,6 +29,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/sem/eval" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/sessiondata" + "github.com/cockroachdb/cockroach/pkg/sql/sqlutil" "github.com/cockroachdb/cockroach/pkg/upgrade" "github.com/cockroachdb/cockroach/pkg/util/tracing" ) @@ -127,6 +128,10 @@ type PlanHookState interface { SpanConfigReconciler() spanconfig.Reconciler BufferClientNotice(ctx context.Context, notice pgnotice.Notice) Txn() *kv.Txn + WithInternalExecutor( + ctx context.Context, + run func(ctx context.Context, txn *kv.Txn, ie sqlutil.InternalExecutor) error, + ) error } // AddPlanHook adds a hook used to short-circuit creating a planNode from a diff --git a/pkg/sql/planner.go b/pkg/sql/planner.go index 89a1632ece9e..f525bb593252 100644 --- a/pkg/sql/planner.go +++ b/pkg/sql/planner.go @@ -835,10 +835,25 @@ func (p *planner) resetPlanner( func (p *planner) GetReplicationStreamManager( ctx context.Context, ) (eval.ReplicationStreamManager, error) { - return streaming.GetReplicationStreamManager(ctx, p.EvalContext(), p.Txn()) + var repStreamManager eval.ReplicationStreamManager + if err := p.WithInternalExecutor(ctx, func(ctx context.Context, txn *kv.Txn, ie sqlutil.InternalExecutor) (err error) { + repStreamManager, err = streaming.GetReplicationStreamManager(ctx, p.EvalContext(), txn, ie) + return err + }); err != nil { + return nil, err + } + return repStreamManager, nil } // GetStreamIngestManager returns a StreamIngestManager. func (p *planner) GetStreamIngestManager(ctx context.Context) (eval.StreamIngestManager, error) { - return streaming.GetStreamIngestManager(ctx, p.EvalContext(), p.Txn()) + var streamIngestManager eval.StreamIngestManager + if err := p.WithInternalExecutor(ctx, func(ctx context.Context, txn *kv.Txn, ie sqlutil.InternalExecutor) (err error) { + streamIngestManager, err = streaming.GetStreamIngestManager(ctx, p.EvalContext(), txn, ie) + return err + }); err != nil { + return nil, err + } + return streamIngestManager, nil + } diff --git a/pkg/sql/row/BUILD.bazel b/pkg/sql/row/BUILD.bazel index 15c884e946a4..c8d138a81739 100644 --- a/pkg/sql/row/BUILD.bazel +++ b/pkg/sql/row/BUILD.bazel @@ -61,6 +61,7 @@ go_library( "//pkg/sql/sessiondatapb", "//pkg/sql/span", "//pkg/sql/sqlerrors", + "//pkg/sql/sqlutil", "//pkg/sql/types", "//pkg/storage", "//pkg/storage/enginepb", @@ -118,6 +119,7 @@ go_test( "//pkg/sql/rowinfra", "//pkg/sql/sem/eval", "//pkg/sql/sem/tree", + "//pkg/sql/sqlutil", "//pkg/storage", "//pkg/testutils", "//pkg/testutils/serverutils", diff --git a/pkg/sql/row/expr_walker.go b/pkg/sql/row/expr_walker.go index f39d64b8364d..f616de018cb4 100644 --- a/pkg/sql/row/expr_walker.go +++ b/pkg/sql/row/expr_walker.go @@ -29,6 +29,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/sem/eval" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/sem/volatility" + "github.com/cockroachdb/cockroach/pkg/sql/sqlutil" "github.com/cockroachdb/cockroach/pkg/sql/types" "github.com/cockroachdb/cockroach/pkg/util/uuid" "github.com/cockroachdb/errors" @@ -286,9 +287,10 @@ func importGenUUID( // SeqChunkProvider uses the import job progress to read and write its sequence // value chunks. type SeqChunkProvider struct { - JobID jobspb.JobID - Registry *jobs.Registry - DB *kv.DB + JobID jobspb.JobID + Registry *jobs.Registry + DB *kv.DB + InternalExecutorFactory sqlutil.InternalExecutorFactory } // RequestChunk updates seqMetadata with information about the chunk of sequence @@ -299,9 +301,9 @@ func (j *SeqChunkProvider) RequestChunk( ctx context.Context, evalCtx *eval.Context, c *CellInfoAnnotation, seqMetadata *SequenceMetadata, ) error { var hasAllocatedChunk bool - return j.DB.Txn(ctx, func(ctx context.Context, txn *kv.Txn) error { + return j.InternalExecutorFactory.TxnWithExecutor(ctx, j.DB, nil /* sessionData */, func(ctx context.Context, txn *kv.Txn, ie sqlutil.InternalExecutor) error { var foundFromPreviouslyAllocatedChunk bool - resolveChunkFunc := func(txn *kv.Txn, md jobs.JobMetadata, ju *jobs.JobUpdater) error { + resolveChunkFunc := func(_ *kv.Txn, _ sqlutil.InternalExecutor, md jobs.JobMetadata, ju *jobs.JobUpdater) error { progress := md.Progress // Check if we have already reserved a chunk corresponding to this row in a @@ -357,7 +359,7 @@ func (j *SeqChunkProvider) RequestChunk( return nil } const useReadLock = true - err := j.Registry.UpdateJobWithTxn(ctx, j.JobID, txn, useReadLock, resolveChunkFunc) + err := j.Registry.UpdateJobWithTxn(ctx, j.JobID, txn, ie, useReadLock, resolveChunkFunc) if err != nil { return err } diff --git a/pkg/sql/row/expr_walker_test.go b/pkg/sql/row/expr_walker_test.go index c6fe414be0b2..0db40ba7b7ac 100644 --- a/pkg/sql/row/expr_walker_test.go +++ b/pkg/sql/row/expr_walker_test.go @@ -24,6 +24,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/catalog/tabledesc" "github.com/cockroachdb/cockroach/pkg/sql/row" "github.com/cockroachdb/cockroach/pkg/sql/sem/eval" + "github.com/cockroachdb/cockroach/pkg/sql/sqlutil" "github.com/cockroachdb/cockroach/pkg/testutils/serverutils" "github.com/cockroachdb/cockroach/pkg/util/leaktest" "github.com/cockroachdb/cockroach/pkg/util/log" @@ -80,6 +81,7 @@ func TestJobBackedSeqChunkProvider(t *testing.T) { ctx := context.Background() s, sqlDB, db := serverutils.StartServer(t, base.TestServerArgs{}) + ief := s.InternalExecutorFactory().(sqlutil.InternalExecutorFactory) defer s.Stopper().Stop(ctx) evalCtx := &eval.Context{ @@ -190,7 +192,7 @@ func TestJobBackedSeqChunkProvider(t *testing.T) { t.Run(test.name, func(t *testing.T) { job := createMockImportJob(ctx, t, registry, test.allocatedChunks, test.resumePos) j := &row.SeqChunkProvider{ - Registry: registry, JobID: job.ID(), DB: db, + Registry: registry, JobID: job.ID(), DB: db, InternalExecutorFactory: ief, } annot := &row.CellInfoAnnotation{ SourceID: 0, diff --git a/pkg/sql/rowexec/BUILD.bazel b/pkg/sql/rowexec/BUILD.bazel index 7e6856c9b8fd..320227e387a4 100644 --- a/pkg/sql/rowexec/BUILD.bazel +++ b/pkg/sql/rowexec/BUILD.bazel @@ -83,6 +83,7 @@ go_library( "//pkg/sql/span", "//pkg/sql/sqlerrors", "//pkg/sql/sqltelemetry", + "//pkg/sql/sqlutil", "//pkg/sql/stats", "//pkg/sql/types", "//pkg/util", diff --git a/pkg/sql/rowexec/backfiller.go b/pkg/sql/rowexec/backfiller.go index 735af8098173..0f39cdfae62a 100644 --- a/pkg/sql/rowexec/backfiller.go +++ b/pkg/sql/rowexec/backfiller.go @@ -26,6 +26,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/execinfrapb" "github.com/cockroachdb/cockroach/pkg/sql/rowinfra" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" + "github.com/cockroachdb/cockroach/pkg/sql/sqlutil" "github.com/cockroachdb/cockroach/pkg/sql/types" "github.com/cockroachdb/cockroach/pkg/util/hlc" "github.com/cockroachdb/cockroach/pkg/util/log" @@ -195,6 +196,7 @@ func GetResumeSpans( ctx context.Context, jobsRegistry *jobs.Registry, txn *kv.Txn, + ie sqlutil.InternalExecutor, codec keys.SQLCodec, col *descs.Collection, tableID descpb.ID, @@ -244,7 +246,7 @@ func GetResumeSpans( "no job found for mutation %d", errors.Safe(mutationID)) } - job, err := jobsRegistry.LoadJobWithTxn(ctx, jobID, txn) + job, err := jobsRegistry.LoadJobWithTxn(ctx, jobID, txn, ie) if err != nil { return nil, nil, 0, errors.Wrapf(err, "can't find job %d", errors.Safe(jobID)) } @@ -268,12 +270,17 @@ func GetResumeSpans( // SetResumeSpansInJob adds a list of resume spans into a job details field. func SetResumeSpansInJob( - ctx context.Context, spans []roachpb.Span, mutationIdx int, txn *kv.Txn, job *jobs.Job, + ctx context.Context, + spans []roachpb.Span, + mutationIdx int, + txn *kv.Txn, + ie sqlutil.InternalExecutor, + job *jobs.Job, ) error { details, ok := job.Details().(jobspb.SchemaChangeDetails) if !ok { return errors.Errorf("expected SchemaChangeDetails job type, got %T", job.Details()) } details.ResumeSpanList[mutationIdx].ResumeSpans = spans - return job.SetDetails(ctx, txn, details) + return job.SetDetails(ctx, txn, ie, details) } diff --git a/pkg/sql/rowexec/backfiller_test.go b/pkg/sql/rowexec/backfiller_test.go index 8530899777b2..0bc807b15ef2 100644 --- a/pkg/sql/rowexec/backfiller_test.go +++ b/pkg/sql/rowexec/backfiller_test.go @@ -27,6 +27,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/catalog/descs" "github.com/cockroachdb/cockroach/pkg/sql/catalog/desctestutils" "github.com/cockroachdb/cockroach/pkg/sql/rowexec" + "github.com/cockroachdb/cockroach/pkg/sql/sqlutil" "github.com/cockroachdb/cockroach/pkg/startupmigrations" "github.com/cockroachdb/cockroach/pkg/testutils/serverutils" "github.com/cockroachdb/cockroach/pkg/testutils/sqlutils" @@ -42,6 +43,7 @@ import ( func TestingWriteResumeSpan( ctx context.Context, txn *kv.Txn, + ie sqlutil.InternalExecutor, codec keys.SQLCodec, col *descs.Collection, id descpb.ID, @@ -54,14 +56,14 @@ func TestingWriteResumeSpan( defer traceSpan.Finish() resumeSpans, job, mutationIdx, err := rowexec.GetResumeSpans( - ctx, jobsRegistry, txn, codec, col, id, mutationID, filter, + ctx, jobsRegistry, txn, ie, codec, col, id, mutationID, filter, ) if err != nil { return err } resumeSpans = roachpb.SubtractSpans(resumeSpans, finished) - return rowexec.SetResumeSpansInJob(ctx, resumeSpans, mutationIdx, txn, job) + return rowexec.SetResumeSpansInJob(ctx, resumeSpans, mutationIdx, txn, ie, job) } func TestWriteResumeSpan(t *testing.T) { @@ -134,13 +136,13 @@ func TestWriteResumeSpan(t *testing.T) { t.Fatal(errors.Wrapf(err, "can't find job %d", jobID)) } - require.NoError(t, job.Update(ctx, nil, /* txn */ - func(_ *kv.Txn, _ jobs.JobMetadata, ju *jobs.JobUpdater) error { + require.NoError(t, job.Update(ctx, nil /* txn */, nil, /* ie */ + func(_ *kv.Txn, _ sqlutil.InternalExecutor, _ jobs.JobMetadata, ju *jobs.JobUpdater) error { ju.UpdateStatus(jobs.StatusRunning) return nil })) - err = job.SetDetails(ctx, nil /* txn */, details) + err = job.SetDetails(ctx, nil /* txn */, nil /* ie */, details) if err != nil { t.Fatal(err) } @@ -179,12 +181,15 @@ func TestWriteResumeSpan(t *testing.T) { if test.resume.Key != nil { finished.EndKey = test.resume.Key } - if err := sql.TestingDescsTxn(ctx, server, func(ctx context.Context, txn *kv.Txn, col *descs.Collection) error { + if err := server.InternalExecutorFactory().(descs.TxnManager).DescsTxnWithExecutor(ctx, server.DB(), nil /* sessionData */, func( + ctx context.Context, txn *kv.Txn, descsCol *descs.Collection, ie sqlutil.InternalExecutor, + ) error { return TestingWriteResumeSpan( ctx, txn, + ie, keys.SystemSQLCodec, - col, + descsCol, tableDesc.ID, mutationID, backfill.IndexMutationFilter, @@ -219,9 +224,10 @@ func TestWriteResumeSpan(t *testing.T) { } var got []roachpb.Span - if err := sql.TestingDescsTxn(ctx, server, func(ctx context.Context, txn *kv.Txn, col *descs.Collection) (err error) { + ief := server.InternalExecutorFactory().(descs.TxnManager) + if err := ief.DescsTxnWithExecutor(ctx, server.DB(), nil /* sessionData */, func(ctx context.Context, txn *kv.Txn, col *descs.Collection, ie sqlutil.InternalExecutor) (err error) { got, _, _, err = rowexec.GetResumeSpans( - ctx, registry, txn, keys.SystemSQLCodec, col, tableDesc.ID, mutationID, backfill.IndexMutationFilter) + ctx, registry, txn, ie, keys.SystemSQLCodec, col, tableDesc.ID, mutationID, backfill.IndexMutationFilter) return err }); err != nil { t.Error(err) diff --git a/pkg/sql/rowexec/sample_aggregator.go b/pkg/sql/rowexec/sample_aggregator.go index f597be7ce925..ba76cfaa2745 100644 --- a/pkg/sql/rowexec/sample_aggregator.go +++ b/pkg/sql/rowexec/sample_aggregator.go @@ -231,10 +231,10 @@ func (s *sampleAggregator) mainLoop(ctx context.Context) (earlyExit bool, err er // If it changed by less than 1%, just check for cancellation (which is more // efficient). if fractionCompleted < 1.0 && fractionCompleted < lastReportedFractionCompleted+0.01 { - return job.CheckStatus(ctx, nil /* txn */) + return job.CheckStatus(ctx, nil /* txn */, nil /* ie */) } lastReportedFractionCompleted = fractionCompleted - return job.FractionProgressed(ctx, nil /* txn */, jobs.FractionUpdater(fractionCompleted)) + return job.FractionProgressed(ctx, nil /* txn */, nil /* ie */, jobs.FractionUpdater(fractionCompleted)) } var rowsProcessed uint64 diff --git a/pkg/sql/schema_change_plan_node.go b/pkg/sql/schema_change_plan_node.go index 2928d275a503..ddaa45abd57c 100644 --- a/pkg/sql/schema_change_plan_node.go +++ b/pkg/sql/schema_change_plan_node.go @@ -33,6 +33,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/sessiondata" "github.com/cockroachdb/cockroach/pkg/sql/sessiondatapb" + "github.com/cockroachdb/cockroach/pkg/sql/sqlutil" "github.com/cockroachdb/cockroach/pkg/util/log" "github.com/cockroachdb/cockroach/pkg/util/retry" "github.com/cockroachdb/cockroach/pkg/util/timeutil" @@ -266,27 +267,34 @@ func (s *schemaChangePlanNode) startExec(params runParams) error { s.plannedState = state } - runDeps := newSchemaChangerTxnRunDependencies( - params.ctx, - p.SessionData(), - p.User(), - p.ExecCfg(), - p.Txn(), - p.Descriptors(), - p.EvalContext(), - p.ExtendedEvalContext().Tracing.KVTracingEnabled(), - scs.jobID, - scs.stmts, - ) - after, jobID, err := scrun.RunStatementPhase( - params.ctx, p.ExecCfg().DeclarativeSchemaChangerTestingKnobs, runDeps, s.plannedState, - ) - if err != nil { + if err := p.WithInternalExecutor(params.ctx, func(ctx context.Context, txn *kv.Txn, ie sqlutil.InternalExecutor) error { + runDeps := newSchemaChangerTxnRunDependencies( + params.ctx, + p.SessionData(), + p.User(), + p.ExecCfg(), + p.Txn(), + ie, + p.Descriptors(), + p.EvalContext(), + p.ExtendedEvalContext().Tracing.KVTracingEnabled(), + scs.jobID, + scs.stmts, + ) + after, jobID, err := scrun.RunStatementPhase( + params.ctx, p.ExecCfg().DeclarativeSchemaChangerTestingKnobs, runDeps, s.plannedState, + ) + if err != nil { + return err + } + scs.state = after + scs.jobID = jobID + return nil + }); err != nil { return err } - scs.state = after - scs.jobID = jobID return nil + } func newSchemaChangerTxnRunDependencies( @@ -295,6 +303,7 @@ func newSchemaChangerTxnRunDependencies( user username.SQLUsername, execCfg *ExecutorConfig, txn *kv.Txn, + ie sqlutil.InternalExecutor, descriptors *descs.Collection, evalContext *eval.Context, kvTrace bool, @@ -314,6 +323,7 @@ func newSchemaChangerTxnRunDependencies( execCfg.Codec, sessionData, txn, + ie, user, descriptors, execCfg.JobRegistry, diff --git a/pkg/sql/schema_changer.go b/pkg/sql/schema_changer.go index 1c3ce9804fac..2d3b2ad7b0c1 100644 --- a/pkg/sql/schema_changer.go +++ b/pkg/sql/schema_changer.go @@ -885,7 +885,9 @@ func (sc *SchemaChanger) handlePermanentSchemaChangeError( // initialize the job running status. func (sc *SchemaChanger) initJobRunningStatus(ctx context.Context) error { - return sc.txn(ctx, func(ctx context.Context, txn *kv.Txn, descriptors *descs.Collection) error { + return sc.txnWithExecutor(ctx, func( + ctx context.Context, txn *kv.Txn, _ *sessiondata.SessionData, descriptors *descs.Collection, ie sqlutil.InternalExecutor, + ) error { flags := tree.ObjectLookupFlagsWithRequired() flags.AvoidLeased = true desc, err := descriptors.GetImmutableTableByID(ctx, txn, sc.descID, flags) @@ -909,7 +911,7 @@ func (sc *SchemaChanger) initJobRunningStatus(ctx context.Context) error { } if runStatus != "" && !desc.Dropped() { if err := sc.job.RunningStatus( - ctx, txn, func(ctx context.Context, details jobspb.Details) (jobs.RunningStatus, error) { + ctx, txn, ie, func(ctx context.Context, details jobspb.Details) (jobs.RunningStatus, error) { return runStatus, nil }); err != nil { return errors.Wrapf(err, "failed to update job status") @@ -1093,8 +1095,8 @@ func (sc *SchemaChanger) RunStateMachineBeforeBackfill(ctx context.Context) erro log.Info(ctx, "stepping through state machine") var runStatus jobs.RunningStatus - if err := sc.txn(ctx, func( - ctx context.Context, txn *kv.Txn, descsCol *descs.Collection, + if err := sc.txnWithExecutor(ctx, func( + ctx context.Context, txn *kv.Txn, _ *sessiondata.SessionData, descsCol *descs.Collection, ie sqlutil.InternalExecutor, ) error { tbl, err := descsCol.GetMutableTableVersionByID(ctx, sc.descID, txn) if err != nil { @@ -1165,7 +1167,7 @@ func (sc *SchemaChanger) RunStateMachineBeforeBackfill(ctx context.Context) erro return err } if sc.job != nil { - if err := sc.job.RunningStatus(ctx, txn, func( + if err := sc.job.RunningStatus(ctx, txn, ie, func( ctx context.Context, details jobspb.Details, ) (jobs.RunningStatus, error) { return runStatus, nil @@ -1207,9 +1209,7 @@ func (sc *SchemaChanger) stepStateMachineAfterIndexBackfill(ctx context.Context) log.Info(ctx, "stepping through state machine") var runStatus jobs.RunningStatus - if err := sc.txn(ctx, func( - ctx context.Context, txn *kv.Txn, descsCol *descs.Collection, - ) error { + if err := sc.txnWithExecutor(ctx, func(ctx context.Context, txn *kv.Txn, _ *sessiondata.SessionData, descsCol *descs.Collection, ie sqlutil.InternalExecutor) error { tbl, err := descsCol.GetMutableTableVersionByID(ctx, sc.descID, txn) if err != nil { return err @@ -1246,7 +1246,7 @@ func (sc *SchemaChanger) stepStateMachineAfterIndexBackfill(ctx context.Context) return err } if sc.job != nil { - if err := sc.job.RunningStatus(ctx, txn, func( + if err := sc.job.RunningStatus(ctx, txn, ie, func( ctx context.Context, details jobspb.Details, ) (jobs.RunningStatus, error) { return runStatus, nil @@ -1935,7 +1935,9 @@ func (sc *SchemaChanger) maybeReverseMutations(ctx context.Context, causingError // Get the other tables whose foreign key backreferences need to be removed. alreadyReversed := false const kvTrace = true // TODO(ajwerner): figure this out - err := sc.txn(ctx, func(ctx context.Context, txn *kv.Txn, descsCol *descs.Collection) error { + err := sc.txnWithExecutor(ctx, func( + ctx context.Context, txn *kv.Txn, _ *sessiondata.SessionData, descsCol *descs.Collection, ie sqlutil.InternalExecutor, + ) error { scTable, err := descsCol.GetMutableTableVersionByID(ctx, sc.descID, txn) if err != nil { return err @@ -2050,7 +2052,7 @@ func (sc *SchemaChanger) maybeReverseMutations(ctx context.Context, causingError tableDesc := scTable.ImmutableCopy().(catalog.TableDescriptor) // Mark the schema change job as failed and create a rollback job. - err = sc.updateJobForRollback(ctx, txn, tableDesc) + err = sc.updateJobForRollback(ctx, txn, ie, tableDesc) if err != nil { return err } @@ -2061,7 +2063,7 @@ func (sc *SchemaChanger) maybeReverseMutations(ctx context.Context, causingError if err != nil { return err } - if err := sc.jobRegistry.Failed(ctx, txn, jobID, causingError); err != nil { + if err := sc.jobRegistry.Failed(ctx, txn, jobID, ie, causingError); err != nil { return err } } @@ -2087,7 +2089,7 @@ func (sc *SchemaChanger) maybeReverseMutations(ctx context.Context, causingError // updateJobForRollback updates the schema change job in the case of a rollback. func (sc *SchemaChanger) updateJobForRollback( - ctx context.Context, txn *kv.Txn, tableDesc catalog.TableDescriptor, + ctx context.Context, txn *kv.Txn, ie sqlutil.InternalExecutor, tableDesc catalog.TableDescriptor, ) error { // Initialize refresh spans to scan the entire table. span := tableDesc.PrimaryIndexSpan(sc.execCfg.Codec) @@ -2103,7 +2105,7 @@ func (sc *SchemaChanger) updateJobForRollback( } oldDetails := sc.job.Details().(jobspb.SchemaChangeDetails) if err := sc.job.SetDetails( - ctx, txn, jobspb.SchemaChangeDetails{ + ctx, txn, ie, jobspb.SchemaChangeDetails{ DescID: sc.descID, TableMutationID: sc.mutationID, ResumeSpanList: spanList, @@ -2112,7 +2114,7 @@ func (sc *SchemaChanger) updateJobForRollback( ); err != nil { return err } - if err := sc.job.SetProgress(ctx, txn, jobspb.SchemaChangeProgress{}); err != nil { + if err := sc.job.SetProgress(ctx, txn, ie, jobspb.SchemaChangeProgress{}); err != nil { return err } diff --git a/pkg/sql/schemachanger/scdeps/exec_deps.go b/pkg/sql/schemachanger/scdeps/exec_deps.go index a4b25abe7daa..0d90eaff621f 100644 --- a/pkg/sql/schemachanger/scdeps/exec_deps.go +++ b/pkg/sql/schemachanger/scdeps/exec_deps.go @@ -35,6 +35,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/sessiondata" "github.com/cockroachdb/cockroach/pkg/sql/sqltelemetry" + "github.com/cockroachdb/cockroach/pkg/sql/sqlutil" "github.com/cockroachdb/cockroach/pkg/util/log" "github.com/cockroachdb/errors" ) @@ -44,9 +45,7 @@ import ( type JobRegistry interface { MakeJobID() jobspb.JobID CreateJobWithTxn(ctx context.Context, record jobs.Record, jobID jobspb.JobID, txn *kv.Txn) (*jobs.Job, error) - UpdateJobWithTxn( - ctx context.Context, jobID jobspb.JobID, txn *kv.Txn, useReadLock bool, updateFunc jobs.UpdateFn, - ) error + UpdateJobWithTxn(ctx context.Context, jobID jobspb.JobID, txn *kv.Txn, ie sqlutil.InternalExecutor, useReadLock bool, updateFunc jobs.UpdateFn) error CheckPausepoint(name string) error } @@ -57,6 +56,7 @@ func NewExecutorDependencies( codec keys.SQLCodec, sessionData *sessiondata.SessionData, txn *kv.Txn, + ie sqlutil.InternalExecutor, user username.SQLUsername, descsCollection *descs.Collection, jobRegistry JobRegistry, @@ -77,6 +77,7 @@ func NewExecutorDependencies( return &execDeps{ txnDeps: txnDeps{ txn: txn, + ie: ie, codec: codec, descsCollection: descsCollection, jobRegistry: jobRegistry, @@ -103,6 +104,7 @@ func NewExecutorDependencies( type txnDeps struct { txn *kv.Txn + ie sqlutil.InternalExecutor codec keys.SQLCodec descsCollection *descs.Collection jobRegistry JobRegistry @@ -122,8 +124,8 @@ func (d *txnDeps) UpdateSchemaChangeJob( ctx context.Context, id jobspb.JobID, callback scexec.JobUpdateCallback, ) error { const useReadLock = false - return d.jobRegistry.UpdateJobWithTxn(ctx, id, d.txn, useReadLock, func( - txn *kv.Txn, md jobs.JobMetadata, ju *jobs.JobUpdater, + return d.jobRegistry.UpdateJobWithTxn(ctx, id, d.txn, d.ie, useReadLock, func( + _ *kv.Txn, _ sqlutil.InternalExecutor, md jobs.JobMetadata, ju *jobs.JobUpdater, ) error { return callback(md, ju.UpdateProgress, ju.UpdatePayload) }) diff --git a/pkg/sql/schemachanger/scdeps/run_deps.go b/pkg/sql/schemachanger/scdeps/run_deps.go index 91a9b9b5cac9..1f539099d132 100644 --- a/pkg/sql/schemachanger/scdeps/run_deps.go +++ b/pkg/sql/schemachanger/scdeps/run_deps.go @@ -26,6 +26,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scrun" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/sessiondata" + "github.com/cockroachdb/cockroach/pkg/sql/sqlutil" "github.com/cockroachdb/cockroach/pkg/util/timeutil" ) @@ -107,13 +108,14 @@ func (d *jobExecutionDeps) ClusterSettings() *cluster.Settings { func (d *jobExecutionDeps) WithTxnInJob(ctx context.Context, fn scrun.JobTxnFunc) error { var createdJobs []jobspb.JobID var tableStatsToRefresh []descpb.ID - err := d.internalExecutorFactory.DescsTxn(ctx, d.db, func( - ctx context.Context, txn *kv.Txn, descriptors *descs.Collection, + err := d.internalExecutorFactory.DescsTxnWithExecutor(ctx, d.db, nil, func( + ctx context.Context, txn *kv.Txn, descriptors *descs.Collection, ie sqlutil.InternalExecutor, ) error { pl := d.job.Payload() ed := &execDeps{ txnDeps: txnDeps{ txn: txn, + ie: ie, codec: d.codec, descsCollection: descriptors, jobRegistry: d.jobRegistry, diff --git a/pkg/sql/schemachanger/scdeps/validator.go b/pkg/sql/schemachanger/scdeps/validator.go index ccd542fe1950..23cb3919ed16 100644 --- a/pkg/sql/schemachanger/scdeps/validator.go +++ b/pkg/sql/schemachanger/scdeps/validator.go @@ -29,6 +29,7 @@ import ( type ValidateForwardIndexesFn func( ctx context.Context, job *jobs.Job, + ief sqlutil.InternalExecutorFactory, tbl catalog.TableDescriptor, indexes []catalog.Index, runHistoricalTxn descs.HistoricalInternalExecTxnRunner, @@ -43,6 +44,7 @@ type ValidateInvertedIndexesFn func( ctx context.Context, codec keys.SQLCodec, job *jobs.Job, + ief sqlutil.InternalExecutorFactory, tbl catalog.TableDescriptor, indexes []catalog.Index, runHistoricalTxn descs.HistoricalInternalExecTxnRunner, @@ -89,7 +91,7 @@ func (vd validator) ValidateForwardIndexes( const withFirstMutationPublic = true const gatherAllInvalid = false return vd.validateForwardIndexes( - ctx, job, tbl, indexes, vd.makeHistoricalInternalExecTxnRunner(), + ctx, job, vd.ieFactory, tbl, indexes, vd.makeHistoricalInternalExecTxnRunner(), withFirstMutationPublic, gatherAllInvalid, override, vd.protectedTimestampProvider, ) } @@ -106,7 +108,7 @@ func (vd validator) ValidateInvertedIndexes( const withFirstMutationPublic = true const gatherAllInvalid = false return vd.validateInvertedIndexes( - ctx, vd.codec, job, tbl, indexes, vd.makeHistoricalInternalExecTxnRunner(), + ctx, vd.codec, job, vd.ieFactory, tbl, indexes, vd.makeHistoricalInternalExecTxnRunner(), withFirstMutationPublic, gatherAllInvalid, override, vd.protectedTimestampProvider, ) } diff --git a/pkg/sql/schemachanger/scexec/BUILD.bazel b/pkg/sql/schemachanger/scexec/BUILD.bazel index 67d7017e304f..24090ec080c8 100644 --- a/pkg/sql/schemachanger/scexec/BUILD.bazel +++ b/pkg/sql/schemachanger/scexec/BUILD.bazel @@ -91,6 +91,7 @@ go_test( "//pkg/sql/sem/catid", # keep "//pkg/sql/sem/tree", "//pkg/sql/sessiondata", + "//pkg/sql/sqlutil", "//pkg/sql/types", "//pkg/testutils/serverutils", "//pkg/testutils/sqlutils", diff --git a/pkg/sql/schemachanger/scexec/backfiller/BUILD.bazel b/pkg/sql/schemachanger/scexec/backfiller/BUILD.bazel index 3cc6cf7463ff..5c612a5235ff 100644 --- a/pkg/sql/schemachanger/scexec/backfiller/BUILD.bazel +++ b/pkg/sql/schemachanger/scexec/backfiller/BUILD.bazel @@ -20,6 +20,7 @@ go_library( "//pkg/sql/backfill", "//pkg/sql/catalog/descpb", "//pkg/sql/schemachanger/scexec", + "//pkg/sql/sqlutil", "//pkg/util", "//pkg/util/syncutil", "//pkg/util/timeutil", diff --git a/pkg/sql/schemachanger/scexec/backfiller/tracker.go b/pkg/sql/schemachanger/scexec/backfiller/tracker.go index cd2353c002e3..630854ff21a9 100644 --- a/pkg/sql/schemachanger/scexec/backfiller/tracker.go +++ b/pkg/sql/schemachanger/scexec/backfiller/tracker.go @@ -21,6 +21,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scexec" + "github.com/cockroachdb/cockroach/pkg/sql/sqlutil" "github.com/cockroachdb/cockroach/pkg/util" "github.com/cockroachdb/cockroach/pkg/util/syncutil" ) @@ -102,15 +103,15 @@ func newTrackerConfig(codec keys.SQLCodec, rc RangeCounter, job *jobs.Job) track numRangesInSpanContainedBy: rc.NumRangesInSpanContainedBy, writeProgressFraction: func(ctx context.Context, fractionProgressed float32) error { if err := job.FractionProgressed( - ctx, nil /* txn */, jobs.FractionUpdater(fractionProgressed), + ctx, nil /* txn */, nil /* ie */, jobs.FractionUpdater(fractionProgressed), ); err != nil { return jobs.SimplifyInvalidStatusError(err) } return nil }, writeCheckpoint: func(ctx context.Context, bps []scexec.BackfillProgress, mps []scexec.MergeProgress) error { - return job.Update(ctx, nil /* txn */, func( - txn *kv.Txn, md jobs.JobMetadata, ju *jobs.JobUpdater, + return job.Update(ctx, nil /* txn */, nil /* ie */, func( + _ *kv.Txn, _ sqlutil.InternalExecutor, md jobs.JobMetadata, ju *jobs.JobUpdater, ) error { pl := md.Payload backfillJobProgress, err := convertToJobBackfillProgress(codec, bps) diff --git a/pkg/sql/schemachanger/scexec/executor_external_test.go b/pkg/sql/schemachanger/scexec/executor_external_test.go index 78377a827cde..893af4968d7b 100644 --- a/pkg/sql/schemachanger/scexec/executor_external_test.go +++ b/pkg/sql/schemachanger/scexec/executor_external_test.go @@ -38,6 +38,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scpb" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/sessiondata" + "github.com/cockroachdb/cockroach/pkg/sql/sqlutil" "github.com/cockroachdb/cockroach/pkg/sql/types" "github.com/cockroachdb/cockroach/pkg/testutils/sqlutils" "github.com/cockroachdb/cockroach/pkg/testutils/testcluster" @@ -59,7 +60,7 @@ type testInfra struct { } func (ti testInfra) newExecDeps( - txn *kv.Txn, descsCollection *descs.Collection, + txn *kv.Txn, ie sqlutil.InternalExecutor, descsCollection *descs.Collection, ) scexec.Dependencies { const kvTrace = true const schemaChangerJobID = 1 @@ -68,6 +69,7 @@ func (ti testInfra) newExecDeps( ti.lm.Codec(), &sessiondata.SessionData{}, txn, + ie, username.RootUserName(), descsCollection, noopJobRegistry{}, @@ -107,6 +109,13 @@ func (ti *testInfra) txn( return ti.ief.DescsTxn(ctx, ti.db, f) } +func (ti *testInfra) txnWithExecutor( + ctx context.Context, + f func(ctx context.Context, txn *kv.Txn, descriptors *descs.Collection, ie sqlutil.InternalExecutor) error, +) error { + return ti.ief.DescsTxnWithExecutor(ctx, ti.db, nil, f) +} + func TestExecutorDescriptorMutationOps(t *testing.T) { defer leaktest.AfterTest(t)() @@ -162,10 +171,10 @@ CREATE TABLE db.t ( return nil })) - require.NoError(t, ti.txn(ctx, func( - ctx context.Context, txn *kv.Txn, descriptors *descs.Collection, + require.NoError(t, ti.txnWithExecutor(ctx, func( + ctx context.Context, txn *kv.Txn, descriptors *descs.Collection, ie sqlutil.InternalExecutor, ) error { - exDeps := ti.newExecDeps(txn, descriptors) + exDeps := ti.newExecDeps(txn, ie, descriptors) _, orig, err := descriptors.GetImmutableTableByName(ctx, txn, &tn, immFlags) require.NoError(t, err) require.Equal(t, c.orig().TableDesc(), orig.TableDesc()) @@ -265,8 +274,8 @@ func TestSchemaChanger(t *testing.T) { ti.tsql.Exec(t, `CREATE TABLE db.foo (i INT PRIMARY KEY)`) var cs scpb.CurrentState - require.NoError(t, ti.txn(ctx, func( - ctx context.Context, txn *kv.Txn, descriptors *descs.Collection, + require.NoError(t, ti.txnWithExecutor(ctx, func( + ctx context.Context, txn *kv.Txn, descriptors *descs.Collection, ie sqlutil.InternalExecutor, ) (err error) { tn := tree.MakeTableNameWithSchema("db", tree.PublicSchemaName, "foo") _, fooTable, err := descriptors.GetImmutableTableByName(ctx, txn, &tn, tree.ObjectLookupFlagsWithRequired()) @@ -337,19 +346,19 @@ func TestSchemaChanger(t *testing.T) { sc := sctestutils.MakePlan(t, initial, scop.PreCommitPhase) stages := sc.StagesForCurrentPhase() for _, s := range stages { - exDeps := ti.newExecDeps(txn, descriptors) + exDeps := ti.newExecDeps(txn, ie, descriptors) require.NoError(t, sc.DecorateErrorWithPlanDetails(scexec.ExecuteStage(ctx, exDeps, s.Ops()))) cs = scpb.CurrentState{TargetState: initial.TargetState, Current: s.After} } return nil })) var after scpb.CurrentState - require.NoError(t, ti.txn(ctx, func( - ctx context.Context, txn *kv.Txn, descriptors *descs.Collection, + require.NoError(t, ti.txnWithExecutor(ctx, func( + ctx context.Context, txn *kv.Txn, descriptors *descs.Collection, ie sqlutil.InternalExecutor, ) error { sc := sctestutils.MakePlan(t, cs, scop.PostCommitPhase) for _, s := range sc.Stages { - exDeps := ti.newExecDeps(txn, descriptors) + exDeps := ti.newExecDeps(txn, ie, descriptors) require.NoError(t, sc.DecorateErrorWithPlanDetails(scexec.ExecuteStage(ctx, exDeps, s.Ops()))) after = scpb.CurrentState{TargetState: cs.TargetState, Current: s.After} } @@ -370,8 +379,8 @@ func TestSchemaChanger(t *testing.T) { ti.tsql.Exec(t, `CREATE TABLE db.foo (i INT PRIMARY KEY)`) var cs scpb.CurrentState - require.NoError(t, ti.txn(ctx, func( - ctx context.Context, txn *kv.Txn, descriptors *descs.Collection, + require.NoError(t, ti.txnWithExecutor(ctx, func( + ctx context.Context, txn *kv.Txn, descriptors *descs.Collection, ie sqlutil.InternalExecutor, ) (err error) { sctestutils.WithBuilderDependenciesFromTestServer(ti.tc.Server(0), func(buildDeps scbuild.Dependencies) { parsed, err := parser.Parse("ALTER TABLE db.foo ADD COLUMN j INT") @@ -383,7 +392,7 @@ func TestSchemaChanger(t *testing.T) { { sc := sctestutils.MakePlan(t, initial, scop.PreCommitPhase) for _, s := range sc.StagesForCurrentPhase() { - exDeps := ti.newExecDeps(txn, descriptors) + exDeps := ti.newExecDeps(txn, ie, descriptors) require.NoError(t, sc.DecorateErrorWithPlanDetails(scexec.ExecuteStage(ctx, exDeps, s.Ops()))) cs = scpb.CurrentState{TargetState: initial.TargetState, Current: s.After} } @@ -391,12 +400,12 @@ func TestSchemaChanger(t *testing.T) { }) return nil })) - require.NoError(t, ti.txn(ctx, func( - ctx context.Context, txn *kv.Txn, descriptors *descs.Collection, + require.NoError(t, ti.txnWithExecutor(ctx, func( + ctx context.Context, txn *kv.Txn, descriptors *descs.Collection, ie sqlutil.InternalExecutor, ) error { sc := sctestutils.MakePlan(t, cs, scop.PostCommitPhase) for _, s := range sc.Stages { - exDeps := ti.newExecDeps(txn, descriptors) + exDeps := ti.newExecDeps(txn, ie, descriptors) require.NoError(t, sc.DecorateErrorWithPlanDetails(scexec.ExecuteStage(ctx, exDeps, s.Ops()))) } return nil @@ -412,7 +421,12 @@ func (n noopJobRegistry) CheckPausepoint(name string) error { } func (n noopJobRegistry) UpdateJobWithTxn( - ctx context.Context, jobID jobspb.JobID, txn *kv.Txn, useReadLock bool, updateFunc jobs.UpdateFn, + ctx context.Context, + jobID jobspb.JobID, + txn *kv.Txn, + ie sqlutil.InternalExecutor, + useReadLock bool, + updateFunc jobs.UpdateFn, ) error { return nil } diff --git a/pkg/sql/schemachanger/scjob/BUILD.bazel b/pkg/sql/schemachanger/scjob/BUILD.bazel index 8ded157a7041..f6e0465e4c04 100644 --- a/pkg/sql/schemachanger/scjob/BUILD.bazel +++ b/pkg/sql/schemachanger/scjob/BUILD.bazel @@ -23,6 +23,7 @@ go_library( "//pkg/sql/schemachanger/scexec", "//pkg/sql/schemachanger/scexec/backfiller", "//pkg/sql/schemachanger/scrun", + "//pkg/sql/sqlutil", "//pkg/util/log", ], ) diff --git a/pkg/sql/schemachanger/scjob/job.go b/pkg/sql/schemachanger/scjob/job.go index 2f3004a202d1..7de360e05971 100644 --- a/pkg/sql/schemachanger/scjob/job.go +++ b/pkg/sql/schemachanger/scjob/job.go @@ -23,6 +23,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scdeps" "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scexec" "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scrun" + "github.com/cockroachdb/cockroach/pkg/sql/sqlutil" "github.com/cockroachdb/cockroach/pkg/util/log" ) @@ -62,7 +63,7 @@ func (n *newSchemaChangeResumer) OnFailOrCancel( func (n *newSchemaChangeResumer) run(ctx context.Context, execCtxI interface{}) error { execCtx := execCtxI.(sql.JobExecContext) execCfg := execCtx.ExecCfg() - if err := n.job.Update(ctx, nil /* txn */, func(txn *kv.Txn, md jobs.JobMetadata, ju *jobs.JobUpdater) error { + if err := n.job.Update(ctx, nil /* txn */, nil /* ie */, func(txn *kv.Txn, ie sqlutil.InternalExecutor, md jobs.JobMetadata, ju *jobs.JobUpdater) error { return nil }); err != nil { // TODO(ajwerner): Detect transient errors and classify as retriable here or diff --git a/pkg/sql/sqlstats/persistedsqlstats/compaction_scheduling.go b/pkg/sql/sqlstats/persistedsqlstats/compaction_scheduling.go index 91fd38685a45..378e36808cf9 100644 --- a/pkg/sql/sqlstats/persistedsqlstats/compaction_scheduling.go +++ b/pkg/sql/sqlstats/persistedsqlstats/compaction_scheduling.go @@ -83,7 +83,11 @@ func CreateSQLStatsCompactionScheduleIfNotYetExist( // We do not need to worry about checking if the job already exist; // at most 1 job semantics are enforced by scheduled jobs system. func CreateCompactionJob( - ctx context.Context, createdByInfo *jobs.CreatedByInfo, txn *kv.Txn, jobRegistry *jobs.Registry, + ctx context.Context, + createdByInfo *jobs.CreatedByInfo, + txn *kv.Txn, + ie sqlutil.InternalExecutor, + jobRegistry *jobs.Registry, ) (jobspb.JobID, error) { record := jobs.Record{ Description: "automatic SQL Stats compaction", @@ -94,7 +98,7 @@ func CreateCompactionJob( } jobID := jobRegistry.MakeJobID() - if _, err := jobRegistry.CreateAdoptableJobWithTxn(ctx, record, jobID, txn); err != nil { + if _, err := jobRegistry.CreateAdoptableJobWithTxn(ctx, record, jobID, txn, ie); err != nil { return jobspb.InvalidJobID, err } return jobID, nil diff --git a/pkg/sql/sqlstats/persistedsqlstats/compaction_test.go b/pkg/sql/sqlstats/persistedsqlstats/compaction_test.go index a4c0ef6a137e..d03ed7ad3f54 100644 --- a/pkg/sql/sqlstats/persistedsqlstats/compaction_test.go +++ b/pkg/sql/sqlstats/persistedsqlstats/compaction_test.go @@ -380,7 +380,7 @@ func TestSQLStatsCompactionJobMarkedAsAutomatic(t *testing.T) { func launchSQLStatsCompactionJob(server serverutils.TestServerInterface) (jobspb.JobID, error) { return persistedsqlstats.CreateCompactionJob( - context.Background(), nil /* createdByInfo */, nil, /* txn */ + context.Background(), nil /* createdByInfo */, nil /* txn */, nil, /* ie */ server.JobRegistry().(*jobs.Registry), ) } diff --git a/pkg/sql/truncate.go b/pkg/sql/truncate.go index 6e117e7d7fef..d732908e9745 100644 --- a/pkg/sql/truncate.go +++ b/pkg/sql/truncate.go @@ -31,6 +31,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scerrors" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/sessiondata" + "github.com/cockroachdb/cockroach/pkg/sql/sqlutil" "github.com/cockroachdb/cockroach/pkg/util/errorutil/unimplemented" "github.com/cockroachdb/cockroach/pkg/util/log" "github.com/cockroachdb/cockroach/pkg/util/log/eventpb" @@ -264,8 +265,11 @@ func (p *planner) truncateTable(ctx context.Context, id descpb.ID, jobDesc strin ctx, clusterversion.V22_2UseDelRangeInGCJob, ), ) - if _, err := p.ExecCfg().JobRegistry.CreateAdoptableJobWithTxn( - ctx, record, p.ExecCfg().JobRegistry.MakeJobID(), p.txn); err != nil { + if err := p.WithInternalExecutor(ctx, func(ctx context.Context, txn *kv.Txn, ie sqlutil.InternalExecutor) error { + _, err = p.ExecCfg().JobRegistry.CreateAdoptableJobWithTxn( + ctx, record, p.ExecCfg().JobRegistry.MakeJobID(), p.txn, ie) + return err + }); err != nil { return err } diff --git a/pkg/sql/ttl/ttljob/ttljob.go b/pkg/sql/ttl/ttljob/ttljob.go index bc86ec6fa419..24a1af0c51f2 100644 --- a/pkg/sql/ttl/ttljob/ttljob.go +++ b/pkg/sql/ttl/ttljob/ttljob.go @@ -31,6 +31,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/physicalplan" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/sqltelemetry" + "github.com/cockroachdb/cockroach/pkg/sql/sqlutil" "github.com/cockroachdb/cockroach/pkg/sql/ttl/ttlbase" "github.com/cockroachdb/cockroach/pkg/sql/types" "github.com/cockroachdb/cockroach/pkg/util/ctxgroup" @@ -258,8 +259,9 @@ func (t rowLevelTTLResumer) Resume(ctx context.Context, execCtx interface{}) err ctx, jobID, nil, /* txn */ + nil, /* ie */ true, /* useReadLock */ - func(_ *kv.Txn, md jobs.JobMetadata, ju *jobs.JobUpdater) error { + func(_ *kv.Txn, _ sqlutil.InternalExecutor, md jobs.JobMetadata, ju *jobs.JobUpdater) error { progress := md.Progress rowLevelTTL := progress.Details.(*jobspb.Progress_RowLevelTTL).RowLevelTTL rowLevelTTL.JobSpanCount = int64(jobSpanCount) diff --git a/pkg/sql/ttl/ttljob/ttljob_processor.go b/pkg/sql/ttl/ttljob/ttljob_processor.go index 06182d87069c..d7add3b0ef02 100644 --- a/pkg/sql/ttl/ttljob/ttljob_processor.go +++ b/pkg/sql/ttl/ttljob/ttljob_processor.go @@ -28,6 +28,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/sessiondata" "github.com/cockroachdb/cockroach/pkg/sql/sessiondatapb" + "github.com/cockroachdb/cockroach/pkg/sql/sqlutil" "github.com/cockroachdb/cockroach/pkg/sql/types" "github.com/cockroachdb/cockroach/pkg/util/ctxgroup" "github.com/cockroachdb/cockroach/pkg/util/log" @@ -182,8 +183,9 @@ func (t *ttlProcessor) work(ctx context.Context) error { ctx, jobID, nil, /* txn */ + nil, /* ie */ true, /* useReadLock */ - func(_ *kv.Txn, md jobs.JobMetadata, ju *jobs.JobUpdater) error { + func(_ *kv.Txn, _ sqlutil.InternalExecutor, md jobs.JobMetadata, ju *jobs.JobUpdater) error { progress := md.Progress rowLevelTTL := progress.Details.(*jobspb.Progress_RowLevelTTL).RowLevelTTL rowLevelTTL.JobRowCount += processorRowCount diff --git a/pkg/sql/ttl/ttlschedule/ttlschedule.go b/pkg/sql/ttl/ttlschedule/ttlschedule.go index 7dd15c9a631b..a812b44e6acf 100644 --- a/pkg/sql/ttl/ttlschedule/ttlschedule.go +++ b/pkg/sql/ttl/ttlschedule/ttlschedule.go @@ -58,6 +58,7 @@ func (s rowLevelTTLExecutor) OnDrop( schedule *jobs.ScheduledJob, txn *kv.Txn, descsCol *descs.Collection, + ie sqlutil.InternalExecutor, ) (int, error) { var args catpb.ScheduledRowLevelTTLArgs @@ -126,6 +127,7 @@ func (s rowLevelTTLExecutor) ExecuteJob( env scheduledjobs.JobSchedulerEnv, sj *jobs.ScheduledJob, txn *kv.Txn, + ie sqlutil.InternalExecutor, ) error { args := &catpb.ScheduledRowLevelTTLArgs{} if err := pbtypes.UnmarshalAny(sj.ExecutionArgs().Args, args); err != nil { @@ -146,6 +148,7 @@ func (s rowLevelTTLExecutor) ExecuteJob( Name: jobs.CreatedByScheduledJobs, }, txn, + ie, p.(sql.PlanHookState).ExtendedEvalContext().Descs, p.(sql.PlanHookState).ExecCfg().JobRegistry, *args, @@ -242,6 +245,7 @@ func createRowLevelTTLJob( ctx context.Context, createdByInfo *jobs.CreatedByInfo, txn *kv.Txn, + ie sqlutil.InternalExecutor, descsCol *descs.Collection, jobRegistry *jobs.Registry, ttlArgs catpb.ScheduledRowLevelTTLArgs, @@ -267,7 +271,7 @@ func createRowLevelTTLJob( } jobID := jobRegistry.MakeJobID() - if _, err := jobRegistry.CreateAdoptableJobWithTxn(ctx, record, jobID, txn); err != nil { + if _, err := jobRegistry.CreateAdoptableJobWithTxn(ctx, record, jobID, txn, ie); err != nil { return jobspb.InvalidJobID, err } return jobID, nil diff --git a/pkg/streaming/BUILD.bazel b/pkg/streaming/BUILD.bazel index e1ce04c319c8..e715db53682e 100644 --- a/pkg/streaming/BUILD.bazel +++ b/pkg/streaming/BUILD.bazel @@ -10,6 +10,7 @@ go_library( "//pkg/ccl/streamingccl/streampb", "//pkg/kv", "//pkg/sql/sem/eval", + "//pkg/sql/sqlutil", "@com_github_cockroachdb_errors//:errors", ], ) diff --git a/pkg/streaming/api.go b/pkg/streaming/api.go index 2d70e4324c12..8559ec2d0090 100644 --- a/pkg/streaming/api.go +++ b/pkg/streaming/api.go @@ -16,6 +16,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/ccl/streamingccl/streampb" "github.com/cockroachdb/cockroach/pkg/kv" "github.com/cockroachdb/cockroach/pkg/sql/sem/eval" + "github.com/cockroachdb/cockroach/pkg/sql/sqlutil" "github.com/cockroachdb/errors" ) @@ -24,28 +25,28 @@ type StreamID = streampb.StreamID // GetReplicationStreamManagerHook is the hook to get access to the producer side replication APIs. // Used by builtin functions to trigger streaming replication. -var GetReplicationStreamManagerHook func(ctx context.Context, evalCtx *eval.Context, txn *kv.Txn) (eval.ReplicationStreamManager, error) +var GetReplicationStreamManagerHook func(ctx context.Context, evalCtx *eval.Context, txn *kv.Txn, ie sqlutil.InternalExecutor) (eval.ReplicationStreamManager, error) // GetStreamIngestManagerHook is the hook to get access to the ingestion side replication APIs. // Used by builtin functions to trigger streaming replication. -var GetStreamIngestManagerHook func(ctx context.Context, evalCtx *eval.Context, txn *kv.Txn) (eval.StreamIngestManager, error) +var GetStreamIngestManagerHook func(ctx context.Context, evalCtx *eval.Context, txn *kv.Txn, ie sqlutil.InternalExecutor) (eval.StreamIngestManager, error) // GetReplicationStreamManager returns a ReplicationStreamManager if a CCL binary is loaded. func GetReplicationStreamManager( - ctx context.Context, evalCtx *eval.Context, txn *kv.Txn, + ctx context.Context, evalCtx *eval.Context, txn *kv.Txn, ie sqlutil.InternalExecutor, ) (eval.ReplicationStreamManager, error) { if GetReplicationStreamManagerHook == nil { return nil, errors.New("replication streaming requires a CCL binary") } - return GetReplicationStreamManagerHook(ctx, evalCtx, txn) + return GetReplicationStreamManagerHook(ctx, evalCtx, txn, ie) } // GetStreamIngestManager returns a StreamIngestManager if a CCL binary is loaded. func GetStreamIngestManager( - ctx context.Context, evalCtx *eval.Context, txn *kv.Txn, + ctx context.Context, evalCtx *eval.Context, txn *kv.Txn, ie sqlutil.InternalExecutor, ) (eval.StreamIngestManager, error) { if GetReplicationStreamManagerHook == nil { return nil, errors.New("replication streaming requires a CCL binary") } - return GetStreamIngestManagerHook(ctx, evalCtx, txn) + return GetStreamIngestManagerHook(ctx, evalCtx, txn, ie) } diff --git a/pkg/upgrade/upgrades/BUILD.bazel b/pkg/upgrade/upgrades/BUILD.bazel index c6ca851fd97d..49926f9c1233 100644 --- a/pkg/upgrade/upgrades/BUILD.bazel +++ b/pkg/upgrade/upgrades/BUILD.bazel @@ -130,6 +130,7 @@ go_test( "//pkg/sql/sem/builtins/builtinconstants", "//pkg/sql/sem/catconstants", "//pkg/sql/sem/tree", + "//pkg/sql/sqlutil", "//pkg/sql/tests", "//pkg/sql/types", "//pkg/storage", diff --git a/pkg/upgrade/upgrades/schema_changes_external_test.go b/pkg/upgrade/upgrades/schema_changes_external_test.go index 72829fd04787..d39c6813d301 100644 --- a/pkg/upgrade/upgrades/schema_changes_external_test.go +++ b/pkg/upgrade/upgrades/schema_changes_external_test.go @@ -30,6 +30,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/catalog/descs" "github.com/cockroachdb/cockroach/pkg/sql/catalog/systemschema" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" + "github.com/cockroachdb/cockroach/pkg/sql/sqlutil" "github.com/cockroachdb/cockroach/pkg/testutils" "github.com/cockroachdb/cockroach/pkg/testutils/serverutils" "github.com/cockroachdb/cockroach/pkg/testutils/skip" @@ -453,11 +454,12 @@ CREATE TABLE test.test_table ( func cancelJob( t *testing.T, ctx context.Context, s serverutils.TestServerInterface, jobID jobspb.JobID, ) { - err := s.DB().Txn(ctx, func(ctx context.Context, txn *kv.Txn) error { + ief := s.InternalExecutorFactory().(sqlutil.InternalExecutorFactory) + err := ief.TxnWithExecutor(ctx, s.DB(), nil /* sessionData */, func(ctx context.Context, txn *kv.Txn, ie sqlutil.InternalExecutor) error { // Using this way of canceling because the migration job us non-cancelable. // Canceling in this way skips the check. return s.JobRegistry().(*jobs.Registry).UpdateJobWithTxn( - ctx, jobID, txn, false /* useReadLock */, func(txn *kv.Txn, md jobs.JobMetadata, ju *jobs.JobUpdater, + ctx, jobID, txn, ie, false /* useReadLock */, func(txn *kv.Txn, _ sqlutil.InternalExecutor, md jobs.JobMetadata, ju *jobs.JobUpdater, ) error { ju.UpdateStatus(jobs.StatusCancelRequested) return nil diff --git a/pkg/util/tracing/zipper/zipper.go b/pkg/util/tracing/zipper/zipper.go index 1be2d098adfd..19e1251b17b4 100644 --- a/pkg/util/tracing/zipper/zipper.go +++ b/pkg/util/tracing/zipper/zipper.go @@ -53,7 +53,7 @@ type InflightTraceZipper interface { type InternalInflightTraceZipper struct { traceStrBuf *bytes.Buffer nodeTraceCollection *tracingpb.TraceCollection - ie sqlutil.InternalExecutor + ieNotBoundToTxn sqlutil.InternalExecutor z *memzipper.Zipper } @@ -78,7 +78,7 @@ func (i *InternalInflightTraceZipper) getZipper() *memzipper.Zipper { func (i *InternalInflightTraceZipper) Zip( ctx context.Context, traceID int64, ) (zipBytes []byte, retErr error) { - it, err := i.ie.QueryIterator(ctx, "internal-zipper", nil, fmt.Sprintf(inflightTracesQuery, traceID)) + it, err := i.ieNotBoundToTxn.QueryIterator(ctx, "internal-zipper", nil, fmt.Sprintf(inflightTracesQuery, traceID)) if err != nil { return nil, err } @@ -192,7 +192,7 @@ func MakeInternalExecutorInflightTraceZipper( t := &InternalInflightTraceZipper{ traceStrBuf: &bytes.Buffer{}, nodeTraceCollection: nil, - ie: ie, + ieNotBoundToTxn: ie, } t.z = &memzipper.Zipper{} t.z.Init()