Skip to content

Commit

Permalink
sql: support array_agg with arrays as inputs
Browse files Browse the repository at this point in the history
This commit extends `array_agg` aggregate function to support arrays as
their input. The result of such aggregation is a nested array, and the
existing implementation simply works for that case.

The main complication of this change is that we currently don't support
multidimensional arrays in all cases, so previously we would disallow it
in a few places (like when creating a command result with a nested array
type). This commit had to remove that limitation since it was too broad.
There are still things that we don't support (e.g. in CREATE TABLE
context, or when the nested array needs to be serialized (we don't have
value encoding for it)), so in those cases we will keep on returning
errors.

Release note (sql change): `array_agg` aggregate function can now
support arrays as the input. Note, however, that CockroachDB still
doesn't have full support of nested arrays (that is tracked by #32552).
  • Loading branch information
yuzefovich committed Jan 17, 2024
1 parent e97d957 commit 51c267a
Show file tree
Hide file tree
Showing 14 changed files with 174 additions and 61 deletions.
48 changes: 48 additions & 0 deletions docs/generated/sql/aggregates.md
Original file line number Diff line number Diff line change
Expand Up @@ -3,52 +3,100 @@
<tbody>
<tr><td><a name="array_agg"></a><code>array_agg(arg1: <a href="bool.html">bool</a>) &rarr; <a href="bool.html">bool</a>[]</code></td><td><span class="funcdesc"><p>Aggregates the selected values into an array.</p>
</span></td><td>Immutable</td></tr>
<tr><td><a name="array_agg"></a><code>array_agg(arg1: <a href="bool.html">bool</a>[]) &rarr; <a href="bool.html">bool</a>[][]</code></td><td><span class="funcdesc"><p>Aggregates the selected values into an array.</p>
</span></td><td>Immutable</td></tr>
<tr><td><a name="array_agg"></a><code>array_agg(arg1: <a href="bytes.html">bytes</a>) &rarr; <a href="bytes.html">bytes</a>[]</code></td><td><span class="funcdesc"><p>Aggregates the selected values into an array.</p>
</span></td><td>Immutable</td></tr>
<tr><td><a name="array_agg"></a><code>array_agg(arg1: <a href="bytes.html">bytes</a>[]) &rarr; <a href="bytes.html">bytes</a>[][]</code></td><td><span class="funcdesc"><p>Aggregates the selected values into an array.</p>
</span></td><td>Immutable</td></tr>
<tr><td><a name="array_agg"></a><code>array_agg(arg1: <a href="date.html">date</a>) &rarr; <a href="date.html">date</a>[]</code></td><td><span class="funcdesc"><p>Aggregates the selected values into an array.</p>
</span></td><td>Immutable</td></tr>
<tr><td><a name="array_agg"></a><code>array_agg(arg1: <a href="date.html">date</a>[]) &rarr; <a href="date.html">date</a>[][]</code></td><td><span class="funcdesc"><p>Aggregates the selected values into an array.</p>
</span></td><td>Immutable</td></tr>
<tr><td><a name="array_agg"></a><code>array_agg(arg1: <a href="decimal.html">decimal</a>) &rarr; <a href="decimal.html">decimal</a>[]</code></td><td><span class="funcdesc"><p>Aggregates the selected values into an array.</p>
</span></td><td>Immutable</td></tr>
<tr><td><a name="array_agg"></a><code>array_agg(arg1: <a href="decimal.html">decimal</a>[]) &rarr; <a href="decimal.html">decimal</a>[][]</code></td><td><span class="funcdesc"><p>Aggregates the selected values into an array.</p>
</span></td><td>Immutable</td></tr>
<tr><td><a name="array_agg"></a><code>array_agg(arg1: <a href="float.html">float</a>) &rarr; <a href="float.html">float</a>[]</code></td><td><span class="funcdesc"><p>Aggregates the selected values into an array.</p>
</span></td><td>Immutable</td></tr>
<tr><td><a name="array_agg"></a><code>array_agg(arg1: <a href="float.html">float</a>[]) &rarr; <a href="float.html">float</a>[][]</code></td><td><span class="funcdesc"><p>Aggregates the selected values into an array.</p>
</span></td><td>Immutable</td></tr>
<tr><td><a name="array_agg"></a><code>array_agg(arg1: <a href="inet.html">inet</a>) &rarr; <a href="inet.html">inet</a>[]</code></td><td><span class="funcdesc"><p>Aggregates the selected values into an array.</p>
</span></td><td>Immutable</td></tr>
<tr><td><a name="array_agg"></a><code>array_agg(arg1: <a href="inet.html">inet</a>[]) &rarr; <a href="inet.html">inet</a>[][]</code></td><td><span class="funcdesc"><p>Aggregates the selected values into an array.</p>
</span></td><td>Immutable</td></tr>
<tr><td><a name="array_agg"></a><code>array_agg(arg1: <a href="int.html">int</a>) &rarr; <a href="int.html">int</a>[]</code></td><td><span class="funcdesc"><p>Aggregates the selected values into an array.</p>
</span></td><td>Immutable</td></tr>
<tr><td><a name="array_agg"></a><code>array_agg(arg1: <a href="int.html">int</a>[]) &rarr; <a href="int.html">int</a>[][]</code></td><td><span class="funcdesc"><p>Aggregates the selected values into an array.</p>
</span></td><td>Immutable</td></tr>
<tr><td><a name="array_agg"></a><code>array_agg(arg1: <a href="interval.html">interval</a>) &rarr; <a href="interval.html">interval</a>[]</code></td><td><span class="funcdesc"><p>Aggregates the selected values into an array.</p>
</span></td><td>Immutable</td></tr>
<tr><td><a name="array_agg"></a><code>array_agg(arg1: <a href="interval.html">interval</a>[]) &rarr; <a href="interval.html">interval</a>[][]</code></td><td><span class="funcdesc"><p>Aggregates the selected values into an array.</p>
</span></td><td>Immutable</td></tr>
<tr><td><a name="array_agg"></a><code>array_agg(arg1: <a href="string.html">string</a>) &rarr; <a href="string.html">string</a>[]</code></td><td><span class="funcdesc"><p>Aggregates the selected values into an array.</p>
</span></td><td>Immutable</td></tr>
<tr><td><a name="array_agg"></a><code>array_agg(arg1: <a href="string.html">string</a>[]) &rarr; <a href="string.html">string</a>[][]</code></td><td><span class="funcdesc"><p>Aggregates the selected values into an array.</p>
</span></td><td>Immutable</td></tr>
<tr><td><a name="array_agg"></a><code>array_agg(arg1: <a href="time.html">time</a>) &rarr; <a href="time.html">time</a>[]</code></td><td><span class="funcdesc"><p>Aggregates the selected values into an array.</p>
</span></td><td>Immutable</td></tr>
<tr><td><a name="array_agg"></a><code>array_agg(arg1: <a href="time.html">time</a>[]) &rarr; <a href="time.html">time</a>[][]</code></td><td><span class="funcdesc"><p>Aggregates the selected values into an array.</p>
</span></td><td>Immutable</td></tr>
<tr><td><a name="array_agg"></a><code>array_agg(arg1: <a href="timestamp.html">timestamp</a>) &rarr; <a href="timestamp.html">timestamp</a>[]</code></td><td><span class="funcdesc"><p>Aggregates the selected values into an array.</p>
</span></td><td>Immutable</td></tr>
<tr><td><a name="array_agg"></a><code>array_agg(arg1: <a href="timestamp.html">timestamp</a>[]) &rarr; <a href="timestamp.html">timestamp</a>[][]</code></td><td><span class="funcdesc"><p>Aggregates the selected values into an array.</p>
</span></td><td>Immutable</td></tr>
<tr><td><a name="array_agg"></a><code>array_agg(arg1: <a href="timestamp.html">timestamptz</a>) &rarr; <a href="timestamp.html">timestamptz</a>[]</code></td><td><span class="funcdesc"><p>Aggregates the selected values into an array.</p>
</span></td><td>Immutable</td></tr>
<tr><td><a name="array_agg"></a><code>array_agg(arg1: <a href="timestamp.html">timestamptz</a>[]) &rarr; <a href="timestamp.html">timestamptz</a>[][]</code></td><td><span class="funcdesc"><p>Aggregates the selected values into an array.</p>
</span></td><td>Immutable</td></tr>
<tr><td><a name="array_agg"></a><code>array_agg(arg1: <a href="uuid.html">uuid</a>) &rarr; <a href="uuid.html">uuid</a>[]</code></td><td><span class="funcdesc"><p>Aggregates the selected values into an array.</p>
</span></td><td>Immutable</td></tr>
<tr><td><a name="array_agg"></a><code>array_agg(arg1: <a href="uuid.html">uuid</a>[]) &rarr; <a href="uuid.html">uuid</a>[][]</code></td><td><span class="funcdesc"><p>Aggregates the selected values into an array.</p>
</span></td><td>Immutable</td></tr>
<tr><td><a name="array_agg"></a><code>array_agg(arg1: anyenum) &rarr; anyenum[]</code></td><td><span class="funcdesc"><p>Aggregates the selected values into an array.</p>
</span></td><td>Immutable</td></tr>
<tr><td><a name="array_agg"></a><code>array_agg(arg1: anyenum[]) &rarr; anyenum[][]</code></td><td><span class="funcdesc"><p>Aggregates the selected values into an array.</p>
</span></td><td>Immutable</td></tr>
<tr><td><a name="array_agg"></a><code>array_agg(arg1: box2d) &rarr; box2d[]</code></td><td><span class="funcdesc"><p>Aggregates the selected values into an array.</p>
</span></td><td>Immutable</td></tr>
<tr><td><a name="array_agg"></a><code>array_agg(arg1: box2d[]) &rarr; box2d[][]</code></td><td><span class="funcdesc"><p>Aggregates the selected values into an array.</p>
</span></td><td>Immutable</td></tr>
<tr><td><a name="array_agg"></a><code>array_agg(arg1: geography) &rarr; geography[]</code></td><td><span class="funcdesc"><p>Aggregates the selected values into an array.</p>
</span></td><td>Immutable</td></tr>
<tr><td><a name="array_agg"></a><code>array_agg(arg1: geography[]) &rarr; geography[][]</code></td><td><span class="funcdesc"><p>Aggregates the selected values into an array.</p>
</span></td><td>Immutable</td></tr>
<tr><td><a name="array_agg"></a><code>array_agg(arg1: geometry) &rarr; geometry[]</code></td><td><span class="funcdesc"><p>Aggregates the selected values into an array.</p>
</span></td><td>Immutable</td></tr>
<tr><td><a name="array_agg"></a><code>array_agg(arg1: geometry[]) &rarr; geometry[][]</code></td><td><span class="funcdesc"><p>Aggregates the selected values into an array.</p>
</span></td><td>Immutable</td></tr>
<tr><td><a name="array_agg"></a><code>array_agg(arg1: jsonb) &rarr; jsonb[]</code></td><td><span class="funcdesc"><p>Aggregates the selected values into an array.</p>
</span></td><td>Immutable</td></tr>
<tr><td><a name="array_agg"></a><code>array_agg(arg1: jsonb[]) &rarr; jsonb[][]</code></td><td><span class="funcdesc"><p>Aggregates the selected values into an array.</p>
</span></td><td>Immutable</td></tr>
<tr><td><a name="array_agg"></a><code>array_agg(arg1: oid) &rarr; oid[]</code></td><td><span class="funcdesc"><p>Aggregates the selected values into an array.</p>
</span></td><td>Immutable</td></tr>
<tr><td><a name="array_agg"></a><code>array_agg(arg1: oid[]) &rarr; oid[][]</code></td><td><span class="funcdesc"><p>Aggregates the selected values into an array.</p>
</span></td><td>Immutable</td></tr>
<tr><td><a name="array_agg"></a><code>array_agg(arg1: pg_lsn) &rarr; pg_lsn[]</code></td><td><span class="funcdesc"><p>Aggregates the selected values into an array.</p>
</span></td><td>Immutable</td></tr>
<tr><td><a name="array_agg"></a><code>array_agg(arg1: pg_lsn[]) &rarr; pg_lsn[][]</code></td><td><span class="funcdesc"><p>Aggregates the selected values into an array.</p>
</span></td><td>Immutable</td></tr>
<tr><td><a name="array_agg"></a><code>array_agg(arg1: refcursor) &rarr; refcursor[]</code></td><td><span class="funcdesc"><p>Aggregates the selected values into an array.</p>
</span></td><td>Immutable</td></tr>
<tr><td><a name="array_agg"></a><code>array_agg(arg1: refcursor[]) &rarr; refcursor[][]</code></td><td><span class="funcdesc"><p>Aggregates the selected values into an array.</p>
</span></td><td>Immutable</td></tr>
<tr><td><a name="array_agg"></a><code>array_agg(arg1: timetz) &rarr; timetz[]</code></td><td><span class="funcdesc"><p>Aggregates the selected values into an array.</p>
</span></td><td>Immutable</td></tr>
<tr><td><a name="array_agg"></a><code>array_agg(arg1: timetz[]) &rarr; timetz[][]</code></td><td><span class="funcdesc"><p>Aggregates the selected values into an array.</p>
</span></td><td>Immutable</td></tr>
<tr><td><a name="array_agg"></a><code>array_agg(arg1: tuple) &rarr; tuple[]</code></td><td><span class="funcdesc"><p>Aggregates the selected values into an array.</p>
</span></td><td>Immutable</td></tr>
<tr><td><a name="array_agg"></a><code>array_agg(arg1: tuple[]) &rarr; tuple[][]</code></td><td><span class="funcdesc"><p>Aggregates the selected values into an array.</p>
</span></td><td>Immutable</td></tr>
<tr><td><a name="array_agg"></a><code>array_agg(arg1: varbit) &rarr; varbit[]</code></td><td><span class="funcdesc"><p>Aggregates the selected values into an array.</p>
</span></td><td>Immutable</td></tr>
<tr><td><a name="array_agg"></a><code>array_agg(arg1: varbit[]) &rarr; varbit[][]</code></td><td><span class="funcdesc"><p>Aggregates the selected values into an array.</p>
</span></td><td>Immutable</td></tr>
<tr><td><a name="array_cat_agg"></a><code>array_cat_agg(arg1: <a href="bool.html">bool</a>[]) &rarr; <a href="bool.html">bool</a>[]</code></td><td><span class="funcdesc"><p>Unnests the selected arrays into elements that are then aggregated into a single array.</p>
</span></td><td>Immutable</td></tr>
<tr><td><a name="array_cat_agg"></a><code>array_cat_agg(arg1: <a href="bytes.html">bytes</a>[]) &rarr; <a href="bytes.html">bytes</a>[]</code></td><td><span class="funcdesc"><p>Unnests the selected arrays into elements that are then aggregated into a single array.</p>
Expand Down
14 changes: 4 additions & 10 deletions pkg/sql/exec_util.go
Original file line number Diff line number Diff line change
Expand Up @@ -109,7 +109,6 @@ import (
"github.com/cockroachdb/cockroach/pkg/upgrade/upgradebase"
"github.com/cockroachdb/cockroach/pkg/util/bitarray"
"github.com/cockroachdb/cockroach/pkg/util/duration"
"github.com/cockroachdb/cockroach/pkg/util/errorutil/unimplemented"
"github.com/cockroachdb/cockroach/pkg/util/hlc"
"github.com/cockroachdb/cockroach/pkg/util/log"
"github.com/cockroachdb/cockroach/pkg/util/metric"
Expand Down Expand Up @@ -2024,15 +2023,10 @@ func checkResultType(typ *types.T) error {
case types.EnumFamily:
case types.VoidFamily:
case types.ArrayFamily:
if typ.ArrayContents().Family() == types.ArrayFamily {
// Technically we could probably return arrays of arrays to a
// client (the encoding exists) but we don't want to give
// mixed signals -- that nested arrays appear to be supported
// in this case, and not in other cases (eg. CREATE). So we
// reject them in every case instead.
return unimplemented.NewWithIssueDetail(32552,
"result", "arrays cannot have arrays as element type")
}
// Note that we support multidimensional arrays in some cases (e.g.
// array_agg with arrays as inputs) but not in others (e.g. CREATE).
// Here we allow them in all cases and rely on each unsupported place to
// have the check.
case types.AnyFamily:
// Placeholder case.
return errors.Errorf("could not determine data type of %s", typ)
Expand Down
9 changes: 9 additions & 0 deletions pkg/sql/logictest/testdata/logic_test/aggregate
Original file line number Diff line number Diff line change
Expand Up @@ -3982,3 +3982,12 @@ SELECT array_cat_agg(
)
----
{(4.166442344843677e+17),(),(-0.12116245180368423)}

statement ok
CREATE TABLE __test_array_agg(a TEXT PRIMARY KEY, b TEXT, c TEXT);
INSERT INTO __test_array_agg VALUES ('a', 'b', 'c'), ('aa', 'bb', 'cc'), ('aaa', 'bbb', 'ccc');

query T
SELECT array_agg(array[a, b, c]) FROM __test_array_agg;
----
{"{a,b,c}","{aa,bb,cc}","{aaa,bbb,ccc}"}
26 changes: 16 additions & 10 deletions pkg/sql/logictest/testdata/logic_test/array
Original file line number Diff line number Diff line change
Expand Up @@ -118,20 +118,23 @@ SELECT ARRAY(VALUES (1),(2),(1))
----
{1,2,1}

statement error arrays cannot have arrays as element type
# This query works in local config but can fail in distributed config since the
# support for nested arrays is incomplete.
onlyif config local
query T
SELECT ARRAY(VALUES (ARRAY[1]))
----
{"{1}"}

query T
SELECT ARRAY(VALUES ('a'),('b'),('c'))
----
{a,b,c}


# TODO(justin): uncomment when #32715 is fixed.
# query T
# SELECT ARRAY(SELECT (1,2))
# ----
# {"(1,2)"}
query T
SELECT ARRAY(SELECT (1,2))
----
{"(1,2)"}

query error subquery must return only one column, found 2
SELECT ARRAY(SELECT 1, 2)
Expand Down Expand Up @@ -443,10 +446,13 @@ DROP TABLE boundedtable
statement error .*unimplemented.*\nHINT.*\n.*32552
CREATE TABLE badtable (b INT[][])

# Nested arrays should be disallowed

query error unimplemented: arrays cannot have arrays as element type.*\nHINT.*\n.*32552
# This query works in local config but can fail in distributed config since the
# support for nested arrays is incomplete.
onlyif config local
query T
SELECT ARRAY[ARRAY[1,2,3]]
----
{"{1,2,3}"}

# The postgres-compat aliases should be disallowed.
# INT2VECTOR is deprecated in Postgres.
Expand Down
2 changes: 0 additions & 2 deletions pkg/sql/opt/exec/execbuilder/testdata/subquery_correlated
Original file line number Diff line number Diff line change
Expand Up @@ -11,8 +11,6 @@ statement ok
CREATE TABLE c (c_id INT PRIMARY KEY, bill TEXT);
CREATE TABLE o (o_id INT PRIMARY KEY, c_id INT, ship TEXT);

# We can't decorrelate cases which don't use a scalar type in the
# ARRAY(...) operator.
query T
EXPLAIN SELECT
c_id,
Expand Down
13 changes: 0 additions & 13 deletions pkg/sql/opt/memo/typing.go
Original file line number Diff line number Diff line change
Expand Up @@ -109,19 +109,6 @@ func BinaryAllowsNullArgs(op opt.Operator, leftType, rightType *types.T) bool {
// functions without importing the builtins package.
var GetBuiltinProperties func(name string) (*tree.FunctionProperties, []tree.Overload)

// AggregateOverloadExists returns whether or not the given operator has a
// unary overload which takes the given type as input.
func AggregateOverloadExists(agg opt.Operator, typ *types.T) bool {
name := opt.AggregateOpReverseMap[agg]
_, overloads := GetBuiltinProperties(name)
for _, o := range overloads {
if o.Types.MatchAt(typ, 0) {
return true
}
}
return false
}

// FindFunction returns the function properties and overload of the function
// with the given name and argument types matching the children of the given
// input.
Expand Down
8 changes: 0 additions & 8 deletions pkg/sql/opt/optbuilder/scalar.go
Original file line number Diff line number Diff line change
Expand Up @@ -146,15 +146,7 @@ func (b *Builder) buildScalar(
s := t.Subquery.(*subquery)

inCol := s.cols[0].id

// This looks kind of arbitrary and strange, because it is:
// We cannot array_agg over some types, but we can only decorrelate via array_agg.
// Thus, we reject a query that is correlated and over a type that we can't array_agg.
typ := b.factory.Metadata().ColumnMeta(inCol).Type
if !s.outerCols.Empty() && !memo.AggregateOverloadExists(opt.ArrayAggOp, typ) {
panic(unimplementedWithIssueDetailf(35710, "", "can't execute a correlated ARRAY(...) over %s", typ))
}

if err := types.CheckArrayElementType(typ); err != nil {
panic(err)
}
Expand Down
24 changes: 24 additions & 0 deletions pkg/sql/opt/optbuilder/testdata/aggregate
Original file line number Diff line number Diff line change
Expand Up @@ -208,29 +208,53 @@ build
SELECT array_agg(NULL)
----
error (42725): ambiguous call: array_agg(unknown), candidates are:
array_agg(bool[]) -> bool[][]
array_agg(box2d) -> box2d[]
array_agg(box2d[]) -> box2d[][]
array_agg(int) -> int[]
array_agg(int[]) -> int[][]
array_agg(float) -> float[]
array_agg(float[]) -> float[][]
array_agg(decimal) -> decimal[]
array_agg(decimal[]) -> decimal[][]
array_agg(date) -> date[]
array_agg(date[]) -> date[][]
array_agg(timestamp) -> timestamp[]
array_agg(timestamp[]) -> timestamp[][]
array_agg(interval) -> interval[]
array_agg(interval[]) -> interval[][]
array_agg(geography) -> geography[]
array_agg(geography[]) -> geography[][]
array_agg(geometry) -> geometry[]
array_agg(geometry[]) -> geometry[][]
array_agg(string) -> string[]
array_agg(string[]) -> string[][]
array_agg(bytes) -> bytes[]
array_agg(bytes[]) -> bytes[][]
array_agg(timestamptz) -> timestamptz[]
array_agg(timestamptz[]) -> timestamptz[][]
array_agg(oid) -> oid[]
array_agg(oid[]) -> oid[][]
array_agg(uuid) -> uuid[]
array_agg(uuid[]) -> uuid[][]
array_agg(inet) -> inet[]
array_agg(inet[]) -> inet[][]
array_agg(pg_lsn) -> pg_lsn[]
array_agg(pg_lsn[]) -> pg_lsn[][]
array_agg(refcursor) -> refcursor[]
array_agg(refcursor[]) -> refcursor[][]
array_agg(time) -> time[]
array_agg(time[]) -> time[][]
array_agg(timetz) -> timetz[]
array_agg(timetz[]) -> timetz[][]
array_agg(jsonb) -> jsonb[]
array_agg(jsonb[]) -> jsonb[][]
array_agg(varbit) -> varbit[]
array_agg(varbit[]) -> varbit[][]
array_agg(anyenum) -> anyenum[]
array_agg(anyenum[]) -> anyenum[][]
array_agg(tuple) -> tuple[]
array_agg(tuple[]) -> tuple[][]
array_agg(bool) -> bool[]

# With an explicit cast, this works as expected.
Expand Down
16 changes: 15 additions & 1 deletion pkg/sql/opt/optbuilder/testdata/scalar
Original file line number Diff line number Diff line change
Expand Up @@ -1279,7 +1279,21 @@ project
build
SELECT ARRAY(SELECT y FROM u ORDER BY x) FROM v
----
error (0A000): unimplemented: can't execute a correlated ARRAY(...) over int[]
project
├── columns: array:10
├── scan v
│ └── columns: v.y:1 v.rowid:2!null v.crdb_internal_mvcc_timestamp:3 v.tableoid:4
└── projections
└── array-flatten col=9 [as=array:10]
└── sort
├── columns: y:9 [hidden: x:5]
├── ordering: +5
└── project
├── columns: y:9 x:5
├── scan u
│ └── columns: x:5 u.rowid:6!null u.crdb_internal_mvcc_timestamp:7 u.tableoid:8
└── projections
└── v.y:1 [as=y:9]

build-scalar
ISERROR(1/0)
Expand Down
1 change: 1 addition & 0 deletions pkg/sql/rowenc/valueside/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -22,6 +22,7 @@ go_library(
"//pkg/sql/sem/tree",
"//pkg/sql/types",
"//pkg/util/encoding",
"//pkg/util/errorutil/unimplemented",
"//pkg/util/ipaddr",
"//pkg/util/json",
"//pkg/util/timeutil/pgdate",
Expand Down
3 changes: 3 additions & 0 deletions pkg/sql/rowenc/valueside/array.go
Original file line number Diff line number Diff line change
Expand Up @@ -14,6 +14,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/sql/sem/tree"
"github.com/cockroachdb/cockroach/pkg/sql/types"
"github.com/cockroachdb/cockroach/pkg/util/encoding"
"github.com/cockroachdb/cockroach/pkg/util/errorutil/unimplemented"
"github.com/cockroachdb/cockroach/pkg/util/json"
"github.com/cockroachdb/cockroach/pkg/util/tsearch"
"github.com/cockroachdb/errors"
Expand Down Expand Up @@ -237,6 +238,8 @@ func DatumTypeToArrayElementEncodingType(t *types.T) (encoding.Type, error) {
return encoding.JSON, nil
case types.TupleFamily:
return encoding.Tuple, nil
case types.ArrayFamily:
return 0, unimplemented.NewWithIssueDetail(32552, "", "nested arrays are not fully supported")
default:
return 0, errors.AssertionFailedf("no known encoding type for %s", t.Family().Name())
}
Expand Down

0 comments on commit 51c267a

Please sign in to comment.