Skip to content

Commit

Permalink
Browse files Browse the repository at this point in the history
69806: kv/kvserver: use generalized engine keys in debug printing r=AlexTalks a=AlexTalks

Previously the CLI debug command only printed `MVCCKey`s, resulting in
the debug printer to error on key decoding whenever a `LockTableKey` was
encountered.  By switching to the more generalized `EngineKey` (and
utilizing the existing MVCC key formatting whenever the key has an MVCC
version), we can increase visibility into our debug logs while investigating
issues.

Related to #69414

Release justification: Non-production bug fix
Release note: None

69944: stats: add a histogram version number r=rytaft a=rytaft

This commit adds a histogram version number to the `HistogramData`
proto. This will allow us to identify what logic was used to construct
a particular histogram and possibly debug future issues.

Release note: None

Release justification: Low risk, high benefit change to existing
functionality.

69963: sql: skip reset sql stats in TestRandomSyntaxFunctions r=maryliag,rafiss a=Azhng

Previously, crdb_internal.reset_sql_stats() causes timeout
in TestRandomSyntaxFunctions. This is very unlikely due to
implementation of the function, and it is likely caused
by contentions.
This commit skip the tests for crdb_internal.reset_sql_stats()
to prevent nightly failures.

Related #69731

Release justification: Non-production code changes

Release note: None

69967: vendor: bump Pebble to 6c12d67b83e6 r=jbowens a=jbowens

```
6c12d67 internal/metamorphic: randomize FormatMajorVersion
e82fb10 db: randomize format major version in unit tests
535b8d6 db: add FormatUpgrade event to EventListener
53dda0f db: introduce format major version
8ec1a49 vfs/atomicfs: add ReadMarker
daf93f0 sstable: Free cache value when checksum type is corrupt
d89613d metamorphic: randomly use disk for tests
e3b6bec metamorphic: transform percentage of SINGLEDEL ops to DELETE ops
41239f8 db: add test demonstrating current SINGLEDEL behavior
```

Release note: none

Release justification: non-production code changes, and bug fix
necessary for a release blocker.

69974: backupccl: set sqlstats testing knobs for scheduled job test r=maryliag,miretskiy a=Azhng

Previsouly, backupccl tests did not set sql stats AOST testing knob
to override the AOST behavior. This causes sql stats error
stack trace to show up in backupccl tests.
This commit added sql stats testing knobs for backupccl test
helpers to mitigate this.

Release justification: Non-production code changes

Release note: None

Co-authored-by: Alex Sarkesian <sarkesian@cockroachlabs.com>
Co-authored-by: Rebecca Taft <becca@cockroachlabs.com>
Co-authored-by: Azhng <archer.xn@gmail.com>
Co-authored-by: Jackson Owens <jackson@cockroachlabs.com>
  • Loading branch information
5 people committed Sep 10, 2021
6 parents 62c5be5 + 6864a06 + 2939e1d + 8ef68b8 + 1e11326 + 7e51c9b commit 5b40b19
Show file tree
Hide file tree
Showing 43 changed files with 16,243 additions and 15,936 deletions.
4 changes: 2 additions & 2 deletions DEPS.bzl
Original file line number Diff line number Diff line change
Expand Up @@ -648,8 +648,8 @@ def go_deps():
name = "com_github_cockroachdb_pebble",
build_file_proto_mode = "disable_global",
importpath = "github.com/cockroachdb/pebble",
sum = "h1:loKopB3apOciC6CvBEVLkbIq4t8DV+UfinxwekbuIjQ=",
version = "v0.0.0-20210907203317-3f8702a60cc1",
sum = "h1:LdrRwalGCnBc7GXqtzShgVvf7m02F7FK6YmziC+cs7M=",
version = "v0.0.0-20210909162603-6c12d67b83e6",
)

go_repository(
Expand Down
2 changes: 1 addition & 1 deletion go.mod
Original file line number Diff line number Diff line change
Expand Up @@ -44,7 +44,7 @@ require (
github.com/cockroachdb/go-test-teamcity v0.0.0-20191211140407-cff980ad0a55
github.com/cockroachdb/gostdlib v1.13.0
github.com/cockroachdb/logtags v0.0.0-20190617123548-eb05cc24525f
github.com/cockroachdb/pebble v0.0.0-20210907203317-3f8702a60cc1
github.com/cockroachdb/pebble v0.0.0-20210909162603-6c12d67b83e6
github.com/cockroachdb/redact v1.1.3
github.com/cockroachdb/returncheck v0.0.0-20200612231554-92cdbca611dd
github.com/cockroachdb/sentry-go v0.6.1-cockroachdb.2
Expand Down
4 changes: 2 additions & 2 deletions go.sum
Original file line number Diff line number Diff line change
Expand Up @@ -279,8 +279,8 @@ github.com/cockroachdb/gostdlib v1.13.0 h1:TzSEPYgkKDNei3gbLc0rrHu4iHyBp7/+NxPOF
github.com/cockroachdb/gostdlib v1.13.0/go.mod h1:eXX95p9QDrYwJfJ6AgeN9QnRa/lqqid9LAzWz/l5OgA=
github.com/cockroachdb/logtags v0.0.0-20190617123548-eb05cc24525f h1:o/kfcElHqOiXqcou5a3rIlMc7oJbMQkeLk0VQJ7zgqY=
github.com/cockroachdb/logtags v0.0.0-20190617123548-eb05cc24525f/go.mod h1:i/u985jwjWRlyHXQbwatDASoW0RMlZ/3i9yJHE2xLkI=
github.com/cockroachdb/pebble v0.0.0-20210907203317-3f8702a60cc1 h1:loKopB3apOciC6CvBEVLkbIq4t8DV+UfinxwekbuIjQ=
github.com/cockroachdb/pebble v0.0.0-20210907203317-3f8702a60cc1/go.mod h1:JXfQr3d+XO4bL1pxGwKKo09xylQSdZ/mpZ9b2wfVcPs=
github.com/cockroachdb/pebble v0.0.0-20210909162603-6c12d67b83e6 h1:LdrRwalGCnBc7GXqtzShgVvf7m02F7FK6YmziC+cs7M=
github.com/cockroachdb/pebble v0.0.0-20210909162603-6c12d67b83e6/go.mod h1:JXfQr3d+XO4bL1pxGwKKo09xylQSdZ/mpZ9b2wfVcPs=
github.com/cockroachdb/redact v1.0.8/go.mod h1:BVNblN9mBWFyMyqK1k3AAiSxhvhfK2oOZZ2lK+dpvRg=
github.com/cockroachdb/redact v1.1.0/go.mod h1:BVNblN9mBWFyMyqK1k3AAiSxhvhfK2oOZZ2lK+dpvRg=
github.com/cockroachdb/redact v1.1.3 h1:AKZds10rFSIj7qADf0g46UixK8NNLwWTNdCIGS5wfSQ=
Expand Down
1 change: 1 addition & 0 deletions pkg/ccl/backupccl/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -198,6 +198,7 @@ go_test(
"//pkg/sql/rowflow",
"//pkg/sql/sem/tree",
"//pkg/sql/sessiondata",
"//pkg/sql/sqlstats",
"//pkg/sql/stats",
"//pkg/storage",
"//pkg/testutils",
Expand Down
4 changes: 4 additions & 0 deletions pkg/ccl/backupccl/create_scheduled_backup_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -32,6 +32,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/sql/parser"
"github.com/cockroachdb/cockroach/pkg/sql/sem/tree"
"github.com/cockroachdb/cockroach/pkg/sql/sessiondata"
"github.com/cockroachdb/cockroach/pkg/sql/sqlstats"
"github.com/cockroachdb/cockroach/pkg/testutils"
"github.com/cockroachdb/cockroach/pkg/testutils/serverutils"
"github.com/cockroachdb/cockroach/pkg/testutils/sqlutils"
Expand Down Expand Up @@ -90,6 +91,9 @@ func newTestHelper(t *testing.T) (*testHelper, func()) {
ExternalIODir: dir,
Knobs: base.TestingKnobs{
JobsTestingKnobs: knobs,
SQLStatsKnobs: &sqlstats.TestingKnobs{
AOSTClause: "AS OF SYSTEM TIME '-1us'",
},
},
}
s, db, _ := serverutils.StartServer(t, args)
Expand Down
12 changes: 6 additions & 6 deletions pkg/cli/debug.go
Original file line number Diff line number Diff line change
Expand Up @@ -269,12 +269,12 @@ func runDebugKeys(cmd *cobra.Command, args []string) error {
}
return strings.Join(pairs, ", "), nil
}
kvserver.DebugSprintKeyValueDecoders = append(kvserver.DebugSprintKeyValueDecoders, fn)
kvserver.DebugSprintMVCCKeyValueDecoders = append(kvserver.DebugSprintMVCCKeyValueDecoders, fn)
}
printer := printKey
if debugCtx.values {
printer = func(kv storage.MVCCKeyValue) (bool, error) {
kvserver.PrintKeyValue(kv)
kvserver.PrintMVCCKeyValue(kv)
return false, nil
}
}
Expand Down Expand Up @@ -512,7 +512,7 @@ func runDebugRangeDescriptors(cmd *cobra.Command, args []string) error {
if kvserver.IsRangeDescriptorKey(kv.Key) != nil {
return nil
}
kvserver.PrintKeyValue(kv)
kvserver.PrintMVCCKeyValue(kv)
return nil
})
}
Expand Down Expand Up @@ -583,7 +583,7 @@ Decode and print a hexadecimal-encoded key-value pair.
}
}

kvserver.PrintKeyValue(storage.MVCCKeyValue{
kvserver.PrintMVCCKeyValue(storage.MVCCKeyValue{
Key: k,
Value: bs[1],
})
Expand Down Expand Up @@ -646,7 +646,7 @@ func runDebugRaftLog(cmd *cobra.Command, args []string) error {

// NB: raft log does not have intents.
return db.MVCCIterate(start, end, storage.MVCCKeyIterKind, func(kv storage.MVCCKeyValue) error {
kvserver.PrintKeyValue(kv)
kvserver.PrintMVCCKeyValue(kv)
return nil
})
}
Expand Down Expand Up @@ -1497,7 +1497,7 @@ func (m mvccValueFormatter) Format(f fmt.State, c rune) {
errors.FormatError(m.err, f, c)
return
}
fmt.Fprint(f, kvserver.SprintKeyValue(m.kv, false /* printKey */))
fmt.Fprint(f, kvserver.SprintMVCCKeyValue(m.kv, false /* printKey */))
}

// lockValueFormatter is a fmt.Formatter for lock values.
Expand Down
1 change: 1 addition & 0 deletions pkg/kv/kvserver/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -388,6 +388,7 @@ go_test(
"@com_github_cockroachdb_errors//:errors",
"@com_github_cockroachdb_errors//oserror",
"@com_github_cockroachdb_logtags//:logtags",
"@com_github_cockroachdb_pebble//:pebble",
"@com_github_cockroachdb_redact//:redact",
"@com_github_gogo_protobuf//proto",
"@com_github_google_btree//:btree",
Expand Down
105 changes: 59 additions & 46 deletions pkg/kv/kvserver/debug_print.go
Original file line number Diff line number Diff line change
Expand Up @@ -27,30 +27,67 @@ import (
"go.etcd.io/etcd/raft/v3/raftpb"
)

// PrintKeyValue attempts to pretty-print the specified MVCCKeyValue to
// PrintEngineKeyValue attempts to print the given key-value pair to
// os.Stdout, utilizing SprintMVCCKeyValue in the case of an MVCCKeyValue.
func PrintEngineKeyValue(k storage.EngineKey, v []byte) {
fmt.Println(SprintEngineKeyValue(k, v))
}

// PrintMVCCKeyValue attempts to pretty-print the specified MVCCKeyValue to
// os.Stdout, falling back to '%q' formatting.
func PrintKeyValue(kv storage.MVCCKeyValue) {
fmt.Println(SprintKeyValue(kv, true /* printKey */))
func PrintMVCCKeyValue(kv storage.MVCCKeyValue) {
fmt.Println(SprintMVCCKeyValue(kv, true /* printKey */))
}

// SprintKey pretty-prints the specified MVCCKey.
func SprintKey(key storage.MVCCKey) string {
// SprintEngineKey pretty-prints the specified EngineKey, using the correct
// MVCC or Lock Table version formatting.
func SprintEngineKey(key storage.EngineKey) string {
if key.IsMVCCKey() {
if mvccKey, err := key.ToMVCCKey(); err == nil {
return SprintMVCCKey(mvccKey)
}
}

return fmt.Sprintf("%s %x (%#x): ", key.Key, key.Version, key.Encode())
}

// SprintMVCCKey pretty-prints the specified MVCCKey.
func SprintMVCCKey(key storage.MVCCKey) string {
return fmt.Sprintf("%s %s (%#x): ", key.Timestamp, key.Key, storage.EncodeKey(key))
}

// DebugSprintKeyValueDecoders allows injecting alternative debug decoders.
var DebugSprintKeyValueDecoders []func(kv storage.MVCCKeyValue) (string, error)
// SprintEngineKeyValue is like PrintEngineKeyValue, but returns a string. In
// the case of an MVCCKey, it will utilize SprintMVCCKeyValue for proper MVCC
// formatting.
func SprintEngineKeyValue(k storage.EngineKey, v []byte) string {
if k.IsMVCCKey() {
if key, err := k.ToMVCCKey(); err == nil {
return SprintMVCCKeyValue(storage.MVCCKeyValue{Key: key, Value: v}, true /* printKey */)
}
}
var sb strings.Builder
fmt.Fprintf(&sb, "%s %x (%#x): ", k.Key, k.Version, k.Encode())
if out, err := tryIntent(storage.MVCCKeyValue{Value: v}); err == nil {
sb.WriteString(out)
} else {
fmt.Fprintf(&sb, "%x", v)
}
return sb.String()
}

// DebugSprintMVCCKeyValueDecoders allows injecting alternative debug decoders.
var DebugSprintMVCCKeyValueDecoders []func(kv storage.MVCCKeyValue) (string, error)

// SprintKeyValue is like PrintKeyValue, but returns a string. If
// SprintMVCCKeyValue is like PrintMVCCKeyValue, but returns a string. If
// printKey is true, prints the key and the value together; otherwise,
// prints just the value.
func SprintKeyValue(kv storage.MVCCKeyValue, printKey bool) string {
func SprintMVCCKeyValue(kv storage.MVCCKeyValue, printKey bool) string {
var sb strings.Builder
if printKey {
sb.WriteString(SprintKey(kv.Key))
sb.WriteString(SprintMVCCKey(kv.Key))
}

decoders := append(DebugSprintKeyValueDecoders,
decoders := append(DebugSprintMVCCKeyValueDecoders,
tryRaftLogEntry,
tryRangeDescriptor,
tryMeta,
Expand Down Expand Up @@ -126,46 +163,40 @@ func decodeWriteBatch(writeBatch *kvserverpb.WriteBatch) (string, error) {
for r.Next() {
switch r.BatchType() {
case storage.BatchTypeDeletion:
mvccKey, err := r.MVCCKey()
engineKey, err := r.EngineKey()
if err != nil {
return sb.String(), err
}
sb.WriteString(fmt.Sprintf("Delete: %s\n", SprintKey(mvccKey)))
sb.WriteString(fmt.Sprintf("Delete: %s\n", SprintEngineKey(engineKey)))
case storage.BatchTypeValue:
mvccKey, err := r.MVCCKey()
engineKey, err := r.EngineKey()
if err != nil {
return sb.String(), err
}
sb.WriteString(fmt.Sprintf("Put: %s\n", SprintKeyValue(storage.MVCCKeyValue{
Key: mvccKey,
Value: r.Value(),
}, true /* printKey */)))
sb.WriteString(fmt.Sprintf("Put: %s\n", SprintEngineKeyValue(engineKey, r.Value())))
case storage.BatchTypeMerge:
mvccKey, err := r.MVCCKey()
engineKey, err := r.EngineKey()
if err != nil {
return sb.String(), err
}
sb.WriteString(fmt.Sprintf("Merge: %s\n", SprintKeyValue(storage.MVCCKeyValue{
Key: mvccKey,
Value: r.Value(),
}, true /* printKey */)))
sb.WriteString(fmt.Sprintf("Merge: %s\n", SprintEngineKeyValue(engineKey, r.Value())))
case storage.BatchTypeSingleDeletion:
mvccKey, err := r.MVCCKey()
engineKey, err := r.EngineKey()
if err != nil {
return sb.String(), err
}
sb.WriteString(fmt.Sprintf("Single Delete: %s\n", SprintKey(mvccKey)))
sb.WriteString(fmt.Sprintf("Single Delete: %s\n", SprintEngineKey(engineKey)))
case storage.BatchTypeRangeDeletion:
mvccStartKey, err := r.MVCCKey()
engineStartKey, err := r.EngineKey()
if err != nil {
return sb.String(), err
}
mvccEndKey, err := r.MVCCEndKey()
engineEndKey, err := r.EngineEndKey()
if err != nil {
return sb.String(), err
}
sb.WriteString(fmt.Sprintf(
"Delete Range: [%s, %s)\n", SprintKey(mvccStartKey), SprintKey(mvccEndKey),
"Delete Range: [%s, %s)\n", SprintEngineKey(engineStartKey), SprintEngineKey(engineEndKey),
))
default:
sb.WriteString(fmt.Sprintf("unsupported batch type: %d\n", r.BatchType()))
Expand Down Expand Up @@ -386,21 +417,3 @@ func (s *stringifyWriteBatch) String() string {
}
return fmt.Sprintf("failed to stringify write batch (%x): %s", s.Data, err)
}

// PrintEngineKeyValue attempts to print the given key-value pair.
func PrintEngineKeyValue(k storage.EngineKey, v []byte) {
if k.IsMVCCKey() {
if key, err := k.ToMVCCKey(); err == nil {
PrintKeyValue(storage.MVCCKeyValue{Key: key, Value: v})
return
}
}
var sb strings.Builder
fmt.Fprintf(&sb, "%s %x (%#x): ", k.Key, k.Version, k.Encode())
if out, err := tryIntent(storage.MVCCKeyValue{Value: v}); err == nil {
sb.WriteString(out)
} else {
fmt.Fprintf(&sb, "%x", v)
}
fmt.Println(sb.String())
}
16 changes: 16 additions & 0 deletions pkg/kv/kvserver/debug_print_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -11,6 +11,7 @@
package kvserver

import (
"encoding/hex"
"math"
"testing"

Expand All @@ -20,6 +21,8 @@ import (
"github.com/cockroachdb/cockroach/pkg/util/hlc"
"github.com/cockroachdb/cockroach/pkg/util/leaktest"
"github.com/cockroachdb/cockroach/pkg/util/log"
"github.com/cockroachdb/pebble"
"github.com/stretchr/testify/require"
)

func TestStringifyWriteBatch(t *testing.T) {
Expand All @@ -42,4 +45,17 @@ func TestStringifyWriteBatch(t *testing.T) {
if str, expStr := swb.String(), "Put: 9223372036.854775807,0 \"/db1\" (0x2f646231007fffffffffffffff09): \"test value\"\n"; str != expStr {
t.Errorf("expected %q for stringified write batch; got %q", expStr, str)
}

var err error
batch := pebble.Batch{}
encodedKey, err := hex.DecodeString("017a6b12c089f704918df70bee8800010003623a9318c0384d07a6f22b858594df6012")
require.NoError(t, err)
err = batch.SingleDelete(encodedKey, nil)
require.NoError(t, err)
wb.Data = batch.Repr()
swb = stringifyWriteBatch(wb)
if str, expStr := swb.String(), "Single Delete: /Local/Lock/Intent/Table/56/1/1169/5/3054/0 "+
"03623a9318c0384d07a6f22b858594df60 (0x017a6b12c089f704918df70bee8800010003623a9318c0384d07a6f22b858594df6012): \n"; str != expStr {
t.Errorf("expected %q for stringified write batch; got %q", expStr, str)
}
}
2 changes: 1 addition & 1 deletion pkg/kv/kvserver/replica_consistency_diff.go
Original file line number Diff line number Diff line change
Expand Up @@ -52,7 +52,7 @@ func (rsds ReplicaSnapshotDiffSlice) SafeFormat(buf redact.SafePrinter, _ rune)
buf.Printf(format,
prefix, d.Timestamp, d.Key,
prefix, d.Timestamp.GoTime(),
prefix, SprintKeyValue(storage.MVCCKeyValue{Key: mvccKey, Value: d.Value}, false /* printKey */),
prefix, SprintMVCCKeyValue(storage.MVCCKeyValue{Key: mvccKey, Value: d.Value}, false /* printKey */),
prefix, storage.EncodeKey(mvccKey), d.Value)
}
}
Expand Down
2 changes: 1 addition & 1 deletion pkg/kv/kvserver/replica_raft.go
Original file line number Diff line number Diff line change
Expand Up @@ -1994,7 +1994,7 @@ func (r *Replica) printRaftTail(
Key: mvccKey,
Value: it.Value(),
}
sb.WriteString(truncateEntryString(SprintKeyValue(kv, true /* printKey */), 2000))
sb.WriteString(truncateEntryString(SprintMVCCKeyValue(kv, true /* printKey */), 2000))
sb.WriteRune('\n')

valid, err := it.PrevEngineKey()
Expand Down
2 changes: 2 additions & 0 deletions pkg/sql/logictest/testdata/logic_test/distsql_stats
Original file line number Diff line number Diff line change
Expand Up @@ -1093,6 +1093,7 @@ FROM [SHOW STATISTICS USING JSON FOR TABLE all_null]
}
],
"histo_col_type": "INT8",
"histo_version": 1,
"name": "s",
"null_count": 0,
"row_count": 1
Expand Down Expand Up @@ -1141,6 +1142,7 @@ FROM [SHOW STATISTICS USING JSON FOR TABLE greeting_stats]
}
],
"histo_col_type": "public.greeting",
"histo_version": 1,
"name": "s",
"null_count": 0,
"row_count": 1
Expand Down
Loading

0 comments on commit 5b40b19

Please sign in to comment.