Skip to content

Commit

Permalink
Add unittest of expr.Eval for JSON_DEPTH
Browse files Browse the repository at this point in the history
Improve codes of function `getFunction` of jsonDepthFunctionClass
  • Loading branch information
pingyu committed Nov 24, 2018
1 parent 1e0876f commit 57dec44
Show file tree
Hide file tree
Showing 6 changed files with 227 additions and 1 deletion.
29 changes: 28 additions & 1 deletion expression/builtin_json.go
Original file line number Diff line number Diff line change
Expand Up @@ -61,6 +61,7 @@ var (
_ builtinFunc = &builtinJSONRemoveSig{}
_ builtinFunc = &builtinJSONMergeSig{}
_ builtinFunc = &builtinJSONContainsSig{}
_ builtinFunc = &builtinJSONDepthSig{}
_ builtinFunc = &builtinJSONKeysSig{}
_ builtinFunc = &builtinJSONKeys2ArgsSig{}
_ builtinFunc = &builtinJSONLengthSig{}
Expand Down Expand Up @@ -759,8 +760,34 @@ type jsonDepthFunctionClass struct {
baseFunctionClass
}

type builtinJSONDepthSig struct {
baseBuiltinFunc
}

func (b *builtinJSONDepthSig) Clone() builtinFunc {
newSig := &builtinJSONDepthSig{}
newSig.cloneFrom(&b.baseBuiltinFunc)
return newSig
}

func (c *jsonDepthFunctionClass) getFunction(ctx sessionctx.Context, args []Expression) (builtinFunc, error) {
return nil, errFunctionNotExists.GenWithStackByArgs("FUNCTION", "JSON_DEPTH")
if err := c.verifyArgs(args); err != nil {
return nil, errors.Trace(err)
}

bf := newBaseBuiltinFuncWithTp(ctx, args, types.ETInt, types.ETJson)
sig := &builtinJSONDepthSig{bf}
sig.setPbCode(tipb.ScalarFuncSig_JsonDepthSig)
return sig, nil
}

func (b *builtinJSONDepthSig) evalInt(row chunk.Row) (res int64, isNull bool, err error) {
obj, isNull, err := b.args[0].EvalJSON(b.ctx, row)
if isNull || err != nil {
return res, isNull, errors.Trace(err)
}

return int64(obj.GetElemDepth()), false, nil
}

type jsonKeysFunctionClass struct {
Expand Down
59 changes: 59 additions & 0 deletions expression/builtin_json_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -565,3 +565,62 @@ func (s *testEvaluatorSuite) TestJSONKeys(c *C) {
}
}
}

func (s *testEvaluatorSuite) TestJSONDepth(c *C) {
defer testleak.AfterTest(c)()
fc := funcs[ast.JSONDepth]
tbl := []struct {
input []interface{}
expected interface{}
success bool
}{
// Tests scalar arguments
{[]interface{}{`null`}, 1, true},
{[]interface{}{`true`}, 1, true},
{[]interface{}{`false`}, 1, true},
{[]interface{}{`1`}, 1, true},
{[]interface{}{`-1`}, 1, true},
{[]interface{}{`1.1`}, 1, true},
{[]interface{}{`"1"`}, 1, true},
// Tests nil arguments
{[]interface{}{nil}, nil, true},
// Tests depth
{[]interface{}{`{}`}, 1, true},
{[]interface{}{`[]`}, 1, true},
{[]interface{}{`[10, 20]`}, 2, true},
{[]interface{}{`[[], {}]`}, 2, true},
{[]interface{}{`{"Name": "Homer"}`}, 2, true},
{[]interface{}{`[10, {"a": 20}]`}, 3, true},
{[]interface{}{`{"Person": {"Name": "Homer", "Age": 39, "Hobbies": ["Eating", "Sleeping"]} }`}, 4, true},
{[]interface{}{`{"a":1}`}, 2, true},
{[]interface{}{`{"a":[1]}`}, 3, true},
{[]interface{}{`{"b":2, "c":3}`}, 2, true},
{[]interface{}{`[1]`}, 2, true},
{[]interface{}{`[1,2]`}, 2, true},
{[]interface{}{`[1,2,[1,3]]`}, 3, true},
{[]interface{}{`[1,2,[1,[5,[3]]]]`}, 5, true},
{[]interface{}{`[1,2,[1,[5,{"a":[2,3]}]]]`}, 6, true},
{[]interface{}{`[{"a":1}]`}, 3, true},
{[]interface{}{`[{"a":1,"b":2}]`}, 3, true},
{[]interface{}{`[{"a":{"a":1},"b":2}]`}, 4, true},
// Tests non-json
{[]interface{}{`a`}, nil, false},
}
for _, t := range tbl {
args := types.MakeDatums(t.input...)
f, err := fc.getFunction(s.ctx, s.datumsToConstants(args))
c.Assert(err, IsNil)
d, err := evalBuiltinFunc(f, chunk.Row{})
if t.success {
c.Assert(err, IsNil)

if t.expected == nil {
c.Assert(d.IsNull(), IsTrue)
} else {
c.Assert(d.GetInt64(), Equals, int64(t.expected.(int)))
}
} else {
c.Assert(err, NotNil)
}
}
}
16 changes: 16 additions & 0 deletions expression/distsql_builtin.go
Original file line number Diff line number Diff line change
Expand Up @@ -438,6 +438,8 @@ func getSignatureByPB(ctx sessionctx.Context, sigCode tipb.ScalarFuncSig, tp *ti
f = &builtinLikeSig{base}
case tipb.ScalarFuncSig_JsonLengthSig:
f = &builtinJSONLengthSig{base}
case tipb.ScalarFuncSig_JsonDepthSig:
f = &builtinJSONDepthSig{base}

case tipb.ScalarFuncSig_InInt:
f = &builtinInIntSig{base}
Expand Down Expand Up @@ -507,6 +509,8 @@ func PBToExpr(expr *tipb.Expr, tps []*types.FieldType, sc *stmtctx.StatementCont
return convertDuration(expr.Val)
case tipb.ExprType_MysqlTime:
return convertTime(expr.Val, expr.FieldType, sc.TimeZone)
case tipb.ExprType_MysqlJson:
return convertJSON(expr.Val)
}
if expr.Tp != tipb.ExprType_ScalarFunc {
panic("should be a tipb.ExprType_ScalarFunc")
Expand Down Expand Up @@ -642,3 +646,15 @@ func convertDuration(val []byte) (*Constant, error) {
d.SetMysqlDuration(types.Duration{Duration: time.Duration(i), Fsp: types.MaxFsp})
return &Constant{Value: d, RetType: types.NewFieldType(mysql.TypeDuration)}, nil
}

func convertJSON(val []byte) (*Constant, error) {
var d types.Datum
_, d, err := codec.DecodeOne(val)
if err != nil {
return nil, errors.Errorf("invalid json % x", val)
}
if d.Kind() != types.KindMysqlJSON {
return nil, errors.Errorf("invalid Datum.Kind() %d", d.Kind())
}
return &Constant{Value: d, RetType: types.NewFieldType(mysql.TypeJSON)}, nil
}
65 changes: 65 additions & 0 deletions expression/distsql_builtin_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -20,6 +20,7 @@ import (
"github.com/pingcap/parser/mysql"
"github.com/pingcap/tidb/sessionctx/stmtctx"
"github.com/pingcap/tidb/types"
"github.com/pingcap/tidb/types/json"
"github.com/pingcap/tidb/util/chunk"
"github.com/pingcap/tidb/util/codec"
"github.com/pingcap/tipb/go-tipb"
Expand Down Expand Up @@ -88,10 +89,26 @@ func (s *testEvalSuite) TestEval(c *C) {
datumExpr(types.NewDecimalDatum(types.NewDecFromFloatForTest(1.1))),
types.NewDecimalDatum(types.NewDecFromFloatForTest(1.1)),
},
// Columns.
{
columnExpr(0),
types.NewIntDatum(100),
},
// Scalar Functions.
{
scalarFunctionExpr(tipb.ScalarFuncSig_JsonDepthSig,
toPBFieldType(newIntFieldType()),
jsonDatumExpr(`true`),
),
types.NewIntDatum(1),
},
{
scalarFunctionExpr(tipb.ScalarFuncSig_JsonDepthSig,
toPBFieldType(newIntFieldType()),
jsonDatumExpr(`[10, {"a": 20}]`),
),
types.NewIntDatum(3),
},
}
sc := new(stmtctx.StatementContext)
for _, tt := range tests {
Expand Down Expand Up @@ -152,15 +169,63 @@ func datumExpr(d types.Datum) *tipb.Expr {
if err != nil {
log.Warnf("err happened when EncodeDecimal in datumExpr:%s", err.Error())
}
case types.KindMysqlJSON:
expr.Tp = tipb.ExprType_MysqlJson
var err error
expr.Val = make([]byte, 0, 1024)
expr.Val, err = codec.EncodeValue(nil, expr.Val, d)
if err != nil {
log.Warnf("err happened when EncodeValue of JSON in datumExpr:%s", err.Error())
}
default:
expr.Tp = tipb.ExprType_Null
}
return expr
}

func jsonDatumExpr(s string) *tipb.Expr {
var d types.Datum
j, err := json.ParseBinaryFromString(s)
if err != nil {
log.Warnf("err happened when json.ParseBinaryFromString in jsonDatumExpr:%s", err.Error())
}
d.SetMysqlJSON(j)
return datumExpr(d)
}

func columnExpr(columnID int64) *tipb.Expr {
expr := new(tipb.Expr)
expr.Tp = tipb.ExprType_ColumnRef
expr.Val = codec.EncodeInt(nil, columnID)
return expr
}

// toPBFieldType converts *types.FieldType to *tipb.FieldType.
func toPBFieldType(ft *types.FieldType) *tipb.FieldType {
return &tipb.FieldType{
Tp: int32(ft.Tp),
Flag: uint32(ft.Flag),
Flen: int32(ft.Flen),
Decimal: int32(ft.Decimal),
Charset: ft.Charset,
Collate: collationToProto(ft.Collate),
}
}

func newIntFieldType() *types.FieldType {
return &types.FieldType{
Tp: mysql.TypeLonglong,
Flen: mysql.MaxIntWidth,
Decimal: 0,
Flag: mysql.BinaryFlag,
}
}

func scalarFunctionExpr(sigCode tipb.ScalarFuncSig, retType *tipb.FieldType, args ...*tipb.Expr) *tipb.Expr {
return &tipb.Expr{
Tp: tipb.ExprType_ScalarFunc,
Sig: sigCode,
Children: args,
FieldType: retType,
}
}
39 changes: 39 additions & 0 deletions types/json/binary_functions.go
Original file line number Diff line number Diff line change
Expand Up @@ -731,3 +731,42 @@ func ContainsBinary(obj, target BinaryJSON) bool {
return CompareBinary(obj, target) == 0
}
}

// GetElemDepth for JSON_DEPTH
// Returns the maximum depth of a JSON document
// rules referenced by MySQL JSON_DEPTH function
// [https://dev.mysql.com/doc/refman/5.7/en/json-attribute-functions.html#function_json-depth]
// 1) An empty array, empty object, or scalar value has depth 1.
// 2) A nonempty array containing only elements of depth 1 or nonempty object containing only member values of depth 1 has depth 2.
// 3) Otherwise, a JSON document has depth greater than 2.
// e.g. depth of '{}', '[]', 'true': 1
// e.g. depth of '[10, 20]', '[[], {}]': 2
// e.g. depth of '[10, {"a": 20}]': 3
func (bj BinaryJSON) GetElemDepth() int {
switch bj.TypeCode {
case TypeCodeObject:
len := bj.GetElemCount()
maxDepth := 0
for i := 0; i < len; i++ {
obj := bj.objectGetVal(i)
depth := obj.GetElemDepth()
if depth > maxDepth {
maxDepth = depth
}
}
return maxDepth + 1
case TypeCodeArray:
len := bj.GetElemCount()
maxDepth := 0
for i := 0; i < len; i++ {
obj := bj.arrayGetElem(i)
depth := obj.GetElemDepth()
if depth > maxDepth {
maxDepth = depth
}
}
return maxDepth + 1
default:
return 1
}
}
20 changes: 20 additions & 0 deletions types/json/binary_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -324,3 +324,23 @@ func (s *testJSONSuite) TestBinaryJSONContains(c *C) {
c.Assert(ContainsBinary(obj, target), Equals, tt.expected)
}
}

func (s *testJSONSuite) TestBinaryJSONDepth(c *C) {
var tests = []struct {
input string
expected int
}{
{`{}`, 1},
{`[]`, 1},
{`true`, 1},
{`[10, 20]`, 2},
{`[[], {}]`, 2},
{`[10, {"a": 20}]`, 3},
{`{"Person": {"Name": "Homer", "Age": 39, "Hobbies": ["Eating", "Sleeping"]} }`, 4},
}

for _, tt := range tests {
obj := mustParseBinaryFromString(c, tt.input)
c.Assert(obj.GetElemDepth(), Equals, tt.expected)
}
}

0 comments on commit 57dec44

Please sign in to comment.