Skip to content

Commit

Permalink
Merge #28151
Browse files Browse the repository at this point in the history
28151: sql: fix the pg text representation of tuples/arrays r=knz a=knz

Forked off #28143.
Fixes #25522.

This patch fixes the conversion of arrays and tuples to text for the
purpose of emitting the arrays/tuples back to a client through pgwire.
This now properly supports nested arrays, arrays containing tuples,
tuples containing arrays, etc. Labels in tuples are never emitted
through pgwire.

Release note (bug fix): CockroachDB supports a wider range of tuple
and array values in query results.

Co-authored-by: Raphael 'kena' Poss <[email protected]>
  • Loading branch information
craig[bot] and knz committed Aug 2, 2018
2 parents 2e4952d + 200d2a6 commit c754381
Show file tree
Hide file tree
Showing 13 changed files with 307 additions and 114 deletions.
19 changes: 0 additions & 19 deletions pkg/sql/lex/encode.go
Original file line number Diff line number Diff line change
Expand Up @@ -28,7 +28,6 @@ import (
"encoding/base64"
"encoding/hex"
"fmt"
"unicode"
"unicode/utf8"

"github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror"
Expand Down Expand Up @@ -130,24 +129,6 @@ func EncodeSQLStringWithFlags(buf *bytes.Buffer, in string, flags EncodeFlags) {
}
}

// EncodeSQLStringInsideArray writes a string literal to buf using the "string
// within array" formatting.
func EncodeSQLStringInsideArray(buf *bytes.Buffer, in string) {
buf.WriteByte('"')
// Loop through each unicode code point.
for i, r := range in {
ch := byte(r)
if unicode.IsPrint(r) && !stringencoding.NeedEscape(ch) && ch != '"' {
// Character is printable doesn't need escaping - just print it out.
buf.WriteRune(r)
} else {
stringencoding.EncodeEscapedChar(buf, in, r, ch, i, '"')
}
}

buf.WriteByte('"')
}

// EncodeUnrestrictedSQLIdent writes the identifier in s to buf.
// The identifier is only quoted if the flags don't tell otherwise and
// the identifier contains special characters.
Expand Down
13 changes: 3 additions & 10 deletions pkg/sql/lex/encode_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -86,16 +86,9 @@ func testEncodeString(t *testing.T, input []byte, encode func(*bytes.Buffer, str

func BenchmarkEncodeSQLString(b *testing.B) {
str := strings.Repeat("foo", 10000)
b.Run("old version", func(b *testing.B) {
for i := 0; i < b.N; i++ {
lex.EncodeSQLStringWithFlags(bytes.NewBuffer(nil), str, lex.EncBareStrings)
}
})
b.Run("new version", func(b *testing.B) {
for i := 0; i < b.N; i++ {
lex.EncodeSQLStringInsideArray(bytes.NewBuffer(nil), str)
}
})
for i := 0; i < b.N; i++ {
lex.EncodeSQLStringWithFlags(bytes.NewBuffer(nil), str, lex.EncBareStrings)
}
}

func TestEncodeRestrictedSQLIdent(t *testing.T) {
Expand Down
10 changes: 5 additions & 5 deletions pkg/sql/logictest/testdata/logic_test/array
Original file line number Diff line number Diff line change
Expand Up @@ -76,7 +76,7 @@ SELECT ARRAY['one', 'two', 'fünf']
query T
SELECT ARRAY[e'\n', e'g\x10h']
----
{"\n","g\x10h"}
{"\x0a","g\x10h"}

query T
SELECT ARRAY['foo', 'bar']
Expand Down Expand Up @@ -582,10 +582,10 @@ query T rowsort
SELECT b FROM a
----
{}
{true}
{false}
{true,true}
{false,true}
{t}
{f}
{t,t}
{f,t}

statement ok
DROP TABLE a
Expand Down
6 changes: 3 additions & 3 deletions pkg/sql/logictest/testdata/logic_test/orms
Original file line number Diff line number Diff line change
Expand Up @@ -68,9 +68,9 @@ GROUP BY i.relname,
ix.indkey
ORDER BY i.relname
----
name primary unique indkey column_indexes column_names definition
customers_id_idx false false 2 {1,2} {"name","id"} CREATE INDEX customers_id_idx ON test.public.customers (id ASC)
primary true true 1 {1,2} {"name","id"} CREATE UNIQUE INDEX "primary" ON test.public.customers (name ASC)
name primary unique indkey column_indexes column_names definition
customers_id_idx false false 2 {1,2} {name,id} CREATE INDEX customers_id_idx ON test.public.customers (id ASC)
primary true true 1 {1,2} {name,id} CREATE UNIQUE INDEX "primary" ON test.public.customers (name ASC)


query TT colnames
Expand Down
22 changes: 11 additions & 11 deletions pkg/sql/logictest/testdata/logic_test/srfs
Original file line number Diff line number Diff line change
Expand Up @@ -284,13 +284,13 @@ query TTT colnames
SELECT 'a' AS a, pg_get_keywords(), 'c' AS c LIMIT 1
----
a pg_get_keywords c
a ('abort','U','unreserved') c
a ("abort","U","unreserved") c

query TTT colnames
SELECT 'a' AS a, pg_get_keywords() AS b, 'c' AS c LIMIT 1
----
a b c
a ('abort','U','unreserved') c
a ("abort","U","unreserved") c

subtest unary_table

Expand Down Expand Up @@ -387,7 +387,7 @@ query T colnames
SELECT information_schema._pg_expandarray(ARRAY['a'])
----
information_schema._pg_expandarray
('a',1)
("a",1)

query TI colnames
SELECT * FROM information_schema._pg_expandarray(ARRAY['a'])
Expand All @@ -399,8 +399,8 @@ query T colnames
SELECT information_schema._pg_expandarray(ARRAY['b', 'a'])
----
information_schema._pg_expandarray
('b',1)
('a',2)
("b",1)
("a",2)

query TI colnames
SELECT * FROM information_schema._pg_expandarray(ARRAY['b', 'a'])
Expand All @@ -413,9 +413,9 @@ query T colnames
SELECT information_schema._pg_expandarray(ARRAY['c', 'b', 'a'])
----
information_schema._pg_expandarray
('c',1)
('b',2)
('a',3)
("c",1)
("b",2)
("a",3)

query TI colnames
SELECT * FROM information_schema._pg_expandarray(ARRAY['c', 'b', 'a'])
Expand Down Expand Up @@ -540,9 +540,9 @@ x n
query T
SELECT ((i.keys).*, 123) FROM (SELECT information_schema._pg_expandarray(ARRAY[3,2,1]) AS keys) AS i
----
((3, 1),123)
((2, 2),123)
((1, 3),123)
("(3,1)",123)
("(2,2)",123)
("(1,3)",123)

subtest generate_subscripts

Expand Down
8 changes: 4 additions & 4 deletions pkg/sql/logictest/testdata/logic_test/tuple
Original file line number Diff line number Diff line change
Expand Up @@ -7,7 +7,7 @@ query TT colnames
SELECT (1, 2, 'hello', NULL, NULL) AS t, (true, NULL, (false, 6.6, false)) AS u
----
t u
(1,2,'hello',,) (true,,(false, 6.6, false))
(1,2,"hello",,) (t,,"(f,6.6,f)")

query BBBBBBBBB colnames
SELECT
Expand Down Expand Up @@ -634,7 +634,7 @@ SELECT ((1, 2, 'hello', NULL, NULL) AS a1, b2, c3, d4, e5) AS r,
((true, NULL, (false, 6.6, false)) AS a1, b2, c3) AS s
----
r s
(1,2,'hello',,) (true,,(false, 6.6, false))
(1,2,"hello",,) (t,,"(f,6.6,f)")

# Comparing tuples
query BBB colnames
Expand Down Expand Up @@ -709,7 +709,7 @@ query T colnames
SELECT ((((((1, '2', 3) AS a, b, c), ((4,'5') AS a, b), (ROW(6) AS a)) AS a, b, c), ((7, 8) AS a, b), (ROW('9') AS a)) AS a, b, c) AS r
----
r
(((1, '2', 3), (4, '5'), (6)),(7, 8),('9'))
("(""(1,""""2"""",3)"",""(4,""""5"""")"",""(6)"")","(7,8)","(""9"")")

subtest labeled_tuple_column_access

Expand Down Expand Up @@ -767,7 +767,7 @@ a b c
query T
SELECT (((ROW(1,'2',true) AS a,b,c)).*, 456)
----
((1, '2', true),456)
("(1,""2"",t)",456)

query I colnames
SELECT ((ROW(1) AS a)).*
Expand Down
42 changes: 11 additions & 31 deletions pkg/sql/pgwire/types.go
Original file line number Diff line number Diff line change
Expand Up @@ -79,12 +79,8 @@ func (b *writeBuffer) writeTextDatum(
}
switch v := tree.UnwrapDatum(nil, d).(type) {
case *tree.DBool:
b.putInt32(1)
if *v {
b.writeByte('t')
} else {
b.writeByte('f')
}
b.textFormatter.FormatNode(v)
b.writeLengthPrefixedVariablePutbuf()

case *tree.DInt:
// Start at offset 4 because `putInt32` clobbers the first 4 bytes.
Expand Down Expand Up @@ -157,40 +153,24 @@ func (b *writeBuffer) writeTextDatum(
b.writeLengthPrefixedString(v.JSON.String())

case *tree.DTuple:
b.variablePutbuf.WriteString("(")
for i, d := range v.D {
if i > 0 {
b.variablePutbuf.WriteString(",")
}
if d == tree.DNull {
// Emit nothing on NULL.
continue
}
b.simpleFormatter.FormatNode(d)
}
b.variablePutbuf.WriteString(")")
b.textFormatter.FormatNode(v)
b.writeLengthPrefixedVariablePutbuf()

case *tree.DArray:
// Arrays are serialized as a string of comma-separated values, surrounded
// by braces.
begin, sep, end := "{", ",", "}"

switch d.ResolvedType().Oid() {
case oid.T_int2vector, oid.T_oidvector:
// vectors are serialized as a string of space-separated values.
begin, sep, end = "", " ", ""
}

b.variablePutbuf.WriteString(begin)
for i, d := range v.Array {
if i > 0 {
sep := ""
// TODO(justin): add a test for nested arrays.
for _, d := range v.Array {
b.variablePutbuf.WriteString(sep)
b.textFormatter.FormatNode(d)
sep = " "
}
// TODO(justin): add a test for nested arrays.
b.arrayFormatter.FormatNode(d)
default:
// Uses the default pgwire text format for arrays.
b.textFormatter.FormatNode(v)
}
b.variablePutbuf.WriteString(end)
b.writeLengthPrefixedVariablePutbuf()

case *tree.DOid:
Expand Down
10 changes: 4 additions & 6 deletions pkg/sql/pgwire/write_buffer.go
Original file line number Diff line number Diff line change
Expand Up @@ -41,10 +41,9 @@ type writeBuffer struct {
// We keep both of these because there are operations that are only possible to
// perform (efficiently) with one or the other, such as strconv.AppendInt with
// putbuf or Datum.Format with variablePutbuf.
putbuf [64]byte
variablePutbuf bytes.Buffer
simpleFormatter tree.FmtCtx
arrayFormatter tree.FmtCtx
putbuf [64]byte
variablePutbuf bytes.Buffer
textFormatter tree.FmtCtx

// bytecount counts the number of bytes written across all pgwire connections, not just this
// buffer. This is passed in so that finishMsg can track all messages we've sent to a network
Expand All @@ -56,8 +55,7 @@ func newWriteBuffer(bytecount *metric.Counter) *writeBuffer {
b := &writeBuffer{
bytecount: bytecount,
}
b.simpleFormatter = tree.MakeFmtCtx(&b.variablePutbuf, tree.FmtSimple)
b.arrayFormatter = tree.MakeFmtCtx(&b.variablePutbuf, tree.FmtArrays)
b.textFormatter = tree.MakeFmtCtx(&b.variablePutbuf, tree.FmtPgwireText)
return b
}

Expand Down
59 changes: 40 additions & 19 deletions pkg/sql/sem/tree/datum.go
Original file line number Diff line number Diff line change
Expand Up @@ -409,6 +409,14 @@ func (*DBool) AmbiguousFormat() bool { return false }

// Format implements the NodeFormatter interface.
func (d *DBool) Format(ctx *FmtCtx) {
if ctx.HasFlags(fmtPgwireFormat) {
if bool(*d) {
ctx.WriteByte('t')
} else {
ctx.WriteByte('f')
}
return
}
ctx.WriteString(strconv.FormatBool(bool(*d)))
}

Expand Down Expand Up @@ -958,8 +966,6 @@ func (d *DString) Format(ctx *FmtCtx) {
buf, f := ctx.Buffer, ctx.flags
if f.HasFlags(fmtUnicodeStrings) {
buf.WriteString(string(*d))
} else if f.HasFlags(fmtWithinArray) {
lex.EncodeSQLStringInsideArray(buf, string(*d))
} else {
lex.EncodeSQLStringWithFlags(buf, string(*d), f.EncodeFlags())
}
Expand Down Expand Up @@ -1026,14 +1032,9 @@ func (*DCollatedString) AmbiguousFormat() bool { return false }

// Format implements the NodeFormatter interface.
func (d *DCollatedString) Format(ctx *FmtCtx) {
buf, f := ctx.Buffer, ctx.flags
if f.HasFlags(fmtWithinArray) {
lex.EncodeSQLStringInsideArray(buf, d.Contents)
} else {
lex.EncodeSQLString(buf, d.Contents)
ctx.WriteString(" COLLATE ")
lex.EncodeUnrestrictedSQLIdent(buf, d.Locale, lex.EncNoFlags)
}
lex.EncodeSQLString(ctx.Buffer, d.Contents)
ctx.WriteString(" COLLATE ")
lex.EncodeUnrestrictedSQLIdent(ctx.Buffer, d.Locale, lex.EncNoFlags)
}

// ResolvedType implements the TypedExpr interface.
Expand Down Expand Up @@ -1192,7 +1193,7 @@ func writeAsHexString(ctx *FmtCtx, d *DBytes) {
// Format implements the NodeFormatter interface.
func (d *DBytes) Format(ctx *FmtCtx) {
f := ctx.flags
if f.HasFlags(fmtWithinArray) {
if f.HasFlags(fmtPgwireFormat) {
ctx.WriteString(`"\\x`)
writeAsHexString(ctx, d)
ctx.WriteString(`"`)
Expand Down Expand Up @@ -2718,14 +2719,24 @@ func (d *DTuple) IsMin(ctx *EvalContext) bool {
func (*DTuple) AmbiguousFormat() bool { return false }

// Format implements the NodeFormatter interface.
// TODO(bram): We don't format tuples in the same way as postgres. See #25522.
// TODO(knz): this is broken if the tuple is labeled. See #26624.
func (d *DTuple) Format(ctx *FmtCtx) {
if ctx.HasFlags(FmtParsable) && (len(d.D) == 0) {
ctx.WriteString("ROW()")
if ctx.HasFlags(fmtPgwireFormat) {
d.pgwireFormat(ctx)
return
}
ctx.FormatNode(&d.D)

if ctx.HasFlags(FmtParsable) && (len(d.D) == 0) {
ctx.WriteString("ROW")
}
ctx.WriteByte('(')
comma := ""
for _, v := range d.D {
ctx.WriteString(comma)
ctx.FormatNode(v)
comma = ", "
}
ctx.WriteByte(')')
}

// Sorted returns true if the tuple is known to be sorted (and contains no
Expand Down Expand Up @@ -2879,6 +2890,11 @@ func (dNull) AmbiguousFormat() bool { return false }

// Format implements the NodeFormatter interface.
func (dNull) Format(ctx *FmtCtx) {
if ctx.HasFlags(fmtPgwireFormat) {
// NULL sub-expressions in pgwire text values are represented with
// the empty string.
return
}
ctx.WriteString("NULL")
}

Expand Down Expand Up @@ -3002,12 +3018,17 @@ func (d *DArray) AmbiguousFormat() bool {

// Format implements the NodeFormatter interface.
func (d *DArray) Format(ctx *FmtCtx) {
if ctx.HasFlags(fmtPgwireFormat) {
d.pgwireFormat(ctx)
return
}

ctx.WriteString("ARRAY[")
for i, v := range d.Array {
if i > 0 {
ctx.WriteString(",")
}
comma := ""
for _, v := range d.Array {
ctx.WriteString(comma)
ctx.FormatNode(v)
comma = ","
}
ctx.WriteByte(']')
}
Expand Down
Loading

0 comments on commit c754381

Please sign in to comment.