diff --git a/pkg/sql/conn_executor.go b/pkg/sql/conn_executor.go index ac182d69cbbd..4361ddd9293c 100644 --- a/pkg/sql/conn_executor.go +++ b/pkg/sql/conn_executor.go @@ -2877,6 +2877,14 @@ func (ex *connExecutor) txnStateTransitionsApplyWrapper( if err != nil { return advanceInfo{}, err } + + if advInfo.txnEvent.eventType == txnUpgradeToExplicit { + ex.extraTxnState.txnFinishClosure.implicit = false + if err = ex.statsCollector.UpgradeImplicitTxn(ex.Ctx()); err != nil { + return advanceInfo{}, err + } + } + } case txnStart: ex.extraTxnState.firstStmtExecuted = false diff --git a/pkg/sql/conn_executor_exec.go b/pkg/sql/conn_executor_exec.go index 986162b7be74..ead8df05aafc 100644 --- a/pkg/sql/conn_executor_exec.go +++ b/pkg/sql/conn_executor_exec.go @@ -2126,12 +2126,11 @@ func (ex *connExecutor) onTxnFinish(ctx context.Context, ev txnEvent) { } } - if !implicit { - ex.statsCollector.EndExplicitTransaction( - ctx, - transactionFingerprintID, - ) - } + ex.statsCollector.EndTransaction( + ctx, + transactionFingerprintID, + ) + if ex.server.cfg.TestingKnobs.BeforeTxnStatsRecorded != nil { ex.server.cfg.TestingKnobs.BeforeTxnStatsRecorded( ex.sessionData(), @@ -2180,7 +2179,6 @@ func (ex *connExecutor) recordTransactionStart(txnID uuid.UUID) { ex.state.mu.RLock() txnStart := ex.state.mu.txnStart ex.state.mu.RUnlock() - implicit := ex.implicitTxn() // Transaction received time is the time at which the statement that prompted // the creation of this transaction was received. @@ -2209,12 +2207,10 @@ func (ex *connExecutor) recordTransactionStart(txnID uuid.UUID) { ex.extraTxnState.shouldExecuteOnTxnFinish = true ex.extraTxnState.txnFinishClosure.txnStartTime = txnStart - ex.extraTxnState.txnFinishClosure.implicit = implicit + ex.extraTxnState.txnFinishClosure.implicit = ex.implicitTxn() ex.extraTxnState.shouldExecuteOnTxnRestart = true - if !implicit { - ex.statsCollector.StartExplicitTransaction() - } + ex.statsCollector.StartTransaction() } func (ex *connExecutor) recordTransactionFinish( diff --git a/pkg/sql/executor_statement_metrics.go b/pkg/sql/executor_statement_metrics.go index 9545675f1cd3..c44dc589c77c 100644 --- a/pkg/sql/executor_statement_metrics.go +++ b/pkg/sql/executor_statement_metrics.go @@ -17,7 +17,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/execinfrapb" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/sqlstats" - "github.com/cockroachdb/cockroach/pkg/util" "github.com/cockroachdb/cockroach/pkg/util/log" "github.com/cockroachdb/cockroach/pkg/util/metric" "github.com/cockroachdb/cockroach/pkg/util/tracing/tracingpb" @@ -168,31 +167,6 @@ func (ex *connExecutor) recordStatementSummary( PlanHash: planner.instrumentation.planGist.Hash(), } - // We only populate the transaction fingerprint ID field if we are in an - // implicit transaction. - // - // TODO(azhng): This will require some big refactoring later, we already - // compute statement's fingerprintID in RecordStatement(). - // However, we need to recompute the Fingerprint() here because this - // is required to populate the transaction fingerprint ID field. - // - // The reason behind it is that: for explicit transactions, we have a final - // callback that will eventually invoke - // statsCollector.EndExplicitTransaction() which will use the extraTxnState - // stored in the connExecutor to compute the transaction fingerprintID. - // Unfortunately, that callback is not invoked for implicit transactions, - // because we don't create temporary stats container for the implicit - // transactions. (The statement stats directly gets written to the actual - // stats container). This means that, unless we populate the transaction - // fingerprintID here, we will not have another chance to do so later. - if ex.implicitTxn() { - stmtFingerprintID := recordedStmtStatsKey.FingerprintID() - txnFingerprintHash := util.MakeFNV64() - txnFingerprintHash.Add(uint64(stmtFingerprintID)) - recordedStmtStatsKey.TransactionFingerprintID = - roachpb.TransactionFingerprintID(txnFingerprintHash.Sum()) - } - recordedStmtStats := sqlstats.RecordedStmtStats{ SessionID: ex.sessionID, StatementID: planner.stmt.QueryID, diff --git a/pkg/sql/instrumentation.go b/pkg/sql/instrumentation.go index 4853b398d91a..c4c334abcc9b 100644 --- a/pkg/sql/instrumentation.go +++ b/pkg/sql/instrumentation.go @@ -300,16 +300,6 @@ func (ih *instrumentationHelper) Finish( Failed: retErr != nil, PlanHash: ih.planGist.Hash(), } - // We populate transaction fingerprint ID if this is an implicit transaction. - // See executor_statement_metrics.go:recordStatementSummary() for further - // explanation. - if ih.implicitTxn { - stmtFingerprintID := stmtStatsKey.FingerprintID() - txnFingerprintHash := util.MakeFNV64() - txnFingerprintHash.Add(uint64(stmtFingerprintID)) - stmtStatsKey.TransactionFingerprintID = - roachpb.TransactionFingerprintID(txnFingerprintHash.Sum()) - } err = statsCollector.RecordStatementExecStats(stmtStatsKey, queryLevelStats) if err != nil { if log.V(2 /* level */) { diff --git a/pkg/sql/logictest/testdata/logic_test/statement_statistics b/pkg/sql/logictest/testdata/logic_test/statement_statistics index 0fa30811b42e..5aaebf4a4a6e 100644 --- a/pkg/sql/logictest/testdata/logic_test/statement_statistics +++ b/pkg/sql/logictest/testdata/logic_test/statement_statistics @@ -8,10 +8,16 @@ SET CLUSTER SETTING sql.stats.flush.enabled = false; # Check that node_statement_statistics report per application statement ok -SET application_name = hello; SELECT 1 +SET application_name = hello statement ok -SET application_name = world; SELECT 2 +SELECT 1 + +statement ok +SET application_name = world + +statement ok +SELECT 2 query B SELECT count > 0 FROM crdb_internal.node_statement_statistics WHERE application_name IN ('hello', 'world') @@ -44,6 +50,25 @@ SELECT key, count >= 1 FROM crdb_internal.node_statement_statistics WHERE applic SELECT _ true SELECT _, _ true +# Check that multiple stmts in a simple query are executed as one txn. + +statement ok +SET application_name = multi_stmts_test; + +statement ok +select 1, 2; select 1, 2, 3; select 'ok' + +statement ok +SET application_name = '' + +query T +SELECT txn_fingerprint_id FROM crdb_internal.node_statement_statistics WHERE application_name = 'multi_stmts_test' ORDER BY txn_fingerprint_id +---- +10413021493801724718 +10413021493801724718 +10413021493801724718 +17854018046052698166 + statement ok CREATE TABLE test(x INT, y INT, z INT); INSERT INTO test(x, y, z) VALUES (0,0,0); @@ -121,7 +146,10 @@ statement ok SHOW CLUSTER SETTING debug.panic_on_failed_assertions statement ok -SET application_name = ''; RESET distsql +SET application_name = ''; + +statement ok +RESET distsql skipif config 3node-tenant-default-configs #52763 query TT colnames @@ -142,7 +170,7 @@ SELECT x FROM test WHERE y IN (_, _, __more3__) + SELECT x FROM test WHERE y NOT IN (_, _, __more3__) · SET CLUSTER SETTING "debug.panic_on_failed_assertions" = DEFAULT · SET CLUSTER SETTING "debug.panic_on_failed_assertions" = _ · -SET application_name = '_' · +SET application_name = '_' · SET distsql = "on" · SHOW CLUSTER SETTING "debug.panic_on_failed_assertions" · SHOW application_name · @@ -204,6 +232,10 @@ SET application_name = 'implicit_txn_test' statement ok BEGIN; SELECT x FROM test where y=1; COMMIT; +# Upgraded implicit txn. +statement ok +select 1; BEGIN; select 1; select 1; COMMIT + statement ok BEGIN; SELECT x, z FROM test; @@ -222,6 +254,7 @@ SELECT key, implicit_txn WHERE application_name = 'implicit_txn_test' ORDER BY key, implicit_txn; ---- key implicit_txn +SELECT _ false SELECT x FROM test WHERE y = _ false SELECT x FROM test WHERE y = _ false SELECT x FROM test WHERE y = _ true diff --git a/pkg/sql/sqlstats/sslocal/sql_stats_test.go b/pkg/sql/sqlstats/sslocal/sql_stats_test.go index 351c50c60f30..dc49fad06951 100644 --- a/pkg/sql/sqlstats/sslocal/sql_stats_test.go +++ b/pkg/sql/sqlstats/sslocal/sql_stats_test.go @@ -456,13 +456,9 @@ func TestExplicitTxnFingerprintAccounting(t *testing.T) { recordStats := func(testCase *tc) { var txnFingerprintID roachpb.TransactionFingerprintID txnFingerprintIDHash := util.MakeFNV64() - if !testCase.implicit { - statsCollector.StartExplicitTransaction() - } + statsCollector.StartTransaction() defer func() { - if !testCase.implicit { - statsCollector.EndExplicitTransaction(ctx, txnFingerprintID) - } + statsCollector.EndTransaction(ctx, txnFingerprintID) require.NoError(t, statsCollector. RecordTransaction(ctx, txnFingerprintID, sqlstats.RecordedTxnStats{})) @@ -568,7 +564,7 @@ func TestAssociatingStmtStatsWithTxnFingerprint(t *testing.T) { for _, txn := range simulatedTxns { // Collect stats for the simulated transaction. txnFingerprintIDHash := util.MakeFNV64() - statsCollector.StartExplicitTransaction() + statsCollector.StartTransaction() for _, fingerprint := range txn.stmtFingerprints { stmtFingerprintID, err := statsCollector.RecordStatement( @@ -581,7 +577,7 @@ func TestAssociatingStmtStatsWithTxnFingerprint(t *testing.T) { } transactionFingerprintID := roachpb.TransactionFingerprintID(txnFingerprintIDHash.Sum()) - statsCollector.EndExplicitTransaction(ctx, transactionFingerprintID) + statsCollector.EndTransaction(ctx, transactionFingerprintID) err := statsCollector.RecordTransaction(ctx, transactionFingerprintID, sqlstats.RecordedTxnStats{}) require.NoError(t, err) diff --git a/pkg/sql/sqlstats/sslocal/sslocal_stats_collector.go b/pkg/sql/sqlstats/sslocal/sslocal_stats_collector.go index 910578c0a391..8d734d33fa27 100644 --- a/pkg/sql/sqlstats/sslocal/sslocal_stats_collector.go +++ b/pkg/sql/sqlstats/sslocal/sslocal_stats_collector.go @@ -67,24 +67,21 @@ func (s *StatsCollector) PreviousPhaseTimes() *sessionphase.Times { // Reset implements sqlstats.StatsCollector interface. func (s *StatsCollector) Reset(appStats sqlstats.ApplicationStats, phaseTime *sessionphase.Times) { previousPhaseTime := s.phaseTimes - if s.isInExplicitTransaction() { - s.flushTarget = appStats - } else { - s.ApplicationStats = appStats - } + s.flushTarget = appStats s.previousPhaseTimes = previousPhaseTime s.phaseTimes = phaseTime.Clone() } -// StartExplicitTransaction implements sqlstats.StatsCollector interface. -func (s *StatsCollector) StartExplicitTransaction() { +// StartTransaction implements sqlstats.StatsCollector interface. +// The current application stats are reset for the new transaction. +func (s *StatsCollector) StartTransaction() { s.flushTarget = s.ApplicationStats s.ApplicationStats = s.flushTarget.NewApplicationStatsWithInheritedOptions() } -// EndExplicitTransaction implements sqlstats.StatsCollector interface. -func (s *StatsCollector) EndExplicitTransaction( +// EndTransaction implements sqlstats.StatsCollector interface. +func (s *StatsCollector) EndTransaction( ctx context.Context, transactionFingerprintID roachpb.TransactionFingerprintID, ) { // We possibly ignore the transactionFingerprintID, for situations where @@ -122,14 +119,23 @@ func (s *StatsCollector) EndExplicitTransaction( func (s *StatsCollector) ShouldSaveLogicalPlanDesc( fingerprint string, implicitTxn bool, database string, ) bool { - if s.isInExplicitTransaction() { - return s.flushTarget.ShouldSaveLogicalPlanDesc(fingerprint, implicitTxn, database) && - s.ApplicationStats.ShouldSaveLogicalPlanDesc(fingerprint, implicitTxn, database) + foundInFlushTarget := true + + if s.flushTarget != nil { + foundInFlushTarget = s.flushTarget.ShouldSaveLogicalPlanDesc(fingerprint, implicitTxn, database) } - return s.ApplicationStats.ShouldSaveLogicalPlanDesc(fingerprint, implicitTxn, database) + return foundInFlushTarget && + s.ApplicationStats.ShouldSaveLogicalPlanDesc(fingerprint, implicitTxn, database) } -func (s *StatsCollector) isInExplicitTransaction() bool { - return s.flushTarget != nil +// UpgradeImplicitTxn implements sqlstats.StatsCollector interface. +func (s *StatsCollector) UpgradeImplicitTxn(ctx context.Context) error { + err := s.ApplicationStats.IterateStatementStats(ctx, &sqlstats.IteratorOptions{}, + func(_ context.Context, statistics *roachpb.CollectedStatementStatistics) error { + statistics.Key.ImplicitTxn = false + return nil + }) + + return err } diff --git a/pkg/sql/sqlstats/ssprovider.go b/pkg/sql/sqlstats/ssprovider.go index e6cbd5660881..791c717ec2c5 100644 --- a/pkg/sql/sqlstats/ssprovider.go +++ b/pkg/sql/sqlstats/ssprovider.go @@ -148,17 +148,20 @@ type StatsCollector interface { // of the sessionphase.Times. Reset(ApplicationStats, *sessionphase.Times) - // StartExplicitTransaction informs StatsCollector that all subsequent - // statements will be executed in the context an explicit transaction. - StartExplicitTransaction() + // StartTransaction sets up the StatsCollector for a new transaction. + StartTransaction() - // EndExplicitTransaction informs the StatsCollector that the explicit txn has + // EndTransaction informs the StatsCollector that the current txn has // finished execution. (Either COMMITTED or ABORTED). This means the txn's // fingerprint ID is now available. StatsCollector will now go back to update // the transaction fingerprint ID field of all the statement statistics for that // txn. - EndExplicitTransaction(ctx context.Context, transactionFingerprintID roachpb.TransactionFingerprintID, - ) + EndTransaction(ctx context.Context, transactionFingerprintID roachpb.TransactionFingerprintID) + + // UpgradeImplicitTxn informs the StatsCollector that the current txn has been + // upgraded to an explicit transaction, thus all previously recorded statements + // should be updated accordingly. + UpgradeImplicitTxn(ctx context.Context) error } // Storage provides clients with interface to perform read and write operations