diff --git a/docs/generated/settings/BUILD.bazel b/docs/generated/settings/BUILD.bazel new file mode 100644 index 000000000000..10b85ecf53eb --- /dev/null +++ b/docs/generated/settings/BUILD.bazel @@ -0,0 +1,13 @@ +genrule( + name = "settings", + outs = ["settings.html"], + cmd = "$(location //pkg/cmd/cockroach-short) gen settings-list --format=html > $@", + exec_tools = ["//pkg/cmd/cockroach-short"], +) + +genrule( + name = "settings_for_tenants", + outs = ["settings-for-tenants.txt"], + cmd = "$(location //pkg/cmd/cockroach-short) gen settings-list --without-system-only > $@", + exec_tools = ["//pkg/cmd/cockroach-short"], +) diff --git a/docs/generated/sql/BUILD.bazel b/docs/generated/sql/BUILD.bazel new file mode 100644 index 000000000000..df2aaa735462 --- /dev/null +++ b/docs/generated/sql/BUILD.bazel @@ -0,0 +1,17 @@ +genrule( + name = "sql", + outs = [ + "aggregates.md", + "functions.md", + "operators.md", + "window_functions.md", + ], + cmd = """ +$(location //pkg/cmd/docgen) functions . --quiet +mv aggregates.md $(location aggregates.md) +mv functions.md $(location functions.md) +mv operators.md $(location operators.md) +mv window_functions.md $(location window_functions.md) +""", + exec_tools = ["//pkg/cmd/docgen"], +) diff --git a/pkg/base/test_server_args.go b/pkg/base/test_server_args.go index 4f0b4e2dcd63..53ad1393625e 100644 --- a/pkg/base/test_server_args.go +++ b/pkg/base/test_server_args.go @@ -268,4 +268,7 @@ type TestTenantArgs struct { // automatically open a connection to the server. That's equivalent to running // SET DATABASE=foo, which works even if the database doesn't (yet) exist. UseDatabase string + + // Skip check for tenant existence when running the test. + SkipTenantCheck bool } diff --git a/pkg/ccl/changefeedccl/BUILD.bazel b/pkg/ccl/changefeedccl/BUILD.bazel index 9bc4f1b50d28..47a17c5568df 100644 --- a/pkg/ccl/changefeedccl/BUILD.bazel +++ b/pkg/ccl/changefeedccl/BUILD.bazel @@ -159,7 +159,9 @@ go_test( "//pkg/sql/catalog/descpb", "//pkg/sql/catalog/schemaexpr", "//pkg/sql/catalog/tabledesc", + "//pkg/sql/catalog/typedesc", "//pkg/sql/distsql", + "//pkg/sql/enum", "//pkg/sql/execinfra", "//pkg/sql/flowinfra", "//pkg/sql/parser", diff --git a/pkg/ccl/changefeedccl/avro.go b/pkg/ccl/changefeedccl/avro.go index a7f04c0c98fa..9ddbd8686c50 100644 --- a/pkg/ccl/changefeedccl/avro.go +++ b/pkg/ccl/changefeedccl/avro.go @@ -143,6 +143,7 @@ type avroDataRecord struct { colIdxByFieldIdx map[int]int fieldIdxByName map[string]int + fieldIdxByColIdx map[int]int // Allocate Go native representation once, to avoid repeated map allocation // when encoding. native map[string]interface{} @@ -468,6 +469,16 @@ func typeToAvroSchema(typ *types.T, reuseMap bool) (*avroSchemaField, error) { return tree.ParseDJSON(x.(string)) }, ) + case types.EnumFamily: + setNullable( + avroSchemaString, + func(d tree.Datum) (interface{}, error) { + return d.(*tree.DEnum).LogicalRep, nil + }, + func(x interface{}) (tree.Datum, error) { + return tree.MakeDEnumFromLogicalRepresentation(typ, x.(string)) + }, + ) case types.ArrayFamily: itemSchema, err := typeToAvroSchema(typ.ArrayContents(), false /*reuse map*/) if err != nil { @@ -545,6 +556,7 @@ func indexToAvroSchema( }, fieldIdxByName: make(map[string]int), colIdxByFieldIdx: make(map[int]int), + fieldIdxByColIdx: make(map[int]int), } colIdxByID := catalog.ColumnIDToOrdinalMap(tableDesc.PublicColumns()) for i := 0; i < index.NumKeyColumns(); i++ { @@ -559,6 +571,7 @@ func indexToAvroSchema( return nil, err } schema.colIdxByFieldIdx[len(schema.Fields)] = colIdx + schema.fieldIdxByColIdx[colIdx] = len(schema.Fields) schema.fieldIdxByName[field.Name] = len(schema.Fields) schema.Fields = append(schema.Fields, field) } @@ -598,6 +611,7 @@ func tableToAvroSchema( }, fieldIdxByName: make(map[string]int), colIdxByFieldIdx: make(map[int]int), + fieldIdxByColIdx: make(map[int]int), } for _, col := range tableDesc.PublicColumns() { field, err := columnToAvroSchema(col) @@ -606,6 +620,7 @@ func tableToAvroSchema( } schema.colIdxByFieldIdx[len(schema.Fields)] = col.Ordinal() schema.fieldIdxByName[field.Name] = len(schema.Fields) + schema.fieldIdxByColIdx[col.Ordinal()] = len(schema.Fields) schema.Fields = append(schema.Fields, field) } schemaJSON, err := json.Marshal(schema) @@ -819,6 +834,16 @@ func (r *avroEnvelopeRecord) BinaryFromRow( return r.codec.BinaryFromNative(buf, native) } +// Refresh the metadata for user-defined types on a cached schema +// The only user-defined type is enum, so this is usually a no-op +func (r *avroDataRecord) refreshTypeMetadata(tbl catalog.TableDescriptor) { + for _, col := range tbl.UserDefinedTypeColumns() { + if fieldIdx, ok := r.fieldIdxByColIdx[col.Ordinal()]; ok { + r.Fields[fieldIdx].typ = col.GetType() + } + } +} + // decimalToRat converts one of our apd decimals to the format expected by the // avro library we use. If the column has a fixed scale (which is always true if // precision is set) this is roundtripable without information loss. diff --git a/pkg/ccl/changefeedccl/avro_test.go b/pkg/ccl/changefeedccl/avro_test.go index 4dcb9a99cab7..7c200199f0dc 100644 --- a/pkg/ccl/changefeedccl/avro_test.go +++ b/pkg/ccl/changefeedccl/avro_test.go @@ -26,6 +26,8 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" "github.com/cockroachdb/cockroach/pkg/sql/catalog/schemaexpr" "github.com/cockroachdb/cockroach/pkg/sql/catalog/tabledesc" + "github.com/cockroachdb/cockroach/pkg/sql/catalog/typedesc" + "github.com/cockroachdb/cockroach/pkg/sql/enum" "github.com/cockroachdb/cockroach/pkg/sql/parser" "github.com/cockroachdb/cockroach/pkg/sql/randgen" "github.com/cockroachdb/cockroach/pkg/sql/rowenc" @@ -43,6 +45,15 @@ import ( "golang.org/x/text/collate" ) +var testTypes = make(map[string]*types.T) +var testTypeResolver = tree.MakeTestingMapTypeResolver(testTypes) + +func makeTestSemaCtx() tree.SemaContext { + testSemaCtx := tree.MakeSemaContext() + testSemaCtx.TypeResolver = testTypeResolver + return testSemaCtx +} + func parseTableDesc(createTableStmt string) (catalog.TableDescriptor, error) { ctx := context.Background() stmt, err := parser.ParseOne(createTableStmt) @@ -56,7 +67,7 @@ func parseTableDesc(createTableStmt string) (catalog.TableDescriptor, error) { st := cluster.MakeTestingClusterSettings() const parentID = descpb.ID(keys.MaxReservedDescID + 1) const tableID = descpb.ID(keys.MaxReservedDescID + 2) - semaCtx := tree.MakeSemaContext() + semaCtx := makeTestSemaCtx() mutDesc, err := importccl.MakeSimpleTableDescriptor( ctx, &semaCtx, st, createTable, parentID, keys.PublicSchemaID, tableID, importccl.NoFKs, hlc.UnixNano()) if err != nil { @@ -67,7 +78,7 @@ func parseTableDesc(createTableStmt string) (catalog.TableDescriptor, error) { func parseValues(tableDesc catalog.TableDescriptor, values string) ([]rowenc.EncDatumRow, error) { ctx := context.Background() - semaCtx := tree.MakeSemaContext() + semaCtx := makeTestSemaCtx() evalCtx := &tree.EvalContext{} valuesStmt, err := parser.ParseOne(values) @@ -136,7 +147,7 @@ func avroFieldMetadataToColDesc(metadata string) (*descpb.ColumnDescriptor, erro } def := parsed.AST.(*tree.AlterTable).Cmds[0].(*tree.AlterTableAddColumn).ColumnDef ctx := context.Background() - semaCtx := tree.MakeSemaContext() + semaCtx := makeTestSemaCtx() col, _, _, err := tabledesc.MakeColumnDefDescs(ctx, def, &semaCtx, &tree.EvalContext{}) return col, err } @@ -147,6 +158,37 @@ func randTime(rng *rand.Rand) time.Time { return timeutil.Unix(0, rng.Int63()) } +//Create a thin, in-memory user-defined enum type +func createEnum(enumLabels tree.EnumValueList, typeName tree.TypeName) *types.T { + + members := make([]descpb.TypeDescriptor_EnumMember, len(enumLabels)) + physReps := enum.GenerateNEvenlySpacedBytes(len(enumLabels)) + for i := range enumLabels { + members[i] = descpb.TypeDescriptor_EnumMember{ + LogicalRepresentation: string(enumLabels[i]), + PhysicalRepresentation: physReps[i], + Capability: descpb.TypeDescriptor_EnumMember_ALL, + } + } + + enumKind := descpb.TypeDescriptor_ENUM + + typeDesc := typedesc.NewBuilder(&descpb.TypeDescriptor{ + Name: typeName.Type(), + ID: 0, + Kind: enumKind, + EnumMembers: members, + Version: 1, + }).BuildCreatedMutableType() + + typ, _ := typeDesc.MakeTypesT(context.Background(), &typeName, nil) + + testTypes[typeName.SQLString()] = typ + + return typ + +} + func TestAvroSchema(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) @@ -278,6 +320,13 @@ func TestAvroSchema(t *testing.T) { } } + testEnum := createEnum( + tree.EnumValueList{tree.EnumValue(`open`), tree.EnumValue(`closed`)}, + tree.MakeUnqualifiedTypeName(`switch`), + ) + + typesToTest = append(typesToTest, testEnum) + // Generate a test for each column type with a random datum of that type. for _, typ := range typesToTest { var datum tree.Datum @@ -516,6 +565,9 @@ func TestAvroSchema(t *testing.T) { {sqlType: `VARCHAR COLLATE "fr"`, sql: `'Bonjour' COLLATE "fr"`, avro: `{"string":"Bonjour"}`}, + {sqlType: `switch`, // User-defined enum with values "open", "closed" + sql: `'open'`, + avro: `{"string":"open"}`}, } for _, test := range goldens { @@ -796,6 +848,14 @@ func BenchmarkEncodeBool(b *testing.B) { benchmarkEncodeType(b, types.Bool, randEncDatumRow(types.Bool)) } +func BenchmarkEncodeEnum(b *testing.B) { + testEnum := createEnum( + tree.EnumValueList{tree.EnumValue(`open`), tree.EnumValue(`closed`)}, + tree.MakeUnqualifiedTypeName(`switch`), + ) + benchmarkEncodeType(b, testEnum, randEncDatumRow(testEnum)) +} + func BenchmarkEncodeFloat(b *testing.B) { benchmarkEncodeType(b, types.Float, randEncDatumRow(types.Float)) } diff --git a/pkg/ccl/changefeedccl/changefeed_test.go b/pkg/ccl/changefeedccl/changefeed_test.go index f3d8f7531e55..f6525da09aad 100644 --- a/pkg/ccl/changefeedccl/changefeed_test.go +++ b/pkg/ccl/changefeedccl/changefeed_test.go @@ -600,6 +600,16 @@ func TestChangefeedUserDefinedTypes(t *testing.T) { `tt: [3]->{"after": {"x": 3, "y": "hiya", "z": "bye"}}`, `tt: [4]->{"after": {"x": 4, "y": "hello", "z": "cya"}}`, }) + + // If we rename a value in an existing type, it doesn't count as a change + // but the rename is reflected in future changes. + sqlDB.Exec(t, `ALTER TYPE t RENAME VALUE 'hi' TO 'yo'`) + sqlDB.Exec(t, `UPDATE tt SET z='cya' where x=2`) + + assertPayloads(t, cf, []string{ + `tt: [2]->{"after": {"x": 2, "y": "yo", "z": "cya"}}`, + }) + } t.Run(`sinkless`, sinklessTest(testFn)) diff --git a/pkg/ccl/changefeedccl/encoder.go b/pkg/ccl/changefeedccl/encoder.go index 72e53a672c68..444b933315f2 100644 --- a/pkg/ccl/changefeedccl/encoder.go +++ b/pkg/ccl/changefeedccl/encoder.go @@ -379,6 +379,10 @@ func (e *confluentAvroEncoder) EncodeKey(ctx context.Context, row encodeRow) ([] e.keyCache[cacheKey] = registered } + if ok { + registered.schema.refreshTypeMetadata(row.tableDesc) + } + // https://docs.confluent.io/current/schema-registry/docs/serializer-formatter.html#wire-format header := []byte{ changefeedbase.ConfluentAvroWireFormatMagic, @@ -432,6 +436,13 @@ func (e *confluentAvroEncoder) EncodeValue(ctx context.Context, row encodeRow) ( // TODO(dan): Bound the size of this cache. e.valueCache[cacheKey] = registered } + if ok { + registered.schema.after.refreshTypeMetadata(row.tableDesc) + if row.prevTableDesc != nil && registered.schema.before != nil { + registered.schema.before.refreshTypeMetadata(row.prevTableDesc) + } + } + var meta avroMetadata if registered.schema.opts.updatedField { meta = map[string]interface{}{ diff --git a/pkg/ccl/changefeedccl/encoder_test.go b/pkg/ccl/changefeedccl/encoder_test.go index b81c424e9a5f..81bbac655a23 100644 --- a/pkg/ccl/changefeedccl/encoder_test.go +++ b/pkg/ccl/changefeedccl/encoder_test.go @@ -529,6 +529,71 @@ func TestAvroCollatedString(t *testing.T) { t.Run(`enterprise`, enterpriseTest(testFn)) } +func TestAvroEnum(t *testing.T) { + defer leaktest.AfterTest(t)() + defer log.Scope(t).Close(t) + + testFn := func(t *testing.T, db *gosql.DB, f cdctest.TestFeedFactory) { + reg := cdctest.StartTestSchemaRegistry() + defer reg.Close() + + sqlDB := sqlutils.MakeSQLRunner(db) + sqlDB.Exec(t, `CREATE TYPE status AS ENUM ('open', 'closed', 'inactive')`) + sqlDB.Exec(t, `CREATE TABLE foo (a INT PRIMARY KEY, b status, c int default 0)`) + sqlDB.Exec(t, `INSERT INTO foo VALUES (1, 'open')`) + sqlDB.Exec(t, `INSERT INTO foo VALUES (2, null)`) + + foo := feed(t, f, `CREATE CHANGEFEED FOR foo `+ + `WITH format=$1, confluent_schema_registry=$2`, + changefeedbase.OptFormatAvro, reg.URL()) + defer closeFeed(t, foo) + assertPayloadsAvro(t, reg, foo, []string{ + `foo: {"a":{"long":1}}->{"after":{"foo":{"a":{"long":1},"b":{"string":"open"},"c":{"long":0}}}}`, + `foo: {"a":{"long":2}}->{"after":{"foo":{"a":{"long":2},"b":null,"c":{"long":0}}}}`, + }) + + sqlDB.Exec(t, `ALTER TYPE status ADD value 'review'`) + sqlDB.Exec(t, `INSERT INTO foo values (4, 'review')`) + + assertPayloadsAvro(t, reg, foo, []string{ + `foo: {"a":{"long":4}}->{"after":{"foo":{"a":{"long":4},"b":{"string":"review"},"c":{"long":0}}}}`, + }) + + // Renaming an enum type doesn't count as a change itself but gets picked up by the encoder + sqlDB.Exec(t, `ALTER TYPE status RENAME value 'open' to 'active'`) + sqlDB.Exec(t, `INSERT INTO foo values (3, 'active')`) + sqlDB.Exec(t, `UPDATE foo set c=1 where a=1`) + + assertPayloadsAvro(t, reg, foo, []string{ + `foo: {"a":{"long":3}}->{"after":{"foo":{"a":{"long":3},"b":{"string":"active"},"c":{"long":0}}}}`, + `foo: {"a":{"long":1}}->{"after":{"foo":{"a":{"long":1},"b":{"string":"active"},"c":{"long":1}}}}`, + }) + + // Enum can be part of a compound primary key + sqlDB.Exec(t, `CREATE TABLE soft_deletes (a INT, b status, c INT default 0, PRIMARY KEY (a,b))`) + sqlDB.Exec(t, `INSERT INTO soft_deletes values (0, 'active')`) + + sd := feed(t, f, `CREATE CHANGEFEED FOR soft_deletes `+ + `WITH format=$1, confluent_schema_registry=$2`, + changefeedbase.OptFormatAvro, reg.URL()) + defer closeFeed(t, sd) + assertPayloadsAvro(t, reg, sd, []string{ + `soft_deletes: {"a":{"long":0},"b":{"string":"active"}}->{"after":{"soft_deletes":{"a":{"long":0},"b":{"string":"active"},"c":{"long":0}}}}`, + }) + + sqlDB.Exec(t, `ALTER TYPE status RENAME value 'active' to 'open'`) + sqlDB.Exec(t, `UPDATE soft_deletes set c=1 where a=0`) + + assertPayloadsAvro(t, reg, sd, []string{ + `soft_deletes: {"a":{"long":0},"b":{"string":"open"}}->{"after":{"soft_deletes":{"a":{"long":0},"b":{"string":"open"},"c":{"long":1}}}}`, + }) + + } + + t.Run(`sinkless`, sinklessTest(testFn)) + t.Run(`enterprise`, enterpriseTest(testFn)) +} + func TestAvroSchemaNaming(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) diff --git a/pkg/ccl/serverccl/server_sql_test.go b/pkg/ccl/serverccl/server_sql_test.go index f530166581ed..f4db08fb21b0 100644 --- a/pkg/ccl/serverccl/server_sql_test.go +++ b/pkg/ccl/serverccl/server_sql_test.go @@ -205,3 +205,21 @@ func TestIdleExit(t *testing.T) { t.Error("stop on idle didn't trigger") } } + +func TestNonExistentTenant(t *testing.T) { + defer leaktest.AfterTest(t)() + defer log.Scope(t).Close(t) + ctx := context.Background() + + tc := serverutils.StartNewTestCluster(t, 1, base.TestClusterArgs{}) + defer tc.Stopper().Stop(ctx) + + _, err := tc.Server(0).StartTenant(ctx, + base.TestTenantArgs{ + TenantID: serverutils.TestTenantID(), + Existing: true, + SkipTenantCheck: true, + }) + require.Error(t, err) + require.Equal(t, "system DB uninitialized, check if tenant is non existent", err.Error()) +} diff --git a/pkg/server/BUILD.bazel b/pkg/server/BUILD.bazel index c7f0d032c4fa..fb3882bf4e40 100644 --- a/pkg/server/BUILD.bazel +++ b/pkg/server/BUILD.bazel @@ -108,6 +108,7 @@ go_library( "//pkg/settings/cluster", "//pkg/sql", "//pkg/sql/catalog/bootstrap", + "//pkg/sql/catalog/catalogkeys", "//pkg/sql/catalog/catconstants", "//pkg/sql/catalog/colinfo", "//pkg/sql/catalog/descpb", diff --git a/pkg/server/server_sql.go b/pkg/server/server_sql.go index ffdeb456937f..b98791b4f2a2 100644 --- a/pkg/server/server_sql.go +++ b/pkg/server/server_sql.go @@ -47,8 +47,10 @@ import ( "github.com/cockroachdb/cockroach/pkg/server/settingswatcher" "github.com/cockroachdb/cockroach/pkg/server/status" "github.com/cockroachdb/cockroach/pkg/sql" + "github.com/cockroachdb/cockroach/pkg/sql/catalog/catalogkeys" "github.com/cockroachdb/cockroach/pkg/sql/catalog/hydratedtables" "github.com/cockroachdb/cockroach/pkg/sql/catalog/lease" + "github.com/cockroachdb/cockroach/pkg/sql/catalog/systemschema" "github.com/cockroachdb/cockroach/pkg/sql/colexec" "github.com/cockroachdb/cockroach/pkg/sql/contention" "github.com/cockroachdb/cockroach/pkg/sql/distsql" @@ -252,7 +254,6 @@ func newSQLServer(ctx context.Context, cfg sqlServerArgs) (*SQLServer, error) { codec = keys.MakeSQLCodec(override) } } - // Create blob service for inter-node file sharing. blobService, err := blobs.NewBlobService(cfg.Settings.ExternalIODir) if err != nil { @@ -746,6 +747,28 @@ func newSQLServer(ctx context.Context, cfg sqlServerArgs) (*SQLServer, error) { }, nil } +// Checks if tenant exists. This function does a very superficial check to see if the system db +// has been bootstrapped for the tenant. This is not a complete check and is only sufficient +// to be used in the dev environment. +func maybeCheckTenantExists(ctx context.Context, codec keys.SQLCodec, db *kv.DB) error { + if codec.ForSystemTenant() { + // Skip check for system tenant and return early. + return nil + } + key := catalogkeys.NewDatabaseKey(systemschema.SystemDatabaseName).Key(codec) + result, err := db.Get(ctx, key) + if err != nil { + return err + } + if result.Value == nil || result.ValueInt() != keys.SystemDatabaseID { + return errors.New("system DB uninitialized, check if tenant is non existent") + } + // Tenant has been confirmed to be bootstrapped successfully + // as the system database, which is a part of the bootstrap data for + // a tenant keyspace, exists in the namespace table. + return nil +} + func (s *SQLServer) preStart( ctx context.Context, stopper *stop.Stopper, @@ -764,6 +787,13 @@ func (s *SQLServer) preStart( return err } } + // Confirm tenant exists prior to initialization. This is a sanity + // check for the dev environment to ensure that a tenant has been + // successfully created before attempting to initialize a SQL + // server for it. + if err := maybeCheckTenantExists(ctx, s.execCfg.Codec, s.execCfg.DB); err != nil { + return err + } s.connManager = connManager s.pgL = pgL s.execCfg.GCJobNotifier.Start(ctx) diff --git a/pkg/server/testserver.go b/pkg/server/testserver.go index 5cdee513aa01..db5d4d54838b 100644 --- a/pkg/server/testserver.go +++ b/pkg/server/testserver.go @@ -716,18 +716,20 @@ func (ts *TestServer) StartTenant( } } - rowCount, err := ts.InternalExecutor().(*sql.InternalExecutor).Exec( - ctx, "testserver-check-tenant-active", nil, - "SELECT 1 FROM system.tenants WHERE id=$1 AND active=true", - params.TenantID.ToUint64(), - ) - if err != nil { - return nil, err - } - if rowCount == 0 { - return nil, errors.New("not found") - } + if !params.SkipTenantCheck { + rowCount, err := ts.InternalExecutor().(*sql.InternalExecutor).Exec( + ctx, "testserver-check-tenant-active", nil, + "SELECT 1 FROM system.tenants WHERE id=$1 AND active=true", + params.TenantID.ToUint64(), + ) + if err != nil { + return nil, err + } + if rowCount == 0 { + return nil, errors.New("not found") + } + } st := params.Settings if st == nil { st = cluster.MakeTestingClusterSettings() diff --git a/pkg/sql/BUILD.bazel b/pkg/sql/BUILD.bazel index 0e973daa51d3..9987921d8236 100644 --- a/pkg/sql/BUILD.bazel +++ b/pkg/sql/BUILD.bazel @@ -402,7 +402,7 @@ go_library( go_test( name = "sql_test", - size = "large", + size = "enormous", srcs = [ "admin_audit_log_test.go", "alter_column_type_test.go", diff --git a/pkg/sql/catalog/descpb/privilege.go b/pkg/sql/catalog/descpb/privilege.go index a1ab5dd73e9e..50b97eeba5d9 100644 --- a/pkg/sql/catalog/descpb/privilege.go +++ b/pkg/sql/catalog/descpb/privilege.go @@ -250,6 +250,25 @@ func MaybeFixUsagePrivForTablesAndDBs(ptr **PrivilegeDescriptor) bool { return modified } +// MaybeFixSchemaPrivileges removes all invalid bits set on a schema's +// PrivilegeDescriptor. +// This is necessary due to ALTER DATABASE ... CONVERT TO SCHEMA originally +// copying all database privileges to the schema. Not all database privileges +// are valid for schemas thus after running ALTER DATABASE ... CONVERT TO SCHEMA, +// the schema may become unusable. +func MaybeFixSchemaPrivileges(ptr **PrivilegeDescriptor) { + if *ptr == nil { + *ptr = &PrivilegeDescriptor{} + } + p := *ptr + + validPrivs := privilege.GetValidPrivilegesForObject(privilege.Schema).ToBitField() + + for i := range p.Users { + p.Users[i].Privileges &= validPrivs + } +} + // MaybeFixPrivileges fixes the privilege descriptor if needed, including: // * adding default privileges for the "admin" role // * fixing default privileges for the "root" user diff --git a/pkg/sql/catalog/descpb/privilege_test.go b/pkg/sql/catalog/descpb/privilege_test.go index c69f4123aace..2cfe477ed02a 100644 --- a/pkg/sql/catalog/descpb/privilege_test.go +++ b/pkg/sql/catalog/descpb/privilege_test.go @@ -889,3 +889,107 @@ func TestMaybeFixUsageAndZoneConfigPrivilege(t *testing.T) { } } + +// TestMaybeFixSchemaPrivileges ensures that invalid privileges are removed +// from a schema's privilege descriptor. +func TestMaybeFixSchemaPrivileges(t *testing.T) { + fooUser := security.MakeSQLUsernameFromPreNormalizedString("foo") + barUser := security.MakeSQLUsernameFromPreNormalizedString("bar") + + type userPrivileges map[security.SQLUsername]privilege.List + + testCases := []struct { + input userPrivileges + output userPrivileges + }{ + { + userPrivileges{ + fooUser: privilege.List{ + privilege.ALL, + privilege.CONNECT, + privilege.CREATE, + privilege.DROP, + privilege.GRANT, + privilege.SELECT, + privilege.INSERT, + privilege.DELETE, + privilege.UPDATE, + privilege.USAGE, + privilege.ZONECONFIG, + }, + barUser: privilege.List{ + privilege.CONNECT, + privilege.CREATE, + privilege.DROP, + privilege.GRANT, + privilege.SELECT, + privilege.INSERT, + privilege.DELETE, + privilege.UPDATE, + privilege.USAGE, + privilege.ZONECONFIG, + }, + }, + userPrivileges{ + fooUser: privilege.List{privilege.ALL}, + barUser: privilege.List{ + privilege.GRANT, + privilege.CREATE, + privilege.USAGE, + }, + }, + }, + { + userPrivileges{ + fooUser: privilege.List{privilege.GRANT}, + }, + userPrivileges{ + fooUser: privilege.List{privilege.GRANT}, + }, + }, + { + userPrivileges{ + fooUser: privilege.List{privilege.CREATE}, + }, + userPrivileges{ + fooUser: privilege.List{privilege.CREATE}, + }, + }, + { + userPrivileges{ + fooUser: privilege.List{privilege.USAGE}, + }, + userPrivileges{ + fooUser: privilege.List{privilege.USAGE}, + }, + }, + } + + for num, tc := range testCases { + desc := &PrivilegeDescriptor{} + for u, p := range tc.input { + desc.Grant(u, p) + } + MaybeFixSchemaPrivileges(&desc) + + for u, p := range tc.output { + outputUser, ok := desc.findUser(u) + if !ok { + t.Errorf("#%d: expected user %s in output, but not found (%v)", + num, u, desc.Users, + ) + } + if a, e := privilege.ListFromBitField(outputUser.Privileges, privilege.Any), p; a.ToBitField() != e.ToBitField() { + t.Errorf("#%d: user %s: expected privileges %v, got %v", + num, u, e, a, + ) + } + + err := privilege.ValidatePrivileges(p, privilege.Schema) + if err != nil { + t.Errorf("%s\n", err.Error()) + } + } + + } +} diff --git a/pkg/sql/catalog/descs/BUILD.bazel b/pkg/sql/catalog/descs/BUILD.bazel index 965d3267c406..c5f4c49eacac 100644 --- a/pkg/sql/catalog/descs/BUILD.bazel +++ b/pkg/sql/catalog/descs/BUILD.bazel @@ -66,8 +66,10 @@ go_test( "//pkg/sql/catalog/tabledesc", "//pkg/sql/pgwire/pgcode", "//pkg/sql/pgwire/pgerror", + "//pkg/sql/privilege", "//pkg/sql/sem/tree", "//pkg/sql/sqlutil", + "//pkg/sql/tests", "//pkg/sql/types", "//pkg/testutils/serverutils", "//pkg/testutils/sqlutils", diff --git a/pkg/sql/catalog/descs/collection_test.go b/pkg/sql/catalog/descs/collection_test.go index a912600a79cc..63eff1c46456 100644 --- a/pkg/sql/catalog/descs/collection_test.go +++ b/pkg/sql/catalog/descs/collection_test.go @@ -28,9 +28,12 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/catalog/tabledesc" "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/tree" "github.com/cockroachdb/cockroach/pkg/sql/sqlutil" + "github.com/cockroachdb/cockroach/pkg/sql/tests" "github.com/cockroachdb/cockroach/pkg/sql/types" + "github.com/cockroachdb/cockroach/pkg/testutils/serverutils" "github.com/cockroachdb/cockroach/pkg/testutils/sqlutils" "github.com/cockroachdb/cockroach/pkg/testutils/testcluster" "github.com/cockroachdb/cockroach/pkg/util/leaktest" @@ -451,3 +454,68 @@ func TestDistSQLTypeResolver_GetTypeDescriptor_WrongType(t *testing.T) { require.Regexp(t, `descriptor \d+ is a relation not a type`, err) require.Equal(t, pgcode.WrongObjectType, pgerror.GetPGCode(err)) } + +// TestMaybeFixSchemaPrivilegesIntegration ensures that schemas that have +// invalid privileges have their privilege descriptors fixed on read-time when +// grabbing the descriptor. +func TestMaybeFixSchemaPrivilegesIntegration(t *testing.T) { + defer leaktest.AfterTest(t)() + defer log.Scope(t).Close(t) + + ctx := context.Background() + params, _ := tests.CreateTestServerParams() + s, db, kvDB := serverutils.StartServer(t, params) + defer s.Stopper().Stop(ctx) + + conn, err := db.Conn(ctx) + require.NoError(t, err) + + _, err = conn.ExecContext(ctx, ` +CREATE DATABASE test; +CREATE SCHEMA test.schema; +CREATE USER testuser; +GRANT CREATE ON SCHEMA test.schema TO testuser; +CREATE TABLE test.schema.t(x INT); +`) + require.NoError(t, err) + + require.NoError( + t, + descs.Txn( + ctx, + s.ClusterSettings(), + s.LeaseManager().(*lease.Manager), + s.InternalExecutor().(sqlutil.InternalExecutor), + kvDB, + func(ctx context.Context, txn *kv.Txn, descsCol *descs.Collection) error { + _, dbDesc, err := descsCol.GetImmutableDatabaseByName(ctx, txn, "test", tree.DatabaseLookupFlags{Required: true}) + if err != nil { + return err + } + _, schemaDesc, err := descsCol.GetMutableSchemaByName(ctx, txn, dbDesc.GetID(), "schema", tree.SchemaLookupFlags{Required: true}) + if err != nil { + return err + } + // Write garbage privileges into the schema desc. + privs := schemaDesc.Desc.GetPrivileges() + for i := range privs.Users { + // SELECT is valid on a database but not a schema, however + // due to issue #65697, after running ALTER DATABASE ... + // CONVERT TO SCHEMA, schemas could end up with + // SELECT on it's privilege descriptor. This test + // mimics a schema that was originally a database. + // We want to ensure the schema's privileges are fixed + // on read. + privs.Users[i].Privileges |= privilege.SELECT.Mask() + } + + descsCol.SkipValidationOnWrite() + return descsCol.WriteDesc(ctx, false, schemaDesc.Desc.(catalog.MutableDescriptor), txn) + }), + ) + + // Make sure using the schema is fine and we don't encounter a + // privilege validation error. + _, err = db.Query("GRANT USAGE ON SCHEMA test.schema TO testuser;") + require.NoError(t, err) +} diff --git a/pkg/sql/catalog/schemadesc/schema_desc_builder.go b/pkg/sql/catalog/schemadesc/schema_desc_builder.go index 0c8540e95dbe..f6cbd7067001 100644 --- a/pkg/sql/catalog/schemadesc/schema_desc_builder.go +++ b/pkg/sql/catalog/schemadesc/schema_desc_builder.go @@ -51,6 +51,8 @@ func (sdb *schemaDescriptorBuilder) DescriptorType() catalog.DescriptorType { func (sdb *schemaDescriptorBuilder) RunPostDeserializationChanges( _ context.Context, _ catalog.DescGetter, ) error { + privDesc := sdb.original.GetPrivileges() + descpb.MaybeFixSchemaPrivileges(&privDesc) return nil } diff --git a/pkg/sql/doctor/doctor_test.go b/pkg/sql/doctor/doctor_test.go index d232a8e617dc..8ed156ff2734 100644 --- a/pkg/sql/doctor/doctor_test.go +++ b/pkg/sql/doctor/doctor_test.go @@ -97,10 +97,14 @@ func TestExamineDescriptors(t *testing.T) { tbl.State = descpb.DescriptorState_DROP } - inSchemaValidTableDesc := protoutil.Clone(validTableDesc).(*descpb.Descriptor) + // Use 51 as the Schema ID, we do not want to use a reserved system ID (1-49) + // for the Schema because there should be no schemas with 1-49. A schema with + // an ID from 1-49 would fail privilege checks due to incompatible privileges + // the privileges returned from the SystemAllowedPrivileges map in privilege.go. + validTableDescWithParentSchema := protoutil.Clone(validTableDesc).(*descpb.Descriptor) { - tbl, _, _, _ := descpb.FromDescriptorWithMVCCTimestamp(inSchemaValidTableDesc, hlc.Timestamp{WallTime: 1}) - tbl.UnexposedParentSchemaID = 3 + tbl, _, _, _ := descpb.FromDescriptorWithMVCCTimestamp(validTableDescWithParentSchema, hlc.Timestamp{WallTime: 1}) + tbl.UnexposedParentSchemaID = 51 } tests := []struct { @@ -199,17 +203,17 @@ func TestExamineDescriptors(t *testing.T) { { // 8 descTable: doctor.DescriptorTable{ { - ID: 1, + ID: 51, DescBytes: toBytes(t, &descpb.Descriptor{Union: &descpb.Descriptor_Schema{ - Schema: &descpb.SchemaDescriptor{Name: "schema", ID: 1, ParentID: 2}, + Schema: &descpb.SchemaDescriptor{Name: "schema", ID: 51, ParentID: 2}, }}), }, }, namespaceTable: doctor.NamespaceTable{ - {NameInfo: descpb.NameInfo{ParentID: 2, Name: "schema"}, ID: 1}, + {NameInfo: descpb.NameInfo{ParentID: 2, Name: "schema"}, ID: 51}, }, expected: `Examining 1 descriptors and 1 namespace entries... - ParentID 2, ParentSchemaID 0: schema "schema" (1): referenced database ID 2: descriptor not found + ParentID 2, ParentSchemaID 0: schema "schema" (51): referenced database ID 2: descriptor not found `, }, { // 9 @@ -295,7 +299,7 @@ func TestExamineDescriptors(t *testing.T) { }, { // 12 descTable: doctor.DescriptorTable{ - {ID: 1, DescBytes: toBytes(t, inSchemaValidTableDesc)}, + {ID: 1, DescBytes: toBytes(t, validTableDescWithParentSchema)}, { ID: 2, DescBytes: toBytes(t, &descpb.Descriptor{Union: &descpb.Descriptor_Database{ @@ -303,9 +307,9 @@ func TestExamineDescriptors(t *testing.T) { }}), }, { - ID: 3, + ID: 51, DescBytes: toBytes(t, &descpb.Descriptor{Union: &descpb.Descriptor_Schema{ - Schema: &descpb.SchemaDescriptor{Name: "schema", ID: 3, ParentID: 4}, + Schema: &descpb.SchemaDescriptor{Name: "schema", ID: 51, ParentID: 4}, }}), }, { @@ -316,14 +320,14 @@ func TestExamineDescriptors(t *testing.T) { }, }, namespaceTable: doctor.NamespaceTable{ - {NameInfo: descpb.NameInfo{ParentID: 2, ParentSchemaID: 3, Name: "t"}, ID: 1}, + {NameInfo: descpb.NameInfo{ParentID: 2, ParentSchemaID: 51, Name: "t"}, ID: 1}, {NameInfo: descpb.NameInfo{Name: "db"}, ID: 2}, - {NameInfo: descpb.NameInfo{ParentID: 4, Name: "schema"}, ID: 3}, + {NameInfo: descpb.NameInfo{ParentID: 4, Name: "schema"}, ID: 51}, {NameInfo: descpb.NameInfo{Name: "db2"}, ID: 4}, }, expected: `Examining 4 descriptors and 4 namespace entries... - ParentID 2, ParentSchemaID 3: relation "t" (1): parent schema 3 is in different database 4 - ParentID 4, ParentSchemaID 0: schema "schema" (3): not present in parent database [4] schemas mapping + ParentID 2, ParentSchemaID 51: relation "t" (1): parent schema 51 is in different database 4 + ParentID 4, ParentSchemaID 0: schema "schema" (51): not present in parent database [4] schemas mapping `, }, { // 13 diff --git a/pkg/sql/opt/idxconstraint/testdata/single-column b/pkg/sql/opt/idxconstraint/testdata/single-column index 7e95892e12e4..8c21da04af37 100644 --- a/pkg/sql/opt/idxconstraint/testdata/single-column +++ b/pkg/sql/opt/idxconstraint/testdata/single-column @@ -153,12 +153,12 @@ NOT a index-constraints vars=(a bool) index=(a) a != true ---- -(/NULL - /false] +[/false - /false] index-constraints vars=(a bool) index=(a) a != false ---- -[/true - ] +[/true - /true] index-constraints vars=(a bool) index=(a) a IS TRUE @@ -195,12 +195,12 @@ a IS DISTINCT FROM 5 index-constraints vars=(a bool) index=(a desc) a != true ---- -[/false - /NULL) +[/false - /false] index-constraints vars=(a bool) index=(a desc) a != false ---- -[ - /true] +[/true - /true] index-constraints vars=(a bool) index=(a desc) a IS TRUE diff --git a/pkg/sql/opt/memo/testdata/logprops/constraints b/pkg/sql/opt/memo/testdata/logprops/constraints index 6b9219830f80..6354cd090444 100644 --- a/pkg/sql/opt/memo/testdata/logprops/constraints +++ b/pkg/sql/opt/memo/testdata/logprops/constraints @@ -545,28 +545,27 @@ SELECT * FROM abc WHERE b != true ---- select ├── columns: a:1(int) b:2(bool!null) c:3(string) + ├── fd: ()-->(2) ├── prune: (1,3) ├── scan abc │ ├── columns: a:1(int) b:2(bool) c:3(string) │ └── prune: (1-3) └── filters - └── ne [type=bool, outer=(2), constraints=(/2: (/NULL - /false]; tight)] - ├── variable: b:2 [type=bool] - └── true [type=bool] + └── not [type=bool, outer=(2), constraints=(/2: [/false - /false]; tight), fd=()-->(2)] + └── variable: b:2 [type=bool] opt SELECT * FROM abc WHERE b != false ---- select ├── columns: a:1(int) b:2(bool!null) c:3(string) + ├── fd: ()-->(2) ├── prune: (1,3) ├── scan abc │ ├── columns: a:1(int) b:2(bool) c:3(string) │ └── prune: (1-3) └── filters - └── ne [type=bool, outer=(2), constraints=(/2: [/true - ]; tight)] - ├── variable: b:2 [type=bool] - └── false [type=bool] + └── variable: b:2 [type=bool, outer=(2), constraints=(/2: [/true - /true]; tight), fd=()-->(2)] opt SELECT * FROM abc WHERE b IS NOT true diff --git a/pkg/sql/opt/memo/testdata/stats/scan b/pkg/sql/opt/memo/testdata/stats/scan index 509ccab962bb..43266dc33a9f 100644 --- a/pkg/sql/opt/memo/testdata/stats/scan +++ b/pkg/sql/opt/memo/testdata/stats/scan @@ -1128,7 +1128,7 @@ select │ └── fd: ()-->(3), (7)-->(5) └── filters ├── a:1 = '37685f26-4b07-40ba-9bbf-42916ed9bc61' [type=bool, outer=(1), constraints=(/1: [/'37685f26-4b07-40ba-9bbf-42916ed9bc61' - /'37685f26-4b07-40ba-9bbf-42916ed9bc61']; tight), fd=()-->(1)] - ├── b:2 = true [type=bool, outer=(2), constraints=(/2: [/true - /true]; tight), fd=()-->(2)] + ├── b:2 [type=bool, outer=(2), constraints=(/2: [/true - /true]; tight), fd=()-->(2)] ├── d:4 = 'foo' [type=bool, outer=(4), constraints=(/4: [/'foo' - /'foo']; tight), fd=()-->(4)] └── f:6 > 0.0 [type=bool, outer=(6), constraints=(/6: [/5e-324 - ]; tight)] @@ -1194,7 +1194,7 @@ select │ └── f:6 > 0.0 [type=bool, outer=(6), constraints=(/6: [/5e-324 - ]; tight)] └── filters ├── a:1 = '37685f26-4b07-40ba-9bbf-42916ed9bc61' [type=bool, outer=(1), constraints=(/1: [/'37685f26-4b07-40ba-9bbf-42916ed9bc61' - /'37685f26-4b07-40ba-9bbf-42916ed9bc61']; tight), fd=()-->(1)] - ├── b:2 = true [type=bool, outer=(2), constraints=(/2: [/true - /true]; tight), fd=()-->(2)] + ├── b:2 [type=bool, outer=(2), constraints=(/2: [/true - /true]; tight), fd=()-->(2)] └── c:3 = 5 [type=bool, outer=(3), constraints=(/3: [/5 - /5]; tight), fd=()-->(3)] # A different combination of predicates. @@ -1225,7 +1225,7 @@ select │ ├── key: (7) │ └── fd: ()-->(3), (7)-->(5) └── filters - ├── b:2 = true [type=bool, outer=(2), constraints=(/2: [/true - /true]; tight), fd=()-->(2)] + ├── b:2 [type=bool, outer=(2), constraints=(/2: [/true - /true]; tight), fd=()-->(2)] └── f:6 > 0.0 [type=bool, outer=(6), constraints=(/6: [/5e-324 - ]; tight)] # Force the alternate index. @@ -1514,7 +1514,7 @@ select │ ├── key: (7) │ └── fd: ()-->(3), (7)-->(5) └── filters - ├── b:2 = true [type=bool, outer=(2), constraints=(/2: [/true - /true]; tight), fd=()-->(2)] + ├── b:2 [type=bool, outer=(2), constraints=(/2: [/true - /true]; tight), fd=()-->(2)] └── f:6 > 0.0 [type=bool, outer=(6), constraints=(/6: [/5e-324 - ]; tight)] # Force the alternate index. @@ -1855,7 +1855,7 @@ select │ ├── key: (7) │ └── fd: ()-->(3), (7)-->(5) └── filters - ├── b:2 = true [type=bool, outer=(2), constraints=(/2: [/true - /true]; tight), fd=()-->(2)] + ├── b:2 [type=bool, outer=(2), constraints=(/2: [/true - /true]; tight), fd=()-->(2)] └── f:6 > 0.0 [type=bool, outer=(6), constraints=(/6: [/5e-324 - ]; tight)] # Force the alternate index. @@ -1959,7 +1959,7 @@ select │ └── fd: ()-->(4-6) └── filters ├── a:1 = '37685f26-4b07-40ba-9bbf-42916ed9bc61' [type=bool, outer=(1), constraints=(/1: [/'37685f26-4b07-40ba-9bbf-42916ed9bc61' - /'37685f26-4b07-40ba-9bbf-42916ed9bc61']; tight), fd=()-->(1)] - └── b:2 = true [type=bool, outer=(2), constraints=(/2: [/true - /true]; tight), fd=()-->(2)] + └── b:2 [type=bool, outer=(2), constraints=(/2: [/true - /true]; tight), fd=()-->(2)] opt SELECT * FROM multi_col diff --git a/pkg/sql/opt/memo/testdata/stats/select b/pkg/sql/opt/memo/testdata/stats/select index 4b98c9a6a9ad..62e694fee5b8 100644 --- a/pkg/sql/opt/memo/testdata/stats/select +++ b/pkg/sql/opt/memo/testdata/stats/select @@ -1730,7 +1730,7 @@ select │ histogram(2)= 0 900 0 100 │ <--- false --- true └── filters - └── b:2 = false [type=bool, outer=(2), constraints=(/2: [/false - /false]; tight), fd=()-->(2)] + └── NOT b:2 [type=bool, outer=(2), constraints=(/2: [/false - /false]; tight), fd=()-->(2)] exec-ddl CREATE TABLE t0(c0 INT) diff --git a/pkg/sql/opt/norm/rules/comp.opt b/pkg/sql/opt/norm/rules/comp.opt index 053050fe58a2..7567658efa9f 100644 --- a/pkg/sql/opt/norm/rules/comp.opt +++ b/pkg/sql/opt/norm/rules/comp.opt @@ -295,3 +295,27 @@ ) => (MakeSTDFullyWithinRight (OpName) $args $left) + +# FoldEqTrue replaces x = True with x. +[FoldEqTrue, Normalize] +(Eq $left:* (True)) +=> +$left + +# FoldEqFalse replaces x = False with NOT x. +[FoldEqFalse, Normalize] +(Eq $left:* (False)) +=> +(Not $left) + +# FoldNeTrue replaces x != True with NOT x. +[FoldNeTrue, Normalize] +(Ne $left:* (True)) +=> +(Not $left) + +# FoldNeFalse replaces x != False with x. +[FoldNeFalse, Normalize] +(Ne $left:* (False)) +=> +$left diff --git a/pkg/sql/opt/norm/testdata/rules/comp b/pkg/sql/opt/norm/testdata/rules/comp index 46ec2ab4db0d..4db9715d2a3b 100644 --- a/pkg/sql/opt/norm/testdata/rules/comp +++ b/pkg/sql/opt/norm/testdata/rules/comp @@ -1114,3 +1114,154 @@ select │ └── columns: geom:1 geog:2 val:3 └── filters └── st_dfullywithinexclusive(geom:1, '010100000000000000000000000000000000000000', val:3) [outer=(1,3), immutable, constraints=(/1: (/NULL - ]; /3: (/NULL - ])] + +# -------------------------------------------------- +# FoldEqTrue + FoldEqFalse +# -------------------------------------------------- + +exec-ddl +CREATE TABLE tbl (k INT PRIMARY KEY, b BOOL) +---- + +norm expect=FoldEqTrue +SELECT * FROM tbl WHERE b=TRUE +---- +select + ├── columns: k:1!null b:2!null + ├── key: (1) + ├── fd: ()-->(2) + ├── scan tbl + │ ├── columns: k:1!null b:2 + │ ├── key: (1) + │ └── fd: (1)-->(2) + └── filters + └── b:2 [outer=(2), constraints=(/2: [/true - /true]; tight), fd=()-->(2)] + +norm expect=FoldEqTrue +SELECT b=TRUE FROM tbl +---- +project + ├── columns: "?column?":4 + ├── scan tbl + │ └── columns: b:2 + └── projections + └── b:2 [as="?column?":4, outer=(2)] + +norm expect=FoldEqFalse +SELECT * FROM tbl WHERE b=FALSE +---- +select + ├── columns: k:1!null b:2!null + ├── key: (1) + ├── fd: ()-->(2) + ├── scan tbl + │ ├── columns: k:1!null b:2 + │ ├── key: (1) + │ └── fd: (1)-->(2) + └── filters + └── NOT b:2 [outer=(2), constraints=(/2: [/false - /false]; tight), fd=()-->(2)] + +norm expect=FoldEqFalse +SELECT b=FALSE FROM tbl +---- +project + ├── columns: "?column?":4 + ├── scan tbl + │ └── columns: b:2 + └── projections + └── NOT b:2 [as="?column?":4, outer=(2)] + +exec-ddl +CREATE INVERTED INDEX ON geom_geog(geom) +---- + +# Regression test for #65684. +# We use opt here to show that the inverted index is used. +opt expect=FoldEqTrue +SELECT count(*) FROM geom_geog WHERE (geom && st_geomfromewkt('SRID=4326;POLYGON((0 0,0 100,100 100,100 0,0 0))'))=TRUE; +---- +scalar-group-by + ├── columns: count:7!null + ├── cardinality: [1 - 1] + ├── immutable + ├── key: () + ├── fd: ()-->(7) + ├── select + │ ├── columns: geom:1!null + │ ├── immutable + │ ├── index-join geom_geog + │ │ ├── columns: geom:1 + │ │ └── inverted-filter + │ │ ├── columns: rowid:4!null + │ │ ├── inverted expression: /6 + │ │ │ ├── tight: false, unique: false + │ │ │ └── union spans + │ │ │ ├── ["B\x89", "B\xfd \x00\x00\x00\x00\x00\x00\x00") + │ │ │ └── ["B\xfd\xff\xff\xff\xff\xff\xff\xff\xff", "B\xfd\xff\xff\xff\xff\xff\xff\xff\xff"] + │ │ ├── pre-filterer expression + │ │ │ └── st_intersects('0103000020E610000001000000050000000000000000000000000000000000000000000000000000000000000000005940000000000000594000000000000059400000000000005940000000000000000000000000000000000000000000000000', geom:1) + │ │ ├── key: (4) + │ │ └── scan geom_geog@secondary + │ │ ├── columns: rowid:4!null geom_inverted_key:6!null + │ │ ├── inverted constraint: /6/4 + │ │ │ └── spans + │ │ │ ├── ["B\x89", "B\xfd \x00\x00\x00\x00\x00\x00\x00") + │ │ │ └── ["B\xfd\xff\xff\xff\xff\xff\xff\xff\xff", "B\xfd\xff\xff\xff\xff\xff\xff\xff\xff"] + │ │ ├── key: (4) + │ │ └── fd: (4)-->(6) + │ └── filters + │ └── geom:1 && '0103000020E610000001000000050000000000000000000000000000000000000000000000000000000000000000005940000000000000594000000000000059400000000000005940000000000000000000000000000000000000000000000000' [outer=(1), immutable, constraints=(/1: (/NULL - ])] + └── aggregations + └── count-rows [as=count_rows:7] + +# -------------------------------------------------- +# FoldNeTrue + FoldNeFalse +# -------------------------------------------------- + +norm expect=FoldNeTrue +SELECT * FROM tbl WHERE b != TRUE +---- +select + ├── columns: k:1!null b:2!null + ├── key: (1) + ├── fd: ()-->(2) + ├── scan tbl + │ ├── columns: k:1!null b:2 + │ ├── key: (1) + │ └── fd: (1)-->(2) + └── filters + └── NOT b:2 [outer=(2), constraints=(/2: [/false - /false]; tight), fd=()-->(2)] + +norm expect=FoldNeTrue +SELECT b != TRUE FROM tbl +---- +project + ├── columns: "?column?":4 + ├── scan tbl + │ └── columns: b:2 + └── projections + └── NOT b:2 [as="?column?":4, outer=(2)] + +norm expect=FoldNeFalse +SELECT * FROM tbl WHERE b != FALSE +---- +select + ├── columns: k:1!null b:2!null + ├── key: (1) + ├── fd: ()-->(2) + ├── scan tbl + │ ├── columns: k:1!null b:2 + │ ├── key: (1) + │ └── fd: (1)-->(2) + └── filters + └── b:2 [outer=(2), constraints=(/2: [/true - /true]; tight), fd=()-->(2)] + +norm expect=FoldNeFalse +SELECT b != FALSE FROM tbl +---- +project + ├── columns: "?column?":4 + ├── scan tbl + │ └── columns: b:2 + └── projections + └── b:2 [as="?column?":4, outer=(2)] diff --git a/pkg/sql/opt/norm/testdata/rules/select b/pkg/sql/opt/norm/testdata/rules/select index 484589d67c6f..eb76646e7d0f 100644 --- a/pkg/sql/opt/norm/testdata/rules/select +++ b/pkg/sql/opt/norm/testdata/rules/select @@ -215,7 +215,7 @@ select ├── scan c │ └── columns: a:1 b:2 c:3 d:4 e:5 └── filters - ├── a:1 AND (a:1 = true) [outer=(1), constraints=(/1: [/true - /true]; tight), fd=()-->(1)] + ├── a:1 [outer=(1), constraints=(/1: [/true - /true]; tight), fd=()-->(1)] ├── b:2 [outer=(2), constraints=(/2: [/true - /true]; tight), fd=()-->(2)] └── b:2 = c:3 [outer=(2,3), constraints=(/2: (/NULL - ]; /3: (/NULL - ]), fd=(2)==(3), (3)==(2)] diff --git a/pkg/sql/opt/xform/testdata/external/liquibase b/pkg/sql/opt/xform/testdata/external/liquibase index 039fbebde068..9d4889faee76 100644 --- a/pkg/sql/opt/xform/testdata/external/liquibase +++ b/pkg/sql/opt/xform/testdata/external/liquibase @@ -217,7 +217,7 @@ project │ │ │ │ │ │ │ │ ├── key: (78) │ │ │ │ │ │ │ │ └── fd: (78)-->(79,85) │ │ │ │ │ │ │ └── filters - │ │ │ │ │ │ │ └── indisclustered:85 = true [outer=(85), constraints=(/85: [/true - /true]; tight), fd=()-->(85)] + │ │ │ │ │ │ │ └── indisclustered:85 [outer=(85), constraints=(/85: [/true - /true]; tight), fd=()-->(85)] │ │ │ │ │ │ ├── left-join (lookup pg_tablespace [as=t]) │ │ │ │ │ │ │ ├── columns: c.oid:1!null c.relname:2!null c.relnamespace:3!null c.relowner:5!null c.reltablespace:8!null c.reltuples:10!null c.relhasindex:13!null c.relpersistence:15!null c.relkind:17!null c.relhasoids:20!null c.relhasrules:22!null c.relhastriggers:23!null c.relacl:26 c.reloptions:27 n.oid:29!null n.nspname:30!null t.oid:34 spcname:35 ftrelid:126 ftserver:127 ftoptions:128 fs.oid:130 srvname:131 │ │ │ │ │ │ │ ├── key columns: [8] = [34] diff --git a/pkg/sql/opt/xform/testdata/external/navicat b/pkg/sql/opt/xform/testdata/external/navicat index c3a6bb8176a9..7d64ff5a2b08 100644 --- a/pkg/sql/opt/xform/testdata/external/navicat +++ b/pkg/sql/opt/xform/testdata/external/navicat @@ -221,7 +221,7 @@ sort │ │ │ │ │ │ │ │ ├── key: (78) │ │ │ │ │ │ │ │ └── fd: (78)-->(79,85) │ │ │ │ │ │ │ └── filters - │ │ │ │ │ │ │ └── indisclustered:85 = true [outer=(85), constraints=(/85: [/true - /true]; tight), fd=()-->(85)] + │ │ │ │ │ │ │ └── indisclustered:85 [outer=(85), constraints=(/85: [/true - /true]; tight), fd=()-->(85)] │ │ │ │ │ │ ├── left-join (lookup pg_tablespace [as=t]) │ │ │ │ │ │ │ ├── columns: c.oid:1!null c.relname:2!null c.relnamespace:3!null c.relowner:5!null c.reltablespace:8!null c.reltuples:10!null c.relhasindex:13!null c.relpersistence:15!null c.relkind:17!null c.relhasoids:20!null c.relhasrules:22!null c.relhastriggers:23!null c.relacl:26 c.reloptions:27 n.oid:29!null n.nspname:30!null t.oid:34 spcname:35 ftrelid:126 ftserver:127 ftoptions:128 fs.oid:130 srvname:131 │ │ │ │ │ │ │ ├── key columns: [8] = [34] diff --git a/pkg/sql/opt/xform/testdata/external/nova b/pkg/sql/opt/xform/testdata/external/nova index fa7823db761b..5ee61f86f756 100644 --- a/pkg/sql/opt/xform/testdata/external/nova +++ b/pkg/sql/opt/xform/testdata/external/nova @@ -260,7 +260,7 @@ project │ │ │ │ │ └── const-agg [as=flavors.id:1, outer=(1)] │ │ │ │ │ └── flavors.id:1 │ │ │ │ └── filters - │ │ │ │ └── (is_public:12 = true) OR (true_agg:25 IS NOT NULL) [outer=(12,25)] + │ │ │ │ └── is_public:12 OR (true_agg:25 IS NOT NULL) [outer=(12,25)] │ │ │ └── $3 │ │ └── $4 │ └── filters (true) @@ -412,7 +412,7 @@ sort │ │ │ │ │ │ │ │ │ │ ├── fd: (1)-->(2-12,14,15), (7)-->(1-6,8-12,14,15), (2)-->(1,3-12,14,15) │ │ │ │ │ │ │ │ │ │ └── ordering: +1 │ │ │ │ │ │ │ │ │ └── filters - │ │ │ │ │ │ │ │ │ └── disabled:11 = false [outer=(11), constraints=(/11: [/false - /false]; tight), fd=()-->(11)] + │ │ │ │ │ │ │ │ │ └── NOT disabled:11 [outer=(11), constraints=(/11: [/false - /false]; tight), fd=()-->(11)] │ │ │ │ │ │ │ │ ├── project │ │ │ │ │ │ │ │ │ ├── columns: true:31!null flavor_projects.flavor_id:18!null │ │ │ │ │ │ │ │ │ ├── has-placeholder @@ -464,7 +464,7 @@ sort │ │ │ │ │ │ │ └── const-agg [as=flavors.updated_at:15, outer=(15)] │ │ │ │ │ │ │ └── flavors.updated_at:15 │ │ │ │ │ │ └── filters - │ │ │ │ │ │ └── (is_public:12 = true) OR (true_agg:32 IS NOT NULL) [outer=(12,32)] + │ │ │ │ │ │ └── is_public:12 OR (true_agg:32 IS NOT NULL) [outer=(12,32)] │ │ │ │ │ ├── project │ │ │ │ │ │ ├── columns: true:34!null flavor_projects.flavor_id:25!null │ │ │ │ │ │ ├── has-placeholder @@ -516,7 +516,7 @@ sort │ │ │ │ └── const-agg [as=flavors.updated_at:15, outer=(15)] │ │ │ │ └── flavors.updated_at:15 │ │ │ └── filters - │ │ │ └── (is_public:12 = true) OR (true_agg:35 IS NOT NULL) [outer=(12,35)] + │ │ │ └── is_public:12 OR (true_agg:35 IS NOT NULL) [outer=(12,35)] │ │ └── $3 │ └── $4 └── filters @@ -708,7 +708,7 @@ sort │ │ │ │ └── const-agg [as=instance_types.updated_at:16, outer=(16)] │ │ │ │ └── instance_types.updated_at:16 │ │ │ └── filters - │ │ │ └── (is_public:12 = true) OR (true_agg:28 IS NOT NULL) [outer=(12,28)] + │ │ │ └── is_public:12 OR (true_agg:28 IS NOT NULL) [outer=(12,28)] │ │ └── $5 │ └── $6 ├── select @@ -869,7 +869,7 @@ sort │ │ └── const-agg [as=instance_types.updated_at:16, outer=(16)] │ │ └── instance_types.updated_at:16 │ └── filters - │ └── (is_public:12 = true) OR (true_agg:37 IS NOT NULL) [outer=(12,37)] + │ └── is_public:12 OR (true_agg:37 IS NOT NULL) [outer=(12,37)] ├── select │ ├── columns: instance_type_extra_specs_1.id:18!null key:19 value:20 instance_type_extra_specs_1.instance_type_id:21!null instance_type_extra_specs_1.deleted:22!null instance_type_extra_specs_1.deleted_at:23 instance_type_extra_specs_1.created_at:24 instance_type_extra_specs_1.updated_at:25 │ ├── has-placeholder @@ -1057,7 +1057,7 @@ project │ │ │ │ │ └── const-agg [as=instance_types.id:1, outer=(1)] │ │ │ │ │ └── instance_types.id:1 │ │ │ │ └── filters - │ │ │ │ └── (is_public:12 = true) OR (true_agg:28 IS NOT NULL) [outer=(12,28)] + │ │ │ │ └── is_public:12 OR (true_agg:28 IS NOT NULL) [outer=(12,28)] │ │ │ └── $5 │ │ └── $6 │ └── filters @@ -1231,7 +1231,7 @@ project │ │ │ │ │ └── const-agg [as=instance_types.id:1, outer=(1)] │ │ │ │ │ └── instance_types.id:1 │ │ │ │ └── filters - │ │ │ │ └── (is_public:12 = true) OR (true_agg:28 IS NOT NULL) [outer=(12,28)] + │ │ │ │ └── is_public:12 OR (true_agg:28 IS NOT NULL) [outer=(12,28)] │ │ │ └── $5 │ │ └── $6 │ └── filters @@ -1389,7 +1389,7 @@ project │ │ │ │ │ └── const-agg [as=flavors.id:1, outer=(1)] │ │ │ │ │ └── flavors.id:1 │ │ │ │ └── filters - │ │ │ │ └── (is_public:12 = true) OR (true_agg:25 IS NOT NULL) [outer=(12,25)] + │ │ │ │ └── is_public:12 OR (true_agg:25 IS NOT NULL) [outer=(12,25)] │ │ │ └── $3 │ │ └── $4 │ └── filters (true) @@ -1546,7 +1546,7 @@ project │ │ │ │ │ └── const-agg [as=flavors.id:1, outer=(1)] │ │ │ │ │ └── flavors.id:1 │ │ │ │ └── filters - │ │ │ │ └── (is_public:12 = true) OR (true_agg:25 IS NOT NULL) [outer=(12,25)] + │ │ │ │ └── is_public:12 OR (true_agg:25 IS NOT NULL) [outer=(12,25)] │ │ │ └── $3 │ │ └── $4 │ └── filters (true) @@ -1727,7 +1727,7 @@ sort │ │ │ │ └── const-agg [as=flavors.updated_at:15, outer=(15)] │ │ │ │ └── flavors.updated_at:15 │ │ │ └── filters - │ │ │ └── (is_public:12 = true) OR (true_agg:25 IS NOT NULL) [outer=(12,25)] + │ │ │ └── is_public:12 OR (true_agg:25 IS NOT NULL) [outer=(12,25)] │ │ └── $5 │ └── $6 └── filters @@ -1917,7 +1917,7 @@ sort │ │ │ │ └── const-agg [as=instance_types.updated_at:16, outer=(16)] │ │ │ │ └── instance_types.updated_at:16 │ │ │ └── filters - │ │ │ └── (is_public:12 = true) OR (true_agg:28 IS NOT NULL) [outer=(12,28)] + │ │ │ └── is_public:12 OR (true_agg:28 IS NOT NULL) [outer=(12,28)] │ │ └── $4 │ └── $5 ├── select @@ -2107,7 +2107,7 @@ project │ │ │ │ │ └── const-agg [as=instance_types.id:1, outer=(1)] │ │ │ │ │ └── instance_types.id:1 │ │ │ │ └── filters - │ │ │ │ └── (is_public:12 = true) OR (true_agg:28 IS NOT NULL) [outer=(12,28)] + │ │ │ │ └── is_public:12 OR (true_agg:28 IS NOT NULL) [outer=(12,28)] │ │ │ └── $5 │ │ └── $6 │ └── filters @@ -2238,7 +2238,7 @@ sort │ │ └── const-agg [as=flavors.updated_at:15, outer=(15)] │ │ └── flavors.updated_at:15 │ └── filters - │ └── (is_public:12 = true) OR (true_agg:32 IS NOT NULL) [outer=(12,32)] + │ └── is_public:12 OR (true_agg:32 IS NOT NULL) [outer=(12,32)] ├── scan flavor_extra_specs [as=flavor_extra_specs_1] │ ├── columns: flavor_extra_specs_1.id:17!null key:18!null value:19 flavor_extra_specs_1.flavor_id:20!null flavor_extra_specs_1.created_at:21 flavor_extra_specs_1.updated_at:22 │ ├── key: (17) @@ -2444,7 +2444,7 @@ sort │ │ │ │ └── const-agg [as=instance_types.updated_at:16, outer=(16)] │ │ │ │ └── instance_types.updated_at:16 │ │ │ └── filters - │ │ │ └── (is_public:12 = true) OR (true_agg:28 IS NOT NULL) [outer=(12,28)] + │ │ │ └── is_public:12 OR (true_agg:28 IS NOT NULL) [outer=(12,28)] │ │ └── $7 │ └── $8 └── filters @@ -2611,7 +2611,7 @@ sort │ │ │ │ └── const-agg [as=flavors.updated_at:15, outer=(15)] │ │ │ │ └── flavors.updated_at:15 │ │ │ └── filters - │ │ │ └── (is_public:12 = true) OR (true_agg:25 IS NOT NULL) [outer=(12,25)] + │ │ │ └── is_public:12 OR (true_agg:25 IS NOT NULL) [outer=(12,25)] │ │ └── $2 │ └── $3 ├── scan flavor_extra_specs [as=flavor_extra_specs_1] @@ -2788,7 +2788,7 @@ sort │ │ │ │ │ │ │ │ │ │ └── ordering: +1 │ │ │ │ │ │ │ │ │ └── filters │ │ │ │ │ │ │ │ │ ├── instance_types.deleted:13 = $1 [outer=(13), constraints=(/13: (/NULL - ]), fd=()-->(13)] - │ │ │ │ │ │ │ │ │ └── disabled:11 = false [outer=(11), constraints=(/11: [/false - /false]; tight), fd=()-->(11)] + │ │ │ │ │ │ │ │ │ └── NOT disabled:11 [outer=(11), constraints=(/11: [/false - /false]; tight), fd=()-->(11)] │ │ │ │ │ │ │ │ ├── project │ │ │ │ │ │ │ │ │ ├── columns: true:36!null instance_type_projects.instance_type_id:19!null │ │ │ │ │ │ │ │ │ ├── has-placeholder @@ -2845,7 +2845,7 @@ sort │ │ │ │ │ │ │ └── const-agg [as=instance_types.updated_at:16, outer=(16)] │ │ │ │ │ │ │ └── instance_types.updated_at:16 │ │ │ │ │ │ └── filters - │ │ │ │ │ │ └── (is_public:12 = true) OR (true_agg:37 IS NOT NULL) [outer=(12,37)] + │ │ │ │ │ │ └── is_public:12 OR (true_agg:37 IS NOT NULL) [outer=(12,37)] │ │ │ │ │ ├── project │ │ │ │ │ │ ├── columns: true:39!null instance_type_projects.instance_type_id:28!null │ │ │ │ │ │ ├── has-placeholder @@ -2903,7 +2903,7 @@ sort │ │ │ │ └── const-agg [as=instance_types.updated_at:16, outer=(16)] │ │ │ │ └── instance_types.updated_at:16 │ │ │ └── filters - │ │ │ └── (is_public:12 = true) OR (true_agg:40 IS NOT NULL) [outer=(12,40)] + │ │ │ └── is_public:12 OR (true_agg:40 IS NOT NULL) [outer=(12,40)] │ │ └── $7 │ └── $8 └── filters @@ -3085,7 +3085,7 @@ project │ │ │ │ │ └── const-agg [as=instance_types.id:1, outer=(1)] │ │ │ │ │ └── instance_types.id:1 │ │ │ │ └── filters - │ │ │ │ └── (is_public:12 = true) OR (true_agg:28 IS NOT NULL) [outer=(12,28)] + │ │ │ │ └── is_public:12 OR (true_agg:28 IS NOT NULL) [outer=(12,28)] │ │ │ └── $5 │ │ └── $6 │ └── filters @@ -3244,7 +3244,7 @@ project │ │ │ │ │ └── const-agg [as=flavors.id:1, outer=(1)] │ │ │ │ │ └── flavors.id:1 │ │ │ │ └── filters - │ │ │ │ └── (is_public:12 = true) OR (true_agg:25 IS NOT NULL) [outer=(12,25)] + │ │ │ │ └── is_public:12 OR (true_agg:25 IS NOT NULL) [outer=(12,25)] │ │ │ └── $3 │ │ └── $4 │ └── filters (true) @@ -3407,7 +3407,7 @@ sort │ │ │ └── const-agg [as=flavors.updated_at:15, outer=(15)] │ │ │ └── flavors.updated_at:15 │ │ └── filters - │ │ └── (is_public:12 = true) OR (true_agg:25 IS NOT NULL) [outer=(12,25)] + │ │ └── is_public:12 OR (true_agg:25 IS NOT NULL) [outer=(12,25)] │ └── $2 ├── scan flavor_extra_specs [as=flavor_extra_specs_1] │ ├── columns: flavor_extra_specs_1.id:27!null key:28!null value:29 flavor_extra_specs_1.flavor_id:30!null flavor_extra_specs_1.created_at:31 flavor_extra_specs_1.updated_at:32