Skip to content

Commit

Permalink
Browse files Browse the repository at this point in the history
86563: ts: fix the pretty-printing of tsd keys r=abarganier a=knz

Found while working on #86524.

Release justification: bug fix

Release note (bug fix): When printing keys and range start/end
boundaries for time series, the displayed structure of keys
was incorrect. This is now fixed.

86904: sql: allow mismatch type numbers in `PREPARE` statement r=rafiss a=ZhouXing19

Previously, we only allow having the same number of parameters and placeholders
in a `PREPARE` statement. This is not compatible with Postgres14's behavior.

This commit is to loosen the restriction and enable this compatibility.
We now take `max(#placeholders, #parameters)` as the true length
 of parameters of the prepare statement. For each parameter, we first
look at the type deduced from the query stmt. If we can't deduce it, 
we take the type hint for this param.

I.e. we now allow queries such as 

```
PREPARE args_test_many(int, int) as select $1
// 2 parameters, but only 1 placeholder in the query.

PREPARE args_test_few(int) as select $1, $2::int
// 1 parameter, but 2 placeholders in the query.
```

fixes #86375

Release justification: Low risk, high benefit changes to existing functionality
Release note: allow mismatch type numbers in `PREPARE` statement

87105: roachtest: skip flaky acceptance/version-upgrade r=tbg a=adityamaru

Skipping the flaky roachtest while we stabilize it.

Informs: #87104

Release note: None

Release justification: testing only change

87117: bazci: fix output path computation r=rail a=rickystewart

These updates were happening in-place so `bazci` was constructing big,
silly paths like `backupccl_test/shard_6_of_16/shard_7_of_16/shard_13_of_16/...`
We just need to copy the variable here.

Release justification: Non-production code changes
Release note: None

Co-authored-by: Raphael 'kena' Poss <[email protected]>
Co-authored-by: Jane Xing <[email protected]>
Co-authored-by: adityamaru <[email protected]>
Co-authored-by: Ricky Stewart <[email protected]>
  • Loading branch information
5 people committed Aug 30, 2022
5 parents f4b491f + 97b3477 + be62a76 + 87b6b07 + 90b139c commit 716626e
Show file tree
Hide file tree
Showing 18 changed files with 151 additions and 45 deletions.
1 change: 1 addition & 0 deletions pkg/cmd/bazci/bazci.go
Original file line number Diff line number Diff line change
Expand Up @@ -185,6 +185,7 @@ func processBuildEventProtocolLog(action, bepLoc string) error {
outputDir = filepath.Join("bazel-testlogs", outputDir)
summary := event.GetTestSummary()
for _, testResult := range testResults[label] {
outputDir := outputDir
if testResult.run > 1 {
outputDir = filepath.Join(outputDir, fmt.Sprintf("run_%d", testResult.run))
}
Expand Down
1 change: 1 addition & 0 deletions pkg/cmd/roachtest/tests/acceptance.go
Original file line number Diff line number Diff line change
Expand Up @@ -75,6 +75,7 @@ func registerAcceptance(r registry.Registry) {
// to head after 19.2 fails.
minVersion: "v19.2.0",
timeout: 30 * time.Minute,
skip: "https://github.com/cockroachdb/cockroach/issues/87104",
},
},
}
Expand Down
11 changes: 6 additions & 5 deletions pkg/sql/conn_executor_exec.go
Original file line number Diff line number Diff line change
Expand Up @@ -562,12 +562,13 @@ func (ex *connExecutor) execStmtInOpenState(
return makeErrEvent(err)
}
var typeHints tree.PlaceholderTypes
// We take max(len(s.Types), stmt.NumPlaceHolders) as the length of types.
numParams := len(s.Types)
if stmt.NumPlaceholders > numParams {
numParams = stmt.NumPlaceholders
}
if len(s.Types) > 0 {
if len(s.Types) > stmt.NumPlaceholders {
err := pgerror.Newf(pgcode.Syntax, "too many types provided")
return makeErrEvent(err)
}
typeHints = make(tree.PlaceholderTypes, stmt.NumPlaceholders)
typeHints = make(tree.PlaceholderTypes, numParams)
for i, t := range s.Types {
resolved, err := tree.ResolveType(ctx, t, ex.planner.semaCtx.GetTypeResolver())
if err != nil {
Expand Down
15 changes: 12 additions & 3 deletions pkg/sql/conn_executor_prepare.go
Original file line number Diff line number Diff line change
Expand Up @@ -240,7 +240,7 @@ func (ex *connExecutor) prepare(

p.stmt = stmt
p.semaCtx.Annotations = tree.MakeAnnotations(stmt.NumAnnotations)
flags, err = ex.populatePrepared(ctx, txn, placeholderHints, p)
flags, err = ex.populatePrepared(ctx, txn, placeholderHints, p, origin)
return err
}

Expand All @@ -260,15 +260,24 @@ func (ex *connExecutor) prepare(
// populatePrepared analyzes and type-checks the query and populates
// stmt.Prepared.
func (ex *connExecutor) populatePrepared(
ctx context.Context, txn *kv.Txn, placeholderHints tree.PlaceholderTypes, p *planner,
ctx context.Context,
txn *kv.Txn,
placeholderHints tree.PlaceholderTypes,
p *planner,
origin PreparedStatementOrigin,
) (planFlags, error) {
if before := ex.server.cfg.TestingKnobs.BeforePrepare; before != nil {
if err := before(ctx, ex.planner.stmt.String(), txn); err != nil {
return 0, err
}
}
stmt := &p.stmt
if err := p.semaCtx.Placeholders.Init(stmt.NumPlaceholders, placeholderHints); err != nil {
var fromSQL bool
if origin == PreparedStatementOriginSQL {
fromSQL = true
}

if err := p.semaCtx.Placeholders.Init(stmt.NumPlaceholders, placeholderHints, fromSQL); err != nil {
return 0, err
}
p.extendedEvalCtx.PrepareOnly = true
Expand Down
32 changes: 30 additions & 2 deletions pkg/sql/logictest/testdata/logic_test/pg_catalog
Original file line number Diff line number Diff line change
Expand Up @@ -4729,6 +4729,34 @@ select name, statement, parameter_types, from_sql from pg_prepared_statements OR
test_insert_statement PREPARE test_insert_statement (int, timestamptz) AS INSERT INTO types VALUES ($2, $1) {bigint,"'timestamp with time zone'"} true
test_select_statement PREPARE test_select_statement AS SELECT * FROM types {} true

subtest pg_catalog.pg_prepare_statement,with_possible_mismatch_num_types

statement ok
PREPARE args_test_many(int, int) as select $1

statement ok
PREPARE args_test_few(int) as select $1, $2::int

statement ok
DROP TABLE IF EXISTS t_prepare;

statement ok
CREATE TABLE t_prepare (x int, y varchar(10), z int2);

statement ok
PREPARE args_deduce_type(int, int, int, int) AS INSERT INTO t_prepare VALUES ($1, $2, $3);

statement ok
PREPARE args_deduce_type_1(int) AS SELECT $1::int, $2::varchar(10), $3::varchar(20);

query TTTB
SELECT name, statement, parameter_types, from_sql FROM pg_prepared_statements WHERE name LIKE 'args_%' ORDER BY 1,2
----
args_deduce_type PREPARE args_deduce_type (int, varchar, int, int) AS INSERT INTO t_prepare VALUES ($1, $2, $3) {bigint,"'character varying'",bigint,bigint} true
args_deduce_type_1 PREPARE args_deduce_type_1 (int, varchar, varchar) AS SELECT $1::INT8, $2::VARCHAR(10), $3::VARCHAR(20) {bigint,"'character varying'","'character varying'"} true
args_test_few PREPARE args_test_few (int, int) AS SELECT $1, $2::INT8 {bigint,bigint} true
args_test_many PREPARE args_test_many (int, int) AS SELECT $1 {bigint,bigint} true

statement ok
DROP TABLE types

Expand Down Expand Up @@ -5130,13 +5158,13 @@ CREATE TABLE jt (a INT PRIMARY KEY); INSERT INTO jt VALUES(1); INSERT INTO jt VA
query ITT
SELECT a, oid, relname FROM jt INNER LOOKUP JOIN pg_class ON a::oid=oid
----
167 167 jt
168 168 jt

query ITT
SELECT a, oid, relname FROM jt LEFT OUTER LOOKUP JOIN pg_class ON a::oid=oid
----
1 NULL NULL
167 167 jt
168 168 jt

subtest regression_49207
statement ok
Expand Down
51 changes: 51 additions & 0 deletions pkg/sql/logictest/testdata/logic_test/prepare
Original file line number Diff line number Diff line change
Expand Up @@ -1431,3 +1431,54 @@ query II
EXECUTE q81315_2(1::DECIMAL)
----
1 1

subtest possible_mismatch_num_types

statement ok
PREPARE args_test_many(int, int) as select $1

query I
EXECUTE args_test_many(1, 2)
----
1

query error wrong number of parameters for prepared statement "args_test_many": expected 2, got 1
EXECUTE args_test_many(1)

statement ok
PREPARE args_test_few(int) as select $1, $2::int

query II
EXECUTE args_test_few(1, 2)
----
1 2

query error wrong number of parameters for prepared statement "args_test_few": expected 2, got 1
EXECUTE args_test_few(1)

statement ok
DROP TABLE IF EXISTS t;

statement ok
CREATE TABLE t (x int, y varchar(10), z int2);

statement ok
PREPARE args_deduce_type(int, int, int, int) AS INSERT INTO t VALUES ($1, $2, $3);

statement ok
EXECUTE args_deduce_type(1,2,3,4);
EXECUTE args_deduce_type('1','2',3,'4');

query ITI
SELECT * FROM t;
----
1 2 3
1 2 3

statement ok
PREPARE args_deduce_type_1(int) AS SELECT $1::int, $2::varchar(10), $3::varchar(20);

query ITT
EXECUTE args_deduce_type_1(1,10,100);
----
1 10 100
2 changes: 1 addition & 1 deletion pkg/sql/opt/bench/bench_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -600,7 +600,7 @@ func newHarness(tb testing.TB, query benchQuery, schemas []string) *harness {
}
}

if err := h.semaCtx.Placeholders.Init(len(query.args), nil /* typeHints */); err != nil {
if err := h.semaCtx.Placeholders.Init(len(query.args), nil /* typeHints */, false /* fromSQL */); err != nil {
tb.Fatal(err)
}
// Run optbuilder to build the memo for Prepare. Even if we will not be using
Expand Down
2 changes: 1 addition & 1 deletion pkg/sql/opt/testutils/build.go
Original file line number Diff line number Diff line change
Expand Up @@ -36,7 +36,7 @@ func BuildQuery(

ctx := context.Background()
semaCtx := tree.MakeSemaContext()
if err := semaCtx.Placeholders.Init(stmt.NumPlaceholders, nil /* typeHints */); err != nil {
if err := semaCtx.Placeholders.Init(stmt.NumPlaceholders, nil /* typeHints */, false /* fromSQL */); err != nil {
t.Fatal(err)
}
semaCtx.Annotations = tree.MakeAnnotations(stmt.NumAnnotations)
Expand Down
2 changes: 1 addition & 1 deletion pkg/sql/opt/testutils/opttester/opt_tester.go
Original file line number Diff line number Diff line change
Expand Up @@ -2174,7 +2174,7 @@ func (ot *OptTester) buildExpr(factory *norm.Factory) error {
if err != nil {
return err
}
if err := ot.semaCtx.Placeholders.Init(stmt.NumPlaceholders, nil /* typeHints */); err != nil {
if err := ot.semaCtx.Placeholders.Init(stmt.NumPlaceholders, nil /* typeHints */, false /* fromSQL */); err != nil {
return err
}
ot.semaCtx.Annotations = tree.MakeAnnotations(stmt.NumAnnotations)
Expand Down
5 changes: 5 additions & 0 deletions pkg/sql/plan_opt.go
Original file line number Diff line number Diff line change
Expand Up @@ -187,6 +187,11 @@ func (p *planner) prepareUsingOptimizer(ctx context.Context) (planFlags, error)
}
}

if p.semaCtx.Placeholders.PlaceholderTypesInfo.FromSQLPrepare {
// Fill blank placeholder types with the type hints.
p.semaCtx.Placeholders.MaybeExtendTypes()
}

// Verify that all placeholder types have been set.
if err := p.semaCtx.Placeholders.Types.AssertAllSet(); err != nil {
return 0, err
Expand Down
2 changes: 1 addition & 1 deletion pkg/sql/schemachange/alter_column_type.go
Original file line number Diff line number Diff line change
Expand Up @@ -209,7 +209,7 @@ func ClassifyConversion(

// See if there's existing cast logic. If so, return general.
semaCtx := tree.MakeSemaContext()
if err := semaCtx.Placeholders.Init(1 /* numPlaceholders */, nil /* typeHints */); err != nil {
if err := semaCtx.Placeholders.Init(1 /* numPlaceholders */, nil /* typeHints */, false /* fromSQL */); err != nil {
return ColumnConversionImpossible, err
}

Expand Down
2 changes: 1 addition & 1 deletion pkg/sql/sem/tree/overload_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -269,7 +269,7 @@ func TestTypeCheckOverloadedExprs(t *testing.T) {
for i, d := range testData {
t.Run(fmt.Sprintf("%v/%v", d.exprs, d.overloads), func(t *testing.T) {
semaCtx := MakeSemaContext()
if err := semaCtx.Placeholders.Init(2 /* numPlaceholders */, nil /* typeHints */); err != nil {
if err := semaCtx.Placeholders.Init(2 /* numPlaceholders */, nil /* typeHints */, false /* fromSQL */); err != nil {
t.Fatal(err)
}
desired := types.Any
Expand Down
45 changes: 26 additions & 19 deletions pkg/sql/sem/tree/placeholders.go
Original file line number Diff line number Diff line change
Expand Up @@ -96,6 +96,10 @@ type PlaceholderTypesInfo struct {
// Types contains the final types set for each placeholder after type
// checking.
Types PlaceholderTypes

// FromSQLPrepare is true when the placeholder is in a statement from a
// PREPARE SQL stmt (rather than a pgwire prepare stmt).
FromSQLPrepare bool
}

// Type returns the known type of a placeholder. If there is no known type yet
Expand Down Expand Up @@ -150,44 +154,47 @@ type PlaceholderInfo struct {

// Init initializes a PlaceholderInfo structure appropriate for the given number
// of placeholders, and with the given (optional) type hints.
func (p *PlaceholderInfo) Init(numPlaceholders int, typeHints PlaceholderTypes) error {
p.Types = make(PlaceholderTypes, numPlaceholders)
func (p *PlaceholderInfo) Init(
numPlaceholders int, typeHints PlaceholderTypes, fromSQL bool,
) error {
if fromSQL {
if typeHints == nil { // This should not happen, but...
return errors.AssertionFailedf("There should be at least one type hint for a sql-level PREPARE statement")
}
p.Types = make(PlaceholderTypes, len(typeHints))
} else {
p.Types = make(PlaceholderTypes, numPlaceholders)
}

if typeHints == nil {
p.TypeHints = make(PlaceholderTypes, numPlaceholders)
} else {
if err := checkPlaceholderArity(len(typeHints), numPlaceholders); err != nil {
return err
}
p.TypeHints = typeHints
}
p.Values = nil
p.FromSQLPrepare = fromSQL
return nil
}

// Assign resets the PlaceholderInfo to the contents of src.
// If src is nil, a new structure is initialized.
func (p *PlaceholderInfo) Assign(src *PlaceholderInfo, numPlaceholders int) error {
if src != nil {
if err := checkPlaceholderArity(len(src.Types), numPlaceholders); err != nil {
return err
}
*p = *src
return nil
}
return p.Init(numPlaceholders, nil /* typeHints */)
return p.Init(numPlaceholders, nil /* typeHints */, false /* fromSQL */)
}

func checkPlaceholderArity(numTypes, numPlaceholders int) error {
if numTypes > numPlaceholders {
return errors.AssertionFailedf(
"unexpected placeholder types: got %d, expected %d",
numTypes, numPlaceholders)
} else if numTypes < numPlaceholders {
return pgerror.Newf(pgcode.UndefinedParameter,
"could not find types for all placeholders: got %d, expected %d",
numTypes, numPlaceholders)
// MaybeExtendTypes is to fill the nil types with the type hints, if exists.
func (p *PlaceholderInfo) MaybeExtendTypes() {
if len(p.TypeHints) >= len(p.Types) {
for i, t := range p.Types {
if t == nil {
p.Types[i] = p.TypeHints[i]
}
}
}
return nil
}

// Value returns the known value of a placeholder. Returns false in
Expand Down
6 changes: 3 additions & 3 deletions pkg/sql/sem/tree/type_check_internal_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -37,7 +37,7 @@ func BenchmarkTypeCheck(b *testing.B) {
b.Fatalf("%s: %v", expr, err)
}
ctx := tree.MakeSemaContext()
if err := ctx.Placeholders.Init(1 /* numPlaceholders */, nil /* typeHints */); err != nil {
if err := ctx.Placeholders.Init(1 /* numPlaceholders */, nil /* typeHints */, false /* fromSQL */); err != nil {
b.Fatal(err)
}
for i := 0; i < b.N; i++ {
Expand Down Expand Up @@ -183,7 +183,7 @@ func attemptTypeCheckSameTypedExprs(t *testing.T, idx int, test sameTypedExprsTe
ctx := context.Background()
forEachPerm(test.exprs, 0, func(exprs []copyableExpr) {
semaCtx := tree.MakeSemaContext()
if err := semaCtx.Placeholders.Init(len(test.ptypes), clonePlaceholderTypes(test.ptypes)); err != nil {
if err := semaCtx.Placeholders.Init(len(test.ptypes), clonePlaceholderTypes(test.ptypes), false /* fromSQL */); err != nil {
t.Fatal(err)
}
desired := types.Any
Expand Down Expand Up @@ -337,7 +337,7 @@ func TestTypeCheckSameTypedExprsError(t *testing.T) {
for i, d := range testData {
t.Run(fmt.Sprintf("test_%d", i), func(t *testing.T) {
semaCtx := tree.MakeSemaContext()
if err := semaCtx.Placeholders.Init(len(d.ptypes), d.ptypes); err != nil {
if err := semaCtx.Placeholders.Init(len(d.ptypes), d.ptypes, false /* fromSQL */); err != nil {
t.Error(err)
}
desired := types.Any
Expand Down
4 changes: 2 additions & 2 deletions pkg/sql/sem/tree/type_check_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -221,7 +221,7 @@ func TestTypeCheck(t *testing.T) {
t.Fatalf("%s: %v", d.expr, err)
}
semaCtx := tree.MakeSemaContext()
if err := semaCtx.Placeholders.Init(1 /* numPlaceholders */, nil /* typeHints */); err != nil {
if err := semaCtx.Placeholders.Init(1 /* numPlaceholders */, nil /* typeHints */, false /* fromSQL */); err != nil {
t.Fatal(err)
}
semaCtx.TypeResolver = mapResolver
Expand Down Expand Up @@ -398,7 +398,7 @@ func TestTypeCheckVolatility(t *testing.T) {

ctx := context.Background()
semaCtx := tree.MakeSemaContext()
if err := semaCtx.Placeholders.Init(len(placeholderTypes), placeholderTypes); err != nil {
if err := semaCtx.Placeholders.Init(len(placeholderTypes), placeholderTypes, false /* fromSQL */); err != nil {
t.Fatal(err)
}

Expand Down
5 changes: 4 additions & 1 deletion pkg/sql/session_state.go
Original file line number Diff line number Diff line change
Expand Up @@ -182,7 +182,10 @@ func (p *planner) DeserializeSessionState(state *tree.DBytes) (*tree.DBool, erro

_, err = evalCtx.statementPreparer.addPreparedStmt(
evalCtx.Ctx(),
prepStmt.Name, stmt, placeholderTypes, prepStmt.PlaceholderTypeHints,
prepStmt.Name,
stmt,
placeholderTypes,
prepStmt.PlaceholderTypeHints,
PreparedStatementOriginSessionMigration,
)
if err != nil {
Expand Down
4 changes: 2 additions & 2 deletions pkg/ts/keys.go
Original file line number Diff line number Diff line change
Expand Up @@ -121,8 +121,8 @@ func prettyPrintKey(key roachpb.Key) string {
// it.
return encoding.PrettyPrintValue(nil /* dirs */, key, "/")
}
return fmt.Sprintf("/%s/%s/%s/%s", name, source, resolution,
timeutil.Unix(0, timestamp).Format(time.RFC3339Nano))
return fmt.Sprintf("/%s/%s/%s/%s", name, resolution,
timeutil.Unix(0, timestamp).Format(time.RFC3339Nano), source)
}

func init() {
Expand Down
Loading

0 comments on commit 716626e

Please sign in to comment.