Skip to content

Commit

Permalink
Browse files Browse the repository at this point in the history
93414: docs: update some paper links r=erikgrinaker a=tbg

The previous links weren't world-readable.

Epic: None
Release note: None

93496: .github: update CODEOWNERS r=erikgrinaker a=tbg

We've had the Replication team for a while and it's about time it
started formally owning some files.

I arrived at these updates by going through

```
go run ./pkg/cmd/whoownsit --walk ./pkg/kv/kvserver/
```

I then noticed Github's validation of the CODEOWNERs file failed, which inspired the subsequent commits. They introduce a `#!` syntax such that we can selectively hide teams from Github.

cc `@dt` and `@nickvigilante,` the `@cockroachdb/tenant-streaming` and `@cockroachdb/docs-infra-prs` need the repo added to them so that Github accepts them. I worked around that for now by using the newly introduced `#!` but the teams won't get PR pings.

Epic: none
Release note: None


93743: opt: reduce allocations in props.histogramIter r=mgartner a=mgartner

This commit reduces allocations in props.histogramIter by delaying datum allocations for bucket bounds until absolutely necessary. Instead, we prefer using the upper bound of the previous bucket as an exclusive lower bound of the current bucket because no datum allocation is necessary.

This significantly reduces the number allocations, especially when filtering a histogram by a span that includes many buckets. It reduces the number of bytes allocated, especially when the histogram bounds are larger values, such as strings larger than several bytes. The benefits can be observed in one of the optimizer micro-benchmarks:

```
name                                                                    old time/op    new time/op    delta
Phases/single-col-histogram-range/Simple/Parse-16                         13.1µs ±19%    10.5µs ± 2%  -19.66%  (p=0.008 n=5+5)
Phases/single-col-histogram-range/Simple/OptBuildNoNorm-16                95.5µs ± 3%    77.4µs ±36%     ~     (p=0.190 n=4+5)
Phases/single-col-histogram-range/Simple/OptBuildNorm-16                   144µs ± 5%     108µs ± 3%  -24.76%  (p=0.008 n=5+5)
Phases/single-col-histogram-range/Simple/Explore-16                        176µs ±17%     114µs ± 2%  -34.93%  (p=0.008 n=5+5)
Phases/single-col-histogram-range/Simple/ExecBuild-16                      161µs ± 2%     121µs ± 1%  -24.61%  (p=0.008 n=5+5)
Phases/single-col-histogram-range/Prepared/AssignPlaceholdersNoNorm-16    56.2µs ± 2%    41.8µs ± 5%  -25.69%  (p=0.008 n=5+5)
Phases/single-col-histogram-range/Prepared/AssignPlaceholdersNorm-16      55.8µs ± 3%    42.1µs ± 3%  -24.53%  (p=0.008 n=5+5)
Phases/single-col-histogram-range/Prepared/Explore-16                     65.0µs ± 8%    50.5µs ± 5%  -22.24%  (p=0.008 n=5+5)
Phases/single-col-histogram-range/Prepared/ExecBuild-16                   73.0µs ± 3%    57.4µs ± 7%  -21.48%  (p=0.008 n=5+5)

name                                                                    old alloc/op   new alloc/op   delta
Phases/single-col-histogram-range/Simple/Parse-16                         3.25kB ± 0%    3.25kB ± 0%     ~     (all equal)
Phases/single-col-histogram-range/Simple/OptBuildNoNorm-16                33.5kB ± 0%    17.9kB ± 0%  -46.52%  (p=0.008 n=5+5)
Phases/single-col-histogram-range/Simple/OptBuildNorm-16                  57.4kB ± 0%    26.2kB ± 0%  -54.37%  (p=0.008 n=5+5)
Phases/single-col-histogram-range/Simple/Explore-16                       60.2kB ± 0%    29.0kB ± 0%  -51.81%  (p=0.008 n=5+5)
Phases/single-col-histogram-range/Simple/ExecBuild-16                     61.7kB ± 0%    30.5kB ± 0%  -50.56%  (p=0.008 n=5+5)
Phases/single-col-histogram-range/Prepared/AssignPlaceholdersNoNorm-16    26.0kB ± 0%    10.5kB ± 0%  -59.83%  (p=0.008 n=5+5)
Phases/single-col-histogram-range/Prepared/AssignPlaceholdersNorm-16      26.0kB ± 0%    10.5kB ± 0%  -59.83%  (p=0.008 n=5+5)
Phases/single-col-histogram-range/Prepared/Explore-16                     28.3kB ± 0%    12.7kB ± 0%  -55.07%  (p=0.008 n=5+5)
Phases/single-col-histogram-range/Prepared/ExecBuild-16                   29.8kB ± 0%    14.2kB ± 0%  -52.32%  (p=0.008 n=5+5)

name                                                                    old allocs/op  new allocs/op  delta
Phases/single-col-histogram-range/Simple/Parse-16                           20.0 ± 0%      20.0 ± 0%     ~     (all equal)
Phases/single-col-histogram-range/Simple/OptBuildNoNorm-16                   608 ± 0%       282 ± 0%  -53.62%  (p=0.008 n=5+5)
Phases/single-col-histogram-range/Simple/OptBuildNorm-16                   1.15k ± 0%     0.50k ± 0%  -56.55%  (p=0.008 n=5+5)
Phases/single-col-histogram-range/Simple/Explore-16                        1.16k ± 0%     0.51k ± 0%  -56.06%  (p=0.008 n=5+5)
Phases/single-col-histogram-range/Simple/ExecBuild-16                      1.17k ± 0%     0.52k ± 0%  -55.63%  (p=0.008 n=5+5)
Phases/single-col-histogram-range/Prepared/AssignPlaceholdersNoNorm-16       564 ± 0%       238 ± 0%  -57.80%  (p=0.008 n=5+5)
Phases/single-col-histogram-range/Prepared/AssignPlaceholdersNorm-16         564 ± 0%       238 ± 0%  -57.80%  (p=0.008 n=5+5)
Phases/single-col-histogram-range/Prepared/Explore-16                        573 ± 0%       247 ± 0%  -56.89%  (p=0.008 n=5+5)
Phases/single-col-histogram-range/Prepared/ExecBuild-16                      582 ± 0%       256 ± 0%  -56.01%  (p=0.008 n=5+5)
```

Fixes #89982

Epic: None

Release note: None

93785: kvserver: revamp raft proto test r=erikgrinaker a=tbg

We previously had this setup where we put an interceptor into
`protoutil.Marshal`, ran all of the `kvserver` tests (or actually just
whichever ones were invoked... since you could run with filters or just
be a bazel shard), and collected the stacks seen for `protoutil.Marshal`
and the message types for which we saw them.

We then attempted to make sure that the set of protos that was marshaled
below raft (though the definition of "below raft" was fuzzy; the code
was really only looking at command application) was special in that
any changes to the marshaling would cause a test failure that would
have to be "signed off on" separately by updating the fixtures.

This was all too clever, and too magical. It also had a real downside:
by injecting its magic the package tests, it also slowed them down, and
in particular it would mess with benchmark results: reflection and extra
allocations can really skew the picture (which is what prompted this
commit).

This commit rips out the existing meta test and replaces it with a
vanilla Go test that keeps an explicit list of protos that we alert on
should their encoding change.  Thanks to `echotest`, it's now easy to
regenerate the fixtures if needed.

Epic: none
Release note: None


93802: bazci: ignore nil `testResult` r=healthy-pod a=rickystewart

Closes #93656.

Release note: None
Epic: None

Co-authored-by: Tobias Grieger <[email protected]>
Co-authored-by: Marcus Gartner <[email protected]>
Co-authored-by: Ricky Stewart <[email protected]>
  • Loading branch information
4 people committed Dec 16, 2022
6 parents a5acdc3 + a954f43 + a1a8ae7 + 2ae6439 + a4da07f + 890d84c commit cc63f9f
Show file tree
Hide file tree
Showing 20 changed files with 333 additions and 398 deletions.
211 changes: 145 additions & 66 deletions .github/CODEOWNERS

Large diffs are not rendered by default.

8 changes: 5 additions & 3 deletions TEAMS.yaml
Original file line number Diff line number Diff line change
Expand Up @@ -19,6 +19,8 @@

cockroachdb/docs:
triage_column_id: 3971225
aliases:
cockroachdb/docs-infra-prs: other
cockroachdb/sql-sessions:
aliases:
cockroachdb/sql-syntax-prs: other
Expand All @@ -45,7 +47,7 @@ cockroachdb/replication:
aliases:
cockroachdb/repl-prs: other
label: T-kv-replication
cockroachdb/geospatial:
cockroachdb/spatial:
triage_column_id: 9487269
cockroachdb/dev-inf:
triage_column_id: 10210759
Expand Down Expand Up @@ -73,9 +75,9 @@ cockroachdb/server:
cockroachdb/server-prs: other
cockroachdb/http-api-prs: other
triage_column_id: 2521812
cockroachdb/cluster-ui:
cockroachdb/admin-ui:
aliases:
cockroachdb/cluster-ui-prs: other
cockroachdb/admin-ui-prs: other
triage_column_id: 6598672
cockroachdb/obs-inf-prs:
triage_column_id: 14196277
Expand Down
4 changes: 2 additions & 2 deletions docs/design.md
Original file line number Diff line number Diff line change
Expand Up @@ -201,11 +201,11 @@ implementation of SSI still requires no locking, but will end up
aborting more transactions. Cockroach’s SI and SSI implementations
prevent starvation scenarios even for arbitrarily long transactions.

See the [Cahill paper](https://drive.google.com/file/d/0B9GCVTp_FHJIcEVyZVdDWEpYYXVVbFVDWElrYUV0NHFhU2Fv/edit?usp=sharing)
See the [Cahill paper](https://ses.library.usyd.edu.au/bitstream/handle/2123/5353/michael-cahill-2009-thesis.pdf)
for one possible implementation of SSI. This is another [great paper](http://cs.yale.edu/homes/thomson/publications/calvin-sigmod12.pdf).
For a discussion of SSI implemented by preventing read-write conflicts
(in contrast to detecting them, called write-snapshot isolation), see
the [Yabandeh paper](https://drive.google.com/file/d/0B9GCVTp_FHJIMjJ2U2t6aGpHLTFUVHFnMTRUbnBwc2pLa1RN/edit?usp=sharing),
the [Yabandeh paper](https://courses.cs.washington.edu/courses/cse444/10sp/544M/READING-LIST/fekete-sigmod2008.pdf),
which is the source of much inspiration for Cockroach’s SSI.

Both SI and SSI require that the outcome of reads must be preserved, i.e.
Expand Down
3 changes: 3 additions & 0 deletions pkg/cmd/bazci/bazci.go
Original file line number Diff line number Diff line change
Expand Up @@ -216,6 +216,9 @@ func (s *monitorBuildServer) handleBuildEvent(
if testResult.attempt > 1 {
outputDir = filepath.Join(outputDir, fmt.Sprintf("attempt_%d", testResult.attempt))
}
if testResult.testResult == nil {
continue
}
for _, output := range testResult.testResult.TestActionOutput {
if output.Name == "test.log" || output.Name == "test.xml" {
src := strings.TrimPrefix(output.GetUri(), "file://")
Expand Down
2 changes: 1 addition & 1 deletion pkg/internal/codeowners/codeowners.go
Original file line number Diff line number Diff line change
Expand Up @@ -53,7 +53,7 @@ func LoadCodeOwners(r io.Reader, teams map[team.Alias]team.Team) (*CodeOwners, e
if s.Err() != nil {
return nil, s.Err()
}
t := s.Text()
t := strings.Replace(s.Text(), "#!", "", -1)
if strings.HasPrefix(t, "#") {
continue
}
Expand Down
5 changes: 5 additions & 0 deletions pkg/internal/codeowners/codeowners_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -29,11 +29,14 @@ func TestMatch(t *testing.T) {
/b/ @cockroachdb/team-b-noreview
/a/b* @cockroachdb/team-b @cockroachdb/team-a
**/c/ @cockroachdb/team-c
#!/q/ @cockroachdb/team-q
/qq/ @cockroachdb/team-q #! @cockroachdb/team-b-noreview
`
teams := map[team.Alias]team.Team{
"cockroachdb/team-a": {},
"cockroachdb/team-b": {},
"cockroachdb/team-c": {},
"cockroachdb/team-q": {},
}

codeOwners, err := LoadCodeOwners(strings.NewReader(owners), teams)
Expand All @@ -49,6 +52,8 @@ func TestMatch(t *testing.T) {
{"a/bob", []team.Team{teams["cockroachdb/team-b"], teams["cockroachdb/team-a"]}},
{"no/owner/", nil},
{"hmm/what/about/c/file", []team.Team{teams["cockroachdb/team-c"]}},
{"q/foo.txt", []team.Team{teams["cockroachdb/team-q"]}},
{"qq/foo.txt", []team.Team{teams["cockroachdb/team-q"], teams["cockroachdb/team-b"]}},
}

for _, tc := range testCases {
Expand Down
1 change: 0 additions & 1 deletion pkg/kv/kvserver/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -94,7 +94,6 @@ go_library(
"stores_base.go",
"stores_server.go",
"testing_knobs.go",
"track_raft_protos.go",
"ts_maintenance_queue.go",
":gen-refreshraftreason-stringer", # keep
],
Expand Down
159 changes: 52 additions & 107 deletions pkg/kv/kvserver/below_raft_protos_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -14,53 +14,32 @@ import (
"fmt"
"hash/fnv"
"math/rand"
"reflect"
"regexp"
"testing"

"github.com/cockroachdb/cockroach/pkg/roachpb"
"github.com/cockroachdb/cockroach/pkg/storage/enginepb"
"github.com/cockroachdb/cockroach/pkg/testutils"
"github.com/cockroachdb/cockroach/pkg/testutils/echotest"
"github.com/cockroachdb/cockroach/pkg/util/leaktest"
"github.com/cockroachdb/cockroach/pkg/util/log"
"github.com/cockroachdb/cockroach/pkg/util/protoutil"
"github.com/stretchr/testify/require"
"go.etcd.io/etcd/raft/v3/raftpb"
)

func verifyHash(b []byte, expectedSum uint64) error {
hash := fnv.New64a()
if _, err := hash.Write(b); err != nil {
return err
}
if sum := hash.Sum64(); sum != expectedSum {
return fmt.Errorf("expected sum %d; got %d", expectedSum, sum)
}
return nil
}

// An arbitrary number chosen to seed the PRNGs used to populate the tested
// protos.
const goldenSeed = 1337

// The count of randomly populated protos that will be concatenated and hashed
// per proto type. Given that the population functions have a chance of leaving
// some fields zero-valued, this number must be greater than `1` to give this
// test a reasonable chance of encountering a non-zero value of every field.
const itersPerProto = 20

type fixture struct {
populatedConstructor func(*rand.Rand) protoutil.Message
emptySum, populatedSum uint64
}
// TestBelowRaftProtosDontChange is a manually curated list of protos that we
// use below Raft. Care must be taken to change these protos, as replica
// divergence could ensue (if the old code and the new code handle the updated
// or old proto differently). Changes to the encoding of these protos will be
// detected by this test. The expectations should only be updated after a
// reflection on the safety of the proposed change.
func TestBelowRaftProtosDontChange(t *testing.T) {
defer leaktest.AfterTest(t)()
defer log.Scope(t).Close(t)

// belowRaftGoldenProtos are protos that we use below Raft. Care must be
// taken to change these protos, as replica divergence could ensue (if the
// old code and the new code handle the updated or old proto differently).
// To reduce the chances of a bug like this, we track the protos that are
// used below Raft and fail on any changes to their structure. When a
// migration was put into place, the map below can be updated with the new
// emptySum and populatedSum for the proto that was changed.
var belowRaftGoldenProtos = map[reflect.Type]fixture{
reflect.TypeOf(&enginepb.MVCCMetadata{}): {
populatedConstructor: func(r *rand.Rand) protoutil.Message {
testCases := []func(r *rand.Rand) protoutil.Message{
func(r *rand.Rand) protoutil.Message {
m := enginepb.NewPopulatedMVCCMetadata(r, false)
m.Txn = nil // never populated below Raft
m.Timestamp.Synthetic = nil // never populated below Raft
Expand All @@ -70,18 +49,10 @@ var belowRaftGoldenProtos = map[reflect.Type]fixture{
m.TxnDidNotUpdateMeta = nil // never populated below Raft
return m
},
emptySum: 7551962144604783939,
populatedSum: 6170112718709472849,
},
reflect.TypeOf(&enginepb.RangeAppliedState{}): {
populatedConstructor: func(r *rand.Rand) protoutil.Message {
func(r *rand.Rand) protoutil.Message {
return enginepb.NewPopulatedRangeAppliedState(r, false)
},
emptySum: 10160370728048384381,
populatedSum: 13858955692092952193,
},
reflect.TypeOf(&raftpb.HardState{}): {
populatedConstructor: func(r *rand.Rand) protoutil.Message {
func(r *rand.Rand) protoutil.Message {
type expectedHardState struct {
Term uint64
Vote uint64
Expand All @@ -98,81 +69,55 @@ var belowRaftGoldenProtos = map[reflect.Type]fixture{
Commit: n % 11,
}
},
emptySum: 13621293256077144893,
populatedSum: 13375098491754757572,
},
// This is used downstream of Raft only to write it into unreplicated keyspace
// as part of VersionUnreplicatedRaftTruncatedState.
// However, it has been sent through Raft for a long time, as part of
// ReplicatedEvalResult.
reflect.TypeOf(&roachpb.RaftTruncatedState{}): {
populatedConstructor: func(r *rand.Rand) protoutil.Message {
func(r *rand.Rand) protoutil.Message {
// This is used downstream of Raft only to write it into unreplicated keyspace
// as part of VersionUnreplicatedRaftTruncatedState.
// However, it has been sent through Raft for a long time, as part of
// ReplicatedEvalResult.
return roachpb.NewPopulatedRaftTruncatedState(r, false)
},
emptySum: 5531676819244041709,
populatedSum: 14781226418259198098,
},
// These are marshaled below Raft by the Pebble merge operator. The Pebble
// merge operator can be called below Raft whenever a Pebble MVCCIterator is
// used.
reflect.TypeOf(&roachpb.InternalTimeSeriesData{}): {
populatedConstructor: func(r *rand.Rand) protoutil.Message {
func(r *rand.Rand) protoutil.Message {

// These are marshaled below Raft by the Pebble merge operator. The Pebble
// merge operator can be called below Raft whenever a Pebble MVCCIterator is
// used.
return roachpb.NewPopulatedInternalTimeSeriesData(r, false)
},
emptySum: 5531676819244041709,
populatedSum: 17471291891947207032,
},
reflect.TypeOf(&enginepb.MVCCMetadataSubsetForMergeSerialization{}): {
populatedConstructor: func(r *rand.Rand) protoutil.Message {
func(r *rand.Rand) protoutil.Message {
m := enginepb.NewPopulatedMVCCMetadataSubsetForMergeSerialization(r, false)
if m.MergeTimestamp != nil {
m.MergeTimestamp.Synthetic = nil // never populated below Raft
}
return m
},
emptySum: 14695981039346656037,
populatedSum: 1187861800212570275,
},
reflect.TypeOf(&roachpb.RaftReplicaID{}): {
populatedConstructor: func(r *rand.Rand) protoutil.Message {
func(r *rand.Rand) protoutil.Message {
return roachpb.NewPopulatedRaftReplicaID(r, false)
},
emptySum: 598336668751268149,
populatedSum: 9313101058286450988,
},
}

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

// Enable the additional checks in TestMain. NB: running this test by itself
// will fail those extra checks - such failures are safe to ignore, so long
// as this test passes when run with the entire package's tests.
verifyBelowRaftProtos = true

slice := make([]byte, 1<<20)
for typ, fix := range belowRaftGoldenProtos {
if b, err := protoutil.Marshal(reflect.New(typ.Elem()).Interface().(protoutil.Message)); err != nil {
t.Fatal(err)
} else if err := verifyHash(b, fix.emptySum); err != nil {
t.Errorf("%s (empty): %s\n", typ, err)
}
}

randGen := rand.New(rand.NewSource(goldenSeed))
// An arbitrary number chosen to seed the PRNGs used to populate the tested
// protos.
const goldenSeed = 1337
// We'll randomly populate, marshal, and hash each proto. Doing this more than
// once is necessary to make it very likely that all fields will be nonzero at
// some point.
const itersPerProto = 50

bytes := slice
numBytes := 0
for i := 0; i < itersPerProto; i++ {
if n, err := fix.populatedConstructor(randGen).MarshalTo(bytes); err != nil {
t.Fatal(err)
} else {
bytes = bytes[n:]
numBytes += n
w := echotest.NewWalker(t, testutils.TestDataPath(t, t.Name()))
for _, fn := range testCases {
name := fmt.Sprintf("%T", fn(rand.New(rand.NewSource(0))))
name = regexp.MustCompile(`.*\.`).ReplaceAllString(name, "")
t.Run(name, w.Run(t, name, func(t *testing.T) string {
rng := rand.New(rand.NewSource(goldenSeed))
hash := fnv.New64a()
for i := 0; i < itersPerProto; i++ {
msg := fn(rng)
dst := make([]byte, msg.Size())
_, err := msg.MarshalTo(dst)
require.NoError(t, err)
hash.Write(dst)
}
}
if err := verifyHash(slice[:numBytes], fix.populatedSum); err != nil {
t.Errorf("%s (populated): %s\n", typ, err)
}
return fmt.Sprint(hash.Sum64())
}))
}
}
68 changes: 1 addition & 67 deletions pkg/kv/kvserver/main_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -11,20 +11,14 @@
package kvserver_test

import (
"fmt"
"os"
"reflect"
"testing"

"github.com/cockroachdb/cockroach/pkg/kv/kvserver"
"github.com/cockroachdb/cockroach/pkg/roachpb"
"github.com/cockroachdb/cockroach/pkg/security/securityassets"
"github.com/cockroachdb/cockroach/pkg/security/securitytest"
"github.com/cockroachdb/cockroach/pkg/server"
"github.com/cockroachdb/cockroach/pkg/storage/enginepb"
"github.com/cockroachdb/cockroach/pkg/testutils/serverutils"
"github.com/cockroachdb/cockroach/pkg/testutils/testcluster"
"github.com/cockroachdb/cockroach/pkg/util"
"github.com/cockroachdb/cockroach/pkg/util/randutil"
)

Expand All @@ -34,69 +28,9 @@ func init() {
securityassets.SetLoader(securitytest.EmbeddedAssets)
}

var verifyBelowRaftProtos bool

func TestMain(m *testing.M) {
randutil.SeedForTests()
serverutils.InitTestServerFactory(server.TestServerFactory)
serverutils.InitTestClusterFactory(testcluster.TestClusterFactory)

// The below-Raft proto tracking is fairly expensive in terms of allocations
// which significantly impacts the tests under -race. We're already doing the
// below-Raft proto tracking in non-race builds, so there is little benefit
// to also doing it in race builds.
if util.RaceEnabled {
os.Exit(m.Run())
}

// Create a set of all protos we believe to be marshaled downstream of raft.
// After the tests are run, we'll subtract the encountered protos from this
// set.
notBelowRaftProtos := make(map[reflect.Type]struct{}, len(belowRaftGoldenProtos))
for typ := range belowRaftGoldenProtos {
notBelowRaftProtos[typ] = struct{}{}
}

// Before running the tests, enable instrumentation that tracks protos which
// are marshaled downstream of raft.
stopTrackingAndGetTypes := kvserver.TrackRaftProtos()

code := m.Run()

// Only do this verification if the associated test was run. Without this
// condition, the verification here would spuriously fail when running a
// small subset of tests e.g. as we often do with `stress`.
if verifyBelowRaftProtos {
failed := false
// Retrieve all the observed downstream-of-raft protos and confirm that they
// are all present in our expected set.
for _, typ := range stopTrackingAndGetTypes() {
if _, ok := belowRaftGoldenProtos[typ]; ok {
delete(notBelowRaftProtos, typ)
} else {
failed = true
fmt.Printf("%s: missing fixture! Please adjust belowRaftGoldenProtos if necessary\n", typ)
}
}

// Confirm that our expected set is now empty; we don't want to cement any
// protos needlessly.
// Two exceptions: these are sometimes observed below raft, sometimes not.
// It supposedly depends on whether any test server runs for long enough to
// write internal time series.
delete(notBelowRaftProtos, reflect.TypeOf(&roachpb.InternalTimeSeriesData{}))
delete(notBelowRaftProtos, reflect.TypeOf(&enginepb.MVCCMetadataSubsetForMergeSerialization{}))
for typ := range notBelowRaftProtos {
// NB: don't set failed=true. In a bazel world, we may just end up sharding in a way that
// doesn't observe some of the protos below raft.
fmt.Printf("%s: not observed below raft!\n", typ)
}

// Make sure our error messages make it out.
if failed && code == 0 {
code = 1
}
}

os.Exit(code)
os.Exit(m.Run())
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,3 @@
echo
----
5484201021249543052
Loading

0 comments on commit cc63f9f

Please sign in to comment.