Skip to content

Commit

Permalink
util/timeutil: don't strip mono time in timeutil.Now
Browse files Browse the repository at this point in the history
Our timeutil.Now() insists on returning UTC timestamps, for better or
worse. It was doing this by calling time.Now.UTC(). The rub is that the
UTC() call strips the monotonic clock reading from the timestamp.
Despite repeatedly trying ([1]), I've failed to find any reasonable
reason for that behavior. To work around it, this patch does unsafe
trickery to get a UTC timestamp without stripping the monos.

Stripping the monos has the following downsides:
1. We lose the benefits of the monotonic clock reading.
2. On OSX, only the monotonic clock seems to have nanosecond resolution. If
we strip it, we only get microsecond resolution. Besides generally sucking,
microsecond resolution is not enough to guarantee that consecutive
timeutil.Now() calls don't return the same instant. This trips up some of
our tests, which assume that they can measure any duration of time.
3. time.Since(t) does one less system calls when t has a monotonic reading,
making it twice as fast as otherwise:
https://cs.opensource.google/go/go/+/refs/tags/go1.17.2:src/time/time.go;l=878;drc=refs%2Ftags%2Fgo1.17.2

An alternative considered was setting the process' timezone to UTC in an
init(): time.Local = time.UTC. That has downsides: it makes cockroach
more unpleasant to link as a library, and setting the process timezone
makes the timestamps not roundtrip through marshalling/unmarshalling
(see [1]).

[1] https://groups.google.com/g/golang-nuts/c/dyPTdi6oem8
  • Loading branch information
andreimatei committed May 27, 2022
1 parent 1e328c8 commit f782e45
Show file tree
Hide file tree
Showing 8 changed files with 122 additions and 8 deletions.
4 changes: 2 additions & 2 deletions pkg/ccl/sqlproxyccl/denylist/file_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -121,12 +121,12 @@ denylist:
Denylist: []*DenyEntry{
{
DenyEntity{"63", ClusterType},
timeutil.Now(),
timeutil.NowNoMono(),
"over usage",
},
{
DenyEntity{"8.8.8.8", IPAddrType},
timeutil.Now().Add(1 * time.Hour),
timeutil.NowNoMono().Add(1 * time.Hour),
"malicious IP",
},
},
Expand Down
12 changes: 9 additions & 3 deletions pkg/ccl/sqlproxyccl/tenant/directory_cache_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -102,13 +102,19 @@ func TestWatchPods(t *testing.T) {
}
require.True(t, tds.AddPod(tenantID, runningPod))
pod := <-podWatcher
require.Equal(t, runningPod, pod)
requirePodsEqual := func(t *testing.T, pod1, pod2 *tenant.Pod) {
p1, p2 := *pod1, *pod2
p1.StateTimestamp = timeutil.StripMono(p1.StateTimestamp)
p2.StateTimestamp = timeutil.StripMono(p2.StateTimestamp)
require.Equal(t, p1, p2)
}
requirePodsEqual(t, runningPod, pod)

// Directory cache should have already been updated.
pods, err := dir.TryLookupTenantPods(ctx, tenantID)
require.NoError(t, err)
require.Len(t, pods, 1)
require.Equal(t, runningPod, pods[0])
requirePodsEqual(t, runningPod, pods[0])

// Drain the pod.
require.True(t, tds.DrainPod(tenantID, runningPod.Addr))
Expand Down Expand Up @@ -152,7 +158,7 @@ func TestWatchPods(t *testing.T) {
// Put the same pod back to running.
require.True(t, tds.AddPod(tenantID, runningPod))
pod = <-podWatcher
require.Equal(t, runningPod, pod)
requirePodsEqual(t, runningPod, pod)

// Directory cache should be updated with the RUNNING pod.
pods, err = dir.TryLookupTenantPods(ctx, tenantID)
Expand Down
2 changes: 1 addition & 1 deletion pkg/security/sessionrevival/token_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -29,7 +29,7 @@ func timestampProto(t *testing.T, ts time.Time) *pbtypes.Timestamp {
}

func TestValidatePayloadContents(t *testing.T) {
now := timeutil.Now().Add(-1 * time.Second)
now := timeutil.NowNoMono().Add(-1 * time.Second)
username := username.MakeSQLUsernameFromPreNormalizedString("testuser")
testCases := []struct {
description string
Expand Down
4 changes: 3 additions & 1 deletion pkg/server/admin_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -1671,7 +1671,9 @@ func TestAdminAPIJobs(t *testing.T) {
if e, a := expected, resIDs; !reflect.DeepEqual(e, a) {
t.Errorf("%d: expected job IDs %v, but got %v", i, e, a)
}
require.Equal(t, now.Add(-retentionTime), res.EarliestRetainedTime)
// We don't use require.Equal() because timestamps don't necessarily
// compare == due to only one of them having a monotonic clock reading.
require.True(t, now.Add(-retentionTime).Equal(res.EarliestRetainedTime))
}
})
}
Expand Down
2 changes: 1 addition & 1 deletion pkg/sql/protoreflect/utils_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -88,7 +88,7 @@ func TestMessageToJSONBRoundTrip(t *testing.T) {
TraceID: 123,
Tags: map[string]string{"one": "1", "two": "2", "three": "3"},
StructuredRecords: []tracingpb.StructuredRecord{{
Time: timeutil.Now(),
Time: timeutil.NowNoMono(),
Payload: makeAny(t, &descpb.ColumnDescriptor{Name: "bogus stats"})}},
},
},
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -142,6 +142,9 @@ func TestSQLStatsJsonEncoding(t *testing.T) {

err = DecodeStmtStatsMetadataJSON(actualMetadataJSON, &actualJSONUnmarshalled)
require.NoError(t, err)
// Strip the monononic part of timestamps, as it doesn't roundtrip. UTC()
// has that stripping side-effect.
input.Stats.LastExecTimestamp = input.Stats.LastExecTimestamp.UTC()

err = DecodeStmtStatsStatisticsJSON(actualStatisticsJSON, &actualJSONUnmarshalled.Stats)
require.NoError(t, err)
Expand Down Expand Up @@ -259,6 +262,9 @@ func TestSQLStatsJsonEncoding(t *testing.T) {
actualStatisticsJSON, _, _ = actualStatisticsJSON.RemovePath([]string{"statistics", "numRows"})
// Initialize the field again to remove the existing value.
expectedStatistics.Stats.NumRows = roachpb.NumericStat{}
// Strip the monononic part of timestamps, as it doesn't roundtrip. UTC()
// has that stripping side-effect.
expectedStatistics.Stats.LastExecTimestamp = expectedStatistics.Stats.LastExecTimestamp.UTC()

err = DecodeStmtStatsStatisticsJSON(actualStatisticsJSON, &actualJSONUnmarshalled.Stats)
require.NoError(t, err)
Expand Down
51 changes: 51 additions & 0 deletions pkg/util/timeutil/time.go
Original file line number Diff line number Diff line change
Expand Up @@ -13,16 +13,67 @@ package timeutil
import (
"strings"
"time"
"unsafe"
)

// LibPQTimePrefix is the prefix lib/pq prints time-type datatypes with.
const LibPQTimePrefix = "0000-01-01"

// Now returns the current UTC time.
//
// We've decided in times immemorial that always returning UTC is a good policy
// across the cluster so that all the timestamps print uniformly across
// different nodes, and also because we were afraid that timestamps leak into
// SQL Datums, and there the timestamp matters. Years later, it's not clear
// whether this was a good decision since it's forcing the nasty implementation
// below.
func Now() time.Time {
t := time.Now()
// HACK: instead of doing t = t.UTC(), we reach inside the
// struct and set the location manually. UTC() strips the monotonic clock reading
// from t, for no good reason: https://groups.google.com/g/golang-nuts/c/dyPTdi6oem8
// Stripping the monotonic part has bad consequences:
// 1. We lose the benefits of the monotonic clock reading.
// 2. On OSX, only the monotonic clock seems to have nanosecond resolution. If
// we strip it, we only get microsecond resolution. Besides generally sucking,
// microsecond resolution is not enough to guarantee that consecutive
// timeutil.Now() calls don't return the same instant. This trips up some of
// our tests, which assume that they can measure any duration of time.
// 3. time.Since(t) does one less system calls when t has a monotonic reading,
// making it twice as fast as otherwise:
// https://cs.opensource.google/go/go/+/refs/tags/go1.17.2:src/time/time.go;l=878;drc=refs%2Ftags%2Fgo1.17.2
x := (*timeLayout)(unsafe.Pointer(&t))
x.loc = nil // nil means UTC
return t
}

// NowNoMono is like Now(), but it strips down the monotonic part of the
// timestamp. This is useful for getting timestamps that rounds-trip through
// various channels that strip out the monotonic part - for example yaml
// marshaling.
func NowNoMono() time.Time {
// UTC has the side-effect of stripping the nanos.
return time.Now().UTC()
}

// StripMono returns a copy of t with its monotonic clock reading stripped. This
// is useful for getting a time.Time that compares == with another one that
// might not have the mono part. time.Time is meant to be compared with
// Time.Equal() (which ignores the mono), not with ==, but sometimes we have a
// time.Time in a bigger struct and we want to use require.Equal() or such.
func StripMono(t time.Time) time.Time {
// UTC() has the side-effect of stripping the mono part.
return t.UTC()
}

// timeLayout mimics time.Time, exposing all the fields. We do an unsafe cast of
// a time.Time to this in order to set the location.
type timeLayout struct {
wall uint64
ext int64
loc *time.Location
}

// Since returns the time elapsed since t.
// It is shorthand for Now().Sub(t), but more efficient.
func Since(t time.Time) time.Duration {
Expand Down
49 changes: 49 additions & 0 deletions pkg/util/timeutil/time_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -13,10 +13,12 @@ package timeutil
import (
"math"
"math/rand"
"reflect"
"testing"
"time"

"github.com/stretchr/testify/assert"
"github.com/stretchr/testify/require"
)

func TestUnixMicros(t *testing.T) {
Expand Down Expand Up @@ -70,3 +72,50 @@ func TestReplaceLibPQTimePrefix(t *testing.T) {
assert.Equal(t, "1970-02-02 11:00", ReplaceLibPQTimePrefix("1970-02-02 11:00"))
assert.Equal(t, "1970-01-01 11:00", ReplaceLibPQTimePrefix("0000-01-01 11:00"))
}

// Test that consecutive timeutil.Now() calls don't return the same instant. We
// have tests that rely on all durations being measurable.
func TestTimeIncreasing(t *testing.T) {
a := Now()
b := Now()
require.NotEqual(t, a, b)
require.NotZero(t, b.Sub(a))
}

func init() {
// We run our tests with the env var TZ="", which means UTC, in order to make
// the timezone predictable. This is not how we run in production, though. For
// this package, let's brutally pretend we're somewhere else by changing the
// locale, in order to make the testing more general. In particular, without
// this, TestTimeIsUTC(), which tests that the timeutil library converts
// timestamps to UTC, would pass regardless of our code because of the testing
// environment. We need to do this in a package init because otherwise the
// update races with the runtime reading it.
loc, err := LoadLocation("Africa/Cairo")
if err != nil {
panic(err)
}
time.Local = loc
}

// Test that Now() returns times in UTC. We've made a decision that this is a
// good policy across the cluster so that all the timestamps print uniformly
// across different nodes, and also because we were afraid that timestamps leak
// into SQL Datums, and there the timestamp matters.
func TestTimeIsUTC(t *testing.T) {
require.Equal(t, time.UTC, Now().Location())
require.Equal(t, time.UTC, Unix(1, 1).Location())
}

// Test that the unsafe cast we do in timeutil.Now() to set the time to UTC is
// sane: check that our hardcoded struct layout corresponds to stdlib.
func TestTimeLayout(t *testing.T) {
stdlib := reflect.TypeOf((*time.Time)(nil)).Elem()
ours := reflect.TypeOf((*timeLayout)(nil)).Elem()
require.Equal(t, stdlib.NumField(), ours.NumField())
for i := 0; i < stdlib.NumField(); i++ {
stdlibF, ourF := stdlib.Field(i), ours.Field(i)
require.Equal(t, stdlibF.Type, ourF.Type)
require.Equal(t, stdlibF.Offset, ourF.Offset)
}
}

0 comments on commit f782e45

Please sign in to comment.