From 36491c4704d671c21bfd7241fd8e04e4616d4abd Mon Sep 17 00:00:00 2001 From: Michael Butler Date: Fri, 6 Dec 2024 15:09:14 -0500 Subject: [PATCH 1/5] sql/externalcatalog: extract type descriptors Epic: none Release: none --- pkg/sql/catalog/externalcatalog/BUILD.bazel | 1 - .../externalcatalog/external_catalog.go | 46 ++++++++++++++++++- .../externalcatalog/external_catalog_test.go | 41 +++++++++++------ .../externalcatalog/externalpb/external.proto | 1 + 4 files changed, 71 insertions(+), 18 deletions(-) diff --git a/pkg/sql/catalog/externalcatalog/BUILD.bazel b/pkg/sql/catalog/externalcatalog/BUILD.bazel index fdb6e115a933..438e261a409b 100644 --- a/pkg/sql/catalog/externalcatalog/BUILD.bazel +++ b/pkg/sql/catalog/externalcatalog/BUILD.bazel @@ -31,7 +31,6 @@ go_test( embed = [":externalcatalog"], deps = [ "//pkg/base", - "//pkg/kv", "//pkg/security/securityassets", "//pkg/security/securitytest", "//pkg/security/username", diff --git a/pkg/sql/catalog/externalcatalog/external_catalog.go b/pkg/sql/catalog/externalcatalog/external_catalog.go index b8281a46c133..f8cb59e678b2 100644 --- a/pkg/sql/catalog/externalcatalog/external_catalog.go +++ b/pkg/sql/catalog/externalcatalog/external_catalog.go @@ -26,9 +26,14 @@ import ( // ExtractExternalCatalog extracts the table descriptors via the schema // resolver. func ExtractExternalCatalog( - ctx context.Context, schemaResolver resolver.SchemaResolver, tableNames ...string, + ctx context.Context, + schemaResolver resolver.SchemaResolver, + txn isql.Txn, + descCol *descs.Collection, + tableNames ...string, ) (externalpb.ExternalCatalog, error) { externalCatalog := externalpb.ExternalCatalog{} + foundTypeDescriptors := make(map[descpb.ID]struct{}) for _, name := range tableNames { uon, err := parser.ParseTableName(name) @@ -40,12 +45,49 @@ func ExtractExternalCatalog( if err != nil { return externalpb.ExternalCatalog{}, err } - + externalCatalog.Types, foundTypeDescriptors, err = getUDTsForTable(ctx, txn, descCol, externalCatalog.Types, foundTypeDescriptors, td) + if err != nil { + return externalpb.ExternalCatalog{}, err + } externalCatalog.Tables = append(externalCatalog.Tables, td.TableDescriptor) } return externalCatalog, nil } +func getUDTsForTable( + ctx context.Context, + txn isql.Txn, + descsCol *descs.Collection, + typeDescriptors []descpb.TypeDescriptor, + foundTypeDescriptors map[descpb.ID]struct{}, + td *tabledesc.Mutable, +) ([]descpb.TypeDescriptor, map[descpb.ID]struct{}, error) { + dbDesc, err := descsCol.ByIDWithLeased(txn.KV()).WithoutNonPublic().Get().Database(ctx, td.GetParentID()) + if err != nil { + return nil, nil, err + } + typeIDs, _, err := td.GetAllReferencedTypeIDs(dbDesc, + func(id descpb.ID) (catalog.TypeDescriptor, error) { + return descsCol.ByIDWithLeased(txn.KV()).WithoutNonPublic().Get().Type(ctx, id) + }) + if err != nil { + return nil, nil, errors.Wrap(err, "resolving type descriptors") + } + for _, typeID := range typeIDs { + if _, ok := foundTypeDescriptors[typeID]; ok { + continue + } + foundTypeDescriptors[typeID] = struct{}{} + + typeDesc, err := descsCol.ByIDWithLeased(txn.KV()).WithoutNonPublic().Get().Type(ctx, typeID) + if err != nil { + return nil, nil, err + } + typeDescriptors = append(typeDescriptors, *typeDesc.TypeDesc()) + } + return typeDescriptors, nil, nil +} + // IngestExternalCatalog ingests the tables in the external catalog into into // the database and schema. // diff --git a/pkg/sql/catalog/externalcatalog/external_catalog_test.go b/pkg/sql/catalog/externalcatalog/external_catalog_test.go index 628bd0e88e00..516d178161b4 100644 --- a/pkg/sql/catalog/externalcatalog/external_catalog_test.go +++ b/pkg/sql/catalog/externalcatalog/external_catalog_test.go @@ -10,7 +10,6 @@ import ( "testing" "github.com/cockroachdb/cockroach/pkg/base" - "github.com/cockroachdb/cockroach/pkg/kv" "github.com/cockroachdb/cockroach/pkg/security/username" "github.com/cockroachdb/cockroach/pkg/sql" "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" @@ -34,7 +33,7 @@ func TestExtractIngestExternalCatalog(t *testing.T) { defer log.Scope(t).Close(t) ctx := context.Background() - srv, conn, kvDB := serverutils.StartServer(t, base.TestServerArgs{}) + srv, conn, _ := serverutils.StartServer(t, base.TestServerArgs{}) defer srv.Stopper().Stop(ctx) s := srv.ApplicationLayer() @@ -51,19 +50,24 @@ func TestExtractIngestExternalCatalog(t *testing.T) { require.NoError(t, err) extractCatalog := func() externalpb.ExternalCatalog { - opName := redact.SafeString("extractCatalog") - planner, close := sql.NewInternalPlanner( - opName, - kv.NewTxn(ctx, kvDB, 0), - sqlUser, - &sql.MemoryMetrics{}, - &execCfg, - sql.NewInternalSessionData(ctx, execCfg.Settings, opName), - ) - defer close() - - catalog, err := ExtractExternalCatalog(ctx, planner.(resolver.SchemaResolver), "db1.sc1.tab1", "db1.sc1.tab2") - require.NoError(t, err) + var catalog externalpb.ExternalCatalog + require.NoError(t, sql.TestingDescsTxn(ctx, srv, func(ctx context.Context, txn isql.Txn, col *descs.Collection) error { + opName := redact.SafeString("extractCatalog") + planner, close := sql.NewInternalPlanner( + opName, + txn.KV(), + sqlUser, + &sql.MemoryMetrics{}, + &execCfg, + sql.NewInternalSessionData(ctx, execCfg.Settings, opName), + ) + defer close() + + catalog, err = ExtractExternalCatalog(ctx, planner.(resolver.SchemaResolver), txn, col, "db1.sc1.tab1", "db1.sc1.tab2") + require.NoError(t, err) + return nil + })) + return catalog } @@ -75,6 +79,13 @@ func TestExtractIngestExternalCatalog(t *testing.T) { sqlDB.Exec(t, "CREATE TABLE db1.sc1.tab3 (a INT PRIMARY KEY, b INT REFERENCES db1.sc1.tab2(a))") sadCatalog := extractCatalog() + // Modify table 1 to have a udt column + sqlDB.Exec(t, "CREATE TYPE db1.sc1.udt AS ENUM ('a', 'b', 'c')") + sqlDB.Exec(t, "ALTER TABLE db1.sc1.tab1 ADD COLUMN c db1.sc1.udt") + udtCatalog := extractCatalog() + require.Equal(t, "udt", udtCatalog.Types[0].Name) + require.Equal(t, "_udt", udtCatalog.Types[1].Name) + srv2, conn2, _ := serverutils.StartServer(t, base.TestServerArgs{}) execCfg2 := srv2.ExecutorConfig().(sql.ExecutorConfig) defer srv2.Stopper().Stop(ctx) diff --git a/pkg/sql/catalog/externalcatalog/externalpb/external.proto b/pkg/sql/catalog/externalcatalog/externalpb/external.proto index 37df1cf7ad24..b13cc402c1c0 100644 --- a/pkg/sql/catalog/externalcatalog/externalpb/external.proto +++ b/pkg/sql/catalog/externalcatalog/externalpb/external.proto @@ -13,5 +13,6 @@ import "sql/catalog/descpb/structured.proto"; message ExternalCatalog { repeated cockroach.sql.sqlbase.TableDescriptor tables = 1 [(gogoproto.nullable) = false]; + repeated cockroach.sql.sqlbase.TypeDescriptor types = 2 [(gogoproto.nullable) = false]; } \ No newline at end of file From 4b1193bc051169117d6bec011e366b31916f3417 Mon Sep 17 00:00:00 2001 From: Michael Butler Date: Fri, 6 Dec 2024 15:51:24 -0500 Subject: [PATCH 2/5] crosscluster/logical: use external catalog to plan LDR job This patch refactors LDR to use the new ExtractExternalCatalog api during job planning. A future pr will leverage this to write offline descriptors when the user requests LDR to create the destination table for them. In the future, we should consier using the ExtractExternalCatalog api during DistSQL planning. Epic: none Release note: none --- .../create_logical_replication_stmt.go | 7 ++-- pkg/crosscluster/producer/BUILD.bazel | 2 +- .../producer/replication_manager.go | 37 ++++++++----------- .../streamclient/randclient/BUILD.bazel | 1 + .../randclient/random_stream_client.go | 5 ++- pkg/repstream/streampb/BUILD.bazel | 2 + pkg/repstream/streampb/stream.proto | 14 +++---- 7 files changed, 30 insertions(+), 38 deletions(-) diff --git a/pkg/crosscluster/logical/create_logical_replication_stmt.go b/pkg/crosscluster/logical/create_logical_replication_stmt.go index 26f0c2e4af5a..36b239e63223 100644 --- a/pkg/crosscluster/logical/create_logical_replication_stmt.go +++ b/pkg/crosscluster/logical/create_logical_replication_stmt.go @@ -233,8 +233,8 @@ func createLogicalReplicationStreamPlanHook( } }() - sourceTypes := make([]*descpb.TypeDescriptor, len(spec.TypeDescriptors)) - for i, desc := range spec.TypeDescriptors { + sourceTypes := make([]*descpb.TypeDescriptor, len(spec.ExternalCatalog.Types)) + for i, desc := range spec.ExternalCatalog.Types { sourceTypes[i] = &desc } crossClusterResolver := crosscluster.MakeCrossClusterTypeResolver(sourceTypes) @@ -244,8 +244,7 @@ func createLogicalReplicationStreamPlanHook( // is used for filtering; if not, they probably forgot that step. throwNoTTLWithCDCIgnoreError := discard == jobspb.LogicalReplicationDetails_DiscardCDCIgnoredTTLDeletes - for i, name := range srcTableNames { - td := spec.TableDescriptors[name] + for i, td := range spec.ExternalCatalog.Tables { cpy := tabledesc.NewBuilder(&td).BuildCreatedMutableTable() if err := typedesc.HydrateTypesInDescriptor(ctx, cpy, crossClusterResolver); err != nil { return err diff --git a/pkg/crosscluster/producer/BUILD.bazel b/pkg/crosscluster/producer/BUILD.bazel index d6f04ab7b198..65dc488c70d7 100644 --- a/pkg/crosscluster/producer/BUILD.bazel +++ b/pkg/crosscluster/producer/BUILD.bazel @@ -41,12 +41,12 @@ go_library( "//pkg/sql/catalog", "//pkg/sql/catalog/descpb", "//pkg/sql/catalog/descs", + "//pkg/sql/catalog/externalcatalog", "//pkg/sql/catalog/resolver", "//pkg/sql/catalog/systemschema", "//pkg/sql/catalog/tabledesc", "//pkg/sql/clusterunique", "//pkg/sql/isql", - "//pkg/sql/parser", "//pkg/sql/pgwire/pgcode", "//pkg/sql/pgwire/pgerror", "//pkg/sql/privilege", diff --git a/pkg/crosscluster/producer/replication_manager.go b/pkg/crosscluster/producer/replication_manager.go index d8d47a08c316..09b444e021c1 100644 --- a/pkg/crosscluster/producer/replication_manager.go +++ b/pkg/crosscluster/producer/replication_manager.go @@ -24,15 +24,14 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/catalog" "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" "github.com/cockroachdb/cockroach/pkg/sql/catalog/descs" + "github.com/cockroachdb/cockroach/pkg/sql/catalog/externalcatalog" "github.com/cockroachdb/cockroach/pkg/sql/catalog/resolver" "github.com/cockroachdb/cockroach/pkg/sql/catalog/tabledesc" "github.com/cockroachdb/cockroach/pkg/sql/clusterunique" - "github.com/cockroachdb/cockroach/pkg/sql/parser" "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgcode" "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror" "github.com/cockroachdb/cockroach/pkg/sql/privilege" "github.com/cockroachdb/cockroach/pkg/sql/sem/eval" - "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/syntheticprivilege" "github.com/cockroachdb/cockroach/pkg/util/hlc" "github.com/cockroachdb/cockroach/pkg/util/uuid" @@ -81,29 +80,21 @@ func (r *replicationStreamManagerImpl) StartReplicationStreamForTables( // Resolve table names to tableIDs and spans. spans := make([]roachpb.Span, 0, len(req.TableNames)) - tableDescs := make(map[string]descpb.TableDescriptor, len(req.TableNames)) mutableTableDescs := make([]*tabledesc.Mutable, 0, len(req.TableNames)) tableIDs := make([]uint32, 0, len(req.TableNames)) - typeDescriptors := make([]descpb.TypeDescriptor, 0) - foundTypeDescriptors := make(map[descpb.ID]struct{}) - for _, name := range req.TableNames { - uon, err := parser.ParseTableName(name) - if err != nil { - return streampb.ReplicationProducerSpec{}, err - } - tn := uon.ToTableName() - _, td, err := resolver.ResolveMutableExistingTableObject(ctx, r.resolver, &tn, true, tree.ResolveRequireTableDesc) - if err != nil { - return streampb.ReplicationProducerSpec{}, err - } - spans = append(spans, td.PrimaryIndexSpan(r.evalCtx.Codec)) - tableIDs = append(tableIDs, uint32(td.GetID())) - mutableTableDescs = append(mutableTableDescs, td) - tableDescs[name] = td.TableDescriptor - typeDescriptors, foundTypeDescriptors, err = getUDTs(ctx, r.txn, typeDescriptors, foundTypeDescriptors, td) + + externalCatalog, err := externalcatalog.ExtractExternalCatalog(ctx, r.resolver, r.txn, r.txn.Descriptors(), req.TableNames...) + if err != nil { + return streampb.ReplicationProducerSpec{}, err + } + + for _, td := range externalCatalog.Tables { + mut, err := r.txn.Descriptors().MutableByID(r.txn.KV()).Table(ctx, td.ID) if err != nil { return streampb.ReplicationProducerSpec{}, err } + mutableTableDescs = append(mutableTableDescs, mut) + tableIDs = append(tableIDs, uint32(td.ID)) } registry := execConfig.JobRegistry @@ -142,8 +133,7 @@ func (r *replicationStreamManagerImpl) StartReplicationStreamForTables( StreamID: streampb.StreamID(jr.JobID), SourceClusterID: r.evalCtx.ClusterID, ReplicationStartTime: replicationStartTime, - TableDescriptors: tableDescs, - TypeDescriptors: typeDescriptors, + ExternalCatalog: externalCatalog, }, nil } @@ -203,6 +193,9 @@ func (r *replicationStreamManagerImpl) PlanLogicalReplication( typeDescriptors := make([]descpb.TypeDescriptor, 0) foundTypeDescriptors := make(map[descpb.ID]struct{}) descriptors := r.txn.Descriptors() + + // TODO(msbutler): use a variant of ExtractExternalCatalog that takes table + // IDs as input. for _, requestedTableID := range req.TableIDs { td, err := descriptors.MutableByID(r.txn.KV()).Table(ctx, descpb.ID(requestedTableID)) diff --git a/pkg/crosscluster/streamclient/randclient/BUILD.bazel b/pkg/crosscluster/streamclient/randclient/BUILD.bazel index 97681099943a..114e5c7890b7 100644 --- a/pkg/crosscluster/streamclient/randclient/BUILD.bazel +++ b/pkg/crosscluster/streamclient/randclient/BUILD.bazel @@ -20,6 +20,7 @@ go_library( "//pkg/sql/catalog/catpb", "//pkg/sql/catalog/descpb", "//pkg/sql/catalog/descs", + "//pkg/sql/catalog/externalcatalog/externalpb", "//pkg/sql/catalog/tabledesc", "//pkg/sql/randgen", "//pkg/sql/rowenc", diff --git a/pkg/crosscluster/streamclient/randclient/random_stream_client.go b/pkg/crosscluster/streamclient/randclient/random_stream_client.go index 66df17e278b3..b77b7dcab9ad 100644 --- a/pkg/crosscluster/streamclient/randclient/random_stream_client.go +++ b/pkg/crosscluster/streamclient/randclient/random_stream_client.go @@ -27,6 +27,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/catalog/catpb" "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" "github.com/cockroachdb/cockroach/pkg/sql/catalog/descs" + "github.com/cockroachdb/cockroach/pkg/sql/catalog/externalcatalog/externalpb" "github.com/cockroachdb/cockroach/pkg/sql/catalog/tabledesc" "github.com/cockroachdb/cockroach/pkg/sql/randgen" "github.com/cockroachdb/cockroach/pkg/sql/rowenc" @@ -829,8 +830,8 @@ func (m *RandomStreamClient) CreateForTables( return &streampb.ReplicationProducerSpec{ StreamID: streampb.StreamID(1), ReplicationStartTime: hlc.Timestamp{WallTime: timeutil.Now().UnixNano()}, - TableDescriptors: map[string]descpb.TableDescriptor{ - req.TableNames[0]: *m.tableDesc.TableDesc(), + ExternalCatalog: externalpb.ExternalCatalog{ + Tables: []descpb.TableDescriptor{*m.tableDesc.TableDesc()}, }, }, nil } diff --git a/pkg/repstream/streampb/BUILD.bazel b/pkg/repstream/streampb/BUILD.bazel index 31f07073e237..d678b129e0de 100644 --- a/pkg/repstream/streampb/BUILD.bazel +++ b/pkg/repstream/streampb/BUILD.bazel @@ -12,6 +12,7 @@ proto_library( "//pkg/kv/kvpb:kvpb_proto", "//pkg/roachpb:roachpb_proto", "//pkg/sql/catalog/descpb:descpb_proto", + "//pkg/sql/catalog/externalcatalog/externalpb:externalpb_proto", "//pkg/util:util_proto", "//pkg/util/hlc:hlc_proto", "@com_github_gogo_protobuf//gogoproto:gogo_proto", @@ -30,6 +31,7 @@ go_proto_library( "//pkg/kv/kvpb", "//pkg/roachpb", "//pkg/sql/catalog/descpb", + "//pkg/sql/catalog/externalcatalog/externalpb", "//pkg/util", "//pkg/util/hlc", "//pkg/util/uuid", # keep diff --git a/pkg/repstream/streampb/stream.proto b/pkg/repstream/streampb/stream.proto index d08eae1dab1d..321dc7d03872 100644 --- a/pkg/repstream/streampb/stream.proto +++ b/pkg/repstream/streampb/stream.proto @@ -19,6 +19,7 @@ import "gogoproto/gogo.proto"; import "google/protobuf/duration.proto"; import "roachpb/span_config.proto"; import "sql/catalog/descpb/structured.proto"; +import "sql/catalog/externalcatalog/externalpb/external.proto"; // ReplicationProducerSpec is the specification returned by the replication // producer job when it is created. @@ -44,15 +45,10 @@ message ReplicationProducerSpec { (gogoproto.nullable) = false, (gogoproto.customname) = "SourceTenantID" ]; - - // TableDescriptors are the descriptors for the tables in the - // ReplicationProducerRequest. - // - // TODO(ssd): We likely want to remove this one-time table - // descriptor fetch with a schemafeed that the consumer starts. - map table_descriptors = 7 [(gogoproto.nullable) = false]; - - repeated cockroach.sql.sqlbase.TypeDescriptor type_descriptors = 8 [(gogoproto.nullable) = false]; + + reserved 7; + reserved 8; + sql.catalog.externalcatalog.externalpb.ExternalCatalog external_catalog = 9 [(gogoproto.nullable) = false]; } // ReplicationProducerRequest is sent by the consuming cluster when From 96d0840a64f3b84cdf1d1415605ce4bb9731ed06 Mon Sep 17 00:00:00 2001 From: Michael Butler Date: Mon, 9 Dec 2024 09:34:57 -0500 Subject: [PATCH 3/5] sql/externalcatalog: rewrite table descriptor ids in IngestExternalCatalog Epic: none Release note: none --- .../externalcatalog/external_catalog.go | 5 +++++ .../externalcatalog/external_catalog_test.go | 22 +++++++------------ 2 files changed, 13 insertions(+), 14 deletions(-) diff --git a/pkg/sql/catalog/externalcatalog/external_catalog.go b/pkg/sql/catalog/externalcatalog/external_catalog.go index f8cb59e678b2..b4b4f7823d82 100644 --- a/pkg/sql/catalog/externalcatalog/external_catalog.go +++ b/pkg/sql/catalog/externalcatalog/external_catalog.go @@ -115,6 +115,10 @@ func IngestExternalCatalog( } else if originalParentID != table.ParentID { return errors.New("all tables must belong to the same parent") } + newID, err := execCfg.DescIDGenerator.GenerateUniqueDescID(ctx) + if err != nil { + return err + } // TODO: rewrite the tables to fresh ids. mutTable := tabledesc.NewBuilder(&table).BuildCreatedMutableTable() if setOffline { @@ -125,6 +129,7 @@ func IngestExternalCatalog( mutTable.UnexposedParentSchemaID = schemaID mutTable.ParentID = dbDesc.GetID() mutTable.Version = 1 + mutTable.ID = newID tablesToWrite = append(tablesToWrite, mutTable) } return ingesting.WriteDescriptors( diff --git a/pkg/sql/catalog/externalcatalog/external_catalog_test.go b/pkg/sql/catalog/externalcatalog/external_catalog_test.go index 516d178161b4..196c5ef49f19 100644 --- a/pkg/sql/catalog/externalcatalog/external_catalog_test.go +++ b/pkg/sql/catalog/externalcatalog/external_catalog_test.go @@ -86,14 +86,10 @@ func TestExtractIngestExternalCatalog(t *testing.T) { require.Equal(t, "udt", udtCatalog.Types[0].Name) require.Equal(t, "_udt", udtCatalog.Types[1].Name) - srv2, conn2, _ := serverutils.StartServer(t, base.TestServerArgs{}) - execCfg2 := srv2.ExecutorConfig().(sql.ExecutorConfig) - defer srv2.Stopper().Stop(ctx) - + // Ingest the catalog into defaultdb. var parentID descpb.ID var schemaID descpb.ID - - require.NoError(t, sql.TestingDescsTxn(ctx, srv2, func(ctx context.Context, txn isql.Txn, col *descs.Collection) error { + require.NoError(t, sql.TestingDescsTxn(ctx, srv, func(ctx context.Context, txn isql.Txn, col *descs.Collection) error { dbDesc, err := col.ByNameWithLeased(txn.KV()).Get().Database(ctx, "defaultdb") require.NoError(t, err) parentID = dbDesc.GetID() @@ -101,14 +97,12 @@ func TestExtractIngestExternalCatalog(t *testing.T) { return err })) - require.ErrorContains(t, sql.TestingDescsTxn(ctx, srv2, func(ctx context.Context, txn isql.Txn, col *descs.Collection) error { - return IngestExternalCatalog(ctx, &execCfg2, sqlUser, sadCatalog, txn, col, parentID, schemaID, false) - }), "invalid foreign key backreference") + require.ErrorContains(t, sql.TestingDescsTxn(ctx, srv, func(ctx context.Context, txn isql.Txn, col *descs.Collection) error { + return IngestExternalCatalog(ctx, &execCfg, sqlUser, sadCatalog, txn, col, parentID, schemaID, false) + }), "invalid inbound foreign key") - require.NoError(t, sql.TestingDescsTxn(ctx, srv2, func(ctx context.Context, txn isql.Txn, col *descs.Collection) error { - return IngestExternalCatalog(ctx, &execCfg2, sqlUser, ingestableCatalog, txn, col, parentID, schemaID, false) + require.NoError(t, sql.TestingDescsTxn(ctx, srv, func(ctx context.Context, txn isql.Txn, col *descs.Collection) error { + return IngestExternalCatalog(ctx, &execCfg, sqlUser, ingestableCatalog, txn, col, parentID, schemaID, false) })) - - sqlDB2 := sqlutils.MakeSQLRunner(conn2) - sqlDB2.CheckQueryResults(t, "SELECT schema_name,table_name FROM [SHOW TABLES]", [][]string{{"public", "tab1"}, {"public", "tab2"}}) + sqlDB.CheckQueryResults(t, "SELECT schema_name,table_name FROM [SHOW TABLES]", [][]string{{"public", "tab1"}, {"public", "tab2"}}) } From 46a3e885aa890344d8605e4afe352b27bca550f9 Mon Sep 17 00:00:00 2001 From: Radu Berinde Date: Mon, 9 Dec 2024 14:41:56 -0800 Subject: [PATCH 4/5] storage: fix incorrect mount association We use `filepath.Rel` to associate a data directory with a mountpoint, but the code incorrectly tolerates a result which starts with `../`. This commit adds a check for this case. Fixes: #137021 Release note (bug fix): Fixed bug that causes an incorrect filesystem to be logged as part of the store information. --- pkg/storage/BUILD.bazel | 1 + pkg/storage/store_properties.go | 23 ++++++++-- pkg/storage/store_properties_test.go | 67 ++++++++++++++++++++++++++++ 3 files changed, 87 insertions(+), 4 deletions(-) create mode 100644 pkg/storage/store_properties_test.go diff --git a/pkg/storage/BUILD.bazel b/pkg/storage/BUILD.bazel index 853c19353bd0..0efbe5e176e7 100644 --- a/pkg/storage/BUILD.bazel +++ b/pkg/storage/BUILD.bazel @@ -148,6 +148,7 @@ go_test( "read_as_of_iterator_test.go", "sst_test.go", "sst_writer_test.go", + "store_properties_test.go", "temp_engine_test.go", ], data = glob(["testdata/**"]), diff --git a/pkg/storage/store_properties.go b/pkg/storage/store_properties.go index 2b7779bd507c..a0d26486f36e 100644 --- a/pkg/storage/store_properties.go +++ b/pkg/storage/store_properties.go @@ -8,6 +8,7 @@ package storage import ( "context" "path/filepath" + "strings" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/util/log" @@ -75,10 +76,7 @@ func getFileSystemProperties(ctx context.Context, dir string) roachpb.FileStoreP // is typically being deployed are well-behaved in that regard: // Kubernetes mirrors /proc/mount in /etc/mtab. for i := len(fslist.List) - 1; i >= 0; i-- { - // filepath.Rel can reliably tell us if a path is relative to - // another: if it is not, an error is returned. - _, err := filepath.Rel(fslist.List[i].DirName, absPath) - if err == nil { + if pathIsInside(fslist.List[i].DirName, absPath) { fsInfo = &fslist.List[i] break } @@ -95,3 +93,20 @@ func getFileSystemProperties(ctx context.Context, dir string) roachpb.FileStoreP fsprops.MountOptions = fsInfo.Options return fsprops } + +// pathIsInside returns true if the absolute target path is inside a base path. +func pathIsInside(basePath string, absTargetPath string) bool { + // filepath.Rel can reliably tell us if a path is relative to + // another: if it is not, an error is returned. + relPath, err := filepath.Rel(basePath, absTargetPath) + if err != nil { + return false + } + if strings.HasPrefix(relPath, "..") { + // This check is consistent with internal filepath code (like isLocal). + if len(relPath) == 2 || relPath[2] == filepath.Separator { + return false + } + } + return true +} diff --git a/pkg/storage/store_properties_test.go b/pkg/storage/store_properties_test.go new file mode 100644 index 000000000000..2e0ff40ac54b --- /dev/null +++ b/pkg/storage/store_properties_test.go @@ -0,0 +1,67 @@ +// Copyright 2024 The Cockroach Authors. +// +// Use of this software is governed by the CockroachDB Software License +// included in the /LICENSE file. + +package storage + +import ( + "path/filepath" + "testing" + + "github.com/cockroachdb/cockroach/pkg/util/leaktest" +) + +func TestPathIsInside(t *testing.T) { + defer leaktest.AfterTest(t)() + + testCases := []struct { + base, target string + expected bool + }{ + { + base: "/", + target: "/cockroach/cockroach-data", + expected: true, + }, + { + base: "/cockroach", + target: "/cockroach/cockroach-data", + expected: true, + }, + { + base: "/cockroach/cockroach-data", + target: "/cockroach/cockroach-data", + expected: true, + }, + { + base: "/cockroach/cockroach-data/foo", + target: "/cockroach/cockroach-data", + expected: false, + }, + { + base: "/cockroach/cockroach-data1", + target: "/cockroach/cockroach-data", + expected: false, + }, + { + base: "/run/user/1001", + target: "/cockroach/cockroach-data", + expected: false, + }, + { + base: "/..foo", + target: "/..foo/data", + expected: true, + }, + } + + for _, tc := range testCases { + t.Run("", func(t *testing.T) { + result := pathIsInside(filepath.FromSlash(tc.base), filepath.FromSlash(tc.target)) + if result != tc.expected { + t.Fatalf("%q, %q: expected %t, got %t", tc.base, tc.target, tc.expected, result) + } + }) + } +} From 660522e7c9eaff11c7fbfdd9f84f2246556caa8f Mon Sep 17 00:00:00 2001 From: Ryan Kuo Date: Mon, 9 Dec 2024 17:36:28 -0500 Subject: [PATCH 5/5] docgen: update BEGIN and SET TRANSACTION diagrams --- docs/generated/sql/bnf/BUILD.bazel | 3 +- docs/generated/sql/bnf/begin_stmt.bnf | 1 + docs/generated/sql/bnf/legacy_begin_stmt.bnf | 29 ++++++++++++++++++- .../sql/bnf/nonpreparable_set_stmt.bnf | 2 ++ docs/generated/sql/bnf/set_transaction.bnf | 23 +++++++++------ docs/generated/sql/bnf/stmt_block.bnf | 13 ++++++++- pkg/cmd/docgen/diagrams.go | 18 ++++++------ pkg/gen/bnf.bzl | 3 +- pkg/gen/diagrams.bzl | 2 +- pkg/gen/docs.bzl | 3 +- pkg/sql/parser/sql.y | 1 - 11 files changed, 70 insertions(+), 28 deletions(-) create mode 100644 docs/generated/sql/bnf/nonpreparable_set_stmt.bnf diff --git a/docs/generated/sql/bnf/BUILD.bazel b/docs/generated/sql/bnf/BUILD.bazel index f00671164367..7adae8b804cd 100644 --- a/docs/generated/sql/bnf/BUILD.bazel +++ b/docs/generated/sql/bnf/BUILD.bazel @@ -72,7 +72,6 @@ FILES = [ "analyze_stmt", "backup", "backup_options", - "begin_stmt", "begin_transaction", "call", "cancel_all_jobs", @@ -168,6 +167,7 @@ FILES = [ "like_table_option_list", "limit_clause", "move_cursor_stmt", + "nonpreparable_set_stmt", "not_null_column_level", "offset_clause", "on_conflict", @@ -223,7 +223,6 @@ FILES = [ "set_rest_more", "set_session_stmt", "set_transaction", - "set_transaction_stmt", "show_backup", "show_cluster_setting", "show_columns_stmt", diff --git a/docs/generated/sql/bnf/begin_stmt.bnf b/docs/generated/sql/bnf/begin_stmt.bnf index 1059df25a0ad..b76ee9186f8f 100644 --- a/docs/generated/sql/bnf/begin_stmt.bnf +++ b/docs/generated/sql/bnf/begin_stmt.bnf @@ -1 +1,2 @@ begin_stmt ::= + 'START' 'TRANSACTION' begin_transaction diff --git a/docs/generated/sql/bnf/legacy_begin_stmt.bnf b/docs/generated/sql/bnf/legacy_begin_stmt.bnf index 3974be960389..f0c5585ba047 100644 --- a/docs/generated/sql/bnf/legacy_begin_stmt.bnf +++ b/docs/generated/sql/bnf/legacy_begin_stmt.bnf @@ -1,2 +1,29 @@ legacy_begin_stmt ::= - 'BEGIN' opt_transaction begin_transaction + 'BEGIN' 'TRANSACTION' 'ISOLATION' 'LEVEL' 'READ' 'UNCOMMITTED' ( ( ( ',' | ) ( ( 'ISOLATION' 'LEVEL' ( 'READ' 'UNCOMMITTED' | 'READ' 'COMMITTED' | 'SNAPSHOT' | 'REPEATABLE' 'READ' | 'SERIALIZABLE' ) ) | ( 'PRIORITY' ( 'LOW' | 'NORMAL' | 'HIGH' ) ) | ( 'READ' 'ONLY' | 'READ' 'WRITE' ) | ( 'AS' 'OF' 'SYSTEM' 'TIME' a_expr ) | ( 'DEFERRABLE' | 'NOT' 'DEFERRABLE' ) ) ) )* + | 'BEGIN' 'TRANSACTION' 'ISOLATION' 'LEVEL' 'READ' 'COMMITTED' ( ( ( ',' | ) ( ( 'ISOLATION' 'LEVEL' ( 'READ' 'UNCOMMITTED' | 'READ' 'COMMITTED' | 'SNAPSHOT' | 'REPEATABLE' 'READ' | 'SERIALIZABLE' ) ) | ( 'PRIORITY' ( 'LOW' | 'NORMAL' | 'HIGH' ) ) | ( 'READ' 'ONLY' | 'READ' 'WRITE' ) | ( 'AS' 'OF' 'SYSTEM' 'TIME' a_expr ) | ( 'DEFERRABLE' | 'NOT' 'DEFERRABLE' ) ) ) )* + | 'BEGIN' 'TRANSACTION' 'ISOLATION' 'LEVEL' 'SNAPSHOT' ( ( ( ',' | ) ( ( 'ISOLATION' 'LEVEL' ( 'READ' 'UNCOMMITTED' | 'READ' 'COMMITTED' | 'SNAPSHOT' | 'REPEATABLE' 'READ' | 'SERIALIZABLE' ) ) | ( 'PRIORITY' ( 'LOW' | 'NORMAL' | 'HIGH' ) ) | ( 'READ' 'ONLY' | 'READ' 'WRITE' ) | ( 'AS' 'OF' 'SYSTEM' 'TIME' a_expr ) | ( 'DEFERRABLE' | 'NOT' 'DEFERRABLE' ) ) ) )* + | 'BEGIN' 'TRANSACTION' 'ISOLATION' 'LEVEL' 'REPEATABLE' 'READ' ( ( ( ',' | ) ( ( 'ISOLATION' 'LEVEL' ( 'READ' 'UNCOMMITTED' | 'READ' 'COMMITTED' | 'SNAPSHOT' | 'REPEATABLE' 'READ' | 'SERIALIZABLE' ) ) | ( 'PRIORITY' ( 'LOW' | 'NORMAL' | 'HIGH' ) ) | ( 'READ' 'ONLY' | 'READ' 'WRITE' ) | ( 'AS' 'OF' 'SYSTEM' 'TIME' a_expr ) | ( 'DEFERRABLE' | 'NOT' 'DEFERRABLE' ) ) ) )* + | 'BEGIN' 'TRANSACTION' 'ISOLATION' 'LEVEL' 'SERIALIZABLE' ( ( ( ',' | ) ( ( 'ISOLATION' 'LEVEL' ( 'READ' 'UNCOMMITTED' | 'READ' 'COMMITTED' | 'SNAPSHOT' | 'REPEATABLE' 'READ' | 'SERIALIZABLE' ) ) | ( 'PRIORITY' ( 'LOW' | 'NORMAL' | 'HIGH' ) ) | ( 'READ' 'ONLY' | 'READ' 'WRITE' ) | ( 'AS' 'OF' 'SYSTEM' 'TIME' a_expr ) | ( 'DEFERRABLE' | 'NOT' 'DEFERRABLE' ) ) ) )* + | 'BEGIN' 'TRANSACTION' 'PRIORITY' 'LOW' ( ( ( ',' | ) ( ( 'ISOLATION' 'LEVEL' ( 'READ' 'UNCOMMITTED' | 'READ' 'COMMITTED' | 'SNAPSHOT' | 'REPEATABLE' 'READ' | 'SERIALIZABLE' ) ) | ( 'PRIORITY' ( 'LOW' | 'NORMAL' | 'HIGH' ) ) | ( 'READ' 'ONLY' | 'READ' 'WRITE' ) | ( 'AS' 'OF' 'SYSTEM' 'TIME' a_expr ) | ( 'DEFERRABLE' | 'NOT' 'DEFERRABLE' ) ) ) )* + | 'BEGIN' 'TRANSACTION' 'PRIORITY' 'NORMAL' ( ( ( ',' | ) ( ( 'ISOLATION' 'LEVEL' ( 'READ' 'UNCOMMITTED' | 'READ' 'COMMITTED' | 'SNAPSHOT' | 'REPEATABLE' 'READ' | 'SERIALIZABLE' ) ) | ( 'PRIORITY' ( 'LOW' | 'NORMAL' | 'HIGH' ) ) | ( 'READ' 'ONLY' | 'READ' 'WRITE' ) | ( 'AS' 'OF' 'SYSTEM' 'TIME' a_expr ) | ( 'DEFERRABLE' | 'NOT' 'DEFERRABLE' ) ) ) )* + | 'BEGIN' 'TRANSACTION' 'PRIORITY' 'HIGH' ( ( ( ',' | ) ( ( 'ISOLATION' 'LEVEL' ( 'READ' 'UNCOMMITTED' | 'READ' 'COMMITTED' | 'SNAPSHOT' | 'REPEATABLE' 'READ' | 'SERIALIZABLE' ) ) | ( 'PRIORITY' ( 'LOW' | 'NORMAL' | 'HIGH' ) ) | ( 'READ' 'ONLY' | 'READ' 'WRITE' ) | ( 'AS' 'OF' 'SYSTEM' 'TIME' a_expr ) | ( 'DEFERRABLE' | 'NOT' 'DEFERRABLE' ) ) ) )* + | 'BEGIN' 'TRANSACTION' 'READ' 'ONLY' ( ( ( ',' | ) ( ( 'ISOLATION' 'LEVEL' ( 'READ' 'UNCOMMITTED' | 'READ' 'COMMITTED' | 'SNAPSHOT' | 'REPEATABLE' 'READ' | 'SERIALIZABLE' ) ) | ( 'PRIORITY' ( 'LOW' | 'NORMAL' | 'HIGH' ) ) | ( 'READ' 'ONLY' | 'READ' 'WRITE' ) | ( 'AS' 'OF' 'SYSTEM' 'TIME' a_expr ) | ( 'DEFERRABLE' | 'NOT' 'DEFERRABLE' ) ) ) )* + | 'BEGIN' 'TRANSACTION' 'READ' 'WRITE' ( ( ( ',' | ) ( ( 'ISOLATION' 'LEVEL' ( 'READ' 'UNCOMMITTED' | 'READ' 'COMMITTED' | 'SNAPSHOT' | 'REPEATABLE' 'READ' | 'SERIALIZABLE' ) ) | ( 'PRIORITY' ( 'LOW' | 'NORMAL' | 'HIGH' ) ) | ( 'READ' 'ONLY' | 'READ' 'WRITE' ) | ( 'AS' 'OF' 'SYSTEM' 'TIME' a_expr ) | ( 'DEFERRABLE' | 'NOT' 'DEFERRABLE' ) ) ) )* + | 'BEGIN' 'TRANSACTION' 'AS' 'OF' 'SYSTEM' 'TIME' a_expr ( ( ( ',' | ) ( ( 'ISOLATION' 'LEVEL' ( 'READ' 'UNCOMMITTED' | 'READ' 'COMMITTED' | 'SNAPSHOT' | 'REPEATABLE' 'READ' | 'SERIALIZABLE' ) ) | ( 'PRIORITY' ( 'LOW' | 'NORMAL' | 'HIGH' ) ) | ( 'READ' 'ONLY' | 'READ' 'WRITE' ) | ( 'AS' 'OF' 'SYSTEM' 'TIME' a_expr ) | ( 'DEFERRABLE' | 'NOT' 'DEFERRABLE' ) ) ) )* + | 'BEGIN' 'TRANSACTION' 'DEFERRABLE' ( ( ( ',' | ) ( ( 'ISOLATION' 'LEVEL' ( 'READ' 'UNCOMMITTED' | 'READ' 'COMMITTED' | 'SNAPSHOT' | 'REPEATABLE' 'READ' | 'SERIALIZABLE' ) ) | ( 'PRIORITY' ( 'LOW' | 'NORMAL' | 'HIGH' ) ) | ( 'READ' 'ONLY' | 'READ' 'WRITE' ) | ( 'AS' 'OF' 'SYSTEM' 'TIME' a_expr ) | ( 'DEFERRABLE' | 'NOT' 'DEFERRABLE' ) ) ) )* + | 'BEGIN' 'TRANSACTION' 'NOT' 'DEFERRABLE' ( ( ( ',' | ) ( ( 'ISOLATION' 'LEVEL' ( 'READ' 'UNCOMMITTED' | 'READ' 'COMMITTED' | 'SNAPSHOT' | 'REPEATABLE' 'READ' | 'SERIALIZABLE' ) ) | ( 'PRIORITY' ( 'LOW' | 'NORMAL' | 'HIGH' ) ) | ( 'READ' 'ONLY' | 'READ' 'WRITE' ) | ( 'AS' 'OF' 'SYSTEM' 'TIME' a_expr ) | ( 'DEFERRABLE' | 'NOT' 'DEFERRABLE' ) ) ) )* + | 'BEGIN' 'TRANSACTION' + | 'BEGIN' 'ISOLATION' 'LEVEL' 'READ' 'UNCOMMITTED' ( ( ( ',' | ) ( ( 'ISOLATION' 'LEVEL' ( 'READ' 'UNCOMMITTED' | 'READ' 'COMMITTED' | 'SNAPSHOT' | 'REPEATABLE' 'READ' | 'SERIALIZABLE' ) ) | ( 'PRIORITY' ( 'LOW' | 'NORMAL' | 'HIGH' ) ) | ( 'READ' 'ONLY' | 'READ' 'WRITE' ) | ( 'AS' 'OF' 'SYSTEM' 'TIME' a_expr ) | ( 'DEFERRABLE' | 'NOT' 'DEFERRABLE' ) ) ) )* + | 'BEGIN' 'ISOLATION' 'LEVEL' 'READ' 'COMMITTED' ( ( ( ',' | ) ( ( 'ISOLATION' 'LEVEL' ( 'READ' 'UNCOMMITTED' | 'READ' 'COMMITTED' | 'SNAPSHOT' | 'REPEATABLE' 'READ' | 'SERIALIZABLE' ) ) | ( 'PRIORITY' ( 'LOW' | 'NORMAL' | 'HIGH' ) ) | ( 'READ' 'ONLY' | 'READ' 'WRITE' ) | ( 'AS' 'OF' 'SYSTEM' 'TIME' a_expr ) | ( 'DEFERRABLE' | 'NOT' 'DEFERRABLE' ) ) ) )* + | 'BEGIN' 'ISOLATION' 'LEVEL' 'SNAPSHOT' ( ( ( ',' | ) ( ( 'ISOLATION' 'LEVEL' ( 'READ' 'UNCOMMITTED' | 'READ' 'COMMITTED' | 'SNAPSHOT' | 'REPEATABLE' 'READ' | 'SERIALIZABLE' ) ) | ( 'PRIORITY' ( 'LOW' | 'NORMAL' | 'HIGH' ) ) | ( 'READ' 'ONLY' | 'READ' 'WRITE' ) | ( 'AS' 'OF' 'SYSTEM' 'TIME' a_expr ) | ( 'DEFERRABLE' | 'NOT' 'DEFERRABLE' ) ) ) )* + | 'BEGIN' 'ISOLATION' 'LEVEL' 'REPEATABLE' 'READ' ( ( ( ',' | ) ( ( 'ISOLATION' 'LEVEL' ( 'READ' 'UNCOMMITTED' | 'READ' 'COMMITTED' | 'SNAPSHOT' | 'REPEATABLE' 'READ' | 'SERIALIZABLE' ) ) | ( 'PRIORITY' ( 'LOW' | 'NORMAL' | 'HIGH' ) ) | ( 'READ' 'ONLY' | 'READ' 'WRITE' ) | ( 'AS' 'OF' 'SYSTEM' 'TIME' a_expr ) | ( 'DEFERRABLE' | 'NOT' 'DEFERRABLE' ) ) ) )* + | 'BEGIN' 'ISOLATION' 'LEVEL' 'SERIALIZABLE' ( ( ( ',' | ) ( ( 'ISOLATION' 'LEVEL' ( 'READ' 'UNCOMMITTED' | 'READ' 'COMMITTED' | 'SNAPSHOT' | 'REPEATABLE' 'READ' | 'SERIALIZABLE' ) ) | ( 'PRIORITY' ( 'LOW' | 'NORMAL' | 'HIGH' ) ) | ( 'READ' 'ONLY' | 'READ' 'WRITE' ) | ( 'AS' 'OF' 'SYSTEM' 'TIME' a_expr ) | ( 'DEFERRABLE' | 'NOT' 'DEFERRABLE' ) ) ) )* + | 'BEGIN' 'PRIORITY' 'LOW' ( ( ( ',' | ) ( ( 'ISOLATION' 'LEVEL' ( 'READ' 'UNCOMMITTED' | 'READ' 'COMMITTED' | 'SNAPSHOT' | 'REPEATABLE' 'READ' | 'SERIALIZABLE' ) ) | ( 'PRIORITY' ( 'LOW' | 'NORMAL' | 'HIGH' ) ) | ( 'READ' 'ONLY' | 'READ' 'WRITE' ) | ( 'AS' 'OF' 'SYSTEM' 'TIME' a_expr ) | ( 'DEFERRABLE' | 'NOT' 'DEFERRABLE' ) ) ) )* + | 'BEGIN' 'PRIORITY' 'NORMAL' ( ( ( ',' | ) ( ( 'ISOLATION' 'LEVEL' ( 'READ' 'UNCOMMITTED' | 'READ' 'COMMITTED' | 'SNAPSHOT' | 'REPEATABLE' 'READ' | 'SERIALIZABLE' ) ) | ( 'PRIORITY' ( 'LOW' | 'NORMAL' | 'HIGH' ) ) | ( 'READ' 'ONLY' | 'READ' 'WRITE' ) | ( 'AS' 'OF' 'SYSTEM' 'TIME' a_expr ) | ( 'DEFERRABLE' | 'NOT' 'DEFERRABLE' ) ) ) )* + | 'BEGIN' 'PRIORITY' 'HIGH' ( ( ( ',' | ) ( ( 'ISOLATION' 'LEVEL' ( 'READ' 'UNCOMMITTED' | 'READ' 'COMMITTED' | 'SNAPSHOT' | 'REPEATABLE' 'READ' | 'SERIALIZABLE' ) ) | ( 'PRIORITY' ( 'LOW' | 'NORMAL' | 'HIGH' ) ) | ( 'READ' 'ONLY' | 'READ' 'WRITE' ) | ( 'AS' 'OF' 'SYSTEM' 'TIME' a_expr ) | ( 'DEFERRABLE' | 'NOT' 'DEFERRABLE' ) ) ) )* + | 'BEGIN' 'READ' 'ONLY' ( ( ( ',' | ) ( ( 'ISOLATION' 'LEVEL' ( 'READ' 'UNCOMMITTED' | 'READ' 'COMMITTED' | 'SNAPSHOT' | 'REPEATABLE' 'READ' | 'SERIALIZABLE' ) ) | ( 'PRIORITY' ( 'LOW' | 'NORMAL' | 'HIGH' ) ) | ( 'READ' 'ONLY' | 'READ' 'WRITE' ) | ( 'AS' 'OF' 'SYSTEM' 'TIME' a_expr ) | ( 'DEFERRABLE' | 'NOT' 'DEFERRABLE' ) ) ) )* + | 'BEGIN' 'READ' 'WRITE' ( ( ( ',' | ) ( ( 'ISOLATION' 'LEVEL' ( 'READ' 'UNCOMMITTED' | 'READ' 'COMMITTED' | 'SNAPSHOT' | 'REPEATABLE' 'READ' | 'SERIALIZABLE' ) ) | ( 'PRIORITY' ( 'LOW' | 'NORMAL' | 'HIGH' ) ) | ( 'READ' 'ONLY' | 'READ' 'WRITE' ) | ( 'AS' 'OF' 'SYSTEM' 'TIME' a_expr ) | ( 'DEFERRABLE' | 'NOT' 'DEFERRABLE' ) ) ) )* + | 'BEGIN' 'AS' 'OF' 'SYSTEM' 'TIME' a_expr ( ( ( ',' | ) ( ( 'ISOLATION' 'LEVEL' ( 'READ' 'UNCOMMITTED' | 'READ' 'COMMITTED' | 'SNAPSHOT' | 'REPEATABLE' 'READ' | 'SERIALIZABLE' ) ) | ( 'PRIORITY' ( 'LOW' | 'NORMAL' | 'HIGH' ) ) | ( 'READ' 'ONLY' | 'READ' 'WRITE' ) | ( 'AS' 'OF' 'SYSTEM' 'TIME' a_expr ) | ( 'DEFERRABLE' | 'NOT' 'DEFERRABLE' ) ) ) )* + | 'BEGIN' 'DEFERRABLE' ( ( ( ',' | ) ( ( 'ISOLATION' 'LEVEL' ( 'READ' 'UNCOMMITTED' | 'READ' 'COMMITTED' | 'SNAPSHOT' | 'REPEATABLE' 'READ' | 'SERIALIZABLE' ) ) | ( 'PRIORITY' ( 'LOW' | 'NORMAL' | 'HIGH' ) ) | ( 'READ' 'ONLY' | 'READ' 'WRITE' ) | ( 'AS' 'OF' 'SYSTEM' 'TIME' a_expr ) | ( 'DEFERRABLE' | 'NOT' 'DEFERRABLE' ) ) ) )* + | 'BEGIN' 'NOT' 'DEFERRABLE' ( ( ( ',' | ) ( ( 'ISOLATION' 'LEVEL' ( 'READ' 'UNCOMMITTED' | 'READ' 'COMMITTED' | 'SNAPSHOT' | 'REPEATABLE' 'READ' | 'SERIALIZABLE' ) ) | ( 'PRIORITY' ( 'LOW' | 'NORMAL' | 'HIGH' ) ) | ( 'READ' 'ONLY' | 'READ' 'WRITE' ) | ( 'AS' 'OF' 'SYSTEM' 'TIME' a_expr ) | ( 'DEFERRABLE' | 'NOT' 'DEFERRABLE' ) ) ) )* + | 'BEGIN' diff --git a/docs/generated/sql/bnf/nonpreparable_set_stmt.bnf b/docs/generated/sql/bnf/nonpreparable_set_stmt.bnf new file mode 100644 index 000000000000..c0cd7be13d8b --- /dev/null +++ b/docs/generated/sql/bnf/nonpreparable_set_stmt.bnf @@ -0,0 +1,2 @@ +nonpreparable_set_stmt ::= + set_transaction_stmt diff --git a/docs/generated/sql/bnf/set_transaction.bnf b/docs/generated/sql/bnf/set_transaction.bnf index 24f8f21d7433..d8a1db215d37 100644 --- a/docs/generated/sql/bnf/set_transaction.bnf +++ b/docs/generated/sql/bnf/set_transaction.bnf @@ -1,9 +1,14 @@ -nonpreparable_set_stmt ::= - 'SET' 'TRANSACTION' 'PRIORITY' 'LOW' ( ( ( ',' | ) ( ( 'PRIORITY' ( 'LOW' | 'NORMAL' | 'HIGH' ) ) | ( 'READ' 'ONLY' | 'READ' 'WRITE' ) | ( 'AS' 'OF' 'SYSTEM' 'TIME' a_expr ) | ( 'DEFERRABLE' | 'NOT' 'DEFERRABLE' ) ) ) )* - | 'SET' 'TRANSACTION' 'PRIORITY' 'NORMAL' ( ( ( ',' | ) ( ( 'PRIORITY' ( 'LOW' | 'NORMAL' | 'HIGH' ) ) | ( 'READ' 'ONLY' | 'READ' 'WRITE' ) | ( 'AS' 'OF' 'SYSTEM' 'TIME' a_expr ) | ( 'DEFERRABLE' | 'NOT' 'DEFERRABLE' ) ) ) )* - | 'SET' 'TRANSACTION' 'PRIORITY' 'HIGH' ( ( ( ',' | ) ( ( 'PRIORITY' ( 'LOW' | 'NORMAL' | 'HIGH' ) ) | ( 'READ' 'ONLY' | 'READ' 'WRITE' ) | ( 'AS' 'OF' 'SYSTEM' 'TIME' a_expr ) | ( 'DEFERRABLE' | 'NOT' 'DEFERRABLE' ) ) ) )* - | 'SET' 'TRANSACTION' 'READ' 'ONLY' ( ( ( ',' | ) ( ( 'PRIORITY' ( 'LOW' | 'NORMAL' | 'HIGH' ) ) | ( 'READ' 'ONLY' | 'READ' 'WRITE' ) | ( 'AS' 'OF' 'SYSTEM' 'TIME' a_expr ) | ( 'DEFERRABLE' | 'NOT' 'DEFERRABLE' ) ) ) )* - | 'SET' 'TRANSACTION' 'READ' 'WRITE' ( ( ( ',' | ) ( ( 'PRIORITY' ( 'LOW' | 'NORMAL' | 'HIGH' ) ) | ( 'READ' 'ONLY' | 'READ' 'WRITE' ) | ( 'AS' 'OF' 'SYSTEM' 'TIME' a_expr ) | ( 'DEFERRABLE' | 'NOT' 'DEFERRABLE' ) ) ) )* - | 'SET' 'TRANSACTION' 'AS' 'OF' 'SYSTEM' 'TIME' a_expr ( ( ( ',' | ) ( ( 'PRIORITY' ( 'LOW' | 'NORMAL' | 'HIGH' ) ) | ( 'READ' 'ONLY' | 'READ' 'WRITE' ) | ( 'AS' 'OF' 'SYSTEM' 'TIME' a_expr ) | ( 'DEFERRABLE' | 'NOT' 'DEFERRABLE' ) ) ) )* - | 'SET' 'TRANSACTION' 'DEFERRABLE' ( ( ( ',' | ) ( ( 'PRIORITY' ( 'LOW' | 'NORMAL' | 'HIGH' ) ) | ( 'READ' 'ONLY' | 'READ' 'WRITE' ) | ( 'AS' 'OF' 'SYSTEM' 'TIME' a_expr ) | ( 'DEFERRABLE' | 'NOT' 'DEFERRABLE' ) ) ) )* - | 'SET' 'TRANSACTION' 'NOT' 'DEFERRABLE' ( ( ( ',' | ) ( ( 'PRIORITY' ( 'LOW' | 'NORMAL' | 'HIGH' ) ) | ( 'READ' 'ONLY' | 'READ' 'WRITE' ) | ( 'AS' 'OF' 'SYSTEM' 'TIME' a_expr ) | ( 'DEFERRABLE' | 'NOT' 'DEFERRABLE' ) ) ) )* +set_transaction_stmt ::= + 'SET' 'TRANSACTION' 'ISOLATION' 'LEVEL' 'READ' 'UNCOMMITTED' ( ( ( ',' | ) ( ( 'ISOLATION' 'LEVEL' ( 'READ' 'UNCOMMITTED' | 'READ' 'COMMITTED' | 'SNAPSHOT' | 'REPEATABLE' 'READ' | 'SERIALIZABLE' ) ) | ( 'PRIORITY' ( 'LOW' | 'NORMAL' | 'HIGH' ) ) | ( 'READ' 'ONLY' | 'READ' 'WRITE' ) | ( 'AS' 'OF' 'SYSTEM' 'TIME' a_expr ) | ( 'DEFERRABLE' | 'NOT' 'DEFERRABLE' ) ) ) )* + | 'SET' 'TRANSACTION' 'ISOLATION' 'LEVEL' 'READ' 'COMMITTED' ( ( ( ',' | ) ( ( 'ISOLATION' 'LEVEL' ( 'READ' 'UNCOMMITTED' | 'READ' 'COMMITTED' | 'SNAPSHOT' | 'REPEATABLE' 'READ' | 'SERIALIZABLE' ) ) | ( 'PRIORITY' ( 'LOW' | 'NORMAL' | 'HIGH' ) ) | ( 'READ' 'ONLY' | 'READ' 'WRITE' ) | ( 'AS' 'OF' 'SYSTEM' 'TIME' a_expr ) | ( 'DEFERRABLE' | 'NOT' 'DEFERRABLE' ) ) ) )* + | 'SET' 'TRANSACTION' 'ISOLATION' 'LEVEL' 'SNAPSHOT' ( ( ( ',' | ) ( ( 'ISOLATION' 'LEVEL' ( 'READ' 'UNCOMMITTED' | 'READ' 'COMMITTED' | 'SNAPSHOT' | 'REPEATABLE' 'READ' | 'SERIALIZABLE' ) ) | ( 'PRIORITY' ( 'LOW' | 'NORMAL' | 'HIGH' ) ) | ( 'READ' 'ONLY' | 'READ' 'WRITE' ) | ( 'AS' 'OF' 'SYSTEM' 'TIME' a_expr ) | ( 'DEFERRABLE' | 'NOT' 'DEFERRABLE' ) ) ) )* + | 'SET' 'TRANSACTION' 'ISOLATION' 'LEVEL' 'REPEATABLE' 'READ' ( ( ( ',' | ) ( ( 'ISOLATION' 'LEVEL' ( 'READ' 'UNCOMMITTED' | 'READ' 'COMMITTED' | 'SNAPSHOT' | 'REPEATABLE' 'READ' | 'SERIALIZABLE' ) ) | ( 'PRIORITY' ( 'LOW' | 'NORMAL' | 'HIGH' ) ) | ( 'READ' 'ONLY' | 'READ' 'WRITE' ) | ( 'AS' 'OF' 'SYSTEM' 'TIME' a_expr ) | ( 'DEFERRABLE' | 'NOT' 'DEFERRABLE' ) ) ) )* + | 'SET' 'TRANSACTION' 'ISOLATION' 'LEVEL' 'SERIALIZABLE' ( ( ( ',' | ) ( ( 'ISOLATION' 'LEVEL' ( 'READ' 'UNCOMMITTED' | 'READ' 'COMMITTED' | 'SNAPSHOT' | 'REPEATABLE' 'READ' | 'SERIALIZABLE' ) ) | ( 'PRIORITY' ( 'LOW' | 'NORMAL' | 'HIGH' ) ) | ( 'READ' 'ONLY' | 'READ' 'WRITE' ) | ( 'AS' 'OF' 'SYSTEM' 'TIME' a_expr ) | ( 'DEFERRABLE' | 'NOT' 'DEFERRABLE' ) ) ) )* + | 'SET' 'TRANSACTION' 'PRIORITY' 'LOW' ( ( ( ',' | ) ( ( 'ISOLATION' 'LEVEL' ( 'READ' 'UNCOMMITTED' | 'READ' 'COMMITTED' | 'SNAPSHOT' | 'REPEATABLE' 'READ' | 'SERIALIZABLE' ) ) | ( 'PRIORITY' ( 'LOW' | 'NORMAL' | 'HIGH' ) ) | ( 'READ' 'ONLY' | 'READ' 'WRITE' ) | ( 'AS' 'OF' 'SYSTEM' 'TIME' a_expr ) | ( 'DEFERRABLE' | 'NOT' 'DEFERRABLE' ) ) ) )* + | 'SET' 'TRANSACTION' 'PRIORITY' 'NORMAL' ( ( ( ',' | ) ( ( 'ISOLATION' 'LEVEL' ( 'READ' 'UNCOMMITTED' | 'READ' 'COMMITTED' | 'SNAPSHOT' | 'REPEATABLE' 'READ' | 'SERIALIZABLE' ) ) | ( 'PRIORITY' ( 'LOW' | 'NORMAL' | 'HIGH' ) ) | ( 'READ' 'ONLY' | 'READ' 'WRITE' ) | ( 'AS' 'OF' 'SYSTEM' 'TIME' a_expr ) | ( 'DEFERRABLE' | 'NOT' 'DEFERRABLE' ) ) ) )* + | 'SET' 'TRANSACTION' 'PRIORITY' 'HIGH' ( ( ( ',' | ) ( ( 'ISOLATION' 'LEVEL' ( 'READ' 'UNCOMMITTED' | 'READ' 'COMMITTED' | 'SNAPSHOT' | 'REPEATABLE' 'READ' | 'SERIALIZABLE' ) ) | ( 'PRIORITY' ( 'LOW' | 'NORMAL' | 'HIGH' ) ) | ( 'READ' 'ONLY' | 'READ' 'WRITE' ) | ( 'AS' 'OF' 'SYSTEM' 'TIME' a_expr ) | ( 'DEFERRABLE' | 'NOT' 'DEFERRABLE' ) ) ) )* + | 'SET' 'TRANSACTION' 'READ' 'ONLY' ( ( ( ',' | ) ( ( 'ISOLATION' 'LEVEL' ( 'READ' 'UNCOMMITTED' | 'READ' 'COMMITTED' | 'SNAPSHOT' | 'REPEATABLE' 'READ' | 'SERIALIZABLE' ) ) | ( 'PRIORITY' ( 'LOW' | 'NORMAL' | 'HIGH' ) ) | ( 'READ' 'ONLY' | 'READ' 'WRITE' ) | ( 'AS' 'OF' 'SYSTEM' 'TIME' a_expr ) | ( 'DEFERRABLE' | 'NOT' 'DEFERRABLE' ) ) ) )* + | 'SET' 'TRANSACTION' 'READ' 'WRITE' ( ( ( ',' | ) ( ( 'ISOLATION' 'LEVEL' ( 'READ' 'UNCOMMITTED' | 'READ' 'COMMITTED' | 'SNAPSHOT' | 'REPEATABLE' 'READ' | 'SERIALIZABLE' ) ) | ( 'PRIORITY' ( 'LOW' | 'NORMAL' | 'HIGH' ) ) | ( 'READ' 'ONLY' | 'READ' 'WRITE' ) | ( 'AS' 'OF' 'SYSTEM' 'TIME' a_expr ) | ( 'DEFERRABLE' | 'NOT' 'DEFERRABLE' ) ) ) )* + | 'SET' 'TRANSACTION' 'AS' 'OF' 'SYSTEM' 'TIME' a_expr ( ( ( ',' | ) ( ( 'ISOLATION' 'LEVEL' ( 'READ' 'UNCOMMITTED' | 'READ' 'COMMITTED' | 'SNAPSHOT' | 'REPEATABLE' 'READ' | 'SERIALIZABLE' ) ) | ( 'PRIORITY' ( 'LOW' | 'NORMAL' | 'HIGH' ) ) | ( 'READ' 'ONLY' | 'READ' 'WRITE' ) | ( 'AS' 'OF' 'SYSTEM' 'TIME' a_expr ) | ( 'DEFERRABLE' | 'NOT' 'DEFERRABLE' ) ) ) )* + | 'SET' 'TRANSACTION' 'DEFERRABLE' ( ( ( ',' | ) ( ( 'ISOLATION' 'LEVEL' ( 'READ' 'UNCOMMITTED' | 'READ' 'COMMITTED' | 'SNAPSHOT' | 'REPEATABLE' 'READ' | 'SERIALIZABLE' ) ) | ( 'PRIORITY' ( 'LOW' | 'NORMAL' | 'HIGH' ) ) | ( 'READ' 'ONLY' | 'READ' 'WRITE' ) | ( 'AS' 'OF' 'SYSTEM' 'TIME' a_expr ) | ( 'DEFERRABLE' | 'NOT' 'DEFERRABLE' ) ) ) )* + | 'SET' 'TRANSACTION' 'NOT' 'DEFERRABLE' ( ( ( ',' | ) ( ( 'ISOLATION' 'LEVEL' ( 'READ' 'UNCOMMITTED' | 'READ' 'COMMITTED' | 'SNAPSHOT' | 'REPEATABLE' 'READ' | 'SERIALIZABLE' ) ) | ( 'PRIORITY' ( 'LOW' | 'NORMAL' | 'HIGH' ) ) | ( 'READ' 'ONLY' | 'READ' 'WRITE' ) | ( 'AS' 'OF' 'SYSTEM' 'TIME' a_expr ) | ( 'DEFERRABLE' | 'NOT' 'DEFERRABLE' ) ) ) )* diff --git a/docs/generated/sql/bnf/stmt_block.bnf b/docs/generated/sql/bnf/stmt_block.bnf index 4ee6e58af28c..b36af937b7a2 100644 --- a/docs/generated/sql/bnf/stmt_block.bnf +++ b/docs/generated/sql/bnf/stmt_block.bnf @@ -2155,7 +2155,8 @@ typename ::= | simple_typename 'ARRAY' transaction_mode ::= - transaction_user_priority + transaction_iso_level + | transaction_user_priority | transaction_read_mode | as_of_clause | transaction_deferrable_mode @@ -2959,6 +2960,9 @@ opt_array_bounds ::= '[' ']' | +transaction_iso_level ::= + 'ISOLATION' 'LEVEL' iso_level + transaction_user_priority ::= 'PRIORITY' user_priority @@ -3483,6 +3487,13 @@ interval_type ::= | 'INTERVAL' interval_qualifier | 'INTERVAL' '(' iconst32 ')' +iso_level ::= + 'READ' 'UNCOMMITTED' + | 'READ' 'COMMITTED' + | 'SNAPSHOT' + | 'REPEATABLE' 'READ' + | 'SERIALIZABLE' + user_priority ::= 'LOW' | 'NORMAL' diff --git a/pkg/cmd/docgen/diagrams.go b/pkg/cmd/docgen/diagrams.go index 153a9ee694fe..9f21fa121de8 100644 --- a/pkg/cmd/docgen/diagrams.go +++ b/pkg/cmd/docgen/diagrams.go @@ -625,22 +625,20 @@ var specs = []stmtSpec{ exclude: []*regexp.Regexp{regexp.MustCompile("'IN'")}, }, { - name: "begin_stmt", + name: "legacy_begin_stmt", inline: []string{ "opt_transaction", "begin_transaction", - "transaction_mode", - "transaction_user_priority", - "user_priority", - "iso_level", "transaction_mode_list", "opt_comma", + "transaction_mode", + "transaction_user_priority", "transaction_read_mode", "as_of_clause", "transaction_deferrable_mode", - }, - exclude: []*regexp.Regexp{ - regexp.MustCompile("'START'"), + "user_priority", + "transaction_iso_level", + "iso_level", }, }, { @@ -1420,7 +1418,7 @@ var specs = []stmtSpec{ }, { name: "set_transaction", - stmt: "nonpreparable_set_stmt", + stmt: "set_transaction_stmt", inline: []string{ "set_transaction_stmt", "transaction_mode", @@ -1431,6 +1429,8 @@ var specs = []stmtSpec{ "as_of_clause", "opt_comma", "transaction_deferrable_mode", + "transaction_iso_level", + "iso_level", }, match: []*regexp.Regexp{regexp.MustCompile("'SET' 'TRANSACTION'")}, }, diff --git a/pkg/gen/bnf.bzl b/pkg/gen/bnf.bzl index b80977b7fccc..a6c4e32a5ffe 100644 --- a/pkg/gen/bnf.bzl +++ b/pkg/gen/bnf.bzl @@ -72,7 +72,6 @@ BNF_SRCS = [ "//docs/generated/sql/bnf:analyze_stmt.bnf", "//docs/generated/sql/bnf:backup.bnf", "//docs/generated/sql/bnf:backup_options.bnf", - "//docs/generated/sql/bnf:begin_stmt.bnf", "//docs/generated/sql/bnf:begin_transaction.bnf", "//docs/generated/sql/bnf:call.bnf", "//docs/generated/sql/bnf:cancel_all_jobs.bnf", @@ -168,6 +167,7 @@ BNF_SRCS = [ "//docs/generated/sql/bnf:like_table_option_list.bnf", "//docs/generated/sql/bnf:limit_clause.bnf", "//docs/generated/sql/bnf:move_cursor_stmt.bnf", + "//docs/generated/sql/bnf:nonpreparable_set_stmt.bnf", "//docs/generated/sql/bnf:not_null_column_level.bnf", "//docs/generated/sql/bnf:offset_clause.bnf", "//docs/generated/sql/bnf:on_conflict.bnf", @@ -223,7 +223,6 @@ BNF_SRCS = [ "//docs/generated/sql/bnf:set_rest_more.bnf", "//docs/generated/sql/bnf:set_session_stmt.bnf", "//docs/generated/sql/bnf:set_transaction.bnf", - "//docs/generated/sql/bnf:set_transaction_stmt.bnf", "//docs/generated/sql/bnf:show_backup.bnf", "//docs/generated/sql/bnf:show_cluster_setting.bnf", "//docs/generated/sql/bnf:show_columns_stmt.bnf", diff --git a/pkg/gen/diagrams.bzl b/pkg/gen/diagrams.bzl index 72cb63109a60..bc2ab870b340 100644 --- a/pkg/gen/diagrams.bzl +++ b/pkg/gen/diagrams.bzl @@ -72,7 +72,6 @@ DIAGRAMS_SRCS = [ "//docs/generated/sql/bnf:analyze.html", "//docs/generated/sql/bnf:backup.html", "//docs/generated/sql/bnf:backup_options.html", - "//docs/generated/sql/bnf:begin.html", "//docs/generated/sql/bnf:begin_transaction.html", "//docs/generated/sql/bnf:call.html", "//docs/generated/sql/bnf:cancel.html", @@ -168,6 +167,7 @@ DIAGRAMS_SRCS = [ "//docs/generated/sql/bnf:like_table_option_list.html", "//docs/generated/sql/bnf:limit_clause.html", "//docs/generated/sql/bnf:move_cursor.html", + "//docs/generated/sql/bnf:nonpreparable_set.html", "//docs/generated/sql/bnf:not_null_column_level.html", "//docs/generated/sql/bnf:offset_clause.html", "//docs/generated/sql/bnf:on_conflict.html", diff --git a/pkg/gen/docs.bzl b/pkg/gen/docs.bzl index aaf3e15f31db..a44bf6fcf520 100644 --- a/pkg/gen/docs.bzl +++ b/pkg/gen/docs.bzl @@ -85,7 +85,6 @@ DOCS_SRCS = [ "//docs/generated/sql/bnf:analyze_stmt.bnf", "//docs/generated/sql/bnf:backup.bnf", "//docs/generated/sql/bnf:backup_options.bnf", - "//docs/generated/sql/bnf:begin_stmt.bnf", "//docs/generated/sql/bnf:begin_transaction.bnf", "//docs/generated/sql/bnf:call.bnf", "//docs/generated/sql/bnf:cancel_all_jobs.bnf", @@ -181,6 +180,7 @@ DOCS_SRCS = [ "//docs/generated/sql/bnf:like_table_option_list.bnf", "//docs/generated/sql/bnf:limit_clause.bnf", "//docs/generated/sql/bnf:move_cursor_stmt.bnf", + "//docs/generated/sql/bnf:nonpreparable_set_stmt.bnf", "//docs/generated/sql/bnf:not_null_column_level.bnf", "//docs/generated/sql/bnf:offset_clause.bnf", "//docs/generated/sql/bnf:on_conflict.bnf", @@ -236,7 +236,6 @@ DOCS_SRCS = [ "//docs/generated/sql/bnf:set_rest_more.bnf", "//docs/generated/sql/bnf:set_session_stmt.bnf", "//docs/generated/sql/bnf:set_transaction.bnf", - "//docs/generated/sql/bnf:set_transaction_stmt.bnf", "//docs/generated/sql/bnf:show_backup.bnf", "//docs/generated/sql/bnf:show_cluster_setting.bnf", "//docs/generated/sql/bnf:show_columns_stmt.bnf", diff --git a/pkg/sql/parser/sql.y b/pkg/sql/parser/sql.y index 0ff0009f229d..508aa82678d8 100644 --- a/pkg/sql/parser/sql.y +++ b/pkg/sql/parser/sql.y @@ -12641,7 +12641,6 @@ opt_comma: transaction_mode: transaction_iso_level { - /* SKIP DOC */ $$.val = tree.TransactionModes{Isolation: $1.isoLevel()} } | transaction_user_priority