Skip to content

Commit

Permalink
Browse files Browse the repository at this point in the history
96045: opt: check UDFs and UDTs by name when checking metadata dependencies r=DrewKimball a=DrewKimball

#### tree: distinguish UDFs and builtins that use a SQL string body

This refactor changes the meaning of the `Overload.IsUDF` field to
be true only for user-defined functions - meaning those created using
`CREATE FUNCTION`. This is contrasted with builtin functions that are
defined with a SQL string set in `Overload.Body`. Logic that cares only
about user-defined functions can continue checking `Overload.IsUDF`,
while cases that deal with the SQL body should use `Overload.HasSQLBody()`.
Note that it is insufficient to check whether `Overload.Body` is empty
because it is possible to define a UDF with an empty body.

#### opt: refactor metadata dependency tracking

This commit performs some refactoring for the way data source objects
are tracked in `opt.Metadata` in order to make future changes to UDT
and UDF dependency tracking easier. More particularly, UDTs and UDFs
will be able to re-resolve any references by name. This is necessary
in order to handle cases where changes to the search-path cause names
in the query to resolve to different objects.

#### opt: track UDT references by name in the Metadata

Previously, it was possible for invalid queries to be kept in the cache
after a schema change, since user-defined types were tracked only by OID.
This missed cases where the search path changed which object a name in
the query resolved to (or whether it resolved at all).

This patch fixes this behavior by tracking UDT references by name, similar
to what was already done for data sources. This ensures that the query
staleness check doesn't miss changes to the search path.

#### opt: track UDFs in the Metadata

This patch adds tracking for user-defined functions in `opt.Metadata`.
This ensures that the query cache will be correctly invalidated after
a schema change or search-path change that could change the query's
semantics, or cause it to error.

Fixes #95214
Fixes #93082
Fixes #93321
Fixes #96674

Release note (bug fix): Fixed a bug that could prevent a cached query from being invalidated when a UDF or UDT referenced by that query was altered or dropped, or when the schema or database of a UDF or UDT was altered or dropped.

98319: pkg/server: fix `/demologin` to properly redirect to home page r=dhartunian a=abarganier

With the introduction of the server controller, we introduced a layer between the HTTP handler and the HTTP server. When this was introduced, the logic to attempt a login to all tenants forgot to handle a specific case for `/demologin`, where the status code is set to a 302 redirect, instead of a 200 status OK.

This broke the redirect piece of the `/demologin` endpoint.

This patch updates the `attemptLoginToAllTenants` HTTP handler to properly set the 302 response code in the case where the underlying login function does so on the sessionWriter.

Release note: none

Epic: CRDB-12100

Fixes: #98253

98696: sql: disallow using cluster_logical_timestamp as column default when backfilling r=Xiang-Gu a=Xiang-Gu

Previously, `ADD COLUMN ... DEFAULT cluster_logical_timestamp()` would crash the node and leave the table in a corrupt state. The root cause is a nil pointer dereference. This commit fixed it by returning an unimplemented error and hence disallow using this builtin function as default value when backfilling.

Fixes: #98269
Release note (bug fix): fixed a bug as detailed in #98269.

Co-authored-by: Drew Kimball <drewk@cockroachlabs.com>
Co-authored-by: Alex Barganier <abarganier@cockroachlabs.com>
Co-authored-by: Xiang Gu <xiang@cockroachlabs.com>
  • Loading branch information
4 people committed Mar 20, 2023
4 parents 5839fe0 + 57dacce + 4a30189 + 796cf66 commit ea389ff
Show file tree
Hide file tree
Showing 25 changed files with 683 additions and 154 deletions.
5 changes: 2 additions & 3 deletions pkg/ccl/changefeedccl/cdceval/func_resolver.go
Original file line number Diff line number Diff line change
Expand Up @@ -73,10 +73,9 @@ func (rs *cdcFunctionResolver) ResolveFunction(
return nil, err
}

// Since we may be dealing with UDFs, ensure it is something
// that's supported.
// Ensure that any overloads defined using a SQL string are supported.
for _, overload := range funcDef.Overloads {
if overload.IsUDF {
if overload.HasSQLBody() {
if err := checkOverloadSupported(fnName, overload.Overload); err != nil {
return nil, err
}
Expand Down
2 changes: 1 addition & 1 deletion pkg/server/authentication.go
Original file line number Diff line number Diff line change
Expand Up @@ -240,7 +240,7 @@ func (s *authenticationServer) demoLogin(w http.ResponseWriter, req *http.Reques

w.Header()["Set-Cookie"] = []string{cookie.String()}
w.Header()["Location"] = []string{"/"}
w.WriteHeader(302)
w.WriteHeader(http.StatusTemporaryRedirect)
_, _ = w.Write([]byte("you can use the UI now"))
}

Expand Down
18 changes: 17 additions & 1 deletion pkg/server/server_controller_http.go
Original file line number Diff line number Diff line change
Expand Up @@ -153,6 +153,8 @@ func (c *serverController) attemptLoginToAllTenants() http.Handler {
}
defer r.Body.Close()

redirect := false
redirectLocation := "/" // default to home page
for _, name := range tenantNames {
server, err := c.getServer(ctx, name)
if err != nil {
Expand Down Expand Up @@ -185,6 +187,16 @@ func (c *serverController) attemptLoginToAllTenants() http.Handler {
name: string(name),
setCookie: setCookieHeader,
})
// In the case of /demologin, we want to redirect to the provided location
// in the header. If we get back a cookie along with an
// http.StatusTemporaryRedirect code, be sure to transfer the response code
// along with the Location into the ResponseWriter later.
if sw.code == http.StatusTemporaryRedirect {
redirect = true
if locationHeader, ok := sw.Header()["Location"]; ok && len(locationHeader) > 0 {
redirectLocation = locationHeader[0]
}
}
}
}
// If the map has entries, the method to create the aggregated session should
Expand Down Expand Up @@ -217,7 +229,11 @@ func (c *serverController) attemptLoginToAllTenants() http.Handler {
return
}
}
w.WriteHeader(http.StatusOK)
if redirect {
http.Redirect(w, r, redirectLocation, http.StatusTemporaryRedirect)
} else {
w.WriteHeader(http.StatusOK)
}
} else {
w.WriteHeader(http.StatusUnauthorized)
}
Expand Down
2 changes: 2 additions & 0 deletions pkg/sql/backfill/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -27,6 +27,8 @@ go_library(
"//pkg/sql/execinfra",
"//pkg/sql/execinfrapb",
"//pkg/sql/isql",
"//pkg/sql/pgwire/pgcode",
"//pkg/sql/pgwire/pgerror",
"//pkg/sql/row",
"//pkg/sql/rowenc",
"//pkg/sql/rowinfra",
Expand Down
12 changes: 12 additions & 0 deletions pkg/sql/backfill/backfill.go
Original file line number Diff line number Diff line change
Expand Up @@ -27,6 +27,8 @@ import (
"github.com/cockroachdb/cockroach/pkg/sql/catalog/typedesc"
"github.com/cockroachdb/cockroach/pkg/sql/execinfra"
"github.com/cockroachdb/cockroach/pkg/sql/isql"
"github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgcode"
"github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror"
"github.com/cockroachdb/cockroach/pkg/sql/row"
"github.com/cockroachdb/cockroach/pkg/sql/rowenc"
"github.com/cockroachdb/cockroach/pkg/sql/rowinfra"
Expand Down Expand Up @@ -372,6 +374,11 @@ func (cb *ColumnBackfiller) RunColumnBackfillChunk(
for j, e := range cb.updateExprs {
val, err := eval.Expr(ctx, cb.evalCtx, e)
if err != nil {
if errors.Is(err, eval.ErrNilTxnInClusterContext) {
// Cannot use expressions that depend on the transaction of the
// evaluation context as the default value for backfill.
return roachpb.Key{}, pgerror.WithCandidateCode(err, pgcode.FeatureNotSupported)
}
return roachpb.Key{}, sqlerrors.NewInvalidSchemaDefinitionError(err)
}
if j < len(cb.added) && !cb.added[j].IsNullable() && val == tree.DNull {
Expand Down Expand Up @@ -876,6 +883,11 @@ func (ib *IndexBackfiller) BuildIndexEntriesChunk(
}
val, err := eval.Expr(ctx, ib.evalCtx, texpr)
if err != nil {
if errors.Is(err, eval.ErrNilTxnInClusterContext) {
// Cannot use expressions that depend on the transaction of the
// evaluation context as the default value for backfill.
err = pgerror.WithCandidateCode(err, pgcode.FeatureNotSupported)
}
return err
}
colIdx, ok := ib.colIdxMap.Get(colID)
Expand Down
1 change: 1 addition & 0 deletions pkg/sql/catalog/funcdesc/func_desc.go
Original file line number Diff line number Diff line change
Expand Up @@ -685,6 +685,7 @@ func (desc *immutable) ToOverload() (ret *tree.Overload, err error) {
ReturnSet: desc.ReturnType.ReturnSet,
Body: desc.FunctionBody,
IsUDF: true,
Version: uint64(desc.Version),
}

argTypes := make(tree.ParamTypes, 0, len(desc.Params))
Expand Down
2 changes: 2 additions & 0 deletions pkg/sql/drop_function.go
Original file line number Diff line number Diff line change
Expand Up @@ -142,6 +142,8 @@ func (p *planner) matchUDF(
}
return nil, err
}
// Note that we don't check ol.HasSQLBody() here, because builtin functions
// can't be dropped even if they are defined using a SQL string.
if !ol.IsUDF {
return nil, errors.Errorf(
"cannot drop function %s%s because it is required by the database system",
Expand Down
11 changes: 11 additions & 0 deletions pkg/sql/logictest/testdata/logic_test/alter_table
Original file line number Diff line number Diff line change
Expand Up @@ -3173,3 +3173,14 @@ SHOW CONSTRAINTS FROM t_96728
table_name constraint_name constraint_type details validated
t_96728 t_96728_j_k_key UNIQUE UNIQUE (j ASC, k ASC) WHERE (i > 0) true
t_96728 t_96728_pkey PRIMARY KEY PRIMARY KEY (i ASC) true

# This subtest disallows using builtin function `cluster_logical_timestamp()`
# as the default expression when backfilling a column.
subtest 98269

statement ok
CREATE TABLE t_98269 (i INT PRIMARY KEY);
INSERT INTO t_98269 VALUES (0);

statement error pgcode 0A000 .* cluster_logical_timestamp\(\): nil txn in cluster context
ALTER TABLE t_98269 ADD COLUMN j DECIMAL NOT NULL DEFAULT cluster_logical_timestamp();
227 changes: 227 additions & 0 deletions pkg/sql/logictest/testdata/logic_test/schema
Original file line number Diff line number Diff line change
Expand Up @@ -911,3 +911,230 @@ DROP SCHEMA sc

statement ok
SET CLUSTER SETTING server.eventlog.enabled = false

subtest alter_table_schema

# Renaming the schema should invalidate a schema-qualified table reference.
statement ok
CREATE SCHEMA sc;
CREATE TABLE sc.xy (x INT, y INT);
INSERT INTO sc.xy VALUES (1, 1);

query II
SELECT * FROM sc.xy;
----
1 1

statement ok
ALTER SCHEMA sc RENAME TO sc1;

query error pq: relation "sc.xy" does not exist
SELECT * FROM sc.xy;

query II
SELECT * FROM sc1.xy;
----
1 1

statement ok
DROP SCHEMA sc1 CASCADE;

# Renaming the database should invalidate a database-qualified table reference.
statement ok
CREATE DATABASE d;
USE d;
CREATE TABLE d.xy (x INT, y INT);
INSERT INTO d.xy VALUES (1, 1);

query II
SELECT * FROM d.xy;
----
1 1

statement ok
ALTER DATABASE d RENAME TO d1;
USE d1;

query error pq: relation "d.xy" does not exist
SELECT * FROM d.xy;

query II
SELECT * FROM d1.xy;
----
1 1

statement ok
USE test;
DROP DATABASE d1 CASCADE;

# Changing the current database should invalidate an unqualified table
# reference.
statement ok
CREATE TABLE xy (x INT, y INT);
INSERT INTO xy VALUES (1, 1);

query II
SELECT * FROM xy;
----
1 1

statement ok
CREATE DATABASE d;
USE d;

query error pq: relation "xy" does not exist
SELECT * FROM xy;

statement ok
USE test;
DROP DATABASE d;
DROP TABLE xy;

# Regression tests for #96674.
subtest alter_udt_schema

# Renaming the schema should invalidate a schema-qualified UDT reference.
statement ok
CREATE SCHEMA sc;
CREATE TYPE sc.t AS ENUM ('HELLO');

query T
SELECT 'HELLO'::sc.t;
----
HELLO

statement ok
ALTER SCHEMA sc RENAME TO sc1;

query error pq: type "sc.t" does not exist
SELECT 'HELLO'::sc.t;

query T
SELECT 'HELLO'::sc1.t;
----
HELLO

statement ok
DROP SCHEMA sc1 CASCADE;

# Renaming the database should invalidate a database-qualified UDT reference.
statement ok
CREATE DATABASE d;
USE d;
CREATE TYPE d.t AS ENUM ('HELLO');

query T
SELECT 'HELLO'::d.t;
----
HELLO

statement ok
ALTER DATABASE d RENAME TO d1;
USE d1;

query error pq: type "d.t" does not exist
SELECT 'HELLO'::d.t;

query T
SELECT 'HELLO'::d1.t;
----
HELLO

statement ok
USE test;
DROP DATABASE d1 CASCADE;

# Changing the current database should invalidate an unqualified UDT reference.
statement ok
CREATE TYPE t AS ENUM ('HELLO');

query T
SELECT 'HELLO'::t;
----
HELLO

statement ok
CREATE DATABASE d;
USE d;

query error pq: type "t" does not exist
SELECT 'HELLO'::t;

statement ok
USE test;
DROP DATABASE d;
DROP TYPE t;

subtest alter_udf_schema

# Renaming the schema should invalidate a schema-qualified UDF reference.
statement ok
CREATE SCHEMA sc;
CREATE FUNCTION sc.fn(INT) RETURNS INT LANGUAGE SQL AS 'SELECT $1';

query I
SELECT sc.fn(1);
----
1

statement ok
ALTER SCHEMA sc RENAME TO sc1;

query error pq: schema "sc" does not exist
SELECT sc.fn(1);

query I
SELECT sc1.fn(1);
----
1

statement ok
DROP SCHEMA sc1 CASCADE;

# Renaming the database should invalidate a database-qualified UDF reference.
statement ok
CREATE DATABASE d;
USE d;
CREATE FUNCTION fn(INT) RETURNS INT LANGUAGE SQL AS 'SELECT $1';

query I
SELECT d.public.fn(1);
----
1

statement ok
ALTER DATABASE d RENAME TO d1;
USE d1;

query error cross-database function references not allowed
SELECT d.public.fn(1);

query I
SELECT d1.public.fn(1);
----
1

statement ok
USE test;
DROP DATABASE d1 CASCADE;

# Changing the current database should invalidate an unqualified UDF reference.
statement ok
CREATE FUNCTION fn(INT) RETURNS INT LANGUAGE SQL AS 'SELECT $1';

query I
SELECT fn(1);
----
1

statement ok
CREATE DATABASE d;
USE d;

query error pq: unknown function: fn\(\): function undefined
SELECT fn(1);

statement ok
USE test;
DROP DATABASE d;
DROP FUNCTION fn;
Loading

0 comments on commit ea389ff

Please sign in to comment.