Skip to content

Commit

Permalink
Merge pull request #58708 from knz/20210111-clusterid-in-logs
Browse files Browse the repository at this point in the history
release-20.2: util/log: conditionally include the tenant+server IDs on every line
  • Loading branch information
andy-kimball authored Jan 13, 2021
2 parents 5471a53 + ebbc531 commit 3e61d3d
Show file tree
Hide file tree
Showing 10 changed files with 193 additions and 19 deletions.
21 changes: 21 additions & 0 deletions pkg/ccl/backupccl/backup_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -5808,6 +5808,8 @@ func TestProtectedTimestampsFailDueToLimits(t *testing.T) {
// Ensure that backing up and restoring tenants succeeds.
func TestBackupRestoreTenant(t *testing.T) {
defer leaktest.AfterTest(t)()
defer log.Scope(t).Close(t)

defer jobs.TestingSetAdoptAndCancelIntervals(100*time.Millisecond, 100*time.Millisecond)()

const numAccounts = 1
Expand All @@ -5824,11 +5826,16 @@ func TestBackupRestoreTenant(t *testing.T) {
tenant10 := sqlutils.MakeSQLRunner(conn10)
tenant10.Exec(t, `CREATE DATABASE foo; CREATE TABLE foo.bar(i int primary key); INSERT INTO foo.bar VALUES (110), (210)`)

// Prevent a logging assertion that the server ID is initialized multiple times.
log.TestingClearServerIdentifiers()

_, conn11 := serverutils.StartTenant(t, srv, base.TestTenantArgs{TenantID: roachpb.MakeTenantID(11)})
defer conn11.Close()
tenant11 := sqlutils.MakeSQLRunner(conn11)
tenant11.Exec(t, `CREATE DATABASE foo; CREATE TABLE foo.baz(i int primary key); INSERT INTO foo.baz VALUES (111), (211)`)

log.TestingClearServerIdentifiers()

_, conn20 := serverutils.StartTenant(t, srv, base.TestTenantArgs{TenantID: roachpb.MakeTenantID(20)})
defer conn20.Close()
tenant20 := sqlutils.MakeSQLRunner(conn20)
Expand Down Expand Up @@ -5880,6 +5887,8 @@ func TestBackupRestoreTenant(t *testing.T) {
[][]string{{`10`, `true`, `{"id": "10", "state": "ACTIVE"}`}},
)

log.TestingClearServerIdentifiers()

ten10Stopper := stop.NewStopper()
_, restoreConn10 := serverutils.StartTenant(
t, restoreTC.Server(0), base.TestTenantArgs{
Expand Down Expand Up @@ -5914,6 +5923,8 @@ func TestBackupRestoreTenant(t *testing.T) {
[][]string{{`10`, `true`, `{"id": "10", "state": "ACTIVE"}`}},
)

log.TestingClearServerIdentifiers()

_, restoreConn10 = serverutils.StartTenant(
t, restoreTC.Server(0), base.TestTenantArgs{TenantID: roachpb.MakeTenantID(10), Existing: true},
)
Expand All @@ -5938,6 +5949,8 @@ func TestBackupRestoreTenant(t *testing.T) {
[][]string{{`10`, `true`, `{"id": "10", "state": "ACTIVE"}`}},
)

log.TestingClearServerIdentifiers()

_, restoreConn10 := serverutils.StartTenant(
t, restoreTC.Server(0), base.TestTenantArgs{TenantID: roachpb.MakeTenantID(10), Existing: true},
)
Expand Down Expand Up @@ -5967,6 +5980,8 @@ func TestBackupRestoreTenant(t *testing.T) {
},
)

log.TestingClearServerIdentifiers()

_, restoreConn10 := serverutils.StartTenant(
t, restoreTC.Server(0), base.TestTenantArgs{TenantID: roachpb.MakeTenantID(10), Existing: true},
)
Expand All @@ -5976,6 +5991,8 @@ func TestBackupRestoreTenant(t *testing.T) {
restoreTenant10.CheckQueryResults(t, `select * from foo.bar`, tenant10.QueryStr(t, `select * from foo.bar`))
restoreTenant10.CheckQueryResults(t, `select * from foo.bar2`, tenant10.QueryStr(t, `select * from foo.bar2`))

log.TestingClearServerIdentifiers()

_, restoreConn11 := serverutils.StartTenant(
t, restoreTC.Server(0), base.TestTenantArgs{TenantID: roachpb.MakeTenantID(11), Existing: true},
)
Expand All @@ -5994,6 +6011,8 @@ func TestBackupRestoreTenant(t *testing.T) {

restoreDB.Exec(t, `RESTORE TENANT 10 FROM 'nodelocal://1/t10' AS OF SYSTEM TIME `+ts1)

log.TestingClearServerIdentifiers()

_, restoreConn10 := serverutils.StartTenant(
t, restoreTC.Server(0), base.TestTenantArgs{TenantID: roachpb.MakeTenantID(10), Existing: true},
)
Expand All @@ -6012,6 +6031,8 @@ func TestBackupRestoreTenant(t *testing.T) {

restoreDB.Exec(t, `RESTORE TENANT 20 FROM 'nodelocal://1/t20'`)

log.TestingClearServerIdentifiers()

_, restoreConn20 := serverutils.StartTenant(
t, restoreTC.Server(0), base.TestTenantArgs{TenantID: roachpb.MakeTenantID(20), Existing: true},
)
Expand Down
11 changes: 11 additions & 0 deletions pkg/ccl/backupccl/create_scheduled_backup_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -35,6 +35,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/testutils/serverutils"
"github.com/cockroachdb/cockroach/pkg/testutils/sqlutils"
"github.com/cockroachdb/cockroach/pkg/util/leaktest"
"github.com/cockroachdb/cockroach/pkg/util/log"
"github.com/cockroachdb/cockroach/pkg/util/timeutil"
pbtypes "github.com/gogo/protobuf/types"
"github.com/stretchr/testify/require"
Expand Down Expand Up @@ -172,6 +173,8 @@ func (t userType) String() string {
// itself with the actual scheduling and the execution of those backups.
func TestSerializesScheduledBackupExecutionArgs(t *testing.T) {
defer leaktest.AfterTest(t)()
defer log.Scope(t).Close(t)

th, cleanup := newTestHelper(t)
defer cleanup()

Expand Down Expand Up @@ -429,6 +432,8 @@ func TestSerializesScheduledBackupExecutionArgs(t *testing.T) {

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

th, cleanup := newTestHelper(t)
defer cleanup()

Expand Down Expand Up @@ -570,6 +575,8 @@ INSERT INTO t1 values (-1), (10), (-100);

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

th, cleanup := newTestHelper(t)
defer cleanup()

Expand All @@ -591,6 +598,8 @@ func TestCreateBackupScheduleRequiresAdminRole(t *testing.T) {

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

th, cleanup := newTestHelper(t)
defer cleanup()

Expand All @@ -611,6 +620,8 @@ func TestCreateBackupScheduleCollectionOverwrite(t *testing.T) {

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

th, cleanup := newTestHelper(t)
defer cleanup()

Expand Down
47 changes: 47 additions & 0 deletions pkg/cli/interactive_tests/test_force_auth_log.tcl
Original file line number Diff line number Diff line change
@@ -0,0 +1,47 @@
#! /usr/bin/env expect -f

source [file join [file dirname $argv0] common.tcl]

set ::env(COCKROACH_INSECURE) "false"
set ::env(COCKROACH_HOST) "localhost"
set certs_dir "/certs"

set ::env(COCKROACH_ALWAYS_LOG_SERVER_IDS) 1
set ::env(COCKROACH_ALWAYS_LOG_AUTHN_EVENTS) 1

proc start_secure_server {argv certs_dir extra} {
report "BEGIN START SECURE SERVER"
system "$argv start-single-node --host=localhost --socket-dir=. --certs-dir=$certs_dir --pid-file=server_pid -s=path=logs/db --background $extra >>expect-cmd.log 2>&1;
$argv sql --certs-dir=$certs_dir -e 'select 1'"
report "END START SECURE SERVER"
}

proc stop_secure_server {argv certs_dir} {
report "BEGIN STOP SECURE SERVER"
system "$argv quit --certs-dir=$certs_dir"
report "END STOP SECURE SERVER"
}


start_secure_server $argv $certs_dir ""

set logfile logs/db/logs/cockroach-auth.log

# run a client command, so we have at least one authn event in the log.
system "$argv sql -e 'create user someuser' --certs-dir=$certs_dir"
system "$argv sql -e 'select 1' --user someuser --certs-dir=$certs_dir</dev/null || true"

start_test "Check that the authentication events are reported"

system "grep -q 'authentication succeeded' $logfile"
system "grep -q 'authentication failed' $logfile"

end_test

start_test "Check that the auth events have both node ID and cluster ID"

system "grep -q '\\\[n1,.*clusterID=........-....-....-....-............\\\] . authentication' $logfile"

end_test

stop_secure_server $argv $certs_dir
7 changes: 7 additions & 0 deletions pkg/server/testserver.go
Original file line number Diff line number Diff line change
Expand Up @@ -57,6 +57,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/ts"
"github.com/cockroachdb/cockroach/pkg/util"
"github.com/cockroachdb/cockroach/pkg/util/hlc"
"github.com/cockroachdb/cockroach/pkg/util/log"
"github.com/cockroachdb/cockroach/pkg/util/metric"
"github.com/cockroachdb/cockroach/pkg/util/netutil"
"github.com/cockroachdb/cockroach/pkg/util/stop"
Expand Down Expand Up @@ -649,6 +650,7 @@ func StartTenant(
if err != nil {
return nil, "", "", err
}

s, err := newSQLServer(ctx, args)
if err != nil {
return nil, "", "", err
Expand Down Expand Up @@ -746,6 +748,11 @@ func StartTenant(
return nil, "", "", err
}

// Inform the logging package of the cluster identifiers.
clusterID := args.rpcContext.ClusterID.Get().String()
log.SetClusterID(clusterID)
log.SetTenantIDs(args.TenantID.String(), int32(s.SQLInstanceID()))

return s, pgLAddr, httpLAddr, nil
}

Expand Down
11 changes: 10 additions & 1 deletion pkg/sql/logictest/logic.go
Original file line number Diff line number Diff line change
Expand Up @@ -1335,6 +1335,10 @@ func (t *logicTest) setup(cfg testClusterConfig, serverArgs TestServerArgs) {
TenantID: roachpb.MakeTenantID(10),
AllowSettingClusterSettings: true,
}

// Prevent a logging assertion that the server ID is initialized multiple times.
log.TestingClearServerIdentifiers()

tenant, err := t.cluster.Server(t.nodeIdx).StartTenant(tenantArgs)
if err != nil {
t.rootT.Fatalf("%+v", err)
Expand Down Expand Up @@ -2877,9 +2881,14 @@ func RunLogicTestWithDefaultConfig(
// - we're generating testfiles, or
// - we are in race mode (where we can hit a limit on alive
// goroutines).
if !*showSQL && !*rewriteResultsInTestfiles && !*rewriteSQL && !util.RaceEnabled {
if !*showSQL && !*rewriteResultsInTestfiles && !*rewriteSQL && !util.RaceEnabled && !cfg.useTenant {
// Skip parallelizing tests that use the kv-batch-size directive since
// the batch size is a global variable.
//
// We also cannot parallelise tests that use tenant servers
// because they change shared state in the logging configuration
// and there is an assertion against conflicting changes.
//
// TODO(jordan, radu): make sqlbase.kvBatchSize non-global to fix this.
if filepath.Base(path) != "select_index_span_ranges" {
t.Parallel() // SAFE FOR TESTING (this comments satisfies the linter)
Expand Down
3 changes: 3 additions & 0 deletions pkg/sql/telemetry_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -144,6 +144,9 @@ func (tt *telemetryTest) Start(t *testing.T) {
tt.server, tt.serverDB, _ = serverutils.StartServer(tt.t, params)
tt.prepareCluster(tt.serverDB)

// Prevent a logging assertion that the server ID is initialized multiple times.
log.TestingClearServerIdentifiers()

tt.tenant, tt.tenantDB = serverutils.StartTenant(tt.t, tt.server, base.TestTenantArgs{
TenantID: roachpb.MakeTenantID(security.EmbeddedTenantIDs()[0]),
AllowSettingClusterSettings: true,
Expand Down
92 changes: 78 additions & 14 deletions pkg/util/log/clog.go
Original file line number Diff line number Diff line change
Expand Up @@ -17,11 +17,14 @@ import (
"context"
"fmt"
"runtime/debug"
"strconv"
"strings"
"sync"
"sync/atomic"
"time"

"github.com/cockroachdb/cockroach/pkg/cli/exit"
"github.com/cockroachdb/cockroach/pkg/util/envutil"
"github.com/cockroachdb/cockroach/pkg/util/syncutil"
"github.com/cockroachdb/logtags"
)
Expand Down Expand Up @@ -62,10 +65,6 @@ type loggingT struct {
hideStack bool // hides stack trace; only in effect when f is not nil
}

// the Cluster ID is reported on every new log file so as to ease the correlation
// of panic reports with self-reported log files.
clusterID string

// fatalCh is closed on fatal errors.
fatalCh chan struct{}

Expand All @@ -74,6 +73,18 @@ type loggingT struct {
active bool
firstUseStack string
}

// the Cluster ID is reported on every new log file so as to ease the correlation
// of panic reports with self-reported log files.
//
// It is also set conditionally by an env var. See the doc string
// for 'alwaysAddClusterID'. (Feature deprecated in v20.2)
clusterID syncutil.AtomicString

// The following identifiers are reported when enabled.
// Note: sqlInstanceID must be accessed atomically.
tenantID syncutil.AtomicString
sqlInstanceID int32
}

type loggerT struct {
Expand Down Expand Up @@ -194,28 +205,49 @@ func (l *loggingT) signalFatalCh() {
}
}

// SetClusterID stores the Cluster ID for further reference.
//
// TODO(knz): This should not be configured per-logger.
// See: https://github.com/cockroachdb/cockroach/issues/40983
// SetClusterID stores the cluster ID for further reference.
func SetClusterID(clusterID string) {
// Ensure that the clusterID is logged with the same format as for
// Ensure that the ID gets logged with the same format as for
// new log files, even on the first log file. This ensures that grep
// will always find it.
ctx := logtags.AddTag(context.Background(), "config", nil)
addStructured(ctx, Severity_INFO, 1, "clusterID: %s", []interface{}{clusterID})

// Perform the change proper.
logging.mu.Lock()
defer logging.mu.Unlock()

if logging.mu.clusterID != "" {
if logging.clusterID.Get() != "" {
panic("clusterID already set")
}

logging.mu.clusterID = clusterID
logging.clusterID.Set(clusterID)
}

// SetTenantIDs stores the tenant ID and instance ID for further reference.
func SetTenantIDs(tenantID string, sqlInstanceID int32) {
// Ensure that the IDs are logged with the same format as for
// new log files, even on the first log file. This ensures that grep
// will always find it.
ctx := logtags.AddTag(context.Background(), "config", nil)
addStructured(ctx, Severity_INFO, 1, "tenantID: %s", []interface{}{tenantID})
addStructured(ctx, Severity_INFO, 1, "instanceID: %d", []interface{}{sqlInstanceID})

// Perform the change proper.
if logging.tenantID.Get() != "" {
panic("tenantID already set")
}

logging.tenantID.Set(tenantID)
atomic.StoreInt32(&logging.sqlInstanceID, sqlInstanceID)
}

// alwaysAddServerIDs, when set, indicates that the cluster and other
// server IDs must be reported in the log tags on every log line.
//
// We use an env var here because we need this feature in SQL pod
// logs, and in v20.2 SQL pods don't have access to cluster settings.
// Note: This feature is immediately obsolete, as v21.1 has JSON
// logging and includes the cluster ID in log events already.
var alwaysAddServerIDs = envutil.EnvOrDefaultBool("COCKROACH_ALWAYS_LOG_SERVER_IDS", false)

// outputLogEntry marshals a log entry proto into bytes, and writes
// the data to the log files. If a trace location is set, stack traces
// are added to the entry before marshaling.
Expand All @@ -228,6 +260,38 @@ func (l *loggerT) outputLogEntry(entry Entry) {
// TODO(tschottdorf): this is a pretty horrible critical section.
l.mu.Lock()

if alwaysAddServerIDs {
// Only emit the cluster ID in tags if requested, and after the
// cluster ID is known already. (It may not be known for
// uninitialized clusters.)
// NB: This code is superseded in CockroachDB v21.1 by JSON
// logging which includes these details unconditionally.
var buf strings.Builder
buf.WriteString(entry.Tags)
if clusterID := logging.clusterID.Get(); len(clusterID) > 0 {
if buf.Len() > 0 {
buf.WriteByte(',')
}
buf.WriteString("clusterID=")
buf.WriteString(clusterID)
}
if tenantID := logging.tenantID.Get(); len(tenantID) > 0 {
if buf.Len() > 0 {
buf.WriteByte(',')
}
buf.WriteString("tenantID=")
buf.WriteString(tenantID)
}
if sqlInstanceID := atomic.LoadInt32(&logging.sqlInstanceID); sqlInstanceID != 0 {
if buf.Len() > 0 {
buf.WriteByte(',')
}
buf.WriteString("instanceID=")
buf.WriteString(strconv.Itoa(int(sqlInstanceID)))
}
entry.Tags = buf.String()
}

// Mark the logger as active, so that further configuration changes
// are disabled. See IsActive() and its callers for details.
setActive()
Expand Down
Loading

0 comments on commit 3e61d3d

Please sign in to comment.