From 82c6ec089a144e1cc2176ccdc708a91dbc7f70a8 Mon Sep 17 00:00:00 2001 From: Andrei Matei Date: Fri, 15 Mar 2019 16:41:56 -0400 Subject: [PATCH] sql: create a span per sql statement Before this patch, statements did not have their own spans. Instead, transactions had spans. This dates from a time when the TxnCoordSender would capture the span of the first write and expect it to live for the duration of the transaction, but that stopped being the case a while ago. This patch makes each statement create its own span. Release note: None --- pkg/sql/conn_executor.go | 458 ++++++++++-------- pkg/sql/conn_io.go | 26 +- .../logictest/testdata/planner_test/select | 28 +- .../testdata/planner_test/show_trace | 90 ++-- .../logictest/testdata/planner_test/upsert | 32 +- pkg/sql/opt/exec/execbuilder/testdata/select | 32 +- .../opt/exec/execbuilder/testdata/show_trace | 90 ++-- pkg/sql/opt/exec/execbuilder/testdata/upsert | 32 +- pkg/sql/trace_test.go | 4 + 9 files changed, 424 insertions(+), 368 deletions(-) diff --git a/pkg/sql/conn_executor.go b/pkg/sql/conn_executor.go index e4f3c30969d8..943b85e33f09 100644 --- a/pkg/sql/conn_executor.go +++ b/pkg/sql/conn_executor.go @@ -50,6 +50,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/util/stop" "github.com/cockroachdb/cockroach/pkg/util/syncutil" "github.com/cockroachdb/cockroach/pkg/util/timeutil" + "github.com/cockroachdb/cockroach/pkg/util/tracing" "github.com/cockroachdb/cockroach/pkg/util/uuid" "github.com/pkg/errors" "golang.org/x/net/trace" @@ -945,6 +946,10 @@ type connExecutor struct { // activated determines whether activate() was called already. // When this is set, close() must be called to release resources. activated bool + + // draining is set if we've received a DrainRequest. Once this is set, we're + // going to find a suitable time to close the connection. + draining bool } // ctxHolder contains a connection's context and, while session tracing is @@ -1142,243 +1147,276 @@ func (ex *connExecutor) run( ex.server.cfg.SessionRegistry.register(ex.sessionID, ex) defer ex.server.cfg.SessionRegistry.deregister(ex.sessionID) - pinfo := &tree.PlaceholderInfo{} - - var draining bool for { ex.curStmt = nil if err := ctx.Err(); err != nil { return err } - cmd, pos, err := ex.stmtBuf.curCmd() - if err != nil { - if err == io.EOF { + var err error + if err = ex.execCmd(ex.Ctx()); err != nil { + if err == io.EOF || err == errDrainingComplete { return nil } return err } - if log.ExpensiveLogEnabled(ex.Ctx(), 2) || ex.eventLog != nil { - ex.sessionEventf(ex.Ctx(), "[%s pos:%d] executing %s", - ex.machine.CurState(), pos, cmd) - } + } +} - var ev fsm.Event - var payload fsm.EventPayload - var res ResultBase +// errDrainingComplete is returned by execCmd when the connExecutor previously got +// a DrainRequest and the time is ripe to finish this session (i.e. we're no +// longer in a transaction). +var errDrainingComplete = fmt.Errorf("draining done. this is a good time to finish this session") - switch tcmd := cmd.(type) { - case ExecStmt: - if tcmd.AST == nil { - res = ex.clientComm.CreateEmptyQueryResult(pos) - break - } - ex.curStmt = tcmd.AST - - stmtRes := ex.clientComm.CreateStatementResult( - tcmd.AST, NeedRowDesc, pos, nil, /* formatCodes */ - ex.sessionData.DataConversion) - res = stmtRes - curStmt := Statement{Statement: tcmd.Statement} - - ex.phaseTimes[sessionQueryReceived] = tcmd.TimeReceived - ex.phaseTimes[sessionStartParse] = tcmd.ParseStart - ex.phaseTimes[sessionEndParse] = tcmd.ParseEnd - - stmtCtx := withStatement(ex.Ctx(), ex.curStmt) - ev, payload, err = ex.execStmt(stmtCtx, curStmt, stmtRes, nil /* pinfo */) - if err != nil { - return err - } - case ExecPortal: - // ExecPortal is handled like ExecStmt, except that the placeholder info - // is taken from the portal. - - portal, ok := ex.extraTxnState.prepStmtsNamespace.portals[tcmd.Name] - if !ok { - err := pgerror.NewErrorf( - pgerror.CodeInvalidCursorNameError, "unknown portal %q", tcmd.Name) - ev = eventNonRetriableErr{IsCommit: fsm.False} - payload = eventNonRetriableErrPayload{err: err} - res = ex.clientComm.CreateErrorResult(pos) - break - } - if log.ExpensiveLogEnabled(ex.Ctx(), 2) { - log.VEventf(ex.Ctx(), 2, "portal resolved to: %s", portal.Stmt.AST.String()) - } - ex.curStmt = portal.Stmt.AST - - *pinfo = tree.PlaceholderInfo{ - PlaceholderTypesInfo: tree.PlaceholderTypesInfo{ - TypeHints: portal.Stmt.TypeHints, - Types: portal.Stmt.Types, - }, - Values: portal.Qargs, - } +// execCmd reads the current command from the stmtBuf and executes it. The +// transaction state is modified accordingly, and the stmtBuf is advanced or +// rewinded accordingly. +// +// Returns an error if communication of results to the client has failed and the +// session should be terminated. Returns io.EOF if the stmtBuf has been closed. +// Returns drainingComplete if the session should finish because draining is +// complete (i.e. we received a DrainRequest - possibly previously - and the +// connection is found to be idle). +func (ex *connExecutor) execCmd(ctx context.Context) error { + cmd, pos, err := ex.stmtBuf.curCmd() + if err != nil { + return err // err could be io.EOF + } - ex.phaseTimes[sessionQueryReceived] = tcmd.TimeReceived - // When parsing has been done earlier, via a separate parse - // message, it is not any more part of the statistics collected - // for this execution. In that case, we simply report that - // parsing took no time. - ex.phaseTimes[sessionStartParse] = time.Time{} - ex.phaseTimes[sessionEndParse] = time.Time{} - - if portal.Stmt.AST == nil { - res = ex.clientComm.CreateEmptyQueryResult(pos) - break - } + ctx, sp := tracing.EnsureChildSpan( + ctx, ex.server.cfg.AmbientCtx.Tracer, + // We print the type of command, not the String() which includes long + // statements. + "exec cmd: "+cmd.command()) + defer sp.Finish() - stmtRes := ex.clientComm.CreateStatementResult( - portal.Stmt.AST, - // The client is using the extended protocol, so no row description is - // needed. - DontNeedRowDesc, - pos, portal.OutFormats, - ex.sessionData.DataConversion) - stmtRes.SetLimit(tcmd.Limit) - res = stmtRes - curStmt := Statement{ - Statement: portal.Stmt.Statement, - Prepared: portal.Stmt, - ExpectedTypes: portal.Stmt.Columns, - AnonymizedStr: portal.Stmt.AnonymizedStr, - } - stmtCtx := withStatement(ex.Ctx(), ex.curStmt) - ev, payload, err = ex.execStmt(stmtCtx, curStmt, stmtRes, pinfo) - if err != nil { - return err - } - case PrepareStmt: - ex.curStmt = tcmd.AST - res = ex.clientComm.CreatePrepareResult(pos) - stmtCtx := withStatement(ex.Ctx(), ex.curStmt) - ev, payload = ex.execPrepare(stmtCtx, tcmd) - case DescribeStmt: - descRes := ex.clientComm.CreateDescribeResult(pos) - res = descRes - ev, payload = ex.execDescribe(ex.Ctx(), tcmd, descRes) - case BindStmt: - res = ex.clientComm.CreateBindResult(pos) - ev, payload = ex.execBind(ex.Ctx(), tcmd) - case DeletePreparedStmt: - res = ex.clientComm.CreateDeleteResult(pos) - ev, payload = ex.execDelPrepStmt(ex.Ctx(), tcmd) - case SendError: - res = ex.clientComm.CreateErrorResult(pos) - ev = eventNonRetriableErr{IsCommit: fsm.False} - payload = eventNonRetriableErrPayload{err: tcmd.Err} - case Sync: - // Note that the Sync result will flush results to the network connection. - res = ex.clientComm.CreateSyncResult(pos) - if draining { - // If we're draining, check whether this is a good time to finish the - // connection. If we're not inside a transaction, we stop processing - // now. If we are inside a transaction, we'll check again the next time - // a Sync is processed. - if snt, ok := ex.machine.CurState().(stateNoTxn); ok { - res.Close(stateToTxnStatusIndicator(snt)) - return nil - } - if snt, ok := ex.machine.CurState().(stateInternalError); ok { - res.Close(stateToTxnStatusIndicator(snt)) - return nil - } - } - case CopyIn: - res = ex.clientComm.CreateCopyInResult(pos) - var err error - ev, payload, err = ex.execCopyIn(ex.Ctx(), tcmd) - if err != nil { - return err - } - case DrainRequest: - // We received a drain request. We terminate immediately if we're not in a - // transaction. If we are in a transaction, we'll finish as soon as a Sync - // command (i.e. the end of a batch) is processed outside of a - // transaction. - draining = true - res = ex.clientComm.CreateDrainResult(pos) - if _, ok := ex.machine.CurState().(stateNoTxn); ok { - return nil - } - case Flush: - // Closing the res will flush the connection's buffer. - res = ex.clientComm.CreateFlushResult(pos) - default: - panic(fmt.Sprintf("unsupported command type: %T", cmd)) + if log.ExpensiveLogEnabled(ctx, 2) || ex.eventLog != nil { + ex.sessionEventf(ctx, "[%s pos:%d] executing %s", + ex.machine.CurState(), pos, cmd) + } + + var ev fsm.Event + var payload fsm.EventPayload + var res ResultBase + + switch tcmd := cmd.(type) { + case ExecStmt: + if tcmd.AST == nil { + res = ex.clientComm.CreateEmptyQueryResult(pos) + break } + ex.curStmt = tcmd.AST - var advInfo advanceInfo + stmtRes := ex.clientComm.CreateStatementResult( + tcmd.AST, NeedRowDesc, pos, nil, /* formatCodes */ + ex.sessionData.DataConversion) + res = stmtRes + curStmt := Statement{Statement: tcmd.Statement} - // If an event was generated, feed it to the state machine. - if ev != nil { - var err error - advInfo, err = ex.txnStateTransitionsApplyWrapper(ev, payload, res, pos) - if err != nil { - return err - } - } else { - // If no event was generated synthesize an advance code. - advInfo = advanceInfo{ - code: advanceOne, - } + ex.phaseTimes[sessionQueryReceived] = tcmd.TimeReceived + ex.phaseTimes[sessionStartParse] = tcmd.ParseStart + ex.phaseTimes[sessionEndParse] = tcmd.ParseEnd + + stmtCtx := withStatement(ctx, ex.curStmt) + ev, payload, err = ex.execStmt(stmtCtx, curStmt, stmtRes, nil /* pinfo */) + if err != nil { + return err + } + case ExecPortal: + // ExecPortal is handled like ExecStmt, except that the placeholder info + // is taken from the portal. + + portal, ok := ex.extraTxnState.prepStmtsNamespace.portals[tcmd.Name] + if !ok { + err := pgerror.NewErrorf( + pgerror.CodeInvalidCursorNameError, "unknown portal %q", tcmd.Name) + ev = eventNonRetriableErr{IsCommit: fsm.False} + payload = eventNonRetriableErrPayload{err: err} + res = ex.clientComm.CreateErrorResult(pos) + break + } + if log.ExpensiveLogEnabled(ctx, 2) { + log.VEventf(ctx, 2, "portal resolved to: %s", portal.Stmt.AST.String()) + } + ex.curStmt = portal.Stmt.AST + + pinfo := &tree.PlaceholderInfo{ + PlaceholderTypesInfo: tree.PlaceholderTypesInfo{ + TypeHints: portal.Stmt.TypeHints, + Types: portal.Stmt.Types, + }, + Values: portal.Qargs, } - // Decide if we need to close the result or not. We don't need to do it if - // we're staying in place or rewinding - the statement will be executed - // again. - if advInfo.code != stayInPlace && advInfo.code != rewind { - // Close the result. In case of an execution error, the result might have - // its error set already or it might not. - resErr := res.Err() - - pe, ok := payload.(payloadWithError) - if ok { - ex.sessionEventf(ex.Ctx(), "execution error: %s", pe.errorCause()) - } - if resErr == nil && ok { - // Depending on whether the result has the error already or not, we have - // to call either Close or CloseWithErr. - res.CloseWithErr(pe.errorCause()) - } else { - ex.recordError(ex.Ctx(), resErr) + ex.phaseTimes[sessionQueryReceived] = tcmd.TimeReceived + // When parsing has been done earlier, via a separate parse + // message, it is not any more part of the statistics collected + // for this execution. In that case, we simply report that + // parsing took no time. + ex.phaseTimes[sessionStartParse] = time.Time{} + ex.phaseTimes[sessionEndParse] = time.Time{} + + if portal.Stmt.AST == nil { + res = ex.clientComm.CreateEmptyQueryResult(pos) + break + } + + stmtRes := ex.clientComm.CreateStatementResult( + portal.Stmt.AST, + // The client is using the extended protocol, so no row description is + // needed. + DontNeedRowDesc, + pos, portal.OutFormats, + ex.sessionData.DataConversion) + stmtRes.SetLimit(tcmd.Limit) + res = stmtRes + curStmt := Statement{ + Statement: portal.Stmt.Statement, + Prepared: portal.Stmt, + ExpectedTypes: portal.Stmt.Columns, + AnonymizedStr: portal.Stmt.AnonymizedStr, + } + stmtCtx := withStatement(ctx, ex.curStmt) + ev, payload, err = ex.execStmt(stmtCtx, curStmt, stmtRes, pinfo) + if err != nil { + return err + } + case PrepareStmt: + ex.curStmt = tcmd.AST + res = ex.clientComm.CreatePrepareResult(pos) + stmtCtx := withStatement(ctx, ex.curStmt) + ev, payload = ex.execPrepare(stmtCtx, tcmd) + case DescribeStmt: + descRes := ex.clientComm.CreateDescribeResult(pos) + res = descRes + ev, payload = ex.execDescribe(ctx, tcmd, descRes) + case BindStmt: + res = ex.clientComm.CreateBindResult(pos) + ev, payload = ex.execBind(ctx, tcmd) + case DeletePreparedStmt: + res = ex.clientComm.CreateDeleteResult(pos) + ev, payload = ex.execDelPrepStmt(ctx, tcmd) + case SendError: + res = ex.clientComm.CreateErrorResult(pos) + ev = eventNonRetriableErr{IsCommit: fsm.False} + payload = eventNonRetriableErrPayload{err: tcmd.Err} + case Sync: + // Note that the Sync result will flush results to the network connection. + res = ex.clientComm.CreateSyncResult(pos) + if ex.draining { + // If we're draining, check whether this is a good time to finish the + // connection. If we're not inside a transaction, we stop processing + // now. If we are inside a transaction, we'll check again the next time + // a Sync is processed. + if ex.idleConn() { + // If we're about to close the connection, close res in order to flush + // now, as we won't have an opportunity to do it later. res.Close(stateToTxnStatusIndicator(ex.machine.CurState())) + return errDrainingComplete } - } else { - res.Discard() } + case CopyIn: + res = ex.clientComm.CreateCopyInResult(pos) + var err error + ev, payload, err = ex.execCopyIn(ctx, tcmd) + if err != nil { + return err + } + case DrainRequest: + // We received a drain request. We terminate immediately if we're not in a + // transaction. If we are in a transaction, we'll finish as soon as a Sync + // command (i.e. the end of a batch) is processed outside of a + // transaction. + ex.draining = true + res = ex.clientComm.CreateDrainResult(pos) + if ex.idleConn() { + return errDrainingComplete + } + case Flush: + // Closing the res will flush the connection's buffer. + res = ex.clientComm.CreateFlushResult(pos) + default: + panic(fmt.Sprintf("unsupported command type: %T", cmd)) + } - // Move the cursor according to what the state transition told us to do. - switch advInfo.code { - case advanceOne: - ex.stmtBuf.advanceOne() - case skipBatch: - // We'll flush whatever results we have to the network. The last one must - // be an error. This flush may seem unnecessary, as we generally only - // flush when the client requests it through a Sync or a Flush but without - // it the Node.js driver isn't happy. That driver likes to send "flush" - // command and only sends Syncs once it received some data. But we ignore - // flush commands (just like we ignore any other commands) when skipping - // to the next batch. - if err := ex.clientComm.Flush(pos); err != nil { - return err - } - if err := ex.stmtBuf.seekToNextBatch(); err != nil { - return err - } - case rewind: - ex.rewindPrepStmtNamespace(ex.Ctx()) - advInfo.rewCap.rewindAndUnlock(ex.Ctx()) - case stayInPlace: - // Nothing to do. The same statement will be executed again. - default: - panic(fmt.Sprintf("unexpected advance code: %s", advInfo.code)) + var advInfo advanceInfo + + // If an event was generated, feed it to the state machine. + if ev != nil { + var err error + advInfo, err = ex.txnStateTransitionsApplyWrapper(ev, payload, res, pos) + if err != nil { + return err } + } else { + // If no event was generated synthesize an advance code. + advInfo = advanceInfo{ + code: advanceOne, + } + } - if err := ex.updateTxnRewindPosMaybe(ex.Ctx(), cmd, pos, advInfo); err != nil { + // Decide if we need to close the result or not. We don't need to do it if + // we're staying in place or rewinding - the statement will be executed + // again. + if advInfo.code != stayInPlace && advInfo.code != rewind { + // Close the result. In case of an execution error, the result might have + // its error set already or it might not. + resErr := res.Err() + + pe, ok := payload.(payloadWithError) + if ok { + ex.sessionEventf(ctx, "execution error: %s", pe.errorCause()) + } + if resErr == nil && ok { + // Depending on whether the result has the error already or not, we have + // to call either Close or CloseWithErr. + res.CloseWithErr(pe.errorCause()) + } else { + ex.recordError(ctx, resErr) + res.Close(stateToTxnStatusIndicator(ex.machine.CurState())) + } + } else { + res.Discard() + } + + // Move the cursor according to what the state transition told us to do. + switch advInfo.code { + case advanceOne: + ex.stmtBuf.advanceOne() + case skipBatch: + // We'll flush whatever results we have to the network. The last one must + // be an error. This flush may seem unnecessary, as we generally only + // flush when the client requests it through a Sync or a Flush but without + // it the Node.js driver isn't happy. That driver likes to send "flush" + // command and only sends Syncs once it received some data. But we ignore + // flush commands (just like we ignore any other commands) when skipping + // to the next batch. + if err := ex.clientComm.Flush(pos); err != nil { return err } + if err := ex.stmtBuf.seekToNextBatch(); err != nil { + return err + } + case rewind: + ex.rewindPrepStmtNamespace(ctx) + advInfo.rewCap.rewindAndUnlock(ctx) + case stayInPlace: + // Nothing to do. The same statement will be executed again. + default: + panic(fmt.Sprintf("unexpected advance code: %s", advInfo.code)) + } + + return ex.updateTxnRewindPosMaybe(ctx, cmd, pos, advInfo) +} + +func (ex *connExecutor) idleConn() bool { + switch ex.machine.CurState().(type) { + case stateNoTxn: + return true + case stateInternalError: + return true + default: + return false } } diff --git a/pkg/sql/conn_io.go b/pkg/sql/conn_io.go index cb3135ea6603..148d93191d6c 100644 --- a/pkg/sql/conn_io.go +++ b/pkg/sql/conn_io.go @@ -117,7 +117,9 @@ type StmtBuf struct { // buffer. type Command interface { fmt.Stringer - command() + // command returns a string representation of the command type (e.g. + // "prepare", "exec stmt"). + command() string } // ExecStmt is the command for running a query sent through the "simple" pgwire @@ -138,7 +140,7 @@ type ExecStmt struct { } // command implements the Command interface. -func (ExecStmt) command() {} +func (ExecStmt) command() string { return "exec stmt" } func (e ExecStmt) String() string { // We have the original SQL, but we still use String() because it obfuscates @@ -160,7 +162,7 @@ type ExecPortal struct { } // command implements the Command interface. -func (ExecPortal) command() {} +func (ExecPortal) command() string { return "exec portal" } func (e ExecPortal) String() string { return fmt.Sprintf("ExecPortal name: %q", e.Name) @@ -187,7 +189,7 @@ type PrepareStmt struct { } // command implements the Command interface. -func (PrepareStmt) command() {} +func (PrepareStmt) command() string { return "prepare" } func (p PrepareStmt) String() string { // We have the original SQL, but we still use String() because it obfuscates @@ -205,7 +207,7 @@ type DescribeStmt struct { } // command implements the Command interface. -func (DescribeStmt) command() {} +func (DescribeStmt) command() string { return "describe" } func (d DescribeStmt) String() string { return fmt.Sprintf("Describe: %q", d.Name) @@ -245,7 +247,7 @@ type BindStmt struct { } // command implements the Command interface. -func (BindStmt) command() {} +func (BindStmt) command() string { return "bind" } func (b BindStmt) String() string { return fmt.Sprintf("BindStmt: %q->%q", b.PreparedStatementName, b.PortalName) @@ -260,7 +262,7 @@ type DeletePreparedStmt struct { } // command implements the Command interface. -func (DeletePreparedStmt) command() {} +func (DeletePreparedStmt) command() string { return "delete" } func (d DeletePreparedStmt) String() string { return fmt.Sprintf("DeletePreparedStmt: %q", d.Name) @@ -280,7 +282,7 @@ var _ Command = DeletePreparedStmt{} type Sync struct{} // command implements the Command interface. -func (Sync) command() {} +func (Sync) command() string { return "sync" } func (Sync) String() string { return "Sync" @@ -293,7 +295,7 @@ var _ Command = Sync{} type Flush struct{} // command implements the Command interface. -func (Flush) command() {} +func (Flush) command() string { return "flush" } func (Flush) String() string { return "Flush" @@ -313,7 +315,7 @@ type CopyIn struct { } // command implements the Command interface. -func (CopyIn) command() {} +func (CopyIn) command() string { return "copy" } func (CopyIn) String() string { return "CopyIn" @@ -328,7 +330,7 @@ var _ Command = CopyIn{} type DrainRequest struct{} // command implements the Command interface. -func (DrainRequest) command() {} +func (DrainRequest) command() string { return "drain" } func (DrainRequest) String() string { return "Drain" @@ -345,7 +347,7 @@ type SendError struct { } // command implements the Command interface. -func (SendError) command() {} +func (SendError) command() string { return "send error" } func (s SendError) String() string { return fmt.Sprintf("SendError: %s", s.Err) diff --git a/pkg/sql/logictest/testdata/planner_test/select b/pkg/sql/logictest/testdata/planner_test/select index a463ecd7c485..ff9deff91a66 100644 --- a/pkg/sql/logictest/testdata/planner_test/select +++ b/pkg/sql/logictest/testdata/planner_test/select @@ -18,21 +18,27 @@ FROM [SHOW TRACE FOR SESSION] WHERE message LIKE '%SPAN START%' OR message LIKE '%pos%executing%'; ---- 0 === SPAN START: session recording === session recording +1 === SPAN START: exec cmd: exec stmt === exec cmd: exec stmt 0 [NoTxn pos:?] executing ExecStmt: BEGIN TRANSACTION session recording -1 === SPAN START: sql txn === sql txn -1 [Open pos:?] executing ExecStmt: SELECT 1 sql txn -2 === SPAN START: consuming rows === consuming rows -3 === SPAN START: flow === flow -7 === SPAN START: values === +2 === SPAN START: sql txn === sql txn +3 === SPAN START: exec cmd: exec stmt === exec cmd: exec stmt +2 [Open pos:?] executing ExecStmt: SELECT 1 sql txn +4 === SPAN START: consuming rows === consuming rows +5 === SPAN START: flow === flow +13 === SPAN START: values === cockroach.processorid: 0 values -1 [Open pos:?] executing ExecStmt: COMMIT TRANSACTION sql txn +6 === SPAN START: exec cmd: exec stmt === exec cmd: exec stmt +2 [Open pos:?] executing ExecStmt: COMMIT TRANSACTION sql txn +7 === SPAN START: exec cmd: exec stmt === exec cmd: exec stmt 0 [NoTxn pos:?] executing ExecStmt: SELECT 2 session recording -4 === SPAN START: sql txn === sql txn -4 [Open pos:?] executing ExecStmt: SELECT 2 sql txn -5 === SPAN START: consuming rows === consuming rows -6 === SPAN START: flow === flow -8 === SPAN START: values === +8 === SPAN START: sql txn === sql txn +9 === SPAN START: exec cmd: exec stmt === exec cmd: exec stmt +8 [Open pos:?] executing ExecStmt: SELECT 2 sql txn +10 === SPAN START: consuming rows === consuming rows +11 === SPAN START: flow === flow +14 === SPAN START: values === cockroach.processorid: 0 values +12 === SPAN START: exec cmd: exec stmt === exec cmd: exec stmt 0 [NoTxn pos:?] executing ExecStmt: SET TRACING = off session recording # ------------------------------------------------------------------------------ diff --git a/pkg/sql/logictest/testdata/planner_test/show_trace b/pkg/sql/logictest/testdata/planner_test/show_trace index cbc34168cd71..a191b50aa5b0 100644 --- a/pkg/sql/logictest/testdata/planner_test/show_trace +++ b/pkg/sql/logictest/testdata/planner_test/show_trace @@ -21,9 +21,9 @@ WHERE message NOT LIKE '%Z/%' AND tag NOT LIKE '%IndexBackfiller%' AND operation != 'dist sender send' ---- -flow CPut /Table/2/1/0/"t"/3/1 -> 53 -flow CPut /Table/3/1/53/2/1 -> database: users: > > -sql txn rows affected: 0 +flow CPut /Table/2/1/0/"t"/3/1 -> 53 +flow CPut /Table/3/1/53/2/1 -> database: users: > > +exec cmd: exec stmt rows affected: 0 # More KV operations. @@ -41,9 +41,9 @@ WHERE message NOT LIKE '%Z/%' AND tag NOT LIKE '%IndexBackfiller%' AND operation != 'dist sender send' ---- -flow CPut /Table/2/1/53/"kv"/3/1 -> 54 -flow CPut /Table/3/1/54/2/1 -> table: columns: nullable:false hidden:false > columns: nullable:true hidden:false > next_column_id:3 families: next_family_id:1 primary_index: interleave:<> partitioning: type:FORWARD > next_index_id:2 privileges: users: > next_mutation_id:1 format_version:3 state:PUBLIC view_query:"" drop_time:0 replacement_of: > audit_mode:DISABLED drop_job_id:0 > -sql txn rows affected: 0 +flow CPut /Table/2/1/53/"kv"/3/1 -> 54 +flow CPut /Table/3/1/54/2/1 -> table: columns: nullable:false hidden:false > columns: nullable:true hidden:false > next_column_id:3 families: next_family_id:1 primary_index: interleave:<> partitioning: type:FORWARD > next_index_id:2 privileges: users: > next_mutation_id:1 format_version:3 state:PUBLIC view_query:"" drop_time:0 replacement_of: > audit_mode:DISABLED drop_job_id:0 > +exec cmd: exec stmt rows affected: 0 # We avoid using the full trace output, because that would make the # ensuing trace especially chatty, as it traces the index backfill at @@ -68,8 +68,8 @@ WHERE message NOT LIKE '%Z/%' AND message NOT LIKE 'querying next range at%' AND tag NOT LIKE '%IndexBackfiller%' AND operation != 'dist sender send' ---- -flow Put /Table/3/1/54/2/1 -> table: columns: nullable:false hidden:false > columns: nullable:true hidden:false > next_column_id:3 families: next_family_id:1 primary_index: interleave:<> partitioning: type:FORWARD > next_index_id:3 privileges: users: > mutations: interleave:<> partitioning: type:FORWARD > state:DELETE_ONLY direction:ADD mutation_id:1 rollback:false > next_mutation_id:2 format_version:3 state:PUBLIC view_query:"" mutationJobs:<...> drop_time:0 replacement_of: > audit_mode:DISABLED drop_job_id:0 > -sql txn rows affected: 0 +flow Put /Table/3/1/54/2/1 -> table: columns: nullable:false hidden:false > columns: nullable:true hidden:false > next_column_id:3 families: next_family_id:1 primary_index: interleave:<> partitioning: type:FORWARD > next_index_id:3 privileges: users: > mutations: interleave:<> partitioning: type:FORWARD > state:DELETE_ONLY direction:ADD mutation_id:1 rollback:false > next_mutation_id:2 format_version:3 state:PUBLIC view_query:"" mutationJobs:<...> drop_time:0 replacement_of: > audit_mode:DISABLED drop_job_id:0 > +exec cmd: exec stmt rows affected: 0 statement ok SET tracing = on,kv,results; INSERT INTO t.kv(k, v) VALUES (1,2); SET tracing = off @@ -78,10 +78,10 @@ query TT SELECT operation, message FROM [SHOW KV TRACE FOR SESSION] WHERE operation != 'dist sender send' ---- -flow CPut /Table/54/1/1/0 -> /TUPLE/2:2:Int/2 -flow InitPut /Table/54/2/2/0 -> /BYTES/0x89 -flow fast path completed -sql txn rows affected: 1 +flow CPut /Table/54/1/1/0 -> /TUPLE/2:2:Int/2 +flow InitPut /Table/54/2/2/0 -> /BYTES/0x89 +flow fast path completed +exec cmd: exec stmt rows affected: 1 statement error duplicate key value @@ -92,9 +92,9 @@ set tracing=off; SELECT operation, message FROM [SHOW KV TRACE FOR SESSION] WHERE operation != 'dist sender send' ---- -flow CPut /Table/54/1/1/0 -> /TUPLE/2:2:Int/2 -flow InitPut /Table/54/2/2/0 -> /BYTES/0x89 -sql txn execution failed after 0 rows: duplicate key value (k)=(1) violates unique constraint "primary" +flow CPut /Table/54/1/1/0 -> /TUPLE/2:2:Int/2 +flow InitPut /Table/54/2/2/0 -> /BYTES/0x89 +exec cmd: exec stmt execution failed after 0 rows: duplicate key value (k)=(1) violates unique constraint "primary" statement error duplicate key value SET tracing = on,kv,results; INSERT INTO t.kv(k, v) VALUES (2,2); SET tracing = off @@ -104,9 +104,9 @@ set tracing=off; SELECT operation, message FROM [SHOW KV TRACE FOR SESSION] WHERE operation != 'dist sender send' ---- -flow CPut /Table/54/1/2/0 -> /TUPLE/2:2:Int/2 -flow InitPut /Table/54/2/2/0 -> /BYTES/0x8a -sql txn execution failed after 0 rows: duplicate key value (v)=(2) violates unique constraint "woo" +flow CPut /Table/54/1/2/0 -> /TUPLE/2:2:Int/2 +flow InitPut /Table/54/2/2/0 -> /BYTES/0x8a +exec cmd: exec stmt execution failed after 0 rows: duplicate key value (v)=(2) violates unique constraint "woo" statement ok SET tracing = on,kv,results; CREATE TABLE t.kv2 AS TABLE t.kv; SET tracing = off @@ -122,13 +122,13 @@ WHERE message NOT LIKE '%Z/%' AND tag NOT LIKE '%IndexBackfiller%' AND operation != 'dist sender send' ---- -table reader Scan /Table/54/{1-2} -flow CPut /Table/2/1/53/"kv2"/3/1 -> 55 -flow CPut /Table/3/1/55/2/1 -> table: columns: nullable:true hidden:false > columns: nullable:true hidden:false > columns: nullable:false default_expr:"unique_rowid()" hidden:true > next_column_id:4 families: next_family_id:1 primary_index: interleave:<> partitioning: type:FORWARD > next_index_id:2 privileges: users: > next_mutation_id:1 format_version:3 state:PUBLIC view_query:"" drop_time:0 replacement_of: > audit_mode:DISABLED drop_job_id:0 > -table reader fetched: /kv/primary/1/v -> /2 -flow CPut /Table/55/1/...PK.../0 -> /TUPLE/1:1:Int/1/1:2:Int/2 -flow fast path completed -sql txn rows affected: 1 +table reader Scan /Table/54/{1-2} +flow CPut /Table/2/1/53/"kv2"/3/1 -> 55 +flow CPut /Table/3/1/55/2/1 -> table: columns: nullable:true hidden:false > columns: nullable:true hidden:false > columns: nullable:false default_expr:"unique_rowid()" hidden:true > next_column_id:4 families: next_family_id:1 primary_index: interleave:<> partitioning: type:FORWARD > next_index_id:2 privileges: users: > next_mutation_id:1 format_version:3 state:PUBLIC view_query:"" drop_time:0 replacement_of: > audit_mode:DISABLED drop_job_id:0 > +table reader fetched: /kv/primary/1/v -> /2 +flow CPut /Table/55/1/...PK.../0 -> /TUPLE/1:1:Int/1/1:2:Int/2 +flow fast path completed +exec cmd: exec stmt rows affected: 1 statement ok SET tracing = on,kv,results; UPDATE t.kv2 SET v = v + 2; SET tracing = off @@ -143,11 +143,11 @@ WHERE message NOT LIKE '%Z/%' AND tag NOT LIKE '%IndexBackfiller%' AND operation != 'dist sender send' ---- -table reader Scan /Table/55/{1-2} -table reader fetched: /kv2/primary/...PK.../k/v -> /1/2 -flow Put /Table/55/1/...PK.../0 -> /TUPLE/1:1:Int/1/1:2:Int/4 -flow fast path completed -sql txn rows affected: 1 +table reader Scan /Table/55/{1-2} +table reader fetched: /kv2/primary/...PK.../k/v -> /1/2 +flow Put /Table/55/1/...PK.../0 -> /TUPLE/1:1:Int/1/1:2:Int/4 +flow fast path completed +exec cmd: exec stmt rows affected: 1 statement ok SET tracing = on,kv,results; DELETE FROM t.kv2; SET tracing = off @@ -156,9 +156,9 @@ query TT SELECT operation, message FROM [SHOW KV TRACE FOR SESSION] WHERE operation != 'dist sender send' ---- -flow DelRange /Table/55/1 - /Table/55/2 -flow fast path completed -sql txn rows affected: 1 +flow DelRange /Table/55/1 - /Table/55/2 +flow fast path completed +exec cmd: exec stmt rows affected: 1 statement ok SET tracing = on,kv,results; DROP TABLE t.kv2; SET tracing = off @@ -175,8 +175,8 @@ WHERE message NOT LIKE '%Z/%' AND message NOT LIKE 'querying next range at%' AND tag NOT LIKE '%IndexBackfiller%' AND operation != 'dist sender send' ---- -flow Put /Table/3/1/55/2/1 -> table: columns: nullable:true hidden:false > columns: nullable:true hidden:false > columns: nullable:false default_expr:"unique_rowid()" hidden:true > next_column_id:4 families: next_family_id:1 primary_index: interleave:<> partitioning: type:FORWARD > next_index_id:2 privileges: users: > next_mutation_id:1 format_version:3 state:DROP draining_names: view_query:"" drop_time:... replacement_of: > audit_mode:DISABLED drop_job_id:... > -sql txn rows affected: 0 +flow Put /Table/3/1/55/2/1 -> table: columns: nullable:true hidden:false > columns: nullable:true hidden:false > columns: nullable:false default_expr:"unique_rowid()" hidden:true > next_column_id:4 families: next_family_id:1 primary_index: interleave:<> partitioning: type:FORWARD > next_index_id:2 privileges: users: > next_mutation_id:1 format_version:3 state:DROP draining_names: view_query:"" drop_time:... replacement_of: > audit_mode:DISABLED drop_job_id:... > +exec cmd: exec stmt rows affected: 0 statement ok SET tracing = on,kv,results; DELETE FROM t.kv; SET tracing = off @@ -185,12 +185,12 @@ query TT SELECT operation, message FROM [SHOW KV TRACE FOR SESSION] WHERE operation != 'dist sender send' ---- -table reader Scan /Table/54/{1-2} -table reader fetched: /kv/primary/1/v -> /2 -flow Del /Table/54/2/2/0 -flow Del /Table/54/1/1/0 -flow fast path completed -sql txn rows affected: 1 +table reader Scan /Table/54/{1-2} +table reader fetched: /kv/primary/1/v -> /2 +flow Del /Table/54/2/2/0 +flow Del /Table/54/1/1/0 +flow fast path completed +exec cmd: exec stmt rows affected: 1 statement ok SET tracing = on,kv,results; DROP INDEX t.kv@woo CASCADE; SET tracing = off @@ -209,8 +209,8 @@ WHERE message NOT LIKE '%Z/%' AND message NOT LIKE 'querying next range at%' AND tag NOT LIKE '%IndexBackfiller%' AND operation != 'dist sender send' ---- -flow Put /Table/3/1/54/2/1 -> table: columns: nullable:false hidden:false > columns: nullable:true hidden:false > next_column_id:3 families: next_family_id:1 primary_index: interleave:<> partitioning: type:FORWARD > next_index_id:3 privileges: users: > mutations: interleave:<> partitioning: type:FORWARD > state:DELETE_AND_WRITE_ONLY direction:DROP mutation_id:2 rollback:false > next_mutation_id:3 format_version:3 state:PUBLIC view_query:"" mutationJobs:<...> drop_time:0 replacement_of: > audit_mode:DISABLED drop_job_id:0 > -sql txn rows affected: 0 +flow Put /Table/3/1/54/2/1 -> table: columns: nullable:false hidden:false > columns: nullable:true hidden:false > next_column_id:3 families: next_family_id:1 primary_index: interleave:<> partitioning: type:FORWARD > next_index_id:3 privileges: users: > mutations: interleave:<> partitioning: type:FORWARD > state:DELETE_AND_WRITE_ONLY direction:DROP mutation_id:2 rollback:false > next_mutation_id:3 format_version:3 state:PUBLIC view_query:"" mutationJobs:<...> drop_time:0 replacement_of: > audit_mode:DISABLED drop_job_id:0 > +exec cmd: exec stmt rows affected: 0 statement ok SET tracing = on,kv,results; DROP TABLE t.kv; SET tracing = off @@ -226,8 +226,8 @@ WHERE message NOT LIKE '%Z/%' AND message NOT LIKE 'querying next range at%' AND tag NOT LIKE '%IndexBackfiller%' AND operation != 'dist sender send' ---- -flow Put /Table/3/1/54/2/1 -> table: columns: nullable:false hidden:false > columns: nullable:true hidden:false > next_column_id:3 families: next_family_id:1 primary_index: interleave:<> partitioning: type:FORWARD > next_index_id:3 privileges: users: > next_mutation_id:3 format_version:3 state:DROP draining_names: view_query:"" drop_time:... replacement_of: > audit_mode:DISABLED drop_job_id:... gc_mutations: > -sql txn rows affected: 0 +flow Put /Table/3/1/54/2/1 -> table: columns: nullable:false hidden:false > columns: nullable:true hidden:false > next_column_id:3 families: next_family_id:1 primary_index: interleave:<> partitioning: type:FORWARD > next_index_id:3 privileges: users: > next_mutation_id:3 format_version:3 state:DROP draining_names: view_query:"" drop_time:... replacement_of: > audit_mode:DISABLED drop_job_id:... gc_mutations: > +exec cmd: exec stmt rows affected: 0 # Check that session tracing does not inhibit the fast path for inserts & # friends (the path resulting in 1PC transactions). diff --git a/pkg/sql/logictest/testdata/planner_test/upsert b/pkg/sql/logictest/testdata/planner_test/upsert index 53fe1570868f..2647cda8770c 100644 --- a/pkg/sql/logictest/testdata/planner_test/upsert +++ b/pkg/sql/logictest/testdata/planner_test/upsert @@ -59,11 +59,11 @@ query TT SELECT operation, message FROM [SHOW KV TRACE FOR SESSION] WHERE operation != 'dist sender send' ---- -flow Scan /Table/56/1/2{-/#} -flow CPut /Table/56/1/2/0 -> /TUPLE/2:2:Int/3 -flow InitPut /Table/56/2/3/0 -> /BYTES/0x8a -flow fast path completed -sql txn rows affected: 1 +flow Scan /Table/56/1/2{-/#} +flow CPut /Table/56/1/2/0 -> /TUPLE/2:2:Int/3 +flow InitPut /Table/56/2/3/0 -> /BYTES/0x8a +flow fast path completed +exec cmd: exec stmt rows affected: 1 statement ok SET tracing = on,kv,results; UPSERT INTO t.kv(k, v) VALUES (1,2); SET tracing = off @@ -72,11 +72,11 @@ query TT SELECT operation, message FROM [SHOW KV TRACE FOR SESSION] WHERE operation != 'dist sender send' ---- -flow Scan /Table/56/1/1{-/#} -flow CPut /Table/56/1/1/0 -> /TUPLE/2:2:Int/2 -flow InitPut /Table/56/2/2/0 -> /BYTES/0x89 -flow fast path completed -sql txn rows affected: 1 +flow Scan /Table/56/1/1{-/#} +flow CPut /Table/56/1/1/0 -> /TUPLE/2:2:Int/2 +flow InitPut /Table/56/2/2/0 -> /BYTES/0x89 +flow fast path completed +exec cmd: exec stmt rows affected: 1 statement error duplicate key value SET tracing = on,kv,results; UPSERT INTO t.kv(k, v) VALUES (2,2); SET tracing = off @@ -86,12 +86,12 @@ set tracing=off; SELECT operation, message FROM [SHOW KV TRACE FOR SESSION] WHERE operation != 'dist sender send' ---- -flow Scan /Table/56/1/2{-/#} -flow fetched: /kv/primary/2/v -> /3 -flow Put /Table/56/1/2/0 -> /TUPLE/2:2:Int/2 -flow Del /Table/56/2/3/0 -flow CPut /Table/56/2/2/0 -> /BYTES/0x8a (expecting does not exist) -sql txn execution failed after 0 rows: duplicate key value (v)=(2) violates unique constraint "woo" +flow Scan /Table/56/1/2{-/#} +flow fetched: /kv/primary/2/v -> /3 +flow Put /Table/56/1/2/0 -> /TUPLE/2:2:Int/2 +flow Del /Table/56/2/3/0 +flow CPut /Table/56/2/2/0 -> /BYTES/0x8a (expecting does not exist) +exec cmd: exec stmt execution failed after 0 rows: duplicate key value (v)=(2) violates unique constraint "woo" subtest regression_32834 diff --git a/pkg/sql/opt/exec/execbuilder/testdata/select b/pkg/sql/opt/exec/execbuilder/testdata/select index 847107c469bf..03cd93294be5 100644 --- a/pkg/sql/opt/exec/execbuilder/testdata/select +++ b/pkg/sql/opt/exec/execbuilder/testdata/select @@ -17,19 +17,25 @@ SELECT FROM [SHOW TRACE FOR SESSION] WHERE message LIKE '%SPAN START%' OR message LIKE '%pos%executing%'; ---- -0 === SPAN START: session recording === session recording -0 [NoTxn pos:?] executing ExecStmt: BEGIN TRANSACTION session recording -1 === SPAN START: sql txn === sql txn -1 [Open pos:?] executing ExecStmt: SELECT 1 sql txn -2 === SPAN START: consuming rows === consuming rows -3 === SPAN START: flow === flow -1 [Open pos:?] executing ExecStmt: COMMIT TRANSACTION sql txn -0 [NoTxn pos:?] executing ExecStmt: SELECT 2 session recording -4 === SPAN START: sql txn === sql txn -4 [Open pos:?] executing ExecStmt: SELECT 2 sql txn -5 === SPAN START: consuming rows === consuming rows -6 === SPAN START: flow === flow -0 [NoTxn pos:?] executing ExecStmt: SET TRACING = off session recording +0 === SPAN START: session recording === session recording +1 === SPAN START: exec cmd: exec stmt === exec cmd: exec stmt +0 [NoTxn pos:?] executing ExecStmt: BEGIN TRANSACTION session recording +2 === SPAN START: sql txn === sql txn +3 === SPAN START: exec cmd: exec stmt === exec cmd: exec stmt +2 [Open pos:?] executing ExecStmt: SELECT 1 sql txn +4 === SPAN START: consuming rows === consuming rows +5 === SPAN START: flow === flow +6 === SPAN START: exec cmd: exec stmt === exec cmd: exec stmt +2 [Open pos:?] executing ExecStmt: COMMIT TRANSACTION sql txn +7 === SPAN START: exec cmd: exec stmt === exec cmd: exec stmt +0 [NoTxn pos:?] executing ExecStmt: SELECT 2 session recording +8 === SPAN START: sql txn === sql txn +9 === SPAN START: exec cmd: exec stmt === exec cmd: exec stmt +8 [Open pos:?] executing ExecStmt: SELECT 2 sql txn +10 === SPAN START: consuming rows === consuming rows +11 === SPAN START: flow === flow +12 === SPAN START: exec cmd: exec stmt === exec cmd: exec stmt +0 [NoTxn pos:?] executing ExecStmt: SET TRACING = off session recording # ------------------------------------------------------------------------------ # Numeric References Tests. diff --git a/pkg/sql/opt/exec/execbuilder/testdata/show_trace b/pkg/sql/opt/exec/execbuilder/testdata/show_trace index 72a1967ed8f3..17e67c89865d 100644 --- a/pkg/sql/opt/exec/execbuilder/testdata/show_trace +++ b/pkg/sql/opt/exec/execbuilder/testdata/show_trace @@ -17,9 +17,9 @@ WHERE message NOT LIKE '%Z/%' AND tag NOT LIKE '%IndexBackfiller%' AND operation != 'dist sender send' ---- -flow CPut /Table/2/1/0/"t"/3/1 -> 53 -flow CPut /Table/3/1/53/2/1 -> database: users: > > -sql txn rows affected: 0 +flow CPut /Table/2/1/0/"t"/3/1 -> 53 +flow CPut /Table/3/1/53/2/1 -> database: users: > > +exec cmd: exec stmt rows affected: 0 # More KV operations. @@ -37,9 +37,9 @@ WHERE message NOT LIKE '%Z/%' AND tag NOT LIKE '%IndexBackfiller%' AND operation != 'dist sender send' ---- -flow CPut /Table/2/1/53/"kv"/3/1 -> 54 -flow CPut /Table/3/1/54/2/1 -> table: columns: nullable:false hidden:false > columns: nullable:true hidden:false > next_column_id:3 families: next_family_id:1 primary_index: interleave:<> partitioning: type:FORWARD > next_index_id:2 privileges: users: > next_mutation_id:1 format_version:3 state:PUBLIC view_query:"" drop_time:0 replacement_of: > audit_mode:DISABLED drop_job_id:0 > -sql txn rows affected: 0 +flow CPut /Table/2/1/53/"kv"/3/1 -> 54 +flow CPut /Table/3/1/54/2/1 -> table: columns: nullable:false hidden:false > columns: nullable:true hidden:false > next_column_id:3 families: next_family_id:1 primary_index: interleave:<> partitioning: type:FORWARD > next_index_id:2 privileges: users: > next_mutation_id:1 format_version:3 state:PUBLIC view_query:"" drop_time:0 replacement_of: > audit_mode:DISABLED drop_job_id:0 > +exec cmd: exec stmt rows affected: 0 # We avoid using the full trace output, because that would make the # ensuing trace especially chatty, as it traces the index backfill at @@ -64,8 +64,8 @@ WHERE message NOT LIKE '%Z/%' AND message NOT LIKE 'querying next range at%' AND tag NOT LIKE '%IndexBackfiller%' AND operation != 'dist sender send' ---- -flow Put /Table/3/1/54/2/1 -> table: columns: nullable:false hidden:false > columns: nullable:true hidden:false > next_column_id:3 families: next_family_id:1 primary_index: interleave:<> partitioning: type:FORWARD > next_index_id:3 privileges: users: > mutations: interleave:<> partitioning: type:FORWARD > state:DELETE_ONLY direction:ADD mutation_id:1 rollback:false > next_mutation_id:2 format_version:3 state:PUBLIC view_query:"" mutationJobs:<...> drop_time:0 replacement_of: > audit_mode:DISABLED drop_job_id:0 > -sql txn rows affected: 0 +flow Put /Table/3/1/54/2/1 -> table: columns: nullable:false hidden:false > columns: nullable:true hidden:false > next_column_id:3 families: next_family_id:1 primary_index: interleave:<> partitioning: type:FORWARD > next_index_id:3 privileges: users: > mutations: interleave:<> partitioning: type:FORWARD > state:DELETE_ONLY direction:ADD mutation_id:1 rollback:false > next_mutation_id:2 format_version:3 state:PUBLIC view_query:"" mutationJobs:<...> drop_time:0 replacement_of: > audit_mode:DISABLED drop_job_id:0 > +exec cmd: exec stmt rows affected: 0 statement ok SET tracing = on,kv,results; INSERT INTO t.kv(k, v) VALUES (1,2); SET tracing = off @@ -74,10 +74,10 @@ query TT SELECT operation, message FROM [SHOW KV TRACE FOR SESSION] WHERE operation != 'dist sender send' ---- -flow CPut /Table/54/1/1/0 -> /TUPLE/2:2:Int/2 -flow InitPut /Table/54/2/2/0 -> /BYTES/0x89 -flow fast path completed -sql txn rows affected: 1 +flow CPut /Table/54/1/1/0 -> /TUPLE/2:2:Int/2 +flow InitPut /Table/54/2/2/0 -> /BYTES/0x89 +flow fast path completed +exec cmd: exec stmt rows affected: 1 statement error duplicate key value @@ -88,9 +88,9 @@ set tracing=off; SELECT operation, message FROM [SHOW KV TRACE FOR SESSION] WHERE operation != 'dist sender send' ---- -flow CPut /Table/54/1/1/0 -> /TUPLE/2:2:Int/2 -flow InitPut /Table/54/2/2/0 -> /BYTES/0x89 -sql txn execution failed after 0 rows: duplicate key value (k)=(1) violates unique constraint "primary" +flow CPut /Table/54/1/1/0 -> /TUPLE/2:2:Int/2 +flow InitPut /Table/54/2/2/0 -> /BYTES/0x89 +exec cmd: exec stmt execution failed after 0 rows: duplicate key value (k)=(1) violates unique constraint "primary" statement error duplicate key value SET tracing = on,kv,results; INSERT INTO t.kv(k, v) VALUES (2,2); SET tracing = off @@ -100,9 +100,9 @@ set tracing=off; SELECT operation, message FROM [SHOW KV TRACE FOR SESSION] WHERE operation != 'dist sender send' ---- -flow CPut /Table/54/1/2/0 -> /TUPLE/2:2:Int/2 -flow InitPut /Table/54/2/2/0 -> /BYTES/0x8a -sql txn execution failed after 0 rows: duplicate key value (v)=(2) violates unique constraint "woo" +flow CPut /Table/54/1/2/0 -> /TUPLE/2:2:Int/2 +flow InitPut /Table/54/2/2/0 -> /BYTES/0x8a +exec cmd: exec stmt execution failed after 0 rows: duplicate key value (v)=(2) violates unique constraint "woo" statement ok SET tracing = on,kv,results; CREATE TABLE t.kv2 AS TABLE t.kv; SET tracing = off @@ -118,13 +118,13 @@ WHERE message NOT LIKE '%Z/%' AND tag NOT LIKE '%IndexBackfiller%' AND operation != 'dist sender send' ---- -table reader Scan /Table/54/{1-2} -flow CPut /Table/2/1/53/"kv2"/3/1 -> 55 -flow CPut /Table/3/1/55/2/1 -> table: columns: nullable:true hidden:false > columns: nullable:true hidden:false > columns: nullable:false default_expr:"unique_rowid()" hidden:true > next_column_id:4 families: next_family_id:1 primary_index: interleave:<> partitioning: type:FORWARD > next_index_id:2 privileges: users: > next_mutation_id:1 format_version:3 state:PUBLIC view_query:"" drop_time:0 replacement_of: > audit_mode:DISABLED drop_job_id:0 > -table reader fetched: /kv/primary/1/v -> /2 -flow CPut /Table/55/1/...PK.../0 -> /TUPLE/1:1:Int/1/1:2:Int/2 -flow fast path completed -sql txn rows affected: 1 +table reader Scan /Table/54/{1-2} +flow CPut /Table/2/1/53/"kv2"/3/1 -> 55 +flow CPut /Table/3/1/55/2/1 -> table: columns: nullable:true hidden:false > columns: nullable:true hidden:false > columns: nullable:false default_expr:"unique_rowid()" hidden:true > next_column_id:4 families: next_family_id:1 primary_index: interleave:<> partitioning: type:FORWARD > next_index_id:2 privileges: users: > next_mutation_id:1 format_version:3 state:PUBLIC view_query:"" drop_time:0 replacement_of: > audit_mode:DISABLED drop_job_id:0 > +table reader fetched: /kv/primary/1/v -> /2 +flow CPut /Table/55/1/...PK.../0 -> /TUPLE/1:1:Int/1/1:2:Int/2 +flow fast path completed +exec cmd: exec stmt rows affected: 1 statement ok SET tracing = on,kv,results; UPDATE t.kv2 SET v = v + 2; SET tracing = off @@ -139,11 +139,11 @@ WHERE message NOT LIKE '%Z/%' AND tag NOT LIKE '%IndexBackfiller%' AND operation != 'dist sender send' ---- -table reader Scan /Table/55/{1-2} -table reader fetched: /kv2/primary/...PK.../k/v -> /1/2 -flow Put /Table/55/1/...PK.../0 -> /TUPLE/1:1:Int/1/1:2:Int/4 -flow fast path completed -sql txn rows affected: 1 +table reader Scan /Table/55/{1-2} +table reader fetched: /kv2/primary/...PK.../k/v -> /1/2 +flow Put /Table/55/1/...PK.../0 -> /TUPLE/1:1:Int/1/1:2:Int/4 +flow fast path completed +exec cmd: exec stmt rows affected: 1 statement ok SET tracing = on,kv,results; DELETE FROM t.kv2; SET tracing = off @@ -152,9 +152,9 @@ query TT SELECT operation, message FROM [SHOW KV TRACE FOR SESSION] WHERE operation != 'dist sender send' ---- -flow DelRange /Table/55/1 - /Table/55/2 -flow fast path completed -sql txn rows affected: 1 +flow DelRange /Table/55/1 - /Table/55/2 +flow fast path completed +exec cmd: exec stmt rows affected: 1 statement ok SET tracing = on,kv,results; DROP TABLE t.kv2; SET tracing = off @@ -171,8 +171,8 @@ WHERE message NOT LIKE '%Z/%' AND message NOT LIKE 'querying next range at%' AND tag NOT LIKE '%IndexBackfiller%' AND operation != 'dist sender send' ---- -flow Put /Table/3/1/55/2/1 -> table: columns: nullable:true hidden:false > columns: nullable:true hidden:false > columns: nullable:false default_expr:"unique_rowid()" hidden:true > next_column_id:4 families: next_family_id:1 primary_index: interleave:<> partitioning: type:FORWARD > next_index_id:2 privileges: users: > next_mutation_id:1 format_version:3 state:DROP draining_names: view_query:"" drop_time:... replacement_of: > audit_mode:DISABLED drop_job_id:... > -sql txn rows affected: 0 +flow Put /Table/3/1/55/2/1 -> table: columns: nullable:true hidden:false > columns: nullable:true hidden:false > columns: nullable:false default_expr:"unique_rowid()" hidden:true > next_column_id:4 families: next_family_id:1 primary_index: interleave:<> partitioning: type:FORWARD > next_index_id:2 privileges: users: > next_mutation_id:1 format_version:3 state:DROP draining_names: view_query:"" drop_time:... replacement_of: > audit_mode:DISABLED drop_job_id:... > +exec cmd: exec stmt rows affected: 0 statement ok SET tracing = on,kv,results; DELETE FROM t.kv; SET tracing = off @@ -181,12 +181,12 @@ query TT SELECT operation, message FROM [SHOW KV TRACE FOR SESSION] WHERE operation != 'dist sender send' ---- -table reader Scan /Table/54/{1-2} -table reader fetched: /kv/primary/1/v -> /2 -flow Del /Table/54/2/2/0 -flow Del /Table/54/1/1/0 -flow fast path completed -sql txn rows affected: 1 +table reader Scan /Table/54/{1-2} +table reader fetched: /kv/primary/1/v -> /2 +flow Del /Table/54/2/2/0 +flow Del /Table/54/1/1/0 +flow fast path completed +exec cmd: exec stmt rows affected: 1 statement ok SET tracing = on,kv,results; DROP INDEX t.kv@woo CASCADE; SET tracing = off @@ -205,8 +205,8 @@ WHERE message NOT LIKE '%Z/%' AND message NOT LIKE 'querying next range at%' AND tag NOT LIKE '%IndexBackfiller%' AND operation != 'dist sender send' ---- -flow Put /Table/3/1/54/2/1 -> table: columns: nullable:false hidden:false > columns: nullable:true hidden:false > next_column_id:3 families: next_family_id:1 primary_index: interleave:<> partitioning: type:FORWARD > next_index_id:3 privileges: users: > mutations: interleave:<> partitioning: type:FORWARD > state:DELETE_AND_WRITE_ONLY direction:DROP mutation_id:2 rollback:false > next_mutation_id:3 format_version:3 state:PUBLIC view_query:"" mutationJobs:<...> drop_time:0 replacement_of: > audit_mode:DISABLED drop_job_id:0 > -sql txn rows affected: 0 +flow Put /Table/3/1/54/2/1 -> table: columns: nullable:false hidden:false > columns: nullable:true hidden:false > next_column_id:3 families: next_family_id:1 primary_index: interleave:<> partitioning: type:FORWARD > next_index_id:3 privileges: users: > mutations: interleave:<> partitioning: type:FORWARD > state:DELETE_AND_WRITE_ONLY direction:DROP mutation_id:2 rollback:false > next_mutation_id:3 format_version:3 state:PUBLIC view_query:"" mutationJobs:<...> drop_time:0 replacement_of: > audit_mode:DISABLED drop_job_id:0 > +exec cmd: exec stmt rows affected: 0 statement ok SET tracing = on,kv,results; DROP TABLE t.kv; SET tracing = off @@ -222,8 +222,8 @@ WHERE message NOT LIKE '%Z/%' AND message NOT LIKE 'querying next range at%' AND tag NOT LIKE '%IndexBackfiller%' AND operation != 'dist sender send' ---- -flow Put /Table/3/1/54/2/1 -> table: columns: nullable:false hidden:false > columns: nullable:true hidden:false > next_column_id:3 families: next_family_id:1 primary_index: interleave:<> partitioning: type:FORWARD > next_index_id:3 privileges: users: > next_mutation_id:3 format_version:3 state:DROP draining_names: view_query:"" drop_time:... replacement_of: > audit_mode:DISABLED drop_job_id:... gc_mutations: > -sql txn rows affected: 0 +flow Put /Table/3/1/54/2/1 -> table: columns: nullable:false hidden:false > columns: nullable:true hidden:false > next_column_id:3 families: next_family_id:1 primary_index: interleave:<> partitioning: type:FORWARD > next_index_id:3 privileges: users: > next_mutation_id:3 format_version:3 state:DROP draining_names: view_query:"" drop_time:... replacement_of: > audit_mode:DISABLED drop_job_id:... gc_mutations: > +exec cmd: exec stmt rows affected: 0 # Check that session tracing does not inhibit the fast path for inserts & # friends (the path resulting in 1PC transactions). diff --git a/pkg/sql/opt/exec/execbuilder/testdata/upsert b/pkg/sql/opt/exec/execbuilder/testdata/upsert index 20725184206f..462898761983 100644 --- a/pkg/sql/opt/exec/execbuilder/testdata/upsert +++ b/pkg/sql/opt/exec/execbuilder/testdata/upsert @@ -226,11 +226,11 @@ query TT SELECT operation, message FROM [SHOW KV TRACE FOR SESSION] WHERE operation != 'dist sender send' ---- -table reader Scan /Table/57/1/2{-/#} -flow CPut /Table/57/1/2/0 -> /TUPLE/2:2:Int/3 -flow InitPut /Table/57/2/3/0 -> /BYTES/0x8a -flow fast path completed -sql txn rows affected: 1 +table reader Scan /Table/57/1/2{-/#} +flow CPut /Table/57/1/2/0 -> /TUPLE/2:2:Int/3 +flow InitPut /Table/57/2/3/0 -> /BYTES/0x8a +flow fast path completed +exec cmd: exec stmt rows affected: 1 statement ok SET tracing = on,kv,results; UPSERT INTO t.kv(k, v) VALUES (1,2); SET tracing = off @@ -239,11 +239,11 @@ query TT SELECT operation, message FROM [SHOW KV TRACE FOR SESSION] WHERE operation != 'dist sender send' ---- -table reader Scan /Table/57/1/1{-/#} -flow CPut /Table/57/1/1/0 -> /TUPLE/2:2:Int/2 -flow InitPut /Table/57/2/2/0 -> /BYTES/0x89 -flow fast path completed -sql txn rows affected: 1 +table reader Scan /Table/57/1/1{-/#} +flow CPut /Table/57/1/1/0 -> /TUPLE/2:2:Int/2 +flow InitPut /Table/57/2/2/0 -> /BYTES/0x89 +flow fast path completed +exec cmd: exec stmt rows affected: 1 statement error duplicate key value SET tracing = on,kv,results; UPSERT INTO t.kv(k, v) VALUES (2,2); SET tracing = off @@ -253,12 +253,12 @@ set tracing=off; SELECT operation, message FROM [SHOW KV TRACE FOR SESSION] WHERE operation != 'dist sender send' ---- -table reader Scan /Table/57/1/2{-/#} -table reader fetched: /kv/primary/2/v -> /3 -flow Put /Table/57/1/2/0 -> /TUPLE/2:2:Int/2 -flow Del /Table/57/2/3/0 -flow CPut /Table/57/2/2/0 -> /BYTES/0x8a (expecting does not exist) -sql txn execution failed after 0 rows: duplicate key value (v)=(2) violates unique constraint "woo" +table reader Scan /Table/57/1/2{-/#} +table reader fetched: /kv/primary/2/v -> /3 +flow Put /Table/57/1/2/0 -> /TUPLE/2:2:Int/2 +flow Del /Table/57/2/3/0 +flow CPut /Table/57/2/2/0 -> /BYTES/0x8a (expecting does not exist) +exec cmd: exec stmt execution failed after 0 rows: duplicate key value (v)=(2) violates unique constraint "woo" subtest regression_32473 diff --git a/pkg/sql/trace_test.go b/pkg/sql/trace_test.go index d6ec2e5def9c..eb80a1e98fa4 100644 --- a/pkg/sql/trace_test.go +++ b/pkg/sql/trace_test.go @@ -71,6 +71,7 @@ func TestTrace(t *testing.T) { "WHERE operation IS NOT NULL ORDER BY op") }, expSpans: []string{ + "exec cmd: exec stmt", "flow", "session recording", "sql txn", @@ -125,6 +126,7 @@ func TestTrace(t *testing.T) { expSpans: []string{ "session recording", "sql txn", + "exec cmd: exec stmt", "flow", "table reader", "consuming rows", @@ -159,6 +161,7 @@ func TestTrace(t *testing.T) { "WHERE operation IS NOT NULL ORDER BY op") }, expSpans: []string{ + "exec cmd: exec stmt", "flow", "session recording", "sql txn", @@ -191,6 +194,7 @@ func TestTrace(t *testing.T) { expSpans: []string{ "session recording", "sql txn", + "exec cmd: exec stmt", "flow", "table reader", "consuming rows",