From 3bd26983091d0f5f062748cb8af6bb63eae27bbe Mon Sep 17 00:00:00 2001 From: Rafi Shamim Date: Thu, 20 Jul 2023 01:09:24 -0400 Subject: [PATCH 1/3] sql: use txn.NewBatch instead of &kv.Batch{} This will make these requests properly passes along the admission control headers. There are more usages in addition to these ones that are not trivial to address. Release note: None --- pkg/sql/catalog/descs/collection_test.go | 2 +- pkg/sql/crdb_internal.go | 9 ++++----- pkg/sql/create_sequence.go | 3 +-- pkg/sql/descriptor.go | 3 +-- pkg/sql/drop_database.go | 5 ++--- pkg/sql/gcjob/descriptor_utils.go | 2 +- pkg/sql/importer/import_job.go | 2 +- pkg/sql/repair.go | 3 +-- pkg/sql/sem/builtins/builtins.go | 11 +++++------ pkg/sql/temporary_schema.go | 3 +-- pkg/sql/truncate.go | 9 ++++----- 11 files changed, 22 insertions(+), 30 deletions(-) diff --git a/pkg/sql/catalog/descs/collection_test.go b/pkg/sql/catalog/descs/collection_test.go index ba2bc5f675ff..d3ca71e751c3 100644 --- a/pkg/sql/catalog/descs/collection_test.go +++ b/pkg/sql/catalog/descs/collection_test.go @@ -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) diff --git a/pkg/sql/crdb_internal.go b/pkg/sql/crdb_internal.go index 0d88accd2a55..5529d102afa2 100644 --- a/pkg/sql/crdb_internal.go +++ b/pkg/sql/crdb_internal.go @@ -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" @@ -6123,12 +6122,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 } @@ -7529,7 +7528,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, @@ -7546,7 +7545,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 } diff --git a/pkg/sql/create_sequence.go b/pkg/sql/create_sequence.go index 403beb033fb4..e07b31388774 100644 --- a/pkg/sql/create_sequence.go +++ b/pkg/sql/create_sequence.go @@ -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" @@ -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 { diff --git a/pkg/sql/descriptor.go b/pkg/sql/descriptor.go index c8074f1e4303..c48a84024073 100644 --- a/pkg/sql/descriptor.go +++ b/pkg/sql/descriptor.go @@ -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" @@ -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 diff --git a/pkg/sql/drop_database.go b/pkg/sql/drop_database.go index 7c5e69519aa8..ef25cffd057c 100644 --- a/pkg/sql/drop_database.go +++ b/pkg/sql/drop_database.go @@ -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" @@ -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 { @@ -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 { diff --git a/pkg/sql/gcjob/descriptor_utils.go b/pkg/sql/gcjob/descriptor_utils.go index 935830df88f7..fc4fc7f4416d 100644 --- a/pkg/sql/gcjob/descriptor_utils.go +++ b/pkg/sql/gcjob/descriptor_utils.go @@ -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. diff --git a/pkg/sql/importer/import_job.go b/pkg/sql/importer/import_job.go index d14d60c005f9..aeea28bd7099 100644 --- a/pkg/sql/importer/import_job.go +++ b/pkg/sql/importer/import_job.go @@ -686,7 +686,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 diff --git a/pkg/sql/repair.go b/pkg/sql/repair.go index 4c53a7244279..4b35cc778d81 100644 --- a/pkg/sql/repair.go +++ b/pkg/sql/repair.go @@ -20,7 +20,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/cloud" "github.com/cockroachdb/cockroach/pkg/config/zonepb" "github.com/cockroachdb/cockroach/pkg/keys" - "github.com/cockroachdb/cockroach/pkg/kv" "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/sql/catalog" @@ -755,7 +754,7 @@ func (p *planner) ForceDeleteTableData(ctx context.Context, descID int64) error requestHeader := kvpb.RequestHeader{ Key: tableSpan.Key, EndKey: tableSpan.EndKey, } - b := &kv.Batch{} + b := p.Txn().NewBatch() if storage.CanUseMVCCRangeTombstones(ctx, p.execCfg.Settings) { b.AddRawRequest(&kvpb.DeleteRangeRequest{ RequestHeader: requestHeader, diff --git a/pkg/sql/sem/builtins/builtins.go b/pkg/sql/sem/builtins/builtins.go index 676ea0206fcc..83978bdfd177 100644 --- a/pkg/sql/sem/builtins/builtins.go +++ b/pkg/sql/sem/builtins/builtins.go @@ -40,7 +40,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/config/zonepb" "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/kvpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvserverbase" "github.com/cockroachdb/cockroach/pkg/roachpb" @@ -5537,17 +5536,17 @@ SELECT Types: tree.ParamTypes{{Name: "key", Typ: types.Bytes}}, ReturnType: tree.FixedReturnType(types.Int), Fn: func(ctx context.Context, evalCtx *eval.Context, args tree.Datums) (tree.Datum, error) { + if evalCtx.Txn == nil { // can occur during backfills + return nil, pgerror.Newf(pgcode.FeatureNotSupported, + "cannot use crdb_internal.lease_holder in this context") + } key := []byte(tree.MustBeDBytes(args[0])) - b := &kv.Batch{} + b := evalCtx.Txn.NewBatch() b.AddRawRequest(&kvpb.LeaseInfoRequest{ RequestHeader: kvpb.RequestHeader{ Key: key, }, }) - if evalCtx.Txn == nil { // can occur during backfills - return nil, pgerror.Newf(pgcode.FeatureNotSupported, - "cannot use crdb_internal.lease_holder in this context") - } if err := evalCtx.Txn.Run(ctx, b); err != nil { return nil, pgerror.Wrap(err, pgcode.InvalidParameterValue, "error fetching leaseholder") } diff --git a/pkg/sql/temporary_schema.go b/pkg/sql/temporary_schema.go index d465dc8082de..f609453eec59 100644 --- a/pkg/sql/temporary_schema.go +++ b/pkg/sql/temporary_schema.go @@ -18,7 +18,6 @@ import ( "time" "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/serverpb" "github.com/cockroachdb/cockroach/pkg/server/telemetry" @@ -108,7 +107,7 @@ func (p *planner) getOrCreateTemporarySchema( if err != nil { return nil, err } - b := &kv.Batch{} + b := p.Txn().NewBatch() if err := p.Descriptors().InsertTempSchemaToBatch( ctx, p.ExtendedEvalContext().Tracing.KVTracingEnabled(), db, tempSchemaName, id, b, ); err != nil { diff --git a/pkg/sql/truncate.go b/pkg/sql/truncate.go index c7eedafeb661..08075349996d 100644 --- a/pkg/sql/truncate.go +++ b/pkg/sql/truncate.go @@ -15,7 +15,6 @@ import ( "math/rand" "github.com/cockroachdb/cockroach/pkg/jobs/jobspb" - "github.com/cockroachdb/cockroach/pkg/kv" "github.com/cockroachdb/cockroach/pkg/kv/kvpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvserverbase" "github.com/cockroachdb/cockroach/pkg/roachpb" @@ -417,7 +416,7 @@ func (p *planner) copySplitPointsToNewIndexes( // Re-split the new set of indexes along the same split points as the old // indexes. - var b kv.Batch + b := p.Txn().NewBatch() tablePrefix := execCfg.Codec.TablePrefix(uint32(tableID)) // Fetch all of the range descriptors for this index. @@ -512,12 +511,12 @@ func (p *planner) copySplitPointsToNewIndexes( }) } - if err = p.txn.DB().Run(ctx, &b); err != nil { + if err = p.txn.DB().Run(ctx, b); err != nil { return err } // Now scatter the ranges, after we've finished splitting them. - b = kv.Batch{} + b = p.Txn().NewBatch() b.AddRawRequest(&kvpb.AdminScatterRequest{ // Scatter all of the data between the start key of the first new index, and // the PrefixEnd of the last new index. @@ -528,7 +527,7 @@ func (p *planner) copySplitPointsToNewIndexes( RandomizeLeases: true, }) - return p.txn.DB().Run(ctx, &b) + return p.txn.DB().Run(ctx, b) } func (p *planner) reassignIndexComments( From b825a901c8986cad44d3ec5b51df253add7e1dde Mon Sep 17 00:00:00 2001 From: adityamaru Date: Sat, 15 Jul 2023 12:21:01 -0400 Subject: [PATCH 2/3] jobs: add table to display execution detail files 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 observability 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 follow-up 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. This change also renames the `Profiler` tab to `Advanced Debugging` as the users of this tab are going to be internal CRDB support and engineering for the time being. 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 --- pkg/jobs/execution_detail_utils.go | 2 +- pkg/ui/workspaces/cluster-ui/src/api/index.ts | 1 + .../cluster-ui/src/api/jobProfilerApi.ts | 29 ++++ .../src/jobs/jobDetailsPage/jobDetails.tsx | 43 +++--- .../jobDetailsPage/jobDetailsConnected.tsx | 19 ++- .../jobProfilerView.module.scss | 5 + .../jobs/jobDetailsPage/jobProfilerView.tsx | 136 ++++++++++++++++++ .../src/store/jobs/jobProfiler.reducer.ts | 64 +++++++++ .../src/store/jobs/jobProfiler.sagas.ts | 41 ++++++ .../cluster-ui/src/store/reducers.ts | 6 + .../db-console/src/redux/apiReducers.ts | 15 ++ pkg/ui/workspaces/db-console/src/util/api.ts | 17 +++ .../db-console/src/views/jobs/jobDetails.tsx | 12 ++ 13 files changed, 364 insertions(+), 26 deletions(-) create mode 100644 pkg/ui/workspaces/cluster-ui/src/api/jobProfilerApi.ts create mode 100644 pkg/ui/workspaces/cluster-ui/src/jobs/jobDetailsPage/jobProfilerView.module.scss create mode 100644 pkg/ui/workspaces/cluster-ui/src/jobs/jobDetailsPage/jobProfilerView.tsx create mode 100644 pkg/ui/workspaces/cluster-ui/src/store/jobs/jobProfiler.reducer.ts create mode 100644 pkg/ui/workspaces/cluster-ui/src/store/jobs/jobProfiler.sagas.ts diff --git a/pkg/jobs/execution_detail_utils.go b/pkg/jobs/execution_detail_utils.go index 7505b852f147..94aee1aed0ac 100644 --- a/pkg/jobs/execution_detail_utils.go +++ b/pkg/jobs/execution_detail_utils.go @@ -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 { diff --git a/pkg/ui/workspaces/cluster-ui/src/api/index.ts b/pkg/ui/workspaces/cluster-ui/src/api/index.ts index 636b33c9cc0b..2e518e43eb29 100644 --- a/pkg/ui/workspaces/cluster-ui/src/api/index.ts +++ b/pkg/ui/workspaces/cluster-ui/src/api/index.ts @@ -26,3 +26,4 @@ export * from "./eventsApi"; export * from "./databaseDetailsApi"; export * from "./tableDetailsApi"; export * from "./types"; +export * from "./jobProfilerApi"; diff --git a/pkg/ui/workspaces/cluster-ui/src/api/jobProfilerApi.ts b/pkg/ui/workspaces/cluster-ui/src/api/jobProfilerApi.ts new file mode 100644 index 000000000000..f13b858ceb7f --- /dev/null +++ b/pkg/ui/workspaces/cluster-ui/src/api/jobProfilerApi.ts @@ -0,0 +1,29 @@ +// Copyright 2023 The Cockroach Authors. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +import { cockroach } from "@cockroachlabs/crdb-protobuf-client"; +import { fetchData } from "./fetchData"; + +export type ListJobProfilerExecutionDetailsRequest = + cockroach.server.serverpb.ListJobProfilerExecutionDetailsRequest; +export type ListJobProfilerExecutionDetailsResponse = + cockroach.server.serverpb.ListJobProfilerExecutionDetailsResponse; + +export const getExecutionDetails = ( + req: ListJobProfilerExecutionDetailsRequest, +): Promise => { + return fetchData( + cockroach.server.serverpb.ListJobProfilerExecutionDetailsResponse, + `/_status/list_job_profiler_execution_details/${req.job_id}`, + null, + null, + "30M", + ); +}; diff --git a/pkg/ui/workspaces/cluster-ui/src/jobs/jobDetailsPage/jobDetails.tsx b/pkg/ui/workspaces/cluster-ui/src/jobs/jobDetailsPage/jobDetails.tsx index 868d19af2497..93135053a98e 100644 --- a/pkg/ui/workspaces/cluster-ui/src/jobs/jobDetailsPage/jobDetails.tsx +++ b/pkg/ui/workspaces/cluster-ui/src/jobs/jobDetailsPage/jobDetails.tsx @@ -39,10 +39,14 @@ import jobStyles from "src/jobs/jobs.module.scss"; import classNames from "classnames/bind"; import { Timestamp } from "../../timestamp"; -import { RequestState } from "../../api"; +import { + ListJobProfilerExecutionDetailsRequest, + ListJobProfilerExecutionDetailsResponse, + RequestState, +} from "../../api"; import moment from "moment-timezone"; import { CockroachCloudContext } from "src/contexts"; -import { InlineAlert } from "@cockroachlabs/ui-components"; +import { JobProfilerView } from "./jobProfilerView"; const { TabPane } = Tabs; @@ -51,15 +55,21 @@ const jobCx = classNames.bind(jobStyles); enum TabKeysEnum { OVERVIEW = "Overview", - PROFILER = "Profiler", + PROFILER = "Advanced Debugging", } export interface JobDetailsStateProps { jobRequest: RequestState; + jobProfilerResponse: RequestState; + jobProfilerLastUpdated: moment.Moment; + jobProfilerDataIsValid: boolean; } export interface JobDetailsDispatchProps { refreshJob: (req: JobRequest) => void; + refreshExecutionDetails: ( + req: ListJobProfilerExecutionDetailsRequest, + ) => void; } export interface JobDetailsState { @@ -117,25 +127,14 @@ export class JobDetails extends React.Component< job: cockroach.server.serverpb.JobResponse, ): React.ReactElement => { const id = job?.id; - // This URL results in a cluster-wide CPU profile to be collected for 5 - // seconds. We set `tagfocus` (tf) to only view the samples corresponding to - // this job's execution. - const url = `debug/pprof/ui/cpu?node=all&seconds=5&labels=true&tf=job.*${id}`; return ( - - - - Profile} - /> - - - - + ); }; @@ -300,7 +299,7 @@ export class JobDetails extends React.Component< {this.renderOverviewTabContent(hasNextRun, nextRun, job)} {!useContext(CockroachCloudContext) && ( - + {this.renderProfilerTabContent(job)} )} diff --git a/pkg/ui/workspaces/cluster-ui/src/jobs/jobDetailsPage/jobDetailsConnected.tsx b/pkg/ui/workspaces/cluster-ui/src/jobs/jobDetailsPage/jobDetailsConnected.tsx index b54302ab81e3..c6b10ba48f47 100644 --- a/pkg/ui/workspaces/cluster-ui/src/jobs/jobDetailsPage/jobDetailsConnected.tsx +++ b/pkg/ui/workspaces/cluster-ui/src/jobs/jobDetailsPage/jobDetailsConnected.tsx @@ -20,7 +20,15 @@ import { import { JobRequest, JobResponse } from "src/api/jobsApi"; import { actions as jobActions } from "src/store/jobDetails"; import { selectID } from "../../selectors"; -import { createInitialState } from "src/api"; +import { + ListJobProfilerExecutionDetailsRequest, + createInitialState, +} from "src/api"; +import { + initialState, + actions as jobProfilerActions, +} from "src/store/jobs/jobProfiler.reducer"; +import { Dispatch } from "redux"; const emptyState = createInitialState(); @@ -31,12 +39,17 @@ const mapStateToProps = ( const jobID = selectID(state, props); return { jobRequest: state.adminUI?.job?.cachedData[jobID] ?? emptyState, + jobProfilerResponse: state.adminUI?.executionDetails ?? initialState, + jobProfilerLastUpdated: state.adminUI?.executionDetails?.lastUpdated, + jobProfilerDataIsValid: state.adminUI?.executionDetails?.valid, }; }; -const mapDispatchToProps = { +const mapDispatchToProps = (dispatch: Dispatch): JobDetailsDispatchProps => ({ refreshJob: (req: JobRequest) => jobActions.refresh(req), -}; + refreshExecutionDetails: (req: ListJobProfilerExecutionDetailsRequest) => + dispatch(jobProfilerActions.refresh(req)), +}); export const JobDetailsPageConnected = withRouter( connect( diff --git a/pkg/ui/workspaces/cluster-ui/src/jobs/jobDetailsPage/jobProfilerView.module.scss b/pkg/ui/workspaces/cluster-ui/src/jobs/jobDetailsPage/jobProfilerView.module.scss new file mode 100644 index 000000000000..d3e955fa1e52 --- /dev/null +++ b/pkg/ui/workspaces/cluster-ui/src/jobs/jobDetailsPage/jobProfilerView.module.scss @@ -0,0 +1,5 @@ +@import "src/core/index.module"; + +.sorted-table { + width: 100%; +} \ No newline at end of file diff --git a/pkg/ui/workspaces/cluster-ui/src/jobs/jobDetailsPage/jobProfilerView.tsx b/pkg/ui/workspaces/cluster-ui/src/jobs/jobDetailsPage/jobProfilerView.tsx new file mode 100644 index 000000000000..e92918c15963 --- /dev/null +++ b/pkg/ui/workspaces/cluster-ui/src/jobs/jobDetailsPage/jobProfilerView.tsx @@ -0,0 +1,136 @@ +// Copyright 2023 The Cockroach Authors. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +import { cockroach } from "@cockroachlabs/crdb-protobuf-client"; +import moment from "moment-timezone"; +import React, { useCallback, useEffect, useState } from "react"; +import { + RequestState, + ListJobProfilerExecutionDetailsResponse, + ListJobProfilerExecutionDetailsRequest, +} from "src/api"; +import { InlineAlert } from "@cockroachlabs/ui-components"; +import { Row, Col } from "antd"; +import "antd/lib/col/style"; +import "antd/lib/row/style"; +import { SummaryCard, SummaryCardItem } from "src/summaryCard"; +import classNames from "classnames"; +import summaryCardStyles from "src/summaryCard/summaryCard.module.scss"; +import long from "long"; +import { ColumnDescriptor, SortSetting, SortedTable } from "src/sortedtable"; +import classnames from "classnames/bind"; +import styles from "./jobProfilerView.module.scss"; +import { EmptyTable } from "src/empty"; +import { useScheduleFunction } from "src/util/hooks"; + +const cardCx = classNames.bind(summaryCardStyles); +const cx = classnames.bind(styles); + +export type JobProfilerStateProps = { + jobID: long; + executionDetailsResponse: RequestState; + lastUpdated: moment.Moment; + isDataValid: boolean; +}; + +export type JobProfilerDispatchProps = { + refreshExecutionDetails: ( + req: ListJobProfilerExecutionDetailsRequest, + ) => void; +}; + +export type JobProfilerViewProps = JobProfilerStateProps & + JobProfilerDispatchProps; + +export function makeJobProfilerViewColumns(): ColumnDescriptor[] { + return [ + { + name: "executionDetailFiles", + title: "Execution Detail Files", + hideTitleUnderline: true, + cell: (executionDetails: string) => executionDetails, + }, + ]; +} + +export const JobProfilerView: React.FC = ({ + jobID, + executionDetailsResponse, + lastUpdated, + isDataValid, + refreshExecutionDetails, +}: JobProfilerViewProps) => { + const columns = makeJobProfilerViewColumns(); + const [sortSetting, setSortSetting] = useState({ + ascending: true, + columnTitle: "executionDetailFiles", + }); + const req = + new cockroach.server.serverpb.ListJobProfilerExecutionDetailsRequest({ + job_id: jobID, + }); + const refresh = useCallback(() => { + refreshExecutionDetails(req); + }, [refreshExecutionDetails, req]); + const [refetch] = useScheduleFunction( + refresh, + true, + 10 * 1000, // 10s polling interval + lastUpdated, + ); + useEffect(() => { + if (!isDataValid) refetch(); + }, [isDataValid, refetch]); + + const onChangeSortSetting = (ss: SortSetting): void => { + setSortSetting(ss); + }; + + // This URL results in a cluster-wide CPU profile to be collected for 5 + // seconds. We set `tagfocus` (tf) to only view the samples corresponding to + // this job's execution. + const url = `debug/pprof/ui/cpu?node=all&seconds=5&labels=true&tf=job.*${jobID}`; + const summaryCardStylesCx = classNames.bind(summaryCardStyles); + return ( +
+ + + + Profile} + /> + + + + + <> +

+ + + + } + /> + + + +

+ ); +}; diff --git a/pkg/ui/workspaces/cluster-ui/src/store/jobs/jobProfiler.reducer.ts b/pkg/ui/workspaces/cluster-ui/src/store/jobs/jobProfiler.reducer.ts new file mode 100644 index 000000000000..4065faa27ced --- /dev/null +++ b/pkg/ui/workspaces/cluster-ui/src/store/jobs/jobProfiler.reducer.ts @@ -0,0 +1,64 @@ +// Copyright 2023 The Cockroach Authors. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +import { createSlice, PayloadAction } from "@reduxjs/toolkit"; +import { DOMAIN_NAME } from "../utils"; +import moment from "moment-timezone"; +import { createInitialState, RequestState } from "src/api/types"; +import { + ListJobProfilerExecutionDetailsRequest, + ListJobProfilerExecutionDetailsResponse, +} from "src/api"; + +export type JobProfilerState = + RequestState; + +export const initialState = + createInitialState(); + +const JobProfilerExecutionDetailsSlice = createSlice({ + name: `${DOMAIN_NAME}/jobProfilerExecutionDetails`, + initialState, + reducers: { + received: ( + state, + action: PayloadAction, + ) => { + state.data = action.payload; + state.valid = true; + state.error = null; + state.inFlight = false; + state.lastUpdated = moment.utc(); + }, + failed: (state, action: PayloadAction) => { + state.inFlight = false; + state.valid = false; + state.error = action.payload; + state.lastUpdated = moment.utc(); + }, + invalidated: state => { + state.valid = false; + }, + refresh: ( + state, + _action: PayloadAction, + ) => { + state.inFlight = true; + }, + request: ( + state, + _action: PayloadAction, + ) => { + state.inFlight = true; + }, + }, +}); + +export const { reducer, actions } = JobProfilerExecutionDetailsSlice; diff --git a/pkg/ui/workspaces/cluster-ui/src/store/jobs/jobProfiler.sagas.ts b/pkg/ui/workspaces/cluster-ui/src/store/jobs/jobProfiler.sagas.ts new file mode 100644 index 000000000000..d0c19679deda --- /dev/null +++ b/pkg/ui/workspaces/cluster-ui/src/store/jobs/jobProfiler.sagas.ts @@ -0,0 +1,41 @@ +// Copyright 2023 The Cockroach Authors. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +import { PayloadAction } from "@reduxjs/toolkit"; +import { actions } from "./jobProfiler.reducer"; +import { call, put, all, takeEvery } from "redux-saga/effects"; +import { + ListJobProfilerExecutionDetailsRequest, + getExecutionDetails, +} from "src/api"; + +export function* refreshJobProfilerSaga( + action: PayloadAction, +) { + yield put(actions.request(action.payload)); +} + +export function* requestJobProfilerSaga( + action: PayloadAction, +): any { + try { + const result = yield call(getExecutionDetails, action.payload); + yield put(actions.received(result)); + } catch (e) { + yield put(actions.failed(e)); + } +} + +export function* jobProfilerSaga() { + yield all([ + takeEvery(actions.refresh, refreshJobProfilerSaga), + takeEvery(actions.request, requestJobProfilerSaga), + ]); +} diff --git a/pkg/ui/workspaces/cluster-ui/src/store/reducers.ts b/pkg/ui/workspaces/cluster-ui/src/store/reducers.ts index e0cf0e1a02f1..c8c2ff2ed0be 100644 --- a/pkg/ui/workspaces/cluster-ui/src/store/reducers.ts +++ b/pkg/ui/workspaces/cluster-ui/src/store/reducers.ts @@ -76,6 +76,10 @@ import { KeyedTableDetailsState, reducer as tableDetails, } from "./databaseTableDetails/tableDetails.reducer"; +import { + JobProfilerState, + reducer as executionDetails, +} from "./jobs/jobProfiler.reducer"; export type AdminUiState = { statementDiagnostics: StatementDiagnosticsState; @@ -91,6 +95,7 @@ export type AdminUiState = { indexStats: IndexStatsReducerState; jobs: JobsState; job: JobDetailsReducerState; + executionDetails: JobProfilerState; clusterLocks: ClusterLocksReqState; databasesList: DatabasesListState; databaseDetails: KeyedDatabaseDetailsState; @@ -124,6 +129,7 @@ export const reducers = combineReducers({ indexStats, jobs, job, + executionDetails, clusterLocks, databasesList, databaseDetails, diff --git a/pkg/ui/workspaces/db-console/src/redux/apiReducers.ts b/pkg/ui/workspaces/db-console/src/redux/apiReducers.ts index b7165c1dfd48..6ab82e29f210 100644 --- a/pkg/ui/workspaces/db-console/src/redux/apiReducers.ts +++ b/pkg/ui/workspaces/db-console/src/redux/apiReducers.ts @@ -212,6 +212,19 @@ const jobReducerObj = new KeyedCachedDataReducer( ); export const refreshJob = jobReducerObj.refresh; +export const jobProfilerRequestKey = ( + req: api.ListJobProfilerExecutionDetailsRequestMessage, +): string => `${req.job_id}`; + +const jobProfilerReducerObj = new KeyedCachedDataReducer( + api.getExecutionDetails, + "jobProfiler", + jobProfilerRequestKey, + null, + moment.duration(10, "m"), +); +export const refreshExecutionDetails = jobProfilerReducerObj.refresh; + export const queryToID = (req: api.QueryPlanRequestMessage): string => req.query; @@ -561,6 +574,7 @@ export interface APIReducersState { nonTableStats: CachedDataReducerState; logs: CachedDataReducerState; liveness: CachedDataReducerState; + jobProfiler: KeyedCachedDataReducerState; jobs: KeyedCachedDataReducerState; job: KeyedCachedDataReducerState; queryPlan: CachedDataReducerState; @@ -623,6 +637,7 @@ export const apiReducersReducer = combineReducers({ [nonTableStatsReducerObj.actionNamespace]: nonTableStatsReducerObj.reducer, [logsReducerObj.actionNamespace]: logsReducerObj.reducer, [livenessReducerObj.actionNamespace]: livenessReducerObj.reducer, + [jobProfilerReducerObj.actionNamespace]: jobProfilerReducerObj.reducer, [jobsReducerObj.actionNamespace]: jobsReducerObj.reducer, [jobReducerObj.actionNamespace]: jobReducerObj.reducer, [queryPlanReducerObj.actionNamespace]: queryPlanReducerObj.reducer, diff --git a/pkg/ui/workspaces/db-console/src/util/api.ts b/pkg/ui/workspaces/db-console/src/util/api.ts index 74e6ce68ea25..b1f3acfc90fe 100644 --- a/pkg/ui/workspaces/db-console/src/util/api.ts +++ b/pkg/ui/workspaces/db-console/src/util/api.ts @@ -90,6 +90,11 @@ export type JobsResponseMessage = protos.cockroach.server.serverpb.JobsResponse; export type JobRequestMessage = protos.cockroach.server.serverpb.JobRequest; export type JobResponseMessage = protos.cockroach.server.serverpb.JobResponse; +export type ListJobProfilerExecutionDetailsRequestMessage = + protos.cockroach.server.serverpb.ListJobProfilerExecutionDetailsRequest; +export type ListJobProfilerExecutionDetailsResponseMessage = + protos.cockroach.server.serverpb.ListJobProfilerExecutionDetailsResponse; + export type QueryPlanRequestMessage = protos.cockroach.server.serverpb.QueryPlanRequest; export type QueryPlanResponseMessage = @@ -471,6 +476,18 @@ export function getJob( ); } +export function getExecutionDetails( + req: ListJobProfilerExecutionDetailsRequestMessage, + timeout?: moment.Duration, +): Promise { + return timeoutFetch( + serverpb.ListJobProfilerExecutionDetailsResponse, + `${STATUS_PREFIX}/list_job_profiler_execution_details/${req.job_id}`, + null, + timeout, + ); +} + // getCluster gets info about the cluster export function getCluster( _req: ClusterRequestMessage, diff --git a/pkg/ui/workspaces/db-console/src/views/jobs/jobDetails.tsx b/pkg/ui/workspaces/db-console/src/views/jobs/jobDetails.tsx index e1f69b13acf9..2c7f2c03971c 100644 --- a/pkg/ui/workspaces/db-console/src/views/jobs/jobDetails.tsx +++ b/pkg/ui/workspaces/db-console/src/views/jobs/jobDetails.tsx @@ -16,23 +16,35 @@ import { connect } from "react-redux"; import { RouteComponentProps, withRouter } from "react-router-dom"; import { createSelectorForKeyedCachedDataField, + refreshExecutionDetails, refreshJob, } from "src/redux/apiReducers"; import { AdminUIState } from "src/redux/state"; +import { ListJobProfilerExecutionDetailsResponseMessage } from "src/util/api"; const selectJob = createSelectorForKeyedCachedDataField("job", selectID); +const selectExecutionDetails = + createSelectorForKeyedCachedDataField( + "jobProfiler", + selectID, + ); const mapStateToProps = ( state: AdminUIState, props: RouteComponentProps, ): JobDetailsStateProps => { + const jobID = selectID(state, props); return { jobRequest: selectJob(state, props), + jobProfilerResponse: selectExecutionDetails(state, props), + jobProfilerLastUpdated: state.cachedData.jobProfiler[jobID]?.setAt, + jobProfilerDataIsValid: state.cachedData.jobProfiler[jobID]?.valid, }; }; const mapDispatchToProps = { refreshJob, + refreshExecutionDetails, }; export default withRouter( From 0477e38d3d801d1803617dd7e9b20f64898012e8 Mon Sep 17 00:00:00 2001 From: Evan Wall Date: Mon, 24 Jul 2023 10:57:36 -0400 Subject: [PATCH 3/3] sql: fix CREATE MATERIALIZED VIEW AS schema change job description 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. --- pkg/sql/create_as_test.go | 16 +++++- pkg/sql/create_view.go | 57 ++++++++----------- pkg/sql/distsql_spec_exec_factory.go | 7 +-- .../logictest/testdata/logic_test/event_log | 2 +- .../testdata/logic_test/event_log_legacy | 2 +- pkg/sql/opt/exec/execbuilder/statement.go | 7 +-- pkg/sql/opt/exec/factory.opt | 7 +-- pkg/sql/opt/memo/expr_format.go | 2 +- pkg/sql/opt/ops/statement.opt | 8 +-- pkg/sql/opt/optbuilder/create_view.go | 32 +++++------ pkg/sql/opt/optgen/cmd/optgen/metadata.go | 1 + pkg/sql/opt_exec_factory.go | 24 +++----- 12 files changed, 71 insertions(+), 94 deletions(-) diff --git a/pkg/sql/create_as_test.go b/pkg/sql/create_as_test.go index b5ccba30c17d..19a556844fec 100644 --- a/pkg/sql/create_as_test.go +++ b/pkg/sql/create_as_test.go @@ -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() @@ -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}}) diff --git a/pkg/sql/create_view.go b/pkg/sql/create_view.go index a83edf95f869..ebddc530d523 100644 --- a/pkg/sql/create_view.go +++ b/pkg/sql/create_view.go @@ -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 @@ -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. @@ -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. @@ -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, @@ -136,16 +129,16 @@ 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( @@ -153,7 +146,7 @@ func (n *createViewNode) startExec(params runParams) error { params.p.txn, n.dbDesc.GetID(), schema.GetID(), - n.viewName.Table(), + createView.Name.Table(), ) if err != nil { return err @@ -174,7 +167,7 @@ func (n *createViewNode) startExec(params runParams) error { } } - if n.persistence.IsTemporary() { + if createView.Persistence.IsTemporary() { telemetry.Inc(sqltelemetry.CreateTempViewCounter) } @@ -235,14 +228,14 @@ func (n *createViewNode) startExec(params runParams) error { ¶ms.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 @@ -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 { @@ -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 } @@ -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 { @@ -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, }) }() @@ -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 { @@ -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 } diff --git a/pkg/sql/distsql_spec_exec_factory.go b/pkg/sql/distsql_spec_exec_factory.go index 4be97afb5e69..4d886ef4f034 100644 --- a/pkg/sql/distsql_spec_exec_factory.go +++ b/pkg/sql/distsql_spec_exec_factory.go @@ -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") } diff --git a/pkg/sql/logictest/testdata/logic_test/event_log b/pkg/sql/logictest/testdata/logic_test/event_log index 3d1dd22595bc..cf458d5293b2 100644 --- a/pkg/sql/logictest/testdata/logic_test/event_log +++ b/pkg/sql/logictest/testdata/logic_test/event_log @@ -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"} diff --git a/pkg/sql/logictest/testdata/logic_test/event_log_legacy b/pkg/sql/logictest/testdata/logic_test/event_log_legacy index 4b803bcf66bd..ac2b7f3e842e 100644 --- a/pkg/sql/logictest/testdata/logic_test/event_log_legacy +++ b/pkg/sql/logictest/testdata/logic_test/event_log_legacy @@ -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"} diff --git a/pkg/sql/opt/exec/execbuilder/statement.go b/pkg/sql/opt/exec/execbuilder/statement.go index eceb09420ce4..e01ee94e1623 100644 --- a/pkg/sql/opt/exec/execbuilder/statement.go +++ b/pkg/sql/opt/exec/execbuilder/statement.go @@ -60,17 +60,12 @@ func (b *Builder) buildCreateView(cv *memo.CreateViewExpr) (execPlan, error) { cols[i].Typ = md.ColumnMeta(cv.Columns[i].ID).Type } root, err := b.factory.ConstructCreateView( + cv.Syntax, schema, - cv.ViewName, - cv.IfNotExists, - cv.Replace, - cv.Persistence, - cv.Materialized, cv.ViewQuery, cols, cv.Deps, cv.TypeDeps, - cv.WithData, ) return execPlan{root: root}, err } diff --git a/pkg/sql/opt/exec/factory.opt b/pkg/sql/opt/exec/factory.opt index 9a509d0e87d7..22c2ef282b13 100644 --- a/pkg/sql/opt/exec/factory.opt +++ b/pkg/sql/opt/exec/factory.opt @@ -621,17 +621,12 @@ define CreateTableAs { # CreateView implements a CREATE VIEW statement. define CreateView { + 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 } # SequenceSelect implements a scan of a sequence as a data source. diff --git a/pkg/sql/opt/memo/expr_format.go b/pkg/sql/opt/memo/expr_format.go index 2f171c9f39a0..ca3042d799b5 100644 --- a/pkg/sql/opt/memo/expr_format.go +++ b/pkg/sql/opt/memo/expr_format.go @@ -1714,7 +1714,7 @@ func FormatPrivate(f *ExprFmtCtx, private interface{}, physProps *physical.Requi case *CreateViewPrivate: schema := f.Memo.Metadata().Schema(t.Schema) - fmt.Fprintf(f.Buffer, " %s.%s", schema.Name(), t.ViewName) + fmt.Fprintf(f.Buffer, " %s.%s", schema.Name(), t.Syntax.Name.Table()) case *JoinPrivate: // Nothing to show; flags are shown separately. diff --git a/pkg/sql/opt/ops/statement.opt b/pkg/sql/opt/ops/statement.opt index beb4463e1cd4..3b90d2568126 100644 --- a/pkg/sql/opt/ops/statement.opt +++ b/pkg/sql/opt/ops/statement.opt @@ -35,13 +35,11 @@ define CreateView { [Private] define CreateViewPrivate { + # Syntax is the CREATE VIEW AST node. + Syntax TreeCreateView + # Schema is the ID of the catalog schema into which the new table goes. Schema SchemaID - ViewName TableName - Persistence Persistence - IfNotExists bool - Replace bool - Materialized bool # ViewQuery contains the query for the view; data sources are always fully # qualified. diff --git a/pkg/sql/opt/optbuilder/create_view.go b/pkg/sql/opt/optbuilder/create_view.go index 7d217bc1cdae..5cb4a11f0fd9 100644 --- a/pkg/sql/opt/optbuilder/create_view.go +++ b/pkg/sql/opt/optbuilder/create_view.go @@ -21,10 +21,6 @@ import ( func (b *Builder) buildCreateView(cv *tree.CreateView, inScope *scope) (outScope *scope) { b.DisableMemoReuse = true - sch, resName := b.resolveSchemaForCreateTable(&cv.Name) - schID := b.factory.Metadata().AddSchema(sch) - viewName := tree.MakeTableNameFromPrefix(resName, tree.Name(cv.Name.Object())) - preFuncResolver := b.semaCtx.FunctionResolver b.semaCtx.FunctionResolver = nil @@ -39,14 +35,21 @@ func (b *Builder) buildCreateView(cv *tree.CreateView, inScope *scope) (outScope if b.sourceViews == nil { b.sourceViews = make(map[string]struct{}) } - b.sourceViews[viewName.FQString()] = struct{}{} + + viewName := &cv.Name + sch, resName := b.resolveSchemaForCreateTable(viewName) + viewName.ObjectNamePrefix = resName + schID := b.factory.Metadata().AddSchema(sch) + + viewFQString := viewName.FQString() + b.sourceViews[viewFQString] = struct{}{} defer func() { b.insideViewDef = false b.trackSchemaDeps = false b.schemaDeps = nil b.schemaTypeDeps = intsets.Fast{} b.qualifyDataSourceNamesInAST = false - delete(b.sourceViews, viewName.FQString()) + delete(b.sourceViews, viewFQString) b.semaCtx.FunctionResolver = preFuncResolver switch recErr := recover().(type) { @@ -106,17 +109,12 @@ func (b *Builder) buildCreateView(cv *tree.CreateView, inScope *scope) (outScope outScope = b.allocScope() outScope.expr = b.factory.ConstructCreateView( &memo.CreateViewPrivate{ - Schema: schID, - ViewName: &viewName, - IfNotExists: cv.IfNotExists, - Replace: cv.Replace, - Persistence: cv.Persistence, - Materialized: cv.Materialized, - ViewQuery: tree.AsStringWithFlags(cv.AsSource, tree.FmtParsable), - Columns: p, - Deps: b.schemaDeps, - TypeDeps: b.schemaTypeDeps, - WithData: cv.WithData, + Syntax: cv, + Schema: schID, + ViewQuery: tree.AsStringWithFlags(cv.AsSource, tree.FmtParsable), + Columns: p, + Deps: b.schemaDeps, + TypeDeps: b.schemaTypeDeps, }, ) return outScope diff --git a/pkg/sql/opt/optgen/cmd/optgen/metadata.go b/pkg/sql/opt/optgen/cmd/optgen/metadata.go index 78f8b7a3f4a3..e84bbf69820e 100644 --- a/pkg/sql/opt/optgen/cmd/optgen/metadata.go +++ b/pkg/sql/opt/optgen/cmd/optgen/metadata.go @@ -251,6 +251,7 @@ func newMetadata(compiled *lang.CompiledExpr, pkg string) *metadata { "Volatility": {fullName: "volatility.V", passByVal: true}, "LiteralRows": {fullName: "opt.LiteralRows", isExpr: true, isPointer: true}, "Distribution": {fullName: "physical.Distribution", passByVal: true}, + "TreeCreateView": {fullName: "tree.CreateView", isPointer: true, usePointerIntern: true}, } // Add types of generated op and private structs. diff --git a/pkg/sql/opt_exec_factory.go b/pkg/sql/opt_exec_factory.go index deb5a3ecff0d..c3bffdb206f6 100644 --- a/pkg/sql/opt_exec_factory.go +++ b/pkg/sql/opt_exec_factory.go @@ -1833,17 +1833,12 @@ func (ef *execFactory) ConstructCreateTableAs( // ConstructCreateView is part of the exec.Factory interface. func (ef *execFactory) 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) { if err := checkSchemaChangeEnabled( @@ -1860,17 +1855,12 @@ func (ef *execFactory) ConstructCreateView( } return &createViewNode{ - viewName: viewName, - ifNotExists: ifNotExists, - replace: replace, - materialized: materialized, - persistence: persistence, - viewQuery: viewQuery, - dbDesc: schema.(*optSchema).database, - columns: columns, - planDeps: planDeps, - typeDeps: typeDepSet, - withData: withData, + createView: createView, + viewQuery: viewQuery, + dbDesc: schema.(*optSchema).database, + columns: columns, + planDeps: planDeps, + typeDeps: typeDepSet, }, nil }