diff --git a/pkg/sql/logictest/testdata/logic_test/json b/pkg/sql/logictest/testdata/logic_test/json index f1229ffdc74d..4ce5321f4a57 100644 --- a/pkg/sql/logictest/testdata/logic_test/json +++ b/pkg/sql/logictest/testdata/logic_test/json @@ -884,3 +884,58 @@ query TBTB SELECT j, j ? 'a', j-1, (j-1) ? 'a' FROM t81647 ---- ["a", "b"] true ["a"] true + +# +# Test JSONB subscripting. +# + +# Constant folding. +query TTT +SELECT + ('{"a": {"b": {"c": 1}}}'::jsonb)['a'], + ('{"a": {"b": {"c": 1}}}'::jsonb)['a']['b']['c'], + ('[1, "2", null]'::jsonb)[1] +---- +{"b": {"c": 1}} 1 "2" + +# Referencing subscript which does not exist. +query TTTT +SELECT + ('{"a": 1}'::jsonb)['b'], + ('{"a": {"b": {"c": 1}}}'::jsonb)['c']['b']['c'], + ('[1, "2", null]'::jsonb)[4], + ('{"a": 1}'::jsonb)[NULL] +---- +NULL NULL NULL NULL + +# Error cases. +statement error unexpected JSON subscript type: TIMESTAMPTZ +SELECT ('{"a": 1}'::jsonb)[now()] + +# Check it works from a JSON table. +statement ok +CREATE TABLE json_subscript_test ( + id SERIAL PRIMARY KEY, + j JSONB, + extract_field TEXT, + extract_int_field INT +); +INSERT INTO json_subscript_test (j, extract_field, extract_int_field) VALUES + ('{"other_field": 2}', 'other_field', 1), + ('{"field": {"field": 2}}', 'field', 0), + ('[1, 2, 3]', 'nothing_to_fetch', 1) + +# Test subscripts with fields using other columns. +query TTITTTT +SELECT j, extract_field, extract_int_field, j['field'], j[extract_field], j[extract_field][extract_field], j[extract_int_field] +FROM json_subscript_test ORDER BY id +---- +{"other_field": 2} other_field 1 NULL 2 NULL NULL +{"field": {"field": 2}} field 0 {"field": 2} {"field": 2} 2 NULL +[1, 2, 3] nothing_to_fetch 1 NULL NULL NULL 2 + +# Test use in a WHERE clause. +query T +SELECT j FROM json_subscript_test WHERE j['other_field'] = '2' ORDER BY id +---- +{"other_field": 2} diff --git a/pkg/sql/opt/memo/typing.go b/pkg/sql/opt/memo/typing.go index 931e1665852a..180256582d06 100644 --- a/pkg/sql/opt/memo/typing.go +++ b/pkg/sql/opt/memo/typing.go @@ -249,7 +249,15 @@ func typeArrayAgg(e opt.ScalarExpr) *types.T { // typeIndirection returns the type of the element of the array. func typeIndirection(e opt.ScalarExpr) *types.T { - return e.Child(0).(opt.ScalarExpr).DataType().ArrayContents() + t := e.Child(0).(opt.ScalarExpr).DataType() + switch t.Family() { + case types.JsonFamily: + return t + case types.ArrayFamily: + return t.ArrayContents() + default: + panic(errors.AssertionFailedf("unknown type indirection type %s", t.SQLString())) + } } // typeCollate returns the collated string typed with the given locale. diff --git a/pkg/sql/opt/norm/fold_constants_funcs.go b/pkg/sql/opt/norm/fold_constants_funcs.go index b678460969c1..c9230b55a496 100644 --- a/pkg/sql/opt/norm/fold_constants_funcs.go +++ b/pkg/sql/opt/norm/fold_constants_funcs.go @@ -497,10 +497,14 @@ func (c *CustomFuncs) FoldIndirection(input, index opt.ScalarExpr) (_ opt.Scalar return nil, false } - // Case 2: The input is a constant DArray. + // Case 2: The input is a constant DArray or DJSON. if memo.CanExtractConstDatum(input) { inputD := memo.ExtractConstDatum(input) - texpr := tree.NewTypedIndirectionExpr(inputD, indexD, input.DataType().ArrayContents()) + resolvedType := input.DataType() + if resolvedType.Family() == types.ArrayFamily { + resolvedType = resolvedType.ArrayContents() + } + texpr := tree.NewTypedIndirectionExpr(inputD, indexD, resolvedType) result, err := eval.Expr(c.f.evalCtx, texpr) if err == nil { return c.f.ConstructConstVal(result, texpr.ResolvedType()), true diff --git a/pkg/sql/opt/optbuilder/scalar.go b/pkg/sql/opt/optbuilder/scalar.go index b576e41665b0..8fb2498c6e7b 100644 --- a/pkg/sql/opt/optbuilder/scalar.go +++ b/pkg/sql/opt/optbuilder/scalar.go @@ -170,22 +170,19 @@ func (b *Builder) buildScalar( out = b.factory.ConstructArrayFlatten(s.node, &subqueryPrivate) case *tree.IndirectionExpr: - expr := b.buildScalar(t.Expr.(tree.TypedExpr), inScope, nil, nil, colRefs) + out = b.buildScalar(t.Expr.(tree.TypedExpr), inScope, nil, nil, colRefs) - if len(t.Indirection) != 1 { - panic(unimplementedWithIssueDetailf(32552, "ind", "multidimensional indexing is not supported")) - } + for _, subscript := range t.Indirection { + if subscript.Slice { + panic(unimplementedWithIssueDetailf(32551, "", "array slicing is not supported")) + } - subscript := t.Indirection[0] - if subscript.Slice { - panic(unimplementedWithIssueDetailf(32551, "", "array slicing is not supported")) + out = b.factory.ConstructIndirection( + out, + b.buildScalar(subscript.Begin.(tree.TypedExpr), inScope, nil, nil, colRefs), + ) } - out = b.factory.ConstructIndirection( - expr, - b.buildScalar(subscript.Begin.(tree.TypedExpr), inScope, nil, nil, colRefs), - ) - case *tree.IfErrExpr: cond := b.buildScalar(t.Cond.(tree.TypedExpr), inScope, nil, nil, colRefs) diff --git a/pkg/sql/opt/optbuilder/testdata/select b/pkg/sql/opt/optbuilder/testdata/select index 057e43cf66da..45786e812e9a 100644 --- a/pkg/sql/opt/optbuilder/testdata/select +++ b/pkg/sql/opt/optbuilder/testdata/select @@ -312,7 +312,7 @@ error (42P01): no data source matches pattern: bar.kv.* build SELECT kv.*[1] FROM kv ---- -error (42804): cannot subscript type tuple{char AS k, char AS v} because it is not an array +error (42804): cannot subscript type tuple{char AS k, char AS v} because it is not an array or json object build SELECT ARRAY[] diff --git a/pkg/sql/sem/eval/expr.go b/pkg/sql/sem/eval/expr.go index 200b37ea95fc..c5e63dc5f269 100644 --- a/pkg/sql/sem/eval/expr.go +++ b/pkg/sql/sem/eval/expr.go @@ -15,6 +15,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree/treecmp" + "github.com/cockroachdb/cockroach/pkg/sql/types" "github.com/cockroachdb/errors" ) @@ -275,20 +276,6 @@ func (e *evaluator) EvalIndexedVar(iv *tree.IndexedVar) (tree.Datum, error) { func (e *evaluator) EvalIndirectionExpr(expr *tree.IndirectionExpr) (tree.Datum, error) { var subscriptIdx int - for i, t := range expr.Indirection { - if t.Slice || i > 0 { - return nil, errors.AssertionFailedf("unsupported feature should have been rejected during planning") - } - - d, err := t.Begin.(tree.TypedExpr).Eval(e) - if err != nil { - return nil, err - } - if d == tree.DNull { - return d, nil - } - subscriptIdx = int(tree.MustBeDInt(d)) - } d, err := expr.Expr.(tree.TypedExpr).Eval(e) if err != nil { @@ -298,17 +285,68 @@ func (e *evaluator) EvalIndirectionExpr(expr *tree.IndirectionExpr) (tree.Datum, return d, nil } - // Index into the DArray, using 1-indexing. - arr := tree.MustBeDArray(d) + switch d.ResolvedType().Family() { + case types.ArrayFamily: + for i, t := range expr.Indirection { + if t.Slice || i > 0 { + return nil, errors.AssertionFailedf("unsupported feature should have been rejected during planning") + } - // VECTOR types use 0-indexing. - if arr.FirstIndex() == 0 { - subscriptIdx++ - } - if subscriptIdx < 1 || subscriptIdx > arr.Len() { - return tree.DNull, nil + beginDatum, err := t.Begin.(tree.TypedExpr).Eval(e) + if err != nil { + return nil, err + } + if beginDatum == tree.DNull { + return tree.DNull, nil + } + subscriptIdx = int(tree.MustBeDInt(beginDatum)) + } + + // Index into the DArray, using 1-indexing. + arr := tree.MustBeDArray(d) + + // VECTOR types use 0-indexing. + if arr.FirstIndex() == 0 { + subscriptIdx++ + } + if subscriptIdx < 1 || subscriptIdx > arr.Len() { + return tree.DNull, nil + } + return arr.Array[subscriptIdx-1], nil + case types.JsonFamily: + j := tree.MustBeDJSON(d) + curr := j.JSON + for _, t := range expr.Indirection { + if t.Slice { + return nil, errors.AssertionFailedf("unsupported feature should have been rejected during planning") + } + + field, err := t.Begin.(tree.TypedExpr).Eval(e) + if err != nil { + return nil, err + } + if field == tree.DNull { + return tree.DNull, nil + } + switch field.ResolvedType().Family() { + case types.StringFamily: + if curr, err = curr.FetchValKeyOrIdx(string(tree.MustBeDString(field))); err != nil { + return nil, err + } + case types.IntFamily: + if curr, err = curr.FetchValIdx(int(tree.MustBeDInt(field))); err != nil { + return nil, err + } + default: + return nil, errors.AssertionFailedf("unsupported feature should have been rejected during planning") + } + if curr == nil { + return tree.DNull, nil + } + } + return tree.NewDJSON(curr), nil } - return arr.Array[subscriptIdx-1], nil + return nil, errors.AssertionFailedf("unsupported feature should have been rejected during planning") } func (e *evaluator) EvalDefaultVal(expr *tree.DefaultVal) (tree.Datum, error) { diff --git a/pkg/sql/sem/tree/type_check.go b/pkg/sql/sem/tree/type_check.go index 5e01859075c6..b0ddfca287bf 100644 --- a/pkg/sql/sem/tree/type_check.go +++ b/pkg/sql/sem/tree/type_check.go @@ -33,6 +33,7 @@ import ( // type checking of the relevant function is made. var ( OnTypeCheckArraySubscript func() + OnTypeCheckJSONBSubscript func() OnTypeCheckIfErr func() OnTypeCheckArrayConstructor func() OnTypeCheckArrayFlatten func() @@ -600,34 +601,64 @@ func (expr *CastExpr) TypeCheck( func (expr *IndirectionExpr) TypeCheck( ctx context.Context, semaCtx *SemaContext, desired *types.T, ) (TypedExpr, error) { - for i, t := range expr.Indirection { - if t.Slice { - return nil, unimplemented.NewWithIssuef(32551, "ARRAY slicing in %s", expr) - } - if i > 0 { - return nil, unimplemented.NewWithIssueDetailf(32552, "ind", "multidimensional indexing: %s", expr) - } - - beginExpr, err := typeCheckAndRequire(ctx, semaCtx, t.Begin, types.Int, "ARRAY subscript") - if err != nil { - return nil, err - } - t.Begin = beginExpr - } - subExpr, err := expr.Expr.TypeCheck(ctx, semaCtx, types.MakeArray(desired)) if err != nil { return nil, err } typ := subExpr.ResolvedType() - if typ.Family() != types.ArrayFamily { - return nil, pgerror.Newf(pgcode.DatatypeMismatch, "cannot subscript type %s because it is not an array", typ) - } expr.Expr = subExpr - expr.typ = typ.ArrayContents() - if OnTypeCheckArraySubscript != nil { - OnTypeCheckArraySubscript() + switch typ.Family() { + case types.ArrayFamily: + expr.typ = typ.ArrayContents() + for i, t := range expr.Indirection { + if t.Slice { + return nil, unimplemented.NewWithIssuef(32551, "ARRAY slicing in %s", expr) + } + if i > 0 { + return nil, unimplemented.NewWithIssueDetailf(32552, "ind", "multidimensional indexing: %s", expr) + } + + beginExpr, err := typeCheckAndRequire(ctx, semaCtx, t.Begin, types.Int, "ARRAY subscript") + if err != nil { + return nil, err + } + t.Begin = beginExpr + } + + if OnTypeCheckArraySubscript != nil { + OnTypeCheckArraySubscript() + } + case types.JsonFamily: + expr.typ = typ + for _, t := range expr.Indirection { + if t.Slice { + return nil, pgerror.Newf(pgcode.DatatypeMismatch, "cannot reference a slice with JSON") + } + beginExpr, err := t.Begin.TypeCheck(ctx, semaCtx, types.Any) + if err != nil { + return nil, err + } + switch beginExpr.ResolvedType().Family() { + case types.IntFamily, types.StringFamily, types.UnknownFamily: + default: + return nil, errors.WithHint( + pgerror.Newf( + pgcode.DatatypeMismatch, + "unexpected JSON subscript type: %s", + beginExpr.ResolvedType().SQLString(), + ), + "subscript type must be integer or text", + ) + } + t.Begin = beginExpr + } + + if OnTypeCheckJSONBSubscript != nil { + OnTypeCheckJSONBSubscript() + } + default: + return nil, pgerror.Newf(pgcode.DatatypeMismatch, "cannot subscript type %s because it is not an array or json object", typ) } return expr, nil } diff --git a/pkg/sql/sqltelemetry/scalar.go b/pkg/sql/sqltelemetry/scalar.go index b7030d706e7d..f7144cd5187c 100644 --- a/pkg/sql/sqltelemetry/scalar.go +++ b/pkg/sql/sqltelemetry/scalar.go @@ -72,6 +72,10 @@ var ArrayFlattenCounter = telemetry.GetCounterOnce("sql.plan.ops.array.flatten") // array subscript expression x[...]. var ArraySubscriptCounter = telemetry.GetCounterOnce("sql.plan.ops.array.ind") +// JSONBSubscriptCounter is to be incremented upon type checking an +// JSONB subscript expression x[...]. +var JSONBSubscriptCounter = telemetry.GetCounterOnce("sql.plan.ops.jsonb.subscript") + // IfErrCounter is to be incremented upon type checking an // IFERROR(...) expression or analogous. var IfErrCounter = telemetry.GetCounterOnce("sql.plan.ops.iferr") diff --git a/pkg/sql/telemetry.go b/pkg/sql/telemetry.go index 2f17533b07f9..706c932f26e5 100644 --- a/pkg/sql/telemetry.go +++ b/pkg/sql/telemetry.go @@ -74,6 +74,9 @@ func init() { tree.OnTypeCheckArraySubscript = func() { telemetry.Inc(sqltelemetry.ArraySubscriptCounter) } + tree.OnTypeCheckJSONBSubscript = func() { + telemetry.Inc(sqltelemetry.JSONBSubscriptCounter) + } tree.OnTypeCheckArrayFlatten = func() { telemetry.Inc(sqltelemetry.ArrayFlattenCounter) } diff --git a/pkg/sql/testdata/telemetry/json b/pkg/sql/testdata/telemetry/json new file mode 100644 index 000000000000..11924ccf01c6 --- /dev/null +++ b/pkg/sql/testdata/telemetry/json @@ -0,0 +1,8 @@ +feature-allowlist +sql.plan.ops.jsonb.subscript +---- + +feature-usage +SELECT ('{"a": {"b": {"c": 1}}}'::jsonb)['a']['b'] +---- +sql.plan.ops.jsonb.subscript