diff --git a/docs/generated/settings/settings-for-tenants.txt b/docs/generated/settings/settings-for-tenants.txt
index fa94521e72e2..e07116e5e52a 100644
--- a/docs/generated/settings/settings-for-tenants.txt
+++ b/docs/generated/settings/settings-for-tenants.txt
@@ -71,6 +71,7 @@ sql.cross_db_fks.enabled boolean false if true, creating foreign key references
sql.cross_db_sequence_owners.enabled boolean false if true, creating sequences owned by tables from other databases is allowed
sql.cross_db_views.enabled boolean false if true, creating views that refer to other databases is allowed
sql.defaults.copy_partitioning_when_deinterleaving_table.enabled boolean false default value for enable_copying_partitioning_when_deinterleaving_table session variable
+sql.defaults.datestyle enumeration iso, mdy default value for DateStyle session setting [iso, mdy = 0, iso, dmy = 1, iso, ymd = 2]
sql.defaults.default_int_size integer 8 the size, in bytes, of an INT type
sql.defaults.disallow_full_table_scans.enabled boolean false setting to true rejects queries that have planned a full table scan
sql.defaults.distsql enumeration auto default distributed SQL execution mode [off = 0, auto = 1, on = 2]
diff --git a/docs/generated/settings/settings.html b/docs/generated/settings/settings.html
index dcb646366245..b9c4338eaea1 100644
--- a/docs/generated/settings/settings.html
+++ b/docs/generated/settings/settings.html
@@ -75,6 +75,7 @@
sql.cross_db_sequence_owners.enabled | boolean | false | if true, creating sequences owned by tables from other databases is allowed |
sql.cross_db_views.enabled | boolean | false | if true, creating views that refer to other databases is allowed |
sql.defaults.copy_partitioning_when_deinterleaving_table.enabled | boolean | false | default value for enable_copying_partitioning_when_deinterleaving_table session variable |
+sql.defaults.datestyle | enumeration | iso, mdy | default value for DateStyle session setting [iso, mdy = 0, iso, dmy = 1, iso, ymd = 2] |
sql.defaults.default_int_size | integer | 8 | the size, in bytes, of an INT type |
sql.defaults.disallow_full_table_scans.enabled | boolean | false | setting to true rejects queries that have planned a full table scan |
sql.defaults.distsql | enumeration | auto | default distributed SQL execution mode [off = 0, auto = 1, on = 2] |
diff --git a/pkg/ccl/cliccl/debug_backup.go b/pkg/ccl/cliccl/debug_backup.go
index be8b31c35a57..7e3133d7090f 100644
--- a/pkg/ccl/cliccl/debug_backup.go
+++ b/pkg/ccl/cliccl/debug_backup.go
@@ -461,7 +461,7 @@ func evalAsOfTimestamp(
}
var err error
// Attempt to parse as timestamp.
- if ts, _, err := pgdate.ParseTimestampWithoutTimezone(timeutil.Now(), pgdate.ParseModeYMD, readTime); err == nil {
+ if ts, _, err := pgdate.ParseTimestampWithoutTimezone(timeutil.Now(), pgdate.DateStyle{Order: pgdate.Order_MDY}, readTime); err == nil {
readTS := hlc.Timestamp{WallTime: ts.UnixNano()}
return readTS, nil
}
diff --git a/pkg/sql/BUILD.bazel b/pkg/sql/BUILD.bazel
index ed95ddb33bfa..e9d146089f62 100644
--- a/pkg/sql/BUILD.bazel
+++ b/pkg/sql/BUILD.bazel
@@ -392,6 +392,7 @@ go_library(
"//pkg/util/stop",
"//pkg/util/syncutil",
"//pkg/util/timeutil",
+ "//pkg/util/timeutil/pgdate",
"//pkg/util/tracing",
"//pkg/util/tracing/collector",
"//pkg/util/tracing/tracingpb",
diff --git a/pkg/sql/colexec/colbuilder/execplan.go b/pkg/sql/colexec/colbuilder/execplan.go
index 58c2c4fa4925..9d9b2b755a4f 100644
--- a/pkg/sql/colexec/colbuilder/execplan.go
+++ b/pkg/sql/colexec/colbuilder/execplan.go
@@ -1914,11 +1914,7 @@ func planSelectionOperators(
case tree.In, tree.NotIn:
negate := cmpOp.Symbol == tree.NotIn
datumTuple, ok := tree.AsDTuple(constArg)
- if !ok || tupleContainsTuples(datumTuple) {
- // Optimized IN operator is supported only on constant
- // expressions that don't contain tuples (because tuples
- // require special null-handling logic), so we fallback to
- // the default comparison operator.
+ if !ok || useDefaultCmpOpForIn(datumTuple) {
break
}
op, err = colexec.GetInOperator(lTyp, leftOp, leftIdx, datumTuple, negate)
@@ -2275,10 +2271,20 @@ func planProjectionExpr(
}
allocator := colmem.NewAllocator(ctx, acc, factory)
resultIdx = -1
+
+ cmpProjOp, isCmpProjOp := projOp.(tree.ComparisonOperator)
+ var hasOptimizedOp bool
+ if isCmpProjOp {
+ switch cmpProjOp.Symbol {
+ case tree.Like, tree.NotLike, tree.In, tree.NotIn, tree.IsDistinctFrom, tree.IsNotDistinctFrom:
+ hasOptimizedOp = true
+ }
+ }
// There are 3 cases. Either the left is constant, the right is constant,
// or neither are constant.
- if lConstArg, lConst := left.(tree.Datum); lConst {
- // Case one: The left is constant.
+ if lConstArg, lConst := left.(tree.Datum); lConst && !hasOptimizedOp {
+ // Case one: The left is constant (and we don't have an optimized
+ // operator for this expression).
// Normally, the optimizer normalizes binary exprs so that the constant
// argument is on the right side. This doesn't happen for
// non-commutative operators such as - and /, though, so we still need
@@ -2318,8 +2324,6 @@ func planProjectionExpr(
right = tupleDatum
}
- cmpProjOp, isCmpProjOp := projOp.(tree.ComparisonOperator)
-
// We have a special case behavior for Is{Not}DistinctFrom before
// checking whether the right expression is constant below in order to
// extract NULL from the cast expression.
@@ -2351,11 +2355,7 @@ func planProjectionExpr(
case tree.In, tree.NotIn:
negate := cmpProjOp.Symbol == tree.NotIn
datumTuple, ok := tree.AsDTuple(rConstArg)
- if !ok || tupleContainsTuples(datumTuple) {
- // Optimized IN operator is supported only on constant
- // expressions that don't contain tuples (because tuples
- // require special null-handling logic), so we fallback to
- // the default comparison operator.
+ if !ok || useDefaultCmpOpForIn(datumTuple) {
break
}
op, err = colexec.GetInProjectionOperator(
@@ -2517,8 +2517,16 @@ func appendOneType(typs []*types.T, t *types.T) []*types.T {
return newTyps
}
-func tupleContainsTuples(tuple *tree.DTuple) bool {
- for _, typ := range tuple.ResolvedType().TupleContents() {
+// useDefaultCmpOpForIn returns whether IN and NOT IN projection/selection
+// operators should be handled via the default operators. This is the case when
+// we have an empty tuple or the tuple contains other tuples (these cases
+// require special null-handling logic).
+func useDefaultCmpOpForIn(tuple *tree.DTuple) bool {
+ tupleContents := tuple.ResolvedType().TupleContents()
+ if len(tupleContents) == 0 {
+ return true
+ }
+ for _, typ := range tupleContents {
if typ.Family() == types.TupleFamily {
return true
}
diff --git a/pkg/sql/colexec/colexeccmp/like_ops.go b/pkg/sql/colexec/colexeccmp/like_ops.go
index f90423f3ddb9..5f03e60498e6 100644
--- a/pkg/sql/colexec/colexeccmp/like_ops.go
+++ b/pkg/sql/colexec/colexeccmp/like_ops.go
@@ -64,10 +64,19 @@ func GetLikeOperatorType(pattern string, negate bool) (LikeOpType, string, error
}
return LikeAlwaysMatch, "", nil
}
- if len(pattern) > 1 && !strings.ContainsAny(pattern[1:len(pattern)-1], "_%") {
- // There are no wildcards in the middle of the string, so we only need to
- // use a regular expression if both the first and last characters are
+ hasEscape := strings.Contains(pattern, `\`)
+ if len(pattern) > 1 && !strings.ContainsAny(pattern[1:len(pattern)-1], "_%") && !hasEscape {
+ // There are no wildcards in the middle of the string as well as no
+ // escape characters in the whole string, so we only need to use a
+ // regular expression if both the first and last characters are
// wildcards.
+ //
+ // The presence of the escape characters breaks the assumptions of the
+ // optimized versions since we no longer could just use the string for a
+ // direct match - we'd need to do some preprocessing here to remove the
+ // escape characters.
+ // TODO(yuzefovich): add that preprocessing (for example, `\\` needs to
+ // be replaced with `\`).
firstChar := pattern[0]
lastChar := pattern[len(pattern)-1]
if !isWildcard(firstChar) && !isWildcard(lastChar) {
diff --git a/pkg/sql/conn_executor.go b/pkg/sql/conn_executor.go
index 6d39696f8f39..deaa0fb78da6 100644
--- a/pkg/sql/conn_executor.go
+++ b/pkg/sql/conn_executor.go
@@ -2405,6 +2405,8 @@ func (ex *connExecutor) resetPlanner(
p.semaCtx.Annotations = nil
p.semaCtx.TypeResolver = p
p.semaCtx.TableNameResolver = p
+ p.semaCtx.DateStyle = ex.sessionData.GetDateStyle()
+ p.semaCtx.IntervalStyle = ex.sessionData.GetIntervalStyle()
ex.resetEvalCtx(&p.extendedEvalCtx, txn, stmtTS)
diff --git a/pkg/sql/exec_util.go b/pkg/sql/exec_util.go
index e57b1c9ea225..2484b03f9685 100644
--- a/pkg/sql/exec_util.go
+++ b/pkg/sql/exec_util.go
@@ -87,6 +87,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/util/metric"
"github.com/cockroachdb/cockroach/pkg/util/mon"
"github.com/cockroachdb/cockroach/pkg/util/syncutil"
+ "github.com/cockroachdb/cockroach/pkg/util/timeutil/pgdate"
"github.com/cockroachdb/cockroach/pkg/util/tracing"
"github.com/cockroachdb/cockroach/pkg/util/tracing/collector"
"github.com/cockroachdb/cockroach/pkg/util/tracing/tracingpb"
@@ -518,6 +519,20 @@ var intervalStyle = settings.RegisterEnumSetting(
}(),
).WithPublic()
+var dateStyleEnumMap = map[int64]string{
+ 0: "ISO, MDY",
+ 1: "ISO, DMY",
+ 2: "ISO, YMD",
+}
+
+// dateStyle controls dates representation.
+var dateStyle = settings.RegisterEnumSetting(
+ "sql.defaults.datestyle",
+ "default value for DateStyle session setting",
+ pgdate.DefaultDateStyle().SQLString(),
+ dateStyleEnumMap,
+).WithPublic()
+
// intervalStyleEnabled controls intervals representation.
// TODO(#sql-experience): remove session setting in v21.1 and have this
// always enabled.
@@ -2568,6 +2583,13 @@ func (m *sessionDataMutator) initSequenceCache() {
// SetIntervalStyle sets the IntervalStyle for the given session.
func (m *sessionDataMutator) SetIntervalStyle(style duration.IntervalStyle) {
m.data.DataConversionConfig.IntervalStyle = style
+ m.paramStatusUpdater.BufferParamStatusUpdate("IntervalStyle", strings.ToLower(style.String()))
+}
+
+// SetDateStyle sets the DateStyle for the given session.
+func (m *sessionDataMutator) SetDateStyle(style pgdate.DateStyle) {
+ m.data.DataConversionConfig.DateStyle = style
+ m.paramStatusUpdater.BufferParamStatusUpdate("DateStyle", style.SQLString())
}
// SetIntervalStyleEnabled sets the IntervalStyleEnabled for the given session.
diff --git a/pkg/sql/logictest/testdata/logic_test/datetime b/pkg/sql/logictest/testdata/logic_test/datetime
index 88543c8c78cb..d71902ff3896 100644
--- a/pkg/sql/logictest/testdata/logic_test/datetime
+++ b/pkg/sql/logictest/testdata/logic_test/datetime
@@ -1457,14 +1457,14 @@ SELECT '-563-12-26'::DATE
-0563-12-26 00:00:00 +0000 +0000
query T
-SELECT '-56-12-26'::DATE
+SELECT '6-12-26 BC'::DATE
----
--0056-12-26 00:00:00 +0000 +0000
+-0025-06-12 00:00:00 +0000 +0000
query T
-SELECT '-5-12-26'::DATE
+SELECT '5-12-26 BC'::DATE
----
--0005-12-26 00:00:00 +0000 +0000
+-0025-05-12 00:00:00 +0000 +0000
# Update: dates now have a much more limited range such that the original
# dates from this issue are no longer possible to express.
@@ -1763,3 +1763,80 @@ query B
SELECT ((SELECT '-infinity'::DATE)) < '2021-04-21':::TIMESTAMP
----
true
+
+subtest datestyle_order
+
+statement ok
+set datestyle = 'dmy'
+
+query T
+SELECT '05-07-2020'::date
+----
+2020-07-05 00:00:00 +0000 +0000
+
+query TTTTT
+SELECT
+ t::timestamptz,
+ t::timestamp,
+ t::timetz,
+ t::time,
+ t::date
+FROM ( VALUES
+ ('2020-09-15 15:17:19.123'),
+ ('15-09-2020 15:17:19.123'),
+ ('03-04-95 15:16:19.123'),
+ ('09-05-2020 15:17:19.123')
+) tbl(t)
+----
+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
+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-04-03 15:16:19.123 +0000 UTC 1995-04-03 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-04-03 00:00:00 +0000 +0000
+2020-05-09 15:17:19.123 +0000 UTC 2020-05-09 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-05-09 00:00:00 +0000 +0000
+
+statement ok
+set datestyle = 'ymd'
+
+statement error field day value 2020 is out of range\nHINT: Perhaps you need a different "datestyle" setting.
+SELECT '05-07-2020'::date
+
+query TTTTT
+SELECT
+ t::timestamptz,
+ t::timestamp,
+ t::timetz,
+ t::time,
+ t::date
+FROM ( VALUES
+ ('2020-09-15 15:17:19.123'),
+ ('95-03-04 15:16:19.123')
+) tbl(t)
+----
+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
+
+statement ok
+set datestyle = 'mdy'
+
+query T
+SELECT '05-07-2020'::date
+----
+2020-05-07 00:00:00 +0000 +0000
+
+query TTTTT
+SELECT
+ t::timestamptz,
+ t::timestamp,
+ t::timetz,
+ t::time,
+ t::date
+FROM ( VALUES
+ ('2020-09-15 15:17:19.123'),
+ ('09-15-2020 15:17:19.123'),
+ ('03-04-95 15:16:19.123'),
+ ('09-05-2020 15:17:19.123')
+) tbl(t)
+----
+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
+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
diff --git a/pkg/sql/logictest/testdata/logic_test/interval b/pkg/sql/logictest/testdata/logic_test/interval
index f6791103889a..d2cd70605d8a 100644
--- a/pkg/sql/logictest/testdata/logic_test/interval
+++ b/pkg/sql/logictest/testdata/logic_test/interval
@@ -423,6 +423,11 @@ INSERT INTO intervals VALUES
(2, '1 day 04:06:08.123'),
(3, '2 years 11 mons -2 days +03:25:45.678')
+query T
+SELECT '-2 years 11 months 1 day 01:02:03'::interval
+----
+-1 years -1 mons +1 day 01:02:03
+
statement ok
create table interval_parsing ( pk INT PRIMARY KEY, i TEXT )
@@ -506,6 +511,11 @@ SELECT i, i::INTERVAL FROM interval_parsing ORDER BY pk
statement ok
SET intervalstyle = 'iso_8601'
+query T
+SELECT '-2 years 11 months 1 day 01:02:03'::interval
+----
+P-1Y-1M1DT1H2M3S
+
query T
SELECT i FROM intervals ORDER BY pk
----
@@ -555,6 +565,11 @@ SELECT i, i::INTERVAL FROM interval_parsing ORDER BY pk
statement ok
SET intervalstyle = 'sql_standard'
+query T
+SELECT '-2 years 11 months 1 day 01:02:03'::interval
+----
+-2-11 -1 -1:02:03
+
query T
SELECT i FROM intervals ORDER BY pk
----
diff --git a/pkg/sql/logictest/testdata/logic_test/set b/pkg/sql/logictest/testdata/logic_test/set
index 51668ddb37f4..0d8e65759623 100644
--- a/pkg/sql/logictest/testdata/logic_test/set
+++ b/pkg/sql/logictest/testdata/logic_test/set
@@ -150,12 +150,52 @@ SET escape_string_warning = 'off'
statement ok
SET datestyle = 'ISO'
+query T
+SHOW datestyle
+----
+ISO, MDY
+
statement ok
SET datestyle = 'ISO, MDY'
+query T
+SHOW datestyle
+----
+ISO, MDY
+
statement ok
SET datestyle = 'mdy, iso'
+query T
+SHOW datestyle
+----
+ISO, MDY
+
+statement ok
+SET datestyle = 'ymd'
+
+query T
+SHOW datestyle
+----
+ISO, YMD
+
+statement ok
+SET datestyle = 'DMY, ISo'
+
+query T
+SHOW datestyle
+----
+ISO, DMY
+
+statement error only ISO style is supported
+SET datestyle = 'postgres'
+
+statement error only ISO style is supported
+SET datestyle = 'german'
+
+statement error only ISO style is supported
+SET datestyle = 'sql'
+
statement error invalid value for parameter "DateStyle": "other"
SET datestyle = 'other'
diff --git a/pkg/sql/logictest/testdata/logic_test/vectorize b/pkg/sql/logictest/testdata/logic_test/vectorize
index b1e93199d272..03435db81ed5 100644
--- a/pkg/sql/logictest/testdata/logic_test/vectorize
+++ b/pkg/sql/logictest/testdata/logic_test/vectorize
@@ -1271,3 +1271,12 @@ SELECT b FROM t66706@u WHERE NOT (b = 'foo')
----
bar
bar
+
+# Regression test for ignoring the escaping in the LIKE pattern (#68040).
+statement ok
+CREATE TABLE t68040 (c) AS SELECT 'string with \ backslash'
+
+query T
+SELECT c FROM t68040 WHERE c LIKE '%\\%'
+----
+string with \ backslash
diff --git a/pkg/sql/opt/memo/BUILD.bazel b/pkg/sql/opt/memo/BUILD.bazel
index 1f457f93dda0..a2beb7c3a038 100644
--- a/pkg/sql/opt/memo/BUILD.bazel
+++ b/pkg/sql/opt/memo/BUILD.bazel
@@ -39,9 +39,11 @@ go_library(
"//pkg/sql/sem/builtins",
"//pkg/sql/sem/tree",
"//pkg/sql/types",
+ "//pkg/util/duration",
"//pkg/util/encoding",
"//pkg/util/json",
"//pkg/util/log",
+ "//pkg/util/timeutil/pgdate",
"//pkg/util/treeprinter",
"@com_github_cockroachdb_errors//:errors",
],
@@ -84,6 +86,7 @@ go_test(
"//pkg/sql/sem/tree",
"//pkg/sql/types",
"//pkg/testutils",
+ "//pkg/util/duration",
"//pkg/util/leaktest",
"//pkg/util/timeofday",
"//pkg/util/timeutil/pgdate",
diff --git a/pkg/sql/opt/memo/memo.go b/pkg/sql/opt/memo/memo.go
index 9ed89a7f13e4..d2cc7d4c9427 100644
--- a/pkg/sql/opt/memo/memo.go
+++ b/pkg/sql/opt/memo/memo.go
@@ -18,7 +18,9 @@ import (
"github.com/cockroachdb/cockroach/pkg/sql/opt/props"
"github.com/cockroachdb/cockroach/pkg/sql/opt/props/physical"
"github.com/cockroachdb/cockroach/pkg/sql/sem/tree"
+ "github.com/cockroachdb/cockroach/pkg/util/duration"
"github.com/cockroachdb/cockroach/pkg/util/log"
+ "github.com/cockroachdb/cockroach/pkg/util/timeutil/pgdate"
"github.com/cockroachdb/errors"
)
@@ -140,6 +142,8 @@ type Memo struct {
preferLookupJoinsForFKs bool
saveTablesPrefix string
intervalStyleEnabled bool
+ dateStyle pgdate.DateStyle
+ intervalStyle duration.IntervalStyle
// curID is the highest currently in-use scalar expression ID.
curID opt.ScalarID
@@ -179,6 +183,8 @@ func (m *Memo) Init(evalCtx *tree.EvalContext) {
preferLookupJoinsForFKs: evalCtx.SessionData.PreferLookupJoinsForFKs,
saveTablesPrefix: evalCtx.SessionData.SaveTablesPrefix,
intervalStyleEnabled: evalCtx.SessionData.IntervalStyleEnabled,
+ dateStyle: evalCtx.SessionData.GetDateStyle(),
+ intervalStyle: evalCtx.SessionData.GetIntervalStyle(),
}
m.metadata.Init()
m.logPropsBuilder.init(evalCtx, m)
@@ -287,7 +293,9 @@ func (m *Memo) IsStale(
m.safeUpdates != evalCtx.SessionData.SafeUpdates ||
m.preferLookupJoinsForFKs != evalCtx.SessionData.PreferLookupJoinsForFKs ||
m.saveTablesPrefix != evalCtx.SessionData.SaveTablesPrefix ||
- m.intervalStyleEnabled != evalCtx.SessionData.IntervalStyleEnabled {
+ m.intervalStyleEnabled != evalCtx.SessionData.IntervalStyleEnabled ||
+ m.dateStyle != evalCtx.SessionData.GetDateStyle() ||
+ m.intervalStyle != evalCtx.SessionData.GetIntervalStyle() {
return true, nil
}
diff --git a/pkg/sql/opt/memo/memo_test.go b/pkg/sql/opt/memo/memo_test.go
index d042699ae792..eeefa2fd2795 100644
--- a/pkg/sql/opt/memo/memo_test.go
+++ b/pkg/sql/opt/memo/memo_test.go
@@ -26,6 +26,8 @@ import (
"github.com/cockroachdb/cockroach/pkg/sql/parser"
"github.com/cockroachdb/cockroach/pkg/sql/sem/tree"
"github.com/cockroachdb/cockroach/pkg/testutils"
+ "github.com/cockroachdb/cockroach/pkg/util/duration"
+ "github.com/cockroachdb/cockroach/pkg/util/timeutil/pgdate"
"github.com/cockroachdb/datadriven"
)
@@ -222,6 +224,18 @@ func TestMemoIsStale(t *testing.T) {
evalCtx.SessionData.IntervalStyleEnabled = false
notStale()
+ // Stale DateStyle.
+ evalCtx.SessionData.DataConversionConfig.DateStyle = pgdate.DateStyle{Order: pgdate.Order_YMD}
+ stale()
+ evalCtx.SessionData.DataConversionConfig.DateStyle = pgdate.DefaultDateStyle()
+ notStale()
+
+ // Stale IntervalStyle.
+ evalCtx.SessionData.DataConversionConfig.IntervalStyle = duration.IntervalStyle_ISO_8601
+ stale()
+ evalCtx.SessionData.DataConversionConfig.IntervalStyle = duration.IntervalStyle_POSTGRES
+ notStale()
+
// Stale prefer lookup joins for FKs.
evalCtx.SessionData.PreferLookupJoinsForFKs = true
stale()
diff --git a/pkg/sql/pgwire/pgwire_test.go b/pkg/sql/pgwire/pgwire_test.go
index 662a7c6ab8b2..05fd486c7925 100644
--- a/pkg/sql/pgwire/pgwire_test.go
+++ b/pkg/sql/pgwire/pgwire_test.go
@@ -1826,7 +1826,7 @@ func TestSessionParameters(t *testing.T) {
{"server_version", "bar", false, false, `parameter "server_version" cannot be changed.*55P02`},
// Erroneous values are also rejected.
{"extra_float_digits", "42", false, false, `42 is outside the valid range for parameter "extra_float_digits".*22023`},
- {"datestyle", "woo", false, false, `invalid value for parameter "DateStyle".*22023`},
+ {"datestyle", "woo", false, false, `invalid value for parameter "DateStyle": "woo".*22023`},
}
for _, test := range testData {
diff --git a/pkg/sql/pgwire/testdata/pgtest/param_status b/pkg/sql/pgwire/testdata/pgtest/param_status
index b017a62a24dc..99a726863eed 100644
--- a/pkg/sql/pgwire/testdata/pgtest/param_status
+++ b/pkg/sql/pgwire/testdata/pgtest/param_status
@@ -68,3 +68,45 @@ ReadyForQuery
{"Type":"ParameterStatus","Name":"TimeZone","Value":"America/New_York"}
{"Type":"CommandComplete","CommandTag":"SET"}
{"Type":"ReadyForQuery","TxStatus":"I"}
+
+# Change interval style.
+
+send
+Query {"String": "SET IntervalStyle_enabled = true"}
+----
+
+until crdb_only
+ReadyForQuery
+----
+{"Type":"CommandComplete","CommandTag":"SET"}
+{"Type":"ReadyForQuery","TxStatus":"I"}
+
+until noncrdb_only
+ErrorResponse
+ReadyForQuery
+----
+{"Type":"ErrorResponse","Code":"42704"}
+{"Type":"ReadyForQuery","TxStatus":"I"}
+
+
+send
+Query {"String": "SET IntervalStyle = 'ISO_8601'"}
+----
+
+until
+ReadyForQuery
+----
+{"Type":"ParameterStatus","Name":"IntervalStyle","Value":"iso_8601"}
+{"Type":"CommandComplete","CommandTag":"SET"}
+{"Type":"ReadyForQuery","TxStatus":"I"}
+
+send
+Query {"String": "SET DateStyle = 'YMD, ISO'"}
+----
+
+until
+ReadyForQuery
+----
+{"Type":"ParameterStatus","Name":"DateStyle","Value":"ISO, YMD"}
+{"Type":"CommandComplete","CommandTag":"SET"}
+{"Type":"ReadyForQuery","TxStatus":"I"}
diff --git a/pkg/sql/planner.go b/pkg/sql/planner.go
index fae4a3aad8ee..079a337f2be7 100644
--- a/pkg/sql/planner.go
+++ b/pkg/sql/planner.go
@@ -340,6 +340,8 @@ func newInternalPlanner(
p.semaCtx.SearchPath = sd.SearchPath
p.semaCtx.IntervalStyleEnabled = sd.IntervalStyleEnabled
p.semaCtx.TypeResolver = p
+ p.semaCtx.DateStyle = sd.GetDateStyle()
+ p.semaCtx.IntervalStyle = sd.GetIntervalStyle()
plannerMon := mon.NewMonitor(fmt.Sprintf("internal-planner.%s.%s", user, opName),
mon.MemoryResource,
diff --git a/pkg/sql/sem/tree/constant.go b/pkg/sql/sem/tree/constant.go
index 362dde8a3fb8..ea3efa19e55a 100644
--- a/pkg/sql/sem/tree/constant.go
+++ b/pkg/sql/sem/tree/constant.go
@@ -22,7 +22,6 @@ import (
"github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgcode"
"github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror"
"github.com/cockroachdb/cockroach/pkg/sql/types"
- "github.com/cockroachdb/cockroach/pkg/util/duration"
"github.com/cockroachdb/errors"
"github.com/lib/pq/oid"
)
@@ -575,7 +574,16 @@ func (expr *StrVal) ResolveAsType(
return ParseDByte(expr.s)
default:
- val, dependsOnContext, err := ParseAndRequireString(typ, expr.s, dummyParseTimeContext{})
+ ptCtx := simpleParseTimeContext{
+ // We can return any time, but not the zero value - it causes an error when
+ // parsing "yesterday".
+ RelativeParseTime: time.Date(2000, time.January, 2, 3, 4, 5, 0, time.UTC),
+ }
+ if semaCtx != nil {
+ ptCtx.DateStyle = semaCtx.DateStyle
+ ptCtx.IntervalStyle = semaCtx.IntervalStyle
+ }
+ val, dependsOnContext, err := ParseAndRequireString(typ, expr.s, ptCtx)
if err != nil {
return nil, err
}
@@ -592,22 +600,3 @@ func (expr *StrVal) ResolveAsType(
return c.TypeCheck(ctx, semaCtx, typ)
}
}
-
-// dummyParseTimeContext is a ParseTimeContext when used for parsing timestamps
-// during type-checking. Note that results that depend on the context are not
-// retained in the AST.
-type dummyParseTimeContext struct{}
-
-var _ ParseTimeContext = dummyParseTimeContext{}
-
-// We can return any time, but not the zero value - it causes an error when
-// parsing "yesterday".
-var dummyTime = time.Date(2000, time.January, 2, 3, 4, 5, 0, time.UTC)
-
-func (dummyParseTimeContext) GetRelativeParseTime() time.Time {
- return dummyTime
-}
-
-func (dummyParseTimeContext) GetIntervalStyle() duration.IntervalStyle {
- return duration.IntervalStyle_POSTGRES
-}
diff --git a/pkg/sql/sem/tree/datum.go b/pkg/sql/sem/tree/datum.go
index 5a313f3ec7a9..c7f7752b86f2 100644
--- a/pkg/sql/sem/tree/datum.go
+++ b/pkg/sql/sem/tree/datum.go
@@ -1772,6 +1772,8 @@ type ParseTimeContext interface {
GetRelativeParseTime() time.Time
// GetIntervalStyle returns the interval style in the session.
GetIntervalStyle() duration.IntervalStyle
+ // GetDateStyle returns the date style in the session.
+ GetDateStyle() pgdate.DateStyle
}
var _ ParseTimeContext = &EvalContext{}
@@ -1787,6 +1789,8 @@ func NewParseTimeContext(relativeParseTime time.Time) ParseTimeContext {
type simpleParseTimeContext struct {
RelativeParseTime time.Time
+ DateStyle pgdate.DateStyle
+ IntervalStyle duration.IntervalStyle
}
// GetRelativeParseTime implements ParseTimeContext.
@@ -1794,9 +1798,14 @@ func (ctx simpleParseTimeContext) GetRelativeParseTime() time.Time {
return ctx.RelativeParseTime
}
-// GetIntervalStyle implements ParseTimeContext..
+// GetIntervalStyle implements ParseTimeContext.
func (ctx simpleParseTimeContext) GetIntervalStyle() duration.IntervalStyle {
- return duration.IntervalStyle_POSTGRES
+ return ctx.IntervalStyle
+}
+
+// GetDateStyle implements ParseTimeContext.
+func (ctx simpleParseTimeContext) GetDateStyle() pgdate.DateStyle {
+ return ctx.DateStyle
}
// relativeParseTime chooses a reasonable "now" value for
@@ -1808,6 +1817,20 @@ func relativeParseTime(ctx ParseTimeContext) time.Time {
return ctx.GetRelativeParseTime()
}
+func dateStyle(ctx ParseTimeContext) pgdate.DateStyle {
+ if ctx == nil {
+ return pgdate.DefaultDateStyle()
+ }
+ return ctx.GetDateStyle()
+}
+
+func intervalStyle(ctx ParseTimeContext) duration.IntervalStyle {
+ if ctx == nil {
+ return duration.IntervalStyle_POSTGRES
+ }
+ return ctx.GetIntervalStyle()
+}
+
// ParseDDate parses and returns the *DDate Datum value represented by the provided
// string in the provided location, or an error if parsing is unsuccessful.
//
@@ -1815,7 +1838,7 @@ func relativeParseTime(ctx ParseTimeContext) time.Time {
// ParseTimeContext (either for the time or the local timezone).
func ParseDDate(ctx ParseTimeContext, s string) (_ *DDate, dependsOnContext bool, _ error) {
now := relativeParseTime(ctx)
- t, dependsOnContext, err := pgdate.ParseDate(now, 0 /* mode */, s)
+ t, dependsOnContext, err := pgdate.ParseDate(now, dateStyle(ctx), s)
return NewDDate(t), dependsOnContext, err
}
@@ -1958,7 +1981,7 @@ func ParseDTime(
s = timeutil.ReplaceLibPQTimePrefix(s)
- t, dependsOnContext, err := pgdate.ParseTimeWithoutTimezone(now, pgdate.ParseModeYMD, s)
+ t, dependsOnContext, err := pgdate.ParseTimeWithoutTimezone(now, dateStyle(ctx), s)
if err != nil {
// Build our own error message to avoid exposing the dummy date.
return nil, false, MakeParseError(s, types.Time, nil)
@@ -2089,7 +2112,7 @@ func ParseDTimeTZ(
ctx ParseTimeContext, s string, precision time.Duration,
) (_ *DTimeTZ, dependsOnContext bool, _ error) {
now := relativeParseTime(ctx)
- d, dependsOnContext, err := timetz.ParseTimeTZ(now, s, precision)
+ d, dependsOnContext, err := timetz.ParseTimeTZ(now, dateStyle(ctx), s, precision)
if err != nil {
return nil, false, err
}
@@ -2218,7 +2241,7 @@ func ParseDTimestamp(
ctx ParseTimeContext, s string, precision time.Duration,
) (_ *DTimestamp, dependsOnContext bool, _ error) {
now := relativeParseTime(ctx)
- t, dependsOnContext, err := pgdate.ParseTimestampWithoutTimezone(now, pgdate.ParseModeMDY, s)
+ t, dependsOnContext, err := pgdate.ParseTimestampWithoutTimezone(now, dateStyle(ctx), s)
if err != nil {
return nil, false, err
}
@@ -2492,7 +2515,7 @@ func ParseDTimestampTZ(
ctx ParseTimeContext, s string, precision time.Duration,
) (_ *DTimestampTZ, dependsOnContext bool, _ error) {
now := relativeParseTime(ctx)
- t, dependsOnContext, err := pgdate.ParseTimestamp(now, pgdate.ParseModeMDY, s)
+ t, dependsOnContext, err := pgdate.ParseTimestamp(now, dateStyle(ctx), s)
if err != nil {
return nil, false, err
}
diff --git a/pkg/sql/sem/tree/eval.go b/pkg/sql/sem/tree/eval.go
index 078ccc25737e..718ad53882d1 100644
--- a/pkg/sql/sem/tree/eval.go
+++ b/pkg/sql/sem/tree/eval.go
@@ -47,6 +47,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/util/mon"
"github.com/cockroachdb/cockroach/pkg/util/timeofday"
"github.com/cockroachdb/cockroach/pkg/util/timeutil"
+ "github.com/cockroachdb/cockroach/pkg/util/timeutil/pgdate"
"github.com/cockroachdb/cockroach/pkg/util/uuid"
"github.com/cockroachdb/errors"
"github.com/lib/pq/oid"
@@ -3798,9 +3799,20 @@ func (ctx *EvalContext) GetLocation() *time.Location {
// GetIntervalStyle returns the session interval style.
func (ctx *EvalContext) GetIntervalStyle() duration.IntervalStyle {
+ if ctx.SessionData == nil {
+ return duration.IntervalStyle_POSTGRES
+ }
return ctx.SessionData.GetIntervalStyle()
}
+// GetDateStyle returns the session date style.
+func (ctx *EvalContext) GetDateStyle() pgdate.DateStyle {
+ if ctx.SessionData == nil {
+ return pgdate.DefaultDateStyle()
+ }
+ return ctx.SessionData.GetDateStyle()
+}
+
// Ctx returns the session's context.
func (ctx *EvalContext) Ctx() context.Context {
return ctx.Context
diff --git a/pkg/sql/sem/tree/parse_string.go b/pkg/sql/sem/tree/parse_string.go
index b437c5bc5a1e..5d5ca41e22a8 100644
--- a/pkg/sql/sem/tree/parse_string.go
+++ b/pkg/sql/sem/tree/parse_string.go
@@ -52,7 +52,7 @@ func ParseAndRequireString(
if typErr != nil {
return nil, false, typErr
}
- d, err = ParseDIntervalWithTypeMetadata(ctx.GetIntervalStyle(), s, itm)
+ d, err = ParseDIntervalWithTypeMetadata(intervalStyle(ctx), s, itm)
case types.Box2DFamily:
d, err = ParseDBox2D(s)
case types.GeographyFamily:
diff --git a/pkg/sql/sem/tree/type_check.go b/pkg/sql/sem/tree/type_check.go
index 7fe9315b7cf7..b8f281da853c 100644
--- a/pkg/sql/sem/tree/type_check.go
+++ b/pkg/sql/sem/tree/type_check.go
@@ -21,8 +21,10 @@ import (
"github.com/cockroachdb/cockroach/pkg/sql/sessiondata"
"github.com/cockroachdb/cockroach/pkg/sql/sqltelemetry"
"github.com/cockroachdb/cockroach/pkg/sql/types"
+ "github.com/cockroachdb/cockroach/pkg/util/duration"
"github.com/cockroachdb/cockroach/pkg/util/errorutil/unimplemented"
"github.com/cockroachdb/cockroach/pkg/util/log"
+ "github.com/cockroachdb/cockroach/pkg/util/timeutil/pgdate"
"github.com/cockroachdb/errors"
"golang.org/x/text/language"
)
@@ -55,6 +57,11 @@ type SemaContext struct {
IntervalStyleEnabled bool
Properties SemaProperties
+
+ // DateStyle refers to the DateStyle to parse as.
+ DateStyle pgdate.DateStyle
+ // IntervalStyle refers to the IntervalStyle to parse as.
+ IntervalStyle duration.IntervalStyle
}
// SemaProperties is a holder for required and derived properties
diff --git a/pkg/sql/sessiondata/BUILD.bazel b/pkg/sql/sessiondata/BUILD.bazel
index 0cdb96b0aa30..d5044de8fe47 100644
--- a/pkg/sql/sessiondata/BUILD.bazel
+++ b/pkg/sql/sessiondata/BUILD.bazel
@@ -22,6 +22,7 @@ go_library(
"//pkg/util/duration",
"//pkg/util/syncutil",
"//pkg/util/timeutil",
+ "//pkg/util/timeutil/pgdate",
],
)
diff --git a/pkg/sql/sessiondata/session_data.go b/pkg/sql/sessiondata/session_data.go
index 533eab755883..860de33f24c3 100644
--- a/pkg/sql/sessiondata/session_data.go
+++ b/pkg/sql/sessiondata/session_data.go
@@ -20,6 +20,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/sql/sessiondatapb"
"github.com/cockroachdb/cockroach/pkg/util/duration"
"github.com/cockroachdb/cockroach/pkg/util/timeutil"
+ "github.com/cockroachdb/cockroach/pkg/util/timeutil/pgdate"
)
// SessionData contains session parameters. They are all user-configurable.
@@ -115,6 +116,14 @@ func (s *SessionData) GetIntervalStyle() duration.IntervalStyle {
return s.DataConversionConfig.IntervalStyle
}
+// GetDateStyle returns the session date style.
+func (s *SessionData) GetDateStyle() pgdate.DateStyle {
+ if s == nil {
+ return pgdate.DefaultDateStyle()
+ }
+ return s.DataConversionConfig.DateStyle
+}
+
// LocalOnlySessionData contains session parameters that only influence the
// execution on the gateway node and don't need to be propagated to the remote
// nodes.
diff --git a/pkg/sql/sessiondatapb/BUILD.bazel b/pkg/sql/sessiondatapb/BUILD.bazel
index 7a191b333cb8..62da25b40b7c 100644
--- a/pkg/sql/sessiondatapb/BUILD.bazel
+++ b/pkg/sql/sessiondatapb/BUILD.bazel
@@ -18,6 +18,7 @@ proto_library(
visibility = ["//visibility:public"],
deps = [
"//pkg/util/duration:duration_proto",
+ "//pkg/util/timeutil/pgdate:pgdate_proto",
"@com_github_gogo_protobuf//gogoproto:gogo_proto",
],
)
@@ -30,6 +31,7 @@ go_proto_library(
visibility = ["//visibility:public"],
deps = [
"//pkg/util/duration",
+ "//pkg/util/timeutil/pgdate",
"@com_github_gogo_protobuf//gogoproto",
],
)
diff --git a/pkg/sql/sessiondatapb/session_data.pb.go b/pkg/sql/sessiondatapb/session_data.pb.go
index e314e1fc7756..017f05fc8d16 100644
--- a/pkg/sql/sessiondatapb/session_data.pb.go
+++ b/pkg/sql/sessiondatapb/session_data.pb.go
@@ -7,6 +7,7 @@ import (
fmt "fmt"
github_com_cockroachdb_cockroach_pkg_security "github.com/cockroachdb/cockroach/pkg/security"
duration "github.com/cockroachdb/cockroach/pkg/util/duration"
+ pgdate "github.com/cockroachdb/cockroach/pkg/util/timeutil/pgdate"
_ "github.com/gogo/protobuf/gogoproto"
proto "github.com/gogo/protobuf/proto"
io "io"
@@ -180,8 +181,10 @@ type DataConversionConfig struct {
// to use for float conversions.This must be set to a value between -15 and
// 3, inclusive.
ExtraFloatDigits int32 `protobuf:"varint,2,opt,name=extra_float_digits,json=extraFloatDigits,proto3" json:"extra_float_digits,omitempty"`
- // IntervalStyle indicates the style to display intervals as.
+ // IntervalStyle indicates the style to parse and display intervals as.
IntervalStyle duration.IntervalStyle `protobuf:"varint,3,opt,name=interval_style,json=intervalStyle,proto3,enum=cockroach.util.duration.IntervalStyle" json:"interval_style,omitempty"`
+ // DateStyle indicates the style to parse and display dates as.
+ DateStyle pgdate.DateStyle `protobuf:"bytes,4,opt,name=date_style,json=dateStyle,proto3" json:"date_style"`
}
func (m *DataConversionConfig) Reset() { *m = DataConversionConfig{} }
@@ -299,63 +302,66 @@ func init() {
}
var fileDescriptor_9fa1c5a4e61eec38 = []byte{
- // 887 bytes of a gzipped FileDescriptorProto
- 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x84, 0x54, 0x41, 0x6f, 0x1a, 0x47,
- 0x14, 0x66, 0x0d, 0xb8, 0x30, 0x18, 0xb2, 0x9e, 0x90, 0x68, 0x45, 0xd2, 0x05, 0xa1, 0xaa, 0xa2,
- 0x56, 0x05, 0xad, 0x1b, 0xf5, 0x58, 0xa9, 0xc4, 0x58, 0x45, 0x8a, 0x5b, 0x77, 0x91, 0x53, 0xa9,
- 0x87, 0x8e, 0x86, 0xdd, 0x07, 0x4c, 0xbd, 0x3b, 0xb3, 0xec, 0x0c, 0xae, 0x71, 0xff, 0x40, 0x8e,
- 0xfd, 0x0f, 0xed, 0xa1, 0x3f, 0xc5, 0x97, 0x4a, 0x39, 0xe6, 0x64, 0xb5, 0xf8, 0x27, 0xf4, 0xd6,
- 0x53, 0x35, 0x83, 0xbd, 0x40, 0x12, 0xd9, 0xb7, 0x79, 0xdf, 0xf7, 0xbe, 0x6f, 0xf6, 0xbd, 0x37,
- 0x6f, 0xd1, 0x47, 0x72, 0x1a, 0x76, 0x24, 0x48, 0xc9, 0x04, 0x0f, 0xa8, 0xa2, 0xf1, 0xf0, 0x36,
- 0x22, 0x3a, 0x6c, 0xc7, 0x89, 0x50, 0x02, 0x3f, 0xf1, 0x85, 0x7f, 0x9a, 0x08, 0xea, 0x4f, 0xda,
- 0x72, 0x1a, 0xb6, 0x37, 0xf2, 0x6b, 0x4f, 0x67, 0x8a, 0x85, 0x9d, 0x60, 0x96, 0x50, 0xc5, 0x04,
- 0x4f, 0x0f, 0x4b, 0x69, 0xad, 0x3a, 0x16, 0x63, 0x61, 0x8e, 0x1d, 0x7d, 0x5a, 0xa2, 0xcd, 0x57,
- 0xdb, 0xa8, 0x34, 0x58, 0xba, 0x1c, 0x50, 0x45, 0x71, 0x0d, 0x15, 0xb4, 0xdb, 0x90, 0x4a, 0x70,
- 0xac, 0x86, 0xd5, 0x2a, 0x7a, 0x69, 0x8c, 0x3f, 0x41, 0x36, 0x8d, 0xe3, 0x90, 0xf9, 0xc6, 0x96,
- 0x70, 0x1a, 0x81, 0xb3, 0x65, 0x72, 0x1e, 0xac, 0xe1, 0xdf, 0xd2, 0x08, 0x30, 0x45, 0x68, 0x26,
- 0x21, 0x21, 0xe6, 0x12, 0x27, 0xab, 0x93, 0xba, 0xdd, 0xff, 0xae, 0xea, 0x5f, 0x8d, 0x99, 0x9a,
- 0xcc, 0x86, 0x6d, 0x5f, 0x44, 0x9d, 0xb4, 0x94, 0x60, 0xb8, 0x3a, 0x77, 0xe2, 0xd3, 0x71, 0x47,
- 0x82, 0x3f, 0x4b, 0x98, 0x9a, 0xb7, 0x07, 0xdf, 0xbf, 0x38, 0x91, 0x90, 0xe8, 0x9b, 0x8e, 0xb5,
- 0x93, 0x57, 0xd4, 0xae, 0xe6, 0x88, 0x23, 0xf4, 0x58, 0x7f, 0x19, 0xf1, 0x05, 0x3f, 0x83, 0xc4,
- 0x34, 0xca, 0x17, 0x7c, 0xc4, 0xc6, 0x4e, 0xae, 0x61, 0xb5, 0x4a, 0xfb, 0x9f, 0xb7, 0xef, 0xe8,
- 0x55, 0x5b, 0x17, 0xfb, 0x3c, 0x55, 0x3e, 0x37, 0xc2, 0x6e, 0xee, 0xf2, 0xaa, 0x9e, 0xf1, 0xaa,
- 0xc1, 0x7b, 0x38, 0x7c, 0x82, 0x2a, 0x67, 0xe0, 0x2b, 0x91, 0xb0, 0x0b, 0x20, 0x91, 0x08, 0xc0,
- 0xc9, 0x37, 0xac, 0x56, 0x65, 0xbf, 0x7d, 0xe7, 0x35, 0x2f, 0x6f, 0x25, 0xbd, 0x73, 0xf0, 0x8f,
- 0x44, 0x00, 0x5e, 0x39, 0x75, 0xd1, 0x21, 0xee, 0xa1, 0xba, 0x02, 0xa9, 0x18, 0x1f, 0x93, 0x95,
- 0x3d, 0xe3, 0x3f, 0x83, 0xaf, 0x48, 0x4c, 0x39, 0xf3, 0xa5, 0xb3, 0xdd, 0xb0, 0x5a, 0x05, 0xef,
- 0xe9, 0x4d, 0x5a, 0xea, 0xd8, 0x37, 0x49, 0xc7, 0x26, 0x07, 0xb7, 0x90, 0x1d, 0xc0, 0x88, 0xce,
- 0x42, 0x45, 0x18, 0x57, 0x44, 0xb2, 0x0b, 0x70, 0x3e, 0x68, 0x58, 0xad, 0xbc, 0x57, 0xb9, 0xc1,
- 0xfb, 0x5c, 0x0d, 0xd8, 0x05, 0xe8, 0x01, 0x87, 0x62, 0x39, 0x29, 0xa7, 0xb0, 0x1c, 0xf0, 0x6d,
- 0x8c, 0xeb, 0xa8, 0x24, 0x81, 0x26, 0xfe, 0x84, 0xc4, 0x54, 0x4d, 0x9c, 0x62, 0x23, 0xdb, 0x2a,
- 0x7a, 0x68, 0x09, 0x1d, 0x53, 0x35, 0xc1, 0xfb, 0xe8, 0x91, 0x82, 0x28, 0x16, 0x09, 0x4d, 0xe6,
- 0x44, 0xfa, 0x13, 0x88, 0xe8, 0xf2, 0x19, 0x20, 0xe3, 0xf4, 0x30, 0x25, 0x07, 0x86, 0x33, 0x4f,
- 0xe1, 0x08, 0x15, 0x25, 0x4c, 0x89, 0x54, 0x54, 0x81, 0x53, 0x32, 0xa3, 0xd9, 0xbb, 0xb3, 0x67,
- 0x03, 0x98, 0xce, 0x80, 0xfb, 0x30, 0xd0, 0x8a, 0x9b, 0x99, 0x14, 0x24, 0x4c, 0x4d, 0x8c, 0x9b,
- 0x68, 0xe7, 0x07, 0x91, 0x9c, 0x1e, 0x41, 0xf4, 0x82, 0x45, 0x4c, 0x39, 0x3b, 0x0d, 0xab, 0x95,
- 0xf5, 0x36, 0x30, 0xfc, 0x0c, 0x3d, 0x66, 0x5c, 0x41, 0x72, 0x46, 0x43, 0x22, 0xd5, 0x3c, 0x04,
- 0x02, 0x9c, 0x0e, 0x43, 0x08, 0x9c, 0xb2, 0xe9, 0x65, 0xf5, 0x96, 0x1d, 0x68, 0xb2, 0xb7, 0xe4,
- 0x9a, 0xff, 0x5a, 0xa8, 0xfa, 0xbe, 0x67, 0x81, 0x7f, 0x42, 0x0f, 0x87, 0x73, 0x05, 0x92, 0x00,
- 0xf7, 0x45, 0x00, 0x64, 0x24, 0x92, 0x88, 0x2a, 0xb3, 0x1e, 0xf7, 0xcd, 0xbf, 0xab, 0x75, 0x3d,
- 0x23, 0x3b, 0x34, 0x2a, 0x6f, 0x77, 0xf8, 0x36, 0x84, 0x3f, 0x45, 0x18, 0xce, 0x55, 0x42, 0xc9,
- 0x28, 0x14, 0x54, 0x91, 0x80, 0x8d, 0x99, 0x92, 0x66, 0xb3, 0xf2, 0x9e, 0x6d, 0x98, 0x43, 0x4d,
- 0x1c, 0x18, 0x1c, 0x1f, 0xa1, 0xca, 0x66, 0x71, 0x66, 0xbd, 0x2a, 0xfb, 0x1f, 0xaf, 0x7d, 0x88,
- 0xfe, 0x11, 0xb4, 0xd3, 0xfd, 0xef, 0xaf, 0x57, 0xeb, 0x95, 0x37, 0x8a, 0x6f, 0xfe, 0x65, 0xa1,
- 0xf2, 0x46, 0xc7, 0x71, 0x17, 0xe5, 0x24, 0x4c, 0xa5, 0x63, 0x35, 0xb2, 0xad, 0xd2, 0x3d, 0xf5,
- 0x6d, 0x28, 0x75, 0xe4, 0x19, 0x2d, 0xfe, 0x0c, 0x55, 0x43, 0x2a, 0x15, 0xd1, 0x93, 0x67, 0xdc,
- 0x4f, 0x20, 0x02, 0xae, 0x20, 0x30, 0x45, 0x95, 0x3d, 0xac, 0xb9, 0x01, 0x4c, 0xfb, 0x2b, 0xa6,
- 0x76, 0x88, 0xb2, 0x03, 0x98, 0xe2, 0x06, 0xda, 0x36, 0x9a, 0xc0, 0xb4, 0xb7, 0xdc, 0x2d, 0x2e,
- 0xae, 0xea, 0x79, 0x9d, 0x7a, 0xe0, 0xe5, 0x25, 0x4c, 0xfb, 0x01, 0xfe, 0x10, 0xa1, 0x90, 0xea,
- 0x65, 0x20, 0x67, 0x34, 0x34, 0x86, 0x59, 0xaf, 0xb8, 0x44, 0x5e, 0xd2, 0x70, 0x8f, 0xa0, 0xdd,
- 0x77, 0x9a, 0x8e, 0x31, 0xaa, 0xac, 0x81, 0xdf, 0xc0, 0xb9, 0x9d, 0xc1, 0x8f, 0x36, 0x12, 0x7b,
- 0xd2, 0xa7, 0x31, 0xd8, 0xd6, 0x5b, 0x70, 0x97, 0x4a, 0xf8, 0xf2, 0x99, 0xbd, 0x55, 0x2b, 0xbc,
- 0xfa, 0xdd, 0xcd, 0xfc, 0xf9, 0x87, 0x9b, 0xd9, 0xfb, 0x15, 0xed, 0xbe, 0xb3, 0xd5, 0xfa, 0x82,
- 0x14, 0x3c, 0xe1, 0x12, 0x94, 0x9d, 0xc1, 0x0f, 0x50, 0x29, 0xc5, 0xbe, 0xe3, 0xf6, 0x16, 0xae,
- 0xa3, 0x27, 0x6b, 0xca, 0x18, 0x12, 0xa6, 0x6b, 0xa7, 0xe1, 0xd7, 0xe1, 0x2f, 0x74, 0x2e, 0xed,
- 0x2c, 0xb6, 0xd1, 0xce, 0x4a, 0x31, 0x1a, 0xd9, 0xb9, 0xd5, 0xb5, 0xcd, 0x5c, 0xc1, 0xb2, 0xad,
- 0x6e, 0xe7, 0xf2, 0x1f, 0x37, 0x73, 0xb9, 0x70, 0xad, 0xd7, 0x0b, 0xd7, 0x7a, 0xb3, 0x70, 0xad,
- 0xbf, 0x17, 0xae, 0xf5, 0xdb, 0xb5, 0x9b, 0x79, 0x7d, 0xed, 0x66, 0xde, 0x5c, 0xbb, 0x99, 0x1f,
- 0xcb, 0x1b, 0x03, 0x1a, 0x6e, 0x9b, 0x3f, 0xf0, 0x17, 0xff, 0x07, 0x00, 0x00, 0xff, 0xff, 0xfa,
- 0xaa, 0x37, 0xa4, 0x5f, 0x06, 0x00, 0x00,
+ // 933 bytes of a gzipped FileDescriptorProto
+ 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x84, 0x54, 0xc1, 0x6e, 0xdb, 0x46,
+ 0x10, 0x15, 0x2d, 0xdb, 0x95, 0x56, 0x96, 0x42, 0x6f, 0x9c, 0x80, 0x50, 0x52, 0x4a, 0x15, 0x8a,
+ 0x42, 0x31, 0x0a, 0xaa, 0x75, 0x83, 0x1e, 0x0b, 0x54, 0xb1, 0x8c, 0x0a, 0x88, 0x53, 0x97, 0x82,
+ 0x53, 0xa0, 0x87, 0x2e, 0x56, 0xe4, 0x48, 0xda, 0x9a, 0xe4, 0x52, 0xdc, 0x95, 0x6b, 0xb9, 0x3f,
+ 0x90, 0x63, 0xff, 0xa1, 0x3d, 0xf4, 0x3f, 0x7a, 0xf1, 0xa5, 0x40, 0x8e, 0x39, 0x19, 0xad, 0xfc,
+ 0x17, 0x3d, 0x05, 0xbb, 0x94, 0x28, 0xc9, 0x09, 0x9c, 0x13, 0x77, 0xde, 0xcc, 0x9b, 0xe1, 0xcc,
+ 0xdb, 0x1d, 0xf4, 0xa9, 0x18, 0x07, 0x2d, 0x01, 0x42, 0x30, 0x1e, 0xf9, 0x54, 0xd2, 0xb8, 0xbf,
+ 0xb0, 0x88, 0x32, 0x9d, 0x38, 0xe1, 0x92, 0xe3, 0x47, 0x1e, 0xf7, 0xce, 0x12, 0x4e, 0xbd, 0x91,
+ 0x23, 0xc6, 0x81, 0xb3, 0x16, 0x5f, 0x7d, 0x3c, 0x91, 0x2c, 0x68, 0xf9, 0x93, 0x84, 0x4a, 0xc6,
+ 0xa3, 0xec, 0x90, 0x52, 0xab, 0x9f, 0x68, 0xaf, 0x64, 0x21, 0xe8, 0x43, 0x3c, 0xf4, 0xa9, 0x84,
+ 0xf9, 0x67, 0x1e, 0xb2, 0x37, 0xe4, 0x43, 0xae, 0x8f, 0x2d, 0x75, 0x4a, 0xd1, 0xc6, 0xab, 0x6d,
+ 0x54, 0xea, 0xa5, 0x85, 0x0e, 0xa9, 0xa4, 0xb8, 0x8a, 0x0a, 0xaa, 0x60, 0x9f, 0x0a, 0xb0, 0x8c,
+ 0xba, 0xd1, 0x2c, 0xba, 0x99, 0x8d, 0x9f, 0x20, 0x93, 0xc6, 0x71, 0xc0, 0x3c, 0x5d, 0x99, 0x44,
+ 0x34, 0x04, 0x6b, 0x43, 0xc7, 0xdc, 0x5b, 0xc1, 0x5f, 0xd0, 0x10, 0x30, 0x45, 0x68, 0x22, 0x20,
+ 0x21, 0xba, 0x88, 0x95, 0x57, 0x41, 0xed, 0xf6, 0xff, 0xd7, 0xb5, 0x6f, 0x86, 0x4c, 0x8e, 0x26,
+ 0x7d, 0xc7, 0xe3, 0x61, 0x2b, 0xeb, 0xd6, 0xef, 0x2f, 0xcf, 0xad, 0xf8, 0x6c, 0xd8, 0x12, 0xe0,
+ 0x4d, 0x12, 0x26, 0xa7, 0x4e, 0xef, 0x87, 0xe7, 0xa7, 0x02, 0x12, 0x55, 0xe9, 0x44, 0x65, 0x72,
+ 0x8b, 0x2a, 0xab, 0x3e, 0xe2, 0x10, 0x3d, 0x54, 0x7f, 0x46, 0x3c, 0x1e, 0x9d, 0x43, 0xa2, 0x67,
+ 0xe9, 0xf1, 0x68, 0xc0, 0x86, 0xd6, 0x66, 0xdd, 0x68, 0x96, 0x0e, 0xbe, 0x74, 0xee, 0x18, 0xa7,
+ 0xa3, 0x9a, 0x7d, 0x96, 0x31, 0x9f, 0x69, 0x62, 0x7b, 0xf3, 0xea, 0xba, 0x96, 0x73, 0xf7, 0xfc,
+ 0xf7, 0xf8, 0xf0, 0x29, 0xaa, 0x9c, 0x83, 0x27, 0x79, 0xc2, 0x2e, 0x81, 0x84, 0xdc, 0x07, 0x6b,
+ 0xab, 0x6e, 0x34, 0x2b, 0x07, 0xce, 0x9d, 0x65, 0x5e, 0x2e, 0x28, 0x9d, 0x0b, 0xf0, 0x8e, 0xb9,
+ 0x0f, 0x6e, 0x39, 0xcb, 0xa2, 0x4c, 0xdc, 0x41, 0x35, 0x09, 0x42, 0xb2, 0x68, 0x48, 0x96, 0xe9,
+ 0x59, 0xf4, 0x0b, 0x78, 0x92, 0xc4, 0x34, 0x62, 0x9e, 0xb0, 0xb6, 0xeb, 0x46, 0xb3, 0xe0, 0x3e,
+ 0x9e, 0x87, 0x65, 0x19, 0xbb, 0x3a, 0xe8, 0x44, 0xc7, 0xe0, 0x26, 0x32, 0x7d, 0x18, 0xd0, 0x49,
+ 0x20, 0x09, 0x8b, 0x24, 0x11, 0xec, 0x12, 0xac, 0x8f, 0xea, 0x46, 0x73, 0xcb, 0xad, 0xcc, 0xf1,
+ 0x6e, 0x24, 0x7b, 0xec, 0x12, 0x94, 0xc0, 0x01, 0x4f, 0x95, 0xb2, 0x0a, 0xa9, 0xc0, 0x0b, 0x1b,
+ 0xd7, 0x50, 0x49, 0x00, 0x4d, 0xbc, 0x11, 0x89, 0xa9, 0x1c, 0x59, 0xc5, 0x7a, 0xbe, 0x59, 0x74,
+ 0x51, 0x0a, 0x9d, 0x50, 0x39, 0xc2, 0x07, 0xe8, 0x81, 0x84, 0x30, 0xe6, 0x09, 0x4d, 0xa6, 0x44,
+ 0x78, 0x23, 0x08, 0x69, 0x7a, 0x0d, 0x90, 0xce, 0x74, 0x3f, 0x73, 0xf6, 0xb4, 0x4f, 0x5f, 0x85,
+ 0x63, 0x54, 0x14, 0x30, 0x26, 0x42, 0x52, 0x09, 0x56, 0x49, 0x4b, 0xb3, 0x7f, 0xe7, 0xcc, 0x7a,
+ 0x30, 0x9e, 0x40, 0xe4, 0x41, 0x4f, 0x31, 0xe6, 0x9a, 0x14, 0x04, 0x8c, 0xb5, 0x8d, 0x1b, 0x68,
+ 0xe7, 0x47, 0x9e, 0x9c, 0x1d, 0x43, 0xf8, 0x9c, 0x85, 0x4c, 0x5a, 0x3b, 0x75, 0xa3, 0x99, 0x77,
+ 0xd7, 0x30, 0xfc, 0x14, 0x3d, 0x64, 0x91, 0x84, 0xe4, 0x9c, 0x06, 0x44, 0xc8, 0x69, 0x00, 0x04,
+ 0x22, 0xda, 0x0f, 0xc0, 0xb7, 0xca, 0x7a, 0x96, 0x7b, 0x0b, 0x6f, 0x4f, 0x39, 0x3b, 0xa9, 0xaf,
+ 0xf1, 0xf7, 0x06, 0xda, 0x7b, 0xdf, 0xb5, 0xc0, 0x3f, 0xa3, 0xfb, 0xfd, 0xa9, 0x04, 0x41, 0x20,
+ 0xf2, 0xb8, 0x0f, 0x64, 0xc0, 0x93, 0x90, 0x4a, 0xfd, 0x3c, 0x3e, 0xa4, 0x7f, 0x5b, 0xf1, 0x3a,
+ 0x9a, 0x76, 0xa4, 0x59, 0xee, 0x6e, 0xff, 0x36, 0x84, 0x3f, 0x47, 0x18, 0x2e, 0x64, 0x42, 0xc9,
+ 0x20, 0xe0, 0x54, 0x12, 0x9f, 0x0d, 0x99, 0x14, 0xfa, 0x65, 0x6d, 0xb9, 0xa6, 0xf6, 0x1c, 0x29,
+ 0xc7, 0xa1, 0xc6, 0xf1, 0x31, 0xaa, 0xac, 0x37, 0xa7, 0x9f, 0x57, 0xe5, 0xe0, 0xb3, 0x95, 0x1f,
+ 0x51, 0x4b, 0xc0, 0xc9, 0x56, 0x44, 0x77, 0xb5, 0x5b, 0xb7, 0xbc, 0xd6, 0x3c, 0x7e, 0x81, 0x90,
+ 0x5a, 0x12, 0xf3, 0x54, 0xe9, 0xd3, 0x79, 0x72, 0x3b, 0xd5, 0x62, 0xb1, 0x38, 0xf3, 0x8d, 0x72,
+ 0x48, 0x25, 0x68, 0xfa, 0x5c, 0x9e, 0xa2, 0xbf, 0x00, 0x1a, 0xff, 0x18, 0xa8, 0xbc, 0xa6, 0x20,
+ 0x6e, 0xa3, 0x4d, 0x01, 0x63, 0x61, 0x19, 0xf5, 0x7c, 0xb3, 0xf4, 0x81, 0x79, 0xad, 0x31, 0x95,
+ 0xe5, 0x6a, 0x2e, 0xfe, 0x02, 0xed, 0x05, 0x54, 0x48, 0xa2, 0x6e, 0x12, 0x8b, 0xbc, 0x04, 0x42,
+ 0x88, 0x24, 0xf8, 0x7a, 0x48, 0x65, 0x17, 0x2b, 0x5f, 0x0f, 0xc6, 0xdd, 0xa5, 0xa7, 0x7a, 0x84,
+ 0xf2, 0x3d, 0x18, 0xe3, 0x3a, 0xda, 0xd6, 0x1c, 0x5f, 0xcb, 0x55, 0x6e, 0x17, 0x67, 0xd7, 0xb5,
+ 0x2d, 0x15, 0x7a, 0xe8, 0x6e, 0x09, 0x18, 0x77, 0x7d, 0xfc, 0x31, 0x42, 0x01, 0x55, 0x8f, 0x8b,
+ 0x9c, 0xd3, 0x40, 0x27, 0xcc, 0xbb, 0xc5, 0x14, 0x79, 0x49, 0x83, 0x7d, 0x82, 0x76, 0xdf, 0x11,
+ 0x11, 0x63, 0x54, 0x59, 0x01, 0xbf, 0x83, 0x0b, 0x33, 0x87, 0x1f, 0xac, 0x05, 0x76, 0x84, 0x47,
+ 0x63, 0x30, 0x8d, 0x5b, 0x70, 0x9b, 0x0a, 0xf8, 0xfa, 0xa9, 0xb9, 0x51, 0x2d, 0xbc, 0xfa, 0xc3,
+ 0xce, 0xfd, 0xf5, 0xa7, 0x9d, 0xdb, 0xff, 0x0d, 0xed, 0xbe, 0xb3, 0x25, 0x54, 0x81, 0x0c, 0x3c,
+ 0x8d, 0x04, 0x48, 0x33, 0x87, 0xef, 0xa1, 0x52, 0x86, 0x7d, 0x1f, 0x99, 0x1b, 0xb8, 0x86, 0x1e,
+ 0xad, 0x30, 0x63, 0x48, 0x98, 0xea, 0x9d, 0x06, 0xdf, 0x06, 0xbf, 0xd2, 0xa9, 0x30, 0xf3, 0xd8,
+ 0x44, 0x3b, 0x4b, 0xc6, 0x60, 0x60, 0x6e, 0x2e, 0xcb, 0x36, 0x36, 0x0b, 0x86, 0x69, 0xb4, 0x5b,
+ 0x57, 0xff, 0xd9, 0xb9, 0xab, 0x99, 0x6d, 0xbc, 0x9e, 0xd9, 0xc6, 0x9b, 0x99, 0x6d, 0xfc, 0x3b,
+ 0xb3, 0x8d, 0xdf, 0x6f, 0xec, 0xdc, 0xeb, 0x1b, 0x3b, 0xf7, 0xe6, 0xc6, 0xce, 0xfd, 0x54, 0x5e,
+ 0x13, 0xa8, 0xbf, 0xad, 0x37, 0xfa, 0x57, 0x6f, 0x03, 0x00, 0x00, 0xff, 0xff, 0xa4, 0x3d, 0xb8,
+ 0xef, 0xd2, 0x06, 0x00, 0x00,
}
func (m *SessionData) Marshal() (dAtA []byte, err error) {
@@ -500,6 +506,16 @@ func (m *DataConversionConfig) MarshalToSizedBuffer(dAtA []byte) (int, error) {
_ = i
var l int
_ = l
+ {
+ size, err := m.DateStyle.MarshalToSizedBuffer(dAtA[:i])
+ if err != nil {
+ return 0, err
+ }
+ i -= size
+ i = encodeVarintSessionData(dAtA, i, uint64(size))
+ }
+ i--
+ dAtA[i] = 0x22
if m.IntervalStyle != 0 {
i = encodeVarintSessionData(dAtA, i, uint64(m.IntervalStyle))
i--
@@ -673,6 +689,8 @@ func (m *DataConversionConfig) Size() (n int) {
if m.IntervalStyle != 0 {
n += 1 + sovSessionData(uint64(m.IntervalStyle))
}
+ l = m.DateStyle.Size()
+ n += 1 + l + sovSessionData(uint64(l))
return n
}
@@ -1206,6 +1224,39 @@ func (m *DataConversionConfig) Unmarshal(dAtA []byte) error {
break
}
}
+ case 4:
+ if wireType != 2 {
+ return fmt.Errorf("proto: wrong wireType = %d for field DateStyle", wireType)
+ }
+ var msglen int
+ for shift := uint(0); ; shift += 7 {
+ if shift >= 64 {
+ return ErrIntOverflowSessionData
+ }
+ if iNdEx >= l {
+ return io.ErrUnexpectedEOF
+ }
+ b := dAtA[iNdEx]
+ iNdEx++
+ msglen |= int(b&0x7F) << shift
+ if b < 0x80 {
+ break
+ }
+ }
+ if msglen < 0 {
+ return ErrInvalidLengthSessionData
+ }
+ postIndex := iNdEx + msglen
+ if postIndex < 0 {
+ return ErrInvalidLengthSessionData
+ }
+ if postIndex > l {
+ return io.ErrUnexpectedEOF
+ }
+ if err := m.DateStyle.Unmarshal(dAtA[iNdEx:postIndex]); err != nil {
+ return err
+ }
+ iNdEx = postIndex
default:
iNdEx = preIndex
skippy, err := skipSessionData(dAtA[iNdEx:])
diff --git a/pkg/sql/sessiondatapb/session_data.proto b/pkg/sql/sessiondatapb/session_data.proto
index e6ae598fdbd0..36a9e5390c14 100644
--- a/pkg/sql/sessiondatapb/session_data.proto
+++ b/pkg/sql/sessiondatapb/session_data.proto
@@ -13,6 +13,7 @@ package cockroach.sql.sessiondatapb;
option go_package = "sessiondatapb";
import "util/duration/duration.proto";
+import "util/timeutil/pgdate/pgdate.proto";
import "gogoproto/gogo.proto";
// SessionData contains session parameters that are easily serializable and are
@@ -75,8 +76,10 @@ message DataConversionConfig {
// to use for float conversions.This must be set to a value between -15 and
// 3, inclusive.
int32 extra_float_digits = 2;
- // IntervalStyle indicates the style to display intervals as.
+ // IntervalStyle indicates the style to parse and display intervals as.
util.duration.IntervalStyle interval_style = 3;
+ // DateStyle indicates the style to parse and display dates as.
+ util.timeutil.pgdate.DateStyle date_style = 4 [(gogoproto.nullable)=false];
}
// BytesEncodeFormat is the configuration for bytes to string conversions.
diff --git a/pkg/sql/vars.go b/pkg/sql/vars.go
index 4d7cfa732570..e4f2721f7796 100644
--- a/pkg/sql/vars.go
+++ b/pkg/sql/vars.go
@@ -37,6 +37,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/util/errorutil/unimplemented"
"github.com/cockroachdb/cockroach/pkg/util/humanizeutil"
"github.com/cockroachdb/cockroach/pkg/util/timeutil"
+ "github.com/cockroachdb/cockroach/pkg/util/timeutil/pgdate"
"github.com/cockroachdb/errors"
)
@@ -253,24 +254,25 @@ var varGen = map[string]sessionVar{
},
},
- // Supported for PG compatibility only.
// See https://www.postgresql.org/docs/10/static/runtime-config-client.html#GUC-DATESTYLE
`datestyle`: {
Set: func(_ context.Context, m *sessionDataMutator, s string) error {
- s = strings.ToLower(s)
- parts := strings.Split(s, ",")
- isOnlyISO := len(parts) == 1 && strings.TrimSpace(parts[0]) == "iso"
- isISOMDY := len(parts) == 2 && strings.TrimSpace(parts[0]) == "iso" && strings.TrimSpace(parts[1]) == "mdy"
- isMDYISO := len(parts) == 2 && strings.TrimSpace(parts[0]) == "mdy" && strings.TrimSpace(parts[1]) == "iso"
- if !(isOnlyISO || isISOMDY || isMDYISO) {
- err := newVarValueError("DateStyle", s, "ISO", "ISO, MDY", "MDY, ISO")
- err = errors.WithDetail(err, compatErrMsg)
- return err
+ ds, err := pgdate.ParseDateStyle(s, m.data.GetDateStyle())
+ if err != nil {
+ return newVarValueError("DateStyle", s, pgdate.AllowedDateStyles()...)
+ }
+ if ds.Style != pgdate.Style_ISO {
+ return unimplemented.NewWithIssue(41773, "only ISO style is supported")
}
+ m.SetDateStyle(ds)
return nil
},
- Get: func(evalCtx *extendedEvalContext) string { return "ISO, MDY" },
- GlobalDefault: func(_ *settings.Values) string { return "ISO, MDY" },
+ Get: func(evalCtx *extendedEvalContext) string {
+ return evalCtx.GetDateStyle().SQLString()
+ },
+ GlobalDefault: func(sv *settings.Values) string {
+ return dateStyleEnumMap[dateStyle.Get(sv)]
+ },
},
// Controls the subsequent parsing of a "naked" INT type.
diff --git a/pkg/util/encoding/BUILD.bazel b/pkg/util/encoding/BUILD.bazel
index 47a3985e3629..435b8e99a111 100644
--- a/pkg/util/encoding/BUILD.bazel
+++ b/pkg/util/encoding/BUILD.bazel
@@ -51,6 +51,7 @@ go_test(
"//pkg/util/timeofday",
"//pkg/util/timetz",
"//pkg/util/timeutil",
+ "//pkg/util/timeutil/pgdate",
"//pkg/util/uuid",
"@com_github_cockroachdb_apd_v2//:apd",
"@com_github_cockroachdb_errors//:errors",
diff --git a/pkg/util/encoding/encoding_test.go b/pkg/util/encoding/encoding_test.go
index f0a1b26e8a88..cc9529f158dc 100644
--- a/pkg/util/encoding/encoding_test.go
+++ b/pkg/util/encoding/encoding_test.go
@@ -30,6 +30,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/uuid"
"github.com/cockroachdb/errors"
"github.com/stretchr/testify/assert"
@@ -1119,7 +1120,7 @@ func TestEncodeDecodeTimeTZ(t *testing.T) {
t.Run(fmt.Sprintf("dir:%d", dir), func(t *testing.T) {
for i := range testCases {
t.Run(fmt.Sprintf("tc:%d", i), func(t *testing.T) {
- current, _, err := timetz.ParseTimeTZ(timeutil.Now(), testCases[i], time.Microsecond)
+ current, _, err := timetz.ParseTimeTZ(timeutil.Now(), pgdate.DefaultDateStyle(), testCases[i], time.Microsecond)
assert.NoError(t, err)
var b []byte
diff --git a/pkg/util/timetz/BUILD.bazel b/pkg/util/timetz/BUILD.bazel
index d59ae12054f2..ecb840461dbd 100644
--- a/pkg/util/timetz/BUILD.bazel
+++ b/pkg/util/timetz/BUILD.bazel
@@ -22,6 +22,7 @@ go_test(
deps = [
"//pkg/util/timeofday",
"//pkg/util/timeutil",
+ "//pkg/util/timeutil/pgdate",
"@com_github_stretchr_testify//assert",
"@com_github_stretchr_testify//require",
],
diff --git a/pkg/util/timetz/timetz.go b/pkg/util/timetz/timetz.go
index ebde37c2a5b3..97ad1a485b28 100644
--- a/pkg/util/timetz/timetz.go
+++ b/pkg/util/timetz/timetz.go
@@ -35,7 +35,7 @@ var (
// timeTZIncludesDateRegex is a regex to check whether there is a date
// associated with the given string when attempting to parse it.
- timeTZIncludesDateRegex = regexp.MustCompile(`^\d{4}-`)
+ timeTZIncludesDateRegex = regexp.MustCompile(`^\d+[-/]`)
// timeTZHasTimeComponent determines whether there is a time component at all
// in a given string.
timeTZHasTimeComponent = regexp.MustCompile(`\d:`)
@@ -101,7 +101,7 @@ func Now() TimeTZ {
// `now` value (either for the time or the local timezone).
//
func ParseTimeTZ(
- now time.Time, s string, precision time.Duration,
+ now time.Time, dateStyle pgdate.DateStyle, s string, precision time.Duration,
) (_ TimeTZ, dependsOnContext bool, _ error) {
// Special case as we have to use `ParseTimestamp` to get the date.
// We cannot use `ParseTime` as it does not have timezone awareness.
@@ -121,7 +121,7 @@ func ParseTimeTZ(
s = timeutil.ReplaceLibPQTimePrefix(s)
}
- t, dependsOnContext, err := pgdate.ParseTimestamp(now, pgdate.ParseModeYMD, s)
+ t, dependsOnContext, err := pgdate.ParseTimestamp(now, dateStyle, s)
if err != nil {
// Build our own error message to avoid exposing the dummy date.
return TimeTZ{}, false, pgerror.Newf(
diff --git a/pkg/util/timetz/timetz_test.go b/pkg/util/timetz/timetz_test.go
index 5be5e3bc6977..cc36054320e1 100644
--- a/pkg/util/timetz/timetz_test.go
+++ b/pkg/util/timetz/timetz_test.go
@@ -17,6 +17,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/util/timeofday"
"github.com/cockroachdb/cockroach/pkg/util/timeutil"
+ "github.com/cockroachdb/cockroach/pkg/util/timeutil/pgdate"
"github.com/stretchr/testify/assert"
"github.com/stretchr/testify/require"
)
@@ -30,10 +31,10 @@ func TestParseTimeTZToStringRoundTrip(t *testing.T) {
}
for _, tc := range testCases {
t.Run(tc, func(t *testing.T) {
- exampleTime, _, err := ParseTimeTZ(timeutil.Now(), tc, time.Microsecond)
+ exampleTime, _, err := ParseTimeTZ(timeutil.Now(), pgdate.DefaultDateStyle(), tc, time.Microsecond)
assert.NoError(t, err)
- exampleTimeFromString, _, err := ParseTimeTZ(timeutil.Now(), exampleTime.String(), time.Microsecond)
+ exampleTimeFromString, _, err := ParseTimeTZ(timeutil.Now(), pgdate.DefaultDateStyle(), exampleTime.String(), time.Microsecond)
assert.NoError(t, err)
assert.True(t, exampleTime.Equal(exampleTimeFromString))
@@ -63,22 +64,22 @@ func TestTimeTZString(t *testing.T) {
}
func TestTimeTZ(t *testing.T) {
- maxTime, depOnCtx, err := ParseTimeTZ(timeutil.Now(), "24:00:00-1559", time.Microsecond)
+ maxTime, depOnCtx, err := ParseTimeTZ(timeutil.Now(), pgdate.DefaultDateStyle(), "24:00:00-1559", time.Microsecond)
require.NoError(t, err)
require.False(t, depOnCtx)
- minTime, depOnCtx, err := ParseTimeTZ(timeutil.Now(), "00:00:00+1559", time.Microsecond)
+ minTime, depOnCtx, err := ParseTimeTZ(timeutil.Now(), pgdate.DefaultDateStyle(), "00:00:00+1559", time.Microsecond)
require.NoError(t, err)
require.False(t, depOnCtx)
// These are all the same UTC time equivalents.
- utcTime, depOnCtx, err := ParseTimeTZ(timeutil.Now(), "11:14:15+0", time.Microsecond)
+ utcTime, depOnCtx, err := ParseTimeTZ(timeutil.Now(), pgdate.DefaultDateStyle(), "11:14:15+0", time.Microsecond)
require.NoError(t, err)
require.False(t, depOnCtx)
- sydneyTime, depOnCtx, err := ParseTimeTZ(timeutil.Now(), "21:14:15+10", time.Microsecond)
+ sydneyTime, depOnCtx, err := ParseTimeTZ(timeutil.Now(), pgdate.DefaultDateStyle(), "21:14:15+10", time.Microsecond)
require.NoError(t, err)
require.False(t, depOnCtx)
- sydneyTimeWithMillisecond, depOnCtx, err := ParseTimeTZ(timeutil.Now(), "21:14:15.001+10", time.Microsecond)
+ sydneyTimeWithMillisecond, depOnCtx, err := ParseTimeTZ(timeutil.Now(), pgdate.DefaultDateStyle(), "21:14:15.001+10", time.Microsecond)
require.NoError(t, err)
require.False(t, depOnCtx)
@@ -187,6 +188,7 @@ func TestParseTimeTZ(t *testing.T) {
testCases := []struct {
str string
precision time.Duration
+ dateStyle pgdate.DateStyle
expected TimeTZ
expectedDepOnCtx bool
@@ -197,6 +199,8 @@ func TestParseTimeTZ(t *testing.T) {
{str: "01:24:00", precision: time.Microsecond, expected: MakeTimeTZ(timeofday.New(1, 24, 0, 0), 0), expectedDepOnCtx: true},
{str: "01:03:24", precision: time.Microsecond, expected: MakeTimeTZ(timeofday.New(1, 3, 24, 0), 0), expectedDepOnCtx: true},
{str: "1970-01-01 01:02:03", precision: time.Microsecond, expected: MakeTimeTZ(timeofday.New(1, 2, 3, 0), 0), expectedDepOnCtx: true},
+ {str: "30/01/1970 01:02:03", dateStyle: pgdate.DateStyle{Order: pgdate.Order_DMY}, precision: time.Microsecond, expected: MakeTimeTZ(timeofday.New(1, 2, 3, 0), 0), expectedDepOnCtx: true},
+ {str: "01-30-1970 01:02:03", dateStyle: pgdate.DateStyle{Order: pgdate.Order_MDY}, precision: time.Microsecond, expected: MakeTimeTZ(timeofday.New(1, 2, 3, 0), 0), expectedDepOnCtx: true},
{str: "1970-01-01T01:02:03", precision: time.Microsecond, expected: MakeTimeTZ(timeofday.New(1, 2, 3, 0), 0), expectedDepOnCtx: true},
{str: "1970-01-01T01:02:03", precision: time.Microsecond, expected: MakeTimeTZ(timeofday.New(1, 2, 3, 0), 0), expectedDepOnCtx: true},
{str: "0000-01-01 01:02:03", precision: time.Microsecond, expected: MakeTimeTZ(timeofday.New(1, 2, 3, 0), 0), expectedDepOnCtx: true},
@@ -236,7 +240,7 @@ func TestParseTimeTZ(t *testing.T) {
}
for i, tc := range testCases {
t.Run(fmt.Sprintf("#%d: %s", i, tc.str), func(t *testing.T) {
- actual, depOnCtx, err := ParseTimeTZ(timeutil.Now(), tc.str, tc.precision)
+ actual, depOnCtx, err := ParseTimeTZ(timeutil.Now(), tc.dateStyle, tc.str, tc.precision)
if tc.expectedError {
assert.Error(t, err)
} else {
diff --git a/pkg/util/timeutil/pgdate/BUILD.bazel b/pkg/util/timeutil/pgdate/BUILD.bazel
index f7d947649720..8475d93af754 100644
--- a/pkg/util/timeutil/pgdate/BUILD.bazel
+++ b/pkg/util/timeutil/pgdate/BUILD.bazel
@@ -1,3 +1,5 @@
+load("@rules_proto//proto:defs.bzl", "proto_library")
+load("@io_bazel_rules_go//proto:def.bzl", "go_proto_library")
load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test")
load("//build:STRINGER.bzl", "stringer")
@@ -12,8 +14,8 @@ go_library(
"setters.go",
"zone_cache.go",
":gen-field-stringer", # keep
- ":gen-parsemode-stringer", # keep
],
+ embed = [":pgdate_go_proto"],
importpath = "github.com/cockroachdb/cockroach/pkg/util/timeutil/pgdate",
visibility = ["//visibility:public"],
deps = [
@@ -42,6 +44,7 @@ go_test(
"//pkg/util/timeutil",
"@com_github_cockroachdb_errors//:errors",
"@com_github_lib_pq//:pq",
+ "@com_github_stretchr_testify//require",
],
)
@@ -51,8 +54,17 @@ stringer(
typ = "field",
)
-stringer(
- name = "gen-parsemode-stringer",
- src = "parsing.go",
- typ = "ParseMode",
+proto_library(
+ name = "pgdate_proto",
+ srcs = ["pgdate.proto"],
+ strip_import_prefix = "/pkg",
+ visibility = ["//visibility:public"],
+)
+
+go_proto_library(
+ name = "pgdate_go_proto",
+ compilers = ["//pkg/cmd/protoc-gen-gogoroach:protoc-gen-gogoroach_compiler"],
+ importpath = "github.com/cockroachdb/cockroach/pkg/util/timeutil/pgdate",
+ proto = ":pgdate_proto",
+ visibility = ["//visibility:public"],
)
diff --git a/pkg/util/timeutil/pgdate/field_extract.go b/pkg/util/timeutil/pgdate/field_extract.go
index e87e5d09463a..068b9403966a 100644
--- a/pkg/util/timeutil/pgdate/field_extract.go
+++ b/pkg/util/timeutil/pgdate/field_extract.go
@@ -60,7 +60,7 @@ type fieldExtract struct {
// location is set to the timezone specified by the timestamp (if any).
location *time.Location
- mode ParseMode
+ dateStyle DateStyle
// The fields that must be present to succeed.
required fieldSet
// Stores a reference to one of the sentinel values, to be returned
@@ -379,7 +379,7 @@ func (fe *fieldExtract) interpretNumber(numbers []numberChunk, idx int, textMont
chunk.v /= 100
return fe.SetChunk(fieldYear, chunk)
- case chunk.magnitude >= 3 || fe.mode == ParseModeYMD:
+ case chunk.magnitude >= 3 || fe.dateStyle.Order == Order_YMD:
// Example: "YYYY MM DD"
// ^^^^
// Example: "YYY MM DD"
@@ -387,7 +387,7 @@ func (fe *fieldExtract) interpretNumber(numbers []numberChunk, idx int, textMont
// Example: "YY MM DD"
// ^^
// A three- or four-digit number must be a year. If we are in a
- // year-first mode, we'll accept the first chunk and possibly
+ // year-first order, we'll accept the first chunk and possibly
// adjust a two-digit value later on. This means that
// 99 would get adjusted to 1999, but 0099 would not.
if chunk.separator == '-' {
@@ -396,15 +396,15 @@ func (fe *fieldExtract) interpretNumber(numbers []numberChunk, idx int, textMont
fe.tweakYear = true
}
return fe.SetChunk(fieldYear, chunk)
- case fe.mode == ParseModeDMY:
+ case fe.dateStyle.Order == Order_DMY:
// Example: "DD MM YY"
// ^^
- // The first value is ambiguous, so we rely on the mode.
+ // The first value is ambiguous, so we rely on the order.
return fe.SetChunk(fieldDay, chunk)
- case fe.mode == ParseModeMDY:
+ case fe.dateStyle.Order == Order_MDY:
// Example: "MM DD YY"
// ^^
- // The first value is ambiguous, so we rely on the mode.
+ // The first value is ambiguous, so we rely on the order.
return fe.SetChunk(fieldMonth, chunk)
}
@@ -421,19 +421,19 @@ func (fe *fieldExtract) interpretNumber(numbers []numberChunk, idx int, textMont
// ^^^^
// Example: "YYY Month DD"
// ^^^
- // Example: "MM DD YY"; only in MDY mode.
+ // Example: "MM DD YY"; only in MDY order.
// ^^
- // Example: "Month DD YY"; only in MDY mode
+ // Example: "Month DD YY"; only in MDY order
// ^^
- // Example: "DD Month YY"; only in DMY mode
+ // Example: "DD Month YY"; only in DMY order
// ^^
- // WARNING: "YY Month DD"; OK in YMD mode. In other modes, we'll
+ // WARNING: "YY Month DD"; OK in YMD order. In other orders, we'll
// ^^ wind up storing the year in the day.
// This is fixed up below.
// The month has been set, but we don't yet have a year. If we know
// that the month was set in the first phase, we'll look for an
- // obvious year or defer to the parsing mode.
- if textMonth && (chunk.magnitude >= 3 || fe.mode == ParseModeYMD) {
+ // obvious year or defer to the parsing order.
+ if textMonth && (chunk.magnitude >= 3 || fe.dateStyle.Order == Order_YMD) {
if chunk.magnitude <= 2 {
fe.tweakYear = true
}
diff --git a/pkg/util/timeutil/pgdate/field_extract_test.go b/pkg/util/timeutil/pgdate/field_extract_test.go
index bcc0b2223b68..ae31e6131945 100644
--- a/pkg/util/timeutil/pgdate/field_extract_test.go
+++ b/pkg/util/timeutil/pgdate/field_extract_test.go
@@ -42,7 +42,7 @@ func TestExtractRelative(t *testing.T) {
now := time.Date(2018, 10, 17, 0, 0, 0, 0, time.UTC)
for _, tc := range tests {
t.Run(tc.s, func(t *testing.T) {
- d, depOnCtx, err := ParseDate(now, ParseModeYMD, tc.s)
+ d, depOnCtx, err := ParseDate(now, DateStyle{Order: Order_YMD}, tc.s)
if err != nil {
t.Fatal(err)
}
diff --git a/pkg/util/timeutil/pgdate/parsemode_string.go b/pkg/util/timeutil/pgdate/parsemode_string.go
deleted file mode 100644
index e8b770b597b5..000000000000
--- a/pkg/util/timeutil/pgdate/parsemode_string.go
+++ /dev/null
@@ -1,25 +0,0 @@
-// Code generated by "stringer -type=ParseMode"; DO NOT EDIT.
-
-package pgdate
-
-import "strconv"
-
-func _() {
- // An "invalid array index" compiler error signifies that the constant values have changed.
- // Re-run the stringer command to generate them again.
- var x [1]struct{}
- _ = x[ParseModeYMD-0]
- _ = x[ParseModeDMY-1]
- _ = x[ParseModeMDY-2]
-}
-
-const _ParseMode_name = "ParseModeYMDParseModeDMYParseModeMDY"
-
-var _ParseMode_index = [...]uint8{0, 12, 24, 36}
-
-func (i ParseMode) String() string {
- if i >= ParseMode(len(_ParseMode_index)-1) {
- return "ParseMode(" + strconv.FormatInt(int64(i), 10) + ")"
- }
- return _ParseMode_name[_ParseMode_index[i]:_ParseMode_index[i+1]]
-}
diff --git a/pkg/util/timeutil/pgdate/parsing.go b/pkg/util/timeutil/pgdate/parsing.go
index 91b6815226b2..7363766397c7 100644
--- a/pkg/util/timeutil/pgdate/parsing.go
+++ b/pkg/util/timeutil/pgdate/parsing.go
@@ -11,6 +11,7 @@
package pgdate
import (
+ "strings"
"time"
"github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgcode"
@@ -81,21 +82,6 @@ var (
TimeNegativeInfinity = timeutil.Unix(-210866803200, 0)
)
-//go:generate stringer -type=ParseMode
-
-// ParseMode controls the resolution of ambiguous date formats such as
-// `01/02/03`.
-type ParseMode uint
-
-// These are the various parsing modes that determine in which order
-// we should look for years, months, and date.
-// ParseModeYMD is the default value.
-const (
- ParseModeYMD ParseMode = iota
- ParseModeDMY
- ParseModeMDY
-)
-
// ParseDate converts a string into Date.
//
// Any specified timezone is inconsequential. Examples:
@@ -106,10 +92,12 @@ const (
// The dependsOnContext return value indicates if we had to consult the given
// `now` value (either for the time or the local timezone).
//
-func ParseDate(now time.Time, mode ParseMode, s string) (_ Date, dependsOnContext bool, _ error) {
+func ParseDate(
+ now time.Time, dateStyle DateStyle, s string,
+) (_ Date, dependsOnContext bool, _ error) {
fe := fieldExtract{
currentTime: now,
- mode: mode,
+ dateStyle: dateStyle,
required: dateRequiredFields,
// We allow time fields to be provided since they occur after
// the date fields that we're really looking for and for
@@ -129,7 +117,7 @@ func ParseDate(now time.Time, mode ParseMode, s string) (_ Date, dependsOnContex
// The dependsOnContext return value indicates if we had to consult the given
// `now` value (either for the time or the local timezone).
func ParseTime(
- now time.Time, mode ParseMode, s string,
+ now time.Time, dateStyle DateStyle, s string,
) (_ time.Time, dependsOnContext bool, _ error) {
fe := fieldExtract{
currentTime: now,
@@ -142,7 +130,7 @@ func ParseTime(
// timestamp string; let's try again, accepting more fields.
fe = fieldExtract{
currentTime: now,
- mode: mode,
+ dateStyle: dateStyle,
required: timeRequiredFields,
wanted: dateTimeFields,
}
@@ -166,7 +154,7 @@ func ParseTime(
// The dependsOnContext return value indicates if we had to consult the given
// `now` value (either for the time or the local timezone).
func ParseTimeWithoutTimezone(
- now time.Time, mode ParseMode, s string,
+ now time.Time, dateStyle DateStyle, s string,
) (_ time.Time, dependsOnContext bool, _ error) {
fe := fieldExtract{
currentTime: now,
@@ -179,7 +167,7 @@ func ParseTimeWithoutTimezone(
// timestamp string; let's try again, accepting more fields.
fe = fieldExtract{
currentTime: now,
- mode: mode,
+ dateStyle: dateStyle,
required: timeRequiredFields,
wanted: dateTimeFields,
}
@@ -197,10 +185,10 @@ func ParseTimeWithoutTimezone(
// The dependsOnContext return value indicates if we had to consult the given
// `now` value (either for the time or the local timezone).
func ParseTimestamp(
- now time.Time, mode ParseMode, s string,
+ now time.Time, dateStyle DateStyle, s string,
) (_ time.Time, dependsOnContext bool, _ error) {
fe := fieldExtract{
- mode: mode,
+ dateStyle: dateStyle,
currentTime: now,
// A timestamp only actually needs a date component; the time
// would be midnight.
@@ -229,10 +217,10 @@ func ParseTimestamp(
// The dependsOnContext return value indicates if we had to consult the given
// `now` value (either for the time or the local timezone).
func ParseTimestampWithoutTimezone(
- now time.Time, mode ParseMode, s string,
+ now time.Time, dateStyle DateStyle, s string,
) (_ time.Time, dependsOnContext bool, _ error) {
fe := fieldExtract{
- mode: mode,
+ dateStyle: dateStyle,
currentTime: now,
// A timestamp only actually needs a date component; the time
// would be midnight.
@@ -262,7 +250,10 @@ func inputErrorf(format string, args ...interface{}) error {
// outOfRangeError returns an error with pg code DatetimeFieldOverflow.
func outOfRangeError(field string, val int) error {
err := errors.Newf("field %s value %d is out of range", errors.Safe(field), errors.Safe(val))
- return pgerror.WithCandidateCode(err, pgcode.DatetimeFieldOverflow)
+ return errors.WithHint(
+ pgerror.WithCandidateCode(err, pgcode.DatetimeFieldOverflow),
+ `Perhaps you need a different "datestyle" setting.`,
+ )
}
// parseError ensures that any error we return to the client will
@@ -272,3 +263,41 @@ func parseError(err error, kind string, s string) error {
errors.Wrapf(err, "parsing as type %s", errors.Safe(kind)),
pgcode.InvalidDatetimeFormat)
}
+
+// DefaultDateStyle returns the default datestyle for Postgres.
+func DefaultDateStyle() DateStyle {
+ return DateStyle{
+ Order: Order_MDY,
+ Style: Style_ISO,
+ }
+}
+
+// ParseDateStyle parses a given DateStyle, modifying the existingDateStyle
+// as appropriate. This is because specifying just Style or Order will leave
+// the other field unchanged.
+func ParseDateStyle(s string, existingDateStyle DateStyle) (DateStyle, error) {
+ ds := existingDateStyle
+ fields := strings.Split(s, ",")
+ for _, field := range fields {
+ field = strings.ToLower(strings.TrimSpace(field))
+ switch field {
+ case "iso":
+ ds.Style = Style_ISO
+ case "german":
+ ds.Style = Style_GERMAN
+ case "sql":
+ ds.Style = Style_SQL
+ case "postgres":
+ ds.Style = Style_POSTGRES
+ case "ymd":
+ ds.Order = Order_YMD
+ case "mdy":
+ ds.Order = Order_MDY
+ case "dmy":
+ ds.Order = Order_DMY
+ default:
+ return ds, pgerror.Newf(pgcode.InvalidParameterValue, "unknown DateStyle parameter: %s", field)
+ }
+ }
+ return ds, nil
+}
diff --git a/pkg/util/timeutil/pgdate/parsing_test.go b/pkg/util/timeutil/pgdate/parsing_test.go
index e37d59ba7677..b5f39dc54813 100644
--- a/pkg/util/timeutil/pgdate/parsing_test.go
+++ b/pkg/util/timeutil/pgdate/parsing_test.go
@@ -24,12 +24,6 @@ import (
_ "github.com/lib/pq"
)
-var modes = []pgdate.ParseMode{
- pgdate.ParseModeDMY,
- pgdate.ParseModeMDY,
- pgdate.ParseModeYMD,
-}
-
var db *gosql.DB
var dbString string
@@ -57,10 +51,10 @@ type timeData struct {
// This text contains a timezone, so we wouldn't expect to be
// able to combine it with another timezone-containing value.
hasTimezone bool
- // Override the expected value for a given ParseMode.
- modeExp map[pgdate.ParseMode]time.Time
- // Override the expected error for a given ParseMode.
- modeErr map[pgdate.ParseMode]bool
+ // Override the expected value for a given Order.
+ orderExp map[pgdate.Order]time.Time
+ // Override the expected error for a given Order.
+ orderErr map[pgdate.Order]bool
// Indicates that we don't implement a feature in PostgreSQL.
unimplemented bool
}
@@ -84,11 +78,11 @@ func (td timeData) concatTime(other timeData) timeData {
concatErr := other.err || td.expectConcatErr || other.expectConcatErr || (td.hasTimezone && other.hasTimezone)
- var concatModeExp map[pgdate.ParseMode]time.Time
- if td.modeExp != nil && !concatErr {
- concatModeExp = make(map[pgdate.ParseMode]time.Time, len(td.modeExp))
- for mode, date := range td.modeExp {
- concatModeExp[mode] = add(date, other.exp)
+ var concatOrderExp map[pgdate.Order]time.Time
+ if td.orderExp != nil && !concatErr {
+ concatOrderExp = make(map[pgdate.Order]time.Time, len(td.orderExp))
+ for order, date := range td.orderExp {
+ concatOrderExp[order] = add(date, other.exp)
}
}
@@ -105,27 +99,27 @@ func (td timeData) concatTime(other timeData) timeData {
expectCrossErr: td.expectCrossErr || other.expectCrossErr,
hasTimezone: td.hasTimezone || other.hasTimezone,
isRolloverTime: td.isRolloverTime || other.isRolloverTime,
- modeExp: concatModeExp,
- modeErr: td.modeErr,
+ orderExp: concatOrderExp,
+ orderErr: td.orderErr,
unimplemented: td.unimplemented || other.unimplemented,
}
}
-// expected returns the expected time or expected error condition for the mode.
-func (td timeData) expected(mode pgdate.ParseMode) (time.Time, bool) {
- if t, ok := td.modeExp[mode]; ok {
+// expected returns the expected time or expected error condition for the order.
+func (td timeData) expected(order pgdate.Order) (time.Time, bool) {
+ if t, ok := td.orderExp[order]; ok {
return t, false
}
- if _, ok := td.modeErr[mode]; ok {
+ if _, ok := td.orderErr[order]; ok {
return pgdate.TimeEpoch, true
}
return td.exp, td.err
}
-func (td timeData) testParseDate(t *testing.T, info string, mode pgdate.ParseMode) {
+func (td timeData) testParseDate(t *testing.T, info string, order pgdate.Order) {
info = fmt.Sprintf("%s ParseDate", info)
- exp, expErr := td.expected(mode)
- dt, _, err := pgdate.ParseDate(time.Time{}, mode, td.s)
+ exp, expErr := td.expected(order)
+ dt, _, err := pgdate.ParseDate(time.Time{}, pgdate.DateStyle{Order: order}, td.s)
res, _ := dt.ToTime()
// HACK: This is a format that parses as a date and timestamp,
@@ -141,18 +135,18 @@ func (td timeData) testParseDate(t *testing.T, info string, mode pgdate.ParseMod
check(t, info, exp, expErr, res, err)
- td.crossCheck(t, info, "date", td.s, mode, exp, expErr)
+ td.crossCheck(t, info, "date", td.s, order, exp, expErr)
}
-func (td timeData) testParseTime(t *testing.T, info string, mode pgdate.ParseMode) {
+func (td timeData) testParseTime(t *testing.T, info string, order pgdate.Order) {
info = fmt.Sprintf("%s ParseTime", info)
- exp, expErr := td.expected(mode)
- res, _, err := pgdate.ParseTime(time.Time{}, mode, td.s)
+ exp, expErr := td.expected(order)
+ res, _, err := pgdate.ParseTime(time.Time{}, pgdate.DateStyle{Order: order}, td.s)
// Weird times like 24:00:00 or 23:59:60 aren't allowed,
// unless there's also a date.
if td.isRolloverTime {
- _, _, err := pgdate.ParseDate(time.Time{}, mode, td.s)
+ _, _, err := pgdate.ParseDate(time.Time{}, pgdate.DateStyle{Order: order}, td.s)
expErr = err != nil
}
@@ -161,13 +155,13 @@ func (td timeData) testParseTime(t *testing.T, info string, mode pgdate.ParseMod
exp = time.Date(0, 1, 1, h, m, sec, td.exp.Nanosecond(), td.exp.Location())
check(t, info, exp, expErr, res, err)
- td.crossCheck(t, info, "timetz", td.s, mode, exp, expErr)
+ td.crossCheck(t, info, "timetz", td.s, order, exp, expErr)
}
-func (td timeData) testParseTimestamp(t *testing.T, info string, mode pgdate.ParseMode) {
+func (td timeData) testParseTimestamp(t *testing.T, info string, order pgdate.Order) {
info = fmt.Sprintf("%s ParseTimestamp", info)
- exp, expErr := td.expected(mode)
- res, _, err := pgdate.ParseTimestamp(time.Time{}, mode, td.s)
+ exp, expErr := td.expected(order)
+ res, _, err := pgdate.ParseTimestamp(time.Time{}, pgdate.DateStyle{Order: order}, td.s)
// HACK: This is a format that parses as a date and timestamp,
// but is not a time.
@@ -181,15 +175,15 @@ func (td timeData) testParseTimestamp(t *testing.T, info string, mode pgdate.Par
}
check(t, info, exp, expErr, res, err)
- td.crossCheck(t, info, "timestamptz", td.s, mode, exp, expErr)
+ td.crossCheck(t, info, "timestamptz", td.s, order, exp, expErr)
}
func (td timeData) testParseTimestampWithoutTimezone(
- t *testing.T, info string, mode pgdate.ParseMode,
+ t *testing.T, info string, order pgdate.Order,
) {
info = fmt.Sprintf("%s ParseTimestampWithoutTimezone", info)
- exp, expErr := td.expected(mode)
- res, _, err := pgdate.ParseTimestampWithoutTimezone(time.Time{}, mode, td.s)
+ exp, expErr := td.expected(order)
+ res, _, err := pgdate.ParseTimestampWithoutTimezone(time.Time{}, pgdate.DateStyle{Order: order}, td.s)
// HACK: This is a format that parses as a date and timestamp,
// but is not a time.
@@ -206,7 +200,7 @@ func (td timeData) testParseTimestampWithoutTimezone(
exp = exp.Add(time.Duration(offset) * time.Second).UTC()
check(t, info, exp, expErr, res, err)
- td.crossCheck(t, info, "timestamp", td.s, mode, exp, expErr)
+ td.crossCheck(t, info, "timestamp", td.s, order, exp, expErr)
}
var dateTestData = []timeData{
@@ -215,61 +209,61 @@ var dateTestData = []timeData{
// and with comments from
// https://www.postgresql.org/docs/10/static/datatype-datetime.html#DATATYPE-DATETIME-DATE-TABLE
{
- //January 8, 1999 unambiguous in any datestyle input mode
+ //January 8, 1999 unambiguous in any datestyle input order
s: "January 8, 1999",
exp: time.Date(1999, time.January, 8, 0, 0, 0, 0, time.UTC),
},
{
- //1999-01-08 ISO 8601; January 8 in any mode (recommended format)
+ //1999-01-08 ISO 8601; January 8 in any order (recommended format)
s: "1999-01-08",
exp: time.Date(1999, time.January, 8, 0, 0, 0, 0, time.UTC),
},
{
- //1999-01-18 ISO 8601; January 18 in any mode (recommended format)
+ //1999-01-18 ISO 8601; January 18 in any order (recommended format)
s: "1999-01-18",
exp: time.Date(1999, time.January, 18, 0, 0, 0, 0, time.UTC),
},
{
- //1/8/1999 January 8 in MDY mode; August 1 in DMY mode
+ //1/8/1999 January 8 in MDY order; August 1 in DMY order
s: "1/8/1999",
err: true,
- modeExp: map[pgdate.ParseMode]time.Time{
- pgdate.ParseModeDMY: time.Date(1999, time.August, 1, 0, 0, 0, 0, time.UTC),
- pgdate.ParseModeMDY: time.Date(1999, time.January, 8, 0, 0, 0, 0, time.UTC),
+ orderExp: map[pgdate.Order]time.Time{
+ pgdate.Order_DMY: time.Date(1999, time.August, 1, 0, 0, 0, 0, time.UTC),
+ pgdate.Order_MDY: time.Date(1999, time.January, 8, 0, 0, 0, 0, time.UTC),
},
},
{
- // 1/18/1999 January 18 in MDY mode; rejected in other modes
+ // 1/18/1999 January 18 in MDY order; rejected in other orders
s: "1/18/1999",
err: true,
- modeExp: map[pgdate.ParseMode]time.Time{
- pgdate.ParseModeMDY: time.Date(1999, time.January, 18, 0, 0, 0, 0, time.UTC),
+ orderExp: map[pgdate.Order]time.Time{
+ pgdate.Order_MDY: time.Date(1999, time.January, 18, 0, 0, 0, 0, time.UTC),
},
},
{
- // 18/1/1999 January 18 in DMY mode; rejected in other modes
+ // 18/1/1999 January 18 in DMY order; rejected in other orders
s: "18/1/1999",
err: true,
- modeExp: map[pgdate.ParseMode]time.Time{
- pgdate.ParseModeDMY: time.Date(1999, time.January, 18, 0, 0, 0, 0, time.UTC),
+ orderExp: map[pgdate.Order]time.Time{
+ pgdate.Order_DMY: time.Date(1999, time.January, 18, 0, 0, 0, 0, time.UTC),
},
},
{
- // 01/02/03 January 2, 2003 in MDY mode; February 1, 2003 in DMY mode; February 3, 2001 in YMD mode
+ // 01/02/03 January 2, 2003 in MDY order; February 1, 2003 in DMY order; February 3, 2001 in YMD order
s: "01/02/03",
- modeExp: map[pgdate.ParseMode]time.Time{
- pgdate.ParseModeYMD: time.Date(2001, time.February, 3, 0, 0, 0, 0, time.UTC),
- pgdate.ParseModeDMY: time.Date(2003, time.February, 1, 0, 0, 0, 0, time.UTC),
- pgdate.ParseModeMDY: time.Date(2003, time.January, 2, 0, 0, 0, 0, time.UTC),
+ orderExp: map[pgdate.Order]time.Time{
+ pgdate.Order_YMD: time.Date(2001, time.February, 3, 0, 0, 0, 0, time.UTC),
+ pgdate.Order_DMY: time.Date(2003, time.February, 1, 0, 0, 0, 0, time.UTC),
+ pgdate.Order_MDY: time.Date(2003, time.January, 2, 0, 0, 0, 0, time.UTC),
},
},
{
- // 19990108 ISO 8601; January 8, 1999 in any mode
+ // 19990108 ISO 8601; January 8, 1999 in any order
s: "19990108",
exp: time.Date(1999, time.January, 8, 0, 0, 0, 0, time.UTC),
},
{
- // 990108 ISO 8601; January 8, 1999 in any mode
+ // 990108 ISO 8601; January 8, 1999 in any order
s: "990108",
exp: time.Date(1999, time.January, 8, 0, 0, 0, 0, time.UTC),
},
@@ -290,53 +284,53 @@ var dateTestData = []timeData{
exp: time.Date(-98, time.January, 8, 0, 0, 0, 0, time.UTC),
// Failure confirmed in pg 10.5:
// https://github.com/postgres/postgres/blob/REL_10_5/src/test/regress/expected/date.out#L135
- modeErr: map[pgdate.ParseMode]bool{
- pgdate.ParseModeYMD: true,
+ orderErr: map[pgdate.Order]bool{
+ pgdate.Order_YMD: true,
},
},
{
- // 99-Jan-08 January 8 in YMD mode, else error
+ // 99-Jan-08 January 8 in YMD order, else error
s: "99-Jan-08",
err: true,
- modeExp: map[pgdate.ParseMode]time.Time{
- pgdate.ParseModeYMD: time.Date(1999, time.January, 8, 0, 0, 0, 0, time.UTC),
+ orderExp: map[pgdate.Order]time.Time{
+ pgdate.Order_YMD: time.Date(1999, time.January, 8, 0, 0, 0, 0, time.UTC),
},
},
{
- // 1999-Jan-08 January 8 in any mode
+ // 1999-Jan-08 January 8 in any order
s: "1999-Jan-08",
exp: time.Date(1999, time.January, 8, 0, 0, 0, 0, time.UTC),
},
{
- // 08-Jan-99 January 8, except error in YMD mode
+ // 08-Jan-99 January 8, except error in YMD order
s: "08-Jan-99",
exp: time.Date(1999, time.January, 8, 0, 0, 0, 0, time.UTC),
- modeErr: map[pgdate.ParseMode]bool{
- pgdate.ParseModeYMD: true,
+ orderErr: map[pgdate.Order]bool{
+ pgdate.Order_YMD: true,
},
},
{
- // 08-Jan-1999 January 8 in any mode
+ // 08-Jan-1999 January 8 in any order
s: "08-Jan-1999",
exp: time.Date(1999, time.January, 8, 0, 0, 0, 0, time.UTC),
},
{
- // Jan-08-99 January 8, except error in YMD mode
+ // Jan-08-99 January 8, except error in YMD order
s: "Jan-08-99",
exp: time.Date(1999, time.January, 8, 0, 0, 0, 0, time.UTC),
- modeErr: map[pgdate.ParseMode]bool{
- pgdate.ParseModeYMD: true,
+ orderErr: map[pgdate.Order]bool{
+ pgdate.Order_YMD: true,
},
},
{
- // Jan-08-1999 January 8 in any mode
+ // Jan-08-1999 January 8 in any order
s: "Jan-08-1999",
exp: time.Date(1999, time.January, 8, 0, 0, 0, 0, time.UTC),
},
{
- // 99-08-Jan Error in all modes, because 99 isn't obviously a year
- // and there's no YDM parse mode.
+ // 99-08-Jan Error in all orders, because 99 isn't obviously a year
+ // and there's no YDM parse order.
s: "99-08-Jan",
err: true,
},
@@ -754,7 +748,7 @@ func TestMain(m *testing.M) {
}
// TestParse does the following:
-// * For each parsing mode:
+// * For each parsing order:
// * Pick an example date input: 2018-01-01
// * Test ParseDate()
// * Pick an example time input: 12:34:56
@@ -766,33 +760,37 @@ func TestMain(m *testing.M) {
// * Pick an example time input:
// * Test ParseTime()
func TestParse(t *testing.T) {
- for _, mode := range modes {
- t.Run(mode.String(), func(t *testing.T) {
+ for _, order := range []pgdate.Order{
+ pgdate.Order_YMD,
+ pgdate.Order_DMY,
+ pgdate.Order_MDY,
+ } {
+ t.Run(order.String(), func(t *testing.T) {
for _, dtc := range dateTestData {
- dtc.testParseDate(t, dtc.s, mode)
+ dtc.testParseDate(t, dtc.s, order)
// Combine times with dates to create timestamps.
for _, ttc := range timeTestData {
info := fmt.Sprintf("%s %s", dtc.s, ttc.s)
tstc := dtc.concatTime(ttc)
- tstc.testParseDate(t, info, mode)
- tstc.testParseTime(t, info, mode)
- tstc.testParseTimestamp(t, info, mode)
- tstc.testParseTimestampWithoutTimezone(t, info, mode)
+ tstc.testParseDate(t, info, order)
+ tstc.testParseTime(t, info, order)
+ tstc.testParseTimestamp(t, info, order)
+ tstc.testParseTimestampWithoutTimezone(t, info, order)
}
}
// Test some other timestamps formats we can't create
// by just concatenating a date + time string.
for _, ttc := range timestampTestData {
- ttc.testParseTime(t, ttc.s, mode)
+ ttc.testParseTime(t, ttc.s, order)
}
})
}
t.Run("ParseTime", func(t *testing.T) {
for _, ttc := range timeTestData {
- ttc.testParseTime(t, ttc.s, 0 /* mode */)
+ ttc.testParseTime(t, ttc.s, 0 /* order */)
}
})
}
@@ -836,7 +834,7 @@ func bench(b *testing.B, layout string, s string, locationName string) {
b.RunParallel(func(pb *testing.PB) {
for pb.Next() {
- if _, _, err := pgdate.ParseTimestamp(time.Time{}, 0, benchS); err != nil {
+ if _, _, err := pgdate.ParseTimestamp(time.Time{}, pgdate.DateStyle{Order: pgdate.Order_MDY}, benchS); err != nil {
b.Fatal(err)
}
b.SetBytes(bytes)
@@ -884,7 +882,7 @@ func check(t testing.TB, info string, expTime time.Time, expErr bool, res time.T
// crossCheck executes the parsing on a remote sql connection.
func (td timeData) crossCheck(
- t *testing.T, info string, kind, s string, mode pgdate.ParseMode, expTime time.Time, expErr bool,
+ t *testing.T, info string, kind, s string, order pgdate.Order, expTime time.Time, expErr bool,
) {
if db == nil {
return
@@ -916,12 +914,12 @@ func (td timeData) crossCheck(
}
var style string
- switch mode {
- case pgdate.ParseModeMDY:
+ switch order {
+ case pgdate.Order_MDY:
style = "MDY"
- case pgdate.ParseModeDMY:
+ case pgdate.Order_DMY:
style = "DMY"
- case pgdate.ParseModeYMD:
+ case pgdate.Order_YMD:
style = "YMD"
}
if _, err := db.Exec(fmt.Sprintf("set datestyle='%s'", style)); err != nil {
@@ -1040,7 +1038,7 @@ func TestDependsOnContext(t *testing.T) {
}
now := time.Date(2001, time.February, 3, 4, 5, 6, 1000, time.FixedZone("foo", 18000))
- mode := pgdate.ParseModeYMD
+ order := pgdate.Order_YMD
for _, tc := range testCases {
t.Run(tc.s, func(t *testing.T) {
toStr := func(result interface{}, depOnCtx bool, err error) string {
@@ -1059,15 +1057,15 @@ func TestDependsOnContext(t *testing.T) {
t.Errorf("%s: expected '%s', got '%s'", what, expected, actual)
}
}
- check("ParseDate", tc.date, toStr(pgdate.ParseDate(now, mode, tc.s)))
- check("ParseTime", tc.time, toStr(pgdate.ParseTime(now, mode, tc.s)))
+ check("ParseDate", tc.date, toStr(pgdate.ParseDate(now, pgdate.DateStyle{Order: order}, tc.s)))
+ check("ParseTime", tc.time, toStr(pgdate.ParseTime(now, pgdate.DateStyle{Order: order}, tc.s)))
check(
"ParseTimeWithoutTimezone", tc.timeNoTZ,
- toStr(pgdate.ParseTimeWithoutTimezone(now, mode, tc.s)),
+ toStr(pgdate.ParseTimeWithoutTimezone(now, pgdate.DateStyle{Order: order}, tc.s)),
)
- check("ParseTimestamp", tc.timestamp, toStr(pgdate.ParseTimestamp(now, mode, tc.s)))
+ check("ParseTimestamp", tc.timestamp, toStr(pgdate.ParseTimestamp(now, pgdate.DateStyle{Order: order}, tc.s)))
check("ParseTimestampWithoutTimezone",
- tc.timestampNoTZ, toStr(pgdate.ParseTimestampWithoutTimezone(now, mode, tc.s)),
+ tc.timestampNoTZ, toStr(pgdate.ParseTimestampWithoutTimezone(now, pgdate.DateStyle{Order: order}, tc.s)),
)
})
}
diff --git a/pkg/util/timeutil/pgdate/pgdate.go b/pkg/util/timeutil/pgdate/pgdate.go
index 4a7f57053af0..9260c1d91696 100644
--- a/pkg/util/timeutil/pgdate/pgdate.go
+++ b/pkg/util/timeutil/pgdate/pgdate.go
@@ -251,3 +251,28 @@ func (d Date) SubDays(days int64) (Date, error) {
}
return MakeDateFromPGEpoch(n)
}
+
+// SQLString formats the Style into a SQL string.
+func (s Style) SQLString() string {
+ switch s {
+ case Style_POSTGRES:
+ return "Postgres"
+ case Style_GERMAN:
+ return "German"
+ }
+ return s.String()
+}
+
+// SQLString formats DateStyle into a SQL format.
+func (ds DateStyle) SQLString() string {
+ return fmt.Sprintf("%s, %s", ds.Style.SQLString(), ds.Order.String())
+}
+
+// AllowedDateStyles returns the list of allowed date styles.
+func AllowedDateStyles() []string {
+ var allowed []string
+ for _, order := range []Order{Order_MDY, Order_DMY, Order_YMD} {
+ allowed = append(allowed, fmt.Sprintf("%s", DateStyle{Style: Style_ISO, Order: order}))
+ }
+ return allowed
+}
diff --git a/pkg/util/timeutil/pgdate/pgdate.pb.go b/pkg/util/timeutil/pgdate/pgdate.pb.go
new file mode 100644
index 000000000000..f7a91e2e6e49
--- /dev/null
+++ b/pkg/util/timeutil/pgdate/pgdate.pb.go
@@ -0,0 +1,387 @@
+// Code generated by protoc-gen-gogo. DO NOT EDIT.
+// source: util/timeutil/pgdate/pgdate.proto
+
+package pgdate
+
+import (
+ fmt "fmt"
+ proto "github.com/gogo/protobuf/proto"
+ io "io"
+ math "math"
+ math_bits "math/bits"
+)
+
+// Reference imports to suppress errors if they are not otherwise used.
+var _ = proto.Marshal
+var _ = fmt.Errorf
+var _ = math.Inf
+
+// This is a compile-time assertion to ensure that this generated file
+// is compatible with the proto package it is being compiled against.
+// A compilation error at this line likely means your copy of the
+// proto package needs to be updated.
+const _ = proto.GoGoProtoPackageIsVersion3 // please upgrade the proto package
+
+// Order refers to the Order component of a DateStyle.
+type Order int32
+
+const (
+ Order_MDY Order = 0
+ Order_DMY Order = 1
+ Order_YMD Order = 2
+)
+
+var Order_name = map[int32]string{
+ 0: "MDY",
+ 1: "DMY",
+ 2: "YMD",
+}
+
+var Order_value = map[string]int32{
+ "MDY": 0,
+ "DMY": 1,
+ "YMD": 2,
+}
+
+func (x Order) String() string {
+ return proto.EnumName(Order_name, int32(x))
+}
+
+func (Order) EnumDescriptor() ([]byte, []int) {
+ return fileDescriptor_1038cae594af4e85, []int{0}
+}
+
+// Style refers to the Style component of a DateStyle.
+type Style int32
+
+const (
+ Style_ISO Style = 0
+ Style_SQL Style = 1
+ Style_POSTGRES Style = 2
+ Style_GERMAN Style = 3
+)
+
+var Style_name = map[int32]string{
+ 0: "ISO",
+ 1: "SQL",
+ 2: "POSTGRES",
+ 3: "GERMAN",
+}
+
+var Style_value = map[string]int32{
+ "ISO": 0,
+ "SQL": 1,
+ "POSTGRES": 2,
+ "GERMAN": 3,
+}
+
+func (x Style) String() string {
+ return proto.EnumName(Style_name, int32(x))
+}
+
+func (Style) EnumDescriptor() ([]byte, []int) {
+ return fileDescriptor_1038cae594af4e85, []int{1}
+}
+
+// DateStyle refers to the PostgreSQL DateStyle allowed variables.
+type DateStyle struct {
+ // Style refers to the style to print output dates.
+ Style Style `protobuf:"varint,1,opt,name=style,proto3,enum=cockroach.util.timeutil.pgdate.Style" json:"style,omitempty"`
+ // Order refers to the order of day, month and year components.
+ Order Order `protobuf:"varint,2,opt,name=order,proto3,enum=cockroach.util.timeutil.pgdate.Order" json:"order,omitempty"`
+}
+
+func (m *DateStyle) Reset() { *m = DateStyle{} }
+func (m *DateStyle) String() string { return proto.CompactTextString(m) }
+func (*DateStyle) ProtoMessage() {}
+func (*DateStyle) Descriptor() ([]byte, []int) {
+ return fileDescriptor_1038cae594af4e85, []int{0}
+}
+func (m *DateStyle) XXX_Unmarshal(b []byte) error {
+ return m.Unmarshal(b)
+}
+func (m *DateStyle) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+ b = b[:cap(b)]
+ n, err := m.MarshalToSizedBuffer(b)
+ if err != nil {
+ return nil, err
+ }
+ return b[:n], nil
+}
+func (m *DateStyle) XXX_Merge(src proto.Message) {
+ xxx_messageInfo_DateStyle.Merge(m, src)
+}
+func (m *DateStyle) XXX_Size() int {
+ return m.Size()
+}
+func (m *DateStyle) XXX_DiscardUnknown() {
+ xxx_messageInfo_DateStyle.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_DateStyle proto.InternalMessageInfo
+
+func init() {
+ proto.RegisterEnum("cockroach.util.timeutil.pgdate.Order", Order_name, Order_value)
+ proto.RegisterEnum("cockroach.util.timeutil.pgdate.Style", Style_name, Style_value)
+ proto.RegisterType((*DateStyle)(nil), "cockroach.util.timeutil.pgdate.DateStyle")
+}
+
+func init() { proto.RegisterFile("util/timeutil/pgdate/pgdate.proto", fileDescriptor_1038cae594af4e85) }
+
+var fileDescriptor_1038cae594af4e85 = []byte{
+ // 245 bytes of a gzipped FileDescriptorProto
+ 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xe2, 0x52, 0x2c, 0x2d, 0xc9, 0xcc,
+ 0xd1, 0x2f, 0xc9, 0xcc, 0x4d, 0x05, 0x33, 0x0a, 0xd2, 0x53, 0x12, 0x4b, 0x52, 0xa1, 0x94, 0x5e,
+ 0x41, 0x51, 0x7e, 0x49, 0xbe, 0x90, 0x5c, 0x72, 0x7e, 0x72, 0x76, 0x51, 0x7e, 0x62, 0x72, 0x86,
+ 0x1e, 0x48, 0x8d, 0x1e, 0x4c, 0xb1, 0x1e, 0x44, 0x95, 0x52, 0x2b, 0x23, 0x17, 0xa7, 0x4b, 0x62,
+ 0x49, 0x6a, 0x70, 0x49, 0x65, 0x4e, 0xaa, 0x90, 0x35, 0x17, 0x6b, 0x31, 0x88, 0x21, 0xc1, 0xa8,
+ 0xc0, 0xa8, 0xc1, 0x67, 0xa4, 0xaa, 0x87, 0x5f, 0xb7, 0x1e, 0x58, 0x57, 0x10, 0x44, 0x0f, 0x48,
+ 0x73, 0x7e, 0x51, 0x4a, 0x6a, 0x91, 0x04, 0x13, 0x71, 0x9a, 0xfd, 0x41, 0x8a, 0x83, 0x20, 0x7a,
+ 0xb4, 0x94, 0xb8, 0x58, 0xc1, 0x7c, 0x21, 0x76, 0x2e, 0x66, 0x5f, 0x97, 0x48, 0x01, 0x06, 0x10,
+ 0xc3, 0xc5, 0x37, 0x52, 0x80, 0x11, 0xc4, 0x88, 0xf4, 0x75, 0x11, 0x60, 0xd2, 0x32, 0xe6, 0x62,
+ 0x85, 0x38, 0x93, 0x9d, 0x8b, 0xd9, 0x33, 0xd8, 0x1f, 0xa2, 0x26, 0x38, 0xd0, 0x47, 0x80, 0x51,
+ 0x88, 0x87, 0x8b, 0x23, 0xc0, 0x3f, 0x38, 0xc4, 0x3d, 0xc8, 0x35, 0x58, 0x80, 0x49, 0x88, 0x8b,
+ 0x8b, 0xcd, 0xdd, 0x35, 0xc8, 0xd7, 0xd1, 0x4f, 0x80, 0xd9, 0x49, 0xe3, 0xc4, 0x43, 0x39, 0x86,
+ 0x13, 0x8f, 0xe4, 0x18, 0x2f, 0x3c, 0x92, 0x63, 0xbc, 0xf1, 0x48, 0x8e, 0xf1, 0xc1, 0x23, 0x39,
+ 0xc6, 0x09, 0x8f, 0xe5, 0x18, 0x2e, 0x3c, 0x96, 0x63, 0xb8, 0xf1, 0x58, 0x8e, 0x21, 0x8a, 0x0d,
+ 0xe2, 0x9e, 0x24, 0x36, 0x70, 0x88, 0x19, 0x03, 0x02, 0x00, 0x00, 0xff, 0xff, 0x7a, 0x4f, 0x44,
+ 0x0b, 0x56, 0x01, 0x00, 0x00,
+}
+
+func (m *DateStyle) Marshal() (dAtA []byte, err error) {
+ size := m.Size()
+ dAtA = make([]byte, size)
+ n, err := m.MarshalToSizedBuffer(dAtA[:size])
+ if err != nil {
+ return nil, err
+ }
+ return dAtA[:n], nil
+}
+
+func (m *DateStyle) MarshalTo(dAtA []byte) (int, error) {
+ size := m.Size()
+ return m.MarshalToSizedBuffer(dAtA[:size])
+}
+
+func (m *DateStyle) MarshalToSizedBuffer(dAtA []byte) (int, error) {
+ i := len(dAtA)
+ _ = i
+ var l int
+ _ = l
+ if m.Order != 0 {
+ i = encodeVarintPgdate(dAtA, i, uint64(m.Order))
+ i--
+ dAtA[i] = 0x10
+ }
+ if m.Style != 0 {
+ i = encodeVarintPgdate(dAtA, i, uint64(m.Style))
+ i--
+ dAtA[i] = 0x8
+ }
+ return len(dAtA) - i, nil
+}
+
+func encodeVarintPgdate(dAtA []byte, offset int, v uint64) int {
+ offset -= sovPgdate(v)
+ base := offset
+ for v >= 1<<7 {
+ dAtA[offset] = uint8(v&0x7f | 0x80)
+ v >>= 7
+ offset++
+ }
+ dAtA[offset] = uint8(v)
+ return base
+}
+func (m *DateStyle) Size() (n int) {
+ if m == nil {
+ return 0
+ }
+ var l int
+ _ = l
+ if m.Style != 0 {
+ n += 1 + sovPgdate(uint64(m.Style))
+ }
+ if m.Order != 0 {
+ n += 1 + sovPgdate(uint64(m.Order))
+ }
+ return n
+}
+
+func sovPgdate(x uint64) (n int) {
+ return (math_bits.Len64(x|1) + 6) / 7
+}
+func sozPgdate(x uint64) (n int) {
+ return sovPgdate(uint64((x << 1) ^ uint64((int64(x) >> 63))))
+}
+func (m *DateStyle) Unmarshal(dAtA []byte) error {
+ l := len(dAtA)
+ iNdEx := 0
+ for iNdEx < l {
+ preIndex := iNdEx
+ var wire uint64
+ for shift := uint(0); ; shift += 7 {
+ if shift >= 64 {
+ return ErrIntOverflowPgdate
+ }
+ if iNdEx >= l {
+ return io.ErrUnexpectedEOF
+ }
+ b := dAtA[iNdEx]
+ iNdEx++
+ wire |= uint64(b&0x7F) << shift
+ if b < 0x80 {
+ break
+ }
+ }
+ fieldNum := int32(wire >> 3)
+ wireType := int(wire & 0x7)
+ if wireType == 4 {
+ return fmt.Errorf("proto: DateStyle: wiretype end group for non-group")
+ }
+ if fieldNum <= 0 {
+ return fmt.Errorf("proto: DateStyle: illegal tag %d (wire type %d)", fieldNum, wire)
+ }
+ switch fieldNum {
+ case 1:
+ if wireType != 0 {
+ return fmt.Errorf("proto: wrong wireType = %d for field Style", wireType)
+ }
+ m.Style = 0
+ for shift := uint(0); ; shift += 7 {
+ if shift >= 64 {
+ return ErrIntOverflowPgdate
+ }
+ if iNdEx >= l {
+ return io.ErrUnexpectedEOF
+ }
+ b := dAtA[iNdEx]
+ iNdEx++
+ m.Style |= Style(b&0x7F) << shift
+ if b < 0x80 {
+ break
+ }
+ }
+ case 2:
+ if wireType != 0 {
+ return fmt.Errorf("proto: wrong wireType = %d for field Order", wireType)
+ }
+ m.Order = 0
+ for shift := uint(0); ; shift += 7 {
+ if shift >= 64 {
+ return ErrIntOverflowPgdate
+ }
+ if iNdEx >= l {
+ return io.ErrUnexpectedEOF
+ }
+ b := dAtA[iNdEx]
+ iNdEx++
+ m.Order |= Order(b&0x7F) << shift
+ if b < 0x80 {
+ break
+ }
+ }
+ default:
+ iNdEx = preIndex
+ skippy, err := skipPgdate(dAtA[iNdEx:])
+ if err != nil {
+ return err
+ }
+ if (skippy < 0) || (iNdEx+skippy) < 0 {
+ return ErrInvalidLengthPgdate
+ }
+ if (iNdEx + skippy) > l {
+ return io.ErrUnexpectedEOF
+ }
+ iNdEx += skippy
+ }
+ }
+
+ if iNdEx > l {
+ return io.ErrUnexpectedEOF
+ }
+ return nil
+}
+func skipPgdate(dAtA []byte) (n int, err error) {
+ l := len(dAtA)
+ iNdEx := 0
+ depth := 0
+ for iNdEx < l {
+ var wire uint64
+ for shift := uint(0); ; shift += 7 {
+ if shift >= 64 {
+ return 0, ErrIntOverflowPgdate
+ }
+ if iNdEx >= l {
+ return 0, io.ErrUnexpectedEOF
+ }
+ b := dAtA[iNdEx]
+ iNdEx++
+ wire |= (uint64(b) & 0x7F) << shift
+ if b < 0x80 {
+ break
+ }
+ }
+ wireType := int(wire & 0x7)
+ switch wireType {
+ case 0:
+ for shift := uint(0); ; shift += 7 {
+ if shift >= 64 {
+ return 0, ErrIntOverflowPgdate
+ }
+ if iNdEx >= l {
+ return 0, io.ErrUnexpectedEOF
+ }
+ iNdEx++
+ if dAtA[iNdEx-1] < 0x80 {
+ break
+ }
+ }
+ case 1:
+ iNdEx += 8
+ case 2:
+ var length int
+ for shift := uint(0); ; shift += 7 {
+ if shift >= 64 {
+ return 0, ErrIntOverflowPgdate
+ }
+ if iNdEx >= l {
+ return 0, io.ErrUnexpectedEOF
+ }
+ b := dAtA[iNdEx]
+ iNdEx++
+ length |= (int(b) & 0x7F) << shift
+ if b < 0x80 {
+ break
+ }
+ }
+ if length < 0 {
+ return 0, ErrInvalidLengthPgdate
+ }
+ iNdEx += length
+ case 3:
+ depth++
+ case 4:
+ if depth == 0 {
+ return 0, ErrUnexpectedEndOfGroupPgdate
+ }
+ depth--
+ case 5:
+ iNdEx += 4
+ default:
+ return 0, fmt.Errorf("proto: illegal wireType %d", wireType)
+ }
+ if iNdEx < 0 {
+ return 0, ErrInvalidLengthPgdate
+ }
+ if depth == 0 {
+ return iNdEx, nil
+ }
+ }
+ return 0, io.ErrUnexpectedEOF
+}
+
+var (
+ ErrInvalidLengthPgdate = fmt.Errorf("proto: negative length found during unmarshaling")
+ ErrIntOverflowPgdate = fmt.Errorf("proto: integer overflow")
+ ErrUnexpectedEndOfGroupPgdate = fmt.Errorf("proto: unexpected end of group")
+)
diff --git a/pkg/util/timeutil/pgdate/pgdate.proto b/pkg/util/timeutil/pgdate/pgdate.proto
new file mode 100644
index 000000000000..fb4d9525c5c4
--- /dev/null
+++ b/pkg/util/timeutil/pgdate/pgdate.proto
@@ -0,0 +1,36 @@
+// Copyright 2021 The Cockroach Authors.
+//
+// Use of this software is governed by the Business Source License
+// included in the file licenses/BSL.txt.
+//
+// As of the Change Date specified in that file, in accordance with
+// the Business Source License, use of this software will be governed
+// by the Apache License, Version 2.0, included in the file
+// licenses/APL.txt.
+
+syntax = "proto3";
+package cockroach.util.timeutil.pgdate;
+option go_package = "pgdate";
+
+// DateStyle refers to the PostgreSQL DateStyle allowed variables.
+message DateStyle {
+ // Style refers to the style to print output dates.
+ Style style = 1;
+ // Order refers to the order of day, month and year components.
+ Order order = 2;
+}
+
+// Order refers to the Order component of a DateStyle.
+enum Order {
+ MDY = 0;
+ DMY = 1;
+ YMD = 2;
+}
+
+// Style refers to the Style component of a DateStyle.
+enum Style {
+ ISO = 0;
+ SQL = 1;
+ POSTGRES = 2;
+ GERMAN = 3;
+}
diff --git a/pkg/util/timeutil/pgdate/pgdate_test.go b/pkg/util/timeutil/pgdate/pgdate_test.go
index 23f82724c222..5147eb68f8a3 100644
--- a/pkg/util/timeutil/pgdate/pgdate_test.go
+++ b/pkg/util/timeutil/pgdate/pgdate_test.go
@@ -16,6 +16,8 @@ import (
"strings"
"testing"
"time"
+
+ "github.com/stretchr/testify/require"
)
func TestParseDate(t *testing.T) {
@@ -70,7 +72,7 @@ func TestParseDate(t *testing.T) {
},
} {
t.Run(tc.s, func(t *testing.T) {
- d, depOnCtx, err := ParseDate(time.Time{}, ParseModeYMD, tc.s)
+ d, depOnCtx, err := ParseDate(time.Time{}, DateStyle{Order: Order_YMD}, tc.s)
if tc.err != "" {
if err == nil || !strings.Contains(err.Error(), tc.err) {
t.Fatalf("got %v, expected %v", err, tc.err)
@@ -161,3 +163,37 @@ func TestMakeDateFromTime(t *testing.T) {
})
}
}
+
+func TestParseDateStyle(t *testing.T) {
+ for _, tc := range []struct {
+ initial DateStyle
+ parse string
+ expected DateStyle
+ sqlString string
+ }{
+ {DefaultDateStyle(), "mdy", DateStyle{Style: Style_ISO, Order: Order_MDY}, "ISO, MDY"},
+ {DefaultDateStyle(), "dmy", DateStyle{Style: Style_ISO, Order: Order_DMY}, "ISO, DMY"},
+ {DefaultDateStyle(), "ymd", DateStyle{Style: Style_ISO, Order: Order_YMD}, "ISO, YMD"},
+
+ {DefaultDateStyle(), "iso", DateStyle{Style: Style_ISO, Order: Order_MDY}, "ISO, MDY"},
+ {DefaultDateStyle(), "german", DateStyle{Style: Style_GERMAN, Order: Order_MDY}, "German, MDY"},
+ {DefaultDateStyle(), "sQl ", DateStyle{Style: Style_SQL, Order: Order_MDY}, "SQL, MDY"},
+ {DefaultDateStyle(), "postgres", DateStyle{Style: Style_POSTGRES, Order: Order_MDY}, "Postgres, MDY"},
+
+ {DefaultDateStyle(), "german, DMY", DateStyle{Style: Style_GERMAN, Order: Order_DMY}, "German, DMY"},
+ {DefaultDateStyle(), "ymd,sql", DateStyle{Style: Style_SQL, Order: Order_YMD}, "SQL, YMD"},
+ {DateStyle{Style: Style_GERMAN, Order: Order_DMY}, "sql,ymd,postgres", DateStyle{Style: Style_POSTGRES, Order: Order_YMD}, "Postgres, YMD"},
+ } {
+ t.Run(fmt.Sprintf("%s/%s", tc.initial.String(), tc.parse), func(t *testing.T) {
+ p, err := ParseDateStyle(tc.parse, tc.initial)
+ require.NoError(t, err)
+ require.Equal(t, tc.expected, p)
+ require.Equal(t, tc.sqlString, p.SQLString())
+ })
+ }
+
+ t.Run("error", func(t *testing.T) {
+ _, err := ParseDateStyle("bad", DefaultDateStyle())
+ require.Error(t, err)
+ })
+}