Skip to content

Commit

Permalink
expression: add json opaque value (#37200)
Browse files Browse the repository at this point in the history
close #9996, close #31918
  • Loading branch information
YangKeao authored Aug 23, 2022
1 parent 75e0885 commit fe40d3c
Show file tree
Hide file tree
Showing 7 changed files with 225 additions and 11 deletions.
18 changes: 17 additions & 1 deletion expression/builtin_cast.go
Original file line number Diff line number Diff line change
Expand Up @@ -705,7 +705,23 @@ func (b *builtinCastStringAsJSONSig) evalJSON(row chunk.Row) (res json.BinaryJSO
if isNull || err != nil {
return res, isNull, err
}
if mysql.HasParseToJSONFlag(b.tp.GetFlag()) {

typ := b.args[0].GetType()
if types.IsBinaryStr(typ) {
buf := []byte(val)
if typ.GetType() == mysql.TypeString {
// the tailing zero should also be in the opaque json
buf = make([]byte, typ.GetFlen())
copy(buf, val)
}

res := json.CreateBinary(json.Opaque{
TypeCode: b.args[0].GetType().GetType(),
Buf: buf,
})

return res, false, err
} else if mysql.HasParseToJSONFlag(b.tp.GetFlag()) {
res, err = json.ParseBinaryFromString(val)
} else {
res = json.CreateBinary(val)
Expand Down
61 changes: 61 additions & 0 deletions expression/builtin_cast_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -1550,3 +1550,64 @@ func TestCastConstAsDecimalFieldType(t *testing.T) {
require.Equal(t, tc.resultDecimal, expr.GetType().GetDecimal())
}
}

func TestCastBinaryStringAsJSONSig(t *testing.T) {
ctx := createContext(t)
sc := ctx.GetSessionVars().StmtCtx
originIgnoreTruncate := sc.IgnoreTruncate
sc.IgnoreTruncate = true
defer func() {
sc.IgnoreTruncate = originIgnoreTruncate
}()

// BINARY STRING will be converted to a JSON opaque
// and yield "base64:typeXX:<base64 encoded value>" finally
var tests = []struct {
str string
tp *types.FieldType
result json.BinaryJSON
resultStr string
}{
{
"a",
types.NewFieldTypeWithCollation(mysql.TypeVarString, charset.CollationBin, 4),
json.BinaryJSON{TypeCode: json.TypeCodeOpaque, Value: []byte{0xfd, 1, 'a'}},
`"base64:type253:YQ=="`,
},
{
"test",
types.NewFieldTypeWithCollation(mysql.TypeVarString, charset.CollationBin, 4),
json.BinaryJSON{TypeCode: json.TypeCodeOpaque, Value: []byte{0xfd, 4, 't', 'e', 's', 't'}},
`"base64:type253:dGVzdA=="`,
},
{
"a",
types.NewFieldTypeWithCollation(mysql.TypeString, charset.CollationBin, 4),
json.BinaryJSON{TypeCode: json.TypeCodeOpaque, Value: []byte{0xfe, 4, 'a', 0, 0, 0}},
`"base64:type254:YQAAAA=="`,
},
{
"a",
types.NewFieldTypeWithCollation(mysql.TypeBlob, charset.CollationBin, 4),
json.BinaryJSON{TypeCode: json.TypeCodeOpaque, Value: []byte{0xfc, 1, 'a'}},
`"base64:type252:YQ=="`,
},
}
for _, tt := range tests {
args := []Expression{&Column{RetType: tt.tp, Index: 0}}
tp := types.NewFieldType(mysql.TypeJSON)
tp.SetDecimal(types.DefaultFsp)
jsonFunc, err := newBaseBuiltinFunc(ctx, "", args, tp)
require.NoError(t, err)
sig := &builtinCastStringAsJSONSig{jsonFunc}

row := chunk.MutRowFromDatums(
[]types.Datum{types.NewCollationStringDatum(tt.str, charset.CollationBin)},
)
res, isNull, err := sig.evalJSON(row.ToRow())
require.NoError(t, err)
require.False(t, isNull)
require.Equal(t, tt.result, res)
require.Equal(t, tt.resultStr, res.String())
}
}
26 changes: 24 additions & 2 deletions expression/builtin_cast_vec.go
Original file line number Diff line number Diff line change
Expand Up @@ -788,8 +788,30 @@ func (b *builtinCastStringAsJSONSig) vecEvalJSON(input *chunk.Chunk, result *chu
}

result.ReserveJSON(n)
hasParse := mysql.HasParseToJSONFlag(b.tp.GetFlag())
if hasParse {
typ := b.args[0].GetType()
if types.IsBinaryStr(typ) {
var res json.BinaryJSON
for i := 0; i < n; i++ {
if buf.IsNull(i) {
result.AppendNull()
continue
}

val := buf.GetBytes(i)
resultBuf := val
if typ.GetType() == mysql.TypeString {
// only for BINARY: the tailing zero should also be in the opaque json
resultBuf = make([]byte, typ.GetFlen())
copy(resultBuf, val)
}

res = json.CreateBinary(json.Opaque{
TypeCode: b.args[0].GetType().GetType(),
Buf: resultBuf,
})
result.AppendJSON(res)
}
} else if mysql.HasParseToJSONFlag(b.tp.GetFlag()) {
var res json.BinaryJSON
for i := 0; i < n; i++ {
if buf.IsNull(i) {
Expand Down
74 changes: 66 additions & 8 deletions types/json/binary.go
Original file line number Diff line number Diff line change
Expand Up @@ -16,6 +16,7 @@ package json

import (
"bytes"
"encoding/base64"
"encoding/binary"
"encoding/json"
"fmt"
Expand Down Expand Up @@ -47,13 +48,15 @@ import (
0x0a | // uint64
0x0b | // double
0x0c | // utf8mb4 string
0x0d | // opaque value
value ::=
object |
array |
literal |
number |
string |
opaque |
object ::= element-count size key-entry* value-entry* key* value*
Expand Down Expand Up @@ -97,6 +100,10 @@ import (
// field. So we need 1 byte to represent
// lengths up to 127, 2 bytes to represent
// lengths up to 16383, and so on...
opaque ::= typeId data-length byte*
typeId ::= byte
*/

// BinaryJSON represents a binary encoded JSON object.
Expand Down Expand Up @@ -128,6 +135,8 @@ func (bj BinaryJSON) MarshalJSON() ([]byte, error) {

func (bj BinaryJSON) marshalTo(buf []byte) ([]byte, error) {
switch bj.TypeCode {
case TypeCodeOpaque:
return marshalOpaqueTo(buf, bj.GetOpaque()), nil
case TypeCodeString:
return marshalStringTo(buf, bj.GetString()), nil
case TypeCodeLiteral:
Expand Down Expand Up @@ -185,13 +194,35 @@ func (bj BinaryJSON) GetFloat64() float64 {

// GetString gets the string value.
func (bj BinaryJSON) GetString() []byte {
strLen, lenLen := uint64(bj.Value[0]), 1
if strLen >= utf8.RuneSelf {
strLen, lenLen = binary.Uvarint(bj.Value)
}
strLen, lenLen := binary.Uvarint(bj.Value)
return bj.Value[lenLen : lenLen+int(strLen)]
}

// Opaque represents a raw binary type
type Opaque struct {
// TypeCode is the same with database type code
TypeCode byte
// Buf is the underlying bytes of the data
Buf []byte
}

// GetOpaque gets the opaque value
func (bj BinaryJSON) GetOpaque() Opaque {
typ := bj.Value[0]

strLen, lenLen := binary.Uvarint(bj.Value[1:])
bufStart := lenLen + 1
return Opaque{
TypeCode: typ,
Buf: bj.Value[bufStart : bufStart+int(strLen)],
}
}

// GetOpaqueFieldType returns the type of opaque value
func (bj BinaryJSON) GetOpaqueFieldType() byte {
return bj.Value[0]
}

// GetKeys gets the keys of the object
func (bj BinaryJSON) GetKeys() BinaryJSON {
count := bj.GetElemCount()
Expand Down Expand Up @@ -231,12 +262,13 @@ func (bj BinaryJSON) valEntryGet(valEntryOff int) BinaryJSON {
case TypeCodeUint64, TypeCodeInt64, TypeCodeFloat64:
return BinaryJSON{TypeCode: tpCode, Value: bj.Value[valOff : valOff+8]}
case TypeCodeString:
strLen, lenLen := uint64(bj.Value[valOff]), 1
if strLen >= utf8.RuneSelf {
strLen, lenLen = binary.Uvarint(bj.Value[valOff:])
}
strLen, lenLen := binary.Uvarint(bj.Value[valOff:])
totalLen := uint32(lenLen) + uint32(strLen)
return BinaryJSON{TypeCode: tpCode, Value: bj.Value[valOff : valOff+totalLen]}
case TypeCodeOpaque:
strLen, lenLen := binary.Uvarint(bj.Value[valOff+1:])
totalLen := 1 + uint32(lenLen) + uint32(strLen)
return BinaryJSON{TypeCode: tpCode, Value: bj.Value[valOff : valOff+totalLen]}
}
dataSize := endian.Uint32(bj.Value[valOff+dataSizeOff:])
return BinaryJSON{TypeCode: tpCode, Value: bj.Value[valOff : valOff+dataSize]}
Expand Down Expand Up @@ -380,6 +412,17 @@ func marshalStringTo(buf, s []byte) []byte {
return buf
}

// opaque value will yield "base64:typeXX:<base64 encoded string>"
func marshalOpaqueTo(buf []byte, opaque Opaque) []byte {
b64 := base64.StdEncoding.EncodeToString(opaque.Buf)
output := fmt.Sprintf(`"base64:type%d:%s"`, opaque.TypeCode, b64)

// as the base64 string is simple and predictable, it could be appended
// to the buf directly.
buf = append(buf, output...)
return buf
}

func marshalLiteralTo(b []byte, litType byte) []byte {
switch litType {
case LiteralFalse:
Expand Down Expand Up @@ -514,6 +557,9 @@ func appendBinary(buf []byte, in interface{}) (TypeCode, []byte, error) {
if err != nil {
return typeCode, nil, errors.Trace(err)
}
case Opaque:
typeCode = TypeCodeOpaque
buf = appendBinaryOpaque(buf, x)
default:
msg := fmt.Sprintf(unknownTypeErrorMsg, reflect.TypeOf(in))
err = errors.New(msg)
Expand Down Expand Up @@ -575,6 +621,18 @@ func appendBinaryString(buf []byte, v string) []byte {
return buf
}

func appendBinaryOpaque(buf []byte, v Opaque) []byte {
buf = append(buf, v.TypeCode)

lenBegin := len(buf)
buf = appendZero(buf, binary.MaxVarintLen64)
lenLen := binary.PutUvarint(buf[lenBegin:], uint64(len(v.Buf)))

buf = buf[:len(buf)-binary.MaxVarintLen64+lenLen]
buf = append(buf, v.Buf...)
return buf
}

func appendBinaryFloat64(buf []byte, v float64) []byte {
off := len(buf)
buf = appendZero(buf, 8)
Expand Down
13 changes: 13 additions & 0 deletions types/json/binary_functions.go
Original file line number Diff line number Diff line change
Expand Up @@ -24,6 +24,7 @@ import (
"unicode/utf8"

"github.com/pingcap/errors"
"github.com/pingcap/tidb/parser/mysql"
"github.com/pingcap/tidb/util/hack"
"github.com/pingcap/tidb/util/stringutil"
"golang.org/x/exp/slices"
Expand Down Expand Up @@ -51,6 +52,16 @@ func (bj BinaryJSON) Type() string {
return "DOUBLE"
case TypeCodeString:
return "STRING"
case TypeCodeOpaque:
typ := bj.GetOpaqueFieldType()
switch typ {
case mysql.TypeTinyBlob, mysql.TypeMediumBlob, mysql.TypeLongBlob, mysql.TypeBlob, mysql.TypeString, mysql.TypeVarString, mysql.TypeVarchar:
return "BLOB"
case mysql.TypeBit:
return "BIT"
default:
return "OPAQUE"
}
default:
msg := fmt.Sprintf(unknownTypeCodeErrorMsg, bj.TypeCode)
panic(msg)
Expand Down Expand Up @@ -791,6 +802,8 @@ func CompareBinary(left, right BinaryJSON) int {
return cmp
}
}
case TypeCodeOpaque:
cmp = bytes.Compare(left.GetString(), right.GetString())
}
} else {
cmp = precedence1 - precedence2
Expand Down
42 changes: 42 additions & 0 deletions types/json/binary_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -644,3 +644,45 @@ func TestBinaryJSONWalk(t *testing.T) {
require.Equal(t, len(test.expected), count)
}
}

func TestBinaryJSONOpaque(t *testing.T) {
var tests = []struct {
bj BinaryJSON
expectedOpaque Opaque
expectedOutput string
}{
{
BinaryJSON{
TypeCode: TypeCodeOpaque,
Value: []byte{233, 1, '9'},
},
Opaque{
TypeCode: 233,
Buf: []byte{'9'},
},
`"base64:type233:OQ=="`,
},
{
BinaryJSON{
TypeCode: TypeCodeOpaque,
Value: append([]byte{233, 0x80, 0x01}, make([]byte, 128)...),
},
Opaque{
TypeCode: 233,
Buf: make([]byte, 128),
},
`"base64:type233:AAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAA="`,
},
}

for _, test := range tests {
buf := []byte{}

require.Equal(t, test.expectedOpaque.TypeCode, test.bj.GetOpaqueFieldType())
require.Equal(t, test.expectedOpaque, test.bj.GetOpaque())

buf, err := test.bj.marshalTo(buf)
require.NoError(t, err)
require.Equal(t, string(buf), test.expectedOutput)
}
}
2 changes: 2 additions & 0 deletions types/json/constants.go
Original file line number Diff line number Diff line change
Expand Up @@ -40,6 +40,8 @@ const (
TypeCodeFloat64 TypeCode = 0x0b
// TypeCodeString indicates the JSON is a string.
TypeCodeString TypeCode = 0x0c
// TypeCodeOpaque indicates the JSON is a opaque
TypeCodeOpaque TypeCode = 0x0d
)

const (
Expand Down

0 comments on commit fe40d3c

Please sign in to comment.