Skip to content

Commit

Permalink
Revert "ssmemstorage: improve lock contention on RecordStatement"
Browse files Browse the repository at this point in the history
This reverts commit a792cd3.
  • Loading branch information
j82w committed Sep 7, 2023
1 parent bd41d5e commit b4dead6
Show file tree
Hide file tree
Showing 4 changed files with 81 additions and 303 deletions.
8 changes: 0 additions & 8 deletions pkg/sql/sqlstats/sslocal/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -45,7 +45,6 @@ go_test(
":sslocal",
"//pkg/base",
"//pkg/ccl",
"//pkg/kv/kvpb",
"//pkg/roachpb",
"//pkg/security/securityassets",
"//pkg/security/securitytest",
Expand All @@ -56,29 +55,22 @@ go_test(
"//pkg/settings/cluster",
"//pkg/sql",
"//pkg/sql/appstatspb",
"//pkg/sql/clusterunique",
"//pkg/sql/execstats",
"//pkg/sql/sessiondata",
"//pkg/sql/sessiondatapb",
"//pkg/sql/sessionphase",
"//pkg/sql/sqlstats",
"//pkg/sql/sqlstats/insights",
"//pkg/sql/sqlstats/persistedsqlstats",
"//pkg/sql/sqlstats/persistedsqlstats/sqlstatsutil",
"//pkg/storage/enginepb",
"//pkg/testutils",
"//pkg/testutils/serverutils",
"//pkg/testutils/sqlutils",
"//pkg/testutils/testcluster",
"//pkg/util",
"//pkg/util/hlc",
"//pkg/util/leaktest",
"//pkg/util/log",
"//pkg/util/metric",
"//pkg/util/mon",
"//pkg/util/timeutil",
"//pkg/util/uint128",
"//pkg/util/uuid",
"@com_github_jackc_pgx_v4//:pgx",
"@com_github_stretchr_testify//require",
],
Expand Down
144 changes: 0 additions & 144 deletions pkg/sql/sqlstats/sslocal/sql_stats_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -14,7 +14,6 @@ import (
"context"
gosql "database/sql"
"encoding/json"
"fmt"
"math"
"net/url"
"sort"
Expand All @@ -23,16 +22,13 @@ import (
"time"

"github.com/cockroachdb/cockroach/pkg/base"
"github.com/cockroachdb/cockroach/pkg/kv/kvpb"
"github.com/cockroachdb/cockroach/pkg/roachpb"
"github.com/cockroachdb/cockroach/pkg/security/username"
"github.com/cockroachdb/cockroach/pkg/server/serverpb"
"github.com/cockroachdb/cockroach/pkg/settings"
"github.com/cockroachdb/cockroach/pkg/settings/cluster"
"github.com/cockroachdb/cockroach/pkg/sql"
"github.com/cockroachdb/cockroach/pkg/sql/appstatspb"
"github.com/cockroachdb/cockroach/pkg/sql/clusterunique"
"github.com/cockroachdb/cockroach/pkg/sql/execstats"
"github.com/cockroachdb/cockroach/pkg/sql/sessiondata"
"github.com/cockroachdb/cockroach/pkg/sql/sessiondatapb"
"github.com/cockroachdb/cockroach/pkg/sql/sessionphase"
Expand All @@ -41,19 +37,14 @@ import (
"github.com/cockroachdb/cockroach/pkg/sql/sqlstats/persistedsqlstats"
"github.com/cockroachdb/cockroach/pkg/sql/sqlstats/persistedsqlstats/sqlstatsutil"
"github.com/cockroachdb/cockroach/pkg/sql/sqlstats/sslocal"
"github.com/cockroachdb/cockroach/pkg/storage/enginepb"
"github.com/cockroachdb/cockroach/pkg/testutils"
"github.com/cockroachdb/cockroach/pkg/testutils/serverutils"
"github.com/cockroachdb/cockroach/pkg/testutils/sqlutils"
"github.com/cockroachdb/cockroach/pkg/util"
"github.com/cockroachdb/cockroach/pkg/util/hlc"
"github.com/cockroachdb/cockroach/pkg/util/leaktest"
"github.com/cockroachdb/cockroach/pkg/util/log"
"github.com/cockroachdb/cockroach/pkg/util/metric"
"github.com/cockroachdb/cockroach/pkg/util/mon"
"github.com/cockroachdb/cockroach/pkg/util/timeutil"
"github.com/cockroachdb/cockroach/pkg/util/uint128"
"github.com/cockroachdb/cockroach/pkg/util/uuid"
"github.com/jackc/pgx/v4"
"github.com/stretchr/testify/require"
)
Expand Down Expand Up @@ -510,141 +501,6 @@ func TestExplicitTxnFingerprintAccounting(t *testing.T) {
}
}

func BenchmarkRecordStatement(b *testing.B) {
defer leaktest.AfterTest(b)()
defer log.Scope(b).Close(b)

ctx := context.Background()

st := cluster.MakeTestingClusterSettings()
monitor := mon.NewUnlimitedMonitor(
context.Background(), "test", mon.MemoryResource,
nil /* curCount */, nil /* maxHist */, math.MaxInt64, st,
)

insightsProvider := insights.New(st, insights.NewMetrics())
sqlStats := sslocal.New(
st,
sqlstats.MaxMemSQLStatsStmtFingerprints,
sqlstats.MaxMemSQLStatsTxnFingerprints,
metric.NewGauge(sql.MetaReportedSQLStatsMemCurBytes), /* curMemoryBytesCount */
metric.NewHistogram(metric.HistogramOptions{
Metadata: sql.MetaReportedSQLStatsMemMaxBytes,
Duration: 10 * time.Second,
MaxVal: 19 * 1000,
SigFigs: 3,
BucketConfig: metric.MemoryUsage64MBBuckets,
}), /* maxMemoryBytesHist */
insightsProvider.Writer,
monitor,
nil, /* reportingSink */
nil, /* knobs */
insightsProvider.LatencyInformation(),
)

appStats := sqlStats.GetApplicationStats("" /* appName */, false /* internal */)
statsCollector := sslocal.NewStatsCollector(
st,
appStats,
sessionphase.NewTimes(),
nil, /* knobs */
)

txnId := uuid.FastMakeV4()
generateRecord := func() sqlstats.RecordedStmtStats {
return sqlstats.RecordedStmtStats{
SessionID: clusterunique.ID{Uint128: uint128.Uint128{Hi: 0x1771ca67e66e6b48, Lo: 0x1}},
StatementID: clusterunique.ID{Uint128: uint128.Uint128{Hi: 0x1771ca67e67262e8, Lo: 0x1}},
TransactionID: txnId,
AutoRetryCount: 0,
AutoRetryReason: error(nil),
RowsAffected: 1,
IdleLatencySec: 0,
ParseLatencySec: 6.5208e-05,
PlanLatencySec: 0.000187041,
RunLatencySec: 0.500771041,
ServiceLatencySec: 0.501024374,
OverheadLatencySec: 1.0839999999845418e-06,
BytesRead: 30,
RowsRead: 1,
RowsWritten: 1,
Nodes: []int64{1},
StatementType: 1,
Plan: (*appstatspb.ExplainTreePlanNode)(nil),
PlanGist: "AgHQAQIAAwIAAAcGBQYh0AEAAQ==",
StatementError: error(nil),
IndexRecommendations: []string(nil),
Query: "UPDATE t SET s = '_' WHERE id = '_'",
StartTime: time.Date(2023, time.July, 14, 16, 58, 2, 837542000, time.UTC),
EndTime: time.Date(2023, time.July, 14, 16, 58, 3, 338566374, time.UTC),
FullScan: false,
ExecStats: &execstats.QueryLevelStats{
NetworkBytesSent: 10,
MaxMemUsage: 40960,
MaxDiskUsage: 197,
KVBytesRead: 30,
KVPairsRead: 1,
KVRowsRead: 1,
KVBatchRequestsIssued: 1,
KVTime: 498771793,
MvccSteps: 1,
MvccStepsInternal: 2,
MvccSeeks: 4,
MvccSeeksInternal: 4,
MvccBlockBytes: 39,
MvccBlockBytesInCache: 25,
MvccKeyBytes: 23,
MvccValueBytes: 250,
MvccPointCount: 6,
MvccPointsCoveredByRangeTombstones: 99,
MvccRangeKeyCount: 9,
MvccRangeKeyContainedPoints: 88,
MvccRangeKeySkippedPoints: 66,
NetworkMessages: 55,
ContentionTime: 498546750,
ContentionEvents: []kvpb.ContentionEvent{{Key: []byte{}, TxnMeta: enginepb.TxnMeta{ID: uuid.FastMakeV4(), Key: []uint8{0xf0, 0x89, 0x12, 0x74, 0x65, 0x73, 0x74, 0x0, 0x1, 0x88}, IsoLevel: 0, Epoch: 0, WriteTimestamp: hlc.Timestamp{WallTime: 1689354396802600000, Logical: 0, Synthetic: false}, MinTimestamp: hlc.Timestamp{WallTime: 1689354396802600000, Logical: 0, Synthetic: false}, Priority: 118164, Sequence: 1, CoordinatorNodeID: 1}, Duration: 498546750}},
RUEstimate: 9999,
CPUTime: 12345,
SqlInstanceIds: map[base.SQLInstanceID]struct{}{1: {}},
Regions: []string{"test"}},
Indexes: []string{"104@1"},
Database: "defaultdb",
}
}

parallel := []int{10}
for _, p := range parallel {
name := fmt.Sprintf("RecordStatement: Parallelism %d", p)
b.Run(name, func(b *testing.B) {
b.SetParallelism(p)
recordValue := generateRecord()
b.RunParallel(func(pb *testing.PB) {
for pb.Next() {
_, err := statsCollector.RecordStatement(
ctx,
appstatspb.StatementStatisticsKey{
Query: "select * from T where t.l = 1000",
ImplicitTxn: true,
Vec: true,
FullScan: true,
DistSQL: true,
Database: "testDb",
App: "myTestApp",
PlanHash: 9001,
QuerySummary: "select * from T",
},
recordValue,
)
// Adds overhead to benchmark and shows up in profile
if err != nil {
require.NoError(b, err)
}
}
})
})
}
}

func TestAssociatingStmtStatsWithTxnFingerprint(t *testing.T) {
defer leaktest.AfterTest(t)()
defer log.Scope(t).Close(t)
Expand Down
Loading

0 comments on commit b4dead6

Please sign in to comment.