diff --git a/pkg/BUILD.bazel b/pkg/BUILD.bazel index c5c76e97e16..a35678eccc9 100644 --- a/pkg/BUILD.bazel +++ b/pkg/BUILD.bazel @@ -549,6 +549,7 @@ ALL_TESTS = [ "//pkg/sql/sem/tree:tree_disallowed_imports_test", "//pkg/sql/sem/tree:tree_test", "//pkg/sql/sessiondata:sessiondata_test", + "//pkg/sql/sessiondatapb:sessiondatapb_test", "//pkg/sql/sessioninit:sessioninit_test", "//pkg/sql/span:span_test", "//pkg/sql/sqlinstance/instancestorage:instancestorage_test", @@ -2051,6 +2052,7 @@ GO_TARGETS = [ "//pkg/sql/sessiondata:sessiondata", "//pkg/sql/sessiondata:sessiondata_test", "//pkg/sql/sessiondatapb:sessiondatapb", + "//pkg/sql/sessiondatapb:sessiondatapb_test", "//pkg/sql/sessioninit:sessioninit", "//pkg/sql/sessioninit:sessioninit_test", "//pkg/sql/sessionphase:sessionphase", diff --git a/pkg/jobs/jobspb/jobs.proto b/pkg/jobs/jobspb/jobs.proto index 483e3a9fa25..6a621ab9245 100644 --- a/pkg/jobs/jobspb/jobs.proto +++ b/pkg/jobs/jobspb/jobs.proto @@ -887,7 +887,9 @@ message SchemaChangeDetails { (gogoproto.customtype) = "github.com/cockroachdb/cockroach/pkg/util/uuid.UUID" ]; - // Next id 13. + sessiondatapb.SessionData session_data = 13; + + // Next id 14. } message SchemaChangeProgress { diff --git a/pkg/sql/colexec/colbuilder/execplan.go b/pkg/sql/colexec/colbuilder/execplan.go index f20bde748d9..7cba0d80670 100644 --- a/pkg/sql/colexec/colbuilder/execplan.go +++ b/pkg/sql/colexec/colbuilder/execplan.go @@ -257,7 +257,7 @@ var ( errExporterWrap = errors.New("core.Exporter is not supported (not an execinfra.RowSource)") errSamplerWrap = errors.New("core.Sampler is not supported (not an execinfra.RowSource)") errSampleAggregatorWrap = errors.New("core.SampleAggregator is not supported (not an execinfra.RowSource)") - errExperimentalWrappingProhibited = errors.New("wrapping for non-JoinReader and non-LocalPlanNode cores is prohibited in vectorize=experimental_always") + errExperimentalWrappingProhibited = errors.Newf("wrapping for non-JoinReader and non-LocalPlanNode cores is prohibited in vectorize=%s", sessiondatapb.VectorizeExperimentalAlways) errWrappedCast = errors.New("mismatched types in NewColOperator and unsupported casts") errLookupJoinUnsupported = errors.New("lookup join reader is unsupported in vectorized") errFilteringAggregation = errors.New("filtering aggregation not supported") diff --git a/pkg/sql/create_as_test.go b/pkg/sql/create_as_test.go index ee18f4a98de..d1c277084ed 100644 --- a/pkg/sql/create_as_test.go +++ b/pkg/sql/create_as_test.go @@ -129,7 +129,6 @@ func TestCreateAsShow(t *testing.T) { testCases := []struct { sql string setup string - skip bool }{ { sql: "SHOW CLUSTER SETTINGS", @@ -165,9 +164,6 @@ func TestCreateAsShow(t *testing.T) { { sql: "SHOW CREATE FUNCTION show_create_fn", setup: "CREATE FUNCTION show_create_fn(i int) RETURNS INT AS 'SELECT i' LANGUAGE SQL", - // TODO(sql-foundations): Fix `unknown function: show_create_fn(): function undefined` error in job. - // See https://github.com/cockroachdb/cockroach/issues/106268. - skip: true, }, { sql: "SHOW CREATE ALL TYPES", @@ -294,9 +290,6 @@ func TestCreateAsShow(t *testing.T) { for i, testCase := range testCases { t.Run(testCase.sql, func(t *testing.T) { - if testCase.skip { - return - } if testCase.setup != "" { if s.StartedDefaultTestTenant() && strings.Contains(testCase.setup, "create_tenant") { // Only the system tenant has the ability to create other diff --git a/pkg/sql/delegate/show_function.go b/pkg/sql/delegate/show_function.go index 1f3d6996c1a..717337aa301 100644 --- a/pkg/sql/delegate/show_function.go +++ b/pkg/sql/delegate/show_function.go @@ -27,12 +27,20 @@ FROM crdb_internal.create_function_statements WHERE schema_name = %[1]s AND function_name = %[2]s ` - un, ok := n.Name.FunctionReference.(*tree.UnresolvedName) + resolvableFunctionReference := &n.Name + un, ok := resolvableFunctionReference.FunctionReference.(*tree.UnresolvedName) if !ok { return nil, errors.AssertionFailedf("not a valid function name") } - fn, err := d.catalog.ResolveFunction(d.ctx, un, &d.evalCtx.SessionData().SearchPath) + searchPath := &d.evalCtx.SessionData().SearchPath + var fn *tree.ResolvedFunctionDefinition + var err error + if d.qualifyDataSourceNamesInAST { + fn, err = resolvableFunctionReference.Resolve(d.ctx, searchPath, d.catalog) + } else { + fn, err = d.catalog.ResolveFunction(d.ctx, un, searchPath) + } if err != nil { return nil, err } @@ -41,12 +49,26 @@ AND function_name = %[2]s for _, o := range fn.Overloads { if o.IsUDF { udfSchema = o.Schema + break } } if udfSchema == "" { return nil, errors.Errorf("function %s does not exist", tree.AsString(un)) } + if d.qualifyDataSourceNamesInAST { + referenceByName := resolvableFunctionReference.ReferenceByName + if !referenceByName.HasExplicitSchema() { + referenceByName.Parts[1] = udfSchema + } + if !referenceByName.HasExplicitCatalog() { + referenceByName.Parts[2] = d.evalCtx.SessionData().Database + } + if referenceByName.NumParts < 3 { + referenceByName.NumParts = 3 + } + } + fullQuery := fmt.Sprintf(query, lexbase.EscapeSQLString(udfSchema), lexbase.EscapeSQLString(un.Parts[0])) return d.parse(fullQuery) } diff --git a/pkg/sql/drop_function.go b/pkg/sql/drop_function.go index 048f85ca9a4..79a54b99012 100644 --- a/pkg/sql/drop_function.go +++ b/pkg/sql/drop_function.go @@ -279,6 +279,7 @@ func (p *planner) writeDropFuncSchemaChange(ctx context.Context, funcDesc *funcd DescriptorIDs: descpb.IDs{funcDesc.ID}, Details: jobspb.SchemaChangeDetails{ DroppedFunctions: descpb.IDs{funcDesc.ID}, + SessionData: &p.SessionData().SessionData, }, Progress: jobspb.TypeSchemaChangeProgress{}, } diff --git a/pkg/sql/drop_schema.go b/pkg/sql/drop_schema.go index acb5b816aea..2cee9052ef4 100644 --- a/pkg/sql/drop_schema.go +++ b/pkg/sql/drop_schema.go @@ -258,6 +258,7 @@ func (p *planner) createDropSchemaJob( // The version distinction for database jobs doesn't matter for jobs that // drop schemas. FormatVersion: jobspb.DatabaseJobFormatVersion, + SessionData: &p.SessionData().SessionData, }, Progress: jobspb.SchemaChangeProgress{}, NonCancelable: true, diff --git a/pkg/sql/exec_util.go b/pkg/sql/exec_util.go index 7bac5de6c98..b210a3b371b 100644 --- a/pkg/sql/exec_util.go +++ b/pkg/sql/exec_util.go @@ -588,12 +588,13 @@ var VectorizeClusterMode = settings.RegisterEnumSetting( VectorizeClusterSettingName, "default vectorize mode", "on", - map[int64]string{ - int64(sessiondatapb.VectorizeUnset): "on", - int64(sessiondatapb.VectorizeOn): "on", - int64(sessiondatapb.VectorizeExperimentalAlways): "experimental_always", - int64(sessiondatapb.VectorizeOff): "off", - }, + func() map[int64]string { + m := make(map[int64]string, len(sessiondatapb.VectorizeExecMode_name)) + for k := range sessiondatapb.VectorizeExecMode_name { + m[int64(k)] = sessiondatapb.VectorizeExecMode(k).String() + } + return m + }(), ).WithPublic() // DistSQLClusterExecMode controls the cluster default for when DistSQL is used. diff --git a/pkg/sql/logictest/testdata/logic_test/builtin_function b/pkg/sql/logictest/testdata/logic_test/builtin_function index 6a22923eb39..966cd754b46 100644 --- a/pkg/sql/logictest/testdata/logic_test/builtin_function +++ b/pkg/sql/logictest/testdata/logic_test/builtin_function @@ -2989,7 +2989,7 @@ select crdb_internal.json_to_pb('cockroach.sql.sqlbase.Descriptor', crdb_interna ---- true -query error pq: crdb_internal.json_to_pb\(\): invalid proto JSON: unmarshaling json to cockroach.sql.sqlbase.Descriptor: unknown field "__redacted__" in descpb.Descriptor +query error pq: crdb_internal.json_to_pb\(\): invalid proto JSON: unmarshaling to cockroach.sql.sqlbase.Descriptor json: .+ unknown field "__redacted__" in descpb.Descriptor select crdb_internal.json_to_pb('cockroach.sql.sqlbase.Descriptor', crdb_internal.pb_to_json('cockroach.sql.sqlbase.Descriptor', descriptor, true, true)) = descriptor from system.descriptor where id = 1 subtest regexp_split diff --git a/pkg/sql/logictest/testdata/logic_test/set b/pkg/sql/logictest/testdata/logic_test/set index 830c8f4bb19..02e508d3fd2 100644 --- a/pkg/sql/logictest/testdata/logic_test/set +++ b/pkg/sql/logictest/testdata/logic_test/set @@ -243,7 +243,7 @@ SET vectorize = experimental_always statement ok SET vectorize = off -statement error invalid value for parameter "vectorize": "bogus" +statement error invalid value for parameter "vectorize": "bogus"\nHINT: Available values: off,on,experimental_always SET vectorize = bogus statement ok diff --git a/pkg/sql/logictest/testdata/logic_test/vectorize b/pkg/sql/logictest/testdata/logic_test/vectorize index 4282fdc23c2..63493a223e4 100644 --- a/pkg/sql/logictest/testdata/logic_test/vectorize +++ b/pkg/sql/logictest/testdata/logic_test/vectorize @@ -3,6 +3,8 @@ # Note that there is no much benefit from running this file with other logic # test configurations because we override vectorize setting. +subtest todo1 + # NOTE: all queries in this file should run with vectorize=experimental_always # unless they are known to be unsupported (like generate_series, etc). If you do # need to execute an unsupported query, follow the pattern of @@ -605,6 +607,14 @@ SELECT EXTRACT(YEAR FROM x) FROM extract_test ---- 2017 +statement ok +RESET vectorize + +subtest end + +subtest 38937 + + # Regression test for #38937 statement ok CREATE TABLE t38937 (_int2) AS SELECT 1::INT2; @@ -614,6 +624,13 @@ SELECT sum_int(_int2) FROM t38937 ---- 1 +subtest end + +subtest todo2 + +statement ok +SET vectorize=experimental_always + # Regression tests for #38959 statement ok @@ -1167,6 +1184,13 @@ SELECT crdb_internal_mvcc_timestamp IS NOT NULL FROM mvcc@i ---- true +statement ok +RESET vectorize + +subtest end + +subtest 51841 + # Regression test for builtin funcs that take no arguments, combined with # data that can't be safely decoded as input. @@ -1176,6 +1200,10 @@ CREATE TABLE t51841 (a) AS SELECT gen_random_uuid(); statement ok SELECT random() from t51841 +subtest end + +subtest todo3 + # Regression test to ensure that we can EXPLAIN (VEC) plans that contain user # defined types. We aren't interested in the output here, but are just ensuring # that we don't panic. @@ -1247,6 +1275,13 @@ SELECT b FROM t66706@u WHERE NOT (b = 'foo') bar bar +statement ok +RESET vectorize + +subtest end + +subtest 68040 + # Regression test for ignoring the escaping in the LIKE pattern (#68040). statement ok CREATE TABLE t68040 (c) AS SELECT 'string with \ backslash' @@ -1256,6 +1291,13 @@ SELECT c FROM t68040 WHERE c LIKE '%\\%' ---- string with \ backslash +subtest end + +subtest todo4 + +statement ok +SET vectorize=experimental_always + # Regression test for #68979. The IN operator should evaluate correctly when the # tuple contents are not sorted by the optimizer. statement ok @@ -1296,3 +1338,8 @@ query I SELECT _int2 * _int2 FROM ints WHERE _int4 + _int4 = _int8 + 2 ---- 4 + +statement ok +RESET vectorize + +subtest end diff --git a/pkg/sql/protoreflect/utils.go b/pkg/sql/protoreflect/utils.go index 643f928e9de..ff75aa34f31 100644 --- a/pkg/sql/protoreflect/utils.go +++ b/pkg/sql/protoreflect/utils.go @@ -123,8 +123,9 @@ func MessageToJSON(msg protoutil.Message, flags FmtFlags) (jsonb.JSON, error) { // Returns serialized byte representation of the protocol message. func JSONBMarshalToMessage(input jsonb.JSON, target protoutil.Message) ([]byte, error) { json := &jsonpb.Unmarshaler{} - if err := json.Unmarshal(strings.NewReader(input.String()), target); err != nil { - return nil, errors.Wrapf(err, "unmarshaling json to %s", proto.MessageName(target)) + jsonString := input.String() + if err := json.Unmarshal(strings.NewReader(jsonString), target); err != nil { + return nil, errors.Wrapf(err, "unmarshaling to %s json: %s", proto.MessageName(target), jsonString) } data, err := protoutil.Marshal(target) if err != nil { diff --git a/pkg/sql/schema.go b/pkg/sql/schema.go index 6495384d0a5..3a82e432855 100644 --- a/pkg/sql/schema.go +++ b/pkg/sql/schema.go @@ -73,6 +73,7 @@ func (p *planner) writeSchemaDescChange( // The version distinction for database jobs doesn't matter for schema // jobs. FormatVersion: jobspb.DatabaseJobFormatVersion, + SessionData: &p.SessionData().SessionData, }, Progress: jobspb.SchemaChangeProgress{}, NonCancelable: true, diff --git a/pkg/sql/schema_changer.go b/pkg/sql/schema_changer.go index 479546a409c..2a24bc6d939 100644 --- a/pkg/sql/schema_changer.go +++ b/pkg/sql/schema_changer.go @@ -51,6 +51,7 @@ import ( "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/sessiondatapb" "github.com/cockroachdb/cockroach/pkg/sql/sqlerrors" "github.com/cockroachdb/cockroach/pkg/sql/sqltelemetry" "github.com/cockroachdb/cockroach/pkg/storage" @@ -156,6 +157,7 @@ type SchemaChanger struct { clock *hlc.Clock settings *cluster.Settings execCfg *ExecutorConfig + sessionData *sessiondatapb.SessionData } // NewSchemaChangerForTesting only for tests. @@ -305,6 +307,8 @@ func (sc *SchemaChanger) backfillQueryIntoTable( return err } + sd := NewInternalSessionData(ctx, sc.execCfg.Settings, "backfillQueryIntoTable") + sd.SessionData = *sc.sessionData // Create an internal planner as the planner used to serve the user query // would have committed by this point. p, cleanup := NewInternalPlanner( @@ -313,7 +317,7 @@ func (sc *SchemaChanger) backfillQueryIntoTable( username.RootUserName(), &MemoryMetrics{}, sc.execCfg, - NewInternalSessionData(ctx, sc.execCfg.Settings, "backfillQueryIntoTable"), + sd, ) defer cleanup() @@ -2155,6 +2159,7 @@ func (sc *SchemaChanger) updateJobForRollback( TableMutationID: sc.mutationID, ResumeSpanList: spanList, FormatVersion: oldDetails.FormatVersion, + SessionData: sc.sessionData, }, ); err != nil { return err @@ -2611,6 +2616,7 @@ func (r schemaChangeResumer) Resume(ctx context.Context, execCtx interface{}) er settings: p.ExecCfg().Settings, execCfg: p.ExecCfg(), metrics: p.ExecCfg().SchemaChangerMetrics, + sessionData: details.SessionData, } opts := retry.Options{ InitialBackoff: 20 * time.Millisecond, @@ -2828,6 +2834,7 @@ func (r schemaChangeResumer) OnFailOrCancel( clock: p.ExecCfg().Clock, settings: p.ExecCfg().Settings, execCfg: p.ExecCfg(), + sessionData: details.SessionData, } if r.job.Payload().FinalResumeError == nil { @@ -2931,6 +2938,7 @@ func (sc *SchemaChanger) queueCleanupJob( // The version distinction for database jobs doesn't matter for jobs on // tables. FormatVersion: jobspb.DatabaseJobFormatVersion, + SessionData: sc.sessionData, }, Progress: jobspb.SchemaChangeProgress{}, NonCancelable: true, diff --git a/pkg/sql/sessiondatapb/BUILD.bazel b/pkg/sql/sessiondatapb/BUILD.bazel index 9dbeb163e92..fb0ad3c96b4 100644 --- a/pkg/sql/sessiondatapb/BUILD.bazel +++ b/pkg/sql/sessiondatapb/BUILD.bazel @@ -1,6 +1,6 @@ load("@rules_proto//proto:defs.bzl", "proto_library") load("@io_bazel_rules_go//proto:def.bzl", "go_proto_library") -load("@io_bazel_rules_go//go:def.bzl", "go_library") +load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") go_library( name = "sessiondatapb", @@ -54,3 +54,14 @@ go_proto_library( "@com_github_gogo_protobuf//gogoproto", ], ) + +go_test( + name = "sessiondatapb_test", + srcs = ["session_data_test.go"], + args = ["-test.timeout=295s"], + embed = [":sessiondatapb"], + deps = [ + "//pkg/sql/protoreflect", + "@com_github_stretchr_testify//require", + ], +) diff --git a/pkg/sql/sessiondatapb/session_data.go b/pkg/sql/sessiondatapb/session_data.go index 4564df0003e..d0db8c1c640 100644 --- a/pkg/sql/sessiondatapb/session_data.go +++ b/pkg/sql/sessiondatapb/session_data.go @@ -55,30 +55,26 @@ func (c DataConversionConfig) GetFloatPrec(typ *types.T) int { } func (m VectorizeExecMode) String() string { - switch m { - case VectorizeOn, VectorizeUnset: - return "on" - case VectorizeExperimentalAlways: - return "experimental_always" - case VectorizeOff: - return "off" - default: + if m == VectorizeUnset { + m = VectorizeOn + } + name, ok := VectorizeExecMode_name[int32(m)] + if !ok { return fmt.Sprintf("invalid (%d)", m) } + return name } // VectorizeExecModeFromString converts a string into a VectorizeExecMode. // False is returned if the conversion was unsuccessful. func VectorizeExecModeFromString(val string) (VectorizeExecMode, bool) { - var m VectorizeExecMode - switch strings.ToUpper(val) { - case "ON": - m = VectorizeOn - case "EXPERIMENTAL_ALWAYS": - m = VectorizeExperimentalAlways - case "OFF": - m = VectorizeOff - default: + lowerVal := strings.ToLower(val) + mInt, ok := VectorizeExecMode_value[lowerVal] + if !ok { + return 0, false + } + m := VectorizeExecMode(mInt) + if m == VectorizeUnset { return 0, false } return m, true diff --git a/pkg/sql/sessiondatapb/session_data.proto b/pkg/sql/sessiondatapb/session_data.proto index 089f23698a4..47656cb7f36 100644 --- a/pkg/sql/sessiondatapb/session_data.proto +++ b/pkg/sql/sessiondatapb/session_data.proto @@ -139,16 +139,16 @@ enum VectorizeExecMode { // VectorizeUnset means the VectorizeExecMode wasn't explicitly set. Having // the first enum value as zero is required by proto3. This is mapped to // VectorizeOn. - VectorizeUnset = 0; + unset = 0 [(gogoproto.enumvalue_customname) = "VectorizeUnset"]; reserved 1; // VectorizeOn means that any supported queries will be run using the // columnar execution. - VectorizeOn = 2; + on = 2 [(gogoproto.enumvalue_customname) = "VectorizeOn"]; // VectorizeExperimentalAlways means that we attempt to vectorize all // queries; unsupported queries will fail. Mostly used for testing. - VectorizeExperimentalAlways = 3; + experimental_always = 3 [(gogoproto.enumvalue_customname) = "VectorizeExperimentalAlways"]; // VectorizeOff means that columnar execution is disabled. - VectorizeOff = 4; + off = 4 [(gogoproto.enumvalue_customname) = "VectorizeOff"]; } // SequenceState is used to marshall the sessiondata.SequenceState struct. diff --git a/pkg/sql/sessiondatapb/session_data_test.go b/pkg/sql/sessiondatapb/session_data_test.go new file mode 100644 index 00000000000..6df46f45545 --- /dev/null +++ b/pkg/sql/sessiondatapb/session_data_test.go @@ -0,0 +1,30 @@ +// Copyright 2023 The Cockroach Authors. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +package sessiondatapb + +import ( + "testing" + + "github.com/cockroachdb/cockroach/pkg/sql/protoreflect" + "github.com/stretchr/testify/require" +) + +func TestSessionDataJsonCompat(t *testing.T) { + expectedSessionData := SessionData{ + VectorizeMode: VectorizeOn, + } + json, err := protoreflect.MessageToJSON(&expectedSessionData, protoreflect.FmtFlags{}) + require.NoError(t, err) + actualSessionData := SessionData{} + _, err = protoreflect.JSONBMarshalToMessage(json, &actualSessionData) + require.NoError(t, err) + require.Equal(t, expectedSessionData, actualSessionData) +} diff --git a/pkg/sql/table.go b/pkg/sql/table.go index 0917b8e950d..22fbf4659fa 100644 --- a/pkg/sql/table.go +++ b/pkg/sql/table.go @@ -71,6 +71,7 @@ func (p *planner) createDropDatabaseJob( DroppedFunctions: funcIDs, DroppedDatabaseID: databaseID, FormatVersion: jobspb.DatabaseJobFormatVersion, + SessionData: &p.SessionData().SessionData, }, Progress: jobspb.SchemaChangeProgress{}, NonCancelable: true, @@ -92,6 +93,7 @@ func (p *planner) createNonDropDatabaseChangeJob( Details: jobspb.SchemaChangeDetails{ DescID: databaseID, FormatVersion: jobspb.DatabaseJobFormatVersion, + SessionData: &p.SessionData().SessionData, }, Progress: jobspb.SchemaChangeProgress{}, NonCancelable: true, @@ -163,6 +165,7 @@ func (p *planner) createOrUpdateSchemaChangeJob( // The version distinction for database jobs doesn't matter for jobs on // tables. FormatVersion: jobspb.DatabaseJobFormatVersion, + SessionData: &p.SessionData().SessionData, }, Progress: jobspb.SchemaChangeProgress{}, // Mark jobs without a mutation ID as non-cancellable, @@ -193,6 +196,7 @@ func (p *planner) createOrUpdateSchemaChangeJob( // The version distinction for database jobs doesn't matter for jobs on // tables. FormatVersion: jobspb.DatabaseJobFormatVersion, + SessionData: &p.SessionData().SessionData, } if oldDetails.TableMutationID != descpb.InvalidMutationID { // The previous queued schema change job was associated with a mutation, diff --git a/pkg/sql/vars.go b/pkg/sql/vars.go index 1ab24352fc2..9664558daf0 100644 --- a/pkg/sql/vars.go +++ b/pkg/sql/vars.go @@ -688,8 +688,13 @@ var varGen = map[string]sessionVar{ Set: func(_ context.Context, m sessionDataMutator, s string) error { mode, ok := sessiondatapb.VectorizeExecModeFromString(s) if !ok { - return newVarValueError(`vectorize`, s, - "off", "on", "experimental_always") + return newVarValueError( + `vectorize`, + s, + sessiondatapb.VectorizeOff.String(), + sessiondatapb.VectorizeOn.String(), + sessiondatapb.VectorizeExperimentalAlways.String(), + ) } m.SetVectorize(mode) return nil