Skip to content

Commit

Permalink
tree: permit constant upcasting to string arrays
Browse files Browse the repository at this point in the history
Release note: None
  • Loading branch information
jordanlewis committed Jul 19, 2019
1 parent 603c904 commit e86e27a
Show file tree
Hide file tree
Showing 10 changed files with 90 additions and 32 deletions.
7 changes: 6 additions & 1 deletion pkg/sql/logictest/testdata/logic_test/array
Original file line number Diff line number Diff line change
Expand Up @@ -862,11 +862,16 @@ DROP TABLE a

# Element append

# Postgres also requires that the string be explicitly casted, so we're no worse
# for wear despite this being a little annoying.
query T
SELECT ARRAY['a','b','c'] || 'd'
SELECT ARRAY['a','b','c'] || 'd'::text
----
{a,b,c,d}

query error unsupported binary operator
SELECT ARRAY['a','b','c'] || 'd'

query T
SELECT ARRAY[1,2,3] || 4
----
Expand Down
8 changes: 7 additions & 1 deletion pkg/sql/logictest/testdata/logic_test/json_builtins
Original file line number Diff line number Diff line change
Expand Up @@ -1003,11 +1003,17 @@ SELECT row_to_json(row(1,'foo')), row_to_json(NULL), row_to_json(row())
{"f1": 1, "f2": "foo"} NULL {}


# TODO(jordan,radu): this should also work with the .*.
# TODO(jordan,radu): this should also work without the .*.
query T
select row_to_json(t.*)
from (
select 1 as a, 2 as b
) t
----
{"a": 1, "b": 2}

query T
SELECT '["a", {"b":1}]'::jsonb #- '{1,b}'
----
["a", {}]

8 changes: 7 additions & 1 deletion pkg/sql/logictest/testdata/logic_test/suboperators
Original file line number Diff line number Diff line change
Expand Up @@ -110,6 +110,12 @@ query error unsupported comparison operator: 1 = ANY ARRAY\['foo', 'bar'\]
SELECT 1 = ANY(ARRAY['foo', 'bar'])

query error unsupported comparison operator: <int> = ANY <string\[\]>
SELECT 1 = ANY(ARRAY['foo'] || 'bar'::string)

# Note that this relatively poor error message is caused by the fact that
# strings are constant castable to string arrays. Postgres also makes this
# same minor mistake in error generation.
query error unsupported binary operator: <string\[\]> || <string> (desired <int\[\]>)
SELECT 1 = ANY(ARRAY['foo'] || 'bar')

# ANY/SOME with subqueries.
Expand Down Expand Up @@ -294,7 +300,7 @@ query error unsupported comparison operator: 1 = ALL ARRAY\['foo', 'bar'\]
SELECT 1 = ALL(ARRAY['foo', 'bar'])

query error unsupported comparison operator: <int> = ALL <string\[\]>
SELECT 1 = ALL(ARRAY['foo'] || 'bar')
SELECT 1 = ALL(ARRAY['foo'] || 'bar'::text)

# ALL with subqueries.

Expand Down
2 changes: 1 addition & 1 deletion pkg/sql/opt/norm/testdata/rules/fold_constants
Original file line number Diff line number Diff line change
Expand Up @@ -326,7 +326,7 @@ values

# Fold constant.
opt expect=FoldBinary
SELECT ARRAY['a','b','c'] || 'd'
SELECT ARRAY['a','b','c'] || 'd'::string
----
values
├── columns: "?column?":1(string[]!null)
Expand Down
4 changes: 4 additions & 0 deletions pkg/sql/sem/tree/constant.go
Original file line number Diff line number Diff line change
Expand Up @@ -398,13 +398,17 @@ func (expr *StrVal) Format(ctx *FmtCtx) {
var (
// StrValAvailAllParsable is the set of parsable string types.
StrValAvailAllParsable = []*types.T{
// Note: String is deliberately first, to make sure that "string" is the
// default type that raw strings get parsed into, without any casts or type
// assertions.
types.String,
types.Bytes,
types.Bool,
types.Int,
types.Float,
types.Decimal,
types.Date,
types.StringArray,
types.Time,
types.Timestamp,
types.TimestampTZ,
Expand Down
13 changes: 13 additions & 0 deletions pkg/sql/sem/tree/constant_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -239,6 +239,14 @@ func mustParseDJSON(t *testing.T, s string) tree.Datum {
}
return d
}
func mustParseDStringArray(t *testing.T, s string) tree.Datum {
evalContext := tree.MakeTestingEvalContext(cluster.MakeTestingClusterSettings())
d, err := tree.ParseDArrayFromString(&evalContext, s, types.String)
if err != nil {
t.Fatal(err)
}
return d
}

var parseFuncs = map[*types.T]func(*testing.T, string) tree.Datum{
types.String: func(t *testing.T, s string) tree.Datum { return tree.NewDString(s) },
Expand All @@ -250,6 +258,7 @@ var parseFuncs = map[*types.T]func(*testing.T, string) tree.Datum{
types.TimestampTZ: mustParseDTimestampTZ,
types.Interval: mustParseDInterval,
types.Jsonb: mustParseDJSON,
types.StringArray: mustParseDStringArray,
}

func typeSet(tys ...*types.T) map[*types.T]struct{} {
Expand Down Expand Up @@ -319,6 +328,10 @@ func TestStringConstantResolveAvailableTypes(t *testing.T) {
c: tree.NewStrVal(`{"a": 1}`),
parseOptions: typeSet(types.String, types.Bytes, types.Jsonb),
},
{
c: tree.NewStrVal(`{a,b}`),
parseOptions: typeSet(types.String, types.Bytes, types.StringArray),
},
{
c: tree.NewBytesStrVal(string([]byte{0xff, 0xfe, 0xfd})),
parseOptions: typeSet(types.String, types.Bytes),
Expand Down
37 changes: 26 additions & 11 deletions pkg/sql/sem/tree/parse_array.go
Original file line number Diff line number Diff line change
Expand Up @@ -20,6 +20,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror"
"github.com/cockroachdb/cockroach/pkg/sql/types"
"github.com/cockroachdb/cockroach/pkg/util/errorutil/unimplemented"
"github.com/cockroachdb/errors"
)

var enclosingError = pgerror.Newf(pgcode.InvalidTextRepresentation, "array must be enclosed in { and }")
Expand Down Expand Up @@ -71,10 +72,10 @@ func (p *parseState) gobbleString(isTerminatingChar func(ch byte) bool) (out str
}

type parseState struct {
s string
evalCtx *EvalContext
result *DArray
t *types.T
s string
ctx ParseTimeContext
result *DArray
t *types.T
}

func (p *parseState) advance() {
Expand Down Expand Up @@ -136,21 +137,35 @@ func (p *parseState) parseElement() error {
}
}

d, err := PerformCast(p.evalCtx, NewDString(next), p.t)
d, err := parseStringAs(p.t, next, p.ctx)
if d == nil && err == nil {
return errors.AssertionFailedf("unknown type %s (%T)", p.t, p.t)
}
if err != nil {
return err
}
return p.result.Append(d)
}

// ParseDArrayFromString parses the string-form of constructing arrays, handling
// cases such as `'{1,2,3}'::INT[]`.
func ParseDArrayFromString(evalCtx *EvalContext, s string, t *types.T) (*DArray, error) {
// cases such as `'{1,2,3}'::INT[]`. The input type t is the type of the
// parameter of the array to parse.
func ParseDArrayFromString(ctx ParseTimeContext, s string, t *types.T) (*DArray, error) {
ret, err := doParseDArrayFromString(ctx, s, t)
if err != nil {
return ret, makeParseError(s, types.MakeArray(t), err)
}
return ret, nil
}

// doParseDArraryFromString does most of the work of ParseDArrayFromString,
// except the error it returns isn't prettified as a parsing error.
func doParseDArrayFromString(ctx ParseTimeContext, s string, t *types.T) (*DArray, error) {
parser := parseState{
s: s,
evalCtx: evalCtx,
result: NewDArray(t),
t: t,
s: s,
ctx: ctx,
result: NewDArray(t),
t: t,
}

parser.eatWhitespace()
Expand Down
32 changes: 16 additions & 16 deletions pkg/sql/sem/tree/parse_array_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -159,24 +159,24 @@ func TestParseArrayError(t *testing.T) {
typ *types.T
expectedError string
}{
{``, types.Int, "array must be enclosed in { and }"},
{`1`, types.Int, "array must be enclosed in { and }"},
{`1,2`, types.Int, "array must be enclosed in { and }"},
{`{1,2`, types.Int, "malformed array"},
{`{1,2,`, types.Int, "malformed array"},
{`{`, types.Int, "malformed array"},
{`{,}`, types.Int, "malformed array"},
{`{}{}`, types.Int, "extra text after closing right brace"},
{`{} {}`, types.Int, "extra text after closing right brace"},
{`{{}}`, types.Int, "unimplemented: nested arrays not supported"},
{`{1, {1}}`, types.Int, "unimplemented: nested arrays not supported"},
{`{hello}`, types.Int, `could not parse "hello" as type int: strconv.ParseInt: parsing "hello": invalid syntax`},
{`{"hello}`, types.String, `malformed array`},
{``, types.Int, `could not parse "" as type int[]: array must be enclosed in { and }`},
{`1`, types.Int, `could not parse "1" as type int[]: array must be enclosed in { and }`},
{`1,2`, types.Int, `could not parse "1,2" as type int[]: array must be enclosed in { and }`},
{`{1,2`, types.Int, `could not parse "{1,2" as type int[]: malformed array`},
{`{1,2,`, types.Int, `could not parse "{1,2," as type int[]: malformed array`},
{`{`, types.Int, `could not parse "{" as type int[]: malformed array`},
{`{,}`, types.Int, `could not parse "{,}" as type int[]: malformed array`},
{`{}{}`, types.Int, `could not parse "{}{}" as type int[]: extra text after closing right brace`},
{`{} {}`, types.Int, `could not parse "{} {}" as type int[]: extra text after closing right brace`},
{`{{}}`, types.Int, `could not parse "{{}}" as type int[]: unimplemented: nested arrays not supported`},
{`{1, {1}}`, types.Int, `could not parse "{1, {1}}" as type int[]: unimplemented: nested arrays not supported`},
{`{hello}`, types.Int, `could not parse "{hello}" as type int[]: could not parse "hello" as type int: strconv.ParseInt: parsing "hello": invalid syntax`},
{`{"hello}`, types.String, `could not parse "{\"hello}" as type string[]: malformed array`},
// It might be unnecessary to disallow this, but Postgres does.
{`{he"lo}`, types.String, "malformed array"},
{`{he"lo}`, types.String, `could not parse "{he\"lo}" as type string[]: malformed array`},

{string([]byte{200}), types.String, "array must be enclosed in { and }"},
{string([]byte{'{', 'a', 200}), types.String, "malformed array"},
{string([]byte{200}), types.String, `could not parse "\xc8" as type string[]: array must be enclosed in { and }`},
{string([]byte{'{', 'a', 200}), types.String, `could not parse "{a\xc8" as type string[]: malformed array`},
}
for _, td := range testData {
t.Run(td.str, func(t *testing.T) {
Expand Down
6 changes: 5 additions & 1 deletion pkg/sql/sem/tree/parse_string.go
Original file line number Diff line number Diff line change
Expand Up @@ -52,14 +52,18 @@ func ParseDatumStringAs(t *types.T, s string, evalCtx *EvalContext) (Datum, erro
}
}

// parseStringAs parses s as type t for simple types. Bytes, arrays, collated
// parseStringAs parses s as type t for simple types. Arrays and collated
// strings are not handled. nil, nil is returned if t is not a supported type.
func parseStringAs(t *types.T, s string, ctx ParseTimeContext) (Datum, error) {
switch t.Family() {
case types.ArrayFamily:
return ParseDArrayFromString(ctx, s, t.ArrayContents())
case types.BitFamily:
return ParseDBitArray(s)
case types.BoolFamily:
return ParseDBool(s)
case types.BytesFamily:
return ParseDByte(s)
case types.DateFamily:
return ParseDDate(ctx, s)
case types.DecimalFamily:
Expand Down
5 changes: 5 additions & 0 deletions pkg/sql/sem/tree/testdata/eval/concat
Original file line number Diff line number Diff line change
Expand Up @@ -14,3 +14,8 @@ eval
b'hello' || 'world'
----
'\x68656c6c6f776f726c64'

eval
array['foo'] || '{a,b}'
----
ARRAY['foo','a','b']

0 comments on commit e86e27a

Please sign in to comment.