Skip to content

Commit 7367308

Browse files
authored
expression: introduce new function for JSON array (#60728)
ref #60649
1 parent 1eb66da commit 7367308

File tree

14 files changed

+11265
-10744
lines changed

14 files changed

+11265
-10744
lines changed

pkg/expression/bench_test.go

Lines changed: 20 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -299,6 +299,22 @@ func (g *charInt64Gener) gen() any {
299299
return int64(nanosecond)
300300
}
301301

302+
type jsonArrayGener struct {
303+
rand *defaultRandGen
304+
}
305+
306+
func newJSONArrayGener() *jsonArrayGener {
307+
return &jsonArrayGener{newDefaultRandGen()}
308+
}
309+
310+
func (g *jsonArrayGener) gen() any {
311+
v := make([]any, 4)
312+
for i := range len(v) {
313+
v[i] = int64(g.rand.Int())
314+
}
315+
return types.CreateBinaryJSON(v)
316+
}
317+
302318
// selectStringGener select one string randomly from the candidates array
303319
type selectStringGener struct {
304320
candidates []string
@@ -1459,7 +1475,10 @@ func genVecBuiltinFuncBenchCase(ctx BuildContext, funcName string, testCase vecE
14591475
}
14601476

14611477
var err error
1462-
if funcName == ast.Cast {
1478+
if funcName == ast.JSONSumCrc32 {
1479+
fc := &jsonSumCRC32FunctionClass{baseFunctionClass{ast.JSONSumCrc32, 1, 1}, fts[0]}
1480+
baseFunc, err = fc.getFunction(ctx, cols)
1481+
} else if funcName == ast.Cast {
14631482
var fc functionClass
14641483
tp := eType2FieldType(testCase.retEvalType)
14651484
switch testCase.retEvalType {

pkg/expression/builtin_json.go

Lines changed: 110 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -18,6 +18,8 @@ import (
1818
"bytes"
1919
"context"
2020
goJSON "encoding/json"
21+
"fmt"
22+
"hash/crc32"
2123
"strconv"
2224
"strings"
2325

@@ -35,6 +37,7 @@ import (
3537

3638
var (
3739
_ functionClass = &jsonTypeFunctionClass{}
40+
_ functionClass = &jsonSumCRC32FunctionClass{}
3841
_ functionClass = &jsonExtractFunctionClass{}
3942
_ functionClass = &jsonUnquoteFunctionClass{}
4043
_ functionClass = &jsonQuoteFunctionClass{}
@@ -64,6 +67,7 @@ var (
6467
_ functionClass = &jsonLengthFunctionClass{}
6568

6669
_ builtinFunc = &builtinJSONTypeSig{}
70+
_ builtinFunc = &builtinJSONSumCRC32Sig{}
6771
_ builtinFunc = &builtinJSONQuoteSig{}
6872
_ builtinFunc = &builtinJSONUnquoteSig{}
6973
_ builtinFunc = &builtinJSONArraySig{}
@@ -180,6 +184,112 @@ func (b *builtinJSONTypeSig) evalString(ctx EvalContext, row chunk.Row) (val str
180184
return j.Type(), false, nil
181185
}
182186

187+
type jsonSumCRC32FunctionClass struct {
188+
baseFunctionClass
189+
190+
tp *types.FieldType
191+
}
192+
193+
func (c *jsonSumCRC32FunctionClass) verifyArgs(ctx EvalContext, args []Expression) error {
194+
if err := c.baseFunctionClass.verifyArgs(args); err != nil {
195+
return err
196+
}
197+
198+
if args[0].GetType(ctx).EvalType() != types.ETJson {
199+
return ErrInvalidTypeForJSON.GenWithStackByArgs(1, "JSON_SUM_CRC32")
200+
}
201+
202+
return nil
203+
}
204+
205+
func (c *jsonSumCRC32FunctionClass) getFunction(ctx BuildContext, args []Expression) (sig builtinFunc, err error) {
206+
if err := c.verifyArgs(ctx.GetEvalCtx(), args); err != nil {
207+
return nil, err
208+
}
209+
arrayType := c.tp.ArrayType()
210+
switch arrayType.GetType() {
211+
case mysql.TypeYear, mysql.TypeJSON, mysql.TypeFloat, mysql.TypeNewDecimal:
212+
return nil, ErrNotSupportedYet.GenWithStackByArgs(fmt.Sprintf("calculating json_sum_crc32 on array of %s", arrayType.String()))
213+
}
214+
if arrayType.EvalType() == types.ETString && arrayType.GetCharset() != charset.CharsetUTF8MB4 && arrayType.GetCharset() != charset.CharsetBin {
215+
return nil, ErrNotSupportedYet.GenWithStackByArgs("unsupported charset")
216+
}
217+
if arrayType.EvalType() == types.ETString && arrayType.GetFlen() == types.UnspecifiedLength {
218+
return nil, ErrNotSupportedYet.GenWithStackByArgs("calculating json_sum_crc32 on array of char/binary BLOBs with unspecified length")
219+
}
220+
221+
bf, err := newBaseBuiltinFunc(ctx, c.funcName, args, c.tp)
222+
if err != nil {
223+
return nil, err
224+
}
225+
sig = &builtinJSONSumCRC32Sig{bf}
226+
return sig, nil
227+
}
228+
229+
type builtinJSONSumCRC32Sig struct {
230+
baseBuiltinFunc
231+
}
232+
233+
func (b *builtinJSONSumCRC32Sig) Clone() builtinFunc {
234+
newSig := &builtinJSONSumCRC32Sig{}
235+
newSig.cloneFrom(&b.baseBuiltinFunc)
236+
return newSig
237+
}
238+
239+
func (b *builtinJSONSumCRC32Sig) evalInt(ctx EvalContext, row chunk.Row) (res int64, isNull bool, err error) {
240+
val, isNull, err := b.args[0].EvalJSON(ctx, row)
241+
if isNull || err != nil {
242+
return res, isNull, err
243+
}
244+
245+
if val.TypeCode != types.JSONTypeCodeArray {
246+
return 0, false, ErrInvalidTypeForJSON.GenWithStackByArgs(1, "JSON_SUM_CRC32")
247+
}
248+
249+
ft := b.tp.ArrayType()
250+
f := convertJSON2Tp(ft.EvalType())
251+
if f == nil {
252+
return 0, false, ErrNotSupportedYet.GenWithStackByArgs(fmt.Sprintf("calculating sum of %s", ft.String()))
253+
}
254+
255+
var sum int64
256+
for i := range val.GetElemCount() {
257+
item, err := f(fakeSctx, val.ArrayGetElem(i), ft)
258+
if err != nil {
259+
if ErrInvalidJSONForFuncIndex.Equal(err) {
260+
err = errors.Errorf("Invalid JSON value for CAST to type %s", ft.CompactStr())
261+
}
262+
return 0, false, err
263+
}
264+
sum += int64(crc32.ChecksumIEEE(fmt.Appendf(nil, "%v", item)))
265+
}
266+
267+
return sum, false, err
268+
}
269+
270+
// BuildJSONSumCrc32FunctionWithCheck builds a JSON_SUM_CRC32 ScalarFunction from the Expression and return error if any.
271+
// The logic is almost the same as build CAST function, except that the return type is fixed to bigint.
272+
func BuildJSONSumCrc32FunctionWithCheck(ctx BuildContext, expr Expression, tp *types.FieldType) (res Expression, err error) {
273+
argType := expr.GetType(ctx.GetEvalCtx())
274+
// If source argument's nullable, then target type should be nullable
275+
if !mysql.HasNotNullFlag(argType.GetFlag()) {
276+
tp.DelFlag(mysql.NotNullFlag)
277+
}
278+
expr = TryPushCastIntoControlFunctionForHybridType(ctx, expr, tp)
279+
280+
if tp.EvalType() != types.ETJson || !tp.IsArray() {
281+
return nil, errors.Errorf("json_sum_crc32 can only built on JSON array, got type %s", tp.EvalType())
282+
}
283+
284+
fc := &jsonSumCRC32FunctionClass{baseFunctionClass{ast.JSONSumCrc32, 1, 1}, tp}
285+
f, err := fc.getFunction(ctx, []Expression{expr})
286+
return &ScalarFunction{
287+
FuncName: ast.NewCIStr(ast.JSONSumCrc32),
288+
RetType: types.NewFieldType(mysql.TypeLonglong),
289+
Function: f,
290+
}, err
291+
}
292+
183293
type jsonExtractFunctionClass struct {
184294
baseFunctionClass
185295
}

pkg/expression/builtin_json_test.go

Lines changed: 108 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -16,10 +16,12 @@ package expression
1616

1717
import (
1818
"fmt"
19+
"hash/crc32"
1920
"testing"
2021

2122
"github.com/pingcap/failpoint"
2223
"github.com/pingcap/tidb/pkg/parser/ast"
24+
"github.com/pingcap/tidb/pkg/parser/charset"
2325
"github.com/pingcap/tidb/pkg/parser/mysql"
2426
"github.com/pingcap/tidb/pkg/parser/terror"
2527
"github.com/pingcap/tidb/pkg/testkit/testutil"
@@ -122,6 +124,112 @@ func TestJSONUnquote(t *testing.T) {
122124
}
123125
}
124126

127+
func TestJSONSumCrc32(t *testing.T) {
128+
checksumFunc := func(vals []any) int64 {
129+
var sum int64
130+
for _, v := range vals {
131+
sum += int64(crc32.ChecksumIEEE(fmt.Appendf(nil, "%v", v)))
132+
}
133+
return sum
134+
}
135+
136+
buildJSON := func(vals []any, path string) (Expression, Expression) {
137+
v := make(map[string]any)
138+
v[path] = vals
139+
return datumsToConstants(types.MakeDatums(types.CreateBinaryJSON(v)))[0],
140+
datumsToConstants(types.MakeDatums(fmt.Sprintf(`$.%s`, path)))[0]
141+
}
142+
143+
buildType := func(tp byte, unsigned bool) *types.FieldType {
144+
b := types.NewFieldTypeBuilder().SetType(tp).SetCharset(charset.CharsetBin).SetCollate(charset.CollationBin).SetArray(true)
145+
if unsigned {
146+
b.AddFlag(mysql.UnsignedFlag)
147+
}
148+
if types.IsString(tp) {
149+
b.SetFlen(16)
150+
}
151+
return b.BuildP()
152+
}
153+
154+
ctx := createContext(t)
155+
tbl := []struct {
156+
input []any
157+
path string
158+
evalError bool
159+
tp *types.FieldType
160+
}{
161+
{
162+
[]any{int64(-1), int64(2), int64(3)},
163+
"",
164+
false,
165+
buildType(mysql.TypeLonglong, false),
166+
},
167+
{
168+
[]any{int64(1), int64(2), int64(3)},
169+
"",
170+
false,
171+
buildType(mysql.TypeLonglong, true),
172+
},
173+
{
174+
[]any{int64(-1), int64(1)},
175+
"",
176+
true,
177+
buildType(mysql.TypeLonglong, true),
178+
},
179+
{
180+
[]any{"a", "b", "c"},
181+
"",
182+
false,
183+
buildType(mysql.TypeVarString, false),
184+
},
185+
{
186+
[]any{float64(1.1), int64(1), float64(3.3)},
187+
"",
188+
false,
189+
buildType(mysql.TypeDouble, false),
190+
},
191+
{
192+
[]any{float64(1.1), int64(1), float64(3.3)},
193+
"",
194+
true,
195+
buildType(mysql.TypeLonglong, false),
196+
},
197+
{
198+
[]any{1.1, 2.2, 3.3},
199+
"arr",
200+
false,
201+
buildType(mysql.TypeDouble, false),
202+
},
203+
{
204+
[]any{1.1, "1.1", 3.3},
205+
"arr",
206+
true,
207+
buildType(mysql.TypeDouble, false),
208+
},
209+
}
210+
211+
var err error
212+
for _, tt := range tbl {
213+
inputExpr := datumsToConstants(types.MakeDatums(types.CreateBinaryJSON(tt.input)))[0]
214+
if tt.path != "" {
215+
json, extract := buildJSON(tt.input, tt.path)
216+
inputExpr, err = NewFunction(ctx, ast.JSONExtract, types.NewFieldType(mysql.TypeJSON), json, extract)
217+
require.NoError(t, err)
218+
}
219+
220+
f, err := BuildJSONSumCrc32FunctionWithCheck(ctx, inputExpr, tt.tp)
221+
require.NoError(t, err, tt.input)
222+
223+
val, _, err := f.EvalInt(ctx, chunk.Row{})
224+
if tt.evalError {
225+
require.Error(t, err)
226+
} else {
227+
require.NoError(t, err, tt.input)
228+
require.EqualValues(t, checksumFunc(tt.input), val)
229+
}
230+
}
231+
}
232+
125233
func TestJSONExtract(t *testing.T) {
126234
ctx := createContext(t)
127235
fc := funcs[ast.JSONExtract]

pkg/expression/builtin_json_vec.go

Lines changed: 56 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -17,8 +17,11 @@ package expression
1717
import (
1818
"bytes"
1919
goJSON "encoding/json"
20+
"fmt"
21+
"hash/crc32"
2022
"strings"
2123

24+
"github.com/pingcap/errors"
2225
"github.com/pingcap/tidb/pkg/parser/ast"
2326
"github.com/pingcap/tidb/pkg/types"
2427
"github.com/pingcap/tidb/pkg/util/chunk"
@@ -914,6 +917,59 @@ func (b *builtinJSONTypeSig) vecEvalString(ctx EvalContext, input *chunk.Chunk,
914917
return nil
915918
}
916919

920+
func (b *builtinJSONSumCRC32Sig) vectorized() bool {
921+
return true
922+
}
923+
924+
func (b *builtinJSONSumCRC32Sig) vecEvalInt(ctx EvalContext, input *chunk.Chunk, result *chunk.Column) error {
925+
ft := b.tp.ArrayType()
926+
f := convertJSON2Tp(ft.EvalType())
927+
if f == nil {
928+
return ErrNotSupportedYet.GenWithStackByArgs(fmt.Sprintf("calculating sum of %s", ft.String()))
929+
}
930+
931+
nr := input.NumRows()
932+
jsonBuf, err := b.bufAllocator.get()
933+
if err != nil {
934+
return err
935+
}
936+
937+
defer b.bufAllocator.put(jsonBuf)
938+
if err = b.args[0].VecEvalJSON(ctx, input, jsonBuf); err != nil {
939+
return err
940+
}
941+
942+
result.ResizeInt64(nr, false)
943+
result.MergeNulls(jsonBuf)
944+
i64s := result.Int64s()
945+
946+
for i := range nr {
947+
if result.IsNull(i) {
948+
continue
949+
}
950+
951+
jsonItem := jsonBuf.GetJSON(i)
952+
if jsonItem.TypeCode != types.JSONTypeCodeArray {
953+
return ErrInvalidTypeForJSON.GenWithStackByArgs(1, "JSON_SUM_CRC32")
954+
}
955+
956+
var sum int64
957+
for j := range jsonItem.GetElemCount() {
958+
item, err := f(fakeSctx, jsonItem.ArrayGetElem(j), ft)
959+
if err != nil {
960+
if ErrInvalidJSONForFuncIndex.Equal(err) {
961+
err = errors.Errorf("Invalid JSON value for CAST to type %s", ft.CompactStr())
962+
}
963+
return err
964+
}
965+
sum += int64(crc32.ChecksumIEEE(fmt.Appendf(nil, "%v", item)))
966+
}
967+
i64s[i] = sum
968+
}
969+
970+
return nil
971+
}
972+
917973
func (b *builtinJSONExtractSig) vectorized() bool {
918974
return true
919975
}

pkg/expression/builtin_json_vec_test.go

Lines changed: 9 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -18,6 +18,8 @@ import (
1818
"testing"
1919

2020
"github.com/pingcap/tidb/pkg/parser/ast"
21+
"github.com/pingcap/tidb/pkg/parser/charset"
22+
"github.com/pingcap/tidb/pkg/parser/mysql"
2123
"github.com/pingcap/tidb/pkg/types"
2224
)
2325

@@ -138,6 +140,13 @@ var vecBuiltinJSONCases = map[string][]vecExprBenchCase{
138140
ast.JSONQuote: {
139141
{retEvalType: types.ETString, childrenTypes: []types.EvalType{types.ETString}},
140142
},
143+
ast.JSONSumCrc32: {
144+
{
145+
retEvalType: types.ETInt,
146+
childrenTypes: []types.EvalType{types.ETJson},
147+
childrenFieldTypes: []*types.FieldType{types.NewFieldTypeBuilder().SetType(mysql.TypeLonglong).SetCharset(charset.CharsetBin).SetCollate(charset.CollationBin).SetArray(true).BuildP()},
148+
geners: []dataGenerator{newJSONArrayGener()}},
149+
},
141150
}
142151

143152
func TestVectorizedBuiltinJSONFunc(t *testing.T) {

0 commit comments

Comments
 (0)