Skip to content

Commit

Permalink
Merge #72587
Browse files Browse the repository at this point in the history
72587: sql: plumb sessiondata into schemaexpr.FormatExprForDisplay r=otan,mgartner a=rafiss

fixes #71776

Release note (bug fix): Previously, introspection tables and error
messages would not correctly display intervals according to the
`intervalstyle` session variable. This is fixed now.

Co-authored-by: Rafi Shamim <[email protected]>
  • Loading branch information
craig[bot] and rafiss committed Nov 11, 2021
2 parents 503f136 + 688c3a1 commit dbad874
Show file tree
Hide file tree
Showing 18 changed files with 125 additions and 62 deletions.
6 changes: 4 additions & 2 deletions pkg/ccl/importccl/read_import_mysql_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -31,6 +31,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/sql/execinfrapb"
"github.com/cockroachdb/cockroach/pkg/sql/row"
"github.com/cockroachdb/cockroach/pkg/sql/sem/tree"
"github.com/cockroachdb/cockroach/pkg/sql/sessiondata"
"github.com/cockroachdb/cockroach/pkg/sql/types"
"github.com/cockroachdb/cockroach/pkg/util/leaktest"
"github.com/cockroachdb/cockroach/pkg/util/log"
Expand Down Expand Up @@ -224,20 +225,21 @@ func compareTables(t *testing.T, expected, got *descpb.TableDescriptor) {
len(expectedIdx), idxNames(expectedIdx), len(gotIdx), idxNames(gotIdx),
)
}
sd := &sessiondata.SessionData{}
for i := range expected.Indexes {
ctx := context.Background()
semaCtx := tree.MakeSemaContext()
tableName := &descpb.AnonymousTable
expectedDesc := tabledesc.NewBuilder(expected).BuildImmutableTable()
gotDesc := tabledesc.NewBuilder(got).BuildImmutableTable()
e, err := catformat.IndexForDisplay(
ctx, expectedDesc, tableName, expectedDesc.PublicNonPrimaryIndexes()[i], "" /* partition */, &semaCtx,
ctx, expectedDesc, tableName, expectedDesc.PublicNonPrimaryIndexes()[i], "" /* partition */, &semaCtx, sd,
)
if err != nil {
t.Fatalf("unexpected error: %s", err)
}
g, err := catformat.IndexForDisplay(
ctx, gotDesc, tableName, gotDesc.PublicNonPrimaryIndexes()[i], "" /* partition */, &semaCtx,
ctx, gotDesc, tableName, gotDesc.PublicNonPrimaryIndexes()[i], "" /* partition */, &semaCtx, sd,
)
if err != nil {
t.Fatalf("unexpected error: %s", err)
Expand Down
2 changes: 1 addition & 1 deletion pkg/sql/alter_table.go
Original file line number Diff line number Diff line change
Expand Up @@ -818,7 +818,7 @@ func (n *alterTableNode) startExec(params runParams) error {
"constraint %q in the middle of being added, try again later", t.Constraint)
}
if err := validateCheckInTxn(
params.ctx, params.p.LeaseMgr(), &params.p.semaCtx, params.EvalContext(), n.tableDesc, params.EvalContext().Txn, ck.Expr,
params.ctx, &params.p.semaCtx, params.EvalContext(), n.tableDesc, params.EvalContext().Txn, ck.Expr,
); err != nil {
return err
}
Expand Down
13 changes: 8 additions & 5 deletions pkg/sql/backfill.go
Original file line number Diff line number Diff line change
Expand Up @@ -715,7 +715,9 @@ func (sc *SchemaChanger) validateConstraints(
// after the check is validated.
defer func() { collection.ReleaseAll(ctx) }()
if c.IsCheck() {
if err := validateCheckInTxn(ctx, sc.leaseMgr, &semaCtx, &evalCtx.EvalContext, desc, txn, c.Check().Expr); err != nil {
if err := validateCheckInTxn(
ctx, &semaCtx, &evalCtx.EvalContext, desc, txn, c.Check().Expr,
); err != nil {
return err
}
} else if c.IsForeignKey() {
Expand All @@ -727,7 +729,9 @@ func (sc *SchemaChanger) validateConstraints(
return err
}
} else if c.IsNotNull() {
if err := validateCheckInTxn(ctx, sc.leaseMgr, &semaCtx, &evalCtx.EvalContext, desc, txn, c.Check().Expr); err != nil {
if err := validateCheckInTxn(
ctx, &semaCtx, &evalCtx.EvalContext, desc, txn, c.Check().Expr,
); err != nil {
// TODO (lucy): This should distinguish between constraint
// validation errors and other types of unexpected errors, and
// return a different error code in the former case
Expand Down Expand Up @@ -2056,7 +2060,7 @@ func runSchemaChangesInTxn(
check := &c.ConstraintToUpdateDesc().Check
if check.Validity == descpb.ConstraintValidity_Validating {
if err := validateCheckInTxn(
ctx, planner.LeaseMgr(), &planner.semaCtx, planner.EvalContext(), tableDesc, planner.txn, check.Expr,
ctx, &planner.semaCtx, planner.EvalContext(), tableDesc, planner.txn, check.Expr,
); err != nil {
return err
}
Expand Down Expand Up @@ -2152,7 +2156,6 @@ func runSchemaChangesInTxn(
// reuse an existing kv.Txn safely.
func validateCheckInTxn(
ctx context.Context,
leaseMgr *lease.Manager,
semaCtx *tree.SemaContext,
evalCtx *tree.EvalContext,
tableDesc *tabledesc.Mutable,
Expand All @@ -2165,7 +2168,7 @@ func validateCheckInTxn(
syntheticDescs = append(syntheticDescs, tableDesc)
}
return ie.WithSyntheticDescriptors(syntheticDescs, func() error {
return validateCheckExpr(ctx, semaCtx, checkExpr, tableDesc, ie, txn)
return validateCheckExpr(ctx, semaCtx, evalCtx.SessionData(), checkExpr, tableDesc, ie, txn)
})
}

Expand Down
2 changes: 2 additions & 0 deletions pkg/sql/catalog/catformat/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -11,6 +11,7 @@ go_library(
"//pkg/sql/catalog/descpb",
"//pkg/sql/catalog/schemaexpr",
"//pkg/sql/sem/tree",
"//pkg/sql/sessiondata",
"@com_github_cockroachdb_errors//:errors",
],
)
Expand All @@ -24,6 +25,7 @@ go_test(
"//pkg/sql/catalog/descpb",
"//pkg/sql/catalog/tabledesc",
"//pkg/sql/sem/tree",
"//pkg/sql/sessiondata",
"//pkg/sql/types",
],
)
14 changes: 8 additions & 6 deletions pkg/sql/catalog/catformat/index.go
Original file line number Diff line number Diff line change
Expand Up @@ -20,6 +20,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb"
"github.com/cockroachdb/cockroach/pkg/sql/catalog/schemaexpr"
"github.com/cockroachdb/cockroach/pkg/sql/sem/tree"
"github.com/cockroachdb/cockroach/pkg/sql/sessiondata"
"github.com/cockroachdb/errors"
)

Expand All @@ -43,8 +44,9 @@ func IndexForDisplay(
index catalog.Index,
partition string,
semaCtx *tree.SemaContext,
sessionData *sessiondata.SessionData,
) (string, error) {
return indexForDisplay(ctx, table, tableName, index.IndexDesc(), index.Primary(), partition, semaCtx)
return indexForDisplay(ctx, table, tableName, index.IndexDesc(), index.Primary(), partition, semaCtx, sessionData)
}

func indexForDisplay(
Expand All @@ -55,6 +57,7 @@ func indexForDisplay(
isPrimary bool,
partition string,
semaCtx *tree.SemaContext,
sessionData *sessiondata.SessionData,
) (string, error) {
f := tree.NewFmtCtx(tree.FmtSimple)
if index.Unique {
Expand All @@ -70,7 +73,7 @@ func indexForDisplay(
f.FormatNode(tableName)
}
f.WriteString(" (")
if err := FormatIndexElements(ctx, table, index, f, semaCtx); err != nil {
if err := FormatIndexElements(ctx, table, index, f, semaCtx, sessionData); err != nil {
return "", err
}
f.WriteByte(')')
Expand Down Expand Up @@ -99,9 +102,7 @@ func indexForDisplay(

if index.IsPartial() {
f.WriteString(" WHERE ")
pred, err := schemaexpr.FormatExprForDisplay(
ctx, table, index.Predicate, semaCtx, tree.FmtParsable,
)
pred, err := schemaexpr.FormatExprForDisplay(ctx, table, index.Predicate, semaCtx, sessionData, tree.FmtParsable)
if err != nil {
return "", err
}
Expand All @@ -122,6 +123,7 @@ func FormatIndexElements(
index *descpb.IndexDescriptor,
f *tree.FmtCtx,
semaCtx *tree.SemaContext,
sessionData *sessiondata.SessionData,
) error {
startIdx := index.ExplicitColumnStartIdx()
for i, n := startIdx, len(index.KeyColumnIDs); i < n; i++ {
Expand All @@ -134,7 +136,7 @@ func FormatIndexElements(
}
if col.IsExpressionIndexColumn() {
expr, err := schemaexpr.FormatExprForExpressionIndexDisplay(
ctx, table, col.GetComputeExpr(), semaCtx, tree.FmtParsable,
ctx, table, col.GetComputeExpr(), semaCtx, sessionData, tree.FmtParsable,
)
if err != nil {
return err
Expand Down
4 changes: 3 additions & 1 deletion pkg/sql/catalog/catformat/index_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -18,6 +18,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb"
"github.com/cockroachdb/cockroach/pkg/sql/catalog/tabledesc"
"github.com/cockroachdb/cockroach/pkg/sql/sem/tree"
"github.com/cockroachdb/cockroach/pkg/sql/sessiondata"
"github.com/cockroachdb/cockroach/pkg/sql/types"
)

Expand Down Expand Up @@ -122,10 +123,11 @@ func TestIndexForDisplay(t *testing.T) {
},
}

sd := &sessiondata.SessionData{}
for testIdx, tc := range testData {
t.Run(strconv.Itoa(testIdx), func(t *testing.T) {
got, err := indexForDisplay(
ctx, tableDesc, &tc.tableName, &tc.index, false /* isPrimary */, tc.partition, &semaCtx,
ctx, tableDesc, &tc.tableName, &tc.index, false /* isPrimary */, tc.partition, &semaCtx, sd,
)
if err != nil {
t.Fatalf("unexpected error: %s", err)
Expand Down
13 changes: 9 additions & 4 deletions pkg/sql/catalog/schemaexpr/column.go
Original file line number Diff line number Diff line change
Expand Up @@ -20,6 +20,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgcode"
"github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror"
"github.com/cockroachdb/cockroach/pkg/sql/sem/tree"
"github.com/cockroachdb/cockroach/pkg/sql/sessiondata"
"github.com/cockroachdb/cockroach/pkg/sql/types"
)

Expand Down Expand Up @@ -78,7 +79,11 @@ func dequalifyColumnRefs(
// converts user defined types in default and computed expressions to a
// human-readable form.
func FormatColumnForDisplay(
ctx context.Context, tbl catalog.TableDescriptor, col catalog.Column, semaCtx *tree.SemaContext,
ctx context.Context,
tbl catalog.TableDescriptor,
col catalog.Column,
semaCtx *tree.SemaContext,
sessionData *sessiondata.SessionData,
) (string, error) {
f := tree.NewFmtCtx(tree.FmtSimple)
name := col.GetName()
Expand Down Expand Up @@ -108,7 +113,7 @@ func FormatColumnForDisplay(

} else {
f.WriteString(" DEFAULT ")
defExpr, err := FormatExprForDisplay(ctx, tbl, col.GetDefaultExpr(), semaCtx, tree.FmtParsable)
defExpr, err := FormatExprForDisplay(ctx, tbl, col.GetDefaultExpr(), semaCtx, sessionData, tree.FmtParsable)
if err != nil {
return "", err
}
Expand All @@ -117,15 +122,15 @@ func FormatColumnForDisplay(
}
if col.HasOnUpdate() {
f.WriteString(" ON UPDATE ")
onUpdateExpr, err := FormatExprForDisplay(ctx, tbl, col.GetOnUpdateExpr(), semaCtx, tree.FmtParsable)
onUpdateExpr, err := FormatExprForDisplay(ctx, tbl, col.GetOnUpdateExpr(), semaCtx, sessionData, tree.FmtParsable)
if err != nil {
return "", err
}
f.WriteString(onUpdateExpr)
}
if col.IsComputed() {
f.WriteString(" AS (")
compExpr, err := FormatExprForDisplay(ctx, tbl, col.GetComputeExpr(), semaCtx, tree.FmtParsable)
compExpr, err := FormatExprForDisplay(ctx, tbl, col.GetComputeExpr(), semaCtx, sessionData, tree.FmtParsable)
if err != nil {
return "", err
}
Expand Down
8 changes: 7 additions & 1 deletion pkg/sql/catalog/schemaexpr/expr.go
Original file line number Diff line number Diff line change
Expand Up @@ -21,6 +21,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgcode"
"github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror"
"github.com/cockroachdb/cockroach/pkg/sql/sem/tree"
"github.com/cockroachdb/cockroach/pkg/sql/sessiondata"
"github.com/cockroachdb/cockroach/pkg/sql/types"
"github.com/cockroachdb/errors"
)
Expand Down Expand Up @@ -160,13 +161,15 @@ func FormatExprForDisplay(
desc catalog.TableDescriptor,
exprStr string,
semaCtx *tree.SemaContext,
sessionData *sessiondata.SessionData,
fmtFlags tree.FmtFlags,
) (string, error) {
return formatExprForDisplayImpl(
ctx,
desc,
exprStr,
semaCtx,
sessionData,
fmtFlags,
false, /* wrapNonFuncExprs */
)
Expand All @@ -181,13 +184,15 @@ func FormatExprForExpressionIndexDisplay(
desc catalog.TableDescriptor,
exprStr string,
semaCtx *tree.SemaContext,
sessionData *sessiondata.SessionData,
fmtFlags tree.FmtFlags,
) (string, error) {
return formatExprForDisplayImpl(
ctx,
desc,
exprStr,
semaCtx,
sessionData,
fmtFlags,
true, /* wrapNonFuncExprs */
)
Expand All @@ -198,6 +203,7 @@ func formatExprForDisplayImpl(
desc catalog.TableDescriptor,
exprStr string,
semaCtx *tree.SemaContext,
sessionData *sessiondata.SessionData,
fmtFlags tree.FmtFlags,
wrapNonFuncExprs bool,
) (string, error) {
Expand All @@ -210,7 +216,7 @@ func formatExprForDisplayImpl(
if err != nil {
return "", err
}
f := tree.NewFmtCtx(fmtFlags)
f := tree.NewFmtCtx(fmtFlags, tree.FmtDataConversionConfig(sessionData.DataConversionConfig))
_, isFunc := expr.(*tree.FuncExpr)
if wrapNonFuncExprs && !isFunc {
f.WriteByte('(')
Expand Down
9 changes: 7 additions & 2 deletions pkg/sql/check.go
Original file line number Diff line number Diff line change
Expand Up @@ -26,6 +26,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgcode"
"github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror"
"github.com/cockroachdb/cockroach/pkg/sql/sem/tree"
"github.com/cockroachdb/cockroach/pkg/sql/sessiondata"
"github.com/cockroachdb/cockroach/pkg/util"
"github.com/cockroachdb/cockroach/pkg/util/log"
"github.com/cockroachdb/errors"
Expand All @@ -39,12 +40,13 @@ import (
func validateCheckExpr(
ctx context.Context,
semaCtx *tree.SemaContext,
sessionData *sessiondata.SessionData,
exprStr string,
tableDesc *tabledesc.Mutable,
ie *InternalExecutor,
txn *kv.Txn,
) error {
expr, err := schemaexpr.FormatExprForDisplay(ctx, tableDesc, exprStr, semaCtx, tree.FmtParsable)
expr, err := schemaexpr.FormatExprForDisplay(ctx, tableDesc, exprStr, semaCtx, sessionData, tree.FmtParsable)
if err != nil {
return err
}
Expand Down Expand Up @@ -446,6 +448,7 @@ type checkSet = util.FastIntSet
func checkMutationInput(
ctx context.Context,
semaCtx *tree.SemaContext,
sessionData *sessiondata.SessionData,
tabDesc catalog.TableDescriptor,
checkOrds checkSet,
checkVals tree.Datums,
Expand All @@ -467,7 +470,9 @@ func checkMutationInput(
} else if !res && checkVals[colIdx] != tree.DNull {
// Failed to satisfy CHECK constraint, so unwrap the serialized
// check expression to display to the user.
expr, err := schemaexpr.FormatExprForDisplay(ctx, tabDesc, checks[i].Expr, semaCtx, tree.FmtParsable)
expr, err := schemaexpr.FormatExprForDisplay(
ctx, tabDesc, checks[i].Expr, semaCtx, sessionData, tree.FmtParsable,
)
if err != nil {
// If we ran into an error trying to read the check constraint, wrap it
// and return.
Expand Down
6 changes: 4 additions & 2 deletions pkg/sql/crdb_internal.go
Original file line number Diff line number Diff line change
Expand Up @@ -2368,7 +2368,7 @@ CREATE TABLE crdb_internal.create_statements (
var err error
if table.IsView() {
descType = typeView
stmt, err = ShowCreateView(ctx, &p.semaCtx, &name, table)
stmt, err = ShowCreateView(ctx, &p.semaCtx, p.SessionData(), &name, table)
} else if table.IsSequence() {
descType = typeSequence
stmt, err = ShowCreateSequence(ctx, &name, table)
Expand Down Expand Up @@ -2489,7 +2489,9 @@ CREATE TABLE crdb_internal.table_columns (
for _, col := range columns {
defStr := tree.DNull
if col.HasDefault() {
defExpr, err := schemaexpr.FormatExprForDisplay(ctx, table, col.GetDefaultExpr(), &p.semaCtx, tree.FmtParsable)
defExpr, err := schemaexpr.FormatExprForDisplay(
ctx, table, col.GetDefaultExpr(), &p.semaCtx, p.SessionData(), tree.FmtParsable,
)
if err != nil {
return err
}
Expand Down
8 changes: 6 additions & 2 deletions pkg/sql/information_schema.go
Original file line number Diff line number Diff line change
Expand Up @@ -435,15 +435,19 @@ https://www.postgresql.org/docs/9.5/infoschema-columns.html`,
}
colDefault := tree.DNull
if column.HasDefault() {
colExpr, err := schemaexpr.FormatExprForDisplay(ctx, table, column.GetDefaultExpr(), &p.semaCtx, tree.FmtParsable)
colExpr, err := schemaexpr.FormatExprForDisplay(
ctx, table, column.GetDefaultExpr(), &p.semaCtx, p.SessionData(), tree.FmtParsable,
)
if err != nil {
return err
}
colDefault = tree.NewDString(colExpr)
}
colComputed := emptyString
if column.IsComputed() {
colExpr, err := schemaexpr.FormatExprForDisplay(ctx, table, column.GetComputeExpr(), &p.semaCtx, tree.FmtSimple)
colExpr, err := schemaexpr.FormatExprForDisplay(
ctx, table, column.GetComputeExpr(), &p.semaCtx, p.SessionData(), tree.FmtSimple,
)
if err != nil {
return err
}
Expand Down
2 changes: 1 addition & 1 deletion pkg/sql/insert.go
Original file line number Diff line number Diff line change
Expand Up @@ -149,7 +149,7 @@ func (r *insertRun) processSourceRow(params runParams, rowVals tree.Datums) erro
if !r.checkOrds.Empty() {
checkVals := rowVals[len(r.insertCols):]
if err := checkMutationInput(
params.ctx, &params.p.semaCtx, r.ti.tableDesc(), r.checkOrds, checkVals,
params.ctx, &params.p.semaCtx, params.p.SessionData(), r.ti.tableDesc(), r.checkOrds, checkVals,
); err != nil {
return err
}
Expand Down
Loading

0 comments on commit dbad874

Please sign in to comment.