Skip to content

Commit

Permalink
Browse files Browse the repository at this point in the history
136820: cliccl: use latest version for the TestList r=RaduBerinde a=RaduBerinde

We were using `MakeClusterSettings()` which initializes the cluster at
the minimum supported version. Switching to
`MakeTestingClusterSettings()` which initializes at the latest
version. It makes more sense for the test to change when the latest
version changes.

Epic: none
Release note: None

136911: sql: use crtime.Mono for phase times r=RaduBerinde a=RaduBerinde

#### go.mod: update crlib dependency

Epic: none
Release note: None

#### sql/sessionphase: separate SessionInit time

The `SessionInit` is not really a phase, it is set just once.
This change separates it out from the phase times (which are used to
measure elapsed times).

Epic: none
Release note: None

#### sql: use crtime.Mono for phase times

Using just the monotonic timer is faster. When we need to convert to a
real `time.Time`, we use `now` as the reference time.

Informs: #133315
Release note: None

Co-authored-by: Radu Berinde <[email protected]>
  • Loading branch information
craig[bot] and RaduBerinde committed Dec 9, 2024
3 parents 4be77ab + 7b563ad + 7d01a1b commit 4c1602e
Show file tree
Hide file tree
Showing 22 changed files with 131 additions and 112 deletions.
6 changes: 3 additions & 3 deletions DEPS.bzl
Original file line number Diff line number Diff line change
Expand Up @@ -1765,10 +1765,10 @@ def go_deps():
name = "com_github_cockroachdb_crlib",
build_file_proto_mode = "disable_global",
importpath = "github.com/cockroachdb/crlib",
sha256 = "4e55a6088c146af4bda503535b067a9c5c980449e0f185a48b5e1e06606d0ef8",
strip_prefix = "github.com/cockroachdb/[email protected]20241112164430-1264a2edc35b",
sha256 = "1afc910b4ff270de79eecb42ab7bd5e6404e6128666c6c55e96db9e27d28e69e",
strip_prefix = "github.com/cockroachdb/[email protected]20241205160938-4a90b184f49c",
urls = [
"https://storage.googleapis.com/cockroach-godeps/gomod/github.com/cockroachdb/crlib/com_github_cockroachdb_crlib-v0.0.0-20241112164430-1264a2edc35b.zip",
"https://storage.googleapis.com/cockroach-godeps/gomod/github.com/cockroachdb/crlib/com_github_cockroachdb_crlib-v0.0.0-20241205160938-4a90b184f49c.zip",
],
)
go_repository(
Expand Down
2 changes: 1 addition & 1 deletion build/bazelutil/distdir_files.bzl
Original file line number Diff line number Diff line change
Expand Up @@ -346,7 +346,7 @@ DISTDIR_FILES = {
"https://storage.googleapis.com/cockroach-godeps/gomod/github.com/cockroachdb/cmux/com_github_cockroachdb_cmux-v0.0.0-20170110192607-30d10be49292.zip": "88f6f9cf33eb535658540b46f6222f029398e590a3ff9cc873d7d561ac6debf0",
"https://storage.googleapis.com/cockroach-godeps/gomod/github.com/cockroachdb/cockroach-go/v2/com_github_cockroachdb_cockroach_go_v2-v2.3.7.zip": "028c29c79c2d373bca3ce9a475291285fdcb68a2f908190f738d5ce605edbd07",
"https://storage.googleapis.com/cockroach-godeps/gomod/github.com/cockroachdb/crlfmt/com_github_cockroachdb_crlfmt-v0.0.0-20221214225007-b2fc5c302548.zip": "fedc01bdd6d964da0425d5eaac8efadc951e78e13f102292cc0774197f09ab63",
"https://storage.googleapis.com/cockroach-godeps/gomod/github.com/cockroachdb/crlib/com_github_cockroachdb_crlib-v0.0.0-20241112164430-1264a2edc35b.zip": "4e55a6088c146af4bda503535b067a9c5c980449e0f185a48b5e1e06606d0ef8",
"https://storage.googleapis.com/cockroach-godeps/gomod/github.com/cockroachdb/crlib/com_github_cockroachdb_crlib-v0.0.0-20241205160938-4a90b184f49c.zip": "1afc910b4ff270de79eecb42ab7bd5e6404e6128666c6c55e96db9e27d28e69e",
"https://storage.googleapis.com/cockroach-godeps/gomod/github.com/cockroachdb/datadriven/com_github_cockroachdb_datadriven-v1.0.3-0.20240530155848-7682d40af056.zip": "f4cb70fec2b2904a56bfbda6a6c8bf9ea1d568a5994ecdb825f770671119b63b",
"https://storage.googleapis.com/cockroach-godeps/gomod/github.com/cockroachdb/errors/com_github_cockroachdb_errors-v1.11.3.zip": "d11ed59d96afef2d1f0ce56892839c62ff5c0cbca8dff0aaefeaef7eb190e73c",
"https://storage.googleapis.com/cockroach-godeps/gomod/github.com/cockroachdb/go-test-teamcity/com_github_cockroachdb_go_test_teamcity-v0.0.0-20191211140407-cff980ad0a55.zip": "bac30148e525b79d004da84d16453ddd2d5cd20528e9187f1d7dac708335674b",
Expand Down
2 changes: 1 addition & 1 deletion go.mod
Original file line number Diff line number Diff line change
Expand Up @@ -128,7 +128,7 @@ require (
github.com/cockroachdb/cmux v0.0.0-20170110192607-30d10be49292
github.com/cockroachdb/cockroach-go/v2 v2.3.7
github.com/cockroachdb/crlfmt v0.0.0-20221214225007-b2fc5c302548
github.com/cockroachdb/crlib v0.0.0-20241112164430-1264a2edc35b
github.com/cockroachdb/crlib v0.0.0-20241205160938-4a90b184f49c
github.com/cockroachdb/datadriven v1.0.3-0.20240530155848-7682d40af056
github.com/cockroachdb/errors v1.11.3
github.com/cockroachdb/go-test-teamcity v0.0.0-20191211140407-cff980ad0a55
Expand Down
4 changes: 2 additions & 2 deletions go.sum
Original file line number Diff line number Diff line change
Expand Up @@ -525,8 +525,8 @@ github.com/cockroachdb/cockroach-go/v2 v2.3.7 h1:nq5GYDuA2zIR/kdLkVLTg7oHTw0UbGU
github.com/cockroachdb/cockroach-go/v2 v2.3.7/go.mod h1:1wNJ45eSXW9AnOc3skntW9ZUZz6gxrQK3cOj3rK+BC8=
github.com/cockroachdb/crlfmt v0.0.0-20221214225007-b2fc5c302548 h1:i0bnjanlWAvM50wHMT7EFyxlt5HQusznWrkwl+HBIsU=
github.com/cockroachdb/crlfmt v0.0.0-20221214225007-b2fc5c302548/go.mod h1:qtkxNlt5i3rrdirfJE/bQeW/IeLajKexErv7jEIV+Uc=
github.com/cockroachdb/crlib v0.0.0-20241112164430-1264a2edc35b h1:SHlYZ/bMx7frnmeqCu+xm0TCxXLzX3jQIVuFbnFGtFU=
github.com/cockroachdb/crlib v0.0.0-20241112164430-1264a2edc35b/go.mod h1:Gq51ZeKaFCXk6QwuGM0w1dnaOqc/F5zKT2zA9D6Xeac=
github.com/cockroachdb/crlib v0.0.0-20241205160938-4a90b184f49c h1:9d6UHu3JQXisFa4JRFzCbX2Ba2ES5e0202N9qpt3VuM=
github.com/cockroachdb/crlib v0.0.0-20241205160938-4a90b184f49c/go.mod h1:Gq51ZeKaFCXk6QwuGM0w1dnaOqc/F5zKT2zA9D6Xeac=
github.com/cockroachdb/datadriven v0.0.0-20190809214429-80d97fb3cbaa/go.mod h1:zn76sxSg3SzpJ0PPJaLDCu+Bu0Lg3sKTORVIj19EIF8=
github.com/cockroachdb/datadriven v1.0.2/go.mod h1:a9RdTaap04u637JoCzcUoIcDmvwSUtcUFtT/C3kJlTU=
github.com/cockroachdb/datadriven v1.0.3-0.20240530155848-7682d40af056 h1:slXychO2uDM6hYRu4c0pD0udNI8uObfeKN6UInWViS8=
Expand Down
2 changes: 1 addition & 1 deletion pkg/ccl/cliccl/ear_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -137,7 +137,7 @@ func TestList(t *testing.T) {
require.NoError(t, err)
env, err := fs.InitEnv(ctx, vfs.Default, dir, fs.EnvConfig{EncryptionOptions: encOpts}, nil /* statsCollector */)
require.NoError(t, err)
p, err := storage.Open(ctx, env, cluster.MakeClusterSettings())
p, err := storage.Open(ctx, env, cluster.MakeTestingClusterSettings())
require.NoError(t, err)

// Write a key and flush, to create a table in the store.
Expand Down
1 change: 1 addition & 0 deletions pkg/sql/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -596,6 +596,7 @@ go_library(
"//pkg/util/uint128",
"//pkg/util/uuid",
"@com_github_cockroachdb_apd_v3//:apd",
"@com_github_cockroachdb_crlib//crtime",
"@com_github_cockroachdb_errors//:errors",
"@com_github_cockroachdb_errors//hintdetail",
"@com_github_cockroachdb_logtags//:logtags",
Expand Down
51 changes: 25 additions & 26 deletions pkg/sql/conn_executor.go
Original file line number Diff line number Diff line change
Expand Up @@ -83,6 +83,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/util/syncutil"
"github.com/cockroachdb/cockroach/pkg/util/timeutil"
"github.com/cockroachdb/cockroach/pkg/util/tochar"
"github.com/cockroachdb/crlib/crtime"
"github.com/cockroachdb/errors"
"github.com/cockroachdb/logtags"
"github.com/cockroachdb/redact"
Expand Down Expand Up @@ -1162,8 +1163,6 @@ func (s *Server) newConnExecutor(
ex.applicationStats = ex.server.sqlStats.GetApplicationStats(newName)
}

ex.phaseTimes.SetSessionPhaseTime(sessionphase.SessionInit, timeutil.Now())

ex.extraTxnState.underOuterTxn = underOuterTxn
ex.extraTxnState.prepStmtsNamespace = prepStmtNamespace{
prepStmts: make(map[string]*PreparedStatement),
Expand Down Expand Up @@ -1549,7 +1548,7 @@ type connExecutor struct {
// txnFinishClosure contains fields that ex.onTxnFinish uses to execute.
txnFinishClosure struct {
// txnStartTime is the time that the transaction started.
txnStartTime time.Time
txnStartTime crtime.Mono
// implicit is whether the transaction was implicit.
implicit bool
}
Expand Down Expand Up @@ -2358,7 +2357,7 @@ func (ex *connExecutor) execCmd() (retErr error) {
// - ex.statsCollector merely contains a copy of the times, that
// was created when the statement started executing (via the
// Reset() method).
ex.statsCollector.PhaseTimes().SetSessionPhaseTime(sessionphase.SessionQueryServiced, timeutil.Now())
ex.statsCollector.PhaseTimes().SetSessionPhaseTime(sessionphase.SessionQueryServiced, crtime.NowMono())
if err != nil {
return err
}
Expand All @@ -2371,8 +2370,8 @@ func (ex *connExecutor) execCmd() (retErr error) {
// 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.SetSessionPhaseTime(sessionphase.SessionStartParse, time.Time{})
ex.phaseTimes.SetSessionPhaseTime(sessionphase.SessionEndParse, time.Time{})
ex.phaseTimes.SetSessionPhaseTime(sessionphase.SessionStartParse, 0)
ex.phaseTimes.SetSessionPhaseTime(sessionphase.SessionEndParse, 0)
// We use a closure for the body of the execution so as to
// guarantee that the full service time is captured below.
err := func() error {
Expand Down Expand Up @@ -2470,7 +2469,7 @@ func (ex *connExecutor) execCmd() (retErr error) {
// reset() method).
// TODO(sql-sessions): fix the phase time for pausable portals.
// https://github.com/cockroachdb/cockroach/issues/99410
ex.statsCollector.PhaseTimes().SetSessionPhaseTime(sessionphase.SessionQueryServiced, timeutil.Now())
ex.statsCollector.PhaseTimes().SetSessionPhaseTime(sessionphase.SessionQueryServiced, crtime.NowMono())
if err != nil {
return err
}
Expand Down Expand Up @@ -2543,7 +2542,7 @@ func (ex *connExecutor) execCmd() (retErr error) {
// - ex.statsCollector merely contains a copy of the times, that
// was created when the statement started executing (via the
// reset() method).
ex.statsCollector.PhaseTimes().SetSessionPhaseTime(sessionphase.SessionQueryServiced, timeutil.Now())
ex.statsCollector.PhaseTimes().SetSessionPhaseTime(sessionphase.SessionQueryServiced, crtime.NowMono())
case CopyOut:
ex.phaseTimes.SetSessionPhaseTime(sessionphase.SessionQueryReceived, tcmd.TimeReceived)
ex.phaseTimes.SetSessionPhaseTime(sessionphase.SessionStartParse, tcmd.ParseStart)
Expand All @@ -2559,7 +2558,7 @@ func (ex *connExecutor) execCmd() (retErr error) {
// - ex.statsCollector merely contains a copy of the times, that
// was created when the statement started executing (via the
// reset() method).
ex.statsCollector.PhaseTimes().SetSessionPhaseTime(sessionphase.SessionQueryServiced, timeutil.Now())
ex.statsCollector.PhaseTimes().SetSessionPhaseTime(sessionphase.SessionQueryServiced, crtime.NowMono())
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
Expand Down Expand Up @@ -3028,7 +3027,7 @@ func (ex *connExecutor) execCopyOut(

if ex.sessionData().StmtTimeout > 0 {
timerDuration :=
ex.sessionData().StmtTimeout - timeutil.Since(ex.phaseTimes.GetSessionPhaseTime(sessionphase.SessionQueryReceived))
ex.sessionData().StmtTimeout - ex.phaseTimes.GetSessionPhaseTime(sessionphase.SessionQueryReceived).Elapsed()
// There's no need to proceed with execution if the timer has already expired.
if timerDuration < 0 {
queryTimedOut = true
Expand All @@ -3045,7 +3044,7 @@ func (ex *connExecutor) execCopyOut(
}
if ex.sessionData().TransactionTimeout > 0 && !ex.implicitTxn() {
timerDuration :=
ex.sessionData().TransactionTimeout - timeutil.Since(ex.phaseTimes.GetSessionPhaseTime(sessionphase.SessionTransactionStarted))
ex.sessionData().TransactionTimeout - ex.phaseTimes.GetSessionPhaseTime(sessionphase.SessionTransactionStarted).Elapsed()

// If the timer already expired, but the transaction is not yet aborted,
// we should error immediately without executing. If the timer
Expand Down Expand Up @@ -3298,7 +3297,7 @@ func (ex *connExecutor) execCopyIn(

if ex.sessionData().StmtTimeout > 0 {
timerDuration :=
ex.sessionData().StmtTimeout - timeutil.Since(ex.phaseTimes.GetSessionPhaseTime(sessionphase.SessionQueryReceived))
ex.sessionData().StmtTimeout - ex.phaseTimes.GetSessionPhaseTime(sessionphase.SessionQueryReceived).Elapsed()
// There's no need to proceed with execution if the timer has already expired.
if timerDuration < 0 {
queryTimedOut = true
Expand All @@ -3315,7 +3314,7 @@ func (ex *connExecutor) execCopyIn(
}
if ex.sessionData().TransactionTimeout > 0 && !ex.implicitTxn() {
timerDuration :=
ex.sessionData().TransactionTimeout - timeutil.Since(ex.phaseTimes.GetSessionPhaseTime(sessionphase.SessionTransactionStarted))
ex.sessionData().TransactionTimeout - ex.phaseTimes.GetSessionPhaseTime(sessionphase.SessionTransactionStarted).Elapsed()

// If the timer already expired, but the transaction is not yet aborted,
// we should error immediately without executing. If the timer
Expand Down Expand Up @@ -4038,11 +4037,11 @@ func (ex *connExecutor) txnStateTransitionsApplyWrapper(
if err != nil {
return advanceInfo{}, err
}
ex.statsCollector.PhaseTimes().SetSessionPhaseTime(sessionphase.SessionStartPostCommitJob, timeutil.Now())
ex.statsCollector.PhaseTimes().SetSessionPhaseTime(sessionphase.SessionStartPostCommitJob, crtime.NowMono())
if err := ex.waitForTxnJobs(); err != nil {
handleErr(err)
}
ex.statsCollector.PhaseTimes().SetSessionPhaseTime(sessionphase.SessionEndPostCommitJob, timeutil.Now())
ex.statsCollector.PhaseTimes().SetSessionPhaseTime(sessionphase.SessionEndPostCommitJob, crtime.NowMono())
if err := ex.waitOneVersionForNewVersionDescriptorsWithoutJobs(descIDsInJobs); err != nil {
return advanceInfo{}, err
}
Expand Down Expand Up @@ -4087,7 +4086,7 @@ func (ex *connExecutor) waitForTxnJobs() error {
jobWaitCtx := ex.ctxHolder.ctx()
var queryTimedout atomic.Bool
if ex.sessionData().StmtTimeout > 0 {
timePassed := timeutil.Since(ex.phaseTimes.GetSessionPhaseTime(sessionphase.SessionQueryReceived))
timePassed := ex.phaseTimes.GetSessionPhaseTime(sessionphase.SessionQueryReceived).Elapsed()
if timePassed > ex.sessionData().StmtTimeout {
queryTimedout.Store(true)
} else {
Expand Down Expand Up @@ -4248,10 +4247,11 @@ func (ex *connExecutor) serialize() serverpb.Session {

if txn != nil {
id := txn.ID()
elapsedTime := crtime.MonoFromTime(timeNow).Sub(ex.state.mu.txnStart)
activeTxnInfo = &serverpb.TxnInfo{
ID: id,
Start: ex.state.mu.txnStart,
ElapsedTime: timeNow.Sub(ex.state.mu.txnStart),
Start: timeNow.Add(-elapsedTime),
ElapsedTime: elapsedTime,
NumStatementsExecuted: int32(ex.state.mu.stmtCount),
NumRetries: int32(txn.Epoch()),
NumAutoRetries: ex.state.mu.autoRetryCounter,
Expand Down Expand Up @@ -4315,12 +4315,12 @@ func (ex *connExecutor) serialize() serverpb.Session {
}
sql := truncateSQL(query.stmt.SQL)
progress := math.Float64frombits(atomic.LoadUint64(&query.progressAtomic))
queryStart := query.start.UTC()
elapsedTime := crtime.MonoFromTime(timeNow).Sub(query.start)
activeQueries = append(activeQueries, serverpb.ActiveQuery{
TxnID: query.txnID,
ID: id.String(),
Start: queryStart,
ElapsedTime: timeNow.Sub(queryStart),
Start: timeNow.Add(-elapsedTime),
ElapsedTime: elapsedTime,
Sql: sql,
SqlNoConstants: sqlNoConstants,
SqlSummary: formatStatementSummary(parsed.AST),
Expand Down Expand Up @@ -4360,11 +4360,10 @@ func (ex *connExecutor) serialize() serverpb.Session {
}

return serverpb.Session{
Username: sd.SessionUser().Normalized(),
ClientAddress: remoteStr,
ApplicationName: ex.applicationName.Load().(string),
// TODO(yuzefovich): this seems like not a concurrency safe call.
Start: ex.phaseTimes.GetSessionPhaseTime(sessionphase.SessionInit).UTC(),
Username: sd.SessionUser().Normalized(),
ClientAddress: remoteStr,
ApplicationName: ex.applicationName.Load().(string),
Start: ex.phaseTimes.InitTime(),
ActiveQueries: activeQueries,
ActiveTxn: activeTxnInfo,
NumTxnsExecuted: ex.extraTxnState.txnCounter.Load(),
Expand Down
Loading

0 comments on commit 4c1602e

Please sign in to comment.