Skip to content

Commit

Permalink
log,server: avoid global variables to log/trace server IDs
Browse files Browse the repository at this point in the history
Prior to this change, the server identifiers (cluster ID, node ID etc)
were stored in global variables in the `log` package.
This was problematic when a single process contains multiple servers,
e.g. in tests, `demo` and multi-tenant CockroachDB.

This change switches the mechanism to use identifiers stored in the
go context. The disadvantage is that the server IDs are not any
more logged at the beginning of each log file (since a given log file
could report data from multiple servers).

Release note (cli change): The server identifiers (cluster ID, node
ID, tenant ID, instance ID) are not any more duplicated at the start
of every new log file (during log file rotations). They are now only
logged when known during server start-up.
(The copy of the identifiers is still included in per-event envelopes
for the various `json` output logging formats.)
  • Loading branch information
knz committed Dec 14, 2021
1 parent 7bf398f commit 0e6abb5
Show file tree
Hide file tree
Showing 35 changed files with 237 additions and 226 deletions.
22 changes: 0 additions & 22 deletions pkg/ccl/backupccl/backup_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -6873,9 +6873,6 @@ func TestBackupRestoreInsideTenant(t *testing.T) {
defer log.Scope(t).Close(t)

makeTenant := func(srv serverutils.TestServerInterface, tenant uint64) (*sqlutils.SQLRunner, func()) {
// Prevent a logging assertion that the server ID is initialized multiple times.
log.TestingClearServerIdentifiers()

_, conn := serverutils.StartTenant(t, srv, base.TestTenantArgs{TenantID: roachpb.MakeTenantID(tenant)})
cleanup := func() { conn.Close() }
return sqlutils.MakeSQLRunner(conn), cleanup
Expand Down Expand Up @@ -7031,16 +7028,11 @@ func TestBackupRestoreTenant(t *testing.T) {
systemDB.Exec(t, `BACKUP system.users TO 'nodelocal://1/users'`)
systemDB.CheckQueryResults(t, `SELECT manifest->>'tenants' FROM [SHOW BACKUP 'nodelocal://1/users' WITH as_json]`, [][]string{{"[]"}})

// 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 @@ -7100,8 +7092,6 @@ func TestBackupRestoreTenant(t *testing.T) {
[][]string{{`100`, `0`, `0`, `0`}},
)

log.TestingClearServerIdentifiers()

ten10Stopper := stop.NewStopper()
_, restoreConn10 := serverutils.StartTenant(
t, restoreTC.Server(0), base.TestTenantArgs{
Expand Down Expand Up @@ -7146,8 +7136,6 @@ 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 @@ -7187,8 +7175,6 @@ 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 @@ -7218,8 +7204,6 @@ func TestBackupRestoreTenant(t *testing.T) {
},
)

log.TestingClearServerIdentifiers()

_, restoreConn10 := serverutils.StartTenant(
t, restoreTC.Server(0), base.TestTenantArgs{TenantID: roachpb.MakeTenantID(10), Existing: true},
)
Expand All @@ -7229,8 +7213,6 @@ 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 @@ -7249,8 +7231,6 @@ 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 @@ -7269,8 +7249,6 @@ 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
1 change: 0 additions & 1 deletion pkg/ccl/changefeedccl/helpers_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -410,7 +410,6 @@ func startTestCluster(t testing.TB) (serverutils.TestClusterInterface, *gosql.DB
func startTestTenant(
t testing.TB, options feedTestOptions,
) (serverutils.TestServerInterface, *gosql.DB, func()) {
log.TestingClearServerIdentifiers()
ctx := context.Background()

kvServer, _, cleanup := startTestFullServer(t, options)
Expand Down
3 changes: 0 additions & 3 deletions pkg/ccl/importccl/import_stmt_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -6875,9 +6875,6 @@ func TestImportInTenant(t *testing.T) {
defer conn10.Close()
t10 := sqlutils.MakeSQLRunner(conn10)

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

// Setup a few tenants, each with a different table.
_, conn11 := serverutils.StartTenant(t, tc.Server(0), base.TestTenantArgs{TenantID: roachpb.MakeTenantID(11)})
defer conn11.Close()
Expand Down
8 changes: 0 additions & 8 deletions pkg/ccl/kvccl/kvtenantccl/tenant_upgrade_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -26,7 +26,6 @@ import (
"github.com/cockroachdb/cockroach/pkg/testutils/sqlutils"
"github.com/cockroachdb/cockroach/pkg/testutils/testcluster"
"github.com/cockroachdb/cockroach/pkg/util/leaktest"
"github.com/cockroachdb/cockroach/pkg/util/log"
"github.com/cockroachdb/cockroach/pkg/util/stop"
"github.com/stretchr/testify/require"
)
Expand Down Expand Up @@ -90,8 +89,6 @@ func TestTenantUpgrade(t *testing.T) {
TestingKnobs: base.TestingKnobs{},
Settings: settings,
}
// Prevent a logging assertion that the server ID is initialized multiple times.
log.TestingClearServerIdentifiers()
tenant, err := tc.Server(0).StartTenant(ctx, tenantArgs)
require.NoError(t, err)
return connectToTenant(t, tenant.SQLAddr())
Expand Down Expand Up @@ -130,7 +127,6 @@ func TestTenantUpgrade(t *testing.T) {
// Restart the tenant and ensure that the version is correct.
cleanup()
{
log.TestingClearServerIdentifiers()
tenantServer, err := tc.Server(0).StartTenant(ctx, base.TestTenantArgs{
TenantID: roachpb.MakeTenantID(initialTenantID),
Existing: true,
Expand All @@ -156,7 +152,6 @@ func TestTenantUpgrade(t *testing.T) {
// Restart the new tenant and ensure it has the right version.
cleanup()
{
log.TestingClearServerIdentifiers()
tenantServer, err := tc.Server(0).StartTenant(ctx, base.TestTenantArgs{
TenantID: roachpb.MakeTenantID(postUpgradeTenantID),
Existing: true,
Expand Down Expand Up @@ -286,8 +281,6 @@ func TestTenantUpgradeFailure(t *testing.T) {
},
Settings: settings,
}
// Prevent a logging assertion that the server ID is initialized multiple times.
log.TestingClearServerIdentifiers()
return &tenantInfo{tenantArgs: &tenantArgs,
v2onMigrationStopper: v2onMigrationStopper}
}
Expand Down Expand Up @@ -334,7 +327,6 @@ func TestTenantUpgradeFailure(t *testing.T) {
// Restart the tenant and ensure that the version is correct.
cleanup()
{
log.TestingClearServerIdentifiers()
tca, cleanup := startAndConnectToTenant(t, tenantInfo)
defer cleanup()
initialTenantRunner = sqlutils.MakeSQLRunner(tca)
Expand Down
1 change: 0 additions & 1 deletion pkg/ccl/serverccl/statusccl/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -16,7 +16,6 @@ go_library(
"//pkg/testutils/serverutils",
"//pkg/testutils/sqlutils",
"//pkg/util/httputil",
"//pkg/util/log",
"//pkg/util/protoutil",
"@com_github_stretchr_testify//require",
],
Expand Down
2 changes: 0 additions & 2 deletions pkg/ccl/serverccl/statusccl/tenant_grpc_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -91,7 +91,6 @@ func TestTenantGRPCServices(t *testing.T) {
sqlRunner.Exec(t, "CREATE TABLE test (id int)")
sqlRunner.Exec(t, "INSERT INTO test VALUES (1)")

log.TestingClearServerIdentifiers()
tenant2, connTenant2 := serverutils.StartTenant(t, server, base.TestTenantArgs{
TenantID: tenantID,
Existing: true,
Expand All @@ -110,7 +109,6 @@ func TestTenantGRPCServices(t *testing.T) {
require.Contains(t, string(body), "INSERT INTO test VALUES")
})

log.TestingClearServerIdentifiers()
tenant3, connTenant3 := serverutils.StartTenant(t, server, base.TestTenantArgs{
TenantID: roachpb.MakeTenantID(11),
TestingKnobs: testingKnobs,
Expand Down
2 changes: 0 additions & 2 deletions pkg/ccl/serverccl/statusccl/tenant_test_utils.go
Original file line number Diff line number Diff line change
Expand Up @@ -25,7 +25,6 @@ import (
"github.com/cockroachdb/cockroach/pkg/testutils/serverutils"
"github.com/cockroachdb/cockroach/pkg/testutils/sqlutils"
"github.com/cockroachdb/cockroach/pkg/util/httputil"
"github.com/cockroachdb/cockroach/pkg/util/log"
"github.com/cockroachdb/cockroach/pkg/util/protoutil"
"github.com/stretchr/testify/require"
)
Expand Down Expand Up @@ -58,7 +57,6 @@ func newTestTenant(
tenantParams.Existing = existing
tenantParams.TestingKnobs = knobs

log.TestingClearServerIdentifiers()
tenant, tenantConn := serverutils.StartTenant(t, server, tenantParams)
sqlDB := sqlutils.MakeSQLRunner(tenantConn)
status := tenant.StatusServer().(serverpb.SQLStatusServer)
Expand Down
1 change: 0 additions & 1 deletion pkg/ccl/sqlproxyccl/proxy_handler_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -1160,7 +1160,6 @@ func newDirectoryServer(
return nil, status.Error(codes.NotFound, "tenant not found")
}

log.TestingClearServerIdentifiers()
tenantStopper := tenantdirsvr.NewSubStopper(tdsStopper)
ten, err := srv.StartTenant(ctx, base.TestTenantArgs{
Existing: true,
Expand Down
1 change: 0 additions & 1 deletion pkg/ccl/sqlproxyccl/tenant/directory_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -486,7 +486,6 @@ func destroyTenant(tc serverutils.TestClusterInterface, id roachpb.TenantID) err
func startTenant(
ctx context.Context, srv serverutils.TestServerInterface, id uint64,
) (*tenantdirsvr.Process, error) {
log.TestingClearServerIdentifiers()
tenantStopper := tenantdirsvr.NewSubStopper(srv.Stopper())
t, err := srv.StartTenant(
ctx,
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -69,9 +69,6 @@ SET CLUSTER SETTING changefeed.experimental_poll_interval = '10ms'
`)
require.NoError(t, err)

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

// Start the destination server.
hDest, cleanupDest := streamingtest.NewReplicationHelper(t, base.TestServerArgs{})
defer cleanupDest()
Expand Down
8 changes: 1 addition & 7 deletions pkg/ccl/testccl/sqlccl/temp_table_clean_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -20,7 +20,6 @@ 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/stop"
"github.com/cockroachdb/cockroach/pkg/util/syncutil"
)
Expand Down Expand Up @@ -100,7 +99,6 @@ func TestTenantTempTableCleanup(t *testing.T) {
},
},
)
log.TestingClearServerIdentifiers()
tenantStoppers := []*stop.Stopper{stop.NewStopper(), stop.NewStopper()}
_, tenantPrimaryDB := serverutils.StartTenant(t, tc.Server(0),
base.TestTenantArgs{
Expand All @@ -118,16 +116,14 @@ func TestTenantTempTableCleanup(t *testing.T) {
tenantSQL.Exec(t, "SET experimental_enable_temp_tables = 'on'")
tenantSQL.Exec(t, "set cluster setting sql.temp_object_cleaner.cleanup_interval='1 seconds'")
tenantSQL.Exec(t, "CREATE TEMP TABLE temp_table (x INT PRIMARY KEY, y INT);")
// Prevent a logging assertion that the server ID is initialized multiple times.
log.TestingClearServerIdentifiers()

_, tenantSecondDB := serverutils.StartTenant(t, tc.Server(1),
base.TestTenantArgs{
Existing: true,
TenantID: serverutils.TestTenantID(),
Settings: settings,
Stopper: tenantStoppers[1],
})
log.TestingClearServerIdentifiers()
tenantSecondSQL := sqlutils.MakeSQLRunner(tenantSecondDB)
tenantSecondSQL.CheckQueryResults(t, "SELECT table_name FROM [SHOW TABLES]",
[][]string{
Expand Down Expand Up @@ -160,8 +156,6 @@ func TestTenantTempTableCleanup(t *testing.T) {
// Enable our hook to allow the database to be
// brought up.
pause()
// Prevent a logging assertion that the server ID is initialized multiple times.
log.TestingClearServerIdentifiers()
// Once we restart the tenant, no sessions should exist
// so all temporary tables should be cleaned up.
tenantStoppers[0] = stop.NewStopper()
Expand Down
1 change: 0 additions & 1 deletion pkg/cli/democluster/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -30,7 +30,6 @@ go_library(
"//pkg/server/serverpb",
"//pkg/server/status",
"//pkg/settings",
"//pkg/sql",
"//pkg/sql/catalog/catalogkeys",
"//pkg/sql/catalog/catconstants",
"//pkg/sql/distsql",
Expand Down
2 changes: 0 additions & 2 deletions pkg/cli/democluster/demo_cluster.go
Original file line number Diff line number Diff line change
Expand Up @@ -37,7 +37,6 @@ import (
"github.com/cockroachdb/cockroach/pkg/server/serverpb"
"github.com/cockroachdb/cockroach/pkg/server/status"
"github.com/cockroachdb/cockroach/pkg/settings"
"github.com/cockroachdb/cockroach/pkg/sql"
"github.com/cockroachdb/cockroach/pkg/sql/catalog/catalogkeys"
"github.com/cockroachdb/cockroach/pkg/sql/catalog/catconstants"
"github.com/cockroachdb/cockroach/pkg/sql/distsql"
Expand Down Expand Up @@ -407,7 +406,6 @@ func (c *transientCluster) Start(
StartingHTTPPort: c.demoCtx.HTTPPort - 2,
Locality: c.demoCtx.Localities[i],
TestingKnobs: base.TestingKnobs{
TenantTestingKnobs: &sql.TenantTestingKnobs{DisableLogTags: true},
Server: &server.TestingKnobs{
ContextTestingKnobs: rpc.ContextTestingKnobs{
ArtificialLatencyMap: latencyMap,
Expand Down
21 changes: 7 additions & 14 deletions pkg/cli/interactive_tests/test_log_config_msg.tcl
Original file line number Diff line number Diff line change
Expand Up @@ -9,30 +9,23 @@ start_server $argv

start_test "Check that the cluster and node ID is reported at the start of the first log file."
spawn tail -n 1000 -F logs/db/logs/cockroach.log
eexpect "\\\[config\\\] * clusterID:"
eexpect "\\\[config\\\] * nodeID:"
eexpect "node startup completed"
eexpect "start.go*clusterID:"
eexpect "start.go*nodeID:"
end_test

stop_server $argv


# Make a server with a tiny log buffer so as to force frequent log rotation.
system "$argv start-single-node --insecure --pid-file=server_pid --background -s=path=logs/db --log-file-max-size=2k >>logs/expect-cmd.log 2>&1;
$argv sql --insecure -e 'select 1'"
# Stop the server, which also flushes and closes the log files.
# Restart the server, to check that the server identifiers are also reported after restarts.
start_server $argv
stop_server $argv

start_test "Check that the cluster and node ID is reported at the start of new log files."
# Verify that the string "restarted pre-existing node" can be found
# somewhere. This ensures that if this string ever changes, the test
# below won't report a false negative.
# somewhere.
system "grep -q 'restarted pre-existing node' logs/db/logs/*.log"
# Verify that "cockroach.log" is not the file where the server reports
# it just started.
system "if grep -q 'restarted pre-existing node' logs/db/logs/cockroach.log; then false; fi"
# Verify that the last log file does contain the cluster ID.
system "grep -qF '\[config\] clusterID:' logs/db/logs/cockroach.log"
system "grep -qF '\[config\] nodeID:' logs/db/logs/cockroach.log"
system "grep -q 'start\.go.*clusterID:' logs/db/logs/cockroach.log"
system "grep -q 'start\.go.*nodeID:' logs/db/logs/cockroach.log"
end_test

6 changes: 0 additions & 6 deletions pkg/cli/start.go
Original file line number Diff line number Diff line change
Expand Up @@ -682,12 +682,6 @@ If problems persist, please see %s.`
return err
}

// Remember the server identifiers for logging.
// TODO(knz): Remove this.
nodeID := s.NodeID()
clusterID := s.ClusterID().String()
log.SetNodeIDs(clusterID, int32(nodeID))

// Now inform the user that the server is running and tell the
// user about its run-time derived parameters.
return reportServerInfo(ctx, tBegin, &serverCfg, s.ClusterSettings(), true /* isHostNode */, initialStart)
Expand Down
8 changes: 4 additions & 4 deletions pkg/cmd/roachtest/tests/multitenant.go
Original file line number Diff line number Diff line change
Expand Up @@ -58,17 +58,17 @@ func runAcceptanceMultitenant(ctx context.Context, t test.Test, c cluster.Cluste
t.Status("checking log file contents")

// Check that the server identifiers are present in the tenant log file.
logFile := filepath.Join(tenant.logDir(), "cockroach.log")
logFile := filepath.Join(tenant.logDir(), "*.log")
if err := c.RunE(ctx, c.Node(1),
"grep", "-q", "'\\[config\\] .* clusterID:'", logFile); err != nil {
"grep", "-q", "'start\\.go.*clusterID:'", logFile); err != nil {
t.Fatal(errors.Wrap(err, "cluster ID not found in log file"))
}
if err := c.RunE(ctx, c.Node(1),
"grep", "-q", "'\\[config\\] .* tenantID:'", logFile); err != nil {
"grep", "-q", "'start\\.go.*tenantID:'", logFile); err != nil {
t.Fatal(errors.Wrap(err, "tenant ID not found in log file"))
}
if err := c.RunE(ctx, c.Node(1),
"grep", "-q", "'\\[config\\] .* instanceID:'", logFile); err != nil {
"grep", "-q", "'start\\.go.*instanceID:'", logFile); err != nil {
t.Fatal(errors.Wrap(err, "SQL instance ID not found in log file"))
}
}
Loading

0 comments on commit 0e6abb5

Please sign in to comment.