Skip to content

Commit

Permalink
Browse files Browse the repository at this point in the history
51865: sql: use the descs.Collection to access types during distributed flows r=rohany a=rohany

This commit enables distributed queries to access user defined type
metadata during flow setup via the lease manager, so that accesses to
this metadata is cached and doesn't have to go through k/v on every
access.

This is achieved by giving the `FlowContext` a `descs.Collection` is
used to access the descriptors through the lease manager.

Release note: None

51939: interval/generic: improve randomized testing, fix upper bound bug r=nvanbenschoten a=nvanbenschoten

In an effort to track down the bug that triggered #51913, this commit
ports the randomized interval btree benchmarks to also be unit tests.
This allows us to run invariant checks (see `btree.Verify`) on randomized
tree configurations.

Doing so revealed a violation of the `isUpperBoundCorrect` invariant.
This was determined to be a bug in `node.removeMax`. When removing an
item from a grandchild node, we were failing to adjust the upper bound
of the child node. It doesn't look like this could cause user-visible
effects because the upper bound of a subtree is only ever decreased on
removal, so at worst, this caused searches in the tree to do more work
than strictly necessary. Still, this is a good bug to fix and it's
encouraging that the new randomized testing using the existing invariant
validation caught it.

52090: sql: support ALTER TABLE SET SCHEMA command r=rohany a=RichardJCai

sql: support ALTER TABLE SET SCHEMA command

Release note (sql change): Added support for
ALTER TABLE/SEQUENCE/VIEW SET SCHEMA to set the schema of
the table to the target schema.

One must have DROP privilege on the table and CREATE privilege
on the schema to perform the operation.

52230: bulkio: Implement `SHOW SCHEDULES` r=miretskiy a=miretskiy

Informs #51850
Informs #51600

Implement `SHOW SCHEDULES` statemen which displays the information
on scheduled jobs.

Display schedule information, optionally filtered
by schedule state (paused or not) and optionally restricted
just to the backup schedules:

```
SHOW [RUNNING|PAUSED] SCHEDULES [FOR BACKUP]
```

In addition, it is possible to display information
for a specific schedule:

```
SHOW SCHEDULE 123
```

Release Notes (enterprise change): `SHOW SCHEDULES` displays
information about the scheduled jobs.

Co-authored-by: Rohan Yadav <[email protected]>
Co-authored-by: Nathan VanBenschoten <[email protected]>
Co-authored-by: richardjcai <[email protected]>
Co-authored-by: Yevgeniy Miretskiy <[email protected]>
  • Loading branch information
5 people committed Aug 3, 2020
5 parents e36a617 + bf9ffe9 + 33f86ff + 655c4cb + 1081966 commit 6ff0528
Show file tree
Hide file tree
Showing 72 changed files with 2,807 additions and 1,338 deletions.
1 change: 1 addition & 0 deletions docs/generated/sql/bnf/show_var.bnf
Original file line number Diff line number Diff line change
Expand Up @@ -9,6 +9,7 @@ show_stmt ::=
| show_indexes_stmt
| show_partitions_stmt
| show_jobs_stmt
| show_schedules_stmt
| show_queries_stmt
| show_ranges_stmt
| show_range_for_row_stmt
Expand Down
30 changes: 30 additions & 0 deletions docs/generated/sql/bnf/stmt_block.bnf
Original file line number Diff line number Diff line change
Expand Up @@ -188,6 +188,7 @@ show_stmt ::=
| show_indexes_stmt
| show_partitions_stmt
| show_jobs_stmt
| show_schedules_stmt
| show_queries_stmt
| show_ranges_stmt
| show_range_for_row_stmt
Expand Down Expand Up @@ -555,6 +556,11 @@ show_jobs_stmt ::=
| 'SHOW' 'JOB' a_expr
| 'SHOW' 'JOB' 'WHEN' 'COMPLETE' a_expr

show_schedules_stmt ::=
'SHOW' 'SCHEDULES' opt_schedule_executor_type
| 'SHOW' schedule_state 'SCHEDULES' opt_schedule_executor_type
| 'SHOW' 'SCHEDULE' a_expr

show_queries_stmt ::=
'SHOW' opt_cluster 'QUERIES'
| 'SHOW' 'ALL' opt_cluster 'QUERIES'
Expand Down Expand Up @@ -818,6 +824,7 @@ unreserved_keyword ::=
| 'PARTITIONS'
| 'PASSWORD'
| 'PAUSE'
| 'PAUSED'
| 'PHYSICAL'
| 'PLAN'
| 'PLANS'
Expand Down Expand Up @@ -853,6 +860,7 @@ unreserved_keyword ::=
| 'ROLLUP'
| 'ROWS'
| 'RULE'
| 'RUNNING'
| 'SCHEDULE'
| 'SCHEDULES'
| 'SETTING'
Expand Down Expand Up @@ -1027,6 +1035,7 @@ alter_table_stmt ::=
| alter_scatter_stmt
| alter_zone_table_stmt
| alter_rename_table_stmt
| alter_table_set_schema_stmt

alter_index_stmt ::=
alter_oneindex_stmt
Expand All @@ -1038,10 +1047,12 @@ alter_index_stmt ::=

alter_view_stmt ::=
alter_rename_view_stmt
| alter_view_set_schema_stmt

alter_sequence_stmt ::=
alter_rename_sequence_stmt
| alter_sequence_options_stmt
| alter_sequence_set_schema_stmt

alter_database_stmt ::=
alter_rename_database_stmt
Expand Down Expand Up @@ -1284,6 +1295,13 @@ for_grantee_clause ::=
'FOR' name_list
|

opt_schedule_executor_type ::=
'FOR' 'BACKUP'

schedule_state ::=
'RUNNING'
| 'PAUSED'

opt_cluster ::=
'CLUSTER'
| 'LOCAL'
Expand Down Expand Up @@ -1376,6 +1394,10 @@ alter_rename_table_stmt ::=
'ALTER' 'TABLE' relation_expr 'RENAME' 'TO' table_name
| 'ALTER' 'TABLE' 'IF' 'EXISTS' relation_expr 'RENAME' 'TO' table_name

alter_table_set_schema_stmt ::=
'ALTER' 'TABLE' relation_expr 'SET' 'SCHEMA' schema_name
| 'ALTER' 'TABLE' 'IF' 'EXISTS' relation_expr 'SET' 'SCHEMA' schema_name

alter_oneindex_stmt ::=
'ALTER' 'INDEX' table_index_name alter_index_cmds
| 'ALTER' 'INDEX' 'IF' 'EXISTS' table_index_name alter_index_cmds
Expand Down Expand Up @@ -1403,6 +1425,10 @@ alter_rename_view_stmt ::=
'ALTER' 'VIEW' relation_expr 'RENAME' 'TO' view_name
| 'ALTER' 'VIEW' 'IF' 'EXISTS' relation_expr 'RENAME' 'TO' view_name

alter_view_set_schema_stmt ::=
'ALTER' 'VIEW' relation_expr 'SET' 'SCHEMA' schema_name
| 'ALTER' 'VIEW' 'IF' 'EXISTS' relation_expr 'SET' 'SCHEMA' schema_name

alter_rename_sequence_stmt ::=
'ALTER' 'SEQUENCE' relation_expr 'RENAME' 'TO' sequence_name
| 'ALTER' 'SEQUENCE' 'IF' 'EXISTS' relation_expr 'RENAME' 'TO' sequence_name
Expand All @@ -1411,6 +1437,10 @@ alter_sequence_options_stmt ::=
'ALTER' 'SEQUENCE' sequence_name sequence_option_list
| 'ALTER' 'SEQUENCE' 'IF' 'EXISTS' sequence_name sequence_option_list

alter_sequence_set_schema_stmt ::=
'ALTER' 'SEQUENCE' relation_expr 'SET' 'SCHEMA' schema_name
| 'ALTER' 'SEQUENCE' 'IF' 'EXISTS' relation_expr 'SET' 'SCHEMA' schema_name

alter_rename_database_stmt ::=
'ALTER' 'DATABASE' database_name 'RENAME' 'TO' database_name

Expand Down
5 changes: 4 additions & 1 deletion pkg/ccl/backupccl/create_scheduled_backup.go
Original file line number Diff line number Diff line change
Expand Up @@ -224,7 +224,10 @@ func doCreateBackupSchedule(
if err != nil {
return err
}
sj.SetExecutionDetails(scheduledBackupExecutorName, jobspb.ExecutionArguments{Args: any})
sj.SetExecutionDetails(
tree.ScheduledBackupExecutor.InternalName(),
jobspb.ExecutionArguments{Args: any},
)

// Create the schedule.
if err := p.ExecCfg().DB.Txn(ctx, func(ctx context.Context, txn *kv.Txn) error {
Expand Down
2 changes: 1 addition & 1 deletion pkg/ccl/backupccl/create_scheduled_backup_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -226,7 +226,7 @@ WITH EXPERIMENTAL SCHEDULE OPTIONS first_run=$1
require.Equal(t, 1, len(schedules))
schedule := schedules[0]

require.Equal(t, scheduledBackupExecutorName, schedule.ExecutorType())
require.Equal(t, tree.ScheduledBackupExecutor.InternalName(), schedule.ExecutorType())
require.EqualValues(t, tc.nextRun, schedule.NextRun())
var arg ScheduledBackupExecutionArgs
require.NoError(t, types.UnmarshalAny(schedule.ExecutionArgs().Args, &arg))
Expand Down
4 changes: 1 addition & 3 deletions pkg/ccl/backupccl/schedule_exec.go
Original file line number Diff line number Diff line change
Expand Up @@ -26,8 +26,6 @@ import (
pbtypes "github.com/gogo/protobuf/types"
)

const scheduledBackupExecutorName = "scheduled-backup-executor"

type scheduledBackupExecutor struct{}

var _ jobs.ScheduledJobExecutor = &scheduledBackupExecutor{}
Expand Down Expand Up @@ -153,7 +151,7 @@ func extractBackupStatement(sj *jobs.ScheduledJob) (*annotatedBackupStatement, e

func init() {
jobs.RegisterScheduledJobExecutorFactory(
scheduledBackupExecutorName,
tree.ScheduledBackupExecutor.InternalName(),
func() (jobs.ScheduledJobExecutor, error) {
return &scheduledBackupExecutor{}, nil
})
Expand Down
3 changes: 2 additions & 1 deletion pkg/ccl/importccl/exportcsv.go
Original file line number Diff line number Diff line change
Expand Up @@ -138,7 +138,8 @@ func newCSVWriterProcessor(
input: input,
output: output,
}
if err := c.out.Init(&execinfrapb.PostProcessSpec{}, c.OutputTypes(), flowCtx.NewEvalCtx(), output); err != nil {
semaCtx := tree.MakeSemaContext()
if err := c.out.Init(&execinfrapb.PostProcessSpec{}, c.OutputTypes(), &semaCtx, flowCtx.NewEvalCtx(), output); err != nil {
return nil, err
}
return c, nil
Expand Down
37 changes: 0 additions & 37 deletions pkg/ccl/importccl/import_processor.go
Original file line number Diff line number Diff line change
Expand Up @@ -16,7 +16,6 @@ import (

"github.com/cockroachdb/cockroach/pkg/ccl/storageccl"
"github.com/cockroachdb/cockroach/pkg/keys"
"github.com/cockroachdb/cockroach/pkg/kv"
"github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvserverbase"
"github.com/cockroachdb/cockroach/pkg/roachpb"
"github.com/cockroachdb/cockroach/pkg/sql/execinfra"
Expand Down Expand Up @@ -120,43 +119,7 @@ func makeInputConverter(
evalCtx *tree.EvalContext,
kvCh chan row.KVBatch,
) (inputConverter, error) {

// installTypeMetadata is a closure that performs the work of installing
// type metadata in all of the tables being imported.
installTypeMetadata := func(evalCtx *tree.EvalContext) error {
for _, table := range spec.Tables {
var colTypes []*types.T
for _, col := range table.Desc.Columns {
colTypes = append(colTypes, col.Type)
}
if err := execinfrapb.HydrateTypeSlice(evalCtx, colTypes); err != nil {
return err
}
}
return nil
}

injectTimeIntoEvalCtx(evalCtx, spec.WalltimeNanos)

if evalCtx.Txn != nil {
// If we have a transaction, then use it.
if err := installTypeMetadata(evalCtx); err != nil {
return nil, err
}
} else if evalCtx.DB != nil {
// Otherwise, open up a new transaction to hydrate type metadata.
// We only perform this logic if evalCtx.DB != nil because there are
// some tests that pass an evalCtx with a nil DB to this function.
// TODO (rohany): Once we lease type descriptors, this should instead
// look into the leased set using the DistSQLTypeResolver.
if err := evalCtx.DB.Txn(ctx, func(ctx context.Context, txn *kv.Txn) error {
evalCtx.Txn = txn
return installTypeMetadata(evalCtx)
}); err != nil {
return nil, err
}
}

var singleTable *sqlbase.TableDescriptor
var singleTableTargetCols tree.NameList
if len(spec.Tables) == 1 {
Expand Down
24 changes: 21 additions & 3 deletions pkg/ccl/importccl/read_import_base.go
Original file line number Diff line number Diff line change
Expand Up @@ -23,6 +23,7 @@ import (
"sync/atomic"
"time"

"github.com/cockroachdb/cockroach/pkg/kv"
"github.com/cockroachdb/cockroach/pkg/roachpb"
"github.com/cockroachdb/cockroach/pkg/sql/execinfra"
"github.com/cockroachdb/cockroach/pkg/sql/execinfrapb"
Expand All @@ -47,9 +48,26 @@ func runImport(
) (*roachpb.BulkOpSummary, error) {
// Used to send ingested import rows to the KV layer.
kvCh := make(chan row.KVBatch, 10)
evalCtx := flowCtx.NewEvalCtx()
evalCtx.DB = flowCtx.Cfg.DB
conv, err := makeInputConverter(ctx, spec, evalCtx, kvCh)

// Install type metadata in all of the import tables. The DB is nil in some
// tests, so check first here.
if flowCtx.Cfg.DB != nil {
if err := flowCtx.Cfg.DB.Txn(ctx, func(ctx context.Context, txn *kv.Txn) error {
resolver := flowCtx.TypeResolverFactory.NewTypeResolver(txn)
for _, table := range spec.Tables {
if err := sqlbase.HydrateTypesInTableDescriptor(ctx, table.Desc, resolver); err != nil {
return err
}
}
return nil
}); err != nil {
return nil, err
}
// Release leases on any accessed types now that type metadata is installed.
flowCtx.TypeResolverFactory.Descriptors.ReleaseAll(ctx)
}

conv, err := makeInputConverter(ctx, spec, flowCtx.NewEvalCtx(), kvCh)
if err != nil {
return nil, err
}
Expand Down
51 changes: 28 additions & 23 deletions pkg/kv/kvserver/concurrency/lockstate_interval_btree.go

Some generated files are not rendered by default. Learn more about how customized files appear on GitHub.

Loading

0 comments on commit 6ff0528

Please sign in to comment.