diff --git a/pkg/ccl/changefeedccl/cdceval/expr_eval.go b/pkg/ccl/changefeedccl/cdceval/expr_eval.go
index 71f3b90a765f..2d72a8e77cc6 100644
--- a/pkg/ccl/changefeedccl/cdceval/expr_eval.go
+++ b/pkg/ccl/changefeedccl/cdceval/expr_eval.go
@@ -26,6 +26,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/sql/sessiondatapb"
"github.com/cockroachdb/cockroach/pkg/sql/types"
"github.com/cockroachdb/cockroach/pkg/util/ctxgroup"
+ "github.com/cockroachdb/cockroach/pkg/util/hlc"
"github.com/cockroachdb/cockroach/pkg/util/log"
"github.com/cockroachdb/cockroach/pkg/util/timeutil"
"github.com/cockroachdb/errors"
@@ -35,6 +36,7 @@ import (
type Evaluator struct {
sc *tree.SelectClause
+ statementTS hlc.Timestamp
// Execution context.
execCfg *sql.ExecutorConfig
user username.SQLUsername
@@ -79,14 +81,16 @@ func NewEvaluator(
execCfg *sql.ExecutorConfig,
user username.SQLUsername,
sd sessiondatapb.SessionData,
-) (*Evaluator, error) {
+ statementTS hlc.Timestamp,
+) *Evaluator {
return &Evaluator{
sc: sc,
execCfg: execCfg,
user: user,
sessionData: sd,
+ statementTS: statementTS,
familyEval: make(map[descpb.FamilyID]*familyEvaluator, 1), // usually, just 1 family.
- }, nil
+ }
}
// NewEvaluator constructs new familyEvaluator for changefeed expression.
@@ -96,6 +100,7 @@ func newFamilyEvaluator(
execCfg *sql.ExecutorConfig,
user username.SQLUsername,
sd sessiondatapb.SessionData,
+ statementTS hlc.Timestamp,
) *familyEvaluator {
e := familyEvaluator{
targetFamilyID: targetFamilyID,
@@ -107,7 +112,7 @@ func newFamilyEvaluator(
},
rowCh: make(chan tree.Datums, 1),
}
-
+ e.rowEvalCtx.startTime = statementTS
// Arrange to be notified when event does not match predicate.
predicateAsProjection(e.norm)
@@ -138,7 +143,7 @@ func (e *Evaluator) Eval(
fe, ok := e.familyEval[updatedRow.FamilyID]
if !ok {
- fe = newFamilyEvaluator(e.sc, updatedRow.FamilyID, e.execCfg, e.user, e.sessionData)
+ fe = newFamilyEvaluator(e.sc, updatedRow.FamilyID, e.execCfg, e.user, e.sessionData, e.statementTS)
e.familyEval[updatedRow.FamilyID] = fe
}
@@ -472,6 +477,7 @@ func (e *familyEvaluator) closeErr() error {
// rowEvalContext represents the context needed to evaluate row expressions.
type rowEvalContext struct {
ctx context.Context
+ startTime hlc.Timestamp
updatedRow cdcevent.Row
}
diff --git a/pkg/ccl/changefeedccl/cdceval/expr_eval_test.go b/pkg/ccl/changefeedccl/cdceval/expr_eval_test.go
index 989ecc3793e1..c136e9eec67d 100644
--- a/pkg/ccl/changefeedccl/cdceval/expr_eval_test.go
+++ b/pkg/ccl/changefeedccl/cdceval/expr_eval_test.go
@@ -781,7 +781,8 @@ func newEvaluatorWithNormCheck(
return nil, err
}
- return NewEvaluator(norm.SelectClause, execCfg, username.RootUserName(), defaultDBSessionData)
+ return NewEvaluator(norm.SelectClause, execCfg, username.RootUserName(),
+ defaultDBSessionData, hlc.Timestamp{}), nil
}
var defaultDBSessionData = sessiondatapb.SessionData{
diff --git a/pkg/ccl/changefeedccl/cdceval/functions.go b/pkg/ccl/changefeedccl/cdceval/functions.go
index 103d805fc618..58c4c5fcd88e 100644
--- a/pkg/ccl/changefeedccl/cdceval/functions.go
+++ b/pkg/ccl/changefeedccl/cdceval/functions.go
@@ -56,12 +56,28 @@ var cdcFunctions = map[string]*tree.ResolvedFunctionDefinition{
// {statement,transaction}_timestamp functions can be supported given that we
// set the statement and transaction timestamp to be equal to MVCC timestamp
// of the event. However, we provide our own override which uses annotation to
- // return the MVCC timestamp of the update.
- "statement_timestamp": makeBuiltinOverride(
- tree.FunDefs["statement_timestamp"], timestampBuiltinOverloads...,
+ // return the MVCC timestamp of the update. In addition, the custom
+ // implementation uses volatility.Volatile since doing so will cause optimizer
+ // to (constant) fold these functions during optimization step -- something we
+ // definitely don't want to do because we need to evaluate those functions for
+ // each event.
+ "statement_timestamp": cdcTimestampBuiltin(
+ "statement_timestamp",
+ "Returns MVCC timestamp of the event",
+ volatility.Volatile,
+ types.TimestampTZ,
+ func(rowEvalCtx *rowEvalContext) hlc.Timestamp {
+ return rowEvalCtx.updatedRow.MvccTimestamp
+ },
),
- "transaction_timestamp": makeBuiltinOverride(
- tree.FunDefs["transaction_timestamp"], timestampBuiltinOverloads...,
+ "transaction_timestamp": cdcTimestampBuiltin(
+ "transaction_timestamp",
+ "Returns MVCC timestamp of the event",
+ volatility.Volatile,
+ types.TimestampTZ,
+ func(rowEvalCtx *rowEvalContext) hlc.Timestamp {
+ return rowEvalCtx.updatedRow.MvccTimestamp
+ },
),
"timezone": useDefaultBuiltin,
@@ -74,6 +90,11 @@ var cdcFunctions = map[string]*tree.ResolvedFunctionDefinition{
//"st_asgeojson",
//"st_estimatedextent",
+ // NB: even though some cdc functions appear to be stable (e.g. cdc_is_delete()),
+ // we should not mark custom CDC functions as stable. Doing so will cause
+ // optimizer to (constant) fold this function during optimization step -- something
+ // we definitely don't want to do because we need to evaluate those functions
+ // for each event.
"cdc_is_delete": makeCDCBuiltIn(
"cdc_is_delete",
tree.Overload{
@@ -86,28 +107,36 @@ var cdcFunctions = map[string]*tree.ResolvedFunctionDefinition{
}
return tree.DBoolFalse, nil
},
- Info: "Returns true if the event is a deletion",
- // NB: even though some cdc functions appear to be stable (e.g. cdc_is_delete()),
- // we should not mark custom CDC functions as stable. Doing so will cause
- // optimizer to (constant) fold this function during optimization step -- something
- // we definitely don't want to do because we need to evaluate those functions
- // for each event.
+ Info: "Returns true if the event is a deletion",
Volatility: volatility.Volatile,
}),
"cdc_mvcc_timestamp": cdcTimestampBuiltin(
"cdc_mvcc_timestamp",
- "Returns event MVCC HLC timestamp",
+ "Returns MVCC timestamp of the event",
+ volatility.Volatile,
+ types.Decimal,
func(rowEvalCtx *rowEvalContext) hlc.Timestamp {
return rowEvalCtx.updatedRow.MvccTimestamp
},
),
"cdc_updated_timestamp": cdcTimestampBuiltin(
"cdc_updated_timestamp",
- "Returns event updated HLC timestamp",
+ "Returns schema timestamp of the event",
+ volatility.Volatile,
+ types.Decimal,
func(rowEvalCtx *rowEvalContext) hlc.Timestamp {
return rowEvalCtx.updatedRow.SchemaTS
},
),
+ "changefeed_creation_timestamp": cdcTimestampBuiltin(
+ "changefeed_creation_timestamp",
+ "Returns changefeed creation time",
+ volatility.Stable,
+ types.Decimal,
+ func(rowEvalCtx *rowEvalContext) hlc.Timestamp {
+ return rowEvalCtx.startTime
+ },
+ ),
}
const cdcFnCategory = "CDC builtin"
@@ -125,7 +154,11 @@ func makeCDCBuiltIn(fnName string, overloads ...tree.Overload) *tree.ResolvedFun
}
func cdcTimestampBuiltin(
- fnName string, doc string, tsFn func(rowEvalCtx *rowEvalContext) hlc.Timestamp,
+ fnName string,
+ doc string,
+ v volatility.V,
+ preferredOverloadReturnType *types.T,
+ tsFn func(rowEvalCtx *rowEvalContext) hlc.Timestamp,
) *tree.ResolvedFunctionDefinition {
def := tree.NewFunctionDefinition(
fnName,
@@ -138,16 +171,35 @@ func cdcTimestampBuiltin(
rowEvalCtx := rowEvalContextFromEvalContext(evalCtx)
return eval.TimestampToDecimalDatum(tsFn(rowEvalCtx)), nil
},
- Info: doc,
- // NB: even though some cdc functions appear to be stable (e.g. cdc_is_delete()),
- // we should not mark custom CDC functions as stable. Doing so will cause
- // optimizer to (constant) fold this function during optimization step -- something
- // we definitely don't want to do because we need to evaluate those functions
- // for each event.
- Volatility: volatility.Volatile,
+ Info: doc + " as HLC timestamp",
+ Volatility: v,
+ PreferredOverload: preferredOverloadReturnType == types.Decimal,
+ },
+ {
+ Types: tree.ParamTypes{},
+ ReturnType: tree.FixedReturnType(types.TimestampTZ),
+ Fn: func(ctx context.Context, evalCtx *eval.Context, args tree.Datums) (tree.Datum, error) {
+ rowEvalCtx := rowEvalContextFromEvalContext(evalCtx)
+ return tree.MakeDTimestampTZ(tsFn(rowEvalCtx).GoTime(), time.Microsecond)
+ },
+ Info: doc + " as TIMESTAMPTZ",
+ Volatility: v,
+ PreferredOverload: preferredOverloadReturnType == types.TimestampTZ,
+ },
+ {
+ Types: tree.ParamTypes{},
+ ReturnType: tree.FixedReturnType(types.Timestamp),
+ Fn: func(ctx context.Context, evalCtx *eval.Context, args tree.Datums) (tree.Datum, error) {
+ rowEvalCtx := rowEvalContextFromEvalContext(evalCtx)
+ return tree.MakeDTimestamp(tsFn(rowEvalCtx).GoTime(), time.Microsecond)
+ },
+ Info: doc + " as TIMESTAMP",
+ Volatility: v,
+ PreferredOverload: preferredOverloadReturnType == types.Timestamp,
},
},
)
+
// The schema name is actually not important since CDC doesn't use any user
// defined functions. And, we're sure that we always return the first
// function definition found.
@@ -185,51 +237,6 @@ func TestingDisableFunctionsBlacklist() func() {
}
}
-// For some functions (specifically the volatile ones), we do
-// not want to use the provided builtin. Instead, we opt for
-// our own function definition.
-func makeBuiltinOverride(
- builtin *tree.FunctionDefinition, overloads ...tree.Overload,
-) *tree.ResolvedFunctionDefinition {
- props := builtin.FunctionProperties
- override := tree.NewFunctionDefinition(builtin.Name, &props, overloads)
- // The schema name is actually not important since CDC doesn't use any user
- // defined functions. And, we're sure that we always return the first
- // function definition found.
- return tree.QualifyBuiltinFunctionDefinition(override, catconstants.PublicSchemaName)
-}
-
-// tree.Overload definitions for statement_timestamp and transaction_timestamp functions.
-var timestampBuiltinOverloads = []tree.Overload{
- {
- Types: tree.ParamTypes{},
- ReturnType: tree.FixedReturnType(types.TimestampTZ),
- PreferredOverload: true,
- Fn: func(ctx context.Context, evalCtx *eval.Context, args tree.Datums) (tree.Datum, error) {
- rowEvalCtx := rowEvalContextFromEvalContext(evalCtx)
- return tree.MakeDTimestampTZ(rowEvalCtx.updatedRow.MvccTimestamp.GoTime(), time.Microsecond)
- },
- Info: "Returns MVCC timestamp of the event",
- // NB: Default builtin implementation uses volatility.Stable
- // We override volatility to be Volatile so that function
- // is not folded.
- Volatility: volatility.Volatile,
- },
- {
- Types: tree.ParamTypes{},
- ReturnType: tree.FixedReturnType(types.Timestamp),
- Fn: func(ctx context.Context, evalCtx *eval.Context, args tree.Datums) (tree.Datum, error) {
- rowEvalCtx := rowEvalContextFromEvalContext(evalCtx)
- return tree.MakeDTimestamp(rowEvalCtx.updatedRow.MvccTimestamp.GoTime(), time.Microsecond)
- },
- Info: "Returns MVCC timestamp of the event",
- // NB: Default builtin implementation uses volatility.Stable
- // We override volatility to be Volatile so that function
- // is not folded.
- Volatility: volatility.Volatile,
- },
-}
-
var functionDenyList = make(map[string]struct{})
func init() {
diff --git a/pkg/ccl/changefeedccl/cdceval/functions_test.go b/pkg/ccl/changefeedccl/cdceval/functions_test.go
index 5e1d4ff88076..9683cfa1bff2 100644
--- a/pkg/ccl/changefeedccl/cdceval/functions_test.go
+++ b/pkg/ccl/changefeedccl/cdceval/functions_test.go
@@ -53,45 +53,106 @@ func TestEvaluatesCDCFunctionOverloads(t *testing.T) {
defer configSemaForCDC(&semaCtx)()
t.Run("time", func(t *testing.T) {
- // We'll run tests against some future time stamp to ensure
- // that time functions use correct values.
- futureTS := s.Clock().Now().Add(int64(60*time.Minute), 0)
- expectTSTZ := func(ts hlc.Timestamp) string {
+ expectTSTZ := func(ts hlc.Timestamp) tree.Datum {
t.Helper()
d, err := tree.MakeDTimestampTZ(ts.GoTime(), time.Microsecond)
require.NoError(t, err)
- return tree.AsStringWithFlags(d, tree.FmtExport)
+ return d
+ }
+ expectTS := func(ts hlc.Timestamp) tree.Datum {
+ t.Helper()
+ d, err := tree.MakeDTimestamp(ts.GoTime(), time.Microsecond)
+ require.NoError(t, err)
+ return d
+ }
+ expectHLC := func(ts hlc.Timestamp) tree.Datum {
+ t.Helper()
+ return eval.TimestampToDecimalDatum(ts)
}
- for _, tc := range []struct {
- fn string
- expect string
- }{
- {fn: "statement_timestamp", expect: expectTSTZ(futureTS)},
- {fn: "transaction_timestamp", expect: expectTSTZ(futureTS)},
+ type preferredFn func(ts hlc.Timestamp) tree.Datum
+ for fn, preferredOverload := range map[string]preferredFn{
+ "statement_timestamp": expectTSTZ,
+ "transaction_timestamp": expectTSTZ,
+ "cdc_mvcc_timestamp": expectHLC,
+ "cdc_updated_timestamp": expectHLC,
+ "changefeed_creation_timestamp": expectHLC,
} {
- t.Run(tc.fn, func(t *testing.T) {
- testRow := makeEventRow(t, desc, s.Clock().Now(), false, futureTS)
+ t.Run(fn, func(t *testing.T) {
+ createTS := s.Clock().Now().Add(-int64(60*time.Minute), 0)
+ schemaTS := s.Clock().Now()
+ rowTS := schemaTS.Add(int64(60*time.Minute), 0)
+
+ targetTS := rowTS
+ switch fn {
+ case "cdc_updated_timestamp":
+ targetTS = schemaTS
+ case "changefeed_creation_timestamp":
+ targetTS = createTS
+ }
+ // We'll run tests against some future time stamp to ensure
+ // that time functions use correct values.
+ testRow := makeEventRow(t, desc, schemaTS, false, rowTS)
e, err := newEvaluator(&execCfg, &semaCtx, testRow.EventDescriptor,
- fmt.Sprintf("SELECT %s() FROM foo", tc.fn))
+ fmt.Sprintf("SELECT "+
+ "%[1]s() AS preferred,"+ // Preferred overload.
+ "%[1]s():::TIMESTAMPTZ AS tstz,"+ // Force timestamptz overload.
+ "%[1]s():::TIMESTAMP AS ts,"+ // Force timestamp overload.
+ "%[1]s():::DECIMAL AS dec,"+ // Force decimal overload.
+ "%[1]s()::STRING AS str"+ // Casts preferred overload to string.
+ " FROM foo", fn))
require.NoError(t, err)
defer e.Close()
+ e.statementTS = createTS
p, err := e.Eval(ctx, testRow, cdcevent.Row{})
require.NoError(t, err)
- require.Equal(t, map[string]string{tc.fn: tc.expect}, slurpValues(t, p))
- // Emit again, this time advancing MVCC timestamp of the row.
- // We want to make sure that optimizer did not constant fold the call
- // to the function, even though this function is marked stable.
+ initialExpectations := map[string]string{
+ "preferred": tree.AsStringWithFlags(preferredOverload(targetTS), tree.FmtExport),
+ "tstz": tree.AsStringWithFlags(expectTSTZ(targetTS), tree.FmtExport),
+ "ts": tree.AsStringWithFlags(expectTS(targetTS), tree.FmtExport),
+ "dec": tree.AsStringWithFlags(eval.TimestampToDecimalDatum(targetTS), tree.FmtExport),
+ "str": tree.AsStringWithFlags(preferredOverload(targetTS), tree.FmtExport),
+ }
+ require.Equal(t, initialExpectations, slurpValues(t, p))
+
+ // Modify row/schema timestamps, and evaluate again.
testRow.MvccTimestamp = testRow.MvccTimestamp.Add(int64(time.Hour), 0)
+ targetTS = testRow.MvccTimestamp
+ testRow.SchemaTS = schemaTS.Add(1, 0)
+ e.statementTS = e.statementTS.Add(-1, 0)
p, err = e.Eval(ctx, testRow, cdcevent.Row{})
require.NoError(t, err)
- require.Equal(t, map[string]string{tc.fn: expectTSTZ(testRow.MvccTimestamp)}, slurpValues(t, p))
+
+ var updatedExpectations map[string]string
+ switch fn {
+ case "changefeed_creation_timestamp":
+ // this function is stable; So, advancing evaluator timestamp
+ // should have no bearing on the returned values -- we should see
+ // the same thing we saw before.
+ updatedExpectations = initialExpectations
+ case "cdc_updated_timestamp":
+ targetTS = testRow.SchemaTS
+ fallthrough
+ default:
+ updatedExpectations = map[string]string{
+ "preferred": tree.AsStringWithFlags(preferredOverload(targetTS), tree.FmtExport),
+ "tstz": tree.AsStringWithFlags(expectTSTZ(targetTS), tree.FmtExport),
+ "ts": tree.AsStringWithFlags(expectTS(targetTS), tree.FmtExport),
+ "dec": tree.AsStringWithFlags(eval.TimestampToDecimalDatum(targetTS), tree.FmtExport),
+ "str": tree.AsStringWithFlags(preferredOverload(targetTS), tree.FmtExport),
+ }
+ }
+ require.Equal(t, updatedExpectations, slurpValues(t, p))
})
}
t.Run("timezone", func(t *testing.T) {
+ // We'll run tests against some future time stamp to ensure
+ // that time functions use correct values.
+ futureTS := s.Clock().Now().Add(int64(60*time.Minute), 0)
+
// Timezone has many overrides, some are immutable, and some are Stable.
// Call "stable" overload which relies on session data containing
// timezone. Since we don't do any special setup with session data, the
@@ -138,31 +199,6 @@ func TestEvaluatesCDCFunctionOverloads(t *testing.T) {
return j
}
- t.Run("cdc_{mvcc,updated}_timestamp", func(t *testing.T) {
- for _, cast := range []string{"", "::decimal", "::string"} {
- t.Run(cast, func(t *testing.T) {
- schemaTS := s.Clock().Now()
- mvccTS := schemaTS.Add(int64(30*time.Minute), 0)
- testRow := makeEventRow(t, desc, schemaTS, false, mvccTS)
- e, err := newEvaluator(&execCfg, &semaCtx, testRow.EventDescriptor, fmt.Sprintf(
- "SELECT cdc_mvcc_timestamp()%[1]s as mvcc, cdc_updated_timestamp()%[1]s as updated FROM foo", cast,
- ))
- require.NoError(t, err)
- defer e.Close()
-
- p, err := e.Eval(ctx, testRow, cdcevent.Row{})
- require.NoError(t, err)
- require.Equal(t,
- map[string]string{
- "mvcc": eval.TimestampToDecimalDatum(mvccTS).String(),
- "updated": eval.TimestampToDecimalDatum(schemaTS).String(),
- },
- slurpValues(t, p),
- )
- })
- }
- })
-
t.Run("pg_collation_for", func(t *testing.T) {
testRow := makeEventRow(t, desc, s.Clock().Now(), false, s.Clock().Now())
@@ -346,5 +382,6 @@ func newEvaluator(
if err != nil {
return nil, err
}
- return NewEvaluator(norm.SelectClause, execCfg, username.RootUserName(), defaultDBSessionData)
+ return NewEvaluator(norm.SelectClause, execCfg, username.RootUserName(),
+ defaultDBSessionData, execCfg.Clock.Now()), nil
}
diff --git a/pkg/ccl/changefeedccl/event_processing.go b/pkg/ccl/changefeedccl/event_processing.go
index 55b031e93be6..5046b5e49d6f 100644
--- a/pkg/ccl/changefeedccl/event_processing.go
+++ b/pkg/ccl/changefeedccl/event_processing.go
@@ -290,7 +290,7 @@ func newEvaluator(
sd = *spec.Feed.SessionData
}
- return cdceval.NewEvaluator(sc, cfg, spec.User(), sd)
+ return cdceval.NewEvaluator(sc, cfg, spec.User(), sd, spec.Feed.StatementTime), nil
}
func (c *kvEventToRowConsumer) topicForEvent(eventMeta cdcevent.Metadata) (TopicDescriptor, error) {
diff --git a/pkg/sql/distsql_plan_changefeed.go b/pkg/sql/distsql_plan_changefeed.go
index e452964d3dcb..7d203ef0aa80 100644
--- a/pkg/sql/distsql_plan_changefeed.go
+++ b/pkg/sql/distsql_plan_changefeed.go
@@ -97,13 +97,17 @@ func PlanCDCExpression(
if err != nil {
return cdcPlan, err
}
+
cdcCat := &cdcOptCatalog{
optCatalog: opc.catalog.(*optCatalog),
cdcConfig: cfg,
targetFamilyID: familyID,
semaCtx: &p.semaCtx,
}
+
+ // Reset catalog to cdc specific implementation.
opc.catalog = cdcCat
+ opc.optimizer.Init(ctx, p.EvalContext(), opc.catalog)
memo, err := opc.buildExecMemo(ctx)
if err != nil {
diff --git a/pkg/sql/logictest/testdata/logic_test/udf b/pkg/sql/logictest/testdata/logic_test/udf
index 7685e386689a..0116557924c9 100644
--- a/pkg/sql/logictest/testdata/logic_test/udf
+++ b/pkg/sql/logictest/testdata/logic_test/udf
@@ -2923,3 +2923,12 @@ SELECT f95240(a) FROM t95240
----
33
NULL
+
+onlyif config local
+query T
+EXPLAIN CREATE FUNCTION f() RETURNS INT LANGUAGE SQL AS 'SELECT 1'
+----
+distribution: local
+vectorized: true
+·
+• create function
diff --git a/pkg/sql/opt/exec/explain/emit.go b/pkg/sql/opt/exec/explain/emit.go
index 68f4fe81cb77..e9f0b1517f35 100644
--- a/pkg/sql/opt/exec/explain/emit.go
+++ b/pkg/sql/opt/exec/explain/emit.go
@@ -284,6 +284,7 @@ var nodeNames = [...]string{
controlJobsOp: "control jobs",
controlSchedulesOp: "control schedules",
createStatisticsOp: "create statistics",
+ createFunctionOp: "create function",
createTableOp: "create table",
createTableAsOp: "create table as",
createViewOp: "create view",
@@ -952,6 +953,7 @@ func (e *emitter) emitNodeAttributes(n *Node) error {
explainOptOp,
explainOp,
showTraceOp,
+ createFunctionOp,
createTableOp,
createTableAsOp,
createViewOp,
diff --git a/pkg/ui/workspaces/cluster-ui/src/databaseDetailsPage/databaseDetailsPage.tsx b/pkg/ui/workspaces/cluster-ui/src/databaseDetailsPage/databaseDetailsPage.tsx
index 0b39635f9900..c7f2f2012007 100644
--- a/pkg/ui/workspaces/cluster-ui/src/databaseDetailsPage/databaseDetailsPage.tsx
+++ b/pkg/ui/workspaces/cluster-ui/src/databaseDetailsPage/databaseDetailsPage.tsx
@@ -19,7 +19,7 @@ import { CaretRight } from "src/icon/caretRight";
import { DatabaseIcon } from "src/icon/databaseIcon";
import { StackIcon } from "src/icon/stackIcon";
import { PageConfig, PageConfigItem } from "src/pageConfig";
-import { Pagination, ResultsPerPageLabel } from "src/pagination";
+import { Pagination } from "src/pagination";
import {
ColumnDescriptor,
ISortedTablePagination,
@@ -27,15 +27,16 @@ import {
SortSetting,
} from "src/sortedtable";
import * as format from "src/util/format";
-import { DATE_FORMAT } from "src/util/format";
+import {
+ DATE_FORMAT,
+ EncodeDatabaseTableUri,
+ EncodeDatabaseUri,
+} from "src/util/format";
import { mvccGarbage, syncHistory, unique } from "../util";
import styles from "./databaseDetailsPage.module.scss";
import sortableTableStyles from "src/sortedtable/sortedtable.module.scss";
-import {
- baseHeadingClasses,
- statisticsClasses,
-} from "src/transactionsPage/transactionsPageClasses";
+import { baseHeadingClasses } from "src/transactionsPage/transactionsPageClasses";
import { Moment } from "moment";
import { Caution } from "@cockroachlabs/icons";
import { Anchor } from "../anchor";
@@ -43,10 +44,10 @@ import LoadingError from "../sqlActivity/errorComponent";
import { Loading } from "../loading";
import { Search } from "../search";
import {
+ calculateActiveFilters,
+ defaultFilters,
Filter,
Filters,
- defaultFilters,
- calculateActiveFilters,
} from "src/queryFilter";
import { UIConfigState } from "src/store";
import { TableStatistics } from "src/tableStatistics";
@@ -509,7 +510,7 @@ export class DatabaseDetailsPage extends React.Component<
),
cell: table => (
@@ -683,7 +684,10 @@ export class DatabaseDetailsPage extends React.Component<
),
cell: table => (
@@ -802,7 +806,10 @@ export class DatabaseDetailsPage extends React.Component<
diff --git a/pkg/ui/workspaces/cluster-ui/src/databaseTablePage/databaseTablePage.tsx b/pkg/ui/workspaces/cluster-ui/src/databaseTablePage/databaseTablePage.tsx
index 57a8a19279a8..33ce856f9912 100644
--- a/pkg/ui/workspaces/cluster-ui/src/databaseTablePage/databaseTablePage.tsx
+++ b/pkg/ui/workspaces/cluster-ui/src/databaseTablePage/databaseTablePage.tsx
@@ -31,7 +31,12 @@ import {
SummaryCardItemBoolSetting,
} from "src/summaryCard";
import * as format from "src/util/format";
-import { DATE_FORMAT, DATE_FORMAT_24_UTC } from "src/util/format";
+import {
+ DATE_FORMAT,
+ DATE_FORMAT_24_UTC,
+ EncodeDatabaseTableUri,
+ EncodeDatabaseUri,
+} from "src/util/format";
import {
ascendingAttr,
columnTitleAttr,
@@ -504,9 +509,15 @@ export class DatabaseTablePage extends React.Component<
(
diff --git a/pkg/ui/workspaces/cluster-ui/src/indexDetailsPage/indexDetailsPage.tsx b/pkg/ui/workspaces/cluster-ui/src/indexDetailsPage/indexDetailsPage.tsx
index 10c03d92f00e..d44911c198ae 100644
--- a/pkg/ui/workspaces/cluster-ui/src/indexDetailsPage/indexDetailsPage.tsx
+++ b/pkg/ui/workspaces/cluster-ui/src/indexDetailsPage/indexDetailsPage.tsx
@@ -34,6 +34,9 @@ import {
calculateTotalWorkload,
Count,
DATE_FORMAT_24_UTC,
+ EncodeDatabaseTableIndexUri,
+ EncodeDatabaseTableUri,
+ EncodeDatabaseUri,
performanceTuningRecipes,
} from "../util";
import {
@@ -258,15 +261,22 @@ export class IndexDetailsPage extends React.Component<
items={[
{ link: "/databases", name: "Databases" },
{
- link: `/database/${this.props.databaseName}`,
+ link: EncodeDatabaseUri(this.props.databaseName),
name: "Tables",
},
{
- link: `/database/${this.props.databaseName}/table/${this.props.tableName}`,
+ link: EncodeDatabaseTableUri(
+ this.props.databaseName,
+ this.props.tableName,
+ ),
name: `Table: ${this.props.tableName}`,
},
{
- link: `/database/${this.props.databaseName}/table/${this.props.tableName}/index/${this.props.indexName}`,
+ link: EncodeDatabaseTableIndexUri(
+ this.props.databaseName,
+ this.props.tableName,
+ this.props.indexName,
+ ),
name: `Index: ${this.props.indexName}`,
},
]}
diff --git a/pkg/ui/workspaces/cluster-ui/src/insightsTable/insightsTable.tsx b/pkg/ui/workspaces/cluster-ui/src/insightsTable/insightsTable.tsx
index 2ca9eab6dc85..0db38f53fe78 100644
--- a/pkg/ui/workspaces/cluster-ui/src/insightsTable/insightsTable.tsx
+++ b/pkg/ui/workspaces/cluster-ui/src/insightsTable/insightsTable.tsx
@@ -19,12 +19,14 @@ import {
clusterSettings,
computeOrUseStmtSummary,
Duration,
+ EncodeDatabasesToIndexUri,
+ EncodeDatabaseTableIndexUri,
performanceBestPractices,
+ performanceTuningRecipes,
statementsRetries,
} from "../util";
import { Anchor } from "../anchor";
import { Link } from "react-router-dom";
-import { performanceTuningRecipes } from "../util";
import { InsightRecommendation, insightType } from "../insights";
const cx = classNames.bind(styles);
@@ -129,8 +131,17 @@ function descriptionCell(
);
const indexLink = isCockroachCloud
- ? `databases/${insightRec.database}/${insightRec.indexDetails?.schema}/${insightRec.indexDetails?.table}/${insightRec.indexDetails?.indexName}`
- : `database/${insightRec.database}/table/${insightRec.indexDetails?.table}/index/${insightRec.indexDetails?.indexName}`;
+ ? EncodeDatabasesToIndexUri(
+ insightRec.database,
+ insightRec.indexDetails?.schema,
+ insightRec.indexDetails?.table,
+ insightRec.indexDetails?.indexName,
+ )
+ : EncodeDatabaseTableIndexUri(
+ insightRec.database,
+ insightRec.indexDetails?.table,
+ insightRec.indexDetails?.indexName,
+ );
switch (insightRec.type) {
case "CreateIndex":
diff --git a/pkg/ui/workspaces/cluster-ui/src/util/format.ts b/pkg/ui/workspaces/cluster-ui/src/util/format.ts
index 5aaa488629f1..c5d9e81b2138 100644
--- a/pkg/ui/workspaces/cluster-ui/src/util/format.ts
+++ b/pkg/ui/workspaces/cluster-ui/src/util/format.ts
@@ -290,3 +290,40 @@ export function capitalize(str: string): string {
if (!str) return str;
return str[0].toUpperCase() + str.substring(1);
}
+
+export function EncodeUriName(name: string) {
+ return encodeURIComponent(name);
+}
+
+export function EncodeDatabasesUri(db: string) {
+ return `/databases/${EncodeUriName(db)}`;
+}
+
+export function EncodeDatabasesToIndexUri(
+ db: string,
+ schema: string,
+ table: string,
+ indexName: string,
+) {
+ return `${EncodeDatabasesUri(db)}/${EncodeUriName(schema)}/${EncodeUriName(
+ table,
+ )}/${EncodeUriName(indexName)}`;
+}
+
+export function EncodeDatabaseTableUri(db: string, table: string) {
+ return `${EncodeDatabaseUri(db)}/table/${EncodeUriName(table)}`;
+}
+
+export function EncodeDatabaseTableIndexUri(
+ db: string,
+ table: string,
+ indexName: string,
+) {
+ return `${EncodeDatabaseTableUri(db, table)}/index/${EncodeUriName(
+ indexName,
+ )}`;
+}
+
+export function EncodeDatabaseUri(db: string) {
+ return `/database/${EncodeUriName(db)}`;
+}
diff --git a/pkg/ui/workspaces/db-console/src/util/api.ts b/pkg/ui/workspaces/db-console/src/util/api.ts
index f294e89e7e33..64dd4434e304 100644
--- a/pkg/ui/workspaces/db-console/src/util/api.ts
+++ b/pkg/ui/workspaces/db-console/src/util/api.ts
@@ -357,9 +357,13 @@ export function getDatabaseDetails(
): Promise {
const queryString = req.include_stats ? "?include_stats=true" : "";
+ const promiseErr = IsValidateUriName(req.database);
+ if (promiseErr) {
+ return promiseErr;
+ }
return timeoutFetch(
serverpb.DatabaseDetailsResponse,
- `${API_PREFIX}/databases/${req.database}${queryString}`,
+ `${API_PREFIX}/databases/${EncodeUriName(req.database)}${queryString}`,
null,
timeout,
);
@@ -370,9 +374,16 @@ export function getTableDetails(
req: TableDetailsRequestMessage,
timeout?: moment.Duration,
): Promise {
+ const promiseErr = IsValidateUriName(req.database, req.table);
+ if (promiseErr) {
+ return promiseErr;
+ }
+
return timeoutFetch(
serverpb.TableDetailsResponse,
- `${API_PREFIX}/databases/${req.database}/tables/${req.table}`,
+ `${API_PREFIX}/databases/${EncodeUriName(
+ req.database,
+ )}/tables/${EncodeUriName(req.table)}`,
null,
timeout,
);
@@ -519,9 +530,16 @@ export function getTableStats(
req: TableStatsRequestMessage,
timeout?: moment.Duration,
): Promise {
+ const promiseErr = IsValidateUriName(req.database, req.table);
+ if (promiseErr) {
+ return promiseErr;
+ }
+
return timeoutFetch(
serverpb.TableStatsResponse,
- `${API_PREFIX}/databases/${req.database}/tables/${req.table}/stats`,
+ `${API_PREFIX}/databases/${EncodeUriName(
+ req.database,
+ )}/tables/${EncodeUriName(req.table)}/stats`,
null,
timeout,
);
@@ -532,9 +550,16 @@ export function getIndexStats(
req: IndexStatsRequestMessage,
timeout?: moment.Duration,
): Promise {
+ const promiseErr = IsValidateUriName(req.database, req.table);
+ if (promiseErr) {
+ return promiseErr;
+ }
+
return timeoutFetch(
serverpb.TableIndexStatsResponse,
- `${STATUS_PREFIX}/databases/${req.database}/tables/${req.table}/indexstats`,
+ `${STATUS_PREFIX}/databases/${EncodeUriName(
+ req.database,
+ )}/tables/${EncodeUriName(req.table)}/indexstats`,
null,
timeout,
);
@@ -877,3 +902,20 @@ export function getKeyVisualizerSamples(
timeout,
);
}
+
+export function IsValidateUriName(...args: string[]): Promise {
+ for (const name of args) {
+ if (name.includes("/")) {
+ return Promise.reject(
+ new Error(
+ `util/api: The entity '${name}' contains '/' which is not currently supported in the UI.`,
+ ),
+ );
+ }
+ }
+ return null;
+}
+
+export function EncodeUriName(name: string): string {
+ return encodeURIComponent(name);
+}