Skip to content

Commit

Permalink
json: add json_storage_free function
Browse files Browse the repository at this point in the history
Signed-off-by: YangKeao <[email protected]>
  • Loading branch information
YangKeao committed Oct 21, 2022
1 parent 8d57ed9 commit 10992ec
Show file tree
Hide file tree
Showing 8 changed files with 128 additions and 1 deletion.
1 change: 1 addition & 0 deletions expression/builtin.go
Original file line number Diff line number Diff line change
Expand Up @@ -883,6 +883,7 @@ var funcs = map[string]functionClass{
ast.JSONPretty: &jsonPrettyFunctionClass{baseFunctionClass{ast.JSONPretty, 1, 1}},
ast.JSONQuote: &jsonQuoteFunctionClass{baseFunctionClass{ast.JSONQuote, 1, 1}},
ast.JSONSearch: &jsonSearchFunctionClass{baseFunctionClass{ast.JSONSearch, 3, -1}},
ast.JSONStorageFree: &jsonStorageFreeFunctionClass{baseFunctionClass{ast.JSONStorageFree, 1, 1}},
ast.JSONStorageSize: &jsonStorageSizeFunctionClass{baseFunctionClass{ast.JSONStorageSize, 1, 1}},
ast.JSONDepth: &jsonDepthFunctionClass{baseFunctionClass{ast.JSONDepth, 1, 1}},
ast.JSONKeys: &jsonKeysFunctionClass{baseFunctionClass{ast.JSONKeys, 1, 2}},
Expand Down
41 changes: 41 additions & 0 deletions expression/builtin_json.go
Original file line number Diff line number Diff line change
Expand Up @@ -1392,6 +1392,43 @@ func (b *builtinJSONSearchSig) evalJSON(row chunk.Row) (res types.BinaryJSON, is
return obj.Search(containType, searchStr, escape, nil)
}

type jsonStorageFreeFunctionClass struct {
baseFunctionClass
}

type builtinJSONStorageFreeSig struct {
baseBuiltinFunc
}

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

func (c *jsonStorageFreeFunctionClass) getFunction(ctx sessionctx.Context, args []Expression) (builtinFunc, error) {
if err := c.verifyArgs(args); err != nil {
return nil, err
}

bf, err := newBaseBuiltinFuncWithTp(ctx, c.funcName, args, types.ETInt, types.ETJson)
if err != nil {
return nil, err
}
sig := &builtinJSONStorageFreeSig{bf}
sig.setPbCode(tipb.ScalarFuncSig_JsonStorageFreeSig)
return sig, nil
}

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

return 0, false, nil
}

type jsonStorageSizeFunctionClass struct {
baseFunctionClass
}
Expand Down Expand Up @@ -1459,6 +1496,10 @@ func (c *jsonDepthFunctionClass) getFunction(ctx sessionctx.Context, args []Expr
}

func (b *builtinJSONDepthSig) evalInt(row chunk.Row) (res int64, isNull bool, err error) {
// as TiDB doesn't support partial update json value, so only check the
// json format and whether it's NULL. For NULL return NULL, for invalid json, return
// an error, otherwise return 0

obj, isNull, err := b.args[0].EvalJSON(b.ctx, row)
if isNull || err != nil {
return res, isNull, err
Expand Down
43 changes: 43 additions & 0 deletions expression/builtin_json_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -985,6 +985,49 @@ func TestJSONValid(t *testing.T) {
}
}

func TestJSONStorageFree(t *testing.T) {
ctx := createContext(t)
fc := funcs[ast.JSONStorageFree]
tbl := []struct {
input []interface{}
expected interface{}
success bool
}{
// Tests scalar arguments
{[]interface{}{`null`}, 0, true},
{[]interface{}{`true`}, 0, true},
{[]interface{}{`1`}, 0, true},
{[]interface{}{`"1"`}, 0, true},
// Tests nil arguments
{[]interface{}{nil}, nil, true},
// Tests valid json documents
{[]interface{}{`{}`}, 0, true},
{[]interface{}{`{"a":1}`}, 0, true},
{[]interface{}{`[{"a":{"a":1},"b":2}]`}, 0, true},
{[]interface{}{`{"a": 1000, "b": "wxyz", "c": "[1, 3, 5, 7]"}`}, 0, true},
// Tests invalid json documents
{[]interface{}{`[{"a":1]`}, 0, false},
{[]interface{}{`[{a":1]`}, 0, false},
}
for _, tt := range tbl {
args := types.MakeDatums(tt.input...)
f, err := fc.getFunction(ctx, datumsToConstants(args))
require.NoError(t, err)
d, err := evalBuiltinFunc(f, chunk.Row{})
if tt.success {
require.NoError(t, err)

if tt.expected == nil {
require.True(t, d.IsNull())
} else {
require.Equal(t, int64(tt.expected.(int)), d.GetInt64())
}
} else {
require.Error(t, err)
}
}
}

func TestJSONStorageSize(t *testing.T) {
ctx := createContext(t)
fc := funcs[ast.JSONStorageSize]
Expand Down
27 changes: 27 additions & 0 deletions expression/builtin_json_vec.go
Original file line number Diff line number Diff line change
Expand Up @@ -103,6 +103,33 @@ func vecJSONModify(ctx sessionctx.Context, args []Expression, bufAllocator colum
return nil
}

func (b *builtinJSONStorageFreeSig) vectorized() bool {
return true
}

func (b *builtinJSONStorageFreeSig) vecEvalInt(input *chunk.Chunk, result *chunk.Column) error {
n := input.NumRows()
buf, err := b.bufAllocator.get()
if err != nil {
return err
}
defer b.bufAllocator.put(buf)
if err := b.args[0].VecEvalJSON(b.ctx, input, buf); err != nil {
return err
}
result.ResizeInt64(n, false)
result.MergeNulls(buf)
int64s := result.Int64s()
for i := 0; i < n; i++ {
if result.IsNull(i) {
continue
}

int64s[i] = 0
}
return nil
}

func (b *builtinJSONStorageSizeSig) vectorized() bool {
return true
}
Expand Down
12 changes: 12 additions & 0 deletions expression/integration_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -7755,3 +7755,15 @@ func TestFix38127(t *testing.T) {
tk.MustQuery("select from_unixtime(dou, '%Y-%m-%d') from t").Check(testkit.Rows("<nil>"))
tk.MustQuery("select from_unixtime(varc, '%Y-%m-%d') from t").Check(testkit.Rows("<nil>"))
}

func TestJSONStorageFree(t *testing.T) {
store := testkit.CreateMockStore(t)
tk := testkit.NewTestKit(t, store)
tk.MustExec("use test")
tk.MustQuery("select json_storage_free(NULL)").Check(testkit.Rows("<nil>"))
tk.MustQuery("select json_storage_free('{}')").Check(testkit.Rows("0"))
tk.MustQuery("select json_storage_free('1')").Check(testkit.Rows("0"))
tk.MustQuery(`select json_storage_free('{"a": "b"}')`).Check(testkit.Rows("0"))
err := tk.ExecToErr(`select json_storage_free('{"c":["a","b"]`)
require.Error(t, err, "[json:3140]Invalid JSON text: The document root must not be followed by other values.")
}
2 changes: 1 addition & 1 deletion go.mod
Original file line number Diff line number Diff line change
Expand Up @@ -71,7 +71,7 @@ require (
github.com/pingcap/log v1.1.0
github.com/pingcap/sysutil v0.0.0-20220114020952-ea68d2dbf5b4
github.com/pingcap/tidb/parser v0.0.0-20211011031125-9b13dc409c5e
github.com/pingcap/tipb v0.0.0-20220824081009-0714a57aff1d
github.com/pingcap/tipb v0.0.0-20221020071514-cd933387bcb5
github.com/pkg/errors v0.9.1
github.com/prometheus/client_golang v1.13.0
github.com/prometheus/client_model v0.2.0
Expand Down
2 changes: 2 additions & 0 deletions go.sum
Original file line number Diff line number Diff line change
Expand Up @@ -768,6 +768,8 @@ github.com/pingcap/sysutil v0.0.0-20220114020952-ea68d2dbf5b4 h1:HYbcxtnkN3s5tqr
github.com/pingcap/sysutil v0.0.0-20220114020952-ea68d2dbf5b4/go.mod h1:sDCsM39cGiv2vwunZkaFA917vVkqDTGSPbbV7z4Oops=
github.com/pingcap/tipb v0.0.0-20220824081009-0714a57aff1d h1:kWYridgsn8xSKYJ2EkXp7uj5HwJnG5snpY3XP8oYmPU=
github.com/pingcap/tipb v0.0.0-20220824081009-0714a57aff1d/go.mod h1:A7mrd7WHBl1o63LE2bIBGEJMTNWXqhgmYiOvMLxozfs=
github.com/pingcap/tipb v0.0.0-20221020071514-cd933387bcb5 h1:Yoo8j5xQGxjlsC3yt0ndsiAz0WZXED9rzsKmEN0U0DY=
github.com/pingcap/tipb v0.0.0-20221020071514-cd933387bcb5/go.mod h1:A7mrd7WHBl1o63LE2bIBGEJMTNWXqhgmYiOvMLxozfs=
github.com/pkg/browser v0.0.0-20180916011732-0a3d74bf9ce4 h1:49lOXmGaUpV9Fz3gd7TFZY106KVlPVa5jcYD1gaQf98=
github.com/pkg/browser v0.0.0-20180916011732-0a3d74bf9ce4/go.mod h1:4OwLy04Bl9Ef3GJJCoec+30X3LQs/0/m4HFRt/2LUSA=
github.com/pkg/errors v0.8.0/go.mod h1:bwawxfHBFNV+L2hUp1rHADufV3IMtnDRdf1r5NINEl0=
Expand Down
1 change: 1 addition & 0 deletions parser/ast/functions.go
Original file line number Diff line number Diff line change
Expand Up @@ -341,6 +341,7 @@ const (
JSONPretty = "json_pretty"
JSONQuote = "json_quote"
JSONSearch = "json_search"
JSONStorageFree = "json_storage_free"
JSONStorageSize = "json_storage_size"
JSONDepth = "json_depth"
JSONKeys = "json_keys"
Expand Down

0 comments on commit 10992ec

Please sign in to comment.