From 0e3822e31ae2be9b50b6e1f9491d7e30a17f4a71 Mon Sep 17 00:00:00 2001 From: Yahor Yuzefovich Date: Wed, 14 Oct 2020 14:55:15 -0700 Subject: [PATCH] sessiondata: extract many fields into a protobuf This commit extract all "simple" fields from `sessiondata.SessionData` object that need to be serialized and sent to the remote nodes in order to get the correct execution into a protobuf. All fields in `SessionData` object have been audited to make sure that all the necessary ones are serialized. Only one omission was found - we were not propagating the default int size - and it is now fixed. Release note: None --- pkg/ccl/importccl/testutils_test.go | 2 +- pkg/cli/sql_util.go | 3 +- pkg/sql/colexec/colbuilder/execplan.go | 12 +- pkg/sql/colexec/window_functions_test.go | 2 - pkg/sql/colflow/vectorized_flow.go | 6 +- pkg/sql/conn_executor.go | 26 +- pkg/sql/conn_executor_prepare.go | 2 +- pkg/sql/conn_io.go | 5 +- pkg/sql/distsql/server.go | 29 +- pkg/sql/distsql_running.go | 12 +- pkg/sql/exec_util.go | 22 +- pkg/sql/execinfra/version.go | 9 +- pkg/sql/execinfrapb/api.go | 28 +- pkg/sql/execinfrapb/api.pb.go | 456 ++++-------- pkg/sql/execinfrapb/api.proto | 20 +- pkg/sql/execinfrapb/expr.go | 5 +- pkg/sql/explain_plan.go | 6 +- pkg/sql/explain_vec.go | 4 +- pkg/sql/internal.go | 7 +- pkg/sql/internal_test.go | 24 +- pkg/sql/lex/encode.go | 66 +- pkg/sql/lex/encode_test.go | 18 +- pkg/sql/parser/parse.go | 47 +- pkg/sql/pgwire/command_result.go | 14 +- pkg/sql/pgwire/conn.go | 14 +- pkg/sql/pgwire/conn_test.go | 15 +- pkg/sql/pgwire/encoding_test.go | 9 +- pkg/sql/pgwire/types.go | 10 +- pkg/sql/pgwire/types_test.go | 41 +- pkg/sql/planner.go | 11 +- pkg/sql/schema_changer.go | 29 +- pkg/sql/sem/builtins/builtins.go | 5 +- pkg/sql/sem/tree/casts.go | 9 +- pkg/sql/sem/tree/datum_invariants_test.go | 4 +- pkg/sql/sem/tree/datum_test.go | 4 +- pkg/sql/sem/tree/eval.go | 17 +- pkg/sql/sessiondata/session_data.go | 143 +--- pkg/sql/sessiondatapb/session_data.go | 111 +++ pkg/sql/sessiondatapb/session_data.pb.go | 813 ++++++++++++++++++++++ pkg/sql/sessiondatapb/session_data.proto | 91 +++ pkg/sql/set_cluster_setting.go | 3 +- pkg/sql/vars.go | 20 +- pkg/workload/dep_test.go | 1 + 43 files changed, 1414 insertions(+), 761 deletions(-) create mode 100644 pkg/sql/sessiondatapb/session_data.go create mode 100644 pkg/sql/sessiondatapb/session_data.pb.go create mode 100644 pkg/sql/sessiondatapb/session_data.proto diff --git a/pkg/ccl/importccl/testutils_test.go b/pkg/ccl/importccl/testutils_test.go index 805d29533782..3c1b7c1d126a 100644 --- a/pkg/ccl/importccl/testutils_test.go +++ b/pkg/ccl/importccl/testutils_test.go @@ -86,7 +86,7 @@ func descForTable( var testEvalCtx = &tree.EvalContext{ SessionData: &sessiondata.SessionData{ - DataConversion: sessiondata.DataConversionConfig{Location: time.UTC}, + Location: time.UTC, }, StmtTimestamp: timeutil.Unix(100000000, 0), Settings: cluster.MakeTestingClusterSettings(), diff --git a/pkg/cli/sql_util.go b/pkg/cli/sql_util.go index 81f8129f993e..1c696cb54ec9 100644 --- a/pkg/cli/sql_util.go +++ b/pkg/cli/sql_util.go @@ -31,6 +31,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/lex" "github.com/cockroachdb/cockroach/pkg/sql/parser" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" + "github.com/cockroachdb/cockroach/pkg/sql/sessiondatapb" "github.com/cockroachdb/cockroach/pkg/util/envutil" "github.com/cockroachdb/cockroach/pkg/util/errorutil/unimplemented" "github.com/cockroachdb/cockroach/pkg/util/log" @@ -1154,7 +1155,7 @@ func formatVal(val driver.Value, showPrintableUnicode bool, showNewLinesAndTabs // that we can let the user see and control the result using // `bytea_output`. return lex.EncodeByteArrayToRawBytes(string(t), - lex.BytesEncodeEscape, false /* skipHexPrefix */) + sessiondatapb.BytesEncodeEscape, false /* skipHexPrefix */) case time.Time: // Since we do not know whether the datum is Timestamp or TimestampTZ, diff --git a/pkg/sql/colexec/colbuilder/execplan.go b/pkg/sql/colexec/colbuilder/execplan.go index d21aa451f04e..c6b37bcfb671 100644 --- a/pkg/sql/colexec/colbuilder/execplan.go +++ b/pkg/sql/colexec/colbuilder/execplan.go @@ -29,7 +29,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/execinfra" "github.com/cockroachdb/cockroach/pkg/sql/execinfrapb" "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/types" "github.com/cockroachdb/cockroach/pkg/util" "github.com/cockroachdb/cockroach/pkg/util/log" @@ -147,10 +147,10 @@ func needHashAggregator(aggSpec *execinfrapb.AggregatorSpec) (bool, error) { // isSupported checks whether we have a columnar operator equivalent to a // processor described by spec. Note that it doesn't perform any other checks // (like validity of the number of inputs). -func isSupported(mode sessiondata.VectorizeExecMode, spec *execinfrapb.ProcessorSpec) error { +func isSupported(mode sessiondatapb.VectorizeExecMode, spec *execinfrapb.ProcessorSpec) error { core := spec.Core - isFullVectorization := mode == sessiondata.VectorizeOn || - mode == sessiondata.VectorizeExperimentalAlways + isFullVectorization := mode == sessiondatapb.VectorizeOn || + mode == sessiondatapb.VectorizeExperimentalAlways switch { case core.Noop != nil: @@ -429,9 +429,9 @@ func (r opResult) createAndWrapRowSource( } if spec.Core.JoinReader == nil { switch flowCtx.EvalCtx.SessionData.VectorizeMode { - case sessiondata.Vectorize201Auto: + case sessiondatapb.Vectorize201Auto: return errors.New("rowexec processor wrapping for non-JoinReader core unsupported in vectorize=201auto mode") - case sessiondata.VectorizeExperimentalAlways: + case sessiondatapb.VectorizeExperimentalAlways: return causeToWrap } } diff --git a/pkg/sql/colexec/window_functions_test.go b/pkg/sql/colexec/window_functions_test.go index 270216905d99..53c423d6ddf2 100644 --- a/pkg/sql/colexec/window_functions_test.go +++ b/pkg/sql/colexec/window_functions_test.go @@ -19,7 +19,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/execinfra" "github.com/cockroachdb/cockroach/pkg/sql/execinfrapb" "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/testutils/colcontainerutils" "github.com/cockroachdb/cockroach/pkg/util/leaktest" @@ -48,7 +47,6 @@ func TestWindowFunctions(t *testing.T) { st := cluster.MakeTestingClusterSettings() evalCtx := tree.MakeTestingEvalContext(st) defer evalCtx.Stop(ctx) - evalCtx.SessionData.VectorizeMode = sessiondata.VectorizeOn flowCtx := &execinfra.FlowCtx{ EvalCtx: &evalCtx, Cfg: &execinfra.ServerConfig{ diff --git a/pkg/sql/colflow/vectorized_flow.go b/pkg/sql/colflow/vectorized_flow.go index 84053cbd20a5..f6f3fc923ce0 100644 --- a/pkg/sql/colflow/vectorized_flow.go +++ b/pkg/sql/colflow/vectorized_flow.go @@ -37,7 +37,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgcode" "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror" "github.com/cockroachdb/cockroach/pkg/sql/rowexec" - "github.com/cockroachdb/cockroach/pkg/sql/sessiondata" + "github.com/cockroachdb/cockroach/pkg/sql/sessiondatapb" "github.com/cockroachdb/cockroach/pkg/sql/types" "github.com/cockroachdb/cockroach/pkg/util" "github.com/cockroachdb/cockroach/pkg/util/log" @@ -1034,7 +1034,7 @@ func (s *vectorizedFlowCreator) setupFlow( if flowCtx.EvalCtx.SessionData.TestingVectorizeInjectPanics { result.Op = colexec.NewPanicInjector(result.Op) } - if flowCtx.EvalCtx.SessionData.VectorizeMode == sessiondata.Vectorize201Auto && + if flowCtx.EvalCtx.SessionData.VectorizeMode == sessiondatapb.Vectorize201Auto && !result.IsStreaming { err = errors.Errorf("non-streaming operator encountered when vectorize=201auto") return @@ -1076,7 +1076,7 @@ func (s *vectorizedFlowCreator) setupFlow( } } - if (flowCtx.EvalCtx.SessionData.VectorizeMode == sessiondata.Vectorize201Auto) && + if (flowCtx.EvalCtx.SessionData.VectorizeMode == sessiondatapb.Vectorize201Auto) && pspec.Output[0].Type == execinfrapb.OutputRouterSpec_BY_HASH { // colexec.HashRouter is not supported when vectorize=auto since it can // buffer an unlimited number of tuples, even though it falls back to diff --git a/pkg/sql/conn_executor.go b/pkg/sql/conn_executor.go index 839eefa946e2..ed57493094e4 100644 --- a/pkg/sql/conn_executor.go +++ b/pkg/sql/conn_executor.go @@ -39,6 +39,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/rowenc" "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/types" "github.com/cockroachdb/cockroach/pkg/util" @@ -462,18 +463,13 @@ type ConnectionHandler struct { // GetUnqualifiedIntSize implements pgwire.sessionDataProvider and returns // the type that INT should be parsed as. func (h ConnectionHandler) GetUnqualifiedIntSize() *types.T { - var size int + var size int32 if h.ex != nil { // The executor will be nil in certain testing situations where // no server is actually present. size = h.ex.sessionData.DefaultIntSize } - switch size { - case 4, 32: - return types.Int4 - default: - return types.Int - } + return parser.NakedIntTypeFromDefaultIntSize(size) } // GetParamStatus retrieves the configured value of the session @@ -512,7 +508,9 @@ func (s *Server) ServeConn( // newSessionData a SessionData that can be passed to newConnExecutor. func (s *Server) newSessionData(args SessionArgs) *sessiondata.SessionData { sd := &sessiondata.SessionData{ - User: args.User, + SessionData: sessiondatapb.SessionData{ + User: args.User, + }, RemoteAddr: args.RemoteAddr, ResultsBufferSize: args.ConnResultsBufferSize, } @@ -537,10 +535,8 @@ func (s *Server) populateMinimalSessionData(sd *sessiondata.SessionData) { if sd.SequenceState == nil { sd.SequenceState = sessiondata.NewSequenceState() } - if sd.DataConversion == (sessiondata.DataConversionConfig{}) { - sd.DataConversion = sessiondata.DataConversionConfig{ - Location: time.UTC, - } + if sd.Location == nil { + sd.Location = time.UTC } if len(sd.SearchPath.GetPathArray()) == 0 { sd.SearchPath = sessiondata.DefaultSearchPathForUser(sd.User) @@ -1462,7 +1458,8 @@ func (ex *connExecutor) execCmd(ctx context.Context) error { NeedRowDesc, pos, nil, /* formatCodes */ - ex.sessionData.DataConversion, + ex.sessionData.DataConversionConfig, + ex.sessionData.Location, 0, /* limit */ "", /* portalName */ ex.implicitTxn(), @@ -1533,7 +1530,8 @@ func (ex *connExecutor) execCmd(ctx context.Context) error { // needed. DontNeedRowDesc, pos, portal.OutFormats, - ex.sessionData.DataConversion, + ex.sessionData.DataConversionConfig, + ex.sessionData.Location, tcmd.Limit, portalName, ex.implicitTxn(), diff --git a/pkg/sql/conn_executor_prepare.go b/pkg/sql/conn_executor_prepare.go index b9041f29715e..c49a82da4a3c 100644 --- a/pkg/sql/conn_executor_prepare.go +++ b/pkg/sql/conn_executor_prepare.go @@ -337,7 +337,7 @@ func (ex *connExecutor) execBind( "expected %d arguments, got %d", numQArgs, len(bindCmd.Args))) } - ptCtx := tree.NewParseTimeContext(ex.state.sqlTimestamp.In(ex.sessionData.DataConversion.Location)) + ptCtx := tree.NewParseTimeContext(ex.state.sqlTimestamp.In(ex.sessionData.Location)) for i, arg := range bindCmd.Args { k := tree.PlaceholderIdx(i) diff --git a/pkg/sql/conn_io.go b/pkg/sql/conn_io.go index f4e925c8fa39..a0e75edd7904 100644 --- a/pkg/sql/conn_io.go +++ b/pkg/sql/conn_io.go @@ -22,7 +22,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgnotice" "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgwirebase" "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/util/log" "github.com/cockroachdb/cockroach/pkg/util/ring" "github.com/cockroachdb/cockroach/pkg/util/syncutil" @@ -591,7 +591,8 @@ type ClientComm interface { descOpt RowDescOpt, pos CmdPos, formatCodes []pgwirebase.FormatCode, - conv sessiondata.DataConversionConfig, + conv sessiondatapb.DataConversionConfig, + location *time.Location, limit int, portalName string, implicitTxn bool, diff --git a/pkg/sql/distsql/server.go b/pkg/sql/distsql/server.go index da2a91fc109e..af7ea8ef454d 100644 --- a/pkg/sql/distsql/server.go +++ b/pkg/sql/distsql/server.go @@ -27,10 +27,10 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/execinfrapb" "github.com/cockroachdb/cockroach/pkg/sql/faketreeeval" "github.com/cockroachdb/cockroach/pkg/sql/flowinfra" - "github.com/cockroachdb/cockroach/pkg/sql/lex" "github.com/cockroachdb/cockroach/pkg/sql/rowflow" "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/sqltelemetry" "github.com/cockroachdb/cockroach/pkg/sql/sqlutil" "github.com/cockroachdb/cockroach/pkg/util/contextutil" @@ -268,34 +268,15 @@ func (ds *ServerImpl) setupFlow( return ctx, nil, err } - var be lex.BytesEncodeFormat - switch req.EvalContext.BytesEncodeFormat { - case execinfrapb.BytesEncodeFormat_HEX: - be = lex.BytesEncodeHex - case execinfrapb.BytesEncodeFormat_ESCAPE: - be = lex.BytesEncodeEscape - case execinfrapb.BytesEncodeFormat_BASE64: - be = lex.BytesEncodeBase64 - default: - return nil, nil, errors.AssertionFailedf("unknown byte encode format: %s", - errors.Safe(req.EvalContext.BytesEncodeFormat)) - } sd := &sessiondata.SessionData{ - ApplicationName: req.EvalContext.ApplicationName, - Database: req.EvalContext.Database, - User: req.EvalContext.User, + SessionData: req.EvalContext.SessionData, SearchPath: sessiondata.MakeSearchPath( req.EvalContext.SearchPath, ).WithTemporarySchemaName( req.EvalContext.TemporarySchemaName, - ).WithUserSchemaName(req.EvalContext.User), + ).WithUserSchemaName(req.EvalContext.SessionData.User), SequenceState: sessiondata.NewSequenceState(), - DataConversion: sessiondata.DataConversionConfig{ - Location: location, - BytesEncodeFormat: be, - ExtraFloatDigits: int(req.EvalContext.ExtraFloatDigits), - }, - VectorizeMode: sessiondata.VectorizeExecMode(req.EvalContext.Vectorize), + Location: location, } ie := &lazyInternalExecutor{ newInternalExecutor: func() sqlutil.InternalExecutor { @@ -352,7 +333,7 @@ func (ds *ServerImpl) setupFlow( // have non-nil localState.EvalContext. We don't want to update EvalContext // itself when the vectorize mode needs to be changed because we would need // to restore the original value which can have data races under stress. - isVectorized := sessiondata.VectorizeExecMode(req.EvalContext.Vectorize) != sessiondata.VectorizeOff + isVectorized := req.EvalContext.SessionData.VectorizeMode != sessiondatapb.VectorizeOff f := newFlow(flowCtx, ds.flowRegistry, syncFlowConsumer, localState.LocalProcs, isVectorized) opt := flowinfra.FuseNormally if localState.IsLocal { diff --git a/pkg/sql/distsql_running.go b/pkg/sql/distsql_running.go index 387389f2d6de..bd977ee0a90a 100644 --- a/pkg/sql/distsql_running.go +++ b/pkg/sql/distsql_running.go @@ -36,7 +36,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/rowenc" "github.com/cockroachdb/cockroach/pkg/sql/rowexec" "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/sqltelemetry" "github.com/cockroachdb/cockroach/pkg/sql/types" "github.com/cockroachdb/cockroach/pkg/util/errorutil" @@ -151,13 +151,13 @@ func (dsp *DistSQLPlanner) setupFlows( resultChan = make(chan runnerResult, len(flows)-1) } - if evalCtx.SessionData.VectorizeMode != sessiondata.VectorizeOff { - if !vectorizeThresholdMet && (evalCtx.SessionData.VectorizeMode == sessiondata.Vectorize201Auto || evalCtx.SessionData.VectorizeMode == sessiondata.VectorizeOn) { + if evalCtx.SessionData.VectorizeMode != sessiondatapb.VectorizeOff { + if !vectorizeThresholdMet && (evalCtx.SessionData.VectorizeMode == sessiondatapb.Vectorize201Auto || evalCtx.SessionData.VectorizeMode == sessiondatapb.VectorizeOn) { // Vectorization is not justified for this flow because the expected // amount of data is too small and the overhead of pre-allocating data // structures needed for the vectorized engine is expected to dominate // the execution time. - setupReq.EvalContext.Vectorize = int32(sessiondata.VectorizeOff) + setupReq.EvalContext.SessionData.VectorizeMode = sessiondatapb.VectorizeOff } else { // Now we check to see whether or not to even try vectorizing the flow. // The goal here is to determine up front whether all of the flows can be @@ -182,7 +182,7 @@ func (dsp *DistSQLPlanner) setupFlows( ); err != nil { // Vectorization attempt failed with an error. returnVectorizationSetupError := false - if evalCtx.SessionData.VectorizeMode == sessiondata.VectorizeExperimentalAlways { + if evalCtx.SessionData.VectorizeMode == sessiondatapb.VectorizeExperimentalAlways { returnVectorizationSetupError = true // If running with VectorizeExperimentalAlways, this check makes sure // that we can still run SET statements (mostly to set vectorize to @@ -206,7 +206,7 @@ func (dsp *DistSQLPlanner) setupFlows( } // Vectorization is not supported for this flow, so we override the // setting. - setupReq.EvalContext.Vectorize = int32(sessiondata.VectorizeOff) + setupReq.EvalContext.SessionData.VectorizeMode = sessiondatapb.VectorizeOff break } } diff --git a/pkg/sql/exec_util.go b/pkg/sql/exec_util.go index eef55ea2c12f..273de60ed4cc 100644 --- a/pkg/sql/exec_util.go +++ b/pkg/sql/exec_util.go @@ -50,7 +50,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/distsql" "github.com/cockroachdb/cockroach/pkg/sql/execinfra" "github.com/cockroachdb/cockroach/pkg/sql/gcjob/gcjobnotifier" - "github.com/cockroachdb/cockroach/pkg/sql/lex" "github.com/cockroachdb/cockroach/pkg/sql/opt" "github.com/cockroachdb/cockroach/pkg/sql/parser" "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgcode" @@ -60,6 +59,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/querycache" "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/sqlliveness" "github.com/cockroachdb/cockroach/pkg/sql/stats" "github.com/cockroachdb/cockroach/pkg/sql/stmtdiagnostics" @@ -305,9 +305,9 @@ var VectorizeClusterMode = settings.RegisterEnumSetting( "default vectorize mode", "on", map[int64]string{ - int64(sessiondata.VectorizeOff): "off", - int64(sessiondata.Vectorize201Auto): "201auto", - int64(sessiondata.VectorizeOn): "on", + int64(sessiondatapb.VectorizeOff): "off", + int64(sessiondatapb.Vectorize201Auto): "201auto", + int64(sessiondatapb.VectorizeOn): "on", }, ) @@ -2054,12 +2054,12 @@ func (m *sessionDataMutator) SetApplicationName(appName string) { m.paramStatusUpdater.BufferParamStatusUpdate("application_name", appName) } -func (m *sessionDataMutator) SetBytesEncodeFormat(val lex.BytesEncodeFormat) { - m.data.DataConversion.BytesEncodeFormat = val +func (m *sessionDataMutator) SetBytesEncodeFormat(val sessiondatapb.BytesEncodeFormat) { + m.data.DataConversionConfig.BytesEncodeFormat = val } -func (m *sessionDataMutator) SetExtraFloatDigits(val int) { - m.data.DataConversion.ExtraFloatDigits = val +func (m *sessionDataMutator) SetExtraFloatDigits(val int32) { + m.data.DataConversionConfig.ExtraFloatDigits = val } func (m *sessionDataMutator) SetDatabase(dbName string) { @@ -2078,7 +2078,7 @@ func (m *sessionDataMutator) SetTemporarySchemaIDForDatabase(dbID uint32, tempSc m.data.DatabaseIDToTempSchemaID[dbID] = tempSchemaID } -func (m *sessionDataMutator) SetDefaultIntSize(size int) { +func (m *sessionDataMutator) SetDefaultIntSize(size int32) { m.data.DefaultIntSize = size } @@ -2128,7 +2128,7 @@ func (m *sessionDataMutator) SetReorderJoinsLimit(val int) { m.data.ReorderJoinsLimit = val } -func (m *sessionDataMutator) SetVectorize(val sessiondata.VectorizeExecMode) { +func (m *sessionDataMutator) SetVectorize(val sessiondatapb.VectorizeExecMode) { m.data.VectorizeMode = val } @@ -2177,7 +2177,7 @@ func (m *sessionDataMutator) UpdateSearchPath(paths []string) { } func (m *sessionDataMutator) SetLocation(loc *time.Location) { - m.data.DataConversion.Location = loc + m.data.Location = loc m.paramStatusUpdater.BufferParamStatusUpdate("TimeZone", sessionDataTimeZoneFormat(loc)) } diff --git a/pkg/sql/execinfra/version.go b/pkg/sql/execinfra/version.go index b24ccad3d83d..b1b9a5dc1858 100644 --- a/pkg/sql/execinfra/version.go +++ b/pkg/sql/execinfra/version.go @@ -39,11 +39,11 @@ import "github.com/cockroachdb/cockroach/pkg/sql/execinfrapb" // // ATTENTION: When updating these fields, add a brief description of what // changed to the version history below. -const Version execinfrapb.DistSQLVersion = 38 +const Version execinfrapb.DistSQLVersion = 39 // MinAcceptedVersion is the oldest version that the server is compatible with. // A server will not accept flows with older versions. -const MinAcceptedVersion execinfrapb.DistSQLVersion = 37 +const MinAcceptedVersion execinfrapb.DistSQLVersion = 39 /* @@ -51,6 +51,11 @@ const MinAcceptedVersion execinfrapb.DistSQLVersion = 37 Please add new entries at the top. +- Version: 39 (MinAcceptedVersion: 39) + - Many parameters from sessiondata.SessionData object were pulled into a + protobuf struct for easier propagation to the remote nodes during the + execution. + - Version: 38 (MinAcceptedVersion: 38) - A paired joiner approach for inverted joins was added, for left outer/semi/anti joins involving the invertedJoiner and joinReader. diff --git a/pkg/sql/execinfrapb/api.go b/pkg/sql/execinfrapb/api.go index 2cc8cd420fcc..45df5e55ba18 100644 --- a/pkg/sql/execinfrapb/api.go +++ b/pkg/sql/execinfrapb/api.go @@ -11,7 +11,6 @@ package execinfrapb import ( - "github.com/cockroachdb/cockroach/pkg/sql/lex" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/util/uuid" ) @@ -33,29 +32,12 @@ type DistSQLVersion uint32 // MakeEvalContext serializes some of the fields of a tree.EvalContext into a // execinfrapb.EvalContext proto. func MakeEvalContext(evalCtx *tree.EvalContext) EvalContext { - var be BytesEncodeFormat - switch evalCtx.SessionData.DataConversion.BytesEncodeFormat { - case lex.BytesEncodeHex: - be = BytesEncodeFormat_HEX - case lex.BytesEncodeEscape: - be = BytesEncodeFormat_ESCAPE - case lex.BytesEncodeBase64: - be = BytesEncodeFormat_BASE64 - default: - panic("unknown format") - } res := EvalContext{ - StmtTimestampNanos: evalCtx.StmtTimestamp.UnixNano(), - TxnTimestampNanos: evalCtx.TxnTimestamp.UnixNano(), - Location: evalCtx.GetLocation().String(), - Database: evalCtx.SessionData.Database, - TemporarySchemaName: evalCtx.SessionData.SearchPath.GetTemporarySchemaName(), - User: evalCtx.SessionData.User, - ApplicationName: evalCtx.SessionData.ApplicationName, - BytesEncodeFormat: be, - ExtraFloatDigits: int32(evalCtx.SessionData.DataConversion.ExtraFloatDigits), - Vectorize: int32(evalCtx.SessionData.VectorizeMode), - VectorizeInjectPanics: evalCtx.SessionData.TestingVectorizeInjectPanics, + SessionData: evalCtx.SessionData.SessionData, + StmtTimestampNanos: evalCtx.StmtTimestamp.UnixNano(), + TxnTimestampNanos: evalCtx.TxnTimestamp.UnixNano(), + Location: evalCtx.GetLocation().String(), + TemporarySchemaName: evalCtx.SessionData.SearchPath.GetTemporarySchemaName(), } // Populate the search path. Make sure not to include the implicit pg_catalog, diff --git a/pkg/sql/execinfrapb/api.pb.go b/pkg/sql/execinfrapb/api.pb.go index 47626059da20..3a0eb49ebc6d 100644 --- a/pkg/sql/execinfrapb/api.pb.go +++ b/pkg/sql/execinfrapb/api.pb.go @@ -13,6 +13,7 @@ import proto "github.com/gogo/protobuf/proto" import fmt "fmt" import math "math" import roachpb "github.com/cockroachdb/cockroach/pkg/roachpb" +import sessiondatapb "github.com/cockroachdb/cockroach/pkg/sql/sessiondatapb" import github_com_cockroachdb_cockroach_pkg_roachpb "github.com/cockroachdb/cockroach/pkg/roachpb" import time "time" @@ -38,46 +39,6 @@ var _ = time.Kitchen // proto package needs to be updated. const _ = proto.GoGoProtoPackageIsVersion2 // please upgrade the proto package -// BytesEncodeFormat is the configuration for bytes to string conversions. -type BytesEncodeFormat int32 - -const ( - BytesEncodeFormat_HEX BytesEncodeFormat = 0 - BytesEncodeFormat_ESCAPE BytesEncodeFormat = 1 - BytesEncodeFormat_BASE64 BytesEncodeFormat = 2 -) - -var BytesEncodeFormat_name = map[int32]string{ - 0: "HEX", - 1: "ESCAPE", - 2: "BASE64", -} -var BytesEncodeFormat_value = map[string]int32{ - "HEX": 0, - "ESCAPE": 1, - "BASE64": 2, -} - -func (x BytesEncodeFormat) Enum() *BytesEncodeFormat { - p := new(BytesEncodeFormat) - *p = x - return p -} -func (x BytesEncodeFormat) String() string { - return proto.EnumName(BytesEncodeFormat_name, int32(x)) -} -func (x *BytesEncodeFormat) UnmarshalJSON(data []byte) error { - value, err := proto.UnmarshalJSONEnum(BytesEncodeFormat_value, data, "BytesEncodeFormat") - if err != nil { - return err - } - *x = BytesEncodeFormat(value) - return nil -} -func (BytesEncodeFormat) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_api_9a2e51711a255dee, []int{0} -} - type SetupFlowRequest struct { // LeafTxnInputState is the input parameter for the *client.Txn needed for // executing the flow. @@ -97,7 +58,7 @@ func (m *SetupFlowRequest) Reset() { *m = SetupFlowRequest{} } func (m *SetupFlowRequest) String() string { return proto.CompactTextString(m) } func (*SetupFlowRequest) ProtoMessage() {} func (*SetupFlowRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_9a2e51711a255dee, []int{0} + return fileDescriptor_api_8a2c20c180cc079a, []int{0} } func (m *SetupFlowRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -135,7 +96,7 @@ func (m *FlowSpec) Reset() { *m = FlowSpec{} } func (m *FlowSpec) String() string { return proto.CompactTextString(m) } func (*FlowSpec) ProtoMessage() {} func (*FlowSpec) Descriptor() ([]byte, []int) { - return fileDescriptor_api_9a2e51711a255dee, []int{1} + return fileDescriptor_api_8a2c20c180cc079a, []int{1} } func (m *FlowSpec) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -166,24 +127,18 @@ type EvalContext struct { TxnTimestampNanos int64 `protobuf:"varint,2,opt,name=txnTimestampNanos" json:"txnTimestampNanos"` // The name of the location according to whose current timezone we're going to // parse timestamps. Used to init EvalContext.Location. - Location string `protobuf:"bytes,4,opt,name=location" json:"location"` - Database string `protobuf:"bytes,5,opt,name=database" json:"database"` - SearchPath []string `protobuf:"bytes,6,rep,name=search_path,json=searchPath" json:"search_path,omitempty"` - User string `protobuf:"bytes,7,opt,name=user" json:"user"` - SeqState SequenceState `protobuf:"bytes,8,opt,name=seq_state,json=seqState" json:"seq_state"` - ApplicationName string `protobuf:"bytes,9,opt,name=application_name,json=applicationName" json:"application_name"` - BytesEncodeFormat BytesEncodeFormat `protobuf:"varint,10,opt,name=bytes_encode_format,json=bytesEncodeFormat,enum=cockroach.sql.distsqlrun.BytesEncodeFormat" json:"bytes_encode_format"` - ExtraFloatDigits int32 `protobuf:"varint,11,opt,name=extra_float_digits,json=extraFloatDigits" json:"extra_float_digits"` - Vectorize int32 `protobuf:"varint,12,opt,name=vectorize" json:"vectorize"` - TemporarySchemaName string `protobuf:"bytes,13,opt,name=temporary_schema_name,json=temporarySchemaName" json:"temporary_schema_name"` - VectorizeInjectPanics bool `protobuf:"varint,14,opt,name=vectorize_inject_panics,json=vectorizeInjectPanics" json:"vectorize_inject_panics"` + Location string `protobuf:"bytes,4,opt,name=location" json:"location"` + SearchPath []string `protobuf:"bytes,6,rep,name=search_path,json=searchPath" json:"search_path,omitempty"` + SeqState SequenceState `protobuf:"bytes,8,opt,name=seq_state,json=seqState" json:"seq_state"` + TemporarySchemaName string `protobuf:"bytes,13,opt,name=temporary_schema_name,json=temporarySchemaName" json:"temporary_schema_name"` + SessionData sessiondatapb.SessionData `protobuf:"bytes,15,opt,name=session_data,json=sessionData" json:"session_data"` } func (m *EvalContext) Reset() { *m = EvalContext{} } func (m *EvalContext) String() string { return proto.CompactTextString(m) } func (*EvalContext) ProtoMessage() {} func (*EvalContext) Descriptor() ([]byte, []int) { - return fileDescriptor_api_9a2e51711a255dee, []int{2} + return fileDescriptor_api_8a2c20c180cc079a, []int{2} } func (m *EvalContext) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -220,7 +175,7 @@ func (m *SequenceState) Reset() { *m = SequenceState{} } func (m *SequenceState) String() string { return proto.CompactTextString(m) } func (*SequenceState) ProtoMessage() {} func (*SequenceState) Descriptor() ([]byte, []int) { - return fileDescriptor_api_9a2e51711a255dee, []int{3} + return fileDescriptor_api_8a2c20c180cc079a, []int{3} } func (m *SequenceState) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -255,7 +210,7 @@ func (m *SequenceState_Seq) Reset() { *m = SequenceState_Seq{} } func (m *SequenceState_Seq) String() string { return proto.CompactTextString(m) } func (*SequenceState_Seq) ProtoMessage() {} func (*SequenceState_Seq) Descriptor() ([]byte, []int) { - return fileDescriptor_api_9a2e51711a255dee, []int{3, 0} + return fileDescriptor_api_8a2c20c180cc079a, []int{3, 0} } func (m *SequenceState_Seq) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -288,7 +243,7 @@ func (m *SimpleResponse) Reset() { *m = SimpleResponse{} } func (m *SimpleResponse) String() string { return proto.CompactTextString(m) } func (*SimpleResponse) ProtoMessage() {} func (*SimpleResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_9a2e51711a255dee, []int{4} + return fileDescriptor_api_8a2c20c180cc079a, []int{4} } func (m *SimpleResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -331,7 +286,7 @@ func (m *ConsumerSignal) Reset() { *m = ConsumerSignal{} } func (m *ConsumerSignal) String() string { return proto.CompactTextString(m) } func (*ConsumerSignal) ProtoMessage() {} func (*ConsumerSignal) Descriptor() ([]byte, []int) { - return fileDescriptor_api_9a2e51711a255dee, []int{5} + return fileDescriptor_api_8a2c20c180cc079a, []int{5} } func (m *ConsumerSignal) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -363,7 +318,7 @@ func (m *DrainRequest) Reset() { *m = DrainRequest{} } func (m *DrainRequest) String() string { return proto.CompactTextString(m) } func (*DrainRequest) ProtoMessage() {} func (*DrainRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_9a2e51711a255dee, []int{6} + return fileDescriptor_api_8a2c20c180cc079a, []int{6} } func (m *DrainRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -414,7 +369,7 @@ func (m *ConsumerHandshake) Reset() { *m = ConsumerHandshake{} } func (m *ConsumerHandshake) String() string { return proto.CompactTextString(m) } func (*ConsumerHandshake) ProtoMessage() {} func (*ConsumerHandshake) Descriptor() ([]byte, []int) { - return fileDescriptor_api_9a2e51711a255dee, []int{7} + return fileDescriptor_api_8a2c20c180cc079a, []int{7} } func (m *ConsumerHandshake) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -449,7 +404,6 @@ func init() { proto.RegisterType((*ConsumerSignal)(nil), "cockroach.sql.distsqlrun.ConsumerSignal") proto.RegisterType((*DrainRequest)(nil), "cockroach.sql.distsqlrun.DrainRequest") proto.RegisterType((*ConsumerHandshake)(nil), "cockroach.sql.distsqlrun.ConsumerHandshake") - proto.RegisterEnum("cockroach.sql.distsqlrun.BytesEncodeFormat", BytesEncodeFormat_name, BytesEncodeFormat_value) } // Reference imports to suppress errors if they are not otherwise used. @@ -806,10 +760,6 @@ func (m *EvalContext) MarshalTo(dAtA []byte) (int, error) { i++ i = encodeVarintApi(dAtA, i, uint64(len(m.Location))) i += copy(dAtA[i:], m.Location) - dAtA[i] = 0x2a - i++ - i = encodeVarintApi(dAtA, i, uint64(len(m.Database))) - i += copy(dAtA[i:], m.Database) if len(m.SearchPath) > 0 { for _, s := range m.SearchPath { dAtA[i] = 0x32 @@ -825,10 +775,6 @@ func (m *EvalContext) MarshalTo(dAtA []byte) (int, error) { i += copy(dAtA[i:], s) } } - dAtA[i] = 0x3a - i++ - i = encodeVarintApi(dAtA, i, uint64(len(m.User))) - i += copy(dAtA[i:], m.User) dAtA[i] = 0x42 i++ i = encodeVarintApi(dAtA, i, uint64(m.SeqState.Size())) @@ -837,31 +783,18 @@ func (m *EvalContext) MarshalTo(dAtA []byte) (int, error) { return 0, err } i += n5 - dAtA[i] = 0x4a - i++ - i = encodeVarintApi(dAtA, i, uint64(len(m.ApplicationName))) - i += copy(dAtA[i:], m.ApplicationName) - dAtA[i] = 0x50 - i++ - i = encodeVarintApi(dAtA, i, uint64(m.BytesEncodeFormat)) - dAtA[i] = 0x58 - i++ - i = encodeVarintApi(dAtA, i, uint64(m.ExtraFloatDigits)) - dAtA[i] = 0x60 - i++ - i = encodeVarintApi(dAtA, i, uint64(m.Vectorize)) dAtA[i] = 0x6a i++ i = encodeVarintApi(dAtA, i, uint64(len(m.TemporarySchemaName))) i += copy(dAtA[i:], m.TemporarySchemaName) - dAtA[i] = 0x70 + dAtA[i] = 0x7a i++ - if m.VectorizeInjectPanics { - dAtA[i] = 1 - } else { - dAtA[i] = 0 + i = encodeVarintApi(dAtA, i, uint64(m.SessionData.Size())) + n6, err := m.SessionData.MarshalTo(dAtA[i:]) + if err != nil { + return 0, err } - i++ + i += n6 return i, nil } @@ -943,11 +876,11 @@ func (m *SimpleResponse) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintApi(dAtA, i, uint64(m.Error.Size())) - n6, err := m.Error.MarshalTo(dAtA[i:]) + n7, err := m.Error.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n6 + i += n7 } return i, nil } @@ -971,31 +904,31 @@ func (m *ConsumerSignal) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintApi(dAtA, i, uint64(m.DrainRequest.Size())) - n7, err := m.DrainRequest.MarshalTo(dAtA[i:]) + n8, err := m.DrainRequest.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n7 + i += n8 } if m.SetupFlowRequest != nil { dAtA[i] = 0x12 i++ i = encodeVarintApi(dAtA, i, uint64(m.SetupFlowRequest.Size())) - n8, err := m.SetupFlowRequest.MarshalTo(dAtA[i:]) + n9, err := m.SetupFlowRequest.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n8 + i += n9 } if m.Handshake != nil { dAtA[i] = 0x1a i++ i = encodeVarintApi(dAtA, i, uint64(m.Handshake.Size())) - n9, err := m.Handshake.MarshalTo(dAtA[i:]) + n10, err := m.Handshake.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n9 + i += n10 } return i, nil } @@ -1045,11 +978,11 @@ func (m *ConsumerHandshake) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x12 i++ i = encodeVarintApi(dAtA, i, uint64(github_com_gogo_protobuf_types.SizeOfStdTime(*m.ConsumerScheduleDeadline))) - n10, err := github_com_gogo_protobuf_types.StdTimeMarshalTo(*m.ConsumerScheduleDeadline, dAtA[i:]) + n11, err := github_com_gogo_protobuf_types.StdTimeMarshalTo(*m.ConsumerScheduleDeadline, dAtA[i:]) if err != nil { return 0, err } - i += n10 + i += n11 } dAtA[i] = 0x18 i++ @@ -1116,26 +1049,18 @@ func (m *EvalContext) Size() (n int) { n += 1 + sovApi(uint64(m.TxnTimestampNanos)) l = len(m.Location) n += 1 + l + sovApi(uint64(l)) - l = len(m.Database) - n += 1 + l + sovApi(uint64(l)) if len(m.SearchPath) > 0 { for _, s := range m.SearchPath { l = len(s) n += 1 + l + sovApi(uint64(l)) } } - l = len(m.User) - n += 1 + l + sovApi(uint64(l)) l = m.SeqState.Size() n += 1 + l + sovApi(uint64(l)) - l = len(m.ApplicationName) - n += 1 + l + sovApi(uint64(l)) - n += 1 + sovApi(uint64(m.BytesEncodeFormat)) - n += 1 + sovApi(uint64(m.ExtraFloatDigits)) - n += 1 + sovApi(uint64(m.Vectorize)) l = len(m.TemporarySchemaName) n += 1 + l + sovApi(uint64(l)) - n += 2 + l = m.SessionData.Size() + n += 1 + l + sovApi(uint64(l)) return n } @@ -1674,35 +1599,6 @@ func (m *EvalContext) Unmarshal(dAtA []byte) error { } m.Location = string(dAtA[iNdEx:postIndex]) iNdEx = postIndex - case 5: - if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Database", wireType) - } - var stringLen uint64 - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflowApi - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - stringLen |= (uint64(b) & 0x7F) << shift - if b < 0x80 { - break - } - } - intStringLen := int(stringLen) - if intStringLen < 0 { - return ErrInvalidLengthApi - } - postIndex := iNdEx + intStringLen - if postIndex > l { - return io.ErrUnexpectedEOF - } - m.Database = string(dAtA[iNdEx:postIndex]) - iNdEx = postIndex case 6: if wireType != 2 { return fmt.Errorf("proto: wrong wireType = %d for field SearchPath", wireType) @@ -1732,35 +1628,6 @@ func (m *EvalContext) Unmarshal(dAtA []byte) error { } m.SearchPath = append(m.SearchPath, string(dAtA[iNdEx:postIndex])) iNdEx = postIndex - case 7: - if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field User", wireType) - } - var stringLen uint64 - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflowApi - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - stringLen |= (uint64(b) & 0x7F) << shift - if b < 0x80 { - break - } - } - intStringLen := int(stringLen) - if intStringLen < 0 { - return ErrInvalidLengthApi - } - postIndex := iNdEx + intStringLen - if postIndex > l { - return io.ErrUnexpectedEOF - } - m.User = string(dAtA[iNdEx:postIndex]) - iNdEx = postIndex case 8: if wireType != 2 { return fmt.Errorf("proto: wrong wireType = %d for field SeqState", wireType) @@ -1791,9 +1658,9 @@ func (m *EvalContext) Unmarshal(dAtA []byte) error { return err } iNdEx = postIndex - case 9: + case 13: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field ApplicationName", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field TemporarySchemaName", wireType) } var stringLen uint64 for shift := uint(0); ; shift += 7 { @@ -1818,70 +1685,13 @@ func (m *EvalContext) Unmarshal(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - m.ApplicationName = string(dAtA[iNdEx:postIndex]) + m.TemporarySchemaName = string(dAtA[iNdEx:postIndex]) iNdEx = postIndex - case 10: - if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field BytesEncodeFormat", wireType) - } - m.BytesEncodeFormat = 0 - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflowApi - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - m.BytesEncodeFormat |= (BytesEncodeFormat(b) & 0x7F) << shift - if b < 0x80 { - break - } - } - case 11: - if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field ExtraFloatDigits", wireType) - } - m.ExtraFloatDigits = 0 - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflowApi - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - m.ExtraFloatDigits |= (int32(b) & 0x7F) << shift - if b < 0x80 { - break - } - } - case 12: - if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field Vectorize", wireType) - } - m.Vectorize = 0 - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflowApi - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - m.Vectorize |= (int32(b) & 0x7F) << shift - if b < 0x80 { - break - } - } - case 13: + case 15: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field TemporarySchemaName", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field SessionData", wireType) } - var stringLen uint64 + var msglen int for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflowApi @@ -1891,41 +1701,22 @@ func (m *EvalContext) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - stringLen |= (uint64(b) & 0x7F) << shift + msglen |= (int(b) & 0x7F) << shift if b < 0x80 { break } } - intStringLen := int(stringLen) - if intStringLen < 0 { + if msglen < 0 { return ErrInvalidLengthApi } - postIndex := iNdEx + intStringLen + postIndex := iNdEx + msglen if postIndex > l { return io.ErrUnexpectedEOF } - m.TemporarySchemaName = string(dAtA[iNdEx:postIndex]) - iNdEx = postIndex - case 14: - if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field VectorizeInjectPanics", wireType) - } - var v int - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflowApi - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - v |= (int(b) & 0x7F) << shift - if b < 0x80 { - break - } + if err := m.SessionData.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err } - m.VectorizeInjectPanics = bool(v != 0) + iNdEx = postIndex default: iNdEx = preIndex skippy, err := skipApi(dAtA[iNdEx:]) @@ -2664,86 +2455,79 @@ var ( ErrIntOverflowApi = fmt.Errorf("proto: integer overflow") ) -func init() { proto.RegisterFile("sql/execinfrapb/api.proto", fileDescriptor_api_9a2e51711a255dee) } - -var fileDescriptor_api_9a2e51711a255dee = []byte{ - // 1246 bytes of a gzipped FileDescriptorProto - 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x94, 0x56, 0x4f, 0x73, 0x1b, 0xc5, - 0x12, 0xd7, 0x4a, 0xb2, 0x2d, 0xb5, 0x2c, 0x3f, 0x79, 0xe2, 0xbc, 0xb7, 0x4f, 0x07, 0x49, 0xa5, - 0x97, 0x07, 0x22, 0x29, 0xa4, 0xa0, 0x04, 0x8a, 0x82, 0x54, 0x51, 0x51, 0xe4, 0x10, 0x39, 0x7f, - 0xca, 0xec, 0x26, 0xa9, 0x14, 0x07, 0xb6, 0x46, 0xbb, 0x2d, 0x69, 0xc9, 0xfe, 0xd3, 0xce, 0xc8, - 0xb1, 0xf9, 0x04, 0x1c, 0xf3, 0x11, 0x38, 0x70, 0xe4, 0x4b, 0x70, 0xf3, 0x31, 0x87, 0x1c, 0x52, - 0x1c, 0x0c, 0xd8, 0xdf, 0x82, 0x13, 0x35, 0xb3, 0xbb, 0xf2, 0x5a, 0xc6, 0xc2, 0xdc, 0x76, 0xbb, - 0x7f, 0xbf, 0x9e, 0xe9, 0xee, 0xdf, 0x4c, 0x0f, 0xfc, 0x97, 0x4d, 0x9d, 0x0e, 0xee, 0xa3, 0x69, - 0x7b, 0xa3, 0x90, 0x06, 0xc3, 0x0e, 0x0d, 0xec, 0x76, 0x10, 0xfa, 0xdc, 0x27, 0xaa, 0xe9, 0x9b, - 0x2f, 0x43, 0x9f, 0x9a, 0x93, 0x36, 0x9b, 0x3a, 0x6d, 0xcb, 0x66, 0x9c, 0x4d, 0x9d, 0x70, 0xe6, - 0x55, 0xb7, 0xc6, 0xfe, 0xd8, 0x97, 0xa0, 0x8e, 0xf8, 0x8a, 0xf0, 0xd5, 0xfa, 0xd8, 0xf7, 0xc7, - 0x0e, 0x76, 0xe4, 0xdf, 0x70, 0x36, 0xea, 0x70, 0xdb, 0x45, 0xc6, 0xa9, 0x1b, 0xc4, 0x00, 0x22, - 0x83, 0x05, 0xc3, 0x8e, 0x45, 0x39, 0x8d, 0x6d, 0xd5, 0xc5, 0xf5, 0x53, 0xbe, 0xc6, 0xa2, 0x2f, - 0x08, 0x7d, 0x13, 0x19, 0xf3, 0x43, 0x16, 0x21, 0x9a, 0x6f, 0xb3, 0x50, 0xd1, 0x91, 0xcf, 0x82, - 0xfb, 0x8e, 0xff, 0x4a, 0xc3, 0xe9, 0x0c, 0x19, 0x27, 0x77, 0x20, 0x3f, 0x72, 0xfc, 0x57, 0x6a, - 0xae, 0xa1, 0xb4, 0x4a, 0xdd, 0x66, 0xfb, 0xa2, 0x34, 0xda, 0x82, 0xa4, 0x07, 0x68, 0xf6, 0xf2, - 0x87, 0x47, 0xf5, 0x8c, 0x26, 0x59, 0xe4, 0x26, 0xac, 0xed, 0x61, 0xc8, 0x6c, 0xdf, 0x53, 0x57, - 0x1a, 0x4a, 0xab, 0xdc, 0xfb, 0xb7, 0x70, 0xfe, 0x71, 0x54, 0xdf, 0xe8, 0xdb, 0x8c, 0xeb, 0x5f, - 0x3d, 0x7a, 0x1e, 0x79, 0xb5, 0x04, 0x46, 0x1e, 0x43, 0x09, 0xf7, 0xa8, 0x73, 0xcf, 0xf7, 0x38, - 0xee, 0x73, 0x75, 0x55, 0x2e, 0xfb, 0xff, 0x8b, 0x97, 0xdd, 0x3e, 0x05, 0xc7, 0x2b, 0xa7, 0xf9, - 0xe4, 0x19, 0x6c, 0x39, 0x48, 0x47, 0x06, 0xdf, 0xf7, 0x0c, 0xdb, 0x0b, 0x66, 0xdc, 0x60, 0x9c, - 0x72, 0x54, 0xd7, 0x64, 0xdc, 0x6b, 0xa9, 0xb8, 0x71, 0x39, 0xdb, 0x8f, 0x90, 0x8e, 0x9e, 0xee, - 0x7b, 0x03, 0x01, 0xd6, 0x05, 0x56, 0xdb, 0x74, 0x16, 0x4d, 0xa4, 0x06, 0x6b, 0x4f, 0x43, 0x6a, - 0xe2, 0xc3, 0xe7, 0x6a, 0xa1, 0xa1, 0xb4, 0x0a, 0xf1, 0xd2, 0x89, 0x71, 0x27, 0x5f, 0x50, 0x2a, - 0xd9, 0x9d, 0x7c, 0x21, 0x5b, 0xc9, 0x35, 0x4f, 0x14, 0x28, 0x24, 0xc5, 0x21, 0x1f, 0xc1, 0x9a, - 0x28, 0x8c, 0x61, 0x5b, 0xaa, 0xd2, 0x50, 0x5a, 0xeb, 0x3d, 0x55, 0x10, 0x7f, 0x39, 0xaa, 0xaf, - 0x0a, 0xc8, 0xa0, 0x7f, 0x3c, 0xff, 0xd2, 0x56, 0x05, 0x70, 0x60, 0x91, 0xc7, 0x00, 0xa7, 0xad, - 0x52, 0xb3, 0x8d, 0x5c, 0xab, 0xd4, 0x7d, 0xff, 0xe2, 0x82, 0xec, 0x26, 0xd8, 0x54, 0x33, 0x52, - 0x01, 0xc8, 0x33, 0x58, 0x1b, 0x53, 0x8e, 0xaf, 0xe8, 0x81, 0xec, 0xe9, 0x4a, 0xef, 0xf3, 0xb8, - 0x25, 0xb7, 0xc6, 0x36, 0x9f, 0xcc, 0x86, 0x6d, 0xd3, 0x77, 0x3b, 0xf3, 0xe8, 0xd6, 0xf0, 0xf4, - 0xbb, 0x13, 0xbc, 0x1c, 0x77, 0x92, 0x32, 0x3d, 0xf1, 0x2d, 0x1c, 0xf4, 0xb5, 0x24, 0x56, 0xf3, - 0xa7, 0x15, 0x28, 0xa5, 0x7a, 0x41, 0x6e, 0x03, 0x61, 0xdc, 0xe5, 0x4f, 0x13, 0xd5, 0x3e, 0xa1, - 0x9e, 0xcf, 0x64, 0xce, 0xb9, 0x78, 0x53, 0x7f, 0xe1, 0x27, 0x5d, 0xd8, 0xe4, 0xfb, 0xde, 0x02, - 0x29, 0x9b, 0x22, 0x9d, 0x77, 0x93, 0x06, 0x14, 0x1c, 0xdf, 0xa4, 0x5c, 0x88, 0x2c, 0xdf, 0x50, - 0x5a, 0xc5, 0x18, 0x3a, 0xb7, 0x0a, 0x84, 0x38, 0x08, 0x43, 0xca, 0x50, 0xca, 0x70, 0x8e, 0x48, - 0xac, 0xa4, 0x0e, 0x25, 0x86, 0x34, 0x34, 0x27, 0x46, 0x40, 0xf9, 0x44, 0x5d, 0x6d, 0xe4, 0x5a, - 0x45, 0x0d, 0x22, 0xd3, 0x2e, 0xe5, 0x13, 0xa2, 0x42, 0x7e, 0xc6, 0x30, 0x94, 0xba, 0x49, 0xe8, - 0xd2, 0x42, 0x76, 0xa0, 0xc8, 0x70, 0x1a, 0xcb, 0xaa, 0x20, 0x65, 0xb5, 0xa4, 0x3b, 0xba, 0x38, - 0x56, 0x9e, 0x89, 0x52, 0x46, 0xc9, 0x36, 0x18, 0x4e, 0x23, 0x59, 0x75, 0xa0, 0x42, 0x83, 0xc0, - 0xb1, 0xa3, 0x7d, 0x1b, 0x1e, 0x75, 0x51, 0x2d, 0xa6, 0x56, 0xfc, 0x57, 0xca, 0xfb, 0x84, 0xba, - 0x48, 0x28, 0x5c, 0x19, 0x1e, 0x70, 0x64, 0x06, 0x7a, 0xa6, 0x6f, 0xa1, 0x31, 0xf2, 0x43, 0x97, - 0x72, 0x15, 0x1a, 0x4a, 0x6b, 0xa3, 0x7b, 0xe3, 0xe2, 0x6d, 0xf4, 0x04, 0x69, 0x5b, 0x72, 0xee, - 0x4b, 0x4a, 0x52, 0xde, 0xe1, 0xa2, 0x83, 0x74, 0x81, 0xe0, 0x3e, 0x0f, 0xa9, 0x31, 0x72, 0x7c, - 0xca, 0x0d, 0xcb, 0x1e, 0xdb, 0x9c, 0xa9, 0x25, 0x29, 0x9d, 0x88, 0x54, 0x91, 0xfe, 0xfb, 0xc2, - 0xdd, 0x97, 0x5e, 0xd2, 0x84, 0xe2, 0x1e, 0x9a, 0xdc, 0x0f, 0xed, 0xef, 0x50, 0x5d, 0x4f, 0x41, - 0x4f, 0xcd, 0xe4, 0x53, 0xb8, 0xca, 0xd1, 0x0d, 0xfc, 0x90, 0x86, 0x07, 0x06, 0x33, 0x27, 0xe8, - 0xd2, 0x28, 0xe1, 0x72, 0x2a, 0xe1, 0x2b, 0x73, 0x88, 0x2e, 0x11, 0x32, 0xe9, 0x3b, 0xf0, 0x9f, - 0x79, 0x18, 0xc3, 0xf6, 0xbe, 0x45, 0x93, 0x1b, 0x01, 0xf5, 0x6c, 0x93, 0xa9, 0x1b, 0xa9, 0xc3, - 0x78, 0x75, 0x0e, 0x1a, 0x48, 0xcc, 0xae, 0x84, 0xec, 0xe4, 0x0b, 0xb9, 0x4a, 0xbe, 0xf9, 0x56, - 0x81, 0xf2, 0x99, 0x5e, 0x90, 0x2f, 0x20, 0xcf, 0x70, 0x2a, 0x24, 0x2a, 0x0e, 0xd8, 0x8d, 0x4b, - 0xb6, 0x50, 0xfc, 0x69, 0x92, 0x48, 0x6e, 0xc2, 0x96, 0x43, 0x19, 0x37, 0x84, 0x1a, 0x6c, 0xcf, - 0x0c, 0xd1, 0x45, 0x8f, 0xa3, 0x25, 0xe5, 0x5b, 0xd6, 0x88, 0xf0, 0xe9, 0x38, 0x1d, 0x9c, 0x7a, - 0xaa, 0xbb, 0x90, 0xd3, 0x71, 0x4a, 0xae, 0xc1, 0xaa, 0xe4, 0x44, 0x57, 0x42, 0xb9, 0x57, 0x16, - 0xdb, 0x3f, 0x3e, 0xaa, 0xaf, 0x08, 0x78, 0x5f, 0x5b, 0x61, 0x38, 0x1d, 0x58, 0xe4, 0x7f, 0x00, - 0x0e, 0xe5, 0xc8, 0xb8, 0xb1, 0x47, 0x9d, 0x33, 0x67, 0xa2, 0x18, 0xd9, 0x9f, 0x53, 0xa7, 0xf9, - 0x25, 0x6c, 0xe8, 0xb6, 0x1b, 0x38, 0xa8, 0x21, 0x0b, 0x7c, 0x8f, 0x21, 0xf9, 0x18, 0x56, 0x30, - 0x0c, 0xfd, 0x50, 0xc6, 0x2e, 0x75, 0xeb, 0x4b, 0x6e, 0x52, 0x01, 0xd3, 0x22, 0x74, 0xf3, 0xfb, - 0x2c, 0x6c, 0xdc, 0xf3, 0x3d, 0x36, 0x73, 0x31, 0xd4, 0xed, 0xb1, 0x47, 0x1d, 0xf2, 0x10, 0xca, - 0x56, 0x48, 0x6d, 0xcf, 0x08, 0xa3, 0xd1, 0x10, 0x47, 0x7c, 0xef, 0xe2, 0x88, 0x7d, 0x01, 0x8f, - 0x07, 0x89, 0xb6, 0x6e, 0xa5, 0xfe, 0xc8, 0x0b, 0x20, 0x4c, 0x8c, 0x1a, 0x43, 0xde, 0x86, 0x49, - 0xc4, 0xac, 0x8c, 0x78, 0x7d, 0x59, 0xed, 0xcf, 0x8e, 0x27, 0xad, 0xc2, 0x16, 0x07, 0xd6, 0x00, - 0x8a, 0x13, 0xea, 0x59, 0x6c, 0x42, 0x5f, 0x62, 0x3c, 0xb5, 0x96, 0x34, 0x33, 0xc9, 0xf1, 0x41, - 0x42, 0xd1, 0x4e, 0xd9, 0x9f, 0xe5, 0x0f, 0x7f, 0xa8, 0x2b, 0xcd, 0x0d, 0x58, 0x4f, 0x27, 0xd2, - 0xfc, 0x31, 0x0b, 0x9b, 0xe7, 0x68, 0xe4, 0x16, 0x10, 0x33, 0x36, 0x4a, 0x35, 0x5b, 0x33, 0x07, - 0xa3, 0x86, 0x26, 0x7a, 0xdc, 0x4c, 0xfc, 0x7a, 0xe2, 0x26, 0xdf, 0x40, 0xf5, 0x1c, 0xc9, 0xb0, - 0x90, 0x5a, 0x8e, 0xed, 0x61, 0x5c, 0x8d, 0x6a, 0x3b, 0x7a, 0x09, 0xb4, 0x93, 0x97, 0x40, 0x7b, - 0x7e, 0xff, 0xf5, 0xf2, 0xaf, 0x7f, 0xad, 0x2b, 0x9a, 0xba, 0x18, 0xb8, 0x1f, 0x47, 0x48, 0x8f, - 0xdf, 0xdc, 0xe5, 0xc6, 0xef, 0x03, 0xd8, 0x72, 0x6d, 0xcf, 0xa0, 0xa6, 0x89, 0x01, 0x47, 0xcb, - 0x48, 0xe8, 0xf9, 0xa5, 0x74, 0xe2, 0xda, 0xde, 0xdd, 0x98, 0x12, 0xdb, 0xae, 0xdf, 0x86, 0xcd, - 0x73, 0xb7, 0x0c, 0x59, 0x83, 0xdc, 0x83, 0xed, 0x17, 0x95, 0x0c, 0x01, 0x58, 0xdd, 0xd6, 0xef, - 0xdd, 0xdd, 0xdd, 0xae, 0x28, 0xe2, 0xbb, 0x77, 0x57, 0xdf, 0xfe, 0xe4, 0x76, 0x25, 0xdb, 0xfd, - 0x39, 0x0b, 0x6b, 0x71, 0x70, 0x32, 0x81, 0x92, 0x36, 0xf3, 0xf4, 0x03, 0xcf, 0x14, 0xfd, 0x25, - 0xad, 0xbf, 0xef, 0x62, 0xa4, 0xd4, 0xea, 0x07, 0x4b, 0xa7, 0xa3, 0x35, 0x33, 0x31, 0x7c, 0x8c, - 0x8c, 0xd1, 0x31, 0x36, 0x33, 0x2d, 0xe5, 0xa6, 0x42, 0x4c, 0x28, 0xce, 0x95, 0x45, 0xfe, 0x81, - 0xfc, 0xaa, 0x4b, 0xf6, 0x74, 0xf6, 0x1c, 0x36, 0x33, 0x64, 0x0c, 0x20, 0x9f, 0x01, 0x3c, 0x44, - 0xea, 0x92, 0xcb, 0xef, 0xb1, 0x7a, 0xe9, 0xc4, 0xa3, 0x6c, 0x7a, 0x1f, 0x1e, 0xfe, 0x5e, 0xcb, - 0x1c, 0x1e, 0xd7, 0x94, 0x37, 0xc7, 0x35, 0xe5, 0xdd, 0x71, 0x4d, 0xf9, 0xed, 0xb8, 0xa6, 0xbc, - 0x3e, 0xa9, 0x65, 0xde, 0x9c, 0xd4, 0x32, 0xef, 0x4e, 0x6a, 0x99, 0xaf, 0x4b, 0xa9, 0x77, 0xe0, - 0x9f, 0x01, 0x00, 0x00, 0xff, 0xff, 0x2d, 0x79, 0x81, 0x4f, 0xb5, 0x0a, 0x00, 0x00, +func init() { proto.RegisterFile("sql/execinfrapb/api.proto", fileDescriptor_api_8a2c20c180cc079a) } + +var fileDescriptor_api_8a2c20c180cc079a = []byte{ + // 1129 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x94, 0x56, 0xcb, 0x72, 0xe3, 0x44, + 0x17, 0xb6, 0x6c, 0xd9, 0x96, 0xdb, 0x76, 0x46, 0xe9, 0x3f, 0x3f, 0x25, 0xbc, 0xb0, 0x5d, 0x26, + 0x80, 0x81, 0xc2, 0x0e, 0x19, 0xa8, 0xa2, 0x80, 0x2a, 0x0a, 0x8f, 0x81, 0xb1, 0x67, 0x92, 0xca, + 0x48, 0x99, 0x14, 0xc5, 0x02, 0x55, 0x5b, 0x3a, 0xb1, 0x55, 0xd1, 0xcd, 0xea, 0x76, 0x2e, 0x6f, + 0xc0, 0x72, 0x1e, 0x81, 0x05, 0x2f, 0x42, 0xb1, 0xc9, 0x72, 0x16, 0xb3, 0x98, 0x62, 0x11, 0xc0, + 0x79, 0x0b, 0x56, 0x54, 0xeb, 0x62, 0x2b, 0x0e, 0x31, 0x61, 0xa7, 0x3e, 0xe7, 0xfb, 0xbe, 0xee, + 0x73, 0xe9, 0xd3, 0x42, 0x6f, 0xd2, 0xa9, 0xdd, 0x85, 0x73, 0x30, 0x2c, 0xf7, 0x38, 0x20, 0xfe, + 0xa8, 0x4b, 0x7c, 0xab, 0xe3, 0x07, 0x1e, 0xf3, 0xb0, 0x62, 0x78, 0xc6, 0x49, 0xe0, 0x11, 0x63, + 0xd2, 0xa1, 0x53, 0xbb, 0x63, 0x5a, 0x94, 0xd1, 0xa9, 0x1d, 0xcc, 0xdc, 0xda, 0xd6, 0xd8, 0x1b, + 0x7b, 0x21, 0xa8, 0xcb, 0xbf, 0x22, 0x7c, 0xad, 0x31, 0xf6, 0xbc, 0xb1, 0x0d, 0xdd, 0x70, 0x35, + 0x9a, 0x1d, 0x77, 0x99, 0xe5, 0x00, 0x65, 0xc4, 0xf1, 0x63, 0x00, 0x0e, 0xc5, 0xfc, 0x51, 0xd7, + 0x24, 0x8c, 0xc4, 0xb6, 0xda, 0xea, 0xfe, 0x29, 0x5f, 0x73, 0xd5, 0xe7, 0x07, 0x9e, 0x01, 0x94, + 0x7a, 0x01, 0x8d, 0x11, 0xdb, 0x1c, 0x41, 0x81, 0x52, 0xcb, 0x73, 0x39, 0xd1, 0x1f, 0x25, 0x2b, + 0x7d, 0xa9, 0xd3, 0x7a, 0x95, 0x45, 0xb2, 0x06, 0x6c, 0xe6, 0x7f, 0x63, 0x7b, 0x67, 0x2a, 0x4c, + 0x67, 0x40, 0x19, 0xfe, 0x02, 0x89, 0xc7, 0xb6, 0x77, 0xa6, 0xe4, 0x9a, 0x42, 0xbb, 0xbc, 0xdb, + 0xea, 0xdc, 0x15, 0x6c, 0x87, 0x93, 0x34, 0x1f, 0x8c, 0x9e, 0x78, 0x79, 0xd5, 0xc8, 0xa8, 0x21, + 0x0b, 0xef, 0xa0, 0xe2, 0x29, 0x04, 0x7c, 0x23, 0x25, 0xdf, 0x14, 0xda, 0xd5, 0xde, 0x1b, 0xdc, + 0xf9, 0xd7, 0x55, 0x63, 0xa3, 0x6f, 0x51, 0xa6, 0x3d, 0x7b, 0x7a, 0x14, 0x79, 0xd5, 0x04, 0x86, + 0xf7, 0x50, 0x19, 0x4e, 0x89, 0xfd, 0xc8, 0x73, 0x19, 0x9c, 0x33, 0xa5, 0x10, 0x6e, 0xfb, 0xf6, + 0xdd, 0xdb, 0x7e, 0xbd, 0x04, 0xc7, 0x3b, 0xa7, 0xf9, 0xf8, 0x39, 0xda, 0xb2, 0x81, 0x1c, 0xeb, + 0xec, 0xdc, 0xd5, 0x2d, 0xd7, 0x9f, 0x31, 0x9d, 0x32, 0xc2, 0x40, 0x29, 0x86, 0xba, 0xdb, 0x29, + 0xdd, 0x38, 0xe9, 0x9d, 0xa7, 0x40, 0x8e, 0x0f, 0xcf, 0xdd, 0x01, 0x07, 0x6b, 0x1c, 0xab, 0x6e, + 0xda, 0xab, 0x26, 0x5c, 0x47, 0xc5, 0xc3, 0x80, 0x18, 0xf0, 0xe4, 0x48, 0x91, 0x9a, 0x42, 0x5b, + 0x8a, 0xb7, 0x4e, 0x8c, 0x43, 0x51, 0x12, 0xe4, 0xec, 0x50, 0x94, 0xb2, 0x72, 0xae, 0x75, 0x2d, + 0x20, 0x29, 0x49, 0x0e, 0xfe, 0x08, 0x15, 0x79, 0x62, 0x74, 0xcb, 0x54, 0x84, 0xa6, 0xd0, 0xae, + 0xf4, 0x14, 0x4e, 0xfc, 0xed, 0xaa, 0x51, 0xe0, 0x90, 0x41, 0x7f, 0xbe, 0xf8, 0x52, 0x0b, 0x1c, + 0x38, 0x30, 0xf1, 0x1e, 0x42, 0xcb, 0x82, 0x2a, 0xd9, 0x66, 0xae, 0x5d, 0xde, 0x7d, 0xf7, 0xee, + 0x84, 0x1c, 0x24, 0xd8, 0x54, 0x31, 0x52, 0x02, 0xf8, 0x39, 0x2a, 0x8e, 0x09, 0x83, 0x33, 0x72, + 0x11, 0xd6, 0x34, 0xdf, 0xfb, 0x3c, 0x2e, 0xc9, 0xc3, 0xb1, 0xc5, 0x26, 0xb3, 0x51, 0xc7, 0xf0, + 0x9c, 0xee, 0x42, 0xdd, 0x1c, 0x2d, 0xbf, 0xbb, 0xfe, 0xc9, 0xb8, 0x9b, 0xa4, 0x69, 0xdf, 0x33, + 0x61, 0xd0, 0x57, 0x13, 0xad, 0xd6, 0xaf, 0x39, 0x54, 0x4e, 0xd5, 0x02, 0x7f, 0x8c, 0x30, 0x65, + 0x0e, 0x3b, 0x4c, 0x7a, 0x7b, 0x9f, 0xb8, 0x1e, 0x0d, 0x63, 0xce, 0xc5, 0x87, 0xfa, 0x07, 0x3f, + 0xde, 0x45, 0x9b, 0xec, 0xdc, 0x5d, 0x21, 0x65, 0x53, 0xa4, 0xdb, 0x6e, 0xdc, 0x44, 0x92, 0xed, + 0x19, 0x84, 0xf1, 0x26, 0x13, 0x9b, 0x42, 0xbb, 0x14, 0x43, 0x17, 0x56, 0xdc, 0x40, 0x65, 0x0a, + 0x24, 0x30, 0x26, 0xba, 0x4f, 0xd8, 0x44, 0x29, 0x34, 0x73, 0xed, 0x92, 0x8a, 0x22, 0xd3, 0x01, + 0x61, 0x13, 0x3c, 0x44, 0x25, 0x0a, 0xd3, 0xb8, 0x35, 0xa4, 0xb0, 0x35, 0xd6, 0x64, 0x58, 0xe3, + 0x57, 0xc3, 0x35, 0x20, 0x6c, 0x85, 0x64, 0x33, 0x0a, 0xd3, 0xa8, 0x35, 0x3e, 0x45, 0xff, 0x67, + 0xe0, 0xf8, 0x5e, 0x40, 0x82, 0x0b, 0x9d, 0x1a, 0x13, 0x70, 0x88, 0xee, 0x12, 0x07, 0x94, 0x6a, + 0xea, 0x6c, 0xff, 0x5b, 0x40, 0xb4, 0x10, 0xb1, 0x4f, 0x1c, 0xc0, 0xcf, 0x50, 0x25, 0x7d, 0x2b, + 0x95, 0x07, 0xe1, 0x41, 0xda, 0x2b, 0x07, 0xb9, 0x71, 0x8d, 0x3b, 0x5a, 0xb4, 0xea, 0x13, 0x46, + 0x92, 0xf6, 0xa7, 0x4b, 0xd3, 0x50, 0x94, 0x72, 0xb2, 0x38, 0x14, 0xa5, 0xbc, 0x5c, 0x18, 0x8a, + 0x52, 0x51, 0x96, 0x86, 0xa2, 0x54, 0x92, 0xd1, 0x50, 0x94, 0x90, 0x5c, 0x1e, 0x8a, 0x52, 0x59, + 0xae, 0x0c, 0x45, 0xa9, 0x22, 0x57, 0x87, 0xa2, 0xb4, 0x21, 0x3f, 0x68, 0xbd, 0x12, 0x50, 0xf5, + 0x46, 0x78, 0xf8, 0x4b, 0x24, 0x52, 0x98, 0xf2, 0xca, 0xf1, 0xbe, 0xfb, 0xe0, 0x9e, 0x59, 0xe1, + 0x2b, 0x35, 0x24, 0xe2, 0x1d, 0xb4, 0x65, 0x13, 0xca, 0x74, 0x9e, 0x60, 0xcb, 0x35, 0x02, 0x70, + 0xc0, 0x65, 0x60, 0x86, 0x55, 0xad, 0xaa, 0x98, 0xfb, 0x34, 0x98, 0x0e, 0x96, 0x9e, 0xda, 0x01, + 0xca, 0x69, 0x30, 0xc5, 0xdb, 0xa8, 0x10, 0x72, 0xa2, 0x9b, 0x52, 0xed, 0x55, 0x79, 0x78, 0xf3, + 0xab, 0x46, 0x9e, 0xc3, 0xfb, 0x6a, 0x9e, 0xc2, 0x74, 0x60, 0xe2, 0xb7, 0x10, 0xb2, 0x09, 0x03, + 0xca, 0xf4, 0x53, 0x62, 0xdf, 0x68, 0x95, 0x52, 0x64, 0x3f, 0x22, 0x76, 0xeb, 0x5b, 0xb4, 0xa1, + 0x59, 0x8e, 0x6f, 0x83, 0x0a, 0xd4, 0xf7, 0x5c, 0x0a, 0xf8, 0x13, 0x94, 0x87, 0x20, 0xf0, 0x82, + 0x50, 0xbb, 0xbc, 0xdb, 0x58, 0x33, 0x60, 0x38, 0x4c, 0x8d, 0xd0, 0xad, 0x1f, 0xb3, 0x68, 0xe3, + 0x91, 0xe7, 0xd2, 0x99, 0x03, 0x81, 0x66, 0x8d, 0x5d, 0x62, 0xe3, 0x27, 0xa8, 0x6a, 0x06, 0xc4, + 0x72, 0xf5, 0x20, 0x9a, 0x98, 0xb1, 0xe2, 0x3b, 0x77, 0x2b, 0xf6, 0x39, 0x3c, 0x9e, 0xaf, 0x6a, + 0xc5, 0x4c, 0xad, 0xf0, 0x77, 0x08, 0x53, 0x3e, 0x81, 0xf5, 0x70, 0x48, 0x24, 0x8a, 0xd9, 0x50, + 0xf1, 0xfd, 0x75, 0xb9, 0xbf, 0x39, 0xb5, 0x55, 0x99, 0xae, 0xce, 0xf1, 0x01, 0x2a, 0x4d, 0x88, + 0x6b, 0xd2, 0x09, 0x39, 0x81, 0x78, 0x98, 0xaf, 0x29, 0x66, 0x12, 0xe3, 0xe3, 0x84, 0xa2, 0x2e, + 0xd9, 0x9f, 0x89, 0x97, 0x3f, 0x35, 0x84, 0xd6, 0x06, 0xaa, 0xa4, 0x03, 0x69, 0xfd, 0x9c, 0x45, + 0x9b, 0xb7, 0x68, 0xf8, 0x21, 0xc2, 0x46, 0x6c, 0x0c, 0x2f, 0x83, 0x39, 0xb3, 0x21, 0x2a, 0x68, + 0x32, 0x33, 0x37, 0x13, 0xbf, 0x96, 0xb8, 0xf1, 0x0f, 0xa8, 0x76, 0x8b, 0xa4, 0x9b, 0x40, 0x4c, + 0xdb, 0x72, 0x21, 0xce, 0x46, 0xad, 0x13, 0x3d, 0xa3, 0x9d, 0xe4, 0x19, 0xed, 0x2c, 0xc6, 0x42, + 0x4f, 0x7c, 0xf1, 0x7b, 0x43, 0x50, 0x95, 0x55, 0xe1, 0x7e, 0xac, 0x90, 0x7e, 0x95, 0x72, 0xf7, + 0x7b, 0x95, 0x1e, 0xa3, 0x2d, 0xc7, 0x72, 0x75, 0x62, 0x18, 0xe0, 0x33, 0x30, 0xf5, 0x84, 0x2e, + 0xae, 0xa5, 0x63, 0xc7, 0x72, 0xbf, 0x8a, 0x29, 0xb1, 0x6d, 0xf7, 0x97, 0x2c, 0x2a, 0xc6, 0x30, + 0x3c, 0x41, 0x65, 0x75, 0xe6, 0x6a, 0x17, 0xae, 0xc1, 0x2b, 0x85, 0xdb, 0xff, 0x5e, 0x8f, 0xa8, + 0xe7, 0x6a, 0xef, 0xad, 0x1d, 0xff, 0xe6, 0xcc, 0x80, 0x60, 0x0f, 0x28, 0x25, 0x63, 0x68, 0x65, + 0xda, 0xc2, 0x8e, 0x80, 0x0d, 0x54, 0x5a, 0xf4, 0x08, 0xfe, 0x0f, 0x8d, 0x54, 0x5b, 0x73, 0xa6, + 0x9b, 0x37, 0xaa, 0x95, 0xc1, 0x63, 0x84, 0xc2, 0x77, 0x8e, 0x05, 0x40, 0x1c, 0x7c, 0xff, 0x33, + 0xd6, 0xee, 0x1d, 0x78, 0x14, 0x4d, 0xef, 0xc3, 0xcb, 0x3f, 0xeb, 0x99, 0xcb, 0x79, 0x5d, 0x78, + 0x39, 0xaf, 0x0b, 0xaf, 0xe7, 0x75, 0xe1, 0x8f, 0x79, 0x5d, 0x78, 0x71, 0x5d, 0xcf, 0xbc, 0xbc, + 0xae, 0x67, 0x5e, 0x5f, 0xd7, 0x33, 0xdf, 0x97, 0x53, 0xbf, 0x43, 0x7f, 0x07, 0x00, 0x00, 0xff, + 0xff, 0x82, 0x86, 0x27, 0x87, 0xbc, 0x09, 0x00, 0x00, } diff --git a/pkg/sql/execinfrapb/api.proto b/pkg/sql/execinfrapb/api.proto index 61dbffc5ef1e..5f74c4f03f05 100644 --- a/pkg/sql/execinfrapb/api.proto +++ b/pkg/sql/execinfrapb/api.proto @@ -23,6 +23,7 @@ import "google/protobuf/timestamp.proto"; import "roachpb/data.proto"; import "sql/execinfrapb/data.proto"; import "sql/execinfrapb/processors.proto"; +import "sql/sessiondatapb/session_data.proto"; message SetupFlowRequest { reserved 1, 2; @@ -67,23 +68,14 @@ message EvalContext { // The name of the location according to whose current timezone we're going to // parse timestamps. Used to init EvalContext.Location. optional string location = 4 [(gogoproto.nullable) = false]; - optional string database = 5 [(gogoproto.nullable) = false]; + reserved 5; repeated string search_path = 6; - optional string user = 7 [(gogoproto.nullable) = false]; + reserved 7; optional SequenceState seq_state = 8 [(gogoproto.nullable) = false]; - optional string application_name = 9 [(gogoproto.nullable) = false]; - optional BytesEncodeFormat bytes_encode_format = 10 [(gogoproto.nullable) = false]; - optional int32 extra_float_digits = 11 [(gogoproto.nullable) = false]; - optional int32 vectorize = 12 [(gogoproto.nullable) = false]; + reserved 9, 10, 11, 12; optional string temporary_schema_name = 13 [(gogoproto.nullable) = false]; - optional bool vectorize_inject_panics = 14 [(gogoproto.nullable) = false]; -} - -// BytesEncodeFormat is the configuration for bytes to string conversions. -enum BytesEncodeFormat { - HEX = 0; - ESCAPE = 1; - BASE64 = 2; + reserved 14; + optional sessiondatapb.SessionData session_data = 15 [(gogoproto.nullable) = false]; } // SequenceState is used to marshall the sessiondata.SequenceState struct. diff --git a/pkg/sql/execinfrapb/expr.go b/pkg/sql/execinfrapb/expr.go index 7a1c4b0d9ee4..22b3974d99c5 100644 --- a/pkg/sql/execinfrapb/expr.go +++ b/pkg/sql/execinfrapb/expr.go @@ -58,7 +58,10 @@ func processExpression( if exprSpec.Expr == "" { return nil, nil } - expr, err := parser.ParseExpr(exprSpec.Expr) + expr, err := parser.ParseExprWithInt( + exprSpec.Expr, + parser.NakedIntTypeFromDefaultIntSize(evalCtx.SessionData.DefaultIntSize), + ) if err != nil { return nil, err } diff --git a/pkg/sql/explain_plan.go b/pkg/sql/explain_plan.go index 3cd922827d34..23079fb6a006 100644 --- a/pkg/sql/explain_plan.go +++ b/pkg/sql/explain_plan.go @@ -25,7 +25,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/opt/exec/explain" "github.com/cockroachdb/cockroach/pkg/sql/physicalplan" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" - "github.com/cockroachdb/cockroach/pkg/sql/sessiondata" + "github.com/cockroachdb/cockroach/pkg/sql/sessiondatapb" ) // explainPlanNode implements EXPLAIN (PLAN); it produces the output of @@ -147,9 +147,9 @@ func explainGetDistributedAndVectorized( ctxSessionData := flowCtx.EvalCtx.SessionData vectorizedThresholdMet := physicalPlan.MaxEstimatedRowCount >= ctxSessionData.VectorizeRowCountThreshold - if ctxSessionData.VectorizeMode == sessiondata.VectorizeOff { + if ctxSessionData.VectorizeMode == sessiondatapb.VectorizeOff { willVectorize = false - } else if !vectorizedThresholdMet && (ctxSessionData.VectorizeMode == sessiondata.Vectorize201Auto || ctxSessionData.VectorizeMode == sessiondata.VectorizeOn) { + } else if !vectorizedThresholdMet && (ctxSessionData.VectorizeMode == sessiondatapb.Vectorize201Auto || ctxSessionData.VectorizeMode == sessiondatapb.VectorizeOn) { willVectorize = false } else { willVectorize = true diff --git a/pkg/sql/explain_vec.go b/pkg/sql/explain_vec.go index 95736c7ab956..be7fce7d736c 100644 --- a/pkg/sql/explain_vec.go +++ b/pkg/sql/explain_vec.go @@ -24,7 +24,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/execinfrapb" "github.com/cockroachdb/cockroach/pkg/sql/physicalplan" "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/util/mon" "github.com/cockroachdb/cockroach/pkg/util/treeprinter" "github.com/cockroachdb/errors" @@ -83,7 +83,7 @@ func (n *explainVecNode) startExec(params runParams) error { // With all other options, we don't change the setting to the // most-inclusive option as we used to because the plan can be different // based on 'vectorize' setting. - if flowCtx.EvalCtx.SessionData.VectorizeMode == sessiondata.VectorizeOff { + if flowCtx.EvalCtx.SessionData.VectorizeMode == sessiondatapb.VectorizeOff { return errors.New("vectorize is set to 'off'") } diff --git a/pkg/sql/internal.go b/pkg/sql/internal.go index f3b338d6aa6c..c81417fb7f22 100644 --- a/pkg/sql/internal.go +++ b/pkg/sql/internal.go @@ -15,6 +15,7 @@ import ( "math" "strings" "sync" + "time" "github.com/cockroachdb/cockroach/pkg/kv" "github.com/cockroachdb/cockroach/pkg/security" @@ -26,6 +27,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgwirebase" "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/sqltelemetry" "github.com/cockroachdb/cockroach/pkg/sql/sqlutil" "github.com/cockroachdb/cockroach/pkg/util/mon" @@ -520,6 +522,8 @@ type internalClientComm struct { sync func([]resWithPos) } +var _ ClientComm = &internalClientComm{} + type resWithPos struct { *bufferedCommandResult pos CmdPos @@ -531,7 +535,8 @@ func (icc *internalClientComm) CreateStatementResult( _ RowDescOpt, pos CmdPos, _ []pgwirebase.FormatCode, - _ sessiondata.DataConversionConfig, + _ sessiondatapb.DataConversionConfig, + _ *time.Location, _ int, _ string, _ bool, diff --git a/pkg/sql/internal_test.go b/pkg/sql/internal_test.go index 32b53b8b7f66..8e395573ffad 100644 --- a/pkg/sql/internal_test.go +++ b/pkg/sql/internal_test.go @@ -27,6 +27,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/roleoption" "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/tests" "github.com/cockroachdb/cockroach/pkg/testutils" "github.com/cockroachdb/cockroach/pkg/testutils/serverutils" @@ -148,9 +149,11 @@ func TestInternalFullTableScan(t *testing.T) { ) ie.SetSessionData( &sessiondata.SessionData{ - Database: "db", + SessionData: sessiondatapb.SessionData{ + Database: "db", + User: security.RootUser, + }, SequenceState: &sessiondata.SequenceState{}, - User: security.RootUser, DisallowFullTableScans: true, }) @@ -277,9 +280,11 @@ func TestSessionBoundInternalExecutor(t *testing.T) { ) ie.SetSessionData( &sessiondata.SessionData{ - Database: expDB, + SessionData: sessiondatapb.SessionData{ + Database: expDB, + User: security.RootUser, + }, SequenceState: &sessiondata.SequenceState{}, - User: security.RootUser, }) row, err := ie.QueryRowEx(ctx, "test", nil, /* txn */ @@ -343,10 +348,13 @@ func TestInternalExecAppNameInitialization(t *testing.T) { ) ie.SetSessionData( &sessiondata.SessionData{ - User: security.RootUser, - Database: "defaultdb", - ApplicationName: "appname_findme", - SequenceState: &sessiondata.SequenceState{}, + SessionData: sessiondatapb.SessionData{ + User: security.RootUser, + Database: "defaultdb", + ApplicationName: "appname_findme", + }, + DisallowFullTableScans: true, + SequenceState: &sessiondata.SequenceState{}, }) testInternalExecutorAppNameInitialization( t, sem, diff --git a/pkg/sql/lex/encode.go b/pkg/sql/lex/encode.go index e1b4151244b8..6868bd2e20dc 100644 --- a/pkg/sql/lex/encode.go +++ b/pkg/sql/lex/encode.go @@ -23,12 +23,11 @@ import ( "bytes" "encoding/base64" "encoding/hex" - "fmt" - "strings" "unicode/utf8" "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgcode" "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror" + "github.com/cockroachdb/cockroach/pkg/sql/sessiondatapb" "github.com/cockroachdb/cockroach/pkg/util/stringencoding" "github.com/cockroachdb/errors" ) @@ -235,9 +234,11 @@ func EncodeSQLBytes(buf *bytes.Buffer, in string) { // If the skipHexPrefix argument is set, the hexadecimal encoding does not // prefix the output with "\x". This is suitable e.g. for the encode() // built-in. -func EncodeByteArrayToRawBytes(data string, be BytesEncodeFormat, skipHexPrefix bool) string { +func EncodeByteArrayToRawBytes( + data string, be sessiondatapb.BytesEncodeFormat, skipHexPrefix bool, +) string { switch be { - case BytesEncodeHex: + case sessiondatapb.BytesEncodeHex: head := 2 if skipHexPrefix { head = 0 @@ -250,7 +251,7 @@ func EncodeByteArrayToRawBytes(data string, be BytesEncodeFormat, skipHexPrefix hex.Encode(res[head:], []byte(data)) return string(res) - case BytesEncodeEscape: + case sessiondatapb.BytesEncodeEscape: // PostgreSQL does not allow all the escapes formats recognized by // CockroachDB's scanner. It only recognizes octal and \\ for the // backslash itself. @@ -272,7 +273,7 @@ func EncodeByteArrayToRawBytes(data string, be BytesEncodeFormat, skipHexPrefix } return string(res) - case BytesEncodeBase64: + case sessiondatapb.BytesEncodeBase64: return base64.StdEncoding.EncodeToString([]byte(data)) default: @@ -285,12 +286,12 @@ func EncodeByteArrayToRawBytes(data string, be BytesEncodeFormat, skipHexPrefix // When using the Hex format, the caller is responsible for skipping the // "\x" prefix, if any. See DecodeRawBytesToByteArrayAuto() below for // an alternative. -func DecodeRawBytesToByteArray(data string, be BytesEncodeFormat) ([]byte, error) { +func DecodeRawBytesToByteArray(data string, be sessiondatapb.BytesEncodeFormat) ([]byte, error) { switch be { - case BytesEncodeHex: + case sessiondatapb.BytesEncodeHex: return hex.DecodeString(data) - case BytesEncodeEscape: + case sessiondatapb.BytesEncodeEscape: // PostgreSQL does not allow all the escapes formats recognized by // CockroachDB's scanner. It only recognizes octal and \\ for the // backslash itself. @@ -329,7 +330,7 @@ func DecodeRawBytesToByteArray(data string, be BytesEncodeFormat) ([]byte, error } return res, nil - case BytesEncodeBase64: + case sessiondatapb.BytesEncodeBase64: return base64.StdEncoding.DecodeString(data) default: @@ -342,48 +343,7 @@ func DecodeRawBytesToByteArray(data string, be BytesEncodeFormat) ([]byte, error // and escape. func DecodeRawBytesToByteArrayAuto(data []byte) ([]byte, error) { if len(data) >= 2 && data[0] == '\\' && (data[1] == 'x' || data[1] == 'X') { - return DecodeRawBytesToByteArray(string(data[2:]), BytesEncodeHex) - } - return DecodeRawBytesToByteArray(string(data), BytesEncodeEscape) -} - -// BytesEncodeFormat controls which format to use for BYTES->STRING -// conversions. -type BytesEncodeFormat int - -const ( - // BytesEncodeHex uses the hex format: e'abc\n'::BYTES::STRING -> '\x61626312'. - // This is the default, for compatibility with PostgreSQL. - BytesEncodeHex BytesEncodeFormat = iota - // BytesEncodeEscape uses the escaped format: e'abc\n'::BYTES::STRING -> 'abc\012'. - BytesEncodeEscape - // BytesEncodeBase64 uses base64 encoding. - BytesEncodeBase64 -) - -func (f BytesEncodeFormat) String() string { - switch f { - case BytesEncodeHex: - return "hex" - case BytesEncodeEscape: - return "escape" - case BytesEncodeBase64: - return "base64" - default: - return fmt.Sprintf("invalid (%d)", f) - } -} - -// BytesEncodeFormatFromString converts a string into a BytesEncodeFormat. -func BytesEncodeFormatFromString(val string) (_ BytesEncodeFormat, ok bool) { - switch strings.ToUpper(val) { - case "HEX": - return BytesEncodeHex, true - case "ESCAPE": - return BytesEncodeEscape, true - case "BASE64": - return BytesEncodeBase64, true - default: - return -1, false + return DecodeRawBytesToByteArray(string(data[2:]), sessiondatapb.BytesEncodeHex) } + return DecodeRawBytesToByteArray(string(data), sessiondatapb.BytesEncodeEscape) } diff --git a/pkg/sql/lex/encode_test.go b/pkg/sql/lex/encode_test.go index e872c71e968c..1abe7966f570 100644 --- a/pkg/sql/lex/encode_test.go +++ b/pkg/sql/lex/encode_test.go @@ -19,6 +19,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/lex" "github.com/cockroachdb/cockroach/pkg/sql/parser" + "github.com/cockroachdb/cockroach/pkg/sql/sessiondatapb" ) func TestEncodeSQLBytes(t *testing.T) { @@ -126,14 +127,14 @@ func TestEncodeRestrictedSQLIdent(t *testing.T) { func TestByteArrayDecoding(t *testing.T) { const ( - fmtHex = lex.BytesEncodeHex - fmtEsc = lex.BytesEncodeEscape - fmtB64 = lex.BytesEncodeBase64 + fmtHex = sessiondatapb.BytesEncodeHex + fmtEsc = sessiondatapb.BytesEncodeEscape + fmtB64 = sessiondatapb.BytesEncodeBase64 ) testData := []struct { in string auto bool - inFmt lex.BytesEncodeFormat + inFmt sessiondatapb.BytesEncodeFormat out string err string }{ @@ -209,8 +210,11 @@ func TestByteArrayEncoding(t *testing.T) { for _, s := range testData { t.Run(s.in, func(t *testing.T) { - for _, format := range []lex.BytesEncodeFormat{ - lex.BytesEncodeHex, lex.BytesEncodeEscape, lex.BytesEncodeBase64} { + for _, format := range []sessiondatapb.BytesEncodeFormat{ + sessiondatapb.BytesEncodeHex, + sessiondatapb.BytesEncodeEscape, + sessiondatapb.BytesEncodeBase64, + } { t.Run(format.String(), func(t *testing.T) { enc := lex.EncodeByteArrayToRawBytes(s.in, format, false) @@ -219,7 +223,7 @@ func TestByteArrayEncoding(t *testing.T) { t.Fatalf("encoded %q, expected %q", enc, expEnc) } - if format == lex.BytesEncodeHex { + if format == sessiondatapb.BytesEncodeHex { // Check that the \x also can be skipped. enc2 := lex.EncodeByteArrayToRawBytes(s.in, format, true) if enc[2:] != enc2 { diff --git a/pkg/sql/parser/parse.go b/pkg/sql/parser/parse.go index bccf244c3a07..c889ccf4b83f 100644 --- a/pkg/sql/parser/parse.go +++ b/pkg/sql/parser/parse.go @@ -88,10 +88,22 @@ type Parser struct { // INT8 is the historical interpretation of INT. This should be left // alone in the future, since there are many sql fragments stored -// in various descriptors. Any user input that was created after +// in various descriptors. Any user input that was created after // INT := INT4 will simply use INT4 in any resulting code. var defaultNakedIntType = types.Int +// NakedIntTypeFromDefaultIntSize given the size in bits or bytes (preferred) +// of how a "naked" INT type should be parsed returns the corresponding integer +// type. +func NakedIntTypeFromDefaultIntSize(defaultIntSize int32) *types.T { + switch defaultIntSize { + case 4, 32: + return types.Int4 + default: + return types.Int + } +} + // Parse parses the sql and returns a list of statements. func (p *Parser) Parse(sql string) (Statements, error) { return p.parseWithDepth(1, sql, defaultNakedIntType) @@ -103,8 +115,8 @@ func (p *Parser) ParseWithInt(sql string, nakedIntType *types.T) (Statements, er return p.parseWithDepth(1, sql, nakedIntType) } -func (p *Parser) parseOneWithDepth(depth int, sql string) (Statement, error) { - stmts, err := p.parseWithDepth(1, sql, defaultNakedIntType) +func (p *Parser) parseOneWithInt(sql string, nakedIntType *types.T) (Statement, error) { + stmts, err := p.parseWithDepth(1, sql, nakedIntType) if err != nil { return Statement{}, err } @@ -232,8 +244,14 @@ func Parse(sql string) (Statements, error) { // bits of SQL from other nodes. In general,earwe expect that all // user-generated SQL has been run through the ParseWithInt() function. func ParseOne(sql string) (Statement, error) { + return ParseOneWithInt(sql, defaultNakedIntType) +} + +// ParseOneWithInt is similar to ParseOn but interprets the INT and SERIAL +// types as the provided integer type. +func ParseOneWithInt(sql string, nakedIntType *types.T) (Statement, error) { var p Parser - return p.parseOneWithDepth(1, sql) + return p.parseOneWithInt(sql, nakedIntType) } // HasMultipleStatements returns true if the sql string contains more than one @@ -306,9 +324,9 @@ func ParseTableNameWithQualifiedNames(sql string) (*tree.UnresolvedObjectName, e return tree.NewUnresolvedObjectName(numParts, nameParts, 0 /* annotationIdx */) } -// parseExprs parses one or more sql expressions. -func parseExprs(exprs []string) (tree.Exprs, error) { - stmt, err := ParseOne(fmt.Sprintf("SET ROW (%s)", strings.Join(exprs, ","))) +// parseExprsWithInt parses one or more sql expressions. +func parseExprsWithInt(exprs []string, nakedIntType *types.T) (tree.Exprs, error) { + stmt, err := ParseOneWithInt(fmt.Sprintf("SET ROW (%s)", strings.Join(exprs, ",")), nakedIntType) if err != nil { return nil, err } @@ -319,17 +337,24 @@ func parseExprs(exprs []string) (tree.Exprs, error) { return set.Values, nil } -// ParseExprs is a short-hand for parseExprs(sql) +// ParseExprs is a short-hand for parseExprsWithInt(sql, defaultNakedIntType). func ParseExprs(sql []string) (tree.Exprs, error) { if len(sql) == 0 { return tree.Exprs{}, nil } - return parseExprs(sql) + return parseExprsWithInt(sql, defaultNakedIntType) } -// ParseExpr is a short-hand for parseExprs([]string{sql}) +// ParseExpr is a short-hand for parseExprsWithInt([]string{sql}, +// defaultNakedIntType). func ParseExpr(sql string) (tree.Expr, error) { - exprs, err := parseExprs([]string{sql}) + return ParseExprWithInt(sql, defaultNakedIntType) +} + +// ParseExprWithInt is a short-hand for parseExprsWithInt([]string{sql}, +// nakedIntType).' +func ParseExprWithInt(sql string, nakedIntType *types.T) (tree.Expr, error) { + exprs, err := parseExprsWithInt([]string{sql}, nakedIntType) if err != nil { return nil, err } diff --git a/pkg/sql/pgwire/command_result.go b/pkg/sql/pgwire/command_result.go index 4bdd9caa5960..3e7b2c066d7b 100644 --- a/pkg/sql/pgwire/command_result.go +++ b/pkg/sql/pgwire/command_result.go @@ -12,6 +12,7 @@ package pgwire import ( "context" + "time" "github.com/cockroachdb/cockroach/pkg/server/telemetry" "github.com/cockroachdb/cockroach/pkg/sql" @@ -19,7 +20,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgnotice" "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgwirebase" "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/sqltelemetry" "github.com/cockroachdb/cockroach/pkg/sql/types" "github.com/cockroachdb/errors" @@ -46,8 +47,9 @@ const ( type commandResult struct { // conn is the parent connection of this commandResult. conn *conn - // conv indicates the conversion settings for SQL values. - conv sessiondata.DataConversionConfig + // conv and location indicate the conversion settings for SQL values. + conv sessiondatapb.DataConversionConfig + location *time.Location // pos identifies the position of the command within the connection. pos sql.CmdPos @@ -201,7 +203,7 @@ func (r *commandResult) AddRow(ctx context.Context, row tree.Datums) error { } r.rowsAffected++ - r.conn.bufferRow(ctx, row, r.formatCodes, r.conv, r.types) + r.conn.bufferRow(ctx, row, r.formatCodes, r.conv, r.location, r.types) var err error if r.bufferingDisabled { err = r.conn.Flush(r.pos) @@ -325,7 +327,8 @@ func (c *conn) newCommandResult( pos sql.CmdPos, stmt tree.Statement, formatCodes []pgwirebase.FormatCode, - conv sessiondata.DataConversionConfig, + conv sessiondatapb.DataConversionConfig, + location *time.Location, limit int, portalName string, implicitTxn bool, @@ -334,6 +337,7 @@ func (c *conn) newCommandResult( *r = commandResult{ conn: c, conv: conv, + location: location, pos: pos, typ: commandComplete, cmdCompleteTag: stmt.StatementTag(), diff --git a/pkg/sql/pgwire/conn.go b/pkg/sql/pgwire/conn.go index 2e07e0621055..076329dfb745 100644 --- a/pkg/sql/pgwire/conn.go +++ b/pkg/sql/pgwire/conn.go @@ -32,7 +32,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgnotice" "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgwirebase" "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/sqltelemetry" "github.com/cockroachdb/cockroach/pkg/sql/types" "github.com/cockroachdb/cockroach/pkg/util/log" @@ -1148,7 +1148,8 @@ func (c *conn) bufferRow( ctx context.Context, row tree.Datums, formatCodes []pgwirebase.FormatCode, - conv sessiondata.DataConversionConfig, + conv sessiondatapb.DataConversionConfig, + sessionLoc *time.Location, types []*types.T, ) { c.msgBuilder.initMsg(pgwirebase.ServerMsgDataRow) @@ -1160,9 +1161,9 @@ func (c *conn) bufferRow( } switch fmtCode { case pgwirebase.FormatText: - c.msgBuilder.writeTextDatum(ctx, col, conv, types[i]) + c.msgBuilder.writeTextDatum(ctx, col, conv, sessionLoc, types[i]) case pgwirebase.FormatBinary: - c.msgBuilder.writeBinaryDatum(ctx, col, conv.Location, types[i]) + c.msgBuilder.writeBinaryDatum(ctx, col, sessionLoc, types[i]) default: c.msgBuilder.setError(errors.Errorf("unsupported format code %s", fmtCode)) } @@ -1433,12 +1434,13 @@ func (c *conn) CreateStatementResult( descOpt sql.RowDescOpt, pos sql.CmdPos, formatCodes []pgwirebase.FormatCode, - conv sessiondata.DataConversionConfig, + conv sessiondatapb.DataConversionConfig, + location *time.Location, limit int, portalName string, implicitTxn bool, ) sql.CommandResult { - return c.newCommandResult(descOpt, pos, stmt, formatCodes, conv, limit, portalName, implicitTxn) + return c.newCommandResult(descOpt, pos, stmt, formatCodes, conv, location, limit, portalName, implicitTxn) } // CreateSyncResult is part of the sql.ClientComm interface. diff --git a/pkg/sql/pgwire/conn_test.go b/pkg/sql/pgwire/conn_test.go index 50b532a538b4..7dc8ea3a699c 100644 --- a/pkg/sql/pgwire/conn_test.go +++ b/pkg/sql/pgwire/conn_test.go @@ -29,12 +29,12 @@ import ( "github.com/cockroachdb/cockroach/pkg/security" "github.com/cockroachdb/cockroach/pkg/sql" "github.com/cockroachdb/cockroach/pkg/sql/catalog/colinfo" - "github.com/cockroachdb/cockroach/pkg/sql/lex" "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgcode" "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror" "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgwirebase" "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/sqlutil" "github.com/cockroachdb/cockroach/pkg/sql/tests" "github.com/cockroachdb/cockroach/pkg/testutils" @@ -547,11 +547,10 @@ func waitForClientConn(ln net.Listener) (*conn, error) { return pgwireConn, nil } -func makeTestingConvCfg() sessiondata.DataConversionConfig { - return sessiondata.DataConversionConfig{ - Location: time.UTC, - BytesEncodeFormat: lex.BytesEncodeHex, - } +func makeTestingConvCfg() (sessiondatapb.DataConversionConfig, *time.Location) { + return sessiondatapb.DataConversionConfig{ + BytesEncodeFormat: sessiondatapb.BytesEncodeHex, + }, time.UTC } // sendResult serializes a set of rows in pgwire format and sends them on a @@ -566,12 +565,12 @@ func sendResult( return err } - defaultConv := makeTestingConvCfg() + defaultConv, defaultLoc := makeTestingConvCfg() for _, row := range rows { c.msgBuilder.initMsg(pgwirebase.ServerMsgDataRow) c.msgBuilder.putInt16(int16(len(row))) for i, col := range row { - c.msgBuilder.writeTextDatum(ctx, col, defaultConv, cols[i].Typ) + c.msgBuilder.writeTextDatum(ctx, col, defaultConv, defaultLoc, cols[i].Typ) } if err := c.msgBuilder.finishMsg(c.conn); err != nil { diff --git a/pkg/sql/pgwire/encoding_test.go b/pkg/sql/pgwire/encoding_test.go index d54134ef3ef4..f5b32277fe5d 100644 --- a/pkg/sql/pgwire/encoding_test.go +++ b/pkg/sql/pgwire/encoding_test.go @@ -26,7 +26,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/parser" "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgwirebase" "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" @@ -134,7 +133,7 @@ func TestEncodings(t *testing.T) { return data } - var conv sessiondata.DataConversionConfig + conv, loc := makeTestingConvCfg() ctx := context.Background() evalCtx := tree.MakeTestingEvalContext(nil) t.Run("encode", func(t *testing.T) { @@ -144,7 +143,7 @@ func TestEncodings(t *testing.T) { buf.reset() buf.textFormatter.Buffer.Reset() - buf.writeTextDatum(ctx, d, conv, tc.T) + buf.writeTextDatum(ctx, d, conv, loc, tc.T) if buf.err != nil { t.Fatal(buf.err) } @@ -257,7 +256,7 @@ func TestExoticNumericEncodings(t *testing.T) { func BenchmarkEncodings(b *testing.B) { tests := readEncodingTests(b) buf := newWriteBuffer(metric.NewCounter(metric.Metadata{})) - var conv sessiondata.DataConversionConfig + conv, loc := makeTestingConvCfg() ctx := context.Background() for _, tc := range tests { @@ -268,7 +267,7 @@ func BenchmarkEncodings(b *testing.B) { for i := 0; i < b.N; i++ { buf.reset() buf.textFormatter.Buffer.Reset() - buf.writeTextDatum(ctx, d, conv, tc.T) + buf.writeTextDatum(ctx, d, conv, loc, tc.T) } }) b.Run("binary", func(b *testing.B) { diff --git a/pkg/sql/pgwire/types.go b/pkg/sql/pgwire/types.go index 919f359c7afe..a6751e1b179f 100644 --- a/pkg/sql/pgwire/types.go +++ b/pkg/sql/pgwire/types.go @@ -26,7 +26,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/lex" "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgwirebase" "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/sqltelemetry" "github.com/cockroachdb/cockroach/pkg/sql/types" "github.com/cockroachdb/cockroach/pkg/util/duration" @@ -80,7 +80,11 @@ func resolveBlankPaddedChar(s string, t *types.T) string { // that have various width encodings and therefore need padding (chars). // It is ignored (and can be nil) for types which do not need padding. func (b *writeBuffer) writeTextDatum( - ctx context.Context, d tree.Datum, conv sessiondata.DataConversionConfig, t *types.T, + ctx context.Context, + d tree.Datum, + conv sessiondatapb.DataConversionConfig, + sessionLoc *time.Location, + t *types.T, ) { if log.V(2) { log.Infof(ctx, "pgwire writing TEXT datum of type: %T, %#v", d, d) @@ -175,7 +179,7 @@ func (b *writeBuffer) writeTextDatum( case *tree.DTimestampTZ: // Start at offset 4 because `putInt32` clobbers the first 4 bytes. - s := formatTs(v.Time, conv.Location, b.putbuf[4:4]) + s := formatTs(v.Time, sessionLoc, b.putbuf[4:4]) b.putInt32(int32(len(s))) b.write(s) diff --git a/pkg/sql/pgwire/types_test.go b/pkg/sql/pgwire/types_test.go index 51f125e3f76c..ca245678ca7f 100644 --- a/pkg/sql/pgwire/types_test.go +++ b/pkg/sql/pgwire/types_test.go @@ -20,10 +20,10 @@ import ( "time" "github.com/cockroachdb/cockroach/pkg/settings/cluster" - "github.com/cockroachdb/cockroach/pkg/sql/lex" "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgwirebase" "github.com/cockroachdb/cockroach/pkg/sql/rowenc" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" + "github.com/cockroachdb/cockroach/pkg/sql/sessiondatapb" "github.com/cockroachdb/cockroach/pkg/sql/types" "github.com/cockroachdb/cockroach/pkg/util/leaktest" "github.com/cockroachdb/cockroach/pkg/util/log" @@ -99,17 +99,17 @@ func TestWriteBinaryArray(t *testing.T) { st := cluster.MakeTestingClusterSettings() ary, _, _ := tree.ParseDArrayFromString(tree.NewTestingEvalContext(st), "{1}", types.Int) - defaultConv := makeTestingConvCfg() + defaultConv, defaultLoc := makeTestingConvCfg() writeBuf1 := newWriteBuffer(nil /* bytecount */) - writeBuf1.writeTextDatum(context.Background(), ary, defaultConv, nil /* t */) + writeBuf1.writeTextDatum(context.Background(), ary, defaultConv, defaultLoc, nil /* t */) writeBuf1.writeBinaryDatum(context.Background(), ary, time.UTC, nil /* t */) writeBuf2 := newWriteBuffer(nil /* bytecount */) - writeBuf2.writeTextDatum(context.Background(), ary, defaultConv, nil /* t */) + writeBuf2.writeTextDatum(context.Background(), ary, defaultConv, defaultLoc, nil /* t */) writeBuf3 := newWriteBuffer(nil /* bytecount */) - writeBuf3.writeBinaryDatum(context.Background(), ary, defaultConv.Location, nil /* t */) + writeBuf3.writeBinaryDatum(context.Background(), ary, defaultLoc, nil /* t */) concatted := bytes.Join([][]byte{writeBuf2.wrapped.Bytes(), writeBuf3.wrapped.Bytes()}, nil) @@ -131,8 +131,8 @@ func TestIntArrayRoundTrip(t *testing.T) { } } - defaultConv := makeTestingConvCfg() - buf.writeTextDatum(context.Background(), d, defaultConv, nil /* t */) + defaultConv, defaultLoc := makeTestingConvCfg() + buf.writeTextDatum(context.Background(), d, defaultConv, defaultLoc, nil /* t */) b := buf.wrapped.Bytes() @@ -172,11 +172,11 @@ func TestFloatConversion(t *testing.T) { buf := newWriteBuffer(nil /* bytecount */) buf.bytecount = metric.NewCounter(metric.Metadata{}) - defaultConv := makeTestingConvCfg() - defaultConv.ExtraFloatDigits = test.extraFloatDigits + defaultConv, defaultLoc := makeTestingConvCfg() + defaultConv.ExtraFloatDigits = int32(test.extraFloatDigits) d := tree.NewDFloat(tree.DFloat(test.val)) - buf.writeTextDatum(context.Background(), d, defaultConv, nil /* t */) + buf.writeTextDatum(context.Background(), d, defaultConv, defaultLoc, nil /* t */) b := buf.wrapped.Bytes() got := string(b[4:]) @@ -199,8 +199,10 @@ func TestByteArrayRoundTrip(t *testing.T) { randValues = append(randValues, d) } - for _, be := range []lex.BytesEncodeFormat{ - lex.BytesEncodeHex, lex.BytesEncodeEscape} { + for _, be := range []sessiondatapb.BytesEncodeFormat{ + sessiondatapb.BytesEncodeHex, + sessiondatapb.BytesEncodeEscape, + } { t.Run(be.String(), func(t *testing.T) { for i, d := range randValues { t.Run(fmt.Sprintf("%d", i), func(t *testing.T) { @@ -209,9 +211,9 @@ func TestByteArrayRoundTrip(t *testing.T) { buf := newWriteBuffer(nil /* bytecount */) buf.bytecount = metric.NewCounter(metric.Metadata{}) - defaultConv := makeTestingConvCfg() + defaultConv, defaultLoc := makeTestingConvCfg() defaultConv.BytesEncodeFormat = be - buf.writeTextDatum(context.Background(), d, defaultConv, nil /* t */) + buf.writeTextDatum(context.Background(), d, defaultConv, defaultLoc, nil /* t */) b := buf.wrapped.Bytes() t.Logf("encoded: %v (%q)", b, b) @@ -239,7 +241,7 @@ func TestCanWriteAllDatums(t *testing.T) { rng := rand.New(rand.NewSource(timeutil.Now().Unix())) - defaultConv := makeTestingConvCfg() + defaultConv, defaultLoc := makeTestingConvCfg() for _, typ := range types.Scalar { buf := newWriteBuffer(nil /* bytecount */) @@ -247,12 +249,12 @@ func TestCanWriteAllDatums(t *testing.T) { for i := 0; i < 10; i++ { d := rowenc.RandDatum(rng, typ, true) - buf.writeTextDatum(context.Background(), d, defaultConv, typ) + buf.writeTextDatum(context.Background(), d, defaultConv, defaultLoc, typ) if buf.err != nil { t.Fatalf("got %s while attempting to write datum %s as text", buf.err, d) } - buf.writeBinaryDatum(context.Background(), d, defaultConv.Location, d.ResolvedType()) + buf.writeBinaryDatum(context.Background(), d, defaultLoc, d.ResolvedType()) if buf.err != nil { t.Fatalf("got %s while attempting to write datum %s as binary", buf.err, d) } @@ -267,9 +269,8 @@ func benchmarkWriteType(b *testing.B, d tree.Datum, format pgwirebase.FormatCode buf.bytecount = metric.NewCounter(metric.Metadata{Name: ""}) writeMethod := func(ctx context.Context, d tree.Datum, loc *time.Location) { - defaultConv := makeTestingConvCfg() - defaultConv.Location = loc - buf.writeTextDatum(ctx, d, defaultConv, d.ResolvedType()) + defaultConv, _ := makeTestingConvCfg() + buf.writeTextDatum(ctx, d, defaultConv, loc, d.ResolvedType()) } if format == pgwirebase.FormatBinary { writeMethod = func(ctx context.Context, d tree.Datum, loc *time.Location) { diff --git a/pkg/sql/planner.go b/pkg/sql/planner.go index 81a2738c661e..f8d37c4e392e 100644 --- a/pkg/sql/planner.go +++ b/pkg/sql/planner.go @@ -32,6 +32,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/sem/transform" "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/types" "github.com/cockroachdb/cockroach/pkg/util/cancelchecker" "github.com/cockroachdb/cockroach/pkg/util/envutil" @@ -262,13 +263,13 @@ func newInternalPlanner( ctx := logtags.AddTag(context.Background(), opName, "") sd := &sessiondata.SessionData{ + SessionData: sessiondatapb.SessionData{ + Database: "system", + User: user, + }, SearchPath: sessiondata.DefaultSearchPathForUser(user), - User: user, - Database: "system", SequenceState: sessiondata.NewSequenceState(), - DataConversion: sessiondata.DataConversionConfig{ - Location: time.UTC, - }, + Location: time.UTC, } // The table collection used by the internal planner does not rely on the // deprecatedDatabaseCache and there are no subscribers to the diff --git a/pkg/sql/schema_changer.go b/pkg/sql/schema_changer.go index c072aab13bd2..47c8e4532eef 100644 --- a/pkg/sql/schema_changer.go +++ b/pkg/sql/schema_changer.go @@ -42,6 +42,7 @@ import ( "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/sessiondatapb" "github.com/cockroachdb/cockroach/pkg/sql/sqlerrors" "github.com/cockroachdb/cockroach/pkg/sql/sqltelemetry" "github.com/cockroachdb/cockroach/pkg/sql/sqlutil" @@ -2018,21 +2019,21 @@ func createSchemaChangeEvalCtx( func newFakeSessionData() *sessiondata.SessionData { sd := &sessiondata.SessionData{ - SearchPath: sessiondata.DefaultSearchPathForUser(security.NodeUser), - // The database is not supposed to be needed in schema changes, as there - // shouldn't be unqualified identifiers in backfills, and the pure functions - // that need it should have already been evaluated. - // - // TODO(andrei): find a way to assert that this field is indeed not used. - // And in fact it is used by `current_schemas()`, which, although is a pure - // function, takes arguments which might be impure (so it can't always be - // pre-evaluated). - Database: "", - SequenceState: sessiondata.NewSequenceState(), - DataConversion: sessiondata.DataConversionConfig{ - Location: time.UTC, + SessionData: sessiondatapb.SessionData{ + // The database is not supposed to be needed in schema changes, as there + // shouldn't be unqualified identifiers in backfills, and the pure functions + // that need it should have already been evaluated. + // + // TODO(andrei): find a way to assert that this field is indeed not used. + // And in fact it is used by `current_schemas()`, which, although is a pure + // function, takes arguments which might be impure (so it can't always be + // pre-evaluated). + Database: "", + User: security.NodeUser, }, - User: security.NodeUser, + SearchPath: sessiondata.DefaultSearchPathForUser(security.NodeUser), + SequenceState: sessiondata.NewSequenceState(), + Location: time.UTC, } return sd } diff --git a/pkg/sql/sem/builtins/builtins.go b/pkg/sql/sem/builtins/builtins.go index c7b0aee283e9..afa9f11ba50a 100644 --- a/pkg/sql/sem/builtins/builtins.go +++ b/pkg/sql/sem/builtins/builtins.go @@ -52,6 +52,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/rowenc" "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/sqlliveness" "github.com/cockroachdb/cockroach/pkg/sql/sqltelemetry" "github.com/cockroachdb/cockroach/pkg/sql/types" @@ -923,7 +924,7 @@ var builtins = map[string]builtinDefinition{ ReturnType: tree.FixedReturnType(types.String), Fn: func(evalCtx *tree.EvalContext, args tree.Datums) (_ tree.Datum, err error) { data, format := *args[0].(*tree.DBytes), string(tree.MustBeDString(args[1])) - be, ok := lex.BytesEncodeFormatFromString(format) + be, ok := sessiondatapb.BytesEncodeFormatFromString(format) if !ok { return nil, pgerror.New(pgcode.InvalidParameterValue, "only 'hex', 'escape', and 'base64' formats are supported for encode()") @@ -942,7 +943,7 @@ var builtins = map[string]builtinDefinition{ ReturnType: tree.FixedReturnType(types.Bytes), Fn: func(evalCtx *tree.EvalContext, args tree.Datums) (_ tree.Datum, err error) { data, format := string(tree.MustBeDString(args[0])), string(tree.MustBeDString(args[1])) - be, ok := lex.BytesEncodeFormatFromString(format) + be, ok := sessiondatapb.BytesEncodeFormatFromString(format) if !ok { return nil, pgerror.New(pgcode.InvalidParameterValue, "only 'hex', 'escape', and 'base64' formats are supported for decode()") diff --git a/pkg/sql/sem/tree/casts.go b/pkg/sql/sem/tree/casts.go index 669a08bd2a7e..416622ab674e 100644 --- a/pkg/sql/sem/tree/casts.go +++ b/pkg/sql/sem/tree/casts.go @@ -616,7 +616,7 @@ func PerformCast(ctx *EvalContext, d Datum, t *types.T) (Datum, error) { s = t.BitArray.String() case *DFloat: s = strconv.FormatFloat(float64(*t), 'g', - ctx.SessionData.DataConversion.GetFloatPrec(), 64) + ctx.SessionData.DataConversionConfig.GetFloatPrec(), 64) case *DBool, *DInt, *DDecimal: s = d.String() case *DTimestamp, *DDate, *DTime, *DTimeTZ, *DGeography, *DGeometry, *DBox2D: @@ -649,8 +649,11 @@ func PerformCast(ctx *EvalContext, d Datum, t *types.T) (Datum, error) { case *DCollatedString: s = t.Contents case *DBytes: - s = lex.EncodeByteArrayToRawBytes(string(*t), - ctx.SessionData.DataConversion.BytesEncodeFormat, false /* skipHexPrefix */) + s = lex.EncodeByteArrayToRawBytes( + string(*t), + ctx.SessionData.DataConversionConfig.BytesEncodeFormat, + false, /* skipHexPrefix */ + ) case *DOid: s = t.String() case *DJSON: diff --git a/pkg/sql/sem/tree/datum_invariants_test.go b/pkg/sql/sem/tree/datum_invariants_test.go index f7546e71d181..35fc83056a88 100644 --- a/pkg/sql/sem/tree/datum_invariants_test.go +++ b/pkg/sql/sem/tree/datum_invariants_test.go @@ -149,9 +149,7 @@ func TestCompareTimestamps(t *testing.T) { func(t *testing.T) { ctx := &EvalContext{ SessionData: &sessiondata.SessionData{ - DataConversion: sessiondata.DataConversionConfig{ - Location: tc.location, - }, + Location: tc.location, }, } assert.Equal(t, tc.expected, compareTimestamps(ctx, tc.left, tc.right)) diff --git a/pkg/sql/sem/tree/datum_test.go b/pkg/sql/sem/tree/datum_test.go index 5f1d7f7cb86d..c3c4a8b2868d 100644 --- a/pkg/sql/sem/tree/datum_test.go +++ b/pkg/sql/sem/tree/datum_test.go @@ -828,9 +828,7 @@ func TestDTimeTZ(t *testing.T) { ctx := &tree.EvalContext{ SessionData: &sessiondata.SessionData{ - DataConversion: sessiondata.DataConversionConfig{ - Location: time.UTC, - }, + Location: time.UTC, }, } diff --git a/pkg/sql/sem/tree/eval.go b/pkg/sql/sem/tree/eval.go index 12a17f94e4c5..9984a569db57 100644 --- a/pkg/sql/sem/tree/eval.go +++ b/pkg/sql/sem/tree/eval.go @@ -34,6 +34,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgnotice" "github.com/cockroachdb/cockroach/pkg/sql/roleoption" "github.com/cockroachdb/cockroach/pkg/sql/sessiondata" + "github.com/cockroachdb/cockroach/pkg/sql/sessiondatapb" "github.com/cockroachdb/cockroach/pkg/sql/sqlliveness" "github.com/cockroachdb/cockroach/pkg/sql/sqltelemetry" "github.com/cockroachdb/cockroach/pkg/sql/types" @@ -3294,11 +3295,13 @@ func MakeTestingEvalContext(st *cluster.Settings) EvalContext { // EvalContext so do not start or close the memory monitor. func MakeTestingEvalContextWithMon(st *cluster.Settings, monitor *mon.BytesMonitor) EvalContext { ctx := EvalContext{ - Codec: keys.SystemSQLCodec, - Txn: &kv.Txn{}, - SessionData: &sessiondata.SessionData{VectorizeMode: sessiondata.VectorizeOn}, - Settings: st, - NodeID: base.TestingIDContainer, + Codec: keys.SystemSQLCodec, + Txn: &kv.Txn{}, + SessionData: &sessiondata.SessionData{SessionData: sessiondatapb.SessionData{ + VectorizeMode: sessiondatapb.VectorizeOn, + }}, + Settings: st, + NodeID: base.TestingIDContainer, } monitor.Start(context.Background(), nil /* pool */, mon.MakeStandaloneBudget(math.MaxInt64)) ctx.Mon = monitor @@ -3494,10 +3497,10 @@ func (ctx *EvalContext) SetStmtTimestamp(ts time.Time) { // GetLocation returns the session timezone. func (ctx *EvalContext) GetLocation() *time.Location { - if ctx.SessionData == nil || ctx.SessionData.DataConversion.Location == nil { + if ctx.SessionData == nil || ctx.SessionData.Location == nil { return time.UTC } - return ctx.SessionData.DataConversion.Location + return ctx.SessionData.Location } // Ctx returns the session's context. diff --git a/pkg/sql/sessiondata/session_data.go b/pkg/sql/sessiondata/session_data.go index a65cdd16b93c..123934dafa30 100644 --- a/pkg/sql/sessiondata/session_data.go +++ b/pkg/sql/sessiondata/session_data.go @@ -16,20 +16,14 @@ import ( "strings" "time" - "github.com/cockroachdb/cockroach/pkg/sql/lex" "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgnotice" + "github.com/cockroachdb/cockroach/pkg/sql/sessiondatapb" ) // SessionData contains session parameters. They are all user-configurable. // A SQL Session changes fields in SessionData through sql.sessionDataMutator. type SessionData struct { - // ApplicationName is the name of the application running the - // current session. This can be used for logging and per-application - // statistics. - ApplicationName string - // Database indicates the "current" database for the purpose of - // resolving names. See searchAndQualifyDatabase() for details. - Database string + sessiondatapb.SessionData // DefaultTxnPriority indicates the default priority of newly created // transactions. // NOTE: we'd prefer to use tree.UserPriority here, but doing so would @@ -77,8 +71,6 @@ type SessionData struct { // idle in a transaction before the session is canceled. // If set to 0, there is no timeout. IdleInTransactionSessionTimeout time.Duration - // User is the name of the user logged into the session. - User string // SafeUpdates causes errors when the client // sends syntax that may have unwanted side effects. SafeUpdates bool @@ -99,28 +91,17 @@ type SessionData struct { // SequenceState gives access to the SQL sequences that have been manipulated // by the session. SequenceState *SequenceState - // DataConversion gives access to the data conversion configuration. - DataConversion DataConversionConfig - // VectorizeMode indicates which kinds of queries to use vectorized execution - // engine for. - VectorizeMode VectorizeExecMode + // Location indicates the current time zone. + Location *time.Location // VectorizeRowCountThreshold indicates the row count above which the // vectorized execution engine will be used if possible. VectorizeRowCountThreshold uint64 - // TestingVectorizeInjectPanics indicates whether random panics are - // injected into the vectorized flow execution. The goal of such behavior - // is making sure that errors that are propagated as panics in the - // vectorized engine are caught in all scenarios. - TestingVectorizeInjectPanics bool // ForceSavepointRestart overrides the default SAVEPOINT behavior // for compatibility with certain ORMs. When this flag is set, // the savepoint name will no longer be compared against the magic // identifier `cockroach_restart` in order use a restartable // transaction. ForceSavepointRestart bool - // DefaultIntSize specifies the size in bits or bytes (preferred) - // of how a "naked" INT type should be parsed. - DefaultIntSize int // ResultsBufferSize specifies the size at which the pgwire results buffer // will self-flush. ResultsBufferSize int64 @@ -150,11 +131,15 @@ type SessionData struct { // AlterColumnTypeGeneralEnabled is true if ALTER TABLE ... ALTER COLUMN ... // TYPE x may be used for general conversions requiring online schema change/ AlterColumnTypeGeneralEnabled bool - // SynchronousCommit is a dummy setting for the synchronous_commit var. SynchronousCommit bool // EnableSeqScan is a dummy setting for the enable_seqscan var. EnableSeqScan bool + /////////////////////////////////////////////////////////////////////////// + // WARNING: consider whether a session parameter you're adding needs to // + // be propagated to the remote nodes. If so, consider adding it to // + // sessiondatapb.SessionData (preferred) or execinfrapb.EvalContext. // + /////////////////////////////////////////////////////////////////////////// } // IsTemporarySchemaID returns true if the given ID refers to any of the temp @@ -176,56 +161,6 @@ func (s *SessionData) GetTemporarySchemaIDForDb(dbID uint32) (uint32, bool) { return schemaID, found } -// DataConversionConfig contains the parameters that influence -// the conversion between SQL data types and strings/byte arrays. -type DataConversionConfig struct { - // Location indicates the current time zone. - Location *time.Location - - // BytesEncodeFormat indicates how to encode byte arrays when converting - // to string. - BytesEncodeFormat lex.BytesEncodeFormat - - // ExtraFloatDigits indicates the number of digits beyond the - // standard number to use for float conversions. - // This must be set to a value between -15 and 3, inclusive. - ExtraFloatDigits int -} - -// GetFloatPrec computes a precision suitable for a call to -// strconv.FormatFloat() or for use with '%.*g' in a printf-like -// function. -func (c *DataConversionConfig) GetFloatPrec() int { - // The user-settable parameter ExtraFloatDigits indicates the number - // of digits to be used to format the float value. PostgreSQL - // combines this with %g. - // The formula is _DIG + extra_float_digits, - // where is either FLT (float4) or DBL (float8). - - // Also the value "3" in PostgreSQL is special and meant to mean - // "all the precision needed to reproduce the float exactly". The Go - // formatter uses the special value -1 for this and activates a - // separate path in the formatter. We compare >= 3 here - // just in case the value is not gated properly in the implementation - // of SET. - if c.ExtraFloatDigits >= 3 { - return -1 - } - - // CockroachDB only implements float8 at this time and Go does not - // expose DBL_DIG, so we use the standard literal constant for - // 64bit floats. - const StdDoubleDigits = 15 - - nDigits := StdDoubleDigits + c.ExtraFloatDigits - if nDigits < 1 { - // Ensure the value is clamped at 1: printf %g does not allow - // values lower than 1. PostgreSQL does this too. - nDigits = 1 - } - return nDigits -} - // ExperimentalDistSQLPlanningMode controls if and when the opt-driven DistSQL // planning is used to create physical plans. type ExperimentalDistSQLPlanningMode int64 @@ -323,66 +258,6 @@ func DistSQLExecModeFromString(val string) (_ DistSQLExecMode, ok bool) { } } -// VectorizeExecMode controls if an when the Executor executes queries using the -// columnar execution engine. -// WARNING: When adding a VectorizeExecMode, note that nodes at previous -// versions might interpret the integer value differently. To avoid this, only -// append to the list or bump the minimum required distsql version (maybe also -// take advantage of that to reorder the list as you see fit). -type VectorizeExecMode int64 - -const ( - // VectorizeOff means that columnar execution is disabled. - VectorizeOff VectorizeExecMode = iota - // Vectorize201Auto means that that any supported queries that use only - // streaming operators (i.e. those that do not require any buffering) will - // be run using the columnar execution. If any part of a query is not - // supported by the vectorized execution engine, the whole query will fall - // back to row execution. - // This is the default setting in 20.1. - Vectorize201Auto - // VectorizeOn means that any supported queries will be run using the - // columnar execution. - VectorizeOn - // VectorizeExperimentalAlways means that we attempt to vectorize all - // queries; unsupported queries will fail. Mostly used for testing. - VectorizeExperimentalAlways -) - -func (m VectorizeExecMode) String() string { - switch m { - case VectorizeOff: - return "off" - case Vectorize201Auto: - return "201auto" - case VectorizeOn: - return "on" - case VectorizeExperimentalAlways: - return "experimental_always" - default: - return fmt.Sprintf("invalid (%d)", m) - } -} - -// 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 "OFF": - m = VectorizeOff - case "201AUTO": - m = Vectorize201Auto - case "ON": - m = VectorizeOn - case "EXPERIMENTAL_ALWAYS": - m = VectorizeExperimentalAlways - default: - return 0, false - } - return m, true -} - // SerialNormalizationMode controls if and when the Executor uses DistSQL. type SerialNormalizationMode int64 diff --git a/pkg/sql/sessiondatapb/session_data.go b/pkg/sql/sessiondatapb/session_data.go new file mode 100644 index 000000000000..b039d59efea1 --- /dev/null +++ b/pkg/sql/sessiondatapb/session_data.go @@ -0,0 +1,111 @@ +// Copyright 2020 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 ( + "fmt" + "strings" +) + +// GetFloatPrec computes a precision suitable for a call to +// strconv.FormatFloat() or for use with '%.*g' in a printf-like +// function. +func (c DataConversionConfig) GetFloatPrec() int { + // The user-settable parameter ExtraFloatDigits indicates the number + // of digits to be used to format the float value. PostgreSQL + // combines this with %g. + // The formula is _DIG + extra_float_digits, + // where is either FLT (float4) or DBL (float8). + + // Also the value "3" in PostgreSQL is special and meant to mean + // "all the precision needed to reproduce the float exactly". The Go + // formatter uses the special value -1 for this and activates a + // separate path in the formatter. We compare >= 3 here + // just in case the value is not gated properly in the implementation + // of SET. + if c.ExtraFloatDigits >= 3 { + return -1 + } + + // CockroachDB only implements float8 at this time and Go does not + // expose DBL_DIG, so we use the standard literal constant for + // 64bit floats. + const StdDoubleDigits = 15 + + nDigits := StdDoubleDigits + c.ExtraFloatDigits + if nDigits < 1 { + // Ensure the value is clamped at 1: printf %g does not allow + // values lower than 1. PostgreSQL does this too. + nDigits = 1 + } + return int(nDigits) +} + +func (f BytesEncodeFormat) String() string { + switch f { + case BytesEncodeHex: + return "hex" + case BytesEncodeEscape: + return "escape" + case BytesEncodeBase64: + return "base64" + default: + return fmt.Sprintf("invalid (%d)", f) + } +} + +// BytesEncodeFormatFromString converts a string into a BytesEncodeFormat. +func BytesEncodeFormatFromString(val string) (_ BytesEncodeFormat, ok bool) { + switch strings.ToUpper(val) { + case "HEX": + return BytesEncodeHex, true + case "ESCAPE": + return BytesEncodeEscape, true + case "BASE64": + return BytesEncodeBase64, true + default: + return -1, false + } +} + +func (m VectorizeExecMode) String() string { + switch m { + case VectorizeOff: + return "off" + case Vectorize201Auto: + return "201auto" + case VectorizeOn: + return "on" + case VectorizeExperimentalAlways: + return "experimental_always" + default: + return fmt.Sprintf("invalid (%d)", m) + } +} + +// 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 "OFF": + m = VectorizeOff + case "201AUTO": + m = Vectorize201Auto + case "ON": + m = VectorizeOn + case "EXPERIMENTAL_ALWAYS": + m = VectorizeExperimentalAlways + default: + return 0, false + } + return m, true +} diff --git a/pkg/sql/sessiondatapb/session_data.pb.go b/pkg/sql/sessiondatapb/session_data.pb.go new file mode 100644 index 000000000000..32d2c6852705 --- /dev/null +++ b/pkg/sql/sessiondatapb/session_data.pb.go @@ -0,0 +1,813 @@ +// Code generated by protoc-gen-gogo. DO NOT EDIT. +// source: sql/sessiondatapb/session_data.proto + +package sessiondatapb + +import proto "github.com/gogo/protobuf/proto" +import fmt "fmt" +import math "math" + +import io "io" + +// Reference imports to suppress errors if they are not otherwise used. +var _ = proto.Marshal +var _ = fmt.Errorf +var _ = math.Inf + +// This is a compile-time assertion to ensure that this generated file +// is compatible with the proto package it is being compiled against. +// A compilation error at this line likely means your copy of the +// proto package needs to be updated. +const _ = proto.GoGoProtoPackageIsVersion2 // please upgrade the proto package + +// BytesEncodeFormat is the configuration for bytes to string conversions. +type BytesEncodeFormat int32 + +const ( + // BytesEncodeHex uses the hex format: e'abc\n'::BYTES::STRING -> '\x61626312'. + // This is the default, for compatibility with PostgreSQL. + BytesEncodeHex BytesEncodeFormat = 0 + // BytesEncodeEscape uses the escaped format: e'abc\n'::BYTES::STRING -> 'abc\012'. + BytesEncodeEscape BytesEncodeFormat = 1 + // BytesEncodeBase64 uses base64 encoding. + BytesEncodeBase64 BytesEncodeFormat = 2 +) + +var BytesEncodeFormat_name = map[int32]string{ + 0: "BytesEncodeHex", + 1: "BytesEncodeEscape", + 2: "BytesEncodeBase64", +} +var BytesEncodeFormat_value = map[string]int32{ + "BytesEncodeHex": 0, + "BytesEncodeEscape": 1, + "BytesEncodeBase64": 2, +} + +func (x BytesEncodeFormat) Enum() *BytesEncodeFormat { + p := new(BytesEncodeFormat) + *p = x + return p +} +func (x BytesEncodeFormat) MarshalJSON() ([]byte, error) { + return proto.MarshalJSONEnum(BytesEncodeFormat_name, int32(x)) +} +func (x *BytesEncodeFormat) UnmarshalJSON(data []byte) error { + value, err := proto.UnmarshalJSONEnum(BytesEncodeFormat_value, data, "BytesEncodeFormat") + if err != nil { + return err + } + *x = BytesEncodeFormat(value) + return nil +} +func (BytesEncodeFormat) EnumDescriptor() ([]byte, []int) { + return fileDescriptor_session_data_38273df3056aef01, []int{0} +} + +// VectorizeExecMode controls if an when the Executor executes queries using +// the columnar execution engine. +type VectorizeExecMode int32 + +const ( + // VectorizeOff means that columnar execution is disabled. + VectorizeOff VectorizeExecMode = 0 + // Vectorize201Auto means that that any supported queries that use only + // streaming operators (i.e. those that do not require any buffering) will + // be run using the columnar execution. If any part of a query is not + // supported by the vectorized execution engine, the whole query will fall + // back to row execution. + // This is the default setting in 20.1. + Vectorize201Auto VectorizeExecMode = 1 + // VectorizeOn means that any supported queries will be run using the + // columnar execution. + VectorizeOn VectorizeExecMode = 2 + // VectorizeExperimentalAlways means that we attempt to vectorize all + // queries; unsupported queries will fail. Mostly used for testing. + VectorizeExperimentalAlways VectorizeExecMode = 3 +) + +var VectorizeExecMode_name = map[int32]string{ + 0: "VectorizeOff", + 1: "Vectorize201Auto", + 2: "VectorizeOn", + 3: "VectorizeExperimentalAlways", +} +var VectorizeExecMode_value = map[string]int32{ + "VectorizeOff": 0, + "Vectorize201Auto": 1, + "VectorizeOn": 2, + "VectorizeExperimentalAlways": 3, +} + +func (x VectorizeExecMode) Enum() *VectorizeExecMode { + p := new(VectorizeExecMode) + *p = x + return p +} +func (x VectorizeExecMode) MarshalJSON() ([]byte, error) { + return proto.MarshalJSONEnum(VectorizeExecMode_name, int32(x)) +} +func (x *VectorizeExecMode) UnmarshalJSON(data []byte) error { + value, err := proto.UnmarshalJSONEnum(VectorizeExecMode_value, data, "VectorizeExecMode") + if err != nil { + return err + } + *x = VectorizeExecMode(value) + return nil +} +func (VectorizeExecMode) EnumDescriptor() ([]byte, []int) { + return fileDescriptor_session_data_38273df3056aef01, []int{1} +} + +// SessionData contains session parameters that are easily serializable and are +// required to be propagated to the remote nodes for the correct execution of +// DistSQL flows. +type SessionData struct { + // Database indicates the "current" database for the purpose of resolving + // names. + Database string `protobuf:"bytes,1,opt,name=database" json:"database"` + // ApplicationName is the name of the application running the current + // session. This can be used for logging and per-application statistics. + ApplicationName string `protobuf:"bytes,2,opt,name=application_name,json=applicationName" json:"application_name"` + // User is the name of the user logged into the session. + User string `protobuf:"bytes,3,opt,name=user" json:"user"` + // DataConversion gives access to the data conversion configuration. + DataConversionConfig DataConversionConfig `protobuf:"bytes,4,opt,name=data_conversion_config,json=dataConversionConfig" json:"data_conversion_config"` + // VectorizeMode indicates which kinds of queries to use vectorized execution + // engine for. + VectorizeMode VectorizeExecMode `protobuf:"varint,5,opt,name=vectorize_mode,json=vectorizeMode,enum=cockroach.sql.sessiondatapb.VectorizeExecMode" json:"vectorize_mode"` + // TestingVectorizeInjectPanics indicates whether random panics are injected + // into the vectorized flow execution. The goal of such behavior is making + // sure that errors that are propagated as panics in the vectorized engine + // are caught in all scenarios. + TestingVectorizeInjectPanics bool `protobuf:"varint,6,opt,name=testing_vectorize_inject_panics,json=testingVectorizeInjectPanics" json:"testing_vectorize_inject_panics"` + // DefaultIntSize specifies the size in bits or bytes (preferred) of how a + // "naked" INT type should be parsed. + DefaultIntSize int32 `protobuf:"varint,7,opt,name=default_int_size,json=defaultIntSize" json:"default_int_size"` +} + +func (m *SessionData) Reset() { *m = SessionData{} } +func (m *SessionData) String() string { return proto.CompactTextString(m) } +func (*SessionData) ProtoMessage() {} +func (*SessionData) Descriptor() ([]byte, []int) { + return fileDescriptor_session_data_38273df3056aef01, []int{0} +} +func (m *SessionData) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *SessionData) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *SessionData) XXX_Merge(src proto.Message) { + xxx_messageInfo_SessionData.Merge(dst, src) +} +func (m *SessionData) XXX_Size() int { + return m.Size() +} +func (m *SessionData) XXX_DiscardUnknown() { + xxx_messageInfo_SessionData.DiscardUnknown(m) +} + +var xxx_messageInfo_SessionData proto.InternalMessageInfo + +// DataConversionConfig contains the parameters that influence the conversion +// between SQL data types and strings/byte arrays. +type DataConversionConfig struct { + // BytesEncodeFormat indicates how to encode byte arrays when converting to + // string. + BytesEncodeFormat BytesEncodeFormat `protobuf:"varint,1,opt,name=bytes_encode_format,json=bytesEncodeFormat,enum=cockroach.sql.sessiondatapb.BytesEncodeFormat" json:"bytes_encode_format"` + // ExtraFloatDigits indicates the number of digits beyond the standard number + // to use for float conversions.This must be set to a value between -15 and + // 3, inclusive. + ExtraFloatDigits int32 `protobuf:"varint,2,opt,name=extra_float_digits,json=extraFloatDigits" json:"extra_float_digits"` +} + +func (m *DataConversionConfig) Reset() { *m = DataConversionConfig{} } +func (m *DataConversionConfig) String() string { return proto.CompactTextString(m) } +func (*DataConversionConfig) ProtoMessage() {} +func (*DataConversionConfig) Descriptor() ([]byte, []int) { + return fileDescriptor_session_data_38273df3056aef01, []int{1} +} +func (m *DataConversionConfig) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *DataConversionConfig) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *DataConversionConfig) XXX_Merge(src proto.Message) { + xxx_messageInfo_DataConversionConfig.Merge(dst, src) +} +func (m *DataConversionConfig) XXX_Size() int { + return m.Size() +} +func (m *DataConversionConfig) XXX_DiscardUnknown() { + xxx_messageInfo_DataConversionConfig.DiscardUnknown(m) +} + +var xxx_messageInfo_DataConversionConfig proto.InternalMessageInfo + +func init() { + proto.RegisterType((*SessionData)(nil), "cockroach.sql.sessiondatapb.SessionData") + proto.RegisterType((*DataConversionConfig)(nil), "cockroach.sql.sessiondatapb.DataConversionConfig") + proto.RegisterEnum("cockroach.sql.sessiondatapb.BytesEncodeFormat", BytesEncodeFormat_name, BytesEncodeFormat_value) + proto.RegisterEnum("cockroach.sql.sessiondatapb.VectorizeExecMode", VectorizeExecMode_name, VectorizeExecMode_value) +} +func (m *SessionData) Marshal() (dAtA []byte, err error) { + size := m.Size() + dAtA = make([]byte, size) + n, err := m.MarshalTo(dAtA) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *SessionData) MarshalTo(dAtA []byte) (int, error) { + var i int + _ = i + var l int + _ = l + dAtA[i] = 0xa + i++ + i = encodeVarintSessionData(dAtA, i, uint64(len(m.Database))) + i += copy(dAtA[i:], m.Database) + dAtA[i] = 0x12 + i++ + i = encodeVarintSessionData(dAtA, i, uint64(len(m.ApplicationName))) + i += copy(dAtA[i:], m.ApplicationName) + dAtA[i] = 0x1a + i++ + i = encodeVarintSessionData(dAtA, i, uint64(len(m.User))) + i += copy(dAtA[i:], m.User) + dAtA[i] = 0x22 + i++ + i = encodeVarintSessionData(dAtA, i, uint64(m.DataConversionConfig.Size())) + n1, err := m.DataConversionConfig.MarshalTo(dAtA[i:]) + if err != nil { + return 0, err + } + i += n1 + dAtA[i] = 0x28 + i++ + i = encodeVarintSessionData(dAtA, i, uint64(m.VectorizeMode)) + dAtA[i] = 0x30 + i++ + if m.TestingVectorizeInjectPanics { + dAtA[i] = 1 + } else { + dAtA[i] = 0 + } + i++ + dAtA[i] = 0x38 + i++ + i = encodeVarintSessionData(dAtA, i, uint64(m.DefaultIntSize)) + return i, nil +} + +func (m *DataConversionConfig) Marshal() (dAtA []byte, err error) { + size := m.Size() + dAtA = make([]byte, size) + n, err := m.MarshalTo(dAtA) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *DataConversionConfig) MarshalTo(dAtA []byte) (int, error) { + var i int + _ = i + var l int + _ = l + dAtA[i] = 0x8 + i++ + i = encodeVarintSessionData(dAtA, i, uint64(m.BytesEncodeFormat)) + dAtA[i] = 0x10 + i++ + i = encodeVarintSessionData(dAtA, i, uint64(m.ExtraFloatDigits)) + return i, nil +} + +func encodeVarintSessionData(dAtA []byte, offset int, v uint64) int { + for v >= 1<<7 { + dAtA[offset] = uint8(v&0x7f | 0x80) + v >>= 7 + offset++ + } + dAtA[offset] = uint8(v) + return offset + 1 +} +func (m *SessionData) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + l = len(m.Database) + n += 1 + l + sovSessionData(uint64(l)) + l = len(m.ApplicationName) + n += 1 + l + sovSessionData(uint64(l)) + l = len(m.User) + n += 1 + l + sovSessionData(uint64(l)) + l = m.DataConversionConfig.Size() + n += 1 + l + sovSessionData(uint64(l)) + n += 1 + sovSessionData(uint64(m.VectorizeMode)) + n += 2 + n += 1 + sovSessionData(uint64(m.DefaultIntSize)) + return n +} + +func (m *DataConversionConfig) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + n += 1 + sovSessionData(uint64(m.BytesEncodeFormat)) + n += 1 + sovSessionData(uint64(m.ExtraFloatDigits)) + return n +} + +func sovSessionData(x uint64) (n int) { + for { + n++ + x >>= 7 + if x == 0 { + break + } + } + return n +} +func sozSessionData(x uint64) (n int) { + return sovSessionData(uint64((x << 1) ^ uint64((int64(x) >> 63)))) +} +func (m *SessionData) Unmarshal(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowSessionData + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= (uint64(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: SessionData: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: SessionData: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Database", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowSessionData + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= (uint64(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return ErrInvalidLengthSessionData + } + postIndex := iNdEx + intStringLen + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Database = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field ApplicationName", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowSessionData + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= (uint64(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return ErrInvalidLengthSessionData + } + postIndex := iNdEx + intStringLen + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.ApplicationName = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 3: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field User", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowSessionData + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= (uint64(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return ErrInvalidLengthSessionData + } + postIndex := iNdEx + intStringLen + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.User = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 4: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field DataConversionConfig", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowSessionData + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthSessionData + } + postIndex := iNdEx + msglen + if postIndex > l { + return io.ErrUnexpectedEOF + } + if err := m.DataConversionConfig.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 5: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field VectorizeMode", wireType) + } + m.VectorizeMode = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowSessionData + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.VectorizeMode |= (VectorizeExecMode(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + case 6: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field TestingVectorizeInjectPanics", wireType) + } + var v int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowSessionData + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + v |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + m.TestingVectorizeInjectPanics = bool(v != 0) + case 7: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field DefaultIntSize", wireType) + } + m.DefaultIntSize = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowSessionData + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.DefaultIntSize |= (int32(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + default: + iNdEx = preIndex + skippy, err := skipSessionData(dAtA[iNdEx:]) + if err != nil { + return err + } + if skippy < 0 { + return ErrInvalidLengthSessionData + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *DataConversionConfig) Unmarshal(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowSessionData + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= (uint64(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: DataConversionConfig: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: DataConversionConfig: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field BytesEncodeFormat", wireType) + } + m.BytesEncodeFormat = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowSessionData + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.BytesEncodeFormat |= (BytesEncodeFormat(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + case 2: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field ExtraFloatDigits", wireType) + } + m.ExtraFloatDigits = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowSessionData + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.ExtraFloatDigits |= (int32(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + default: + iNdEx = preIndex + skippy, err := skipSessionData(dAtA[iNdEx:]) + if err != nil { + return err + } + if skippy < 0 { + return ErrInvalidLengthSessionData + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func skipSessionData(dAtA []byte) (n int, err error) { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return 0, ErrIntOverflowSessionData + } + if iNdEx >= l { + return 0, io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= (uint64(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + wireType := int(wire & 0x7) + switch wireType { + case 0: + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return 0, ErrIntOverflowSessionData + } + if iNdEx >= l { + return 0, io.ErrUnexpectedEOF + } + iNdEx++ + if dAtA[iNdEx-1] < 0x80 { + break + } + } + return iNdEx, nil + case 1: + iNdEx += 8 + return iNdEx, nil + case 2: + var length int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return 0, ErrIntOverflowSessionData + } + if iNdEx >= l { + return 0, io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + length |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + iNdEx += length + if length < 0 { + return 0, ErrInvalidLengthSessionData + } + return iNdEx, nil + case 3: + for { + var innerWire uint64 + var start int = iNdEx + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return 0, ErrIntOverflowSessionData + } + if iNdEx >= l { + return 0, io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + innerWire |= (uint64(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + innerWireType := int(innerWire & 0x7) + if innerWireType == 4 { + break + } + next, err := skipSessionData(dAtA[start:]) + if err != nil { + return 0, err + } + iNdEx = start + next + } + return iNdEx, nil + case 4: + return iNdEx, nil + case 5: + iNdEx += 4 + return iNdEx, nil + default: + return 0, fmt.Errorf("proto: illegal wireType %d", wireType) + } + } + panic("unreachable") +} + +var ( + ErrInvalidLengthSessionData = fmt.Errorf("proto: negative length found during unmarshaling") + ErrIntOverflowSessionData = fmt.Errorf("proto: integer overflow") +) + +func init() { + proto.RegisterFile("sql/sessiondatapb/session_data.proto", fileDescriptor_session_data_38273df3056aef01) +} + +var fileDescriptor_session_data_38273df3056aef01 = []byte{ + // 548 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x84, 0x92, 0xcb, 0x6e, 0xd4, 0x3c, + 0x1c, 0xc5, 0xe3, 0x5e, 0xbe, 0xaf, 0xb8, 0x74, 0xea, 0x9a, 0x01, 0x45, 0x2d, 0x4a, 0xa3, 0x8a, + 0xc5, 0xa8, 0x8b, 0x0c, 0x1d, 0x21, 0xf6, 0x9d, 0x5e, 0x44, 0x85, 0xb8, 0x88, 0x4a, 0x2c, 0x60, + 0x61, 0x79, 0x9c, 0x7f, 0x06, 0x43, 0x62, 0xa7, 0xb1, 0x7b, 0x7d, 0x02, 0x96, 0xbc, 0x03, 0x2c, + 0x58, 0xf2, 0x0e, 0x6c, 0xba, 0xec, 0xb2, 0x2b, 0x04, 0xd3, 0x17, 0x41, 0x71, 0xd3, 0x74, 0xa6, + 0xad, 0xca, 0x2e, 0x3e, 0xe7, 0x77, 0xfe, 0xc7, 0x97, 0xe0, 0x47, 0x66, 0x27, 0x6d, 0x1b, 0x30, + 0x46, 0x6a, 0x15, 0x73, 0xcb, 0xf3, 0xde, 0xc5, 0x8a, 0x95, 0xcb, 0x28, 0x2f, 0xb4, 0xd5, 0x74, + 0x41, 0x68, 0xf1, 0xa9, 0xd0, 0x5c, 0x7c, 0x88, 0xcc, 0x4e, 0x1a, 0x8d, 0xf0, 0xf3, 0xcd, 0xbe, + 0xee, 0x6b, 0xc7, 0xb5, 0xcb, 0xaf, 0xf3, 0xc8, 0xd2, 0xcf, 0x71, 0x3c, 0xbd, 0x7d, 0xce, 0xad, + 0x73, 0xcb, 0x69, 0x88, 0xa7, 0x4a, 0xbe, 0xc7, 0x0d, 0xf8, 0x28, 0x44, 0xad, 0x3b, 0xdd, 0x89, + 0xe3, 0x5f, 0x8b, 0xde, 0x9b, 0x5a, 0xa5, 0x6d, 0x4c, 0x78, 0x9e, 0xa7, 0x52, 0x70, 0x5b, 0xd6, + 0x2b, 0x9e, 0x81, 0x3f, 0x36, 0x44, 0xce, 0x0e, 0xb9, 0x2f, 0x79, 0x06, 0xd4, 0xc7, 0x13, 0xbb, + 0x06, 0x0a, 0x7f, 0x7c, 0x08, 0x72, 0x0a, 0xcd, 0xf0, 0x83, 0x72, 0x2c, 0x13, 0x5a, 0xed, 0x41, + 0xe1, 0x4e, 0x23, 0xb4, 0x4a, 0x64, 0xdf, 0x9f, 0x08, 0x51, 0x6b, 0xba, 0xb3, 0x12, 0xdd, 0x72, + 0xa0, 0xa8, 0xdc, 0xef, 0x5a, 0x9d, 0x5c, 0x73, 0xc1, 0x6a, 0x7c, 0x33, 0xbe, 0xc1, 0xa3, 0xef, + 0x71, 0x63, 0x0f, 0x84, 0xd5, 0x85, 0x3c, 0x02, 0x96, 0xe9, 0x18, 0xfc, 0xc9, 0x10, 0xb5, 0x1a, + 0x9d, 0xe8, 0xd6, 0x9a, 0xb7, 0x17, 0x91, 0x8d, 0x03, 0x10, 0x2f, 0x74, 0x0c, 0x55, 0xc7, 0x4c, + 0x3d, 0xab, 0x14, 0xe9, 0x73, 0xbc, 0x68, 0xc1, 0x58, 0xa9, 0xfa, 0xec, 0xb2, 0x44, 0xaa, 0x8f, + 0x20, 0x2c, 0xcb, 0xb9, 0x92, 0xc2, 0xf8, 0xff, 0x85, 0xa8, 0x35, 0x55, 0xa5, 0x1f, 0x56, 0x70, + 0x3d, 0x7d, 0xcb, 0xa1, 0xaf, 0x1d, 0x49, 0x23, 0x4c, 0x62, 0x48, 0xf8, 0x6e, 0x6a, 0x99, 0x54, + 0x96, 0x19, 0x79, 0x04, 0xfe, 0xff, 0x21, 0x6a, 0x4d, 0x56, 0xe9, 0x46, 0xe5, 0x6e, 0x29, 0xbb, + 0x2d, 0x8f, 0x60, 0xe9, 0x07, 0xc2, 0xcd, 0x9b, 0xae, 0x83, 0xc6, 0xf8, 0x5e, 0xef, 0xd0, 0x82, + 0x61, 0xa0, 0x84, 0x8e, 0x81, 0x25, 0xba, 0xc8, 0xb8, 0x75, 0x2f, 0xfb, 0xaf, 0x73, 0x77, 0xcb, + 0xdc, 0x86, 0x8b, 0x6d, 0xba, 0x54, 0xd5, 0x3d, 0xd7, 0xbb, 0x6a, 0xd0, 0x0e, 0xa6, 0x70, 0x60, + 0x0b, 0xce, 0x92, 0x54, 0x73, 0xcb, 0x62, 0xd9, 0x97, 0xd6, 0xb8, 0x9f, 0xe2, 0x62, 0xc3, 0xc4, + 0xf9, 0x9b, 0xa5, 0xbd, 0xee, 0xdc, 0x65, 0x86, 0xe7, 0xae, 0x35, 0x50, 0x8a, 0x1b, 0x43, 0xe2, + 0x33, 0x38, 0x20, 0x1e, 0xbd, 0x3f, 0x02, 0x6e, 0x18, 0xc1, 0x73, 0x20, 0xe8, 0x8a, 0xdc, 0xe5, + 0x06, 0x9e, 0x3e, 0x21, 0x63, 0xf3, 0x53, 0x9f, 0xbf, 0x06, 0xde, 0xf7, 0x6f, 0x81, 0xb7, 0xbc, + 0x8f, 0xe7, 0xae, 0x3d, 0x1d, 0x25, 0xf8, 0x6e, 0x2d, 0xbe, 0x4a, 0x12, 0xe2, 0xd1, 0x26, 0x26, + 0xb5, 0xd2, 0x79, 0xbc, 0xb2, 0xba, 0x6b, 0x35, 0x41, 0x74, 0x16, 0x4f, 0x5f, 0x72, 0x8a, 0x8c, + 0xd1, 0x45, 0xbc, 0x30, 0x34, 0x2d, 0x87, 0x42, 0x66, 0xa0, 0x2c, 0x4f, 0x57, 0xd3, 0x7d, 0x7e, + 0x68, 0xc8, 0xf8, 0x65, 0x71, 0xb7, 0x7d, 0xfc, 0x27, 0xf0, 0x8e, 0x07, 0x01, 0x3a, 0x19, 0x04, + 0xe8, 0x74, 0x10, 0xa0, 0xdf, 0x83, 0x00, 0x7d, 0x39, 0x0b, 0xbc, 0x93, 0xb3, 0xc0, 0x3b, 0x3d, + 0x0b, 0xbc, 0x77, 0x33, 0x23, 0x37, 0xfd, 0x37, 0x00, 0x00, 0xff, 0xff, 0x0f, 0xf2, 0xdf, 0x93, + 0xdd, 0x03, 0x00, 0x00, +} diff --git a/pkg/sql/sessiondatapb/session_data.proto b/pkg/sql/sessiondatapb/session_data.proto new file mode 100644 index 000000000000..f7832bc74826 --- /dev/null +++ b/pkg/sql/sessiondatapb/session_data.proto @@ -0,0 +1,91 @@ +// Copyright 2020 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. + +syntax = "proto2"; +package cockroach.sql.sessiondatapb; +option go_package = "sessiondatapb"; + +import "gogoproto/gogo.proto"; + +// SessionData contains session parameters that are easily serializable and are +// required to be propagated to the remote nodes for the correct execution of +// DistSQL flows. +message SessionData { + // Database indicates the "current" database for the purpose of resolving + // names. + optional string database = 1 [(gogoproto.nullable) = false]; + // ApplicationName is the name of the application running the current + // session. This can be used for logging and per-application statistics. + optional string application_name = 2 [(gogoproto.nullable) = false]; + // User is the name of the user logged into the session. + optional string user = 3 [(gogoproto.nullable) = false]; + // DataConversion gives access to the data conversion configuration. + optional DataConversionConfig data_conversion_config = 4 [(gogoproto.nullable) = false]; + // VectorizeMode indicates which kinds of queries to use vectorized execution + // engine for. + optional VectorizeExecMode vectorize_mode = 5 [(gogoproto.nullable) = false]; + // TestingVectorizeInjectPanics indicates whether random panics are injected + // into the vectorized flow execution. The goal of such behavior is making + // sure that errors that are propagated as panics in the vectorized engine + // are caught in all scenarios. + optional bool testing_vectorize_inject_panics = 6 [(gogoproto.nullable) = false]; + // DefaultIntSize specifies the size in bits or bytes (preferred) of how a + // "naked" INT type should be parsed. + optional int32 default_int_size = 7 [(gogoproto.nullable) = false]; +} + +// DataConversionConfig contains the parameters that influence the conversion +// between SQL data types and strings/byte arrays. +message DataConversionConfig { + // BytesEncodeFormat indicates how to encode byte arrays when converting to + // string. + optional BytesEncodeFormat bytes_encode_format = 1 [(gogoproto.nullable) = false]; + // ExtraFloatDigits indicates the number of digits beyond the standard number + // to use for float conversions.This must be set to a value between -15 and + // 3, inclusive. + optional int32 extra_float_digits = 2 [(gogoproto.nullable) = false]; +} + +// BytesEncodeFormat is the configuration for bytes to string conversions. +enum BytesEncodeFormat { + option (gogoproto.goproto_enum_prefix) = false; + option (gogoproto.goproto_enum_stringer) = false; + + // BytesEncodeHex uses the hex format: e'abc\n'::BYTES::STRING -> '\x61626312'. + // This is the default, for compatibility with PostgreSQL. + BytesEncodeHex = 0; + // BytesEncodeEscape uses the escaped format: e'abc\n'::BYTES::STRING -> 'abc\012'. + BytesEncodeEscape = 1; + // BytesEncodeBase64 uses base64 encoding. + BytesEncodeBase64 = 2; +} + +// VectorizeExecMode controls if an when the Executor executes queries using +// the columnar execution engine. +enum VectorizeExecMode { + option (gogoproto.goproto_enum_prefix) = false; + option (gogoproto.goproto_enum_stringer) = false; + + // VectorizeOff means that columnar execution is disabled. + VectorizeOff = 0; + // Vectorize201Auto means that that any supported queries that use only + // streaming operators (i.e. those that do not require any buffering) will + // be run using the columnar execution. If any part of a query is not + // supported by the vectorized execution engine, the whole query will fall + // back to row execution. + // This is the default setting in 20.1. + Vectorize201Auto = 1; + // VectorizeOn means that any supported queries will be run using the + // columnar execution. + VectorizeOn = 2; + // VectorizeExperimentalAlways means that we attempt to vectorize all + // queries; unsupported queries will fail. Mostly used for testing. + VectorizeExperimentalAlways = 3; +} diff --git a/pkg/sql/set_cluster_setting.go b/pkg/sql/set_cluster_setting.go index f7bfd98766db..78838aa001b3 100644 --- a/pkg/sql/set_cluster_setting.go +++ b/pkg/sql/set_cluster_setting.go @@ -28,6 +28,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/roleoption" "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/sqltelemetry" "github.com/cockroachdb/cockroach/pkg/sql/stats" "github.com/cockroachdb/cockroach/pkg/sql/types" @@ -275,7 +276,7 @@ func (n *setClusterSettingNode) startExec(params runParams) error { if err != nil { break } - validatedExecMode, isValid := sessiondata.VectorizeExecModeFromString(sessiondata.VectorizeExecMode(val).String()) + validatedExecMode, isValid := sessiondatapb.VectorizeExecModeFromString(sessiondatapb.VectorizeExecMode(val).String()) if !isValid { break } diff --git a/pkg/sql/vars.go b/pkg/sql/vars.go index 6fa7c15a8dfb..1694a0f74801 100644 --- a/pkg/sql/vars.go +++ b/pkg/sql/vars.go @@ -23,7 +23,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/server/telemetry" "github.com/cockroachdb/cockroach/pkg/settings" "github.com/cockroachdb/cockroach/pkg/sql/delegate" - "github.com/cockroachdb/cockroach/pkg/sql/lex" "github.com/cockroachdb/cockroach/pkg/sql/paramparse" "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgcode" "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror" @@ -31,6 +30,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/sem/builtins" "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/sqltelemetry" "github.com/cockroachdb/cockroach/pkg/util/errorutil/unimplemented" "github.com/cockroachdb/cockroach/pkg/util/timeutil" @@ -142,7 +142,7 @@ var varGen = map[string]sessionVar{ Set: func( _ context.Context, m *sessionDataMutator, s string, ) error { - mode, ok := lex.BytesEncodeFormatFromString(s) + mode, ok := sessiondatapb.BytesEncodeFormatFromString(s) if !ok { return newVarValueError(`bytea_output`, s, "hex", "escape", "base64") } @@ -150,9 +150,9 @@ var varGen = map[string]sessionVar{ return nil }, Get: func(evalCtx *extendedEvalContext) string { - return evalCtx.SessionData.DataConversion.BytesEncodeFormat.String() + return evalCtx.SessionData.DataConversionConfig.BytesEncodeFormat.String() }, - GlobalDefault: func(sv *settings.Values) string { return lex.BytesEncodeHex.String() }, + GlobalDefault: func(sv *settings.Values) string { return sessiondatapb.BytesEncodeHex.String() }, }, `client_min_messages`: { @@ -288,7 +288,7 @@ var varGen = map[string]sessionVar{ if i == 4 { telemetry.Inc(sqltelemetry.DefaultIntSize4Counter) } - m.SetDefaultIntSize(int(i)) + m.SetDefaultIntSize(int32(i)) return nil }, GlobalDefault: func(sv *settings.Values) string { @@ -490,7 +490,7 @@ var varGen = map[string]sessionVar{ // CockroachDB extension. `vectorize`: { Set: func(_ context.Context, m *sessionDataMutator, s string) error { - mode, ok := sessiondata.VectorizeExecModeFromString(s) + mode, ok := sessiondatapb.VectorizeExecModeFromString(s) if !ok { return newVarValueError(`vectorize`, s, "off", "201auto", "on", "experimental_always") @@ -502,7 +502,7 @@ var varGen = map[string]sessionVar{ return evalCtx.SessionData.VectorizeMode.String() }, GlobalDefault: func(sv *settings.Values) string { - return sessiondata.VectorizeExecMode( + return sessiondatapb.VectorizeExecMode( VectorizeClusterMode.Get(sv)).String() }, }, @@ -702,11 +702,11 @@ var varGen = map[string]sessionVar{ return pgerror.Newf(pgcode.InvalidParameterValue, `%d is outside the valid range for parameter "extra_float_digits" (-15 .. 3)`, i) } - m.SetExtraFloatDigits(int(i)) + m.SetExtraFloatDigits(int32(i)) return nil }, Get: func(evalCtx *extendedEvalContext) string { - return fmt.Sprintf("%d", evalCtx.SessionData.DataConversion.ExtraFloatDigits) + return fmt.Sprintf("%d", evalCtx.SessionData.DataConversionConfig.ExtraFloatDigits) }, GlobalDefault: func(sv *settings.Values) string { return "0" }, }, @@ -952,7 +952,7 @@ var varGen = map[string]sessionVar{ // See https://www.postgresql.org/docs/10/static/runtime-config-client.html#GUC-TIMEZONE `timezone`: { Get: func(evalCtx *extendedEvalContext) string { - return sessionDataTimeZoneFormat(evalCtx.SessionData.DataConversion.Location) + return sessionDataTimeZoneFormat(evalCtx.SessionData.Location) }, GetStringVal: timeZoneVarGetStringVal, Set: timeZoneVarSet, diff --git a/pkg/workload/dep_test.go b/pkg/workload/dep_test.go index 58523995ea6d..36fdf458a3cf 100644 --- a/pkg/workload/dep_test.go +++ b/pkg/workload/dep_test.go @@ -30,6 +30,7 @@ func TestDepAllowlist(t *testing.T) { `github.com/cockroachdb/cockroach/pkg/geo/geopb`, `github.com/cockroachdb/cockroach/pkg/sql/lex`, `github.com/cockroachdb/cockroach/pkg/sql/oidext`, + `github.com/cockroachdb/cockroach/pkg/sql/sessiondatapb`, `github.com/cockroachdb/cockroach/pkg/sql/types`, `github.com/cockroachdb/cockroach/pkg/util`, `github.com/cockroachdb/cockroach/pkg/util/arith`,