Skip to content

Commit

Permalink
Browse files Browse the repository at this point in the history
85199: sql: move function properties to overload level r=chengxiong-ruan a=chengxiong-ruan

There're 3 commits:
(1) sql: use Overload in FunctionDefinition instead of overloadImpl

It feels a bit annoying to always need to cast overloadImpl
to Overload. In fact, all `overloadImpl`s in FunctionDefinition
are `Overload`s

(2) sql: move function properties to overload level

Moving function propterties to overload level, but still keep
it in FunctionDefiniton since it's still needed for internal
usage like docgen. Later on, the function resolution interface
will return a resolved version of function definition which
won't have the properties field. Instead, we'll go through
overloads to fetch properties for resolved function definition.
Current FunctionDefiniton will be kept for builtin functions only.

(3) sql: remove usage of GetBuiltinProperties from GetSequenceFromFunc

In `GetSequenceFromFunc` we resolve a functiong and then use the
name to get builtin properties. This seems unnecessary because
we may just get function properties from the resolved function
definition. With this change, we are eleminating a good amount of
usage cases of "GetBuiltinProperties", which seems to be the
majority of the usages of "GetBuiltinProperties".

There shouldn't be any functionality changes since we don't have UDFs yet.

85259: roachtest: fix cdc/schemareg r=ajwerner a=ajwerner

Now that DROP COLUMN uses the declarative schema changer, you no longer see
touch writes.

Fixes #84789

Release note: None

85281: backupccl: show backup was incorrectly capturing a ctx r=yuzefovich a=adityamaru

This diff fixes a span use after finish that was a result
of show backup code incorrectly capturing a context.

Fixes: #85201

Release note: None

85284: dev: fix `test --changed` when no files are changed r=rail a=rickystewart

With the previous version of the code this would fail with a confusing
error message if no files were changed.

Release note: None

Co-authored-by: Chengxiong Ruan <[email protected]>
Co-authored-by: Andrew Werner <[email protected]>
Co-authored-by: Aditya Maru <[email protected]>
Co-authored-by: Ricky Stewart <[email protected]>
  • Loading branch information
5 people committed Jul 29, 2022
5 parents 933df73 + 5409013 + dc25058 + 7245e31 + df5a016 commit 4c8e32a
Show file tree
Hide file tree
Showing 39 changed files with 256 additions and 159 deletions.
2 changes: 1 addition & 1 deletion dev
Original file line number Diff line number Diff line change
Expand Up @@ -8,7 +8,7 @@ fi
set -euo pipefail

# Bump this counter to force rebuilding `dev` on all machines.
DEV_VERSION=47
DEV_VERSION=48

THIS_DIR=$(cd "$(dirname "$0")" && pwd)
BINARY_DIR=$THIS_DIR/bin/dev-versions
Expand Down
25 changes: 12 additions & 13 deletions pkg/ccl/backupccl/show.go
Original file line number Diff line number Diff line change
Expand Up @@ -105,9 +105,9 @@ func (m manifestInfoReader) header() colinfo.ResultColumns {
func (m manifestInfoReader) showBackup(
ctx context.Context,
mem *mon.BoundAccount,
mkStore cloud.ExternalStorageFromURIFactory,
_ cloud.ExternalStorageFromURIFactory,
info backupInfo,
user username.SQLUsername,
_ username.SQLUsername,
resultsCh chan<- tree.Datums,
) error {
var memReserved int64
Expand All @@ -129,7 +129,7 @@ func (m manifestInfoReader) showBackup(
return err
}

datums, err := m.shower.fn(info)
datums, err := m.shower.fn(ctx, info)
if err != nil {
return err
}
Expand Down Expand Up @@ -158,7 +158,7 @@ func (m metadataSSTInfoReader) header() colinfo.ResultColumns {

func (m metadataSSTInfoReader) showBackup(
ctx context.Context,
mem *mon.BoundAccount,
_ *mon.BoundAccount,
mkStore cloud.ExternalStorageFromURIFactory,
info backupInfo,
user username.SQLUsername,
Expand Down Expand Up @@ -249,15 +249,14 @@ func showBackupPlanHook(
case tree.BackupFileDetails:
shower = backupShowerFileSetup(backup.InCollection)
case tree.BackupSchemaDetails:
shower = backupShowerDefault(ctx, p, true, opts)
shower = backupShowerDefault(p, true, opts)
default:
shower = backupShowerDefault(ctx, p, false, opts)
shower = backupShowerDefault(p, false, opts)
}
infoReader = manifestInfoReader{shower: shower}
}

fn := func(ctx context.Context, _ []sql.PlanNode, resultsCh chan<- tree.Datums) error {
// TODO(dan): Move this span into sql.
ctx, span := tracing.ChildSpan(ctx, stmt.StatementTag())
defer span.Finish()

Expand Down Expand Up @@ -612,7 +611,7 @@ type backupShower struct {

// fn is the specific implementation of the shower that can either be a default, ranges, files,
// or JSON shower.
fn func(info backupInfo) ([]tree.Datums, error)
fn func(ctx context.Context, info backupInfo) ([]tree.Datums, error)
}

// backupShowerHeaders defines the schema for the table presented to the user.
Expand Down Expand Up @@ -656,11 +655,11 @@ func backupShowerHeaders(showSchemas bool, opts map[string]string) colinfo.Resul
}

func backupShowerDefault(
ctx context.Context, p sql.PlanHookState, showSchemas bool, opts map[string]string,
p sql.PlanHookState, showSchemas bool, opts map[string]string,
) backupShower {
return backupShower{
header: backupShowerHeaders(showSchemas, opts),
fn: func(info backupInfo) ([]tree.Datums, error) {
fn: func(ctx context.Context, info backupInfo) ([]tree.Datums, error) {
var rows []tree.Datums
for layer, manifest := range info.manifests {
// Map database ID to descriptor name.
Expand Down Expand Up @@ -998,7 +997,7 @@ var backupShowerRanges = backupShower{
{Name: "end_key", Typ: types.Bytes},
},

fn: func(info backupInfo) (rows []tree.Datums, err error) {
fn: func(ctx context.Context, info backupInfo) (rows []tree.Datums, err error) {
for _, manifest := range info.manifests {
for _, span := range manifest.Spans {
rows = append(rows, tree.Datums{
Expand Down Expand Up @@ -1027,7 +1026,7 @@ func backupShowerFileSetup(inCol tree.StringOrPlaceholderOptList) backupShower {
{Name: "file_bytes", Typ: types.Int},
},

fn: func(info backupInfo) (rows []tree.Datums, err error) {
fn: func(ctx context.Context, info backupInfo) (rows []tree.Datums, err error) {

var manifestDirs []string
var localityAware bool
Expand Down Expand Up @@ -1161,7 +1160,7 @@ var jsonShower = backupShower{
{Name: "manifest", Typ: types.Jsonb},
},

fn: func(info backupInfo) ([]tree.Datums, error) {
fn: func(ctx context.Context, info backupInfo) ([]tree.Datums, error) {
rows := make([]tree.Datums, len(info.manifests))
for i, manifest := range info.manifests {
j, err := protoreflect.MessageToJSON(
Expand Down
3 changes: 1 addition & 2 deletions pkg/ccl/changefeedccl/cdceval/expr_eval.go
Original file line number Diff line number Diff line change
Expand Up @@ -624,8 +624,7 @@ func checkFunctionSupported(
fnVolatility = fnCall.ResolvedOverload().Volatility
} else {
// Pick highest volatility overload.
for _, o := range fn.Definition {
overload := o.(*tree.Overload)
for _, overload := range fn.Definition {
if overload.Volatility > fnVolatility {
fnVolatility = overload.Volatility
}
Expand Down
6 changes: 5 additions & 1 deletion pkg/cmd/dev/test.go
Original file line number Diff line number Diff line change
Expand Up @@ -454,7 +454,11 @@ func (d *dev) determineAffectedTargets(ctx context.Context) ([]string, error) {
if err != nil {
return nil, err
}
changedFilesList := strings.Split(strings.TrimSpace(string(changedFiles)), "\n")
trimmedOutput := strings.TrimSpace(string(changedFiles))
if trimmedOutput == "" {
return nil, nil
}
changedFilesList := strings.Split(trimmedOutput, "\n")
// Each file in this list needs to be munged somewhat to match up to the
// Bazel target syntax.
for idx, file := range changedFilesList {
Expand Down
5 changes: 1 addition & 4 deletions pkg/cmd/roachtest/tests/cdc.go
Original file line number Diff line number Diff line change
Expand Up @@ -529,7 +529,7 @@ func runCDCSchemaRegistry(ctx context.Context, t test.Test, c cluster.Cluster) {
updatedMap := make(map[string]struct{})
var resolved []string
pagesFetched := 0
pageSize := 14
pageSize := 7

for len(updatedMap) < 10 && pagesFetched < 5 {
result, err := c.RunWithDetailsSingleNode(ctx, t.L(), kafkaNode,
Expand Down Expand Up @@ -569,11 +569,8 @@ func runCDCSchemaRegistry(ctx context.Context, t test.Test, c cluster.Cluster) {
`{"before":null,"after":{"foo":{"a":{"long":3},"b":{"string":"3"},"c":{"long":3}}},"updated":{"string":""}}`,
`{"before":null,"after":{"foo":{"a":{"long":4},"c":{"long":4}}},"updated":{"string":""}}`,
`{"before":{"foo_before":{"a":{"long":1},"b":null,"c":null}},"after":{"foo":{"a":{"long":1},"c":null}},"updated":{"string":""}}`,
`{"before":{"foo_before":{"a":{"long":1},"c":null}},"after":{"foo":{"a":{"long":1},"c":null}},"updated":{"string":""}}`,
`{"before":{"foo_before":{"a":{"long":2},"b":{"string":"2"},"c":null}},"after":{"foo":{"a":{"long":2},"c":null}},"updated":{"string":""}}`,
`{"before":{"foo_before":{"a":{"long":2},"c":null}},"after":{"foo":{"a":{"long":2},"c":null}},"updated":{"string":""}}`,
`{"before":{"foo_before":{"a":{"long":3},"b":{"string":"3"},"c":{"long":3}}},"after":{"foo":{"a":{"long":3},"c":{"long":3}}},"updated":{"string":""}}`,
`{"before":{"foo_before":{"a":{"long":3},"c":{"long":3}}},"after":{"foo":{"a":{"long":3},"c":{"long":3}}},"updated":{"string":""}}`,
}
} else {
expected = []string{
Expand Down
2 changes: 1 addition & 1 deletion pkg/internal/sqlsmith/relational.go
Original file line number Diff line number Diff line change
Expand Up @@ -752,7 +752,7 @@ var countStar = func() tree.TypedExpr {
nil, /* window */
typ,
&fn.FunctionProperties,
fn.Definition[0].(*tree.Overload),
fn.Definition[0],
)
}()

Expand Down
1 change: 0 additions & 1 deletion pkg/internal/sqlsmith/schema.go
Original file line number Diff line number Diff line change
Expand Up @@ -534,7 +534,6 @@ var functions = func() map[tree.FunctionClass]map[oid.Oid][]function {
continue
}
for _, ov := range def.Definition {
ov := ov.(*tree.Overload)
// Ignore documented unusable functions.
if strings.Contains(ov.Info, "Not usable") {
continue
Expand Down
1 change: 0 additions & 1 deletion pkg/sql/catalog/seqexpr/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -27,7 +27,6 @@ go_test(
"//pkg/sql/catalog/descpb",
"//pkg/sql/parser",
"//pkg/sql/sem/builtins",
"//pkg/sql/sem/builtins/builtinsregistry",
"//pkg/sql/sem/tree",
"//pkg/sql/types",
"@com_github_stretchr_testify//require",
Expand Down
44 changes: 19 additions & 25 deletions pkg/sql/catalog/seqexpr/sequence.go
Original file line number Diff line number Diff line change
Expand Up @@ -47,24 +47,27 @@ func (si *SeqIdentifier) IsByID() bool {
// Returns the identifier of the sequence or nil if no sequence was found.
//
// `getBuiltinProperties` argument is commonly builtinsregistry.GetBuiltinProperties.
func GetSequenceFromFunc(
funcExpr *tree.FuncExpr,
getBuiltinProperties func(name string) (*tree.FunctionProperties, []tree.Overload),
) (*SeqIdentifier, error) {
func GetSequenceFromFunc(funcExpr *tree.FuncExpr) (*SeqIdentifier, error) {

// Resolve doesn't use the searchPath for resolving FunctionDefinitions
// so we can pass in an empty SearchPath.
// TODO(mgartner): Plumb a function resolver here, or determine that the
// function should have already been resolved.
// TODO(chengxiong): Since we have funcExpr here, it's possible to narrow down
// overloads by using input types.
def, err := funcExpr.Func.Resolve(tree.EmptySearchPath, nil /* resolver */)
if err != nil {
return nil, err
}

fnProps, overloads := getBuiltinProperties(def.Name)
if fnProps != nil && fnProps.HasSequenceArguments {
hasSequenceArguments, err := def.GetHasSequenceArguments()
if err != nil {
return nil, err
}

if hasSequenceArguments {
found := false
for _, overload := range overloads {
for _, overload := range def.Definition {
// Find the overload that matches funcExpr.
if len(funcExpr.Exprs) == overload.Types.Length() {
found = true
Expand Down Expand Up @@ -137,17 +140,14 @@ func getSequenceIdentifier(expr tree.Expr) *SeqIdentifier {
// e.g. nextval('foo') => "foo"; nextval(123::regclass) => 123; <some other expression> => nil
//
// `getBuiltinProperties` argument is commonly builtinsregistry.GetBuiltinProperties.
func GetUsedSequences(
defaultExpr tree.Expr,
getBuiltinProperties func(name string) (*tree.FunctionProperties, []tree.Overload),
) ([]SeqIdentifier, error) {
func GetUsedSequences(defaultExpr tree.Expr) ([]SeqIdentifier, error) {
var seqIdentifiers []SeqIdentifier
_, err := tree.SimpleVisit(
defaultExpr,
func(expr tree.Expr) (recurse bool, newExpr tree.Expr, err error) {
switch t := expr.(type) {
case *tree.FuncExpr:
identifier, err := GetSequenceFromFunc(t, getBuiltinProperties)
identifier, err := GetSequenceFromFunc(t)
if err != nil {
return false, nil, err
}
Expand All @@ -170,14 +170,12 @@ func GetUsedSequences(
//
// `getBuiltinProperties` argument is commonly builtinsregistry.GetBuiltinProperties.
func ReplaceSequenceNamesWithIDs(
defaultExpr tree.Expr,
nameToID map[string]descpb.ID,
getBuiltinProperties func(name string) (*tree.FunctionProperties, []tree.Overload),
defaultExpr tree.Expr, nameToID map[string]descpb.ID,
) (tree.Expr, error) {
replaceFn := func(expr tree.Expr) (recurse bool, newExpr tree.Expr, err error) {
switch t := expr.(type) {
case *tree.FuncExpr:
identifier, err := GetSequenceFromFunc(t, getBuiltinProperties)
identifier, err := GetSequenceFromFunc(t)
if err != nil {
return false, nil, err
}
Expand Down Expand Up @@ -219,13 +217,11 @@ func ReplaceSequenceNamesWithIDs(
//
// `getBuiltinProperties` argument is commonly builtinsregistry.GetBuiltinProperties.
func UpgradeSequenceReferenceInExpr(
expr *string,
usedSequenceIDsToNames map[descpb.ID]*tree.TableName,
getBuiltinProperties func(name string) (*tree.FunctionProperties, []tree.Overload),
expr *string, usedSequenceIDsToNames map[descpb.ID]*tree.TableName,
) (hasUpgraded bool, err error) {
// Find the "reverse" mapping from sequence name to their IDs for those
// sequences referenced by-name in `expr`.
usedSequenceNamesToIDs, err := seqNameToIDMappingInExpr(*expr, usedSequenceIDsToNames, getBuiltinProperties)
usedSequenceNamesToIDs, err := seqNameToIDMappingInExpr(*expr, usedSequenceIDsToNames)
if err != nil {
return false, err
}
Expand All @@ -237,7 +233,7 @@ func UpgradeSequenceReferenceInExpr(
return false, err
}

newExpr, err := ReplaceSequenceNamesWithIDs(parsedExpr, usedSequenceNamesToIDs, getBuiltinProperties)
newExpr, err := ReplaceSequenceNamesWithIDs(parsedExpr, usedSequenceNamesToIDs)
if err != nil {
return false, err
}
Expand Down Expand Up @@ -265,15 +261,13 @@ func UpgradeSequenceReferenceInExpr(
//
// See its unit test for some examples.
func seqNameToIDMappingInExpr(
expr string,
seqIDToNameMapping map[descpb.ID]*tree.TableName,
getBuiltinProperties func(name string) (*tree.FunctionProperties, []tree.Overload),
expr string, seqIDToNameMapping map[descpb.ID]*tree.TableName,
) (map[string]descpb.ID, error) {
parsedExpr, err := parser.ParseExpr(expr)
if err != nil {
return nil, err
}
seqRefs, err := GetUsedSequences(parsedExpr, getBuiltinProperties)
seqRefs, err := GetUsedSequences(parsedExpr)
if err != nil {
return nil, err
}
Expand Down
23 changes: 11 additions & 12 deletions pkg/sql/catalog/seqexpr/sequence_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -19,7 +19,6 @@ import (
"github.com/cockroachdb/cockroach/pkg/sql/catalog/seqexpr"
"github.com/cockroachdb/cockroach/pkg/sql/parser"
_ "github.com/cockroachdb/cockroach/pkg/sql/sem/builtins" // register all builtins in builtins:init() for seqexpr package
"github.com/cockroachdb/cockroach/pkg/sql/sem/builtins/builtinsregistry"
"github.com/cockroachdb/cockroach/pkg/sql/sem/tree"
"github.com/cockroachdb/cockroach/pkg/sql/types"
"github.com/stretchr/testify/require"
Expand Down Expand Up @@ -53,7 +52,7 @@ func TestGetSequenceFromFunc(t *testing.T) {
if !ok {
t.Fatal("Expr is not a FuncExpr")
}
identifier, err := seqexpr.GetSequenceFromFunc(funcExpr, builtinsregistry.GetBuiltinProperties)
identifier, err := seqexpr.GetSequenceFromFunc(funcExpr)
if err != nil {
t.Fatal(err)
}
Expand Down Expand Up @@ -99,7 +98,7 @@ func TestGetUsedSequences(t *testing.T) {
if err != nil {
t.Fatal(err)
}
identifiers, err := seqexpr.GetUsedSequences(typedExpr, builtinsregistry.GetBuiltinProperties)
identifiers, err := seqexpr.GetUsedSequences(typedExpr)
if err != nil {
t.Fatal(err)
}
Expand Down Expand Up @@ -150,7 +149,7 @@ func TestReplaceSequenceNamesWithIDs(t *testing.T) {
if err != nil {
t.Fatal(err)
}
newExpr, err := seqexpr.ReplaceSequenceNamesWithIDs(typedExpr, namesToID, builtinsregistry.GetBuiltinProperties)
newExpr, err := seqexpr.ReplaceSequenceNamesWithIDs(typedExpr, namesToID)
if err != nil {
t.Fatal(err)
}
Expand All @@ -169,7 +168,7 @@ func TestUpgradeSequenceReferenceInExpr(t *testing.T) {
usedSequenceIDsToNames[1] = &tbl1
usedSequenceIDsToNames[2] = &tbl2
expr := "nextval('testdb.sc1.t') + nextval('sc1.t')"
hasUpgraded, err := seqexpr.UpgradeSequenceReferenceInExpr(&expr, usedSequenceIDsToNames, builtinsregistry.GetBuiltinProperties)
hasUpgraded, err := seqexpr.UpgradeSequenceReferenceInExpr(&expr, usedSequenceIDsToNames)
require.NoError(t, err)
require.True(t, hasUpgraded)
require.Equal(t,
Expand All @@ -184,7 +183,7 @@ func TestUpgradeSequenceReferenceInExpr(t *testing.T) {
usedSequenceIDsToNames[1] = &tbl1
usedSequenceIDsToNames[2] = &tbl2
expr := "nextval('testdb.sc1.t') + nextval('sc1.t')"
hasUpgraded, err := seqexpr.UpgradeSequenceReferenceInExpr(&expr, usedSequenceIDsToNames, builtinsregistry.GetBuiltinProperties)
hasUpgraded, err := seqexpr.UpgradeSequenceReferenceInExpr(&expr, usedSequenceIDsToNames)
require.NoError(t, err)
require.True(t, hasUpgraded)
require.Equal(t,
Expand All @@ -200,7 +199,7 @@ func TestUpgradeSequenceReferenceInExpr(t *testing.T) {
usedSequenceIDsToNames[1] = &tbl1
usedSequenceIDsToNames[2] = &tbl2
expr := "nextval('testdb.public.t') + nextval('testdb.t')"
hasUpgraded, err := seqexpr.UpgradeSequenceReferenceInExpr(&expr, usedSequenceIDsToNames, builtinsregistry.GetBuiltinProperties)
hasUpgraded, err := seqexpr.UpgradeSequenceReferenceInExpr(&expr, usedSequenceIDsToNames)
require.NoError(t, err)
require.True(t, hasUpgraded)
require.Equal(t,
Expand All @@ -215,7 +214,7 @@ func TestUpgradeSequenceReferenceInExpr(t *testing.T) {
usedSequenceIDsToNames[1] = &tbl1
usedSequenceIDsToNames[2] = &tbl2
expr := "nextval('t')"
_, err := seqexpr.UpgradeSequenceReferenceInExpr(&expr, usedSequenceIDsToNames, builtinsregistry.GetBuiltinProperties)
_, err := seqexpr.UpgradeSequenceReferenceInExpr(&expr, usedSequenceIDsToNames)
require.Error(t, err, "ambiguous name matching for 't'; both 'sc1.t' and 'sc2.t' match it.")
require.Equal(t, "more than 1 matches found for \"t\"", err.Error())
})
Expand All @@ -227,7 +226,7 @@ func TestUpgradeSequenceReferenceInExpr(t *testing.T) {
usedSequenceIDsToNames[1] = &tbl1
usedSequenceIDsToNames[2] = &tbl2
expr := "nextval('t2')"
_, err := seqexpr.UpgradeSequenceReferenceInExpr(&expr, usedSequenceIDsToNames, builtinsregistry.GetBuiltinProperties)
_, err := seqexpr.UpgradeSequenceReferenceInExpr(&expr, usedSequenceIDsToNames)
require.Error(t, err, "no matching name for 't2'; neither 'sc1.t' nor 'sc2.t' match it.")
require.Equal(t, "no table name found to match input \"t2\"", err.Error())
})
Expand All @@ -241,7 +240,7 @@ func TestUpgradeSequenceReferenceInExpr(t *testing.T) {
usedSequenceIDsToNames[2] = &tbl2
usedSequenceIDsToNames[3] = &tbl3
expr := "((nextval(1::REGCLASS) + nextval(2::REGCLASS)) + currval(3::REGCLASS)) + nextval(3::REGCLASS)"
hasUpgraded, err := seqexpr.UpgradeSequenceReferenceInExpr(&expr, usedSequenceIDsToNames, builtinsregistry.GetBuiltinProperties)
hasUpgraded, err := seqexpr.UpgradeSequenceReferenceInExpr(&expr, usedSequenceIDsToNames)
require.NoError(t, err)
require.False(t, hasUpgraded)
require.Equal(t,
Expand All @@ -258,7 +257,7 @@ func TestUpgradeSequenceReferenceInExpr(t *testing.T) {
usedSequenceIDsToNames[2] = &tbl2
usedSequenceIDsToNames[3] = &tbl3
expr := "nextval('testdb.public.s1') + nextval('testdb.public.s2') + currval('testdb.sc1.s3') + nextval('testdb.sc1.s3')"
hasUpgraded, err := seqexpr.UpgradeSequenceReferenceInExpr(&expr, usedSequenceIDsToNames, builtinsregistry.GetBuiltinProperties)
hasUpgraded, err := seqexpr.UpgradeSequenceReferenceInExpr(&expr, usedSequenceIDsToNames)
require.NoError(t, err)
require.True(t, hasUpgraded)
require.Equal(t,
Expand All @@ -275,7 +274,7 @@ func TestUpgradeSequenceReferenceInExpr(t *testing.T) {
usedSequenceIDsToNames[2] = &tbl2
usedSequenceIDsToNames[3] = &tbl3
expr := "nextval('testdb.public.s1') + nextval(2::REGCLASS) + currval('testdb.sc1.s3') + nextval('testdb.sc1.s3')"
hasUpgraded, err := seqexpr.UpgradeSequenceReferenceInExpr(&expr, usedSequenceIDsToNames, builtinsregistry.GetBuiltinProperties)
hasUpgraded, err := seqexpr.UpgradeSequenceReferenceInExpr(&expr, usedSequenceIDsToNames)
require.NoError(t, err)
require.True(t, hasUpgraded)
require.Equal(t,
Expand Down
Loading

0 comments on commit 4c8e32a

Please sign in to comment.