Skip to content

Commit

Permalink
Merge #29437
Browse files Browse the repository at this point in the history
29437: roachtest: add acceptance/build-info r=benesch a=petermattis

Move the build-info acceptance test to a new acceptance/build-info
roachtest. Move the existing cli/node-status to
acceptance/cli/node-status. The acceptance roachtests are all going to
share the same cluster to amortize the cluster setup/teardown time.

Fun fact: the acceptance/build-info roachtest is about twice as fast as
the old build-info acceptance test.

See #29151

Release note: None

Co-authored-by: Peter Mattis <[email protected]>
  • Loading branch information
craig[bot] and petermattis committed Aug 31, 2018
2 parents 61325cd + 8a08936 commit dc6c7b5
Show file tree
Hide file tree
Showing 7 changed files with 114 additions and 125 deletions.
2 changes: 1 addition & 1 deletion build/teamcity-local-roachtest.sh
Original file line number Diff line number Diff line change
Expand Up @@ -23,7 +23,7 @@ tc_end_block "Compile workload/roachtest"

tc_start_block "Run local roachtests"
# TODO(peter,dan): curate a suite of the tests that works locally.
run build/builder.sh ./bin/roachtest run '(cli|kv/splits)' \
run build/builder.sh ./bin/roachtest run '(acceptance|kv/splits)' \
--local \
--cockroach "cockroach" \
--workload "bin/workload" \
Expand Down
43 changes: 43 additions & 0 deletions pkg/cmd/roachtest/acceptance.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,43 @@
// Copyright 2018 The Cockroach Authors.
//
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or
// implied. See the License for the specific language governing
// permissions and limitations under the License. See the AUTHORS file
// for names of contributors.

package main

import "context"

func registerAcceptance(r *registry) {
spec := testSpec{
Name: "acceptance",
Nodes: nodes(3),
}

testCases := []struct {
name string
fn func(ctx context.Context, t *test, c *cluster)
}{
{"build-info", runBuildInfo},
{"cli/node-status", runCLINodeStatus},
{"event-log", runEventLog},
}
for _, tc := range testCases {
spec.SubTests = append(spec.SubTests, testSpec{
Name: tc.name,
Stable: true, // DO NOT COPY to new tests
Run: tc.fn,
})
}

r.Add(spec)
}
Original file line number Diff line number Diff line change
@@ -1,4 +1,4 @@
// Copyright 2015 The Cockroach Authors.
// Copyright 2018 The Cockroach Authors.
//
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
Expand All @@ -10,48 +10,31 @@
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or
// implied. See the License for the specific language governing
// permissions and limitations under the License.
// permissions and limitations under the License. See the AUTHORS file
// for names of contributors.

package acceptance
package main

import (
"context"
"testing"
"time"
"net/http"

"github.com/cockroachdb/cockroach/pkg/acceptance/cluster"
"github.com/cockroachdb/cockroach/pkg/server/serverpb"
"github.com/cockroachdb/cockroach/pkg/testutils"
"github.com/cockroachdb/cockroach/pkg/util/httputil"
"github.com/cockroachdb/cockroach/pkg/util/log"
)

func TestBuildInfo(t *testing.T) {
s := log.Scope(t)
defer s.Close(t)
func runBuildInfo(ctx context.Context, t *test, c *cluster) {
c.Put(ctx, cockroach, "./cockroach", c.All())
c.Wipe(ctx)
c.Start(ctx)

RunLocal(t, func(t *testing.T) {
runTestWithCluster(t, testBuildInfoInner)
})
}

func testBuildInfoInner(
ctx context.Context, t *testing.T, c cluster.Cluster, cfg cluster.TestConfig,
) {
if err := CheckGossip(ctx, c, 20*time.Second, HasPeers(c.NumNodes())); err != nil {
var details serverpb.DetailsResponse
url := `http://` + c.ExternalAdminUIAddr(ctx, c.Node(1))[0] + `/_status/details/local`
err := httputil.GetJSON(http.Client{}, url, &details)
if err != nil {
t.Fatal(err)
}

var details serverpb.DetailsResponse
testutils.SucceedsSoon(t, func() error {
select {
case <-stopper.ShouldStop():
t.Fatalf("interrupted")
default:
}
return httputil.GetJSON(cluster.HTTPClient, c.URL(ctx, 0)+"/_status/details/local", &details)
})

bi := details.BuildInfo
testData := map[string]string{
"go_version": bi.GoVersion,
Expand All @@ -61,7 +44,7 @@ func testBuildInfoInner(
}
for key, val := range testData {
if val == "" {
t.Errorf("build info not set for \"%s\"", key)
t.Fatalf("build info not set for \"%s\"", key)
}
}
}
27 changes: 2 additions & 25 deletions pkg/cmd/roachtest/cli.go
Original file line number Diff line number Diff line change
Expand Up @@ -24,7 +24,8 @@ import (

func runCLINodeStatus(ctx context.Context, t *test, c *cluster) {
c.Put(ctx, cockroach, "./cockroach", c.All())
c.Start(ctx, c.All())
c.Wipe(ctx)
c.Start(ctx)

db := c.Conn(ctx, 1)
defer db.Close()
Expand Down Expand Up @@ -82,27 +83,3 @@ func runCLINodeStatus(ctx context.Context, t *test, c *cluster) {
}
}
}

func registerCLI(r *registry) {
spec := testSpec{
Name: "cli",
Stable: true, // DO NOT COPY to new tests
Nodes: nodes(3),
}

testCases := []struct {
name string
fn func(ctx context.Context, t *test, c *cluster)
}{
{"node-status", runCLINodeStatus},
}
for _, tc := range testCases {
spec.SubTests = append(spec.SubTests, testSpec{
Name: tc.name,
Stable: true, // DO NOT COPY to new tests
Run: tc.fn,
})
}

r.Add(spec)
}
10 changes: 10 additions & 0 deletions pkg/cmd/roachtest/cluster.go
Original file line number Diff line number Diff line change
Expand Up @@ -1039,6 +1039,16 @@ func (c *cluster) InternalAdminUIAddr(ctx context.Context, node nodeListOption)
return addrs
}

// ExternalAdminUIAddr returns the internal Admin UI address in the form host:port
// for the specified node.
func (c *cluster) ExternalAdminUIAddr(ctx context.Context, node nodeListOption) []string {
var addrs []string
for _, u := range c.ExternalAddr(ctx, node) {
addrs = append(addrs, addrToAdminUIAddr(c, u))
}
return addrs
}

// InternalAddr returns the internal address in the form host:port for the
// specified nodes.
func (c *cluster) InternalAddr(ctx context.Context, node nodeListOption) []string {
Expand Down
110 changes: 43 additions & 67 deletions pkg/acceptance/event_log_test.go → pkg/cmd/roachtest/event_log.go
Original file line number Diff line number Diff line change
@@ -1,4 +1,4 @@
// Copyright 2016 The Cockroach Authors.
// Copyright 2018 The Cockroach Authors.
//
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
Expand All @@ -10,68 +10,49 @@
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or
// implied. See the License for the specific language governing
// permissions and limitations under the License.
// permissions and limitations under the License. See the AUTHORS file
// for names of contributors.

package acceptance
package main

import (
"context"
gosql "database/sql"
"encoding/json"
"testing"
"fmt"
"time"

"github.com/pkg/errors"

"github.com/cockroachdb/cockroach/pkg/acceptance/cluster"
"github.com/cockroachdb/cockroach/pkg/roachpb"
csql "github.com/cockroachdb/cockroach/pkg/sql"
"github.com/cockroachdb/cockroach/pkg/testutils"
"github.com/cockroachdb/cockroach/pkg/util/log"
"github.com/cockroachdb/cockroach/pkg/util/retry"
"github.com/cockroachdb/cockroach/pkg/util/uuid"
)

// TestEventLog verifies that "node joined" and "node restart" events are
// recorded whenever a node starts and contacts the cluster.
func TestEventLog(t *testing.T) {
s := log.Scope(t)
defer s.Close(t)

RunLocal(t, func(t *testing.T) {
runTestWithCluster(t, testEventLogInner)
})
}

func testEventLogInner(
ctx context.Context, t *testing.T, c cluster.Cluster, cfg cluster.TestConfig,
) {
num := c.NumNodes()
if num <= 0 {
t.Fatalf("%d nodes in cluster", num)
}

var confirmedClusterID uuid.UUID
func runEventLog(ctx context.Context, t *test, c *cluster) {
type nodeEventInfo struct {
Descriptor roachpb.NodeDescriptor
ClusterID uuid.UUID
}

// Verify that a node_join message was logged for each node in the cluster.
// We expect there to eventually be one such message for each node in the
// cluster, and each message must be correctly formatted.
testutils.SucceedsSoon(t, func() error {
db := makePGClient(t, c.PGUrl(ctx, 0))
defer db.Close()
c.Put(ctx, cockroach, "./cockroach", c.All())
c.Wipe(ctx)
c.Start(ctx)

// Verify that "node joined" and "node restart" events are recorded whenever
// a node starts and contacts the cluster.
db := c.Conn(ctx, 1)
defer db.Close()
waitForFullReplication(t, db)
var clusterID uuid.UUID

// Query all node join events. There should be one for each node in the
// cluster.
err := retry.ForDuration(10*time.Second, func() error {
rows, err := db.Query(
`SELECT "targetID", info FROM system.eventlog WHERE "eventType" = $1`,
string(csql.EventLogNodeJoin))
`SELECT "targetID", info FROM system.eventlog WHERE "eventType" = 'node_join'`,
)
if err != nil {
return err
t.Fatal(err)
}
clusterID = uuid.UUID{}
seenIds := make(map[int64]struct{})
var clusterID uuid.UUID
for rows.Next() {
var targetID int64
var infoStr gosql.NullString
Expand Down Expand Up @@ -110,33 +91,27 @@ func testEventLogInner(
seenIds[targetID] = struct{}{}
}
if err := rows.Err(); err != nil {
return err
t.Fatal(err)
}

if a, e := len(seenIds), c.NumNodes(); a != e {
return errors.Errorf("expected %d node join messages, found %d: %v", e, a, seenIds)
if c.nodes != len(seenIds) {
return fmt.Errorf("expected %d node join messages, found %d: %v",
c.nodes, len(seenIds), seenIds)
}

confirmedClusterID = clusterID
return nil
})

// Stop and Start Node 0, and verify the node restart message.
if err := c.Kill(ctx, 0); err != nil {
t.Fatal(err)
}
if err := c.Restart(ctx, 0); err != nil {
if err != nil {
t.Fatal(err)
}

testutils.SucceedsSoon(t, func() error {
db := makePGClient(t, c.PGUrl(ctx, 0))
defer db.Close()
// Stop and Start Node 3, and verify the node restart message.
c.Stop(ctx, c.Node(3))
c.Start(ctx, c.Node(3))

err = retry.ForDuration(10*time.Second, func() error {
// Query all node restart events. There should only be one.
rows, err := db.Query(
`SELECT "targetID", info FROM system.eventlog WHERE "eventType" = $1`,
string(csql.EventLogNodeRestart))
`SELECT "targetID", info FROM system.eventlog WHERE "eventType" = 'node_restart'`,
)
if err != nil {
return err
}
Expand All @@ -146,7 +121,7 @@ func testEventLogInner(
var targetID int64
var infoStr gosql.NullString
if err := rows.Scan(&targetID, &infoStr); err != nil {
t.Fatal(err)
return err
}

// Verify the stored node descriptor.
Expand All @@ -158,24 +133,25 @@ func testEventLogInner(
t.Fatal(err)
}
if a, e := int64(info.Descriptor.NodeID), targetID; a != e {
t.Fatalf("Node join with targetID %d had descriptor for wrong node %d", e, a)
t.Fatalf("node join with targetID %d had descriptor for wrong node %d", e, a)
}

// Verify cluster ID is recorded, and is the same for all nodes.
if confirmedClusterID != info.ClusterID {
t.Fatalf(
"Node restart recorded different cluster ID than earlier join. Expected %s, got %s. Info: %v",
confirmedClusterID, info.ClusterID, info)
if clusterID != info.ClusterID {
t.Fatalf("expected cluser ID %s, got %s\n%v", clusterID, info.ClusterID, info)
}

seenCount++
}
if err := rows.Err(); err != nil {
return err
t.Fatal(err)
}
if seenCount != 1 {
return errors.Errorf("Expected only one node restart event, found %d", seenCount)
return fmt.Errorf("expected one node restart event, found %d", seenCount)
}
return nil
})
if err != nil {
t.Fatal(err)
}
}
2 changes: 1 addition & 1 deletion pkg/cmd/roachtest/registry.go
Original file line number Diff line number Diff line change
Expand Up @@ -20,12 +20,12 @@ func registerTests(r *registry) {
//
// grep -h -E 'func register[^(]+\(.*registry\) {' *.go | grep -E -o 'register[^(]+' | grep -E -v '^register(Tests|Benchmarks)$' | grep -v '^\w*Bench$' | sort -f | awk '{printf "\t%s(r)\n", $0}'

registerAcceptance(r)
registerAllocator(r)
registerBackup(r)
registerCancel(r)
registerCDC(r)
registerClearRange(r)
registerCLI(r)
registerClock(r)
registerCopy(r)
registerDebug(r)
Expand Down

0 comments on commit dc6c7b5

Please sign in to comment.