Skip to content

Commit

Permalink
add json_type function and compare json with primitive types. (pingca…
Browse files Browse the repository at this point in the history
  • Loading branch information
hicqu authored May 24, 2017
1 parent aef9db4 commit 6962814
Showing 17 changed files with 486 additions and 40 deletions.
14 changes: 14 additions & 0 deletions ast/functions.go
Original file line number Diff line number Diff line change
@@ -277,6 +277,20 @@ const (
Uncompress = "uncompress"
UncompressedLength = "uncompressed_length"
ValidatePasswordStrength = "validate_password_strength"

// json functions
JSONType = "json_type"
JSONExtract = "json_extract"
JSONUnquote = "json_unquote"
JSONArray = "json_array"
JSONObject = "json_object"
JSONMerge = "json_merge"
JSONValid = "json_valid"
JSONSet = "json_set"
JSONInsert = "json_insert"
JSONReplace = "json_replace"
JSONRemove = "json_remove"
JSONContains = "json_contains"
)

// FuncCallExpr is for function expression.
17 changes: 16 additions & 1 deletion executor/executor_test.go
Original file line number Diff line number Diff line change
@@ -965,11 +965,26 @@ func (s *testSuite) TestJSON(c *C) {
tk.MustExec(`insert into test_json (id, a) values (3, null)`)
tk.MustExec(`insert into test_json (id, a) values (4, 'true')`)
tk.MustExec(`insert into test_json (id, a) values (5, '3')`)
tk.MustExec(`insert into test_json (id, a) values (5, '4.0')`)
tk.MustExec(`insert into test_json (id, a) values (6, '"string"')`)

var result *testkit.Result
result = tk.MustQuery(`select tj.a from test_json tj order by tj.id`)
result.Check(testkit.Rows(`{"a":[1,"2",{"aa":"bb"},4],"b":true}`, "null", "<nil>", "true", "3", `"string"`))
result.Check(testkit.Rows(`{"a":[1,"2",{"aa":"bb"},4],"b":true}`, "null", "<nil>", "true", "3", "4", `"string"`))

// check json_type function
result = tk.MustQuery(`select json_type(a) from test_json tj order by tj.id`)
result.Check(testkit.Rows("OBJECT", "NULL", "<nil>", "BOOLEAN", "INTEGER", "DOUBLE", "STRING"))

// check json compare with primitives.
result = tk.MustQuery(`select a from test_json tj where a = 3`)
result.Check(testkit.Rows("3"))
result = tk.MustQuery(`select a from test_json tj where a = 4.0`)
result.Check(testkit.Rows("4"))
result = tk.MustQuery(`select a from test_json tj where a = true`)
result.Check(testkit.Rows("true"))
result = tk.MustQuery(`select a from test_json tj where a = "string"`)
result.Check(testkit.Rows(`"string"`))
}

func (s *testSuite) TestToPBExpr(c *C) {
3 changes: 3 additions & 0 deletions expression/builtin.go
Original file line number Diff line number Diff line change
@@ -688,4 +688,7 @@ var funcs = map[string]functionClass{
ast.Uncompress: &uncompressFunctionClass{baseFunctionClass{ast.Uncompress, 1, 1}},
ast.UncompressedLength: &uncompressedLengthFunctionClass{baseFunctionClass{ast.UncompressedLength, 1, 1}},
ast.ValidatePasswordStrength: &validatePasswordStrengthFunctionClass{baseFunctionClass{ast.ValidatePasswordStrength, 1, 1}},

// json functions
ast.JSONType: &jsonTypeFunctionClass{baseFunctionClass{ast.JSONType, 1, 1}},
}
16 changes: 10 additions & 6 deletions expression/builtin_compare.go
Original file line number Diff line number Diff line change
@@ -231,14 +231,10 @@ func (s *builtinCompareSig) eval(row []types.Datum) (d types.Datum, err error) {
if err != nil {
return types.Datum{}, errors.Trace(err)
}

sc := s.ctx.GetSessionVars().StmtCtx
var a, b = args[0], args[1]
if s.op != opcode.NullEQ {
a, b, err = types.CoerceDatum(sc, a, b)
if err != nil {
return d, errors.Trace(err)
}
}

if a.IsNull() || b.IsNull() {
// For <=>, if a and b are both nil, return true.
// If a or b is nil, return false.
@@ -252,8 +248,16 @@ func (s *builtinCompareSig) eval(row []types.Datum) (d types.Datum, err error) {
return
}

if s.op != opcode.NullEQ {
if aa, bb, err := types.CoerceDatum(sc, a, b); err == nil {
a = aa
b = bb
}
}

n, err := a.CompareDatum(sc, b)
if err != nil {
// TODO: should deal with error here.
return d, errors.Trace(err)
}
var result bool
72 changes: 72 additions & 0 deletions expression/builtin_json.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,72 @@
// Copyright 2017 PingCAP, Inc.
//
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// See the License for the specific language governing permissions and
// limitations under the License.

package expression

import (
"github.com/juju/errors"
"github.com/pingcap/tidb/ast"
"github.com/pingcap/tidb/context"
"github.com/pingcap/tidb/mysql"
"github.com/pingcap/tidb/util/types"
"github.com/pingcap/tipb/go-tipb"
)

// jsonFunctionNameToPB is for pushdown json functions to storage engine.
var jsonFunctionNameToPB = map[string]tipb.ExprType{
ast.JSONType: tipb.ExprType_JsonType,
ast.JSONExtract: tipb.ExprType_JsonExtract,
ast.JSONUnquote: tipb.ExprType_JsonUnquote,
ast.JSONValid: tipb.ExprType_JsonValid,
ast.JSONObject: tipb.ExprType_JsonObject,
ast.JSONArray: tipb.ExprType_JsonArray,
ast.JSONMerge: tipb.ExprType_JsonMerge,
ast.JSONSet: tipb.ExprType_JsonSet,
ast.JSONInsert: tipb.ExprType_JsonInsert,
ast.JSONReplace: tipb.ExprType_JsonReplace,
ast.JSONRemove: tipb.ExprType_JsonRemove,
ast.JSONContains: tipb.ExprType_JsonContains,
}

var (
_ functionClass = &jsonTypeFunctionClass{}
)

type jsonTypeFunctionClass struct {
baseFunctionClass
}

type builtinJSONTypeSig struct {
baseBuiltinFunc
}

func (c *jsonTypeFunctionClass) getFunction(args []Expression, ctx context.Context) (builtinFunc, error) {
return &builtinJSONTypeSig{newBaseBuiltinFunc(args, ctx)}, errors.Trace(c.verifyArgs(args))
}

func (b *builtinJSONTypeSig) eval(row []types.Datum) (d types.Datum, err error) {
args, err := b.evalArgs(row)
if err != nil {
return d, errors.Trace(err)
}
sc := b.ctx.GetSessionVars().StmtCtx

if args[0].Kind() != types.KindNull {
var djson types.Datum
djson, err = args[0].ConvertTo(sc, types.NewFieldType(mysql.TypeJSON))
if err == nil {
d.SetString(djson.GetMysqlJSON().Type())
}
}
return d, errors.Trace(err)
}
3 changes: 3 additions & 0 deletions expression/typeinferer.go
Original file line number Diff line number Diff line change
@@ -423,6 +423,9 @@ func (v *typeInferrer) handleFuncCallExpr(x *ast.FuncCallExpr) {
tp = types.NewFieldType(mysql.TypeBlob)
case ast.Uncompress:
tp = types.NewFieldType(mysql.TypeLongBlob)
case ast.JSONType:
tp = types.NewFieldType(mysql.TypeVarString)
chs = v.defaultCharset
case ast.AnyValue:
tp = x.Args[0].GetType()
case ast.RowFunc:
1 change: 1 addition & 0 deletions expression/typeinferer_test.go
Original file line number Diff line number Diff line change
@@ -327,6 +327,7 @@ func (ts *testTypeInferrerSuite) TestInferType(c *C) {
{`time_to_sec("23:59:59")`, mysql.TypeLonglong, charset.CharsetBin, mysql.BinaryFlag},
{`inet6_aton('FE80::AAAA:0000:00C2:0002')`, mysql.TypeVarString, charset.CharsetUTF8, 0},
{`is_ipv4_mapped(c_varbinary)`, mysql.TypeLonglong, charset.CharsetBin, mysql.BinaryFlag},
{`json_type('3')`, mysql.TypeVarString, charset.CharsetUTF8, 0},
}
for _, tt := range tests {
ctx := testKit.Se.(context.Context)
1 change: 1 addition & 0 deletions parser/misc.go
Original file line number Diff line number Diff line change
@@ -534,6 +534,7 @@ var tokenMap = map[string]int{
"JSON": jsonType,
"JSON_EXTRACT": jsonExtract,
"JSON_UNQUOTE": jsonUnquote,
"JSON_TYPE": jsonTypeFunc,
"SECOND_MICROSECOND": secondMicrosecond,
"MINUTE_MICROSECOND": minuteMicrosecond,
"MINUTE_SECOND": minuteSecond,
10 changes: 9 additions & 1 deletion parser/parser.y
Original file line number Diff line number Diff line change
@@ -285,6 +285,7 @@ import (
isNull "ISNULL"
jsonExtract "JSON_EXTRACT"
jsonUnquote "JSON_UNQUOTE"
jsonTypeFunc "JSON_TYPE"
kill "KILL"
lastInsertID "LAST_INSERT_ID"
lcase "LCASE"
@@ -2348,7 +2349,7 @@ NotKeywordToken:
| "AES_DECRYPT" | "AES_ENCRYPT" | "QUOTE"
| "ANY_VALUE" | "INET_ATON" | "INET_NTOA" | "INET6_ATON" | "INET6_NTOA" | "IS_FREE_LOCK" | "IS_IPV4" | "IS_IPV4_COMPAT" | "IS_IPV4_MAPPED" | "IS_IPV6" | "IS_USED_LOCK" | "MASTER_POS_WAIT" | "NAME_CONST" | "RELEASE_ALL_LOCKS" | "UUID" | "UUID_SHORT"
| "COMPRESS" | "DECODE" | "DES_DECRYPT" | "DES_ENCRYPT" | "ENCODE" | "ENCRYPT" | "MD5" | "OLD_PASSWORD" | "RANDOM_BYTES" | "SHA1" | "SHA" | "SHA2" | "UNCOMPRESS" | "UNCOMPRESSED_LENGTH" | "VALIDATE_PASSWORD_STRENGTH"
| "JSON_EXTRACT" | "JSON_UNQUOTE"
| "JSON_EXTRACT" | "JSON_UNQUOTE" | "JSON_TYPE"

/************************************************************************************
*
@@ -3657,6 +3658,13 @@ FunctionCallNonKeyword:
Args: []ast.ExprNode{$3.(ast.ExprNode)},
}
}
| "JSON_TYPE" '(' Expression ')'
{
$$ = &ast.FuncCallExpr{
FnName: model.NewCIStr($1),
Args: []ast.ExprNode{$3.(ast.ExprNode)},
}
}

GetFormatSelector:
"DATE"
3 changes: 3 additions & 0 deletions parser/parser_test.go
Original file line number Diff line number Diff line change
@@ -1367,6 +1367,9 @@ func (s *testParserSuite) TestType(c *C) {
// for https://github.com/pingcap/tidb/issues/312
{`create table t (c float(53));`, true},
{`create table t (c float(54));`, false},

// for json type
{`create table t (a JSON);`, true},
}
s.RunTest(c, table)
}
4 changes: 0 additions & 4 deletions util/types/compare.go
Original file line number Diff line number Diff line change
@@ -1,7 +1,3 @@
// Copyright 2014 The ql Authors. All rights reserved.
// Use of this source code is governed by a BSD-style
// license that can be found in the LICENSES/QL-LICENSE file.

// Copyright 2015 PingCAP, Inc.
//
// Licensed under the Apache License, Version 2.0 (the "License");
52 changes: 39 additions & 13 deletions util/types/datum.go
Original file line number Diff line number Diff line change
@@ -406,6 +406,10 @@ func (d *Datum) SetValue(val interface{}) {
// CompareDatum compares datum to another datum.
// TODO: return error properly.
func (d *Datum) CompareDatum(sc *variable.StatementContext, ad Datum) (int, error) {
if d.k == KindMysqlJSON && ad.k != KindMysqlJSON {
cmp, err := ad.CompareDatum(sc, *d)
return cmp * -1, errors.Trace(err)
}
switch ad.k {
case KindNull:
if d.k == KindNull {
@@ -446,6 +450,8 @@ func (d *Datum) CompareDatum(sc *variable.StatementContext, ad Datum) (int, erro
return d.compareMysqlHex(sc, ad.GetMysqlHex())
case KindMysqlSet:
return d.compareMysqlSet(sc, ad.GetMysqlSet())
case KindMysqlJSON:
return d.compareMysqlJSON(sc, ad.GetMysqlJSON())
case KindMysqlTime:
return d.compareMysqlTime(sc, ad.GetMysqlTime())
case KindRow:
@@ -629,6 +635,31 @@ func (d *Datum) compareMysqlSet(sc *variable.StatementContext, set Set) (int, er
}
}

func (d *Datum) compareMysqlJSON(sc *variable.StatementContext, target json.JSON) (int, error) {
var origin json.JSON

switch d.Kind() {
case KindMysqlJSON:
origin = d.x.(json.JSON)
case KindInt64, KindUint64:
i64 := d.GetInt64()
origin = json.CreateJSON(i64)
case KindFloat32, KindFloat64:
f64 := d.GetFloat64()
origin = json.CreateJSON(f64)
case KindMysqlDecimal:
f64, _ := d.GetMysqlDecimal().ToFloat64()
origin = json.CreateJSON(f64)
case KindString, KindBytes:
s := d.GetString()
origin = json.CreateJSON(s)
default:
s, _ := d.ToString()
origin = json.CreateJSON(s)
}
return json.CompareJSON(origin, target)
}

func (d *Datum) compareMysqlTime(sc *variable.StatementContext, time Time) (int, error) {
switch d.k {
case KindString, KindBytes:
@@ -1145,24 +1176,19 @@ func (d *Datum) convertToMysqlSet(sc *variable.StatementContext, target *FieldTy
return ret, nil
}

func (d *Datum) convertToMysqlJSON(sc *variable.StatementContext, target *FieldType) (Datum, error) {
var (
s string
j json.JSON
ret Datum
err error
)
func (d *Datum) convertToMysqlJSON(sc *variable.StatementContext, target *FieldType) (ret Datum, err error) {
switch d.k {
case KindString, KindBytes:
s = d.GetString()
var j json.JSON
if j, err = json.ParseFromString(d.GetString()); err == nil {
ret.SetMysqlJSON(j)
}
case KindMysqlJSON:
ret = *d
default:
return invalidConv(d, target.Tp)
}
j, err = json.ParseFromString(s)
if err == nil {
ret.SetValue(j)
}
return ret, err
return ret, errors.Trace(err)
}

// ToBool converts to a bool.
Loading

0 comments on commit 6962814

Please sign in to comment.