Skip to content

Commit

Permalink
Browse files Browse the repository at this point in the history
52758: backupccl: add SHOW BACKUPS IN Y and SHOW BACKUP x IN Y r=dt a=dt

This change teaches SHOW BACKUP about collections, specifically adding
support for listing the backups in a collection with SHOW BACKUPS IN dest
and for showing an individual backup with dest using SHOW BACKUP x IN dest.
The former is a thin wrapper around 'ls' on the destination while the latter
is an even thinner wrapper around 'SHOW BACKUP dest/x', but both should make
it easier to interact with backups in collections, which often have long and
cumbersome URIs (including keys), so the more we can keep the collection URI
constant and let you just specify which element in the collection you want
to SHOW, RESTORE, etc, the easier it will be.

Release note (enterprise change): SHOW BACKUPS can be used to list backups in a backup collection created by BACKUP INTO.

52892: geo: implement ST_PointFromGeoHash({text,int4}) -> geometry r=otan a=himanshuchawla009

fixes: #48820
This PR adds a function to calculate the convert 
geohash representation to point representation,
imitating the PostGIS funcitonality.

if no precision is specified ST_PointFromGeoHash returns a point based on full precision of the input GeoHash string.
If precision is specified ST_PointFromGeoHash will use that many characters from the GeoHash to create the point.

Release note (sql change): Implements ST_PointFromGeoHash
which converts geohash to point

52990: sql: gate certain box2d comparison ops by a cluster setting r=rytaft a=otan

Release note (sql change): box2d comparison operators are now gated by
the cluster setting
`sql.spatial.experimental_box2d_comparison_operators.enabled`.



52993: bulkio: Dry run backup when creating schedules. r=miretskiy a=miretskiy

Fixes #52924

Dry run execution of the scheduled backup.

Invoking backup planning and execution in dry run ensures
that the scheduled backup is sane since all of the required
sanity checks are performed when attempting to run the backup.
The actual effects of this dry run are removed by rolling back
to the safepoint.

Release Notes: None

53003: sql: more EXPLAIN improvements r=RaduBerinde a=RaduBerinde

#### sql: omit various EXPLAIN fields in non-verbose mode

This change removes some unnecessary fields, and relegates others to the VERBOSE
variant.

Release note (sql change): various improvements to EXPLAIN plans.

#### sql: improve EXPLAIN output for join nodes

This change folds the type of join (inner/outer/etc) into the node name itself
(e.g. `hash join (left outer)`). The "inner" type is omitted in non-verbose
mode.

Release note (sql change): various improvements to EXPLAIN plans.

#### sql: improve EXPLAIN output for scalar group by

Move "scalar" to the node name instead of a separate field.

Release note (sql change): various improvements to EXPLAIN plans.

#### sql: improve EXPLAIN output for set operations

Improve the node names for set operation: `union/intersect/except [all]`.

Release note (sql change): various improvements in EXPLAIN plans.


53032: pkg/jobs: skip flaky TestJobSchedulerDaemonProcessesJobs r=jbowens a=jbowens

See #52959.

Release note: None

Co-authored-by: David Taylor <[email protected]>
Co-authored-by: himanshuchawla009 <[email protected]>
Co-authored-by: Oliver Tan <[email protected]>
Co-authored-by: Yevgeniy Miretskiy <[email protected]>
Co-authored-by: Radu Berinde <[email protected]>
Co-authored-by: Jackson Owens <[email protected]>
  • Loading branch information
7 people committed Aug 19, 2020
7 parents 697e455 + 3f1c901 + 6278dec + d60618d + 696b9e5 + 266c711 + 9c8d12f commit 935ae2e
Show file tree
Hide file tree
Showing 56 changed files with 4,917 additions and 5,049 deletions.
1 change: 1 addition & 0 deletions docs/generated/settings/settings.html
Original file line number Diff line number Diff line change
Expand Up @@ -57,6 +57,7 @@
<tr><td><code>sql.metrics.statement_details.threshold</code></td><td>duration</td><td><code>0s</code></td><td>minimum execution time to cause statistics to be collected</td></tr>
<tr><td><code>sql.metrics.transaction_details.enabled</code></td><td>boolean</td><td><code>true</code></td><td>collect per-application transaction statistics</td></tr>
<tr><td><code>sql.notices.enabled</code></td><td>boolean</td><td><code>true</code></td><td>enable notices in the server/client protocol being sent</td></tr>
<tr><td><code>sql.spatial.experimental_box2d_comparison_operators.enabled</code></td><td>boolean</td><td><code>false</code></td><td>enables the use of certain experimental box2d comparison operators</td></tr>
<tr><td><code>sql.stats.automatic_collection.enabled</code></td><td>boolean</td><td><code>true</code></td><td>automatic statistics collection mode</td></tr>
<tr><td><code>sql.stats.automatic_collection.fraction_stale_rows</code></td><td>float</td><td><code>0.2</code></td><td>target fraction of stale rows per table that will trigger a statistics refresh</td></tr>
<tr><td><code>sql.stats.automatic_collection.min_stale_rows</code></td><td>integer</td><td><code>500</code></td><td>target minimum number of stale rows per table that will trigger a statistics refresh</td></tr>
Expand Down
1 change: 1 addition & 0 deletions docs/generated/sql/bnf/show_backup.bnf
Original file line number Diff line number Diff line change
@@ -1,3 +1,4 @@
show_backup_stmt ::=
'SHOW' 'BACKUP' location opt_with_options
| 'SHOW' 'BACKUP' location 'IN' location opt_with_options
| 'SHOW' 'BACKUP' 'SCHEMAS' location opt_with_options
5 changes: 4 additions & 1 deletion docs/generated/sql/bnf/stmt_block.bnf
Original file line number Diff line number Diff line change
Expand Up @@ -539,7 +539,9 @@ use_stmt ::=
'USE' var_value

show_backup_stmt ::=
'SHOW' 'BACKUP' string_or_placeholder opt_with_options
'SHOW' 'BACKUPS' 'IN' string_or_placeholder
| 'SHOW' 'BACKUP' string_or_placeholder opt_with_options
| 'SHOW' 'BACKUP' string_or_placeholder 'IN' string_or_placeholder opt_with_options
| 'SHOW' 'BACKUP' 'SCHEMAS' string_or_placeholder opt_with_options

show_columns_stmt ::=
Expand Down Expand Up @@ -714,6 +716,7 @@ unreserved_keyword ::=
| 'AUTOMATIC'
| 'AUTHORIZATION'
| 'BACKUP'
| 'BACKUPS'
| 'BEFORE'
| 'BEGIN'
| 'BINARY'
Expand Down
4 changes: 4 additions & 0 deletions docs/generated/sql/functions.md
Original file line number Diff line number Diff line change
Expand Up @@ -1450,6 +1450,10 @@ calculated, the result is transformed back into a Geography with SRID 4326.</p>
</span></td></tr>
<tr><td><a name="st_point"></a><code>st_point(x: <a href="float.html">float</a>, y: <a href="float.html">float</a>) &rarr; geometry</code></td><td><span class="funcdesc"><p>Returns a new Point with the given X and Y coordinates.</p>
</span></td></tr>
<tr><td><a name="st_pointfromgeohash"></a><code>st_pointfromgeohash(geohash: <a href="string.html">string</a>) &rarr; geometry</code></td><td><span class="funcdesc"><p>Return a Geometry point from a GeoHash string with max precision.</p>
</span></td></tr>
<tr><td><a name="st_pointfromgeohash"></a><code>st_pointfromgeohash(geohash: <a href="string.html">string</a>, precision: <a href="int.html">int</a>) &rarr; geometry</code></td><td><span class="funcdesc"><p>Return a Geometry point from a GeoHash string with supplied precision.</p>
</span></td></tr>
<tr><td><a name="st_pointfromtext"></a><code>st_pointfromtext(str: <a href="string.html">string</a>, srid: <a href="int.html">int</a>) &rarr; geometry</code></td><td><span class="funcdesc"><p>Returns the Geometry from a WKT or EWKT representation with an SRID. If the shape underneath is not Point, NULL is returned. If the SRID is present in both the EWKT and the argument, the argument value is used.</p>
</span></td></tr>
<tr><td><a name="st_pointfromtext"></a><code>st_pointfromtext(val: <a href="string.html">string</a>) &rarr; geometry</code></td><td><span class="funcdesc"><p>Returns the Geometry from a WKT or EWKT representation. If the shape underneath is not Point, NULL is returned.</p>
Expand Down
65 changes: 32 additions & 33 deletions pkg/ccl/backupccl/create_scheduled_backup.go
Original file line number Diff line number Diff line change
Expand Up @@ -244,24 +244,8 @@ func doCreateBackupSchedules(
AppendToLatest: false,
}

if backupNode.Options.CaptureRevisionHistory && !eval.isEnterpriseUser {
// TODO(yevgeniy): Pull license check logic into a common helper.
if err := utilccl.CheckEnterpriseEnabled(
p.ExecCfg().Settings, p.ExecCfg().ClusterID(), p.ExecCfg().Organization(),
"revision_history"); err != nil {
return err
}
}

// Evaluate encryption passphrase if set.
if eval.encryptionPassphrase != nil {
if !eval.isEnterpriseUser {
if err := utilccl.CheckEnterpriseEnabled(
p.ExecCfg().Settings, p.ExecCfg().ClusterID(), p.ExecCfg().Organization(),
"encryption"); err != nil {
return err
}
}
pw, err := eval.encryptionPassphrase()
if err != nil {
return errors.Wrapf(err, "failed to evaluate backup encryption_passphrase")
Expand All @@ -275,21 +259,18 @@ func doCreateBackupSchedules(
return errors.Wrapf(err, "failed to evaluate backup destination paths")
}

if len(destinations) > 1 {
if !eval.isEnterpriseUser {
if err := utilccl.CheckEnterpriseEnabled(
p.ExecCfg().Settings, p.ExecCfg().ClusterID(), p.ExecCfg().Organization(),
"partitioned destinations"); err != nil {
return err
}
}
}
for _, dest := range destinations {
backupNode.To = append(backupNode.To, tree.NewDString(dest))
}

backupNode.Targets = eval.Targets

// Run full backup in dry-run mode. This will do all of the sanity checks
// and validation we need to make in order to ensure the schedule is sane.
if err := dryRunBackup(ctx, p, backupNode); err != nil {
return errors.Wrapf(err, "failed to dry run backup")
}

var fullScheduleName string
if eval.scheduleName != nil {
scheduleName, err := eval.scheduleName()
Expand Down Expand Up @@ -389,10 +370,6 @@ func createBackupSchedule(
sj.SetNextRun(*firstRun)
}

// TODO(yevgeniy): Validate backup schedule:
// * Verify targets exist. Provide a way for user to override this via option.
// * Verify destination paths sane (i.e. valid schema://, etc)

// We do not set backupNode.AsOf: this is done when the scheduler kicks off the backup.
// Serialize backup statement and set schedule executor and its args.
args.BackupStatement = tree.AsString(backupNode)
Expand Down Expand Up @@ -431,6 +408,28 @@ func createBackupSchedule(
return nil
}

// dryRunBackup executes backup in dry-run mode: we simply execute backup
// under transaction savepoint, and then rollback to that save point.
func dryRunBackup(ctx context.Context, p sql.PlanHookState, backupNode *tree.Backup) error {
sp, err := p.ExtendedEvalContext().Txn.CreateSavepoint(ctx)
if err != nil {
return err
}
err = dryRunInvokeBackup(ctx, p, backupNode)
if rollbackErr := p.ExtendedEvalContext().Txn.RollbackToSavepoint(ctx, sp); rollbackErr != nil {
return rollbackErr
}
return err
}

func dryRunInvokeBackup(ctx context.Context, p sql.PlanHookState, backupNode *tree.Backup) error {
backupFn, err := planBackup(ctx, p, backupNode)
if err != nil {
return err
}
return invokeBackup(ctx, backupFn)
}

// makeScheduleBackupEval prepares helper scheduledBackupEval struct to assist in evaluation
// of various schedule and backup specific components.
func makeScheduledBackupEval(
Expand Down Expand Up @@ -530,10 +529,6 @@ func createBackupScheduleHook(
return fn, scheduledBackupHeader, nil, false, nil
}

func init() {
sql.AddPlanHook(createBackupScheduleHook)
}

// MarshalJSONPB provides a custom Marshaller for jsonpb that redacts secrets in
// URI fields.
func (m ScheduledBackupExecutionArgs) MarshalJSONPB(x *jsonpb.Marshaler) ([]byte, error) {
Expand Down Expand Up @@ -591,3 +586,7 @@ func (m ScheduledBackupExecutionArgs) MarshalJSONPB(x *jsonpb.Marshaler) ([]byte
m.BackupStatement = backup.String()
return json.Marshal(m)
}

func init() {
sql.AddPlanHook(createBackupScheduleHook)
}
63 changes: 33 additions & 30 deletions pkg/ccl/backupccl/create_scheduled_backup_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -193,121 +193,121 @@ func TestSerializesScheduledBackupExecutionArgs(t *testing.T) {
}{
{
name: "full-cluster",
query: "CREATE SCHEDULE FOR BACKUP INTO 'somewhere?AWS_SECRET_ACCESS_KEY=neverappars' RECURRING '@hourly'",
query: "CREATE SCHEDULE FOR BACKUP INTO 'nodelocal://0/backup?AWS_SECRET_ACCESS_KEY=neverappears' RECURRING '@hourly'",
user: freeUser,
expectedSchedules: []expectedSchedule{
{
nameRe: "BACKUP .+",
backupStmt: "BACKUP INTO 'somewhere?AWS_SECRET_ACCESS_KEY=neverappars' WITH detached",
shownStmt: "BACKUP INTO 'somewhere?AWS_SECRET_ACCESS_KEY=redacted' WITH detached",
backupStmt: "BACKUP INTO 'nodelocal://0/backup?AWS_SECRET_ACCESS_KEY=neverappears' WITH detached",
shownStmt: "BACKUP INTO 'nodelocal://0/backup?AWS_SECRET_ACCESS_KEY=redacted' WITH detached",
period: time.Hour,
},
},
},
{
name: "full-cluster-with-name",
query: "CREATE SCHEDULE 'my-backup' FOR BACKUP INTO 'somewhere' RECURRING '@hourly'",
query: "CREATE SCHEDULE 'my-backup' FOR BACKUP INTO 'nodelocal://0/backup' RECURRING '@hourly'",
user: freeUser,
expectedSchedules: []expectedSchedule{
{
nameRe: "my-backup",
backupStmt: "BACKUP INTO 'somewhere' WITH detached",
backupStmt: "BACKUP INTO 'nodelocal://0/backup' WITH detached",
period: time.Hour,
},
},
},
{
name: "full-cluster-always",
query: "CREATE SCHEDULE FOR BACKUP INTO 'somewhere' RECURRING '@hourly' FULL BACKUP ALWAYS",
query: "CREATE SCHEDULE FOR BACKUP INTO 'nodelocal://0/backup' RECURRING '@hourly' FULL BACKUP ALWAYS",
user: freeUser,
expectedSchedules: []expectedSchedule{
{
nameRe: "BACKUP .+",
backupStmt: "BACKUP INTO 'somewhere' WITH detached",
backupStmt: "BACKUP INTO 'nodelocal://0/backup' WITH detached",
period: time.Hour,
},
},
},
{
name: "full-cluster",
query: "CREATE SCHEDULE FOR BACKUP INTO 'somewhere' RECURRING '@hourly'",
query: "CREATE SCHEDULE FOR BACKUP INTO 'nodelocal://0/backup' RECURRING '@hourly'",
user: enterpriseUser,
expectedSchedules: []expectedSchedule{
{
nameRe: "BACKUP .*: INCREMENTAL",
backupStmt: "BACKUP INTO LATEST IN 'somewhere' WITH detached",
backupStmt: "BACKUP INTO LATEST IN 'nodelocal://0/backup' WITH detached",
period: time.Hour,
},
{
nameRe: "BACKUP .+",
backupStmt: "BACKUP INTO 'somewhere' WITH detached",
backupStmt: "BACKUP INTO 'nodelocal://0/backup' WITH detached",
period: 24 * time.Hour,
runsNow: true,
},
},
},
{
name: "full-cluster-with-name",
query: "CREATE SCHEDULE 'my-backup' FOR BACKUP INTO 'somewhere' RECURRING '@hourly'",
query: "CREATE SCHEDULE 'my-backup' FOR BACKUP INTO 'nodelocal://0/backup' RECURRING '@hourly'",
user: enterpriseUser,
expectedSchedules: []expectedSchedule{
{
nameRe: "my-backup: INCREMENTAL",
backupStmt: "BACKUP INTO LATEST IN 'somewhere' WITH detached",
backupStmt: "BACKUP INTO LATEST IN 'nodelocal://0/backup' WITH detached",
period: time.Hour,
},
{
nameRe: "my-backup",
backupStmt: "BACKUP INTO 'somewhere' WITH detached",
backupStmt: "BACKUP INTO 'nodelocal://0/backup' WITH detached",
period: 24 * time.Hour,
runsNow: true,
},
},
},
{
name: "full-cluster-always",
query: "CREATE SCHEDULE FOR BACKUP INTO 'somewhere' RECURRING '@hourly' FULL BACKUP ALWAYS",
query: "CREATE SCHEDULE FOR BACKUP INTO 'nodelocal://0/backup' RECURRING '@hourly' FULL BACKUP ALWAYS",
user: enterpriseUser,
expectedSchedules: []expectedSchedule{
{
nameRe: "BACKUP .+",
backupStmt: "BACKUP INTO 'somewhere' WITH detached",
backupStmt: "BACKUP INTO 'nodelocal://0/backup' WITH detached",
period: time.Hour,
},
},
},
{
name: "enterprise-license-required-for-incremental",
query: "CREATE SCHEDULE FOR BACKUP INTO 'somewhere' RECURRING '@hourly' FULL BACKUP '@weekly'",
query: "CREATE SCHEDULE FOR BACKUP INTO 'nodelocal://0/backup' RECURRING '@hourly' FULL BACKUP '@weekly'",
user: freeUser,
errMsg: "use of BACKUP INTO LATEST requires an enterprise license",
},
{
name: "enterprise-license-required-for-revision-history",
query: "CREATE SCHEDULE FOR BACKUP INTO 'somewhere' WITH revision_history RECURRING '@hourly'",
query: "CREATE SCHEDULE FOR BACKUP INTO 'nodelocal://0/backup' WITH revision_history RECURRING '@hourly'",
user: freeUser,
errMsg: "use of revision_history requires an enterprise license",
errMsg: "use of BACKUP with revision_history requires an enterprise license",
},
{
name: "enterprise-license-required-for-encryption",
query: "CREATE SCHEDULE FOR BACKUP INTO 'somewhere' WITH encryption_passphrase='secret' RECURRING '@hourly'",
query: "CREATE SCHEDULE FOR BACKUP INTO 'nodelocal://0/backup' WITH encryption_passphrase='secret' RECURRING '@hourly'",
user: freeUser,
errMsg: "use of encryption requires an enterprise license",
errMsg: "use of BACKUP with encryption requires an enterprise license",
},
{
name: "full-cluster-with-name-arg",
query: `CREATE SCHEDULE $1 FOR BACKUP INTO 'somewhere' WITH revision_history, detached RECURRING '@hourly'`,
query: `CREATE SCHEDULE $1 FOR BACKUP INTO 'nodelocal://0/backup' WITH revision_history, detached RECURRING '@hourly'`,
queryArgs: []interface{}{"my_backup_name"},
user: enterpriseUser,
expectedSchedules: []expectedSchedule{
{
nameRe: "my_backup_name: INCREMENTAL",
backupStmt: "BACKUP INTO LATEST IN 'somewhere' WITH revision_history, detached",
backupStmt: "BACKUP INTO LATEST IN 'nodelocal://0/backup' WITH revision_history, detached",
period: time.Hour,
},
{
nameRe: "my_backup_name",
backupStmt: "BACKUP INTO 'somewhere' WITH revision_history, detached",
backupStmt: "BACKUP INTO 'nodelocal://0/backup' WITH revision_history, detached",
period: 24 * time.Hour,
runsNow: true,
},
Expand All @@ -317,13 +317,13 @@ func TestSerializesScheduledBackupExecutionArgs(t *testing.T) {
name: "multiple-tables-with-encryption",
user: enterpriseUser,
query: `
CREATE SCHEDULE FOR BACKUP TABLE db.*, other_db.foo INTO 'somewhere'
CREATE SCHEDULE FOR BACKUP TABLE system.jobs, system.scheduled_jobs INTO 'nodelocal://0/backup'
WITH encryption_passphrase='secret' RECURRING '@weekly'`,
expectedSchedules: []expectedSchedule{
{
nameRe: "BACKUP .*",
backupStmt: "BACKUP TABLE db.*, other_db.foo INTO 'somewhere' WITH encryption_passphrase='secret', detached",
shownStmt: "BACKUP TABLE db.*, other_db.foo INTO 'somewhere' WITH encryption_passphrase='redacted', detached",
backupStmt: "BACKUP TABLE system.jobs, system.scheduled_jobs INTO 'nodelocal://0/backup' WITH encryption_passphrase='secret', detached",
shownStmt: "BACKUP TABLE system.jobs, system.scheduled_jobs INTO 'nodelocal://0/backup' WITH encryption_passphrase='redacted', detached",
period: 7 * 24 * time.Hour,
},
},
Expand All @@ -332,7 +332,8 @@ func TestSerializesScheduledBackupExecutionArgs(t *testing.T) {
name: "partitioned-backup",
user: enterpriseUser,
query: `
CREATE SCHEDULE FOR BACKUP DATABASE db1, db2 INTO ('somewhere', 'anywhere')
CREATE SCHEDULE FOR BACKUP DATABASE system
INTO ('nodelocal://0/backup?COCKROACH_LOCALITY=x%3Dy', 'nodelocal://0/backup2?COCKROACH_LOCALITY=default')
WITH revision_history
RECURRING '1 2 * * *'
FULL BACKUP ALWAYS
Expand All @@ -341,9 +342,11 @@ func TestSerializesScheduledBackupExecutionArgs(t *testing.T) {
queryArgs: []interface{}{th.env.Now().Add(time.Minute)},
expectedSchedules: []expectedSchedule{
{
nameRe: "BACKUP .+",
backupStmt: "BACKUP DATABASE db1, db2 INTO ('somewhere', 'anywhere') WITH revision_history, detached",
period: 24 * time.Hour,
nameRe: "BACKUP .+",
backupStmt: "BACKUP DATABASE system INTO " +
"('nodelocal://0/backup?COCKROACH_LOCALITY=x%3Dy', 'nodelocal://0/backup2?COCKROACH_LOCALITY=default') " +
"WITH revision_history, detached",
period: 24 * time.Hour,
},
},
},
Expand Down
36 changes: 23 additions & 13 deletions pkg/ccl/backupccl/schedule_exec.go
Original file line number Diff line number Diff line change
Expand Up @@ -86,18 +86,14 @@ func (e *scheduledBackupExecutor) executeBackup(
// Invoke backup plan hook.
hook, cleanup := cfg.PlanHookMaker("exec-backup", txn, sj.Owner())
defer cleanup()
planBackup, cols, _, _, err := backupPlanHook(ctx, backupStmt, hook.(sql.PlanHookState))

backupFn, err := planBackup(ctx, hook.(sql.PlanHookState), backupStmt)
if err != nil {
return errors.Wrapf(err, "backup eval: %q", tree.AsString(backupStmt))
}
if planBackup == nil {
return errors.Newf("backup eval: %q", tree.AsString(backupStmt))
}
if len(cols) != len(utilccl.DetachedJobExecutionResultHeader) {
return errors.Newf("unexpected result columns")
return err
}
return invokeBackup(ctx, backupFn)
}

func invokeBackup(ctx context.Context, backupFn sql.PlanHookRowFn) error {
resultCh := make(chan tree.Datums) // No need to close
g := ctxgroup.WithContext(ctx)

Expand All @@ -111,15 +107,29 @@ func (e *scheduledBackupExecutor) executeBackup(
})

g.GoCtx(func(ctx context.Context) error {
if err := planBackup(ctx, nil, resultCh); err != nil {
return errors.Wrapf(err, "backup planning error: %q", tree.AsString(backupStmt))
}
return nil
return backupFn(ctx, nil, resultCh)
})

return g.Wait()
}

func planBackup(
ctx context.Context, p sql.PlanHookState, backupStmt tree.Statement,
) (sql.PlanHookRowFn, error) {
fn, cols, _, _, err := backupPlanHook(ctx, backupStmt, p)

if err != nil {
return nil, errors.Wrapf(err, "backup eval: %q", tree.AsString(backupStmt))
}
if fn == nil {
return nil, errors.Newf("backup eval: %q", tree.AsString(backupStmt))
}
if len(cols) != len(utilccl.DetachedJobExecutionResultHeader) {
return nil, errors.Newf("unexpected result columns")
}
return fn, nil
}

// NotifyJobTermination implements jobs.ScheduledJobExecutor interface.
func (e *scheduledBackupExecutor) NotifyJobTermination(
ctx context.Context, jobID int64, jobStatus jobs.Status, schedule *jobs.ScheduledJob, _ *kv.Txn,
Expand Down
Loading

0 comments on commit 935ae2e

Please sign in to comment.