Skip to content

Commit

Permalink
Browse files Browse the repository at this point in the history
56379: workload/schemachange: improve error screening r=ajwerner a=jayshrivastava

workload/schemachange: improve error screening

Several ops will be updated to screen for errors (#56119):
- dropTable
- dropView
- createSequence
- renameSequence
- dropSequence
- insertRow

Sequences were also updated to have more interesting cases: non-existing sequences can randomly be returned and sequences can be owned by columns.

Furthermore, this change fixes a bug where non-existing tables were getting returned instead of existing ones.

Finally, error screening logic is updated to ignore transaction retry errors. This fixes one of the issues in #56230.

57034: cli: add reset-quorum command r=knz a=TheSamHuang

This adds `reset-quorum` as a command in the debug CLI. This command utilizes the functionality added in #56333 to recover unavailable ranges where all replicas are lost. 

57272: roachtest: Add missing gossiped StoreDescriptor to expected relocate errors r=nvanbenschoten a=nvanbenschoten

Closes #57191.

This seems possible shortly after the cluster starts up, if gossip hasn't propagated by the time the `ALTER TABLE _ EXPERIMENTAL_RELOCATE _` statement is run.

57274: roachtest: remove initial disk space check from drop test r=nvanbenschoten a=nvanbenschoten

Fixes #56040.

Will need to be backported.

This assertion seemed like a good idea, but it was repeatedly (dec148a)
a source of flakiness when fixtures changed. Now that we're using IMPORT
for TPC-C, the check is even harder to get right without making it so
small as to be useless. It doesn't seem to be worth the instability, so
remove it.

57454: util/log: de-flake TestSecondaryGC and some SQL tests r=irfansharif a=knz

Fixes #57442
Fixes #57444 

See individual commits for details.

Co-authored-by: Jayant Shrivastava <[email protected]>
Co-authored-by: Sam Huang <[email protected]>
Co-authored-by: Nathan VanBenschoten <[email protected]>
Co-authored-by: Raphael 'kena' Poss <[email protected]>
  • Loading branch information
5 people committed Dec 3, 2020
6 parents ced36e3 + 868a286 + a352d26 + b9d4389 + 187198a + 40f01b9 commit ac6463f
Show file tree
Hide file tree
Showing 14 changed files with 649 additions and 95 deletions.
1 change: 1 addition & 0 deletions pkg/cli/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -13,6 +13,7 @@ go_library(
"debug_check_store.go",
"debug_logconfig.go",
"debug_merge_logs.go",
"debug_reset_quorum.go",
"debug_synctest.go",
"decode.go",
"demo.go",
Expand Down
1 change: 1 addition & 0 deletions pkg/cli/debug.go
Original file line number Diff line number Diff line change
Expand Up @@ -1179,6 +1179,7 @@ var debugCmds = append(DebugCmdsForRocksDB,
debugEnvCmd,
debugZipCmd,
debugMergeLogsCommand,
debugResetQuorumCmd,
)

// DebugCmd is the root of all debug commands. Exported to allow modification by CCL code.
Expand Down
68 changes: 68 additions & 0 deletions pkg/cli/debug_reset_quorum.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,68 @@
// Copyright 2020 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 cli

import (
"context"
"fmt"
"strconv"

"github.com/cockroachdb/cockroach/pkg/roachpb"
"github.com/spf13/cobra"
)

var debugResetQuorumCmd = &cobra.Command{
Use: "reset-quorum [range ID]",
Short: "Reset quorum on the given range" +
" by designating the target node as the sole voter.",
Long: `
Reset quorum on the given range by designating the current node as
the sole voter. Any existing data for the range is discarded.
This command is UNSAFE and should only be used with the supervision
of Cockroach Labs support. It is a last-resort option to recover a
specified range after multiple node failures and loss of quorum.
Data on any surviving replicas will not be used to restore quorum.
Instead, these replicas will be removed irrevocably.
`,
Args: cobra.ExactArgs(1),
RunE: MaybeDecorateGRPCError(runDebugResetQuorum),
}

func runDebugResetQuorum(cmd *cobra.Command, args []string) error {
ctx, cancel := context.WithCancel(context.Background())
defer cancel()

rangeID, err := strconv.ParseInt(args[0], 10, 32)
if err != nil {
return err
}

// Set up GRPC Connection for running ResetQuorum.
cc, _, finish, err := getClientGRPCConn(ctx, serverCfg)
if err != nil {
return err
}
defer finish()

// Call ResetQuorum to reset quorum for given range on target node.
_, err = roachpb.NewInternalClient(cc).ResetQuorum(ctx, &roachpb.ResetQuorumRequest{
RangeID: int32(rangeID),
})
if err != nil {
return err
}

fmt.Printf("ok; please verify https://<ui>/#/reports/range/%d", rangeID)

return nil
}
12 changes: 2 additions & 10 deletions pkg/cmd/roachtest/drop.go
Original file line number Diff line number Diff line change
Expand Up @@ -29,7 +29,7 @@ func registerDrop(r *testRegistry) {
// by a truncation for the `stock` table (which contains warehouses*100k
// rows). Next, it issues a `DROP` for the whole database, and sets the GC TTL
// to one second.
runDrop := func(ctx context.Context, t *test, c *cluster, warehouses, nodes int, initDiskSpace int) {
runDrop := func(ctx context.Context, t *test, c *cluster, warehouses, nodes int) {
c.Put(ctx, cockroach, "./cockroach", c.Range(1, nodes))
c.Put(ctx, workload, "./workload", c.Range(1, nodes))
c.Start(ctx, t, c.Range(1, nodes), startArgs("-e", "COCKROACH_MEMPROF_INTERVAL=15s"))
Expand Down Expand Up @@ -86,12 +86,6 @@ func registerDrop(r *testRegistry) {
}

t.l.Printf("Node %d space used: %s\n", j, humanizeutil.IBytes(int64(size)))

// Return if the size of the directory is less than expected.
if size < initDiskSpace {
t.Fatalf("Node %d space used: %s less than %s", j, humanizeutil.IBytes(int64(size)),
humanizeutil.IBytes(int64(initDiskSpace)))
}
}

for i := minWarehouse; i <= maxWarehouse; i++ {
Expand Down Expand Up @@ -159,7 +153,6 @@ func registerDrop(r *testRegistry) {

warehouses := 100
numNodes := 9
initDiskSpace := 256 << 20 // 256 MB

r.Add(testSpec{
Name: fmt.Sprintf("drop/tpcc/w=%d,nodes=%d", warehouses, numNodes),
Expand All @@ -172,10 +165,9 @@ func registerDrop(r *testRegistry) {
if local {
numNodes = 4
warehouses = 1
initDiskSpace = 5 << 20 // 5 MB
fmt.Printf("running with w=%d,nodes=%d in local mode\n", warehouses, numNodes)
}
runDrop(ctx, t, c, warehouses, numNodes, initDiskSpace)
runDrop(ctx, t, c, warehouses, numNodes)
},
})
}
2 changes: 1 addition & 1 deletion pkg/kv/test_utils.go
Original file line number Diff line number Diff line change
Expand Up @@ -38,7 +38,6 @@ func OnlyFollowerReads(rec tracing.Recording) bool {
// IsExpectedRelocateError maintains an allowlist of errors related to
// atomic-replication-changes we want to ignore / retry on for tests.
// See:
// https://github.com/cockroachdb/cockroach/issues/33732
// https://github.com/cockroachdb/cockroach/issues/33708
// https://github.cm/cockroachdb/cockroach/issues/34012
// https://github.com/cockroachdb/cockroach/issues/33683#issuecomment-454889149
Expand All @@ -57,6 +56,7 @@ func IsExpectedRelocateError(err error) bool {
"snapshot failed:",
"breaker open",
"unable to select removal target", // https://github.com/cockroachdb/cockroach/issues/49513
"cannot up-replicate to .*; missing gossiped StoreDescriptor",
}
pattern := "(" + strings.Join(allowlist, "|") + ")"
return testutils.IsError(err, pattern)
Expand Down
18 changes: 17 additions & 1 deletion pkg/sql/rowenc/testutils.go
Original file line number Diff line number Diff line change
Expand Up @@ -188,7 +188,13 @@ func RandDatumWithNullChance(rng *rand.Rand, typ *types.T, nullChance int) tree.
return &tree.DBitArray{BitArray: r}
case types.StringFamily:
// Generate a random ASCII string.
p := make([]byte, rng.Intn(10))
var length int
if typ.Oid() == oid.T_char || typ.Oid() == oid.T_bpchar {
length = 1
} else {
length = rng.Intn(10)
}
p := make([]byte, length)
for i := range p {
p[i] = byte(1 + rng.Intn(127))
}
Expand Down Expand Up @@ -750,6 +756,16 @@ func randInterestingDatum(rng *rand.Rand, typ *types.T) tree.Datum {
default:
panic(errors.AssertionFailedf("float with an unexpected width %d", typ.Width()))
}
case types.BitFamily:
// A width of 64 is used by all special BitFamily datums in randInterestingDatums.
// If the provided bit type, typ, has a width of 0 (representing an arbitrary width) or 64 exactly,
// then the special datum will be valid for the provided type. Otherwise, the special type
// must be resized to match the width of the provided type.
if typ.Width() == 0 || typ.Width() == 64 {
return special
}
return &tree.DBitArray{BitArray: special.(*tree.DBitArray).ToWidth(uint(typ.Width()))}

default:
return special
}
Expand Down
1 change: 0 additions & 1 deletion pkg/util/log/file_log_gc.go
Original file line number Diff line number Diff line change
Expand Up @@ -21,7 +21,6 @@ import (

// gcDaemon runs the GC loop for the given logger.
func (l *fileSink) gcDaemon(ctx context.Context) {
l.gcOldFiles()
for {
select {
case <-ctx.Done():
Expand Down
6 changes: 4 additions & 2 deletions pkg/util/log/file_log_gc_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -88,8 +88,6 @@ func testLogGC(
}(logging.mu.disableDaemons)
logging.mu.Unlock()

const newLogFiles = 20

// Make an entry in the target logger. This ensures That there is at
// least one file in the target directory for the logger being
// tested. This serves two
Expand Down Expand Up @@ -132,6 +130,8 @@ func testLogGC(
// Pick a max total size that's between 2 and 3 log files in size.
maxTotalLogFileSize := logFileSize*expectedFilesAfterGC + logFileSize // 2

t.Logf("new max total log file size: %d", maxTotalLogFileSize)

// We want to create multiple log files below. For this we need to
// override the size/number limits first to the values suitable for
// the test.
Expand All @@ -143,6 +143,8 @@ func testLogGC(
atomic.StoreInt64(&fileSink.logFilesCombinedMaxSize, maxTotalLogFileSize)

// Create the number of expected log files.
const newLogFiles = 20

for i := 1; i < newLogFiles; i++ {
logFn(context.Background(), "%d", i)
Flush()
Expand Down
8 changes: 5 additions & 3 deletions pkg/util/log/flags.go
Original file line number Diff line number Diff line change
Expand Up @@ -179,8 +179,10 @@ func ApplyConfig(config logconfig.Config) (cleanupFn func(), err error) {
allSinkInfos.put(fileSinkInfo)

if fileSink.logFilesCombinedMaxSize > 0 {
// Do a start round of GC, so clear up past accumulated files.
fileSink.gcOldFiles()
// Start the GC process. This ensures that old capture files get
// erased as necessary.
// erased as new files get created.
go fileSink.gcDaemon(secLoggersCtx)
}

Expand Down Expand Up @@ -223,7 +225,7 @@ func ApplyConfig(config logconfig.Config) (cleanupFn func(), err error) {
}

// Apply the stderr sink configuration.
logging.stderrSink.noColor = config.Sinks.Stderr.NoColor
logging.stderrSink.noColor.Set(config.Sinks.Stderr.NoColor)
if err := logging.stderrSinkInfoTemplate.applyConfig(config.Sinks.Stderr.CommonSinkConfig); err != nil {
cleanupFn()
return nil, err
Expand Down Expand Up @@ -354,7 +356,7 @@ func DescribeAppliedConfig() string {
}

// Describe the stderr sink.
config.Sinks.Stderr.NoColor = logging.stderrSink.noColor
config.Sinks.Stderr.NoColor = logging.stderrSink.noColor.Get()
config.Sinks.Stderr.CommonSinkConfig = logging.stderrSinkInfoTemplate.describeAppliedConfig()

describeConnections := func(l *loggerT, ch Channel,
Expand Down
4 changes: 2 additions & 2 deletions pkg/util/log/format_crdb_v1.go
Original file line number Diff line number Diff line change
Expand Up @@ -62,7 +62,7 @@ func (formatCrdbV1TTY) formatterName() string { return "crdb-v1-tty" }

func (formatCrdbV1TTY) formatEntry(entry logpb.Entry, stacks []byte) *buffer {
cp := ttycolor.StderrProfile
if logging.stderrSink.noColor {
if logging.stderrSink.noColor.Get() {
cp = nil
}
return formatLogEntryInternal(entry, false /*showCounter*/, cp, stacks)
Expand All @@ -77,7 +77,7 @@ func (formatCrdbV1TTYWithCounter) formatterName() string { return "crdb-v1-tty-c

func (formatCrdbV1TTYWithCounter) formatEntry(entry logpb.Entry, stacks []byte) *buffer {
cp := ttycolor.StderrProfile
if logging.stderrSink.noColor {
if logging.stderrSink.noColor.Get() {
cp = nil
}
return formatLogEntryInternal(entry, true /*showCounter*/, cp, stacks)
Expand Down
7 changes: 5 additions & 2 deletions pkg/util/log/stderr_sink.go
Original file line number Diff line number Diff line change
Expand Up @@ -10,13 +10,16 @@

package log

import "github.com/cockroachdb/cockroach/pkg/cli/exit"
import (
"github.com/cockroachdb/cockroach/pkg/cli/exit"
"github.com/cockroachdb/cockroach/pkg/util/syncutil"
)

// Type of a stderr copy sink.
type stderrSink struct {
// the --no-color flag. When set it disables escapes code on the
// stderr copy.
noColor bool
noColor syncutil.AtomicBool
}

// activeAtSeverity implements the logSink interface.
Expand Down
Loading

0 comments on commit ac6463f

Please sign in to comment.