Skip to content

Commit

Permalink
Browse files Browse the repository at this point in the history
51691: kvserver: remove ToIsLearner flag from RaftHeartbeat r=yuzefovich a=nvanbenschoten

This commit removes the ToIsLearner flag from the RaftHeartbeat message. This is allowed because 20.2 no longer need to deal with preemptive snapshots. This completes @ajwerner's heroic migration started in 99a031d and followed up by b2850e1.

51692: tree: cast TIMESTAMP to TEXT without timezone data r=rohany,adityamaru a=otan

This better matches postgres. I'm guessing it used to be done this way
to match the CLI output (which we can't fix without changing lib/pq),
but this affects output results from ORM tests. Note that CLI output is
not affected unless TIMESTAMP is cast to text.

Release note (sql change): Casting TIMESTAMP types to TEXT related types
now omits the timezone component. For example, '2001-12-15
15:14:13'::TIMESTAMP will now format as '2001-12-15 15:14:13' instead of
'2001-12-15 15:14:13+00:00'.



51770: cmd/skipped-tests: utility to post skipped tests to slack r=yuzefovich a=petermattis

Release note: None

51773: sql: panic with assertion errors, not raw strings r=jordanlewis a=jordanlewis

Panicking with raw strings will cause the strings to be redacted.

Release note: None

51780: opt: fix flake in TestExecBuild due to new stats cache refresh mechanism r=rytaft a=rytaft

This commit fixes a flake caused by the new stats cache update mechanism,
which refreshes the stats asynchronously. The solution is to retry the
test query until the new stats are available in the cache.

Fixes #50863

Release note: None

51784: sql: disable distribution of plans when transaction has modified a type r=rohany a=rohany

Fixes #50897.

Release note (sql change): Transactions that have modified or created a
type will execute queries on the local node, rather than distributing
the queries to other nodes in the cluster.

51806: roachtest: deflake disk-stalled roachtests r=irfansharif a=irfansharif

Fixes #51724, which failed with the salient error:

  ERROR: no --join flags provided to 'cockroach start'

Since this test is spinning up a single node cluster to test disk
stalls, it suffices to use start-single-node instead.

Release note: None

Co-authored-by: Nathan VanBenschoten <[email protected]>
Co-authored-by: Oliver Tan <[email protected]>
Co-authored-by: Peter Mattis <[email protected]>
Co-authored-by: Jordan Lewis <[email protected]>
Co-authored-by: Rebecca Taft <[email protected]>
Co-authored-by: Rohan Yadav <[email protected]>
Co-authored-by: irfan sharif <[email protected]>
  • Loading branch information
8 people committed Jul 23, 2020
8 parents 89dda79 + fd89146 + 79949e1 + 50a803d + 3dd344f + 399f590 + 123a67c + a25cab9 commit 2032f53
Show file tree
Hide file tree
Showing 113 changed files with 716 additions and 531 deletions.
6 changes: 3 additions & 3 deletions pkg/ccl/backupccl/show_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -36,7 +36,7 @@ func TestShowBackup(t *testing.T) {

const full, inc, inc2 = LocalFoo + "/full", LocalFoo + "/inc", LocalFoo + "/inc2"

beforeTS := sqlDB.QueryStr(t, `SELECT now()::string`)[0][0]
beforeTS := sqlDB.QueryStr(t, `SELECT now()::timestamp::string`)[0][0]
sqlDB.Exec(t, fmt.Sprintf(`BACKUP DATABASE data TO $1 AS OF SYSTEM TIME '%s'`, beforeTS), full)

res := sqlDB.QueryStr(t, `SELECT table_name, start_time::string, end_time::string, rows, is_full_cluster FROM [SHOW BACKUP $1]`, full)
Expand All @@ -51,7 +51,7 @@ func TestShowBackup(t *testing.T) {

// Backup the changes by appending to the base and by making a separate
// inc backup.
incTS := sqlDB.QueryStr(t, `SELECT now()::string`)[0][0]
incTS := sqlDB.QueryStr(t, `SELECT now()::timestamp::string`)[0][0]
sqlDB.Exec(t, fmt.Sprintf(`BACKUP DATABASE data TO $1 AS OF SYSTEM TIME '%s'`, incTS), full)
sqlDB.Exec(t, fmt.Sprintf(`BACKUP DATABASE data TO $1 AS OF SYSTEM TIME '%s' INCREMENTAL FROM $2`, incTS), inc, full)

Expand All @@ -75,7 +75,7 @@ func TestShowBackup(t *testing.T) {

// Backup the changes again, by appending to the base and by making a
// separate inc backup.
inc2TS := sqlDB.QueryStr(t, `SELECT now()::string`)[0][0]
inc2TS := sqlDB.QueryStr(t, `SELECT now()::timestamp::string`)[0][0]
sqlDB.Exec(t, fmt.Sprintf(`BACKUP DATABASE data TO $1 AS OF SYSTEM TIME '%s'`, inc2TS), full)
sqlDB.Exec(t, fmt.Sprintf(`BACKUP DATABASE data TO $1 AS OF SYSTEM TIME '%s' INCREMENTAL FROM $2, $3`, inc2TS), inc2, full, inc)

Expand Down
4 changes: 3 additions & 1 deletion pkg/cli/sql_util.go
Original file line number Diff line number Diff line change
Expand Up @@ -927,7 +927,9 @@ func formatVal(val driver.Value, showPrintableUnicode bool, showNewLinesAndTabs
lex.BytesEncodeEscape, false /* skipHexPrefix */)

case time.Time:
return t.Format(tree.TimestampOutputFormat)
// Since we do not know whether the datum is Timestamp or TimestampTZ,
// output the full format.
return t.Format(tree.TimestampTZOutputFormat)
}

return fmt.Sprint(val)
Expand Down
2 changes: 1 addition & 1 deletion pkg/cli/testdata/dump/row
Original file line number Diff line number Diff line change
Expand Up @@ -90,7 +90,7 @@ CREATE TABLE t (
);

INSERT INTO t (i, f, s, b, d, t, ts, n, o, e, u, ip, j, ary, tz, e1, e2, s1, s2, oi) VALUES
(1, 2.3, 'striiing', '\x613162326333', '2016-03-26', '01:02:03.456', '2016-01-25 10:10:10+00:00', '02:30:30', true, 1.2345, 'e9716c74-2638-443d-90ed-ffde7bea7d1d', '192.168.0.1', '{"a": "b"}', ARRAY['hello','world'], '2016-01-25 10:10:10+00:00', 3, 4.5, 's', 'hello' COLLATE en_u_ks_level2, 6),
(1, 2.3, 'striiing', '\x613162326333', '2016-03-26', '01:02:03.456', '2016-01-25 10:10:10', '02:30:30', true, 1.2345, 'e9716c74-2638-443d-90ed-ffde7bea7d1d', '192.168.0.1', '{"a": "b"}', ARRAY['hello','world'], '2016-01-25 10:10:10+00:00', 3, 4.5, 's', 'hello' COLLATE en_u_ks_level2, 6),
(NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL),
(NULL, '+Inf', NULL, NULL, NULL, NULL, NULL, NULL, NULL, 'Infinity', NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL),
(NULL, '-Inf', NULL, NULL, NULL, NULL, NULL, NULL, NULL, '-Infinity', NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL),
Expand Down
7 changes: 6 additions & 1 deletion pkg/cmd/roachtest/disk_stall.go
Original file line number Diff line number Diff line change
Expand Up @@ -14,6 +14,7 @@ import (
"context"
"fmt"
"math/rand"
"runtime"
"strings"
"time"

Expand Down Expand Up @@ -46,6 +47,10 @@ func registerDiskStalledDetection(r *testRegistry) {
func runDiskStalledDetection(
ctx context.Context, t *test, c *cluster, affectsLogDir bool, affectsDataDir bool,
) {
if local && runtime.GOOS != "linux" {
t.Fatalf("must run on linux os, found %s", runtime.GOOS)
}

n := c.Node(1)

c.Put(ctx, cockroach, "./cockroach")
Expand Down Expand Up @@ -100,7 +105,7 @@ func runDiskStalledDetection(
out, err := c.RunWithBuffer(ctx, l, n,
fmt.Sprintf("timeout --signal 9 %ds env COCKROACH_ENGINE_MAX_SYNC_DURATION_FATAL=true "+
"COCKROACH_ENGINE_MAX_SYNC_DURATION=%s COCKROACH_LOG_MAX_SYNC_DURATION=%s "+
"./cockroach start --insecure --logtostderr=INFO --store {store-dir}/%s --log-dir {store-dir}/%s",
"./cockroach start-single-node --insecure --logtostderr=INFO --store {store-dir}/%s --log-dir {store-dir}/%s",
int(dur.Seconds()), maxDataSync, maxLogSync, dataDir, logDir,
),
)
Expand Down
189 changes: 189 additions & 0 deletions pkg/cmd/skipped-tests/main.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,189 @@
// 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 main

import (
"bytes"
"flag"
"fmt"
"go/build"
"log"
"os/exec"
"path/filepath"
"regexp"
"sort"
"strings"

"github.com/cockroachdb/errors"
"github.com/ghemawat/stream"
"github.com/nlopes/slack"
)

var slackToken = flag.String("slack-token", "", "Slack bot token")
var slackChannel = flag.String("slack-channel", "test-infra-ops", "Slack channel")

type skippedTest struct {
file string
test string
}

func dirCmd(dir string, name string, args ...string) stream.Filter {
cmd := exec.Command(name, args...)
cmd.Dir = dir
out, err := cmd.CombinedOutput()
switch {
case err == nil:
case errors.HasType(err, (*exec.ExitError)(nil)):
// Non-zero exit is expected.
default:
log.Fatal(err)
}
return stream.ReadLines(bytes.NewReader(out))
}

func makeSlackClient() *slack.Client {
if *slackToken == "" {
return nil
}
return slack.New(*slackToken)
}

func findChannel(client *slack.Client, name string) (string, error) {
if client != nil {
channels, err := client.GetChannels(true)
if err != nil {
return "", err
}
for _, channel := range channels {
if channel.Name == name {
return channel.ID, nil
}
}
}
return "", fmt.Errorf("not found")
}

func postReport(skipped []skippedTest) {
client := makeSlackClient()
if client == nil {
fmt.Printf("no slack client\n")
return
}

channel, _ := findChannel(client, *slackChannel)
if channel == "" {
fmt.Printf("unable to find slack channel: %q\n", *slackChannel)
return
}

params := slack.PostMessageParameters{
Username: "Craig Cockroach",
}

status := "good"
switch n := len(skipped); {
case n >= 100:
status = "danger"
case n > 10:
status = "warning"
}

message := fmt.Sprintf("%d skipped tests", len(skipped))
fmt.Println(message)

params.Attachments = append(params.Attachments,
slack.Attachment{
Color: status,
Title: message,
Fallback: message,
})

fileMap := make(map[string]int)
for i := range skipped {
fileMap[skipped[i].file]++
}
files := make([]string, 0, len(fileMap))
for file := range fileMap {
files = append(files, file)
}
sort.Strings(files)

var buf bytes.Buffer
for _, file := range files {
fmt.Fprintf(&buf, "%3d %s\n", fileMap[file], file)
}
fmt.Print(buf.String())

params.Attachments = append(params.Attachments,
slack.Attachment{
Color: status,
Text: fmt.Sprintf("```\n%s```\n", buf.String()),
})

if _, _, err := client.PostMessage(channel, "", params); err != nil {
fmt.Printf("unable to post slack report: %v\n", err)
return
}

fmt.Printf("posted slack report\n")
}

func main() {
flag.Parse()

const root = "github.com/cockroachdb/cockroach"

crdb, err := build.Import(root, "", build.FindOnly)
if err != nil {
log.Fatal(err)
}
pkgDir := filepath.Join(crdb.Dir, "pkg")

// NB: This only matches tests, not benchmarks. We can change this to match
// benchmarks if we want to report on skipped benchmarks too.
testnameRE := regexp.MustCompile(`([^:]+):func (Test[^(]*).*`)

// We grep for all test and benchmark names, along with calls to Skip and
// Skipf. We then loop over this output keeping track of the most recent
// test/benchmark seen so we can associate the Skip call with the correct
// test.
filter := stream.Sequence(
dirCmd(pkgDir, "git", "grep", "-E", `^func (Test|Benchmark)|\.Skipf?\(`),
// Short-list of skip reasons to exclude from reporting.
//
// TODO(peter): We should probably have an explicit exclude marker.
stream.GrepNot(`short|PKG specified`),
)

var skipped []skippedTest
var lastTest string
if err := stream.ForEach(filter, func(s string) {
switch {
case strings.Contains(s, ":func "):
lastTest = s
case strings.Contains(s, ".Skip"):
m := testnameRE.FindStringSubmatch(lastTest)
if m != nil {
file, test := m[1], m[2]
skipped = append(skipped, skippedTest{
file: file,
test: test,
})
// Clear the test so we don't report it more than once.
lastTest = ""
}
}
}); err != nil {
log.Fatal(err)
}

postReport(skipped)
}
2 changes: 1 addition & 1 deletion pkg/internal/sqlsmith/bulkio.go
Original file line number Diff line number Diff line change
Expand Up @@ -71,7 +71,7 @@ func makeAsOf(s *Smither) tree.AsOfClause {
case 1:
expr = tree.NewStrVal("-2s")
case 2:
expr = tree.NewStrVal(timeutil.Now().Add(-2 * time.Second).Format(tree.TimestampOutputFormat))
expr = tree.NewStrVal(timeutil.Now().Add(-2 * time.Second).Format(tree.TimestampTZOutputFormat))
case 3:
expr = sqlbase.RandDatum(s.rnd, types.Interval, false /* nullOk */)
case 4:
Expand Down
Loading

0 comments on commit 2032f53

Please sign in to comment.