Skip to content

Commit

Permalink
Browse files Browse the repository at this point in the history
117769: cli/demo: expand demo://tenant to postgres:// URL r=dt a=dt

The SQL connection URLs are large and somewhat unwieldy when used with \c to reconnect or in PCR control statements. One particular source of friction, beyond the sheer length of the string making statements harder to read or modify by hand, is that the connection string in it includes values that are specific to a given `demo` session, meaning a subsequent session cannot simply re-run the same commands from the recorded shell history as they contain the ephemeral ports and passwords.

This change adds a small special-case to the shell when it is running in demo mode, so that it will replace any occurrence of a string of the form 'demo://<cluster_name>' with the pgurl for the virtual cluster of that name on its first server.

For example, when used reconnecting the shell itself to a different pgurl:

```
demo@127.0.0.1:26257/system/defaultdb> \c demo://demoapp
using new connection URL: postgresql://demo:demo36905@127.0.0.1:26257/defaultdb?options=-ccluster%3Ddemoapp&sslmode=require&sslrootcert=%2FUsers%2Fuser%2F.cockroach-demo%2Fca.crt

demo@127.0.0.1:26257/demoapp/defaultdb> \c demo://system
using new connection URL: postgresql://demo:demo36905@127.0.0.1:26257/defaultdb?options=-ccluster%3Dsystem&sslmode=require&sslrootcert=%2FUsers%2Fuser%2F.cockroach-demo%2Fca.crt
```

For another example, when used in the content of a statement that will be sent to the database:
```
demo@127.0.0.1:26257/system/defaultdb> create virtual cluster t2 from replication of demoapp on 'demo://system';
CREATE VIRTUAL CLUSTER FROM REPLICATION 0

demo@127.0.0.1:26257/system/defaultdb> SELECT source_cluster_uri FROM [SHOW VIRTUAL CLUSTER t2 WITH REPLICATION STATUS];
                                                   source_cluster_uri
------------------------------------------------------------------------------------------------------------------------
  postgresql://demo:redacted@127.0.0.1:26257/defaultdb?options=-ccluster%3Dsystem&sslmode=require&sslrootcert=redacted
```

The substitutions are performed after recording the line in the shell history, so that the history is kept in terms of the short name that remains constant across demo sessions.

Release note: none.
Epic: none.

117976: sql: remove unused functions r=mgartner a=mgartner

#### sql: remove unused functions

The best code is no code.

Release note: None

#### sql/tests: add "_test" suffix to file name

The commit renames a test file, `virtual_cluster_name.go`, to match the
naming convention. I believe the test in the was being run in CI before,
despite the misnaming.

Epic: None

Release note: None


117990: generate-logictest: skip certain CCL configurations under `race` r=rail a=rickystewart

For whatever reason these particular configurations are particularly prone to timing out under `race`. Presumably they are more sensitive to the extra load. The other configurations seem not to have this issue.

Epic: none
Release note: None

Co-authored-by: David Taylor <tinystatemachine@gmail.com>
Co-authored-by: Marcus Gartner <marcus@cockroachlabs.com>
Co-authored-by: Ricky Stewart <ricky@cockroachlabs.com>
  • Loading branch information
4 people committed Jan 19, 2024
4 parents 981d187 + 28a8207 + 77c3dbf + 3296952 commit 8a34679
Show file tree
Hide file tree
Showing 28 changed files with 37 additions and 155 deletions.
1 change: 1 addition & 0 deletions pkg/ccl/logictestccl/tests/3node-tenant/generated_test.go

Some generated files are not rendered by default. Learn more about how customized files appear on GitHub.

Some generated files are not rendered by default. Learn more about how customized files appear on GitHub.

Some generated files are not rendered by default. Learn more about how customized files appear on GitHub.

Some generated files are not rendered by default. Learn more about how customized files appear on GitHub.

Some generated files are not rendered by default. Learn more about how customized files appear on GitHub.

Some generated files are not rendered by default. Learn more about how customized files appear on GitHub.

Some generated files are not rendered by default. Learn more about how customized files appear on GitHub.

8 changes: 8 additions & 0 deletions pkg/cli/clisqlshell/sql.go
Original file line number Diff line number Diff line change
Expand Up @@ -1392,6 +1392,10 @@ func (c *cliState) doHandleCliCmd(loopState, nextState cliStateEnum) cliStateEnu
// to handle it as a statement, so save the history.
c.addHistory(c.lastInputLine)

if c.sqlCtx.DemoCluster != nil {
c.lastInputLine = c.sqlCtx.DemoCluster.ExpandShortDemoURLs(c.lastInputLine)
}

// As a convenience to the user, we strip the final semicolon, if
// any, in all cases.
line := strings.TrimRight(c.lastInputLine, "; ")
Expand Down Expand Up @@ -1980,6 +1984,10 @@ func (c *cliState) doPrepareStatementLine(
c.addHistory(c.concatLines)
}

if c.sqlCtx.DemoCluster != nil {
c.concatLines = c.sqlCtx.DemoCluster.ExpandShortDemoURLs(c.concatLines)
}

if !c.iCtx.checkSyntax {
return execState
}
Expand Down
3 changes: 3 additions & 0 deletions pkg/cli/democluster/api/api.go
Original file line number Diff line number Diff line change
Expand Up @@ -26,6 +26,9 @@ type DemoCluster interface {
// Listing is printed to 'w'. Errors/warnings are printed to 'ew'.
ListDemoNodes(w, ew io.Writer, justOne, verbose bool)

// ExpandShortDemoURLs expands `demo://` in a string URLs to `postgres://...`.
ExpandShortDemoURLs(string) string

// AddNode creates a new node with the given locality string.
AddNode(ctx context.Context, localityString string) (newNodeID int32, err error)

Expand Down
11 changes: 11 additions & 0 deletions pkg/cli/democluster/demo_cluster.go
Original file line number Diff line number Diff line change
Expand Up @@ -2038,6 +2038,17 @@ func (c *transientCluster) ListDemoNodes(w, ew io.Writer, justOne, verbose bool)
}
}

func (c *transientCluster) ExpandShortDemoURLs(s string) string {
if !strings.Contains(s, "demo://") {
return s
}
u, err := c.getNetworkURLForServer(context.Background(), 0, false, forSystemTenant)
if err != nil {
return s
}
return regexp.MustCompile(`demo://([[:alnum:]]+)`).ReplaceAllString(s, strings.ReplaceAll(u.String(), "-ccluster%3Dsystem", "-ccluster%3D$1"))
}

func (c *transientCluster) printURLs(
w, ew io.Writer,
sqlURL *pgurl.URL,
Expand Down
4 changes: 4 additions & 0 deletions pkg/cmd/generate-logictest/main.go
Original file line number Diff line number Diff line change
Expand Up @@ -32,6 +32,7 @@ type testFileTemplateConfig struct {
CockroachGoTestserverTest bool
Ccl bool
ForceProductionValues bool
SkipCclUnderRace bool
Package, TestRuleName, RelDir string
ConfigIdx int
TestCount int
Expand Down Expand Up @@ -178,6 +179,9 @@ func (t *testdir) dump() error {
// allocate the tests which use 3-node clusters 2 vCPUs, and
// the ones which use more a bit more.
tplCfg.NumCPU = (cfg.NumNodes / 2) + 1
if cfg.Name == "3node-tenant" || strings.HasPrefix(cfg.Name, "multiregion-") {
tplCfg.SkipCclUnderRace = true
}
subdir := filepath.Join(t.dir, cfg.Name)
f, buildF, cleanup, err := openTestSubdir(subdir)
if err != nil {
Expand Down
3 changes: 2 additions & 1 deletion pkg/cmd/generate-logictest/templates.go
Original file line number Diff line number Diff line change
Expand Up @@ -60,7 +60,8 @@ func runLogicTest(t *testing.T, file string) {
{{- define "runCCLLogicTest" }}
{{- if .CclLogicTest -}}
func runCCLLogicTest(t *testing.T, file string) {
skip.UnderDeadlock(t, "times out and/or hangs")
{{if .SkipCclUnderRace}}skip.UnderRace(t, "times out and/or hangs")
{{end}}skip.UnderDeadlock(t, "times out and/or hangs")
logictest.RunLogicTest(t, logictest.TestServerArgs{}, configIdx, filepath.Join(cclLogicTestDir, file))
}
{{ end }}
Expand Down
8 changes: 0 additions & 8 deletions pkg/sql/catalog/catalogkeys/keys.go
Original file line number Diff line number Diff line change
Expand Up @@ -298,14 +298,6 @@ func MakeObjectCommentsMetadataPrefix(
return encoding.EncodeUvarintAscending(k, uint64(descID))
}

// MakeSubObjectCommentsMetadataPrefix returns the key
func MakeSubObjectCommentsMetadataPrefix(
codec keys.SQLCodec, cmtKey CommentType, descID descpb.ID, subID uint32,
) roachpb.Key {
k := MakeObjectCommentsMetadataPrefix(codec, cmtKey, descID)
return encoding.EncodeUvarintAscending(k, uint64(subID))
}

// DecodeCommentMetadataID decodes a CommentKey from comments metadata key.
func DecodeCommentMetadataID(codec keys.SQLCodec, key roachpb.Key) ([]byte, CommentKey, error) {
remaining, tableID, indexID, err := codec.DecodeIndexPrefix(key)
Expand Down
5 changes: 0 additions & 5 deletions pkg/sql/catalog/errors.go
Original file line number Diff line number Diff line change
Expand Up @@ -177,11 +177,6 @@ func AsFunctionDescriptor(desc Descriptor) (FunctionDescriptor, error) {
return typ, nil
}

// WrapDescRefErr wraps an error pertaining to a descriptor id.
func WrapDescRefErr(id descpb.ID, err error) error {
return errors.Wrapf(err, "referenced descriptor ID %d", errors.Safe(id))
}

// WrapDatabaseDescRefErr wraps an error pertaining to a database descriptor id.
func WrapDatabaseDescRefErr(id descpb.ID, err error) error {
return errors.Wrapf(err, "referenced database ID %d", errors.Safe(id))
Expand Down
35 changes: 1 addition & 34 deletions pkg/sql/catalog/schemaexpr/expr.go
Original file line number Diff line number Diff line change
Expand Up @@ -34,40 +34,6 @@ import (
"github.com/cockroachdb/errors"
)

// DequalifyAndTypeCheckExpr type checks the given expression and returns the
// type-checked expression disregarding volatility. The typed expression, which contains dummyColumns,
// does not support evaluation.
func DequalifyAndTypeCheckExpr(
ctx context.Context,
desc catalog.TableDescriptor,
expr tree.Expr,
semaCtx *tree.SemaContext,
tn *tree.TableName,
) (tree.TypedExpr, error) {
nonDropColumns := desc.NonDropColumns()
sourceInfo := colinfo.NewSourceInfoForSingleTable(
*tn, colinfo.ResultColumnsFromColumns(desc.GetID(), nonDropColumns),
)
expr, err := dequalifyColumnRefs(ctx, sourceInfo, expr)
if err != nil {
return nil, err
}

// Replace the column variables with dummyColumns so that they can be
// type-checked.
replacedExpr, _, err := replaceColumnVars(desc, expr)
if err != nil {
return nil, err
}

typedExpr, err := tree.TypeCheck(ctx, replacedExpr, semaCtx, types.Any)
if err != nil {
return nil, err
}

return typedExpr, nil
}

// DequalifyAndValidateExprImpl validates that an expression has the given type and
// contains no functions with a volatility greater than maxVolatility. The
// type-checked and constant-folded expression, the type of the expression, and
Expand All @@ -77,6 +43,7 @@ func DequalifyAndTypeCheckExpr(
// tree.TypedExpr would be dangerous. It contains dummyColumns which do not
// support evaluation and are not useful outside the context of type-checking
// the expression.
// TODO(mgartner): Rename this function without "Impl".
func DequalifyAndValidateExprImpl(
ctx context.Context,
expr tree.Expr,
Expand Down
12 changes: 0 additions & 12 deletions pkg/sql/catalog/schemaexpr/select_name_resolution.go
Original file line number Diff line number Diff line change
Expand Up @@ -112,15 +112,3 @@ func ResolveNamesUsingVisitor(
func (v *NameResolutionVisitor) Err() error {
return v.err
}

// MakeNameResolutionVisitor returns initialized name resolution visitor.
func MakeNameResolutionVisitor(
source *colinfo.DataSourceInfo, ivarHelper tree.IndexedVarHelper,
) NameResolutionVisitor {
return NameResolutionVisitor{
iVarHelper: ivarHelper,
resolver: colinfo.ColumnResolver{
Source: source,
},
}
}
5 changes: 0 additions & 5 deletions pkg/sql/catalog/table_elements.go
Original file line number Diff line number Diff line change
Expand Up @@ -970,11 +970,6 @@ func ColumnIDToOrdinalMap(columns []Column) TableColMap {
return m
}

// ColumnTypes returns the types of the given columns
func ColumnTypes(columns []Column) []*types.T {
return ColumnTypesWithInvertedCol(columns, nil /* invertedCol */)
}

// ColumnTypesWithInvertedCol returns the types of all given columns,
// If invertedCol is non-nil, substitutes the type of the inverted
// column instead of the column with the same ID.
Expand Down
15 changes: 0 additions & 15 deletions pkg/sql/parser/scanner.go
Original file line number Diff line number Diff line change
Expand Up @@ -56,21 +56,6 @@ func Tokens(sql string) (tokens []TokenString, ok bool) {
return tokens, true
}

// TokensIgnoreErrors decomposes the input into lexical tokens and
// ignores errors.
func TokensIgnoreErrors(sql string) (tokens []TokenString) {
s := makeSQLScanner(sql)
for {
var lval = &sqlSymType{}
s.Scan(lval)
if lval.ID() == 0 {
break
}
tokens = append(tokens, TokenString{TokenID: lval.ID(), Str: lval.Str()})
}
return tokens
}

// TokenString is the unit value returned by Tokens.
type TokenString struct {
TokenID int32
Expand Down
11 changes: 0 additions & 11 deletions pkg/sql/schemachanger/scdeps/sctestdeps/database_state.go
Original file line number Diff line number Diff line change
Expand Up @@ -37,17 +37,6 @@ import (
"github.com/stretchr/testify/require"
)

// WaitForNoRunningSchemaChanges schema changes waits for no schema changes
// to exist.
func WaitForNoRunningSchemaChanges(t *testing.T, tdb *sqlutils.SQLRunner) {
tdb.CheckQueryResultsRetry(t, `
SELECT count(*)
FROM [SHOW JOBS]
WHERE job_type = 'SCHEMA CHANGE'
AND status NOT IN ('succeeded', 'failed', 'aborted')`,
[][]string{{"0"}})
}

// ReadDescriptorsFromDB reads the set of descriptors from tdb.
func ReadDescriptorsFromDB(
ctx context.Context, t *testing.T, tdb *sqlutils.SQLRunner,
Expand Down
8 changes: 0 additions & 8 deletions pkg/sql/schemachanger/scexec/scmutationexec/helpers.go
Original file line number Diff line number Diff line change
Expand Up @@ -213,14 +213,6 @@ func MakeColumnIDMutationSelector(columnID descpb.ColumnID) MutationSelector {
}
}

// MakeMutationIDMutationSelector returns a MutationSelector which matches the
// first mutation with this ID.
func MakeMutationIDMutationSelector(mutationID descpb.MutationID) MutationSelector {
return func(mut catalog.Mutation) bool {
return mut.MutationID() == mutationID
}
}

// enqueueNonIndexMutation enqueues a non-index mutation `m` (of generic type M)
// with direction `dir` without increasing the next mutation ID.
// The mutation state will be DELETE_ONLY if `dir=ADD` and WRITE_ONLY if `dir=DROP`.
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -55,12 +55,6 @@ func isDescriptor(e scpb.Element) bool {
return false
}

// IsDescriptor returns true for a descriptor-element, i.e. an element which
// owns its corresponding descriptor. This is only used for exports
func IsDescriptor(e scpb.Element) bool {
return isDescriptor(e)
}

func isSubjectTo2VersionInvariant(e scpb.Element) bool {
// TODO(ajwerner): This should include constraints and enum values but it
// currently does not because we do not support dropping them unless we're
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -55,12 +55,6 @@ func isDescriptor(e scpb.Element) bool {
return false
}

// IsDescriptor returns true for a descriptor-element, i.e. an element which
// owns its corresponding descriptor. This is only used for exports
func IsDescriptor(e scpb.Element) bool {
return isDescriptor(e)
}

func isSubjectTo2VersionInvariant(e scpb.Element) bool {
// TODO(ajwerner): This should include constraints and enum values but it
// currently does not because we do not support dropping them unless we're
Expand Down
9 changes: 0 additions & 9 deletions pkg/sql/schemachanger/screl/scalars.go
Original file line number Diff line number Diff line change
Expand Up @@ -32,15 +32,6 @@ func GetDescID(e scpb.Element) catid.DescID {
return id.(catid.DescID)
}

// GetIndexID retrieves the index ID from the element if it has one.
func GetIndexID(e scpb.Element) (catid.IndexID, bool) {
v, err := Schema.GetAttribute(IndexID, e)
if err != nil {
return 0, false
}
return v.(catid.IndexID), true
}

// AllTargetStateDescIDs applies AllTargetDescIDs to the whole target state.
func AllTargetStateDescIDs(s scpb.TargetState) (ids catalog.DescriptorIDSet) {
for i := range s.Targets {
Expand Down
5 changes: 0 additions & 5 deletions pkg/sql/scrub/errors.go
Original file line number Diff line number Diff line change
Expand Up @@ -73,11 +73,6 @@ func WrapError(code string, err error) *Error {
}
}

// IsScrubError checks if an error is a Error.
func IsScrubError(err error) bool {
return errors.HasType(err, (*Error)(nil))
}

// UnwrapScrubError gets the underlying error if err is a scrub.Error.
// If err is not a scrub.Error, the error is returned unchanged.
func UnwrapScrubError(err error) error {
Expand Down
18 changes: 0 additions & 18 deletions pkg/sql/sem/plpgsqltree/utils/plpg_visitor.go
Original file line number Diff line number Diff line change
Expand Up @@ -100,24 +100,6 @@ func CountPLpgSQLStmt(sql string) (PLpgSQLStmtCounter, error) {
return v.StmtCnt, v.Err
}

// ParseAndCollectTelemetryForPLpgSQLFunc takes a plpgsql function and parses and collects
// telemetry on the parsable statements.
func ParseAndCollectTelemetryForPLpgSQLFunc(stmt *tree.CreateRoutine) error {
// Assert that the function language is PLPGSQL.
var funcBodyStr string
for _, option := range stmt.Options {
switch opt := option.(type) {
case tree.RoutineBodyStr:
funcBodyStr = string(opt)
}
}

if _, err := CountPLpgSQLStmt(funcBodyStr); err != nil {
return errors.Wrap(err, "plpgsql not supported in user-defined functions")
}
return unimp.New("plpgsql", "plpgsql not supported in user-defined functions")
}

// SQLStmtVisitor calls Fn for every SQL statement and expression found while
// walking the PLpgSQL AST. Since PLpgSQL nodes may have statement and
// expression fields that are nil, Fn should handle the nil case.
Expand Down
7 changes: 0 additions & 7 deletions pkg/sql/sqlerrors/errors.go
Original file line number Diff line number Diff line change
Expand Up @@ -193,13 +193,6 @@ func NewUnsupportedUnvalidatedConstraintError(constraintType catconstants.Constr
"%v constraints cannot be marked NOT VALID", constraintType)
}

// WrapErrorWhileConstructingObjectAlreadyExistsErr is used to wrap an error
// when an error occurs while trying to get the colliding object for an
// ObjectAlreadyExistsErr.
func WrapErrorWhileConstructingObjectAlreadyExistsErr(err error) error {
return pgerror.WithCandidateCode(errors.Wrap(err, "object already exists"), pgcode.DuplicateObject)
}

// MakeObjectAlreadyExistsError creates an error for a namespace collision
// with an arbitrary descriptor type.
func MakeObjectAlreadyExistsError(collidingObject *descpb.Descriptor, name string) error {
Expand Down
Loading

0 comments on commit 8a34679

Please sign in to comment.