Skip to content

Commit

Permalink
Merge pull request #62211 from ajwerner/backport20.2-61933
Browse files Browse the repository at this point in the history
release-20.2: sql/sem/tree: properly serialize physical values of enums
  • Loading branch information
ajwerner authored Mar 23, 2021
2 parents 0345d86 + 5a0f834 commit 628fb8b
Show file tree
Hide file tree
Showing 5 changed files with 133 additions and 16 deletions.
9 changes: 6 additions & 3 deletions pkg/sql/sem/tree/datum.go
Original file line number Diff line number Diff line change
Expand Up @@ -12,6 +12,7 @@ package tree

import (
"bytes"
"encoding/hex"
"fmt"
"math"
"math/big"
Expand Down Expand Up @@ -1442,12 +1443,14 @@ func (d *DBytes) Format(ctx *FmtCtx) {
ctx.WriteString(`"\\x`)
writeAsHexString(ctx, d)
ctx.WriteString(`"`)
} else if f.HasFlags(fmtFormatByteLiterals) {
ctx.WriteByte('x')
ctx.WriteByte('\'')
_, _ = hex.NewEncoder(ctx).Write([]byte(*d))
ctx.WriteByte('\'')
} else {
withQuotes := !f.HasFlags(FmtFlags(lex.EncBareStrings))
if withQuotes {
if f.HasFlags(fmtFormatByteLiterals) {
ctx.WriteByte('b')
}
ctx.WriteByte('\'')
}
ctx.WriteString("\\x")
Expand Down
4 changes: 2 additions & 2 deletions pkg/sql/sem/tree/format.go
Original file line number Diff line number Diff line change
Expand Up @@ -133,8 +133,8 @@ const (
fmtStaticallyFormatUserDefinedTypes

// fmtFormatByteLiterals instructs bytes to be formatted as byte literals
// rather than string literals. For example, the bytes \x40 will be formatted
// as b'\x40' rather than '\x40'.
// rather than string literals. For example, the bytes \x40ab will be formatted
// as x'40ab' rather than '\x40ab'.
fmtFormatByteLiterals
)

Expand Down
11 changes: 7 additions & 4 deletions pkg/sql/sem/tree/format_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -336,8 +336,11 @@ func TestFormatExpr2(t *testing.T) {
// enum related code in tree expects that the length of
// PhysicalRepresentations is equal to the length of
// LogicalRepresentations.
PhysicalRepresentations: make([][]byte, len(enumMembers)),
IsMemberReadOnly: make([]bool, len(enumMembers)),
PhysicalRepresentations: [][]byte{
{0x42, 0x1},
{0x42},
},
IsMemberReadOnly: make([]bool, len(enumMembers)),
},
}
enumHi, err := tree.MakeDEnumFromLogicalRepresentation(enumType, enumMembers[0])
Expand Down Expand Up @@ -400,13 +403,13 @@ func TestFormatExpr2(t *testing.T) {
types.MakeTuple([]*types.T{enumType, enumType}),
enumHi, enumHello),
tree.FmtSerializable,
`(b'\x':::@100500, b'\x':::@100500)`,
`(x'4201':::@100500, x'42':::@100500)`,
},
{tree.NewDTuple(
types.MakeTuple([]*types.T{enumType, enumType}),
tree.DNull, enumHi),
tree.FmtSerializable,
`(NULL:::@100500, b'\x':::@100500)`,
`(NULL:::@100500, x'4201':::@100500)`,
},
}

Expand Down
14 changes: 7 additions & 7 deletions pkg/sql/table_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -394,30 +394,30 @@ func TestSerializedUDTsInTableDescriptor(t *testing.T) {
// Test a simple UDT as the default value.
{
"x greeting DEFAULT ('hello')",
`b'\x80':::@100053`,
`x'80':::@100053`,
getDefault,
},
{
"x greeting DEFAULT ('hello':::greeting)",
`b'\x80':::@100053`,
`x'80':::@100053`,
getDefault,
},
// Test when a UDT is used in a default value, but isn't the
// final type of the column.
{
"x INT DEFAULT (CASE WHEN 'hello'::greeting = 'hello'::greeting THEN 0 ELSE 1 END)",
`CASE WHEN b'\x80':::@100053 = b'\x80':::@100053 THEN 0:::INT8 ELSE 1:::INT8 END`,
`CASE WHEN x'80':::@100053 = x'80':::@100053 THEN 0:::INT8 ELSE 1:::INT8 END`,
getDefault,
},
{
"x BOOL DEFAULT ('hello'::greeting IS OF (greeting, greeting))",
`b'\x80':::@100053 IS OF (@100053, @100053)`,
`x'80':::@100053 IS OF (@100053, @100053)`,
getDefault,
},
// Test check constraints.
{
"x greeting, CHECK (x = 'hello')",
`x = b'\x80':::@100053`,
`x = x'80':::@100053`,
getCheck,
},
{
Expand All @@ -428,12 +428,12 @@ func TestSerializedUDTsInTableDescriptor(t *testing.T) {
// Test a computed column in the same cases as above.
{
"x greeting AS ('hello') STORED",
`b'\x80':::@100053`,
`x'80':::@100053`,
getComputed,
},
{
"x INT AS (CASE WHEN 'hello'::greeting = 'hello'::greeting THEN 0 ELSE 1 END) STORED",
`CASE WHEN b'\x80':::@100053 = b'\x80':::@100053 THEN 0:::INT8 ELSE 1:::INT8 END`,
`CASE WHEN x'80':::@100053 = x'80':::@100053 THEN 0:::INT8 ELSE 1:::INT8 END`,
getComputed,
},
}
Expand Down
111 changes: 111 additions & 0 deletions pkg/sql/tests/enum_test.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,111 @@
// 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.

package tests

import (
"context"
"fmt"
"math/rand"
"sort"
"strings"
"testing"

"github.com/cockroachdb/cockroach/pkg/base"
"github.com/cockroachdb/cockroach/pkg/testutils/sqlutils"
"github.com/cockroachdb/cockroach/pkg/testutils/testcluster"
"github.com/cockroachdb/cockroach/pkg/util/leaktest"
"github.com/google/btree"
"github.com/stretchr/testify/require"
)

func TestLargeEnums(t *testing.T) {
defer leaktest.AfterTest(t)()

ctx := context.Background()
tc := testcluster.StartTestCluster(t, 1, base.TestClusterArgs{})
defer tc.Stopper().Stop(ctx)

// The idea here is that we're going to create a very large number of
// enum elements corresponding to non-negative integers and then we're
// going to add them to the enum in a random order. Then we're going to
// make sure that we can cast the integers to strings to the enums, order
// them as enums then cast them back to ints and make sure it's what we want.
// Ideally we'd make this number even bigger but it's slow enough as it is.
const N = 100

order := rand.Perm(N)
tdb := sqlutils.MakeSQLRunner(tc.ServerConn(0))

tdb.Exec(t, "CREATE TYPE e AS ENUM ()")
// Construct a single transaction to insert all the values; otherwise we'd
// have to wait for lots of versions and it would take a very long time.
var createEnumsQuery string
{
alreadyInserted := btree.New(8)
next := func(n int) (next int, ok bool) {
alreadyInserted.AscendGreaterOrEqual(intItem(n), func(i btree.Item) (wantMore bool) {
next, ok = int(i.(intItem)), true
return false
})
return next, ok
}
prev := func(n int) (prev int, ok bool) {
alreadyInserted.DescendLessOrEqual(intItem(n), func(i btree.Item) (wantMore bool) {
prev, ok = int(i.(intItem)), true
return false
})
return prev, ok
}
var buf strings.Builder
buf.WriteString("BEGIN;\n")
for i, n := range order {
fmt.Fprintf(&buf, "\tALTER TYPE e ADD VALUE '%d'", n)
if alreadyInserted.Len() == 0 {
buf.WriteString(";\n")
} else if next, ok := next(n); ok {
fmt.Fprintf(&buf, " BEFORE '%d';\n", next)
} else {
prev, ok := prev(n)
require.Truef(t, ok, "prev %v %v", n, order[:i])
fmt.Fprintf(&buf, " AFTER '%d';\n", prev)
}
alreadyInserted.ReplaceOrInsert(intItem(n))
}
buf.WriteString("COMMIT;")
createEnumsQuery = buf.String()
}
tdb.Exec(t, createEnumsQuery)

// Okay, now we have enum values for all of these numbers.
tdb.Exec(t, `CREATE TABLE t (i e PRIMARY KEY);`)
tdb.Exec(t, `
INSERT INTO t SELECT i
FROM (
SELECT i::STRING::e AS i
FROM generate_series(0, $1 - 1, 1) AS t (i)
);`, N)
rows := tdb.Query(t, "SELECT i::STRING::INT FROM t")
var read []int
for rows.Next() {
var i int
require.NoError(t, rows.Scan(&i))
read = append(read, i)
}
require.NoError(t, rows.Err())
require.Len(t, read, N)
require.Truef(t, sort.IntsAreSorted(read), "%v", read)
}

type intItem int

func (i intItem) Less(o btree.Item) bool {
return i < o.(intItem)
}

0 comments on commit 628fb8b

Please sign in to comment.