Skip to content

Commit

Permalink
eval: clean up usage of UnwrapDatum
Browse files Browse the repository at this point in the history
This commit removes many calls to `eval.UnwrapDatum` where the first
argument was `nil`. In such a case that function is equivalent to
`tree.UnwrapDOidWrapper`, so this commit uses the latter wherever
possible.

Next, this commit adds an explicit `context.Context` argument to the
signature to avoid the usage of the deprecated stored context from
`eval.Context`. This required a little bit of plumbing around the index
encoding methods.

Release note: None
  • Loading branch information
yuzefovich committed Oct 26, 2022
1 parent 1b1c8da commit 08b815b
Show file tree
Hide file tree
Showing 24 changed files with 110 additions and 110 deletions.
10 changes: 5 additions & 5 deletions pkg/sql/importer/exportparquet.go
Original file line number Diff line number Diff line change
Expand Up @@ -28,7 +28,6 @@ import (
"github.com/cockroachdb/cockroach/pkg/sql/rowenc"
"github.com/cockroachdb/cockroach/pkg/sql/rowexec"
"github.com/cockroachdb/cockroach/pkg/sql/sem/builtins"
"github.com/cockroachdb/cockroach/pkg/sql/sem/eval"
"github.com/cockroachdb/cockroach/pkg/sql/sem/tree"
"github.com/cockroachdb/cockroach/pkg/sql/types"
"github.com/cockroachdb/cockroach/pkg/util/bitarray"
Expand Down Expand Up @@ -776,10 +775,11 @@ func (sp *parquetWriterProcessor) Run(ctx context.Context) {
return err
}

// If we're encoding a DOidWrapper, then we want to cast the wrapped datum.
// Note that we pass in nil as the first argument since we're not interested
// in evaluating the evalCtx's placeholders.
edNative, err := exporter.parquetColumns[i].encodeFn(eval.UnwrapDatum(nil, ed.Datum))
// If we're encoding a DOidWrapper, then we want to cast
// the wrapped datum. Note that we don't use
// eval.UnwrapDatum since we're not interested in
// evaluating the placeholders.
edNative, err := exporter.parquetColumns[i].encodeFn(tree.UnwrapDOidWrapper(ed.Datum))
if err != nil {
return err
}
Expand Down
10 changes: 4 additions & 6 deletions pkg/sql/importer/exportparquet_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -27,7 +27,6 @@ import (
"github.com/cockroachdb/cockroach/pkg/sql/catalog/colinfo"
"github.com/cockroachdb/cockroach/pkg/sql/importer"
"github.com/cockroachdb/cockroach/pkg/sql/randgen"
"github.com/cockroachdb/cockroach/pkg/sql/sem/eval"
"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 @@ -147,11 +146,10 @@ func validateParquetFile(
return err
}

// If we're encoding a DOidWrapper, then we want to cast the wrapped datum.
// Note that we pass in nil as the first argument since we're not interested
// in evaluating the placeholders.
validateDatum(t, eval.UnwrapDatum(nil, test.datums[i][j]), eval.UnwrapDatum(nil, datum),
test.cols[j].Typ)
// If we're encoding a DOidWrapper, then we want to cast the wrapped
// datum. Note that we don't use eval.UnwrapDatum since we're not
// interested in evaluating the placeholders.
validateDatum(t, tree.UnwrapDOidWrapper(test.datums[i][j]), tree.UnwrapDOidWrapper(datum), test.cols[j].Typ)
}
i++
}
Expand Down
2 changes: 1 addition & 1 deletion pkg/sql/opt/invertedidx/inverted_index_expr.go
Original file line number Diff line number Diff line change
Expand Up @@ -41,7 +41,7 @@ func NewDatumsToInvertedExpr(
return NewGeoDatumsToInvertedExpr(ctx, evalCtx, colTypes, expr, geoConfig)
}

return NewJSONOrArrayDatumsToInvertedExpr(evalCtx, colTypes, expr)
return NewJSONOrArrayDatumsToInvertedExpr(ctx, evalCtx, colTypes, expr)
}

// NewBoundPreFilterer returns a PreFilterer for the given expr where the type
Expand Down
80 changes: 42 additions & 38 deletions pkg/sql/opt/invertedidx/json_array.go
Original file line number Diff line number Diff line change
Expand Up @@ -126,9 +126,9 @@ func (j *jsonOrArrayJoinPlanner) extractJSONOrArrayJoinCondition(
// through the JSON or Array. This function is used when checking if an indexed
// column contains (@>) a constant.
func getInvertedExprForJSONOrArrayIndexForContaining(
evalCtx *eval.Context, d tree.Datum,
ctx context.Context, evalCtx *eval.Context, d tree.Datum,
) inverted.Expression {
invertedExpr, err := rowenc.EncodeContainingInvertedIndexSpans(evalCtx, d)
invertedExpr, err := rowenc.EncodeContainingInvertedIndexSpans(ctx, evalCtx, d)
if err != nil {
panic(err)
}
Expand All @@ -141,9 +141,9 @@ func getInvertedExprForJSONOrArrayIndexForContaining(
// through the JSON or Array. This function is only used when checking if an
// indexed column is contained by (<@) a constant.
func getInvertedExprForJSONOrArrayIndexForContainedBy(
evalCtx *eval.Context, d tree.Datum,
ctx context.Context, evalCtx *eval.Context, d tree.Datum,
) inverted.Expression {
invertedExpr, err := rowenc.EncodeContainedInvertedIndexSpans(evalCtx, d)
invertedExpr, err := rowenc.EncodeContainedInvertedIndexSpans(ctx, evalCtx, d)
if err != nil {
panic(err)
}
Expand All @@ -157,9 +157,9 @@ func getInvertedExprForJSONOrArrayIndexForContainedBy(
// If d is an array, then the inverted expression is a conjunction if all is
// true, and a disjunction otherwise.
func getInvertedExprForJSONIndexForExists(
evalCtx *eval.Context, d tree.Datum, all bool,
ctx context.Context, evalCtx *eval.Context, d tree.Datum, all bool,
) inverted.Expression {
invertedExpr, err := rowenc.EncodeExistsInvertedIndexSpans(evalCtx, d, all)
invertedExpr, err := rowenc.EncodeExistsInvertedIndexSpans(ctx, evalCtx, d, all)
if err != nil {
panic(err)
}
Expand All @@ -172,9 +172,9 @@ func getInvertedExprForJSONIndexForExists(
// through the Array. This function is only used when checking if an
// indexed Array column overlaps (&&) with a constant.
func getInvertedExprForArrayIndexForOverlaps(
evalCtx *eval.Context, d tree.Datum,
ctx context.Context, evalCtx *eval.Context, d tree.Datum,
) inverted.Expression {
invertedExpr, err := rowenc.EncodeOverlapsInvertedIndexSpans(evalCtx, d)
invertedExpr, err := rowenc.EncodeOverlapsInvertedIndexSpans(ctx, evalCtx, d)
if err != nil {
panic(err)
}
Expand Down Expand Up @@ -232,7 +232,7 @@ func (g *jsonOrArrayDatumsToInvertedExpr) IndexedVarNodeFormatter(idx int) tree.
// NewJSONOrArrayDatumsToInvertedExpr returns a new
// jsonOrArrayDatumsToInvertedExpr.
func NewJSONOrArrayDatumsToInvertedExpr(
evalCtx *eval.Context, colTypes []*types.T, expr tree.TypedExpr,
ctx context.Context, evalCtx *eval.Context, colTypes []*types.T, expr tree.TypedExpr,
) (invertedexpr.DatumsToInvertedExpr, error) {
g := &jsonOrArrayDatumsToInvertedExpr{
evalCtx: evalCtx,
Expand All @@ -259,17 +259,17 @@ func NewJSONOrArrayDatumsToInvertedExpr(
var invertedExpr inverted.Expression
switch t.Operator.Symbol {
case treecmp.ContainedBy:
invertedExpr = getInvertedExprForJSONOrArrayIndexForContainedBy(evalCtx, d)
invertedExpr = getInvertedExprForJSONOrArrayIndexForContainedBy(ctx, evalCtx, d)
case treecmp.Contains:
invertedExpr = getInvertedExprForJSONOrArrayIndexForContaining(evalCtx, d)
invertedExpr = getInvertedExprForJSONOrArrayIndexForContaining(ctx, evalCtx, d)
case treecmp.Overlaps:
invertedExpr = getInvertedExprForArrayIndexForOverlaps(evalCtx, d)
invertedExpr = getInvertedExprForArrayIndexForOverlaps(ctx, evalCtx, d)
case treecmp.JSONExists:
invertedExpr = getInvertedExprForJSONIndexForExists(evalCtx, d, true /* all */)
invertedExpr = getInvertedExprForJSONIndexForExists(ctx, evalCtx, d, true /* all */)
case treecmp.JSONSomeExists:
invertedExpr = getInvertedExprForJSONIndexForExists(evalCtx, d, false /* all */)
invertedExpr = getInvertedExprForJSONIndexForExists(ctx, evalCtx, d, false /* all */)
case treecmp.JSONAllExists:
invertedExpr = getInvertedExprForJSONIndexForExists(evalCtx, d, true /* all */)
invertedExpr = getInvertedExprForJSONIndexForExists(ctx, evalCtx, d, true /* all */)
default:
return nil, fmt.Errorf("%s cannot be index-accelerated", t)
}
Expand Down Expand Up @@ -319,10 +319,10 @@ func (g *jsonOrArrayDatumsToInvertedExpr) Convert(
}
switch t.Operator.Symbol {
case treecmp.Contains:
return getInvertedExprForJSONOrArrayIndexForContaining(g.evalCtx, d), nil
return getInvertedExprForJSONOrArrayIndexForContaining(ctx, g.evalCtx, d), nil

case treecmp.ContainedBy:
return getInvertedExprForJSONOrArrayIndexForContainedBy(g.evalCtx, d), nil
return getInvertedExprForJSONOrArrayIndexForContainedBy(ctx, g.evalCtx, d), nil

default:
return nil, fmt.Errorf("unsupported expression %v", t)
Expand Down Expand Up @@ -379,21 +379,21 @@ func (j *jsonOrArrayFilterPlanner) extractInvertedFilterConditionFromLeaf(
) {
switch t := expr.(type) {
case *memo.ContainsExpr:
invertedExpr = j.extractJSONOrArrayContainsCondition(evalCtx, t.Left, t.Right, false /* containedBy */)
invertedExpr = j.extractJSONOrArrayContainsCondition(ctx, evalCtx, t.Left, t.Right, false /* containedBy */)
case *memo.ContainedByExpr:
invertedExpr = j.extractJSONOrArrayContainsCondition(evalCtx, t.Left, t.Right, true /* containedBy */)
invertedExpr = j.extractJSONOrArrayContainsCondition(ctx, evalCtx, t.Left, t.Right, true /* containedBy */)
case *memo.JsonExistsExpr:
invertedExpr = j.extractJSONExistsCondition(evalCtx, t.Left, t.Right, false /* all */)
invertedExpr = j.extractJSONExistsCondition(ctx, evalCtx, t.Left, t.Right, false /* all */)
case *memo.JsonSomeExistsExpr:
invertedExpr = j.extractJSONExistsCondition(evalCtx, t.Left, t.Right, false /* all */)
invertedExpr = j.extractJSONExistsCondition(ctx, evalCtx, t.Left, t.Right, false /* all */)
case *memo.JsonAllExistsExpr:
invertedExpr = j.extractJSONExistsCondition(evalCtx, t.Left, t.Right, true /* all */)
invertedExpr = j.extractJSONExistsCondition(ctx, evalCtx, t.Left, t.Right, true /* all */)
case *memo.EqExpr:
if fetch, ok := t.Left.(*memo.FetchValExpr); ok {
invertedExpr = j.extractJSONFetchValEqCondition(evalCtx, fetch, t.Right)
invertedExpr = j.extractJSONFetchValEqCondition(ctx, evalCtx, fetch, t.Right)
}
case *memo.OverlapsExpr:
invertedExpr = j.extractArrayOverlapsCondition(evalCtx, t.Left, t.Right)
invertedExpr = j.extractArrayOverlapsCondition(ctx, evalCtx, t.Left, t.Right)
}

if invertedExpr == nil {
Expand All @@ -417,7 +417,7 @@ func (j *jsonOrArrayFilterPlanner) extractInvertedFilterConditionFromLeaf(
// on the given left and right expression arguments. Returns an empty
// InvertedExpression if no inverted filter could be extracted.
func (j *jsonOrArrayFilterPlanner) extractArrayOverlapsCondition(
evalCtx *eval.Context, left, right opt.ScalarExpr,
ctx context.Context, evalCtx *eval.Context, left, right opt.ScalarExpr,
) inverted.Expression {
var constantVal opt.ScalarExpr
if isIndexColumn(j.tabID, j.index, left, j.computedColumns) && memo.CanExtractConstDatum(right) {
Expand All @@ -434,15 +434,15 @@ func (j *jsonOrArrayFilterPlanner) extractArrayOverlapsCondition(
// If none of the conditions are met, we cannot create an InvertedExpression.
return inverted.NonInvertedColExpression{}
}
return getInvertedExprForArrayIndexForOverlaps(evalCtx, memo.ExtractConstDatum(constantVal))
return getInvertedExprForArrayIndexForOverlaps(ctx, evalCtx, memo.ExtractConstDatum(constantVal))
}

// extractJSONOrArrayContainsCondition extracts an InvertedExpression
// representing an inverted filter over the planner's inverted index, based
// on the given left and right expression arguments. Returns an empty
// InvertedExpression if no inverted filter could be extracted.
func (j *jsonOrArrayFilterPlanner) extractJSONOrArrayContainsCondition(
evalCtx *eval.Context, left, right opt.ScalarExpr, containedBy bool,
ctx context.Context, evalCtx *eval.Context, left, right opt.ScalarExpr, containedBy bool,
) inverted.Expression {
var indexColumn, constantVal opt.ScalarExpr
if isIndexColumn(j.tabID, j.index, left, j.computedColumns) && memo.CanExtractConstDatum(right) {
Expand All @@ -460,12 +460,12 @@ func (j *jsonOrArrayFilterPlanner) extractJSONOrArrayContainsCondition(
if fetch, ok := left.(*memo.FetchValExpr); ok {
// When the expression has a JSON fetch operator on the left, it is
// handled in extractJSONFetchValContainsCondition.
return j.extractJSONFetchValContainsCondition(evalCtx, fetch, right, containedBy)
return j.extractJSONFetchValContainsCondition(ctx, evalCtx, fetch, right, containedBy)
} else if fetch, ok := right.(*memo.FetchValExpr); ok {
// When the expression has a JSON fetch operator on the right, it is
// handled in extractJSONFetchValContainsCondition as an equivalent
// expression with right and left swapped.
return j.extractJSONFetchValContainsCondition(evalCtx, fetch, left, !containedBy)
return j.extractJSONFetchValContainsCondition(ctx, evalCtx, fetch, left, !containedBy)
}
// If none of the conditions are met, we cannot create an InvertedExpression.
return inverted.NonInvertedColExpression{}
Expand All @@ -480,9 +480,9 @@ func (j *jsonOrArrayFilterPlanner) extractJSONOrArrayContainsCondition(
}
}
if containedBy {
return getInvertedExprForJSONOrArrayIndexForContainedBy(evalCtx, d)
return getInvertedExprForJSONOrArrayIndexForContainedBy(ctx, evalCtx, d)
}
return getInvertedExprForJSONOrArrayIndexForContaining(evalCtx, d)
return getInvertedExprForJSONOrArrayIndexForContaining(ctx, evalCtx, d)
}

// extractJSONExistsCondition extracts an InvertedExpression representing an
Expand All @@ -491,15 +491,15 @@ func (j *jsonOrArrayFilterPlanner) extractJSONOrArrayContainsCondition(
// arguments. Returns an empty InvertedExpression if no inverted filter could be
// extracted.
func (j *jsonOrArrayFilterPlanner) extractJSONExistsCondition(
evalCtx *eval.Context, left, right opt.ScalarExpr, all bool,
ctx context.Context, evalCtx *eval.Context, left, right opt.ScalarExpr, all bool,
) inverted.Expression {
if isIndexColumn(j.tabID, j.index, left, j.computedColumns) && memo.CanExtractConstDatum(right) {
// When the first argument is a variable or expression corresponding to the
// index column and the second argument is a constant, we get the
// InvertedExpression for left ? right.
constantVal := right
d := memo.ExtractConstDatum(constantVal)
return getInvertedExprForJSONIndexForExists(evalCtx, d, all)
return getInvertedExprForJSONIndexForExists(ctx, evalCtx, d, all)
}
// If none of the conditions are met, we cannot create an InvertedExpression.
return inverted.NonInvertedColExpression{}
Expand All @@ -517,7 +517,7 @@ func (j *jsonOrArrayFilterPlanner) extractJSONExistsCondition(
// index and each index is a constant string. The right expression must be a
// constant JSON value.
func (j *jsonOrArrayFilterPlanner) extractJSONFetchValEqCondition(
evalCtx *eval.Context, left *memo.FetchValExpr, right opt.ScalarExpr,
ctx context.Context, evalCtx *eval.Context, left *memo.FetchValExpr, right opt.ScalarExpr,
) inverted.Expression {
// The right side of the expression should be a constant JSON value.
if !memo.CanExtractConstDatum(right) {
Expand All @@ -540,7 +540,7 @@ func (j *jsonOrArrayFilterPlanner) extractJSONFetchValEqCondition(

// For Equals expressions, we will generate the inverted expression for the
// single object built from the keys and val.
invertedExpr := getInvertedExprForJSONOrArrayIndexForContaining(evalCtx, tree.NewDJSON(obj))
invertedExpr := getInvertedExprForJSONOrArrayIndexForContaining(ctx, evalCtx, tree.NewDJSON(obj))

// When the right side is an array or object, the InvertedExpression
// generated is not tight. We must indicate it is non-tight so an additional
Expand Down Expand Up @@ -568,7 +568,11 @@ func (j *jsonOrArrayFilterPlanner) extractJSONFetchValEqCondition(
// The type of operator is indicated by the containedBy parameter, which is
// true for <@ and false for @>.
func (j *jsonOrArrayFilterPlanner) extractJSONFetchValContainsCondition(
evalCtx *eval.Context, left *memo.FetchValExpr, right opt.ScalarExpr, containedBy bool,
ctx context.Context,
evalCtx *eval.Context,
left *memo.FetchValExpr,
right opt.ScalarExpr,
containedBy bool,
) inverted.Expression {
// The right side of the expression should be a constant JSON value.
if !memo.CanExtractConstDatum(right) {
Expand Down Expand Up @@ -603,9 +607,9 @@ func (j *jsonOrArrayFilterPlanner) extractJSONFetchValContainsCondition(
for i := range objs {
var expr inverted.Expression
if containedBy {
expr = getInvertedExprForJSONOrArrayIndexForContainedBy(evalCtx, tree.NewDJSON(objs[i]))
expr = getInvertedExprForJSONOrArrayIndexForContainedBy(ctx, evalCtx, tree.NewDJSON(objs[i]))
} else {
expr = getInvertedExprForJSONOrArrayIndexForContaining(evalCtx, tree.NewDJSON(objs[i]))
expr = getInvertedExprForJSONOrArrayIndexForContaining(ctx, evalCtx, tree.NewDJSON(objs[i]))
}
if invertedExpr == nil {
invertedExpr = expr
Expand Down
4 changes: 2 additions & 2 deletions pkg/sql/paramparse/paramparse.go
Original file line number Diff line number Diff line change
Expand Up @@ -43,7 +43,7 @@ func DatumAsFloat(
if err != nil {
return 0, err
}
switch v := eval.UnwrapDatum(evalCtx, val).(type) {
switch v := eval.UnwrapDatum(ctx, evalCtx, val).(type) {
case *tree.DString:
return strconv.ParseFloat(string(*v), 64)
case *tree.DInt:
Expand All @@ -70,7 +70,7 @@ func DatumAsDuration(
return 0, err
}
var d duration.Duration
switch v := eval.UnwrapDatum(evalCtx, val).(type) {
switch v := eval.UnwrapDatum(ctx, evalCtx, val).(type) {
case *tree.DString:
datum, err := tree.ParseDInterval(evalCtx.SessionData().GetIntervalStyle(), string(*v))
if err != nil {
Expand Down
1 change: 0 additions & 1 deletion pkg/sql/pgwire/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -44,7 +44,6 @@ go_library(
"//pkg/sql/pgwire/pgwirebase",
"//pkg/sql/pgwire/pgwirecancel",
"//pkg/sql/sem/catconstants",
"//pkg/sql/sem/eval",
"//pkg/sql/sem/tree",
"//pkg/sql/sessiondatapb",
"//pkg/sql/sqltelemetry",
Expand Down
5 changes: 2 additions & 3 deletions pkg/sql/pgwire/types.go
Original file line number Diff line number Diff line change
Expand Up @@ -25,7 +25,6 @@ import (
"github.com/cockroachdb/cockroach/pkg/server/telemetry"
"github.com/cockroachdb/cockroach/pkg/sql/lex"
"github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgwirebase"
"github.com/cockroachdb/cockroach/pkg/sql/sem/eval"
"github.com/cockroachdb/cockroach/pkg/sql/sem/tree"
"github.com/cockroachdb/cockroach/pkg/sql/sessiondatapb"
"github.com/cockroachdb/cockroach/pkg/sql/sqltelemetry"
Expand Down Expand Up @@ -160,7 +159,7 @@ func writeTextDatumNotNull(
) {
oldDCC := b.textFormatter.SetDataConversionConfig(conv)
defer b.textFormatter.SetDataConversionConfig(oldDCC)
switch v := eval.UnwrapDatum(nil, d).(type) {
switch v := tree.UnwrapDOidWrapper(d).(type) {
case *tree.DBitArray:
b.textFormatter.FormatNode(v)
b.writeFromFmtCtx(b.textFormatter)
Expand Down Expand Up @@ -560,7 +559,7 @@ func (b *writeBuffer) writeBinaryDatum(
func writeBinaryDatumNotNull(
ctx context.Context, b *writeBuffer, d tree.Datum, sessionLoc *time.Location, t *types.T,
) {
switch v := eval.UnwrapDatum(nil, d).(type) {
switch v := tree.UnwrapDOidWrapper(d).(type) {
case *tree.DBitArray:
words, lastBitsUsed := v.EncodingParts()
if len(words) == 0 {
Expand Down
Loading

0 comments on commit 08b815b

Please sign in to comment.