Skip to content

Commit

Permalink
rditer: add functions to compute the MVCC stats for user-only and non…
Browse files Browse the repository at this point in the history
…-user spans

ComputeStatsForRange iterates over an entire range and computes its MVCC
stats.

This patch adds ComputeStatsForRangeUserOnly and
ComputeStatsForRangeExcludingUser, which compute the MVCC stats
corresponding to the user-only and non-user spans in the range,
respectively. These functions are useful for computing the estimated
stats proposed in cockroachdb#119499 by allowing to quickly iterate only over the
non-user key spans in the range, and separately iterate over the
user-only key spans.

Part of: cockroachdb#119499
Release note: None
  • Loading branch information
miraradeva committed Mar 11, 2024
1 parent 8ad5029 commit 4b963ea
Show file tree
Hide file tree
Showing 5 changed files with 141 additions and 2 deletions.
3 changes: 3 additions & 0 deletions pkg/kv/kvserver/rditer/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -29,6 +29,7 @@ go_test(
srcs = [
"replica_data_iter_test.go",
"select_test.go",
"stats_test.go",
],
data = glob(["testdata/**"]),
embed = [":rditer"],
Expand All @@ -39,12 +40,14 @@ go_test(
"//pkg/roachpb",
"//pkg/settings/cluster",
"//pkg/storage",
"//pkg/storage/enginepb",
"//pkg/testutils",
"//pkg/testutils/datapathutils",
"//pkg/testutils/echotest",
"//pkg/testutils/skip",
"//pkg/util/hlc",
"//pkg/util/leaktest",
"//pkg/util/protoutil",
"//pkg/util/randutil",
"//pkg/util/uuid",
"@com_github_olekukonko_tablewriter//:tablewriter",
Expand Down
19 changes: 19 additions & 0 deletions pkg/kv/kvserver/rditer/replica_data_iter.go
Original file line number Diff line number Diff line change
Expand Up @@ -137,6 +137,25 @@ func MakeReplicatedKeySpanSet(d *roachpb.RangeDescriptor) *spanset.SpanSet {
return ss
}

// MakeReplicatedKeySpansUserOnly returns all key spans corresponding to user
// keys.
func MakeReplicatedKeySpansUserOnly(d *roachpb.RangeDescriptor) []roachpb.Span {
return Select(d.RangeID, SelectOpts{
ReplicatedBySpan: d.RSpan(),
ReplicatedSpansFilter: ReplicatedSpansUserOnly,
})
}

// MakeReplicatedKeySpansExcludingUser returns all key spans corresponding to
// non-user keys.
func MakeReplicatedKeySpansExcludingUser(d *roachpb.RangeDescriptor) []roachpb.Span {
return Select(d.RangeID, SelectOpts{
ReplicatedBySpan: d.RSpan(),
ReplicatedByRangeID: true,
ReplicatedSpansFilter: ReplicatedSpansExcludeUser,
})
}

// makeReplicatedKeySpansExceptLockTable returns all key spans that are fully Raft
// replicated for the given Range, except for the lock table spans. These are
// returned in the following sorted order:
Expand Down
11 changes: 10 additions & 1 deletion pkg/kv/kvserver/rditer/replica_data_iter_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -24,12 +24,14 @@ import (
"github.com/cockroachdb/cockroach/pkg/roachpb"
"github.com/cockroachdb/cockroach/pkg/settings/cluster"
"github.com/cockroachdb/cockroach/pkg/storage"
"github.com/cockroachdb/cockroach/pkg/storage/enginepb"
"github.com/cockroachdb/cockroach/pkg/testutils"
"github.com/cockroachdb/cockroach/pkg/testutils/datapathutils"
"github.com/cockroachdb/cockroach/pkg/testutils/echotest"
"github.com/cockroachdb/cockroach/pkg/testutils/skip"
"github.com/cockroachdb/cockroach/pkg/util/hlc"
"github.com/cockroachdb/cockroach/pkg/util/leaktest"
"github.com/cockroachdb/cockroach/pkg/util/protoutil"
"github.com/cockroachdb/cockroach/pkg/util/randutil"
"github.com/cockroachdb/cockroach/pkg/util/uuid"
"github.com/olekukonko/tablewriter"
Expand Down Expand Up @@ -118,6 +120,13 @@ func createRangeData(
TxnUUID: testTxnID,
},
}
lockMeta := enginepb.MVCCMetadata{
Txn: &enginepb.TxnMeta{ID: testTxnID},
KeyBytes: storage.MVCCVersionTimestampSize,
ValBytes: int64(len(value.RawBytes)),
}
lockVal, err := protoutil.Marshal(&lockMeta)
require.NoError(t, err)

rs = append(rs, storage.MVCCRangeKey{ // emitted last because we emit all point keys before range keys
StartKey: desc.StartKey.AsRawKey().Clone(),
Expand All @@ -134,7 +143,7 @@ func createRangeData(
}
for _, l := range locks {
sl, _ := l.ToEngineKey(nil)
require.NoError(t, eng.PutEngineKey(sl, []byte("fake lock")))
require.NoError(t, eng.PutEngineKey(sl, lockVal))
}

return ps, rs
Expand Down
56 changes: 55 additions & 1 deletion pkg/kv/kvserver/rditer/stats.go
Original file line number Diff line number Diff line change
Expand Up @@ -27,6 +27,24 @@ func ComputeStatsForRange(
ctx, d, reader, nowNanos, storage.ComputeStatsVisitors{})
}

// ComputeStatsForRangeUserOnly is like ComputeStatsForRange but iterates only
// over spans corresponding to user keys.
func ComputeStatsForRangeUserOnly(
ctx context.Context, d *roachpb.RangeDescriptor, reader storage.Reader, nowNanos int64,
) (enginepb.MVCCStats, error) {
return ComputeStatsForRangeWithVisitorsUserOnly(
ctx, d, reader, nowNanos, storage.ComputeStatsVisitors{})
}

// ComputeStatsForRangeExcludingUser is like ComputeStatsForRange but iterates
// only over spans *not* corresponding to user keys.
func ComputeStatsForRangeExcludingUser(
ctx context.Context, d *roachpb.RangeDescriptor, reader storage.Reader, nowNanos int64,
) (enginepb.MVCCStats, error) {
return ComputeStatsForRangeWithVisitorsExcludingUser(
ctx, d, reader, nowNanos, storage.ComputeStatsVisitors{})
}

// ComputeStatsForRangeWithVisitors is like ComputeStatsForRange but also
// calls the given callbacks for every key.
func ComputeStatsForRangeWithVisitors(
Expand All @@ -35,9 +53,45 @@ func ComputeStatsForRangeWithVisitors(
reader storage.Reader,
nowNanos int64,
visitors storage.ComputeStatsVisitors,
) (enginepb.MVCCStats, error) {
return computeStatsForSpansWithVisitors(ctx, MakeReplicatedKeySpans(d), reader, nowNanos, visitors)
}

// ComputeStatsForRangeWithVisitorsUserOnly is like
// ComputeStatsForRangeWithVisitors but computes stats only for spans
// corresponding to user keys.
func ComputeStatsForRangeWithVisitorsUserOnly(
ctx context.Context,
d *roachpb.RangeDescriptor,
reader storage.Reader,
nowNanos int64,
visitors storage.ComputeStatsVisitors,
) (enginepb.MVCCStats, error) {
return computeStatsForSpansWithVisitors(ctx, MakeReplicatedKeySpansUserOnly(d), reader, nowNanos, visitors)
}

// ComputeStatsForRangeWithVisitorsExcludingUser is like
// ComputeStatsForRangeWithVisitors but computes stats only for spans
// corresponding to non-user keys.
func ComputeStatsForRangeWithVisitorsExcludingUser(
ctx context.Context,
d *roachpb.RangeDescriptor,
reader storage.Reader,
nowNanos int64,
visitors storage.ComputeStatsVisitors,
) (enginepb.MVCCStats, error) {
return computeStatsForSpansWithVisitors(ctx, MakeReplicatedKeySpansExcludingUser(d), reader, nowNanos, visitors)
}

func computeStatsForSpansWithVisitors(
ctx context.Context,
spans []roachpb.Span,
reader storage.Reader,
nowNanos int64,
visitors storage.ComputeStatsVisitors,
) (enginepb.MVCCStats, error) {
var ms enginepb.MVCCStats
for _, keySpan := range MakeReplicatedKeySpans(d) {
for _, keySpan := range spans {
msDelta, err := storage.ComputeStatsWithVisitors(
ctx, reader, keySpan.Key, keySpan.EndKey, nowNanos, visitors)
if err != nil {
Expand Down
54 changes: 54 additions & 0 deletions pkg/kv/kvserver/rditer/stats_test.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,54 @@
// Copyright 2024 The Cockroach Authors.
//
// Use of this software is governed by the Business Source License
// included in the file licenses/BSL.txt.
//
// As of the Change Date specified in that file, in accordance with
// the Business Source License, use of this software will be governed
// by the Apache License, Version 2.0, included in the file
// licenses/APL.txt.

package rditer

import (
"context"
"testing"
"time"

"github.com/cockroachdb/cockroach/pkg/roachpb"
"github.com/cockroachdb/cockroach/pkg/storage"
"github.com/cockroachdb/cockroach/pkg/util/leaktest"
"github.com/stretchr/testify/require"
)

// TestComputeStats verifies that the sum of the stats returned by
// ComputeStatsForRangeUserOnly and ComputeStatsForRangeExcludingUser equals the
// total range stats returned by ComputeStatsForRange.
func TestComputeStats(t *testing.T) {
defer leaktest.AfterTest(t)()

storage.DisableMetamorphicSimpleValueEncoding(t)

eng := storage.NewDefaultInMemForTesting()
defer eng.Close()

desc := roachpb.RangeDescriptor{
RangeID: 1,
StartKey: roachpb.RKey("a"),
EndKey: roachpb.RKey("b"),
}

createRangeData(t, eng, desc)
nowNanos := time.Now().UnixNano()

userOnly, err := ComputeStatsForRangeUserOnly(context.Background(), &desc, eng, nowNanos)
require.NoError(t, err)
nonUserOnly, err := ComputeStatsForRangeExcludingUser(context.Background(), &desc, eng, nowNanos)
require.NoError(t, err)
all, err := ComputeStatsForRange(context.Background(), &desc, eng, nowNanos)
require.NoError(t, err)

userPlusNonUser := userOnly
userPlusNonUser.Add(nonUserOnly)
require.Equal(t, all, userPlusNonUser)
}

0 comments on commit 4b963ea

Please sign in to comment.