Skip to content

Commit

Permalink
Merge #42928 #43012
Browse files Browse the repository at this point in the history
42928: builtins: add CURRENT_TIME functionality r=otan a=otan

Resolves #31708
Refs #26097

This PR adds CURRENT_TIME as a builtin.

Release note (sql change): This PR adds the CURRENT_TIME builtin, which
can be used with precision, e.g. `SELECT CURRENT_TIME, CURRENT_TIME(3)`.

43012: sql: fix current_timestamp behaviour with time zone set r=otan a=otan

With time zones set, current_timestamp needs to localise to the timezone
set in the context for time options, e.g. for `TIME` with `UTC+3` at
`UTC midnight`, `current_timestamp()` should return `3am`. This was
previously not handled correctly by Timestamp, and is rectified in this
PR.

Release note (bug fix): Previously, current_timestamp would not
correctly account for `SET TIME ZONE` in the background when storing
results, storing the timestamp as `UTC` instead. This is fixed in this
PR.

Co-authored-by: Oliver Tan <otan@cockroachlabs.com>
  • Loading branch information
craig[bot] and otan committed Dec 6, 2019
3 parents 28f216e + 7d30f95 + b78a228 commit 1e7bb50
Show file tree
Hide file tree
Showing 10 changed files with 215 additions and 9 deletions.
3 changes: 3 additions & 0 deletions docs/generated/sql/bnf/stmt_block.bnf
Original file line number Diff line number Diff line change
Expand Up @@ -2130,6 +2130,7 @@ func_expr_common_subexpr ::=
| 'CURRENT_SCHEMA'
| 'CURRENT_CATALOG'
| 'CURRENT_TIMESTAMP'
| 'CURRENT_TIME'
| 'CURRENT_USER'
| 'CURRENT_ROLE'
| 'SESSION_USER'
Expand Down Expand Up @@ -2310,6 +2311,8 @@ special_function ::=
| 'CURRENT_SCHEMA' '(' ')'
| 'CURRENT_TIMESTAMP' '(' ')'
| 'CURRENT_TIMESTAMP' '(' a_expr ')'
| 'CURRENT_TIME' '(' ')'
| 'CURRENT_TIME' '(' a_expr ')'
| 'CURRENT_USER' '(' ')'
| 'EXTRACT' '(' extract_list ')'
| 'EXTRACT_DURATION' '(' extract_list ')'
Expand Down
15 changes: 15 additions & 0 deletions docs/generated/sql/functions.md
Original file line number Diff line number Diff line change
Expand Up @@ -1056,6 +1056,21 @@ SELECT * FROM crdb_internal.check_consistency(true, ‘\x02’, ‘\x04’)</p>
</span></td></tr></tbody>
</table>

### TIMETZ functions

<table>
<thead><tr><th>Function &rarr; Returns</th><th>Description</th></tr></thead>
<tbody>
<tr><td><a name="current_time"></a><code>current_time() &rarr; <a href="time.html">time</a></code></td><td><span class="funcdesc"><p>Returns the current transaction’s time with no time zone.</p>
</span></td></tr>
<tr><td><a name="current_time"></a><code>current_time() &rarr; timetz</code></td><td><span class="funcdesc"><p>Returns the current transaction’s time with time zone.</p>
</span></td></tr>
<tr><td><a name="current_time"></a><code>current_time(precision: <a href="int.html">int</a>) &rarr; <a href="time.html">time</a></code></td><td><span class="funcdesc"><p>Returns the current transaction’s time with no time zone.</p>
</span></td></tr>
<tr><td><a name="current_time"></a><code>current_time(precision: <a href="int.html">int</a>) &rarr; timetz</code></td><td><span class="funcdesc"><p>Returns the current transaction’s time with time zone.</p>
</span></td></tr></tbody>
</table>

### TUPLE functions

<table>
Expand Down
1 change: 0 additions & 1 deletion pkg/cmd/roachtest/hibernate_blacklist.go
Original file line number Diff line number Diff line change
Expand Up @@ -23,7 +23,6 @@ var hibernateBlacklists = blacklistsForVersion{
// After a failed run, an updated version of this blacklist should be available
// in the test log.
var hibernateBlackList20_1 = blacklist{
"org.hibernate.jpa.test.criteria.QueryBuilderTest.testDateTimeFunctions": "31708",
"org.hibernate.jpa.test.criteria.basic.PredicateTest.testQuotientConversion": "26732",
"org.hibernate.jpa.test.lock.LockTest.testFindWithTimeoutHint": "40476",
"org.hibernate.jpa.test.lock.LockTest.testUpdateWithPessimisticReadLockSkipLocked": "40476",
Expand Down
48 changes: 48 additions & 0 deletions pkg/sql/logictest/testdata/logic_test/time
Original file line number Diff line number Diff line change
Expand Up @@ -419,3 +419,51 @@ query T
SELECT '2001-01-01 01:24:00'::time
----
0000-01-01 01:24:00 +0000 UTC

subtest current_time_tests

statement ok
CREATE TABLE current_time_test (
id INTEGER PRIMARY KEY,
a TIME(3) DEFAULT CURRENT_TIME,
b TIME DEFAULT CURRENT_TIME
)

statement ok
INSERT INTO current_time_test (id) VALUES (1)

statement ok
INSERT INTO current_time_test (id, a, b) VALUES
(2, current_time, current_time),
(3, current_time, current_time(3))

query I
SELECT id FROM current_time_test WHERE
('1970-01-01 ' || b::string)::timestamp -
('1970-01-01 ' || a::string)::timestamp
> '1ms'::interval ORDER BY id ASC
----

# test that current_time is correct in different timezones.

statement ok
set time zone +3

statement ok
create table current_time_tzset_test (id integer, a time, b time)

statement ok
insert into current_time_tzset_test (id, a) values (1, current_time)

statement ok
set time zone 0

statement ok
update current_time_tzset_test set b = current_time

# a was written at an interval 3 hours ahead, and should persist that way.
# make sure they're roughly 3 hours apart.
query I
select id from current_time_tzset_test WHERE interval '2h59m' < a - b and a - b < interval '3h'
----
1
22 changes: 21 additions & 1 deletion pkg/sql/logictest/testdata/logic_test/timestamp
Original file line number Diff line number Diff line change
Expand Up @@ -213,7 +213,6 @@ SELECT extract(timezone FROM '2001-01-01 13:00:00+01:15'::timestamptz)
----
10800


subtest regression_41776

statement ok
Expand All @@ -223,3 +222,24 @@ query T
SELECT '2001-01-01 00:00:00'::TIMESTAMP::TIMESTAMPTZ
----
2001-01-01 00:00:00 +0100 +0100

# test that current_timestamp is correct in different timezones.
subtest current_timestamp_correct_in_timezone

statement ok
set time zone +3

statement ok
create table current_timestamp_test (a timestamp, b timestamptz)

statement ok
insert into current_timestamp_test values (current_timestamp, current_timestamp)

statement ok
set time zone 0

# a was written at an interval 3 hours ahead, and should persist that way.
# b will remember the timezone, so should be "constant" for comparison's sake.
query TT
select * from current_timestamp_test WHERE a - interval '3h' <> b
----
45 changes: 45 additions & 0 deletions pkg/sql/logictest/testdata/logic_test/timetz
Original file line number Diff line number Diff line change
Expand Up @@ -94,6 +94,51 @@ SELECT '2001-01-01 11:00+04:00'::timestamptz::timetz
statement ok
SET TIME ZONE UTC

subtest current_time_tests

# current_time cannot be subtracted, but can as a timestamp.
query B
select
('1970-01-01 ' || current_time(3)::string)::timestamp -
('1970-01-01 ' || current_time::string)::timestamp
<= '1ms'::interval
----
true

statement ok
CREATE TABLE current_time_test (
id INTEGER PRIMARY KEY,
a TIMETZ(3) DEFAULT CURRENT_TIME,
b TIMETZ DEFAULT CURRENT_TIME
)

statement ok
INSERT INTO current_time_test (id) VALUES (1)

statement ok
INSERT INTO current_time_test (id, a, b) VALUES
(2, current_time, current_time),
(3, current_time, current_time(3))

query I
SELECT id FROM current_time_test WHERE
('1970-01-01 ' || b::string)::timestamp -
('1970-01-01 ' || a::string)::timestamp
> '1ms'::interval ORDER BY id ASC
----

# switching timezones should make current_time() change timezones too.
statement ok
set time zone +4

query B
select current_time() + current_timestamp()::date = current_timestamp()
----
true

statement ok
set time zone UTC

subtest precision_tests

query error precision 7 out of range
Expand Down
6 changes: 4 additions & 2 deletions pkg/sql/parser/parse_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -1578,6 +1578,10 @@ func TestParse2(t *testing.T) {
`SELECT current_timestamp()`},
{`SELECT current_timestamp(6)`,
`SELECT current_timestamp(6)`},
{`SELECT CURRENT_TIME`,
`SELECT current_time()`},
{`SELECT current_time(6)`,
`SELECT current_time(6)`},
{`SELECT CURRENT_DATE`,
`SELECT current_date()`},
{`SELECT POSITION(a IN b)`,
Expand Down Expand Up @@ -3145,8 +3149,6 @@ func TestUnimplementedSyntax(t *testing.T) {
{`SELECT GROUPING (a,b,c)`, 0, `d_expr grouping`},
{`SELECT a(VARIADIC b)`, 0, `variadic`},
{`SELECT a(b, c, VARIADIC b)`, 0, `variadic`},
{`SELECT CURRENT_TIME`, 26097, `current_time`},
{`SELECT CURRENT_TIME()`, 26097, `current_time`},
{`SELECT TREAT (a AS INT8)`, 0, `treat`},
{`SELECT a(b) WITHIN GROUP (ORDER BY c)`, 0, `within group`},

Expand Down
8 changes: 6 additions & 2 deletions pkg/sql/parser/sql.y
Original file line number Diff line number Diff line change
Expand Up @@ -8202,7 +8202,7 @@ func_expr_common_subexpr:
}
| CURRENT_TIME
{
return unimplementedWithIssueDetail(sqllex, 26097, "current_time")
$$.val = &tree.FuncExpr{Func: tree.WrapFunction($1)}
}
| CURRENT_USER
{
Expand Down Expand Up @@ -8286,7 +8286,11 @@ special_function:
| CURRENT_TIMESTAMP '(' error { return helpWithFunctionByName(sqllex, $1) }
| CURRENT_TIME '(' ')'
{
return unimplementedWithIssueDetail(sqllex, 26097, "current_time")
$$.val = &tree.FuncExpr{Func: tree.WrapFunction($1)}
}
| CURRENT_TIME '(' a_expr ')'
{
$$.val = &tree.FuncExpr{Func: tree.WrapFunction($1), Exprs: tree.Exprs{$3.expr()}}
}
| CURRENT_TIME '(' error { return helpWithFunctionByName(sqllex, $1) }
| CURRENT_USER '(' ')'
Expand Down
47 changes: 46 additions & 1 deletion pkg/sql/sem/builtins/builtins.go
Original file line number Diff line number Diff line change
Expand Up @@ -1674,7 +1674,52 @@ CockroachDB supports the following flags:
},
),

"now": txnTSImpl,
"now": txnTSImpl,
"current_time": makeBuiltin(
tree.FunctionProperties{Impure: true},
tree.Overload{
Types: tree.ArgTypes{},
ReturnType: tree.FixedReturnType(types.TimeTZ),
PreferredOverload: true,
Fn: func(ctx *tree.EvalContext, args tree.Datums) (tree.Datum, error) {
return ctx.GetTxnTime(time.Microsecond), nil
},
Info: "Returns the current transaction's time with time zone.",
},
tree.Overload{
Types: tree.ArgTypes{},
ReturnType: tree.FixedReturnType(types.Time),
Fn: func(ctx *tree.EvalContext, args tree.Datums) (tree.Datum, error) {
return ctx.GetTxnTimeNoZone(time.Microsecond), nil
},
Info: "Returns the current transaction's time with no time zone.",
},
tree.Overload{
Types: tree.ArgTypes{{"precision", types.Int}},
ReturnType: tree.FixedReturnType(types.TimeTZ),
PreferredOverload: true,
Fn: func(ctx *tree.EvalContext, args tree.Datums) (tree.Datum, error) {
prec := int32(tree.MustBeDInt(args[0]))
if prec < 0 || prec > 6 {
return nil, pgerror.Newf(pgcode.NumericValueOutOfRange, "precision %d out of range", prec)
}
return ctx.GetTxnTime(tree.TimeFamilyPrecisionToRoundDuration(prec)), nil
},
Info: "Returns the current transaction's time with time zone.",
},
tree.Overload{
Types: tree.ArgTypes{{"precision", types.Int}},
ReturnType: tree.FixedReturnType(types.Time),
Fn: func(ctx *tree.EvalContext, args tree.Datums) (tree.Datum, error) {
prec := int32(tree.MustBeDInt(args[0]))
if prec < 0 || prec > 6 {
return nil, pgerror.Newf(pgcode.NumericValueOutOfRange, "precision %d out of range", prec)
}
return ctx.GetTxnTimeNoZone(tree.TimeFamilyPrecisionToRoundDuration(prec)), nil
},
Info: "Returns the current transaction's time with no time zone.",
},
),
"current_timestamp": txnTSWithPrecisionImpl,
"transaction_timestamp": txnTSImpl,

Expand Down
29 changes: 27 additions & 2 deletions pkg/sql/sem/tree/eval.go
Original file line number Diff line number Diff line change
Expand Up @@ -2932,7 +2932,7 @@ func (ctx *EvalContext) GetTxnTimestamp(precision time.Duration) *DTimestampTZ {
if !ctx.PrepareOnly && ctx.TxnTimestamp.IsZero() {
panic(errors.AssertionFailedf("zero transaction timestamp in EvalContext"))
}
return MakeDTimestampTZ(ctx.TxnTimestamp, precision)
return MakeDTimestampTZ(ctx.GetRelativeParseTime(), precision)
}

// GetTxnTimestampNoZone retrieves the current transaction timestamp as per
Expand All @@ -2943,7 +2943,32 @@ func (ctx *EvalContext) GetTxnTimestampNoZone(precision time.Duration) *DTimesta
if !ctx.PrepareOnly && ctx.TxnTimestamp.IsZero() {
panic(errors.AssertionFailedf("zero transaction timestamp in EvalContext"))
}
return MakeDTimestamp(ctx.TxnTimestamp, precision)
// Move the time to UTC, but keeping the location's time.
t := ctx.GetRelativeParseTime()
_, offsetSecs := t.Zone()
return MakeDTimestamp(t.Add(time.Second*time.Duration(offsetSecs)).In(time.UTC), precision)
}

// GetTxnTime retrieves the current transaction time as per
// the evaluation context.
func (ctx *EvalContext) GetTxnTime(precision time.Duration) *DTimeTZ {
// TODO(knz): a zero timestamp should never be read, even during
// Prepare. This will need to be addressed.
if !ctx.PrepareOnly && ctx.TxnTimestamp.IsZero() {
panic(errors.AssertionFailedf("zero transaction timestamp in EvalContext"))
}
return NewDTimeTZFromTime(ctx.GetRelativeParseTime().Round(precision))
}

// GetTxnTimeNoZone retrieves the current transaction time as per
// the evaluation context.
func (ctx *EvalContext) GetTxnTimeNoZone(precision time.Duration) *DTime {
// TODO(knz): a zero timestamp should never be read, even during
// Prepare. This will need to be addressed.
if !ctx.PrepareOnly && ctx.TxnTimestamp.IsZero() {
panic(errors.AssertionFailedf("zero transaction timestamp in EvalContext"))
}
return MakeDTime(timeofday.FromTime(ctx.GetRelativeParseTime().Round(precision)))
}

// SetTxnTimestamp sets the corresponding timestamp in the EvalContext.
Expand Down

0 comments on commit 1e7bb50

Please sign in to comment.