Skip to content

Commit

Permalink
Merge #56634 #56698
Browse files Browse the repository at this point in the history
56634: timeutil: embed Go time zone data into CockroachDB r=knz a=otan

Refs #36864

* Use go's time/tzdata to embed timezone data.
* Modify timeutil package to always use this package.
* Remove cli checks for tzdata.
* Add lint rule to use timeutil.LoadLocation

The binary size changes by ~1MB here.

Release note (general change): The timezone data is now built in to the
CockroachDB binary, which is the fallback source of time if tzdata is
not located by the default Go standard library.

56698: stopper: assert that stoppers don't leak r=tbg a=knz

Fixes #56697
Fixes #56696

Until now leaked stoppers were just reported in tests but did not
cause failures. This PR switches the check to cause errors instead.

Release note: None

Co-authored-by: Oliver Tan <[email protected]>
Co-authored-by: Raphael 'kena' Poss <[email protected]>
  • Loading branch information
3 people committed Nov 16, 2020
3 parents b4bf42c + 90efaaf + c0aa632 commit e7e0f26
Show file tree
Hide file tree
Showing 21 changed files with 71 additions and 91 deletions.
2 changes: 1 addition & 1 deletion pkg/ccl/serverccl/role_authentication_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -47,7 +47,7 @@ func TestVerifyPassword(t *testing.T) {
}

//location is used for timezone testing.
shanghaiLoc, err := time.LoadLocation("Asia/Shanghai")
shanghaiLoc, err := timeutil.LoadLocation("Asia/Shanghai")
if err != nil {
t.Fatal(err)
}
Expand Down
4 changes: 0 additions & 4 deletions pkg/cli/demo.go
Original file line number Diff line number Diff line change
Expand Up @@ -269,10 +269,6 @@ func runDemo(cmd *cobra.Command, gen workload.Generator) (err error) {
}
defer c.cleanup(ctx)

if err := checkTzDatabaseAvailability(ctx); err != nil {
return err
}

loc, err := geos.EnsureInit(geos.EnsureInitErrorDisplayPrivate, startCtx.geoLibsDir)
if err != nil {
log.Infof(ctx, "could not initialize GEOS - spatial functions may not be available: %v", err)
Expand Down
39 changes: 22 additions & 17 deletions pkg/cli/demo_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -11,6 +11,7 @@
package cli

import (
"context"
"fmt"
"testing"

Expand Down Expand Up @@ -65,27 +66,31 @@ func TestTestServerArgsForTransientCluster(t *testing.T) {
},
}

for _, tc := range testCases {
demoCtxTemp := demoCtx
demoCtx.sqlPoolMemorySize = tc.sqlPoolMemorySize
demoCtx.cacheSize = tc.cacheSize
for i, tc := range testCases {
t.Run(fmt.Sprint(i), func(t *testing.T) {
demoCtxTemp := demoCtx
demoCtx.sqlPoolMemorySize = tc.sqlPoolMemorySize
demoCtx.cacheSize = tc.cacheSize

actual := testServerArgsForTransientCluster(unixSocketDetails{}, tc.nodeID, tc.joinAddr, "")
actual := testServerArgsForTransientCluster(unixSocketDetails{}, tc.nodeID, tc.joinAddr, "")
stopper := actual.Stopper
defer stopper.Stop(context.Background())

assert.Len(t, actual.StoreSpecs, 1)
assert.Equal(
t,
fmt.Sprintf("demo-node%d", tc.nodeID),
actual.StoreSpecs[0].StickyInMemoryEngineID,
)
assert.Len(t, actual.StoreSpecs, 1)
assert.Equal(
t,
fmt.Sprintf("demo-node%d", tc.nodeID),
actual.StoreSpecs[0].StickyInMemoryEngineID,
)

// We cannot compare these.
actual.Stopper = nil
actual.StoreSpecs = nil
// We cannot compare these.
actual.Stopper = nil
actual.StoreSpecs = nil

assert.Equal(t, tc.expected, actual)
assert.Equal(t, tc.expected, actual)

// Restore demoCtx state after each test.
demoCtx = demoCtxTemp
// Restore demoCtx state after each test.
demoCtx = demoCtxTemp
})
}
}
36 changes: 0 additions & 36 deletions pkg/cli/start.go
Original file line number Diff line number Diff line change
Expand Up @@ -403,13 +403,6 @@ func runStart(cmd *cobra.Command, args []string, startSingleNode bool) error {
// registered.
reportConfiguration(ctx)

// Until/unless CockroachDB embeds its own tz database, we want
// an early sanity check. It's better to inform the user early
// than to get surprising errors during SQL queries.
if err := checkTzDatabaseAvailability(ctx); err != nil {
return errors.Wrap(err, "failed to initialize node")
}

// ReadyFn will be called when the server has started listening on
// its network sockets, but perhaps before it has done bootstrapping
// and thus before Start() completes.
Expand Down Expand Up @@ -923,35 +916,6 @@ func clientFlagsRPC() string {
return strings.Join(flags, " ")
}

func checkTzDatabaseAvailability(ctx context.Context) error {
if _, err := timeutil.LoadLocation("America/New_York"); err != nil {
log.Errorf(ctx, "timeutil.LoadLocation: %v", err)
reportedErr := errors.WithHint(
errors.WithIssueLink(
errors.New("unable to load named timezones"),
errors.IssueLink{IssueURL: unimplemented.MakeURL(36864)}),
"Check that the time zone database is installed on your system, or\n"+
"set the ZONEINFO environment variable to a Go time zone .zip archive.")

if envutil.EnvOrDefaultBool("COCKROACH_INCONSISTENT_TIME_ZONES", false) {
// The user tells us they really know what they want.
reportedErr := &formattedError{err: reportedErr}
log.Shoutf(ctx, log.Severity_WARNING, "%v", reportedErr)
} else {
// Prevent a successful start.
//
// In the past, we were simply using log.Shout to emit an error,
// informing the user that startup could continue with degraded
// behavior. However, usage demonstrated that users typically do
// not see the error and instead run into silently incorrect SQL
// results. To avoid this situation altogether, it's better to
// stop early.
return reportedErr
}
}
return nil
}

func reportConfiguration(ctx context.Context) {
serverCfg.Report(ctx)
if envVarsUsed := envutil.GetEnvVarsUsed(); len(envVarsUsed) > 0 {
Expand Down
4 changes: 3 additions & 1 deletion pkg/internal/client/requestbatcher/batcher_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -548,10 +548,12 @@ func TestMaxKeysPerBatchReq(t *testing.T) {

func TestPanicWithNilSender(t *testing.T) {
defer leaktest.AfterTest(t)()
stopper := stop.NewStopper()
defer stopper.Stop(context.Background())
defer func() {
if r := recover(); r == nil {
t.Fatalf("failed to panic with a nil Sender")
}
}()
New(Config{Stopper: stop.NewStopper()})
New(Config{Stopper: stopper})
}
16 changes: 15 additions & 1 deletion pkg/kv/kvclient/kvcoord/range_cache_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -35,6 +35,7 @@ import (

type testDescriptorDB struct {
data llrb.Tree
stopper *stop.Stopper
cache *RangeDescriptorCache
lookupCount int64
disablePrefetch bool
Expand Down Expand Up @@ -256,10 +257,13 @@ func initTestDescriptorDB(t *testing.T) *testDescriptorDB {
}
}
// TODO(andrei): don't leak this Stopper. Someone needs to Stop() it.
db.cache = NewRangeDescriptorCache(st, db, staticSize(2<<10), stop.NewStopper())
db.stopper = stop.NewStopper()
db.cache = NewRangeDescriptorCache(st, db, staticSize(2<<10), db.stopper)
return db
}

func (db *testDescriptorDB) stop() { db.stopper.Stop(context.Background()) }

// assertLookupCountEq fails unless exactly the number of lookups have been observed.
func (db *testDescriptorDB) assertLookupCountEq(t *testing.T, exp int64, key string) {
t.Helper()
Expand Down Expand Up @@ -331,6 +335,7 @@ func TestDescriptorDBGetDescriptors(t *testing.T) {
defer leaktest.AfterTest(t)()
defer log.Scope(t).Close(t)
db := initTestDescriptorDB(t)
defer db.stop()

key := roachpb.RKey("k")
expectedRspansMap := map[bool][]roachpb.RSpan{
Expand Down Expand Up @@ -388,6 +393,7 @@ func TestRangeCache(t *testing.T) {
defer leaktest.AfterTest(t)()
defer log.Scope(t).Close(t)
db := initTestDescriptorDB(t)
defer db.stop()
ctx := context.Background()

// Totally uncached range.
Expand Down Expand Up @@ -504,6 +510,7 @@ func TestRangeCacheCoalescedRequests(t *testing.T) {
defer leaktest.AfterTest(t)()
defer log.Scope(t).Close(t)
db := initTestDescriptorDB(t)
defer db.stop()
ctx := context.Background()

pauseLookupResumeAndAssert := func(key string, expected int64) {
Expand Down Expand Up @@ -561,6 +568,7 @@ func TestRangeCacheContextCancellation(t *testing.T) {
defer leaktest.AfterTest(t)()
defer log.Scope(t).Close(t)
db := initTestDescriptorDB(t)
defer db.stop()

// lookupAndWaitUntilJoin performs a RangeDescriptor lookup in a new
// goroutine and blocks until the request is added to the inflight request
Expand Down Expand Up @@ -627,6 +635,7 @@ func TestRangeCacheDetectSplit(t *testing.T) {
defer leaktest.AfterTest(t)()
defer log.Scope(t).Close(t)
db := initTestDescriptorDB(t)
defer db.stop()
ctx := context.Background()

pauseLookupResumeAndAssert := func(key string, evictToken EvictionToken) {
Expand Down Expand Up @@ -696,6 +705,7 @@ func TestRangeCacheDetectSplitReverseScan(t *testing.T) {
defer leaktest.AfterTest(t)()
defer log.Scope(t).Close(t)
db := initTestDescriptorDB(t)
defer db.stop()
ctx := context.Background()

// A request initially looks up the range descriptor ["a"-"b").
Expand Down Expand Up @@ -827,6 +837,8 @@ func TestRangeCacheHandleDoubleSplit(t *testing.T) {
for _, tc := range testCases {
t.Run(fmt.Sprintf("reverse=%t", tc.reverseScan), func(t *testing.T) {
db := initTestDescriptorDB(t)
defer db.stop()

db.disablePrefetch = true
ctx := context.Background()

Expand Down Expand Up @@ -951,6 +963,8 @@ func TestRangeCacheUseIntents(t *testing.T) {
defer leaktest.AfterTest(t)()
defer log.Scope(t).Close(t)
db := initTestDescriptorDB(t)
defer db.stop()

ctx := context.Background()

// A request initially looks up the range descriptor ["a"-"b").
Expand Down
1 change: 1 addition & 0 deletions pkg/kv/kvserver/client_merge_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -3401,6 +3401,7 @@ func TestStoreRangeMergeDuringShutdown(t *testing.T) {
startWithSingleRange: true,
}
mtc.Start(t, 1)
defer mtc.Stop()
store := mtc.Store(0)
stopper := mtc.engineStoppers[0]

Expand Down
1 change: 1 addition & 0 deletions pkg/kv/kvserver/closedts/container/container_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -123,6 +123,7 @@ func setupTwoNodeTest() (_ *TestContainer, _ *TestContainer, shutdown func()) {
defer wg.Done()
c2.Stopper.Stop(context.Background())
}()
wg.Wait()
}
}

Expand Down
4 changes: 3 additions & 1 deletion pkg/kv/kvserver/rangefeed/processor_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -432,7 +432,9 @@ func TestNilProcessor(t *testing.T) {

// The following should panic because they are not safe
// to call on a nil Processor.
require.Panics(t, func() { p.Start(stop.NewStopper(), nil) })
stopper := stop.NewStopper()
defer stopper.Stop(context.Background())
require.Panics(t, func() { p.Start(stopper, nil) })
require.Panics(t, func() { p.Register(roachpb.RSpan{}, hlc.Timestamp{}, nil, false, nil, nil) })
}

Expand Down
2 changes: 1 addition & 1 deletion pkg/server/authentication_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -205,7 +205,7 @@ func TestVerifyPassword(t *testing.T) {
}

//location is used for timezone testing.
shanghaiLoc, err := time.LoadLocation("Asia/Shanghai")
shanghaiLoc, err := timeutil.LoadLocation("Asia/Shanghai")
if err != nil {
t.Fatal(err)
}
Expand Down
4 changes: 3 additions & 1 deletion pkg/sql/distsql_physical_planner_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -261,7 +261,9 @@ func TestDistSQLReceiverUpdatesCaches(t *testing.T) {

size := func() int64 { return 2 << 10 }
st := cluster.MakeTestingClusterSettings()
rangeCache := kvcoord.NewRangeDescriptorCache(st, nil /* db */, size, stop.NewStopper())
stopper := stop.NewStopper()
defer stopper.Stop(ctx)
rangeCache := kvcoord.NewRangeDescriptorCache(st, nil /* db */, size, stopper)
r := MakeDistSQLReceiver(
ctx, nil /* resultWriter */, tree.Rows,
rangeCache, nil /* txn */, nil /* updateClock */, &SessionTracing{})
Expand Down
3 changes: 2 additions & 1 deletion pkg/sql/sem/tree/datum_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -27,6 +27,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/util/log"
"github.com/cockroachdb/cockroach/pkg/util/timeofday"
"github.com/cockroachdb/cockroach/pkg/util/timetz"
"github.com/cockroachdb/cockroach/pkg/util/timeutil"
"github.com/stretchr/testify/assert"
"github.com/stretchr/testify/require"
)
Expand Down Expand Up @@ -848,7 +849,7 @@ func TestDTimeTZ(t *testing.T) {
require.False(t, depOnCtx)

// No daylight savings in Hawaii!
hawaiiZone, err := time.LoadLocation("Pacific/Honolulu")
hawaiiZone, err := timeutil.LoadLocation("Pacific/Honolulu")
require.NoError(t, err)
hawaiiTime := tree.NewDTimeTZFromLocation(timeofday.New(1, 14, 15, 0), hawaiiZone)

Expand Down
5 changes: 3 additions & 2 deletions pkg/testutils/lint/lint_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -709,13 +709,14 @@ func TestLint(t *testing.T) {
"git",
"grep",
"-nE",
`\btime\.(Now|Since|Unix)\(`,
`\btime\.(Now|Since|Unix|LoadLocation)\(`,
"--",
"*.go",
":!**/embedded.go",
":!util/timeutil/time.go",
":!util/timeutil/now_unix.go",
":!util/timeutil/now_windows.go",
":!util/timeutil/time.go",
":!util/timeutil/zoneinfo.go",
":!util/tracing/span.go",
":!util/tracing/tracer.go",
)
Expand Down
2 changes: 0 additions & 2 deletions pkg/util/leaktest/leaktest.go
Original file line number Diff line number Diff line change
Expand Up @@ -112,8 +112,6 @@ func AfterTest(t testing.TB) func() {
return
}

// TODO(tbg): make this call 't.Error' instead of 't.Logf' once there is
// enough Stopper discipline.
PrintLeakedStoppers(t)

// Loop, waiting for goroutines to shut down.
Expand Down
2 changes: 1 addition & 1 deletion pkg/util/stop/stopper.go
Original file line number Diff line number Diff line change
Expand Up @@ -95,7 +95,7 @@ func PrintLeakedStoppers(t testing.TB) {
trackedStoppers.Lock()
defer trackedStoppers.Unlock()
for _, tracked := range trackedStoppers.stoppers {
t.Logf("leaked stopper, created at:\n%s", tracked.createdAt)
t.Errorf("leaked stopper, created at:\n%s", tracked.createdAt)
}
}

Expand Down
1 change: 1 addition & 0 deletions pkg/util/stop/stopper_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -346,6 +346,7 @@ func TestStopperRunTaskPanic(t *testing.T) {
s := stop.NewStopper(stop.OnPanic(func(v interface{}) {
ch <- v
}))
defer s.Stop(context.Background())
// If RunTask were not panic-safe, Stop() would deadlock.
type testFn func()
explode := func(context.Context) { panic(ch) }
Expand Down
2 changes: 1 addition & 1 deletion pkg/util/timetz/timetz_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -83,7 +83,7 @@ func TestTimeTZ(t *testing.T) {
require.False(t, depOnCtx)

// No daylight savings in Hawaii!
hawaiiZone, err := time.LoadLocation("Pacific/Honolulu")
hawaiiZone, err := timeutil.LoadLocation("Pacific/Honolulu")
require.NoError(t, err)
hawaiiTime := MakeTimeTZFromLocation(timeofday.New(1, 14, 15, 0), hawaiiZone)

Expand Down
5 changes: 3 additions & 2 deletions pkg/util/timeutil/pgdate/parsing_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -19,6 +19,7 @@ import (
"testing"
"time"

"github.com/cockroachdb/cockroach/pkg/util/timeutil"
"github.com/cockroachdb/cockroach/pkg/util/timeutil/pgdate"
_ "github.com/lib/pq"
)
Expand Down Expand Up @@ -819,7 +820,7 @@ func BenchmarkParseTimestampComparison(b *testing.B) {
}

// bench compares our ParseTimestamp to ParseInLocation, optionally
// chained with a time.LoadLocation() for resolving named zones.
// chained with a timeutil.LoadLocation() for resolving named zones.
// The layout parameter is only used for time.ParseInLocation().
// When a named timezone is used, it must be passed via locationName
// so that it may be resolved to a time.Location. It will be
Expand Down Expand Up @@ -850,7 +851,7 @@ func bench(b *testing.B, layout string, s string, locationName string) {
loc := time.UTC
if locationName != "" {
var err error
loc, err = time.LoadLocation(locationName)
loc, err = timeutil.LoadLocation(locationName)
if err != nil {
b.Fatal(err)
}
Expand Down
Loading

0 comments on commit e7e0f26

Please sign in to comment.