Skip to content

Commit

Permalink
Merge #68093 #68289
Browse files Browse the repository at this point in the history
68093: sql: allow YMD, DMY for DateStyle as a session variable r=rafiss a=otan

Refs: #41773

See individual commits for details.

68289: colexec: fix LIKE operators when patterns have escape characters r=yuzefovich a=yuzefovich

**colexec: fix LIKE operators when patterns have escape characters**

Fixes: #68040.

Release note (bug fix): Previously, CockroachDB could incorrectly
evaluate LIKE expressions when the pattern contained the escape
characters `\` if the expressions were executed via the vectorized
engine.

**colbuilder: force planning of optimized projection operators**

Whenever we're planning a projection expression, we have 3 cases: the
left is constant, the right is constants, and neither are constants. For
the second case we have some optimized operators. Previously, those
operators weren't exercised via the TestEval/vectorized because in the
eval tests the left side is constant. This commit switches the planning
to force planning of those optimized operators. This shouldn't really
have an effect on planning of actual queries.

This was prompted by the bug in LIKE operators that is fixed in the
previous commit. Had we forced the planning for our eval tests, we would
have caught it earlier.

This also revealed an incompatibility for our IN operator implementation when
the right side is an empty tuple which this commit also fixes. However,
I don't think this scenario can be hit in production because the
optimizer folds such an expression into correct `false`. Thus, there is
no release note.

Release note: None

Co-authored-by: Oliver Tan <[email protected]>
Co-authored-by: Yahor Yuzefovich <[email protected]>
  • Loading branch information
3 people committed Aug 2, 2021
3 parents 635c263 + 0d6c10e + 9b590d3 commit 9501adb
Show file tree
Hide file tree
Showing 44 changed files with 1,163 additions and 305 deletions.
1 change: 1 addition & 0 deletions docs/generated/settings/settings-for-tenants.txt
Original file line number Diff line number Diff line change
Expand Up @@ -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]
Expand Down
1 change: 1 addition & 0 deletions docs/generated/settings/settings.html
Original file line number Diff line number Diff line change
Expand Up @@ -75,6 +75,7 @@
<tr><td><code>sql.cross_db_sequence_owners.enabled</code></td><td>boolean</td><td><code>false</code></td><td>if true, creating sequences owned by tables from other databases is allowed</td></tr>
<tr><td><code>sql.cross_db_views.enabled</code></td><td>boolean</td><td><code>false</code></td><td>if true, creating views that refer to other databases is allowed</td></tr>
<tr><td><code>sql.defaults.copy_partitioning_when_deinterleaving_table.enabled</code></td><td>boolean</td><td><code>false</code></td><td>default value for enable_copying_partitioning_when_deinterleaving_table session variable</td></tr>
<tr><td><code>sql.defaults.datestyle</code></td><td>enumeration</td><td><code>iso, mdy</code></td><td>default value for DateStyle session setting [iso, mdy = 0, iso, dmy = 1, iso, ymd = 2]</td></tr>
<tr><td><code>sql.defaults.default_int_size</code></td><td>integer</td><td><code>8</code></td><td>the size, in bytes, of an INT type</td></tr>
<tr><td><code>sql.defaults.disallow_full_table_scans.enabled</code></td><td>boolean</td><td><code>false</code></td><td>setting to true rejects queries that have planned a full table scan</td></tr>
<tr><td><code>sql.defaults.distsql</code></td><td>enumeration</td><td><code>auto</code></td><td>default distributed SQL execution mode [off = 0, auto = 1, on = 2]</td></tr>
Expand Down
2 changes: 1 addition & 1 deletion pkg/ccl/cliccl/debug_backup.go
Original file line number Diff line number Diff line change
Expand Up @@ -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
}
Expand Down
1 change: 1 addition & 0 deletions pkg/sql/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -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",
Expand Down
40 changes: 24 additions & 16 deletions pkg/sql/colexec/colbuilder/execplan.go
Original file line number Diff line number Diff line change
Expand Up @@ -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)
Expand Down Expand Up @@ -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
Expand Down Expand Up @@ -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.
Expand Down Expand Up @@ -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(
Expand Down Expand Up @@ -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
}
Expand Down
15 changes: 12 additions & 3 deletions pkg/sql/colexec/colexeccmp/like_ops.go
Original file line number Diff line number Diff line change
Expand Up @@ -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) {
Expand Down
2 changes: 2 additions & 0 deletions pkg/sql/conn_executor.go
Original file line number Diff line number Diff line change
Expand Up @@ -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)

Expand Down
22 changes: 22 additions & 0 deletions pkg/sql/exec_util.go
Original file line number Diff line number Diff line change
Expand Up @@ -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"
Expand Down Expand Up @@ -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.
Expand Down Expand Up @@ -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.
Expand Down
85 changes: 81 additions & 4 deletions pkg/sql/logictest/testdata/logic_test/datetime
Original file line number Diff line number Diff line change
Expand Up @@ -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.
Expand Down Expand Up @@ -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
15 changes: 15 additions & 0 deletions pkg/sql/logictest/testdata/logic_test/interval
Original file line number Diff line number Diff line change
Expand Up @@ -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 )

Expand Down Expand Up @@ -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
----
Expand Down Expand Up @@ -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
----
Expand Down
40 changes: 40 additions & 0 deletions pkg/sql/logictest/testdata/logic_test/set
Original file line number Diff line number Diff line change
Expand Up @@ -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'

Expand Down
9 changes: 9 additions & 0 deletions pkg/sql/logictest/testdata/logic_test/vectorize
Original file line number Diff line number Diff line change
Expand Up @@ -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
Loading

0 comments on commit 9501adb

Please sign in to comment.