Skip to content

Commit

Permalink
Merge #82017 #82088
Browse files Browse the repository at this point in the history
82017: storage: add `IterOptions.RangeKeyMaskingBelow` r=jbowens a=erikgrinaker

This patch adds an iterator option `RangeKeyMaskingBelow`, which will
cause range keys at or below the given timestamp to mask any point keys
below them. This can be used to optimize callers that do not care about
deleted point keys, since Pebble will not emit these.

Touches #70412.

Release note: None

82088: roachtest: extract version map into separate file r=tbg a=rail

Previously, we had to patch the code in order to update the version
upgrade test's version mapping. This made automation a bit complicated,
thus it was done manually.

Additionally, when we upgraded to a new major version, we would need to
patch the same test in order to generate fixtures.

This patch extracts the version mapping into a separate file, which will
be easier to patch using automation. Additionally, it extracts the fixture
generation code into a separate test which is skipped by CI, but can be
run manually.

Release note: None

Co-authored-by: Erik Grinaker <[email protected]>
Co-authored-by: Rail Aliiev <[email protected]>
  • Loading branch information
3 people committed Jun 7, 2022
3 parents 873c397 + 28e600d + d56b11b commit 3b5aaee
Show file tree
Hide file tree
Showing 13 changed files with 636 additions and 73 deletions.
2 changes: 2 additions & 0 deletions pkg/cmd/roachtest/tests/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -40,6 +40,7 @@ go_library(
"encryption.go",
"engine_switch.go",
"event_log.go",
"fixtures.go",
"flowable.go",
"follower_reads.go",
"go_helpers.go",
Expand Down Expand Up @@ -143,6 +144,7 @@ go_library(
"versionupgrade.go",
"ycsb.go",
],
embedsrcs = ["predecessor_version.json"],
importpath = "github.com/cockroachdb/cockroach/pkg/cmd/roachtest/tests",
visibility = ["//visibility:public"],
deps = [
Expand Down
70 changes: 70 additions & 0 deletions pkg/cmd/roachtest/tests/fixtures.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,70 @@
// Copyright 2018 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 tests

import (
"context"
"strings"
"time"

"github.com/cockroachdb/cockroach/pkg/cmd/roachtest/cluster"
"github.com/cockroachdb/cockroach/pkg/cmd/roachtest/registry"
"github.com/cockroachdb/cockroach/pkg/cmd/roachtest/test"
)

func registerFixtures(r registry.Registry) {
// Run this test to create a new fixture for the version upgrade test. This
// is necessary after every release. For example, the day `master` becomes
// the 20.2 release, this test will fail because it is missing a fixture for
// 20.1; run the test (on 20.1). Check it in (instructions will be logged
// below) and off we go.
//
// The version to create/update the fixture for. Must be released (i.e.
// can download it from the homepage); if that is not the case use the
// empty string which uses the local cockroach binary. Make sure that
// this binary then has the correct version. For example, to make a
// "v20.2" fixture, you will need a binary that has "v20.2" in the
// output of `./cockroach version`, and this process will end up
// creating fixtures that have "v20.2" in them. This would be part
// of tagging the master branch as v21.1 in the process of going
// through the major release for v20.2. The version is passed in as
// FIXTURE_VERSION environment variable.
//
// In the common case, one should populate this with the version (instead of
// using the empty string) as this is the most straightforward and least
// error-prone way to generate the fixtures.
//
// Please note that you do *NOT* need to update the fixtures in a patch
// release. This only happens as part of preparing the master branch for the
// next release. The release team runbooks, at time of writing, reflect
// this.
//
// Example invocation:
// roachtest --local run generate-fixtures --debug --cockroach ./cockroach \
// --build-tag v22.1.0-beta.3 tag:fixtures
runFixtures := func(
ctx context.Context,
t test.Test,
c cluster.Cluster,
) {
fixtureVersion := strings.TrimPrefix(t.BuildVersion().String(), "v")
makeVersionFixtureAndFatal(ctx, t, c, fixtureVersion)
}
spec := registry.TestSpec{
Name: "generate-fixtures",
Timeout: 30 * time.Minute,
Tags: []string{"fixtures"},
Owner: registry.OwnerDevInf,
Cluster: r.MakeClusterSpec(4),
Run: runFixtures,
}
r.Add(spec)
}
48 changes: 28 additions & 20 deletions pkg/cmd/roachtest/tests/predecessor_version.go
Original file line number Diff line number Diff line change
Expand Up @@ -11,12 +11,36 @@
package tests

import (
// Import embed for the version map
_ "embed"
"encoding/json"
"fmt"

"github.com/cockroachdb/cockroach/pkg/util/version"
"github.com/cockroachdb/errors"
)

// You can update the values in predecessor_version.json to point at newer patch releases.
//
// NB: If a new key was added (e.g. if you're releasing a new major
// release), you'll also need to regenerate fixtures. To regenerate
// fixtures, you will need to run acceptance/version-upgrade with the
// checkpoint option enabled to create the missing store directory
// fixture (see runVersionUpgrade).
//go:embed predecessor_version.json
var verMapJSON []byte

type versionMap map[string]string

func unmarshalVersionMap() (versionMap, error) {
var res versionMap
err := json.Unmarshal(verMapJSON, &res)
if err != nil {
return versionMap{}, err
}
return res, nil
}

// PredecessorVersion returns a recent predecessor of the build version (i.e.
// the build tag of the main binary). For example, if the running binary is from
// the master branch prior to releasing 19.2.0, this will return a recent
Expand All @@ -26,27 +50,11 @@ func PredecessorVersion(buildVersion version.Version) (string, error) {
return "", errors.Errorf("buildVersion not set")
}

buildVersionMajorMinor := fmt.Sprintf("%d.%d", buildVersion.Major(), buildVersion.Minor())

// You can update the values in verMap to point at newer patch releases.
//
// NB: If a new key was added (e.g. if you're releasing a new major
// release), you'll also need to regenerate fixtures. To regenerate
// fixtures, you will need to run acceptance/version-upgrade with the
// checkpoint option enabled to create the missing store directory
// fixture (see runVersionUpgrade).
verMap := map[string]string{
"22.2": "22.1.0",
"22.1": "21.2.7",
"21.2": "21.1.12",
"21.1": "20.2.12",
"20.2": "20.1.16",
"20.1": "19.2.11",
"19.2": "19.1.11",
"19.1": "2.1.9",
"2.2": "2.1.9",
"2.1": "2.0.7",
verMap, err := unmarshalVersionMap()
if err != nil {
return "", errors.Wrap(err, "cannot load version map")
}
buildVersionMajorMinor := fmt.Sprintf("%d.%d", buildVersion.Major(), buildVersion.Minor())
v, ok := verMap[buildVersionMajorMinor]
if !ok {
return "", errors.Errorf("prev version not set for version: %s", buildVersionMajorMinor)
Expand Down
12 changes: 12 additions & 0 deletions pkg/cmd/roachtest/tests/predecessor_version.json
Original file line number Diff line number Diff line change
@@ -0,0 +1,12 @@
{
"19.1": "2.1.9",
"19.2": "19.1.11",
"2.1": "2.0.7",
"2.2": "2.1.9",
"20.1": "19.2.11",
"20.2": "20.1.16",
"21.1": "20.2.12",
"21.2": "21.1.12",
"22.1": "21.2.7",
"22.2": "22.1.1"
}
1 change: 1 addition & 0 deletions pkg/cmd/roachtest/tests/registry.go
Original file line number Diff line number Diff line change
Expand Up @@ -40,6 +40,7 @@ func RegisterTests(r registry.Registry) {
registerDrop(r)
registerEncryption(r)
registerEngineSwitch(r)
registerFixtures(r)
registerFlowable(r)
registerFollowerReads(r)
registerGopg(r)
Expand Down
30 changes: 1 addition & 29 deletions pkg/cmd/roachtest/tests/versionupgrade.go
Original file line number Diff line number Diff line change
Expand Up @@ -86,34 +86,6 @@ func runVersionUpgrade(ctx context.Context, t test.Test, c cluster.Cluster) {
t.Fatal(err)
}

// Set the bool within to true to create a new fixture for this test. This
// is necessary after every release. For example, the day `master` becomes
// the 20.2 release, this test will fail because it is missing a fixture for
// 20.1; run the test (on 20.1) with the bool flipped to create the fixture.
// Check it in (instructions will be logged below) and off we go.
if false {
// The version to create/update the fixture for. Must be released (i.e.
// can download it from the homepage); if that is not the case use the
// empty string which uses the local cockroach binary. Make sure that
// this binary then has the correct version. For example, to make a
// "v20.2" fixture, you will need a binary that has "v20.2" in the
// output of `./cockroach version`, and this process will end up
// creating fixtures that have "v20.2" in them. This would be part
// of tagging the master branch as v21.1 in the process of going
// through the major release for v20.2.
//
// In the common case, one should populate this with the version (instead of
// using the empty string) as this is the most straightforward and least
// error-prone way to generate the fixtures.
//
// Please note that you do *NOT* need to update the fixtures in a patch
// release. This only happens as part of preparing the master branch for the
// next release. The release team runbooks, at time of writing, reflect
// this.
makeFixtureVersion := "22.1.0-beta.3" // for 21.2 release in late 2021
makeVersionFixtureAndFatal(ctx, t, c, makeFixtureVersion)
}

testFeaturesStep := versionUpgradeTestFeatures.step(c.All())
schemaChangeStep := runSchemaChangeWorkloadStep(c.All().RandNode()[0], 10 /* maxOps */, 2 /* concurrency */)
// TODO(irfansharif): All schema change instances were commented out while
Expand Down Expand Up @@ -634,7 +606,7 @@ result:
for i in 1 2 3 4; do
mkdir -p pkg/cmd/roachtest/fixtures/${i} && \
mv artifacts/acceptance/version-upgrade/run_1/logs/${i}.unredacted/checkpoint-*.tgz \
mv artifacts/generate-fixtures/run_1/logs/${i}.unredacted/checkpoint-*.tgz \
pkg/cmd/roachtest/fixtures/${i}/
done
`)
Expand Down
14 changes: 14 additions & 0 deletions pkg/storage/engine.go
Original file line number Diff line number Diff line change
Expand Up @@ -398,6 +398,20 @@ type IterOptions struct {
// TODO(erikgrinaker): Consider separating the options structs for
// EngineIterator and MVCCIterator.
KeyTypes IterKeyType
// RangeKeyMaskingBelow enables masking (hiding) of point keys by range keys.
// Any range key with a timestamp at or below RangeKeyMaskingBelow
// will mask point keys below it, preventing them from being surfaced.
// Consider the following example:
//
// 4 o---------------o RangeKeyMaskingBelow=4 emits b3
// 3 b3 d3 RangeKeyMaskingBelow=3 emits b3,d3,f2
// 2 o---------------o f2 RangeKeyMaskingBelow=2 emits b3,d3,f2
// 1 a1 b1 o-------o RangeKeyMaskingBelow=1 emits a1,b3,b1,d3,f2
// a b c d e f g
//
// Range keys themselves are not affected by the masking, and will be
// emitted as normal.
RangeKeyMaskingBelow hlc.Timestamp
// useL6Filters allows the caller to opt into reading filter blocks for
// L6 sstables. Only for use with Prefix = true. Helpful if a lot of prefix
// Seeks are expected in quick succession, that are also likely to not
Expand Down
5 changes: 3 additions & 2 deletions pkg/storage/engine_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -2086,8 +2086,9 @@ func TestEngineRangeKeysUnsupported(t *testing.T) {
require.False(t, r.SupportsRangeKeys())

iter := r.NewMVCCIterator(MVCCKeyAndIntentsIterKind, IterOptions{
KeyTypes: keyType,
UpperBound: keys.MaxKey,
KeyTypes: keyType,
UpperBound: keys.MaxKey,
RangeKeyMaskingBelow: hlc.Timestamp{WallTime: 1}, // should get disabled when unsupported
})
defer iter.Close()

Expand Down
2 changes: 2 additions & 0 deletions pkg/storage/intent_interleaving_iter.go
Original file line number Diff line number Diff line change
Expand Up @@ -21,6 +21,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/roachpb"
"github.com/cockroachdb/cockroach/pkg/storage/enginepb"
"github.com/cockroachdb/cockroach/pkg/util"
"github.com/cockroachdb/cockroach/pkg/util/hlc"
"github.com/cockroachdb/cockroach/pkg/util/protoutil"
"github.com/cockroachdb/cockroach/pkg/util/uuid"
"github.com/cockroachdb/errors"
Expand Down Expand Up @@ -222,6 +223,7 @@ func newIntentInterleavingIterator(reader Reader, opts IterOptions) MVCCIterator
return reader.NewMVCCIterator(MVCCKeyIterKind, opts)
}
intentOpts.KeyTypes = IterKeyTypePointsOnly
intentOpts.RangeKeyMaskingBelow = hlc.Timestamp{}

iiIter := intentInterleavingIterPool.Get().(*intentInterleavingIter)
intentKeyBuf := iiIter.intentKeyBuf
Expand Down
5 changes: 4 additions & 1 deletion pkg/storage/mvcc_history_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -69,7 +69,7 @@ import (
// get [t=<name>] [ts=<int>[,<int>]] [resolve [status=<txnstatus>]] k=<key> [inconsistent] [tombstones] [failOnMoreRecent] [localUncertaintyLimit=<int>[,<int>]] [globalUncertaintyLimit=<int>[,<int>]]
// scan [t=<name>] [ts=<int>[,<int>]] [resolve [status=<txnstatus>]] k=<key> [end=<key>] [inconsistent] [tombstones] [reverse] [failOnMoreRecent] [localUncertaintyLimit=<int>[,<int>]] [globalUncertaintyLimit=<int>[,<int>]] [max=<max>] [targetbytes=<target>] [avoidExcess] [allowEmpty]
//
// iter_new [k=<key>] [end=<key>] [prefix] [kind=key|keyAndIntents] [types=pointsOnly|pointsWithRanges|pointsAndRanges|rangesOnly]
// iter_new [k=<key>] [end=<key>] [prefix] [kind=key|keyAndIntents] [types=pointsOnly|pointsWithRanges|pointsAndRanges|rangesOnly] [maskBelow=<int>[,<int>]]
// iter_seek_ge k=<key> [ts=<int>[,<int>]]
// iter_seek_lt k=<key> [ts=<int>[,<int>]]
// iter_seek_intent_ge k=<key> txn=<name>
Expand Down Expand Up @@ -983,6 +983,9 @@ func cmdIterNew(e *evalCtx) error {
return errors.Errorf("unknown key type %s", arg)
}
}
if e.hasArg("maskBelow") {
opts.RangeKeyMaskingBelow = e.getTsWithName("maskBelow")
}

var r, closeReader Reader
rType := util.ConstantWithMetamorphicTestChoice(
Expand Down
19 changes: 16 additions & 3 deletions pkg/storage/mvcc_key.go
Original file line number Diff line number Diff line change
Expand Up @@ -219,13 +219,26 @@ func encodeMVCCTimestamp(ts hlc.Timestamp) []byte {
// representation, including the length suffix but excluding the sentinel byte.
// This is equivalent to the Pebble suffix.
func EncodeMVCCTimestampSuffix(ts hlc.Timestamp) []byte {
return encodeMVCCTimestampSuffixToBuf(nil, ts)
}

// encodeMVCCTimestampSuffixToBuf encodes an MVCC timestamp into its Pebble
// representation, including the length suffix but excluding the sentinel byte.
// This is equivalent to the Pebble suffix. It reuses the given byte buffer if
// it has sufficient capacity.
func encodeMVCCTimestampSuffixToBuf(buf []byte, ts hlc.Timestamp) []byte {
tsLen := encodedMVCCTimestampLength(ts)
if tsLen == 0 {
return nil
return buf[:0]
}
suffixLen := tsLen + mvccEncodedTimeLengthLen
if cap(buf) < suffixLen {
buf = make([]byte, suffixLen)
} else {
buf = buf[:suffixLen]
}
buf := make([]byte, tsLen+mvccEncodedTimeLengthLen)
encodeMVCCTimestampToBuf(buf, ts)
buf[tsLen] = byte(tsLen + mvccEncodedTimeLengthLen)
buf[tsLen] = byte(suffixLen)
return buf
}

Expand Down
Loading

0 comments on commit 3b5aaee

Please sign in to comment.