Skip to content

Commit

Permalink
Merge #31758
Browse files Browse the repository at this point in the history
31758: sql: Generalize date/time parsing r=bobvawter a=bobvawter

sql: Generalize date/time parsing

The current date/time parsing code relies on `time.ParseInLocation()`. It does
not support all of the various date/time formats accepted by PostgreSQL and
also requires multiple invocation to try the various date/time formats that we
do accept.

This change updates the date/time parsing code with a new implementation that
does not delegate to `time.ParseInLocation()` and is able to parse all
supported formats in a single pass.

In order to support parsing named timezones like `America/New_York`, we
delegate to `time.LoadLocation()` as we did previously.  `LoadLocation()` is
rather expensive, since it looks for tzinfo files on disk every time it is
invoked. A per-node, in-memory cache has been added to amortize this overhead.
Per #31978, the tzinfo used on each node could already be inconsistent,
depending on the tzinfo files present in the underlying OS.

The following table compares the new `ParseTimestamp()` function to calling
`ParseInLocation()`.  While it is true that `ParseInLocation()` is generally
faster for any given pattern, the current parsing code must call it repeatedly,
trying each supported date format until one succeeds. The test with the named
timezone also shows the significant overhead of calling `LoadLocation()`.

```
2003-06-12/ParseTimestamp-8             10000000               122 ns/op          81.53 MB/s
2003-06-12/ParseInLocation-8            30000000                35.6 ns/op       281.29 MB/s
2003-06-12_01:02:03/ParseTimestamp-8            10000000               163 ns/op         116.45 MB/s
2003-06-12_01:02:03/ParseInLocation-8           30000000                54.4 ns/op       349.16 MB/s
2003-06-12_04:05:06.789-04:00/ParseTimestamp-8          10000000               238 ns/op         121.69 MB/s
2003-06-12_04:05:06.789-04:00/ParseInLocation-8         10000000               161 ns/op         180.05 MB/s
2000-01-01T02:02:02.567+09:30/ParseTimestamp-8           5000000               233 ns/op         124.01 MB/s
2000-01-01T02:02:02.567+09:30/ParseInLocation-8         10000000               158 ns/op         182.41 MB/s
2003-06-12_04:05:06.789_America/New_York/ParseTimestamp-8                3000000               475 ns/op          84.06 MB/s
2003-06-12_04:05:06.789_America/New_York/ParseInLocation-8                200000              7313 ns/op           3.15 MB/s
```

The tests in `parsing_test.go` have an optional mode to cross-check the test
data aginst a PostgreSQL server.  This is useful for developing, but is not
part of the automated build.

Parsing of BC dates is supported, #28099 could then be completed by changing
the date-formatting code to print a BC date.

This change would allow #30697 (incomplete handling of datestyle) to be
re-evaluated, since the parser does allow configuration of YMD, DMY, or MDY
input styles.

Resolves #27500
Resolves #27501
Resolves #31954

Release note (sql change): A wider variety of date, time, and timestamp formats
are now accepted by the SQL frontend.

Release note (bug fix): Prepared statements that bind temporal values now
respect the session's timezone setting. Previously, bound temporal values were
always interpreted as though the session time zone were UTC.

Release note (backward-incompatible change): Timezone abbreviations, such as
EST, are no longer allowed when parsing or converting to a date/time type.
Previously, an abbreviation would be accepted if it were an alias for the
session's timezone.


Co-authored-by: Bob Vawter <bob@cockroachlabs.com>
  • Loading branch information
craig[bot] and bobvawter committed Oct 30, 2018
2 parents d7a53a3 + 06b03bb commit c8e87ae
Show file tree
Hide file tree
Showing 28 changed files with 2,999 additions and 184 deletions.
2 changes: 1 addition & 1 deletion pkg/ccl/importccl/read_import_mysql_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -275,7 +275,7 @@ func TestMysqlValueToDatum(t *testing.T) {
defer leaktest.AfterTest(t)()

date := func(s string) tree.Datum {
d, err := tree.ParseDDate(s, nil)
d, err := tree.ParseDDate(nil, s)
if err != nil {
t.Fatal(err)
}
Expand Down
5 changes: 4 additions & 1 deletion pkg/sql/conn_executor_prepare.go
Original file line number Diff line number Diff line change
Expand Up @@ -326,14 +326,17 @@ func (ex *connExecutor) execBind(
"expected %d arguments, got %d", numQArgs, len(bindCmd.Args)))
}

ptCtx := tree.NewParseTimeContext(ex.sessionData.DurationAdditionMode,
ex.state.sqlTimestamp.In(ex.sessionData.DataConversion.Location))

for i, arg := range bindCmd.Args {
k := strconv.Itoa(i + 1)
t := ps.InTypes[i]
if arg == nil {
// nil indicates a NULL argument value.
qargs[k] = tree.DNull
} else {
d, err := pgwirebase.DecodeOidDatum(t, qArgFormatCodes[i], arg)
d, err := pgwirebase.DecodeOidDatum(ptCtx, t, qArgFormatCodes[i], arg)
if err != nil {
if _, ok := err.(*pgerror.Error); ok {
return retErr(err)
Expand Down
7 changes: 3 additions & 4 deletions pkg/sql/logictest/testdata/logic_test/datetime
Original file line number Diff line number Diff line change
Expand Up @@ -646,14 +646,13 @@ SELECT '2015-08-25 05:45:45.53453'::timestamp
statement ok
SET TIME ZONE 'Europe/Rome'

query T
query error pq: unimplemented: timestamp abbreviations not supported
SELECT '2015-08-25 05:45:45.53453 CET'::timestamptz WHERE false
----

statement ok
SET TIME ZONE +1

query error unknown zone "CET"
query error pq: unimplemented: timestamp abbreviations not supported
SELECT '2015-08-25 05:45:45.53453 CET'::timestamptz WHERE false

query T
Expand Down Expand Up @@ -690,7 +689,7 @@ SELECT '2015-08-25 05:45:45-01:00'::timestamptz
statement ok
SET TIMEZONE = +2

query error unknown zone "CET"
query error pq: unimplemented: timestamp abbreviations not supported
SELECT '2015-08-25 05:45:45.53453 CET'::timestamptz WHERE false

query T
Expand Down
8 changes: 4 additions & 4 deletions pkg/sql/logictest/testdata/logic_test/typing
Original file line number Diff line number Diff line change
Expand Up @@ -136,11 +136,11 @@ SELECT ts FROM untyped WHERE ts != '2015-09-18 00:00:00'

# Regression tests for #15050

statement error error type checking constant value: could not parse "2017-04-18 18:00" as type timestamp
CREATE TABLE t15050a (c DECIMAL DEFAULT CASE WHEN now() < '2017-04-18 18:00' THEN 2 ELSE 2 END);
statement error pq: error type checking constant value: could not parse "Not Timestamp" as type timestamp
CREATE TABLE t15050a (c DECIMAL DEFAULT CASE WHEN now() < 'Not Timestamp' THEN 2 ELSE 2 END);

statement error error type checking constant value: could not parse "2017-04-18 18:00" as type timestamp
CREATE TABLE t15050b (c DECIMAL DEFAULT IF(now() < '2017-04-18 18:00', 2, 2));
statement error pq: error type checking constant value: could not parse "Not Timestamp" as type timestamp
CREATE TABLE t15050b (c DECIMAL DEFAULT IF(now() < 'Not Timestamp', 2, 2));

# Regression tests for #15632

Expand Down
12 changes: 6 additions & 6 deletions pkg/sql/pgwire/binary_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -84,7 +84,7 @@ func testBinaryDatumType(t *testing.T, typ string, datumConstructor func(val str
}

datum, err := pgwirebase.DecodeOidDatum(
oid, pgwirebase.FormatBinary, got[4:],
evalCtx, oid, pgwirebase.FormatBinary, got[4:],
)
if err != nil {
t.Fatalf("unable to decode %v: %s", got[4:], err)
Expand Down Expand Up @@ -123,7 +123,7 @@ func TestBinaryTimestamp(t *testing.T) {
func TestBinaryTimestampTZ(t *testing.T) {
defer leaktest.AfterTest(t)()
testBinaryDatumType(t, "timestamptz", func(val string) tree.Datum {
tstz, err := tree.ParseDTimestampTZ(val, time.UTC, time.Microsecond)
tstz, err := tree.ParseDTimestampTZ(nil, val, time.Microsecond)
if err != nil {
t.Fatal(err)
}
Expand All @@ -145,7 +145,7 @@ func TestBinaryInterval(t *testing.T) {
func TestBinaryDate(t *testing.T) {
defer leaktest.AfterTest(t)()
testBinaryDatumType(t, "date", func(val string) tree.Datum {
d, err := tree.ParseDDate(val, time.UTC)
d, err := tree.ParseDDate(nil, val)
if err != nil {
t.Fatal(err)
}
Expand All @@ -156,7 +156,7 @@ func TestBinaryDate(t *testing.T) {
func TestBinaryTime(t *testing.T) {
defer leaktest.AfterTest(t)()
testBinaryDatumType(t, "time", func(val string) tree.Datum {
d, err := tree.ParseDTime(val)
d, err := tree.ParseDTime(nil, val)
if err != nil {
t.Fatal(err)
}
Expand Down Expand Up @@ -232,7 +232,7 @@ func TestBinaryIntArray(t *testing.T) {

b := buf.wrapped.Bytes()

got, err := pgwirebase.DecodeOidDatum(oid.T__int8, pgwirebase.FormatBinary, b[4:])
got, err := pgwirebase.DecodeOidDatum(nil, oid.T__int8, pgwirebase.FormatBinary, b[4:])
if err != nil {
t.Fatal(err)
}
Expand Down Expand Up @@ -304,7 +304,7 @@ func TestRandomBinaryDecimal(t *testing.T) {
if got := buf.wrapped.Bytes(); !bytes.Equal(got, test.Expect) {
t.Errorf("%q:\n\t%v found,\n\t%v expected", test.In, got, test.Expect)
} else if datum, err := pgwirebase.DecodeOidDatum(
oid.T_numeric, pgwirebase.FormatBinary, got[4:],
nil, oid.T_numeric, pgwirebase.FormatBinary, got[4:],
); err != nil {
t.Errorf("%q: unable to decode %v: %s", test.In, got[4:], err)
} else if dec.Compare(evalCtx, datum) != 0 {
Expand Down
9 changes: 9 additions & 0 deletions pkg/sql/pgwire/pgerror/errors.go
Original file line number Diff line number Diff line change
Expand Up @@ -195,6 +195,15 @@ func UnimplementedWithIssueError(issue int, msg string) error {
return err.SetHintf("See: https://github.com/cockroachdb/cockroach/issues/%d", issue)
}

// UnimplementedWithIssueDetailError constructs an error with the given message
// and a link to the passed issue. Recorded as "#<issue>.detail" in tracking.
// This is useful when we need an extra axis of information to drill down into.
func UnimplementedWithIssueDetailError(issue int, detail, msg string) error {
err := NewErrorWithDepthf(1, CodeFeatureNotSupportedError, "unimplemented: %s", msg)
err.InternalCommand = fmt.Sprintf("#%d.%s", issue, detail)
return err.SetHintf("See: https://github.com/cockroachdb/cockroach/issues/%d", issue)
}

const unimplementedErrorHint = `This feature is not yet implemented in CockroachDB.
Please check https://github.com/cockroachdb/cockroach/issues to check
Expand Down
39 changes: 39 additions & 0 deletions pkg/sql/pgwire/pgwire_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -1144,6 +1144,45 @@ func (p preparedExecTest) RowsAffectedErr(err string) preparedExecTest {
return p
}

// Verify that bound dates are evaluated using session timezone.
func TestPGPrepareDate(t *testing.T) {
defer leaktest.AfterTest(t)()
s, db, _ := serverutils.StartServer(t, base.TestServerArgs{})
defer s.Stopper().Stop(context.TODO())

if _, err := db.Exec("CREATE TABLE test (t TIMESTAMPTZ)"); err != nil {
t.Fatal(err)
}

if _, err := db.Exec("SET TIME ZONE +08"); err != nil {
t.Fatal(err)
}

stmt, err := db.Prepare("INSERT INTO test VALUES ($1)")
if err != nil {
t.Fatal(err)
}

if _, err := stmt.Exec("2018-01-01 12:34:56"); err != nil {
t.Fatal(err)
}

// Reset to UTC for the query.
if _, err := db.Exec("SET TIME ZONE UTC"); err != nil {
t.Fatal(err)
}

var ts time.Time
if err := db.QueryRow("SELECT t FROM test").Scan(&ts); err != nil {
t.Fatal(err)
}

exp := time.Date(2018, 1, 1, 4, 34, 56, 0, time.UTC)
if !exp.Equal(ts) {
t.Fatalf("expected %s, got %s", exp, ts)
}
}

func TestPGPreparedExec(t *testing.T) {
defer leaktest.AfterTest(t)()
var baseTest preparedExecTest
Expand Down
30 changes: 14 additions & 16 deletions pkg/sql/pgwire/pgwirebase/encoding.go
Original file line number Diff line number Diff line change
Expand Up @@ -190,8 +190,11 @@ func NewProtocolViolationErrorf(format string, args ...interface{}) error {
}

// DecodeOidDatum decodes bytes with specified Oid and format code into
// a datum.
func DecodeOidDatum(id oid.Oid, code FormatCode, b []byte) (tree.Datum, error) {
// a datum. If the ParseTimeContext is nil, reasonable defaults
// will be applied.
func DecodeOidDatum(
ctx tree.ParseTimeContext, id oid.Oid, code FormatCode, b []byte,
) (tree.Datum, error) {
switch code {
case FormatText:
switch id {
Expand Down Expand Up @@ -238,30 +241,25 @@ func DecodeOidDatum(id oid.Oid, code FormatCode, b []byte) (tree.Datum, error) {
}
return tree.NewDBytes(tree.DBytes(res)), nil
case oid.T_timestamp:
d, err := tree.ParseDTimestamp(nil, string(b), time.Microsecond)
d, err := tree.ParseDTimestamp(ctx, string(b), time.Microsecond)
if err != nil {
return nil, errors.Errorf("could not parse string %q as timestamp", b)
}
return d, nil
case oid.T_timestamptz:
d, err := tree.ParseDTimestampTZ(string(b), time.UTC, time.Microsecond)
d, err := tree.ParseDTimestampTZ(ctx, string(b), time.Microsecond)
if err != nil {
return nil, errors.Errorf("could not parse string %q as timestamptz", b)
}
return d, nil
case oid.T_date:
ts, err := tree.ParseDTimestamp(nil, string(b), time.Microsecond)
d, err := tree.ParseDDate(ctx, string(b))
if err != nil {
res, err := tree.ParseDDate(string(b), time.UTC)
if err != nil {
return nil, errors.Errorf("could not parse string %q as date", b)
}
return res, nil
return nil, errors.Errorf("could not parse string %q as date", b)
}
daysSinceEpoch := ts.Unix() / secondsInDay
return tree.NewDDate(tree.DDate(daysSinceEpoch)), nil
return d, nil
case oid.T_time:
d, err := tree.ParseDTime(string(b))
d, err := tree.ParseDTime(nil, string(b))
if err != nil {
return nil, errors.Errorf("could not parse string %q as time", b)
}
Expand Down Expand Up @@ -513,7 +511,7 @@ func DecodeOidDatum(id oid.Oid, code FormatCode, b []byte) (tree.Datum, error) {
return tree.ParseDJSON(string(b))
default:
if _, ok := types.ArrayOids[id]; ok {
return decodeBinaryArray(b, code)
return decodeBinaryArray(ctx, b, code)
}
}
default:
Expand Down Expand Up @@ -618,7 +616,7 @@ func pgBinaryToIPAddr(b []byte) (ipaddr.IPAddr, error) {
}, nil
}

func decodeBinaryArray(b []byte, code FormatCode) (tree.Datum, error) {
func decodeBinaryArray(ctx tree.ParseTimeContext, b []byte, code FormatCode) (tree.Datum, error) {
hdr := struct {
Ndims int32
// Nullflag
Expand Down Expand Up @@ -648,7 +646,7 @@ func decodeBinaryArray(b []byte, code FormatCode) (tree.Datum, error) {
return nil, err
}
buf := r.Next(int(vlen))
elem, err := DecodeOidDatum(elemOid, code, buf)
elem, err := DecodeOidDatum(ctx, elemOid, code, buf)
if err != nil {
return nil, err
}
Expand Down
10 changes: 5 additions & 5 deletions pkg/sql/pgwire/types_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -136,7 +136,7 @@ func TestIntArrayRoundTrip(t *testing.T) {

b := buf.wrapped.Bytes()

got, err := pgwirebase.DecodeOidDatum(oid.T__int8, pgwirebase.FormatText, b[4:])
got, err := pgwirebase.DecodeOidDatum(nil, oid.T__int8, pgwirebase.FormatText, b[4:])
if err != nil {
t.Fatal(err)
}
Expand Down Expand Up @@ -213,7 +213,7 @@ func TestByteArrayRoundTrip(t *testing.T) {
b := buf.wrapped.Bytes()
t.Logf("encoded: %v (%q)", b, b)

got, err := pgwirebase.DecodeOidDatum(oid.T_bytea, pgwirebase.FormatText, b[4:])
got, err := pgwirebase.DecodeOidDatum(nil, oid.T_bytea, pgwirebase.FormatText, b[4:])
if err != nil {
t.Fatal(err)
}
Expand Down Expand Up @@ -321,7 +321,7 @@ func benchmarkWriteString(b *testing.B, format pgwirebase.FormatCode) {
}

func benchmarkWriteDate(b *testing.B, format pgwirebase.FormatCode) {
d, err := tree.ParseDDate("2010-09-28", time.UTC)
d, err := tree.ParseDDate(nil, "2010-09-28")
if err != nil {
b.Fatal(err)
}
Expand All @@ -337,7 +337,7 @@ func benchmarkWriteTimestamp(b *testing.B, format pgwirebase.FormatCode) {
}

func benchmarkWriteTimestampTZ(b *testing.B, format pgwirebase.FormatCode) {
tstz, err := tree.ParseDTimestampTZ("2010-09-28 12:00:00.1", time.UTC, time.Microsecond)
tstz, err := tree.ParseDTimestampTZ(nil, "2010-09-28 12:00:00.1", time.Microsecond)
if err != nil {
b.Fatal(err)
}
Expand Down Expand Up @@ -470,7 +470,7 @@ func BenchmarkDecodeBinaryDecimal(b *testing.B) {
b.ResetTimer()
for i := 0; i < b.N; i++ {
b.StartTimer()
got, err := pgwirebase.DecodeOidDatum(oid.T_numeric, pgwirebase.FormatBinary, bytes)
got, err := pgwirebase.DecodeOidDatum(nil, oid.T_numeric, pgwirebase.FormatBinary, bytes)
b.StopTimer()
evalCtx := tree.NewTestingEvalContext(cluster.MakeTestingClusterSettings())
defer evalCtx.Stop(context.Background())
Expand Down
16 changes: 12 additions & 4 deletions pkg/sql/sem/tree/constant_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -199,7 +199,14 @@ func mustParseDBool(t *testing.T, s string) tree.Datum {
return d
}
func mustParseDDate(t *testing.T, s string) tree.Datum {
d, err := tree.ParseDDate(s, time.UTC)
d, err := tree.ParseDDate(nil, s)
if err != nil {
t.Fatal(err)
}
return d
}
func mustParseDTime(t *testing.T, s string) tree.Datum {
d, err := tree.ParseDTime(nil, s)
if err != nil {
t.Fatal(err)
}
Expand All @@ -213,7 +220,7 @@ func mustParseDTimestamp(t *testing.T, s string) tree.Datum {
return d
}
func mustParseDTimestampTZ(t *testing.T, s string) tree.Datum {
d, err := tree.ParseDTimestampTZ(s, time.UTC, time.Millisecond)
d, err := tree.ParseDTimestampTZ(nil, s, time.Millisecond)
if err != nil {
t.Fatal(err)
}
Expand All @@ -239,6 +246,7 @@ var parseFuncs = map[types.T]func(*testing.T, string) tree.Datum{
types.Bytes: func(t *testing.T, s string) tree.Datum { return tree.NewDBytes(tree.DBytes(s)) },
types.Bool: mustParseDBool,
types.Date: mustParseDDate,
types.Time: mustParseDTime,
types.Timestamp: mustParseDTimestamp,
types.TimestampTZ: mustParseDTimestampTZ,
types.Interval: mustParseDInterval,
Expand Down Expand Up @@ -277,11 +285,11 @@ func TestStringConstantResolveAvailableTypes(t *testing.T) {
},
{
c: tree.NewStrVal("2010-09-28 12:00:00.1"),
parseOptions: typeSet(types.String, types.Bytes, types.Timestamp, types.TimestampTZ, types.Date),
parseOptions: typeSet(types.String, types.Bytes, types.Time, types.Timestamp, types.TimestampTZ, types.Date),
},
{
c: tree.NewStrVal("2006-07-08T00:00:00.000000123Z"),
parseOptions: typeSet(types.String, types.Bytes, types.Timestamp, types.TimestampTZ, types.Date),
parseOptions: typeSet(types.String, types.Bytes, types.Time, types.Timestamp, types.TimestampTZ, types.Date),
},
{
c: tree.NewStrVal("PT12H2M"),
Expand Down
Loading

0 comments on commit c8e87ae

Please sign in to comment.