Skip to content

Commit 879f32a

Browse files
craig[bot]miraradevamsbutlerxinhaoz
committed
142982: kvserver,raft: reduce leader lease logging r=iskettaneh a=miraradeva Under some circumstances (e.g. successive splitting, overload), certain Raft and store liveness logging is excessive and might appear worrisome even though it is harmless or expected. This commit pushes some of that logging to a higher verbosity. Fixes: cockroachdb#141854 Release note: None 143007: upgrades: skip creating defaultdb/postgres dbs during PCR reader tenant startup r=jeffswenson a=msbutler Informs cockroachdb#142960 Informs: cockroachdb#141592 Release note: none 143027: sqlstats: set session meta fields directly on sqlstats.RecordedTxnStats r=xinhaoz a=xinhaoz This commit replaces the SessionData pointer field on sqlstas.RecordedTxnStats with direct Application and UserNormalized fields for simpler access to session metadata. Epic: none Release note: None Co-authored-by: Mira Radeva <[email protected]> Co-authored-by: Michael Butler <[email protected]> Co-authored-by: Xin Hao Zhang <[email protected]>
4 parents d729d9b + d50947f + 1dd0a72 + 6bf8615 commit 879f32a

16 files changed

+59
-46
lines changed

pkg/kv/kvserver/replica_raft.go

Lines changed: 3 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -2220,7 +2220,9 @@ func (r *Replica) maybeMarkReplicaUnavailableInLeaderlessWatcher(
22202220
err := errors.Errorf("have been leaderless for %.2fs, setting the "+
22212221
"leaderless watcher replica's state as unavailable",
22222222
durationSinceLeaderless.Seconds())
2223-
log.Warningf(ctx, "%s", err)
2223+
if log.ExpensiveLogEnabled(ctx, 1) {
2224+
log.VEventf(ctx, 1, "%s", err)
2225+
}
22242226
r.LeaderlessWatcher.mu.unavailable = true
22252227
}
22262228
}

pkg/kv/kvserver/replica_store_liveness.go

Lines changed: 6 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -63,7 +63,9 @@ func (r *replicaRLockedStoreLiveness) SupportFor(replicaID raftpb.PeerID) (raftp
6363
storeID, ok := r.getStoreIdent(replicaID)
6464
if !ok {
6565
ctx := r.AnnotateCtx(context.TODO())
66-
log.Warningf(ctx, "store not found for replica %d in SupportFor", replicaID)
66+
if log.ExpensiveLogEnabled(ctx, 1) {
67+
log.VEventf(ctx, 1, "store not found for replica %d in SupportFor", replicaID)
68+
}
6769
return 0, false
6870
}
6971
epoch, ok := r.store.storeLiveness.SupportFor(storeID)
@@ -77,7 +79,9 @@ func (r *replicaRLockedStoreLiveness) SupportFrom(
7779
storeID, ok := r.getStoreIdent(replicaID)
7880
if !ok {
7981
ctx := r.AnnotateCtx(context.TODO())
80-
log.Warningf(ctx, "store not found for replica %d in SupportFrom", replicaID)
82+
if log.ExpensiveLogEnabled(ctx, 1) {
83+
log.VEventf(ctx, 1, "store not found for replica %d in SupportFrom", replicaID)
84+
}
8185
return 0, hlc.Timestamp{}
8286
}
8387
epoch, exp := r.store.storeLiveness.SupportFrom(storeID)

pkg/raft/raft.go

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -2356,7 +2356,7 @@ func (r *raft) checkQuorumActive() {
23562356
r.logger.Debugf("%x does not have store liveness support from a quorum of peers", r.id)
23572357
}
23582358
if !quorumActiveByHeartbeats && !quorumActiveByFortification {
2359-
r.logger.Warningf("%x stepped down to follower since quorum is not active", r.id)
2359+
r.logger.Infof("%x stepped down to follower since quorum is not active", r.id)
23602360
r.becomeFollower(r.Term, None)
23612361
}
23622362
// Mark everyone (but ourselves) as inactive in preparation for the next

pkg/raft/testdata/de_fortification_checkquorum.txt

Lines changed: 2 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -68,7 +68,7 @@ INFO 1 leader at term 1 does not support itself in the liveness fabric
6868
INFO 1 leader at term 1 does not support itself in the liveness fabric
6969
DEBUG 1 has not received messages from a quorum of peers in the last election timeout
7070
DEBUG 1 does not have store liveness support from a quorum of peers
71-
WARN 1 stepped down to follower since quorum is not active
71+
INFO 1 stepped down to follower since quorum is not active
7272
DEBUG 1 setting election elapsed to start from 3 ticks after store liveness support expired
7373
INFO 1 became follower at term 1
7474
DEBUG 1 reset election elapsed to 0
@@ -232,7 +232,7 @@ INFO 2 leader at term 2 does not support itself in the liveness fabric
232232
INFO 2 leader at term 2 does not support itself in the liveness fabric
233233
DEBUG 2 has not received messages from a quorum of peers in the last election timeout
234234
DEBUG 2 does not have store liveness support from a quorum of peers
235-
WARN 2 stepped down to follower since quorum is not active
235+
INFO 2 stepped down to follower since quorum is not active
236236
DEBUG 2 setting election elapsed to start from 3 ticks after store liveness support expired
237237
INFO 2 became follower at term 2
238238
DEBUG 2 reset election elapsed to 0

pkg/raft/testdata/fortification_checkquorum.txt

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -84,7 +84,7 @@ INFO 1 leader at term 1 does not support itself in the liveness fabric
8484
INFO 1 leader at term 1 does not support itself in the liveness fabric
8585
DEBUG 1 has not received messages from a quorum of peers in the last election timeout
8686
DEBUG 1 does not have store liveness support from a quorum of peers
87-
WARN 1 stepped down to follower since quorum is not active
87+
INFO 1 stepped down to follower since quorum is not active
8888
DEBUG 1 setting election elapsed to start from 3 ticks after store liveness support expired
8989
INFO 1 became follower at term 1
9090
DEBUG 1 reset election elapsed to 0

pkg/raft/testdata/liveness_recovered_after_temporary_loss_of_quorum.txt

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -65,7 +65,7 @@ tick-election 1
6565
----
6666
INFO 1 leader at term 1 does not support itself in the liveness fabric
6767
INFO 1 leader at term 1 does not support itself in the liveness fabric
68-
WARN 1 stepped down to follower since quorum is not active
68+
INFO 1 stepped down to follower since quorum is not active
6969
INFO 1 became follower at term 1
7070

7171
stabilize

pkg/sql/conn_executor_exec.go

Lines changed: 3 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -4356,8 +4356,9 @@ func (ex *connExecutor) recordTransactionFinish(
43564356
// TODO(107318): add qos
43574357
// TODO(107318): add asoftime or ishistorical
43584358
// TODO(107318): add readonly
4359-
SessionData: ex.sessionData(),
4360-
TxnErr: txnErr,
4359+
TxnErr: txnErr,
4360+
Application: ex.applicationName.Load().(string),
4361+
UserNormalized: ex.sessionData().User().Normalized(),
43614362
}
43624363

43634364
if ex.server.cfg.TestingKnobs.OnRecordTxnFinish != nil {

pkg/sql/exec_log.go

Lines changed: 2 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -471,8 +471,8 @@ func (p *planner) logTransaction(
471471

472472
*sampledTxn = eventpb.SampledTransaction{
473473
SkippedTransactions: int64(skippedTransactions),
474-
User: txnStats.SessionData.SessionUser().Normalized(),
475-
ApplicationName: txnStats.SessionData.ApplicationName,
474+
User: txnStats.UserNormalized,
475+
ApplicationName: txnStats.Application,
476476
TxnCounter: uint32(txnCounter),
477477
SessionID: txnStats.SessionID.String(),
478478
TransactionID: txnStats.TransactionID.String(),

pkg/sql/instrumentation_test.go

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -215,7 +215,7 @@ func TestSampledStatsCollectionOnNewFingerprint(t *testing.T) {
215215
OnRecordTxnFinish: func(isInternal bool, _ *sessionphase.Times, stmt string, txnStats sqlstats.RecordedTxnStats) {
216216
// We won't run into a race here because we'll only observe
217217
// txns from a single connection.
218-
if txnStats.SessionData.ApplicationName == testApp {
218+
if txnStats.Application == testApp {
219219
if strings.Contains(stmt, `SET application_name`) {
220220
return
221221
}

pkg/sql/sqlstats/BUILD.bazel

Lines changed: 0 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -20,7 +20,6 @@ go_library(
2020
"//pkg/sql/clusterunique",
2121
"//pkg/sql/execstats",
2222
"//pkg/sql/sem/tree",
23-
"//pkg/sql/sessiondata",
2423
"//pkg/sql/sqlstats/persistedsqlstats/sqlstatsutil",
2524
"//pkg/util/log",
2625
"//pkg/util/metric",

pkg/sql/sqlstats/insights/util.go

Lines changed: 2 additions & 8 deletions
Original file line numberDiff line numberDiff line change
@@ -36,12 +36,6 @@ func MakeTxnInsight(value sqlstats.RecordedTxnStats) *Transaction {
3636
status = Transaction_Completed
3737
}
3838

39-
var user, appName string
40-
if value.SessionData != nil {
41-
user = value.SessionData.User().Normalized()
42-
appName = value.SessionData.ApplicationName
43-
}
44-
4539
insight := &Transaction{
4640
ID: value.TransactionID,
4741
FingerprintID: value.FingerprintID,
@@ -50,8 +44,8 @@ func MakeTxnInsight(value sqlstats.RecordedTxnStats) *Transaction {
5044
Contention: &value.ExecStats.ContentionTime,
5145
StartTime: value.StartTime,
5246
EndTime: value.EndTime,
53-
User: user,
54-
ApplicationName: appName,
47+
User: value.UserNormalized,
48+
ApplicationName: value.Application,
5549
RowsRead: value.RowsRead,
5650
RowsWritten: value.RowsWritten,
5751
RetryCount: value.RetryCount,

pkg/sql/sqlstats/sslocal/BUILD.bazel

Lines changed: 0 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -63,7 +63,6 @@ go_test(
6363
"//pkg/sql/sem/catconstants",
6464
"//pkg/sql/sem/tree",
6565
"//pkg/sql/sessiondata",
66-
"//pkg/sql/sessiondatapb",
6766
"//pkg/sql/sessionphase",
6867
"//pkg/sql/sqlstats",
6968
"//pkg/sql/sqlstats/insights",

pkg/sql/sqlstats/sslocal/sql_stats_test.go

Lines changed: 8 additions & 20 deletions
Original file line numberDiff line numberDiff line change
@@ -30,7 +30,6 @@ import (
3030
"github.com/cockroachdb/cockroach/pkg/sql/sem/catconstants"
3131
"github.com/cockroachdb/cockroach/pkg/sql/sem/tree"
3232
"github.com/cockroachdb/cockroach/pkg/sql/sessiondata"
33-
"github.com/cockroachdb/cockroach/pkg/sql/sessiondatapb"
3433
"github.com/cockroachdb/cockroach/pkg/sql/sessionphase"
3534
"github.com/cockroachdb/cockroach/pkg/sql/sqlstats"
3635
"github.com/cockroachdb/cockroach/pkg/sql/sqlstats/insights"
@@ -474,17 +473,11 @@ func TestExplicitTxnFingerprintAccounting(t *testing.T) {
474473
defer func() {
475474
statsCollector.EndTransaction(ctx, txnFingerprintID)
476475
require.NoError(t,
477-
statsCollector.
478-
RecordTransaction(ctx, sqlstats.RecordedTxnStats{
479-
FingerprintID: txnFingerprintID,
480-
SessionData: &sessiondata.SessionData{
481-
SessionData: sessiondatapb.SessionData{
482-
UserProto: username.RootUserName().EncodeProto(),
483-
Database: "defaultdb",
484-
ApplicationName: "appname_findme",
485-
},
486-
},
487-
}))
476+
statsCollector.RecordTransaction(ctx, sqlstats.RecordedTxnStats{
477+
FingerprintID: txnFingerprintID,
478+
UserNormalized: username.RootUser,
479+
Application: "appname_findme",
480+
}))
488481
}()
489482
for _, fingerprint := range testCase.fingerprints {
490483
stmtFingerprintID := appstatspb.ConstructStatementFingerprintID(fingerprint, testCase.implicit, "defaultdb")
@@ -609,14 +602,9 @@ func TestAssociatingStmtStatsWithTxnFingerprint(t *testing.T) {
609602
transactionFingerprintID := appstatspb.TransactionFingerprintID(txnFingerprintIDHash.Sum())
610603
statsCollector.EndTransaction(ctx, transactionFingerprintID)
611604
err := statsCollector.RecordTransaction(ctx, sqlstats.RecordedTxnStats{
612-
FingerprintID: transactionFingerprintID,
613-
SessionData: &sessiondata.SessionData{
614-
SessionData: sessiondatapb.SessionData{
615-
UserProto: username.RootUserName().EncodeProto(),
616-
Database: "defaultdb",
617-
ApplicationName: "appname_findme",
618-
},
619-
},
605+
FingerprintID: transactionFingerprintID,
606+
UserNormalized: username.RootUser,
607+
Application: "appname_findme",
620608
})
621609
require.NoError(t, err)
622610

pkg/sql/sqlstats/ssprovider.go

Lines changed: 3 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -17,7 +17,6 @@ import (
1717
"github.com/cockroachdb/cockroach/pkg/sql/clusterunique"
1818
"github.com/cockroachdb/cockroach/pkg/sql/execstats"
1919
"github.com/cockroachdb/cockroach/pkg/sql/sem/tree"
20-
"github.com/cockroachdb/cockroach/pkg/sql/sessiondata"
2120
"github.com/cockroachdb/cockroach/pkg/util/uuid"
2221
)
2322

@@ -119,8 +118,10 @@ type RecordedTxnStats struct {
119118
RowsWritten int64
120119
BytesRead int64
121120
Priority roachpb.UserPriority
122-
SessionData *sessiondata.SessionData
123121
TxnErr error
122+
Application string
123+
// Normalized user name.
124+
UserNormalized string
124125
}
125126

126127
// SSDrainer is the interface for draining or resetting sql stats.

pkg/upgrade/upgrades/permanent_maybe_setup_pcr_reader.go

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -23,7 +23,7 @@ func maybeSetupPCRStandbyReader(
2323
if d.TenantInfoAccessor == nil {
2424
return nil
2525
}
26-
id, ts, err := d.TenantInfoAccessor.ReadFromTenantInfo(ctx)
26+
id, ts, err := readerTenantInfo(ctx, d)
2727
if err != nil {
2828
return err
2929
}

pkg/upgrade/upgrades/permanent_upgrades.go

Lines changed: 25 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -20,6 +20,7 @@ import (
2020
"github.com/cockroachdb/cockroach/pkg/sql/sem/tree"
2121
"github.com/cockroachdb/cockroach/pkg/sql/sessiondata"
2222
"github.com/cockroachdb/cockroach/pkg/upgrade"
23+
"github.com/cockroachdb/cockroach/pkg/util/hlc"
2324
"github.com/cockroachdb/cockroach/pkg/util/log"
2425
"github.com/cockroachdb/cockroach/pkg/util/protoutil"
2526
"github.com/cockroachdb/errors"
@@ -254,6 +255,12 @@ func createDefaultDbs(
254255
// already.
255256
const createDbStmt = `CREATE DATABASE IF NOT EXISTS "%s" WITH OWNER root`
256257

258+
id, _, _ := readerTenantInfo(ctx, deps)
259+
if id.IsSet() {
260+
// Don't create the default databases for read from standby tenants.
261+
return nil
262+
}
263+
257264
var err error
258265
for _, dbName := range []string{catalogkeys.DefaultDatabaseName, catalogkeys.PgDatabaseName} {
259266
stmt := fmt.Sprintf(createDbStmt, dbName)
@@ -265,3 +272,21 @@ func createDefaultDbs(
265272
}
266273
return nil
267274
}
275+
276+
// readerTenantInfo returns the tenant ID and timestamp if we're spinning up a
277+
// read from standby tenant.
278+
func readerTenantInfo(
279+
ctx context.Context, d upgrade.TenantDeps,
280+
) (roachpb.TenantID, hlc.Timestamp, error) {
281+
if d.TenantInfoAccessor == nil {
282+
return roachpb.TenantID{}, hlc.Timestamp{}, nil
283+
}
284+
id, ts, err := d.TenantInfoAccessor.ReadFromTenantInfo(ctx)
285+
if err != nil {
286+
return roachpb.TenantID{}, hlc.Timestamp{}, err
287+
}
288+
if !id.IsSet() {
289+
return roachpb.TenantID{}, hlc.Timestamp{}, nil
290+
}
291+
return id, ts, nil
292+
}

0 commit comments

Comments
 (0)