From a3af55eaf6e3bf31d49c19043ba5bb338f802fd5 Mon Sep 17 00:00:00 2001
From: Oliver Tan
Date: Wed, 28 Jul 2021 13:02:20 +1000
Subject: [PATCH 1/4] builtins: introduce 1 arg variant parse_interval /
to_char
Release note (sql change): Introduce parse_interval and
to_char which takes in 1 string or interval and assumes the
Postgres IntervalStyle to make it's output.
---
docs/generated/sql/functions.md | 8 +++++--
.../logictest/testdata/logic_test/interval | 21 +++++++++-------
pkg/sql/sem/builtins/builtins.go | 24 +++++++++++++++++++
pkg/sql/sem/tree/casts.go | 4 ++--
4 files changed, 44 insertions(+), 13 deletions(-)
diff --git a/docs/generated/sql/functions.md b/docs/generated/sql/functions.md
index 799bb0ae8618..51c3f80fba2d 100644
--- a/docs/generated/sql/functions.md
+++ b/docs/generated/sql/functions.md
@@ -500,8 +500,6 @@ and which stays constant throughout the transaction. This timestamp
has no relationship with the commit order of concurrent transactions.
This function is the preferred overload and will be evaluated by default.
-parse_interval(string: string, style: string) → interval | Convert a string to an interval using the given IntervalStyle.
- |
statement_timestamp() → timestamp | Returns the start time of the current statement.
|
statement_timestamp() → timestamptz | Returns the start time of the current statement.
@@ -518,6 +516,8 @@ has no relationship with the commit order of concurrent transactions.
|
timezone(timezone: string, timetz: timetz) → timetz | Convert given time with time zone to the new time zone.
|
+to_char(interval: interval) → string | Convert an interval to a string assuming the Postgres IntervalStyle.
+ |
transaction_timestamp() → date | Returns the time of the current transaction.
The value is based on a timestamp picked when the transaction starts
and which stays constant throughout the transaction. This timestamp
@@ -2522,6 +2522,10 @@ The output can be used to recreate a database.’
|
overlay(input: string, overlay_val: string, start_pos: int, end_pos: int) → string | Deletes the characters in input between start_pos and end_pos (count starts at 1), and then insert overlay_val at start_pos .
|
+parse_interval(string: string, style: string) → interval | Convert a string to an interval using the given IntervalStyle.
+ |
+parse_interval(val: string) → interval | Convert a string to an interval assuming the Postgres IntervalStyle.
+ |
parse_timestamp(string: string) → timestamp | Convert a string containing an absolute timestamp to the corresponding timestamp.
|
pg_collation_for(str: anyelement) → string | Returns the collation of the argument
diff --git a/pkg/sql/logictest/testdata/logic_test/interval b/pkg/sql/logictest/testdata/logic_test/interval
index d2cd70605d8a..ba4b79d9cd12 100644
--- a/pkg/sql/logictest/testdata/logic_test/interval
+++ b/pkg/sql/logictest/testdata/logic_test/interval
@@ -404,12 +404,12 @@ SET intervalstyle = 'sql_standard'
statement ok
SET intervalstyle_enabled = 'on'
-statement error context-dependent operators are not allowed in computed column\nHINT: INTERVAL to STRING casts depends on IntervalStyle; consider using to_char_with_style\(interval, 'postgres'\)
+statement error context-dependent operators are not allowed in computed column\nHINT: INTERVAL to STRING casts depends on IntervalStyle; consider using to_char\(interval\)
CREATE TABLE invalid_table (
invalid_col string AS ('1 hour'::interval::string) STORED
)
-statement error string::interval: context-dependent operators are not allowed in computed column\nHINT: STRING to INTERVAL casts depend on session IntervalStyle; use parse_interval\(string, 'postgres'\) instead
+statement error string::interval: context-dependent operators are not allowed in computed column\nHINT: STRING to INTERVAL casts depend on session IntervalStyle; use parse_interval\(string\) instead
CREATE TABLE invalid_table (
invalid_col interval AS ('1 hour'::string::interval) STORED
)
@@ -445,29 +445,32 @@ SELECT i FROM intervals ORDER BY pk
1 day 04:06:08.123
2 years 11 mons -2 days +03:25:45.678
-query TTTBBB
-WITH tbl(pk, i, pg, iso, sql_std) AS (
+query TTTTBBBB
+WITH tbl(pk, i, pg, iso, sql_std, default_style) AS (
SELECT
pk,
i,
to_char_with_style(i, 'postgres') AS pg,
to_char_with_style(i, 'iso_8601') AS iso,
- to_char_with_style(i, 'sql_standard') AS sql_std
+ to_char_with_style(i, 'sql_standard') AS sql_std,
+ to_char(i) AS default_style
FROM intervals
)
SELECT
pg,
iso,
sql_std,
+ default_style,
i = parse_interval(pg, 'postgres'),
i = parse_interval(iso, 'iso_8601'),
- i = parse_interval(sql_std, 'sql_standard')
+ i = parse_interval(sql_std, 'sql_standard'),
+ i = parse_interval(default_style) AND pg = default_style
FROM tbl
ORDER BY pk
----
--2 years -11 mons +1 day 04:05:06.123 P-2Y-11M1DT4H5M6.123S -2-11 +1 +4:05:06.123 true true true
-1 day 04:06:08.123 P1DT4H6M8.123S 1 4:06:08.123 true true true
-2 years 11 mons -2 days +03:25:45.678 P2Y11M-2DT3H25M45.678S +2-11 -2 +3:25:45.678 true true true
+-2 years -11 mons +1 day 04:05:06.123 P-2Y-11M1DT4H5M6.123S -2-11 +1 +4:05:06.123 -2 years -11 mons +1 day 04:05:06.123 true true true true
+1 day 04:06:08.123 P1DT4H6M8.123S 1 4:06:08.123 1 day 04:06:08.123 true true true true
+2 years 11 mons -2 days +03:25:45.678 P2Y11M-2DT3H25M45.678S +2-11 -2 +3:25:45.678 2 years 11 mons -2 days +03:25:45.678 true true true true
query T
SELECT array_to_string(array_agg(i ORDER BY pk), ' ') FROM intervals
diff --git a/pkg/sql/sem/builtins/builtins.go b/pkg/sql/sem/builtins/builtins.go
index 2f9dbe40a6eb..c4bdcb0a885b 100644
--- a/pkg/sql/sem/builtins/builtins.go
+++ b/pkg/sql/sem/builtins/builtins.go
@@ -2344,6 +2344,22 @@ var builtins = map[string]builtinDefinition{
},
),
+ "to_char": makeBuiltin(
+ defProps(),
+ tree.Overload{
+ Types: tree.ArgTypes{{"interval", types.Interval}},
+ ReturnType: tree.FixedReturnType(types.String),
+ Fn: func(ctx *tree.EvalContext, args tree.Datums) (tree.Datum, error) {
+ d := tree.MustBeDInterval(args[0]).Duration
+ var buf bytes.Buffer
+ d.FormatWithStyle(&buf, duration.IntervalStyle_POSTGRES)
+ return tree.NewDString(buf.String()), nil
+ },
+ Info: "Convert an interval to a string assuming the Postgres IntervalStyle.",
+ Volatility: tree.VolatilityImmutable,
+ },
+ ),
+
"to_char_with_style": makeBuiltin(
defProps(),
tree.Overload{
@@ -3022,6 +3038,14 @@ may increase either contention or retry errors, or both.`,
"parse_interval": makeBuiltin(
defProps(),
+ stringOverload1(
+ func(evalCtx *tree.EvalContext, s string) (tree.Datum, error) {
+ return tree.ParseDInterval(duration.IntervalStyle_POSTGRES, s)
+ },
+ types.Interval,
+ "Convert a string to an interval assuming the Postgres IntervalStyle.",
+ tree.VolatilityImmutable,
+ ),
tree.Overload{
Types: tree.ArgTypes{{"string", types.String}, {"style", types.String}},
ReturnType: tree.FixedReturnType(types.Interval),
diff --git a/pkg/sql/sem/tree/casts.go b/pkg/sql/sem/tree/casts.go
index 718c5502bdef..c2f41b2b2e31 100644
--- a/pkg/sql/sem/tree/casts.go
+++ b/pkg/sql/sem/tree/casts.go
@@ -184,7 +184,7 @@ var validCasts = []castInfo{
from: types.IntervalFamily,
to: types.StringFamily,
volatility: VolatilityImmutable,
- volatilityHint: "INTERVAL to STRING casts depends on IntervalStyle; consider using to_char_with_style(interval, 'postgres')",
+ volatilityHint: "INTERVAL to STRING casts depends on IntervalStyle; consider using to_char(interval)",
},
{from: types.UuidFamily, to: types.StringFamily, volatility: VolatilityImmutable},
{from: types.DateFamily, to: types.StringFamily, volatility: VolatilityImmutable},
@@ -288,7 +288,7 @@ var validCasts = []castInfo{
from: types.StringFamily,
to: types.IntervalFamily,
volatility: VolatilityImmutable,
- volatilityHint: "STRING to INTERVAL casts depend on session IntervalStyle; use parse_interval(string, 'postgres') instead",
+ volatilityHint: "STRING to INTERVAL casts depend on session IntervalStyle; use parse_interval(string) instead",
},
{from: types.CollatedStringFamily, to: types.IntervalFamily, volatility: VolatilityImmutable},
{from: types.IntFamily, to: types.IntervalFamily, volatility: VolatilityImmutable},
From 530a93725a9c93b456bb023d26605c98ba81d09e Mon Sep 17 00:00:00 2001
From: Oliver Tan
Date: Wed, 4 Aug 2021 17:46:39 +1000
Subject: [PATCH 2/4] builtins: add timestamp/DateStyle variant for
parse_timestamp/to_char
Release note (sql change): parse_timestamp now has a two argument
variant, which takes in a DateStyle and parses timestamps according to
that DateStyle. The one argument version assumes MDY. These builtins
have an immutable volatility.
Release note (sql change): Introduce a timestamp,DateStyle variant
to to_char_with_style, which converts timestamps to a string with
an immutable volatility. There is also a 1 arg to_char for timestamp
values which assumes the ISO,MDY output style.
---
docs/generated/sql/functions.md | 8 +-
.../logictest/testdata/logic_test/datetime | 57 +++++++++---
pkg/sql/sem/builtins/BUILD.bazel | 1 +
pkg/sql/sem/builtins/builtins.go | 91 ++++++++++++++++++-
pkg/sql/sem/tree/datum.go | 20 +++-
5 files changed, 155 insertions(+), 22 deletions(-)
diff --git a/docs/generated/sql/functions.md b/docs/generated/sql/functions.md
index 51c3f80fba2d..668aa40316c4 100644
--- a/docs/generated/sql/functions.md
+++ b/docs/generated/sql/functions.md
@@ -518,6 +518,8 @@ has no relationship with the commit order of concurrent transactions.
|
to_char(interval: interval) → string | Convert an interval to a string assuming the Postgres IntervalStyle.
|
+to_char(timestamp: timestamp) → string | Convert an timestamp to a string assuming the ISO, MDY DateStyle.
+ |
transaction_timestamp() → date | Returns the time of the current transaction.
The value is based on a timestamp picked when the transaction starts
and which stays constant throughout the transaction. This timestamp
@@ -2526,7 +2528,9 @@ The output can be used to recreate a database.’
|
parse_interval(val: string) → interval | Convert a string to an interval assuming the Postgres IntervalStyle.
|
-parse_timestamp(string: string) → timestamp | Convert a string containing an absolute timestamp to the corresponding timestamp.
+parse_timestamp(string: string, datestyle: string) → timestamp | Convert a string containing an absolute timestamp to the corresponding timestamp assuming dates formatted using the given DateStyle.
+ |
+parse_timestamp(val: string) → timestamp | Convert a string containing an absolute timestamp to the corresponding timestamp assuming dates are in MDY format.
|
pg_collation_for(str: anyelement) → string | Returns the collation of the argument
|
@@ -2718,6 +2722,8 @@ The output can be used to recreate a database.’
|
to_char_with_style(interval: interval, style: string) → string | Convert an interval to a string using the given IntervalStyle.
|
+to_char_with_style(timestamp: timestamp, datestyle: string) → string | Convert an timestamp to a string assuming the string is formatted using the given DateStyle.
+ |
to_english(val: int) → string | This function enunciates the value of its argument using English cardinals.
|
to_hex(val: bytes) → string | Converts val to its hexadecimal representation.
diff --git a/pkg/sql/logictest/testdata/logic_test/datetime b/pkg/sql/logictest/testdata/logic_test/datetime
index d71902ff3896..f782d17b6075 100644
--- a/pkg/sql/logictest/testdata/logic_test/datetime
+++ b/pkg/sql/logictest/testdata/logic_test/datetime
@@ -1704,20 +1704,6 @@ SELECT 'infinity'::timestamp, '-infinity'::timestamptz
----
294276-12-31 23:59:59.999999 +0000 +0000 -4713-11-24 00:00:00 +0000 +0000
-query T
-SELECT parse_timestamp('2020-01-02 01:02:03')
-----
-2020-01-02 01:02:03 +0000 +0000
-
-query error could not parse
-SELECT parse_timestamp('foo')
-
-query error parse_timestamp\(\): relative timestamps are not supported
-SELECT parse_timestamp('now')
-
-query error parse_timestamp\(\): relative timestamps are not supported
-SELECT parse_timestamp('tomorrow')
-
# Verify that parse_timestamp can be used in computed column expressions.
statement ok
CREATE TABLE timestamps (s STRING, ts TIMESTAMP AS (parse_timestamp(s)) STORED)
@@ -1840,3 +1826,46 @@ FROM ( VALUES
2020-09-15 15:17:19.123 +0000 UTC 2020-09-15 15:17:19.123 +0000 +0000 0000-01-01 15:17:19.123 +0000 UTC 0000-01-01 15:17:19.123 +0000 UTC 2020-09-15 00:00:00 +0000 +0000
1995-03-04 15:16:19.123 +0000 UTC 1995-03-04 15:16:19.123 +0000 +0000 0000-01-01 15:16:19.123 +0000 UTC 0000-01-01 15:16:19.123 +0000 UTC 1995-03-04 00:00:00 +0000 +0000
2020-09-05 15:17:19.123 +0000 UTC 2020-09-05 15:17:19.123 +0000 +0000 0000-01-01 15:17:19.123 +0000 UTC 0000-01-01 15:17:19.123 +0000 UTC 2020-09-05 00:00:00 +0000 +0000
+
+statement error only ISO style is supported
+SELECT parse_timestamp('01-02-2020 01:02:03', 'postgres')
+
+query error could not parse
+SELECT parse_timestamp('foo')
+
+query error parse_timestamp\(\): relative timestamps are not supported
+SELECT parse_timestamp('now')
+
+query error parse_timestamp\(\): relative timestamps are not supported
+SELECT parse_timestamp('tomorrow')
+
+query error parse_timestamp\(\): relative timestamps are not supported
+SELECT parse_timestamp('now', 'mdy')
+
+statement error only ISO style is supported
+SELECT to_char_with_style(now()::timestamp, 'postgres')
+
+query TT
+SELECT
+ to_char('2020-01-02 01:02:03'::timestamp),
+ to_char_with_style('2020-01-02 01:02:03'::timestamp, 'DMY')
+----
+2020-01-02 01:02:03 2020-01-02 01:02:03
+
+statement ok
+CREATE TABLE timestamp_datestyle_parse(pk SERIAL PRIMARY KEY, s string);
+INSERT INTO timestamp_datestyle_parse VALUES
+ (1, '07-09-12 11:30:45.123'),
+ (2, '07-09-12')
+
+query TTTT
+SELECT
+ parse_timestamp(s),
+ parse_timestamp(s, 'iso,mdy'),
+ parse_timestamp(s, 'iso,dmy'),
+ parse_timestamp(s, 'iso,ymd')
+FROM timestamp_datestyle_parse
+ORDER BY pk
+----
+2012-07-09 11:30:45.123 +0000 +0000 2012-07-09 11:30:45.123 +0000 +0000 2012-09-07 11:30:45.123 +0000 +0000 2007-09-12 11:30:45.123 +0000 +0000
+2012-07-09 00:00:00 +0000 +0000 2012-07-09 00:00:00 +0000 +0000 2012-09-07 00:00:00 +0000 +0000 2007-09-12 00:00:00 +0000 +0000
diff --git a/pkg/sql/sem/builtins/BUILD.bazel b/pkg/sql/sem/builtins/BUILD.bazel
index 27bbd8037b93..86e14cda1d8f 100644
--- a/pkg/sql/sem/builtins/BUILD.bazel
+++ b/pkg/sql/sem/builtins/BUILD.bazel
@@ -83,6 +83,7 @@ go_library(
"//pkg/util/timeofday",
"//pkg/util/timetz",
"//pkg/util/timeutil",
+ "//pkg/util/timeutil/pgdate",
"//pkg/util/tracing",
"//pkg/util/ulid",
"//pkg/util/unaccent",
diff --git a/pkg/sql/sem/builtins/builtins.go b/pkg/sql/sem/builtins/builtins.go
index c4bdcb0a885b..618b18cddb9e 100644
--- a/pkg/sql/sem/builtins/builtins.go
+++ b/pkg/sql/sem/builtins/builtins.go
@@ -64,6 +64,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/sql/types"
"github.com/cockroachdb/cockroach/pkg/streaming"
"github.com/cockroachdb/cockroach/pkg/util/duration"
+ "github.com/cockroachdb/cockroach/pkg/util/errorutil/unimplemented"
"github.com/cockroachdb/cockroach/pkg/util/fuzzystrmatch"
"github.com/cockroachdb/cockroach/pkg/util/hlc"
"github.com/cockroachdb/cockroach/pkg/util/humanizeutil"
@@ -73,6 +74,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/util/timeofday"
"github.com/cockroachdb/cockroach/pkg/util/timetz"
"github.com/cockroachdb/cockroach/pkg/util/timeutil"
+ "github.com/cockroachdb/cockroach/pkg/util/timeutil/pgdate"
"github.com/cockroachdb/cockroach/pkg/util/tracing"
"github.com/cockroachdb/cockroach/pkg/util/ulid"
"github.com/cockroachdb/cockroach/pkg/util/unaccent"
@@ -2358,6 +2360,16 @@ var builtins = map[string]builtinDefinition{
Info: "Convert an interval to a string assuming the Postgres IntervalStyle.",
Volatility: tree.VolatilityImmutable,
},
+ tree.Overload{
+ Types: tree.ArgTypes{{"timestamp", types.Timestamp}},
+ ReturnType: tree.FixedReturnType(types.String),
+ Fn: func(ctx *tree.EvalContext, args tree.Datums) (tree.Datum, error) {
+ ts := tree.MustBeDTimestamp(args[0])
+ return tree.NewDString(tree.AsStringWithFlags(&ts, tree.FmtBareStrings)), nil
+ },
+ Info: "Convert an timestamp to a string assuming the ISO, MDY DateStyle.",
+ Volatility: tree.VolatilityImmutable,
+ },
),
"to_char_with_style": makeBuiltin(
@@ -2383,6 +2395,24 @@ var builtins = map[string]builtinDefinition{
Info: "Convert an interval to a string using the given IntervalStyle.",
Volatility: tree.VolatilityImmutable,
},
+ tree.Overload{
+ Types: tree.ArgTypes{{"timestamp", types.Timestamp}, {"datestyle", types.String}},
+ ReturnType: tree.FixedReturnType(types.String),
+ Fn: func(ctx *tree.EvalContext, args tree.Datums) (tree.Datum, error) {
+ ts := tree.MustBeDTimestamp(args[0])
+ dateStyleStr := string(tree.MustBeDString(args[1]))
+ ds, err := pgdate.ParseDateStyle(dateStyleStr, pgdate.DefaultDateStyle())
+ if err != nil {
+ return nil, err
+ }
+ if ds.Style != pgdate.Style_ISO {
+ return nil, unimplemented.NewWithIssue(41773, "only ISO style is supported")
+ }
+ return tree.NewDString(tree.AsStringWithFlags(&ts, tree.FmtBareStrings)), nil
+ },
+ Info: "Convert an timestamp to a string assuming the string is formatted using the given DateStyle.",
+ Volatility: tree.VolatilityImmutable,
+ },
),
// https://www.postgresql.org/docs/10/static/functions-datetime.html
@@ -3016,22 +3046,57 @@ may increase either contention or retry errors, or both.`,
// expressions or partial index predicates. Only absolute timestamps that do
// not depend on the current context are supported (relative timestamps like
// 'now' are not supported).
- "parse_timestamp": makeBuiltin(defProps(),
+ "parse_timestamp": makeBuiltin(
+ defProps(),
+ stringOverload1(
+ func(ctx *tree.EvalContext, s string) (tree.Datum, error) {
+ ts, dependsOnContext, err := tree.ParseDTimestamp(
+ tree.NewParseTimeContext(ctx.GetTxnTimestamp(time.Microsecond).Time),
+ s,
+ time.Microsecond,
+ )
+ if err != nil {
+ return nil, err
+ }
+ if dependsOnContext {
+ return nil, pgerror.Newf(
+ pgcode.InvalidParameterValue,
+ "relative timestamps are not supported",
+ )
+ }
+ return ts, nil
+ },
+ types.Timestamp,
+ "Convert a string containing an absolute timestamp to the corresponding timestamp assuming dates are in MDY format.",
+ tree.VolatilityImmutable,
+ ),
tree.Overload{
- Types: tree.ArgTypes{{"string", types.String}},
+ Types: tree.ArgTypes{{"string", types.String}, {"datestyle", types.String}},
ReturnType: tree.FixedReturnType(types.Timestamp),
Fn: func(ctx *tree.EvalContext, args tree.Datums) (tree.Datum, error) {
arg := string(tree.MustBeDString(args[0]))
- ts, dependsOnContext, err := tree.ParseDTimestamp(ctx, arg, time.Microsecond)
+ dateStyle := string(tree.MustBeDString(args[1]))
+ parseCtx, err := parseContextFromDateStyle(ctx, dateStyle)
+ if err != nil {
+ return nil, err
+ }
+ ts, dependsOnContext, err := tree.ParseDTimestamp(
+ parseCtx,
+ arg,
+ time.Microsecond,
+ )
if err != nil {
return nil, err
}
if dependsOnContext {
- return nil, pgerror.Newf(pgcode.InvalidParameterValue, "relative timestamps are not supported")
+ return nil, pgerror.Newf(
+ pgcode.InvalidParameterValue,
+ "relative timestamps are not supported",
+ )
}
return ts, nil
},
- Info: "Convert a string containing an absolute timestamp to the corresponding timestamp.",
+ Info: "Convert a string containing an absolute timestamp to the corresponding timestamp assuming dates formatted using the given DateStyle.",
Volatility: tree.VolatilityImmutable,
},
),
@@ -7946,3 +8011,19 @@ func arrayNumInvertedIndexEntries(
}
return tree.NewDInt(tree.DInt(len(keys))), nil
}
+
+func parseContextFromDateStyle(
+ ctx *tree.EvalContext, dateStyleStr string,
+) (tree.ParseTimeContext, error) {
+ ds, err := pgdate.ParseDateStyle(dateStyleStr, pgdate.DefaultDateStyle())
+ if err != nil {
+ return nil, err
+ }
+ if ds.Style != pgdate.Style_ISO {
+ return nil, unimplemented.NewWithIssue(41773, "only ISO style is supported")
+ }
+ return tree.NewParseTimeContext(
+ ctx.GetTxnTimestamp(time.Microsecond).Time,
+ tree.NewParseTimeContextOptionDateStyle(ds),
+ ), nil
+}
diff --git a/pkg/sql/sem/tree/datum.go b/pkg/sql/sem/tree/datum.go
index c7f7752b86f2..3be1a0f0f1dd 100644
--- a/pkg/sql/sem/tree/datum.go
+++ b/pkg/sql/sem/tree/datum.go
@@ -1779,12 +1779,28 @@ type ParseTimeContext interface {
var _ ParseTimeContext = &EvalContext{}
var _ ParseTimeContext = &simpleParseTimeContext{}
+// NewParseTimeContextOption is an option to NewParseTimeContext.
+type NewParseTimeContextOption func(ret *simpleParseTimeContext)
+
+// NewParseTimeContextOptionDateStyle sets the DateStyle for the context.
+func NewParseTimeContextOptionDateStyle(dateStyle pgdate.DateStyle) NewParseTimeContextOption {
+ return func(ret *simpleParseTimeContext) {
+ ret.DateStyle = dateStyle
+ }
+}
+
// NewParseTimeContext constructs a ParseTimeContext that returns
// the given values.
-func NewParseTimeContext(relativeParseTime time.Time) ParseTimeContext {
- return &simpleParseTimeContext{
+func NewParseTimeContext(
+ relativeParseTime time.Time, opts ...NewParseTimeContextOption,
+) ParseTimeContext {
+ ret := &simpleParseTimeContext{
RelativeParseTime: relativeParseTime,
}
+ for _, opt := range opts {
+ opt(ret)
+ }
+ return ret
}
type simpleParseTimeContext struct {
From cd04e35cce3be0b7bd9d109849c52c68c7e8ceb0 Mon Sep 17 00:00:00 2001
From: Oliver Tan
Date: Wed, 28 Jul 2021 17:08:43 +1000
Subject: [PATCH 3/4] builtins: add parse_date and to_char_with_style(date,
...)
Release note (sql change): Introduce a parse_date builtin with two
variants - the single arg variants parses a date and assumes ISO,MDY
datestyle and the two arg variants parses a date assuming the datestyle
variant on the second argument. This provides an immutable way of
casting strings to dates.
Release note (sql change): Introduce to_char(date), which
assumes a DateStyle of ISO,MDY and outputs date in that format. There is
also a to_char_with_style(date, DateStyle) variant which outputs the
date in the chosen DateStyle. This provides an immutable way of casting
dates to strings.
---
docs/generated/sql/functions.md | 8 ++
.../logictest/testdata/logic_test/datetime | 31 ++++++++
pkg/sql/sem/builtins/builtins.go | 78 +++++++++++++++++++
pkg/sql/sem/tree/datum.go | 24 ++++++
4 files changed, 141 insertions(+)
diff --git a/docs/generated/sql/functions.md b/docs/generated/sql/functions.md
index 668aa40316c4..8e6fa1e4f1f7 100644
--- a/docs/generated/sql/functions.md
+++ b/docs/generated/sql/functions.md
@@ -516,6 +516,8 @@ has no relationship with the commit order of concurrent transactions.
|
timezone(timezone: string, timetz: timetz) → timetz | Convert given time with time zone to the new time zone.
|
+to_char(date: date) → string | Convert an date to a string assuming the ISO, MDY DateStyle.
+ |
to_char(interval: interval) → string | Convert an interval to a string assuming the Postgres IntervalStyle.
|
to_char(timestamp: timestamp) → string | Convert an timestamp to a string assuming the ISO, MDY DateStyle.
@@ -2524,6 +2526,10 @@ The output can be used to recreate a database.’
|
overlay(input: string, overlay_val: string, start_pos: int, end_pos: int) → string | Deletes the characters in input between start_pos and end_pos (count starts at 1), and then insert overlay_val at start_pos .
|
+parse_date(string: string, datestyle: string) → date | Parses a date assuming it is in format specified by DateStyle.
+ |
+parse_date(val: string) → date | Parses a date assuming it is in MDY format.
+ |
parse_interval(string: string, style: string) → interval | Convert a string to an interval using the given IntervalStyle.
|
parse_interval(val: string) → interval | Convert a string to an interval assuming the Postgres IntervalStyle.
@@ -2720,6 +2726,8 @@ The output can be used to recreate a database.’
|
substring(input: varbit, start_pos: int, length: int) → varbit | Returns a bit subarray of input starting at start_pos (count starts at 1) and including up to length characters.
|
+to_char_with_style(date: date, datestyle: string) → string | Convert an date to a string assuming the string is formatted using the given DateStyle.
+ |
to_char_with_style(interval: interval, style: string) → string | Convert an interval to a string using the given IntervalStyle.
|
to_char_with_style(timestamp: timestamp, datestyle: string) → string | Convert an timestamp to a string assuming the string is formatted using the given DateStyle.
diff --git a/pkg/sql/logictest/testdata/logic_test/datetime b/pkg/sql/logictest/testdata/logic_test/datetime
index f782d17b6075..0daf4e6912f5 100644
--- a/pkg/sql/logictest/testdata/logic_test/datetime
+++ b/pkg/sql/logictest/testdata/logic_test/datetime
@@ -1869,3 +1869,34 @@ ORDER BY pk
----
2012-07-09 11:30:45.123 +0000 +0000 2012-07-09 11:30:45.123 +0000 +0000 2012-09-07 11:30:45.123 +0000 +0000 2007-09-12 11:30:45.123 +0000 +0000
2012-07-09 00:00:00 +0000 +0000 2012-07-09 00:00:00 +0000 +0000 2012-09-07 00:00:00 +0000 +0000 2007-09-12 00:00:00 +0000 +0000
+
+statement error only ISO style is supported
+SELECT parse_date('01-02-2020 01:02:03', 'postgres')
+
+query error parse_date\(\): relative dates are not supported
+SELECT parse_date('now')
+
+query error parse_date\(\): relative dates are not supported
+SELECT parse_date('tomorrow', 'iso,mdy')
+
+query TTTT
+SELECT
+ parse_date(s),
+ parse_date(s, 'iso,mdy'),
+ parse_date(s, 'iso,dmy'),
+ parse_date(s, 'iso,ymd')
+FROM timestamp_datestyle_parse
+ORDER BY pk
+----
+2012-07-09 00:00:00 +0000 +0000 2012-07-09 00:00:00 +0000 +0000 2012-09-07 00:00:00 +0000 +0000 2007-09-12 00:00:00 +0000 +0000
+2012-07-09 00:00:00 +0000 +0000 2012-07-09 00:00:00 +0000 +0000 2012-09-07 00:00:00 +0000 +0000 2007-09-12 00:00:00 +0000 +0000
+
+statement error only ISO style is supported
+SELECT to_char_with_style(now()::date, 'postgres')
+
+query TT
+SELECT
+ to_char('2020-01-02 01:02:03'::date),
+ to_char_with_style('2020-01-02 01:02:03'::date, 'DMY')
+----
+2020-01-02 2020-01-02
diff --git a/pkg/sql/sem/builtins/builtins.go b/pkg/sql/sem/builtins/builtins.go
index 618b18cddb9e..cb3f2ac66a01 100644
--- a/pkg/sql/sem/builtins/builtins.go
+++ b/pkg/sql/sem/builtins/builtins.go
@@ -2370,6 +2370,16 @@ var builtins = map[string]builtinDefinition{
Info: "Convert an timestamp to a string assuming the ISO, MDY DateStyle.",
Volatility: tree.VolatilityImmutable,
},
+ tree.Overload{
+ Types: tree.ArgTypes{{"date", types.Date}},
+ ReturnType: tree.FixedReturnType(types.String),
+ Fn: func(ctx *tree.EvalContext, args tree.Datums) (tree.Datum, error) {
+ ts := tree.MustBeDDate(args[0])
+ return tree.NewDString(tree.AsStringWithFlags(&ts, tree.FmtBareStrings)), nil
+ },
+ Info: "Convert an date to a string assuming the ISO, MDY DateStyle.",
+ Volatility: tree.VolatilityImmutable,
+ },
),
"to_char_with_style": makeBuiltin(
@@ -2413,6 +2423,24 @@ var builtins = map[string]builtinDefinition{
Info: "Convert an timestamp to a string assuming the string is formatted using the given DateStyle.",
Volatility: tree.VolatilityImmutable,
},
+ tree.Overload{
+ Types: tree.ArgTypes{{"date", types.Date}, {"datestyle", types.String}},
+ ReturnType: tree.FixedReturnType(types.String),
+ Fn: func(ctx *tree.EvalContext, args tree.Datums) (tree.Datum, error) {
+ ts := tree.MustBeDDate(args[0])
+ dateStyleStr := string(tree.MustBeDString(args[1]))
+ ds, err := pgdate.ParseDateStyle(dateStyleStr, pgdate.DefaultDateStyle())
+ if err != nil {
+ return nil, err
+ }
+ if ds.Style != pgdate.Style_ISO {
+ return nil, unimplemented.NewWithIssue(41773, "only ISO style is supported")
+ }
+ return tree.NewDString(tree.AsStringWithFlags(&ts, tree.FmtBareStrings)), nil
+ },
+ Info: "Convert an date to a string assuming the string is formatted using the given DateStyle.",
+ Volatility: tree.VolatilityImmutable,
+ },
),
// https://www.postgresql.org/docs/10/static/functions-datetime.html
@@ -3101,6 +3129,56 @@ may increase either contention or retry errors, or both.`,
},
),
+ "parse_date": makeBuiltin(
+ defProps(),
+ stringOverload1(
+ func(ctx *tree.EvalContext, s string) (tree.Datum, error) {
+ ts, dependsOnContext, err := tree.ParseDDate(
+ tree.NewParseTimeContext(ctx.GetTxnTimestamp(time.Microsecond).Time),
+ s,
+ )
+ if err != nil {
+ return nil, err
+ }
+ if dependsOnContext {
+ return nil, pgerror.Newf(
+ pgcode.InvalidParameterValue,
+ "relative dates are not supported",
+ )
+ }
+ return ts, nil
+ },
+ types.Date,
+ "Parses a date assuming it is in MDY format.",
+ tree.VolatilityImmutable,
+ ),
+ tree.Overload{
+ Types: tree.ArgTypes{{"string", types.String}, {"datestyle", types.String}},
+ ReturnType: tree.FixedReturnType(types.Date),
+ Fn: func(ctx *tree.EvalContext, args tree.Datums) (tree.Datum, error) {
+ arg := string(tree.MustBeDString(args[0]))
+ dateStyle := string(tree.MustBeDString(args[1]))
+ parseCtx, err := parseContextFromDateStyle(ctx, dateStyle)
+ if err != nil {
+ return nil, err
+ }
+ ts, dependsOnContext, err := tree.ParseDDate(parseCtx, arg)
+ if err != nil {
+ return nil, err
+ }
+ if dependsOnContext {
+ return nil, pgerror.Newf(
+ pgcode.InvalidParameterValue,
+ "relative dates are not supported",
+ )
+ }
+ return ts, nil
+ },
+ Info: "Parses a date assuming it is in format specified by DateStyle.",
+ Volatility: tree.VolatilityImmutable,
+ },
+ ),
+
"parse_interval": makeBuiltin(
defProps(),
stringOverload1(
diff --git a/pkg/sql/sem/tree/datum.go b/pkg/sql/sem/tree/datum.go
index 3be1a0f0f1dd..d418c2a416df 100644
--- a/pkg/sql/sem/tree/datum.go
+++ b/pkg/sql/sem/tree/datum.go
@@ -1858,6 +1858,30 @@ func ParseDDate(ctx ParseTimeContext, s string) (_ *DDate, dependsOnContext bool
return NewDDate(t), dependsOnContext, err
}
+// AsDDate attempts to retrieve a DDate from an Expr, returning a DDate and
+// a flag signifying whether the assertion was successful. The function should
+// be used instead of direct type assertions wherever a *DDate wrapped by a
+// *DOidWrapper is possible.
+func AsDDate(e Expr) (DDate, bool) {
+ switch t := e.(type) {
+ case *DDate:
+ return *t, true
+ case *DOidWrapper:
+ return AsDDate(t.Wrapped)
+ }
+ return DDate{}, false
+}
+
+// MustBeDDate attempts to retrieve a DDate from an Expr, panicking if the
+// assertion fails.
+func MustBeDDate(e Expr) DDate {
+ t, ok := AsDDate(e)
+ if !ok {
+ panic(errors.AssertionFailedf("expected *DDate, found %T", e))
+ }
+ return t
+}
+
// ResolvedType implements the TypedExpr interface.
func (*DDate) ResolvedType() *types.T {
return types.Date
From ed97f6ec33d0dc928547ae5b7b42e6ad51922351 Mon Sep 17 00:00:00 2001
From: Oliver Tan
Date: Tue, 3 Aug 2021 10:25:59 +1000
Subject: [PATCH 4/4] builtins: implement parse_time, parse_timetz
Note to_char_with_style is not needed as these types do not need date to
output.
Release note (sql change): Implement parse_time and parse_timetz
builtins, which parses a time or timetz with immutable volatility.
---
docs/generated/sql/functions.md | 8 ++
.../logictest/testdata/logic_test/datetime | 36 +++++++
pkg/sql/sem/builtins/builtins.go | 102 ++++++++++++++++++
3 files changed, 146 insertions(+)
diff --git a/docs/generated/sql/functions.md b/docs/generated/sql/functions.md
index 8e6fa1e4f1f7..07b0588a58bc 100644
--- a/docs/generated/sql/functions.md
+++ b/docs/generated/sql/functions.md
@@ -2534,10 +2534,18 @@ The output can be used to recreate a database.’
|
parse_interval(val: string) → interval | Convert a string to an interval assuming the Postgres IntervalStyle.
|
+parse_time(string: string, timestyle: string) → time | Parses a time assuming the date (if any) is in format specified by DateStyle.
+ |
+parse_time(val: string) → time | Parses a time assuming the date (if any) is in MDY format.
+ |
parse_timestamp(string: string, datestyle: string) → timestamp | Convert a string containing an absolute timestamp to the corresponding timestamp assuming dates formatted using the given DateStyle.
|
parse_timestamp(val: string) → timestamp | Convert a string containing an absolute timestamp to the corresponding timestamp assuming dates are in MDY format.
|
+parse_timetz(string: string, timestyle: string) → timetz | Parses a timetz assuming the date (if any) is in format specified by DateStyle.
+ |
+parse_timetz(val: string) → timetz | Parses a timetz assuming the date (if any) is in MDY format.
+ |
pg_collation_for(str: anyelement) → string | Returns the collation of the argument
|
quote_ident(val: string) → string | Return val suitably quoted to serve as identifier in a SQL statement.
diff --git a/pkg/sql/logictest/testdata/logic_test/datetime b/pkg/sql/logictest/testdata/logic_test/datetime
index 0daf4e6912f5..d3512770f23f 100644
--- a/pkg/sql/logictest/testdata/logic_test/datetime
+++ b/pkg/sql/logictest/testdata/logic_test/datetime
@@ -1900,3 +1900,39 @@ SELECT
to_char_with_style('2020-01-02 01:02:03'::date, 'DMY')
----
2020-01-02 2020-01-02
+
+statement ok
+CREATE TABLE time_datestyle_parse(pk SERIAL PRIMARY KEY, s string);
+INSERT INTO time_datestyle_parse VALUES
+ (1, '2007-09-12 11:30:45.123+06'),
+ (2, '2007-09-12 11:30:45.123+03')
+
+statement error only ISO style is supported
+SELECT parse_time('01-02-2020 01:02:03', 'postgres')
+
+query TTTT
+SELECT
+ parse_time(s),
+ parse_time(s, 'iso,mdy'),
+ parse_time(s, 'iso,dmy'),
+ parse_time(s, 'iso,ymd')
+FROM time_datestyle_parse
+ORDER BY pk
+----
+0000-01-01 11:30:45.123 +0000 UTC 0000-01-01 11:30:45.123 +0000 UTC 0000-01-01 11:30:45.123 +0000 UTC 0000-01-01 11:30:45.123 +0000 UTC
+0000-01-01 11:30:45.123 +0000 UTC 0000-01-01 11:30:45.123 +0000 UTC 0000-01-01 11:30:45.123 +0000 UTC 0000-01-01 11:30:45.123 +0000 UTC
+
+statement error only ISO style is supported
+SELECT parse_timetz('01-02-2020 01:02:03', 'postgres')
+
+query TTTT
+SELECT
+ parse_timetz(s),
+ parse_timetz(s, 'iso,mdy'),
+ parse_timetz(s, 'iso,dmy'),
+ parse_timetz(s, 'iso,ymd')
+FROM time_datestyle_parse
+ORDER BY pk
+----
+0000-01-01 11:30:45.123 +0600 +0600 0000-01-01 11:30:45.123 +0600 +0600 0000-01-01 11:30:45.123 +0600 +0600 0000-01-01 11:30:45.123 +0600 +0600
+0000-01-01 11:30:45.123 +0300 +0300 0000-01-01 11:30:45.123 +0300 +0300 0000-01-01 11:30:45.123 +0300 +0300 0000-01-01 11:30:45.123 +0300 +0300
diff --git a/pkg/sql/sem/builtins/builtins.go b/pkg/sql/sem/builtins/builtins.go
index cb3f2ac66a01..85a1196ef051 100644
--- a/pkg/sql/sem/builtins/builtins.go
+++ b/pkg/sql/sem/builtins/builtins.go
@@ -3179,6 +3179,57 @@ may increase either contention or retry errors, or both.`,
},
),
+ "parse_time": makeBuiltin(
+ defProps(),
+ stringOverload1(
+ func(ctx *tree.EvalContext, s string) (tree.Datum, error) {
+ t, dependsOnContext, err := tree.ParseDTime(
+ tree.NewParseTimeContext(ctx.GetTxnTimestamp(time.Microsecond).Time),
+ s,
+ time.Microsecond,
+ )
+ if err != nil {
+ return nil, err
+ }
+ if dependsOnContext {
+ return nil, pgerror.Newf(
+ pgcode.InvalidParameterValue,
+ "relative times are not supported",
+ )
+ }
+ return t, nil
+ },
+ types.Time,
+ "Parses a time assuming the date (if any) is in MDY format.",
+ tree.VolatilityImmutable,
+ ),
+ tree.Overload{
+ Types: tree.ArgTypes{{"string", types.String}, {"timestyle", types.String}},
+ ReturnType: tree.FixedReturnType(types.Time),
+ Fn: func(ctx *tree.EvalContext, args tree.Datums) (tree.Datum, error) {
+ arg := string(tree.MustBeDString(args[0]))
+ dateStyle := string(tree.MustBeDString(args[1]))
+ parseCtx, err := parseContextFromDateStyle(ctx, dateStyle)
+ if err != nil {
+ return nil, err
+ }
+ t, dependsOnContext, err := tree.ParseDTime(parseCtx, arg, time.Microsecond)
+ if err != nil {
+ return nil, err
+ }
+ if dependsOnContext {
+ return nil, pgerror.Newf(
+ pgcode.InvalidParameterValue,
+ "relative times are not supported",
+ )
+ }
+ return t, nil
+ },
+ Info: "Parses a time assuming the date (if any) is in format specified by DateStyle.",
+ Volatility: tree.VolatilityImmutable,
+ },
+ ),
+
"parse_interval": makeBuiltin(
defProps(),
stringOverload1(
@@ -3210,6 +3261,57 @@ may increase either contention or retry errors, or both.`,
},
),
+ "parse_timetz": makeBuiltin(
+ defProps(),
+ stringOverload1(
+ func(ctx *tree.EvalContext, s string) (tree.Datum, error) {
+ t, dependsOnContext, err := tree.ParseDTimeTZ(
+ tree.NewParseTimeContext(ctx.GetTxnTimestamp(time.Microsecond).Time),
+ s,
+ time.Microsecond,
+ )
+ if err != nil {
+ return nil, err
+ }
+ if dependsOnContext {
+ return nil, pgerror.Newf(
+ pgcode.InvalidParameterValue,
+ "relative times are not supported",
+ )
+ }
+ return t, nil
+ },
+ types.TimeTZ,
+ "Parses a timetz assuming the date (if any) is in MDY format.",
+ tree.VolatilityImmutable,
+ ),
+ tree.Overload{
+ Types: tree.ArgTypes{{"string", types.String}, {"timestyle", types.String}},
+ ReturnType: tree.FixedReturnType(types.TimeTZ),
+ Fn: func(ctx *tree.EvalContext, args tree.Datums) (tree.Datum, error) {
+ arg := string(tree.MustBeDString(args[0]))
+ dateStyle := string(tree.MustBeDString(args[1]))
+ parseCtx, err := parseContextFromDateStyle(ctx, dateStyle)
+ if err != nil {
+ return nil, err
+ }
+ t, dependsOnContext, err := tree.ParseDTimeTZ(parseCtx, arg, time.Microsecond)
+ if err != nil {
+ return nil, err
+ }
+ if dependsOnContext {
+ return nil, pgerror.Newf(
+ pgcode.InvalidParameterValue,
+ "relative times are not supported",
+ )
+ }
+ return t, nil
+ },
+ Info: "Parses a timetz assuming the date (if any) is in format specified by DateStyle.",
+ Volatility: tree.VolatilityImmutable,
+ },
+ ),
+
// Array functions.
"string_to_array": makeBuiltin(arrayPropsNullableArgs(),
|