Skip to content

Commit

Permalink
roachtest: add acceptance/event-log
Browse files Browse the repository at this point in the history
Move the event-log acceptance test to an acceptance/event-log roachtest.

Release note: None
  • Loading branch information
petermattis committed Aug 31, 2018
1 parent 6223b8e commit 8a08936
Show file tree
Hide file tree
Showing 2 changed files with 44 additions and 67 deletions.
1 change: 1 addition & 0 deletions pkg/cmd/roachtest/acceptance.go
Original file line number Diff line number Diff line change
Expand Up @@ -29,6 +29,7 @@ func registerAcceptance(r *registry) {
}{
{"build-info", runBuildInfo},
{"cli/node-status", runCLINodeStatus},
{"event-log", runEventLog},
}
for _, tc := range testCases {
spec.SubTests = append(spec.SubTests, testSpec{
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)
}
}

0 comments on commit 8a08936

Please sign in to comment.