Skip to content

Commit

Permalink
Browse files Browse the repository at this point in the history
71896: sql/server: add function to reset index usage stats r=lindseyjin a=lindseyjin

Previously, there was no existing way to clear index usage statistics
from the crdb_internal.index_usage_statistics table. This commit adds a
function that enables developers to clear index usage metrics using RPC
fanout to reach all nodes in a cluster. We have also added a new
metadata field that tracks the last reset time. Currently, this
functionality can be accessed via the SQL shell, and is not yet
accessible from the frontend console.

![image](https://user-images.githubusercontent.com/29153209/138530064-31eb8bbf-aafc-4e5f-995c-bd575b0ba60a.png)

Release note (sql change): Add function
crdb_internal.reset_index_usage_stats() to clear index usage stats. This
can be invoked from the SQL shell.

72042: kvserver: batch intents in `MVCCIterator.CheckForKeyCollisions` r=nvanbenschoten a=erikgrinaker

`MVCCIterator.CheckForKeyCollisions()` is used by `AddSSTable` to check
for key collisions when `DisallowShadowing` is set. If it encounters any
intents, it returns `WriteIntentError` to resolve these before retrying.

However, this returned an error for each individual intent, which has
quadratic performance. This patch changes it to instead collect and
return a batch of intents, for more efficient intent resolution.

The batch size is controlled by the existing setting
`storage.mvcc.max_intents_per_error`, which defaults to 5000.

Resolves cockroachdb#71697.

Release note (performance improvement): Improved `IMPORT INTO`
performance in cases where it encounters large numbers of unresolved
write intents.

72136: opt: fix regression in many-columns-and-indexes microbenchmark r=rytaft a=rytaft

This commit fixes a regression that was due to unnecessary computation
of interesting orderings. We now only compute interesting orderings
when it may be beneficial for planning purposes.

Fixes cockroachdb#72001

Release note (performance improvement): Fixed a performance regression
in planning that could occur for simple queries on schemas with a large
number of indexes.

Co-authored-by: Lindsey Jin <[email protected]>
Co-authored-by: Erik Grinaker <[email protected]>
Co-authored-by: Rebecca Taft <[email protected]>
  • Loading branch information
4 people committed Oct 29, 2021
4 parents b7dceca + f85b8ff + af360f8 + b8b3e37 commit 2e326b8
Show file tree
Hide file tree
Showing 32 changed files with 1,565 additions and 600 deletions.
41 changes: 41 additions & 0 deletions docs/generated/http/full.md
Original file line number Diff line number Diff line change
Expand Up @@ -3956,6 +3956,47 @@ Response object returned by IndexUsageStatistics.
| Field | Type | Label | Description | Support status |
| ----- | ---- | ----- | ----------- | -------------- |
| statistics | [cockroach.sql.CollectedIndexUsageStatistics](#cockroach.server.serverpb.IndexUsageStatisticsResponse-cockroach.sql.CollectedIndexUsageStatistics) | repeated | | [reserved](#support-status) |
| last_reset | [google.protobuf.Timestamp](#cockroach.server.serverpb.IndexUsageStatisticsResponse-google.protobuf.Timestamp) | | Timestamp of the last index usage stats reset. | [reserved](#support-status) |







## ResetIndexUsageStats

`POST /_status/resetindexusagestats`



Support status: [reserved](#support-status)

#### Request Parameters




Request object for issuing a index usage stats reset request.


| Field | Type | Label | Description | Support status |
| ----- | ---- | ----- | ----------- | -------------- |
| node_id | [string](#cockroach.server.serverpb.ResetIndexUsageStatsRequest-string) | | | [reserved](#support-status) |







#### Response Parameters




Response object returned by ResetIndexUsageStatsRequest.




Expand Down
2 changes: 2 additions & 0 deletions docs/generated/sql/functions.md
Original file line number Diff line number Diff line change
Expand Up @@ -2924,6 +2924,8 @@ SELECT * FROM crdb_internal.check_consistency(true, ‘\x02’, ‘\x04’)</p>
</span></td></tr>
<tr><td><a name="crdb_internal.range_stats"></a><code>crdb_internal.range_stats(key: <a href="bytes.html">bytes</a>) &rarr; jsonb</code></td><td><span class="funcdesc"><p>This function is used to retrieve range statistics information as a JSON object.</p>
</span></td></tr>
<tr><td><a name="crdb_internal.reset_index_usage_stats"></a><code>crdb_internal.reset_index_usage_stats() &rarr; <a href="bool.html">bool</a></code></td><td><span class="funcdesc"><p>This function is used to clear the collected index usage statistics.</p>
</span></td></tr>
<tr><td><a name="crdb_internal.reset_sql_stats"></a><code>crdb_internal.reset_sql_stats() &rarr; <a href="bool.html">bool</a></code></td><td><span class="funcdesc"><p>This function is used to clear the collected SQL statistics.</p>
</span></td></tr>
<tr><td><a name="crdb_internal.round_decimal_values"></a><code>crdb_internal.round_decimal_values(val: <a href="decimal.html">decimal</a>, scale: <a href="int.html">int</a>) &rarr; <a href="decimal.html">decimal</a></code></td><td><span class="funcdesc"><p>This function is used internally to round decimal values during mutations.</p>
Expand Down
106 changes: 106 additions & 0 deletions pkg/ccl/serverccl/tenant_status_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -300,6 +300,112 @@ func TestResetSQLStatsRPCForTenant(t *testing.T) {
}
}

func TestResetIndexUsageStatsRPCForTenant(t *testing.T) {
defer leaktest.AfterTest(t)()
defer log.Scope(t).Close(t)

skip.UnderStressRace(t, "expensive tests")

ctx := context.Background()

statsIngestionCb, statsIngestionNotifier := idxusage.CreateIndexStatsIngestedCallbackForTest()

knobs := tests.CreateTestingKnobs()
knobs.IndexUsageStatsKnobs = &idxusage.TestingKnobs{
OnIndexUsageStatsProcessedCallback: statsIngestionCb,
}

testHelper := newTestTenantHelper(t, 3 /* tenantClusterSize */, knobs)
defer testHelper.cleanup(ctx, t)

testingCluster := testHelper.testCluster()
controlCluster := testHelper.controlCluster()

for _, cluster := range []tenantCluster{testingCluster, controlCluster} {
// Create tables and insert data.
cluster.tenantConn(0).Exec(t, `
CREATE TABLE test (
k INT PRIMARY KEY,
a INT,
b INT,
INDEX(a)
)
`)

cluster.tenantConn(0).Exec(t, `
INSERT INTO test
VALUES (1, 10, 100), (2, 20, 200), (3, 30, 300)
`)

// Record scan on primary index.
cluster.tenantConn(0).Exec(t, "SELECT * FROM test")

// Record scan on secondary index.
cluster.tenantConn(1).Exec(t, "SELECT * FROM test@test_a_idx")
testTableIDStr := cluster.tenantConn(2).QueryStr(t, "SELECT 'test'::regclass::oid")[0][0]
testTableID, err := strconv.Atoi(testTableIDStr)
require.NoError(t, err)

// Wait for the stats to be ingested.
require.NoError(t,
idxusage.WaitForIndexStatsIngestionForTest(statsIngestionNotifier, map[roachpb.IndexUsageKey]struct{}{
{
TableID: roachpb.TableID(testTableID),
IndexID: 1,
}: {},
{
TableID: roachpb.TableID(testTableID),
IndexID: 2,
}: {},
}, 2 /* expectedEventCnt*/, 5*time.Second /* timeout */),
)

query := `
SELECT
table_id,
index_id,
total_reads,
extract_duration('second', now() - last_read) < 5
FROM
crdb_internal.index_usage_statistics
WHERE
table_id = $1
`
// Assert index usage data was inserted.
actual := cluster.tenantConn(2).QueryStr(t, query, testTableID)
expected := [][]string{
{testTableIDStr, "1", "1", "true"},
{testTableIDStr, "2", "1", "true"},
}
require.Equal(t, expected, actual)
}

// Reset index usage stats.
status := testingCluster.tenantStatusSrv(1 /* idx */)
_, err := status.ResetIndexUsageStats(ctx, &serverpb.ResetIndexUsageStatsRequest{})
require.NoError(t, err)

resp, err := status.IndexUsageStatistics(ctx, &serverpb.IndexUsageStatisticsRequest{})
require.NoError(t, err)

// Require index usage metrics to be reset.
for _, stats := range resp.Statistics {
require.Equal(t, uint64(0), stats.Stats.TotalReadCount)
require.Equal(t, time.Time{}, stats.Stats.LastRead)
}

// Ensure tenant data isolation.
status = controlCluster.tenantStatusSrv(1 /* idx */)
resp, err = status.IndexUsageStatistics(ctx, &serverpb.IndexUsageStatisticsRequest{})
require.NoError(t, err)

// Require index usage metrics to not be reset.
for _, stats := range resp.Statistics {
require.NotEqual(t, uint64(0), stats.Stats.TotalReadCount)
require.NotEqual(t, time.Time{}, stats.Stats.LastRead)
}
}

func ensureExpectedStmtFingerprintExistsInRPCResponse(
t *testing.T, expectedStmts []string, resp *serverpb.StatementsResponse, clusterType string,
) {
Expand Down
8 changes: 6 additions & 2 deletions pkg/kv/kvserver/batcheval/cmd_add_sstable.go
Original file line number Diff line number Diff line change
Expand Up @@ -121,7 +121,10 @@ func EvalAddSSTable(
var skippedKVStats enginepb.MVCCStats
var err error
if args.DisallowShadowing {
if skippedKVStats, err = checkForKeyCollisions(ctx, readWriter, mvccStartKey, mvccEndKey, args.Data); err != nil {
maxIntents := storage.MaxIntentsPerWriteIntentError.Get(&cArgs.EvalCtx.ClusterSettings().SV)
skippedKVStats, err = checkForKeyCollisions(
ctx, readWriter, mvccStartKey, mvccEndKey, args.Data, maxIntents)
if err != nil {
return result.Result{}, errors.Wrap(err, "checking for key collisions")
}
}
Expand Down Expand Up @@ -294,6 +297,7 @@ func checkForKeyCollisions(
mvccStartKey storage.MVCCKey,
mvccEndKey storage.MVCCKey,
data []byte,
maxIntents int64,
) (enginepb.MVCCStats, error) {
// Create iterator over the existing data.
existingDataIter := reader.NewMVCCIterator(storage.MVCCKeyAndIntentsIterKind, storage.IterOptions{UpperBound: mvccEndKey.Key})
Expand All @@ -306,5 +310,5 @@ func checkForKeyCollisions(
return enginepb.MVCCStats{}, nil
}

return existingDataIter.CheckForKeyCollisions(data, mvccStartKey.Key, mvccEndKey.Key)
return existingDataIter.CheckForKeyCollisions(data, mvccStartKey.Key, mvccEndKey.Key, maxIntents)
}
39 changes: 31 additions & 8 deletions pkg/kv/kvserver/batcheval/cmd_add_sstable_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -26,6 +26,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/kv/kvserver/batcheval"
"github.com/cockroachdb/cockroach/pkg/roachpb"
"github.com/cockroachdb/cockroach/pkg/server"
"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"
Expand Down Expand Up @@ -498,6 +499,9 @@ func TestAddSSTableDisallowShadowing(t *testing.T) {
e := engineImpl.create()
defer e.Close()

st := cluster.MakeTestingClusterSettings()
evalCtx := (&batcheval.MockEvalCtx{ClusterSettings: st}).EvalContext()

for _, kv := range mvccKVsFromStrs([]strKv{
{"a", 2, "aa"},
{"b", 1, "bb"},
Expand Down Expand Up @@ -553,6 +557,7 @@ func TestAddSSTableDisallowShadowing(t *testing.T) {
sstBytes := getSSTBytes(sstKVs)
stats := getStats(roachpb.Key("a"), roachpb.Key("b"), sstBytes)
cArgs := batcheval.CommandArgs{
EvalCtx: evalCtx,
Header: roachpb.Header{
Timestamp: hlc.Timestamp{WallTime: 7},
},
Expand Down Expand Up @@ -580,6 +585,7 @@ func TestAddSSTableDisallowShadowing(t *testing.T) {

sstBytes := getSSTBytes(sstKVs)
cArgs := batcheval.CommandArgs{
EvalCtx: evalCtx,
Header: roachpb.Header{
Timestamp: hlc.Timestamp{WallTime: 7},
},
Expand Down Expand Up @@ -609,6 +615,7 @@ func TestAddSSTableDisallowShadowing(t *testing.T) {

sstBytes := getSSTBytes(sstKVs)
cArgs := batcheval.CommandArgs{
EvalCtx: evalCtx,
Header: roachpb.Header{
Timestamp: hlc.Timestamp{WallTime: 7},
},
Expand Down Expand Up @@ -636,6 +643,7 @@ func TestAddSSTableDisallowShadowing(t *testing.T) {
sstBytes := getSSTBytes(sstKVs)
stats := getStats(roachpb.Key("c"), roachpb.Key("i"), sstBytes)
cArgs := batcheval.CommandArgs{
EvalCtx: evalCtx,
Header: roachpb.Header{
Timestamp: hlc.Timestamp{WallTime: 7},
},
Expand Down Expand Up @@ -668,6 +676,7 @@ func TestAddSSTableDisallowShadowing(t *testing.T) {

sstBytes := getSSTBytes(sstKVs)
cArgs := batcheval.CommandArgs{
EvalCtx: evalCtx,
Header: roachpb.Header{
Timestamp: hlc.Timestamp{WallTime: 7},
},
Expand Down Expand Up @@ -696,6 +705,7 @@ func TestAddSSTableDisallowShadowing(t *testing.T) {

sstBytes := getSSTBytes(sstKVs)
cArgs := batcheval.CommandArgs{
EvalCtx: evalCtx,
Header: roachpb.Header{
Timestamp: hlc.Timestamp{WallTime: 7},
},
Expand Down Expand Up @@ -725,6 +735,7 @@ func TestAddSSTableDisallowShadowing(t *testing.T) {

sstBytes := getSSTBytes(sstKVs)
cArgs := batcheval.CommandArgs{
EvalCtx: evalCtx,
Header: roachpb.Header{
Timestamp: hlc.Timestamp{WallTime: 7},
},
Expand All @@ -742,16 +753,16 @@ func TestAddSSTableDisallowShadowing(t *testing.T) {
}
}

// Test key collision when ingesting a key which has a write intent in the
// Test key collision when ingesting keys which have write intents in the
// existing data.
{
sstKVs := mvccKVsFromStrs([]strKv{
{"f", 2, "ff"},
{"q", 4, "qq"},
{"t", 3, "ttt"}, // has a write intent in the existing data.
{"q", 4, "qq"}, // has a write intent in the existing data
{"t", 3, "ttt"}, // has a write intent in the existing data
})

// Add in a write intent.
// Add in two write intents.
ts := hlc.Timestamp{WallTime: 7}
txn := roachpb.MakeTransaction(
"test",
Expand All @@ -760,18 +771,24 @@ func TestAddSSTableDisallowShadowing(t *testing.T) {
ts,
base.DefaultMaxClockOffset.Nanoseconds(),
)
if err := storage.MVCCPut(
ctx, e, nil, []byte("q"), ts,
roachpb.MakeValueFromBytes([]byte("q")),
&txn,
); err != nil {
t.Fatalf("%+v", err)
}
if err := storage.MVCCPut(
ctx, e, nil, []byte("t"), ts,
roachpb.MakeValueFromBytes([]byte("tt")),
&txn,
); err != nil {
if !errors.HasType(err, (*roachpb.WriteIntentError)(nil)) {
t.Fatalf("%+v", err)
}
t.Fatalf("%+v", err)
}

sstBytes := getSSTBytes(sstKVs)
cArgs := batcheval.CommandArgs{
EvalCtx: evalCtx,
Header: roachpb.Header{
Timestamp: hlc.Timestamp{WallTime: 7},
},
Expand All @@ -784,7 +801,7 @@ func TestAddSSTableDisallowShadowing(t *testing.T) {
}

_, err := batcheval.EvalAddSSTable(ctx, e, cArgs, nil)
if !testutils.IsError(err, "conflicting intents on \"t") {
if !testutils.IsError(err, "conflicting intents on \"q\", \"t\"") {
t.Fatalf("%+v", err)
}
}
Expand All @@ -810,6 +827,7 @@ func TestAddSSTableDisallowShadowing(t *testing.T) {

sstBytes := getSSTBytes(sstKVs)
cArgs := batcheval.CommandArgs{
EvalCtx: evalCtx,
Header: roachpb.Header{
Timestamp: hlc.Timestamp{WallTime: 7},
},
Expand Down Expand Up @@ -839,6 +857,7 @@ func TestAddSSTableDisallowShadowing(t *testing.T) {
sstBytes := getSSTBytes(sstKVs)
stats := getStats(roachpb.Key("e"), roachpb.Key("zz"), sstBytes)
cArgs := batcheval.CommandArgs{
EvalCtx: evalCtx,
Header: roachpb.Header{
Timestamp: hlc.Timestamp{WallTime: 7},
},
Expand Down Expand Up @@ -868,6 +887,7 @@ func TestAddSSTableDisallowShadowing(t *testing.T) {

sstBytes := getSSTBytes(sstKVs)
cArgs := batcheval.CommandArgs{
EvalCtx: evalCtx,
Header: roachpb.Header{
Timestamp: hlc.Timestamp{WallTime: 7},
},
Expand Down Expand Up @@ -896,6 +916,7 @@ func TestAddSSTableDisallowShadowing(t *testing.T) {

sstBytes := getSSTBytes(sstKVs)
cArgs := batcheval.CommandArgs{
EvalCtx: evalCtx,
Header: roachpb.Header{
Timestamp: hlc.Timestamp{WallTime: 7},
},
Expand Down Expand Up @@ -924,6 +945,7 @@ func TestAddSSTableDisallowShadowing(t *testing.T) {

sstBytes := getSSTBytes(sstKVs)
cArgs := batcheval.CommandArgs{
EvalCtx: evalCtx,
Header: roachpb.Header{
Timestamp: hlc.Timestamp{WallTime: 7},
},
Expand Down Expand Up @@ -963,6 +985,7 @@ func TestAddSSTableDisallowShadowing(t *testing.T) {
commandStats := enginepb.MVCCStats{}

cArgs := batcheval.CommandArgs{
EvalCtx: evalCtx,
Header: roachpb.Header{
Timestamp: hlc.Timestamp{WallTime: 7},
},
Expand Down
6 changes: 6 additions & 0 deletions pkg/kv/kvserver/batcheval/cmd_export.go
Original file line number Diff line number Diff line change
Expand Up @@ -151,6 +151,11 @@ func evalExport(

maxRunTime := exportRequestMaxIterationTime.Get(&cArgs.EvalCtx.ClusterSettings().SV)

var maxIntents uint64
if m := storage.MaxIntentsPerWriteIntentError.Get(&cArgs.EvalCtx.ClusterSettings().SV); m > 0 {
maxIntents = uint64(m)
}

// Time-bound iterators only make sense to use if the start time is set.
useTBI := args.EnableTimeBoundIteratorOptimization && !args.StartTime.IsEmpty()
// Only use resume timestamp if splitting mid key is enabled.
Expand All @@ -170,6 +175,7 @@ func evalExport(
ExportAllRevisions: exportAllRevisions,
TargetSize: targetSize,
MaxSize: maxSize,
MaxIntents: maxIntents,
StopMidKey: args.SplitMidKey,
UseTBI: useTBI,
ResourceLimiter: storage.NewResourceLimiter(storage.ResourceLimiterOptions{MaxRunTime: maxRunTime}, timeutil.DefaultTimeSource{}),
Expand Down
Loading

0 comments on commit 2e326b8

Please sign in to comment.