Skip to content

Commit

Permalink
outliers: require session, statement, and transaction ids
Browse files Browse the repository at this point in the history
Release note: None
  • Loading branch information
matthewtodd committed Jul 6, 2022
1 parent c393aac commit ccb1eac
Show file tree
Hide file tree
Showing 5 changed files with 27 additions and 31 deletions.
2 changes: 1 addition & 1 deletion pkg/sql/crdb_internal.go
Original file line number Diff line number Diff line change
Expand Up @@ -6204,7 +6204,7 @@ CREATE TABLE crdb_internal.node_execution_outliers (
) {
err = errors.CombineErrors(err, addRow(
tree.NewDString(hex.EncodeToString(o.Session.ID.GetBytes())),
tree.NewDUuid(tree.DUuid{UUID: *o.Transaction.ID}),
tree.NewDUuid(tree.DUuid{UUID: o.Transaction.ID}),
tree.NewDString(hex.EncodeToString(o.Statement.ID.GetBytes())),
tree.NewDBytes(tree.DBytes(sqlstatsutil.EncodeUint64ToBytes(uint64(o.Statement.FingerprintID)))),
))
Expand Down
9 changes: 6 additions & 3 deletions pkg/sql/sqlstats/outliers/outliers.proto
Original file line number Diff line number Diff line change
Expand Up @@ -16,17 +16,20 @@ import "gogoproto/gogo.proto";

message Session {
bytes id = 1 [(gogoproto.customname) = "ID",
(gogoproto.customtype) = "github.com/cockroachdb/cockroach/pkg/sql/clusterunique.ID"];
(gogoproto.customtype) = "github.com/cockroachdb/cockroach/pkg/sql/clusterunique.ID",
(gogoproto.nullable) = false];
}

message Transaction {
bytes id = 1 [(gogoproto.customname) = "ID",
(gogoproto.customtype) = "github.com/cockroachdb/cockroach/pkg/util/uuid.UUID"];
(gogoproto.customtype) = "github.com/cockroachdb/cockroach/pkg/util/uuid.UUID",
(gogoproto.nullable) = false];
}

message Statement {
bytes id = 1 [(gogoproto.customname) = "ID",
(gogoproto.customtype) = "github.com/cockroachdb/cockroach/pkg/sql/clusterunique.ID"];
(gogoproto.customtype) = "github.com/cockroachdb/cockroach/pkg/sql/clusterunique.ID",
(gogoproto.nullable) = false];
uint64 fingerprint_id = 2 [(gogoproto.customname) = "FingerprintID",
(gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/roachpb.StmtFingerprintID"];
double latency_in_seconds = 3;
Expand Down
41 changes: 17 additions & 24 deletions pkg/sql/sqlstats/outliers/outliers_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -28,13 +28,10 @@ import (
func TestOutliers(t *testing.T) {
ctx := context.Background()

sessionID := clusterunique.IDFromBytes([]byte("aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa"))
session := &outliers.Session{ID: &sessionID}
txnID := uuid.FastMakeV4()
transaction := &outliers.Transaction{ID: &txnID}
statementID := clusterunique.IDFromBytes([]byte("bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb"))
session := &outliers.Session{ID: clusterunique.IDFromBytes([]byte("aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa"))}
transaction := &outliers.Transaction{ID: uuid.FastMakeV4()}
statement := &outliers.Statement{
ID: &statementID,
ID: clusterunique.IDFromBytes([]byte("bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb")),
FingerprintID: roachpb.StmtFingerprintID(100),
LatencyInSeconds: 2,
}
Expand All @@ -43,8 +40,8 @@ func TestOutliers(t *testing.T) {
st := cluster.MakeTestingClusterSettings()
outliers.LatencyThreshold.Override(ctx, &st.SV, 1*time.Second)
registry := outliers.New(st, outliers.NewMetrics())
registry.ObserveStatement(sessionID, statement)
registry.ObserveTransaction(sessionID, transaction)
registry.ObserveStatement(session.ID, statement)
registry.ObserveTransaction(session.ID, transaction)

expected := []*outliers.Outlier{{
Session: session,
Expand All @@ -67,8 +64,8 @@ func TestOutliers(t *testing.T) {
st := cluster.MakeTestingClusterSettings()
outliers.LatencyThreshold.Override(ctx, &st.SV, 0)
registry := outliers.New(st, outliers.NewMetrics())
registry.ObserveStatement(sessionID, statement)
registry.ObserveTransaction(sessionID, transaction)
registry.ObserveStatement(session.ID, statement)
registry.ObserveTransaction(session.ID, transaction)

var actual []*outliers.Outlier
registry.IterateOutliers(
Expand All @@ -83,15 +80,14 @@ func TestOutliers(t *testing.T) {
t.Run("too fast", func(t *testing.T) {
st := cluster.MakeTestingClusterSettings()
outliers.LatencyThreshold.Override(ctx, &st.SV, 1*time.Second)
statement2ID := clusterunique.IDFromBytes([]byte("bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb"))
statement2 := &outliers.Statement{
ID: &statement2ID,
ID: clusterunique.IDFromBytes([]byte("bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb")),
FingerprintID: roachpb.StmtFingerprintID(100),
LatencyInSeconds: 0.5,
}
registry := outliers.New(st, outliers.NewMetrics())
registry.ObserveStatement(sessionID, statement2)
registry.ObserveTransaction(sessionID, transaction)
registry.ObserveStatement(session.ID, statement2)
registry.ObserveTransaction(session.ID, transaction)

var actual []*outliers.Outlier
registry.IterateOutliers(
Expand All @@ -104,24 +100,21 @@ func TestOutliers(t *testing.T) {
})

t.Run("buffering statements per session", func(t *testing.T) {
otherSessionID := clusterunique.IDFromBytes([]byte("cccccccccccccccccccccccccccccccc"))
otherSession := &outliers.Session{ID: &otherSessionID}
otherTxnID := uuid.FastMakeV4()
otherTransaction := &outliers.Transaction{ID: &otherTxnID}
otherStatementID := clusterunique.IDFromBytes([]byte("dddddddddddddddddddddddddddddddd"))
otherSession := &outliers.Session{ID: clusterunique.IDFromBytes([]byte("cccccccccccccccccccccccccccccccc"))}
otherTransaction := &outliers.Transaction{ID: uuid.FastMakeV4()}
otherStatement := &outliers.Statement{
ID: &otherStatementID,
ID: clusterunique.IDFromBytes([]byte("dddddddddddddddddddddddddddddddd")),
FingerprintID: roachpb.StmtFingerprintID(101),
LatencyInSeconds: 3,
}

st := cluster.MakeTestingClusterSettings()
outliers.LatencyThreshold.Override(ctx, &st.SV, 1*time.Second)
registry := outliers.New(st, outliers.NewMetrics())
registry.ObserveStatement(sessionID, statement)
registry.ObserveStatement(otherSessionID, otherStatement)
registry.ObserveTransaction(sessionID, transaction)
registry.ObserveTransaction(otherSessionID, otherTransaction)
registry.ObserveStatement(session.ID, statement)
registry.ObserveStatement(otherSession.ID, otherStatement)
registry.ObserveTransaction(session.ID, transaction)
registry.ObserveTransaction(otherSession.ID, otherTransaction)

expected := []*outliers.Outlier{{
Session: session,
Expand Down
2 changes: 1 addition & 1 deletion pkg/sql/sqlstats/outliers/registry.go
Original file line number Diff line number Diff line change
Expand Up @@ -89,7 +89,7 @@ func (r *registry) ObserveTransaction(sessionID clusterunique.ID, transaction *T
if hasOutlier {
for _, s := range statements {
r.mu.outliers.Add(s.ID, &Outlier{
Session: &Session{ID: &sessionID},
Session: &Session{ID: sessionID},
Transaction: transaction,
Statement: s,
})
Expand Down
4 changes: 2 additions & 2 deletions pkg/sql/sqlstats/ssmemstorage/ss_mem_writer.go
Original file line number Diff line number Diff line change
Expand Up @@ -161,7 +161,7 @@ func (s *Container) RecordStatement(
}

s.outliersRegistry.ObserveStatement(value.SessionID, &outliers.Statement{
ID: &value.StatementID,
ID: value.StatementID,
FingerprintID: stmtFingerprintID,
LatencyInSeconds: value.ServiceLatency,
})
Expand Down Expand Up @@ -274,7 +274,7 @@ func (s *Container) RecordTransaction(
stats.mu.data.ExecStats.MaxDiskUsage.Record(stats.mu.data.ExecStats.Count, float64(value.ExecStats.MaxDiskUsage))
}

s.outliersRegistry.ObserveTransaction(value.SessionID, &outliers.Transaction{ID: &value.TransactionID})
s.outliersRegistry.ObserveTransaction(value.SessionID, &outliers.Transaction{ID: value.TransactionID})

return nil
}
Expand Down

0 comments on commit ccb1eac

Please sign in to comment.