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.enabledbooleanfalseif true, creating sequences owned by tables from other databases is allowed sql.cross_db_views.enabledbooleanfalseif true, creating views that refer to other databases is allowed sql.defaults.copy_partitioning_when_deinterleaving_table.enabledbooleanfalsedefault value for enable_copying_partitioning_when_deinterleaving_table session variable +sql.defaults.datestyleenumerationiso, mdydefault value for DateStyle session setting [iso, mdy = 0, iso, dmy = 1, iso, ymd = 2] sql.defaults.default_int_sizeinteger8the size, in bytes, of an INT type sql.defaults.disallow_full_table_scans.enabledbooleanfalsesetting to true rejects queries that have planned a full table scan sql.defaults.distsqlenumerationautodefault 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) + }) +}