You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@druid.apache.org by cw...@apache.org on 2023/03/27 19:42:47 UTC

[druid] branch master updated: nested columns + arrays = array columns! (#13803)

This is an automated email from the ASF dual-hosted git repository.

cwylie pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/druid.git


The following commit(s) were added to refs/heads/master by this push:
     new d5b1b5bc8e nested columns + arrays = array columns! (#13803)
d5b1b5bc8e is described below

commit d5b1b5bc8e751b8a45b2a2bf5a02ee2018968a9f
Author: Clint Wylie <cw...@apache.org>
AuthorDate: Mon Mar 27 12:42:35 2023 -0700

    nested columns + arrays = array columns! (#13803)
    
    array columns!
    changes:
    * add support for storing nested arrays of string, long, and double values as specialized nested columns instead of breaking them into separate element columns
    * nested column type mimic behavior means that columns ingested with only root arrays of primitive values will be ARRAY typed columns
    * neat test refactor stuff
    * add v4 segment test
    * add array element indexes
    * add tests for unnest and array columns
    * fix unnest column value selector cursor handling of null and empty arrays
---
 .../data/input/AvroStreamInputFormatTest.java      |    1 -
 .../data/input/AvroStreamInputRowParserTest.java   |    1 -
 .../apache/druid/data/input/orc/OrcReaderTest.java |    6 -
 .../parquet/NestedColumnParquetReaderTest.java     |    4 +-
 .../input/protobuf/ProtobufInputFormatTest.java    |    6 +-
 .../data/input/impl/DelimitedInputFormat.java      |    1 +
 .../druid/data/input/impl/JsonInputFormat.java     |    1 -
 .../druid/data/input/impl/MapInputRowParser.java   |   26 +-
 .../org/apache/druid/math/expr/ConstantExpr.java   |    1 -
 .../java/org/apache/druid/math/expr/ExprEval.java  |   33 +-
 .../druid/math/expr/ExpressionProcessing.java      |   20 +-
 .../math/expr/ExpressionProcessingConfig.java      |   11 -
 .../org/apache/druid/math/expr/ExpressionType.java |    8 -
 .../java/org/apache/druid/math/expr/Function.java  |    1 -
 .../query/expression/NestedDataExpressions.java    |   19 +-
 ...ngleValueStringGroupByVectorColumnSelector.java |    3 +-
 .../druid/query/metadata/SegmentAnalyzer.java      |   18 +
 .../druid/segment/DimensionHandlerUtils.java       |   11 +
 .../druid/segment/NestedDataColumnIndexer.java     |  169 ++-
 .../druid/segment/NestedDataColumnMerger.java      |  348 ++++--
 .../segment/UnnestColumnValueSelectorCursor.java   |   33 +-
 .../apache/druid/segment/UnnestStorageAdapter.java |   69 +-
 .../druid/segment/data/FrontCodedIndexed.java      |   12 +-
 ...Indexed.java => FrontCodedIntArrayIndexed.java} |  413 ++-----
 .../data/FrontCodedIntArrayIndexedWriter.java      |  409 +++++++
 .../incremental/IncrementalIndexSchema.java        |    5 +
 ...lumnWriter.java => ArrayFieldColumnWriter.java} |   42 +-
 .../nested/CompressedNestedDataComplexColumn.java  |  478 +++++++-
 .../GlobalDictionaryEncodedFieldColumnWriter.java  |   38 +-
 .../segment/nested/GlobalDictionaryIdLookup.java   |   20 +
 .../nested/GlobalDictionarySortedCollector.java    |   48 +-
 .../segment/nested/GlobalDimensionDictionary.java  |  143 ++-
 .../segment/nested/LocalDimensionDictionary.java   |    4 +-
 .../segment/nested/NestedDataColumnSerializer.java |  125 +-
 .../segment/nested/NestedDataColumnSupplier.java   |   68 +-
 .../druid/segment/nested/NestedDataColumnV3.java   |    3 +-
 .../druid/segment/nested/NestedDataColumnV4.java   |    3 +-
 ...edDataColumnV4.java => NestedDataColumnV5.java} |    9 +-
 ...er.java => NestedFieldColumnIndexSupplier.java} |   23 +-
 ...ava => NestedFieldDictionaryEncodedColumn.java} |  128 +-
 ...teralTypeInfo.java => NestedFieldTypeInfo.java} |   95 +-
 .../segment/nested/StringFieldColumnWriter.java    |    2 +-
 .../segment/nested/StructuredDataProcessor.java    |   91 +-
 ...mnWriter.java => VariantFieldColumnWriter.java} |   33 +-
 .../segment/virtual/NestedFieldVirtualColumn.java  |   12 +-
 .../druid/data/input/ResourceInputSource.java      |  110 ++
 .../data/input/impl/InputRowParserSerdeTest.java   |    1 -
 .../druid/data/input/impl/JsonLineReaderTest.java  |    6 +-
 .../java/org/apache/druid/math/expr/EvalTest.java  |  194 ++-
 .../org/apache/druid/math/expr/OutputTypeTest.java |    8 +-
 .../org/apache/druid/math/expr/ParserTest.java     |   47 +-
 .../druid/math/expr/VectorExprSanityTest.java      |    2 +-
 .../druid/query/MultiValuedDimensionTest.java      |    2 +-
 .../apache/druid/query/NestedDataTestUtils.java    |  633 +++++-----
 .../query/aggregation/AggregationTestHelper.java   |   46 -
 .../expression/NestedDataExpressionsTest.java      |    2 +-
 .../expression/VectorExpressionsSanityTest.java    |    2 +-
 .../query/groupby/NestedDataGroupByQueryTest.java  |   64 +-
 .../druid/query/scan/NestedDataScanQueryTest.java  |  137 ++-
 .../druid/query/topn/NestedDataTopNQueryTest.java  |   32 +-
 .../org/apache/druid/segment/IndexBuilder.java     |  200 +++-
 .../druid/segment/NestedDataColumnIndexerTest.java |   29 +-
 .../UnnestColumnValueSelectorCursorTest.java       |  128 +-
 .../druid/segment/data/FrontCodedIndexedTest.java  |   46 +-
 .../data/FrontCodedIntArrayIndexedTest.java        |  467 ++++++++
 .../druid/segment/filter/ExpressionFilterTest.java |    2 +-
 .../nested/NestedDataColumnSupplierTest.java       |  432 ++++++-
 ...ava => NestedFieldColumnIndexSupplierTest.java} |  162 +--
 ...st.java => NestedFieldColumnSelectorsTest.java} |   31 +-
 .../segment/nested/NestedFieldTypeInfoTest.java    |  174 +++
 .../druid/segment/transform/TransformSpecTest.java |    4 +-
 .../druid/testing/InitializedNullHandlingTest.java |    2 +-
 .../src/test/resources/nested-array-test-data.json |   14 +
 .../test/resources/nested-numeric-test-data.json   |   10 +
 .../test/resources/nested-simple-test-data.json    |    8 +
 .../src/test/resources/nested-simple-test-data.tsv |    8 +
 .../src/test/resources/nested-types-test-data.json |    8 +
 .../src/test/resources/nested_segment_v4/index.zip |  Bin 0 -> 2062 bytes
 .../resources/numeric-nested-test-data-parser.json |   20 -
 .../test/resources/numeric-nested-test-data.json   |   10 -
 .../resources/simple-nested-test-data-aggs.json    |    6 -
 .../resources/simple-nested-test-data-parser.json  |   33 -
 .../simple-nested-test-data-tsv-parser.json        |   41 -
 .../simple-nested-test-data-tsv-transform.json     |   24 -
 .../test/resources/simple-nested-test-data.json    |    8 -
 .../src/test/resources/simple-nested-test-data.tsv |    8 -
 processing/src/test/resources/types-test-data.json |    8 -
 .../java/org/apache/druid/cli/DumpSegment.java     |    6 +-
 .../builtin/ArrayContainsOperatorConversion.java   |    2 +-
 .../builtin/NestedDataOperatorConversions.java     |  159 +++
 .../sql/calcite/planner/DruidOperatorTable.java    |    3 +
 .../druid/sql/calcite/BaseCalciteQueryTest.java    |    1 +
 .../druid/sql/calcite/CalciteArraysQueryTest.java  |  219 ++--
 .../calcite/CalciteMultiValueStringQueryTest.java  |    2 +-
 .../sql/calcite/CalciteNestedDataQueryTest.java    | 1244 +++++++++++++++++++-
 .../calcite/SqlVectorizedExpressionSanityTest.java |    2 +-
 .../druid/sql/calcite/util/CalciteTestBase.java    |    2 +-
 .../druid/sql/calcite/util/TestDataBuilder.java    |   75 +-
 ...d.jgz => wikiticker-2015-09-12-sampled.json.gz} |  Bin
 99 files changed, 6032 insertions(+), 1846 deletions(-)

diff --git a/extensions-core/avro-extensions/src/test/java/org/apache/druid/data/input/AvroStreamInputFormatTest.java b/extensions-core/avro-extensions/src/test/java/org/apache/druid/data/input/AvroStreamInputFormatTest.java
index 177766510d..222c5472cd 100644
--- a/extensions-core/avro-extensions/src/test/java/org/apache/druid/data/input/AvroStreamInputFormatTest.java
+++ b/extensions-core/avro-extensions/src/test/java/org/apache/druid/data/input/AvroStreamInputFormatTest.java
@@ -106,7 +106,6 @@ public class AvroStreamInputFormatTest extends InitializedNullHandlingTest
   private static final String TOPIC = "aTopic";
   static final List<String> DIMENSIONS = Arrays.asList(EVENT_TYPE, ID, SOME_OTHER_ID, IS_VALID);
   private static final List<String> DIMENSIONS_SCHEMALESS = Arrays.asList(
-      "nested",
       SOME_OTHER_ID,
       "someIntArray",
       "someFloat",
diff --git a/extensions-core/avro-extensions/src/test/java/org/apache/druid/data/input/AvroStreamInputRowParserTest.java b/extensions-core/avro-extensions/src/test/java/org/apache/druid/data/input/AvroStreamInputRowParserTest.java
index 3bcec4e3c1..b4a3514bce 100644
--- a/extensions-core/avro-extensions/src/test/java/org/apache/druid/data/input/AvroStreamInputRowParserTest.java
+++ b/extensions-core/avro-extensions/src/test/java/org/apache/druid/data/input/AvroStreamInputRowParserTest.java
@@ -86,7 +86,6 @@ public class AvroStreamInputRowParserTest
   private static final ZonedDateTime DATE_TIME = ZonedDateTime.of(2015, 10, 25, 19, 30, 0, 0, ZoneOffset.UTC);
   static final List<String> DIMENSIONS = Arrays.asList(EVENT_TYPE, ID, SOME_OTHER_ID, IS_VALID);
   private static final List<String> DIMENSIONS_SCHEMALESS = Arrays.asList(
-      "nested",
       SOME_OTHER_ID,
       "someIntArray",
       "someFloat",
diff --git a/extensions-core/orc-extensions/src/test/java/org/apache/druid/data/input/orc/OrcReaderTest.java b/extensions-core/orc-extensions/src/test/java/org/apache/druid/data/input/orc/OrcReaderTest.java
index 3013b4439e..6a395af8e9 100644
--- a/extensions-core/orc-extensions/src/test/java/org/apache/druid/data/input/orc/OrcReaderTest.java
+++ b/extensions-core/orc-extensions/src/test/java/org/apache/druid/data/input/orc/OrcReaderTest.java
@@ -36,7 +36,6 @@ import org.apache.druid.java.util.common.parsers.CloseableIterator;
 import org.apache.druid.java.util.common.parsers.JSONPathFieldSpec;
 import org.apache.druid.java.util.common.parsers.JSONPathFieldType;
 import org.apache.druid.java.util.common.parsers.JSONPathSpec;
-import org.apache.druid.math.expr.ExpressionProcessing;
 import org.apache.druid.query.expression.TestExprMacroTable;
 import org.apache.druid.segment.NestedDataDimensionSchema;
 import org.apache.druid.segment.transform.ExpressionTransform;
@@ -453,7 +452,6 @@ public class OrcReaderTest extends InitializedNullHandlingTest
         "middle",
         "list",
         "map",
-        "ts",
         "decimal1"
     );
     try (CloseableIterator<InputRow> iterator = reader.read()) {
@@ -593,7 +591,6 @@ public class OrcReaderTest extends InitializedNullHandlingTest
   @Test
   public void testNestedArray() throws IOException
   {
-    ExpressionProcessing.initializeForTests(true);
     final InputFormat inputFormat = new OrcInputFormat(
         new JSONPathSpec(
             true,
@@ -669,9 +666,6 @@ public class OrcReaderTest extends InitializedNullHandlingTest
       Assert.assertArrayEquals(new Object[]{1L, 2L}, (Object[]) row.getRaw("t_d_0"));
       Assert.assertFalse(iterator.hasNext());
     }
-    finally {
-      ExpressionProcessing.initializeForTests(null);
-    }
   }
 
   @Test
diff --git a/extensions-core/parquet-extensions/src/test/java/org/apache/druid/data/input/parquet/NestedColumnParquetReaderTest.java b/extensions-core/parquet-extensions/src/test/java/org/apache/druid/data/input/parquet/NestedColumnParquetReaderTest.java
index cd587f444f..96d2a8c578 100644
--- a/extensions-core/parquet-extensions/src/test/java/org/apache/druid/data/input/parquet/NestedColumnParquetReaderTest.java
+++ b/extensions-core/parquet-extensions/src/test/java/org/apache/druid/data/input/parquet/NestedColumnParquetReaderTest.java
@@ -181,7 +181,7 @@ public class NestedColumnParquetReaderTest extends BaseParquetReaderTest
     );
 
     List<InputRow> rows = readAllRows(reader);
-    Assert.assertEquals(ImmutableList.of("dim1", "metric1", "timestamp"), rows.get(0).getDimensions());
+    Assert.assertEquals(ImmutableList.of("dim1", "metric1"), rows.get(0).getDimensions());
     Assert.assertEquals(FlattenSpecParquetInputTest.TS1, rows.get(0).getTimestamp().toString());
     Assert.assertEquals(ImmutableList.of("d1v1"), rows.get(0).getDimension("dim1"));
     Assert.assertEquals("d1v1", rows.get(0).getRaw("dim1"));
@@ -218,7 +218,7 @@ public class NestedColumnParquetReaderTest extends BaseParquetReaderTest
     );
 
     List<InputRow> rows = readAllRows(reader);
-    Assert.assertEquals(ImmutableList.of("nestedData", "dim1", "metric1", "timestamp"), rows.get(0).getDimensions());
+    Assert.assertEquals(ImmutableList.of("nestedData", "dim1", "metric1"), rows.get(0).getDimensions());
     Assert.assertEquals(FlattenSpecParquetInputTest.TS1, rows.get(0).getTimestamp().toString());
     Assert.assertEquals(ImmutableList.of("d1v1"), rows.get(0).getDimension("dim1"));
     Assert.assertEquals("d1v1", rows.get(0).getRaw("dim1"));
diff --git a/extensions-core/protobuf-extensions/src/test/java/org/apache/druid/data/input/protobuf/ProtobufInputFormatTest.java b/extensions-core/protobuf-extensions/src/test/java/org/apache/druid/data/input/protobuf/ProtobufInputFormatTest.java
index 498174852b..86b5ae4edc 100644
--- a/extensions-core/protobuf-extensions/src/test/java/org/apache/druid/data/input/protobuf/ProtobufInputFormatTest.java
+++ b/extensions-core/protobuf-extensions/src/test/java/org/apache/druid/data/input/protobuf/ProtobufInputFormatTest.java
@@ -75,7 +75,7 @@ public class ProtobufInputFormatTest
   public void setUp() throws Exception
   {
     NullHandling.initializeForTests();
-    ExpressionProcessing.initializeForTests(null);
+    ExpressionProcessing.initializeForTests();
     timestampSpec = new TimestampSpec("timestamp", "iso", null);
     dimensionsSpec = new DimensionsSpec(Lists.newArrayList(
         new StringDimensionSchema("event"),
@@ -243,7 +243,6 @@ public class ProtobufInputFormatTest
                      .add("someFloatColumn")
                      .add("id")
                      .add("someBytesColumn")
-                     .add("timestamp")
                      .build(),
         row.getDimensions()
     );
@@ -380,8 +379,7 @@ public class ProtobufInputFormatTest
             "someFloatColumn",
             "eventType",
             "id",
-            "someBytesColumn",
-            "timestamp"
+            "someBytesColumn"
         ),
         row.getDimensions()
     );
diff --git a/processing/src/main/java/org/apache/druid/data/input/impl/DelimitedInputFormat.java b/processing/src/main/java/org/apache/druid/data/input/impl/DelimitedInputFormat.java
index f1a0a48522..d409f7ef12 100644
--- a/processing/src/main/java/org/apache/druid/data/input/impl/DelimitedInputFormat.java
+++ b/processing/src/main/java/org/apache/druid/data/input/impl/DelimitedInputFormat.java
@@ -41,6 +41,7 @@ import java.util.List;
 public class DelimitedInputFormat extends FlatTextInputFormat
 {
   public static final String TYPE_KEY = "tsv";
+
   private static final String DEFAULT_DELIMITER = "\t";
 
   @JsonCreator
diff --git a/processing/src/main/java/org/apache/druid/data/input/impl/JsonInputFormat.java b/processing/src/main/java/org/apache/druid/data/input/impl/JsonInputFormat.java
index 821c5bf435..200c621e13 100644
--- a/processing/src/main/java/org/apache/druid/data/input/impl/JsonInputFormat.java
+++ b/processing/src/main/java/org/apache/druid/data/input/impl/JsonInputFormat.java
@@ -40,7 +40,6 @@ import java.util.Objects;
 public class JsonInputFormat extends NestedInputFormat
 {
   public static final String TYPE_KEY = "json";
-
   private final Map<String, Boolean> featureSpec;
   private final ObjectMapper objectMapper;
   private final boolean keepNullColumns;
diff --git a/processing/src/main/java/org/apache/druid/data/input/impl/MapInputRowParser.java b/processing/src/main/java/org/apache/druid/data/input/impl/MapInputRowParser.java
index 58930804d0..47ef09e527 100644
--- a/processing/src/main/java/org/apache/druid/data/input/impl/MapInputRowParser.java
+++ b/processing/src/main/java/org/apache/druid/data/input/impl/MapInputRowParser.java
@@ -23,7 +23,6 @@ import com.fasterxml.jackson.annotation.JsonCreator;
 import com.fasterxml.jackson.annotation.JsonProperty;
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.collect.ImmutableList;
-import com.google.common.collect.Sets;
 import org.apache.druid.data.input.InputRow;
 import org.apache.druid.data.input.InputRowSchema;
 import org.apache.druid.data.input.MapBasedInputRow;
@@ -36,6 +35,7 @@ import java.util.ArrayList;
 import java.util.LinkedHashSet;
 import java.util.List;
 import java.util.Map;
+import java.util.Set;
 
 public class MapInputRowParser implements InputRowParser<Map<String, Object>>
 {
@@ -77,22 +77,38 @@ public class MapInputRowParser implements InputRowParser<Map<String, Object>>
    * 3) If isIncludeAllDimensions is not set and {@link DimensionsSpec#getDimensionNames()} is empty,
    *    the dimensions in the given map is returned.
    *
-   * In any case, the returned list does not include any dimensions in {@link DimensionsSpec#getDimensionExclusions()}.
+   * In any case, the returned list does not include any dimensions in {@link DimensionsSpec#getDimensionExclusions()}
+   * or {@link TimestampSpec#getTimestampColumn()}.
    */
   private static List<String> findDimensions(
+      TimestampSpec timestampSpec,
       DimensionsSpec dimensionsSpec,
       Map<String, Object> rawInputRow
   )
   {
+    final String timestampColumn = timestampSpec.getTimestampColumn();
+    final Set<String> exclusions = dimensionsSpec.getDimensionExclusions();
     if (dimensionsSpec.isIncludeAllDimensions()) {
       LinkedHashSet<String> dimensions = new LinkedHashSet<>(dimensionsSpec.getDimensionNames());
-      dimensions.addAll(Sets.difference(rawInputRow.keySet(), dimensionsSpec.getDimensionExclusions()));
+      for (String field : rawInputRow.keySet()) {
+        if (timestampColumn.equals(field) || exclusions.contains(field)) {
+          continue;
+        }
+        dimensions.add(field);
+      }
       return new ArrayList<>(dimensions);
     } else {
       if (!dimensionsSpec.getDimensionNames().isEmpty()) {
         return dimensionsSpec.getDimensionNames();
       } else {
-        return new ArrayList<>(Sets.difference(rawInputRow.keySet(), dimensionsSpec.getDimensionExclusions()));
+        List<String> dimensions = new ArrayList<>();
+        for (String field : rawInputRow.keySet()) {
+          if (timestampColumn.equals(field) || exclusions.contains(field)) {
+            continue;
+          }
+          dimensions.add(field);
+        }
+        return dimensions;
       }
     }
   }
@@ -104,7 +120,7 @@ public class MapInputRowParser implements InputRowParser<Map<String, Object>>
       Map<String, Object> theMap
   ) throws ParseException
   {
-    final List<String> dimensionsToUse = findDimensions(dimensionsSpec, theMap);
+    final List<String> dimensionsToUse = findDimensions(timestampSpec, dimensionsSpec, theMap);
 
     final DateTime timestamp;
     try {
diff --git a/processing/src/main/java/org/apache/druid/math/expr/ConstantExpr.java b/processing/src/main/java/org/apache/druid/math/expr/ConstantExpr.java
index ff4cedbecb..fdf6f080ee 100644
--- a/processing/src/main/java/org/apache/druid/math/expr/ConstantExpr.java
+++ b/processing/src/main/java/org/apache/druid/math/expr/ConstantExpr.java
@@ -379,7 +379,6 @@ class ArrayExpr extends ConstantExpr<Object[]>
   {
     super(outputType, value);
     Preconditions.checkArgument(outputType.isArray(), "Output type %s is not an array", outputType);
-    ExpressionType.checkNestedArrayAllowed(outputType);
   }
 
   @Override
diff --git a/processing/src/main/java/org/apache/druid/math/expr/ExprEval.java b/processing/src/main/java/org/apache/druid/math/expr/ExprEval.java
index 062ef9805d..cca67bc12a 100644
--- a/processing/src/main/java/org/apache/druid/math/expr/ExprEval.java
+++ b/processing/src/main/java/org/apache/druid/math/expr/ExprEval.java
@@ -370,11 +370,26 @@ public abstract class ExprEval<T>
     return new ComplexExprEval(outputType, value);
   }
 
+  public static ExprEval bestEffortArray(@Nullable List<?> theList)
+  {
+    // do not convert empty lists to arrays with a single null element here, because that should have been done
+    // by the selectors preparing their ObjectBindings if necessary. If we get to this point it was legitimately
+    // empty
+    NonnullPair<ExpressionType, Object[]> coerced = coerceListToArray(theList, false);
+    if (coerced == null) {
+      return bestEffortOf(null);
+    }
+    return ofArray(coerced.lhs, coerced.rhs);
+  }
+
   /**
    * Examine java type to find most appropriate expression type
    */
   public static ExprEval bestEffortOf(@Nullable Object val)
   {
+    if (val == null) {
+      return new StringExprEval(null);
+    }
     if (val instanceof ExprEval) {
       return (ExprEval) val;
     }
@@ -468,14 +483,7 @@ public abstract class ExprEval<T>
 
     if (val instanceof List || val instanceof Object[]) {
       final List<?> theList = val instanceof List ? ((List<?>) val) : Arrays.asList((Object[]) val);
-      // do not convert empty lists to arrays with a single null element here, because that should have been done
-      // by the selectors preparing their ObjectBindings if necessary. If we get to this point it was legitimately
-      // empty
-      NonnullPair<ExpressionType, Object[]> coerced = coerceListToArray(theList, false);
-      if (coerced == null) {
-        return bestEffortOf(null);
-      }
-      return ofArray(coerced.lhs, coerced.rhs);
+      return bestEffortArray(theList);
     }
 
     // in 'best effort' mode, we couldn't possibly use byte[] as a complex or anything else useful without type
@@ -485,12 +493,8 @@ public abstract class ExprEval<T>
       return new StringExprEval(StringUtils.encodeBase64String((byte[]) val));
     }
 
-    if (val != null) {
-      // is this cool?
-      return new ComplexExprEval(ExpressionType.UNKNOWN_COMPLEX, val);
-    }
-
-    return new StringExprEval(null);
+    // is this cool?
+    return new ComplexExprEval(ExpressionType.UNKNOWN_COMPLEX, val);
   }
 
   public static ExprEval ofType(@Nullable ExpressionType type, @Nullable Object value)
@@ -1109,7 +1113,6 @@ public abstract class ExprEval<T>
       super(value);
       this.arrayType = arrayType;
       Preconditions.checkArgument(arrayType.isArray(), "Output type %s is not an array", arrayType);
-      ExpressionType.checkNestedArrayAllowed(arrayType);
     }
 
     @Override
diff --git a/processing/src/main/java/org/apache/druid/math/expr/ExpressionProcessing.java b/processing/src/main/java/org/apache/druid/math/expr/ExpressionProcessing.java
index 905e0850f7..ae1d5fb297 100644
--- a/processing/src/main/java/org/apache/druid/math/expr/ExpressionProcessing.java
+++ b/processing/src/main/java/org/apache/druid/math/expr/ExpressionProcessing.java
@@ -22,8 +22,6 @@ package org.apache.druid.math.expr;
 import com.google.common.annotations.VisibleForTesting;
 import com.google.inject.Inject;
 
-import javax.annotation.Nullable;
-
 /**
  * Like {@link org.apache.druid.common.config.NullHandling}, except for expressions processing configs
  */
@@ -43,33 +41,23 @@ public class ExpressionProcessing
   /**
    * Many unit tests do not setup modules for this value to be injected, this method provides a manual way to initialize
    * {@link #INSTANCE}
-   * @param allowNestedArrays
    */
   @VisibleForTesting
-  public static void initializeForTests(@Nullable Boolean allowNestedArrays)
+  public static void initializeForTests()
   {
-    INSTANCE = new ExpressionProcessingConfig(allowNestedArrays, null, null, null);
+    INSTANCE = new ExpressionProcessingConfig(null, null, null);
   }
 
   @VisibleForTesting
   public static void initializeForStrictBooleansTests(boolean useStrict)
   {
-    INSTANCE = new ExpressionProcessingConfig(null, useStrict, null, null);
+    INSTANCE = new ExpressionProcessingConfig(useStrict, null, null);
   }
 
   @VisibleForTesting
   public static void initializeForHomogenizeNullMultiValueStrings()
   {
-    INSTANCE = new ExpressionProcessingConfig(null, null, null, true);
-  }
-
-  /**
-   * [['is expression support for'],['nested arrays'],['enabled?']]
-   */
-  public static boolean allowNestedArrays()
-  {
-    checkInitialized();
-    return INSTANCE.allowNestedArrays();
+    INSTANCE = new ExpressionProcessingConfig(null, null, true);
   }
 
   /**
diff --git a/processing/src/main/java/org/apache/druid/math/expr/ExpressionProcessingConfig.java b/processing/src/main/java/org/apache/druid/math/expr/ExpressionProcessingConfig.java
index b832578fe3..44e3b9409d 100644
--- a/processing/src/main/java/org/apache/druid/math/expr/ExpressionProcessingConfig.java
+++ b/processing/src/main/java/org/apache/druid/math/expr/ExpressionProcessingConfig.java
@@ -26,7 +26,6 @@ import javax.annotation.Nullable;
 
 public class ExpressionProcessingConfig
 {
-  public static final String NESTED_ARRAYS_CONFIG_STRING = "druid.expressions.allowNestedArrays";
   public static final String NULL_HANDLING_LEGACY_LOGICAL_OPS_STRING = "druid.expressions.useStrictBooleans";
   // Coerce arrays to multi value strings
   public static final String PROCESS_ARRAYS_AS_MULTIVALUE_STRINGS_CONFIG_STRING =
@@ -35,9 +34,6 @@ public class ExpressionProcessingConfig
   public static final String HOMOGENIZE_NULL_MULTIVALUE_STRING_ARRAYS =
       "druid.expressions.homogenizeNullMultiValueStringArrays";
 
-  @JsonProperty("allowNestedArrays")
-  private final boolean allowNestedArrays;
-
   @JsonProperty("useStrictBooleans")
   private final boolean useStrictBooleans;
 
@@ -49,13 +45,11 @@ public class ExpressionProcessingConfig
 
   @JsonCreator
   public ExpressionProcessingConfig(
-      @JsonProperty("allowNestedArrays") @Nullable Boolean allowNestedArrays,
       @JsonProperty("useStrictBooleans") @Nullable Boolean useStrictBooleans,
       @JsonProperty("processArraysAsMultiValueStrings") @Nullable Boolean processArraysAsMultiValueStrings,
       @JsonProperty("homogenizeNullMultiValueStringArrays") @Nullable Boolean homogenizeNullMultiValueStringArrays
   )
   {
-    this.allowNestedArrays = getWithPropertyFallbackFalse(allowNestedArrays, NESTED_ARRAYS_CONFIG_STRING);
     this.useStrictBooleans = getWithPropertyFallbackFalse(useStrictBooleans, NULL_HANDLING_LEGACY_LOGICAL_OPS_STRING);
     this.processArraysAsMultiValueStrings = getWithPropertyFallbackFalse(
         processArraysAsMultiValueStrings,
@@ -67,11 +61,6 @@ public class ExpressionProcessingConfig
     );
   }
 
-  public boolean allowNestedArrays()
-  {
-    return allowNestedArrays;
-  }
-
   public boolean isUseStrictBooleans()
   {
     return useStrictBooleans;
diff --git a/processing/src/main/java/org/apache/druid/math/expr/ExpressionType.java b/processing/src/main/java/org/apache/druid/math/expr/ExpressionType.java
index 6f6b7c3025..f2f17f2ef7 100644
--- a/processing/src/main/java/org/apache/druid/math/expr/ExpressionType.java
+++ b/processing/src/main/java/org/apache/druid/math/expr/ExpressionType.java
@@ -23,7 +23,6 @@ import com.fasterxml.jackson.annotation.JsonCreator;
 import com.fasterxml.jackson.annotation.JsonProperty;
 import com.fasterxml.jackson.databind.annotation.JsonSerialize;
 import com.fasterxml.jackson.databind.ser.std.ToStringSerializer;
-import org.apache.druid.java.util.common.IAE;
 import org.apache.druid.java.util.common.ISE;
 import org.apache.druid.segment.column.BaseTypeSignature;
 import org.apache.druid.segment.column.ColumnType;
@@ -210,11 +209,4 @@ public class ExpressionType extends BaseTypeSignature<ExprType>
         throw new ISE("Unsupported expression type[%s]", exprType);
     }
   }
-
-  public static void checkNestedArrayAllowed(ExpressionType outputType)
-  {
-    if (outputType.isArray() && outputType.getElementType().isArray() && !ExpressionProcessing.allowNestedArrays()) {
-      throw new IAE("Cannot create a nested array type [%s], 'druid.expressions.allowNestedArrays' must be set to true", outputType);
-    }
-  }
 }
diff --git a/processing/src/main/java/org/apache/druid/math/expr/Function.java b/processing/src/main/java/org/apache/druid/math/expr/Function.java
index 70cd0f8f27..03eb95820f 100644
--- a/processing/src/main/java/org/apache/druid/math/expr/Function.java
+++ b/processing/src/main/java/org/apache/druid/math/expr/Function.java
@@ -3054,7 +3054,6 @@ public interface Function extends NamedFunction
       if (arrayType == null) {
         arrayType = ExpressionTypeFactory.getInstance().ofArray(evaluated.type());
       }
-      ExpressionType.checkNestedArrayAllowed(arrayType);
       if (arrayType.getElementType().isNumeric() && evaluated.isNumericNull()) {
         out[i] = null;
       } else if (!evaluated.asArrayType().equals(arrayType)) {
diff --git a/processing/src/main/java/org/apache/druid/query/expression/NestedDataExpressions.java b/processing/src/main/java/org/apache/druid/query/expression/NestedDataExpressions.java
index 833f46428b..2eabe194dd 100644
--- a/processing/src/main/java/org/apache/druid/query/expression/NestedDataExpressions.java
+++ b/processing/src/main/java/org/apache/druid/query/expression/NestedDataExpressions.java
@@ -480,10 +480,25 @@ public class NestedDataExpressions
       final StructuredDataProcessor processor = new StructuredDataProcessor()
       {
         @Override
-        public StructuredDataProcessor.ProcessedLiteral<?> processLiteralField(ArrayList<NestedPathPart> fieldPath, Object fieldValue)
+        public ProcessedValue<?> processField(ArrayList<NestedPathPart> fieldPath, @Nullable Object fieldValue)
         {
           // do nothing, we only want the list of fields returned by this processor
-          return StructuredDataProcessor.ProcessedLiteral.NULL_LITERAL;
+          return ProcessedValue.NULL_LITERAL;
+        }
+
+        @Nullable
+        @Override
+        public ProcessedValue<?> processArrayField(
+            ArrayList<NestedPathPart> fieldPath,
+            @Nullable List<?> array
+        )
+        {
+          // we only want to return a non-null value here if the value is an array of primitive values
+          ExprEval<?> eval = ExprEval.bestEffortArray(array);
+          if (eval.type().isArray() && eval.type().getElementType().isPrimitive()) {
+            return ProcessedValue.NULL_LITERAL;
+          }
+          return null;
         }
       };
 
diff --git a/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/vector/DictionaryBuildingSingleValueStringGroupByVectorColumnSelector.java b/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/vector/DictionaryBuildingSingleValueStringGroupByVectorColumnSelector.java
index 937ddc339c..83f49e1c83 100644
--- a/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/vector/DictionaryBuildingSingleValueStringGroupByVectorColumnSelector.java
+++ b/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/vector/DictionaryBuildingSingleValueStringGroupByVectorColumnSelector.java
@@ -25,6 +25,7 @@ import org.apache.druid.common.config.NullHandling;
 import org.apache.druid.query.groupby.ResultRow;
 import org.apache.druid.query.groupby.epinephelinae.DictionaryBuilding;
 import org.apache.druid.query.groupby.epinephelinae.collection.MemoryPointer;
+import org.apache.druid.segment.DimensionHandlerUtils;
 import org.apache.druid.segment.vector.VectorObjectSelector;
 
 import java.util.ArrayList;
@@ -71,7 +72,7 @@ public class DictionaryBuildingSingleValueStringGroupByVectorColumnSelector impl
     int stateFootprintIncrease = 0;
 
     for (int i = startRow, j = keyOffset; i < endRow; i++, j += keySize) {
-      final String value = (String) vector[i];
+      final String value = DimensionHandlerUtils.convertObjectToString(vector[i]);
       final int dictId = reverseDictionary.getInt(value);
       if (dictId < 0) {
         final int nextId = dictionary.size();
diff --git a/processing/src/main/java/org/apache/druid/query/metadata/SegmentAnalyzer.java b/processing/src/main/java/org/apache/druid/query/metadata/SegmentAnalyzer.java
index 859396170b..7704a8e4c8 100644
--- a/processing/src/main/java/org/apache/druid/query/metadata/SegmentAnalyzer.java
+++ b/processing/src/main/java/org/apache/druid/query/metadata/SegmentAnalyzer.java
@@ -134,6 +134,9 @@ public class SegmentAnalyzer
             analysis = analyzeStringColumn(capabilities, storageAdapter, columnName);
           }
           break;
+        case ARRAY:
+          analysis = analyzeArrayColumn(capabilities);
+          break;
         case COMPLEX:
           final ColumnHolder columnHolder = index != null ? index.getColumnHolder(columnName) : null;
           analysis = analyzeComplexColumn(capabilities, numRows, columnHolder);
@@ -385,4 +388,19 @@ public class SegmentAnalyzer
       );
     }
   }
+
+  private ColumnAnalysis analyzeArrayColumn(final ColumnCapabilities capabilities)
+  {
+    return new ColumnAnalysis(
+        capabilities.toColumnType(),
+        capabilities.getType().name(),
+        false,
+        capabilities.hasNulls().isTrue(),
+        0L,
+        null,
+        null,
+        null,
+        null
+    );
+  }
 }
diff --git a/processing/src/main/java/org/apache/druid/segment/DimensionHandlerUtils.java b/processing/src/main/java/org/apache/druid/segment/DimensionHandlerUtils.java
index b509c7f058..8009afd654 100644
--- a/processing/src/main/java/org/apache/druid/segment/DimensionHandlerUtils.java
+++ b/processing/src/main/java/org/apache/druid/segment/DimensionHandlerUtils.java
@@ -624,4 +624,15 @@ public final class DimensionHandlerUtils
   {
     return number == null ? ZERO_FLOAT : number;
   }
+
+  public static boolean isNumericNull(@Nullable Object o)
+  {
+    if (o instanceof Number) {
+      return false;
+    }
+    if (o instanceof String && Doubles.tryParse((String) o) != null) {
+      return false;
+    }
+    return true;
+  }
 }
diff --git a/processing/src/main/java/org/apache/druid/segment/NestedDataColumnIndexer.java b/processing/src/main/java/org/apache/druid/segment/NestedDataColumnIndexer.java
index df64897eb7..9b0de5fcdc 100644
--- a/processing/src/main/java/org/apache/druid/segment/NestedDataColumnIndexer.java
+++ b/processing/src/main/java/org/apache/druid/segment/NestedDataColumnIndexer.java
@@ -21,6 +21,8 @@ package org.apache.druid.segment;
 
 import org.apache.druid.collections.bitmap.BitmapFactory;
 import org.apache.druid.collections.bitmap.MutableBitmap;
+import org.apache.druid.java.util.common.IAE;
+import org.apache.druid.java.util.common.UOE;
 import org.apache.druid.math.expr.ExprEval;
 import org.apache.druid.math.expr.ExpressionType;
 import org.apache.druid.query.dimension.DimensionSpec;
@@ -34,7 +36,7 @@ import org.apache.druid.segment.incremental.IncrementalIndexRowHolder;
 import org.apache.druid.segment.nested.GlobalDictionarySortedCollector;
 import org.apache.druid.segment.nested.GlobalDimensionDictionary;
 import org.apache.druid.segment.nested.NestedDataComplexTypeSerde;
-import org.apache.druid.segment.nested.NestedLiteralTypeInfo;
+import org.apache.druid.segment.nested.NestedFieldTypeInfo;
 import org.apache.druid.segment.nested.NestedPathFinder;
 import org.apache.druid.segment.nested.NestedPathPart;
 import org.apache.druid.segment.nested.StructuredData;
@@ -42,6 +44,8 @@ import org.apache.druid.segment.nested.StructuredDataProcessor;
 
 import javax.annotation.Nullable;
 import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
 import java.util.Map;
 import java.util.Objects;
 import java.util.SortedMap;
@@ -51,7 +55,7 @@ public class NestedDataColumnIndexer implements DimensionIndexer<StructuredData,
 {
   protected volatile boolean hasNulls = false;
 
-  protected SortedMap<String, LiteralFieldIndexer> fieldIndexers = new TreeMap<>();
+  protected SortedMap<String, FieldIndexer> fieldIndexers = new TreeMap<>();
   protected final GlobalDimensionDictionary globalDictionary = new GlobalDimensionDictionary();
 
   int estimatedFieldKeySize = 0;
@@ -59,16 +63,42 @@ public class NestedDataColumnIndexer implements DimensionIndexer<StructuredData,
   protected final StructuredDataProcessor indexerProcessor = new StructuredDataProcessor()
   {
     @Override
-    public ProcessedLiteral<?> processLiteralField(ArrayList<NestedPathPart> fieldPath, Object fieldValue)
+    public ProcessedValue<?> processField(ArrayList<NestedPathPart> fieldPath, @Nullable Object fieldValue)
     {
-      final String fieldName = NestedPathFinder.toNormalizedJsonPath(fieldPath);
-      LiteralFieldIndexer fieldIndexer = fieldIndexers.get(fieldName);
-      if (fieldIndexer == null) {
-        estimatedFieldKeySize += StructuredDataProcessor.estimateStringSize(fieldName);
-        fieldIndexer = new LiteralFieldIndexer(globalDictionary);
-        fieldIndexers.put(fieldName, fieldIndexer);
+      // null value is always added to the global dictionary as id 0, so we can ignore them here
+      if (fieldValue != null) {
+        final String fieldName = NestedPathFinder.toNormalizedJsonPath(fieldPath);
+        ExprEval<?> eval = ExprEval.bestEffortOf(fieldValue);
+        FieldIndexer fieldIndexer = fieldIndexers.get(fieldName);
+        if (fieldIndexer == null) {
+          estimatedFieldKeySize += StructuredDataProcessor.estimateStringSize(fieldName);
+          fieldIndexer = new FieldIndexer(globalDictionary);
+          fieldIndexers.put(fieldName, fieldIndexer);
+        }
+        return fieldIndexer.processValue(eval);
       }
-      return fieldIndexer.processValue(fieldValue);
+      return ProcessedValue.NULL_LITERAL;
+    }
+
+    @Nullable
+    @Override
+    public ProcessedValue<?> processArrayField(
+        ArrayList<NestedPathPart> fieldPath,
+        @Nullable List<?> array
+    )
+    {
+      final ExprEval<?> eval = ExprEval.bestEffortArray(array);
+      if (eval.type().isArray() && eval.type().getElementType().isPrimitive()) {
+        final String fieldName = NestedPathFinder.toNormalizedJsonPath(fieldPath);
+        FieldIndexer fieldIndexer = fieldIndexers.get(fieldName);
+        if (fieldIndexer == null) {
+          estimatedFieldKeySize += StructuredDataProcessor.estimateStringSize(fieldName);
+          fieldIndexer = new FieldIndexer(globalDictionary);
+          fieldIndexers.put(fieldName, fieldIndexer);
+        }
+        return fieldIndexer.processValue(eval);
+      }
+      return null;
     }
   };
   
@@ -145,6 +175,15 @@ public class NestedDataColumnIndexer implements DimensionIndexer<StructuredData,
     final int dimIndex = desc.getIndex();
     final ColumnValueSelector<?> rootLiteralSelector = getRootLiteralValueSelector(currEntry, dimIndex);
     if (rootLiteralSelector != null) {
+      final FieldIndexer root = fieldIndexers.get(NestedPathFinder.JSON_PATH_ROOT);
+      final ColumnType rootType = root.getTypes().getSingleType();
+      if (rootType.isArray()) {
+        throw new UOE(
+            "makeDimensionSelector is not supported, column [%s] is [%s] typed and should only use makeColumnValueSelector",
+            spec.getOutputName(),
+            rootType
+        );
+      }
       return new BaseSingleValueDimensionSelector()
       {
         @Nullable
@@ -165,7 +204,12 @@ public class NestedDataColumnIndexer implements DimensionIndexer<StructuredData,
         }
       };
     }
-    throw new UnsupportedOperationException("Not supported");
+    // column has nested data or is of mixed root type, cannot use
+    throw new UOE(
+        "makeDimensionSelector is not supported, column [%s] is [%s] typed and should only use makeColumnValueSelector",
+        spec.getOutputName(),
+        NestedDataComplexTypeSerde.TYPE
+    );
   }
 
   @Override
@@ -212,7 +256,7 @@ public class NestedDataColumnIndexer implements DimensionIndexer<StructuredData,
   public ColumnCapabilities getColumnCapabilities()
   {
     if (fieldIndexers.size() == 1 && fieldIndexers.containsKey(NestedPathFinder.JSON_PATH_ROOT)) {
-      LiteralFieldIndexer rootField = fieldIndexers.get(NestedPathFinder.JSON_PATH_ROOT);
+      FieldIndexer rootField = fieldIndexers.get(NestedPathFinder.JSON_PATH_ROOT);
       if (rootField.isSingleType()) {
         return ColumnCapabilitiesImpl.createDefault()
                                      .setType(rootField.getTypes().getSingleType())
@@ -265,7 +309,7 @@ public class NestedDataColumnIndexer implements DimensionIndexer<StructuredData,
   @Override
   public ColumnValueSelector convertUnsortedValuesToSorted(ColumnValueSelector selectorWithUnsortedValues)
   {
-    final LiteralFieldIndexer rootIndexer = fieldIndexers.get(NestedPathFinder.JSON_PATH_ROOT);
+    final FieldIndexer rootIndexer = fieldIndexers.get(NestedPathFinder.JSON_PATH_ROOT);
     if (fieldIndexers.size() == 1 && rootIndexer != null && rootIndexer.isSingleType()) {
       // for root only literals, makeColumnValueSelector and makeDimensionSelector automatically unwrap StructuredData
       // we need to do the opposite here, wrapping selector values with a StructuredData so that they are consistently
@@ -330,9 +374,9 @@ public class NestedDataColumnIndexer implements DimensionIndexer<StructuredData,
     throw new UnsupportedOperationException("Not supported");
   }
 
-  public void mergeFields(SortedMap<String, NestedLiteralTypeInfo.MutableTypeSet> mergedFields)
+  public void mergeFields(SortedMap<String, NestedFieldTypeInfo.MutableTypeSet> mergedFields)
   {
-    for (Map.Entry<String, NestedDataColumnIndexer.LiteralFieldIndexer> entry : fieldIndexers.entrySet()) {
+    for (Map.Entry<String, FieldIndexer> entry : fieldIndexers.entrySet()) {
       // skip adding the field if no types are in the set, meaning only null values have been processed
       if (!entry.getValue().getTypes().isEmpty()) {
         mergedFields.put(entry.getKey(), entry.getValue().getTypes());
@@ -354,7 +398,7 @@ public class NestedDataColumnIndexer implements DimensionIndexer<StructuredData,
     if (fieldIndexers.size() > 1) {
       return null;
     }
-    final LiteralFieldIndexer root = fieldIndexers.get(NestedPathFinder.JSON_PATH_ROOT);
+    final FieldIndexer root = fieldIndexers.get(NestedPathFinder.JSON_PATH_ROOT);
     if (root == null || !root.isSingleType()) {
       return null;
     }
@@ -411,7 +455,7 @@ public class NestedDataColumnIndexer implements DimensionIndexer<StructuredData,
         if (0 <= dimIndex && dimIndex < dims.length) {
           final StructuredData data = (StructuredData) dims[dimIndex];
           if (data != null) {
-            return data.getValue();
+            return ExprEval.bestEffortOf(data.getValue()).valueOrDefault();
           }
         }
 
@@ -426,55 +470,72 @@ public class NestedDataColumnIndexer implements DimensionIndexer<StructuredData,
     };
   }
 
-  static class LiteralFieldIndexer
+  static class FieldIndexer
   {
     private final GlobalDimensionDictionary globalDimensionDictionary;
-    private final NestedLiteralTypeInfo.MutableTypeSet typeSet;
+    private final NestedFieldTypeInfo.MutableTypeSet typeSet;
 
-    LiteralFieldIndexer(GlobalDimensionDictionary globalDimensionDictionary)
+    FieldIndexer(GlobalDimensionDictionary globalDimensionDictionary)
     {
       this.globalDimensionDictionary = globalDimensionDictionary;
-      this.typeSet = new NestedLiteralTypeInfo.MutableTypeSet();
+      this.typeSet = new NestedFieldTypeInfo.MutableTypeSet();
     }
 
-    private StructuredDataProcessor.ProcessedLiteral<?> processValue(@Nullable Object value)
+    private StructuredDataProcessor.ProcessedValue<?> processValue(ExprEval<?> eval)
     {
-      // null value is always added to the global dictionary as id 0, so we can ignore them here
-      if (value != null) {
-        // why not
-        ExprEval<?> eval = ExprEval.bestEffortOf(value);
-        final ColumnType columnType = ExpressionType.toColumnType(eval.type());
-
-        switch (columnType.getType()) {
-          case LONG:
-            globalDimensionDictionary.addLongValue(eval.asLong());
-            typeSet.add(ColumnType.LONG);
-            return new StructuredDataProcessor.ProcessedLiteral<>(
-                eval.asLong(),
-                StructuredDataProcessor.getLongObjectEstimateSize()
-            );
-          case DOUBLE:
-            globalDimensionDictionary.addDoubleValue(eval.asDouble());
-            typeSet.add(ColumnType.DOUBLE);
-            return new StructuredDataProcessor.ProcessedLiteral<>(
-                eval.asDouble(),
-                StructuredDataProcessor.getDoubleObjectEstimateSize()
+      final ColumnType columnType = ExpressionType.toColumnType(eval.type());
+      int sizeEstimate;
+      switch (columnType.getType()) {
+        case LONG:
+          typeSet.add(ColumnType.LONG);
+          sizeEstimate = globalDimensionDictionary.addLongValue(eval.asLong());
+          return new StructuredDataProcessor.ProcessedValue<>(eval.asLong(), sizeEstimate);
+        case DOUBLE:
+          typeSet.add(ColumnType.DOUBLE);
+          sizeEstimate = globalDimensionDictionary.addDoubleValue(eval.asDouble());
+          return new StructuredDataProcessor.ProcessedValue<>(eval.asDouble(), sizeEstimate);
+        case ARRAY:
+          // sanity check, this should never happen
+          if (columnType.getElementType() == null) {
+            throw new IAE(
+                "Array type [%s] missing element type, how did this possibly happen?",
+                eval.type()
             );
-          case STRING:
-          default:
-            final String asString = eval.asString();
-            globalDimensionDictionary.addStringValue(asString);
-            typeSet.add(ColumnType.STRING);
-            return new StructuredDataProcessor.ProcessedLiteral<>(
-                eval.asString(),
-                StructuredDataProcessor.estimateStringSize(asString)
-            );
-        }
+          }
+          switch (columnType.getElementType().getType()) {
+            case LONG:
+              typeSet.add(ColumnType.LONG_ARRAY);
+              final Object[] longArray = eval.asArray();
+              sizeEstimate = globalDimensionDictionary.addLongArray(longArray);
+              return new StructuredDataProcessor.ProcessedValue<>(longArray, sizeEstimate);
+            case DOUBLE:
+              typeSet.add(ColumnType.DOUBLE_ARRAY);
+              final Object[] doubleArray = eval.asArray();
+              sizeEstimate = globalDimensionDictionary.addDoubleArray(doubleArray);
+              return new StructuredDataProcessor.ProcessedValue<>(doubleArray, sizeEstimate);
+            case STRING:
+              final Object[] stringArray = eval.asArray();
+              // empty arrays and arrays with all nulls are detected as string arrays, but dont count them as part of
+              // the type set
+              if (stringArray.length > 0 && !Arrays.stream(stringArray).allMatch(Objects::isNull)) {
+                typeSet.add(ColumnType.STRING_ARRAY);
+              }
+              sizeEstimate = globalDimensionDictionary.addStringArray(stringArray);
+              return new StructuredDataProcessor.ProcessedValue<>(stringArray, sizeEstimate);
+            default:
+              throw new IAE("Unhandled type: %s", columnType);
+          }
+        case STRING:
+          typeSet.add(ColumnType.STRING);
+          final String asString = eval.asString();
+          sizeEstimate = globalDimensionDictionary.addStringValue(asString);
+          return new StructuredDataProcessor.ProcessedValue<>(asString, sizeEstimate);
+        default:
+          throw new IAE("Unhandled type: %s", columnType);
       }
-      return StructuredDataProcessor.ProcessedLiteral.NULL_LITERAL;
     }
 
-    public NestedLiteralTypeInfo.MutableTypeSet getTypes()
+    public NestedFieldTypeInfo.MutableTypeSet getTypes()
     {
       return typeSet;
     }
diff --git a/processing/src/main/java/org/apache/druid/segment/NestedDataColumnMerger.java b/processing/src/main/java/org/apache/druid/segment/NestedDataColumnMerger.java
index bf672c3522..95a52b83c2 100644
--- a/processing/src/main/java/org/apache/druid/segment/NestedDataColumnMerger.java
+++ b/processing/src/main/java/org/apache/druid/segment/NestedDataColumnMerger.java
@@ -19,6 +19,8 @@
 
 package org.apache.druid.segment;
 
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Iterators;
 import com.google.common.collect.PeekingIterator;
 import org.apache.druid.java.util.common.ISE;
 import org.apache.druid.java.util.common.io.Closer;
@@ -28,22 +30,28 @@ import org.apache.druid.segment.column.ColumnDescriptor;
 import org.apache.druid.segment.column.ColumnHolder;
 import org.apache.druid.segment.column.StringEncodingStrategies;
 import org.apache.druid.segment.column.ValueType;
+import org.apache.druid.segment.data.FrontCodedIntArrayIndexedWriter;
 import org.apache.druid.segment.data.Indexed;
 import org.apache.druid.segment.incremental.IncrementalIndex;
 import org.apache.druid.segment.incremental.IncrementalIndexAdapter;
 import org.apache.druid.segment.nested.CompressedNestedDataComplexColumn;
+import org.apache.druid.segment.nested.GlobalDictionaryIdLookup;
 import org.apache.druid.segment.nested.GlobalDictionarySortedCollector;
 import org.apache.druid.segment.nested.NestedDataColumnSerializer;
 import org.apache.druid.segment.nested.NestedDataComplexTypeSerde;
-import org.apache.druid.segment.nested.NestedLiteralTypeInfo;
+import org.apache.druid.segment.nested.NestedFieldTypeInfo;
 import org.apache.druid.segment.serde.ComplexColumnPartSerde;
 import org.apache.druid.segment.writeout.SegmentWriteOutMedium;
 
 import javax.annotation.Nullable;
 import java.io.IOException;
 import java.nio.IntBuffer;
+import java.util.Arrays;
 import java.util.Comparator;
+import java.util.Iterator;
 import java.util.List;
+import java.util.NoSuchElementException;
+import java.util.PriorityQueue;
 import java.util.SortedMap;
 import java.util.TreeMap;
 
@@ -86,99 +94,129 @@ public class NestedDataColumnMerger implements DimensionMergerV9
   @Override
   public void writeMergedValueDictionary(List<IndexableAdapter> adapters) throws IOException
   {
+    try {
+      long dimStartTime = System.currentTimeMillis();
+
+      int numMergeIndex = 0;
+      GlobalDictionarySortedCollector sortedLookup = null;
+      final Indexed[] sortedLookups = new Indexed[adapters.size()];
+      final Indexed[] sortedLongLookups = new Indexed[adapters.size()];
+      final Indexed[] sortedDoubleLookups = new Indexed[adapters.size()];
+      final Iterable<Object[]>[] sortedArrayLookups = new Iterable[adapters.size()];
+
+      final SortedMap<String, NestedFieldTypeInfo.MutableTypeSet> mergedFields = new TreeMap<>();
+
+      for (int i = 0; i < adapters.size(); i++) {
+        final IndexableAdapter adapter = adapters.get(i);
+        final GlobalDictionarySortedCollector dimValues;
+        if (adapter instanceof IncrementalIndexAdapter) {
+          dimValues = getSortedIndexFromIncrementalAdapter((IncrementalIndexAdapter) adapter, mergedFields);
+        } else if (adapter instanceof QueryableIndexIndexableAdapter) {
+          dimValues = getSortedIndexesFromQueryableAdapter((QueryableIndexIndexableAdapter) adapter, mergedFields);
+        } else {
+          throw new ISE("Unable to merge columns of unsupported adapter [%s]", adapter.getClass());
+        }
 
-    long dimStartTime = System.currentTimeMillis();
-
-    int numMergeIndex = 0;
-    GlobalDictionarySortedCollector sortedLookup = null;
-    final Indexed[] sortedLookups = new Indexed[adapters.size()];
-    final Indexed[] sortedLongLookups = new Indexed[adapters.size()];
-    final Indexed[] sortedDoubleLookups = new Indexed[adapters.size()];
+        boolean allNulls = dimValues == null || dimValues.allNull();
+        sortedLookup = dimValues;
+        if (!allNulls) {
+          sortedLookups[i] = dimValues.getSortedStrings();
+          sortedLongLookups[i] = dimValues.getSortedLongs();
+          sortedDoubleLookups[i] = dimValues.getSortedDoubles();
+          sortedArrayLookups[i] = dimValues.getSortedArrays();
+          numMergeIndex++;
+        }
+      }
 
-    final SortedMap<String, NestedLiteralTypeInfo.MutableTypeSet> mergedFields = new TreeMap<>();
+      descriptorBuilder = new ColumnDescriptor.Builder();
 
-    for (int i = 0; i < adapters.size(); i++) {
-      final IndexableAdapter adapter = adapters.get(i);
-      final GlobalDictionarySortedCollector dimValues;
-      if (adapter instanceof IncrementalIndexAdapter) {
-        dimValues = getSortedIndexFromIncrementalAdapter((IncrementalIndexAdapter) adapter, mergedFields);
-      } else if (adapter instanceof QueryableIndexIndexableAdapter) {
-        dimValues = getSortedIndexesFromQueryableAdapter((QueryableIndexIndexableAdapter) adapter, mergedFields);
+      final NestedDataColumnSerializer defaultSerializer = new NestedDataColumnSerializer(
+          name,
+          indexSpec,
+          segmentWriteOutMedium,
+          progressIndicator,
+          closer
+      );
+      serializer = defaultSerializer;
+
+      final ComplexColumnPartSerde partSerde = ComplexColumnPartSerde.serializerBuilder()
+                                                                     .withTypeName(NestedDataComplexTypeSerde.TYPE_NAME)
+                                                                     .withDelegate(serializer)
+                                                                     .build();
+      descriptorBuilder.setValueType(ValueType.COMPLEX)
+                       .setHasMultipleValues(false)
+                       .addSerde(partSerde);
+
+      defaultSerializer.open();
+      defaultSerializer.serializeFields(mergedFields);
+
+      int stringCardinality;
+      int longCardinality;
+      int doubleCardinality;
+      int arrayCardinality;
+      if (numMergeIndex == 1) {
+        defaultSerializer.serializeDictionaries(
+            sortedLookup.getSortedStrings(),
+            sortedLookup.getSortedLongs(),
+            sortedLookup.getSortedDoubles(),
+            () -> new ArrayDictionaryMergingIterator(
+                sortedArrayLookups,
+                defaultSerializer.getGlobalLookup()
+            )
+        );
+        stringCardinality = sortedLookup.getStringCardinality();
+        longCardinality = sortedLookup.getLongCardinality();
+        doubleCardinality = sortedLookup.getDoubleCardinality();
+        arrayCardinality = sortedLookup.getArrayCardinality();
       } else {
-        throw new ISE("Unable to merge columns of unsupported adapter %s", adapter.getClass());
+        final SimpleDictionaryMergingIterator<String> stringIterator = new SimpleDictionaryMergingIterator<>(
+            sortedLookups,
+            STRING_MERGING_COMPARATOR
+        );
+        final SimpleDictionaryMergingIterator<Long> longIterator = new SimpleDictionaryMergingIterator<>(
+            sortedLongLookups,
+            LONG_MERGING_COMPARATOR
+        );
+        final SimpleDictionaryMergingIterator<Double> doubleIterator = new SimpleDictionaryMergingIterator<>(
+            sortedDoubleLookups,
+            DOUBLE_MERGING_COMPARATOR
+        );
+        final ArrayDictionaryMergingIterator arrayIterator = new ArrayDictionaryMergingIterator(
+            sortedArrayLookups,
+            defaultSerializer.getGlobalLookup()
+        );
+        defaultSerializer.serializeDictionaries(
+            () -> stringIterator,
+            () -> longIterator,
+            () -> doubleIterator,
+            () -> arrayIterator
+        );
+        stringCardinality = stringIterator.getCardinality();
+        longCardinality = longIterator.getCardinality();
+        doubleCardinality = doubleIterator.getCardinality();
+        arrayCardinality = arrayIterator.getCardinality();
       }
 
-      boolean allNulls = allNull(dimValues.getSortedStrings()) &&
-                         allNull(dimValues.getSortedLongs()) &&
-                         allNull(dimValues.getSortedDoubles());
-      sortedLookup = dimValues;
-      if (!allNulls) {
-        sortedLookups[i] = dimValues.getSortedStrings();
-        sortedLongLookups[i] = dimValues.getSortedLongs();
-        sortedDoubleLookups[i] = dimValues.getSortedDoubles();
-        numMergeIndex++;
-      }
-    }
-
-    int cardinality = 0;
-    descriptorBuilder = new ColumnDescriptor.Builder();
-
-    final NestedDataColumnSerializer defaultSerializer = new NestedDataColumnSerializer(
-        name,
-        indexSpec,
-        segmentWriteOutMedium,
-        progressIndicator,
-        closer
-    );
-    serializer = defaultSerializer;
-
-    final ComplexColumnPartSerde partSerde = ComplexColumnPartSerde.serializerBuilder()
-                                                                   .withTypeName(NestedDataComplexTypeSerde.TYPE_NAME)
-                                                                   .withDelegate(serializer)
-                                                                   .build();
-    descriptorBuilder.setValueType(ValueType.COMPLEX)
-                     .setHasMultipleValues(false)
-                     .addSerde(partSerde);
-
-    defaultSerializer.open();
-    defaultSerializer.serializeFields(mergedFields);
-
-    if (numMergeIndex > 1) {
-      SimpleDictionaryMergingIterator<String> dictionaryMergeIterator = new SimpleDictionaryMergingIterator<>(
-          sortedLookups,
-          STRING_MERGING_COMPARATOR
+      log.debug(
+          "Completed dim[%s] conversions with string cardinality[%,d], long cardinality[%,d], double cardinality[%,d], array cardinality[%,d] in %,d millis.",
+          name,
+          stringCardinality,
+          longCardinality,
+          doubleCardinality,
+          arrayCardinality,
+          System.currentTimeMillis() - dimStartTime
       );
-      SimpleDictionaryMergingIterator<Long> longDictionaryMergeIterator = new SimpleDictionaryMergingIterator<>(
-          sortedLongLookups,
-          LONG_MERGING_COMPARATOR
-      );
-      SimpleDictionaryMergingIterator<Double> doubleDictionaryMergeIterator = new SimpleDictionaryMergingIterator<>(
-          sortedDoubleLookups,
-          DOUBLE_MERGING_COMPARATOR
-      );
-      defaultSerializer.serializeStringDictionary(() -> dictionaryMergeIterator);
-      defaultSerializer.serializeLongDictionary(() -> longDictionaryMergeIterator);
-      defaultSerializer.serializeDoubleDictionary(() -> doubleDictionaryMergeIterator);
-      cardinality = dictionaryMergeIterator.getCardinality();
-    } else if (numMergeIndex == 1) {
-      defaultSerializer.serializeStringDictionary(sortedLookup.getSortedStrings());
-      defaultSerializer.serializeLongDictionary(sortedLookup.getSortedLongs());
-      defaultSerializer.serializeDoubleDictionary(sortedLookup.getSortedDoubles());
-      cardinality = sortedLookup.size();
     }
-
-    log.debug(
-        "Completed dim[%s] conversions with cardinality[%,d] in %,d millis.",
-        name,
-        cardinality,
-        System.currentTimeMillis() - dimStartTime
-    );
+    catch (IOException ioe) {
+      log.error(ioe, "Failed to merge dictionary for column [%s]", name);
+      throw ioe;
+    }
   }
 
   @Nullable
   private GlobalDictionarySortedCollector getSortedIndexFromIncrementalAdapter(
       IncrementalIndexAdapter adapter,
-      SortedMap<String, NestedLiteralTypeInfo.MutableTypeSet> mergedFields
+      SortedMap<String, NestedFieldTypeInfo.MutableTypeSet> mergedFields
   )
   {
     final IncrementalIndex index = adapter.getIncrementalIndex();
@@ -194,7 +232,7 @@ public class NestedDataColumnMerger implements DimensionMergerV9
   @Nullable
   private GlobalDictionarySortedCollector getSortedIndexesFromQueryableAdapter(
       QueryableIndexIndexableAdapter adapter,
-      SortedMap<String, NestedLiteralTypeInfo.MutableTypeSet> mergedFields
+      SortedMap<String, NestedFieldTypeInfo.MutableTypeSet> mergedFields
   )
   {
     final ColumnHolder columnHolder = adapter.getQueryableIndex().getColumnHolder(name);
@@ -214,7 +252,7 @@ public class NestedDataColumnMerger implements DimensionMergerV9
   }
 
   private GlobalDictionarySortedCollector getSortedIndexFromV1QueryableAdapterNestedColumn(
-      SortedMap<String, NestedLiteralTypeInfo.MutableTypeSet> mergedFields,
+      SortedMap<String, NestedFieldTypeInfo.MutableTypeSet> mergedFields,
       BaseColumn col
   )
   {
@@ -223,10 +261,10 @@ public class NestedDataColumnMerger implements DimensionMergerV9
     closer.register(column);
     for (int i = 0; i < column.getFields().size(); i++) {
       String fieldPath = column.getFields().get(i);
-      NestedLiteralTypeInfo.TypeSet types = column.getFieldInfo().getTypes(i);
+      NestedFieldTypeInfo.TypeSet types = column.getFieldInfo().getTypes(i);
       mergedFields.compute(fieldPath, (k, v) -> {
         if (v == null) {
-          return new NestedLiteralTypeInfo.MutableTypeSet(types.getByteValue());
+          return new NestedFieldTypeInfo.MutableTypeSet(types.getByteValue());
         }
         return v.merge(types.getByteValue());
       });
@@ -234,7 +272,9 @@ public class NestedDataColumnMerger implements DimensionMergerV9
     return new GlobalDictionarySortedCollector(
         new StringEncodingStrategies.Utf8ToStringIndexed(column.getStringDictionary()),
         column.getLongDictionary(),
-        column.getDoubleDictionary()
+        column.getDoubleDictionary(),
+        column.getArraysIterable(),
+        column.getArrayDictionary().size()
     );
   }
 
@@ -271,13 +311,139 @@ public class NestedDataColumnMerger implements DimensionMergerV9
     return descriptorBuilder.build();
   }
 
-  private <T> boolean allNull(Indexed<T> dimValues)
+  public static class ArrayDictionaryMergingIterator implements Iterator<int[]>
   {
-    for (int i = 0, size = dimValues.size(); i < size; i++) {
-      if (dimValues.get(i) != null) {
-        return false;
+    private static final Comparator<PeekingIterator<int[]>> PEEKING_ITERATOR_COMPARATOR =
+        (lhs, rhs) -> FrontCodedIntArrayIndexedWriter.ARRAY_COMPARATOR.compare(lhs.peek(), rhs.peek());
+
+    protected final PriorityQueue<PeekingIterator<int[]>> pQueue;
+    private final Iterable<Object[]>[] dimValueLookups;
+    private final GlobalDictionaryIdLookup idLookup;
+
+    protected int counter;
+    private boolean initialized;
+
+    public ArrayDictionaryMergingIterator(Iterable<Object[]>[] dimValueLookups, GlobalDictionaryIdLookup idLookup)
+    {
+      this.pQueue = new PriorityQueue<>(PEEKING_ITERATOR_COMPARATOR);
+      this.dimValueLookups = dimValueLookups;
+      this.idLookup = idLookup;
+    }
+
+    private void initialize()
+    {
+      // we initialize lazily because the global id lookup might not be populated because the lower dictionary mergers
+      // have not been iterated yet, so wait until we iterate this one while serializing to populate it
+      for (Iterable<Object[]> dimValueLookup : dimValueLookups) {
+        if (dimValueLookup == null) {
+          continue;
+        }
+        final PeekingIterator<int[]> iter = Iterators.peekingIterator(
+            new IdLookupArrayIterator(idLookup, dimValueLookup.iterator())
+        );
+        if (iter.hasNext()) {
+          pQueue.add(iter);
+        }
+      }
+      initialized = true;
+    }
+
+    @Override
+    public boolean hasNext()
+    {
+      if (!initialized) {
+        initialize();
+      }
+      return !pQueue.isEmpty();
+    }
+
+    @Override
+    public int[] next()
+    {
+      if (!initialized) {
+        initialize();
+      }
+      PeekingIterator<int[]> smallest = pQueue.remove();
+      if (smallest == null) {
+        throw new NoSuchElementException();
+      }
+      final int[] value = smallest.next();
+      if (smallest.hasNext()) {
+        pQueue.add(smallest);
+      }
+
+      while (!pQueue.isEmpty() && Arrays.equals(value, pQueue.peek().peek())) {
+        PeekingIterator<int[]> same = pQueue.remove();
+        same.next();
+        if (same.hasNext()) {
+          pQueue.add(same);
+        }
+      }
+      counter++;
+
+      return value;
+    }
+
+    public int getCardinality()
+    {
+      return counter;
+    }
+
+    @Override
+    public void remove()
+    {
+      throw new UnsupportedOperationException("remove");
+    }
+  }
+
+  public static class IdLookupArrayIterator implements Iterator<int[]>
+  {
+    private final GlobalDictionaryIdLookup idLookup;
+    private final Iterator<Object[]> delegate;
+
+    public IdLookupArrayIterator(
+        GlobalDictionaryIdLookup idLookup,
+        Iterator<Object[]> delegate
+    )
+    {
+      this.idLookup = idLookup;
+      this.delegate = delegate;
+    }
+
+    @Override
+    public boolean hasNext()
+    {
+      return delegate.hasNext();
+    }
+
+    @Override
+    public int[] next()
+    {
+      final Object[] next = delegate.next();
+      if (next == null) {
+        return null;
+      }
+      final int[] globalIds = new int[next.length];
+      for (int i = 0; i < next.length; i++) {
+        if (next[i] == null) {
+          globalIds[i] = 0;
+        } else if (next[i] instanceof String) {
+          globalIds[i] = idLookup.lookupString((String) next[i]);
+        } else if (next[i] instanceof Long) {
+          globalIds[i] = idLookup.lookupLong((Long) next[i]);
+        } else if (next[i] instanceof Double) {
+          globalIds[i] = idLookup.lookupDouble((Double) next[i]);
+        } else {
+          globalIds[i] = -1;
+        }
+        Preconditions.checkArgument(
+            globalIds[i] >= 0,
+            "unknown global id [%s] for value [%s]",
+            globalIds[i],
+            next[i]
+        );
       }
+      return globalIds;
     }
-    return true;
   }
 }
diff --git a/processing/src/main/java/org/apache/druid/segment/UnnestColumnValueSelectorCursor.java b/processing/src/main/java/org/apache/druid/segment/UnnestColumnValueSelectorCursor.java
index 9179d55dc1..40a3aeeda2 100644
--- a/processing/src/main/java/org/apache/druid/segment/UnnestColumnValueSelectorCursor.java
+++ b/processing/src/main/java/org/apache/druid/segment/UnnestColumnValueSelectorCursor.java
@@ -187,10 +187,7 @@ public class UnnestColumnValueSelectorCursor implements Cursor
           @Override
           public Object getObject()
           {
-            if (!unnestListForCurrentRow.isEmpty()) {
-              return unnestListForCurrentRow.get(index);
-            }
-            return null;
+            return unnestListForCurrentRow.get(index);
           }
 
           @Override
@@ -279,7 +276,7 @@ public class UnnestColumnValueSelectorCursor implements Cursor
   {
     currentVal = this.columnValueSelector.getObject();
     if (currentVal == null) {
-      unnestListForCurrentRow = Collections.singletonList(null);
+      unnestListForCurrentRow = Collections.emptyList();
     } else if (currentVal instanceof List) {
       unnestListForCurrentRow = (List<Object>) currentVal;
     } else if (currentVal instanceof Object[]) {
@@ -296,9 +293,25 @@ public class UnnestColumnValueSelectorCursor implements Cursor
   private void initialize()
   {
     getNextRow();
+    if (unnestListForCurrentRow.isEmpty()) {
+      moveToNextNonEmptyRow();
+    }
     needInitialization = false;
   }
 
+  private void moveToNextNonEmptyRow()
+  {
+    index = 0;
+    do {
+      baseCursor.advance();
+      if (!baseCursor.isDone()) {
+        getNextRow();
+      } else {
+        return;
+      }
+    } while (unnestListForCurrentRow.isEmpty());
+  }
+
   /**
    * This advances the cursor to move to the next element to be unnested.
    * When the last element in a row is unnested, it is also responsible
@@ -307,14 +320,8 @@ public class UnnestColumnValueSelectorCursor implements Cursor
    */
   private void advanceAndUpdate()
   {
-    if (unnestListForCurrentRow.isEmpty() || index >= unnestListForCurrentRow.size() - 1) {
-      index = 0;
-      baseCursor.advance();
-      if (!baseCursor.isDone()) {
-        getNextRow();
-      }
-    } else {
-      index++;
+    if (++index >= unnestListForCurrentRow.size()) {
+      moveToNextNonEmptyRow();
     }
   }
 }
diff --git a/processing/src/main/java/org/apache/druid/segment/UnnestStorageAdapter.java b/processing/src/main/java/org/apache/druid/segment/UnnestStorageAdapter.java
index 8506a99d02..74ea48ad4e 100644
--- a/processing/src/main/java/org/apache/druid/segment/UnnestStorageAdapter.java
+++ b/processing/src/main/java/org/apache/druid/segment/UnnestStorageAdapter.java
@@ -75,7 +75,7 @@ public class UnnestStorageAdapter implements StorageAdapter
   public UnnestStorageAdapter(
       final StorageAdapter baseAdapter,
       final VirtualColumn unnestColumn,
-      final DimFilter unnestFilter
+      @Nullable final DimFilter unnestFilter
   )
   {
     this.baseAdapter = baseAdapter;
@@ -100,9 +100,7 @@ public class UnnestStorageAdapter implements StorageAdapter
         unnestFilter != null ? unnestFilter.toFilter() : null,
         virtualColumns,
         inputColumn,
-        inputColumn == null || virtualColumns.exists(inputColumn)
-        ? null
-        : baseAdapter.getColumnCapabilities(inputColumn)
+        inputColumn == null ? null : virtualColumns.getColumnCapabilitiesWithFallback(baseAdapter, inputColumn)
     );
 
     final Sequence<Cursor> baseCursorSequence = baseAdapter.makeCursors(
@@ -118,37 +116,29 @@ public class UnnestStorageAdapter implements StorageAdapter
         baseCursorSequence,
         cursor -> {
           Objects.requireNonNull(cursor);
-          Cursor retVal = cursor;
-          ColumnCapabilities capabilities = unnestColumn.capabilities(
+          final ColumnCapabilities capabilities = unnestColumn.capabilities(
               cursor.getColumnSelectorFactory(),
               unnestColumn.getOutputName()
           );
-          if (capabilities != null) {
-            if (capabilities.isDictionaryEncoded().and(capabilities.areDictionaryValuesUnique()).isTrue()) {
-              retVal = new UnnestDimensionCursor(
-                  retVal,
-                  retVal.getColumnSelectorFactory(),
-                  unnestColumn,
-                  outputColumnName
-              );
-            } else {
-              retVal = new UnnestColumnValueSelectorCursor(
-                  retVal,
-                  retVal.getColumnSelectorFactory(),
-                  unnestColumn,
-                  outputColumnName
-              );
-            }
+          final Cursor unnestCursor;
+
+          if (useDimensionCursor(capabilities)) {
+            unnestCursor = new UnnestDimensionCursor(
+                cursor,
+                cursor.getColumnSelectorFactory(),
+                unnestColumn,
+                outputColumnName
+            );
           } else {
-            retVal = new UnnestColumnValueSelectorCursor(
-                retVal,
-                retVal.getColumnSelectorFactory(),
+            unnestCursor = new UnnestColumnValueSelectorCursor(
+                cursor,
+                cursor.getColumnSelectorFactory(),
                 unnestColumn,
                 outputColumnName
             );
           }
           return PostJoinCursor.wrap(
-              retVal,
+              unnestCursor,
               virtualColumns,
               filterPair.rhs
           );
@@ -478,4 +468,31 @@ public class UnnestStorageAdapter implements StorageAdapter
              || filter instanceof BoundFilter;
     }
   }
+
+  /**
+   * Array and nested array columns are dictionary encoded, but not correctly for {@link UnnestDimensionCursor} which
+   * is tailored for scalar logical type values that are {@link ColumnCapabilities#isDictionaryEncoded()} and possibly
+   * with {@link ColumnCapabilities#hasMultipleValues()} (specifically {@link ValueType#STRING}), so we don't want to
+   * use this cursor if the capabilities are unknown or if the column type is {@link ValueType#ARRAY}.
+   */
+  private static boolean useDimensionCursor(@Nullable ColumnCapabilities capabilities)
+  {
+    if (capabilities == null) {
+      // capabilities being null here should be indicative of the column not existing or being a virtual column with
+      // no type information, chances are it is not going to be using a very cool dimension selector and so wont work
+      // with this, which requires real dictionary ids for the value matcher to work correctly
+      return false;
+    }
+    // the column needs real, unique value dictionary so that the value matcher id lookup works correctly, otherwise
+    // we must not use the dimension selector
+    if (capabilities.isDictionaryEncoded().and(capabilities.areDictionaryValuesUnique()).isTrue()) {
+      // if we got here, we only actually want to do this for dictionary encoded strings, since no other dictionary
+      // encoded column type should ever have multiple values set. nested and array columns are also dictionary encoded,
+      // but for arrays, the row is always a single dictionary id which maps to the entire array instead of an array
+      // of ids for each element, so we don't want to ever use the dimension selector cursor for that
+      return capabilities.is(ValueType.STRING);
+    }
+    // wasn't a dictionary encoded string, use the value selector
+    return false;
+  }
 }
diff --git a/processing/src/main/java/org/apache/druid/segment/data/FrontCodedIndexed.java b/processing/src/main/java/org/apache/druid/segment/data/FrontCodedIndexed.java
index cc0f280405..27a84b5df3 100644
--- a/processing/src/main/java/org/apache/druid/segment/data/FrontCodedIndexed.java
+++ b/processing/src/main/java/org/apache/druid/segment/data/FrontCodedIndexed.java
@@ -179,8 +179,9 @@ public final class FrontCodedIndexed implements Indexed<ByteBuffer>
     }
     Indexed.checkIndex(index, adjustedNumValues);
 
-    // due to vbyte encoding, the null value is not actually stored in the bucket (no negative values), so we adjust
-    // the index
+    // due to vbyte encoding, the null value is not actually stored in the bucket. we would typically represent it as a
+    // length of -1, since 0 is the empty string, but VByte encoding cannot have negative values, so if the null value
+    // is present, we adjust the index by 1 since it is always stored as position 0 due to sorting first
     final int adjustedIndex = index - adjustIndex;
     // find the bucket which contains the value with maths
     final int bucket = adjustedIndex >> div;
@@ -199,6 +200,10 @@ public final class FrontCodedIndexed implements Indexed<ByteBuffer>
       return hasNull ? 0 : -1;
     }
 
+    if (numBuckets == 0) {
+      return hasNull ? -2 : -1;
+    }
+
     int minBucketIndex = 0;
     int maxBucketIndex = numBuckets - 1;
     while (minBucketIndex < maxBucketIndex) {
@@ -298,6 +303,9 @@ public final class FrontCodedIndexed implements Indexed<ByteBuffer>
   @Override
   public Iterator<ByteBuffer> iterator()
   {
+    if (adjustedNumValues == 0) {
+      return Collections.emptyIterator();
+    }
     if (hasNull && adjustedNumValues == 1) {
       return Collections.<ByteBuffer>singletonList(null).iterator();
     }
diff --git a/processing/src/main/java/org/apache/druid/segment/data/FrontCodedIndexed.java b/processing/src/main/java/org/apache/druid/segment/data/FrontCodedIntArrayIndexed.java
similarity index 58%
copy from processing/src/main/java/org/apache/druid/segment/data/FrontCodedIndexed.java
copy to processing/src/main/java/org/apache/druid/segment/data/FrontCodedIntArrayIndexed.java
index cc0f280405..de7ce2d977 100644
--- a/processing/src/main/java/org/apache/druid/segment/data/FrontCodedIndexed.java
+++ b/processing/src/main/java/org/apache/druid/segment/data/FrontCodedIntArrayIndexed.java
@@ -21,7 +21,6 @@ package org.apache.druid.segment.data;
 
 import com.google.common.base.Preconditions;
 import com.google.common.base.Supplier;
-import org.apache.druid.annotations.SuppressFBWarnings;
 import org.apache.druid.common.config.NullHandling;
 import org.apache.druid.java.util.common.ISE;
 import org.apache.druid.java.util.common.StringUtils;
@@ -35,52 +34,50 @@ import java.util.Iterator;
 import java.util.NoSuchElementException;
 
 /**
- * {@link Indexed} specialized for storing variable-width binary values (such as utf8 encoded strings), which must be
- * sorted and unique, using 'front coding'. Front coding is a type of delta encoding for byte arrays, where sorted
- * values are grouped into buckets. The first value of the bucket is written entirely, and remaining values are stored
- * as a pair of an integer which indicates how much of the first byte array of the bucket to use as a prefix, followed
- * by the remaining bytes after the prefix to complete the value. If using 'incremental' buckets, instead of using the
- * prefix of the first bucket value, instead the prefix is computed against the immediately preceding value in the
- * bucket.
- * <p>
+ * {@link Indexed} specialized for storing int arrays, which must be sorted and unique, using 'front coding'.
+ *
+ * Front coding is a type of delta encoding, where sorted values are grouped into buckets. The first value of the bucket
+ * is written entirely, and remaining values are stored as a pair of an integer which indicates how much of the first
+ * int array of the bucket to use as a prefix, followed by the remaining ints after the prefix to complete the value.
+ *
  * front coded indexed layout:
  * | version | bucket size | has null? | number of values | size of "offsets" + "buckets" | "offsets" | "buckets" |
  * | ------- | ----------- | --------- | ---------------- | ----------------------------- | --------- | --------- |
  * |    byte |        byte |      byte |        vbyte int |                     vbyte int |     int[] |  bucket[] |
- * <p>
+ *
  * "offsets" are the ending offsets of each bucket stored in order, stored as plain integers for easy random access.
- * <p>
+ *
  * bucket layout:
  * | first value | prefix length | fragment | ... | prefix length | fragment |
  * | ----------- | ------------- | -------- | --- | ------------- | -------- |
- * |        blob |     vbyte int |     blob | ... |     vbyte int |     blob |
- * <p>
- * blob layout:
- * | blob length | blob bytes |
- * | ----------- | ---------- |
- * |   vbyte int |     byte[] |
- * <p>
- * <p>
+ * |       int[] |     vbyte int |    int[] | ... |     vbyte int |    int[] |
+ *
+ * int array layout:
+ * | length      |  ints |
+ * | ----------- | ----- |
+ * |   vbyte int | int[] |
+ *
+ *
  * Getting a value first picks the appropriate bucket, finds its offset in the underlying buffer, then scans the bucket
  * values to seek to the correct position of the value within the bucket in order to reconstruct it using the prefix
  * length.
- * <p>
+ *
  * Finding the index of a value involves binary searching the first values of each bucket to find the correct bucket,
  * then a linear scan within the bucket to find the matching value (or negative insertion point -1 for values that
  * are not present).
- * <p>
+ *
  * The value iterator reads an entire bucket at a time, reconstructing the values into an array to iterate within the
  * bucket before moving onto the next bucket as the iterator is consumed.
- * <p>
+ *
  * This class is not thread-safe since during operation modifies positions of a shared buffer.
  */
-public final class FrontCodedIndexed implements Indexed<ByteBuffer>
+public final class FrontCodedIntArrayIndexed implements Indexed<int[]>
 {
-  public static Supplier<FrontCodedIndexed> read(ByteBuffer buffer, ByteOrder ordering)
+  public static Supplier<FrontCodedIntArrayIndexed> read(ByteBuffer buffer, ByteOrder ordering)
   {
     final ByteBuffer orderedBuffer = buffer.asReadOnlyBuffer().order(ordering);
     final byte version = orderedBuffer.get();
-    Preconditions.checkArgument(version == 0 || version == 1, "only V0 and V1 exist, encountered " + version);
+    Preconditions.checkArgument(version == 0, "only V0 exists, encountered " + version);
     final int bucketSize = Byte.toUnsignedInt(orderedBuffer.get());
     final boolean hasNull = NullHandling.IS_NULL_BYTE == orderedBuffer.get();
     final int numValues = VByte.readInt(orderedBuffer);
@@ -90,14 +87,13 @@ public final class FrontCodedIndexed implements Indexed<ByteBuffer>
     // move position to end of buffer
     buffer.position(offsetsPosition + size);
 
-    return () -> new FrontCodedIndexed(
+    return () -> new FrontCodedIntArrayIndexed(
         buffer,
         ordering,
         bucketSize,
         numValues,
         hasNull,
-        offsetsPosition,
-        version
+        offsetsPosition
     );
   }
 
@@ -105,8 +101,6 @@ public final class FrontCodedIndexed implements Indexed<ByteBuffer>
   private final int adjustedNumValues;
   private final int adjustIndex;
   private final int bucketSize;
-  private final int[] unwindPrefixLength;
-  private final int[] unwindBufferPosition;
   private final int numBuckets;
   private final int div;
   private final int rem;
@@ -114,20 +108,16 @@ public final class FrontCodedIndexed implements Indexed<ByteBuffer>
   private final int bucketsPosition;
   private final boolean hasNull;
   private final int lastBucketNumValues;
+  private final int[] unwindPrefixLength;
+  private final int[] unwindBufferPosition;
 
-  private final GetBucketValue getBucketValueFn;
-  private final ReadBucket readBucketFn;
-  private final FindInBucket findInBucketFn;
-
-  @SuppressFBWarnings(value = "NP_STORE_INTO_NONNULL_FIELD", justification = "V0 does not use unwindPrefixLength or unwindBufferPosition")
-  private FrontCodedIndexed(
+  private FrontCodedIntArrayIndexed(
       ByteBuffer buffer,
       ByteOrder order,
       int bucketSize,
       int numValues,
       boolean hasNull,
-      int offsetsPosition,
-      byte version
+      int offsetsPosition
   )
   {
     if (Integer.bitCount(bucketSize) != 1) {
@@ -145,23 +135,8 @@ public final class FrontCodedIndexed implements Indexed<ByteBuffer>
     this.lastBucketNumValues = (numValues & rem) == 0 ? bucketSize : numValues & rem;
     this.offsetsPosition = offsetsPosition;
     this.bucketsPosition = offsetsPosition + ((numBuckets - 1) * Integer.BYTES);
-    if (version == 0) {
-      // version zero, all prefixes are computed against the first value in the bucket
-      this.getBucketValueFn = FrontCodedIndexed::getFromBucketV0;
-      this.readBucketFn = FrontCodedIndexed::readBucketV0;
-      this.findInBucketFn = this::findValueInBucketV0;
-      //noinspection DataFlowIssue
-      this.unwindPrefixLength = null;
-      //noinspection DataFlowIssue
-      this.unwindBufferPosition = null;
-    } else {
-      // version one uses 'incremental' buckets, where the prefix is computed against the previous value
-      this.unwindPrefixLength = new int[bucketSize];
-      this.unwindBufferPosition = new int[bucketSize];
-      this.getBucketValueFn = this::getFromBucketV1;
-      this.readBucketFn = this::readBucketV1;
-      this.findInBucketFn = this::findValueInBucketV1;
-    }
+    this.unwindPrefixLength = new int[bucketSize];
+    this.unwindBufferPosition = new int[bucketSize];
   }
 
   @Override
@@ -172,26 +147,27 @@ public final class FrontCodedIndexed implements Indexed<ByteBuffer>
 
   @Nullable
   @Override
-  public ByteBuffer get(int index)
+  public int[] get(int index)
   {
     if (hasNull && index == 0) {
       return null;
     }
     Indexed.checkIndex(index, adjustedNumValues);
 
-    // due to vbyte encoding, the null value is not actually stored in the bucket (no negative values), so we adjust
-    // the index
+    // due to vbyte encoding, the null value is not actually stored in the bucket. we would typically represent it as a
+    // length of -1, since 0 is the empty array, but VByte encoding cannot have negative values, so if the null value
+    // is present, we adjust the index by 1 since it is always stored as position 0 due to sorting first
     final int adjustedIndex = index - adjustIndex;
     // find the bucket which contains the value with maths
     final int bucket = adjustedIndex >> div;
     final int bucketIndex = adjustedIndex & rem;
     final int offset = getBucketOffset(bucket);
     buffer.position(offset);
-    return getBucketValueFn.get(buffer, bucketIndex);
+    return getFromBucket(buffer, bucketIndex);
   }
 
   @Override
-  public int indexOf(@Nullable ByteBuffer value)
+  public int indexOf(@Nullable int[] value)
   {
     // performs binary search using the first values of each bucket to locate the appropriate bucket, and then does
     // a linear scan to find the value within the bucket
@@ -199,6 +175,10 @@ public final class FrontCodedIndexed implements Indexed<ByteBuffer>
       return hasNull ? 0 : -1;
     }
 
+    if (numBuckets == 0) {
+      return hasNull ? -2 : -1;
+    }
+
     int minBucketIndex = 0;
     int maxBucketIndex = numBuckets - 1;
     while (minBucketIndex < maxBucketIndex) {
@@ -214,13 +194,13 @@ public final class FrontCodedIndexed implements Indexed<ByteBuffer>
       // save the length of the shared prefix with the first value of the bucket and the value to match so we
       // can use it later to skip over all values in the bucket that share a longer prefix with the first value
       // (the bucket is sorted, so the prefix length gets smaller as values increase)
-      final int sharedPrefix = buffer.position() - firstOffset;
+      final int sharedPrefix = (buffer.position() - firstOffset) / Integer.BYTES;
       if (comparison == 0) {
-        if (firstLength == value.remaining()) {
+        if (firstLength == value.length) {
           // it turns out that the first value in current bucket is what we are looking for, short circuit
           return currBucketFirstValueIndex + adjustIndex;
         } else {
-          comparison = Integer.compare(firstLength, value.remaining());
+          comparison = Integer.compare(firstLength, value.length);
         }
       }
 
@@ -231,21 +211,21 @@ public final class FrontCodedIndexed implements Indexed<ByteBuffer>
       final int nextLength = VByte.readInt(buffer);
       int comparisonNext = compareBucketFirstValue(buffer, nextLength, value);
       if (comparisonNext == 0) {
-        if (nextLength == value.remaining()) {
+        if (nextLength == value.length) {
           // it turns out that the first value in next bucket is what we are looking for, go ahead and short circuit
           // for that as well, even though we weren't going to scan that bucket on this iteration...
           return (currBucketFirstValueIndex + adjustIndex) + bucketSize;
         } else {
-          comparisonNext = Integer.compare(nextLength, value.remaining());
+          comparisonNext = Integer.compare(nextLength, value.length);
         }
       }
 
       if (comparison < 0 && comparisonNext > 0) {
         // this is exactly the right bucket
         // find the value in the bucket (or where it would be if it were present)
-        buffer.position(firstOffset + firstLength);
+        buffer.position(firstOffset + (firstLength * Integer.BYTES));
 
-        return findInBucketFn.find(value, currBucketFirstValueIndex, bucketSize, sharedPrefix);
+        return findValueInBucket(value, currBucketFirstValueIndex, bucketSize, sharedPrefix);
       } else if (comparison < 0) {
         minBucketIndex = currentBucket + 1;
       } else {
@@ -268,24 +248,24 @@ public final class FrontCodedIndexed implements Indexed<ByteBuffer>
     final int firstLength = VByte.readInt(buffer);
     final int firstOffset = buffer.position();
     int comparison = compareBucketFirstValue(buffer, firstLength, value);
-    final int sharedPrefix = buffer.position() - firstOffset;
+    final int sharedPrefix = (buffer.position() - firstOffset) / Integer.BYTES;
     if (comparison == 0) {
-      if (firstLength == value.remaining()) {
+      if (firstLength == value.length) {
         // it turns out that the first value in current bucket is what we are looking for, short circuit
         return bucketIndexBase + adjustIndex;
       } else {
-        comparison = Integer.compare(firstLength, value.remaining());
+        comparison = Integer.compare(firstLength, value.length);
       }
     }
 
     if (comparison > 0) {
       // value preceedes bucket, so bail out
-      return ~(bucketIndexBase + adjustIndex);
+      return -(bucketIndexBase + adjustIndex) - 1;
     }
 
-    buffer.position(firstOffset + firstLength);
+    buffer.position(firstOffset + (firstLength * Integer.BYTES));
 
-    return findInBucketFn.find(value, bucketIndexBase, numValuesInBucket, sharedPrefix);
+    return findValueInBucket(value, bucketIndexBase, numValuesInBucket, sharedPrefix);
   }
 
   @Override
@@ -296,20 +276,24 @@ public final class FrontCodedIndexed implements Indexed<ByteBuffer>
   }
 
   @Override
-  public Iterator<ByteBuffer> iterator()
+  public Iterator<int[]> iterator()
   {
+    if (adjustedNumValues == 0) {
+      return Collections.emptyIterator();
+    }
     if (hasNull && adjustedNumValues == 1) {
-      return Collections.<ByteBuffer>singletonList(null).iterator();
+      return Collections.<int[]>singletonList(null).iterator();
     }
+
     ByteBuffer copy = buffer.asReadOnlyBuffer().order(buffer.order());
     copy.position(bucketsPosition);
-    final ByteBuffer[] firstBucket = readBucketFn.readBucket(copy, numBuckets > 1 ? bucketSize : lastBucketNumValues);
+    final int[][] firstBucket = readBucket(copy, numBuckets > 1 ? bucketSize : lastBucketNumValues);
     // iterator decodes and buffers a bucket at a time, paging through buckets as the iterator is consumed
-    return new Iterator<ByteBuffer>()
+    return new Iterator<int[]>()
     {
       private int currIndex = 0;
       private int currentBucketIndex = 0;
-      private ByteBuffer[] currentBucket = firstBucket;
+      private int[][] currentBucket = firstBucket;
 
       @Override
       public boolean hasNext()
@@ -318,7 +302,7 @@ public final class FrontCodedIndexed implements Indexed<ByteBuffer>
       }
 
       @Override
-      public ByteBuffer next()
+      public int[] next()
       {
         // null is handled special
         if (hasNull && currIndex == 0) {
@@ -334,7 +318,7 @@ public final class FrontCodedIndexed implements Indexed<ByteBuffer>
         if (bucketNum != currentBucketIndex) {
           final int offset = copy.getInt(offsetsPosition + ((bucketNum - 1) * Integer.BYTES));
           copy.position(bucketsPosition + offset);
-          currentBucket = readBucketFn.readBucket(
+          currentBucket = readBucket(
               copy,
               bucketNum < (numBuckets - 1) ? bucketSize : lastBucketNumValues
           );
@@ -374,23 +358,23 @@ public final class FrontCodedIndexed implements Indexed<ByteBuffer>
    * MUST be prepared before calling, as it expects the length of the first value to have already been read externally,
    * and the buffer position to be at the start of the first bucket value. The final buffer position will be the
    * 'shared prefix length' of the first value in the bucket and the value to compare.
-   * <p>
+   *
    * Bytes are compared using {@link StringUtils#compareUtf8UsingJavaStringOrdering(byte, byte)}. Therefore, when the
    * values are UTF-8 encoded strings, the ordering is compatible with {@link String#compareTo(String)}.
    */
-  private static int compareBucketFirstValue(ByteBuffer bucketBuffer, int length, ByteBuffer value)
+  private static int compareBucketFirstValue(ByteBuffer bucketBuffer, int length, int[] value)
   {
     final int startOffset = bucketBuffer.position();
-    final int commonLength = Math.min(length, value.remaining());
+    final int commonLength = Math.min(length, value.length);
     // save the length of the shared prefix with the first value of the bucket and the value to match so we
     // can use it later to skip over all values in the bucket that share a longer prefix with the first value
     // (the bucket is sorted, so the prefix length gets smaller as values increase)
     int sharedPrefix;
     int comparison = 0;
     for (sharedPrefix = 0; sharedPrefix < commonLength; sharedPrefix++) {
-      comparison = StringUtils.compareUtf8UsingJavaStringOrdering(bucketBuffer.get(), value.get(sharedPrefix));
+      comparison = Integer.compare(bucketBuffer.getInt(), value[sharedPrefix]);
       if (comparison != 0) {
-        bucketBuffer.position(startOffset + sharedPrefix);
+        bucketBuffer.position(startOffset + (sharedPrefix * Integer.BYTES));
         break;
       }
     }
@@ -400,162 +384,19 @@ public final class FrontCodedIndexed implements Indexed<ByteBuffer>
   /**
    * Finds a value in a bucket among the fragments. The first value is assumed to have been already compared against
    * and be smaller than the value we are looking for. This comparison is the source of the 'shared prefix', which is
-   * the length which the value has in common with the first value of the bucket.
-   * <p>
+   * the length which the value has in common with the previous values of the bucket.
+   *
    * This method uses this shared prefix length to skip more expensive byte by byte full value comparisons when
    * possible by comparing the shared prefix length with the prefix length of the fragment. Since the bucket is always
    * sorted, prefix lengths shrink as you progress to higher indexes, and we can use this to reason that a fragment
    * with a longer prefix length than the shared prefix will always sort before the value we are looking for, and values
    * which have a shorter prefix will always be greater than the value we are looking for, so we only need to do a
    * full comparison if the prefix length is the same
-   * <p>
+   *
    * this method modifies the position of {@link #buffer}
    */
-  private int findValueInBucketV0(
-      ByteBuffer value,
-      int currBucketFirstValueIndex,
-      int bucketSize,
-      int sharedPrefix
-  )
-  {
-    int relativePosition = 0;
-    int prefixLength;
-    // scan through bucket values until we find match or compare numValues
-    int insertionPoint = 1;
-    while (++relativePosition < bucketSize) {
-      prefixLength = VByte.readInt(buffer);
-      if (prefixLength > sharedPrefix) {
-        // this value shares more in common with the first value, so the value we are looking for comes after
-        final int skip = VByte.readInt(buffer);
-        buffer.position(buffer.position() + skip);
-        insertionPoint++;
-      } else if (prefixLength < sharedPrefix) {
-        // prefix is smaller, that means this value sorts ahead of it
-        break;
-      } else {
-        final int fragmentLength = VByte.readInt(buffer);
-        final int common = Math.min(fragmentLength, value.remaining() - prefixLength);
-        int fragmentComparison = 0;
-        for (int i = 0; i < common; i++) {
-          fragmentComparison = StringUtils.compareUtf8UsingJavaStringOrdering(
-              buffer.get(buffer.position() + i),
-              value.get(prefixLength + i)
-          );
-          if (fragmentComparison != 0) {
-            break;
-          }
-        }
-        if (fragmentComparison == 0) {
-          fragmentComparison = Integer.compare(prefixLength + fragmentLength, value.remaining());
-        }
-
-        if (fragmentComparison == 0) {
-          return (currBucketFirstValueIndex + adjustIndex) + relativePosition;
-        } else if (fragmentComparison < 0) {
-          buffer.position(buffer.position() + fragmentLength);
-          insertionPoint++;
-        } else {
-          break;
-        }
-      }
-    }
-    // (-(insertion point) - 1)
-    return -(currBucketFirstValueIndex + adjustIndex) + (~insertionPoint);
-  }
-
-  /**
-   * Get a value from a bucket at a relative position.
-   * <p>
-   * This method modifies the position of the buffer.
-   */
-  static ByteBuffer getFromBucketV0(ByteBuffer buffer, int offset)
-  {
-    int prefixPosition;
-    if (offset == 0) {
-      final int length = VByte.readInt(buffer);
-      final ByteBuffer firstValue = buffer.asReadOnlyBuffer();
-      firstValue.limit(firstValue.position() + length);
-      return firstValue;
-    } else {
-      final int firstLength = VByte.readInt(buffer);
-      prefixPosition = buffer.position();
-      buffer.position(buffer.position() + firstLength);
-    }
-    int pos = 0;
-    int prefixLength;
-    int fragmentLength;
-    int fragmentPosition;
-    // scan through bucket values until we reach offset
-    do {
-      prefixLength = VByte.readInt(buffer);
-      if (++pos < offset) {
-        // not there yet, no need to read anything other than the length to skip ahead
-        final int skipLength = VByte.readInt(buffer);
-        buffer.position(buffer.position() + skipLength);
-      } else {
-        // we've reached our destination
-        fragmentLength = VByte.readInt(buffer);
-        fragmentPosition = buffer.position();
-        break;
-      }
-    } while (true);
-    final int valueLength = prefixLength + fragmentLength;
-    ByteBuffer value = ByteBuffer.allocate(valueLength);
-    for (int i = 0; i < valueLength; i++) {
-      if (i < prefixLength) {
-        value.put(buffer.get(prefixPosition + i));
-      } else {
-        value.put(buffer.get(fragmentPosition + i - prefixLength));
-      }
-    }
-    value.flip();
-    return value;
-  }
-
-
-  /**
-   * Read an entire bucket from a {@link ByteBuffer}, returning an array of reconstructed value bytes.
-   * <p>
-   * This method modifies the position of the buffer.
-   */
-  private static ByteBuffer[] readBucketV0(ByteBuffer bucket, int numValues)
-  {
-    final int length = VByte.readInt(bucket);
-    final byte[] prefixBytes = new byte[length];
-    bucket.get(prefixBytes, 0, length);
-    final ByteBuffer[] bucketBuffers = new ByteBuffer[numValues];
-    bucketBuffers[0] = ByteBuffer.wrap(prefixBytes);
-    int pos = 1;
-    while (pos < numValues) {
-      final int prefixLength = VByte.readInt(bucket);
-      final int fragmentLength = VByte.readInt(bucket);
-      final byte[] fragment = new byte[fragmentLength];
-      bucket.get(fragment, 0, fragmentLength);
-      final ByteBuffer value = ByteBuffer.allocate(prefixLength + fragmentLength);
-      value.put(prefixBytes, 0, prefixLength);
-      value.put(fragment);
-      value.flip();
-      bucketBuffers[pos++] = value;
-    }
-    return bucketBuffers;
-  }
-
-  /**
-   * Finds a value in a bucket among the fragments. The first value is assumed to have been already compared against
-   * and be smaller than the value we are looking for. This comparison is the source of the 'shared prefix', which is
-   * the length which the value has in common with the previous value of the bucket.
-   * <p>
-   * This method uses this shared prefix length to skip more expensive byte by byte full value comparisons when
-   * possible by comparing the shared prefix length with the prefix length of the fragment. Since the bucket is always
-   * sorted, prefix lengths shrink as you progress to higher indexes, and we can use this to reason that a fragment
-   * with a longer prefix length than the shared prefix will always sort before the value we are looking for, and values
-   * which have a shorter prefix will always be greater than the value we are looking for, so we only need to do a
-   * full comparison if the prefix length is the same
-   * <p>
-   * this method modifies the position of {@link #buffer}
-   */
-  private int findValueInBucketV1(
-      ByteBuffer value,
+  private int findValueInBucket(
+      int[] value,
       int currBucketFirstValueIndex,
       int bucketSize,
       int sharedPrefixLength
@@ -570,7 +411,7 @@ public final class FrontCodedIndexed implements Indexed<ByteBuffer>
       if (prefixLength > sharedPrefixLength) {
         // bucket value shares more in common with the preceding value, so the value we are looking for comes after
         final int skip = VByte.readInt(buffer);
-        buffer.position(buffer.position() + skip);
+        buffer.position(buffer.position() + (skip * Integer.BYTES));
         insertionPoint++;
       } else if (prefixLength < sharedPrefixLength) {
         // bucket value prefix is smaller, that means the value we are looking for sorts ahead of it
@@ -578,13 +419,13 @@ public final class FrontCodedIndexed implements Indexed<ByteBuffer>
       } else {
         // value has the same shared prefix, so compare additional values to find
         final int fragmentLength = VByte.readInt(buffer);
-        final int common = Math.min(fragmentLength, value.remaining() - prefixLength);
+        final int common = Math.min(fragmentLength, value.length - prefixLength);
         int fragmentComparison = 0;
         boolean shortCircuit = false;
         for (int i = 0; i < common; i++) {
-          fragmentComparison = StringUtils.compareUtf8UsingJavaStringOrdering(
-              buffer.get(buffer.position() + i),
-              value.get(prefixLength + i)
+          fragmentComparison = Integer.compare(
+              buffer.getInt(buffer.position() + (i * Integer.BYTES)),
+              value[prefixLength + i]
           );
           if (fragmentComparison != 0) {
             sharedPrefixLength = prefixLength + i;
@@ -593,7 +434,7 @@ public final class FrontCodedIndexed implements Indexed<ByteBuffer>
           }
         }
         if (fragmentComparison == 0) {
-          fragmentComparison = Integer.compare(prefixLength + fragmentLength, value.remaining());
+          fragmentComparison = Integer.compare(prefixLength + fragmentLength, value.length);
         }
 
         if (fragmentComparison == 0) {
@@ -603,7 +444,7 @@ public final class FrontCodedIndexed implements Indexed<ByteBuffer>
           if (!shortCircuit) {
             sharedPrefixLength = prefixLength + common;
           }
-          buffer.position(buffer.position() + fragmentLength);
+          buffer.position(buffer.position() + (fragmentLength * Integer.BYTES));
           insertionPoint++;
         } else {
           break;
@@ -614,15 +455,21 @@ public final class FrontCodedIndexed implements Indexed<ByteBuffer>
     return -(currBucketFirstValueIndex + adjustIndex) + (~insertionPoint);
   }
 
-  private ByteBuffer getFromBucketV1(ByteBuffer buffer, int offset)
+  /**
+   * Get a value from a bucket at a relative position.
+   *
+   * This method modifies the position of the buffer.
+   */
+  int[] getFromBucket(ByteBuffer buffer, int offset)
   {
     // first value is written whole
     final int length = VByte.readInt(buffer);
     if (offset == 0) {
-      // return first value directly from underlying buffer since it is stored whole
-      final ByteBuffer value = buffer.asReadOnlyBuffer();
-      value.limit(value.position() + length);
-      return value;
+      final int[] firstValue = new int[length];
+      for (int i = 0; i < length; i++) {
+        firstValue[i] = buffer.getInt();
+      }
+      return firstValue;
     }
     int pos = 0;
     int prefixLength;
@@ -630,7 +477,7 @@ public final class FrontCodedIndexed implements Indexed<ByteBuffer>
     unwindPrefixLength[pos] = 0;
     unwindBufferPosition[pos] = buffer.position();
 
-    buffer.position(buffer.position() + length);
+    buffer.position(buffer.position() + (length * Integer.BYTES));
     do {
       prefixLength = VByte.readInt(buffer);
       if (++pos < offset) {
@@ -638,22 +485,26 @@ public final class FrontCodedIndexed implements Indexed<ByteBuffer>
         final int skipLength = VByte.readInt(buffer);
         unwindPrefixLength[pos] = prefixLength;
         unwindBufferPosition[pos] = buffer.position();
-        buffer.position(buffer.position() + skipLength);
+        buffer.position(buffer.position() + (skipLength * Integer.BYTES));
       } else {
         // we've reached our destination
         fragmentLength = VByte.readInt(buffer);
         if (prefixLength == 0) {
-          // no prefix, return it directly from the underlying buffer
-          final ByteBuffer value = buffer.asReadOnlyBuffer();
-          value.limit(value.position() + fragmentLength);
+          // no prefix, return it directly
+          final int[] value = new int[fragmentLength];
+          for (int i = 0; i < fragmentLength; i++) {
+            value[i] = buffer.getInt();
+          }
           return value;
         }
         break;
       }
     } while (true);
     final int valueLength = prefixLength + fragmentLength;
-    final byte[] valueBytes = new byte[valueLength];
-    buffer.get(valueBytes, prefixLength, fragmentLength);
+    final int[] value = new int[valueLength];
+    for (int i = prefixLength; i < valueLength; i++) {
+      value[i] = buffer.getInt();
+    }
     for (int i = prefixLength; i > 0;) {
       // previous value had a larger prefix than or the same as the value we are looking for
       // skip it since the fragment doesn't have anything we need
@@ -661,55 +512,47 @@ public final class FrontCodedIndexed implements Indexed<ByteBuffer>
         continue;
       }
       buffer.position(unwindBufferPosition[pos]);
-      buffer.get(valueBytes, unwindPrefixLength[pos], i - unwindPrefixLength[pos]);
+      final int prevLength = unwindPrefixLength[pos];
+      for (int fragmentOffset = 0; fragmentOffset < i - prevLength; fragmentOffset++) {
+        value[prevLength + fragmentOffset] = buffer.getInt();
+      }
+
       i = unwindPrefixLength[pos];
     }
-    return ByteBuffer.wrap(valueBytes);
+    return value;
   }
 
+
   /**
    * Read an entire bucket from a {@link ByteBuffer}, returning an array of reconstructed value bytes.
-   * <p>
+   *
    * This method modifies the position of the buffer.
    */
-  private ByteBuffer[] readBucketV1(ByteBuffer bucket, int numValues)
+  private static int[][] readBucket(ByteBuffer bucket, int numValues)
   {
-    final ByteBuffer[] bucketBuffers = new ByteBuffer[numValues];
+    final int[][] bucketValues = new int[numValues][];
 
     // first value is written whole
     final int length = VByte.readInt(bucket);
-    byte[] prefixBytes = new byte[length];
-    bucket.get(prefixBytes, 0, length);
-    bucketBuffers[0] = ByteBuffer.wrap(prefixBytes);
+    int[] prefix = new int[length];
+    for (int i = 0; i < length; i++) {
+      prefix[i] = bucket.getInt();
+    }
+    bucketValues[0] = prefix;
     int pos = 1;
     while (pos < numValues) {
       final int prefixLength = VByte.readInt(bucket);
       final int fragmentLength = VByte.readInt(bucket);
-      byte[] nextValueBytes = new byte[prefixLength + fragmentLength];
-      System.arraycopy(prefixBytes, 0, nextValueBytes, 0, prefixLength);
-      bucket.get(nextValueBytes, prefixLength, fragmentLength);
-      final ByteBuffer value = ByteBuffer.wrap(nextValueBytes);
-      prefixBytes = nextValueBytes;
-      bucketBuffers[pos++] = value;
+      final int[] value = new int[prefixLength + fragmentLength];
+      for (int i = 0; i < prefixLength; i++) {
+        value[i] = prefix[i];
+      }
+      for (int i = prefixLength; i < value.length; i++) {
+        value[i] = bucket.getInt();
+      }
+      prefix = value;
+      bucketValues[pos++] = value;
     }
-    return bucketBuffers;
-  }
-
-  @FunctionalInterface
-  interface GetBucketValue
-  {
-    ByteBuffer get(ByteBuffer buffer, int offset);
-  }
-
-  @FunctionalInterface
-  interface ReadBucket
-  {
-    ByteBuffer[] readBucket(ByteBuffer buffer, int bucketSize);
-  }
-
-  @FunctionalInterface
-  interface FindInBucket
-  {
-    int find(ByteBuffer value, int currBucketFirstValueIndex, int bucketSize, int sharedPrefixLength);
+    return bucketValues;
   }
 }
diff --git a/processing/src/main/java/org/apache/druid/segment/data/FrontCodedIntArrayIndexedWriter.java b/processing/src/main/java/org/apache/druid/segment/data/FrontCodedIntArrayIndexedWriter.java
new file mode 100644
index 0000000000..00cc2079fc
--- /dev/null
+++ b/processing/src/main/java/org/apache/druid/segment/data/FrontCodedIntArrayIndexedWriter.java
@@ -0,0 +1,409 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.druid.segment.data;
+
+import com.google.common.primitives.Ints;
+import org.apache.druid.common.config.NullHandling;
+import org.apache.druid.io.Channels;
+import org.apache.druid.java.util.common.IAE;
+import org.apache.druid.java.util.common.ISE;
+import org.apache.druid.java.util.common.io.smoosh.FileSmoosher;
+import org.apache.druid.segment.writeout.SegmentWriteOutMedium;
+import org.apache.druid.segment.writeout.WriteOutBytes;
+
+import javax.annotation.Nullable;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.ByteOrder;
+import java.nio.channels.WritableByteChannel;
+import java.util.Arrays;
+import java.util.Comparator;
+
+public class FrontCodedIntArrayIndexedWriter implements DictionaryWriter<int[]>
+{
+  private static final int MAX_LOG_BUFFER_SIZE = 26;
+
+  public static final Comparator<int[]> ARRAY_COMPARATOR = (o1, o2) -> {
+    //noinspection ArrayEquality
+    if (o1 == o2) {
+      return 0;
+    }
+    if (o1 == null) {
+      return -1;
+    }
+    if (o2 == null) {
+      return 1;
+    }
+    final int iter = Math.min(o1.length, o2.length);
+    for (int i = 0; i < iter; i++) {
+      final int cmp = Integer.compare(o1[i], o2[i]);
+      if (cmp == 0) {
+        continue;
+      }
+      return cmp;
+    }
+    return Integer.compare(o1.length, o2.length);
+  };
+
+  private final SegmentWriteOutMedium segmentWriteOutMedium;
+  private final int bucketSize;
+  private final ByteOrder byteOrder;
+  private final int[][] bucketBuffer;
+  private final ByteBuffer getOffsetBuffer;
+  private final int div;
+
+  @Nullable
+  private int[] prevObject = null;
+  @Nullable
+  private WriteOutBytes headerOut = null;
+  @Nullable
+  private WriteOutBytes valuesOut = null;
+  private int numWritten = 0;
+  private ByteBuffer scratch;
+  private int logScratchSize = 10;
+  private boolean isClosed = false;
+  private boolean hasNulls = false;
+
+  public FrontCodedIntArrayIndexedWriter(
+      SegmentWriteOutMedium segmentWriteOutMedium,
+      ByteOrder byteOrder,
+      int bucketSize
+  )
+  {
+    if (Integer.bitCount(bucketSize) != 1 || bucketSize < 1 || bucketSize > 128) {
+      throw new IAE("bucketSize must be a power of two (from 1 up to 128) but was[%,d]", bucketSize);
+    }
+    this.segmentWriteOutMedium = segmentWriteOutMedium;
+    this.scratch = ByteBuffer.allocate(1 << logScratchSize).order(byteOrder);
+    this.bucketSize = bucketSize;
+    this.byteOrder = byteOrder;
+    this.bucketBuffer = new int[bucketSize][];
+    this.getOffsetBuffer = ByteBuffer.allocate(Integer.BYTES).order(byteOrder);
+    this.div = Integer.numberOfTrailingZeros(bucketSize);
+  }
+
+  @Override
+  public void open() throws IOException
+  {
+    headerOut = segmentWriteOutMedium.makeWriteOutBytes();
+    valuesOut = segmentWriteOutMedium.makeWriteOutBytes();
+  }
+
+  @Override
+  public void write(@Nullable int[] value) throws IOException
+  {
+
+    if (prevObject != null && ARRAY_COMPARATOR.compare(prevObject, value) >= 0) {
+      throw new ISE(
+          "Values must be sorted and unique. Element [%s] with value [%s] is before or equivalent to [%s]",
+          numWritten,
+          value == null ? null : Arrays.toString(value),
+          Arrays.toString(prevObject)
+      );
+    }
+
+    if (value == null) {
+      hasNulls = true;
+      return;
+    }
+
+    // if the bucket buffer is full, write the bucket
+    if (numWritten > 0 && (numWritten % bucketSize) == 0) {
+      resetScratch();
+      int written;
+      // write the bucket, growing scratch buffer as necessary
+      do {
+        written = writeBucket(scratch, bucketBuffer, bucketSize);
+        if (written < 0) {
+          growScratch();
+        }
+      } while (written < 0);
+      scratch.flip();
+      Channels.writeFully(valuesOut, scratch);
+
+      resetScratch();
+      // write end offset for current value
+      scratch.putInt((int) valuesOut.size());
+      scratch.flip();
+      Channels.writeFully(headerOut, scratch);
+    }
+
+    bucketBuffer[numWritten % bucketSize] = value;
+
+    ++numWritten;
+    prevObject = value;
+  }
+
+
+  @Override
+  public long getSerializedSize() throws IOException
+  {
+    if (!isClosed) {
+      flush();
+    }
+    int headerAndValues = Ints.checkedCast(headerOut.size() + valuesOut.size());
+    return Byte.BYTES +
+           Byte.BYTES +
+           Byte.BYTES +
+           VByte.computeIntSize(numWritten) +
+           VByte.computeIntSize(headerAndValues) +
+           headerAndValues;
+  }
+
+  @Override
+  public void writeTo(WritableByteChannel channel, FileSmoosher smoosher) throws IOException
+  {
+    if (!isClosed) {
+      flush();
+    }
+    resetScratch();
+    // version 0
+    scratch.put((byte) 0);
+    scratch.put((byte) bucketSize);
+    scratch.put(hasNulls ? NullHandling.IS_NULL_BYTE : NullHandling.IS_NOT_NULL_BYTE);
+    VByte.writeInt(scratch, numWritten);
+    VByte.writeInt(scratch, Ints.checkedCast(headerOut.size() + valuesOut.size()));
+    scratch.flip();
+    Channels.writeFully(channel, scratch);
+    headerOut.writeTo(channel);
+    valuesOut.writeTo(channel);
+  }
+
+  @Override
+  public boolean isSorted()
+  {
+    return true;
+  }
+
+  @Nullable
+  @Override
+  public int[] get(int index) throws IOException
+  {
+    if (index == 0 && hasNulls) {
+      return null;
+    }
+    final int adjustedIndex = hasNulls ? index - 1 : index;
+    final int relativeIndex = adjustedIndex % bucketSize;
+    // check for current page
+    if (adjustedIndex >= numWritten - bucketSize) {
+      return bucketBuffer[relativeIndex];
+    } else {
+      final int bucket = adjustedIndex >> div;
+      long startOffset;
+      if (bucket == 0) {
+        startOffset = 0;
+      } else {
+        startOffset = getBucketOffset(bucket - 1);
+      }
+      long endOffset = getBucketOffset(bucket);
+      int currentBucketSize = Ints.checkedCast(endOffset - startOffset);
+      if (currentBucketSize == 0) {
+        return null;
+      }
+      final ByteBuffer bucketBuffer = ByteBuffer.allocate(currentBucketSize).order(byteOrder);
+      valuesOut.readFully(startOffset, bucketBuffer);
+      bucketBuffer.clear();
+      return getFromBucket(bucketBuffer, relativeIndex);
+    }
+  }
+
+  private long getBucketOffset(int index) throws IOException
+  {
+    getOffsetBuffer.clear();
+    headerOut.readFully(index * (long) Integer.BYTES, getOffsetBuffer);
+    return getOffsetBuffer.getInt(0);
+  }
+
+  private void flush() throws IOException
+  {
+    if (numWritten == 0) {
+      return;
+    }
+    int remainder = numWritten % bucketSize;
+    resetScratch();
+    int written;
+    do {
+      written = writeBucket(scratch, bucketBuffer, remainder == 0 ? bucketSize : remainder);
+      if (written < 0) {
+        growScratch();
+      }
+    } while (written < 0);
+    scratch.flip();
+    Channels.writeFully(valuesOut, scratch);
+    resetScratch();
+    isClosed = true;
+  }
+
+  private void resetScratch()
+  {
+    scratch.position(0);
+    scratch.limit(scratch.capacity());
+  }
+
+  private void growScratch()
+  {
+    if (logScratchSize < MAX_LOG_BUFFER_SIZE) {
+      this.scratch = ByteBuffer.allocate(1 << ++logScratchSize).order(byteOrder);
+    } else {
+      throw new IllegalStateException("scratch buffer to big to write buckets");
+    }
+  }
+
+  /**
+   * Write bucket of values to a {@link ByteBuffer}. The first value is written completely, subsequent values are
+   * written with an integer to indicate how much of the first value in the bucket is a prefix of the value, followed
+   * by the remaining bytes of the value.
+   *
+   * Uses {@link VByte} encoded integers to indicate prefix length and value length.
+   */
+  public static int writeBucket(ByteBuffer buffer, int[][] values, int numValues)
+  {
+    int written = 0;
+    int[] prev = null;
+    while (written < numValues) {
+      int[] next = values[written];
+      if (written == 0) {
+        prev = next;
+        // the first value in the bucket is written completely as it is
+        int rem = writeValue(buffer, prev);
+        // wasn't enough room, bail out
+        if (rem < 0) {
+          return rem;
+        }
+      } else {
+        // all other values must be partitioned into a prefix length and suffix bytes
+        int prefixLength = 0;
+        for (; prefixLength < prev.length; prefixLength++) {
+          final int cmp = Integer.compare(prev[prefixLength], next[prefixLength]);
+          if (cmp != 0) {
+            break;
+          }
+        }
+        // convert to bytes because not every char is a single byte
+        final int[] suffix = new int[next.length - prefixLength];
+        System.arraycopy(next, prefixLength, suffix, 0, suffix.length);
+        int rem = buffer.remaining() - VByte.computeIntSize(prefixLength);
+        // wasn't enough room, bail out
+        if (rem < 0) {
+          return rem;
+        }
+        VByte.writeInt(buffer, prefixLength);
+        rem = writeValue(buffer, suffix);
+        prev = next;
+        // wasn't enough room, bail out
+        if (rem < 0) {
+          return rem;
+        }
+      }
+      written++;
+    }
+    return written;
+  }
+
+  /**
+   * Write a variable length int[] value to a {@link ByteBuffer}, storing the length as a {@link VByte} encoded
+   * integer followed by the value itself. Returns the number of bytes written to the buffer. This method returns a
+   * negative value if there is no room available in the buffer, so that it can be grown if needed.
+   */
+  public static int writeValue(ByteBuffer buffer, int[] ints)
+  {
+    int remaining = buffer.remaining() - VByte.computeIntSize(ints.length) - ints.length;
+    if (remaining < 0) {
+      return remaining;
+    }
+    final int pos = buffer.position();
+    VByte.writeInt(buffer, ints.length);
+    for (int anInt : ints) {
+      remaining = buffer.remaining() - Integer.BYTES;
+      if (remaining < 0) {
+        return remaining;
+      }
+      buffer.putInt(anInt);
+    }
+    return buffer.position() - pos;
+  }
+
+  /**
+   * Copy of {@link FrontCodedIntArrayIndexed#getFromBucket(ByteBuffer, int)} but with local declarations of arrays
+   * for unwinding stuff
+   */
+  int[] getFromBucket(ByteBuffer buffer, int offset)
+  {
+    int[] unwindPrefixLength = new int[bucketSize];
+    int[] unwindBufferPosition = new int[bucketSize];
+    // first value is written whole
+    final int length = VByte.readInt(buffer);
+    if (offset == 0) {
+      final int[] firstValue = new int[length];
+      for (int i = 0; i < length; i++) {
+        firstValue[i] = buffer.getInt();
+      }
+      return firstValue;
+    }
+    int pos = 0;
+    int prefixLength;
+    int fragmentLength;
+    unwindPrefixLength[pos] = 0;
+    unwindBufferPosition[pos] = buffer.position();
+
+    buffer.position(buffer.position() + (length * Integer.BYTES));
+    do {
+      prefixLength = VByte.readInt(buffer);
+      if (++pos < offset) {
+        // not there yet, no need to read anything other than the length to skip ahead
+        final int skipLength = VByte.readInt(buffer);
+        unwindPrefixLength[pos] = prefixLength;
+        unwindBufferPosition[pos] = buffer.position();
+        buffer.position(buffer.position() + (skipLength * Integer.BYTES));
+      } else {
+        // we've reached our destination
+        fragmentLength = VByte.readInt(buffer);
+        if (prefixLength == 0) {
+          // no prefix, return it directly
+          final int[] value = new int[fragmentLength];
+          for (int i = 0; i < fragmentLength; i++) {
+            value[i] = buffer.getInt();
+          }
+          return value;
+        }
+        break;
+      }
+    } while (true);
+    final int valueLength = prefixLength + fragmentLength;
+    final int[] value = new int[valueLength];
+    for (int i = prefixLength; i < valueLength; i++) {
+      value[i] = buffer.getInt();
+    }
+    for (int i = prefixLength; i > 0;) {
+      // previous value had a larger prefix than or the same as the value we are looking for
+      // skip it since the fragment doesn't have anything we need
+      if (unwindPrefixLength[--pos] >= i) {
+        continue;
+      }
+      buffer.position(unwindBufferPosition[pos]);
+      final int prevLength = unwindPrefixLength[pos];
+      for (int fragmentOffset = 0; fragmentOffset < i - prevLength; fragmentOffset++) {
+        value[prevLength + fragmentOffset] = buffer.getInt();
+      }
+      i = unwindPrefixLength[pos];
+    }
+    return value;
+  }
+}
diff --git a/processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndexSchema.java b/processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndexSchema.java
index 2931eaeb2b..d127f3d89b 100644
--- a/processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndexSchema.java
+++ b/processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndexSchema.java
@@ -33,6 +33,11 @@ import javax.annotation.Nullable;
  */
 public class IncrementalIndexSchema
 {
+  public static IncrementalIndexSchema.Builder builder()
+  {
+    return new Builder();
+  }
+
   private final long minTimestamp;
   private final TimestampSpec timestampSpec;
   private final Granularity gran;
diff --git a/processing/src/main/java/org/apache/druid/segment/nested/StringFieldColumnWriter.java b/processing/src/main/java/org/apache/druid/segment/nested/ArrayFieldColumnWriter.java
similarity index 55%
copy from processing/src/main/java/org/apache/druid/segment/nested/StringFieldColumnWriter.java
copy to processing/src/main/java/org/apache/druid/segment/nested/ArrayFieldColumnWriter.java
index c9f10a269c..1ae465d25c 100644
--- a/processing/src/main/java/org/apache/druid/segment/nested/StringFieldColumnWriter.java
+++ b/processing/src/main/java/org/apache/druid/segment/nested/ArrayFieldColumnWriter.java
@@ -19,7 +19,7 @@
 
 package org.apache.druid.segment.nested;
 
-import org.apache.druid.common.config.NullHandling;
+import com.google.common.base.Preconditions;
 import org.apache.druid.java.util.common.io.smoosh.FileSmoosher;
 import org.apache.druid.segment.IndexSpec;
 import org.apache.druid.segment.writeout.SegmentWriteOutMedium;
@@ -27,12 +27,10 @@ import org.apache.druid.segment.writeout.SegmentWriteOutMedium;
 import java.io.IOException;
 import java.nio.channels.WritableByteChannel;
 
-/**
- * Literal field writer for string type nested columns of {@link NestedDataColumnSerializer}
- */
-public final class StringFieldColumnWriter extends GlobalDictionaryEncodedFieldColumnWriter<String>
+public class ArrayFieldColumnWriter extends GlobalDictionaryEncodedFieldColumnWriter<int[]>
 {
-  public StringFieldColumnWriter(
+
+  protected ArrayFieldColumnWriter(
       String columnName,
       String fieldName,
       SegmentWriteOutMedium segmentWriteOutMedium,
@@ -44,18 +42,38 @@ public final class StringFieldColumnWriter extends GlobalDictionaryEncodedFieldC
   }
 
   @Override
-  String processValue(Object value)
+  int[] processValue(int row, Object value)
   {
-    if (value == null) {
-      return null;
+    if (value instanceof Object[]) {
+      Object[] array = (Object[]) value;
+      final int[] globalIds = new int[array.length];
+      for (int i = 0; i < array.length; i++) {
+        if (array[i] == null) {
+          globalIds[i] = 0;
+        } else if (array[i] instanceof String) {
+          globalIds[i] = globalDictionaryIdLookup.lookupString((String) array[i]);
+        } else if (array[i] instanceof Long) {
+          globalIds[i] = globalDictionaryIdLookup.lookupLong((Long) array[i]);
+        } else if (array[i] instanceof Double) {
+          globalIds[i] = globalDictionaryIdLookup.lookupDouble((Double) array[i]);
+        } else {
+          globalIds[i] = -1;
+        }
+        Preconditions.checkArgument(globalIds[i] >= 0, "unknown global id [%s] for value [%s]", globalIds[i], array[i]);
+        arrayElements.computeIfAbsent(
+            globalIds[i],
+            (id) -> indexSpec.getBitmapSerdeFactory().getBitmapFactory().makeEmptyMutableBitmap()
+        ).add(row);
+      }
+      return globalIds;
     }
-    return NullHandling.emptyToNullIfNeeded(String.valueOf(value));
+    return null;
   }
 
   @Override
-  int lookupGlobalId(String value)
+  int lookupGlobalId(int[] value)
   {
-    return globalDictionaryIdLookup.lookupString(value);
+    return globalDictionaryIdLookup.lookupArray(value);
   }
 
   @Override
diff --git a/processing/src/main/java/org/apache/druid/segment/nested/CompressedNestedDataComplexColumn.java b/processing/src/main/java/org/apache/druid/segment/nested/CompressedNestedDataComplexColumn.java
index 77919c4bff..1d4a683384 100644
--- a/processing/src/main/java/org/apache/druid/segment/nested/CompressedNestedDataComplexColumn.java
+++ b/processing/src/main/java/org/apache/druid/segment/nested/CompressedNestedDataComplexColumn.java
@@ -22,7 +22,9 @@ package org.apache.druid.segment.nested;
 import com.google.common.base.Preconditions;
 import com.google.common.base.Supplier;
 import com.google.common.collect.ImmutableList;
+import com.google.common.primitives.Doubles;
 import org.apache.druid.collections.bitmap.ImmutableBitmap;
+import org.apache.druid.java.util.common.IAE;
 import org.apache.druid.java.util.common.ISE;
 import org.apache.druid.java.util.common.RE;
 import org.apache.druid.java.util.common.StringUtils;
@@ -30,6 +32,7 @@ import org.apache.druid.java.util.common.io.Closer;
 import org.apache.druid.java.util.common.io.smoosh.SmooshedFileMapper;
 import org.apache.druid.query.extraction.ExtractionFn;
 import org.apache.druid.query.monomorphicprocessing.RuntimeShapeInspector;
+import org.apache.druid.segment.BaseSingleValueDimensionSelector;
 import org.apache.druid.segment.ColumnValueSelector;
 import org.apache.druid.segment.DimensionSelector;
 import org.apache.druid.segment.NilColumnValueSelector;
@@ -41,7 +44,6 @@ import org.apache.druid.segment.column.ColumnHolder;
 import org.apache.druid.segment.column.ColumnIndexSupplier;
 import org.apache.druid.segment.column.ColumnType;
 import org.apache.druid.segment.column.DictionaryEncodedColumn;
-import org.apache.druid.segment.column.ValueType;
 import org.apache.druid.segment.data.ColumnarDoubles;
 import org.apache.druid.segment.data.ColumnarInts;
 import org.apache.druid.segment.data.ColumnarLongs;
@@ -51,6 +53,7 @@ import org.apache.druid.segment.data.CompressedVSizeColumnarIntsSupplier;
 import org.apache.druid.segment.data.CompressedVariableSizedBlobColumn;
 import org.apache.druid.segment.data.CompressedVariableSizedBlobColumnSupplier;
 import org.apache.druid.segment.data.FixedIndexed;
+import org.apache.druid.segment.data.FrontCodedIntArrayIndexed;
 import org.apache.druid.segment.data.GenericIndexed;
 import org.apache.druid.segment.data.Indexed;
 import org.apache.druid.segment.data.ObjectStrategy;
@@ -58,6 +61,7 @@ import org.apache.druid.segment.data.ReadableOffset;
 import org.apache.druid.segment.data.VSizeColumnarInts;
 import org.apache.druid.segment.data.WritableSupplier;
 import org.apache.druid.segment.serde.DictionaryEncodedColumnPartSerde;
+import org.apache.druid.segment.serde.NoIndexesColumnIndexSupplier;
 import org.apache.druid.segment.vector.NilVectorSelector;
 import org.apache.druid.segment.vector.ReadableVectorInspector;
 import org.apache.druid.segment.vector.ReadableVectorOffset;
@@ -71,6 +75,7 @@ import java.io.IOException;
 import java.nio.ByteBuffer;
 import java.util.ArrayList;
 import java.util.Collections;
+import java.util.Iterator;
 import java.util.List;
 import java.util.Set;
 import java.util.concurrent.ConcurrentHashMap;
@@ -89,16 +94,18 @@ public abstract class CompressedNestedDataComplexColumn<TStringDictionary extend
   private final ImmutableBitmap nullValues;
 
   private final GenericIndexed<String> fields;
-  private final NestedLiteralTypeInfo fieldInfo;
+  private final NestedFieldTypeInfo fieldInfo;
 
   private final Supplier<TStringDictionary> stringDictionarySupplier;
   private final Supplier<FixedIndexed<Long>> longDictionarySupplier;
   private final Supplier<FixedIndexed<Double>> doubleDictionarySupplier;
+  private final Supplier<FrontCodedIntArrayIndexed> arrayDictionarySupplier;
+
   private final SmooshedFileMapper fileMapper;
 
   private final String rootFieldPath;
 
-  private final ConcurrentHashMap<String, ColumnHolder> columns = new ConcurrentHashMap<>();
+  private final ConcurrentHashMap<Integer, ColumnHolder> columns = new ConcurrentHashMap<>();
 
   private static final ObjectStrategy<Object> STRATEGY = NestedDataComplexTypeSerde.INSTANCE.getObjectStrategy();
 
@@ -108,10 +115,11 @@ public abstract class CompressedNestedDataComplexColumn<TStringDictionary extend
       CompressedVariableSizedBlobColumnSupplier compressedRawColumnSupplier,
       ImmutableBitmap nullValues,
       GenericIndexed<String> fields,
-      NestedLiteralTypeInfo fieldInfo,
+      NestedFieldTypeInfo fieldInfo,
       Supplier<TStringDictionary> stringDictionary,
       Supplier<FixedIndexed<Long>> longDictionarySupplier,
       Supplier<FixedIndexed<Double>> doubleDictionarySupplier,
+      Supplier<FrontCodedIntArrayIndexed> arrayDictionarySupplier,
       SmooshedFileMapper fileMapper,
       String rootFieldPath
   )
@@ -123,6 +131,7 @@ public abstract class CompressedNestedDataComplexColumn<TStringDictionary extend
     this.stringDictionarySupplier = stringDictionary;
     this.longDictionarySupplier = longDictionarySupplier;
     this.doubleDictionarySupplier = doubleDictionarySupplier;
+    this.arrayDictionarySupplier = arrayDictionarySupplier;
     this.fileMapper = fileMapper;
     this.closer = Closer.create();
     this.compressedRawColumnSupplier = compressedRawColumnSupplier;
@@ -150,7 +159,7 @@ public abstract class CompressedNestedDataComplexColumn<TStringDictionary extend
     return fieldParts;
   }
 
-  public NestedLiteralTypeInfo getFieldInfo()
+  public NestedFieldTypeInfo getFieldInfo()
   {
     return fieldInfo;
   }
@@ -170,6 +179,60 @@ public abstract class CompressedNestedDataComplexColumn<TStringDictionary extend
     return doubleDictionarySupplier.get();
   }
 
+  public FrontCodedIntArrayIndexed getArrayDictionary()
+  {
+    return arrayDictionarySupplier.get();
+  }
+
+  public Iterable<Object[]> getArraysIterable()
+  {
+    Iterable<Object[]> arrays = () -> {
+      final TStringDictionary stringDictionary = stringDictionarySupplier.get();
+      final FixedIndexed<Long> longDictionary = longDictionarySupplier.get();
+      final FixedIndexed<Double> doubleDictionary = doubleDictionarySupplier.get();
+
+      return new Iterator<Object[]>()
+      {
+        final Iterator<int[]> delegate = arrayDictionarySupplier.get().iterator();
+
+        @Override
+        public boolean hasNext()
+        {
+          return delegate.hasNext();
+        }
+
+        @Override
+        public Object[] next()
+        {
+          final int[] next = delegate.next();
+          final Object[] nextArray = new Object[next.length];
+          for (int i = 0; i < nextArray.length; i++) {
+            nextArray[i] = lookupId(next[i]);
+          }
+          return nextArray;
+        }
+
+        private Object lookupId(int globalId)
+        {
+          if (globalId == 0) {
+            return null;
+          }
+          final int adjustLongId = stringDictionary.size();
+          final int adjustDoubleId = stringDictionary.size() + longDictionary.size();
+          if (globalId < adjustLongId) {
+            return StringUtils.fromUtf8Nullable(stringDictionary.get(globalId));
+          } else if (globalId < adjustDoubleId) {
+            return longDictionary.get(globalId - adjustLongId);
+          } else if (globalId < adjustDoubleId + doubleDictionary.size()) {
+            return doubleDictionary.get(globalId - adjustDoubleId);
+          }
+          throw new IAE("Unknown globalId [%s]", globalId);
+        }
+      };
+    };
+    return arrays;
+  }
+
   public ImmutableBitmap getNullValues()
   {
     return nullValues;
@@ -327,17 +390,59 @@ public abstract class CompressedNestedDataComplexColumn<TStringDictionary extend
   }
 
   @Override
-  public DimensionSelector makeDimensionSelector(List<NestedPathPart> path, ReadableOffset readableOffset, ExtractionFn fn)
+  public DimensionSelector makeDimensionSelector(
+      List<NestedPathPart> path,
+      ReadableOffset readableOffset,
+      ExtractionFn fn
+  )
   {
     final String field = getField(path);
     Preconditions.checkNotNull(field, "Null field");
-
-    if (fields.indexOf(field) >= 0) {
-      DictionaryEncodedColumn<?> col = (DictionaryEncodedColumn<?>) getColumnHolder(field).getColumn();
+    final int fieldIndex = fields.indexOf(field);
+    if (fieldIndex >= 0) {
+      DictionaryEncodedColumn<?> col = (DictionaryEncodedColumn<?>) getColumnHolder(field, fieldIndex).getColumn();
       return col.makeDimensionSelector(readableOffset, fn);
-    } else {
-      return DimensionSelector.constant(null);
     }
+    if (!path.isEmpty() && path.get(path.size() - 1) instanceof NestedPathArrayElement) {
+      final NestedPathPart lastPath = path.get(path.size() - 1);
+      final String arrayField = getField(path.subList(0, path.size() - 1));
+      final int arrayFieldIndex = fields.indexOf(arrayField);
+      if (arrayFieldIndex >= 0) {
+        final int elementNumber = ((NestedPathArrayElement) lastPath).getIndex();
+        if (elementNumber < 0) {
+          throw new IAE("Cannot make array element selector for path [%s], negative array index not supported for this selector", path);
+        }
+        DictionaryEncodedColumn<?> col = (DictionaryEncodedColumn<?>) getColumnHolder(arrayField, arrayFieldIndex).getColumn();
+        ColumnValueSelector<?> arraySelector = col.makeColumnValueSelector(readableOffset);
+        return new BaseSingleValueDimensionSelector()
+        {
+          @Nullable
+          @Override
+          protected String getValue()
+          {
+            Object o = arraySelector.getObject();
+            if (o instanceof Object[]) {
+              Object[] array = (Object[]) o;
+              if (elementNumber < array.length) {
+                Object element = array[elementNumber];
+                if (element == null) {
+                  return null;
+                }
+                return String.valueOf(element);
+              }
+            }
+            return null;
+          }
+
+          @Override
+          public void inspectRuntimeShape(RuntimeShapeInspector inspector)
+          {
+            arraySelector.inspectRuntimeShape(inspector);
+          }
+        };
+      }
+    }
+    return DimensionSelector.constant(null);
   }
 
   @Override
@@ -346,12 +451,81 @@ public abstract class CompressedNestedDataComplexColumn<TStringDictionary extend
     final String field = getField(path);
     Preconditions.checkNotNull(field, "Null field");
 
-    if (fields.indexOf(field) >= 0) {
-      BaseColumn col = getColumnHolder(field).getColumn();
+    final int fieldIndex = fields.indexOf(field);
+    if (fieldIndex >= 0) {
+      BaseColumn col = getColumnHolder(field, fieldIndex).getColumn();
       return col.makeColumnValueSelector(readableOffset);
-    } else {
-      return NilColumnValueSelector.instance();
     }
+    if (!path.isEmpty() && path.get(path.size() - 1) instanceof NestedPathArrayElement) {
+      final NestedPathPart lastPath = path.get(path.size() - 1);
+      final String arrayField = getField(path.subList(0, path.size() - 1));
+      final int arrayFieldIndex = fields.indexOf(arrayField);
+      if (arrayFieldIndex >= 0) {
+        final int elementNumber = ((NestedPathArrayElement) lastPath).getIndex();
+        if (elementNumber < 0) {
+          throw new IAE("Cannot make array element selector for path [%s], negative array index not supported for this selector", path);
+        }
+        DictionaryEncodedColumn<?> col = (DictionaryEncodedColumn<?>) getColumnHolder(arrayField, arrayFieldIndex).getColumn();
+        ColumnValueSelector arraySelector = col.makeColumnValueSelector(readableOffset);
+        return new ColumnValueSelector<Object>()
+        {
+          @Override
+          public boolean isNull()
+          {
+            Object o = getObject();
+            return !(o instanceof Number);
+          }
+
+          @Override
+          public long getLong()
+          {
+            Object o = getObject();
+            return o instanceof Number ? ((Number) o).longValue() : 0L;
+          }
+
+          @Override
+          public float getFloat()
+          {
+            Object o = getObject();
+            return o instanceof Number ? ((Number) o).floatValue() : 0f;
+          }
+
+          @Override
+          public double getDouble()
+          {
+            Object o = getObject();
+            return o instanceof Number ? ((Number) o).doubleValue() : 0.0;
+          }
+
+          @Override
+          public void inspectRuntimeShape(RuntimeShapeInspector inspector)
+          {
+            arraySelector.inspectRuntimeShape(inspector);
+          }
+
+          @Nullable
+          @Override
+          public Object getObject()
+          {
+            Object o = arraySelector.getObject();
+            if (o instanceof Object[]) {
+              Object[] array = (Object[]) o;
+              if (elementNumber < array.length) {
+                return array[elementNumber];
+              }
+            }
+            return null;
+          }
+
+          @Override
+          public Class<?> classOfObject()
+          {
+            return Object.class;
+          }
+        };
+      }
+    }
+    return NilColumnValueSelector.instance();
   }
 
   @Override
@@ -362,9 +536,9 @@ public abstract class CompressedNestedDataComplexColumn<TStringDictionary extend
   {
     final String field = getField(path);
     Preconditions.checkNotNull(field, "Null field");
-
-    if (fields.indexOf(field) >= 0) {
-      DictionaryEncodedColumn<?> col = (DictionaryEncodedColumn<?>) getColumnHolder(field).getColumn();
+    final int fieldIndex = fields.indexOf(field);
+    if (fieldIndex >= 0) {
+      DictionaryEncodedColumn<?> col = (DictionaryEncodedColumn<?>) getColumnHolder(field, fieldIndex).getColumn();
       return col.makeSingleValueDimensionVectorSelector(readableOffset);
     } else {
       return NilVectorSelector.create(readableOffset);
@@ -376,13 +550,66 @@ public abstract class CompressedNestedDataComplexColumn<TStringDictionary extend
   {
     final String field = getField(path);
     Preconditions.checkNotNull(field, "Null field");
-
-    if (fields.indexOf(field) >= 0) {
-      BaseColumn col = getColumnHolder(field).getColumn();
+    final int fieldIndex = fields.indexOf(field);
+    if (fieldIndex >= 0) {
+      BaseColumn col = getColumnHolder(field, fieldIndex).getColumn();
       return col.makeVectorObjectSelector(readableOffset);
-    } else {
-      return NilVectorSelector.create(readableOffset);
     }
+    if (!path.isEmpty() && path.get(path.size() - 1) instanceof NestedPathArrayElement) {
+      final NestedPathPart lastPath = path.get(path.size() - 1);
+      final String arrayField = getField(path.subList(0, path.size() - 1));
+      final int arrayFieldIndex = fields.indexOf(arrayField);
+      if (arrayFieldIndex >= 0) {
+        final int elementNumber = ((NestedPathArrayElement) lastPath).getIndex();
+        if (elementNumber < 0) {
+          throw new IAE("Cannot make array element selector for path [%s], negative array index not supported for this selector", path);
+        }
+        DictionaryEncodedColumn<?> col = (DictionaryEncodedColumn<?>) getColumnHolder(arrayField, arrayFieldIndex).getColumn();
+        VectorObjectSelector arraySelector = col.makeVectorObjectSelector(readableOffset);
+
+        return new VectorObjectSelector()
+        {
+          private final Object[] elements = new Object[arraySelector.getMaxVectorSize()];
+          private int id = ReadableVectorInspector.NULL_ID;
+
+          @Override
+          public Object[] getObjectVector()
+          {
+            if (readableOffset.getId() != id) {
+              final Object[] delegate = arraySelector.getObjectVector();
+              for (int i = 0; i < arraySelector.getCurrentVectorSize(); i++) {
+                Object maybeArray = delegate[i];
+                if (maybeArray instanceof Object[]) {
+                  Object[] anArray = (Object[]) maybeArray;
+                  if (elementNumber < anArray.length) {
+                    elements[i] = anArray[elementNumber];
+                  } else {
+                    elements[i] = null;
+                  }
+                } else {
+                  elements[i] = null;
+                }
+              }
+              id = readableOffset.getId();
+            }
+            return elements;
+          }
+
+          @Override
+          public int getMaxVectorSize()
+          {
+            return arraySelector.getMaxVectorSize();
+          }
+
+          @Override
+          public int getCurrentVectorSize()
+          {
+            return arraySelector.getCurrentVectorSize();
+          }
+        };
+      }
+    }
+    return NilVectorSelector.create(readableOffset);
   }
 
   @Override
@@ -390,15 +617,132 @@ public abstract class CompressedNestedDataComplexColumn<TStringDictionary extend
   {
     final String field = getField(path);
     Preconditions.checkNotNull(field, "Null field");
-
-    if (fields.indexOf(field) >= 0) {
-      BaseColumn col = getColumnHolder(field).getColumn();
+    final int fieldIndex = fields.indexOf(field);
+    if (fieldIndex >= 0) {
+      BaseColumn col = getColumnHolder(field, fieldIndex).getColumn();
       return col.makeVectorValueSelector(readableOffset);
-    } else {
-      return NilVectorSelector.create(readableOffset);
     }
-  }
+    if (!path.isEmpty() && path.get(path.size() - 1) instanceof NestedPathArrayElement) {
+      final NestedPathPart lastPath = path.get(path.size() - 1);
+      final String arrayField = getField(path.subList(0, path.size() - 1));
+      final int arrayFieldIndex = fields.indexOf(arrayField);
+      if (arrayFieldIndex >= 0) {
+        final int elementNumber = ((NestedPathArrayElement) lastPath).getIndex();
+        if (elementNumber < 0) {
+          throw new IAE("Cannot make array element selector for path [%s], negative array index not supported for this selector", path);
+        }
+        DictionaryEncodedColumn<?> col = (DictionaryEncodedColumn<?>) getColumnHolder(arrayField, arrayFieldIndex).getColumn();
+        VectorObjectSelector arraySelector = col.makeVectorObjectSelector(readableOffset);
+
+        return new VectorValueSelector()
+        {
+          private final long[] longs = new long[readableOffset.getMaxVectorSize()];
+          private final double[] doubles = new double[readableOffset.getMaxVectorSize()];
+          private final float[] floats = new float[readableOffset.getMaxVectorSize()];
+          private final boolean[] nulls = new boolean[readableOffset.getMaxVectorSize()];
+          private int id = ReadableVectorInspector.NULL_ID;
+
+          private void computeNumbers()
+          {
+            if (readableOffset.getId() != id) {
+              final Object[] maybeArrays = arraySelector.getObjectVector();
+              for (int i = 0; i < arraySelector.getCurrentVectorSize(); i++) {
+                Object maybeArray = maybeArrays[i];
+                if (maybeArray instanceof Object[]) {
+                  Object[] anArray = (Object[]) maybeArray;
+                  if (elementNumber < anArray.length) {
+                    if (anArray[elementNumber] instanceof Number) {
+                      Number n = (Number) anArray[elementNumber];
+                      longs[i] = n.longValue();
+                      doubles[i] = n.doubleValue();
+                      floats[i] = n.floatValue();
+                      nulls[i] = false;
+                    } else {
+                      Double d = anArray[elementNumber] instanceof String
+                                 ? Doubles.tryParse((String) anArray[elementNumber])
+                                 : null;
+                      if (d != null) {
+                        longs[i] = d.longValue();
+                        doubles[i] = d;
+                        floats[i] = d.floatValue();
+                        nulls[i] = false;
+                      } else {
+                        nullElement(i);
+                      }
+                    }
+                  } else {
+                    nullElement(i);
+                  }
+                } else {
+                  // not an array?
+                  nullElement(i);
+                }
+              }
+              id = readableOffset.getId();
+            }
+          }
+
+          private void nullElement(int i)
+          {
+            longs[i] = 0L;
+            doubles[i] = 0L;
+            floats[i] = 0L;
+            nulls[i] = true;
+          }
+
+          @Override
+          public long[] getLongVector()
+          {
+            if (readableOffset.getId() != id) {
+              computeNumbers();
+            }
+            return longs;
+          }
 
+          @Override
+          public float[] getFloatVector()
+          {
+            if (readableOffset.getId() != id) {
+              computeNumbers();
+            }
+            return floats;
+          }
+
+          @Override
+          public double[] getDoubleVector()
+          {
+            if (readableOffset.getId() != id) {
+              computeNumbers();
+            }
+            return doubles;
+          }
+
+          @Nullable
+          @Override
+          public boolean[] getNullVector()
+          {
+            if (readableOffset.getId() != id) {
+              computeNumbers();
+            }
+            return nulls;
+          }
+
+          @Override
+          public int getMaxVectorSize()
+          {
+            return arraySelector.getMaxVectorSize();
+          }
+
+          @Override
+          public int getCurrentVectorSize()
+          {
+            return arraySelector.getCurrentVectorSize();
+          }
+        };
+      }
+    }
+    return NilVectorSelector.create(readableOffset);
+  }
 
 
   @Nullable
@@ -410,14 +754,16 @@ public abstract class CompressedNestedDataComplexColumn<TStringDictionary extend
     if (index < 0) {
       return null;
     }
-    return NestedLiteralTypeInfo.convertToSet(fieldInfo.getTypes(index).getByteValue());
+    return NestedFieldTypeInfo.convertToSet(fieldInfo.getTypes(index).getByteValue());
   }
 
   @Nullable
   @Override
   public ColumnHolder getColumnHolder(List<NestedPathPart> path)
   {
-    return getColumnHolder(getField(path));
+    final String field = getField(path);
+    final int fieldIndex = fields.indexOf(field);
+    return getColumnHolder(field, fieldIndex);
   }
 
   @Nullable
@@ -425,35 +771,44 @@ public abstract class CompressedNestedDataComplexColumn<TStringDictionary extend
   public ColumnIndexSupplier getColumnIndexSupplier(List<NestedPathPart> path)
   {
     final String field = getField(path);
-    if (fields.indexOf(field) < 0) {
-      return null;
+    int fieldIndex = fields.indexOf(field);
+    if (fieldIndex >= 0) {
+      return getColumnHolder(field, fieldIndex).getIndexSupplier();
+    }
+    if (!path.isEmpty() && path.get(path.size() - 1) instanceof NestedPathArrayElement) {
+      final String arrayField = getField(path.subList(0, path.size() - 1));
+      final int arrayFieldIndex = fields.indexOf(arrayField);
+      if (arrayFieldIndex >= 0) {
+        return NoIndexesColumnIndexSupplier.getInstance();
+      }
     }
-    return getColumnHolder(field).getIndexSupplier();
+    return null;
   }
 
   @Override
   public boolean isNumeric(List<NestedPathPart> path)
   {
     final String field = getField(path);
-    if (fields.indexOf(field) < 0) {
+    final int fieldIndex = fields.indexOf(field);
+    if (fieldIndex < 0) {
       return true;
     }
-    return getColumnHolder(field).getCapabilities().isNumeric();
+    return getColumnHolder(field, fieldIndex).getCapabilities().isNumeric();
   }
 
-  private ColumnHolder getColumnHolder(String field)
+  private ColumnHolder getColumnHolder(String field, int fieldIndex)
   {
-    return columns.computeIfAbsent(field, this::readNestedFieldColumn);
+    return columns.computeIfAbsent(fieldIndex, (f) -> readNestedFieldColumn(field, fieldIndex));
   }
 
-  private ColumnHolder readNestedFieldColumn(String field)
+  @Nullable
+  private ColumnHolder readNestedFieldColumn(String field, int fieldIndex)
   {
     try {
-      if (fields.indexOf(field) < 0) {
+      if (fieldIndex < 0) {
         return null;
       }
-      final int fieldIndex = fields.indexOf(field);
-      final NestedLiteralTypeInfo.TypeSet types = fieldInfo.getTypes(fieldIndex);
+      final NestedFieldTypeInfo.TypeSet types = fieldInfo.getTypes(fieldIndex);
       final String fieldFileName = getFieldFileName(metadata.getFileNameBase(), field, fieldIndex);
       final ByteBuffer dataBuffer = fileMapper.mapFile(fieldFileName);
       if (dataBuffer == null) {
@@ -491,10 +846,16 @@ public abstract class CompressedNestedDataComplexColumn<TStringDictionary extend
       int doublesLength = bb.getInt();
       dataBuffer.position(dataBuffer.position() + Integer.BYTES + Integer.BYTES);
       int pos = dataBuffer.position();
-      final Supplier<ColumnarLongs> longs = longsLength > 0 ? CompressedColumnarLongsSupplier.fromByteBuffer(dataBuffer, metadata.getByteOrder()) : () -> null;
+      final Supplier<ColumnarLongs> longs = longsLength > 0 ? CompressedColumnarLongsSupplier.fromByteBuffer(
+          dataBuffer,
+          metadata.getByteOrder()
+      ) : () -> null;
       dataBuffer.position(pos + longsLength);
       pos = dataBuffer.position();
-      final Supplier<ColumnarDoubles> doubles = doublesLength > 0 ? CompressedColumnarDoublesSuppliers.fromByteBuffer(dataBuffer, metadata.getByteOrder()) : () -> null;
+      final Supplier<ColumnarDoubles> doubles = doublesLength > 0 ? CompressedColumnarDoublesSuppliers.fromByteBuffer(
+          dataBuffer,
+          metadata.getByteOrder()
+      ) : () -> null;
       dataBuffer.position(pos + doublesLength);
       final WritableSupplier<ColumnarInts> ints;
       if (version == DictionaryEncodedColumnPartSerde.VERSION.COMPRESSED) {
@@ -503,17 +864,35 @@ public abstract class CompressedNestedDataComplexColumn<TStringDictionary extend
         ints = VSizeColumnarInts.readFromByteBuffer(dataBuffer);
       }
       ColumnType theType = types.getSingleType();
-      columnBuilder.setType(theType == null ? ValueType.STRING : theType.getType());
+      columnBuilder.setType(theType == null ? ColumnType.STRING : theType);
 
       GenericIndexed<ImmutableBitmap> rBitmaps = GenericIndexed.read(
           dataBuffer,
           metadata.getBitmapSerdeFactory().getObjectStrategy(),
           columnBuilder.getFileMapper()
       );
+      final Supplier<FixedIndexed<Integer>> arrayElementDictionarySupplier;
+      final GenericIndexed<ImmutableBitmap> arrayElementBitmaps;
+      if (dataBuffer.hasRemaining()) {
+        arrayElementDictionarySupplier = FixedIndexed.read(
+            dataBuffer,
+            NestedDataColumnSerializer.INT_TYPE_STRATEGY,
+            metadata.getByteOrder(),
+            Integer.BYTES
+        );
+        arrayElementBitmaps = GenericIndexed.read(
+            dataBuffer,
+            metadata.getBitmapSerdeFactory().getObjectStrategy(),
+            columnBuilder.getFileMapper()
+        );
+      } else {
+        arrayElementDictionarySupplier = null;
+        arrayElementBitmaps = null;
+      }
       final boolean hasNull = localDictionarySupplier.get().get(0) == 0;
       Supplier<DictionaryEncodedColumn<?>> columnSupplier = () -> {
         FixedIndexed<Integer> localDict = localDictionarySupplier.get();
-        return closer.register(new NestedFieldLiteralDictionaryEncodedColumn(
+        return closer.register(new NestedFieldDictionaryEncodedColumn(
             types,
             longs.get(),
             doubles.get(),
@@ -521,6 +900,7 @@ public abstract class CompressedNestedDataComplexColumn<TStringDictionary extend
             stringDictionarySupplier.get(),
             longDictionarySupplier.get(),
             doubleDictionarySupplier.get(),
+            arrayDictionarySupplier != null ? arrayDictionarySupplier.get() : null,
             localDict,
             hasNull
             ? rBitmaps.get(0)
@@ -531,14 +911,16 @@ public abstract class CompressedNestedDataComplexColumn<TStringDictionary extend
                    .setHasNulls(hasNull)
                    .setDictionaryEncodedColumnSupplier(columnSupplier);
       columnBuilder.setIndexSupplier(
-          new NestedFieldLiteralColumnIndexSupplier(
+          new NestedFieldColumnIndexSupplier(
               types,
               metadata.getBitmapSerdeFactory().getBitmapFactory(),
               rBitmaps,
               localDictionarySupplier,
               stringDictionarySupplier,
               longDictionarySupplier,
-              doubleDictionarySupplier
+              doubleDictionarySupplier,
+              arrayElementDictionarySupplier,
+              arrayElementBitmaps
           ),
           true,
           false
diff --git a/processing/src/main/java/org/apache/druid/segment/nested/GlobalDictionaryEncodedFieldColumnWriter.java b/processing/src/main/java/org/apache/druid/segment/nested/GlobalDictionaryEncodedFieldColumnWriter.java
index def741a107..0c308ff6b4 100644
--- a/processing/src/main/java/org/apache/druid/segment/nested/GlobalDictionaryEncodedFieldColumnWriter.java
+++ b/processing/src/main/java/org/apache/druid/segment/nested/GlobalDictionaryEncodedFieldColumnWriter.java
@@ -22,6 +22,8 @@ package org.apache.druid.segment.nested;
 import com.google.common.base.Preconditions;
 import com.google.common.primitives.Ints;
 import it.unimi.dsi.fastutil.ints.Int2IntOpenHashMap;
+import it.unimi.dsi.fastutil.ints.Int2ObjectMap;
+import it.unimi.dsi.fastutil.ints.Int2ObjectRBTreeMap;
 import it.unimi.dsi.fastutil.ints.IntArrays;
 import it.unimi.dsi.fastutil.ints.IntIterator;
 import org.apache.druid.collections.bitmap.ImmutableBitmap;
@@ -70,6 +72,8 @@ public abstract class GlobalDictionaryEncodedFieldColumnWriter<T>
   protected final GlobalDictionaryIdLookup globalDictionaryIdLookup;
   protected final LocalDimensionDictionary localDictionary = new LocalDimensionDictionary();
 
+  protected final Int2ObjectRBTreeMap<MutableBitmap> arrayElements = new Int2ObjectRBTreeMap<>();
+
   protected FixedIndexedIntWriter intermediateValueWriter;
   // maybe someday we allow no bitmap indexes or multi-value columns
   protected int flags = DictionaryEncodedColumnPartSerde.NO_FLAGS;
@@ -96,7 +100,7 @@ public abstract class GlobalDictionaryEncodedFieldColumnWriter<T>
   /**
    * Perform any value conversion needed before storing the value in the
    */
-  T processValue(Object value)
+  T processValue(int row, Object value)
   {
     return (T) value;
   }
@@ -133,7 +137,7 @@ public abstract class GlobalDictionaryEncodedFieldColumnWriter<T>
     if (row > cursorPosition) {
       fillNull(row);
     }
-    final T value = processValue(val);
+    final T value = processValue(row, val);
     final int localId;
     // null is always 0
     if (value == null) {
@@ -150,7 +154,7 @@ public abstract class GlobalDictionaryEncodedFieldColumnWriter<T>
 
   private void fillNull(int row) throws IOException
   {
-    final T value = processValue(null);
+    final T value = processValue(row, null);
     final int localId = localDictionary.add(0);
     while (cursorPosition < row) {
       intermediateValueWriter.write(localId);
@@ -184,6 +188,8 @@ public abstract class GlobalDictionaryEncodedFieldColumnWriter<T>
     final SegmentWriteOutMedium tmpWriteoutMedium = segmentWriteOutMedium.makeChildWriteOutMedium();
     final FixedIndexedIntWriter sortedDictionaryWriter = new FixedIndexedIntWriter(tmpWriteoutMedium, true);
     sortedDictionaryWriter.open();
+    final FixedIndexedIntWriter arrayElementDictionaryWriter = new FixedIndexedIntWriter(tmpWriteoutMedium, true);
+    arrayElementDictionaryWriter.open();
     GenericIndexedWriter<ImmutableBitmap> bitmapIndexWriter = new GenericIndexedWriter<>(
         tmpWriteoutMedium,
         columnName,
@@ -191,6 +197,14 @@ public abstract class GlobalDictionaryEncodedFieldColumnWriter<T>
     );
     bitmapIndexWriter.open();
     bitmapIndexWriter.setObjectsNotSorted();
+    GenericIndexedWriter<ImmutableBitmap> arrayElementIndexWriter = new GenericIndexedWriter<>(
+        tmpWriteoutMedium,
+        columnName,
+        indexSpec.getBitmapSerdeFactory().getObjectStrategy()
+    );
+    arrayElementIndexWriter.open();
+    arrayElementIndexWriter.setObjectsNotSorted();
+
     final Int2IntOpenHashMap globalToUnsorted = localDictionary.getGlobalIdToLocalId();
     final int[] unsortedToGlobal = new int[localDictionary.size()];
     for (int key : globalToUnsorted.keySet()) {
@@ -210,6 +224,13 @@ public abstract class GlobalDictionaryEncodedFieldColumnWriter<T>
       bitmaps[index] = indexSpec.getBitmapSerdeFactory().getBitmapFactory().makeEmptyMutableBitmap();
     }
 
+    for (Int2ObjectMap.Entry<MutableBitmap> arrayElement : arrayElements.int2ObjectEntrySet()) {
+      arrayElementDictionaryWriter.write(arrayElement.getIntKey());
+      arrayElementIndexWriter.write(
+          indexSpec.getBitmapSerdeFactory().getBitmapFactory().makeImmutableBitmap(arrayElement.getValue())
+      );
+    }
+
     openColumnSerializer(tmpWriteoutMedium, sortedGlobal[sortedGlobal.length - 1]);
     final IntIterator rows = intermediateValueWriter.getIterator();
     int rowCount = 0;
@@ -233,9 +254,16 @@ public abstract class GlobalDictionaryEncodedFieldColumnWriter<T>
       @Override
       public long getSerializedSize() throws IOException
       {
+        final long arraySize;
+        if (arrayElements.size() > 0) {
+          arraySize = arrayElementDictionaryWriter.getSerializedSize() + arrayElementIndexWriter.getSerializedSize();
+        } else {
+          arraySize = 0;
+        }
         return 1 + Integer.BYTES +
                sortedDictionaryWriter.getSerializedSize() +
                bitmapIndexWriter.getSerializedSize() +
+               arraySize +
                getSerializedColumnSize();
       }
 
@@ -247,6 +275,10 @@ public abstract class GlobalDictionaryEncodedFieldColumnWriter<T>
         sortedDictionaryWriter.writeTo(channel, smoosher);
         writeColumnTo(channel, smoosher);
         bitmapIndexWriter.writeTo(channel, smoosher);
+        if (arrayElements.size() > 0) {
+          arrayElementDictionaryWriter.writeTo(channel, smoosher);
+          arrayElementIndexWriter.writeTo(channel, smoosher);
+        }
       }
     };
     final String fieldFileName = NestedDataColumnSerializer.getInternalFileName(columnName, fieldName);
diff --git a/processing/src/main/java/org/apache/druid/segment/nested/GlobalDictionaryIdLookup.java b/processing/src/main/java/org/apache/druid/segment/nested/GlobalDictionaryIdLookup.java
index 821f3ecc26..2675b0392c 100644
--- a/processing/src/main/java/org/apache/druid/segment/nested/GlobalDictionaryIdLookup.java
+++ b/processing/src/main/java/org/apache/druid/segment/nested/GlobalDictionaryIdLookup.java
@@ -24,8 +24,10 @@ import it.unimi.dsi.fastutil.doubles.Double2IntLinkedOpenHashMap;
 import it.unimi.dsi.fastutil.doubles.Double2IntMap;
 import it.unimi.dsi.fastutil.longs.Long2IntLinkedOpenHashMap;
 import it.unimi.dsi.fastutil.longs.Long2IntMap;
+import it.unimi.dsi.fastutil.objects.Object2IntAVLTreeMap;
 import it.unimi.dsi.fastutil.objects.Object2IntLinkedOpenHashMap;
 import it.unimi.dsi.fastutil.objects.Object2IntMap;
+import org.apache.druid.segment.data.FrontCodedIntArrayIndexedWriter;
 
 import javax.annotation.Nullable;
 
@@ -41,6 +43,8 @@ public class GlobalDictionaryIdLookup
 
   private final Double2IntMap doubleLookup;
 
+  private final Object2IntMap<int[]> arrayLookup;
+
   private int dictionarySize;
 
   public GlobalDictionaryIdLookup()
@@ -51,6 +55,8 @@ public class GlobalDictionaryIdLookup
     longLookup.defaultReturnValue(-1);
     this.doubleLookup = new Double2IntLinkedOpenHashMap();
     doubleLookup.defaultReturnValue(-1);
+    this.arrayLookup = new Object2IntAVLTreeMap<>(FrontCodedIntArrayIndexedWriter.ARRAY_COMPARATOR);
+    this.arrayLookup.defaultReturnValue(-1);
   }
 
   public void addString(@Nullable String value)
@@ -99,4 +105,18 @@ public class GlobalDictionaryIdLookup
     }
     return doubleLookup.get(value.doubleValue());
   }
+
+  public void addArray(int[] value)
+  {
+    int id = dictionarySize++;
+    arrayLookup.put(value, id);
+  }
+
+  public int lookupArray(@Nullable int[] value)
+  {
+    if (value == null) {
+      return 0;
+    }
+    return arrayLookup.getInt(value);
+  }
 }
diff --git a/processing/src/main/java/org/apache/druid/segment/nested/GlobalDictionarySortedCollector.java b/processing/src/main/java/org/apache/druid/segment/nested/GlobalDictionarySortedCollector.java
index 7cb775cc54..70040d7b52 100644
--- a/processing/src/main/java/org/apache/druid/segment/nested/GlobalDictionarySortedCollector.java
+++ b/processing/src/main/java/org/apache/druid/segment/nested/GlobalDictionarySortedCollector.java
@@ -32,16 +32,22 @@ public class GlobalDictionarySortedCollector
   private final Indexed<String> sortedStrings;
   private final Indexed<Long> sortedLongs;
   private final Indexed<Double> sortedDoubles;
+  private final Iterable<Object[]> sortedArrays;
+  private final int arrayCount;
 
   public GlobalDictionarySortedCollector(
       Indexed<String> sortedStrings,
       Indexed<Long> sortedLongs,
-      Indexed<Double> sortedDoubles
+      Indexed<Double> sortedDoubles,
+      Iterable<Object[]> sortedArrays,
+      int arrayCount
   )
   {
     this.sortedStrings = sortedStrings;
     this.sortedLongs = sortedLongs;
     this.sortedDoubles = sortedDoubles;
+    this.sortedArrays = sortedArrays;
+    this.arrayCount = arrayCount;
   }
 
   public Indexed<String> getSortedStrings()
@@ -59,8 +65,44 @@ public class GlobalDictionarySortedCollector
     return sortedDoubles;
   }
 
-  public int size()
+  public Iterable<Object[]> getSortedArrays()
   {
-    return sortedStrings.size() + sortedLongs.size() + sortedDoubles.size();
+    return sortedArrays;
+  }
+
+  public int getStringCardinality()
+  {
+    return sortedStrings.size();
+  }
+
+  public int getLongCardinality()
+  {
+    return sortedLongs.size();
+  }
+
+  public int getDoubleCardinality()
+  {
+    return sortedDoubles.size();
+  }
+
+  public int getArrayCardinality()
+  {
+    return arrayCount;
+  }
+
+  public boolean allNull()
+  {
+    for (String s : sortedStrings) {
+      if (s != null) {
+        return false;
+      }
+    }
+    if (sortedLongs.size() > 0) {
+      return false;
+    }
+    if (sortedDoubles.size() > 0) {
+      return false;
+    }
+    return arrayCount == 0;
   }
 }
diff --git a/processing/src/main/java/org/apache/druid/segment/nested/GlobalDimensionDictionary.java b/processing/src/main/java/org/apache/druid/segment/nested/GlobalDimensionDictionary.java
index 4f22577b41..48b06d4e45 100644
--- a/processing/src/main/java/org/apache/druid/segment/nested/GlobalDimensionDictionary.java
+++ b/processing/src/main/java/org/apache/druid/segment/nested/GlobalDimensionDictionary.java
@@ -19,18 +19,23 @@
 
 package org.apache.druid.segment.nested;
 
+import com.google.common.base.Preconditions;
 import org.apache.druid.query.monomorphicprocessing.RuntimeShapeInspector;
 import org.apache.druid.segment.ComparatorDimensionDictionary;
 import org.apache.druid.segment.ComparatorSortedDimensionDictionary;
 import org.apache.druid.segment.DimensionDictionary;
 import org.apache.druid.segment.NestedDataColumnIndexer;
 import org.apache.druid.segment.column.ColumnType;
+import org.apache.druid.segment.data.FrontCodedIntArrayIndexedWriter;
 import org.apache.druid.segment.data.GenericIndexed;
 import org.apache.druid.segment.data.Indexed;
 import org.apache.druid.segment.data.IndexedIterable;
 
 import javax.annotation.Nullable;
+import java.util.Comparator;
 import java.util.Iterator;
+import java.util.Set;
+import java.util.TreeSet;
 
 /**
  * Used by {@link NestedDataColumnIndexer} to build the global value dictionary, which can be converted into a
@@ -42,45 +47,107 @@ public class GlobalDimensionDictionary
   private final ComparatorDimensionDictionary<String> stringDictionary;
   private final ComparatorDimensionDictionary<Long> longDictionary;
   private final ComparatorDimensionDictionary<Double> doubleDictionary;
+  private final Set<Object[]> stringArrays;
+  private final Set<Object[]> longArrays;
+  private final Set<Object[]> doubleArrays;
+
+  private int arrayBytesSizeEstimate;
 
   public GlobalDimensionDictionary()
   {
-    this.stringDictionary = new ComparatorDimensionDictionary<String>(GenericIndexed.STRING_STRATEGY) {
+    this.stringDictionary = new ComparatorDimensionDictionary<String>(GenericIndexed.STRING_STRATEGY)
+    {
       @Override
       public long estimateSizeOfValue(String value)
       {
         return StructuredDataProcessor.estimateStringSize(value);
       }
     };
-    this.longDictionary = new ComparatorDimensionDictionary<Long>(ColumnType.LONG.getNullableStrategy()) {
+    this.longDictionary = new ComparatorDimensionDictionary<Long>(ColumnType.LONG.getNullableStrategy())
+    {
       @Override
       public long estimateSizeOfValue(Long value)
       {
         return StructuredDataProcessor.getLongObjectEstimateSize();
       }
     };
-    this.doubleDictionary = new ComparatorDimensionDictionary<Double>(ColumnType.DOUBLE.getNullableStrategy()) {
+    this.doubleDictionary = new ComparatorDimensionDictionary<Double>(ColumnType.DOUBLE.getNullableStrategy())
+    {
       @Override
       public long estimateSizeOfValue(Double value)
       {
         return StructuredDataProcessor.getDoubleObjectEstimateSize();
       }
     };
+    this.stringArrays = new TreeSet<>(ColumnType.STRING_ARRAY.getNullableStrategy());
+    this.longArrays = new TreeSet<>(ColumnType.LONG_ARRAY.getNullableStrategy());
+    this.doubleArrays = new TreeSet<>(ColumnType.DOUBLE_ARRAY.getNullableStrategy());
   }
 
-  public void addLongValue(@Nullable Long value)
+  public int addLongValue(@Nullable Long value)
   {
     longDictionary.add(value);
+    return StructuredDataProcessor.getLongObjectEstimateSize();
   }
 
-  public void addDoubleValue(@Nullable Double value)
+  public int addDoubleValue(@Nullable Double value)
   {
     doubleDictionary.add(value);
+    return StructuredDataProcessor.getDoubleObjectEstimateSize();
   }
 
-  public void addStringValue(@Nullable String value)
+  public int addStringValue(@Nullable String value)
   {
     stringDictionary.add(value);
+    return StructuredDataProcessor.estimateStringSize(value);
+  }
+
+  public int addStringArray(@Nullable Object[] value)
+  {
+    if (value == null) {
+      return 0;
+    }
+    stringArrays.add(value);
+    int sizeEstimate = 0;
+    for (Object o : value) {
+      if (o != null) {
+        sizeEstimate += addStringValue((String) o);
+      }
+    }
+    arrayBytesSizeEstimate += sizeEstimate;
+    return sizeEstimate;
+  }
+
+  public int addLongArray(@Nullable Object[] value)
+  {
+    if (value == null) {
+      return 0;
+    }
+    longArrays.add(value);
+    int sizeEstimate = 0;
+    for (Object o : value) {
+      if (o != null) {
+        sizeEstimate += addLongValue((Long) o);
+      }
+    }
+    arrayBytesSizeEstimate += sizeEstimate;
+    return sizeEstimate;
+  }
+
+  public int addDoubleArray(@Nullable Object[] value)
+  {
+    if (value == null) {
+      return 0;
+    }
+    doubleArrays.add(value);
+    int sizeEstimate = 0;
+    for (Object o : value) {
+      if (o != null) {
+        sizeEstimate += addDoubleValue((Double) o);
+      }
+    }
+    arrayBytesSizeEstimate += sizeEstimate;
+    return sizeEstimate;
   }
 
   public GlobalDictionarySortedCollector getSortedCollector()
@@ -184,7 +251,9 @@ public class GlobalDimensionDictionary
       public int indexOf(Double value)
       {
         int id = doubleDictionary.getId(value);
-        return id < 0 ? DimensionDictionary.ABSENT_VALUE_ID : sortedDoubleDimensionDictionary.getSortedIdFromUnsortedId(id);
+        return id < 0
+               ? DimensionDictionary.ABSENT_VALUE_ID
+               : sortedDoubleDimensionDictionary.getSortedIdFromUnsortedId(id);
       }
 
       @Override
@@ -199,16 +268,70 @@ public class GlobalDimensionDictionary
         // nothing to inspect
       }
     };
-    return new GlobalDictionarySortedCollector(strings, longs, doubles);
+
+    // offset by 1 because nulls are ignored by the indexer, but always global id 0
+    final int adjustLongs = 1 + strings.size();
+    final int adjustDoubles = adjustLongs + longs.size();
+    TreeSet<Object[]> sortedArrays = new TreeSet<>(new Comparator<Object[]>()
+    {
+      @Override
+      public int compare(Object[] o1, Object[] o2)
+      {
+        return FrontCodedIntArrayIndexedWriter.ARRAY_COMPARATOR.compare(convertArray(o1), convertArray(o2));
+      }
+
+      @Nullable
+      private int[] convertArray(Object[] array)
+      {
+        if (array == null) {
+          return null;
+        }
+        final int[] globalIds = new int[array.length];
+        for (int i = 0; i < array.length; i++) {
+          if (array[i] == null) {
+            globalIds[i] = 0;
+          } else if (array[i] instanceof String) {
+            // offset by 1 because nulls are ignored by the indexer, but always global id 0
+            globalIds[i] = 1 + strings.indexOf((String) array[i]);
+          } else if (array[i] instanceof Long) {
+            globalIds[i] = longs.indexOf((Long) array[i]) + adjustLongs;
+          } else if (array[i] instanceof Double) {
+            globalIds[i] = doubles.indexOf((Double) array[i]) + adjustDoubles;
+          } else {
+            globalIds[i] = -1;
+          }
+          Preconditions.checkArgument(
+              globalIds[i] >= 0,
+              "unknown global id [%s] for value [%s]",
+              globalIds[i],
+              array[i]
+          );
+        }
+        return globalIds;
+      }
+    });
+    sortedArrays.addAll(stringArrays);
+    sortedArrays.addAll(longArrays);
+    sortedArrays.addAll(doubleArrays);
+
+    return new GlobalDictionarySortedCollector(strings, longs, doubles, sortedArrays, sortedArrays.size());
   }
 
   public long sizeInBytes()
   {
-    return stringDictionary.sizeInBytes() + longDictionary.sizeInBytes() + doubleDictionary.sizeInBytes();
+    return stringDictionary.sizeInBytes()
+           + longDictionary.sizeInBytes()
+           + doubleDictionary.sizeInBytes()
+           + arrayBytesSizeEstimate;
   }
 
   public int getCardinality()
   {
-    return stringDictionary.size() + longDictionary.size() + doubleDictionary.size();
+    return stringDictionary.size()
+           + longDictionary.size()
+           + doubleDictionary.size()
+           + stringArrays.size()
+           + longArrays.size()
+           + doubleArrays.size();
   }
 }
diff --git a/processing/src/main/java/org/apache/druid/segment/nested/LocalDimensionDictionary.java b/processing/src/main/java/org/apache/druid/segment/nested/LocalDimensionDictionary.java
index 97b447dadd..21a5467818 100644
--- a/processing/src/main/java/org/apache/druid/segment/nested/LocalDimensionDictionary.java
+++ b/processing/src/main/java/org/apache/druid/segment/nested/LocalDimensionDictionary.java
@@ -25,6 +25,8 @@ public class LocalDimensionDictionary
 {
   private final Int2IntOpenHashMap globalIdToLocalId = new Int2IntOpenHashMap();
 
+  private int nextLocalId = 0;
+
   public LocalDimensionDictionary()
   {
     this.globalIdToLocalId.defaultReturnValue(-1);
@@ -35,8 +37,6 @@ public class LocalDimensionDictionary
     return globalIdToLocalId;
   }
   
-  private int nextLocalId = 0;
-  
   public int add(int originalValue)
   {
     int prev = globalIdToLocalId.get(originalValue);
diff --git a/processing/src/main/java/org/apache/druid/segment/nested/NestedDataColumnSerializer.java b/processing/src/main/java/org/apache/druid/segment/nested/NestedDataColumnSerializer.java
index f73b72209c..020dc10ad0 100644
--- a/processing/src/main/java/org/apache/druid/segment/nested/NestedDataColumnSerializer.java
+++ b/processing/src/main/java/org/apache/druid/segment/nested/NestedDataColumnSerializer.java
@@ -24,6 +24,8 @@ import com.google.common.collect.Maps;
 import org.apache.druid.collections.bitmap.ImmutableBitmap;
 import org.apache.druid.collections.bitmap.MutableBitmap;
 import org.apache.druid.common.config.NullHandling;
+import org.apache.druid.java.util.common.ISE;
+import org.apache.druid.java.util.common.RE;
 import org.apache.druid.java.util.common.StringUtils;
 import org.apache.druid.java.util.common.io.Closer;
 import org.apache.druid.java.util.common.io.smoosh.FileSmoosher;
@@ -46,17 +48,20 @@ import org.apache.druid.segment.data.CompressedVariableSizedBlobColumnSerializer
 import org.apache.druid.segment.data.CompressionStrategy;
 import org.apache.druid.segment.data.DictionaryWriter;
 import org.apache.druid.segment.data.FixedIndexedWriter;
+import org.apache.druid.segment.data.FrontCodedIntArrayIndexedWriter;
 import org.apache.druid.segment.data.GenericIndexed;
 import org.apache.druid.segment.data.GenericIndexedWriter;
 import org.apache.druid.segment.serde.Serializer;
 import org.apache.druid.segment.writeout.SegmentWriteOutMedium;
 
+import javax.annotation.Nullable;
 import java.io.ByteArrayOutputStream;
 import java.io.IOException;
 import java.nio.ByteBuffer;
 import java.nio.ByteOrder;
 import java.nio.channels.WritableByteChannel;
 import java.util.ArrayList;
+import java.util.List;
 import java.util.Map;
 import java.util.SortedMap;
 
@@ -67,6 +72,7 @@ public class NestedDataColumnSerializer implements GenericColumnSerializer<Struc
   public static final String STRING_DICTIONARY_FILE_NAME = "__stringDictionary";
   public static final String LONG_DICTIONARY_FILE_NAME = "__longDictionary";
   public static final String DOUBLE_DICTIONARY_FILE_NAME = "__doubleDictionary";
+  public static final String ARRAY_DICTIONARY_FILE_NAME = "__arrayDictionary";
   public static final String RAW_FILE_NAME = "__raw";
   public static final String NULL_BITMAP_FILE_NAME = "__nullIndex";
 
@@ -81,14 +87,14 @@ public class NestedDataColumnSerializer implements GenericColumnSerializer<Struc
   private final StructuredDataProcessor fieldProcessor = new StructuredDataProcessor()
   {
     @Override
-    public StructuredDataProcessor.ProcessedLiteral<?> processLiteralField(ArrayList<NestedPathPart> fieldPath, Object fieldValue)
+    public ProcessedValue<?> processField(ArrayList<NestedPathPart> fieldPath, @Nullable Object fieldValue)
     {
       final GlobalDictionaryEncodedFieldColumnWriter<?> writer = fieldWriters.get(
           NestedPathFinder.toNormalizedJsonPath(fieldPath)
       );
       if (writer != null) {
         try {
-          ExprEval<?> eval = ExprEval.bestEffortOf(fieldValue);
+          final ExprEval<?> eval = ExprEval.bestEffortOf(fieldValue);
           if (eval.type().isPrimitive() || (eval.type().isArray() && eval.type().getElementType().isPrimitive())) {
             writer.addValue(rowCount, eval.value());
           } else {
@@ -96,24 +102,51 @@ public class NestedDataColumnSerializer implements GenericColumnSerializer<Struc
             writer.addValue(rowCount, eval.asString());
           }
           // serializer doesn't use size estimate
-          return StructuredDataProcessor.ProcessedLiteral.NULL_LITERAL;
+          return ProcessedValue.NULL_LITERAL;
         }
         catch (IOException e) {
-          throw new RuntimeException(":(");
+          throw new RE(e, "Failed to write field [%s], unhandled value", fieldPath);
         }
       }
-      return StructuredDataProcessor.ProcessedLiteral.NULL_LITERAL;
+      return ProcessedValue.NULL_LITERAL;
+    }
+
+    @Nullable
+    @Override
+    public ProcessedValue<?> processArrayField(
+        ArrayList<NestedPathPart> fieldPath,
+        @Nullable List<?> array
+    )
+    {
+      final ExprEval<?> eval = ExprEval.bestEffortArray(array);
+      if (eval.type().isArray() && eval.type().getElementType().isPrimitive()) {
+        final GlobalDictionaryEncodedFieldColumnWriter<?> writer = fieldWriters.get(
+            NestedPathFinder.toNormalizedJsonPath(fieldPath)
+        );
+        if (writer != null) {
+          try {
+            writer.addValue(rowCount, eval.value());
+            // serializer doesn't use size estimate
+            return ProcessedValue.NULL_LITERAL;
+          }
+          catch (IOException e) {
+            throw new RE(e, "Failed to write field [%s] value [%s]", fieldPath, array);
+          }
+        }
+      }
+      return null;
     }
   };
 
   private byte[] metadataBytes;
   private GlobalDictionaryIdLookup globalDictionaryIdLookup;
-  private SortedMap<String, NestedLiteralTypeInfo.MutableTypeSet> fields;
+  private SortedMap<String, NestedFieldTypeInfo.MutableTypeSet> fields;
   private GenericIndexedWriter<String> fieldsWriter;
-  private NestedLiteralTypeInfo.Writer fieldsInfoWriter;
+  private NestedFieldTypeInfo.Writer fieldsInfoWriter;
   private DictionaryWriter<String> dictionaryWriter;
   private FixedIndexedWriter<Long> longDictionaryWriter;
   private FixedIndexedWriter<Double> doubleDictionaryWriter;
+  private FrontCodedIntArrayIndexedWriter arrayDictionaryWriter;
   private CompressedVariableSizedBlobColumnSerializer rawWriter;
   private ByteBufferWriter<ImmutableBitmap> nullBitmapWriter;
   private MutableBitmap nullRowsBitmap;
@@ -121,6 +154,8 @@ public class NestedDataColumnSerializer implements GenericColumnSerializer<Struc
   private int rowCount = 0;
   private boolean closedForWrite = false;
 
+  private boolean dictionarySerialized = false;
+
   public NestedDataColumnSerializer(
       String name,
       IndexSpec indexSpec,
@@ -136,13 +171,18 @@ public class NestedDataColumnSerializer implements GenericColumnSerializer<Struc
     this.globalDictionaryIdLookup = new GlobalDictionaryIdLookup();
   }
 
+  public GlobalDictionaryIdLookup getGlobalLookup()
+  {
+    return globalDictionaryIdLookup;
+  }
+
   @Override
   public void open() throws IOException
   {
     fieldsWriter = new GenericIndexedWriter<>(segmentWriteOutMedium, name, GenericIndexed.STRING_STRATEGY);
     fieldsWriter.open();
 
-    fieldsInfoWriter = new NestedLiteralTypeInfo.Writer(segmentWriteOutMedium);
+    fieldsInfoWriter = new NestedFieldTypeInfo.Writer(segmentWriteOutMedium);
     fieldsInfoWriter.open();
 
     dictionaryWriter = StringEncodingStrategies.getStringDictionaryWriter(
@@ -170,6 +210,13 @@ public class NestedDataColumnSerializer implements GenericColumnSerializer<Struc
     );
     doubleDictionaryWriter.open();
 
+    arrayDictionaryWriter = new FrontCodedIntArrayIndexedWriter(
+        segmentWriteOutMedium,
+        ByteOrder.nativeOrder(),
+        4
+    );
+    arrayDictionaryWriter.open();
+
     rawWriter = new CompressedVariableSizedBlobColumnSerializer(
         getInternalFileName(name, RAW_FILE_NAME),
         segmentWriteOutMedium,
@@ -186,12 +233,12 @@ public class NestedDataColumnSerializer implements GenericColumnSerializer<Struc
     nullRowsBitmap = indexSpec.getBitmapSerdeFactory().getBitmapFactory().makeEmptyMutableBitmap();
   }
 
-  public void serializeFields(SortedMap<String, NestedLiteralTypeInfo.MutableTypeSet> fields) throws IOException
+  public void serializeFields(SortedMap<String, NestedFieldTypeInfo.MutableTypeSet> fields) throws IOException
   {
     this.fields = fields;
     this.fieldWriters = Maps.newHashMapWithExpectedSize(fields.size());
     int ctr = 0;
-    for (Map.Entry<String, NestedLiteralTypeInfo.MutableTypeSet> field : fields.entrySet()) {
+    for (Map.Entry<String, NestedFieldTypeInfo.MutableTypeSet> field : fields.entrySet()) {
       final String fieldName = field.getKey();
       final String fieldFileName = NESTED_FIELD_PREFIX + ctr++;
       fieldsWriter.write(fieldName);
@@ -215,7 +262,7 @@ public class NestedDataColumnSerializer implements GenericColumnSerializer<Struc
               indexSpec,
               globalDictionaryIdLookup
           );
-        } else {
+        } else if (Types.is(type, ValueType.DOUBLE)) {
           writer = new DoubleFieldColumnWriter(
               name,
               fieldFileName,
@@ -223,9 +270,19 @@ public class NestedDataColumnSerializer implements GenericColumnSerializer<Struc
               indexSpec,
               globalDictionaryIdLookup
           );
+        } else if (Types.is(type, ValueType.ARRAY)) {
+          writer = new ArrayFieldColumnWriter(
+              name,
+              fieldFileName,
+              segmentWriteOutMedium,
+              indexSpec,
+              globalDictionaryIdLookup
+          );
+        } else {
+          throw new ISE("Invalid field type [%s], how did this happen?", type);
         }
       } else {
-        writer = new VariantLiteralFieldColumnWriter(
+        writer = new VariantFieldColumnWriter(
             name,
             fieldFileName,
             segmentWriteOutMedium,
@@ -238,11 +295,21 @@ public class NestedDataColumnSerializer implements GenericColumnSerializer<Struc
     }
   }
 
-  public void serializeStringDictionary(Iterable<String> dictionaryValues) throws IOException
+  public void serializeDictionaries(
+      Iterable<String> strings,
+      Iterable<Long> longs,
+      Iterable<Double> doubles,
+      Iterable<int[]> arrays
+  ) throws IOException
   {
+    if (dictionarySerialized) {
+      throw new ISE("String dictionary already serialized for column [%s], cannot serialize again", name);
+    }
+
+    // null is always 0
     dictionaryWriter.write(null);
     globalDictionaryIdLookup.addString(null);
-    for (String value : dictionaryValues) {
+    for (String value : strings) {
       value = NullHandling.emptyToNullIfNeeded(value);
       if (value == null) {
         continue;
@@ -251,33 +318,40 @@ public class NestedDataColumnSerializer implements GenericColumnSerializer<Struc
       dictionaryWriter.write(value);
       globalDictionaryIdLookup.addString(value);
     }
-  }
+    dictionarySerialized = true;
 
-  public void serializeLongDictionary(Iterable<Long> dictionaryValues) throws IOException
-  {
-    for (Long value : dictionaryValues) {
+    for (Long value : longs) {
       if (value == null) {
         continue;
       }
       longDictionaryWriter.write(value);
       globalDictionaryIdLookup.addLong(value);
     }
-  }
 
-  public void serializeDoubleDictionary(Iterable<Double> dictionaryValues) throws IOException
-  {
-    for (Double value : dictionaryValues) {
+    for (Double value : doubles) {
       if (value == null) {
         continue;
       }
       doubleDictionaryWriter.write(value);
       globalDictionaryIdLookup.addDouble(value);
     }
+
+    for (int[] value : arrays) {
+      if (value == null) {
+        continue;
+      }
+      arrayDictionaryWriter.write(value);
+      globalDictionaryIdLookup.addArray(value);
+    }
+    dictionarySerialized = true;
   }
 
   @Override
   public void serialize(ColumnValueSelector<? extends StructuredData> selector) throws IOException
   {
+    if (!dictionarySerialized) {
+      throw new ISE("Must serialize value dictionaries before serializing values for column [%s]", name);
+    }
     StructuredData data = StructuredData.wrap(selector.getObject());
     if (data == null) {
       nullRowsBitmap.add(rowCount);
@@ -330,8 +404,8 @@ public class NestedDataColumnSerializer implements GenericColumnSerializer<Struc
   {
     Preconditions.checkState(closedForWrite, "Not closed yet!");
     Preconditions.checkArgument(dictionaryWriter.isSorted(), "Dictionary not sorted?!?");
-    // version 4
-    channel.write(ByteBuffer.wrap(new byte[]{0x04}));
+    // version 5
+    channel.write(ByteBuffer.wrap(new byte[]{0x05}));
     channel.write(ByteBuffer.wrap(metadataBytes));
     fieldsWriter.writeTo(channel, smoosher);
     fieldsInfoWriter.writeTo(channel, smoosher);
@@ -340,6 +414,7 @@ public class NestedDataColumnSerializer implements GenericColumnSerializer<Struc
     writeInternal(smoosher, dictionaryWriter, STRING_DICTIONARY_FILE_NAME);
     writeInternal(smoosher, longDictionaryWriter, LONG_DICTIONARY_FILE_NAME);
     writeInternal(smoosher, doubleDictionaryWriter, DOUBLE_DICTIONARY_FILE_NAME);
+    writeInternal(smoosher, arrayDictionaryWriter, ARRAY_DICTIONARY_FILE_NAME);
     writeInternal(smoosher, rawWriter, RAW_FILE_NAME);
     if (!nullRowsBitmap.isEmpty()) {
       writeInternal(smoosher, nullBitmapWriter, NULL_BITMAP_FILE_NAME);
@@ -355,7 +430,7 @@ public class NestedDataColumnSerializer implements GenericColumnSerializer<Struc
       channel.close();
     }
 
-    for (Map.Entry<String, NestedLiteralTypeInfo.MutableTypeSet> field : fields.entrySet()) {
+    for (Map.Entry<String, NestedFieldTypeInfo.MutableTypeSet> field : fields.entrySet()) {
       // remove writer so that it can be collected when we are done with it
       GlobalDictionaryEncodedFieldColumnWriter<?> writer = fieldWriters.remove(field.getKey());
       writer.writeTo(rowCount, smoosher);
diff --git a/processing/src/main/java/org/apache/druid/segment/nested/NestedDataColumnSupplier.java b/processing/src/main/java/org/apache/druid/segment/nested/NestedDataColumnSupplier.java
index 507a531f4b..15dc2d888f 100644
--- a/processing/src/main/java/org/apache/druid/segment/nested/NestedDataColumnSupplier.java
+++ b/processing/src/main/java/org/apache/druid/segment/nested/NestedDataColumnSupplier.java
@@ -36,6 +36,7 @@ import org.apache.druid.segment.data.CompressedVariableSizedBlobColumnSupplier;
 import org.apache.druid.segment.data.EncodedStringDictionaryWriter;
 import org.apache.druid.segment.data.FixedIndexed;
 import org.apache.druid.segment.data.FrontCodedIndexed;
+import org.apache.druid.segment.data.FrontCodedIntArrayIndexed;
 import org.apache.druid.segment.data.GenericIndexed;
 
 import javax.annotation.Nullable;
@@ -72,18 +73,20 @@ public class NestedDataColumnSupplier implements Supplier<ComplexColumn>
   {
     final byte version = bb.get();
 
-    if (version == 0x03 || version == 0x04) {
+    if (version == 0x03 || version == 0x04 || version == 0x05) {
       try {
         final SmooshedFileMapper mapper = columnBuilder.getFileMapper();
         final NestedDataColumnMetadata metadata;
         final GenericIndexed<String> fields;
-        final NestedLiteralTypeInfo fieldInfo;
+        final NestedFieldTypeInfo fieldInfo;
         final CompressedVariableSizedBlobColumnSupplier compressedRawColumnSupplier;
         final ImmutableBitmap nullValues;
         final GenericIndexed<ByteBuffer> stringDictionary;
         final Supplier<FrontCodedIndexed> frontCodedStringDictionarySupplier;
         final Supplier<FixedIndexed<Long>> longDictionarySupplier;
         final Supplier<FixedIndexed<Double>> doubleDictionarySupplier;
+        final Supplier<FrontCodedIntArrayIndexed> arrayDictionarySupplier;
+
         ColumnType simpleType;
 
         metadata = jsonMapper.readValue(
@@ -91,7 +94,7 @@ public class NestedDataColumnSupplier implements Supplier<ComplexColumn>
             NestedDataColumnMetadata.class
         );
         fields = GenericIndexed.read(bb, GenericIndexed.STRING_STRATEGY, mapper);
-        fieldInfo = NestedLiteralTypeInfo.read(bb, fields.size());
+        fieldInfo = NestedFieldTypeInfo.read(bb, fields.size());
 
         if (fields.size() == 0) {
           // all nulls, in the future we'll deal with this better... but for now lets just call it a string because
@@ -99,7 +102,7 @@ public class NestedDataColumnSupplier implements Supplier<ComplexColumn>
           simpleType = ColumnType.STRING;
         } else if (fields.size() == 1 &&
                    ((version == 0x03 && NestedPathFinder.JQ_PATH_ROOT.equals(fields.get(0))) ||
-                    (version == 0x04 && NestedPathFinder.JSON_PATH_ROOT.equals(fields.get(0))))
+                    ((version == 0x04 || version == 0x05) && NestedPathFinder.JSON_PATH_ROOT.equals(fields.get(0))))
         ) {
           simpleType = fieldInfo.getTypes(0).getSingleType();
         } else {
@@ -162,6 +165,19 @@ public class NestedDataColumnSupplier implements Supplier<ComplexColumn>
             metadata.getByteOrder(),
             Double.BYTES
         );
+        if (version == 0x05) {
+          final ByteBuffer arrayDictionarybuffer = loadInternalFile(
+              mapper,
+              metadata,
+              NestedDataColumnSerializer.ARRAY_DICTIONARY_FILE_NAME
+          );
+          arrayDictionarySupplier = FrontCodedIntArrayIndexed.read(
+              arrayDictionarybuffer,
+              metadata.getByteOrder()
+          );
+        } else {
+          arrayDictionarySupplier = null;
+        }
         final ByteBuffer rawBuffer = loadInternalFile(mapper, metadata, NestedDataColumnSerializer.RAW_FILE_NAME);
         compressedRawColumnSupplier = CompressedVariableSizedBlobColumnSupplier.fromByteBuffer(
             NestedDataColumnSerializer.getInternalFileName(
@@ -194,6 +210,7 @@ public class NestedDataColumnSupplier implements Supplier<ComplexColumn>
             frontCodedStringDictionarySupplier,
             longDictionarySupplier,
             doubleDictionarySupplier,
+            arrayDictionarySupplier,
             columnConfig,
             mapper,
             simpleType
@@ -210,13 +227,14 @@ public class NestedDataColumnSupplier implements Supplier<ComplexColumn>
   private final byte version;
   private final NestedDataColumnMetadata metadata;
   private final GenericIndexed<String> fields;
-  private final NestedLiteralTypeInfo fieldInfo;
+  private final NestedFieldTypeInfo fieldInfo;
   private final CompressedVariableSizedBlobColumnSupplier compressedRawColumnSupplier;
   private final ImmutableBitmap nullValues;
   private final GenericIndexed<ByteBuffer> stringDictionary;
   private final Supplier<FrontCodedIndexed> frontCodedStringDictionarySupplier;
   private final Supplier<FixedIndexed<Long>> longDictionarySupplier;
   private final Supplier<FixedIndexed<Double>> doubleDictionarySupplier;
+  private final Supplier<FrontCodedIntArrayIndexed> arrayDictionarySupplier;
   private final ColumnConfig columnConfig;
   private final SmooshedFileMapper fileMapper;
 
@@ -227,13 +245,14 @@ public class NestedDataColumnSupplier implements Supplier<ComplexColumn>
       byte version,
       NestedDataColumnMetadata metadata,
       GenericIndexed<String> fields,
-      NestedLiteralTypeInfo fieldInfo,
+      NestedFieldTypeInfo fieldInfo,
       CompressedVariableSizedBlobColumnSupplier compressedRawColumnSupplier,
       ImmutableBitmap nullValues,
       GenericIndexed<ByteBuffer> stringDictionary,
       Supplier<FrontCodedIndexed> frontCodedStringDictionarySupplier,
       Supplier<FixedIndexed<Long>> longDictionarySupplier,
       Supplier<FixedIndexed<Double>> doubleDictionarySupplier,
+      Supplier<FrontCodedIntArrayIndexed> arrayDictionarySupplier,
       ColumnConfig columnConfig,
       SmooshedFileMapper fileMapper,
       @Nullable ColumnType simpleType
@@ -249,6 +268,7 @@ public class NestedDataColumnSupplier implements Supplier<ComplexColumn>
     this.frontCodedStringDictionarySupplier = frontCodedStringDictionarySupplier;
     this.longDictionarySupplier = longDictionarySupplier;
     this.doubleDictionarySupplier = doubleDictionarySupplier;
+    this.arrayDictionarySupplier = arrayDictionarySupplier;
     this.columnConfig = columnConfig;
     this.fileMapper = fileMapper;
     this.simpleType = simpleType;
@@ -259,8 +279,10 @@ public class NestedDataColumnSupplier implements Supplier<ComplexColumn>
   {
     if (version == 0x03) {
       return makeV3();
+    } else if (version == 0x04) {
+      return makeV4();
     }
-    return makeV4();
+    return makeV5();
   }
 
   @Nullable
@@ -329,6 +351,38 @@ public class NestedDataColumnSupplier implements Supplier<ComplexColumn>
     );
   }
 
+  private NestedDataColumnV5 makeV5()
+  {
+    if (frontCodedStringDictionarySupplier != null) {
+      return new NestedDataColumnV5<>(
+          metadata,
+          columnConfig,
+          compressedRawColumnSupplier,
+          nullValues,
+          fields,
+          fieldInfo,
+          frontCodedStringDictionarySupplier,
+          longDictionarySupplier,
+          doubleDictionarySupplier,
+          arrayDictionarySupplier,
+          fileMapper
+      );
+    }
+    return new NestedDataColumnV5<>(
+        metadata,
+        columnConfig,
+        compressedRawColumnSupplier,
+        nullValues,
+        fields,
+        fieldInfo,
+        stringDictionary::singleThreaded,
+        longDictionarySupplier,
+        doubleDictionarySupplier,
+        arrayDictionarySupplier,
+        fileMapper
+    );
+  }
+
   private static ByteBuffer loadInternalFile(
       SmooshedFileMapper fileMapper,
       NestedDataColumnMetadata metadata,
diff --git a/processing/src/main/java/org/apache/druid/segment/nested/NestedDataColumnV3.java b/processing/src/main/java/org/apache/druid/segment/nested/NestedDataColumnV3.java
index 8c2cc91963..eadfca84ce 100644
--- a/processing/src/main/java/org/apache/druid/segment/nested/NestedDataColumnV3.java
+++ b/processing/src/main/java/org/apache/druid/segment/nested/NestedDataColumnV3.java
@@ -41,7 +41,7 @@ public final class NestedDataColumnV3<TStringDictionary extends Indexed<ByteBuff
       CompressedVariableSizedBlobColumnSupplier compressedRawColumnSupplier,
       ImmutableBitmap nullValues,
       GenericIndexed<String> fields,
-      NestedLiteralTypeInfo fieldInfo,
+      NestedFieldTypeInfo fieldInfo,
       Supplier<TStringDictionary> stringDictionary,
       Supplier<FixedIndexed<Long>> longDictionarySupplier,
       Supplier<FixedIndexed<Double>> doubleDictionarySupplier,
@@ -58,6 +58,7 @@ public final class NestedDataColumnV3<TStringDictionary extends Indexed<ByteBuff
         stringDictionary,
         longDictionarySupplier,
         doubleDictionarySupplier,
+        null,
         fileMapper,
         NestedPathFinder.JQ_PATH_ROOT
     );
diff --git a/processing/src/main/java/org/apache/druid/segment/nested/NestedDataColumnV4.java b/processing/src/main/java/org/apache/druid/segment/nested/NestedDataColumnV4.java
index 96c5f56e2b..26631ee7fd 100644
--- a/processing/src/main/java/org/apache/druid/segment/nested/NestedDataColumnV4.java
+++ b/processing/src/main/java/org/apache/druid/segment/nested/NestedDataColumnV4.java
@@ -40,7 +40,7 @@ public final class NestedDataColumnV4<TStringDictionary extends Indexed<ByteBuff
       CompressedVariableSizedBlobColumnSupplier compressedRawColumnSupplier,
       ImmutableBitmap nullValues,
       GenericIndexed<String> fields,
-      NestedLiteralTypeInfo fieldInfo,
+      NestedFieldTypeInfo fieldInfo,
       Supplier<TStringDictionary> stringDictionary,
       Supplier<FixedIndexed<Long>> longDictionarySupplier,
       Supplier<FixedIndexed<Double>> doubleDictionarySupplier,
@@ -57,6 +57,7 @@ public final class NestedDataColumnV4<TStringDictionary extends Indexed<ByteBuff
         stringDictionary,
         longDictionarySupplier,
         doubleDictionarySupplier,
+        null,
         fileMapper,
         NestedPathFinder.JSON_PATH_ROOT
     );
diff --git a/processing/src/main/java/org/apache/druid/segment/nested/NestedDataColumnV4.java b/processing/src/main/java/org/apache/druid/segment/nested/NestedDataColumnV5.java
similarity index 89%
copy from processing/src/main/java/org/apache/druid/segment/nested/NestedDataColumnV4.java
copy to processing/src/main/java/org/apache/druid/segment/nested/NestedDataColumnV5.java
index 96c5f56e2b..7559226181 100644
--- a/processing/src/main/java/org/apache/druid/segment/nested/NestedDataColumnV4.java
+++ b/processing/src/main/java/org/apache/druid/segment/nested/NestedDataColumnV5.java
@@ -25,25 +25,27 @@ import org.apache.druid.java.util.common.io.smoosh.SmooshedFileMapper;
 import org.apache.druid.segment.column.ColumnConfig;
 import org.apache.druid.segment.data.CompressedVariableSizedBlobColumnSupplier;
 import org.apache.druid.segment.data.FixedIndexed;
+import org.apache.druid.segment.data.FrontCodedIntArrayIndexed;
 import org.apache.druid.segment.data.GenericIndexed;
 import org.apache.druid.segment.data.Indexed;
 
 import java.nio.ByteBuffer;
 import java.util.List;
 
-public final class NestedDataColumnV4<TStringDictionary extends Indexed<ByteBuffer>>
+public class NestedDataColumnV5<TStringDictionary extends Indexed<ByteBuffer>>
     extends CompressedNestedDataComplexColumn<TStringDictionary>
 {
-  public NestedDataColumnV4(
+  public NestedDataColumnV5(
       NestedDataColumnMetadata metadata,
       ColumnConfig columnConfig,
       CompressedVariableSizedBlobColumnSupplier compressedRawColumnSupplier,
       ImmutableBitmap nullValues,
       GenericIndexed<String> fields,
-      NestedLiteralTypeInfo fieldInfo,
+      NestedFieldTypeInfo fieldInfo,
       Supplier<TStringDictionary> stringDictionary,
       Supplier<FixedIndexed<Long>> longDictionarySupplier,
       Supplier<FixedIndexed<Double>> doubleDictionarySupplier,
+      Supplier<FrontCodedIntArrayIndexed> arrayDictionarySupplier,
       SmooshedFileMapper fileMapper
   )
   {
@@ -57,6 +59,7 @@ public final class NestedDataColumnV4<TStringDictionary extends Indexed<ByteBuff
         stringDictionary,
         longDictionarySupplier,
         doubleDictionarySupplier,
+        arrayDictionarySupplier,
         fileMapper,
         NestedPathFinder.JSON_PATH_ROOT
     );
diff --git a/processing/src/main/java/org/apache/druid/segment/nested/NestedFieldLiteralColumnIndexSupplier.java b/processing/src/main/java/org/apache/druid/segment/nested/NestedFieldColumnIndexSupplier.java
similarity index 97%
rename from processing/src/main/java/org/apache/druid/segment/nested/NestedFieldLiteralColumnIndexSupplier.java
rename to processing/src/main/java/org/apache/druid/segment/nested/NestedFieldColumnIndexSupplier.java
index 22ba59835e..b4e59d486b 100644
--- a/processing/src/main/java/org/apache/druid/segment/nested/NestedFieldLiteralColumnIndexSupplier.java
+++ b/processing/src/main/java/org/apache/druid/segment/nested/NestedFieldColumnIndexSupplier.java
@@ -68,10 +68,10 @@ import java.util.NoSuchElementException;
 import java.util.SortedSet;
 
 /**
- * Supplies indexes for nested field columns {@link NestedFieldLiteralDictionaryEncodedColumn} of
+ * Supplies indexes for nested field columns {@link NestedFieldDictionaryEncodedColumn} of
  * {@link NestedDataComplexColumn}.
  */
-public class NestedFieldLiteralColumnIndexSupplier<TStringDictionary extends Indexed<ByteBuffer>>
+public class NestedFieldColumnIndexSupplier<TStringDictionary extends Indexed<ByteBuffer>>
     implements ColumnIndexSupplier
 {
   @Nullable
@@ -83,17 +83,26 @@ public class NestedFieldLiteralColumnIndexSupplier<TStringDictionary extends Ind
   private final Supplier<FixedIndexed<Long>> globalLongDictionarySupplier;
   private final Supplier<FixedIndexed<Double>> globalDoubleDictionarySupplier;
 
+  @SuppressWarnings({"FieldCanBeLocal", "unused"})
+  @Nullable
+  private final GenericIndexed<ImmutableBitmap> arrayElementBitmaps;
+  @SuppressWarnings({"FieldCanBeLocal", "unused"})
+  @Nullable
+  private final Supplier<FixedIndexed<Integer>> arrayElementDictionarySupplier;
+
   private final int adjustLongId;
   private final int adjustDoubleId;
 
-  public NestedFieldLiteralColumnIndexSupplier(
-      NestedLiteralTypeInfo.TypeSet types,
+  public NestedFieldColumnIndexSupplier(
+      NestedFieldTypeInfo.TypeSet types,
       BitmapFactory bitmapFactory,
       GenericIndexed<ImmutableBitmap> bitmaps,
       Supplier<FixedIndexed<Integer>> localDictionarySupplier,
       Supplier<TStringDictionary> globalStringDictionarySupplier,
       Supplier<FixedIndexed<Long>> globalLongDictionarySupplier,
-      Supplier<FixedIndexed<Double>> globalDoubleDictionarySupplier
+      Supplier<FixedIndexed<Double>> globalDoubleDictionarySupplier,
+      @Nullable Supplier<FixedIndexed<Integer>> arrayElementDictionarySupplier,
+      @Nullable GenericIndexed<ImmutableBitmap> arrayElementBitmaps
   )
   {
     this.singleType = types.getSingleType();
@@ -103,6 +112,8 @@ public class NestedFieldLiteralColumnIndexSupplier<TStringDictionary extends Ind
     this.globalStringDictionarySupplier = globalStringDictionarySupplier;
     this.globalLongDictionarySupplier = globalLongDictionarySupplier;
     this.globalDoubleDictionarySupplier = globalDoubleDictionarySupplier;
+    this.arrayElementDictionarySupplier = arrayElementDictionarySupplier;
+    this.arrayElementBitmaps = arrayElementBitmaps;
     this.adjustLongId = globalStringDictionarySupplier.get().size();
     this.adjustDoubleId = adjustLongId + globalLongDictionarySupplier.get().size();
   }
@@ -320,7 +331,7 @@ public class NestedFieldLiteralColumnIndexSupplier<TStringDictionary extends Ind
     @Override
     public ImmutableBitmap getBitmap(int idx)
     {
-      return NestedFieldLiteralColumnIndexSupplier.this.getBitmap(idx);
+      return NestedFieldColumnIndexSupplier.this.getBitmap(idx);
     }
   }
 
diff --git a/processing/src/main/java/org/apache/druid/segment/nested/NestedFieldLiteralDictionaryEncodedColumn.java b/processing/src/main/java/org/apache/druid/segment/nested/NestedFieldDictionaryEncodedColumn.java
similarity index 85%
rename from processing/src/main/java/org/apache/druid/segment/nested/NestedFieldLiteralDictionaryEncodedColumn.java
rename to processing/src/main/java/org/apache/druid/segment/nested/NestedFieldDictionaryEncodedColumn.java
index be942a9dfd..3449d3410a 100644
--- a/processing/src/main/java/org/apache/druid/segment/nested/NestedFieldLiteralDictionaryEncodedColumn.java
+++ b/processing/src/main/java/org/apache/druid/segment/nested/NestedFieldDictionaryEncodedColumn.java
@@ -33,6 +33,7 @@ import org.apache.druid.query.filter.ValueMatcher;
 import org.apache.druid.query.monomorphicprocessing.RuntimeShapeInspector;
 import org.apache.druid.segment.AbstractDimensionSelector;
 import org.apache.druid.segment.ColumnValueSelector;
+import org.apache.druid.segment.DimensionHandlerUtils;
 import org.apache.druid.segment.DimensionSelector;
 import org.apache.druid.segment.DoubleColumnSelector;
 import org.apache.druid.segment.IdLookup;
@@ -46,6 +47,7 @@ import org.apache.druid.segment.data.ColumnarDoubles;
 import org.apache.druid.segment.data.ColumnarInts;
 import org.apache.druid.segment.data.ColumnarLongs;
 import org.apache.druid.segment.data.FixedIndexed;
+import org.apache.druid.segment.data.FrontCodedIntArrayIndexed;
 import org.apache.druid.segment.data.Indexed;
 import org.apache.druid.segment.data.IndexedInts;
 import org.apache.druid.segment.data.ReadableOffset;
@@ -69,10 +71,10 @@ import java.io.IOException;
 import java.nio.ByteBuffer;
 import java.util.BitSet;
 
-public class NestedFieldLiteralDictionaryEncodedColumn<TStringDictionary extends Indexed<ByteBuffer>>
+public class NestedFieldDictionaryEncodedColumn<TStringDictionary extends Indexed<ByteBuffer>>
     implements DictionaryEncodedColumn<String>
 {
-  private final NestedLiteralTypeInfo.TypeSet types;
+  private final NestedFieldTypeInfo.TypeSet types;
   @Nullable
   private final ColumnType singleType;
   private final ColumnarLongs longsColumn;
@@ -82,20 +84,25 @@ public class NestedFieldLiteralDictionaryEncodedColumn<TStringDictionary extends
   private final FixedIndexed<Long> globalLongDictionary;
   private final FixedIndexed<Double> globalDoubleDictionary;
 
+  private final FrontCodedIntArrayIndexed globalArrayDictionary;
+
   private final FixedIndexed<Integer> dictionary;
   private final ImmutableBitmap nullBitmap;
 
   private final int adjustLongId;
   private final int adjustDoubleId;
+  private final int adjustArrayId;
+
 
-  public NestedFieldLiteralDictionaryEncodedColumn(
-      NestedLiteralTypeInfo.TypeSet types,
+  public NestedFieldDictionaryEncodedColumn(
+      NestedFieldTypeInfo.TypeSet types,
       ColumnarLongs longsColumn,
       ColumnarDoubles doublesColumn,
       ColumnarInts column,
       TStringDictionary globalDictionary,
       FixedIndexed<Long> globalLongDictionary,
       FixedIndexed<Double> globalDoubleDictionary,
+      @Nullable FrontCodedIntArrayIndexed globalArrayDictionary,
       FixedIndexed<Integer> dictionary,
       ImmutableBitmap nullBitmap
   )
@@ -108,10 +115,12 @@ public class NestedFieldLiteralDictionaryEncodedColumn<TStringDictionary extends
     this.globalDictionary = globalDictionary;
     this.globalLongDictionary = globalLongDictionary;
     this.globalDoubleDictionary = globalDoubleDictionary;
+    this.globalArrayDictionary = globalArrayDictionary;
     this.dictionary = dictionary;
     this.nullBitmap = nullBitmap;
     this.adjustLongId = globalDictionary.size();
     this.adjustDoubleId = adjustLongId + globalLongDictionary.size();
+    this.adjustArrayId = adjustDoubleId + globalDoubleDictionary.size();
   }
 
   @Override
@@ -215,6 +224,18 @@ public class NestedFieldLiteralDictionaryEncodedColumn<TStringDictionary extends
     }
   }
 
+  private Object lookupGlobalScalarObject(int globalId)
+  {
+    if (globalId < globalDictionary.size()) {
+      return StringUtils.fromUtf8Nullable(globalDictionary.get(globalId));
+    } else if (globalId < globalDictionary.size() + globalLongDictionary.size()) {
+      return globalLongDictionary.get(globalId - adjustLongId);
+    } else if (globalId < globalDictionary.size() + globalLongDictionary.size() + globalDoubleDictionary.size()) {
+      return globalDoubleDictionary.get(globalId - adjustDoubleId);
+    }
+    throw new IllegalArgumentException("not a scalar in the dictionary");
+  }
+
   @Override
   public DimensionSelector makeDimensionSelector(
       ReadableOffset offset,
@@ -299,7 +320,10 @@ public class NestedFieldLiteralDictionaryEncodedColumn<TStringDictionary extends
       @Override
       public boolean isNull()
       {
-        return dictionary.get(getRowValue()) == 0;
+        if (dictionary.get(getRowValue()) == 0) {
+          return true;
+        }
+        return DimensionHandlerUtils.isNumericNull(getObject());
       }
 
       @Override
@@ -332,7 +356,7 @@ public class NestedFieldLiteralDictionaryEncodedColumn<TStringDictionary extends
               @Override
               public void inspectRuntimeShape(RuntimeShapeInspector inspector)
               {
-                inspector.visit("column", NestedFieldLiteralDictionaryEncodedColumn.this);
+                inspector.visit("column", NestedFieldDictionaryEncodedColumn.this);
               }
             };
           } else {
@@ -373,7 +397,7 @@ public class NestedFieldLiteralDictionaryEncodedColumn<TStringDictionary extends
           @Override
           public void inspectRuntimeShape(RuntimeShapeInspector inspector)
           {
-            inspector.visit("column", NestedFieldLiteralDictionaryEncodedColumn.this);
+            inspector.visit("column", NestedFieldDictionaryEncodedColumn.this);
           }
         };
       }
@@ -381,7 +405,7 @@ public class NestedFieldLiteralDictionaryEncodedColumn<TStringDictionary extends
       @Override
       public Object getObject()
       {
-        return NestedFieldLiteralDictionaryEncodedColumn.this.lookupName(getRowValue());
+        return NestedFieldDictionaryEncodedColumn.this.lookupName(getRowValue());
       }
 
       @Override
@@ -407,7 +431,7 @@ public class NestedFieldLiteralDictionaryEncodedColumn<TStringDictionary extends
       @Override
       public String lookupName(int id)
       {
-        final String value = NestedFieldLiteralDictionaryEncodedColumn.this.lookupName(id);
+        final String value = NestedFieldDictionaryEncodedColumn.this.lookupName(id);
         return extractionFn == null ? value : extractionFn.apply(value);
       }
 
@@ -428,7 +452,7 @@ public class NestedFieldLiteralDictionaryEncodedColumn<TStringDictionary extends
       public int lookupId(String name)
       {
         if (extractionFn == null) {
-          return NestedFieldLiteralDictionaryEncodedColumn.this.lookupId(name);
+          return NestedFieldDictionaryEncodedColumn.this.lookupId(name);
         }
         throw new UnsupportedOperationException("cannot perform lookup when applying an extraction function");
       }
@@ -524,7 +548,7 @@ public class NestedFieldLiteralDictionaryEncodedColumn<TStringDictionary extends
         };
       }
     }
-    if (singleType == null) {
+    if (singleType == null || singleType.isArray()) {
       return new ColumnValueSelector<Object>()
       {
 
@@ -538,12 +562,18 @@ public class NestedFieldLiteralDictionaryEncodedColumn<TStringDictionary extends
         {
           final int localId = column.get(offset.getOffset());
           final int globalId = dictionary.get(localId);
-          if (globalId < adjustLongId) {
-            return StringUtils.fromUtf8Nullable(globalDictionary.get(globalId));
-          } else if (globalId < adjustDoubleId) {
-            return globalLongDictionary.get(globalId - adjustLongId);
+          if (globalId < adjustArrayId) {
+            return lookupGlobalScalarObject(globalId);
           } else {
-            return globalDoubleDictionary.get(globalId - adjustDoubleId);
+            int[] arr = globalArrayDictionary.get(globalId - adjustArrayId);
+            if (arr == null) {
+              return null;
+            }
+            final Object[] array = new Object[arr.length];
+            for (int i = 0; i < arr.length; i++) {
+              array[i] = lookupGlobalScalarObject(arr[i]);
+            }
+            return array;
           }
         }
 
@@ -620,7 +650,10 @@ public class NestedFieldLiteralDictionaryEncodedColumn<TStringDictionary extends
               nullMark = nullIterator.next();
             }
           }
-          return nullMark == offsetMark;
+          if (nullMark == offsetMark) {
+            return true;
+          }
+          return DimensionHandlerUtils.isNumericNull(getObject());
         }
 
         @Override
@@ -662,7 +695,7 @@ public class NestedFieldLiteralDictionaryEncodedColumn<TStringDictionary extends
       @Override
       public String lookupName(final int id)
       {
-        return NestedFieldLiteralDictionaryEncodedColumn.this.lookupName(id);
+        return NestedFieldDictionaryEncodedColumn.this.lookupName(id);
       }
 
       @Nullable
@@ -682,7 +715,7 @@ public class NestedFieldLiteralDictionaryEncodedColumn<TStringDictionary extends
       @Override
       public int lookupId(@Nullable String name)
       {
-        return NestedFieldLiteralDictionaryEncodedColumn.this.lookupId(name);
+        return NestedFieldDictionaryEncodedColumn.this.lookupId(name);
       }
     }
 
@@ -698,6 +731,61 @@ public class NestedFieldLiteralDictionaryEncodedColumn<TStringDictionary extends
   @Override
   public VectorObjectSelector makeVectorObjectSelector(ReadableVectorOffset offset)
   {
+    if (singleType != null && singleType.isArray()) {
+      return new VectorObjectSelector()
+      {
+        private final int[] vector = new int[offset.getMaxVectorSize()];
+        private final Object[] objects = new Object[offset.getMaxVectorSize()];
+        private int id = ReadableVectorInspector.NULL_ID;
+
+        @Override
+
+        public Object[] getObjectVector()
+        {
+          if (id == offset.getId()) {
+            return objects;
+          }
+
+          if (offset.isContiguous()) {
+            column.get(vector, offset.getStartOffset(), offset.getCurrentVectorSize());
+          } else {
+            column.get(vector, offset.getOffsets(), offset.getCurrentVectorSize());
+          }
+          for (int i = 0; i < offset.getCurrentVectorSize(); i++) {
+            final int globalId = dictionary.get(vector[i]);
+            if (globalId < adjustArrayId) {
+              objects[i] = lookupGlobalScalarObject(globalId);
+            } else {
+              int[] arr = globalArrayDictionary.get(globalId - adjustArrayId);
+              if (arr == null) {
+                objects[i] = null;
+              } else {
+                final Object[] array = new Object[arr.length];
+                for (int j = 0; j < arr.length; j++) {
+                  array[j] = lookupGlobalScalarObject(arr[j]);
+                }
+                objects[i] = array;
+              }
+            }
+          }
+          id = offset.getId();
+
+          return objects;
+        }
+
+        @Override
+        public int getMaxVectorSize()
+        {
+          return offset.getMaxVectorSize();
+        }
+
+        @Override
+        public int getCurrentVectorSize()
+        {
+          return offset.getCurrentVectorSize();
+        }
+      };
+    }
     final class StringVectorSelector extends StringDictionaryEncodedColumn.StringVectorObjectSelector
     {
       public StringVectorSelector()
@@ -709,7 +797,7 @@ public class NestedFieldLiteralDictionaryEncodedColumn<TStringDictionary extends
       @Override
       public String lookupName(int id)
       {
-        return NestedFieldLiteralDictionaryEncodedColumn.this.lookupName(id);
+        return NestedFieldDictionaryEncodedColumn.this.lookupName(id);
       }
     }
 
diff --git a/processing/src/main/java/org/apache/druid/segment/nested/NestedLiteralTypeInfo.java b/processing/src/main/java/org/apache/druid/segment/nested/NestedFieldTypeInfo.java
similarity index 68%
rename from processing/src/main/java/org/apache/druid/segment/nested/NestedLiteralTypeInfo.java
rename to processing/src/main/java/org/apache/druid/segment/nested/NestedFieldTypeInfo.java
index 4b4b89df83..1d6d251fab 100644
--- a/processing/src/main/java/org/apache/druid/segment/nested/NestedLiteralTypeInfo.java
+++ b/processing/src/main/java/org/apache/druid/segment/nested/NestedFieldTypeInfo.java
@@ -19,6 +19,7 @@
 
 package org.apache.druid.segment.nested;
 
+import com.google.common.base.Preconditions;
 import com.google.common.collect.Sets;
 import org.apache.druid.java.util.common.ISE;
 import org.apache.druid.java.util.common.io.smoosh.FileSmoosher;
@@ -31,21 +32,28 @@ import javax.annotation.Nullable;
 import java.io.IOException;
 import java.nio.ByteBuffer;
 import java.nio.channels.WritableByteChannel;
+import java.util.Objects;
 import java.util.Set;
 
 /**
  * Binary serialization for nested field type info, translated into this compact format for storage in segments.
  * The index of the type info here is the same as the field index in {@link CompressedNestedDataComplexColumn#fields}
  */
-public class NestedLiteralTypeInfo
+public class NestedFieldTypeInfo
 {
   private static final byte STRING_MASK = 1;
   private static final byte LONG_MASK = 1 << 2;
   private static final byte DOUBLE_MASK = 1 << 3;
 
-  public static NestedLiteralTypeInfo read(ByteBuffer buffer, int length)
+  private static final byte STRING_ARRAY_MASK = 1 << 4;
+
+  private static final byte LONG_ARRAY_MASK = 1 << 5;
+
+  private static final byte DOUBLE_ARRAY_MASK = 1 << 6;
+
+  public static NestedFieldTypeInfo read(ByteBuffer buffer, int length)
   {
-    NestedLiteralTypeInfo typeInfo = new NestedLiteralTypeInfo(buffer);
+    NestedFieldTypeInfo typeInfo = new NestedFieldTypeInfo(buffer);
     buffer.position(buffer.position() + length);
     return typeInfo;
   }
@@ -53,7 +61,7 @@ public class NestedLiteralTypeInfo
   private final ByteBuffer buffer;
   private final int startOffset;
 
-  public NestedLiteralTypeInfo(ByteBuffer buffer)
+  public NestedFieldTypeInfo(ByteBuffer buffer)
   {
     this.buffer = buffer;
     this.startOffset = buffer.position();
@@ -79,7 +87,7 @@ public class NestedLiteralTypeInfo
     @Nullable
     public ColumnType getSingleType()
     {
-      return NestedLiteralTypeInfo.getSingleType(types);
+      return NestedFieldTypeInfo.getSingleType(types);
     }
 
     public byte getByteValue()
@@ -92,6 +100,25 @@ public class NestedLiteralTypeInfo
     {
       return convertToSet(types).toString();
     }
+
+    @Override
+    public boolean equals(Object o)
+    {
+      if (this == o) {
+        return true;
+      }
+      if (o == null || getClass() != o.getClass()) {
+        return false;
+      }
+      TypeSet typeSet = (TypeSet) o;
+      return types == typeSet.types;
+    }
+
+    @Override
+    public int hashCode()
+    {
+      return Objects.hash(types);
+    }
   }
 
   public static class MutableTypeSet
@@ -120,6 +147,22 @@ public class NestedLiteralTypeInfo
         case DOUBLE:
           types |= DOUBLE_MASK;
           break;
+        case ARRAY:
+          Preconditions.checkNotNull(type.getElementType(), "ElementType must not be null");
+          switch (type.getElementType().getType()) {
+            case STRING:
+              types |= STRING_ARRAY_MASK;
+              break;
+            case LONG:
+              types |= LONG_ARRAY_MASK;
+              break;
+            case DOUBLE:
+              types |= DOUBLE_ARRAY_MASK;
+              break;
+            default:
+              throw new ISE("Unsupported nested array type: [%s]", type.asTypeString());
+          }
+          break;
         default:
           throw new ISE("Unsupported nested type: [%s]", type.asTypeString());
       }
@@ -135,7 +178,7 @@ public class NestedLiteralTypeInfo
     @Nullable
     public ColumnType getSingleType()
     {
-      return NestedLiteralTypeInfo.getSingleType(types);
+      return NestedFieldTypeInfo.getSingleType(types);
     }
 
     public boolean isEmpty()
@@ -154,6 +197,25 @@ public class NestedLiteralTypeInfo
     {
       return convertToSet(types).toString();
     }
+
+    @Override
+    public boolean equals(Object o)
+    {
+      if (this == o) {
+        return true;
+      }
+      if (o == null || getClass() != o.getClass()) {
+        return false;
+      }
+      MutableTypeSet typeSet = (MutableTypeSet) o;
+      return types == typeSet.types;
+    }
+
+    @Override
+    public int hashCode()
+    {
+      return Objects.hash(types);
+    }
   }
 
   public static class Writer implements Serializer
@@ -209,6 +271,18 @@ public class NestedLiteralTypeInfo
       singleType = ColumnType.DOUBLE;
       count++;
     }
+    if ((types & STRING_ARRAY_MASK) > 0) {
+      singleType = ColumnType.STRING_ARRAY;
+      count++;
+    }
+    if ((types & LONG_ARRAY_MASK) > 0) {
+      singleType = ColumnType.LONG_ARRAY;
+      count++;
+    }
+    if ((types & DOUBLE_ARRAY_MASK) > 0) {
+      singleType = ColumnType.DOUBLE_ARRAY;
+      count++;
+    }
     return count == 1 ? singleType : null;
   }
 
@@ -224,6 +298,15 @@ public class NestedLiteralTypeInfo
     if ((types & DOUBLE_MASK) > 0) {
       theTypes.add(ColumnType.DOUBLE);
     }
+    if ((types & STRING_ARRAY_MASK) > 0) {
+      theTypes.add(ColumnType.STRING_ARRAY);
+    }
+    if ((types & DOUBLE_ARRAY_MASK) > 0) {
+      theTypes.add(ColumnType.DOUBLE_ARRAY);
+    }
+    if ((types & LONG_ARRAY_MASK) > 0) {
+      theTypes.add(ColumnType.LONG_ARRAY);
+    }
     return theTypes;
   }
 }
diff --git a/processing/src/main/java/org/apache/druid/segment/nested/StringFieldColumnWriter.java b/processing/src/main/java/org/apache/druid/segment/nested/StringFieldColumnWriter.java
index c9f10a269c..5bb2f186b0 100644
--- a/processing/src/main/java/org/apache/druid/segment/nested/StringFieldColumnWriter.java
+++ b/processing/src/main/java/org/apache/druid/segment/nested/StringFieldColumnWriter.java
@@ -44,7 +44,7 @@ public final class StringFieldColumnWriter extends GlobalDictionaryEncodedFieldC
   }
 
   @Override
-  String processValue(Object value)
+  String processValue(int row, Object value)
   {
     if (value == null) {
       return null;
diff --git a/processing/src/main/java/org/apache/druid/segment/nested/StructuredDataProcessor.java b/processing/src/main/java/org/apache/druid/segment/nested/StructuredDataProcessor.java
index 8549deda44..e517bdc0a0 100644
--- a/processing/src/main/java/org/apache/druid/segment/nested/StructuredDataProcessor.java
+++ b/processing/src/main/java/org/apache/druid/segment/nested/StructuredDataProcessor.java
@@ -22,6 +22,7 @@ package org.apache.druid.segment.nested;
 import javax.annotation.Nullable;
 import java.util.ArrayDeque;
 import java.util.ArrayList;
+import java.util.Arrays;
 import java.util.HashSet;
 import java.util.List;
 import java.util.Map;
@@ -31,11 +32,31 @@ import java.util.Set;
 
 public abstract class StructuredDataProcessor
 {
-  public abstract ProcessedLiteral<?> processLiteralField(ArrayList<NestedPathPart> fieldPath, Object fieldValue);
+  protected StructuredDataProcessor()
+  {
+  }
+
+  /**
+   * process a value that is definitely not a {@link Map}, {@link List}, or {@link Object[]}
+   */
+  public abstract ProcessedValue<?> processField(ArrayList<NestedPathPart> fieldPath, @Nullable Object fieldValue);
+
+  /**
+   * Process a {@link List} or {@link Object[]} returning a {@link ProcessedValue} if no further processing should
+   * be performed by the {@link StructuredDataProcessor}, else a return value of null indicates that each element
+   * of the array will be processed separately as a new {@link NestedPathArrayElement} part.
+   */
+  @Nullable
+  public abstract ProcessedValue<?> processArrayField(ArrayList<NestedPathPart> fieldPath, @Nullable List<?> array);
+
 
   /**
-   * Process fields, returning a list of all paths to literal fields, represented as an ordered sequence of
-   * {@link NestedPathPart}.
+   * Process some object, traversing any nested structure and returning a list of all paths which created a
+   * {@link ProcessedValue} during processing, represented as an ordered sequence of {@link NestedPathPart}.
+   *
+   * This method processes plain java objects, for each {@link Map} it adds a {@link MapField} to the path, for
+   * {@link List} a {@link ArrayField}, {@link Object[]} a {@link ArrayField}, and so on. {@link ArrayField} and
+   * {@link ArrayField} will be processed by {@link #processArrayField(ArrayList, List)}
    */
   public ProcessResults processFields(Object raw)
   {
@@ -45,9 +66,11 @@ public abstract class StructuredDataProcessor
     if (raw instanceof Map) {
       toProcess.add(new MapField(newPath, (Map<String, ?>) raw));
     } else if (raw instanceof List) {
-      toProcess.add(new ListField(newPath, (List<?>) raw));
+      toProcess.add(new ArrayField(newPath, (List<?>) raw));
+    } else if (raw instanceof Object[]) {
+      toProcess.add(new ArrayField(newPath, Arrays.asList((Object[]) raw)));
     } else {
-      return new ProcessResults().addLiteralField(newPath, processLiteralField(newPath, raw).getSize());
+      return new ProcessResults().addLiteralField(newPath, processField(newPath, raw).getSize());
     }
 
     final ProcessResults accumulator = new ProcessResults();
@@ -56,8 +79,8 @@ public abstract class StructuredDataProcessor
       Field next = toProcess.poll();
       if (next instanceof MapField) {
         accumulator.merge(processMapField(toProcess, (MapField) next));
-      } else if (next instanceof ListField) {
-        accumulator.merge(processListField(toProcess, (ListField) next));
+      } else if (next instanceof ArrayField) {
+        accumulator.merge(processArrayField(toProcess, (ArrayField) next));
       }
     }
     return accumulator;
@@ -76,40 +99,50 @@ public abstract class StructuredDataProcessor
       newPath.add(new NestedPathField(entry.getKey()));
       if (value instanceof List) {
         List<?> theList = (List<?>) value;
-        toProcess.add(new ListField(newPath, theList));
+        toProcess.add(new ArrayField(newPath, theList));
+      } else if (value instanceof Object[]) {
+        toProcess.add(new ArrayField(newPath, Arrays.asList((Object[]) value)));
       } else if (value instanceof Map) {
         toProcess.add(new MapField(newPath, (Map<String, ?>) value));
       } else {
         // literals get processed
-        processResults.addLiteralField(newPath, processLiteralField(newPath, value).getSize());
+        processResults.addLiteralField(newPath, processField(newPath, value).getSize());
       }
     }
     return processResults;
   }
 
-  private ProcessResults processListField(Queue<Field> toProcess, ListField list)
+  private ProcessResults processArrayField(Queue<Field> toProcess, ArrayField list)
   {
     // start with object reference, is probably a bit bigger than this...
     final ProcessResults results = new ProcessResults().withSize(8);
     final List<?> theList = list.getList();
-    for (int i = 0; i < theList.size(); i++) {
-      final ArrayList<NestedPathPart> newPath = new ArrayList<>(list.getPath());
-      newPath.add(new NestedPathArrayElement(i));
-      final Object element = StructuredData.unwrap(theList.get(i));
-      // maps and lists go back into the queue
-      if (element instanceof Map) {
-        toProcess.add(new MapField(newPath, (Map<String, ?>) element));
-      } else if (element instanceof List) {
-        toProcess.add(new ListField(newPath, (List<?>) element));
-      } else {
-        // literals get processed
-        results.addLiteralField(newPath, processLiteralField(newPath, element).getSize());
+    // check to see if the processor handled the array, indicated by a non-null result, if so we can stop here
+    final ProcessedValue<?> maybeProcessed = processArrayField(list.getPath(), theList);
+    if (maybeProcessed != null) {
+      results.addLiteralField(list.getPath(), maybeProcessed.getSize());
+    } else {
+      // else we have to dig into the list and process each element
+      for (int i = 0; i < theList.size(); i++) {
+        final ArrayList<NestedPathPart> newPath = new ArrayList<>(list.getPath());
+        newPath.add(new NestedPathArrayElement(i));
+        final Object element = StructuredData.unwrap(theList.get(i));
+        // maps and lists go back into the queue
+        if (element instanceof Map) {
+          toProcess.add(new MapField(newPath, (Map<String, ?>) element));
+        } else if (element instanceof List) {
+          toProcess.add(new ArrayField(newPath, (List<?>) element));
+        } else if (element instanceof Object[]) {
+          toProcess.add(new ArrayField(newPath, Arrays.asList((Object[]) element)));
+        } else {
+          results.addLiteralField(newPath, processField(newPath, element).getSize());
+        }
       }
     }
     return results;
   }
 
-  abstract static class Field
+  private abstract static class Field
   {
     private final ArrayList<NestedPathPart> path;
 
@@ -124,11 +157,11 @@ public abstract class StructuredDataProcessor
     }
   }
 
-  static class ListField extends Field
+  static class ArrayField extends Field
   {
     private final List<?> list;
 
-    ListField(ArrayList<NestedPathPart> path, List<?> list)
+    ArrayField(ArrayList<NestedPathPart> path, List<?> list)
     {
       super(path);
       this.list = list;
@@ -156,14 +189,14 @@ public abstract class StructuredDataProcessor
     }
   }
 
-  public static class ProcessedLiteral<T>
+  public static class ProcessedValue<T>
   {
-    public static final ProcessedLiteral<?> NULL_LITERAL = new ProcessedLiteral<>(null, 0);
+    public static final ProcessedValue<?> NULL_LITERAL = new ProcessedValue<>(null, 0);
     @Nullable
     private final T value;
     private final int size;
 
-    public ProcessedLiteral(@Nullable T value, int size)
+    public ProcessedValue(@Nullable T value, int size)
     {
       this.value = value;
       this.size = size;
@@ -190,7 +223,7 @@ public abstract class StructuredDataProcessor
       if (o == null || getClass() != o.getClass()) {
         return false;
       }
-      ProcessedLiteral<?> that = (ProcessedLiteral<?>) o;
+      ProcessedValue<?> that = (ProcessedValue<?>) o;
       return size == that.size && Objects.equals(value, that.value);
     }
 
diff --git a/processing/src/main/java/org/apache/druid/segment/nested/VariantLiteralFieldColumnWriter.java b/processing/src/main/java/org/apache/druid/segment/nested/VariantFieldColumnWriter.java
similarity index 63%
rename from processing/src/main/java/org/apache/druid/segment/nested/VariantLiteralFieldColumnWriter.java
rename to processing/src/main/java/org/apache/druid/segment/nested/VariantFieldColumnWriter.java
index 7440c12004..121f388583 100644
--- a/processing/src/main/java/org/apache/druid/segment/nested/VariantLiteralFieldColumnWriter.java
+++ b/processing/src/main/java/org/apache/druid/segment/nested/VariantFieldColumnWriter.java
@@ -19,6 +19,7 @@
 
 package org.apache.druid.segment.nested;
 
+import com.google.common.base.Preconditions;
 import org.apache.druid.java.util.common.io.smoosh.FileSmoosher;
 import org.apache.druid.segment.IndexSpec;
 import org.apache.druid.segment.writeout.SegmentWriteOutMedium;
@@ -29,9 +30,9 @@ import java.nio.channels.WritableByteChannel;
 /**
  * Literal field writer for mixed type nested columns of {@link NestedDataColumnSerializer}
  */
-public final class VariantLiteralFieldColumnWriter extends GlobalDictionaryEncodedFieldColumnWriter<Object>
+public final class VariantFieldColumnWriter extends GlobalDictionaryEncodedFieldColumnWriter<Object>
 {
-  public VariantLiteralFieldColumnWriter(
+  public VariantFieldColumnWriter(
       String columnName,
       String fieldName,
       SegmentWriteOutMedium segmentWriteOutMedium,
@@ -42,6 +43,32 @@ public final class VariantLiteralFieldColumnWriter extends GlobalDictionaryEncod
     super(columnName, fieldName, segmentWriteOutMedium, indexSpec, globalDictionaryIdLookup);
   }
 
+
+  @Override
+  Object processValue(int row, Object value)
+  {
+    if (value instanceof Object[]) {
+      Object[] array = (Object[]) value;
+      final int[] globalIds = new int[array.length];
+      for (int i = 0; i < array.length; i++) {
+        if (array[i] == null) {
+          globalIds[i] = 0;
+        } else if (array[i] instanceof String) {
+          globalIds[i] = globalDictionaryIdLookup.lookupString((String) array[i]);
+        } else if (array[i] instanceof Long) {
+          globalIds[i] = globalDictionaryIdLookup.lookupLong((Long) array[i]);
+        } else if (array[i] instanceof Double) {
+          globalIds[i] = globalDictionaryIdLookup.lookupDouble((Double) array[i]);
+        } else {
+          globalIds[i] = -1;
+        }
+        Preconditions.checkArgument(globalIds[i] >= 0, "unknown global id [%s] for value [%s]", globalIds[i], array[i]);
+      }
+      return globalIds;
+    }
+    return super.processValue(row, value);
+  }
+
   @Override
   int lookupGlobalId(Object value)
   {
@@ -52,6 +79,8 @@ public final class VariantLiteralFieldColumnWriter extends GlobalDictionaryEncod
       return globalDictionaryIdLookup.lookupLong((Long) value);
     } else if (value instanceof Double) {
       return globalDictionaryIdLookup.lookupDouble((Double) value);
+    } else if (value instanceof int[]) {
+      return globalDictionaryIdLookup.lookupArray((int[]) value);
     } else {
       return globalDictionaryIdLookup.lookupString(String.valueOf(value));
     }
diff --git a/processing/src/main/java/org/apache/druid/segment/virtual/NestedFieldVirtualColumn.java b/processing/src/main/java/org/apache/druid/segment/virtual/NestedFieldVirtualColumn.java
index 6541e1e79c..f4902946bf 100644
--- a/processing/src/main/java/org/apache/druid/segment/virtual/NestedFieldVirtualColumn.java
+++ b/processing/src/main/java/org/apache/druid/segment/virtual/NestedFieldVirtualColumn.java
@@ -57,6 +57,7 @@ import org.apache.druid.segment.data.ReadableOffset;
 import org.apache.druid.segment.nested.CompressedNestedDataComplexColumn;
 import org.apache.druid.segment.nested.NestedDataComplexColumn;
 import org.apache.druid.segment.nested.NestedDataComplexTypeSerde;
+import org.apache.druid.segment.nested.NestedFieldDictionaryEncodedColumn;
 import org.apache.druid.segment.nested.NestedPathArrayElement;
 import org.apache.druid.segment.nested.NestedPathFinder;
 import org.apache.druid.segment.nested.NestedPathPart;
@@ -78,7 +79,7 @@ import java.util.Objects;
 
 /**
  * Optimized virtual column that can make direct selectors into a {@link NestedDataComplexColumn} or any associated
- * nested fields ({@link org.apache.druid.segment.nested.NestedFieldLiteralDictionaryEncodedColumn}) including using
+ * nested fields ({@link NestedFieldDictionaryEncodedColumn}) including using
  * their indexes.
  * <p>
  * This virtual column is used for the SQL operators JSON_VALUE (if {@link #processFromRaw} is set to false) or
@@ -655,12 +656,13 @@ public class NestedFieldVirtualColumn implements VirtualColumn
       if (capabilities.is(ValueType.COMPLEX) &&
           capabilities.getComplexTypeName().equals(NestedDataComplexTypeSerde.TYPE_NAME) &&
           capabilities.isDictionaryEncoded().isTrue()) {
+        final boolean useDictionary = parts.isEmpty() || !(parts.get(parts.size() - 1) instanceof NestedPathArrayElement);
         return ColumnCapabilitiesImpl.createDefault()
                                      .setType(expectedType != null ? expectedType : ColumnType.STRING)
-                                     .setDictionaryEncoded(true)
-                                     .setDictionaryValuesSorted(true)
-                                     .setDictionaryValuesUnique(true)
-                                     .setHasBitmapIndexes(true)
+                                     .setDictionaryEncoded(useDictionary)
+                                     .setDictionaryValuesSorted(useDictionary)
+                                     .setDictionaryValuesUnique(useDictionary)
+                                     .setHasBitmapIndexes(useDictionary)
                                      .setHasNulls(expectedType == null || (expectedType.isNumeric()
                                                                            && NullHandling.sqlCompatible()));
       }
diff --git a/processing/src/test/java/org/apache/druid/data/input/ResourceInputSource.java b/processing/src/test/java/org/apache/druid/data/input/ResourceInputSource.java
new file mode 100644
index 0000000000..44aa52615d
--- /dev/null
+++ b/processing/src/test/java/org/apache/druid/data/input/ResourceInputSource.java
@@ -0,0 +1,110 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.druid.data.input;
+
+import org.apache.druid.data.input.impl.InputEntityIteratingReader;
+import org.apache.druid.utils.CompressionUtils;
+
+import javax.annotation.Nullable;
+import java.io.File;
+import java.io.IOException;
+import java.io.InputStream;
+import java.net.URI;
+import java.net.URISyntaxException;
+import java.util.Collections;
+
+/**
+ * {@link InputSource} backed by {@link ClassLoader#getResourceAsStream(String)}, for all your embedded test data
+ * ingestion needs
+ */
+public class ResourceInputSource extends AbstractInputSource
+{
+  public static ResourceInputSource of(ClassLoader loader, String resourceFile)
+  {
+    return new ResourceInputSource(loader, resourceFile);
+  }
+
+  private final ClassLoader classLoader;
+  private final String resourceFile;
+
+  private ResourceInputSource(ClassLoader classLoader, String resourceFile)
+  {
+    this.classLoader = classLoader;
+    this.resourceFile = resourceFile;
+  }
+
+  @Override
+  public boolean isSplittable()
+  {
+    return false;
+  }
+
+  @Override
+  public boolean needsFormat()
+  {
+    return true;
+  }
+
+  @Override
+  protected InputSourceReader formattableReader(
+      InputRowSchema inputRowSchema,
+      InputFormat inputFormat,
+      File temporaryDirectory
+  )
+  {
+    return new InputEntityIteratingReader(
+        inputRowSchema,
+        inputFormat,
+        Collections.singletonList(new ResourceStreamEntity(classLoader, resourceFile)).iterator(),
+        temporaryDirectory
+    );
+  }
+
+  public static class ResourceStreamEntity implements InputEntity
+  {
+    private final ClassLoader classLoader;
+    private final String resourceFile;
+
+    public ResourceStreamEntity(ClassLoader classLoader, String resourceFile)
+    {
+      this.classLoader = classLoader;
+      this.resourceFile = resourceFile;
+    }
+
+    @Nullable
+    @Override
+    public URI getUri()
+    {
+      try {
+        return classLoader.getResource(resourceFile).toURI();
+      }
+      catch (URISyntaxException e) {
+        throw new RuntimeException(e);
+      }
+    }
+
+    @Override
+    public InputStream open() throws IOException
+    {
+      final InputStream resourceStream = classLoader.getResourceAsStream(resourceFile);
+      return CompressionUtils.decompress(resourceStream, resourceFile);
+    }
+  }
+}
diff --git a/processing/src/test/java/org/apache/druid/data/input/impl/InputRowParserSerdeTest.java b/processing/src/test/java/org/apache/druid/data/input/impl/InputRowParserSerdeTest.java
index 7e1b487cae..34a3722fc2 100644
--- a/processing/src/test/java/org/apache/druid/data/input/impl/InputRowParserSerdeTest.java
+++ b/processing/src/test/java/org/apache/druid/data/input/impl/InputRowParserSerdeTest.java
@@ -233,7 +233,6 @@ public class InputRowParserSerdeTest
         "hey0barx",
         "metA",
         "missing",
-        "timestamp",
         "foo.bar1",
         "blah",
         "newmet",
diff --git a/processing/src/test/java/org/apache/druid/data/input/impl/JsonLineReaderTest.java b/processing/src/test/java/org/apache/druid/data/input/impl/JsonLineReaderTest.java
index 5b0a3a391a..dea2efb019 100644
--- a/processing/src/test/java/org/apache/druid/data/input/impl/JsonLineReaderTest.java
+++ b/processing/src/test/java/org/apache/druid/data/input/impl/JsonLineReaderTest.java
@@ -188,7 +188,7 @@ public class JsonLineReaderTest
       int numActualIterations = 0;
       while (iterator.hasNext()) {
         final InputRow row = iterator.next();
-        Assert.assertEquals(Arrays.asList("path_omg", "tree_omg", "timestamp", "bar", "foo"), row.getDimensions());
+        Assert.assertEquals(Arrays.asList("path_omg", "tree_omg", "bar", "foo"), row.getDimensions());
         Assert.assertTrue(row.getDimension("bar").isEmpty());
         Assert.assertEquals("x", Iterables.getOnlyElement(row.getDimension("foo")));
         Assert.assertTrue(row.getDimension("path_omg").isEmpty());
@@ -234,7 +234,7 @@ public class JsonLineReaderTest
       int numActualIterations = 0;
       while (iterator.hasNext()) {
         final InputRow row = iterator.next();
-        Assert.assertEquals(Arrays.asList("path_omg", "tree_omg", "timestamp", "bar", "foo"), row.getDimensions());
+        Assert.assertEquals(Arrays.asList("path_omg", "tree_omg", "bar", "foo"), row.getDimensions());
         Assert.assertEquals("1", Iterables.getOnlyElement(row.getDimension("bar")));
         Assert.assertEquals("x", Iterables.getOnlyElement(row.getDimension("foo")));
         Assert.assertEquals("a", Iterables.getOnlyElement(row.getDimension("path_omg")));
@@ -280,7 +280,7 @@ public class JsonLineReaderTest
       int numActualIterations = 0;
       while (iterator.hasNext()) {
         final InputRow row = iterator.next();
-        Assert.assertEquals(Arrays.asList("path_omg", "tree_omg", "timestamp", "foo"), row.getDimensions());
+        Assert.assertEquals(Arrays.asList("path_omg", "tree_omg", "foo"), row.getDimensions());
         Assert.assertTrue(row.getDimension("bar").isEmpty());
         Assert.assertEquals("x", Iterables.getOnlyElement(row.getDimension("foo")));
         Assert.assertEquals("a", Iterables.getOnlyElement(row.getDimension("path_omg")));
diff --git a/processing/src/test/java/org/apache/druid/math/expr/EvalTest.java b/processing/src/test/java/org/apache/druid/math/expr/EvalTest.java
index 8ec8929478..a2ddb78dc3 100644
--- a/processing/src/test/java/org/apache/druid/math/expr/EvalTest.java
+++ b/processing/src/test/java/org/apache/druid/math/expr/EvalTest.java
@@ -102,7 +102,7 @@ public class EvalTest extends InitializedNullHandlingTest
       Assert.assertFalse(evalDouble("!2.0", bindings) > 0.0);
     }
     finally {
-      ExpressionProcessing.initializeForTests(null);
+      ExpressionProcessing.initializeForTests();
     }
     try {
       ExpressionProcessing.initializeForStrictBooleansTests(true);
@@ -137,7 +137,7 @@ public class EvalTest extends InitializedNullHandlingTest
       assertEquals(3.0, evalDouble("if(0.0, 2.0, 3.0)", bindings), 0.0001);
     }
     finally {
-      ExpressionProcessing.initializeForTests(null);
+      ExpressionProcessing.initializeForTests();
     }
   }
 
@@ -435,7 +435,7 @@ public class EvalTest extends InitializedNullHandlingTest
       assertEquals(ExpressionType.DOUBLE, eval.type());
     }
     finally {
-      ExpressionProcessing.initializeForTests(null);
+      ExpressionProcessing.initializeForTests();
     }
     try {
       ExpressionProcessing.initializeForStrictBooleansTests(true);
@@ -464,7 +464,7 @@ public class EvalTest extends InitializedNullHandlingTest
       assertEquals(ExpressionType.LONG, eval.type());
     }
     finally {
-      ExpressionProcessing.initializeForTests(null);
+      ExpressionProcessing.initializeForTests();
     }
   }
 
@@ -532,7 +532,7 @@ public class EvalTest extends InitializedNullHandlingTest
     }
     finally {
       // reset
-      ExpressionProcessing.initializeForTests(null);
+      ExpressionProcessing.initializeForTests();
     }
 
     try {
@@ -607,7 +607,7 @@ public class EvalTest extends InitializedNullHandlingTest
     }
     finally {
       // reset
-      ExpressionProcessing.initializeForTests(null);
+      ExpressionProcessing.initializeForTests();
     }
   }
 
@@ -664,7 +664,7 @@ public class EvalTest extends InitializedNullHandlingTest
     }
     finally {
       // reset
-      ExpressionProcessing.initializeForTests(null);
+      ExpressionProcessing.initializeForTests();
     }
 
     try {
@@ -703,7 +703,7 @@ public class EvalTest extends InitializedNullHandlingTest
     }
     finally {
       // reset
-      ExpressionProcessing.initializeForTests(null);
+      ExpressionProcessing.initializeForTests();
     }
   }
 
@@ -948,103 +948,95 @@ public class EvalTest extends InitializedNullHandlingTest
     Assert.assertArrayEquals(new Object[] {"1.0", "2", "3", "true", "false"}, (Object[]) eval.value());
 
     // nested arrays
-    try {
-      ExpressionProcessing.initializeForTests(true);
-
-      ExpressionType nestedLongArray = ExpressionTypeFactory.getInstance().ofArray(ExpressionType.LONG_ARRAY);
-      final Object[] expectedLongArray = new Object[]{
-          new Object[] {1L, 2L, 3L},
-          new Object[] {5L, null, 9L},
-          null,
-          new Object[] {2L, 4L, 6L}
-      };
-
-      List<?> longArrayInputs = Arrays.asList(
-          new Object[]{
-              new Object[] {1L, 2L, 3L},
-              new Object[] {5L, null, 9L},
-              null,
-              new Object[] {2L, 4L, 6L}
-          },
-          Arrays.asList(
-              new Object[] {1L, 2L, 3L},
-              new Object[] {5L, null, 9L},
-              null,
-              new Object[] {2L, 4L, 6L}
-          ),
-          Arrays.asList(
-              Arrays.asList(1L, 2L, 3L),
-              Arrays.asList(5L, null, 9L),
-              null,
-              Arrays.asList(2L, 4L, 6L)
-          ),
-          Arrays.asList(
-              Arrays.asList(1L, 2L, 3L),
-              Arrays.asList("5", "hello", "9"),
-              null,
-              new Object[]{2.2, 4.4, 6.6}
-          )
-      );
-
-      for (Object o : longArrayInputs) {
-        eval = ExprEval.ofType(nestedLongArray, o);
-        Assert.assertEquals(nestedLongArray, eval.type());
-        Object[] val = (Object[]) eval.value();
-        Assert.assertEquals(expectedLongArray.length, val.length);
-        for (int i = 0; i < expectedLongArray.length; i++) {
-          Assert.assertArrayEquals((Object[]) expectedLongArray[i], (Object[]) val[i]);
-        }
+    ExpressionType nestedLongArray = ExpressionTypeFactory.getInstance().ofArray(ExpressionType.LONG_ARRAY);
+    final Object[] expectedLongArray = new Object[]{
+        new Object[] {1L, 2L, 3L},
+        new Object[] {5L, null, 9L},
+        null,
+        new Object[] {2L, 4L, 6L}
+    };
+
+    List<?> longArrayInputs = Arrays.asList(
+        new Object[]{
+            new Object[] {1L, 2L, 3L},
+            new Object[] {5L, null, 9L},
+            null,
+            new Object[] {2L, 4L, 6L}
+        },
+        Arrays.asList(
+            new Object[] {1L, 2L, 3L},
+            new Object[] {5L, null, 9L},
+            null,
+            new Object[] {2L, 4L, 6L}
+        ),
+        Arrays.asList(
+            Arrays.asList(1L, 2L, 3L),
+            Arrays.asList(5L, null, 9L),
+            null,
+            Arrays.asList(2L, 4L, 6L)
+        ),
+        Arrays.asList(
+            Arrays.asList(1L, 2L, 3L),
+            Arrays.asList("5", "hello", "9"),
+            null,
+            new Object[]{2.2, 4.4, 6.6}
+        )
+    );
+
+    for (Object o : longArrayInputs) {
+      eval = ExprEval.ofType(nestedLongArray, o);
+      Assert.assertEquals(nestedLongArray, eval.type());
+      Object[] val = (Object[]) eval.value();
+      Assert.assertEquals(expectedLongArray.length, val.length);
+      for (int i = 0; i < expectedLongArray.length; i++) {
+        Assert.assertArrayEquals((Object[]) expectedLongArray[i], (Object[]) val[i]);
       }
+    }
 
-      ExpressionType nestedDoubleArray = ExpressionTypeFactory.getInstance().ofArray(ExpressionType.DOUBLE_ARRAY);
-      final Object[] expectedDoubleArray = new Object[]{
-          new Object[] {1.1, 2.2, 3.3},
-          new Object[] {5.5, null, 9.9},
-          null,
-          new Object[] {2.2, 4.4, 6.6}
-      };
-
-      List<?> doubleArrayInputs = Arrays.asList(
-          new Object[]{
-              new Object[] {1.1, 2.2, 3.3},
-              new Object[] {5.5, null, 9.9},
-              null,
-              new Object[] {2.2, 4.4, 6.6}
-          },
-          new Object[]{
-              Arrays.asList(1.1, 2.2, 3.3),
-              Arrays.asList(5.5, null, 9.9),
-              null,
-              Arrays.asList(2.2, 4.4, 6.6)
-          },
-          Arrays.asList(
-              Arrays.asList(1.1, 2.2, 3.3),
-              Arrays.asList(5.5, null, 9.9),
-              null,
-              Arrays.asList(2.2, 4.4, 6.6)
-          ),
-          new Object[]{
-              new Object[] {"1.1", "2.2", "3.3"},
-              Arrays.asList("5.5", null, "9.9"),
-              null,
-              new String[] {"2.2", "4.4", "6.6"}
-          }
-      );
-
-      for (Object o : doubleArrayInputs) {
-        eval = ExprEval.ofType(nestedDoubleArray, o);
-        Assert.assertEquals(nestedDoubleArray, eval.type());
-        Object[] val = (Object[]) eval.value();
-        Assert.assertEquals(expectedLongArray.length, val.length);
-        for (int i = 0; i < expectedLongArray.length; i++) {
-          Assert.assertArrayEquals((Object[]) expectedDoubleArray[i], (Object[]) val[i]);
+    ExpressionType nestedDoubleArray = ExpressionTypeFactory.getInstance().ofArray(ExpressionType.DOUBLE_ARRAY);
+    final Object[] expectedDoubleArray = new Object[]{
+        new Object[] {1.1, 2.2, 3.3},
+        new Object[] {5.5, null, 9.9},
+        null,
+        new Object[] {2.2, 4.4, 6.6}
+    };
+
+    List<?> doubleArrayInputs = Arrays.asList(
+        new Object[]{
+            new Object[] {1.1, 2.2, 3.3},
+            new Object[] {5.5, null, 9.9},
+            null,
+            new Object[] {2.2, 4.4, 6.6}
+        },
+        new Object[]{
+            Arrays.asList(1.1, 2.2, 3.3),
+            Arrays.asList(5.5, null, 9.9),
+            null,
+            Arrays.asList(2.2, 4.4, 6.6)
+        },
+        Arrays.asList(
+            Arrays.asList(1.1, 2.2, 3.3),
+            Arrays.asList(5.5, null, 9.9),
+            null,
+            Arrays.asList(2.2, 4.4, 6.6)
+        ),
+        new Object[]{
+            new Object[] {"1.1", "2.2", "3.3"},
+            Arrays.asList("5.5", null, "9.9"),
+            null,
+            new String[] {"2.2", "4.4", "6.6"}
         }
+    );
+
+    for (Object o : doubleArrayInputs) {
+      eval = ExprEval.ofType(nestedDoubleArray, o);
+      Assert.assertEquals(nestedDoubleArray, eval.type());
+      Object[] val = (Object[]) eval.value();
+      Assert.assertEquals(expectedLongArray.length, val.length);
+      for (int i = 0; i < expectedLongArray.length; i++) {
+        Assert.assertArrayEquals((Object[]) expectedDoubleArray[i], (Object[]) val[i]);
       }
     }
-    finally {
-      // reset
-      ExpressionProcessing.initializeForTests(null);
-    }
   }
 
   @Test
@@ -1073,7 +1065,7 @@ public class EvalTest extends InitializedNullHandlingTest
     }
     finally {
       // reset
-      ExpressionProcessing.initializeForTests(null);
+      ExpressionProcessing.initializeForTests();
     }
 
     // doubles
diff --git a/processing/src/test/java/org/apache/druid/math/expr/OutputTypeTest.java b/processing/src/test/java/org/apache/druid/math/expr/OutputTypeTest.java
index e2f14dd712..00682b9632 100644
--- a/processing/src/test/java/org/apache/druid/math/expr/OutputTypeTest.java
+++ b/processing/src/test/java/org/apache/druid/math/expr/OutputTypeTest.java
@@ -86,7 +86,7 @@ public class OutputTypeTest extends InitializedNullHandlingTest
     }
     finally {
       // reset
-      ExpressionProcessing.initializeForTests(null);
+      ExpressionProcessing.initializeForTests();
     }
 
     try {
@@ -96,7 +96,7 @@ public class OutputTypeTest extends InitializedNullHandlingTest
     }
     finally {
       // reset
-      ExpressionProcessing.initializeForTests(null);
+      ExpressionProcessing.initializeForTests();
     }
   }
 
@@ -161,7 +161,7 @@ public class OutputTypeTest extends InitializedNullHandlingTest
       assertOutputType("z_ || z", inspector, ExpressionType.LONG);
     }
     finally {
-      ExpressionProcessing.initializeForTests(null);
+      ExpressionProcessing.initializeForTests();
     }
     try {
       ExpressionProcessing.initializeForStrictBooleansTests(false);
@@ -184,7 +184,7 @@ public class OutputTypeTest extends InitializedNullHandlingTest
       assertOutputType("z_ || z", inspector, ExpressionType.DOUBLE);
     }
     finally {
-      ExpressionProcessing.initializeForTests(null);
+      ExpressionProcessing.initializeForTests();
     }
     assertOutputType("1*(2 + 3.0)", inspector, ExpressionType.DOUBLE);
   }
diff --git a/processing/src/test/java/org/apache/druid/math/expr/ParserTest.java b/processing/src/test/java/org/apache/druid/math/expr/ParserTest.java
index 87cc6f1b64..f6f5eccd10 100644
--- a/processing/src/test/java/org/apache/druid/math/expr/ParserTest.java
+++ b/processing/src/test/java/org/apache/druid/math/expr/ParserTest.java
@@ -21,7 +21,6 @@ package org.apache.druid.math.expr;
 
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.ImmutableSet;
-import org.apache.druid.java.util.common.IAE;
 import org.apache.druid.java.util.common.RE;
 import org.apache.druid.java.util.common.StringUtils;
 import org.apache.druid.segment.column.TypeStrategies;
@@ -353,36 +352,28 @@ public class ParserTest extends InitializedNullHandlingTest
   @Test
   public void testLiteralExplicitTypedArrays()
   {
-    ExpressionProcessing.initializeForTests(true);
+    validateConstantExpression("ARRAY<DOUBLE>[1.0, 2.0, null, 3.0]", new Object[]{1.0, 2.0, null, 3.0});
+    validateConstantExpression("ARRAY<LONG>[1, 2, null, 3]", new Object[]{1L, 2L, null, 3L});
+    validateConstantExpression("ARRAY<STRING>['1', '2', null, '3.0']", new Object[]{"1", "2", null, "3.0"});
 
-    try {
-      validateConstantExpression("ARRAY<DOUBLE>[1.0, 2.0, null, 3.0]", new Object[]{1.0, 2.0, null, 3.0});
-      validateConstantExpression("ARRAY<LONG>[1, 2, null, 3]", new Object[]{1L, 2L, null, 3L});
-      validateConstantExpression("ARRAY<STRING>['1', '2', null, '3.0']", new Object[]{"1", "2", null, "3.0"});
-
-      // mixed type tests
-      validateConstantExpression("ARRAY<DOUBLE>[3, null, 4, 2.345]", new Object[]{3.0, null, 4.0, 2.345});
-      validateConstantExpression("ARRAY<LONG>[1.0, null, 2000.0]", new Object[]{1L, null, 2000L});
+    // mixed type tests
+    validateConstantExpression("ARRAY<DOUBLE>[3, null, 4, 2.345]", new Object[]{3.0, null, 4.0, 2.345});
+    validateConstantExpression("ARRAY<LONG>[1.0, null, 2000.0]", new Object[]{1L, null, 2000L});
 
-      // explicit typed string arrays should accept any literal and convert
-      validateConstantExpression("ARRAY<STRING>['1', null, 2000, 1.1]", new Object[]{"1", null, "2000", "1.1"});
-      validateConstantExpression("ARRAY<LONG>['1', null, 2000, 1.1]", new Object[]{1L, null, 2000L, 1L});
-      validateConstantExpression("ARRAY<DOUBLE>['1', null, 2000, 1.1]", new Object[]{1.0, null, 2000.0, 1.1});
+    // explicit typed string arrays should accept any literal and convert
+    validateConstantExpression("ARRAY<STRING>['1', null, 2000, 1.1]", new Object[]{"1", null, "2000", "1.1"});
+    validateConstantExpression("ARRAY<LONG>['1', null, 2000, 1.1]", new Object[]{1L, null, 2000L, 1L});
+    validateConstantExpression("ARRAY<DOUBLE>['1', null, 2000, 1.1]", new Object[]{1.0, null, 2000.0, 1.1});
 
-      // the gramar isn't cool enough yet to parse populated nested-arrays or complex arrays..., but empty ones can
-      // be defined...
-      validateConstantExpression("ARRAY<COMPLEX<nullableLongPair>>[]", new Object[]{});
-      validateConstantExpression("ARRAY<ARRAY<LONG>>[]", new Object[]{});
-    }
-    finally {
-      ExpressionProcessing.initializeForTests(null);
-    }
+    // the gramar isn't cool enough yet to parse populated nested-arrays or complex arrays..., but empty ones can
+    // be defined...
+    validateConstantExpression("ARRAY<COMPLEX<nullableLongPair>>[]", new Object[]{});
+    validateConstantExpression("ARRAY<ARRAY<LONG>>[]", new Object[]{});
   }
 
   @Test
   public void testConstantComplexAndNestedArrays()
   {
-    ExpressionProcessing.initializeForTests(true);
     // they can be built with array builder functions though...
     validateConstantExpression(
         "array(['foo', 'bar', 'baz'], ['baz','foo','bar'])",
@@ -426,18 +417,8 @@ public class ParserTest extends InitializedNullHandlingTest
         StringUtils.format("array(%s,%s)", l1String, l2String),
         new Object[]{l1, l2}
     );
-    ExpressionProcessing.initializeForTests(null);
   }
 
-  @Test
-  public void nestedArraysExplodeIfNotEnabled()
-  {
-    expectedException.expect(IAE.class);
-    expectedException.expectMessage("Cannot create a nested array type [ARRAY<ARRAY<LONG>>], 'druid.expressions.allowNestedArrays' must be set to true");
-    validateConstantExpression("ARRAY<ARRAY<LONG>>[]", new Object[]{});
-  }
-
-
   @Test
   public void testLiteralArrayImplicitStringParseException()
   {
diff --git a/processing/src/test/java/org/apache/druid/math/expr/VectorExprSanityTest.java b/processing/src/test/java/org/apache/druid/math/expr/VectorExprSanityTest.java
index 2e20a486fa..a8724db8a0 100644
--- a/processing/src/test/java/org/apache/druid/math/expr/VectorExprSanityTest.java
+++ b/processing/src/test/java/org/apache/druid/math/expr/VectorExprSanityTest.java
@@ -73,7 +73,7 @@ public class VectorExprSanityTest extends InitializedNullHandlingTest
   @AfterClass
   public static void teardownTests()
   {
-    ExpressionProcessing.initializeForTests(null);
+    ExpressionProcessing.initializeForTests();
   }
 
   @Test
diff --git a/processing/src/test/java/org/apache/druid/query/MultiValuedDimensionTest.java b/processing/src/test/java/org/apache/druid/query/MultiValuedDimensionTest.java
index 152e7c908a..dafa7ff9a8 100644
--- a/processing/src/test/java/org/apache/druid/query/MultiValuedDimensionTest.java
+++ b/processing/src/test/java/org/apache/druid/query/MultiValuedDimensionTest.java
@@ -550,7 +550,7 @@ public class MultiValuedDimensionTest extends InitializedNullHandlingTest
       TestHelper.assertExpectedObjects(expectedResults, result.toList(), "expr-multi-multi");
     }
     finally {
-      ExpressionProcessing.initializeForTests(null);
+      ExpressionProcessing.initializeForTests();
     }
   }
 
diff --git a/processing/src/test/java/org/apache/druid/query/NestedDataTestUtils.java b/processing/src/test/java/org/apache/druid/query/NestedDataTestUtils.java
index d75f92e7f1..8e6f3485d4 100644
--- a/processing/src/test/java/org/apache/druid/query/NestedDataTestUtils.java
+++ b/processing/src/test/java/org/apache/druid/query/NestedDataTestUtils.java
@@ -19,432 +19,456 @@
 
 package org.apache.druid.query;
 
-import com.fasterxml.jackson.core.type.TypeReference;
 import com.fasterxml.jackson.databind.ObjectMapper;
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.Lists;
-import com.google.common.io.Files;
-import org.apache.commons.io.IOUtils;
-import org.apache.commons.io.LineIterator;
-import org.apache.druid.data.input.impl.StringInputRowParser;
+import org.apache.druid.data.input.InputFormat;
+import org.apache.druid.data.input.InputRowSchema;
+import org.apache.druid.data.input.InputSource;
+import org.apache.druid.data.input.ResourceInputSource;
+import org.apache.druid.data.input.impl.DelimitedInputFormat;
+import org.apache.druid.data.input.impl.DimensionsSpec;
+import org.apache.druid.data.input.impl.JsonInputFormat;
+import org.apache.druid.data.input.impl.LocalInputSource;
+import org.apache.druid.data.input.impl.StringDimensionSchema;
+import org.apache.druid.data.input.impl.TimestampSpec;
 import org.apache.druid.guice.NestedDataModule;
 import org.apache.druid.java.util.common.StringUtils;
 import org.apache.druid.java.util.common.granularity.Granularities;
 import org.apache.druid.java.util.common.granularity.Granularity;
-import org.apache.druid.java.util.common.guava.nary.TrinaryFn;
 import org.apache.druid.java.util.common.io.Closer;
-import org.apache.druid.query.aggregation.AggregationTestHelper;
+import org.apache.druid.java.util.common.parsers.JSONPathSpec;
 import org.apache.druid.query.aggregation.AggregatorFactory;
+import org.apache.druid.query.aggregation.CountAggregatorFactory;
+import org.apache.druid.query.expression.TestExprMacroTable;
 import org.apache.druid.segment.IncrementalIndexSegment;
+import org.apache.druid.segment.IndexBuilder;
+import org.apache.druid.segment.NestedDataDimensionSchema;
 import org.apache.druid.segment.QueryableIndexSegment;
 import org.apache.druid.segment.Segment;
 import org.apache.druid.segment.TestHelper;
-import org.apache.druid.segment.incremental.IncrementalIndex;
+import org.apache.druid.segment.incremental.IncrementalIndexSchema;
+import org.apache.druid.segment.transform.ExpressionTransform;
+import org.apache.druid.segment.transform.TransformSpec;
 import org.apache.druid.timeline.SegmentId;
 import org.junit.rules.TemporaryFolder;
 
 import java.io.ByteArrayInputStream;
 import java.io.File;
-import java.io.FileInputStream;
 import java.io.IOException;
 import java.io.InputStream;
+import java.io.OutputStream;
 import java.io.SequenceInputStream;
-import java.nio.charset.StandardCharsets;
+import java.nio.file.Files;
 import java.util.ArrayList;
+import java.util.Arrays;
 import java.util.Collections;
 import java.util.List;
+import java.util.function.BiFunction;
 
 public class NestedDataTestUtils
 {
-  public static final String SIMPLE_DATA_FILE = "simple-nested-test-data.json";
-  public static final String SIMPLE_PARSER_FILE = "simple-nested-test-data-parser.json";
-  public static final String SIMPLE_DATA_TSV_FILE = "simple-nested-test-data.tsv";
-  public static final String SIMPLE_PARSER_TSV_FILE = "simple-nested-test-data-tsv-parser.json";
-  public static final String SIMPLE_PARSER_TSV_TRANSFORM_FILE = "simple-nested-test-data-tsv-transform.json";
-  public static final String SIMPLE_AGG_FILE = "simple-nested-test-data-aggs.json";
-
-  public static final String TYPES_DATA_FILE = "types-test-data.json";
-  public static final String TYPES_PARSER_FILE = "types-test-data-parser.json";
-
-  public static final String NUMERIC_DATA_FILE = "numeric-nested-test-data.json";
-  public static final String NUMERIC_PARSER_FILE = "numeric-nested-test-data-parser.json";
+  public static final String SIMPLE_DATA_FILE = "nested-simple-test-data.json";
+  public static final String SIMPLE_DATA_TSV_FILE = "nested-simple-test-data.tsv";
+  public static final String NUMERIC_DATA_FILE = "nested-numeric-test-data.json";
+  public static final String TYPES_DATA_FILE = "nested-types-test-data.json";
+  public static final String ARRAY_TYPES_DATA_FILE = "nested-array-test-data.json";
 
   public static final ObjectMapper JSON_MAPPER;
 
+  public static final TimestampSpec TIMESTAMP_SPEC = new TimestampSpec("timestamp", null, null);
+
+  public static final JsonInputFormat DEFAULT_JSON_INPUT_FORMAT = new JsonInputFormat(
+      JSONPathSpec.DEFAULT,
+      null,
+      null,
+      null,
+      null
+  );
+
+  public static final DimensionsSpec AUTO_DISCOVERY =
+      DimensionsSpec.builder()
+                    .useSchemaDiscovery(true)
+                    .build();
+
+  public static final DimensionsSpec TSV_SCHEMA =
+      DimensionsSpec.builder()
+                    .setDimensions(
+                        Arrays.asList(
+                            new StringDimensionSchema("dim"),
+                            new NestedDataDimensionSchema("nest_json"),
+                            new NestedDataDimensionSchema("nester_json"),
+                            new NestedDataDimensionSchema("variant_json"),
+                            new NestedDataDimensionSchema("list_json")
+                        )
+                    )
+                    .build();
+  public static final InputRowSchema AUTO_SCHEMA = new InputRowSchema(
+      TIMESTAMP_SPEC,
+      AUTO_DISCOVERY,
+      null
+  );
+
+  public static final InputRowSchema SIMPLE_DATA_TSV_SCHEMA = new InputRowSchema(
+      TIMESTAMP_SPEC,
+      TSV_SCHEMA,
+      null
+  );
+
+  public static DelimitedInputFormat SIMPLE_DATA_TSV_INPUT_FORMAT = new DelimitedInputFormat(
+      Arrays.asList(
+          "timestamp",
+          "dim",
+          "nest",
+          "nester",
+          "variant",
+          "list"
+      ),
+      null,
+      null,
+      false,
+      false,
+      0
+  );
+
+  public static final TransformSpec SIMPLE_DATA_TSV_TRANSFORM = new TransformSpec(
+      null,
+      Arrays.asList(
+          new ExpressionTransform("nest_json", "parse_json(nest)", TestExprMacroTable.INSTANCE),
+          new ExpressionTransform("nester_json", "parse_json(nester)", TestExprMacroTable.INSTANCE),
+          new ExpressionTransform("variant_json", "parse_json(variant)", TestExprMacroTable.INSTANCE),
+          new ExpressionTransform("list_json", "parse_json(list)", TestExprMacroTable.INSTANCE)
+      )
+  );
+
+  public static final AggregatorFactory[] COUNT = new AggregatorFactory[]{
+      new CountAggregatorFactory("count")
+  };
+
   static {
     JSON_MAPPER = TestHelper.makeJsonMapper();
     JSON_MAPPER.registerModules(NestedDataModule.getJacksonModulesList());
   }
 
-  public static List<Segment> createSegments(
-      AggregationTestHelper helper,
+  public static List<Segment> createSimpleSegmentsTsv(
       TemporaryFolder tempFolder,
-      Closer closer,
-      Granularity granularity,
-      boolean rollup,
-      int maxRowCount
-  ) throws Exception
+      Closer closer
+  )
+      throws Exception
   {
-    return createSegments(
-        helper,
+    return createSimpleNestedTestDataTsvSegments(
         tempFolder,
         closer,
-        SIMPLE_DATA_FILE,
-        SIMPLE_PARSER_FILE,
-        SIMPLE_AGG_FILE,
-        granularity,
-        rollup,
-        maxRowCount
+        Granularities.NONE,
+        true
     );
   }
 
-  public static List<Segment> createTsvSegments(
-      AggregationTestHelper helper,
+  public static List<Segment> createSimpleNestedTestDataTsvSegments(
       TemporaryFolder tempFolder,
       Closer closer,
       Granularity granularity,
-      boolean rollup,
-      int maxRowCount
+      boolean rollup
   ) throws Exception
   {
     return createSegments(
-        helper,
         tempFolder,
         closer,
         SIMPLE_DATA_TSV_FILE,
-        SIMPLE_PARSER_TSV_FILE,
-        SIMPLE_PARSER_TSV_TRANSFORM_FILE,
-        SIMPLE_AGG_FILE,
+        SIMPLE_DATA_TSV_INPUT_FORMAT,
+        TIMESTAMP_SPEC,
+        SIMPLE_DATA_TSV_SCHEMA.getDimensionsSpec(),
+        SIMPLE_DATA_TSV_TRANSFORM,
+        COUNT,
         granularity,
-        rollup,
-        maxRowCount
+        rollup
     );
   }
 
-  public static Segment createIncrementalIndex(
-      Granularity granularity,
-      boolean rollup,
-      boolean deserializeComplexMetrics,
-      int maxRowCount
-  )
-      throws Exception
+  public static Segment createSimpleNestedTestDataIncrementalIndex(TemporaryFolder tempFolder) throws Exception
   {
-    return createIncrementalIndex(
+    return createIncrementalIndexForJsonInput(
+        tempFolder,
         SIMPLE_DATA_FILE,
-        SIMPLE_PARSER_FILE,
-        SIMPLE_AGG_FILE,
-        granularity,
-        rollup,
-        deserializeComplexMetrics,
-        maxRowCount
+        Granularities.NONE,
+        true
     );
   }
 
-  public static List<Segment> createSegments(
-      AggregationTestHelper helper,
+  public static List<Segment> createSimpleNestedTestDataSegments(
       TemporaryFolder tempFolder,
-      Closer closer,
-      String inputFileName,
-      String parserJsonFileName,
-      String aggJsonFileName,
-      Granularity granularity,
-      boolean rollup,
-      int maxRowCount
-  ) throws Exception
+      Closer closer
+  )
+      throws Exception
   {
-    File segmentDir = tempFolder.newFolder();
-    File inputFile = readFileFromClasspath(inputFileName);
-    FileInputStream inputDataStream = new FileInputStream(inputFile);
-    String parserJson = readFileFromClasspathAsString(parserJsonFileName);
-    String aggJson = readFileFromClasspathAsString(aggJsonFileName);
-
-    helper.createIndex(
-        inputDataStream,
-        parserJson,
-        aggJson,
-        segmentDir,
-        0,
-        granularity,
-        maxRowCount,
-        rollup
-    );
-    inputDataStream.close();
-
-    final List<Segment> segments = Lists.transform(
-        ImmutableList.of(segmentDir),
-        dir -> {
-          try {
-            return closer.register(new QueryableIndexSegment(helper.getIndexIO().loadIndex(dir), SegmentId.dummy("")));
-          }
-          catch (IOException ex) {
-            throw new RuntimeException(ex);
-          }
-        }
+    return createSegmentsForJsonInput(
+        tempFolder,
+        closer,
+        SIMPLE_DATA_FILE,
+        Granularities.NONE,
+        true
     );
+  }
 
-    return segments;
+  public static Segment createIncrementalIndexForJsonInput(TemporaryFolder tempFolder, String fileName)
+      throws Exception
+  {
+    return createIncrementalIndexForJsonInput(
+        tempFolder,
+        fileName,
+        Granularities.NONE,
+        true
+    );
   }
 
-  public static List<Segment> createSegments(
-      AggregationTestHelper helper,
+  public static Segment createIncrementalIndexForJsonInput(
       TemporaryFolder tempFolder,
-      Closer closer,
-      String inputFileName,
-      String parserJsonFileName,
-      String transformSpecJsonFileName,
-      String aggJsonFileName,
+      String file,
       Granularity granularity,
-      boolean rollup,
-      int maxRowCount
-  ) throws Exception
+      boolean rollup
+  )
+      throws Exception
   {
-    File segmentDir = tempFolder.newFolder();
-    File inputFile = readFileFromClasspath(inputFileName);
-    FileInputStream inputDataStream = new FileInputStream(inputFile);
-    String parserJson = readFileFromClasspathAsString(parserJsonFileName);
-    String transformSpecJson = readFileFromClasspathAsString(transformSpecJsonFileName);
-    String aggJson = readFileFromClasspathAsString(aggJsonFileName);
-
-    helper.createIndex(
-        inputDataStream,
-        parserJson,
-        transformSpecJson,
-        aggJson,
-        segmentDir,
-        0,
+    return createIncrementalIndex(
+        tempFolder,
+        file,
+        DEFAULT_JSON_INPUT_FORMAT,
+        TIMESTAMP_SPEC,
+        AUTO_DISCOVERY,
+        TransformSpec.NONE,
+        COUNT,
         granularity,
-        maxRowCount,
         rollup
     );
-    inputDataStream.close();
-
-    final List<Segment> segments = Lists.transform(
-        ImmutableList.of(segmentDir),
-        dir -> {
-          try {
-            return closer.register(new QueryableIndexSegment(helper.getIndexIO().loadIndex(dir), SegmentId.dummy("")));
-          }
-          catch (IOException ex) {
-            throw new RuntimeException(ex);
-          }
-        }
-    );
-
-    return segments;
   }
 
-  public static List<Segment> createSegmentsWithConcatenatedInput(
-      AggregationTestHelper helper,
+  public static List<Segment> createSegmentsForJsonInput(
       TemporaryFolder tempFolder,
       Closer closer,
+      String inputFile,
       Granularity granularity,
-      boolean rollup,
-      int maxRowCount,
-      int numCopies,
-      int numSegments
+      boolean rollup
   ) throws Exception
   {
-    return createSegmentsWithConcatenatedInput(
-        helper,
+    return createSegments(
         tempFolder,
         closer,
-        SIMPLE_DATA_FILE,
-        SIMPLE_PARSER_FILE,
-        null,
-        SIMPLE_AGG_FILE,
+        inputFile,
+        DEFAULT_JSON_INPUT_FORMAT,
+        TIMESTAMP_SPEC,
+        AUTO_DISCOVERY,
+        TransformSpec.NONE,
+        COUNT,
         granularity,
-        rollup,
-        maxRowCount,
-        numCopies,
-        numSegments
+        rollup
     );
   }
 
-  /**
-   * turn small test data into bigger test data by duplicating itself into a bigger stream
-   */
-  public static List<Segment> createSegmentsWithConcatenatedInput(
-      AggregationTestHelper helper,
+  public static List<Segment> createSegmentsWithConcatenatedJsonInput(
       TemporaryFolder tempFolder,
       Closer closer,
-      String inputFileName,
-      String parserJsonFileName,
-      String transformSpecJsonFileName,
-      String aggJsonFileName,
+      String inputFile,
       Granularity granularity,
       boolean rollup,
-      int maxRowCount,
       int numCopies,
       int numSegments
   ) throws Exception
   {
-    String parserJson = readFileFromClasspathAsString(parserJsonFileName);
-    String transformSpecJson = transformSpecJsonFileName != null ? readFileFromClasspathAsString(transformSpecJsonFileName) : null;
-    String aggJson = readFileFromClasspathAsString(aggJsonFileName);
-
-    List<File> segmentDirs = Lists.newArrayListWithCapacity(numSegments);
+    List<InputSource> inputFiles = Lists.newArrayListWithCapacity(numSegments);
     for (int i = 0; i < numSegments; i++) {
-      List<InputStream> inputStreams = Lists.newArrayListWithCapacity(numCopies);
-      for (int j = 0; j < numCopies; j++) {
-        inputStreams.add(new FileInputStream(readFileFromClasspath(inputFileName)));
-        if (j + 1 < numCopies) {
-          inputStreams.add(new ByteArrayInputStream(StringUtils.toUtf8("\n")));
-        }
-      }
-      SequenceInputStream inputDataStream = new SequenceInputStream(Collections.enumeration(inputStreams));
-      File segmentDir = tempFolder.newFolder();
-      helper.createIndex(
-          inputDataStream,
-          parserJson,
-          transformSpecJson,
-          aggJson,
-          segmentDir,
-          0,
-          granularity,
-          maxRowCount,
-          rollup
-      );
-      inputDataStream.close();
-      segmentDirs.add(segmentDir);
+      File file = selfConcatenateResourceFile(tempFolder, inputFile, numCopies);
+      inputFiles.add(new LocalInputSource(file.getParentFile(), file.getName()));
     }
-
-    final List<Segment> segments = Lists.transform(
-        segmentDirs,
-        dir -> {
-          try {
-            return closer.register(new QueryableIndexSegment(helper.getIndexIO().loadIndex(dir), SegmentId.dummy("")));
-          }
-          catch (IOException ex) {
-            throw new RuntimeException(ex);
-          }
-        }
-    );
-
-    return segments;
-  }
-
-  public static Segment createIncrementalIndex(
-      String inputFileName,
-      String parserJsonFileName,
-      String aggJsonFileName,
-      Granularity granularity,
-      boolean rollup,
-      boolean deserializeComplexMetrics,
-      int maxRowCount
-  )
-      throws Exception
-  {
-    File inputFile = readFileFromClasspath(inputFileName);
-    FileInputStream inputDataStream = new FileInputStream(inputFile);
-    String parserJson = readFileFromClasspathAsString(parserJsonFileName);
-    String aggJson = readFileFromClasspathAsString(aggJsonFileName);
-    StringInputRowParser parser = JSON_MAPPER.readValue(parserJson, StringInputRowParser.class);
-
-    LineIterator iter = IOUtils.lineIterator(inputDataStream, "UTF-8");
-    List<AggregatorFactory> aggregatorSpecs = JSON_MAPPER.readValue(
-        aggJson,
-        new TypeReference<List<AggregatorFactory>>()
-        {
-        }
-    );
-    IncrementalIndex index = AggregationTestHelper.createIncrementalIndex(
-        iter,
-        parser,
-        parser.getParseSpec().getDimensionsSpec().getDimensions(),
-        aggregatorSpecs.toArray(new AggregatorFactory[0]),
-        0,
+    return createSegments(
+        tempFolder,
+        closer,
+        inputFiles,
+        DEFAULT_JSON_INPUT_FORMAT,
+        TIMESTAMP_SPEC,
+        AUTO_DISCOVERY,
+        TransformSpec.NONE,
+        COUNT,
         granularity,
-        deserializeComplexMetrics,
-        maxRowCount,
         rollup
     );
-    inputDataStream.close();
-    return new IncrementalIndexSegment(index, SegmentId.dummy("test_datasource"));
   }
 
-  public static Segment createDefaultHourlyIncrementalIndex() throws Exception
-  {
-    return createIncrementalIndex(Granularities.HOUR, true, true, 1000);
-  }
-
-  public static Segment createDefaultDailyIncrementalIndex() throws Exception
-  {
-    return createIncrementalIndex(Granularities.DAY, true, true, 1000);
-  }
-
-  public static List<Segment> createDefaultHourlySegments(
-      AggregationTestHelper helper,
+  public static List<Segment> createSegmentsForJsonInput(
       TemporaryFolder tempFolder,
-      Closer closer
+      Closer closer,
+      String inputFile
   )
       throws Exception
   {
-    return createSegments(
-        helper,
+    return createSegmentsForJsonInput(
         tempFolder,
         closer,
-        Granularities.HOUR,
-        true,
-        1000
+        inputFile,
+        Granularities.NONE,
+        true
     );
   }
 
-  public static List<Segment> createDefaultHourlySegmentsTsv(
-      AggregationTestHelper helper,
+  public static Segment createIncrementalIndex(
       TemporaryFolder tempFolder,
-      Closer closer
+      String inputFileName,
+      InputFormat inputFormat,
+      TimestampSpec timestampSpec,
+      DimensionsSpec dimensionsSpec,
+      TransformSpec transformSpec,
+      AggregatorFactory[] aggregators,
+      Granularity queryGranularity,
+      boolean rollup
   )
       throws Exception
   {
-    return createTsvSegments(
-        helper,
-        tempFolder,
-        closer,
-        Granularities.HOUR,
-        true,
-        1000
-    );
+    IndexBuilder bob = IndexBuilder.create()
+                                   .schema(
+                                       IncrementalIndexSchema.builder()
+                                                             .withTimestampSpec(timestampSpec)
+                                                             .withDimensionsSpec(dimensionsSpec)
+                                                             .withMetrics(aggregators)
+                                                             .withQueryGranularity(queryGranularity)
+                                                             .withRollup(rollup)
+                                                             .withMinTimestamp(0)
+                                                             .build()
+                                   )
+                                   .inputSource(
+                                       ResourceInputSource.of(
+                                           NestedDataTestUtils.class.getClassLoader(),
+                                           inputFileName
+                                       )
+                                   )
+                                   .inputFormat(inputFormat)
+                                   .transform(transformSpec)
+                                   .inputTmpDir(tempFolder.newFolder());
+
+    return new IncrementalIndexSegment(bob.buildIncrementalIndex(), SegmentId.dummy("test_datasource"));
   }
 
-  public static List<Segment> createDefaultDaySegments(
-      AggregationTestHelper helper,
+  public static List<Segment> createSegments(
       TemporaryFolder tempFolder,
-      Closer closer
-  )
-      throws Exception
+      Closer closer,
+      String input,
+      InputFormat inputFormat,
+      TimestampSpec timestampSpec,
+      DimensionsSpec dimensionsSpec,
+      TransformSpec transformSpec,
+      AggregatorFactory[] aggregators,
+      Granularity queryGranularity,
+      boolean rollup
+  ) throws Exception
   {
     return createSegments(
-        helper,
         tempFolder,
         closer,
-        Granularities.DAY,
-        true,
-        1000
+        Collections.singletonList(ResourceInputSource.of(NestedDataTestUtils.class.getClassLoader(), input)),
+        inputFormat,
+        timestampSpec,
+        dimensionsSpec,
+        transformSpec,
+        aggregators,
+        queryGranularity,
+        rollup
     );
   }
 
-  public static File readFileFromClasspath(String fileName)
+  public static List<Segment> createSegments(
+      TemporaryFolder tempFolder,
+      Closer closer,
+      List<InputSource> inputs,
+      InputFormat inputFormat,
+      TimestampSpec timestampSpec,
+      DimensionsSpec dimensionsSpec,
+      TransformSpec transformSpec,
+      AggregatorFactory[] aggregators,
+      Granularity queryGranularity,
+      boolean rollup
+  ) throws Exception
   {
-    return new File(NestedDataTestUtils.class.getClassLoader().getResource(fileName).getFile());
+    final List<Segment> segments = Lists.newArrayListWithCapacity(inputs.size());
+    for (InputSource inputSource : inputs) {
+      final File segmentDir = tempFolder.newFolder();
+      IndexBuilder bob = IndexBuilder.create()
+                                     .tmpDir(segmentDir)
+                                     .schema(
+                                         IncrementalIndexSchema.builder()
+                                                               .withTimestampSpec(timestampSpec)
+                                                               .withDimensionsSpec(dimensionsSpec)
+                                                               .withMetrics(aggregators)
+                                                               .withQueryGranularity(queryGranularity)
+                                                               .withRollup(rollup)
+                                                               .withMinTimestamp(0)
+                                                               .build()
+                                     )
+                                     .inputSource(inputSource)
+                                     .inputFormat(inputFormat)
+                                     .transform(transformSpec)
+                                     .inputTmpDir(tempFolder.newFolder());
+      segments.add(
+          new QueryableIndexSegment(
+              closer.register(bob.buildMMappedIndex()),
+              SegmentId.dummy("test_datasource")
+          )
+      );
+    }
+
+    return segments;
   }
 
-  public static String readFileFromClasspathAsString(String fileName) throws IOException
+  /**
+   * turn a small file into bigger file with a bunch of copies of itself
+   */
+  public static File selfConcatenateResourceFile(
+      TemporaryFolder tempFolder,
+      String inputFileName,
+      int numCopies
+  ) throws IOException
   {
-    return Files.asCharSource(readFileFromClasspath(fileName), StandardCharsets.UTF_8).read();
+    List<InputStream> inputStreams = Lists.newArrayListWithCapacity(numCopies);
+    for (int i = 0; i < numCopies; i++) {
+      InputStream stream = NestedDataTestUtils.class.getClassLoader().getResourceAsStream(inputFileName);
+      inputStreams.add(stream);
+      if (i + 1 < numCopies) {
+        inputStreams.add(new ByteArrayInputStream(StringUtils.toUtf8("\n")));
+      }
+    }
+    File tmpFile = tempFolder.newFile();
+    try (
+        SequenceInputStream inputDataStream = new SequenceInputStream(Collections.enumeration(inputStreams));
+        OutputStream outStream = Files.newOutputStream(tmpFile.toPath())
+    ) {
+      final byte[] buffer = new byte[8096];
+      int bytesRead;
+      while ((bytesRead = inputDataStream.read(buffer)) != -1) {
+        outStream.write(buffer, 0, bytesRead);
+      }
+    }
+
+    return tmpFile;
   }
 
-  public static List<TrinaryFn<AggregationTestHelper, TemporaryFolder, Closer, List<Segment>>> getSegmentGenerators()
+  public static List<BiFunction<TemporaryFolder, Closer, List<Segment>>> getSegmentGenerators(
+      String jsonInputFile
+  )
   {
-    final List<TrinaryFn<AggregationTestHelper, TemporaryFolder, Closer, List<Segment>>> segmentsGenerators = new ArrayList<>();
-    segmentsGenerators.add(new TrinaryFn<AggregationTestHelper, TemporaryFolder, Closer, List<Segment>>()
+    final List<BiFunction<TemporaryFolder, Closer, List<Segment>>> segmentsGenerators =
+        new ArrayList<>();
+    segmentsGenerators.add(new BiFunction<TemporaryFolder, Closer, List<Segment>>()
     {
       @Override
-      public List<Segment> apply(AggregationTestHelper helper, TemporaryFolder tempFolder, Closer closer)
+      public List<Segment> apply(TemporaryFolder tempFolder, Closer closer)
       {
         try {
           return ImmutableList.<Segment>builder()
-                              .addAll(NestedDataTestUtils.createDefaultHourlySegments(helper, tempFolder, closer))
-                              .add(NestedDataTestUtils.createDefaultHourlyIncrementalIndex())
+                              .addAll(
+                                  NestedDataTestUtils.createSegmentsForJsonInput(
+                                      tempFolder,
+                                      closer,
+                                      jsonInputFile
+                                  )
+                              )
+                              .add(NestedDataTestUtils.createIncrementalIndexForJsonInput(tempFolder, jsonInputFile))
                               .build();
         }
         catch (Exception e) {
@@ -458,15 +482,15 @@ public class NestedDataTestUtils
         return "mixed";
       }
     });
-    segmentsGenerators.add(new TrinaryFn<AggregationTestHelper, TemporaryFolder, Closer, List<Segment>>()
+    segmentsGenerators.add(new BiFunction<TemporaryFolder, Closer, List<Segment>>()
     {
       @Override
-      public List<Segment> apply(AggregationTestHelper helper, TemporaryFolder tempFolder, Closer closer)
+      public List<Segment> apply(TemporaryFolder tempFolder, Closer closer)
       {
         try {
           return ImmutableList.of(
-              NestedDataTestUtils.createDefaultHourlyIncrementalIndex(),
-              NestedDataTestUtils.createDefaultHourlyIncrementalIndex()
+              NestedDataTestUtils.createIncrementalIndexForJsonInput(tempFolder, jsonInputFile),
+              NestedDataTestUtils.createIncrementalIndexForJsonInput(tempFolder, jsonInputFile)
           );
         }
         catch (Exception e) {
@@ -480,15 +504,26 @@ public class NestedDataTestUtils
         return "incremental";
       }
     });
-    segmentsGenerators.add(new TrinaryFn<AggregationTestHelper, TemporaryFolder, Closer, List<Segment>>()
+    segmentsGenerators.add(new BiFunction<TemporaryFolder, Closer, List<Segment>>()
     {
       @Override
-      public List<Segment> apply(AggregationTestHelper helper, TemporaryFolder tempFolder, Closer closer)
+      public List<Segment> apply(TemporaryFolder tempFolder, Closer closer)
       {
         try {
           return ImmutableList.<Segment>builder()
-                              .addAll(NestedDataTestUtils.createDefaultHourlySegments(helper, tempFolder, closer))
-                              .addAll(NestedDataTestUtils.createDefaultHourlySegments(helper, tempFolder, closer))
+                              .addAll(
+                                  NestedDataTestUtils.createSegmentsForJsonInput(
+                                      tempFolder,
+                                      closer,
+                                      jsonInputFile
+                                  )
+                              )
+                              .addAll(NestedDataTestUtils.createSegmentsForJsonInput(
+                                  tempFolder,
+                                          closer,
+                                          jsonInputFile
+                                      )
+                              )
                               .build();
         }
         catch (Exception e) {
diff --git a/processing/src/test/java/org/apache/druid/query/aggregation/AggregationTestHelper.java b/processing/src/test/java/org/apache/druid/query/aggregation/AggregationTestHelper.java
index 5cad59dffb..65a59fc528 100644
--- a/processing/src/test/java/org/apache/druid/query/aggregation/AggregationTestHelper.java
+++ b/processing/src/test/java/org/apache/druid/query/aggregation/AggregationTestHelper.java
@@ -82,8 +82,6 @@ import org.apache.druid.segment.column.ColumnConfig;
 import org.apache.druid.segment.incremental.IncrementalIndex;
 import org.apache.druid.segment.incremental.IncrementalIndexSchema;
 import org.apache.druid.segment.incremental.OnheapIncrementalIndex;
-import org.apache.druid.segment.transform.TransformSpec;
-import org.apache.druid.segment.transform.TransformingStringInputRowParser;
 import org.apache.druid.segment.writeout.OffHeapMemorySegmentWriteOutMediumFactory;
 import org.apache.druid.timeline.SegmentId;
 import org.apache.druid.utils.CloseableUtils;
@@ -509,50 +507,6 @@ public class AggregationTestHelper implements Closeable
     }
   }
 
-  public void createIndex(
-      InputStream inputDataStream,
-      String parserJson,
-      String transformSpecJson,
-      String aggregators,
-      File outDir,
-      long minTimestamp,
-      Granularity gran,
-      int maxRowCount,
-      boolean rollup
-  ) throws Exception
-  {
-    try {
-      StringInputRowParser parser = mapper.readValue(parserJson, StringInputRowParser.class);
-      TransformSpec transformSpec;
-      if (transformSpecJson != null) {
-        transformSpec = mapper.readValue(transformSpecJson, TransformSpec.class);
-        parser = new TransformingStringInputRowParser(parser.getParseSpec(), parser.getEncoding(), transformSpec);
-      }
-
-      LineIterator iter = IOUtils.lineIterator(inputDataStream, "UTF-8");
-      List<AggregatorFactory> aggregatorSpecs = mapper.readValue(
-          aggregators,
-          new TypeReference<List<AggregatorFactory>>()
-          {
-          }
-      );
-
-      createIndex(
-          iter,
-          parser,
-          aggregatorSpecs.toArray(new AggregatorFactory[0]),
-          outDir,
-          minTimestamp,
-          gran,
-          true,
-          maxRowCount,
-          rollup
-      );
-    }
-    finally {
-      Closeables.close(inputDataStream, true);
-    }
-  }
 
   public void createIndex(
       Iterator rows,
diff --git a/processing/src/test/java/org/apache/druid/query/expression/NestedDataExpressionsTest.java b/processing/src/test/java/org/apache/druid/query/expression/NestedDataExpressionsTest.java
index 1b7e899509..235579b19f 100644
--- a/processing/src/test/java/org/apache/druid/query/expression/NestedDataExpressionsTest.java
+++ b/processing/src/test/java/org/apache/druid/query/expression/NestedDataExpressionsTest.java
@@ -131,7 +131,7 @@ public class NestedDataExpressionsTest extends InitializedNullHandlingTest
     expr = Parser.parse("json_paths(nester)", MACRO_TABLE);
     eval = expr.eval(inputBindings);
     Assert.assertEquals(ExpressionType.STRING_ARRAY, eval.type());
-    Assert.assertArrayEquals(new Object[]{"$.x[0]", "$.y.a", "$.x[1]", "$.y.b", "$.x[2]"}, (Object[]) eval.value());
+    Assert.assertArrayEquals(new Object[]{"$.x", "$.y.a", "$.y.b"}, (Object[]) eval.value());
   }
 
   @Test
diff --git a/processing/src/test/java/org/apache/druid/query/expression/VectorExpressionsSanityTest.java b/processing/src/test/java/org/apache/druid/query/expression/VectorExpressionsSanityTest.java
index 87ae80fa10..71e977909f 100644
--- a/processing/src/test/java/org/apache/druid/query/expression/VectorExpressionsSanityTest.java
+++ b/processing/src/test/java/org/apache/druid/query/expression/VectorExpressionsSanityTest.java
@@ -71,7 +71,7 @@ public class VectorExpressionsSanityTest extends InitializedNullHandlingTest
   @AfterClass
   public static void teardownTests()
   {
-    ExpressionProcessing.initializeForTests(null);
+    ExpressionProcessing.initializeForTests();
   }
 
   static void testExpression(String expr, Expr parsed, Map<String, ExpressionType> types)
diff --git a/processing/src/test/java/org/apache/druid/query/groupby/NestedDataGroupByQueryTest.java b/processing/src/test/java/org/apache/druid/query/groupby/NestedDataGroupByQueryTest.java
index 3038ccaedf..030a40b29c 100644
--- a/processing/src/test/java/org/apache/druid/query/groupby/NestedDataGroupByQueryTest.java
+++ b/processing/src/test/java/org/apache/druid/query/groupby/NestedDataGroupByQueryTest.java
@@ -25,8 +25,6 @@ import org.apache.druid.common.config.NullHandling;
 import org.apache.druid.guice.NestedDataModule;
 import org.apache.druid.java.util.common.Intervals;
 import org.apache.druid.java.util.common.granularity.Granularities;
-import org.apache.druid.java.util.common.guava.Sequence;
-import org.apache.druid.java.util.common.guava.nary.TrinaryFn;
 import org.apache.druid.java.util.common.io.Closer;
 import org.apache.druid.java.util.common.logger.Logger;
 import org.apache.druid.query.NestedDataTestUtils;
@@ -60,6 +58,8 @@ import java.util.ArrayList;
 import java.util.Collection;
 import java.util.List;
 import java.util.Map;
+import java.util.function.BiFunction;
+import java.util.function.Supplier;
 
 @RunWith(Parameterized.class)
 public class NestedDataGroupByQueryTest extends InitializedNullHandlingTest
@@ -73,12 +73,12 @@ public class NestedDataGroupByQueryTest extends InitializedNullHandlingTest
   private final GroupByQueryConfig config;
   private final QueryContexts.Vectorize vectorize;
   private final AggregationTestHelper helper;
-  private final TrinaryFn<AggregationTestHelper, TemporaryFolder, Closer, List<Segment>> segmentsGenerator;
+  private final BiFunction<TemporaryFolder, Closer, List<Segment>> segmentsGenerator;
   private final String segmentsName;
 
   public NestedDataGroupByQueryTest(
       GroupByQueryConfig config,
-      TrinaryFn<AggregationTestHelper, TemporaryFolder, Closer, List<Segment>> segmentGenerator,
+      BiFunction<TemporaryFolder, Closer, List<Segment>> segmentGenerator,
       String vectorize
   )
   {
@@ -107,11 +107,11 @@ public class NestedDataGroupByQueryTest extends InitializedNullHandlingTest
   public static Collection<?> constructorFeeder()
   {
     final List<Object[]> constructors = new ArrayList<>();
-    final List<TrinaryFn<AggregationTestHelper, TemporaryFolder, Closer, List<Segment>>> segmentsGenerators =
-        NestedDataTestUtils.getSegmentGenerators();
+    final List<BiFunction<TemporaryFolder, Closer, List<Segment>>> segmentsGenerators =
+        NestedDataTestUtils.getSegmentGenerators(NestedDataTestUtils.SIMPLE_DATA_FILE);
 
     for (GroupByQueryConfig config : GroupByQueryRunnerTest.testConfigs()) {
-      for (TrinaryFn<AggregationTestHelper, TemporaryFolder, Closer, List<Segment>> generatorFn : segmentsGenerators) {
+      for (BiFunction<TemporaryFolder, Closer, List<Segment>> generatorFn : segmentsGenerators) {
         for (String vectorize : new String[]{"false", "true", "force"}) {
           constructors.add(new Object[]{config, generatorFn, vectorize});
         }
@@ -444,8 +444,8 @@ public class NestedDataGroupByQueryTest extends InitializedNullHandlingTest
     runResults(
         groupQuery,
         ImmutableList.of(
-            new Object[]{1609459200000L, NullHandling.defaultLongValue(), 8L},
-            new Object[]{1609545600000L, NullHandling.defaultLongValue(), 8L}
+            new Object[]{1672531200000L, NullHandling.defaultLongValue(), 8L},
+            new Object[]{1672617600000L, NullHandling.defaultLongValue(), 8L}
         ),
         false,
         true
@@ -466,7 +466,7 @@ public class NestedDataGroupByQueryTest extends InitializedNullHandlingTest
                                               new NestedFieldVirtualColumn("__time", "$", "v0")
                                           )
                                           .setAggregatorSpecs(new CountAggregatorFactory("count"))
-                                          .setDimFilter(new SelectorDimFilter("v0", "1609459200000", null))
+                                          .setDimFilter(new SelectorDimFilter("v0", "1672531200000", null))
                                           .setContext(getContext())
                                           .build();
 
@@ -474,7 +474,7 @@ public class NestedDataGroupByQueryTest extends InitializedNullHandlingTest
     runResults(
         groupQuery,
         ImmutableList.of(
-            new Object[]{1609459200000L, 8L}
+            new Object[]{1672531200000L, 8L}
         ),
         false,
         true
@@ -495,7 +495,7 @@ public class NestedDataGroupByQueryTest extends InitializedNullHandlingTest
                                               new NestedFieldVirtualColumn("__time", "$", "v0", ColumnType.STRING)
                                           )
                                           .setAggregatorSpecs(new CountAggregatorFactory("count"))
-                                          .setDimFilter(new SelectorDimFilter("v0", "1609459200000", null))
+                                          .setDimFilter(new SelectorDimFilter("v0", "1672531200000", null))
                                           .setContext(getContext())
                                           .build();
 
@@ -503,7 +503,7 @@ public class NestedDataGroupByQueryTest extends InitializedNullHandlingTest
     runResults(
         groupQuery,
         ImmutableList.of(
-            new Object[]{"1609459200000", 8L}
+            new Object[]{"1672531200000", 8L}
         ),
         true,
         false
@@ -542,14 +542,18 @@ public class NestedDataGroupByQueryTest extends InitializedNullHandlingTest
     runResults(groupQuery, expectedResults, false, false);
   }
 
-  private void runResults(GroupByQuery groupQuery, List<Object[]> expectedResults, boolean hasUnknownCardinality, boolean hasNonStringOutput)
+  private void runResults(
+      GroupByQuery groupQuery,
+      List<Object[]> expectedResults,
+      boolean hasUnknownCardinality,
+      boolean hasNonStringOutput
+  )
   {
+    Supplier<List<ResultRow>> runner =
+        () -> helper.runQueryOnSegmentsObjs(segmentsGenerator.apply(tempFolder, closer), groupQuery).toList();
     if (GroupByStrategySelector.STRATEGY_V1.equals(config.getDefaultStrategy())) {
       if (hasUnknownCardinality) {
-        Throwable t = Assert.assertThrows(
-            RuntimeException.class,
-            () -> helper.runQueryOnSegmentsObjs(segmentsGenerator.apply(helper, tempFolder, closer), groupQuery).toList()
-        );
+        Throwable t = Assert.assertThrows(RuntimeException.class, runner::get);
         Assert.assertEquals(
             "java.lang.UnsupportedOperationException: GroupBy v1 does not support dimension selectors with unknown cardinality.",
             t.getMessage()
@@ -557,10 +561,7 @@ public class NestedDataGroupByQueryTest extends InitializedNullHandlingTest
         return;
       }
       if (hasNonStringOutput) {
-        Throwable t = Assert.assertThrows(
-            RuntimeException.class,
-            () -> helper.runQueryOnSegmentsObjs(segmentsGenerator.apply(helper, tempFolder, closer), groupQuery).toList()
-        );
+        Throwable t = Assert.assertThrows(RuntimeException.class, runner::get);
         Assert.assertEquals(
             "java.lang.UnsupportedOperationException: GroupBy v1 only supports dimensions with an outputType of STRING.",
             t.getMessage()
@@ -570,22 +571,14 @@ public class NestedDataGroupByQueryTest extends InitializedNullHandlingTest
     }
     if (!"segments".equals(segmentsName)) {
       if (GroupByStrategySelector.STRATEGY_V1.equals(config.getDefaultStrategy())) {
-        Throwable t = Assert.assertThrows(
-            RuntimeException.class,
-            () -> helper.runQueryOnSegmentsObjs(segmentsGenerator.apply(helper, tempFolder, closer), groupQuery)
-                        .toList()
-        );
+        Throwable t = Assert.assertThrows(RuntimeException.class, runner::get);
         Assert.assertEquals(
             "java.lang.UnsupportedOperationException: GroupBy v1 does not support dimension selectors with unknown cardinality.",
             t.getMessage()
         );
         return;
       } else if (vectorize == QueryContexts.Vectorize.FORCE) {
-        Throwable t = Assert.assertThrows(
-            RuntimeException.class,
-            () -> helper.runQueryOnSegmentsObjs(segmentsGenerator.apply(helper, tempFolder, closer), groupQuery)
-                        .toList()
-        );
+        Throwable t = Assert.assertThrows(RuntimeException.class, runner::get);
         Assert.assertEquals(
             "java.util.concurrent.ExecutionException: java.lang.RuntimeException: org.apache.druid.java.util.common.ISE: Cannot vectorize!",
             t.getMessage()
@@ -594,12 +587,7 @@ public class NestedDataGroupByQueryTest extends InitializedNullHandlingTest
       }
     }
 
-    Sequence<ResultRow> seq = helper.runQueryOnSegmentsObjs(
-        segmentsGenerator.apply(helper, tempFolder, closer),
-        groupQuery
-    );
-
-    List<ResultRow> results = seq.toList();
+    List<ResultRow> results = runner.get();
     verifyResults(
         groupQuery.getResultRowSignature(),
         results,
diff --git a/processing/src/test/java/org/apache/druid/query/scan/NestedDataScanQueryTest.java b/processing/src/test/java/org/apache/druid/query/scan/NestedDataScanQueryTest.java
index 268c19bf91..8593f746cc 100644
--- a/processing/src/test/java/org/apache/druid/query/scan/NestedDataScanQueryTest.java
+++ b/processing/src/test/java/org/apache/druid/query/scan/NestedDataScanQueryTest.java
@@ -22,6 +22,7 @@ package org.apache.druid.query.scan;
 import com.fasterxml.jackson.databind.Module;
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.ImmutableMap;
+import org.apache.druid.common.config.NullHandling;
 import org.apache.druid.guice.NestedDataModule;
 import org.apache.druid.java.util.common.Intervals;
 import org.apache.druid.java.util.common.granularity.Granularities;
@@ -32,6 +33,8 @@ import org.apache.druid.query.Druids;
 import org.apache.druid.query.NestedDataTestUtils;
 import org.apache.druid.query.Query;
 import org.apache.druid.query.aggregation.AggregationTestHelper;
+import org.apache.druid.query.aggregation.AggregatorFactory;
+import org.apache.druid.query.aggregation.CountAggregatorFactory;
 import org.apache.druid.query.filter.BoundDimFilter;
 import org.apache.druid.query.filter.SelectorDimFilter;
 import org.apache.druid.query.ordering.StringComparators;
@@ -39,6 +42,7 @@ import org.apache.druid.query.spec.MultipleIntervalSegmentSpec;
 import org.apache.druid.segment.Segment;
 import org.apache.druid.segment.column.ColumnType;
 import org.apache.druid.segment.nested.NestedDataComplexTypeSerde;
+import org.apache.druid.segment.transform.TransformSpec;
 import org.apache.druid.segment.virtual.NestedFieldVirtualColumn;
 import org.apache.druid.testing.InitializedNullHandlingTest;
 import org.junit.After;
@@ -71,10 +75,7 @@ public class NestedDataScanQueryTest extends InitializedNullHandlingTest
   {
     NestedDataModule.registerHandlersAndSerde();
     List<? extends Module> mods = NestedDataModule.getJacksonModulesList();
-    this.helper = AggregationTestHelper.createScanQueryAggregationTestHelper(
-        mods,
-        tempFolder
-    );
+    this.helper = AggregationTestHelper.createScanQueryAggregationTestHelper(mods, tempFolder);
     this.closer = Closer.create();
   }
 
@@ -97,7 +98,7 @@ public class NestedDataScanQueryTest extends InitializedNullHandlingTest
                                              .limit(100)
                                              .context(ImmutableMap.of())
                                              .build();
-    List<Segment> segs = NestedDataTestUtils.createDefaultHourlySegments(helper, tempFolder, closer);
+    List<Segment> segs = NestedDataTestUtils.createSimpleNestedTestDataSegments(tempFolder, closer);
 
     final Sequence<ScanResultValue> seq = helper.runQueryOnSegmentsObjs(segs, scanQuery);
 
@@ -126,15 +127,16 @@ public class NestedDataScanQueryTest extends InitializedNullHandlingTest
                                              .build();
     List<Segment> segs = ImmutableList.<Segment>builder().addAll(
         NestedDataTestUtils.createSegments(
-            helper,
             tempFolder,
             closer,
             NestedDataTestUtils.NUMERIC_DATA_FILE,
-            NestedDataTestUtils.NUMERIC_PARSER_FILE,
-            NestedDataTestUtils.SIMPLE_AGG_FILE,
+            NestedDataTestUtils.DEFAULT_JSON_INPUT_FORMAT,
+            NestedDataTestUtils.TIMESTAMP_SPEC,
+            NestedDataTestUtils.AUTO_DISCOVERY,
+            TransformSpec.NONE,
+            NestedDataTestUtils.COUNT,
             Granularities.YEAR,
-            true,
-            1000
+            true
         )
     ).build();
 
@@ -171,9 +173,9 @@ public class NestedDataScanQueryTest extends InitializedNullHandlingTest
                                              .context(ImmutableMap.of())
                                              .build();
     List<Segment> realtimeSegs = ImmutableList.of(
-        NestedDataTestUtils.createDefaultHourlyIncrementalIndex()
+        NestedDataTestUtils.createSimpleNestedTestDataIncrementalIndex(tempFolder)
     );
-    List<Segment> segs = NestedDataTestUtils.createDefaultHourlySegments(helper, tempFolder, closer);
+    List<Segment> segs = NestedDataTestUtils.createSimpleNestedTestDataSegments(tempFolder, closer);
 
 
     final Sequence<ScanResultValue> seq = helper.runQueryOnSegmentsObjs(realtimeSegs, scanQuery);
@@ -242,9 +244,9 @@ public class NestedDataScanQueryTest extends InitializedNullHandlingTest
                                              .context(ImmutableMap.of())
                                              .build();
     List<Segment> realtimeSegs = ImmutableList.of(
-        NestedDataTestUtils.createDefaultHourlyIncrementalIndex()
+        NestedDataTestUtils.createSimpleNestedTestDataIncrementalIndex(tempFolder)
     );
-    List<Segment> segs = NestedDataTestUtils.createDefaultHourlySegments(helper, tempFolder, closer);
+    List<Segment> segs = NestedDataTestUtils.createSimpleNestedTestDataSegments(tempFolder, closer);
 
 
     final Sequence<ScanResultValue> seq = helper.runQueryOnSegmentsObjs(realtimeSegs, scanQuery);
@@ -278,7 +280,7 @@ public class NestedDataScanQueryTest extends InitializedNullHandlingTest
                                              .limit(100)
                                              .context(ImmutableMap.of())
                                              .build();
-    List<Segment> segs = NestedDataTestUtils.createDefaultHourlySegmentsTsv(helper, tempFolder, closer);
+    List<Segment> segs = NestedDataTestUtils.createSimpleSegmentsTsv(tempFolder, closer);
 
     final Sequence<ScanResultValue> seq = helper.runQueryOnSegmentsObjs(segs, scanQuery);
 
@@ -302,13 +304,12 @@ public class NestedDataScanQueryTest extends InitializedNullHandlingTest
                                              .limit(100)
                                              .context(ImmutableMap.of())
                                              .build();
-    List<Segment> segs = NestedDataTestUtils.createSegments(
-        helper,
+    List<Segment> segs = NestedDataTestUtils.createSegmentsForJsonInput(
         tempFolder,
         closer,
+        NestedDataTestUtils.SIMPLE_DATA_FILE,
         Granularities.HOUR,
-        true,
-        3
+        true
     );
     final Sequence<ScanResultValue> seq = helper.runQueryOnSegmentsObjs(segs, scanQuery);
 
@@ -334,13 +335,12 @@ public class NestedDataScanQueryTest extends InitializedNullHandlingTest
                                              .build();
 
 
-    List<Segment> segs = NestedDataTestUtils.createSegmentsWithConcatenatedInput(
-        helper,
+    List<Segment> segs = NestedDataTestUtils.createSegmentsWithConcatenatedJsonInput(
         tempFolder,
         closer,
+        NestedDataTestUtils.SIMPLE_DATA_FILE,
         Granularities.HOUR,
         false,
-        5,
         10,
         1
     );
@@ -369,13 +369,12 @@ public class NestedDataScanQueryTest extends InitializedNullHandlingTest
 
 
     // same rows over and over so expect same 8 rows after rollup
-    List<Segment> segs = NestedDataTestUtils.createSegmentsWithConcatenatedInput(
-        helper,
+    List<Segment> segs = NestedDataTestUtils.createSegmentsWithConcatenatedJsonInput(
         tempFolder,
         closer,
-        Granularities.HOUR,
+        NestedDataTestUtils.SIMPLE_DATA_FILE,
+        Granularities.YEAR,
         true,
-        5,
         100,
         1
     );
@@ -407,7 +406,7 @@ public class NestedDataScanQueryTest extends InitializedNullHandlingTest
                                              .limit(100)
                                              .context(ImmutableMap.of())
                                              .build();
-    List<Segment> segs = NestedDataTestUtils.createDefaultHourlySegments(helper, tempFolder, closer);
+    List<Segment> segs = NestedDataTestUtils.createSimpleNestedTestDataSegments(tempFolder, closer);
 
     final Sequence<ScanResultValue> seq = helper.runQueryOnSegmentsObjs(segs, scanQuery);
 
@@ -446,7 +445,7 @@ public class NestedDataScanQueryTest extends InitializedNullHandlingTest
                                              .limit(100)
                                              .context(ImmutableMap.of())
                                              .build();
-    List<Segment> segs = NestedDataTestUtils.createDefaultHourlySegments(helper, tempFolder, closer);
+    List<Segment> segs = NestedDataTestUtils.createSimpleNestedTestDataSegments(tempFolder, closer);
 
     final Sequence<ScanResultValue> seq = helper.runQueryOnSegmentsObjs(segs, scanQuery);
 
@@ -472,25 +471,87 @@ public class NestedDataScanQueryTest extends InitializedNullHandlingTest
                                              .build();
     List<Segment> realtimeSegs = ImmutableList.of(
         NestedDataTestUtils.createIncrementalIndex(
+            tempFolder,
             NestedDataTestUtils.TYPES_DATA_FILE,
-            NestedDataTestUtils.TYPES_PARSER_FILE,
-            NestedDataTestUtils.SIMPLE_AGG_FILE,
+            NestedDataTestUtils.DEFAULT_JSON_INPUT_FORMAT,
+            NestedDataTestUtils.TIMESTAMP_SPEC,
+            NestedDataTestUtils.AUTO_DISCOVERY,
+            TransformSpec.NONE,
+            NestedDataTestUtils.COUNT,
             Granularities.DAY,
-            true,
-            false,
-            1000
+            true
         )
     );
     List<Segment> segs = NestedDataTestUtils.createSegments(
-        helper,
         tempFolder,
         closer,
         NestedDataTestUtils.TYPES_DATA_FILE,
-        NestedDataTestUtils.TYPES_PARSER_FILE,
-        NestedDataTestUtils.SIMPLE_AGG_FILE,
+        NestedDataTestUtils.DEFAULT_JSON_INPUT_FORMAT,
+        NestedDataTestUtils.TIMESTAMP_SPEC,
+        NestedDataTestUtils.AUTO_DISCOVERY,
+        TransformSpec.NONE,
+        NestedDataTestUtils.COUNT,
         Granularities.DAY,
-        true,
-        100
+        true
+    );
+
+
+    final Sequence<ScanResultValue> seq = helper.runQueryOnSegmentsObjs(realtimeSegs, scanQuery);
+    final Sequence<ScanResultValue> seq2 = helper.runQueryOnSegmentsObjs(segs, scanQuery);
+
+    List<ScanResultValue> resultsRealtime = seq.toList();
+    List<ScanResultValue> resultsSegments = seq2.toList();
+    logResults(resultsSegments);
+    logResults(resultsRealtime);
+    Assert.assertEquals(1, resultsRealtime.size());
+    Assert.assertEquals(resultsRealtime.size(), resultsSegments.size());
+    Assert.assertEquals(resultsSegments.get(0).getEvents().toString(), resultsRealtime.get(0).getEvents().toString());
+  }
+
+  @Test
+  public void testIngestAndScanSegmentsRealtimeSchemaDiscoveryArrayTypes() throws Exception
+  {
+    Druids.ScanQueryBuilder builder = Druids.newScanQueryBuilder()
+                                            .dataSource("test_datasource")
+                                            .intervals(
+                                                new MultipleIntervalSegmentSpec(
+                                                    Collections.singletonList(Intervals.ETERNITY)
+                                                )
+                                            )
+                                            .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST)
+                                            .limit(100)
+                                            .context(ImmutableMap.of());
+    if (NullHandling.replaceWithDefault()) {
+      // null elements are replaced with default values if druid.generic.useDefaultValueForNull=true
+      // ... but not until after they are persisted, so realtime query results don't match of course...
+      builder.columns("arrayString", "arrayLong", "arrayDouble");
+    }
+    Query<ScanResultValue> scanQuery = builder.build();
+    final AggregatorFactory[] aggs = new AggregatorFactory[]{new CountAggregatorFactory("count")};
+    List<Segment> realtimeSegs = ImmutableList.of(
+        NestedDataTestUtils.createIncrementalIndex(
+            tempFolder,
+            NestedDataTestUtils.ARRAY_TYPES_DATA_FILE,
+            NestedDataTestUtils.DEFAULT_JSON_INPUT_FORMAT,
+            NestedDataTestUtils.TIMESTAMP_SPEC,
+            NestedDataTestUtils.AUTO_DISCOVERY,
+            TransformSpec.NONE,
+            aggs,
+            Granularities.NONE,
+            true
+        )
+    );
+    List<Segment> segs = NestedDataTestUtils.createSegments(
+        tempFolder,
+        closer,
+        NestedDataTestUtils.ARRAY_TYPES_DATA_FILE,
+        NestedDataTestUtils.DEFAULT_JSON_INPUT_FORMAT,
+        NestedDataTestUtils.TIMESTAMP_SPEC,
+        NestedDataTestUtils.AUTO_DISCOVERY,
+        TransformSpec.NONE,
+        aggs,
+        Granularities.NONE,
+        true
     );
 
 
diff --git a/processing/src/test/java/org/apache/druid/query/topn/NestedDataTopNQueryTest.java b/processing/src/test/java/org/apache/druid/query/topn/NestedDataTopNQueryTest.java
index 58cbf46c5c..392df70e1d 100644
--- a/processing/src/test/java/org/apache/druid/query/topn/NestedDataTopNQueryTest.java
+++ b/processing/src/test/java/org/apache/druid/query/topn/NestedDataTopNQueryTest.java
@@ -26,7 +26,6 @@ import org.apache.druid.java.util.common.Intervals;
 import org.apache.druid.java.util.common.StringUtils;
 import org.apache.druid.java.util.common.granularity.Granularities;
 import org.apache.druid.java.util.common.guava.Sequence;
-import org.apache.druid.java.util.common.guava.nary.TrinaryFn;
 import org.apache.druid.java.util.common.io.Closer;
 import org.apache.druid.java.util.common.logger.Logger;
 import org.apache.druid.query.NestedDataTestUtils;
@@ -55,6 +54,7 @@ import java.util.Arrays;
 import java.util.Collection;
 import java.util.Collections;
 import java.util.List;
+import java.util.function.BiFunction;
 
 @RunWith(Parameterized.class)
 public class NestedDataTopNQueryTest extends InitializedNullHandlingTest
@@ -65,11 +65,11 @@ public class NestedDataTopNQueryTest extends InitializedNullHandlingTest
   public final TemporaryFolder tempFolder = new TemporaryFolder();
 
   private final AggregationTestHelper helper;
-  private final TrinaryFn<AggregationTestHelper, TemporaryFolder, Closer, List<Segment>> segmentsGenerator;
+  private final BiFunction<TemporaryFolder, Closer, List<Segment>> segmentsGenerator;
   private final Closer closer;
 
   public NestedDataTopNQueryTest(
-      TrinaryFn<AggregationTestHelper, TemporaryFolder, Closer, List<Segment>> segmentGenerator
+      BiFunction<TemporaryFolder, Closer, List<Segment>> segmentGenerator
   )
   {
     NestedDataModule.registerHandlersAndSerde();
@@ -85,10 +85,10 @@ public class NestedDataTopNQueryTest extends InitializedNullHandlingTest
   public static Collection<?> constructorFeeder()
   {
     final List<Object[]> constructors = new ArrayList<>();
-    final List<TrinaryFn<AggregationTestHelper, TemporaryFolder, Closer, List<Segment>>> segmentsGenerators =
-        NestedDataTestUtils.getSegmentGenerators();
+    final List<BiFunction<TemporaryFolder, Closer, List<Segment>>> segmentsGenerators =
+        NestedDataTestUtils.getSegmentGenerators(NestedDataTestUtils.SIMPLE_DATA_FILE);
 
-    for (TrinaryFn<AggregationTestHelper, TemporaryFolder, Closer, List<Segment>> generatorFn : segmentsGenerators) {
+    for (BiFunction<TemporaryFolder, Closer, List<Segment>> generatorFn : segmentsGenerators) {
       constructors.add(new Object[]{generatorFn});
     }
     return constructors;
@@ -114,7 +114,7 @@ public class NestedDataTopNQueryTest extends InitializedNullHandlingTest
                                            .build();
 
 
-    Sequence<Result<TopNResultValue>> seq = helper.runQueryOnSegmentsObjs(segmentsGenerator.apply(helper, tempFolder, closer), topN);
+    Sequence<Result<TopNResultValue>> seq = helper.runQueryOnSegmentsObjs(segmentsGenerator.apply(tempFolder, closer), topN);
 
     Sequence<Object[]> resultsSeq = new TopNQueryQueryToolChest(new TopNQueryConfig()).resultsAsArrays(topN, seq);
 
@@ -123,10 +123,10 @@ public class NestedDataTopNQueryTest extends InitializedNullHandlingTest
     verifyResults(
         results,
         ImmutableList.of(
-            new Object[]{1609459200000L, null, 8L},
-            new Object[]{1609459200000L, "100", 2L},
-            new Object[]{1609459200000L, "200", 2L},
-            new Object[]{1609459200000L, "300", 4L}
+            new Object[]{1672531200000L, null, 8L},
+            new Object[]{1672531200000L, "100", 2L},
+            new Object[]{1672531200000L, "200", 2L},
+            new Object[]{1672531200000L, "300", 4L}
         )
     );
   }
@@ -148,7 +148,7 @@ public class NestedDataTopNQueryTest extends InitializedNullHandlingTest
                                            .build();
 
 
-    Sequence<Result<TopNResultValue>> seq = helper.runQueryOnSegmentsObjs(segmentsGenerator.apply(helper, tempFolder, closer), topN);
+    Sequence<Result<TopNResultValue>> seq = helper.runQueryOnSegmentsObjs(segmentsGenerator.apply(tempFolder, closer), topN);
 
     Sequence<Object[]> resultsSeq = new TopNQueryQueryToolChest(new TopNQueryConfig()).resultsAsArrays(topN, seq);
 
@@ -157,10 +157,10 @@ public class NestedDataTopNQueryTest extends InitializedNullHandlingTest
     verifyResults(
         results,
         ImmutableList.of(
-            new Object[]{1609459200000L, null, NullHandling.defaultDoubleValue()},
-            new Object[]{1609459200000L, "100", 200.0},
-            new Object[]{1609459200000L, "200", 400.0},
-            new Object[]{1609459200000L, "300", 1200.0}
+            new Object[]{1672531200000L, null, NullHandling.defaultDoubleValue()},
+            new Object[]{1672531200000L, "100", 200.0},
+            new Object[]{1672531200000L, "200", 400.0},
+            new Object[]{1672531200000L, "300", 1200.0}
         )
     );
   }
diff --git a/processing/src/test/java/org/apache/druid/segment/IndexBuilder.java b/processing/src/test/java/org/apache/druid/segment/IndexBuilder.java
index f0d9b1fc32..90a4411c87 100644
--- a/processing/src/test/java/org/apache/druid/segment/IndexBuilder.java
+++ b/processing/src/test/java/org/apache/druid/segment/IndexBuilder.java
@@ -22,12 +22,18 @@ package org.apache.druid.segment;
 import com.fasterxml.jackson.databind.ObjectMapper;
 import com.google.common.base.Preconditions;
 import com.google.common.collect.Iterables;
+import org.apache.druid.data.input.InputFormat;
 import org.apache.druid.data.input.InputRow;
+import org.apache.druid.data.input.InputRowSchema;
+import org.apache.druid.data.input.InputSource;
+import org.apache.druid.data.input.InputSourceReader;
 import org.apache.druid.frame.FrameType;
 import org.apache.druid.frame.segment.FrameSegment;
 import org.apache.druid.frame.testutil.FrameTestUtil;
+import org.apache.druid.java.util.common.CloseableIterators;
 import org.apache.druid.java.util.common.StringUtils;
 import org.apache.druid.java.util.common.guava.Sequences;
+import org.apache.druid.java.util.common.parsers.CloseableIterator;
 import org.apache.druid.query.aggregation.AggregatorFactory;
 import org.apache.druid.query.aggregation.CountAggregatorFactory;
 import org.apache.druid.segment.column.ColumnConfig;
@@ -36,10 +42,13 @@ import org.apache.druid.segment.incremental.IncrementalIndex;
 import org.apache.druid.segment.incremental.IncrementalIndexSchema;
 import org.apache.druid.segment.incremental.IndexSizeExceededException;
 import org.apache.druid.segment.incremental.OnheapIncrementalIndex;
+import org.apache.druid.segment.transform.TransformSpec;
 import org.apache.druid.segment.writeout.OffHeapMemorySegmentWriteOutMediumFactory;
 import org.apache.druid.segment.writeout.SegmentWriteOutMediumFactory;
 import org.apache.druid.timeline.SegmentId;
 
+import javax.annotation.Nonnull;
+import javax.annotation.Nullable;
 import java.io.File;
 import java.io.IOException;
 import java.util.ArrayList;
@@ -56,18 +65,27 @@ public class IndexBuilder
   private static final int ROWS_PER_INDEX_FOR_MERGING = 1;
   private static final int DEFAULT_MAX_ROWS = Integer.MAX_VALUE;
 
-  private IncrementalIndexSchema schema = new IncrementalIndexSchema.Builder()
-      .withMetrics(new CountAggregatorFactory("count"))
-      .build();
+  private final ObjectMapper jsonMapper;
+  private final IndexIO indexIO;
+  private final List<InputRow> rows = new ArrayList<>();
+
   private SegmentWriteOutMediumFactory segmentWriteOutMediumFactory = OffHeapMemorySegmentWriteOutMediumFactory.instance();
   private IndexMerger indexMerger;
   private File tmpDir;
   private IndexSpec indexSpec = new IndexSpec();
   private int maxRows = DEFAULT_MAX_ROWS;
-
-  private final ObjectMapper jsonMapper;
-  private final IndexIO indexIO;
-  private final List<InputRow> rows = new ArrayList<>();
+  private int intermediatePersistSize = ROWS_PER_INDEX_FOR_MERGING;
+  private IncrementalIndexSchema schema = new IncrementalIndexSchema.Builder()
+      .withMetrics(new CountAggregatorFactory("count"))
+      .build();
+  @Nullable
+  private InputSource inputSource = null;
+  @Nullable
+  private InputFormat inputFormat = null;
+  @Nullable
+  private TransformSpec transformSpec = null;
+  @Nullable
+  private File inputSourceTmpDir = null;
 
   private IndexBuilder(ObjectMapper jsonMapper, ColumnConfig columnConfig)
   {
@@ -121,6 +139,50 @@ public class IndexBuilder
     return this;
   }
 
+  public IndexBuilder inputSource(InputSource inputSource)
+  {
+    this.inputSource = inputSource;
+    return this;
+  }
+
+  public IndexBuilder inputFormat(InputFormat inputFormat)
+  {
+    this.inputFormat = inputFormat;
+    return this;
+  }
+
+  public IndexBuilder transform(TransformSpec transformSpec)
+  {
+    this.transformSpec = transformSpec;
+    return this;
+  }
+
+  public IndexBuilder inputTmpDir(File inputSourceTmpDir)
+  {
+    this.inputSourceTmpDir = inputSourceTmpDir;
+    return this;
+  }
+
+  public IndexBuilder rows(
+      InputSource inputSource,
+      InputFormat inputFormat,
+      InputRowSchema rowSchema,
+      TransformSpec transformSpec,
+      File tmp
+  )
+      throws IOException
+  {
+    rows.clear();
+    InputSourceReader reader = inputSource.reader(rowSchema, inputFormat, tmp);
+    InputSourceReader transformingReader = transformSpec.decorate(reader);
+    try (CloseableIterator<InputRow> rowIterator = transformingReader.read()) {
+      while (rowIterator.hasNext()) {
+        rows.add(rowIterator.next());
+      }
+    }
+    return this;
+  }
+
   public IndexBuilder rows(Iterable<InputRow> rows)
   {
     this.rows.clear();
@@ -128,8 +190,30 @@ public class IndexBuilder
     return this;
   }
 
+  public IndexBuilder maxRows(int maxRows)
+  {
+    this.maxRows = maxRows;
+    return this;
+  }
+
+  public IndexBuilder intermediaryPersistSize(int rows)
+  {
+    this.intermediatePersistSize = rows;
+    return this;
+  }
+
   public IncrementalIndex buildIncrementalIndex()
   {
+    if (inputSource != null) {
+      return buildIncrementalIndexWithInputSource(
+          schema,
+          inputSource,
+          inputFormat,
+          transformSpec,
+          inputSourceTmpDir,
+          maxRows
+      );
+    }
     return buildIncrementalIndexWithRows(schema, maxRows, rows);
   }
 
@@ -157,20 +241,63 @@ public class IndexBuilder
 
   public QueryableIndex buildMMappedMergedIndex()
   {
-    IndexMerger indexMerger = TestHelper.getTestIndexMergerV9(segmentWriteOutMediumFactory);
     Preconditions.checkNotNull(tmpDir, "tmpDir");
-
     final List<QueryableIndex> persisted = new ArrayList<>();
-    try {
-      for (int i = 0; i < rows.size(); i += ROWS_PER_INDEX_FOR_MERGING) {
+    if (inputSource != null) {
+      Preconditions.checkNotNull(inputSource, "inputSource");
+      Preconditions.checkNotNull(inputFormat, "inputFormat");
+      Preconditions.checkNotNull(inputSourceTmpDir, "inputSourceTmpDir");
+
+      TransformSpec transformer = transformSpec != null ? transformSpec : TransformSpec.NONE;
+      InputRowSchema rowSchema = new InputRowSchema(schema.getTimestampSpec(), schema.getDimensionsSpec(), null);
+      InputSourceReader reader = inputSource.reader(rowSchema, inputFormat, inputSourceTmpDir);
+      InputSourceReader transformingReader = transformer.decorate(reader);
+      return mergeIndexes(indexMerger, persisted, transformingReader::read);
+    }
+
+    return mergeIndexes(indexMerger, persisted, () -> CloseableIterators.withEmptyBaggage(rows.iterator()));
+  }
+
+  @Nonnull
+  private QueryableIndex mergeIndexes(
+      IndexMerger indexMerger,
+      List<QueryableIndex> persisted,
+      IteratorSupplier iteratorSupplier
+  )
+  {
+    try (CloseableIterator<InputRow> rowIterator = iteratorSupplier.get()) {
+      int i = 0;
+      IncrementalIndex incrementalIndex = new OnheapIncrementalIndex.Builder()
+          .setIndexSchema(schema)
+          .setMaxRowCount(maxRows)
+          .build();
+      while (rowIterator.hasNext()) {
+        if (i < intermediatePersistSize) {
+          incrementalIndex.add(rowIterator.next());
+          i++;
+        } else {
+          persisted.add(
+              TestHelper.getTestIndexIO().loadIndex(
+                  indexMerger.persist(
+                      incrementalIndex,
+                      new File(tmpDir, StringUtils.format("testIndex-%s", UUID.randomUUID().toString())),
+                      indexSpec,
+                      null
+                  )
+              )
+          );
+          incrementalIndex = new OnheapIncrementalIndex.Builder()
+              .setIndexSchema(schema)
+              .setMaxRowCount(maxRows)
+              .build();
+          i = 0;
+        }
+      }
+      if (i != 0) {
         persisted.add(
             TestHelper.getTestIndexIO().loadIndex(
                 indexMerger.persist(
-                    buildIncrementalIndexWithRows(
-                        schema,
-                        maxRows,
-                        rows.subList(i, Math.min(rows.size(), i + ROWS_PER_INDEX_FOR_MERGING))
-                    ),
+                    incrementalIndex,
                     new File(tmpDir, StringUtils.format("testIndex-%s", UUID.randomUUID().toString())),
                     indexSpec,
                     null
@@ -178,6 +305,7 @@ public class IndexBuilder
             )
         );
       }
+
       final QueryableIndex merged = TestHelper.getTestIndexIO().loadIndex(
           indexMerger.mergeQueryableIndex(
               persisted,
@@ -267,4 +395,44 @@ public class IndexBuilder
     }
     return incrementalIndex;
   }
+
+  private static IncrementalIndex buildIncrementalIndexWithInputSource(
+      IncrementalIndexSchema schema,
+      InputSource inputSource,
+      InputFormat inputFormat,
+      @Nullable TransformSpec transformSpec,
+      File inputSourceTmpDir,
+      int maxRows
+  )
+  {
+    Preconditions.checkNotNull(schema, "schema");
+    Preconditions.checkNotNull(inputSource, "inputSource");
+    Preconditions.checkNotNull(inputFormat, "inputFormat");
+    Preconditions.checkNotNull(inputSourceTmpDir, "inputSourceTmpDir");
+
+    final IncrementalIndex incrementalIndex = new OnheapIncrementalIndex.Builder()
+        .setIndexSchema(schema)
+        .setMaxRowCount(maxRows)
+        .build();
+    TransformSpec tranformer = transformSpec != null ? transformSpec : TransformSpec.NONE;
+    InputRowSchema rowSchema = new InputRowSchema(schema.getTimestampSpec(), schema.getDimensionsSpec(), null);
+    InputSourceReader reader = inputSource.reader(rowSchema, inputFormat, inputSourceTmpDir);
+    InputSourceReader transformingReader = tranformer.decorate(reader);
+    try (CloseableIterator<InputRow> rowIterator = transformingReader.read()) {
+      while (rowIterator.hasNext()) {
+        incrementalIndex.add(rowIterator.next());
+      }
+    }
+    catch (IOException e) {
+      throw new RuntimeException(e);
+    }
+    return incrementalIndex;
+  }
+
+
+  @FunctionalInterface
+  interface IteratorSupplier
+  {
+    CloseableIterator<InputRow> get() throws IOException;
+  }
 }
diff --git a/processing/src/test/java/org/apache/druid/segment/NestedDataColumnIndexerTest.java b/processing/src/test/java/org/apache/druid/segment/NestedDataColumnIndexerTest.java
index d489bb5c1c..ce28ed0066 100644
--- a/processing/src/test/java/org/apache/druid/segment/NestedDataColumnIndexerTest.java
+++ b/processing/src/test/java/org/apache/druid/segment/NestedDataColumnIndexerTest.java
@@ -94,34 +94,34 @@ public class NestedDataColumnIndexerTest extends InitializedNullHandlingTest
 
     // new raw value, new fields
     key = indexer.processRowValsToUnsortedEncodedKeyComponent(ImmutableList.of(1L, 2L, 10L), false);
-    Assert.assertEquals(276, key.getEffectiveSizeBytes());
-    Assert.assertEquals(5, indexer.getCardinality());
+    Assert.assertEquals(168, key.getEffectiveSizeBytes());
+    Assert.assertEquals(6, indexer.getCardinality());
     // new raw value, re-use fields and dictionary
     key = indexer.processRowValsToUnsortedEncodedKeyComponent(ImmutableList.of(1L, 2L, 10L), false);
-    Assert.assertEquals(56, key.getEffectiveSizeBytes());
-    Assert.assertEquals(5, indexer.getCardinality());
+    Assert.assertEquals(104, key.getEffectiveSizeBytes());
+    Assert.assertEquals(6, indexer.getCardinality());
     // new raw value, new fields
     key = indexer.processRowValsToUnsortedEncodedKeyComponent(
         ImmutableMap.of("x", ImmutableList.of(1L, 2L, 10L)),
         false
     );
-    Assert.assertEquals(286, key.getEffectiveSizeBytes());
-    Assert.assertEquals(5, indexer.getCardinality());
+    Assert.assertEquals(166, key.getEffectiveSizeBytes());
+    Assert.assertEquals(6, indexer.getCardinality());
     // new raw value
     key = indexer.processRowValsToUnsortedEncodedKeyComponent(
         ImmutableMap.of("x", ImmutableList.of(1L, 2L, 10L)),
         false
     );
-    Assert.assertEquals(118, key.getEffectiveSizeBytes());
-    Assert.assertEquals(5, indexer.getCardinality());
+    Assert.assertEquals(166, key.getEffectiveSizeBytes());
+    Assert.assertEquals(6, indexer.getCardinality());
 
     key = indexer.processRowValsToUnsortedEncodedKeyComponent("", false);
     if (NullHandling.replaceWithDefault()) {
       Assert.assertEquals(0, key.getEffectiveSizeBytes());
-      Assert.assertEquals(6, indexer.getCardinality());
+      Assert.assertEquals(7, indexer.getCardinality());
     } else {
       Assert.assertEquals(104, key.getEffectiveSizeBytes());
-      Assert.assertEquals(6, indexer.getCardinality());
+      Assert.assertEquals(7, indexer.getCardinality());
     }
   }
 
@@ -364,7 +364,7 @@ public class NestedDataColumnIndexerTest extends InitializedNullHandlingTest
         UnsupportedOperationException.class,
         () -> cursorList.get(0).getColumnSelectorFactory().makeDimensionSelector(dimensionSpec)
     );
-    Assert.assertEquals(StructuredData.wrap(new Object[]{"a"}), valueSelector.getObject());
+    Assert.assertArrayEquals(new Object[]{"a"}, (Object[]) valueSelector.getObject());
 
     columnSelectorFactory = cursorList.get(1).getColumnSelectorFactory();
     valueSelector = columnSelectorFactory.makeColumnValueSelector(STRING_ARRAY_COL);
@@ -372,8 +372,7 @@ public class NestedDataColumnIndexerTest extends InitializedNullHandlingTest
         UnsupportedOperationException.class,
         () -> cursorList.get(1).getColumnSelectorFactory().makeDimensionSelector(dimensionSpec)
     );
-    Assert.assertEquals(StructuredData.wrap(new Object[]{"b", "c"}), valueSelector.getObject());
-    Assert.assertFalse(valueSelector.isNull());
+    Assert.assertArrayEquals(new Object[]{"b", "c"}, (Object[]) valueSelector.getObject());
 
     columnSelectorFactory = cursorList.get(2).getColumnSelectorFactory();
     valueSelector = columnSelectorFactory.makeColumnValueSelector(STRING_ARRAY_COL);
@@ -381,9 +380,7 @@ public class NestedDataColumnIndexerTest extends InitializedNullHandlingTest
         UnsupportedOperationException.class,
         () -> cursorList.get(2).getColumnSelectorFactory().makeDimensionSelector(dimensionSpec)
     );
-    // raw data is left as is, so is currently still a list while in incremental index...
-    Assert.assertEquals(StructuredData.wrap(ImmutableList.of("d", "e")), valueSelector.getObject());
-    Assert.assertFalse(valueSelector.isNull());
+    Assert.assertArrayEquals(new Object[]{"d", "e"}, (Object[]) valueSelector.getObject());
 
     columnSelectorFactory = cursorList.get(3).getColumnSelectorFactory();
     valueSelector = columnSelectorFactory.makeColumnValueSelector(STRING_ARRAY_COL);
diff --git a/processing/src/test/java/org/apache/druid/segment/UnnestColumnValueSelectorCursorTest.java b/processing/src/test/java/org/apache/druid/segment/UnnestColumnValueSelectorCursorTest.java
index bafb50f708..475f268338 100644
--- a/processing/src/test/java/org/apache/druid/segment/UnnestColumnValueSelectorCursorTest.java
+++ b/processing/src/test/java/org/apache/druid/segment/UnnestColumnValueSelectorCursorTest.java
@@ -21,7 +21,6 @@ package org.apache.druid.segment;
 
 import org.apache.druid.common.config.NullHandling;
 import org.apache.druid.math.expr.ExprMacroTable;
-import org.apache.druid.math.expr.ExpressionProcessing;
 import org.apache.druid.query.dimension.DefaultDimensionSpec;
 import org.apache.druid.query.monomorphicprocessing.StringRuntimeShape;
 import org.apache.druid.segment.column.ColumnType;
@@ -45,13 +44,11 @@ public class UnnestColumnValueSelectorCursorTest extends InitializedNullHandling
   public static void setUpClass()
   {
     NullHandling.initializeForTests();
-    ExpressionProcessing.initializeForTests(true); // Allow nested arrays
   }
 
   @AfterClass
   public static void tearDownClass()
   {
-    ExpressionProcessing.initializeForTests(null); // Clear special expression-processing config.
   }
 
   @Test
@@ -123,7 +120,8 @@ public class UnnestColumnValueSelectorCursorTest extends InitializedNullHandling
     List<Object> inputList = Arrays.asList(
         Collections.singletonList(null),
         Arrays.asList(null, null),
-        Collections.singletonList(null)
+        Collections.singletonList(null),
+        Collections.emptyList()
     );
 
     //Create base cursor
@@ -145,6 +143,74 @@ public class UnnestColumnValueSelectorCursorTest extends InitializedNullHandling
       k++;
       unnestCursor.advance();
     }
+    // since type is 'STRING', it follows multi-value string rules so single element arrays become scalar values,
+    // so [null] becomes null, meaning we only have 2 rows
+    Assert.assertEquals(k, 2);
+  }
+
+  @Test
+  public void test_list_unnest_cursors_user_supplied_list_only_nulls_mv_to_array()
+  {
+    List<Object> inputList = Arrays.asList(
+        Collections.singletonList(null),
+        Arrays.asList(null, null),
+        Collections.singletonList(null),
+        Collections.emptyList()
+    );
+
+    //Create base cursor
+    ListCursor listCursor = new ListCursor(inputList);
+
+    //Create unnest cursor
+    UnnestColumnValueSelectorCursor unnestCursor = new UnnestColumnValueSelectorCursor(
+        listCursor,
+        listCursor.getColumnSelectorFactory(),
+        new ExpressionVirtualColumn("__unnest__", "mv_to_array(\"dummy\")", ColumnType.STRING, ExprMacroTable.nil()),
+        OUTPUT_NAME
+    );
+    ColumnValueSelector unnestColumnValueSelector = unnestCursor.getColumnSelectorFactory()
+                                                                .makeColumnValueSelector(OUTPUT_NAME);
+    int k = 0;
+    while (!unnestCursor.isDone()) {
+      Object valueSelectorVal = unnestColumnValueSelector.getObject();
+      Assert.assertNull(valueSelectorVal);
+      k++;
+      unnestCursor.advance();
+    }
+    // since type is 'STRING', it follows multi-value string rules so single element arrays become scalar values,
+    // so [null] becomes null, meaning we only have 2 rows
+    Assert.assertEquals(k, 2);
+  }
+
+  @Test
+  public void test_list_unnest_cursors_user_supplied_list_only_nulls_array()
+  {
+    List<Object> inputList = Arrays.asList(
+        Collections.singletonList(null),
+        Arrays.asList(null, null),
+        Collections.singletonList(null),
+        Collections.emptyList()
+    );
+
+    //Create base cursor
+    ListCursor listCursor = new ListCursor(inputList);
+
+    //Create unnest cursor
+    UnnestColumnValueSelectorCursor unnestCursor = new UnnestColumnValueSelectorCursor(
+        listCursor,
+        listCursor.getColumnSelectorFactory(),
+        new ExpressionVirtualColumn("__unnest__", "\"dummy\"", ColumnType.STRING_ARRAY, ExprMacroTable.nil()),
+        OUTPUT_NAME
+    );
+    ColumnValueSelector unnestColumnValueSelector = unnestCursor.getColumnSelectorFactory()
+                                                                .makeColumnValueSelector(OUTPUT_NAME);
+    int k = 0;
+    while (!unnestCursor.isDone()) {
+      Object valueSelectorVal = unnestColumnValueSelector.getObject();
+      Assert.assertNull(valueSelectorVal);
+      k++;
+      unnestCursor.advance();
+    }
     Assert.assertEquals(k, 4);
   }
 
@@ -155,12 +221,15 @@ public class UnnestColumnValueSelectorCursorTest extends InitializedNullHandling
         Arrays.asList("a", "b"),
         Arrays.asList("b", "c"),
         "d",
+        Collections.singletonList(null),
+        Arrays.asList(null, null),
+        Collections.emptyList(),
         null,
         null,
         null
     );
 
-    List<String> expectedResults = Arrays.asList("a", "b", "b", "c", "d", null, null, null);
+    List<String> expectedResults = Arrays.asList("a", "b", "b", "c", "d", null, null);
 
     //Create base cursor
     ListCursor listCursor = new ListCursor(inputList);
@@ -178,7 +247,52 @@ public class UnnestColumnValueSelectorCursorTest extends InitializedNullHandling
     while (!unnestCursor.isDone()) {
       Object valueSelectorVal = unnestColumnValueSelector.getObject();
       if (valueSelectorVal == null) {
-        Assert.assertEquals(expectedResults.get(k), null);
+        Assert.assertNull(expectedResults.get(k));
+      } else {
+        Assert.assertEquals(expectedResults.get(k), valueSelectorVal.toString());
+      }
+      k++;
+      unnestCursor.advance();
+    }
+    // since type is 'STRING', it follows multi-value string rules so single element arrays become scalar values,
+    // so [null] becomes null, meaning we only have 7 rows
+    Assert.assertEquals(k, 7);
+  }
+
+  @Test
+  public void test_list_unnest_cursors_user_supplied_list_mixed_with_nulls_array()
+  {
+    List<Object> inputList = Arrays.asList(
+        Arrays.asList("a", "b"),
+        Arrays.asList("b", "c"),
+        "d",
+        Collections.singletonList(null),
+        Arrays.asList(null, null),
+        Collections.emptyList(),
+        null,
+        null,
+        null
+    );
+
+    List<String> expectedResults = Arrays.asList("a", "b", "b", "c", "d", null, null, null);
+
+    //Create base cursor
+    ListCursor listCursor = new ListCursor(inputList);
+
+    //Create unnest cursor
+    UnnestColumnValueSelectorCursor unnestCursor = new UnnestColumnValueSelectorCursor(
+        listCursor,
+        listCursor.getColumnSelectorFactory(),
+        new ExpressionVirtualColumn("__unnest__", "\"dummy\"", ColumnType.STRING_ARRAY, ExprMacroTable.nil()),
+        OUTPUT_NAME
+    );
+    ColumnValueSelector unnestColumnValueSelector = unnestCursor.getColumnSelectorFactory()
+                                                                .makeColumnValueSelector(OUTPUT_NAME);
+    int k = 0;
+    while (!unnestCursor.isDone()) {
+      Object valueSelectorVal = unnestColumnValueSelector.getObject();
+      if (valueSelectorVal == null) {
+        Assert.assertNull(expectedResults.get(k));
       } else {
         Assert.assertEquals(expectedResults.get(k), valueSelectorVal.toString());
       }
@@ -568,7 +682,7 @@ public class UnnestColumnValueSelectorCursorTest extends InitializedNullHandling
       k++;
       unnestCursor.advance();
     }
-    Assert.assertEquals(k, 9);
+    Assert.assertEquals(k, 8);
     unnestCursor.reset();
     Assert.assertNotNull(unnestDimSelector);
   }
diff --git a/processing/src/test/java/org/apache/druid/segment/data/FrontCodedIndexedTest.java b/processing/src/test/java/org/apache/druid/segment/data/FrontCodedIndexedTest.java
index 88a4a5d3ac..4c1c51231a 100644
--- a/processing/src/test/java/org/apache/druid/segment/data/FrontCodedIndexedTest.java
+++ b/processing/src/test/java/org/apache/druid/segment/data/FrontCodedIndexedTest.java
@@ -68,7 +68,7 @@ public class FrontCodedIndexedTest extends InitializedNullHandlingTest
   {
     ByteBuffer buffer = ByteBuffer.allocate(1 << 12).order(order);
     List<String> theList = ImmutableList.of("hello", "helloo", "hellooo", "hellooz", "helloozy");
-    fillBuffer(buffer, theList, 4, useIncrementalBuckets);
+    persistToBuffer(buffer, theList, 4, useIncrementalBuckets);
 
     buffer.position(0);
     FrontCodedIndexed codedUtf8Indexed = FrontCodedIndexed.read(
@@ -99,7 +99,7 @@ public class FrontCodedIndexedTest extends InitializedNullHandlingTest
   {
     ByteBuffer buffer = ByteBuffer.allocate(1 << 12).order(order);
     List<String> theList = ImmutableList.of("hello", "helloo", "hellooo", "hellooz", "helloozy");
-    fillBuffer(buffer, theList, 16, useIncrementalBuckets);
+    persistToBuffer(buffer, theList, 16, useIncrementalBuckets);
 
     FrontCodedIndexed codedUtf8Indexed = FrontCodedIndexed.read(
         buffer,
@@ -137,7 +137,7 @@ public class FrontCodedIndexedTest extends InitializedNullHandlingTest
       for (int i = 0; i < sizeBase + sizeAdjust; i++) {
         values.add(IdUtils.getRandomId() + IdUtils.getRandomId() + IdUtils.getRandomId() + IdUtils.getRandomId());
       }
-      fillBuffer(buffer, values, bucketSize, useIncrementalBuckets);
+      persistToBuffer(buffer, values, bucketSize, useIncrementalBuckets);
 
       FrontCodedIndexed codedUtf8Indexed = FrontCodedIndexed.read(
           buffer,
@@ -173,7 +173,7 @@ public class FrontCodedIndexedTest extends InitializedNullHandlingTest
       for (int i = 0; i < sizeBase + sizeAdjust; i++) {
         values.add(IdUtils.getRandomId() + IdUtils.getRandomId() + IdUtils.getRandomId() + IdUtils.getRandomId());
       }
-      fillBuffer(buffer, values, bucketSize, useIncrementalBuckets);
+      persistToBuffer(buffer, values, bucketSize, useIncrementalBuckets);
 
       FrontCodedIndexed codedUtf8Indexed = FrontCodedIndexed.read(
           buffer,
@@ -207,7 +207,7 @@ public class FrontCodedIndexedTest extends InitializedNullHandlingTest
     ByteBuffer buffer = ByteBuffer.allocate(1 << 12).order(order);
     List<String> theList = ImmutableList.of("hello", "helloo", "hellooo", "hellooz", "helloozy");
 
-    fillBuffer(buffer, theList, 4, useIncrementalBuckets);
+    persistToBuffer(buffer, theList, 4, useIncrementalBuckets);
 
     FrontCodedIndexed codedUtf8Indexed = FrontCodedIndexed.read(
         buffer,
@@ -231,7 +231,7 @@ public class FrontCodedIndexedTest extends InitializedNullHandlingTest
     TreeSet<String> values = new TreeSet<>(GenericIndexed.STRING_STRATEGY);
     values.add(null);
     values.addAll(theList);
-    fillBuffer(buffer, values, 4, useIncrementalBuckets);
+    persistToBuffer(buffer, values, 4, useIncrementalBuckets);
 
     FrontCodedIndexed codedUtf8Indexed = FrontCodedIndexed.read(
         buffer,
@@ -254,7 +254,7 @@ public class FrontCodedIndexedTest extends InitializedNullHandlingTest
 
     // "\uD83D\uDCA9" and "(請參見已被刪除版本)" are a regression test for https://github.com/apache/druid/pull/13364
     List<String> theList = ImmutableList.of("Győ-Moson-Sopron", "Győr", "\uD83D\uDCA9", "(請參見已被刪除版本)");
-    fillBuffer(buffer, theList, 4, useIncrementalBuckets);
+    persistToBuffer(buffer, theList, 4, useIncrementalBuckets);
 
     buffer.position(0);
     FrontCodedIndexed codedUtf8Indexed = FrontCodedIndexed.read(
@@ -282,7 +282,7 @@ public class FrontCodedIndexedTest extends InitializedNullHandlingTest
   {
     ByteBuffer buffer = ByteBuffer.allocate(1 << 12).order(order);
     List<String> theList = Collections.singletonList(null);
-    fillBuffer(buffer, theList, 4, useIncrementalBuckets);
+    persistToBuffer(buffer, theList, 4, useIncrementalBuckets);
 
     buffer.position(0);
     FrontCodedIndexed codedUtf8Indexed = FrontCodedIndexed.read(
@@ -303,6 +303,32 @@ public class FrontCodedIndexedTest extends InitializedNullHandlingTest
     Assert.assertFalse(utf8Iterator.hasNext());
   }
 
+  @Test
+  public void testFrontCodedEmpty() throws IOException
+  {
+    ByteBuffer buffer = ByteBuffer.allocate(1 << 6).order(order);
+    List<String> theList = Collections.emptyList();
+    persistToBuffer(buffer, theList, 4, useIncrementalBuckets);
+
+    buffer.position(0);
+    FrontCodedIndexed codedUtf8Indexed = FrontCodedIndexed.read(
+        buffer,
+        buffer.order()
+    ).get();
+
+    Assert.assertEquals(0, codedUtf8Indexed.size());
+    Throwable t = Assert.assertThrows(IAE.class, () -> codedUtf8Indexed.get(0));
+    Assert.assertEquals("Index[0] >= size[0]", t.getMessage());
+    Assert.assertThrows(IllegalArgumentException.class, () -> codedUtf8Indexed.get(-1));
+    Assert.assertThrows(IllegalArgumentException.class, () -> codedUtf8Indexed.get(theList.size()));
+
+    Assert.assertEquals(-1, codedUtf8Indexed.indexOf(null));
+    Assert.assertEquals(-1, codedUtf8Indexed.indexOf(StringUtils.toUtf8ByteBuffer("hello")));
+
+    Iterator<ByteBuffer> utf8Iterator = codedUtf8Indexed.iterator();
+    Assert.assertFalse(utf8Iterator.hasNext());
+  }
+
   @Test
   public void testBucketSizes() throws IOException
   {
@@ -325,7 +351,7 @@ public class FrontCodedIndexedTest extends InitializedNullHandlingTest
       values.add(IdUtils.getRandomId() + IdUtils.getRandomId() + IdUtils.getRandomId() + IdUtils.getRandomId());
     }
     for (int bucketSize : bucketSizes) {
-      fillBuffer(buffer, values, bucketSize, useIncrementalBuckets);
+      persistToBuffer(buffer, values, bucketSize, useIncrementalBuckets);
       FrontCodedIndexed codedUtf8Indexed = FrontCodedIndexed.read(
           buffer,
           buffer.order()
@@ -388,7 +414,7 @@ public class FrontCodedIndexedTest extends InitializedNullHandlingTest
     );
   }
 
-  private static long fillBuffer(ByteBuffer buffer, Iterable<String> sortedIterable, int bucketSize, boolean useIncrementalBuckets) throws IOException
+  private static long persistToBuffer(ByteBuffer buffer, Iterable<String> sortedIterable, int bucketSize, boolean useIncrementalBuckets) throws IOException
   {
     Iterator<String> sortedStrings = sortedIterable.iterator();
     buffer.position(0);
diff --git a/processing/src/test/java/org/apache/druid/segment/data/FrontCodedIntArrayIndexedTest.java b/processing/src/test/java/org/apache/druid/segment/data/FrontCodedIntArrayIndexedTest.java
new file mode 100644
index 0000000000..8a08a2a972
--- /dev/null
+++ b/processing/src/test/java/org/apache/druid/segment/data/FrontCodedIntArrayIndexedTest.java
@@ -0,0 +1,467 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.druid.segment.data;
+
+import com.google.common.collect.ImmutableList;
+import junitparams.converters.Nullable;
+import org.apache.druid.java.util.common.IAE;
+import org.apache.druid.java.util.common.StringUtils;
+import org.apache.druid.segment.writeout.OnHeapMemorySegmentWriteOutMedium;
+import org.junit.Assert;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.ByteOrder;
+import java.nio.channels.WritableByteChannel;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.TreeSet;
+import java.util.concurrent.ThreadLocalRandom;
+
+@RunWith(Parameterized.class)
+public class FrontCodedIntArrayIndexedTest
+{
+  @Parameterized.Parameters(name = "{0}")
+  public static Collection<Object[]> constructorFeeder()
+  {
+    return ImmutableList.of(new Object[]{ByteOrder.LITTLE_ENDIAN}, new Object[]{ByteOrder.BIG_ENDIAN});
+  }
+
+  private final ByteOrder order;
+
+  public FrontCodedIntArrayIndexedTest(ByteOrder byteOrder)
+  {
+    this.order = byteOrder;
+  }
+
+  @Test
+  public void testFrontCodedIntArrayIndexed() throws IOException
+  {
+    ByteBuffer buffer = ByteBuffer.allocate(1 << 12).order(order);
+    TreeSet<int[]> values = new TreeSet<>(FrontCodedIntArrayIndexedWriter.ARRAY_COMPARATOR);
+    values.add(new int[]{1, 2, 3});
+    values.add(new int[]{1, 2});
+    values.add(new int[]{1, 3});
+    values.add(new int[]{1, 2, 4});
+    values.add(new int[]{1, 3, 4});
+    values.add(new int[]{1, 2, 1});
+    values.add(new int[]{2, 1});
+    values.add(new int[]{2, 2, 1});
+
+    persistToBuffer(buffer, values, 4);
+
+    buffer.position(0);
+    FrontCodedIntArrayIndexed codedIndexed = FrontCodedIntArrayIndexed.read(
+        buffer,
+        buffer.order()
+    ).get();
+
+    Iterator<int[]> indexedIterator = codedIndexed.iterator();
+    Iterator<int[]> expectedIterator = values.iterator();
+    int ctr = 0;
+    while (expectedIterator.hasNext() && indexedIterator.hasNext()) {
+      final int[] expectedNext = expectedIterator.next();
+      final int[] next = indexedIterator.next();
+      assertSame(ctr, expectedNext, next);
+      assertSame(ctr, expectedNext, codedIndexed.get(ctr));
+      Assert.assertEquals("row " + ctr, ctr, codedIndexed.indexOf(next));
+      ctr++;
+    }
+    Assert.assertEquals(expectedIterator.hasNext(), indexedIterator.hasNext());
+  }
+
+
+  @Test
+  public void testFrontCodedIntArrayIndexedSingleBucket() throws IOException
+  {
+    ByteBuffer buffer = ByteBuffer.allocate(1 << 12).order(order);
+    TreeSet<int[]> values = new TreeSet<>(FrontCodedIntArrayIndexedWriter.ARRAY_COMPARATOR);
+    values.add(new int[]{1, 2, 3});
+    values.add(new int[]{1, 2});
+    values.add(new int[]{1, 3});
+    values.add(new int[]{1, 2, 4});
+    values.add(new int[]{1, 3, 4});
+    values.add(new int[]{1, 2, 1});
+    persistToBuffer(buffer, values, 16);
+
+    FrontCodedIntArrayIndexed codedIndexed = FrontCodedIntArrayIndexed.read(
+        buffer,
+        buffer.order()
+    ).get();
+
+    Iterator<int[]> expectedIterator = values.iterator();
+    Iterator<int[]> indexedIterator = codedIndexed.iterator();
+    int ctr = 0;
+    while (indexedIterator.hasNext() && expectedIterator.hasNext()) {
+      final int[] expectedNext = expectedIterator.next();
+      final int[] next = indexedIterator.next();
+      assertSame(ctr, expectedNext, next);
+      assertSame(ctr, expectedNext, codedIndexed.get(ctr));
+      Assert.assertEquals(ctr, codedIndexed.indexOf(next));
+      ctr++;
+    }
+    Assert.assertEquals(expectedIterator.hasNext(), indexedIterator.hasNext());
+  }
+
+  @Test
+  public void testFrontCodedIntArrayIndexedBigger() throws IOException
+  {
+    final int sizeBase = 10000;
+    final int bucketSize = 16;
+    final ByteBuffer buffer = ByteBuffer.allocate(1 << 24).order(order);
+    for (int sizeAdjust = 0; sizeAdjust < bucketSize; sizeAdjust++) {
+      final TreeSet<int[]> values = new TreeSet<>(FrontCodedIntArrayIndexedWriter.ARRAY_COMPARATOR);
+      while (values.size() < sizeBase + sizeAdjust) {
+        int length = ThreadLocalRandom.current().nextInt(10);
+        final int[] val = new int[length];
+        for (int j = 0; j < length; j++) {
+          val[j] = ThreadLocalRandom.current().nextInt(0, 10_000);
+        }
+        values.add(val);
+      }
+      persistToBuffer(buffer, values, bucketSize);
+
+      FrontCodedIntArrayIndexed codedIndexed = FrontCodedIntArrayIndexed.read(
+          buffer,
+          buffer.order()
+      ).get();
+
+      Iterator<int[]> expectedIterator = values.iterator();
+      Iterator<int[]> indexedIterator = codedIndexed.iterator();
+      int ctr = 0;
+      while (indexedIterator.hasNext() && expectedIterator.hasNext()) {
+        final int[] expectedNext = expectedIterator.next();
+        final int[] next = indexedIterator.next();
+        assertSame(ctr, expectedNext, next);
+        assertSame(ctr, expectedNext, codedIndexed.get(ctr));
+        Assert.assertEquals(ctr, codedIndexed.indexOf(next));
+        ctr++;
+      }
+      Assert.assertEquals(expectedIterator.hasNext(), indexedIterator.hasNext());
+      Assert.assertEquals(ctr, sizeBase + sizeAdjust);
+    }
+  }
+
+  @Test
+  public void testFrontCodedIntArrayIndexedBiggerWithNulls() throws IOException
+  {
+    final int sizeBase = 10000;
+    final int bucketSize = 16;
+    final ByteBuffer buffer = ByteBuffer.allocate(1 << 25).order(order);
+    for (int sizeAdjust = 0; sizeAdjust < bucketSize; sizeAdjust++) {
+      TreeSet<int[]> values = new TreeSet<>(FrontCodedIntArrayIndexedWriter.ARRAY_COMPARATOR);
+      values.add(null);
+      while (values.size() < sizeBase + sizeAdjust + 1) {
+        int length = ThreadLocalRandom.current().nextInt(10);
+        final int[] val = new int[length];
+        for (int j = 0; j < length; j++) {
+          val[j] = ThreadLocalRandom.current().nextInt(0, 10_000);
+        }
+        values.add(val);
+      }
+      persistToBuffer(buffer, values, bucketSize);
+
+      FrontCodedIntArrayIndexed codedIndexed = FrontCodedIntArrayIndexed.read(
+          buffer,
+          buffer.order()
+      ).get();
+
+      Iterator<int[]> expectedIterator = values.iterator();
+      Iterator<int[]> indexedIterator = codedIndexed.iterator();
+      int ctr = 0;
+      while (indexedIterator.hasNext() && expectedIterator.hasNext()) {
+        final int[] expectedNext = expectedIterator.next();
+        final int[] next = indexedIterator.next();
+        assertSame(ctr, expectedNext, next);
+        assertSame(ctr, expectedNext, codedIndexed.get(ctr));
+        Assert.assertEquals(ctr, codedIndexed.indexOf(next));
+        ctr++;
+      }
+      Assert.assertEquals(expectedIterator.hasNext(), indexedIterator.hasNext());
+      Assert.assertEquals(ctr, sizeBase + sizeAdjust + 1);
+    }
+  }
+
+  @Test
+  public void testFrontCodedIntArrayIndexedIndexOf() throws IOException
+  {
+    ByteBuffer buffer = ByteBuffer.allocate(1 << 12).order(order);
+    TreeSet<int[]> values = new TreeSet<>(FrontCodedIntArrayIndexedWriter.ARRAY_COMPARATOR);
+    values.add(new int[]{1, 2});
+    values.add(new int[]{1, 2, 1});
+    values.add(new int[]{1, 2, 3});
+    values.add(new int[]{1, 2, 4});
+    values.add(new int[]{1, 3});
+    values.add(new int[]{1, 3, 4});
+
+    persistToBuffer(buffer, values, 4);
+
+    FrontCodedIntArrayIndexed codedIndexed = FrontCodedIntArrayIndexed.read(
+        buffer,
+        buffer.order()
+    ).get();
+    Assert.assertEquals(-1, codedIndexed.indexOf(new int[]{1}));
+    Assert.assertEquals(0, codedIndexed.indexOf(new int[]{1, 2}));
+    Assert.assertEquals(1, codedIndexed.indexOf(new int[]{1, 2, 1}));
+    Assert.assertEquals(-3, codedIndexed.indexOf(new int[]{1, 2, 2}));
+    Assert.assertEquals(4, codedIndexed.indexOf(new int[]{1, 3}));
+    Assert.assertEquals(-7, codedIndexed.indexOf(new int[]{1, 4, 4}));
+    Assert.assertEquals(-7, codedIndexed.indexOf(new int[]{9, 1, 1}));
+  }
+
+
+  @Test
+  public void testFrontCodedIntArrayIndexedIndexOfWithNull() throws IOException
+  {
+    ByteBuffer buffer = ByteBuffer.allocate(1 << 12).order(order);
+    TreeSet<int[]> values = new TreeSet<>(FrontCodedIntArrayIndexedWriter.ARRAY_COMPARATOR);
+    values.add(null);
+    values.add(new int[]{1, 2});
+    values.add(new int[]{1, 2, 1});
+    values.add(new int[]{1, 2, 3});
+    values.add(new int[]{1, 2, 4});
+    values.add(new int[]{1, 3});
+    values.add(new int[]{1, 3, 4});
+    persistToBuffer(buffer, values, 4);
+
+    FrontCodedIntArrayIndexed codedIndexed = FrontCodedIntArrayIndexed.read(
+        buffer,
+        buffer.order()
+    ).get();
+    Assert.assertEquals(0, codedIndexed.indexOf(null));
+    Assert.assertEquals(-2, codedIndexed.indexOf(new int[]{1}));
+    Assert.assertEquals(1, codedIndexed.indexOf(new int[]{1, 2}));
+    Assert.assertEquals(2, codedIndexed.indexOf(new int[]{1, 2, 1}));
+    Assert.assertEquals(-4, codedIndexed.indexOf(new int[]{1, 2, 2}));
+    Assert.assertEquals(5, codedIndexed.indexOf(new int[]{1, 3}));
+    Assert.assertEquals(-8, codedIndexed.indexOf(new int[]{1, 4, 4}));
+    Assert.assertEquals(-8, codedIndexed.indexOf(new int[]{9, 1, 1}));
+  }
+
+
+  @Test
+  public void testFrontCodedOnlyNull() throws IOException
+  {
+    ByteBuffer buffer = ByteBuffer.allocate(1 << 12).order(order);
+    List<int[]> theList = Collections.singletonList(null);
+    persistToBuffer(buffer, theList, 4);
+
+    buffer.position(0);
+    FrontCodedIntArrayIndexed codedIndexed = FrontCodedIntArrayIndexed.read(
+        buffer,
+        buffer.order()
+    ).get();
+
+    Assert.assertNull(codedIndexed.get(0));
+    Assert.assertThrows(IllegalArgumentException.class, () -> codedIndexed.get(-1));
+    Assert.assertThrows(IllegalArgumentException.class, () -> codedIndexed.get(theList.size()));
+
+    Assert.assertEquals(0, codedIndexed.indexOf(null));
+    Assert.assertEquals(-2, codedIndexed.indexOf(new int[]{1, 2, 3, 4}));
+
+    Iterator<int[]> iterator = codedIndexed.iterator();
+    Assert.assertTrue(iterator.hasNext());
+    Assert.assertNull(iterator.next());
+    Assert.assertFalse(iterator.hasNext());
+  }
+
+  @Test
+  public void testFrontCodedEmpty() throws IOException
+  {
+    ByteBuffer buffer = ByteBuffer.allocate(1 << 6).order(order);
+    List<int[]> theList = Collections.emptyList();
+    persistToBuffer(buffer, theList, 4);
+
+    buffer.position(0);
+    FrontCodedIndexed codedUtf8Indexed = FrontCodedIndexed.read(
+        buffer,
+        buffer.order()
+    ).get();
+
+    Assert.assertEquals(0, codedUtf8Indexed.size());
+    Throwable t = Assert.assertThrows(IAE.class, () -> codedUtf8Indexed.get(0));
+    Assert.assertEquals("Index[0] >= size[0]", t.getMessage());
+    Assert.assertThrows(IllegalArgumentException.class, () -> codedUtf8Indexed.get(-1));
+    Assert.assertThrows(IllegalArgumentException.class, () -> codedUtf8Indexed.get(theList.size()));
+
+    Assert.assertEquals(-1, codedUtf8Indexed.indexOf(null));
+    Assert.assertEquals(-1, codedUtf8Indexed.indexOf(StringUtils.toUtf8ByteBuffer("hello")));
+
+    Iterator<ByteBuffer> utf8Iterator = codedUtf8Indexed.iterator();
+    Assert.assertFalse(utf8Iterator.hasNext());
+  }
+
+  @Test
+  public void testBucketSizes() throws IOException
+  {
+    final int numValues = 10000;
+    final ByteBuffer buffer = ByteBuffer.allocate(1 << 25).order(order);
+    final int[] bucketSizes = new int[]{
+        1,
+        1 << 1,
+        1 << 2,
+        1 << 3,
+        1 << 4,
+        1 << 5,
+        1 << 6,
+        1 << 7
+    };
+
+    TreeSet<int[]> values = new TreeSet<>(FrontCodedIntArrayIndexedWriter.ARRAY_COMPARATOR);
+    values.add(null);
+    while (values.size() < numValues + 1) {
+      int length = ThreadLocalRandom.current().nextInt(10);
+      final int[] val = new int[length];
+      for (int j = 0; j < length; j++) {
+        val[j] = ThreadLocalRandom.current().nextInt(0, 10_000);
+      }
+      values.add(val);
+    }
+    for (int bucketSize : bucketSizes) {
+      persistToBuffer(buffer, values, bucketSize);
+      FrontCodedIntArrayIndexed codedIndexed = FrontCodedIntArrayIndexed.read(
+          buffer,
+          buffer.order()
+      ).get();
+
+      Iterator<int[]> expectedIterator = values.iterator();
+      Iterator<int[]> iterator = codedIndexed.iterator();
+      int ctr = 0;
+      while (iterator.hasNext() && expectedIterator.hasNext()) {
+        final int[] expectedNext = expectedIterator.next();
+        final int[] next = iterator.next();
+        assertSame(ctr, expectedNext, next);
+        assertSame(ctr, expectedNext, codedIndexed.get(ctr));
+        Assert.assertEquals(ctr, codedIndexed.indexOf(next));
+        ctr++;
+      }
+      Assert.assertEquals(expectedIterator.hasNext(), iterator.hasNext());
+      Assert.assertEquals(ctr, numValues + 1);
+    }
+  }
+
+  @Test
+  public void testBadBucketSize()
+  {
+    OnHeapMemorySegmentWriteOutMedium medium = new OnHeapMemorySegmentWriteOutMedium();
+
+    Assert.assertThrows(
+        IAE.class,
+        () -> new FrontCodedIntArrayIndexedWriter(
+            medium,
+            ByteOrder.nativeOrder(),
+            0
+        )
+    );
+
+    Assert.assertThrows(
+        IAE.class,
+        () -> new FrontCodedIntArrayIndexedWriter(
+            medium,
+            ByteOrder.nativeOrder(),
+            15
+        )
+    );
+
+    Assert.assertThrows(
+        IAE.class,
+        () -> new FrontCodedIntArrayIndexedWriter(
+            medium,
+            ByteOrder.nativeOrder(),
+            256
+        )
+    );
+  }
+
+  private static long persistToBuffer(ByteBuffer buffer, Iterable<int[]> sortedIterable, int bucketSize) throws IOException
+  {
+    Iterator<int[]> sortedInts = sortedIterable.iterator();
+    buffer.position(0);
+    OnHeapMemorySegmentWriteOutMedium medium = new OnHeapMemorySegmentWriteOutMedium();
+    FrontCodedIntArrayIndexedWriter writer = new FrontCodedIntArrayIndexedWriter(
+        medium,
+        buffer.order(),
+        bucketSize
+    );
+    writer.open();
+    int index = 0;
+    while (sortedInts.hasNext()) {
+      final int[] next = sortedInts.next();
+      writer.write(next);
+      assertSame(index, next, writer.get(index));
+      index++;
+    }
+
+    // check 'get' again so that we aren't always reading from current page
+    index = 0;
+    sortedInts = sortedIterable.iterator();
+    while (sortedInts.hasNext()) {
+      assertSame(index, sortedInts.next(), writer.get(index));
+      index++;
+    }
+
+    WritableByteChannel channel = new WritableByteChannel()
+    {
+      @Override
+      public int write(ByteBuffer src)
+      {
+        int size = src.remaining();
+        buffer.put(src);
+        return size;
+      }
+
+      @Override
+      public boolean isOpen()
+      {
+        return true;
+      }
+
+      @Override
+      public void close()
+      {
+      }
+    };
+    long size = writer.getSerializedSize();
+    buffer.position(0);
+    writer.writeTo(channel, null);
+    Assert.assertEquals(size, buffer.position());
+    buffer.position(0);
+    return size;
+  }
+
+  private static void assertSame(int index, @Nullable int[] expected, @Nullable int[] actual)
+  {
+    if (expected == null) {
+      Assert.assertNull("row " + index, actual);
+    } else {
+      Assert.assertArrayEquals(
+          "row " + index + " expected: " + Arrays.toString(expected) + " actual: " + Arrays.toString(actual),
+          expected,
+          actual
+      );
+    }
+  }
+}
diff --git a/processing/src/test/java/org/apache/druid/segment/filter/ExpressionFilterTest.java b/processing/src/test/java/org/apache/druid/segment/filter/ExpressionFilterTest.java
index d0e089a7f7..289843f600 100644
--- a/processing/src/test/java/org/apache/druid/segment/filter/ExpressionFilterTest.java
+++ b/processing/src/test/java/org/apache/druid/segment/filter/ExpressionFilterTest.java
@@ -128,7 +128,7 @@ public class ExpressionFilterTest extends BaseFilterTest
   @After
   public void teardown()
   {
-    ExpressionProcessing.initializeForTests(null);
+    ExpressionProcessing.initializeForTests();
   }
 
   @AfterClass
diff --git a/processing/src/test/java/org/apache/druid/segment/nested/NestedDataColumnSupplierTest.java b/processing/src/test/java/org/apache/druid/segment/nested/NestedDataColumnSupplierTest.java
index 38764b093a..ba2be32008 100644
--- a/processing/src/test/java/org/apache/druid/segment/nested/NestedDataColumnSupplierTest.java
+++ b/processing/src/test/java/org/apache/druid/segment/nested/NestedDataColumnSupplierTest.java
@@ -26,7 +26,9 @@ import com.google.common.util.concurrent.Futures;
 import com.google.common.util.concurrent.ListenableFuture;
 import com.google.common.util.concurrent.ListeningExecutorService;
 import com.google.common.util.concurrent.MoreExecutors;
+import org.apache.druid.collections.bitmap.ImmutableBitmap;
 import org.apache.druid.collections.bitmap.RoaringBitmapFactory;
+import org.apache.druid.collections.bitmap.WrappedRoaringBitmap;
 import org.apache.druid.common.config.NullHandling;
 import org.apache.druid.guice.NestedDataModule;
 import org.apache.druid.java.util.common.concurrent.Execs;
@@ -42,6 +44,7 @@ import org.apache.druid.segment.ColumnValueSelector;
 import org.apache.druid.segment.DimensionSelector;
 import org.apache.druid.segment.IndexSpec;
 import org.apache.druid.segment.NestedDataColumnIndexer;
+import org.apache.druid.segment.NestedDataColumnMerger;
 import org.apache.druid.segment.ObjectColumnSelector;
 import org.apache.druid.segment.QueryableIndex;
 import org.apache.druid.segment.SimpleAscendingOffset;
@@ -55,6 +58,11 @@ import org.apache.druid.segment.column.DruidPredicateIndex;
 import org.apache.druid.segment.column.NullValueIndex;
 import org.apache.druid.segment.column.StringValueSetIndex;
 import org.apache.druid.segment.column.TypeStrategy;
+import org.apache.druid.segment.vector.BitmapVectorOffset;
+import org.apache.druid.segment.vector.NoFilterVectorOffset;
+import org.apache.druid.segment.vector.VectorObjectSelector;
+import org.apache.druid.segment.vector.VectorValueSelector;
+import org.apache.druid.segment.writeout.SegmentWriteOutMediumFactory;
 import org.apache.druid.segment.writeout.TmpFileSegmentWriteOutMediumFactory;
 import org.apache.druid.testing.InitializedNullHandlingTest;
 import org.apache.druid.utils.CompressionUtils;
@@ -102,12 +110,39 @@ public class NestedDataColumnSupplierTest extends InitializedNullHandlingTest
       TestHelper.makeMap("x", 4L, "y", 2.0, "z", "e", "v", 11111L, "nullish", null)
   );
 
+  List<Map<String, Object>> arrayTestData = ImmutableList.of(
+      TestHelper.makeMap("s", new Object[]{"a", "b", "c"}, "l", new Object[]{1L, 2L, 3L}, "d", new Object[]{1.1, 2.2}),
+      TestHelper.makeMap(
+          "s",
+          new Object[]{null, "b", "c"},
+          "l",
+          new Object[]{1L, null, 3L},
+          "d",
+          new Object[]{2.2, 2.2}
+      ),
+      TestHelper.makeMap(
+          "s",
+          new Object[]{"b", "c"},
+          "l",
+          new Object[]{null, null},
+          "d",
+          new Object[]{1.1, null, 2.2}
+      ),
+      TestHelper.makeMap("s", new Object[]{"a", "b", "c", "d"}, "l", new Object[]{4L, 2L, 3L}),
+      TestHelper.makeMap("s", new Object[]{"d", "b", "c", "a"}, "d", new Object[]{1.1, 2.2}),
+      TestHelper.makeMap("l", new Object[]{1L, 2L, 3L}, "d", new Object[]{3.1, 2.2, 1.9})
+  );
+
   Closer closer = Closer.create();
 
   SmooshedFileMapper fileMapper;
 
   ByteBuffer baseBuffer;
 
+  SmooshedFileMapper arrayFileMapper;
+
+  ByteBuffer arrayBaseBuffer;
+
   @BeforeClass
   public static void staticSetup()
   {
@@ -118,11 +153,22 @@ public class NestedDataColumnSupplierTest extends InitializedNullHandlingTest
   public void setup() throws IOException
   {
     final String fileNameBase = "test";
-    TmpFileSegmentWriteOutMediumFactory writeOutMediumFactory = TmpFileSegmentWriteOutMediumFactory.instance();
-    final File tmpFile = tempFolder.newFolder();
-    try (final FileSmoosher smoosher = new FileSmoosher(tmpFile)) {
-
+    final String arrayFileNameBase = "array";
+    fileMapper = smooshify(fileNameBase, tempFolder.newFolder(), data);
+    baseBuffer = fileMapper.mapFile(fileNameBase);
+    arrayFileMapper = smooshify(arrayFileNameBase, tempFolder.newFolder(), arrayTestData);
+    arrayBaseBuffer = arrayFileMapper.mapFile(arrayFileNameBase);
+  }
 
+  private SmooshedFileMapper smooshify(
+      String fileNameBase,
+      File tmpFile,
+      List<Map<String, Object>> data
+  )
+      throws IOException
+  {
+    SegmentWriteOutMediumFactory writeOutMediumFactory = TmpFileSegmentWriteOutMediumFactory.instance();
+    try (final FileSmoosher smoosher = new FileSmoosher(tmpFile)) {
       NestedDataColumnSerializer serializer = new NestedDataColumnSerializer(
           fileNameBase,
           new IndexSpec(),
@@ -135,16 +181,22 @@ public class NestedDataColumnSupplierTest extends InitializedNullHandlingTest
       for (Object o : data) {
         indexer.processRowValsToUnsortedEncodedKeyComponent(o, false);
       }
-      SortedMap<String, NestedLiteralTypeInfo.MutableTypeSet> sortedFields = new TreeMap<>();
+      SortedMap<String, NestedFieldTypeInfo.MutableTypeSet> sortedFields = new TreeMap<>();
       indexer.mergeFields(sortedFields);
 
       GlobalDictionarySortedCollector globalDictionarySortedCollector = indexer.getSortedCollector();
 
       serializer.open();
       serializer.serializeFields(sortedFields);
-      serializer.serializeStringDictionary(globalDictionarySortedCollector.getSortedStrings());
-      serializer.serializeLongDictionary(globalDictionarySortedCollector.getSortedLongs());
-      serializer.serializeDoubleDictionary(globalDictionarySortedCollector.getSortedDoubles());
+      serializer.serializeDictionaries(
+          globalDictionarySortedCollector.getSortedStrings(),
+          globalDictionarySortedCollector.getSortedLongs(),
+          globalDictionarySortedCollector.getSortedDoubles(),
+          () -> new NestedDataColumnMerger.ArrayDictionaryMergingIterator(
+              new Iterable[]{globalDictionarySortedCollector.getSortedArrays()},
+              serializer.getGlobalLookup()
+          )
+      );
 
       SettableSelector valueSelector = new SettableSelector();
       for (Object o : data) {
@@ -156,8 +208,7 @@ public class NestedDataColumnSupplierTest extends InitializedNullHandlingTest
         serializer.writeTo(writer, smoosher);
       }
       smoosher.close();
-      fileMapper = closer.register(SmooshedFileMapper.load(tmpFile));
-      baseBuffer = fileMapper.mapFile(fileNameBase);
+      return closer.register(SmooshedFileMapper.load(tmpFile));
     }
   }
 
@@ -185,6 +236,24 @@ public class NestedDataColumnSupplierTest extends InitializedNullHandlingTest
     }
   }
 
+  @Test
+  public void testArrayFunctionality() throws IOException
+  {
+    ColumnBuilder bob = new ColumnBuilder();
+    bob.setFileMapper(arrayFileMapper);
+    NestedDataColumnSupplier supplier = NestedDataColumnSupplier.read(
+        arrayBaseBuffer,
+        bob,
+        () -> 0,
+        NestedDataComplexTypeSerde.OBJECT_MAPPER,
+        new OnlyPositionalReadsTypeStrategy<>(ColumnType.LONG.getStrategy()),
+        new OnlyPositionalReadsTypeStrategy<>(ColumnType.DOUBLE.getStrategy())
+    );
+    try (NestedDataComplexColumn column = (NestedDataComplexColumn) supplier.get()) {
+      smokeTestArrays(column);
+    }
+  }
+
   @Test
   public void testConcurrency() throws ExecutionException, InterruptedException
   {
@@ -234,10 +303,10 @@ public class NestedDataColumnSupplierTest extends InitializedNullHandlingTest
     String columnName = "shipTo";
     String firstValue = "Cole";
     File tmpLocation = tempFolder.newFolder();
-    File v3Segment = new File(
-        NestedDataColumnSupplierTest.class.getClassLoader().getResource("nested_segment_v3/index.zip").getFile()
+    CompressionUtils.unzip(
+        NestedDataColumnSupplierTest.class.getClassLoader().getResourceAsStream("nested_segment_v3/index.zip"),
+        tmpLocation
     );
-    CompressionUtils.unzip(v3Segment, tmpLocation);
     try (Closer closer = Closer.create()) {
       QueryableIndex theIndex = closer.register(TestHelper.getTestIndexIO().loadIndex(tmpLocation));
       ColumnHolder holder = theIndex.getColumnHolder(columnName);
@@ -251,8 +320,8 @@ public class NestedDataColumnSupplierTest extends InitializedNullHandlingTest
       ColumnHolder nestedColumnHolder = v3.getColumnHolder(path);
       Assert.assertNotNull(nestedColumnHolder);
       Assert.assertEquals(ColumnType.STRING, nestedColumnHolder.getCapabilities().toColumnType());
-      NestedFieldLiteralDictionaryEncodedColumn<?> nestedColumn =
-          (NestedFieldLiteralDictionaryEncodedColumn<?>) nestedColumnHolder.getColumn();
+      NestedFieldDictionaryEncodedColumn<?> nestedColumn =
+          (NestedFieldDictionaryEncodedColumn<?>) nestedColumnHolder.getColumn();
 
       Assert.assertNotNull(nestedColumn);
 
@@ -271,6 +340,50 @@ public class NestedDataColumnSupplierTest extends InitializedNullHandlingTest
     }
   }
 
+  @Test
+  public void testLegacyV4ReaderFormat() throws IOException
+  {
+    String columnName = "shipTo";
+    // i accidentally didn't use same segment granularity for v3 and v4 segments... so they have different first value
+    String firstValue = "Beatty";
+    File tmpLocation = tempFolder.newFolder();
+    CompressionUtils.unzip(
+        NestedDataColumnSupplierTest.class.getClassLoader().getResourceAsStream("nested_segment_v4/index.zip"),
+        tmpLocation
+    );
+    try (Closer closer = Closer.create()) {
+      QueryableIndex theIndex = closer.register(TestHelper.getTestIndexIO().loadIndex(tmpLocation));
+      ColumnHolder holder = theIndex.getColumnHolder(columnName);
+      Assert.assertNotNull(holder);
+      Assert.assertEquals(NestedDataComplexTypeSerde.TYPE, holder.getCapabilities().toColumnType());
+
+      NestedDataColumnV4<?> v4 = closer.register((NestedDataColumnV4<?>) holder.getColumn());
+      Assert.assertNotNull(v4);
+
+      List<NestedPathPart> path = ImmutableList.of(new NestedPathField("lastName"));
+      ColumnHolder nestedColumnHolder = v4.getColumnHolder(path);
+      Assert.assertNotNull(nestedColumnHolder);
+      Assert.assertEquals(ColumnType.STRING, nestedColumnHolder.getCapabilities().toColumnType());
+      NestedFieldDictionaryEncodedColumn<?> nestedColumn =
+          (NestedFieldDictionaryEncodedColumn<?>) nestedColumnHolder.getColumn();
+
+      Assert.assertNotNull(nestedColumn);
+
+      ColumnValueSelector<?> selector = nestedColumn.makeColumnValueSelector(
+          new SimpleAscendingOffset(theIndex.getNumRows())
+      );
+
+      ColumnIndexSupplier indexSupplier = v4.getColumnIndexSupplier(path);
+      Assert.assertNotNull(indexSupplier);
+      StringValueSetIndex valueSetIndex = indexSupplier.as(StringValueSetIndex.class);
+      Assert.assertNotNull(valueSetIndex);
+
+      BitmapColumnIndex indexForValue = valueSetIndex.forValue(firstValue);
+      Assert.assertEquals(firstValue, selector.getObject());
+      Assert.assertTrue(indexForValue.computeBitmapResult(resultFactory).get(0));
+    }
+  }
+
   private void smokeTest(NestedDataComplexColumn column) throws IOException
   {
     SimpleAscendingOffset offset = new SimpleAscendingOffset(data.size());
@@ -310,7 +423,10 @@ public class NestedDataColumnSupplierTest extends InitializedNullHandlingTest
     NullValueIndex zNulls = zIndexSupplier.as(NullValueIndex.class);
 
     final List<NestedPathPart> vPath = NestedPathFinder.parseJsonPath("$.v");
-    Assert.assertEquals(ImmutableSet.of(ColumnType.STRING, ColumnType.LONG, ColumnType.DOUBLE), column.getColumnTypes(vPath));
+    Assert.assertEquals(
+        ImmutableSet.of(ColumnType.STRING, ColumnType.LONG, ColumnType.DOUBLE),
+        column.getColumnTypes(vPath)
+    );
     Assert.assertEquals(ColumnType.STRING, column.getColumnHolder(vPath).getCapabilities().toColumnType());
     ColumnValueSelector<?> vSelector = column.makeColumnValueSelector(vPath, offset);
     DimensionSelector vDimSelector = column.makeDimensionSelector(vPath, offset, null);
@@ -344,12 +460,265 @@ public class NestedDataColumnSupplierTest extends InitializedNullHandlingTest
       testPath(row, i, "x", xSelector, xDimSelector, xValueIndex, xPredicateIndex, xNulls, ColumnType.LONG);
       testPath(row, i, "y", ySelector, yDimSelector, yValueIndex, yPredicateIndex, yNulls, ColumnType.DOUBLE);
       testPath(row, i, "z", zSelector, zDimSelector, zValueIndex, zPredicateIndex, zNulls, ColumnType.STRING);
-      testPath(row, i, "nullish", nullishSelector, nullishDimSelector, nullishValueIndex, nullishPredicateIndex, nullishNulls, ColumnType.STRING);
+      testPath(
+          row,
+          i,
+          "nullish",
+          nullishSelector,
+          nullishDimSelector,
+          nullishValueIndex,
+          nullishPredicateIndex,
+          nullishNulls,
+          ColumnType.STRING
+      );
 
       offset.increment();
     }
   }
 
+  private void smokeTestArrays(NestedDataComplexColumn column) throws IOException
+  {
+    SimpleAscendingOffset offset = new SimpleAscendingOffset(arrayTestData.size());
+    NoFilterVectorOffset vectorOffset = new NoFilterVectorOffset(4, 0, arrayTestData.size());
+    WrappedRoaringBitmap bitmap = new WrappedRoaringBitmap();
+    for (int i = 0; i < arrayTestData.size(); i++) {
+      if (i % 2 == 0) {
+        bitmap.add(i);
+      }
+    }
+    BitmapVectorOffset bitmapVectorOffset = new BitmapVectorOffset(
+        4,
+        bitmap.toImmutableBitmap(),
+        0,
+        arrayTestData.size()
+    );
+
+    ColumnValueSelector<?> rawSelector = column.makeColumnValueSelector(offset);
+    VectorObjectSelector rawVectorSelector = column.makeVectorObjectSelector(vectorOffset);
+    VectorObjectSelector rawVectorSelectorFiltered = column.makeVectorObjectSelector(bitmapVectorOffset);
+
+    final List<NestedPathPart> sPath = NestedPathFinder.parseJsonPath("$.s");
+    Assert.assertEquals(ImmutableSet.of(ColumnType.STRING_ARRAY), column.getColumnTypes(sPath));
+    Assert.assertEquals(ColumnType.STRING_ARRAY, column.getColumnHolder(sPath).getCapabilities().toColumnType());
+    ColumnValueSelector<?> sSelector = column.makeColumnValueSelector(sPath, offset);
+    VectorObjectSelector sVectorSelector = column.makeVectorObjectSelector(sPath, vectorOffset);
+    VectorObjectSelector sVectorSelectorFiltered = column.makeVectorObjectSelector(sPath, bitmapVectorOffset);
+    ColumnIndexSupplier sIndexSupplier = column.getColumnIndexSupplier(sPath);
+    Assert.assertNotNull(sIndexSupplier);
+    Assert.assertNull(sIndexSupplier.as(StringValueSetIndex.class));
+    Assert.assertNull(sIndexSupplier.as(DruidPredicateIndex.class));
+    NullValueIndex sNulls = sIndexSupplier.as(NullValueIndex.class);
+
+    final List<NestedPathPart> sElementPath = NestedPathFinder.parseJsonPath("$.s[1]");
+    ColumnValueSelector<?> sElementSelector = column.makeColumnValueSelector(sElementPath, offset);
+    VectorObjectSelector sElementVectorSelector = column.makeVectorObjectSelector(sElementPath, vectorOffset);
+    VectorObjectSelector sElementFilteredVectorSelector = column.makeVectorObjectSelector(
+        sElementPath,
+        bitmapVectorOffset
+    );
+    ColumnIndexSupplier sElementIndexSupplier = column.getColumnIndexSupplier(sElementPath);
+    Assert.assertNotNull(sElementIndexSupplier);
+    Assert.assertNull(sElementIndexSupplier.as(StringValueSetIndex.class));
+    Assert.assertNull(sElementIndexSupplier.as(DruidPredicateIndex.class));
+    Assert.assertNull(sElementIndexSupplier.as(NullValueIndex.class));
+
+    final List<NestedPathPart> lPath = NestedPathFinder.parseJsonPath("$.l");
+    Assert.assertEquals(ImmutableSet.of(ColumnType.LONG_ARRAY), column.getColumnTypes(lPath));
+    Assert.assertEquals(ColumnType.LONG_ARRAY, column.getColumnHolder(lPath).getCapabilities().toColumnType());
+    ColumnValueSelector<?> lSelector = column.makeColumnValueSelector(lPath, offset);
+    VectorObjectSelector lVectorSelector = column.makeVectorObjectSelector(lPath, vectorOffset);
+    VectorObjectSelector lVectorSelectorFiltered = column.makeVectorObjectSelector(lPath, bitmapVectorOffset);
+    ColumnIndexSupplier lIndexSupplier = column.getColumnIndexSupplier(lPath);
+    Assert.assertNotNull(lIndexSupplier);
+    Assert.assertNull(lIndexSupplier.as(StringValueSetIndex.class));
+    Assert.assertNull(lIndexSupplier.as(DruidPredicateIndex.class));
+    NullValueIndex lNulls = lIndexSupplier.as(NullValueIndex.class);
+
+    final List<NestedPathPart> lElementPath = NestedPathFinder.parseJsonPath("$.l[1]");
+    ColumnValueSelector<?> lElementSelector = column.makeColumnValueSelector(lElementPath, offset);
+    VectorValueSelector lElementVectorSelector = column.makeVectorValueSelector(lElementPath, vectorOffset);
+    VectorObjectSelector lElementVectorObjectSelector = column.makeVectorObjectSelector(lElementPath, vectorOffset);
+    VectorValueSelector lElementFilteredVectorSelector = column.makeVectorValueSelector(
+        lElementPath,
+        bitmapVectorOffset
+    );
+    ColumnIndexSupplier lElementIndexSupplier = column.getColumnIndexSupplier(lElementPath);
+    Assert.assertNotNull(lElementIndexSupplier);
+    Assert.assertNull(lElementIndexSupplier.as(StringValueSetIndex.class));
+    Assert.assertNull(lElementIndexSupplier.as(DruidPredicateIndex.class));
+    Assert.assertNull(lElementIndexSupplier.as(NullValueIndex.class));
+
+    final List<NestedPathPart> dPath = NestedPathFinder.parseJsonPath("$.d");
+    Assert.assertEquals(ImmutableSet.of(ColumnType.DOUBLE_ARRAY), column.getColumnTypes(dPath));
+    Assert.assertEquals(ColumnType.DOUBLE_ARRAY, column.getColumnHolder(dPath).getCapabilities().toColumnType());
+    ColumnValueSelector<?> dSelector = column.makeColumnValueSelector(dPath, offset);
+    VectorObjectSelector dVectorSelector = column.makeVectorObjectSelector(dPath, vectorOffset);
+    VectorObjectSelector dVectorSelectorFiltered = column.makeVectorObjectSelector(dPath, bitmapVectorOffset);
+    ColumnIndexSupplier dIndexSupplier = column.getColumnIndexSupplier(dPath);
+    Assert.assertNotNull(dIndexSupplier);
+    Assert.assertNull(dIndexSupplier.as(StringValueSetIndex.class));
+    Assert.assertNull(dIndexSupplier.as(DruidPredicateIndex.class));
+    NullValueIndex dNulls = dIndexSupplier.as(NullValueIndex.class);
+
+    final List<NestedPathPart> dElementPath = NestedPathFinder.parseJsonPath("$.d[1]");
+    ColumnValueSelector<?> dElementSelector = column.makeColumnValueSelector(dElementPath, offset);
+    VectorValueSelector dElementVectorSelector = column.makeVectorValueSelector(dElementPath, vectorOffset);
+    VectorObjectSelector dElementVectorObjectSelector = column.makeVectorObjectSelector(dElementPath, vectorOffset);
+    VectorValueSelector dElementFilteredVectorSelector = column.makeVectorValueSelector(
+        dElementPath,
+        bitmapVectorOffset
+    );
+    ColumnIndexSupplier dElementIndexSupplier = column.getColumnIndexSupplier(dElementPath);
+    Assert.assertNotNull(dElementIndexSupplier);
+    Assert.assertNull(dElementIndexSupplier.as(StringValueSetIndex.class));
+    Assert.assertNull(dElementIndexSupplier.as(DruidPredicateIndex.class));
+    Assert.assertNull(dElementIndexSupplier.as(NullValueIndex.class));
+
+
+    ImmutableBitmap sNullIndex = sNulls.forNull().computeBitmapResult(resultFactory);
+    ImmutableBitmap lNullIndex = lNulls.forNull().computeBitmapResult(resultFactory);
+    ImmutableBitmap dNullIndex = dNulls.forNull().computeBitmapResult(resultFactory);
+
+    int rowCounter = 0;
+    while (offset.withinBounds()) {
+      Map row = arrayTestData.get(rowCounter);
+      Assert.assertEquals(
+          JSON_MAPPER.writeValueAsString(row),
+          JSON_MAPPER.writeValueAsString(StructuredData.unwrap(rawSelector.getObject()))
+      );
+
+      Object[] s = (Object[]) row.get("s");
+      Object[] l = (Object[]) row.get("l");
+      Object[] d = (Object[]) row.get("d");
+      Assert.assertArrayEquals(s, (Object[]) sSelector.getObject());
+      Assert.assertArrayEquals(l, (Object[]) lSelector.getObject());
+      Assert.assertArrayEquals(d, (Object[]) dSelector.getObject());
+      Assert.assertEquals(s == null, sNullIndex.get(rowCounter));
+      Assert.assertEquals(l == null, lNullIndex.get(rowCounter));
+      Assert.assertEquals(d == null, dNullIndex.get(rowCounter));
+
+      if (s == null || s.length < 1) {
+        Assert.assertNull(sElementSelector.getObject());
+      } else {
+        Assert.assertEquals(s[1], sElementSelector.getObject());
+      }
+      if (l == null || l.length < 1 || l[1] == null) {
+        Assert.assertTrue(lElementSelector.isNull());
+        Assert.assertNull(lElementSelector.getObject());
+      } else {
+        Assert.assertEquals(l[1], lElementSelector.getLong());
+        Assert.assertEquals(l[1], lElementSelector.getObject());
+      }
+      if (d == null || d.length < 1 || d[1] == null) {
+        Assert.assertTrue(dElementSelector.isNull());
+        Assert.assertNull(dElementSelector.getObject());
+      } else {
+        Assert.assertEquals((Double) d[1], dElementSelector.getDouble(), 0.0);
+        Assert.assertEquals(d[1], dElementSelector.getObject());
+      }
+
+      offset.increment();
+      rowCounter++;
+    }
+
+    rowCounter = 0;
+    while (!vectorOffset.isDone()) {
+      final Object[] rawVector = rawVectorSelector.getObjectVector();
+      final Object[] sVector = sVectorSelector.getObjectVector();
+      final Object[] lVector = lVectorSelector.getObjectVector();
+      final Object[] dVector = dVectorSelector.getObjectVector();
+      final Object[] sElementVector = sElementVectorSelector.getObjectVector();
+      final long[] lElementVector = lElementVectorSelector.getLongVector();
+      final boolean[] lElementNulls = lElementVectorSelector.getNullVector();
+      final Object[] lElementObjectVector = lElementVectorObjectSelector.getObjectVector();
+      final double[] dElementVector = dElementVectorSelector.getDoubleVector();
+      final boolean[] dElementNulls = dElementVectorSelector.getNullVector();
+      final Object[] dElementObjectVector = dElementVectorObjectSelector.getObjectVector();
+
+      for (int i = 0; i < vectorOffset.getCurrentVectorSize(); i++, rowCounter++) {
+
+        Map row = arrayTestData.get(rowCounter);
+        Assert.assertEquals(
+            JSON_MAPPER.writeValueAsString(row),
+            JSON_MAPPER.writeValueAsString(StructuredData.unwrap(rawVector[i]))
+        );
+        Object[] s = (Object[]) row.get("s");
+        Object[] l = (Object[]) row.get("l");
+        Object[] d = (Object[]) row.get("d");
+
+        Assert.assertArrayEquals(s, (Object[]) sVector[i]);
+        Assert.assertArrayEquals(l, (Object[]) lVector[i]);
+        Assert.assertArrayEquals(d, (Object[]) dVector[i]);
+
+        if (s == null || s.length < 1) {
+          Assert.assertNull(sElementVector[i]);
+        } else {
+          Assert.assertEquals(s[1], sElementVector[i]);
+        }
+        if (l == null || l.length < 1 || l[1] == null) {
+          Assert.assertTrue(lElementNulls[i]);
+          Assert.assertNull(lElementObjectVector[i]);
+        } else {
+          Assert.assertEquals(l[1], lElementVector[i]);
+          Assert.assertEquals(l[1], lElementObjectVector[i]);
+        }
+        if (d == null || d.length < 1 || d[1] == null) {
+          Assert.assertTrue(dElementNulls[i]);
+          Assert.assertNull(dElementObjectVector[i]);
+        } else {
+          Assert.assertEquals((Double) d[1], dElementVector[i], 0.0);
+          Assert.assertEquals(d[1], dElementObjectVector[i]);
+        }
+      }
+      vectorOffset.advance();
+    }
+
+    rowCounter = 0;
+    while (!bitmapVectorOffset.isDone()) {
+      final Object[] rawVector = rawVectorSelectorFiltered.getObjectVector();
+      final Object[] sVector = sVectorSelectorFiltered.getObjectVector();
+      final Object[] lVector = lVectorSelectorFiltered.getObjectVector();
+      final Object[] dVector = dVectorSelectorFiltered.getObjectVector();
+      final Object[] sElementVector = sElementFilteredVectorSelector.getObjectVector();
+      final long[] lElementVector = lElementFilteredVectorSelector.getLongVector();
+      final boolean[] lElementNulls = lElementFilteredVectorSelector.getNullVector();
+      final double[] dElementVector = dElementFilteredVectorSelector.getDoubleVector();
+      final boolean[] dElementNulls = dElementFilteredVectorSelector.getNullVector();
+
+      for (int i = 0; i < bitmapVectorOffset.getCurrentVectorSize(); i++, rowCounter += 2) {
+        Map row = arrayTestData.get(rowCounter);
+        Assert.assertEquals(
+            JSON_MAPPER.writeValueAsString(row),
+            JSON_MAPPER.writeValueAsString(StructuredData.unwrap(rawVector[i]))
+        );
+        Object[] s = (Object[]) row.get("s");
+        Object[] l = (Object[]) row.get("l");
+        Object[] d = (Object[]) row.get("d");
+
+        Assert.assertArrayEquals(s, (Object[]) sVector[i]);
+        Assert.assertArrayEquals(l, (Object[]) lVector[i]);
+        Assert.assertArrayEquals(d, (Object[]) dVector[i]);
+
+        if (s == null || s.length < 1) {
+          Assert.assertNull(sElementVector[i]);
+        } else {
+          Assert.assertEquals(s[1], sElementVector[i]);
+        }
+        if (l == null || l.length < 1 || l[1] == null) {
+          Assert.assertTrue(lElementNulls[i]);
+        } else {
+          Assert.assertEquals(l[1], lElementVector[i]);
+        }
+        if (d == null || d.length < 1 || d[1] == null) {
+          Assert.assertTrue(dElementNulls[i]);
+        } else {
+          Assert.assertEquals((Double) d[1], dElementVector[i], 0.0);
+        }
+      }
+      bitmapVectorOffset.advance();
+    }
+  }
+
   private void testPath(
       Map row,
       int rowNumber,
@@ -372,10 +741,11 @@ public class NestedDataColumnSupplierTest extends InitializedNullHandlingTest
       Assert.assertEquals(inputValue, valueSelector.getObject());
       if (ColumnType.LONG.equals(singleType)) {
         Assert.assertEquals(inputValue, valueSelector.getLong());
+        Assert.assertFalse(path + " is not null", valueSelector.isNull());
       } else if (ColumnType.DOUBLE.equals(singleType)) {
         Assert.assertEquals((double) inputValue, valueSelector.getDouble(), 0.0);
+        Assert.assertFalse(path + " is not null", valueSelector.isNull());
       }
-      Assert.assertFalse(valueSelector.isNull());
 
       final String theString = String.valueOf(inputValue);
       Assert.assertEquals(theString, dimSelector.getObject());
@@ -384,11 +754,19 @@ public class NestedDataColumnSupplierTest extends InitializedNullHandlingTest
       Assert.assertEquals(dimSelector.idLookup().lookupId(dimSelectorLookupVal), dimSelector.getRow().get(0));
 
       Assert.assertTrue(valueSetIndex.forValue(theString).computeBitmapResult(resultFactory).get(rowNumber));
-      Assert.assertTrue(valueSetIndex.forSortedValues(new TreeSet<>(ImmutableSet.of(theString))).computeBitmapResult(resultFactory).get(rowNumber));
-      Assert.assertTrue(predicateIndex.forPredicate(new SelectorPredicateFactory(theString)).computeBitmapResult(resultFactory).get(rowNumber));
+      Assert.assertTrue(valueSetIndex.forSortedValues(new TreeSet<>(ImmutableSet.of(theString)))
+                                     .computeBitmapResult(resultFactory)
+                                     .get(rowNumber));
+      Assert.assertTrue(predicateIndex.forPredicate(new SelectorPredicateFactory(theString))
+                                      .computeBitmapResult(resultFactory)
+                                      .get(rowNumber));
       Assert.assertFalse(valueSetIndex.forValue(NO_MATCH).computeBitmapResult(resultFactory).get(rowNumber));
-      Assert.assertFalse(valueSetIndex.forSortedValues(new TreeSet<>(ImmutableSet.of(NO_MATCH))).computeBitmapResult(resultFactory).get(rowNumber));
-      Assert.assertFalse(predicateIndex.forPredicate(new SelectorPredicateFactory(NO_MATCH)).computeBitmapResult(resultFactory).get(rowNumber));
+      Assert.assertFalse(valueSetIndex.forSortedValues(new TreeSet<>(ImmutableSet.of(NO_MATCH)))
+                                      .computeBitmapResult(resultFactory)
+                                      .get(rowNumber));
+      Assert.assertFalse(predicateIndex.forPredicate(new SelectorPredicateFactory(NO_MATCH))
+                                       .computeBitmapResult(resultFactory)
+                                       .get(rowNumber));
       Assert.assertFalse(nullValueIndex.forNull().computeBitmapResult(resultFactory).get(rowNumber));
 
       Assert.assertTrue(dimSelector.makeValueMatcher(theString).matches());
@@ -397,7 +775,7 @@ public class NestedDataColumnSupplierTest extends InitializedNullHandlingTest
       Assert.assertFalse(dimSelector.makeValueMatcher(x -> Objects.equals(x, NO_MATCH)).matches());
     } else {
       Assert.assertNull(valueSelector.getObject());
-      Assert.assertTrue(valueSelector.isNull());
+      Assert.assertTrue(path, valueSelector.isNull());
 
       Assert.assertEquals(0, dimSelector.getRow().get(0));
       Assert.assertNull(dimSelector.getObject());
@@ -406,9 +784,13 @@ public class NestedDataColumnSupplierTest extends InitializedNullHandlingTest
       Assert.assertTrue(valueSetIndex.forValue(null).computeBitmapResult(resultFactory).get(rowNumber));
       Assert.assertFalse(valueSetIndex.forValue(NO_MATCH).computeBitmapResult(resultFactory).get(rowNumber));
       Assert.assertTrue(nullValueIndex.forNull().computeBitmapResult(resultFactory).get(rowNumber));
-      Assert.assertTrue(predicateIndex.forPredicate(new SelectorPredicateFactory(null)).computeBitmapResult(resultFactory).get(rowNumber));
+      Assert.assertTrue(predicateIndex.forPredicate(new SelectorPredicateFactory(null))
+                                      .computeBitmapResult(resultFactory)
+                                      .get(rowNumber));
       Assert.assertFalse(valueSetIndex.forValue(NO_MATCH).computeBitmapResult(resultFactory).get(rowNumber));
-      Assert.assertFalse(predicateIndex.forPredicate(new SelectorPredicateFactory(NO_MATCH)).computeBitmapResult(resultFactory).get(rowNumber));
+      Assert.assertFalse(predicateIndex.forPredicate(new SelectorPredicateFactory(NO_MATCH))
+                                       .computeBitmapResult(resultFactory)
+                                       .get(rowNumber));
 
       Assert.assertTrue(dimSelector.makeValueMatcher((String) null).matches());
       Assert.assertFalse(dimSelector.makeValueMatcher(NO_MATCH).matches());
diff --git a/processing/src/test/java/org/apache/druid/segment/nested/NestedFieldLiteralColumnIndexSupplierTest.java b/processing/src/test/java/org/apache/druid/segment/nested/NestedFieldColumnIndexSupplierTest.java
similarity index 92%
rename from processing/src/test/java/org/apache/druid/segment/nested/NestedFieldLiteralColumnIndexSupplierTest.java
rename to processing/src/test/java/org/apache/druid/segment/nested/NestedFieldColumnIndexSupplierTest.java
index 46e881a9cb..13e5e2b535 100644
--- a/processing/src/test/java/org/apache/druid/segment/nested/NestedFieldLiteralColumnIndexSupplierTest.java
+++ b/processing/src/test/java/org/apache/druid/segment/nested/NestedFieldColumnIndexSupplierTest.java
@@ -60,7 +60,7 @@ import java.nio.ByteOrder;
 import java.nio.channels.WritableByteChannel;
 import java.util.TreeSet;
 
-public class NestedFieldLiteralColumnIndexSupplierTest extends InitializedNullHandlingTest
+public class NestedFieldColumnIndexSupplierTest extends InitializedNullHandlingTest
 {
   BitmapSerdeFactory roaringFactory = RoaringBitmapSerdeFactory.getInstance();
   BitmapResultFactory<ImmutableBitmap> bitmapResultFactory = new DefaultBitmapResultFactory(
@@ -139,7 +139,7 @@ public class NestedFieldLiteralColumnIndexSupplierTest extends InitializedNullHa
   @Test
   public void testSingleTypeStringColumnValueIndex() throws IOException
   {
-    NestedFieldLiteralColumnIndexSupplier<?> indexSupplier = makeSingleTypeStringSupplier();
+    NestedFieldColumnIndexSupplier<?> indexSupplier = makeSingleTypeStringSupplier();
 
     NullValueIndex nullIndex = indexSupplier.as(NullValueIndex.class);
     Assert.assertNotNull(nullIndex);
@@ -161,7 +161,7 @@ public class NestedFieldLiteralColumnIndexSupplierTest extends InitializedNullHa
   @Test
   public void testSingleTypeStringColumnValueSetIndex() throws IOException
   {
-    NestedFieldLiteralColumnIndexSupplier<?> indexSupplier = makeSingleTypeStringSupplier();
+    NestedFieldColumnIndexSupplier<?> indexSupplier = makeSingleTypeStringSupplier();
 
     StringValueSetIndex valueSetIndex = indexSupplier.as(StringValueSetIndex.class);
     Assert.assertNotNull(valueSetIndex);
@@ -194,7 +194,7 @@ public class NestedFieldLiteralColumnIndexSupplierTest extends InitializedNullHa
   @Test
   public void testSingleTypeStringColumnRangeIndex() throws IOException
   {
-    NestedFieldLiteralColumnIndexSupplier<?> indexSupplier = makeSingleTypeStringSupplier();
+    NestedFieldColumnIndexSupplier<?> indexSupplier = makeSingleTypeStringSupplier();
 
     LexicographicalRangeIndex rangeIndex = indexSupplier.as(LexicographicalRangeIndex.class);
     Assert.assertNotNull(rangeIndex);
@@ -351,7 +351,7 @@ public class NestedFieldLiteralColumnIndexSupplierTest extends InitializedNullHa
   @Test
   public void testSingleTypeStringColumnRangeIndexWithPredicate() throws IOException
   {
-    NestedFieldLiteralColumnIndexSupplier<?> indexSupplier = makeSingleTypeStringSupplier();
+    NestedFieldColumnIndexSupplier<?> indexSupplier = makeSingleTypeStringSupplier();
 
     LexicographicalRangeIndex rangeIndex = indexSupplier.as(LexicographicalRangeIndex.class);
     Assert.assertNotNull(rangeIndex);
@@ -419,7 +419,7 @@ public class NestedFieldLiteralColumnIndexSupplierTest extends InitializedNullHa
   @Test
   public void testSingleTypeStringColumnPredicateIndex() throws IOException
   {
-    NestedFieldLiteralColumnIndexSupplier<?> indexSupplier = makeSingleTypeStringSupplier();
+    NestedFieldColumnIndexSupplier<?> indexSupplier = makeSingleTypeStringSupplier();
 
     DruidPredicateIndex predicateIndex = indexSupplier.as(DruidPredicateIndex.class);
     Assert.assertNotNull(predicateIndex);
@@ -442,7 +442,7 @@ public class NestedFieldLiteralColumnIndexSupplierTest extends InitializedNullHa
   @Test
   public void testSingleTypeStringColumnWithNullValueIndex() throws IOException
   {
-    NestedFieldLiteralColumnIndexSupplier<?> indexSupplier = makeSingleTypeStringWithNullsSupplier();
+    NestedFieldColumnIndexSupplier<?> indexSupplier = makeSingleTypeStringWithNullsSupplier();
 
     NullValueIndex nullIndex = indexSupplier.as(NullValueIndex.class);
     Assert.assertNotNull(nullIndex);
@@ -461,7 +461,7 @@ public class NestedFieldLiteralColumnIndexSupplierTest extends InitializedNullHa
   @Test
   public void testSingleTypeStringColumnWithNullValueSetIndex() throws IOException
   {
-    NestedFieldLiteralColumnIndexSupplier<?> indexSupplier = makeSingleTypeStringWithNullsSupplier();
+    NestedFieldColumnIndexSupplier<?> indexSupplier = makeSingleTypeStringWithNullsSupplier();
 
     StringValueSetIndex valueSetIndex = indexSupplier.as(StringValueSetIndex.class);
     Assert.assertNotNull(valueSetIndex);
@@ -494,7 +494,7 @@ public class NestedFieldLiteralColumnIndexSupplierTest extends InitializedNullHa
   @Test
   public void testSingleValueStringWithNullRangeIndex() throws IOException
   {
-    NestedFieldLiteralColumnIndexSupplier<?> indexSupplier = makeSingleTypeStringWithNullsSupplier();
+    NestedFieldColumnIndexSupplier<?> indexSupplier = makeSingleTypeStringWithNullsSupplier();
 
     LexicographicalRangeIndex rangeIndex = indexSupplier.as(LexicographicalRangeIndex.class);
     Assert.assertNotNull(rangeIndex);
@@ -584,7 +584,7 @@ public class NestedFieldLiteralColumnIndexSupplierTest extends InitializedNullHa
   @Test
   public void testSingleValueStringWithNullPredicateIndex() throws IOException
   {
-    NestedFieldLiteralColumnIndexSupplier<?> indexSupplier = makeSingleTypeStringWithNullsSupplier();
+    NestedFieldColumnIndexSupplier<?> indexSupplier = makeSingleTypeStringWithNullsSupplier();
 
     DruidPredicateIndex predicateIndex = indexSupplier.as(DruidPredicateIndex.class);
     Assert.assertNotNull(predicateIndex);
@@ -607,7 +607,7 @@ public class NestedFieldLiteralColumnIndexSupplierTest extends InitializedNullHa
   @Test
   public void testSingleTypeLongColumnValueSetIndex() throws IOException
   {
-    NestedFieldLiteralColumnIndexSupplier<?> indexSupplier = makeSingleTypeLongSupplier();
+    NestedFieldColumnIndexSupplier<?> indexSupplier = makeSingleTypeLongSupplier();
 
     StringValueSetIndex valueSetIndex = indexSupplier.as(StringValueSetIndex.class);
     Assert.assertNotNull(valueSetIndex);
@@ -636,7 +636,7 @@ public class NestedFieldLiteralColumnIndexSupplierTest extends InitializedNullHa
   @Test
   public void testSingleTypeLongColumnRangeIndex() throws IOException
   {
-    NestedFieldLiteralColumnIndexSupplier<?> indexSupplier = makeSingleTypeLongSupplier();
+    NestedFieldColumnIndexSupplier<?> indexSupplier = makeSingleTypeLongSupplier();
 
     NumericRangeIndex rangeIndex = indexSupplier.as(NumericRangeIndex.class);
     Assert.assertNotNull(rangeIndex);
@@ -705,7 +705,7 @@ public class NestedFieldLiteralColumnIndexSupplierTest extends InitializedNullHa
   @Test
   public void testSingleTypeLongColumnPredicateIndex() throws IOException
   {
-    NestedFieldLiteralColumnIndexSupplier<?> indexSupplier = makeSingleTypeLongSupplier();
+    NestedFieldColumnIndexSupplier<?> indexSupplier = makeSingleTypeLongSupplier();
 
     DruidPredicateIndex predicateIndex = indexSupplier.as(DruidPredicateIndex.class);
     Assert.assertNotNull(predicateIndex);
@@ -728,7 +728,7 @@ public class NestedFieldLiteralColumnIndexSupplierTest extends InitializedNullHa
   @Test
   public void testSingleTypeLongColumnWithNullValueIndex() throws IOException
   {
-    NestedFieldLiteralColumnIndexSupplier<?> indexSupplier = makeSingleTypeLongSupplierWithNull();
+    NestedFieldColumnIndexSupplier<?> indexSupplier = makeSingleTypeLongSupplierWithNull();
 
     NullValueIndex nullIndex = indexSupplier.as(NullValueIndex.class);
     Assert.assertNotNull(nullIndex);
@@ -747,7 +747,7 @@ public class NestedFieldLiteralColumnIndexSupplierTest extends InitializedNullHa
   @Test
   public void testSingleTypeLongColumnWithNullValueSetIndex() throws IOException
   {
-    NestedFieldLiteralColumnIndexSupplier<?> indexSupplier = makeSingleTypeLongSupplierWithNull();
+    NestedFieldColumnIndexSupplier<?> indexSupplier = makeSingleTypeLongSupplierWithNull();
 
     StringValueSetIndex valueSetIndex = indexSupplier.as(StringValueSetIndex.class);
     Assert.assertNotNull(valueSetIndex);
@@ -792,7 +792,7 @@ public class NestedFieldLiteralColumnIndexSupplierTest extends InitializedNullHa
   @Test
   public void testSingleValueLongWithNullRangeIndex() throws IOException
   {
-    NestedFieldLiteralColumnIndexSupplier<?> indexSupplier = makeSingleTypeLongSupplierWithNull();
+    NestedFieldColumnIndexSupplier<?> indexSupplier = makeSingleTypeLongSupplierWithNull();
 
     NumericRangeIndex rangeIndex = indexSupplier.as(NumericRangeIndex.class);
     Assert.assertNotNull(rangeIndex);
@@ -857,7 +857,7 @@ public class NestedFieldLiteralColumnIndexSupplierTest extends InitializedNullHa
   @Test
   public void testSingleValueLongWithNullPredicateIndex() throws IOException
   {
-    NestedFieldLiteralColumnIndexSupplier<?> indexSupplier = makeSingleTypeLongSupplierWithNull();
+    NestedFieldColumnIndexSupplier<?> indexSupplier = makeSingleTypeLongSupplierWithNull();
 
     DruidPredicateIndex predicateIndex = indexSupplier.as(DruidPredicateIndex.class);
     Assert.assertNotNull(predicateIndex);
@@ -880,7 +880,7 @@ public class NestedFieldLiteralColumnIndexSupplierTest extends InitializedNullHa
   @Test
   public void testSingleTypeDoubleColumnValueSetIndex() throws IOException
   {
-    NestedFieldLiteralColumnIndexSupplier<?> indexSupplier = makeSingleTypeDoubleSupplier();
+    NestedFieldColumnIndexSupplier<?> indexSupplier = makeSingleTypeDoubleSupplier();
 
     StringValueSetIndex valueSetIndex = indexSupplier.as(StringValueSetIndex.class);
     Assert.assertNotNull(valueSetIndex);
@@ -909,7 +909,7 @@ public class NestedFieldLiteralColumnIndexSupplierTest extends InitializedNullHa
   @Test
   public void testSingleTypeDoubleColumnRangeIndex() throws IOException
   {
-    NestedFieldLiteralColumnIndexSupplier<?> indexSupplier = makeSingleTypeDoubleSupplier();
+    NestedFieldColumnIndexSupplier<?> indexSupplier = makeSingleTypeDoubleSupplier();
 
     NumericRangeIndex rangeIndex = indexSupplier.as(NumericRangeIndex.class);
     Assert.assertNotNull(rangeIndex);
@@ -1002,7 +1002,7 @@ public class NestedFieldLiteralColumnIndexSupplierTest extends InitializedNullHa
   @Test
   public void testSingleTypeDoubleColumnPredicateIndex() throws IOException
   {
-    NestedFieldLiteralColumnIndexSupplier<?> indexSupplier = makeSingleTypeDoubleSupplier();
+    NestedFieldColumnIndexSupplier<?> indexSupplier = makeSingleTypeDoubleSupplier();
 
     DruidPredicateIndex predicateIndex = indexSupplier.as(DruidPredicateIndex.class);
     Assert.assertNotNull(predicateIndex);
@@ -1025,7 +1025,7 @@ public class NestedFieldLiteralColumnIndexSupplierTest extends InitializedNullHa
   @Test
   public void testSingleTypeDoubleColumnWithNullValueIndex() throws IOException
   {
-    NestedFieldLiteralColumnIndexSupplier<?> indexSupplier = makeSingleTypeDoubleSupplierWithNull();
+    NestedFieldColumnIndexSupplier<?> indexSupplier = makeSingleTypeDoubleSupplierWithNull();
 
     NullValueIndex nullIndex = indexSupplier.as(NullValueIndex.class);
     Assert.assertNotNull(nullIndex);
@@ -1044,7 +1044,7 @@ public class NestedFieldLiteralColumnIndexSupplierTest extends InitializedNullHa
   @Test
   public void testSingleTypeDoubleColumnWithNullValueSetIndex() throws IOException
   {
-    NestedFieldLiteralColumnIndexSupplier<?> indexSupplier = makeSingleTypeDoubleSupplierWithNull();
+    NestedFieldColumnIndexSupplier<?> indexSupplier = makeSingleTypeDoubleSupplierWithNull();
 
     StringValueSetIndex valueSetIndex = indexSupplier.as(StringValueSetIndex.class);
     Assert.assertNotNull(valueSetIndex);
@@ -1089,7 +1089,7 @@ public class NestedFieldLiteralColumnIndexSupplierTest extends InitializedNullHa
   @Test
   public void testSingleValueDoubleWithNullRangeIndex() throws IOException
   {
-    NestedFieldLiteralColumnIndexSupplier<?> indexSupplier = makeSingleTypeDoubleSupplierWithNull();
+    NestedFieldColumnIndexSupplier<?> indexSupplier = makeSingleTypeDoubleSupplierWithNull();
 
     NumericRangeIndex rangeIndex = indexSupplier.as(NumericRangeIndex.class);
     Assert.assertNotNull(rangeIndex);
@@ -1139,7 +1139,7 @@ public class NestedFieldLiteralColumnIndexSupplierTest extends InitializedNullHa
   @Test
   public void testSingleValueDoubleWithNullPredicateIndex() throws IOException
   {
-    NestedFieldLiteralColumnIndexSupplier<?> indexSupplier = makeSingleTypeDoubleSupplierWithNull();
+    NestedFieldColumnIndexSupplier<?> indexSupplier = makeSingleTypeDoubleSupplierWithNull();
 
     DruidPredicateIndex predicateIndex = indexSupplier.as(DruidPredicateIndex.class);
     Assert.assertNotNull(predicateIndex);
@@ -1162,7 +1162,7 @@ public class NestedFieldLiteralColumnIndexSupplierTest extends InitializedNullHa
   @Test
   public void testVariantNullValueIndex() throws IOException
   {
-    NestedFieldLiteralColumnIndexSupplier<?> indexSupplier = makeVariantSupplierWithNull();
+    NestedFieldColumnIndexSupplier<?> indexSupplier = makeVariantSupplierWithNull();
 
     NullValueIndex nullIndex = indexSupplier.as(NullValueIndex.class);
     Assert.assertNotNull(nullIndex);
@@ -1184,7 +1184,7 @@ public class NestedFieldLiteralColumnIndexSupplierTest extends InitializedNullHa
   @Test
   public void testVariantValueSetIndex() throws IOException
   {
-    NestedFieldLiteralColumnIndexSupplier<?> indexSupplier = makeVariantSupplierWithNull();
+    NestedFieldColumnIndexSupplier<?> indexSupplier = makeVariantSupplierWithNull();
 
     StringValueSetIndex valueSetIndex = indexSupplier.as(StringValueSetIndex.class);
     Assert.assertNotNull(valueSetIndex);
@@ -1242,7 +1242,7 @@ public class NestedFieldLiteralColumnIndexSupplierTest extends InitializedNullHa
   @Test
   public void testVariantRangeIndex() throws IOException
   {
-    NestedFieldLiteralColumnIndexSupplier<?> indexSupplier = makeVariantSupplierWithNull();
+    NestedFieldColumnIndexSupplier<?> indexSupplier = makeVariantSupplierWithNull();
 
     LexicographicalRangeIndex rangeIndex = indexSupplier.as(LexicographicalRangeIndex.class);
     Assert.assertNull(rangeIndex);
@@ -1254,7 +1254,7 @@ public class NestedFieldLiteralColumnIndexSupplierTest extends InitializedNullHa
   @Test
   public void testVariantPredicateIndex() throws IOException
   {
-    NestedFieldLiteralColumnIndexSupplier<?> indexSupplier = makeVariantSupplierWithNull();
+    NestedFieldColumnIndexSupplier<?> indexSupplier = makeVariantSupplierWithNull();
 
     DruidPredicateIndex predicateIndex = indexSupplier.as(DruidPredicateIndex.class);
     Assert.assertNotNull(predicateIndex);
@@ -1277,7 +1277,7 @@ public class NestedFieldLiteralColumnIndexSupplierTest extends InitializedNullHa
   @Test
   public void testDictionaryEncodedStringValueIndex() throws IOException
   {
-    NestedFieldLiteralColumnIndexSupplier<?> indexSupplier = makeVariantSupplierWithNull();
+    NestedFieldColumnIndexSupplier<?> indexSupplier = makeVariantSupplierWithNull();
 
     DictionaryEncodedStringValueIndex lowLevelIndex = indexSupplier.as(DictionaryEncodedStringValueIndex.class);
     Assert.assertNotNull(lowLevelIndex);
@@ -1398,18 +1398,20 @@ public class NestedFieldLiteralColumnIndexSupplierTest extends InitializedNullHa
 
     GenericIndexed<ImmutableBitmap> bitmaps = GenericIndexed.read(bitmapsBuffer, roaringFactory.getObjectStrategy());
 
-    NestedFieldLiteralColumnIndexSupplier<?> indexSupplier = new NestedFieldLiteralColumnIndexSupplier<>(
-        new NestedLiteralTypeInfo.TypeSet(
-            new NestedLiteralTypeInfo.MutableTypeSet().add(ColumnType.STRING)
-                                                      .add(ColumnType.LONG)
-                                                      .getByteValue()
+    NestedFieldColumnIndexSupplier<?> indexSupplier = new NestedFieldColumnIndexSupplier<>(
+        new NestedFieldTypeInfo.TypeSet(
+            new NestedFieldTypeInfo.MutableTypeSet().add(ColumnType.STRING)
+                                                    .add(ColumnType.LONG)
+                                                    .getByteValue()
         ),
         roaringFactory.getBitmapFactory(),
         bitmaps,
         dictionarySupplier,
         stringIndexed,
         longIndexed,
-        doubleIndexed
+        doubleIndexed,
+        null,
+        null
     );
 
     StringValueSetIndex valueSetIndex = indexSupplier.as(StringValueSetIndex.class);
@@ -1438,7 +1440,7 @@ public class NestedFieldLiteralColumnIndexSupplierTest extends InitializedNullHa
     checkBitmap(bitmap);
   }
 
-  private NestedFieldLiteralColumnIndexSupplier<?> makeSingleTypeStringSupplier() throws IOException
+  private NestedFieldColumnIndexSupplier<?> makeSingleTypeStringSupplier() throws IOException
   {
     ByteBuffer localDictionaryBuffer = ByteBuffer.allocate(1 << 12).order(ByteOrder.nativeOrder());
     ByteBuffer bitmapsBuffer = ByteBuffer.allocate(1 << 12);
@@ -1496,20 +1498,22 @@ public class NestedFieldLiteralColumnIndexSupplierTest extends InitializedNullHa
 
     GenericIndexed<ImmutableBitmap> bitmaps = GenericIndexed.read(bitmapsBuffer, roaringFactory.getObjectStrategy());
 
-    return new NestedFieldLiteralColumnIndexSupplier<>(
-        new NestedLiteralTypeInfo.TypeSet(
-            new NestedLiteralTypeInfo.MutableTypeSet().add(ColumnType.STRING).getByteValue()
+    return new NestedFieldColumnIndexSupplier<>(
+        new NestedFieldTypeInfo.TypeSet(
+            new NestedFieldTypeInfo.MutableTypeSet().add(ColumnType.STRING).getByteValue()
         ),
         roaringFactory.getBitmapFactory(),
         bitmaps,
         dictionarySupplier,
         globalStrings,
         globalLongs,
-        globalDoubles
+        globalDoubles,
+        null,
+        null
     );
   }
 
-  private NestedFieldLiteralColumnIndexSupplier<?> makeSingleTypeStringWithNullsSupplier() throws IOException
+  private NestedFieldColumnIndexSupplier<?> makeSingleTypeStringWithNullsSupplier() throws IOException
   {
     ByteBuffer localDictionaryBuffer = ByteBuffer.allocate(1 << 12).order(ByteOrder.nativeOrder());
     ByteBuffer bitmapsBuffer = ByteBuffer.allocate(1 << 12);
@@ -1570,20 +1574,22 @@ public class NestedFieldLiteralColumnIndexSupplierTest extends InitializedNullHa
 
     GenericIndexed<ImmutableBitmap> bitmaps = GenericIndexed.read(bitmapsBuffer, roaringFactory.getObjectStrategy());
 
-    return new NestedFieldLiteralColumnIndexSupplier<>(
-        new NestedLiteralTypeInfo.TypeSet(
-            new NestedLiteralTypeInfo.MutableTypeSet().add(ColumnType.STRING).getByteValue()
+    return new NestedFieldColumnIndexSupplier<>(
+        new NestedFieldTypeInfo.TypeSet(
+            new NestedFieldTypeInfo.MutableTypeSet().add(ColumnType.STRING).getByteValue()
         ),
         roaringFactory.getBitmapFactory(),
         bitmaps,
         dictionarySupplier,
         globalStrings,
         globalLongs,
-        globalDoubles
+        globalDoubles,
+        null,
+        null
     );
   }
 
-  private NestedFieldLiteralColumnIndexSupplier<?> makeSingleTypeLongSupplier() throws IOException
+  private NestedFieldColumnIndexSupplier<?> makeSingleTypeLongSupplier() throws IOException
   {
     ByteBuffer localDictionaryBuffer = ByteBuffer.allocate(1 << 12).order(ByteOrder.nativeOrder());
     ByteBuffer bitmapsBuffer = ByteBuffer.allocate(1 << 12);
@@ -1641,20 +1647,22 @@ public class NestedFieldLiteralColumnIndexSupplierTest extends InitializedNullHa
 
     GenericIndexed<ImmutableBitmap> bitmaps = GenericIndexed.read(bitmapsBuffer, roaringFactory.getObjectStrategy());
 
-    return new NestedFieldLiteralColumnIndexSupplier<>(
-        new NestedLiteralTypeInfo.TypeSet(
-            new NestedLiteralTypeInfo.MutableTypeSet().add(ColumnType.LONG).getByteValue()
+    return new NestedFieldColumnIndexSupplier<>(
+        new NestedFieldTypeInfo.TypeSet(
+            new NestedFieldTypeInfo.MutableTypeSet().add(ColumnType.LONG).getByteValue()
         ),
         roaringFactory.getBitmapFactory(),
         bitmaps,
         dictionarySupplier,
         globalStrings,
         globalLongs,
-        globalDoubles
+        globalDoubles,
+        null,
+        null
     );
   }
 
-  private NestedFieldLiteralColumnIndexSupplier<?> makeSingleTypeLongSupplierWithNull() throws IOException
+  private NestedFieldColumnIndexSupplier<?> makeSingleTypeLongSupplierWithNull() throws IOException
   {
     ByteBuffer localDictionaryBuffer = ByteBuffer.allocate(1 << 12).order(ByteOrder.nativeOrder());
     ByteBuffer bitmapsBuffer = ByteBuffer.allocate(1 << 12);
@@ -1716,20 +1724,22 @@ public class NestedFieldLiteralColumnIndexSupplierTest extends InitializedNullHa
 
     GenericIndexed<ImmutableBitmap> bitmaps = GenericIndexed.read(bitmapsBuffer, roaringFactory.getObjectStrategy());
 
-    return new NestedFieldLiteralColumnIndexSupplier<>(
-        new NestedLiteralTypeInfo.TypeSet(
-            new NestedLiteralTypeInfo.MutableTypeSet().add(ColumnType.LONG).getByteValue()
+    return new NestedFieldColumnIndexSupplier<>(
+        new NestedFieldTypeInfo.TypeSet(
+            new NestedFieldTypeInfo.MutableTypeSet().add(ColumnType.LONG).getByteValue()
         ),
         roaringFactory.getBitmapFactory(),
         bitmaps,
         dictionarySupplier,
         globalStrings,
         globalLongs,
-        globalDoubles
+        globalDoubles,
+        null,
+        null
     );
   }
 
-  private NestedFieldLiteralColumnIndexSupplier<?> makeSingleTypeDoubleSupplier() throws IOException
+  private NestedFieldColumnIndexSupplier<?> makeSingleTypeDoubleSupplier() throws IOException
   {
     ByteBuffer localDictionaryBuffer = ByteBuffer.allocate(1 << 12).order(ByteOrder.nativeOrder());
     ByteBuffer bitmapsBuffer = ByteBuffer.allocate(1 << 12);
@@ -1787,20 +1797,22 @@ public class NestedFieldLiteralColumnIndexSupplierTest extends InitializedNullHa
 
     GenericIndexed<ImmutableBitmap> bitmaps = GenericIndexed.read(bitmapsBuffer, roaringFactory.getObjectStrategy());
 
-    return new NestedFieldLiteralColumnIndexSupplier<>(
-        new NestedLiteralTypeInfo.TypeSet(
-            new NestedLiteralTypeInfo.MutableTypeSet().add(ColumnType.DOUBLE).getByteValue()
+    return new NestedFieldColumnIndexSupplier<>(
+        new NestedFieldTypeInfo.TypeSet(
+            new NestedFieldTypeInfo.MutableTypeSet().add(ColumnType.DOUBLE).getByteValue()
         ),
         roaringFactory.getBitmapFactory(),
         bitmaps,
         dictionarySupplier,
         globalStrings,
         globalLongs,
-        globalDoubles
+        globalDoubles,
+        null,
+        null
     );
   }
 
-  private NestedFieldLiteralColumnIndexSupplier<?> makeSingleTypeDoubleSupplierWithNull() throws IOException
+  private NestedFieldColumnIndexSupplier<?> makeSingleTypeDoubleSupplierWithNull() throws IOException
   {
     ByteBuffer localDictionaryBuffer = ByteBuffer.allocate(1 << 12).order(ByteOrder.nativeOrder());
     ByteBuffer bitmapsBuffer = ByteBuffer.allocate(1 << 12);
@@ -1862,20 +1874,22 @@ public class NestedFieldLiteralColumnIndexSupplierTest extends InitializedNullHa
 
     GenericIndexed<ImmutableBitmap> bitmaps = GenericIndexed.read(bitmapsBuffer, roaringFactory.getObjectStrategy());
 
-    return new NestedFieldLiteralColumnIndexSupplier<>(
-        new NestedLiteralTypeInfo.TypeSet(
-            new NestedLiteralTypeInfo.MutableTypeSet().add(ColumnType.DOUBLE).getByteValue()
+    return new NestedFieldColumnIndexSupplier<>(
+        new NestedFieldTypeInfo.TypeSet(
+            new NestedFieldTypeInfo.MutableTypeSet().add(ColumnType.DOUBLE).getByteValue()
         ),
         roaringFactory.getBitmapFactory(),
         bitmaps,
         dictionarySupplier,
         globalStrings,
         globalLongs,
-        globalDoubles
+        globalDoubles,
+        null,
+        null
     );
   }
 
-  private NestedFieldLiteralColumnIndexSupplier<?> makeVariantSupplierWithNull() throws IOException
+  private NestedFieldColumnIndexSupplier<?> makeVariantSupplierWithNull() throws IOException
   {
     ByteBuffer localDictionaryBuffer = ByteBuffer.allocate(1 << 12).order(ByteOrder.nativeOrder());
     ByteBuffer bitmapsBuffer = ByteBuffer.allocate(1 << 12);
@@ -1945,19 +1959,21 @@ public class NestedFieldLiteralColumnIndexSupplierTest extends InitializedNullHa
 
     GenericIndexed<ImmutableBitmap> bitmaps = GenericIndexed.read(bitmapsBuffer, roaringFactory.getObjectStrategy());
 
-    return new NestedFieldLiteralColumnIndexSupplier<>(
-        new NestedLiteralTypeInfo.TypeSet(
-            new NestedLiteralTypeInfo.MutableTypeSet().add(ColumnType.STRING)
-                                                      .add(ColumnType.LONG)
-                                                      .add(ColumnType.DOUBLE)
-                                                      .getByteValue()
+    return new NestedFieldColumnIndexSupplier<>(
+        new NestedFieldTypeInfo.TypeSet(
+            new NestedFieldTypeInfo.MutableTypeSet().add(ColumnType.STRING)
+                                                    .add(ColumnType.LONG)
+                                                    .add(ColumnType.DOUBLE)
+                                                    .getByteValue()
         ),
         roaringFactory.getBitmapFactory(),
         bitmaps,
         dictionarySupplier,
         globalStrings,
         globalLongs,
-        globalDoubles
+        globalDoubles,
+        null,
+        null
     );
   }
 
diff --git a/processing/src/test/java/org/apache/druid/segment/nested/NestedFieldLiteralColumnSelectorsTest.java b/processing/src/test/java/org/apache/druid/segment/nested/NestedFieldColumnSelectorsTest.java
similarity index 95%
rename from processing/src/test/java/org/apache/druid/segment/nested/NestedFieldLiteralColumnSelectorsTest.java
rename to processing/src/test/java/org/apache/druid/segment/nested/NestedFieldColumnSelectorsTest.java
index cf65e483ba..486bb6df4f 100644
--- a/processing/src/test/java/org/apache/druid/segment/nested/NestedFieldLiteralColumnSelectorsTest.java
+++ b/processing/src/test/java/org/apache/druid/segment/nested/NestedFieldColumnSelectorsTest.java
@@ -41,6 +41,7 @@ import org.apache.druid.segment.Segment;
 import org.apache.druid.segment.StorageAdapter;
 import org.apache.druid.segment.VirtualColumns;
 import org.apache.druid.segment.column.ColumnType;
+import org.apache.druid.segment.transform.TransformSpec;
 import org.apache.druid.segment.vector.BaseDoubleVectorValueSelector;
 import org.apache.druid.segment.vector.BaseLongVectorValueSelector;
 import org.apache.druid.segment.vector.SingleValueDimensionVectorSelector;
@@ -58,7 +59,7 @@ import org.junit.rules.TemporaryFolder;
 import java.io.IOException;
 import java.util.List;
 
-public class NestedFieldLiteralColumnSelectorsTest
+public class NestedFieldColumnSelectorsTest
 {
   private static final String NESTED_LONG_FIELD = "long";
   private static final String NESTED_DOUBLE_FIELD = "double";
@@ -76,7 +77,7 @@ public class NestedFieldLiteralColumnSelectorsTest
   private final AggregationTestHelper helper;
   private final Closer closer;
 
-  public NestedFieldLiteralColumnSelectorsTest()
+  public NestedFieldColumnSelectorsTest()
   {
     NestedDataModule.registerHandlersAndSerde();
     List<? extends Module> mods = NestedDataModule.getJacksonModulesList();
@@ -326,15 +327,16 @@ public class NestedFieldLiteralColumnSelectorsTest
   private ColumnSelectorFactory getNumericColumnSelectorFactory(VirtualColumns virtualColumns) throws Exception
   {
     List<Segment> segments = NestedDataTestUtils.createSegments(
-        helper,
         tempFolder,
         closer,
         NestedDataTestUtils.NUMERIC_DATA_FILE,
-        NestedDataTestUtils.NUMERIC_PARSER_FILE,
-        NestedDataTestUtils.SIMPLE_AGG_FILE,
-        Granularities.DAY,
-        true,
-        1000
+        NestedDataTestUtils.DEFAULT_JSON_INPUT_FORMAT,
+        NestedDataTestUtils.TIMESTAMP_SPEC,
+        NestedDataTestUtils.AUTO_DISCOVERY,
+        TransformSpec.NONE,
+        NestedDataTestUtils.COUNT,
+        Granularities.NONE,
+        true
     );
     Assert.assertEquals(1, segments.size());
     StorageAdapter storageAdapter = segments.get(0).asStorageAdapter();
@@ -355,15 +357,16 @@ public class NestedFieldLiteralColumnSelectorsTest
   private VectorColumnSelectorFactory getVectorColumnSelectorFactory(VirtualColumns virtualColumns) throws Exception
   {
     List<Segment> segments = NestedDataTestUtils.createSegments(
-        helper,
         tempFolder,
         closer,
         NestedDataTestUtils.NUMERIC_DATA_FILE,
-        NestedDataTestUtils.NUMERIC_PARSER_FILE,
-        NestedDataTestUtils.SIMPLE_AGG_FILE,
-        Granularities.DAY,
-        true,
-        1000
+        NestedDataTestUtils.DEFAULT_JSON_INPUT_FORMAT,
+        NestedDataTestUtils.TIMESTAMP_SPEC,
+        NestedDataTestUtils.AUTO_DISCOVERY,
+        TransformSpec.NONE,
+        NestedDataTestUtils.COUNT,
+        Granularities.NONE,
+        true
     );
     Assert.assertEquals(1, segments.size());
     StorageAdapter storageAdapter = segments.get(0).asStorageAdapter();
diff --git a/processing/src/test/java/org/apache/druid/segment/nested/NestedFieldTypeInfoTest.java b/processing/src/test/java/org/apache/druid/segment/nested/NestedFieldTypeInfoTest.java
new file mode 100644
index 0000000000..aa100cf7af
--- /dev/null
+++ b/processing/src/test/java/org/apache/druid/segment/nested/NestedFieldTypeInfoTest.java
@@ -0,0 +1,174 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.druid.segment.nested;
+
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableSet;
+import nl.jqno.equalsverifier.EqualsVerifier;
+import nl.jqno.equalsverifier.Warning;
+import org.apache.druid.segment.column.ColumnType;
+import org.apache.druid.segment.writeout.OnHeapMemorySegmentWriteOutMedium;
+import org.junit.Assert;
+import org.junit.Test;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.ByteOrder;
+import java.nio.channels.WritableByteChannel;
+import java.util.List;
+import java.util.Set;
+
+public class NestedFieldTypeInfoTest
+{
+  private static final ByteBuffer BUFFER = ByteBuffer.allocate(1024).order(ByteOrder.nativeOrder());
+
+  @Test
+  public void testSingleType() throws IOException
+  {
+    List<ColumnType> supportedTypes = ImmutableList.of(
+        ColumnType.STRING,
+        ColumnType.LONG,
+        ColumnType.DOUBLE,
+        ColumnType.STRING_ARRAY,
+        ColumnType.LONG_ARRAY,
+        ColumnType.DOUBLE_ARRAY
+    );
+
+    for (ColumnType type : supportedTypes) {
+      testSingleType(type);
+    }
+  }
+
+  @Test
+  public void testMultiType() throws IOException
+  {
+    List<Set<ColumnType>> tests = ImmutableList.of(
+        ImmutableSet.of(ColumnType.STRING, ColumnType.LONG),
+        ImmutableSet.of(ColumnType.LONG, ColumnType.DOUBLE),
+        ImmutableSet.of(ColumnType.STRING, ColumnType.LONG, ColumnType.DOUBLE),
+        ImmutableSet.of(ColumnType.DOUBLE, ColumnType.DOUBLE_ARRAY),
+        ImmutableSet.of(ColumnType.LONG_ARRAY, ColumnType.DOUBLE_ARRAY)
+    );
+
+    for (Set<ColumnType> typeSet : tests) {
+      testMultiType(typeSet);
+    }
+  }
+
+  @Test
+  public void testEqualsAndHashCode()
+  {
+    EqualsVerifier.forClass(NestedFieldTypeInfo.TypeSet.class)
+                  .usingGetClass()
+                  .verify();
+
+    EqualsVerifier.forClass(NestedFieldTypeInfo.MutableTypeSet.class)
+                  .suppress(Warning.NONFINAL_FIELDS)
+                  .usingGetClass()
+                  .verify();
+  }
+
+  private void testSingleType(ColumnType columnType) throws IOException
+  {
+    NestedFieldTypeInfo.MutableTypeSet typeSet = new NestedFieldTypeInfo.MutableTypeSet();
+    Assert.assertNull(typeSet.getSingleType());
+    Assert.assertTrue(typeSet.isEmpty());
+
+    typeSet.add(columnType);
+
+    Assert.assertEquals(columnType, typeSet.getSingleType());
+    Assert.assertEquals(ImmutableSet.of(columnType), NestedFieldTypeInfo.convertToSet(typeSet.getByteValue()));
+
+    writeTypeSet(typeSet);
+    NestedFieldTypeInfo info = new NestedFieldTypeInfo(BUFFER);
+    Assert.assertEquals(0, BUFFER.position());
+
+    NestedFieldTypeInfo.TypeSet roundTrip = info.getTypes(0);
+    Assert.assertEquals(columnType, roundTrip.getSingleType());
+
+    NestedFieldTypeInfo info2 = NestedFieldTypeInfo.read(BUFFER, 1);
+    Assert.assertEquals(info.getTypes(0), info2.getTypes(0));
+    Assert.assertEquals(1, BUFFER.position());
+  }
+
+  private void testMultiType(Set<ColumnType> columnTypes) throws IOException
+  {
+    NestedFieldTypeInfo.MutableTypeSet typeSet = new NestedFieldTypeInfo.MutableTypeSet();
+    Assert.assertNull(typeSet.getSingleType());
+    Assert.assertTrue(typeSet.isEmpty());
+
+    NestedFieldTypeInfo.MutableTypeSet merge = new NestedFieldTypeInfo.MutableTypeSet();
+    for (ColumnType columnType : columnTypes) {
+      typeSet.add(columnType);
+      merge.merge(new NestedFieldTypeInfo.MutableTypeSet().add(columnType).getByteValue());
+    }
+
+    Assert.assertEquals(merge.getByteValue(), typeSet.getByteValue());
+    Assert.assertNull(typeSet.getSingleType());
+    Assert.assertEquals(columnTypes, NestedFieldTypeInfo.convertToSet(typeSet.getByteValue()));
+
+    writeTypeSet(typeSet);
+    NestedFieldTypeInfo info = new NestedFieldTypeInfo(BUFFER);
+    Assert.assertEquals(0, BUFFER.position());
+
+    NestedFieldTypeInfo.TypeSet roundTrip = info.getTypes(0);
+    Assert.assertNull(roundTrip.getSingleType());
+    Assert.assertEquals(columnTypes, NestedFieldTypeInfo.convertToSet(roundTrip.getByteValue()));
+
+    NestedFieldTypeInfo info2 = NestedFieldTypeInfo.read(BUFFER, 1);
+    Assert.assertEquals(info.getTypes(0), info2.getTypes(0));
+    Assert.assertEquals(1, BUFFER.position());
+  }
+
+  private static void writeTypeSet(NestedFieldTypeInfo.MutableTypeSet typeSet) throws IOException
+  {
+    BUFFER.position(0);
+    NestedFieldTypeInfo.Writer writer = new NestedFieldTypeInfo.Writer(new OnHeapMemorySegmentWriteOutMedium());
+    writer.open();
+    writer.write(typeSet);
+    Assert.assertEquals(1, writer.getSerializedSize());
+
+    WritableByteChannel channel = new WritableByteChannel()
+    {
+      @Override
+      public int write(ByteBuffer src)
+      {
+        int size = src.remaining();
+        BUFFER.put(src);
+        return size;
+      }
+
+      @Override
+      public boolean isOpen()
+      {
+        return true;
+      }
+
+      @Override
+      public void close()
+      {
+      }
+    };
+    writer.writeTo(channel, null);
+    Assert.assertEquals(1, BUFFER.position());
+
+    BUFFER.position(0);
+  }
+}
diff --git a/processing/src/test/java/org/apache/druid/segment/transform/TransformSpecTest.java b/processing/src/test/java/org/apache/druid/segment/transform/TransformSpecTest.java
index b9244b3ae1..3ed0a51bf0 100644
--- a/processing/src/test/java/org/apache/druid/segment/transform/TransformSpecTest.java
+++ b/processing/src/test/java/org/apache/druid/segment/transform/TransformSpecTest.java
@@ -237,7 +237,7 @@ public class TransformSpecTest extends InitializedNullHandlingTest
       Assert.assertEquals(0L, row2.getRaw("truthy2"));
     }
     finally {
-      ExpressionProcessing.initializeForTests(null);
+      ExpressionProcessing.initializeForTests();
     }
     try {
       ExpressionProcessing.initializeForStrictBooleansTests(false);
@@ -268,7 +268,7 @@ public class TransformSpecTest extends InitializedNullHandlingTest
       Assert.assertEquals(0L, row2.getRaw("truthy2"));
     }
     finally {
-      ExpressionProcessing.initializeForTests(null);
+      ExpressionProcessing.initializeForTests();
     }
   }
 
diff --git a/processing/src/test/java/org/apache/druid/testing/InitializedNullHandlingTest.java b/processing/src/test/java/org/apache/druid/testing/InitializedNullHandlingTest.java
index 1ba482d38a..b033475700 100644
--- a/processing/src/test/java/org/apache/druid/testing/InitializedNullHandlingTest.java
+++ b/processing/src/test/java/org/apache/druid/testing/InitializedNullHandlingTest.java
@@ -26,6 +26,6 @@ public class InitializedNullHandlingTest
 {
   static {
     NullHandling.initializeForTests();
-    ExpressionProcessing.initializeForTests(null);
+    ExpressionProcessing.initializeForTests();
   }
 }
diff --git a/processing/src/test/resources/nested-array-test-data.json b/processing/src/test/resources/nested-array-test-data.json
new file mode 100644
index 0000000000..b8ae3ace38
--- /dev/null
+++ b/processing/src/test/resources/nested-array-test-data.json
@@ -0,0 +1,14 @@
+{"timestamp": "2023-01-01T00:00:00", "arrayString": ["a", "b"],       "arrayStringNulls": ["a", "b"],         "arrayLong":[1, 2, 3],    "arrayLongNulls":[1, null,3],   "arrayDouble":[1.1, 2.2, 3.3],  "arrayDoubleNulls":[1.1, 2.2, null],  "arrayVariant":["a", 1, 2.2],      "arrayNestedLong":[[1, 2, null], [3, 4]],     "arrayObject":[{"x": 1},{"x":2}]}
+{"timestamp": "2023-01-01T00:00:00", "arrayString": ["a", "b", "c"],  "arrayStringNulls": [null, "b"],        "arrayLong":[2, 3],                                       "arrayDouble":[3.3, 4.4, 5.5],  "arrayDoubleNulls":[999, null, 5.5],  "arrayVariant":[null, null, 2.2],  "arrayNestedLong":[null, [null], []],          "arrayObject":[{"x": 3},{"x":4}]}
+{"timestamp": "2023-01-01T00:00:00", "arrayString": ["b", "c"],       "arrayStringNulls": ["d", null, "b"],   "arrayLong":[1, 2, 3, 4], "arrayLongNulls":[1, 2, 3],     "arrayDouble":[1.1, 3.3],       "arrayDoubleNulls":[null, 2.2, null], "arrayVariant":[1, null, 1],       "arrayNestedLong":[[1], null, [1, 2, 3]],      "arrayObject":[null,{"x":2}]}
+{"timestamp": "2023-01-01T00:00:00", "arrayString": ["d", "e"],       "arrayStringNulls": ["b", "b"],         "arrayLong":[1, 4],       "arrayLongNulls":[1],           "arrayDouble":[2.2, 3.3, 4.0],                                        "arrayVariant":["a", "b", "c"],    "arrayNestedLong":[[1, 2], [3, 4], [5, 6, 7]], "arrayObject":[{"x": null},{"x":2}]}
+{"timestamp": "2023-01-01T00:00:00", "arrayString": null,                                                     "arrayLong":[1, 2, 3],    "arrayLongNulls":[],            "arrayDouble":[1.1, 2.2, 3.3],  "arrayDoubleNulls":null,                                                                                                "arrayObject":[{"x": 1000},{"y":2000}]}
+{"timestamp": "2023-01-01T00:00:00", "arrayString": ["a", "b"],       "arrayStringNulls": null,                                         "arrayLongNulls":[null, 2, 9],  "arrayDouble":null,             "arrayDoubleNulls":[999, 5.5, null],  "arrayVariant":["a", 1, 2.2],      "arrayNestedLong":[[1], [1, 2, null]],         "arrayObject":[{"a": 1},{"b":2}]}
+{"timestamp": "2023-01-01T00:00:00",                                  "arrayStringNulls": ["a", "b"],         "arrayLong":null,         "arrayLongNulls":[2, 3],                                        "arrayDoubleNulls":[null],            "arrayVariant":null,               "arrayNestedLong":null,                        "arrayObject":[{"x": 1},{"x":2}]}
+{"timestamp": "2023-01-02T00:00:00", "arrayString": ["a", "b"],       "arrayStringNulls": [],                 "arrayLong":[1, 2, 3],    "arrayLongNulls":[1, null,3],   "arrayDouble":[1.1, 2.2, 3.3],  "arrayDoubleNulls":[1.1, 2.2, null],  "arrayVariant":["a", 1, 2.2],      "arrayNestedLong":[[2, 3], [1, 5]],            "arrayObject":[{"x": 1},{"x":2}]}
+{"timestamp": "2023-01-02T00:00:00", "arrayString": ["a", "b", "c"],  "arrayStringNulls": [null, "b"],        "arrayLong":[2, 3],                                       "arrayDouble":[3.3, 4.4, 5.5],  "arrayDoubleNulls":[999, null, 5.5],  "arrayVariant":[null, null, 2.2],  "arrayNestedLong":[null],                      "arrayObject":[{"x": 3},{"x":4}]}
+{"timestamp": "2023-01-02T00:00:00", "arrayString": ["b", "c"],       "arrayStringNulls": ["d", null, "b"],   "arrayLong":[1, 2, 3, 4], "arrayLongNulls":[1, 2, 3],     "arrayDouble":[1.1, 3.3],       "arrayDoubleNulls":[null, 2.2, null], "arrayVariant":[1, null, 1],       "arrayNestedLong":[[1], null, [1]],            "arrayObject":[null,{"x":2}]}
+{"timestamp": "2023-01-02T00:00:00", "arrayString": ["d", "e"],       "arrayStringNulls": ["b", "b"],         "arrayLong":[1, 4],       "arrayLongNulls":[null],        "arrayDouble":[2.2, 3.3, 4.0],                                        "arrayVariant":["a", "b", "c"],    "arrayNestedLong":[[1, 2], [3, 4], [5, 6, 7]], "arrayObject":[{"x": null},{"x":2}]}
+{"timestamp": "2023-01-02T00:00:00", "arrayString": null,                                                     "arrayLong":[1, 2, 3],    "arrayLongNulls":null,          "arrayDouble":[1.1, 2.2, 3.3],  "arrayDoubleNulls":[],                                                                                                  "arrayObject":[{"x": 1000},{"y":2000}]}
+{"timestamp": "2023-01-02T00:00:00", "arrayString": ["a", "b"],       "arrayStringNulls": [null],                                       "arrayLongNulls":[null, 2, 9],  "arrayDouble":null,             "arrayDoubleNulls":[999, 5.5, null],  "arrayVariant":["a", 1, 2.2],      "arrayNestedLong":[],                          "arrayObject":[{"a": 1},{"b":2}]}
+{"timestamp": "2023-01-02T00:00:00",                                  "arrayStringNulls": ["a", "b"],         "arrayLong":null,         "arrayLongNulls":[2, 3],                                        "arrayDoubleNulls":[null, 1.1],       "arrayVariant":null,               "arrayNestedLong":null,                        "arrayObject":[{"x": 1},{"x":2}]}
\ No newline at end of file
diff --git a/processing/src/test/resources/nested-numeric-test-data.json b/processing/src/test/resources/nested-numeric-test-data.json
new file mode 100644
index 0000000000..ba79dca0e9
--- /dev/null
+++ b/processing/src/test/resources/nested-numeric-test-data.json
@@ -0,0 +1,10 @@
+{"timestamp": "2023-01-01", "nest": {"long": 100, "double": 100.1, "mixed_numeric": 100,    "mixed":"foo",  "sparse_long": 100,                         "sparse_mixed_numeric": 100,    "sparse_mixed": "foo"}}
+{"timestamp": "2023-01-01", "nest": {"long": 200, "double": 200.2, "mixed_numeric": 200.4,  "mixed":"200",                      "sparse_double": 200.2, "sparse_mixed_numeric": 200.2}}
+{"timestamp": "2023-01-01", "nest": {"long": 300, "double": 300.3, "mixed_numeric": 300,    "mixed":300,    "sparse_long": 300, "sparse_double": 300.3,                                 "sparse_mixed": 300}}
+{"timestamp": "2023-01-01", "nest": {"long": 400, "double": 400.4, "mixed_numeric": 400.4,  "mixed":"bar",                      "sparse_double": 400.4, "sparse_mixed_numeric": 400}}
+{"timestamp": "2023-01-01", "nest": {"long": 500, "double": 500.5, "mixed_numeric": 500,    "mixed":500.5,  "sparse_long": 500,                         "sparse_mixed_numeric": 500.5, "sparse_mixed": 500.5}}
... 2350 lines suppressed ...


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org