Skip to content

Commit

Permalink
Browse files Browse the repository at this point in the history
106879: jobs: add table to display execution details r=maryliag a=adityamaru

In #105384 and #106629 we added support to collect and list files that had been collected as part of
a job's execution details. These files are meant
to provide improved obersvability into the state
of a job.

This change is the first of a few that exposes these endpoints on the DBConsole job details page. This change only adds support for listing files that have been requested as part of a job's execution details.
A future change will add support to request these files, sort them and download them from the job details page.

This page is not available on the Cloud Console as it is meant for advanced debugging.

Informs: #105076

Release note (ui change): add table in the Profiler job details page that lists all the available files describing a job's execution details
<img width="1505" alt="Screenshot 2023-07-18 at 2 26 50 PM" src="https://github.com/cockroachdb/cockroach/assets/13837382/aebe18a6-9c25-4c9a-ad7c-a94e2e4c97ff">
<img width="1510" alt="Screenshot 2023-07-18 at 2 27 03 PM" src="https://github.com/cockroachdb/cockroach/assets/13837382/da9b3a21-8dc6-47ca-ac02-24d8bb7d09e7">



107236: sql: use txn.NewBatch instead of &kv.Batch{} r=fqazi a=rafiss

This will make these requests properly passes along the admission control headers.

informs #79212
Epic: None
Release note: None

107447: sql: fix CREATE MATERIALIZED VIEW AS schema change job description r=fqazi a=ecwall

Fixes #107445

This changes the CREATE MATERIALIZED VIEW AS schema change job description SQL syntax. For example
```
CREATE VIEW "v" AS "SELECT t.id FROM movr.public.t";
```
becomes
```
CREATE MATERIALIZED VIEW defaultdb.public.v AS SELECT t.id FROM defaultdb.public.t WITH DATA;
```

Release note (bug fix): Fix CREATE MATERIALIZED VIEW AS schema change job description SQL syntax.

Co-authored-by: adityamaru <adityamaru@gmail.com>
Co-authored-by: Rafi Shamim <rafi@cockroachlabs.com>
Co-authored-by: Evan Wall <wall@cockroachlabs.com>
  • Loading branch information
4 people committed Jul 24, 2023
4 parents c917a6b + b825a90 + 3bd2698 + 0477e38 commit 4618dbf
Show file tree
Hide file tree
Showing 36 changed files with 457 additions and 150 deletions.
2 changes: 1 addition & 1 deletion pkg/jobs/execution_detail_utils.go
Expand Up @@ -143,7 +143,7 @@ func ListExecutionDetailFiles(
func(infoKey string, value []byte) error {
// Look for the final chunk of each file to find the unique file name.
if strings.HasSuffix(infoKey, finalChunkSuffix) {
files = append(files, strings.TrimSuffix(infoKey, finalChunkSuffix))
files = append(files, strings.TrimPrefix(strings.TrimSuffix(infoKey, finalChunkSuffix), profilerconstants.ExecutionDetailsChunkKeyPrefix))
}
return nil
}); err != nil {
Expand Down
2 changes: 1 addition & 1 deletion pkg/sql/catalog/descs/collection_test.go
Expand Up @@ -236,7 +236,7 @@ func TestAddUncommittedDescriptorAndMutableResolution(t *testing.T) {
require.Same(t, immByName, immByID)

// Don't write the descriptor, just write the namespace entry.
b := &kv.Batch{}
b := txn.KV().NewBatch()
err = descriptors.InsertNamespaceEntryToBatch(ctx, false /* kvTrace */, mut, b)
require.NoError(t, err)
err = txn.KV().Run(ctx, b)
Expand Down
9 changes: 4 additions & 5 deletions pkg/sql/crdb_internal.go
Expand Up @@ -31,7 +31,6 @@ import (
"github.com/cockroachdb/cockroach/pkg/jobs/jobsauth"
"github.com/cockroachdb/cockroach/pkg/jobs/jobspb"
"github.com/cockroachdb/cockroach/pkg/keys"
"github.com/cockroachdb/cockroach/pkg/kv"
"github.com/cockroachdb/cockroach/pkg/kv/kvclient/kvcoord"
"github.com/cockroachdb/cockroach/pkg/kv/kvpb"
"github.com/cockroachdb/cockroach/pkg/kv/kvserver/concurrency/lock"
Expand Down Expand Up @@ -6139,12 +6138,12 @@ CREATE TABLE crdb_internal.lost_descriptors_with_data (
hasData := func(startID, endID descpb.ID) (found bool, _ error) {
startPrefix := p.extendedEvalCtx.Codec.TablePrefix(uint32(startID))
endPrefix := p.extendedEvalCtx.Codec.TablePrefix(uint32(endID - 1)).PrefixEnd()
var b kv.Batch
b := p.Txn().NewBatch()
b.Header.MaxSpanRequestKeys = 1
scanRequest := kvpb.NewScan(startPrefix, endPrefix, false).(*kvpb.ScanRequest)
scanRequest.ScanFormat = kvpb.BATCH_RESPONSE
b.AddRawRequest(scanRequest)
err = p.execCfg.DB.Run(ctx, &b)
err = p.execCfg.DB.Run(ctx, b)
if err != nil {
return false, err
}
Expand Down Expand Up @@ -7560,7 +7559,7 @@ func genClusterLocksGenerator(
var resumeSpan *roachpb.Span

fetchLocks := func(key, endKey roachpb.Key) error {
b := kv.Batch{}
b := p.Txn().NewBatch()
queryLocksRequest := &kvpb.QueryLocksRequest{
RequestHeader: kvpb.RequestHeader{
Key: key,
Expand All @@ -7577,7 +7576,7 @@ func genClusterLocksGenerator(
b.Header.MaxSpanRequestKeys = int64(rowinfra.ProductionKVBatchSize)
b.Header.TargetBytes = int64(rowinfra.GetDefaultBatchBytesLimit(p.extendedEvalCtx.TestingKnobs.ForceProductionValues))

err := p.txn.Run(ctx, &b)
err := p.txn.Run(ctx, b)
if err != nil {
return err
}
Expand Down
16 changes: 15 additions & 1 deletion pkg/sql/create_as_test.go
Expand Up @@ -351,6 +351,16 @@ func TestFormat(t *testing.T) {
setup: "CREATE TABLE ctas_explicit_columns_source_tbl (id int PRIMARY KEY)",
expectedFormat: "CREATE TABLE defaultdb.public.ctas_explicit_columns_tbl (id) AS SELECT * FROM defaultdb.public.ctas_explicit_columns_source_tbl",
},
{
sql: "CREATE MATERIALIZED VIEW cmvas_implicit_columns_tbl AS SELECT * FROM cmvas_implicit_columns_source_tbl",
setup: "CREATE TABLE cmvas_implicit_columns_source_tbl (id int PRIMARY KEY)",
expectedFormat: "CREATE MATERIALIZED VIEW defaultdb.public.cmvas_implicit_columns_tbl AS SELECT cmvas_implicit_columns_source_tbl.id FROM defaultdb.public.cmvas_implicit_columns_source_tbl WITH DATA",
},
{
sql: "CREATE MATERIALIZED VIEW cmvas_explicit_columns_tbl (id2) AS SELECT * FROM cmvas_explicit_columns_source_tbl",
setup: "CREATE TABLE cmvas_explicit_columns_source_tbl (id int PRIMARY KEY)",
expectedFormat: "CREATE MATERIALIZED VIEW defaultdb.public.cmvas_explicit_columns_tbl (id2) AS SELECT cmvas_explicit_columns_source_tbl.id FROM defaultdb.public.cmvas_explicit_columns_source_tbl WITH DATA",
},
}

ctx := context.Background()
Expand All @@ -371,14 +381,18 @@ func TestFormat(t *testing.T) {
switch stmt := statements[0].AST.(type) {
case *tree.CreateTable:
name = stmt.Table.Table()
case *tree.CreateView:
name = stmt.Name.Table()
default:
require.Failf(t, "missing case", "unexpected type %T", stmt)
}
// Filter description starting with CREATE to filter out CMVAS
// "updating view reference" job.
query := fmt.Sprintf(
`SELECT description
FROM [SHOW JOBS]
WHERE job_type IN ('SCHEMA CHANGE', 'NEW SCHEMA CHANGE')
AND description LIKE '%%%s%%'`,
AND description LIKE 'CREATE%%%s%%'`,
name,
)
sqlRunner.CheckQueryResults(t, query, [][]string{{tc.expectedFormat}})
Expand Down
3 changes: 1 addition & 2 deletions pkg/sql/create_sequence.go
Expand Up @@ -15,7 +15,6 @@ import (
"fmt"

"github.com/cockroachdb/cockroach/pkg/keys"
"github.com/cockroachdb/cockroach/pkg/kv"
"github.com/cockroachdb/cockroach/pkg/server/telemetry"
clustersettings "github.com/cockroachdb/cockroach/pkg/settings/cluster"
"github.com/cockroachdb/cockroach/pkg/sql/catalog"
Expand Down Expand Up @@ -155,7 +154,7 @@ func doCreateSequence(

// Initialize the sequence value.
seqValueKey := p.ExecCfg().Codec.SequenceKey(uint32(id))
b := &kv.Batch{}
b := p.Txn().NewBatch()

startVal := desc.SequenceOpts.Start
for _, option := range opts {
Expand Down
57 changes: 24 additions & 33 deletions pkg/sql/create_view.go
Expand Up @@ -44,17 +44,12 @@ import (

// createViewNode represents a CREATE VIEW statement.
type createViewNode struct {
// viewName is the fully qualified name of the new view.
viewName *tree.TableName
createView *tree.CreateView
// viewQuery contains the view definition, with all table names fully
// qualified.
viewQuery string
ifNotExists bool
replace bool
persistence tree.Persistence
materialized bool
dbDesc catalog.DatabaseDescriptor
columns colinfo.ResultColumns
viewQuery string
dbDesc catalog.DatabaseDescriptor
columns colinfo.ResultColumns

// planDeps tracks which tables and views the view being created
// depends on. This is collected during the construction of
Expand All @@ -65,9 +60,6 @@ type createViewNode struct {
// depends on. This is collected during the construction of
// the view query's logical plan.
typeDeps typeDependencies
// withData indicates if a materialized view should be populated
// with data by executing the underlying query.
withData bool
}

// ReadingOwnWrites implements the planNodeReadingOwnWrites interface.
Expand All @@ -76,16 +68,17 @@ type createViewNode struct {
func (n *createViewNode) ReadingOwnWrites() {}

func (n *createViewNode) startExec(params runParams) error {
createView := n.createView
tableType := tree.GetTableType(
false /* isSequence */, true /* isView */, n.materialized,
false /* isSequence */, true /* isView */, createView.Materialized,
)
if n.replace {
if createView.Replace {
telemetry.Inc(sqltelemetry.SchemaChangeCreateCounter(fmt.Sprintf("or_replace_%s", tableType)))
} else {
telemetry.Inc(sqltelemetry.SchemaChangeCreateCounter(tableType))
}

viewName := n.viewName.Object()
viewName := createView.Name.Object()
log.VEventf(params.ctx, 2, "dependencies for view %s:\n%s", viewName, n.planDeps.String())

// Check that the view does not contain references to other databases.
Expand Down Expand Up @@ -120,14 +113,14 @@ func (n *createViewNode) startExec(params runParams) error {
if err != nil {
return err
}
if !n.persistence.IsTemporary() && backRefMutable.Temporary {
if !createView.Persistence.IsTemporary() && backRefMutable.Temporary {
hasTempBackref = true
}
backRefMutables[id] = backRefMutable
}
}
if hasTempBackref {
n.persistence = tree.PersistenceTemporary
createView.Persistence = tree.PersistenceTemporary
// This notice is sent from pg, let's imitate.
params.p.BufferClientNotice(
params.ctx,
Expand All @@ -136,24 +129,24 @@ func (n *createViewNode) startExec(params runParams) error {
}

var replacingDesc *tabledesc.Mutable
schema, err := getSchemaForCreateTable(params, n.dbDesc, n.persistence, n.viewName,
tree.ResolveRequireViewDesc, n.ifNotExists)
schema, err := getSchemaForCreateTable(params, n.dbDesc, createView.Persistence, &createView.Name,
tree.ResolveRequireViewDesc, createView.IfNotExists)
if err != nil && !sqlerrors.IsRelationAlreadyExistsError(err) {
return err
}
if err != nil {
switch {
case n.ifNotExists:
case createView.IfNotExists:
return nil
case n.replace:
case createView.Replace:
// If we are replacing an existing view see if what we are
// replacing is actually a view.
id, err := params.p.Descriptors().LookupObjectID(
params.ctx,
params.p.txn,
n.dbDesc.GetID(),
schema.GetID(),
n.viewName.Table(),
createView.Name.Table(),
)
if err != nil {
return err
Expand All @@ -174,7 +167,7 @@ func (n *createViewNode) startExec(params runParams) error {
}
}

if n.persistence.IsTemporary() {
if createView.Persistence.IsTemporary() {
telemetry.Inc(sqltelemetry.CreateTempViewCounter)
}

Expand Down Expand Up @@ -235,14 +228,14 @@ func (n *createViewNode) startExec(params runParams) error {
&params.p.semaCtx,
params.p.EvalContext(),
params.p.EvalContext().Settings,
n.persistence,
createView.Persistence,
n.dbDesc.IsMultiRegion(),
params.p)
if err != nil {
return err
}

if n.materialized {
if createView.Materialized {
// If the view is materialized, set up some more state on the view descriptor.
// In particular,
// * mark the descriptor as a materialized view
Expand All @@ -254,7 +247,7 @@ func (n *createViewNode) startExec(params runParams) error {
// the table descriptor as requiring a REFRESH VIEW to indicate the view
// should only be accessed after a REFRESH VIEW operation has been called
// on it.
desc.RefreshViewRequired = !n.withData
desc.RefreshViewRequired = !createView.WithData
desc.State = descpb.DescriptorState_ADD
version := params.ExecCfg().Settings.Version.ActiveVersion(params.ctx)
if err := desc.AllocateIDs(params.ctx, version); err != nil {
Expand Down Expand Up @@ -282,12 +275,10 @@ func (n *createViewNode) startExec(params runParams) error {
desc.DependsOnTypes = append(desc.DependsOnTypes, orderedTypeDeps.Ordered()...)
newDesc = &desc

// TODO (lucy): I think this needs a NodeFormatter implementation. For now,
// do some basic string formatting (not accurate in the general case).
if err = params.p.createDescriptor(
params.ctx,
newDesc,
fmt.Sprintf("CREATE VIEW %q AS %q", n.viewName, n.viewQuery),
tree.AsStringWithFQNames(n.createView, params.Ann()),
); err != nil {
return err
}
Expand Down Expand Up @@ -317,7 +308,7 @@ func (n *createViewNode) startExec(params runParams) error {
params.ctx,
backRefMutable,
descpb.InvalidMutationID,
fmt.Sprintf("updating view reference %q in table %s(%d)", n.viewName,
fmt.Sprintf("updating view reference %q in table %s(%d)", &createView.Name,
updated.desc.GetName(), updated.desc.GetID(),
),
); err != nil {
Expand Down Expand Up @@ -362,7 +353,7 @@ func (n *createViewNode) startExec(params runParams) error {
return params.p.logEvent(params.ctx,
newDesc.ID,
&eventpb.CreateView{
ViewName: n.viewName.FQString(),
ViewName: createView.Name.FQString(),
ViewQuery: n.viewQuery,
})
}()
Expand Down Expand Up @@ -659,7 +650,7 @@ func (p *planner) replaceViewDesc(
ctx,
desc,
descpb.InvalidMutationID,
fmt.Sprintf("removing view reference for %q from %s(%d)", n.viewName,
fmt.Sprintf("removing view reference for %q from %s(%d)", &n.createView.Name,
desc.Name, desc.ID,
),
); err != nil {
Expand Down Expand Up @@ -695,7 +686,7 @@ func (p *planner) replaceViewDesc(
// Since we are replacing an existing view here, we need to write the new
// descriptor into place.
if err := p.writeSchemaChange(ctx, toReplace, descpb.InvalidMutationID,
fmt.Sprintf("CREATE OR REPLACE VIEW %q AS %q", n.viewName, n.viewQuery),
fmt.Sprintf("CREATE OR REPLACE VIEW %q AS %q", &n.createView.Name, n.viewQuery),
); err != nil {
return nil, err
}
Expand Down
3 changes: 1 addition & 2 deletions pkg/sql/descriptor.go
Expand Up @@ -16,7 +16,6 @@ import (
"strings"

"github.com/cockroachdb/cockroach/pkg/keys"
"github.com/cockroachdb/cockroach/pkg/kv"
"github.com/cockroachdb/cockroach/pkg/security/username"
"github.com/cockroachdb/cockroach/pkg/server/telemetry"
"github.com/cockroachdb/cockroach/pkg/settings"
Expand Down Expand Up @@ -227,7 +226,7 @@ func (p *planner) createDescriptor(
"expected new descriptor, not a modification of version %d",
descriptor.OriginalVersion())
}
b := &kv.Batch{}
b := p.Txn().NewBatch()
kvTrace := p.ExtendedEvalContext().Tracing.KVTracingEnabled()
if err := p.Descriptors().WriteDescToBatch(ctx, kvTrace, descriptor, b); err != nil {
return err
Expand Down
7 changes: 1 addition & 6 deletions pkg/sql/distsql_spec_exec_factory.go
Expand Up @@ -1042,17 +1042,12 @@ func (e *distSQLSpecExecFactory) ConstructCreateTableAs(
}

func (e *distSQLSpecExecFactory) ConstructCreateView(
createView *tree.CreateView,
schema cat.Schema,
viewName *cat.DataSourceName,
ifNotExists bool,
replace bool,
persistence tree.Persistence,
materialized bool,
viewQuery string,
columns colinfo.ResultColumns,
deps opt.SchemaDeps,
typeDeps opt.SchemaTypeDeps,
withData bool,
) (exec.Node, error) {
return nil, unimplemented.NewWithIssue(47473, "experimental opt-driven distsql planning: create view")
}
Expand Down
5 changes: 2 additions & 3 deletions pkg/sql/drop_database.go
Expand Up @@ -13,7 +13,6 @@ package sql
import (
"context"

"github.com/cockroachdb/cockroach/pkg/kv"
"github.com/cockroachdb/cockroach/pkg/server/telemetry"
"github.com/cockroachdb/cockroach/pkg/sql/catalog"
"github.com/cockroachdb/cockroach/pkg/sql/catalog/catalogkeys"
Expand Down Expand Up @@ -141,7 +140,7 @@ func (n *dropDatabaseNode) startExec(params runParams) error {
schemaToDelete := schemaWithDbDesc.schema
switch schemaToDelete.SchemaKind() {
case catalog.SchemaPublic:
b := &kv.Batch{}
b := p.Txn().NewBatch()
if err := p.Descriptors().DeleteDescriptorlessPublicSchemaToBatch(
ctx, p.ExtendedEvalContext().Tracing.KVTracingEnabled(), n.dbDesc, b,
); err != nil {
Expand All @@ -151,7 +150,7 @@ func (n *dropDatabaseNode) startExec(params runParams) error {
return err
}
case catalog.SchemaTemporary:
b := &kv.Batch{}
b := p.Txn().NewBatch()
if err := p.Descriptors().DeleteTempSchemaToBatch(
ctx, p.ExtendedEvalContext().Tracing.KVTracingEnabled(), n.dbDesc, schemaToDelete.GetName(), b,
); err != nil {
Expand Down
2 changes: 1 addition & 1 deletion pkg/sql/gcjob/descriptor_utils.go
Expand Up @@ -32,7 +32,7 @@ func deleteDatabaseZoneConfig(
return nil
}
return db.Txn(ctx, func(ctx context.Context, txn *kv.Txn) error {
b := &kv.Batch{}
b := txn.NewBatch()

// Delete the zone config entry for the dropped database associated with the
// job, if it exists.
Expand Down
2 changes: 1 addition & 1 deletion pkg/sql/importer/import_job.go
Expand Up @@ -691,7 +691,7 @@ func (r *importResumer) prepareSchemasForIngestion(

// Finally create the schemas on disk.
for i, mutDesc := range mutableSchemaDescs {
b := &kv.Batch{}
b := txn.KV().NewBatch()
kvTrace := p.ExtendedEvalContext().Tracing.KVTracingEnabled()
if err := descsCol.WriteDescToBatch(ctx, kvTrace, mutDesc, b); err != nil {
return nil, err
Expand Down
2 changes: 1 addition & 1 deletion pkg/sql/logictest/testdata/logic_test/event_log
Expand Up @@ -578,7 +578,7 @@ SELECT "eventType", "reportingID", info::JSONB - 'Timestamp' - 'DescriptorID'
WHERE "eventType" in ('create_view', 'drop_view')
ORDER BY "timestamp", info
----
create_view 1 {"EventType": "create_view", "Statement": "CREATE VIEW \"\".\"\".v AS SELECT 1", "Tag": "CREATE VIEW", "User": "root", "ViewName": "test.public.v", "ViewQuery": "SELECT 1"}
create_view 1 {"EventType": "create_view", "Statement": "CREATE VIEW test.public.v AS SELECT 1", "Tag": "CREATE VIEW", "User": "root", "ViewName": "test.public.v", "ViewQuery": "SELECT 1"}
drop_view 1 {"EventType": "drop_view", "Statement": "DROP VIEW test.public.v", "Tag": "DROP VIEW", "User": "root", "ViewName": "test.public.v"}


Expand Down
2 changes: 1 addition & 1 deletion pkg/sql/logictest/testdata/logic_test/event_log_legacy
Expand Up @@ -579,7 +579,7 @@ SELECT "eventType", "reportingID", info::JSONB - 'Timestamp' - 'DescriptorID'
WHERE "eventType" in ('create_view', 'drop_view')
ORDER BY "timestamp", info
----
create_view 1 {"EventType": "create_view", "Statement": "CREATE VIEW \"\".\"\".v AS SELECT 1", "Tag": "CREATE VIEW", "User": "root", "ViewName": "test.public.v", "ViewQuery": "SELECT 1"}
create_view 1 {"EventType": "create_view", "Statement": "CREATE VIEW test.public.v AS SELECT 1", "Tag": "CREATE VIEW", "User": "root", "ViewName": "test.public.v", "ViewQuery": "SELECT 1"}
drop_view 1 {"EventType": "drop_view", "Statement": "DROP VIEW test.public.v", "Tag": "DROP VIEW", "User": "root", "ViewName": "test.public.v"}


Expand Down

0 comments on commit 4618dbf

Please sign in to comment.