You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@druid.apache.org by gi...@apache.org on 2019/07/12 19:54:17 UTC

[incubator-druid] branch master updated: Query vectorization. (#6794)

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

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


The following commit(s) were added to refs/heads/master by this push:
     new ffa25b7  Query vectorization. (#6794)
ffa25b7 is described below

commit ffa25b78321a56b210a8ad5b6c9ca61b3b737153
Author: Gian Merlino <gi...@gmail.com>
AuthorDate: Fri Jul 12 12:54:07 2019 -0700

    Query vectorization. (#6794)
    
    * Benchmarks: New SqlBenchmark, add caching & vectorization to some others.
    
    - Introduce a new SqlBenchmark geared towards benchmarking a wide
      variety of SQL queries. Rename the old SqlBenchmark to
      SqlVsNativeBenchmark.
    - Add (optional) caching to SegmentGenerator to enable easier
      benchmarking of larger segments.
    - Add vectorization to FilteredAggregatorBenchmark and GroupByBenchmark.
    
    * Query vectorization.
    
    This patch includes vectorized timeseries and groupBy engines, as well
    as some analogs of your favorite Druid classes:
    
    - VectorCursor is like Cursor. (It comes from StorageAdapter.makeVectorCursor.)
    - VectorColumnSelectorFactory is like ColumnSelectorFactory, and it has
      methods to create analogs of the column selectors you know and love.
    - VectorOffset and ReadableVectorOffset are like Offset and ReadableOffset.
    - VectorAggregator is like BufferAggregator.
    - VectorValueMatcher is like ValueMatcher.
    
    There are some noticeable differences between vectorized and regular
    execution:
    
    - Unlike regular cursors, vector cursors do not understand time
      granularity. They expect query engines to handle this on their own,
      which a new VectorCursorGranularizer class helps with. This is to
      avoid too much batch-splitting and to respect the fact that vector
      selectors are somewhat more heavyweight than regular selectors.
    - Unlike FilteredOffset, FilteredVectorOffset does not leverage indexes
      for filters that might partially support them (like an OR of one
      filter that supports indexing and another that doesn't). I'm not sure
      that this behavior is desirable anyway (it is potentially too eager)
      but, at any rate, it'd be better to harmonize it between the two
      classes. Potentially they should both do some different thing that
      is smarter than what either of them is doing right now.
    - When vector cursors are created by QueryableIndexCursorSequenceBuilder,
      they use a morphing binary-then-linear search to find their start and
      end rows, rather than linear search.
    
    Limitations in this patch are:
    
    - Only timeseries and groupBy have vectorized engines.
    - GroupBy doesn't handle multi-value dimensions yet.
    - Vector cursors cannot handle virtual columns or descending order.
    - Only some filters have vectorized matchers: "selector", "bound", "in",
      "like", "regex", "search", "and", "or", and "not".
    - Only some aggregators have vectorized implementations: "count",
      "doubleSum", "floatSum", "longSum", "hyperUnique", and "filtered".
    - Dimension specs other than "default" don't work yet (no extraction
      functions or filtered dimension specs).
    
    Currently, the testing strategy includes adding vectorization-enabled
    tests to TimeseriesQueryRunnerTest, GroupByQueryRunnerTest,
    GroupByTimeseriesQueryRunnerTest, CalciteQueryTest, and all of the
    filtering tests that extend BaseFilterTest. In all of those classes,
    there are some test cases that don't support vectorization. They are
    marked by special function calls like "cannotVectorize" or "skipVectorize"
    that tell the test harness to either expect an exception or to skip the
    test case.
    
    Testing should be expanded in the future -- a project in and of itself.
    
    Related to #3011.
    
    * WIP
    
    * Adjustments for unused things.
    
    * Adjust javadocs.
    
    * DimensionDictionarySelector adjustments.
    
    * Add "clone" to BatchIteratorAdapter.
    
    * ValueMatcher javadocs.
    
    * Fix benchmark.
    
    * Fixups post-merge.
    
    * Expect exception on testGroupByWithStringVirtualColumn for IncrementalIndex.
    
    * BloomDimFilterSqlTest: Tag two non-vectorizable tests.
    
    * Minor adjustments.
    
    * Update surefire, bump up Xmx in Travis.
    
    * Some more adjustments.
    
    * Javadoc adjustments
    
    * AggregatorAdapters adjustments.
    
    * Additional comments.
    
    * Remove switching search.
    
    * Only missiles.
---
 .travis.yml                                        |  12 +-
 benchmarks/pom.xml                                 |  12 +
 .../benchmark/ExpressionAggregationBenchmark.java  |  21 +-
 .../benchmark/ExpressionSelectorBenchmark.java     |  21 +-
 .../benchmark/FilteredAggregatorBenchmark.java     |  33 +-
 .../benchmark/datagen/BenchmarkColumnSchema.java   |  22 +
 .../benchmark/datagen/BenchmarkSchemaInfo.java     |  11 +
 .../druid/benchmark/datagen/BenchmarkSchemas.java  |   2 +-
 .../druid/benchmark/datagen/SegmentGenerator.java  | 127 ++-
 .../druid/benchmark/query/GroupByBenchmark.java    |  47 +-
 .../apache/druid/benchmark/query/SqlBenchmark.java | 209 +++--
 ...SqlBenchmark.java => SqlVsNativeBenchmark.java} |  69 +-
 benchmarks/src/main/resources/log4j2.xml           |  32 +
 .../java/util/common/guava/FilteredSequence.java   |   4 +-
 .../util/common/guava/FilteringAccumulator.java    |   4 +-
 .../common/guava/FilteringYieldingAccumulator.java |   4 +-
 .../druid/java/util/common/guava/Sequence.java     |  11 +-
 .../druid/java/util/common/guava/Sequences.java    |   2 +-
 docs/content/querying/query-context.md             |  28 +
 ...ouperUsingSketchMergeAggregatorFactoryTest.java |  15 +-
 .../druid/query/filter/BloomDimFilterTest.java     |  15 +-
 .../query/filter/sql/BloomDimFilterSqlTest.java    |   6 +
 .../variance/VarianceGroupByQueryTest.java         |  18 +-
 .../variance/VarianceTimeseriesQueryTest.java      |  15 +-
 pom.xml                                            |   2 +-
 .../bitmap/BatchIteratorAdapter.java}              |  48 +-
 .../druid/collections/bitmap/ImmutableBitmap.java  |   9 +
 .../bitmap/WrappedImmutableRoaringBitmap.java      |   7 +
 .../apache/druid/query/DefaultQueryMetrics.java    |   6 +
 .../java/org/apache/druid/query/QueryContexts.java |  61 ++
 .../java/org/apache/druid/query/QueryMetrics.java  |   6 +
 .../org/apache/druid/query/QueryRunnerHelper.java  |  24 +-
 .../query/aggregation/AggregatorAdapters.java      | 389 +++++++++
 .../druid/query/aggregation/AggregatorFactory.java |  18 +
 .../druid/query/aggregation/BufferAggregator.java  |   4 +
 .../query/aggregation/CountAggregatorFactory.java  |  13 +
 .../aggregation/CountVectorAggregator.java}        |  48 +-
 .../aggregation/DoubleSumAggregatorFactory.java    |  23 +
 .../aggregation/DoubleSumVectorAggregator.java     |  83 ++
 .../aggregation/FilteredAggregatorFactory.java     |  60 +-
 .../aggregation/FilteredVectorAggregator.java      | 153 ++++
 .../aggregation/FloatSumAggregatorFactory.java     |  23 +
 .../aggregation/FloatSumVectorAggregator.java      |  84 ++
 .../aggregation/LongSumAggregatorFactory.java      |  27 +-
 .../query/aggregation/LongSumVectorAggregator.java |  83 ++
 .../aggregation/NoopVectorAggregator.java}         |  48 +-
 .../aggregation/NullableAggregatorFactory.java     |  71 +-
 .../aggregation/NullableBufferAggregator.java      |  26 +-
 .../aggregation/NullableVectorAggregator.java      | 165 ++++
 .../aggregation/SuppressedAggregatorFactory.java   |  93 ++
 .../druid/query/aggregation/VectorAggregator.java  |  86 ++
 .../cardinality/CardinalityBufferAggregator.java   |  13 +-
 .../hyperloglog/HyperUniquesAggregatorFactory.java |  22 +
 .../hyperloglog/HyperUniquesBufferAggregator.java  |  30 +-
 .../hyperloglog/HyperUniquesVectorAggregator.java  | 116 +++
 .../query/dimension/DefaultDimensionSpec.java      |  20 +
 .../druid/query/dimension/DimensionSpec.java       |  22 +
 .../dimension/VectorColumnStrategizer.java}        |  28 +-
 .../java/org/apache/druid/query/filter/Filter.java |  22 +
 .../druid/query/filter/IntervalDimFilter.java      |   6 +
 .../StringValueMatcherColumnSelectorStrategy.java  |  64 +-
 .../apache/druid/query/filter/ValueMatcher.java    |   5 +
 .../ValueMatcherColumnSelectorStrategyFactory.java |   2 +-
 .../filter/vector/BaseVectorValueMatcher.java}     |  28 +-
 .../filter/vector/BooleanVectorValueMatcher.java   |  63 ++
 .../filter/vector/DoubleVectorValueMatcher.java    | 105 +++
 .../filter/vector/FloatVectorValueMatcher.java     | 105 +++
 .../filter/vector/LongVectorValueMatcher.java      | 105 +++
 .../vector/MultiValueStringVectorValueMatcher.java | 208 +++++
 .../query/filter/vector/ReadableVectorMatch.java   |  68 ++
 .../SingleValueStringVectorValueMatcher.java       | 192 +++++
 .../druid/query/filter/vector/VectorMatch.java     | 267 ++++++
 .../query/filter/vector/VectorValueMatcher.java    |  42 +
 .../VectorValueMatcherColumnStrategizer.java       |  74 ++
 .../filter/vector/VectorValueMatcherFactory.java}  |  13 +-
 .../apache/druid/query/groupby/GroupByQuery.java   |   1 +
 .../druid/query/groupby/GroupByQueryConfig.java    |  16 +-
 .../druid/query/groupby/GroupByQueryEngine.java    |   3 +-
 .../groupby/RowBasedColumnSelectorFactory.java     |   3 +-
 .../epinephelinae/AbstractBufferHashGrouper.java   |  49 +-
 .../groupby/epinephelinae/AggregateResult.java     |  48 +-
 .../groupby/epinephelinae/BufferArrayGrouper.java  | 165 ++--
 .../groupby/epinephelinae/BufferHashGrouper.java   | 166 +++-
 .../groupby/epinephelinae/ByteBufferHashTable.java |  17 +-
 .../groupby/epinephelinae/ByteBufferKeySerde.java  |  91 ++
 .../epinephelinae/CloseableGrouperIterator.java    |   5 +-
 .../groupby/epinephelinae/ConcurrentGrouper.java   |   5 +-
 .../epinephelinae/GroupByQueryEngineV2.java        | 198 +++--
 .../druid/query/groupby/epinephelinae/Grouper.java |   8 +-
 .../query/groupby/epinephelinae/Groupers.java      |  66 +-
 .../epinephelinae/LimitedBufferHashGrouper.java    |  40 +-
 .../epinephelinae/RowBasedGrouperHelper.java       |  17 +-
 .../groupby/epinephelinae/SpillingGrouper.java     |  21 +-
 .../query/groupby/epinephelinae/VectorGrouper.java |  77 ++
 .../vector/DoubleGroupByVectorColumnSelector.java  |  71 ++
 .../vector/FloatGroupByVectorColumnSelector.java   |  69 ++
 .../vector/GroupByVectorColumnSelector.java}       |  21 +-
 .../vector/GroupByVectorColumnStrategizer.java     |  70 ++
 .../vector/LongGroupByVectorColumnSelector.java    |  70 ++
 ...ngleValueStringGroupByVectorColumnSelector.java |  69 ++
 .../epinephelinae/vector/VectorGroupByEngine.java  | 436 ++++++++++
 .../query/search/DefaultSearchQueryMetrics.java    |   6 +
 .../query/select/DefaultSelectQueryMetrics.java    |   6 +
 .../query/timeseries/TimeseriesQueryEngine.java    | 272 +++++-
 .../timeseries/TimeseriesQueryQueryToolChest.java  |   2 +-
 .../query/timeseries/TimeseriesResultBuilder.java  |   7 +-
 .../types/StringTopNColumnSelectorStrategy.java    |   3 +-
 .../query/vector/VectorCursorGranularizer.java     | 173 ++++
 .../druid/segment/ColumnSelectorFactory.java       |   2 +
 .../main/java/org/apache/druid/segment/Cursor.java |   4 +-
 .../org/apache/druid/segment/CursorFactory.java    |  43 +
 .../druid/segment/DimensionDictionarySelector.java | 104 +++
 .../druid/segment/DimensionHandlerUtils.java       |  78 ++
 .../apache/druid/segment/DimensionSelector.java    |  78 +-
 .../druid/segment/DoubleDimensionIndexer.java      |   2 +-
 .../druid/segment/FloatDimensionIndexer.java       |   2 +-
 .../apache/druid/segment/LongDimensionIndexer.java |   2 +-
 .../QueryableIndexCursorSequenceBuilder.java       | 602 +++++++++++++
 .../druid/segment/QueryableIndexSegment.java       |   4 +-
 .../segment/QueryableIndexStorageAdapter.java      | 603 +++++--------
 .../org/apache/druid/segment/VirtualColumns.java   |   5 +
 .../apache/druid/segment/column/BaseColumn.java    |  14 +
 .../segment/column/ColumnCapabilitiesImpl.java     |   8 +
 .../apache/druid/segment/column/ComplexColumn.java |  52 ++
 .../segment/column/DictionaryEncodedColumn.java    |  13 +
 .../apache/druid/segment/column/DoublesColumn.java |   9 +-
 .../segment/column/DoublesColumnWithNulls.java     |   8 +
 .../apache/druid/segment/column/FloatsColumn.java  |   8 +
 .../segment/column/FloatsColumnWithNulls.java      |   8 +
 .../apache/druid/segment/column/LongsColumn.java   |   8 +
 .../druid/segment/column/LongsColumnWithNulls.java |   8 +
 .../column/StringDictionaryEncodedColumn.java      | 163 +++-
 .../data/BlockLayoutColumnarDoublesSupplier.java   |  65 +-
 .../data/BlockLayoutColumnarFloatsSupplier.java    |  69 +-
 .../data/BlockLayoutColumnarLongsSupplier.java     |  52 +-
 .../apache/druid/segment/data/ColumnarDoubles.java |  76 +-
 .../apache/druid/segment/data/ColumnarFloats.java  |  75 +-
 .../apache/druid/segment/data/ColumnarLongs.java   |  75 +-
 .../druid/segment/data/ColumnarMultiInts.java      |  10 +
 .../data/CompressedVSizeColumnarIntsSupplier.java  |  64 +-
 .../CompressedVSizeColumnarMultiIntsSupplier.java  |  41 +-
 .../druid/segment/data/CompressionFactory.java     |  21 +
 .../data/EntireLayoutColumnarFloatsSupplier.java   |  16 -
 .../data/EntireLayoutColumnarLongsSupplier.java    |  16 -
 .../org/apache/druid/segment/data/IndexedInts.java |  15 +
 .../segment/data/LongsLongEncodingReader.java      |  13 +
 .../apache/druid/segment/data/ReadableOffset.java  |   2 +
 .../druid/segment/data/VSizeColumnarMultiInts.java |   6 +
 .../org/apache/druid/segment/filter/AndFilter.java |  50 +-
 .../apache/druid/segment/filter/BoundFilter.java   |  20 +
 .../segment/filter/DimensionPredicateFilter.java   |  20 +
 .../org/apache/druid/segment/filter/InFilter.java  |  20 +
 .../apache/druid/segment/filter/LikeFilter.java    |  20 +
 .../org/apache/druid/segment/filter/NotFilter.java |  33 +
 .../org/apache/druid/segment/filter/OrFilter.java  | 125 ++-
 .../druid/segment/filter/SelectorFilter.java       |  20 +
 .../IncrementalIndexColumnSelectorFactory.java     |   9 +-
 .../incremental/IncrementalIndexRowIterator.java   |   7 +-
 .../IncrementalIndexStorageAdapter.java            |  34 +-
 .../vector/BaseDoubleVectorValueSelector.java      |  88 ++
 .../vector/BaseFloatVectorValueSelector.java       |  88 ++
 .../vector/BaseLongVectorValueSelector.java        |  88 ++
 .../druid/segment/vector/BitmapVectorOffset.java   | 134 +++
 .../druid/segment/vector/FilteredVectorOffset.java | 175 ++++
 .../MultiValueDimensionVectorSelector.java}        |  22 +-
 .../druid/segment/vector/NilVectorSelector.java    | 178 ++++
 .../druid/segment/vector/NoFilterVectorOffset.java |  90 ++
 .../QueryableIndexVectorColumnSelectorFactory.java | 194 +++++
 .../druid/segment/vector/ReadableVectorOffset.java |  67 ++
 .../SingleValueDimensionVectorSelector.java}       |  22 +-
 .../vector/VectorColumnSelectorFactory.java        |  69 ++
 .../apache/druid/segment/vector/VectorCursor.java  |  76 ++
 .../VectorObjectSelector.java}                     |  17 +-
 .../VectorOffset.java}                             |  27 +-
 .../druid/segment/vector/VectorSelectorUtils.java  |  63 ++
 .../VectorSizeInspector.java}                      |  28 +-
 .../druid/segment/vector/VectorValueSelector.java  |  55 ++
 ...gInputCachingExpressionColumnValueSelector.java |   3 +-
 .../SingleStringInputDimensionSelector.java        |   3 +-
 .../apache/druid/query/QueryRunnerTestHelper.java  |   9 +
 .../druid/query/filter/vector/VectorMatchTest.java | 127 +++
 .../query/groupby/GroupByQueryRunnerTest.java      | 934 ++++++++++-----------
 .../groupby/GroupByTimeseriesQueryRunnerTest.java  | 131 +--
 .../epinephelinae/BufferArrayGrouperTest.java      |  15 +-
 .../epinephelinae/BufferHashGrouperTest.java       |  26 +-
 .../LimitedBufferHashGrouperTest.java              |  15 +-
 .../query/metadata/SegmentMetadataQueryTest.java   |  24 +-
 .../metadata/SegmentMetadataUnionQueryTest.java    |   2 +-
 .../query/spec/SpecificSegmentQueryRunnerTest.java |   2 +-
 .../timeseries/TimeseriesQueryRunnerTest.java      | 173 +++-
 .../QueryableIndexCursorSequenceBuilderTest.java   | 117 +++
 .../segment/data/CompressedFloatsSerdeTest.java    |   2 +-
 .../segment/data/CompressedLongsSerdeTest.java     |   2 +-
 .../druid/segment/filter/BaseFilterTest.java       | 240 +++++-
 .../druid/segment/filter/BoundFilterTest.java      |   4 +-
 .../segment/filter/ColumnComparisonFilterTest.java |  52 +-
 .../druid/segment/filter/ExpressionFilterTest.java | 123 +--
 .../filter/FloatAndDoubleFilteringTest.java        |   6 +-
 .../druid/segment/filter/JavaScriptFilterTest.java | 118 ++-
 .../druid/segment/filter/LongFilteringTest.java    |   4 +-
 .../druid/segment/filter/SelectorFilterTest.java   |  61 +-
 .../druid/segment/filter/TimeFilteringTest.java    |   6 +-
 .../druid/segment/virtual/VirtualColumnsTest.java  |   3 +-
 .../druid/sql/calcite/BaseCalciteQueryTest.java    |  64 +-
 .../apache/druid/sql/calcite/CalciteQueryTest.java | 232 ++++-
 205 files changed, 10929 insertions(+), 2143 deletions(-)

diff --git a/.travis.yml b/.travis.yml
index 0b8c577..a8aca05 100644
--- a/.travis.yml
+++ b/.travis.yml
@@ -60,7 +60,7 @@ matrix:
       before_script: unset _JAVA_OPTIONS
       script:
         # Set MAVEN_OPTS for Surefire launcher
-        - MAVEN_OPTS='-Xmx512m' mvn test -B -pl processing
+        - MAVEN_OPTS='-Xmx800m' mvn test -B -pl processing
         - sh -c "dmesg | egrep -i '(oom|out of memory|kill process|killed).*' -C 1 || exit 0"
         - free -m
 
@@ -71,7 +71,7 @@ matrix:
       before_script: unset _JAVA_OPTIONS
       script:
         # Set MAVEN_OPTS for Surefire launcher
-        - MAVEN_OPTS='-Xmx512m' mvn test -B -Ddruid.generic.useDefaultValueForNull=false -pl processing
+        - MAVEN_OPTS='-Xmx800m' mvn test -B -Ddruid.generic.useDefaultValueForNull=false -pl processing
         - sh -c "dmesg | egrep -i '(oom|out of memory|kill process|killed).*' -C 1 || exit 0"
         - free -m
 
@@ -82,7 +82,7 @@ matrix:
       before_script: unset _JAVA_OPTIONS
       script:
         # Set MAVEN_OPTS for Surefire launcher
-        - MAVEN_OPTS='-Xmx512m' mvn test -B -pl server
+        - MAVEN_OPTS='-Xmx800m' mvn test -B -pl server
 
       # server module test with SQL Compatibility enabled
     - env:
@@ -91,7 +91,7 @@ matrix:
       before_script: unset _JAVA_OPTIONS
       script:
         # Set MAVEN_OPTS for Surefire launcher
-        - MAVEN_OPTS='-Xmx512m' mvn test -B -pl server -Ddruid.generic.useDefaultValueForNull=false
+        - MAVEN_OPTS='-Xmx800m' mvn test -B -pl server -Ddruid.generic.useDefaultValueForNull=false
 
 
       # other modules test
@@ -101,7 +101,7 @@ matrix:
       before_script: unset _JAVA_OPTIONS
       script:
         # Set MAVEN_OPTS for Surefire launcher
-        - MAVEN_OPTS='-Xmx512m' mvn test -B -pl '!processing,!server'
+        - MAVEN_OPTS='-Xmx800m' mvn test -B -pl '!processing,!server'
         - sh -c "dmesg | egrep -i '(oom|out of memory|kill process|killed).*' -C 1 || exit 0"
         - free -m
 
@@ -112,7 +112,7 @@ matrix:
       before_script: unset _JAVA_OPTIONS
       script:
         # Set MAVEN_OPTS for Surefire launcher
-        - MAVEN_OPTS='-Xmx512m' mvn test -B -Ddruid.generic.useDefaultValueForNull=false -pl '!processing,!server'
+        - MAVEN_OPTS='-Xmx800m' mvn test -B -Ddruid.generic.useDefaultValueForNull=false -pl '!processing,!server'
         - sh -c "dmesg | egrep -i '(oom|out of memory|kill process|killed).*' -C 1 || exit 0"
         - free -m
 
diff --git a/benchmarks/pom.xml b/benchmarks/pom.xml
index ffc8933..37f657b 100644
--- a/benchmarks/pom.xml
+++ b/benchmarks/pom.xml
@@ -85,12 +85,24 @@
     </dependency>
     <dependency>
       <groupId>org.apache.druid</groupId>
+      <artifactId>druid-core</artifactId>
+      <version>${project.parent.version}</version>
+      <type>test-jar</type>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.druid</groupId>
       <artifactId>druid-processing</artifactId>
       <version>${project.parent.version}</version>
       <type>test-jar</type>
     </dependency>
     <dependency>
       <groupId>org.apache.druid</groupId>
+      <artifactId>druid-server</artifactId>
+      <version>${project.parent.version}</version>
+      <type>test-jar</type>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.druid</groupId>
       <artifactId>druid-sql</artifactId>
       <version>${project.parent.version}</version>
       <type>test-jar</type>
diff --git a/benchmarks/src/main/java/org/apache/druid/benchmark/ExpressionAggregationBenchmark.java b/benchmarks/src/main/java/org/apache/druid/benchmark/ExpressionAggregationBenchmark.java
index 69b8f80..bde4da9 100644
--- a/benchmarks/src/main/java/org/apache/druid/benchmark/ExpressionAggregationBenchmark.java
+++ b/benchmarks/src/main/java/org/apache/druid/benchmark/ExpressionAggregationBenchmark.java
@@ -27,6 +27,7 @@ import org.apache.druid.benchmark.datagen.SegmentGenerator;
 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.io.Closer;
 import org.apache.druid.js.JavaScriptConfig;
 import org.apache.druid.query.aggregation.BufferAggregator;
 import org.apache.druid.query.aggregation.DoubleSumAggregatorFactory;
@@ -73,15 +74,17 @@ public class ExpressionAggregationBenchmark
   @Param({"1000000"})
   private int rowsPerSegment;
 
-  private SegmentGenerator segmentGenerator;
   private QueryableIndex index;
   private JavaScriptAggregatorFactory javaScriptAggregatorFactory;
   private DoubleSumAggregatorFactory expressionAggregatorFactory;
   private ByteBuffer aggregationBuffer = ByteBuffer.allocate(Double.BYTES);
+  private Closer closer;
 
   @Setup(Level.Trial)
   public void setup()
   {
+    this.closer = Closer.create();
+
     final BenchmarkSchemaInfo schemaInfo = new BenchmarkSchemaInfo(
         ImmutableList.of(
             BenchmarkColumnSchema.makeNormal("x", ValueType.FLOAT, false, 1, 0d, 0d, 10000d, false),
@@ -99,8 +102,10 @@ public class ExpressionAggregationBenchmark
                                                .shardSpec(new LinearShardSpec(0))
                                                .build();
 
-    this.segmentGenerator = new SegmentGenerator();
-    this.index = segmentGenerator.generate(dataSegment, schemaInfo, Granularities.NONE, rowsPerSegment);
+    final SegmentGenerator segmentGenerator = closer.register(new SegmentGenerator());
+    this.index = closer.register(
+        segmentGenerator.generate(dataSegment, schemaInfo, Granularities.NONE, rowsPerSegment)
+    );
     this.javaScriptAggregatorFactory = new JavaScriptAggregatorFactory(
         "name",
         ImmutableList.of("x", "y"),
@@ -120,15 +125,7 @@ public class ExpressionAggregationBenchmark
   @TearDown(Level.Trial)
   public void tearDown() throws Exception
   {
-    if (index != null) {
-      index.close();
-      index = null;
-    }
-
-    if (segmentGenerator != null) {
-      segmentGenerator.close();
-      segmentGenerator = null;
-    }
+    closer.close();
   }
 
   @Benchmark
diff --git a/benchmarks/src/main/java/org/apache/druid/benchmark/ExpressionSelectorBenchmark.java b/benchmarks/src/main/java/org/apache/druid/benchmark/ExpressionSelectorBenchmark.java
index 9953c0e..2f92f0b 100644
--- a/benchmarks/src/main/java/org/apache/druid/benchmark/ExpressionSelectorBenchmark.java
+++ b/benchmarks/src/main/java/org/apache/druid/benchmark/ExpressionSelectorBenchmark.java
@@ -26,6 +26,7 @@ import org.apache.druid.benchmark.datagen.SegmentGenerator;
 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.io.Closer;
 import org.apache.druid.query.dimension.DefaultDimensionSpec;
 import org.apache.druid.query.dimension.ExtractionDimensionSpec;
 import org.apache.druid.query.expression.TestExprMacroTable;
@@ -72,12 +73,14 @@ public class ExpressionSelectorBenchmark
   @Param({"1000000"})
   private int rowsPerSegment;
 
-  private SegmentGenerator segmentGenerator;
   private QueryableIndex index;
+  private Closer closer;
 
   @Setup(Level.Trial)
   public void setup()
   {
+    this.closer = Closer.create();
+
     final BenchmarkSchemaInfo schemaInfo = new BenchmarkSchemaInfo(
         ImmutableList.of(
             BenchmarkColumnSchema.makeZipf(
@@ -113,22 +116,16 @@ public class ExpressionSelectorBenchmark
                                                .shardSpec(new LinearShardSpec(0))
                                                .build();
 
-    this.segmentGenerator = new SegmentGenerator();
-    this.index = segmentGenerator.generate(dataSegment, schemaInfo, Granularities.HOUR, rowsPerSegment);
+    final SegmentGenerator segmentGenerator = closer.register(new SegmentGenerator());
+    this.index = closer.register(
+        segmentGenerator.generate(dataSegment, schemaInfo, Granularities.HOUR, rowsPerSegment)
+    );
   }
 
   @TearDown(Level.Trial)
   public void tearDown() throws Exception
   {
-    if (index != null) {
-      index.close();
-      index = null;
-    }
-
-    if (segmentGenerator != null) {
-      segmentGenerator.close();
-      segmentGenerator = null;
-    }
+    closer.close();
   }
 
   @Benchmark
diff --git a/benchmarks/src/main/java/org/apache/druid/benchmark/FilteredAggregatorBenchmark.java b/benchmarks/src/main/java/org/apache/druid/benchmark/FilteredAggregatorBenchmark.java
index 448cc00..d7a98e8 100644
--- a/benchmarks/src/main/java/org/apache/druid/benchmark/FilteredAggregatorBenchmark.java
+++ b/benchmarks/src/main/java/org/apache/druid/benchmark/FilteredAggregatorBenchmark.java
@@ -20,6 +20,7 @@
 package org.apache.druid.benchmark;
 
 import com.fasterxml.jackson.databind.ObjectMapper;
+import com.google.common.collect.ImmutableMap;
 import com.google.common.io.Files;
 import org.apache.commons.io.FileUtils;
 import org.apache.druid.benchmark.datagen.BenchmarkDataGenerator;
@@ -31,7 +32,6 @@ import org.apache.druid.jackson.DefaultObjectMapper;
 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.logger.Logger;
-import org.apache.druid.js.JavaScriptConfig;
 import org.apache.druid.query.Druids;
 import org.apache.druid.query.FinalizeResultsQueryRunner;
 import org.apache.druid.query.Query;
@@ -47,7 +47,6 @@ import org.apache.druid.query.aggregation.hyperloglog.HyperUniquesSerde;
 import org.apache.druid.query.filter.BoundDimFilter;
 import org.apache.druid.query.filter.DimFilter;
 import org.apache.druid.query.filter.InDimFilter;
-import org.apache.druid.query.filter.JavaScriptDimFilter;
 import org.apache.druid.query.filter.OrDimFilter;
 import org.apache.druid.query.filter.RegexDimFilter;
 import org.apache.druid.query.filter.SearchQueryDimFilter;
@@ -106,6 +105,9 @@ public class FilteredAggregatorBenchmark
   @Param({"basic"})
   private String schema;
 
+  @Param({"false", "true"})
+  private String vectorize;
+
   private static final Logger log = new Logger(FilteredAggregatorBenchmark.class);
   private static final int RNG_SEED = 9999;
   private static final IndexMergerV9 INDEX_MERGER_V9;
@@ -160,12 +162,6 @@ public class FilteredAggregatorBenchmark
     filter = new OrDimFilter(
         Arrays.asList(
             new BoundDimFilter("dimSequential", "-1", "-1", true, true, null, null, StringComparators.ALPHANUMERIC),
-            new JavaScriptDimFilter(
-                "dimSequential",
-                "function(x) { return false }",
-                null,
-                JavaScriptConfig.getEnabledInstance()
-            ),
             new RegexDimFilter("dimSequential", "X", null),
             new SearchQueryDimFilter("dimSequential", new ContainsSearchQuerySpec("X", false), null),
             new InDimFilter("dimSequential", Collections.singletonList("X"), null)
@@ -233,7 +229,7 @@ public class FilteredAggregatorBenchmark
         .buildOnheap();
   }
 
-  private static <T> List<T> runQuery(QueryRunnerFactory factory, QueryRunner runner, Query<T> query)
+  private static <T> List<T> runQuery(QueryRunnerFactory factory, QueryRunner runner, Query<T> query, String vectorize)
   {
     QueryToolChest toolChest = factory.getToolchest();
     QueryRunner<T> theRunner = new FinalizeResultsQueryRunner<>(
@@ -241,7 +237,10 @@ public class FilteredAggregatorBenchmark
         toolChest
     );
 
-    Sequence<T> queryResult = theRunner.run(QueryPlus.wrap(query), new HashMap<>());
+    final QueryPlus<T> queryToRun = QueryPlus.wrap(
+        query.withOverriddenContext(ImmutableMap.of("vectorize", vectorize))
+    );
+    Sequence<T> queryResult = theRunner.run(queryToRun, new HashMap<>());
     return queryResult.toList();
   }
 
@@ -268,7 +267,12 @@ public class FilteredAggregatorBenchmark
         new IncrementalIndexSegment(incIndex, SegmentId.dummy("incIndex"))
     );
 
-    List<Result<TimeseriesResultValue>> results = FilteredAggregatorBenchmark.runQuery(factory, runner, query);
+    List<Result<TimeseriesResultValue>> results = FilteredAggregatorBenchmark.runQuery(
+        factory,
+        runner,
+        query,
+        vectorize
+    );
     for (Result<TimeseriesResultValue> result : results) {
       blackhole.consume(result);
     }
@@ -285,7 +289,12 @@ public class FilteredAggregatorBenchmark
         new QueryableIndexSegment(qIndex, SegmentId.dummy("qIndex"))
     );
 
-    List<Result<TimeseriesResultValue>> results = FilteredAggregatorBenchmark.runQuery(factory, runner, query);
+    List<Result<TimeseriesResultValue>> results = FilteredAggregatorBenchmark.runQuery(
+        factory,
+        runner,
+        query,
+        vectorize
+    );
     for (Result<TimeseriesResultValue> result : results) {
       blackhole.consume(result);
     }
diff --git a/benchmarks/src/main/java/org/apache/druid/benchmark/datagen/BenchmarkColumnSchema.java b/benchmarks/src/main/java/org/apache/druid/benchmark/datagen/BenchmarkColumnSchema.java
index 0cf6d7c..549140e 100644
--- a/benchmarks/src/main/java/org/apache/druid/benchmark/datagen/BenchmarkColumnSchema.java
+++ b/benchmarks/src/main/java/org/apache/druid/benchmark/datagen/BenchmarkColumnSchema.java
@@ -424,4 +424,26 @@ public class BenchmarkColumnSchema
     schema.enumeratedProbabilities = enumeratedProbabilities;
     return schema;
   }
+
+  @Override
+  public String toString()
+  {
+    return "BenchmarkColumnSchema{" +
+           "distributionType=" + distributionType +
+           ", name='" + name + '\'' +
+           ", type=" + type +
+           ", isMetric=" + isMetric +
+           ", rowSize=" + rowSize +
+           ", nullProbability=" + nullProbability +
+           ", enumeratedValues=" + enumeratedValues +
+           ", enumeratedProbabilities=" + enumeratedProbabilities +
+           ", startInt=" + startInt +
+           ", endInt=" + endInt +
+           ", startDouble=" + startDouble +
+           ", endDouble=" + endDouble +
+           ", zipfExponent=" + zipfExponent +
+           ", mean=" + mean +
+           ", standardDeviation=" + standardDeviation +
+           '}';
+  }
 }
diff --git a/benchmarks/src/main/java/org/apache/druid/benchmark/datagen/BenchmarkSchemaInfo.java b/benchmarks/src/main/java/org/apache/druid/benchmark/datagen/BenchmarkSchemaInfo.java
index 1f43ce2..7a27209 100644
--- a/benchmarks/src/main/java/org/apache/druid/benchmark/datagen/BenchmarkSchemaInfo.java
+++ b/benchmarks/src/main/java/org/apache/druid/benchmark/datagen/BenchmarkSchemaInfo.java
@@ -68,4 +68,15 @@ public class BenchmarkSchemaInfo
   {
     return withRollup;
   }
+
+  @Override
+  public String toString()
+  {
+    return "BenchmarkSchemaInfo{" +
+           "columnSchemas=" + columnSchemas +
+           ", aggs=" + aggs +
+           ", dataInterval=" + dataInterval +
+           ", withRollup=" + withRollup +
+           '}';
+  }
 }
diff --git a/benchmarks/src/main/java/org/apache/druid/benchmark/datagen/BenchmarkSchemas.java b/benchmarks/src/main/java/org/apache/druid/benchmark/datagen/BenchmarkSchemas.java
index 3e35a66..69a9d90 100644
--- a/benchmarks/src/main/java/org/apache/druid/benchmark/datagen/BenchmarkSchemas.java
+++ b/benchmarks/src/main/java/org/apache/druid/benchmark/datagen/BenchmarkSchemas.java
@@ -94,7 +94,7 @@ public class BenchmarkSchemas
     basicSchemaIngestAggsExpression.add(new DoubleMinAggregatorFactory("minFloatZipf", "metFloatZipf"));
     basicSchemaIngestAggsExpression.add(new HyperUniquesAggregatorFactory("hyper", "dimHyperUnique"));
 
-    Interval basicSchemaDataInterval = Intervals.utc(0, 1000000);
+    Interval basicSchemaDataInterval = Intervals.of("2000-01-01/P1D");
 
     BenchmarkSchemaInfo basicSchema = new BenchmarkSchemaInfo(
         basicSchemaColumns,
diff --git a/benchmarks/src/main/java/org/apache/druid/benchmark/datagen/SegmentGenerator.java b/benchmarks/src/main/java/org/apache/druid/benchmark/datagen/SegmentGenerator.java
index c716d47..74466d0 100644
--- a/benchmarks/src/main/java/org/apache/druid/benchmark/datagen/SegmentGenerator.java
+++ b/benchmarks/src/main/java/org/apache/druid/benchmark/datagen/SegmentGenerator.java
@@ -20,7 +20,7 @@
 package org.apache.druid.benchmark.datagen;
 
 import com.google.common.collect.ImmutableList;
-import com.google.common.collect.Iterables;
+import com.google.common.hash.Hashing;
 import com.google.common.io.Files;
 import org.apache.commons.io.FileUtils;
 import org.apache.druid.data.input.InputRow;
@@ -31,6 +31,7 @@ import org.apache.druid.data.input.impl.FloatDimensionSchema;
 import org.apache.druid.data.input.impl.LongDimensionSchema;
 import org.apache.druid.data.input.impl.StringDimensionSchema;
 import org.apache.druid.java.util.common.ISE;
+import org.apache.druid.java.util.common.StringUtils;
 import org.apache.druid.java.util.common.granularity.Granularity;
 import org.apache.druid.java.util.common.logger.Logger;
 import org.apache.druid.query.aggregation.AggregatorFactory;
@@ -40,18 +41,20 @@ import org.apache.druid.segment.IndexSpec;
 import org.apache.druid.segment.QueryableIndex;
 import org.apache.druid.segment.QueryableIndexIndexableAdapter;
 import org.apache.druid.segment.TestHelper;
+import org.apache.druid.segment.data.RoaringBitmapSerdeFactory;
 import org.apache.druid.segment.incremental.IncrementalIndexSchema;
 import org.apache.druid.segment.serde.ComplexMetrics;
 import org.apache.druid.segment.writeout.OffHeapMemorySegmentWriteOutMediumFactory;
 import org.apache.druid.timeline.DataSegment;
 import org.apache.druid.timeline.SegmentId;
 
+import javax.annotation.Nullable;
 import java.io.Closeable;
 import java.io.File;
 import java.io.IOException;
+import java.nio.charset.StandardCharsets;
 import java.util.ArrayList;
 import java.util.List;
-import java.util.concurrent.atomic.AtomicInteger;
 import java.util.stream.Collectors;
 
 public class SegmentGenerator implements Closeable
@@ -59,15 +62,43 @@ public class SegmentGenerator implements Closeable
   private static final Logger log = new Logger(SegmentGenerator.class);
 
   private static final int MAX_ROWS_IN_MEMORY = 200000;
-  private static final int STARTING_SEED = 9999; // Consistent seed for reproducibility
 
-  private final File tempDir;
-  private final AtomicInteger seed;
+  // Setup can take a long time due to the need to generate large segments.
+  // Allow users to specify a cache directory via a JVM property or an environment variable.
+  private static final String CACHE_DIR_PROPERTY = "druid.benchmark.cacheDir";
+  private static final String CACHE_DIR_ENV_VAR = "DRUID_BENCHMARK_CACHE_DIR";
+
+  private final File cacheDir;
+  private final boolean cleanupCacheDir;
 
   public SegmentGenerator()
   {
-    this.tempDir = Files.createTempDir();
-    this.seed = new AtomicInteger(STARTING_SEED);
+    this(null);
+  }
+
+  public SegmentGenerator(@Nullable final File cacheDir)
+  {
+    if (cacheDir != null) {
+      this.cacheDir = cacheDir;
+      this.cleanupCacheDir = false;
+    } else {
+      final String userConfiguredCacheDir = System.getProperty(CACHE_DIR_PROPERTY, System.getenv(CACHE_DIR_ENV_VAR));
+      if (userConfiguredCacheDir != null) {
+        this.cacheDir = new File(userConfiguredCacheDir);
+        this.cleanupCacheDir = false;
+      } else {
+        log.warn("No cache directory provided; benchmark data caching is disabled. "
+                 + "Set the 'druid.benchmark.cacheDir' property or 'DRUID_BENCHMARK_CACHE_DIR' environment variable "
+                 + "to use caching.");
+        this.cacheDir = Files.createTempDir();
+        this.cleanupCacheDir = true;
+      }
+    }
+  }
+
+  public File getCacheDir()
+  {
+    return cacheDir;
   }
 
   public QueryableIndex generate(
@@ -80,9 +111,32 @@ public class SegmentGenerator implements Closeable
     // In case we need to generate hyperUniques.
     ComplexMetrics.registerSerde("hyperUnique", new HyperUniquesSerde());
 
+    final String dataHash = Hashing.sha256()
+                                   .newHasher()
+                                   .putString(dataSegment.getId().toString(), StandardCharsets.UTF_8)
+                                   .putString(schemaInfo.toString(), StandardCharsets.UTF_8)
+                                   .putString(granularity.toString(), StandardCharsets.UTF_8)
+                                   .putInt(numRows)
+                                   .hash()
+                                   .toString();
+
+    final File outDir = new File(getSegmentDir(dataSegment.getId(), dataHash), "merged");
+
+    if (outDir.exists()) {
+      try {
+        log.info("Found segment with hash[%s] cached in directory[%s].", dataHash, outDir);
+        return TestHelper.getTestIndexIO().loadIndex(outDir);
+      }
+      catch (IOException e) {
+        throw new RuntimeException(e);
+      }
+    }
+
+    log.info("Writing segment with hash[%s] to directory[%s].", dataHash, outDir);
+
     final BenchmarkDataGenerator dataGenerator = new BenchmarkDataGenerator(
         schemaInfo.getColumnSchemas(),
-        seed.getAndIncrement(),
+        dataSegment.getId().hashCode(), /* Use segment identifier hashCode as seed */
         schemaInfo.getDataInterval(),
         numRows
     );
@@ -124,61 +178,69 @@ public class SegmentGenerator implements Closeable
       rows.add(row);
 
       if ((i + 1) % 20000 == 0) {
-        log.info("%,d/%,d rows generated.", i + 1, numRows);
+        log.info("%,d/%,d rows generated for[%s].", i + 1, numRows, dataSegment);
       }
 
       if (rows.size() % MAX_ROWS_IN_MEMORY == 0) {
-        indexes.add(makeIndex(dataSegment.getId(), indexes.size(), rows, indexSchema));
+        indexes.add(makeIndex(dataSegment.getId(), dataHash, indexes.size(), rows, indexSchema));
         rows.clear();
       }
     }
 
-    log.info("%,d/%,d rows generated.", numRows, numRows);
+    log.info("%,d/%,d rows generated for[%s].", numRows, numRows, dataSegment);
 
     if (rows.size() > 0) {
-      indexes.add(makeIndex(dataSegment.getId(), indexes.size(), rows, indexSchema));
+      indexes.add(makeIndex(dataSegment.getId(), dataHash, indexes.size(), rows, indexSchema));
       rows.clear();
     }
 
+    final QueryableIndex retVal;
+
     if (indexes.isEmpty()) {
       throw new ISE("No rows to index?");
-    } else if (indexes.size() == 1) {
-      return Iterables.getOnlyElement(indexes);
     } else {
       try {
-        final QueryableIndex merged = TestHelper.getTestIndexIO().loadIndex(
-            TestHelper.getTestIndexMergerV9(OffHeapMemorySegmentWriteOutMediumFactory.instance()).merge(
-                indexes.stream().map(QueryableIndexIndexableAdapter::new).collect(Collectors.toList()),
-                false,
-                schemaInfo.getAggs()
-                          .stream()
-                          .map(AggregatorFactory::getCombiningFactory)
-                          .toArray(AggregatorFactory[]::new),
-                new File(tempDir, "merged"),
-                new IndexSpec()
-            )
-        );
+        retVal = TestHelper
+            .getTestIndexIO()
+            .loadIndex(
+                TestHelper.getTestIndexMergerV9(OffHeapMemorySegmentWriteOutMediumFactory.instance())
+                          .merge(
+                              indexes.stream().map(QueryableIndexIndexableAdapter::new).collect(Collectors.toList()),
+                              false,
+                              schemaInfo.getAggs()
+                                        .stream()
+                                        .map(AggregatorFactory::getCombiningFactory)
+                                        .toArray(AggregatorFactory[]::new),
+                              outDir,
+                              new IndexSpec(new RoaringBitmapSerdeFactory(true), null, null, null)
+                          )
+            );
 
         for (QueryableIndex index : indexes) {
           index.close();
         }
-
-        return merged;
       }
       catch (IOException e) {
         throw new RuntimeException(e);
       }
     }
+
+    log.info("Finished writing segment[%s] to[%s]", dataSegment, outDir);
+
+    return retVal;
   }
 
   @Override
   public void close() throws IOException
   {
-    FileUtils.deleteDirectory(tempDir);
+    if (cleanupCacheDir) {
+      FileUtils.deleteDirectory(cacheDir);
+    }
   }
 
   private QueryableIndex makeIndex(
       final SegmentId identifier,
+      final String dataHash,
       final int indexNumber,
       final List<InputRow> rows,
       final IncrementalIndexSchema indexSchema
@@ -187,9 +249,14 @@ public class SegmentGenerator implements Closeable
     return IndexBuilder
         .create()
         .schema(indexSchema)
-        .tmpDir(new File(new File(tempDir, identifier.toString()), String.valueOf(indexNumber)))
+        .tmpDir(new File(getSegmentDir(identifier, dataHash), String.valueOf(indexNumber)))
         .segmentWriteOutMediumFactory(OffHeapMemorySegmentWriteOutMediumFactory.instance())
         .rows(rows)
         .buildMMappedIndex();
   }
+
+  private File getSegmentDir(final SegmentId identifier, final String dataHash)
+  {
+    return new File(cacheDir, StringUtils.format("%s_%s", identifier, dataHash));
+  }
 }
diff --git a/benchmarks/src/main/java/org/apache/druid/benchmark/query/GroupByBenchmark.java b/benchmarks/src/main/java/org/apache/druid/benchmark/query/GroupByBenchmark.java
index 2a61341..2eb7426 100644
--- a/benchmarks/src/main/java/org/apache/druid/benchmark/query/GroupByBenchmark.java
+++ b/benchmarks/src/main/java/org/apache/druid/benchmark/query/GroupByBenchmark.java
@@ -51,6 +51,7 @@ import org.apache.druid.query.QueryRunner;
 import org.apache.druid.query.QueryRunnerFactory;
 import org.apache.druid.query.QueryToolChest;
 import org.apache.druid.query.aggregation.AggregatorFactory;
+import org.apache.druid.query.aggregation.CountAggregatorFactory;
 import org.apache.druid.query.aggregation.DoubleMinAggregatorFactory;
 import org.apache.druid.query.aggregation.DoubleSumAggregatorFactory;
 import org.apache.druid.query.aggregation.LongSumAggregatorFactory;
@@ -134,6 +135,9 @@ public class GroupByBenchmark
   @Param({"all", "day"})
   private String queryGranularity;
 
+  @Param({"force", "false"})
+  private String vectorize;
+
   private static final Logger log = new Logger(GroupByBenchmark.class);
   private static final int RNG_SEED = 9999;
   private static final IndexMergerV9 INDEX_MERGER_V9;
@@ -178,10 +182,8 @@ public class GroupByBenchmark
     { // basic.A
       QuerySegmentSpec intervalSpec = new MultipleIntervalSegmentSpec(Collections.singletonList(basicSchema.getDataInterval()));
       List<AggregatorFactory> queryAggs = new ArrayList<>();
-      queryAggs.add(new LongSumAggregatorFactory(
-          "sumLongSequential",
-          "sumLongSequential"
-      ));
+      queryAggs.add(new CountAggregatorFactory("cnt"));
+      queryAggs.add(new LongSumAggregatorFactory("sumLongSequential", "sumLongSequential"));
       GroupByQuery queryA = GroupByQuery
           .builder()
           .setDataSource("blah")
@@ -189,6 +191,7 @@ public class GroupByBenchmark
           .setDimensions(new DefaultDimensionSpec("dimSequential", null), new DefaultDimensionSpec("dimZipf", null))
           .setAggregatorSpecs(queryAggs)
           .setGranularity(Granularity.fromString(queryGranularity))
+          .setContext(ImmutableMap.of("vectorize", vectorize))
           .build();
 
       basicQueries.put("A", queryA);
@@ -209,6 +212,7 @@ public class GroupByBenchmark
           .setDimensions(new DefaultDimensionSpec("dimSequential", null), new DefaultDimensionSpec("dimZipf", null))
           .setAggregatorSpecs(queryAggs)
           .setGranularity(Granularities.DAY)
+          .setContext(ImmutableMap.of("vectorize", vectorize))
           .build();
 
       GroupByQuery queryA = GroupByQuery
@@ -218,6 +222,7 @@ public class GroupByBenchmark
           .setDimensions(new DefaultDimensionSpec("dimSequential", null))
           .setAggregatorSpecs(queryAggs)
           .setGranularity(Granularities.WEEK)
+          .setContext(ImmutableMap.of("vectorize", vectorize))
           .build();
 
       basicQueries.put("nested", queryA);
@@ -242,6 +247,7 @@ public class GroupByBenchmark
           .setAggregatorSpecs(queryAggs)
           .setGranularity(Granularity.fromString(queryGranularity))
           .setDimFilter(new BoundDimFilter("dimUniform", "0", "100", true, true, null, null, null))
+          .setContext(ImmutableMap.of("vectorize", vectorize))
           .build();
 
       basicQueries.put("filter", queryA);
@@ -265,6 +271,7 @@ public class GroupByBenchmark
           .setDimensions(new DefaultDimensionSpec("dimZipf", null))
           .setAggregatorSpecs(queryAggs)
           .setGranularity(Granularity.fromString(queryGranularity))
+          .setContext(ImmutableMap.of("vectorize", vectorize))
           .build();
 
       basicQueries.put("singleZipf", queryA);
@@ -292,6 +299,7 @@ public class GroupByBenchmark
               queryAggs
           )
           .setGranularity(Granularity.fromString(queryGranularity))
+          .setContext(ImmutableMap.of("vectorize", vectorize))
           .build();
 
       simpleQueries.put("A", queryA);
@@ -317,6 +325,7 @@ public class GroupByBenchmark
               queryAggs
           )
           .setGranularity(Granularity.fromString(queryGranularity))
+          .setContext(ImmutableMap.of("vectorize", vectorize))
           .build();
 
       simpleLongQueries.put("A", queryA);
@@ -340,6 +349,7 @@ public class GroupByBenchmark
           .setDimensions(new DefaultDimensionSpec("dimSequential", "dimSequential", ValueType.FLOAT))
           .setAggregatorSpecs(queryAggs)
           .setGranularity(Granularity.fromString(queryGranularity))
+          .setContext(ImmutableMap.of("vectorize", vectorize))
           .build();
 
       simpleFloatQueries.put("A", queryA);
@@ -502,9 +512,9 @@ public class GroupByBenchmark
     return new IncrementalIndex.Builder()
         .setIndexSchema(
             new IncrementalIndexSchema.Builder()
-            .withMetrics(schemaInfo.getAggsArray())
-            .withRollup(withRollup)
-            .build()
+                .withMetrics(schemaInfo.getAggsArray())
+                .withRollup(withRollup)
+                .build()
         )
         .setReportParseExceptions(false)
         .setConcurrentEventAdd(true)
@@ -536,7 +546,7 @@ public class GroupByBenchmark
     }
   }
 
-  private static <T> List<T> runQuery(QueryRunnerFactory factory, QueryRunner runner, Query<T> query)
+  private static <T> Sequence<T> runQuery(QueryRunnerFactory factory, QueryRunner runner, Query<T> query)
   {
     QueryToolChest toolChest = factory.getToolchest();
     QueryRunner<T> theRunner = new FinalizeResultsQueryRunner<>(
@@ -544,8 +554,7 @@ public class GroupByBenchmark
         toolChest
     );
 
-    Sequence<T> queryResult = theRunner.run(QueryPlus.wrap(query), new HashMap<>());
-    return queryResult.toList();
+    return theRunner.run(QueryPlus.wrap(query), new HashMap<>());
   }
 
   @Benchmark
@@ -559,8 +568,13 @@ public class GroupByBenchmark
         new IncrementalIndexSegment(anIncrementalIndex, SegmentId.dummy("incIndex"))
     );
 
-    List<Row> results = GroupByBenchmark.runQuery(factory, runner, query);
-    blackhole.consume(results);
+    final Sequence<Row> results = GroupByBenchmark.runQuery(factory, runner, query);
+    final Row lastRow = results.accumulate(
+        null,
+        (accumulated, in) -> in
+    );
+
+    blackhole.consume(lastRow);
   }
 
   @Benchmark
@@ -574,8 +588,13 @@ public class GroupByBenchmark
         new QueryableIndexSegment(queryableIndexes.get(0), SegmentId.dummy("qIndex"))
     );
 
-    List<Row> results = GroupByBenchmark.runQuery(factory, runner, query);
-    blackhole.consume(results);
+    final Sequence<Row> results = GroupByBenchmark.runQuery(factory, runner, query);
+    final Row lastRow = results.accumulate(
+        null,
+        (accumulated, in) -> in
+    );
+
+    blackhole.consume(lastRow);
   }
 
   @Benchmark
diff --git a/benchmarks/src/main/java/org/apache/druid/benchmark/query/SqlBenchmark.java b/benchmarks/src/main/java/org/apache/druid/benchmark/query/SqlBenchmark.java
index 529886a..667210c 100644
--- a/benchmarks/src/main/java/org/apache/druid/benchmark/query/SqlBenchmark.java
+++ b/benchmarks/src/main/java/org/apache/druid/benchmark/query/SqlBenchmark.java
@@ -19,30 +19,26 @@
 
 package org.apache.druid.benchmark.query;
 
-import com.google.common.io.Files;
-import org.apache.commons.io.FileUtils;
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableMap;
 import org.apache.druid.benchmark.datagen.BenchmarkSchemaInfo;
 import org.apache.druid.benchmark.datagen.BenchmarkSchemas;
 import org.apache.druid.benchmark.datagen.SegmentGenerator;
-import org.apache.druid.data.input.Row;
-import org.apache.druid.java.util.common.Intervals;
 import org.apache.druid.java.util.common.Pair;
 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.io.Closer;
 import org.apache.druid.java.util.common.logger.Logger;
-import org.apache.druid.query.QueryPlus;
 import org.apache.druid.query.QueryRunnerFactoryConglomerate;
-import org.apache.druid.query.aggregation.CountAggregatorFactory;
-import org.apache.druid.query.dimension.DefaultDimensionSpec;
-import org.apache.druid.query.groupby.GroupByQuery;
 import org.apache.druid.segment.QueryableIndex;
 import org.apache.druid.server.security.AuthTestUtils;
+import org.apache.druid.server.security.AuthenticationResult;
 import org.apache.druid.server.security.NoopEscalator;
-import org.apache.druid.sql.SqlLifecycle;
-import org.apache.druid.sql.SqlLifecycleFactory;
+import org.apache.druid.sql.calcite.planner.Calcites;
+import org.apache.druid.sql.calcite.planner.DruidPlanner;
 import org.apache.druid.sql.calcite.planner.PlannerConfig;
 import org.apache.druid.sql.calcite.planner.PlannerFactory;
+import org.apache.druid.sql.calcite.planner.PlannerResult;
 import org.apache.druid.sql.calcite.schema.DruidSchema;
 import org.apache.druid.sql.calcite.schema.SystemSchema;
 import org.apache.druid.sql.calcite.util.CalciteTests;
@@ -64,39 +60,112 @@ import org.openjdk.jmh.annotations.TearDown;
 import org.openjdk.jmh.annotations.Warmup;
 import org.openjdk.jmh.infra.Blackhole;
 
-import java.io.File;
-import java.util.HashMap;
+import javax.annotation.Nullable;
 import java.util.List;
+import java.util.Map;
 import java.util.concurrent.TimeUnit;
 
 /**
- * Benchmark that compares the same groupBy query through the native query layer and through the SQL layer.
+ * Benchmark that tests various SQL queries.
  */
 @State(Scope.Benchmark)
 @Fork(value = 1)
 @Warmup(iterations = 15)
-@Measurement(iterations = 30)
+@Measurement(iterations = 25)
 public class SqlBenchmark
 {
-  @Param({"200000", "1000000"})
-  private int rowsPerSegment;
+  static {
+    Calcites.setSystemProperties();
+  }
 
   private static final Logger log = new Logger(SqlBenchmark.class);
 
-  private File tmpDir;
-  private SegmentGenerator segmentGenerator;
-  private SpecificSegmentsQuerySegmentWalker walker;
-  private SqlLifecycleFactory sqlLifecycleFactory;
-  private GroupByQuery groupByQuery;
-  private String sqlQuery;
-  private Closer resourceCloser;
+  private static final List<String> QUERIES = ImmutableList.of(
+      // 0, 1, 2, 3: Timeseries, unfiltered
+      "SELECT COUNT(*) FROM foo",
+      "SELECT COUNT(DISTINCT hyper) FROM foo",
+      "SELECT SUM(sumLongSequential), SUM(sumFloatNormal) FROM foo",
+      "SELECT FLOOR(__time TO MINUTE), SUM(sumLongSequential), SUM(sumFloatNormal) FROM foo GROUP BY 1",
+
+      // 4: Timeseries, low selectivity filter (90% of rows match)
+      "SELECT SUM(sumLongSequential), SUM(sumFloatNormal) FROM foo WHERE dimSequential NOT LIKE '%3'",
+
+      // 5: Timeseries, high selectivity filter (0.1% of rows match)
+      "SELECT SUM(sumLongSequential), SUM(sumFloatNormal) FROM foo WHERE dimSequential = '311'",
+
+      // 6: Timeseries, mixing low selectivity index-capable filter (90% of rows match) + cursor filter
+      "SELECT SUM(sumLongSequential), SUM(sumFloatNormal) FROM foo\n"
+      + "WHERE dimSequential NOT LIKE '%3' AND maxLongUniform > 10",
+
+      // 7: Timeseries, low selectivity toplevel filter (90%), high selectivity filtered aggregator (0.1%)
+      "SELECT\n"
+      + "  SUM(sumLongSequential) FILTER(WHERE dimSequential = '311'),\n"
+      + "  SUM(sumFloatNormal)\n"
+      + "FROM foo\n"
+      + "WHERE dimSequential NOT LIKE '%3'",
+
+      // 8: Timeseries, no toplevel filter, various filtered aggregators with clauses repeated.
+      "SELECT\n"
+      + "  SUM(sumLongSequential) FILTER(WHERE dimSequential = '311'),\n"
+      + "  SUM(sumLongSequential) FILTER(WHERE dimSequential <> '311'),\n"
+      + "  SUM(sumLongSequential) FILTER(WHERE dimSequential LIKE '%3'),\n"
+      + "  SUM(sumLongSequential) FILTER(WHERE dimSequential NOT LIKE '%3'),\n"
+      + "  SUM(sumLongSequential),\n"
+      + "  SUM(sumFloatNormal) FILTER(WHERE dimSequential = '311'),\n"
+      + "  SUM(sumFloatNormal) FILTER(WHERE dimSequential <> '311'),\n"
+      + "  SUM(sumFloatNormal) FILTER(WHERE dimSequential LIKE '%3'),\n"
+      + "  SUM(sumFloatNormal) FILTER(WHERE dimSequential NOT LIKE '%3'),\n"
+      + "  SUM(sumFloatNormal),\n"
+      + "  COUNT(*) FILTER(WHERE dimSequential = '311'),\n"
+      + "  COUNT(*) FILTER(WHERE dimSequential <> '311'),\n"
+      + "  COUNT(*) FILTER(WHERE dimSequential LIKE '%3'),\n"
+      + "  COUNT(*) FILTER(WHERE dimSequential NOT LIKE '%3'),\n"
+      + "  COUNT(*)\n"
+      + "FROM foo",
+
+      // 9: Timeseries, toplevel time filter, time-comparison filtered aggregators
+      "SELECT\n"
+      + "  SUM(sumLongSequential)\n"
+      + "    FILTER(WHERE __time >= TIMESTAMP '2000-01-01 00:00:00' AND __time < TIMESTAMP '2000-01-01 12:00:00'),\n"
+      + "  SUM(sumLongSequential)\n"
+      + "    FILTER(WHERE __time >= TIMESTAMP '2000-01-01 12:00:00' AND __time < TIMESTAMP '2000-01-02 00:00:00')\n"
+      + "FROM foo\n"
+      + "WHERE __time >= TIMESTAMP '2000-01-01 00:00:00' AND __time < TIMESTAMP '2000-01-02 00:00:00'",
+
+      // 10, 11: GroupBy two strings, unfiltered, unordered
+      "SELECT dimSequential, dimZipf, SUM(sumLongSequential) FROM foo GROUP BY 1, 2",
+      "SELECT dimSequential, dimZipf, SUM(sumLongSequential), COUNT(*) FROM foo GROUP BY 1, 2",
+
+      // 12, 13, 14: GroupBy one string, unfiltered, various aggregator configurations
+      "SELECT dimZipf FROM foo GROUP BY 1",
+      "SELECT dimZipf, COUNT(*) FROM foo GROUP BY 1 ORDER BY COUNT(*) DESC",
+      "SELECT dimZipf, SUM(sumLongSequential), COUNT(*) FROM foo GROUP BY 1 ORDER BY COUNT(*) DESC",
+
+      // 15, 16: GroupBy long, unfiltered, unordered; with and without aggregators
+      "SELECT maxLongUniform FROM foo GROUP BY 1",
+      "SELECT maxLongUniform, SUM(sumLongSequential), COUNT(*) FROM foo GROUP BY 1",
+
+      // 17, 18: GroupBy long, filter by long, unordered; with and without aggregators
+      "SELECT maxLongUniform FROM foo WHERE maxLongUniform > 10 GROUP BY 1",
+      "SELECT maxLongUniform, SUM(sumLongSequential), COUNT(*) FROM foo WHERE maxLongUniform > 10 GROUP BY 1"
+  );
+
+  @Param({"5000000"})
+  private int rowsPerSegment;
+
+  @Param({"false", "force"})
+  private String vectorize;
+
+  @Param({"10", "15"})
+  private String query;
+
+  @Nullable
+  private PlannerFactory plannerFactory;
+  private Closer closer = Closer.create();
 
   @Setup(Level.Trial)
   public void setup()
   {
-    tmpDir = Files.createTempDir();
-    log.info("Starting benchmark setup using tmpDir[%s], rows[%,d].", tmpDir, rowsPerSegment);
-
     final BenchmarkSchemaInfo schemaInfo = BenchmarkSchemas.SCHEMA_MAP.get("basic");
 
     final DataSegment dataSegment = DataSegment.builder()
@@ -106,87 +175,55 @@ public class SqlBenchmark
                                                .shardSpec(new LinearShardSpec(0))
                                                .build();
 
-    this.segmentGenerator = new SegmentGenerator();
+    final PlannerConfig plannerConfig = new PlannerConfig();
 
+    final SegmentGenerator segmentGenerator = closer.register(new SegmentGenerator());
+    log.info("Starting benchmark setup using cacheDir[%s], rows[%,d].", segmentGenerator.getCacheDir(), rowsPerSegment);
     final QueryableIndex index = segmentGenerator.generate(dataSegment, schemaInfo, Granularities.NONE, rowsPerSegment);
-    final Pair<QueryRunnerFactoryConglomerate, Closer> conglomerateCloserPair = CalciteTests
-        .createQueryRunnerFactoryConglomerate();
-    final QueryRunnerFactoryConglomerate conglomerate = conglomerateCloserPair.lhs;
-    final PlannerConfig plannerConfig = new PlannerConfig();
-    final DruidSchema druidSchema = CalciteTests.createMockSchema(conglomerate, walker, plannerConfig);
+
+    final Pair<QueryRunnerFactoryConglomerate, Closer> conglomerate = CalciteTests.createQueryRunnerFactoryConglomerate();
+    closer.register(conglomerate.rhs);
+
+    final SpecificSegmentsQuerySegmentWalker walker = new SpecificSegmentsQuerySegmentWalker(conglomerate.lhs).add(
+        dataSegment,
+        index
+    );
+    closer.register(walker);
+
+    final DruidSchema druidSchema = CalciteTests.createMockSchema(conglomerate.lhs, walker, plannerConfig);
     final SystemSchema systemSchema = CalciteTests.createMockSystemSchema(druidSchema, walker, plannerConfig);
-    this.walker = new SpecificSegmentsQuerySegmentWalker(conglomerate).add(dataSegment, index);
-    final PlannerFactory plannerFactory = new PlannerFactory(
+
+    plannerFactory = new PlannerFactory(
         druidSchema,
         systemSchema,
-        CalciteTests.createMockQueryLifecycleFactory(walker, conglomerate),
+        CalciteTests.createMockQueryLifecycleFactory(walker, conglomerate.lhs),
         CalciteTests.createOperatorTable(),
         CalciteTests.createExprMacroTable(),
         plannerConfig,
         AuthTestUtils.TEST_AUTHORIZER_MAPPER,
         CalciteTests.getJsonMapper()
     );
-    this.sqlLifecycleFactory = CalciteTests.createSqlLifecycleFactory(plannerFactory);
-    groupByQuery = GroupByQuery
-        .builder()
-        .setDataSource("foo")
-        .setInterval(Intervals.ETERNITY)
-        .setDimensions(new DefaultDimensionSpec("dimZipf", "d0"), new DefaultDimensionSpec("dimSequential", "d1"))
-        .setAggregatorSpecs(new CountAggregatorFactory("c"))
-        .setGranularity(Granularities.ALL)
-        .build();
-
-    sqlQuery = "SELECT\n"
-               + "  dimZipf AS d0,"
-               + "  dimSequential AS d1,\n"
-               + "  COUNT(*) AS c\n"
-               + "FROM druid.foo\n"
-               + "GROUP BY dimZipf, dimSequential";
   }
 
   @TearDown(Level.Trial)
   public void tearDown() throws Exception
   {
-    if (walker != null) {
-      walker.close();
-      walker = null;
-    }
-
-    if (segmentGenerator != null) {
-      segmentGenerator.close();
-      segmentGenerator = null;
-    }
-
-    if (resourceCloser != null) {
-      resourceCloser.close();
-    }
-
-    if (tmpDir != null) {
-      FileUtils.deleteDirectory(tmpDir);
-    }
+    closer.close();
   }
 
   @Benchmark
   @BenchmarkMode(Mode.AverageTime)
   @OutputTimeUnit(TimeUnit.MILLISECONDS)
-  public void queryNative(Blackhole blackhole)
+  public void querySql(Blackhole blackhole) throws Exception
   {
-    final Sequence<Row> resultSequence = QueryPlus.wrap(groupByQuery).run(walker, new HashMap<>());
-    final List<Row> resultList = resultSequence.toList();
-    blackhole.consume(resultList);
-  }
-
-  @Benchmark
-  @BenchmarkMode(Mode.AverageTime)
-  @OutputTimeUnit(TimeUnit.MILLISECONDS)
-  public void queryPlanner(Blackhole blackhole) throws Exception
-  {
-    SqlLifecycle sqlLifecycle = sqlLifecycleFactory.factorize();
-    final List<Object[]> results = sqlLifecycle.runSimple(
-        sqlQuery,
-        null,
-        NoopEscalator.getInstance().createEscalatedAuthenticationResult()
-    ).toList();
-    blackhole.consume(results);
+    final Map<String, Object> context = ImmutableMap.of("vectorize", vectorize);
+    final AuthenticationResult authenticationResult = NoopEscalator.getInstance()
+                                                                   .createEscalatedAuthenticationResult();
+    try (final DruidPlanner planner = plannerFactory.createPlanner(context, authenticationResult)) {
+      final PlannerResult plannerResult = planner.plan(QUERIES.get(Integer.parseInt(query)));
+      final Sequence<Object[]> resultSequence = plannerResult.run();
+      final Object[] lastRow = resultSequence.accumulate(null, (accumulated, in) -> in);
+      blackhole.consume(lastRow);
+    }
   }
 }
diff --git a/benchmarks/src/main/java/org/apache/druid/benchmark/query/SqlBenchmark.java b/benchmarks/src/main/java/org/apache/druid/benchmark/query/SqlVsNativeBenchmark.java
similarity index 80%
copy from benchmarks/src/main/java/org/apache/druid/benchmark/query/SqlBenchmark.java
copy to benchmarks/src/main/java/org/apache/druid/benchmark/query/SqlVsNativeBenchmark.java
index 529886a..fdac5ff 100644
--- a/benchmarks/src/main/java/org/apache/druid/benchmark/query/SqlBenchmark.java
+++ b/benchmarks/src/main/java/org/apache/druid/benchmark/query/SqlVsNativeBenchmark.java
@@ -19,8 +19,6 @@
 
 package org.apache.druid.benchmark.query;
 
-import com.google.common.io.Files;
-import org.apache.commons.io.FileUtils;
 import org.apache.druid.benchmark.datagen.BenchmarkSchemaInfo;
 import org.apache.druid.benchmark.datagen.BenchmarkSchemas;
 import org.apache.druid.benchmark.datagen.SegmentGenerator;
@@ -38,11 +36,12 @@ import org.apache.druid.query.dimension.DefaultDimensionSpec;
 import org.apache.druid.query.groupby.GroupByQuery;
 import org.apache.druid.segment.QueryableIndex;
 import org.apache.druid.server.security.AuthTestUtils;
+import org.apache.druid.server.security.AuthenticationResult;
 import org.apache.druid.server.security.NoopEscalator;
-import org.apache.druid.sql.SqlLifecycle;
-import org.apache.druid.sql.SqlLifecycleFactory;
+import org.apache.druid.sql.calcite.planner.DruidPlanner;
 import org.apache.druid.sql.calcite.planner.PlannerConfig;
 import org.apache.druid.sql.calcite.planner.PlannerFactory;
+import org.apache.druid.sql.calcite.planner.PlannerResult;
 import org.apache.druid.sql.calcite.schema.DruidSchema;
 import org.apache.druid.sql.calcite.schema.SystemSchema;
 import org.apache.druid.sql.calcite.util.CalciteTests;
@@ -64,9 +63,7 @@ import org.openjdk.jmh.annotations.TearDown;
 import org.openjdk.jmh.annotations.Warmup;
 import org.openjdk.jmh.infra.Blackhole;
 
-import java.io.File;
 import java.util.HashMap;
-import java.util.List;
 import java.util.concurrent.TimeUnit;
 
 /**
@@ -76,26 +73,23 @@ import java.util.concurrent.TimeUnit;
 @Fork(value = 1)
 @Warmup(iterations = 15)
 @Measurement(iterations = 30)
-public class SqlBenchmark
+public class SqlVsNativeBenchmark
 {
   @Param({"200000", "1000000"})
   private int rowsPerSegment;
 
-  private static final Logger log = new Logger(SqlBenchmark.class);
+  private static final Logger log = new Logger(SqlVsNativeBenchmark.class);
 
-  private File tmpDir;
-  private SegmentGenerator segmentGenerator;
   private SpecificSegmentsQuerySegmentWalker walker;
-  private SqlLifecycleFactory sqlLifecycleFactory;
+  private PlannerFactory plannerFactory;
   private GroupByQuery groupByQuery;
   private String sqlQuery;
-  private Closer resourceCloser;
+  private Closer closer;
 
   @Setup(Level.Trial)
   public void setup()
   {
-    tmpDir = Files.createTempDir();
-    log.info("Starting benchmark setup using tmpDir[%s], rows[%,d].", tmpDir, rowsPerSegment);
+    this.closer = Closer.create();
 
     final BenchmarkSchemaInfo schemaInfo = BenchmarkSchemas.SCHEMA_MAP.get("basic");
 
@@ -106,17 +100,20 @@ public class SqlBenchmark
                                                .shardSpec(new LinearShardSpec(0))
                                                .build();
 
-    this.segmentGenerator = new SegmentGenerator();
+    final SegmentGenerator segmentGenerator = closer.register(new SegmentGenerator());
+    log.info("Starting benchmark setup using tmpDir[%s], rows[%,d].", segmentGenerator.getCacheDir(), rowsPerSegment);
 
     final QueryableIndex index = segmentGenerator.generate(dataSegment, schemaInfo, Granularities.NONE, rowsPerSegment);
     final Pair<QueryRunnerFactoryConglomerate, Closer> conglomerateCloserPair = CalciteTests
         .createQueryRunnerFactoryConglomerate();
     final QueryRunnerFactoryConglomerate conglomerate = conglomerateCloserPair.lhs;
     final PlannerConfig plannerConfig = new PlannerConfig();
+
+    this.walker = closer.register(new SpecificSegmentsQuerySegmentWalker(conglomerate).add(dataSegment, index));
     final DruidSchema druidSchema = CalciteTests.createMockSchema(conglomerate, walker, plannerConfig);
     final SystemSchema systemSchema = CalciteTests.createMockSystemSchema(druidSchema, walker, plannerConfig);
-    this.walker = new SpecificSegmentsQuerySegmentWalker(conglomerate).add(dataSegment, index);
-    final PlannerFactory plannerFactory = new PlannerFactory(
+
+    plannerFactory = new PlannerFactory(
         druidSchema,
         systemSchema,
         CalciteTests.createMockQueryLifecycleFactory(walker, conglomerate),
@@ -126,7 +123,6 @@ public class SqlBenchmark
         AuthTestUtils.TEST_AUTHORIZER_MAPPER,
         CalciteTests.getJsonMapper()
     );
-    this.sqlLifecycleFactory = CalciteTests.createSqlLifecycleFactory(plannerFactory);
     groupByQuery = GroupByQuery
         .builder()
         .setDataSource("foo")
@@ -147,23 +143,7 @@ public class SqlBenchmark
   @TearDown(Level.Trial)
   public void tearDown() throws Exception
   {
-    if (walker != null) {
-      walker.close();
-      walker = null;
-    }
-
-    if (segmentGenerator != null) {
-      segmentGenerator.close();
-      segmentGenerator = null;
-    }
-
-    if (resourceCloser != null) {
-      resourceCloser.close();
-    }
-
-    if (tmpDir != null) {
-      FileUtils.deleteDirectory(tmpDir);
-    }
+    closer.close();
   }
 
   @Benchmark
@@ -172,8 +152,8 @@ public class SqlBenchmark
   public void queryNative(Blackhole blackhole)
   {
     final Sequence<Row> resultSequence = QueryPlus.wrap(groupByQuery).run(walker, new HashMap<>());
-    final List<Row> resultList = resultSequence.toList();
-    blackhole.consume(resultList);
+    final Row lastRow = resultSequence.accumulate(null, (accumulated, in) -> in);
+    blackhole.consume(lastRow);
   }
 
   @Benchmark
@@ -181,12 +161,13 @@ public class SqlBenchmark
   @OutputTimeUnit(TimeUnit.MILLISECONDS)
   public void queryPlanner(Blackhole blackhole) throws Exception
   {
-    SqlLifecycle sqlLifecycle = sqlLifecycleFactory.factorize();
-    final List<Object[]> results = sqlLifecycle.runSimple(
-        sqlQuery,
-        null,
-        NoopEscalator.getInstance().createEscalatedAuthenticationResult()
-    ).toList();
-    blackhole.consume(results);
+    final AuthenticationResult authenticationResult = NoopEscalator.getInstance()
+                                                                   .createEscalatedAuthenticationResult();
+    try (final DruidPlanner planner = plannerFactory.createPlanner(null, authenticationResult)) {
+      final PlannerResult plannerResult = planner.plan(sqlQuery);
+      final Sequence<Object[]> resultSequence = plannerResult.run();
+      final Object[] lastRow = resultSequence.accumulate(null, (accumulated, in) -> in);
+      blackhole.consume(lastRow);
+    }
   }
 }
diff --git a/benchmarks/src/main/resources/log4j2.xml b/benchmarks/src/main/resources/log4j2.xml
new file mode 100644
index 0000000..dbce142
--- /dev/null
+++ b/benchmarks/src/main/resources/log4j2.xml
@@ -0,0 +1,32 @@
+<?xml version="1.0" encoding="UTF-8" ?>
+<!--
+  ~ 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.
+  -->
+
+<Configuration status="WARN">
+  <Appenders>
+    <Console name="Console" target="SYSTEM_OUT">
+      <PatternLayout pattern="%d{ISO8601} %p [%t] %c - %m%n"/>
+    </Console>
+  </Appenders>
+  <Loggers>
+    <Root level="info">
+      <AppenderRef ref="Console"/>
+    </Root>
+  </Loggers>
+</Configuration>
diff --git a/core/src/main/java/org/apache/druid/java/util/common/guava/FilteredSequence.java b/core/src/main/java/org/apache/druid/java/util/common/guava/FilteredSequence.java
index d3851ae..81210d7 100644
--- a/core/src/main/java/org/apache/druid/java/util/common/guava/FilteredSequence.java
+++ b/core/src/main/java/org/apache/druid/java/util/common/guava/FilteredSequence.java
@@ -28,11 +28,11 @@ import java.io.IOException;
 public class FilteredSequence<T> implements Sequence<T>
 {
   private final Sequence<T> baseSequence;
-  private final Predicate<T> pred;
+  private final Predicate<? super T> pred;
 
   public FilteredSequence(
       Sequence<T> baseSequence,
-      Predicate<T> pred
+      Predicate<? super T> pred
   )
   {
     this.baseSequence = baseSequence;
diff --git a/core/src/main/java/org/apache/druid/java/util/common/guava/FilteringAccumulator.java b/core/src/main/java/org/apache/druid/java/util/common/guava/FilteringAccumulator.java
index 87b5b27..36d36bc 100644
--- a/core/src/main/java/org/apache/druid/java/util/common/guava/FilteringAccumulator.java
+++ b/core/src/main/java/org/apache/druid/java/util/common/guava/FilteringAccumulator.java
@@ -25,10 +25,10 @@ import com.google.common.base.Predicate;
 */
 public class FilteringAccumulator<OutType, T> implements Accumulator<OutType, T>
 {
-  private final Predicate<T> pred;
+  private final Predicate<? super T> pred;
   private final Accumulator<OutType, T> accumulator;
 
-  public FilteringAccumulator(Predicate<T> pred, Accumulator<OutType, T> accumulator)
+  public FilteringAccumulator(Predicate<? super T> pred, Accumulator<OutType, T> accumulator)
   {
     this.pred = pred;
     this.accumulator = accumulator;
diff --git a/core/src/main/java/org/apache/druid/java/util/common/guava/FilteringYieldingAccumulator.java b/core/src/main/java/org/apache/druid/java/util/common/guava/FilteringYieldingAccumulator.java
index 5faee68..014d54e 100644
--- a/core/src/main/java/org/apache/druid/java/util/common/guava/FilteringYieldingAccumulator.java
+++ b/core/src/main/java/org/apache/druid/java/util/common/guava/FilteringYieldingAccumulator.java
@@ -25,12 +25,12 @@ import com.google.common.base.Predicate;
 */
 public class FilteringYieldingAccumulator<OutType, T> extends YieldingAccumulator<OutType, T>
 {
-  private final Predicate<T> pred;
+  private final Predicate<? super T> pred;
   private final YieldingAccumulator<OutType, T> accumulator;
 
   private volatile boolean didSomething = false;
 
-  public FilteringYieldingAccumulator(Predicate<T> pred, YieldingAccumulator<OutType, T> accumulator)
+  public FilteringYieldingAccumulator(Predicate<? super T> pred, YieldingAccumulator<OutType, T> accumulator)
   {
     this.pred = pred;
     this.accumulator = accumulator;
diff --git a/core/src/main/java/org/apache/druid/java/util/common/guava/Sequence.java b/core/src/main/java/org/apache/druid/java/util/common/guava/Sequence.java
index 555c768..c17a638 100644
--- a/core/src/main/java/org/apache/druid/java/util/common/guava/Sequence.java
+++ b/core/src/main/java/org/apache/druid/java/util/common/guava/Sequence.java
@@ -19,6 +19,7 @@
 
 package org.apache.druid.java.util.common.guava;
 
+import com.google.common.base.Predicate;
 import com.google.common.collect.Ordering;
 
 import java.io.Closeable;
@@ -53,7 +54,7 @@ public interface Sequence<T>
    */
   <OutType> OutType accumulate(OutType initValue, Accumulator<OutType, T> accumulator);
 
- /**
+  /**
    * Return a Yielder for accumulated sequence.
    *
    * @param initValue   the initial value to pass along to start the accumulation.
@@ -71,11 +72,13 @@ public interface Sequence<T>
     return new MappedSequence<>(this, mapper);
   }
 
+  default Sequence<T> filter(Predicate<? super T> predicate)
+  {
+    return Sequences.filter(this, predicate);
+  }
+
   /**
    * This will materialize the entire sequence.  Use at your own risk.
-   *
-   * Several benchmarks rely on this method to eagerly accumulate Sequences to ArrayLists.  e.g.
-   * GroupByBenchmark.
    */
   default List<T> toList()
   {
diff --git a/core/src/main/java/org/apache/druid/java/util/common/guava/Sequences.java b/core/src/main/java/org/apache/druid/java/util/common/guava/Sequences.java
index 2bab971..df6fbe5 100644
--- a/core/src/main/java/org/apache/druid/java/util/common/guava/Sequences.java
+++ b/core/src/main/java/org/apache/druid/java/util/common/guava/Sequences.java
@@ -84,7 +84,7 @@ public class Sequences
     return new MappedSequence<>(sequence, fn::apply);
   }
 
-  public static <T> Sequence<T> filter(Sequence<T> sequence, Predicate<T> pred)
+  public static <T> Sequence<T> filter(Sequence<T> sequence, Predicate<? super T> pred)
   {
     return new FilteredSequence<>(sequence, pred);
   }
diff --git a/docs/content/querying/query-context.md b/docs/content/querying/query-context.md
index d9d8218..b668cd7 100644
--- a/docs/content/querying/query-context.md
+++ b/docs/content/querying/query-context.md
@@ -60,3 +60,31 @@ In addition, some query types offer context parameters specific to that query ty
 ### GroupBy queries
 
 See [GroupBy query context](groupbyquery.html#query-context).
+
+### Vectorizable queries
+
+The GroupBy and Timeseries query types can run in _vectorized_ mode, which speeds up query execution by processing
+batches of rows at a time. Not all queries can be vectorized. In particular, vectorization currently has the following
+requirements:
+
+- All query-level filters must either be able to run on bitmap indexes or must offer vectorized row-matchers. These
+include "selector", "bound", "in", "like", "regex", "search", "and", "or", and "not".
+- All filters in filtered aggregators must offer vectorized row-matchers.
+- All aggregators must offer vectorized implementations. These include "count", "doubleSum", "floatSum", "longSum",
+"hyperUnique", and "filtered".
+- No virtual columns.
+- For GroupBy: All dimension specs must be "default" (no extraction functions or filtered dimension specs).
+- For GroupBy: No multi-value dimensions.
+- For Timeseries: No "descending" order.
+- Only immutable segments (not real-time).
+
+Other query types (like TopN, Scan, Select, and Search) ignore the "vectorize" parameter, and will execute without
+vectorization. These query types will ignore the "vectorize" parameter even if it is set to `"force"`.
+
+Vectorization is an alpha-quality feature as of Druid #{DRUIDVERSION}. We heartily welcome any feedback and testing
+from the community as we work to battle-test it.
+
+|property|default| description|
+|--------|-------|------------|
+|vectorize|`false`|Enables or disables vectorized query execution. Possible values are `false` (disabled), `true` (enabled if possible, disabled otherwise, on a per-segment basis), and `force` (enabled, and groupBy or timeseries queries that cannot be vectorized will fail). The `"force"` setting is meant to aid in testing, and is not generally useful in production (since real-time segments can never be processed with vectorized execution, any queries on real-time data will fail).|
+|vectorSize|`512`|Sets the row batching size for a particular query.|
diff --git a/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/theta/BufferHashGrouperUsingSketchMergeAggregatorFactoryTest.java b/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/theta/BufferHashGrouperUsingSketchMergeAggregatorFactoryTest.java
index f10ae2c..db7c377 100644
--- a/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/theta/BufferHashGrouperUsingSketchMergeAggregatorFactoryTest.java
+++ b/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/theta/BufferHashGrouperUsingSketchMergeAggregatorFactoryTest.java
@@ -20,12 +20,13 @@
 package org.apache.druid.query.aggregation.datasketches.theta;
 
 import com.google.common.base.Suppliers;
+import com.google.common.collect.ImmutableList;
 import com.google.common.collect.ImmutableMap;
 import com.google.common.collect.Lists;
 import com.yahoo.sketches.theta.Sketches;
 import com.yahoo.sketches.theta.UpdateSketch;
 import org.apache.druid.data.input.MapBasedRow;
-import org.apache.druid.query.aggregation.AggregatorFactory;
+import org.apache.druid.query.aggregation.AggregatorAdapters;
 import org.apache.druid.query.aggregation.CountAggregatorFactory;
 import org.apache.druid.query.groupby.epinephelinae.BufferHashGrouper;
 import org.apache.druid.query.groupby.epinephelinae.Grouper;
@@ -47,11 +48,13 @@ public class BufferHashGrouperUsingSketchMergeAggregatorFactoryTest
     final BufferHashGrouper<Integer> grouper = new BufferHashGrouper<>(
         Suppliers.ofInstance(ByteBuffer.allocate(bufferSize)),
         GrouperTestUtil.intKeySerde(),
-        columnSelectorFactory,
-        new AggregatorFactory[]{
-            new SketchMergeAggregatorFactory("sketch", "sketch", 16, false, true, 2),
-            new CountAggregatorFactory("count")
-        },
+        AggregatorAdapters.factorizeBuffered(
+            columnSelectorFactory,
+            ImmutableList.of(
+                new SketchMergeAggregatorFactory("sketch", "sketch", 16, false, true, 2),
+                new CountAggregatorFactory("count")
+            )
+        ),
         Integer.MAX_VALUE,
         0.75f,
         initialBuckets,
diff --git a/extensions-core/druid-bloom-filter/src/test/java/org/apache/druid/query/filter/BloomDimFilterTest.java b/extensions-core/druid-bloom-filter/src/test/java/org/apache/druid/query/filter/BloomDimFilterTest.java
index fb8d31a..d01b4c5 100644
--- a/extensions-core/druid-bloom-filter/src/test/java/org/apache/druid/query/filter/BloomDimFilterTest.java
+++ b/extensions-core/druid-bloom-filter/src/test/java/org/apache/druid/query/filter/BloomDimFilterTest.java
@@ -246,21 +246,24 @@ public class BloomDimFilterTest extends BaseFilterTest
   @Test
   public void testExpressionVirtualColumn() throws IOException
   {
-    assertFilterMatches(
+    assertFilterMatchesSkipVectorize(
         new BloomDimFilter("expr", bloomKFilter(1000, 1.1F), null),
         ImmutableList.of("0", "1", "2", "3", "4", "5")
     );
-    assertFilterMatches(new BloomDimFilter("expr", bloomKFilter(1000, 1.2F), null), ImmutableList.of());
-    assertFilterMatches(
+    assertFilterMatchesSkipVectorize(new BloomDimFilter("expr", bloomKFilter(1000, 1.2F), null), ImmutableList.of());
+    assertFilterMatchesSkipVectorize(
         new BloomDimFilter("exprDouble", bloomKFilter(1000, 2.1D), null),
         ImmutableList.of("0", "1", "2", "3", "4", "5")
     );
-    assertFilterMatches(new BloomDimFilter("exprDouble", bloomKFilter(1000, 2.2D), null), ImmutableList.of());
-    assertFilterMatches(
+    assertFilterMatchesSkipVectorize(
+        new BloomDimFilter("exprDouble", bloomKFilter(1000, 2.2D), null),
+        ImmutableList.of()
+    );
+    assertFilterMatchesSkipVectorize(
         new BloomDimFilter("exprLong", bloomKFilter(1000, 3L), null),
         ImmutableList.of("0", "1", "2", "3", "4", "5")
     );
-    assertFilterMatches(new BloomDimFilter("exprLong", bloomKFilter(1000, 4L), null), ImmutableList.of());
+    assertFilterMatchesSkipVectorize(new BloomDimFilter("exprLong", bloomKFilter(1000, 4L), null), ImmutableList.of());
   }
 
   @Test
diff --git a/extensions-core/druid-bloom-filter/src/test/java/org/apache/druid/query/filter/sql/BloomDimFilterSqlTest.java b/extensions-core/druid-bloom-filter/src/test/java/org/apache/druid/query/filter/sql/BloomDimFilterSqlTest.java
index be32f01..fd9f5f5 100644
--- a/extensions-core/druid-bloom-filter/src/test/java/org/apache/druid/query/filter/sql/BloomDimFilterSqlTest.java
+++ b/extensions-core/druid-bloom-filter/src/test/java/org/apache/druid/query/filter/sql/BloomDimFilterSqlTest.java
@@ -169,6 +169,9 @@ public class BloomDimFilterSqlTest extends BaseCalciteQueryTest
   @Test
   public void testBloomFilterVirtualColumn() throws Exception
   {
+    // Cannot vectorize due to expression virtual columns.
+    cannotVectorize();
+
     BloomKFilter filter = new BloomKFilter(1500);
     filter.addString("def-foo");
     byte[] bytes = BloomFilterSerializersModule.bloomKFilterToBytes(filter);
@@ -199,6 +202,9 @@ public class BloomDimFilterSqlTest extends BaseCalciteQueryTest
   @Test
   public void testBloomFilterVirtualColumnNumber() throws Exception
   {
+    // Cannot vectorize due to expression virtual columns.
+    cannotVectorize();
+
     BloomKFilter filter = new BloomKFilter(1500);
     filter.addFloat(20.2f);
     byte[] bytes = BloomFilterSerializersModule.bloomKFilterToBytes(filter);
diff --git a/extensions-core/stats/src/test/java/org/apache/druid/query/aggregation/variance/VarianceGroupByQueryTest.java b/extensions-core/stats/src/test/java/org/apache/druid/query/aggregation/variance/VarianceGroupByQueryTest.java
index 1b0ef25..b302aef 100644
--- a/extensions-core/stats/src/test/java/org/apache/druid/query/aggregation/variance/VarianceGroupByQueryTest.java
+++ b/extensions-core/stats/src/test/java/org/apache/druid/query/aggregation/variance/VarianceGroupByQueryTest.java
@@ -45,6 +45,7 @@ import org.junit.runners.Parameterized;
 import java.util.Collection;
 import java.util.Collections;
 import java.util.List;
+import java.util.stream.Collectors;
 
 /**
  *
@@ -58,9 +59,22 @@ public class VarianceGroupByQueryTest
   private final String testName;
 
   @Parameterized.Parameters(name = "{0}")
-  public static Collection<?> constructorFeeder()
+  public static Collection<Object[]> constructorFeeder()
   {
-    return GroupByQueryRunnerTest.constructorFeeder();
+    // Use GroupByQueryRunnerTest's constructorFeeder, but remove vectorized tests, since this aggregator
+    // can't vectorize yet.
+    return GroupByQueryRunnerTest.constructorFeeder().stream()
+                                 .filter(constructor -> !((boolean) constructor[4]) /* !vectorize */)
+                                 .map(
+                                     constructor ->
+                                         new Object[]{
+                                             constructor[0],
+                                             constructor[1],
+                                             constructor[2],
+                                             constructor[3]
+                                         }
+                                 )
+                                 .collect(Collectors.toList());
   }
 
   public VarianceGroupByQueryTest(
diff --git a/extensions-core/stats/src/test/java/org/apache/druid/query/aggregation/variance/VarianceTimeseriesQueryTest.java b/extensions-core/stats/src/test/java/org/apache/druid/query/aggregation/variance/VarianceTimeseriesQueryTest.java
index c5dcece..7e0e059 100644
--- a/extensions-core/stats/src/test/java/org/apache/druid/query/aggregation/variance/VarianceTimeseriesQueryTest.java
+++ b/extensions-core/stats/src/test/java/org/apache/druid/query/aggregation/variance/VarianceTimeseriesQueryTest.java
@@ -37,6 +37,8 @@ import org.junit.runners.Parameterized;
 import java.util.Arrays;
 import java.util.HashMap;
 import java.util.List;
+import java.util.stream.Collectors;
+import java.util.stream.StreamSupport;
 
 @RunWith(Parameterized.class)
 public class VarianceTimeseriesQueryTest
@@ -44,13 +46,22 @@ public class VarianceTimeseriesQueryTest
   @Parameterized.Parameters(name = "{0}:descending={1}")
   public static Iterable<Object[]> constructorFeeder()
   {
-    return TimeseriesQueryRunnerTest.constructorFeeder();
+    // Use TimeseriesQueryRunnerTest's constructorFeeder, but remove vectorized tests, since this aggregator
+    // can't vectorize yet.
+    return StreamSupport.stream(TimeseriesQueryRunnerTest.constructorFeeder().spliterator(), false)
+                        .filter(constructor -> !((boolean) constructor[2]) /* !vectorize */)
+                        .map(constructor -> new Object[]{constructor[0], constructor[1], constructor[3]})
+                        .collect(Collectors.toList());
   }
 
   private final QueryRunner runner;
   private final boolean descending;
 
-  public VarianceTimeseriesQueryTest(QueryRunner runner, boolean descending, List<AggregatorFactory> aggregatorFactories)
+  public VarianceTimeseriesQueryTest(
+      QueryRunner runner,
+      boolean descending,
+      List<AggregatorFactory> aggregatorFactories
+  )
   {
     this.runner = runner;
     this.descending = descending;
diff --git a/pom.xml b/pom.xml
index 836307a..1a802c9 100644
--- a/pom.xml
+++ b/pom.xml
@@ -1227,7 +1227,7 @@
                 <plugin>
                     <groupId>org.apache.maven.plugins</groupId>
                     <artifactId>maven-surefire-plugin</artifactId>
-                    <version>2.19.1</version>
+                    <version>2.22.2</version>
                     <configuration>
                         <!-- locale settings must be set on the command line before startup -->
                         <!-- set default options -->
diff --git a/processing/src/main/java/org/apache/druid/segment/QueryableIndexSegment.java b/processing/src/main/java/org/apache/druid/collections/bitmap/BatchIteratorAdapter.java
similarity index 52%
copy from processing/src/main/java/org/apache/druid/segment/QueryableIndexSegment.java
copy to processing/src/main/java/org/apache/druid/collections/bitmap/BatchIteratorAdapter.java
index 8f68178..225999f 100644
--- a/processing/src/main/java/org/apache/druid/segment/QueryableIndexSegment.java
+++ b/processing/src/main/java/org/apache/druid/collections/bitmap/BatchIteratorAdapter.java
@@ -17,52 +17,42 @@
  * under the License.
  */
 
-package org.apache.druid.segment;
+package org.apache.druid.collections.bitmap;
 
-import org.apache.druid.timeline.SegmentId;
-import org.joda.time.Interval;
+import com.google.common.base.Preconditions;
+import org.roaringbitmap.BatchIterator;
+import org.roaringbitmap.IntIterator;
 
-/**
- */
-public class QueryableIndexSegment extends AbstractSegment
+public final class BatchIteratorAdapter implements BatchIterator
 {
-  private final QueryableIndex index;
-  private final SegmentId segmentId;
-
-  public QueryableIndexSegment(QueryableIndex index, final SegmentId segmentId)
-  {
-    this.index = index;
-    this.segmentId = segmentId;
-  }
+  private final IntIterator iterator;
 
-  @Override
-  public SegmentId getId()
+  public BatchIteratorAdapter(IntIterator iterator)
   {
-    return segmentId;
+    this.iterator = Preconditions.checkNotNull(iterator, "iterator");
   }
 
   @Override
-  public Interval getDataInterval()
+  public int nextBatch(int[] buffer)
   {
-    return index.getDataInterval();
-  }
+    int i;
+    for (i = 0; i < buffer.length && iterator.hasNext(); i++) {
+      buffer[i] = iterator.next();
+    }
 
-  @Override
-  public QueryableIndex asQueryableIndex()
-  {
-    return index;
+    return i;
   }
 
   @Override
-  public StorageAdapter asStorageAdapter()
+  public boolean hasNext()
   {
-    return new QueryableIndexStorageAdapter(index);
+    return iterator.hasNext();
   }
 
   @Override
-  public void close()
+  public BatchIterator clone()
   {
-    // this is kinda nasty
-    index.close();
+    // It's okay to make a "new BatchIteratorAdapter" instead of calling super.clone(), since this class is final.
+    return new BatchIteratorAdapter(iterator.clone());
   }
 }
diff --git a/processing/src/main/java/org/apache/druid/collections/bitmap/ImmutableBitmap.java b/processing/src/main/java/org/apache/druid/collections/bitmap/ImmutableBitmap.java
index 6e8247c..d02e237 100644
--- a/processing/src/main/java/org/apache/druid/collections/bitmap/ImmutableBitmap.java
+++ b/processing/src/main/java/org/apache/druid/collections/bitmap/ImmutableBitmap.java
@@ -19,6 +19,7 @@
 
 package org.apache.druid.collections.bitmap;
 
+import org.roaringbitmap.BatchIterator;
 import org.roaringbitmap.IntIterator;
 
 /**
@@ -33,6 +34,14 @@ public interface ImmutableBitmap
   IntIterator iterator();
 
   /**
+   * @return a batched iterator over the set bits of this bitmap
+   */
+  default BatchIterator batchIterator()
+  {
+    return new BatchIteratorAdapter(iterator());
+  }
+
+  /**
    * @return The number of bits set to true in this bitmap
    */
   int size();
diff --git a/processing/src/main/java/org/apache/druid/collections/bitmap/WrappedImmutableRoaringBitmap.java b/processing/src/main/java/org/apache/druid/collections/bitmap/WrappedImmutableRoaringBitmap.java
index 98c45b6..763445a 100644
--- a/processing/src/main/java/org/apache/druid/collections/bitmap/WrappedImmutableRoaringBitmap.java
+++ b/processing/src/main/java/org/apache/druid/collections/bitmap/WrappedImmutableRoaringBitmap.java
@@ -19,6 +19,7 @@
 
 package org.apache.druid.collections.bitmap;
 
+import org.roaringbitmap.BatchIterator;
 import org.roaringbitmap.IntIterator;
 import org.roaringbitmap.buffer.ImmutableRoaringBitmap;
 
@@ -77,6 +78,12 @@ public class WrappedImmutableRoaringBitmap implements ImmutableBitmap
   }
 
   @Override
+  public BatchIterator batchIterator()
+  {
+    return bitmap.getBatchIterator();
+  }
+
+  @Override
   public int size()
   {
     return bitmap.getCardinality();
diff --git a/processing/src/main/java/org/apache/druid/query/DefaultQueryMetrics.java b/processing/src/main/java/org/apache/druid/query/DefaultQueryMetrics.java
index 1a029b4..4d5e3c5 100644
--- a/processing/src/main/java/org/apache/druid/query/DefaultQueryMetrics.java
+++ b/processing/src/main/java/org/apache/druid/query/DefaultQueryMetrics.java
@@ -197,6 +197,12 @@ public class DefaultQueryMetrics<QueryType extends Query<?>> implements QueryMet
   }
 
   @Override
+  public void vectorized(final boolean vectorized)
+  {
+    // Emit nothing by default.
+  }
+
+  @Override
   public BitmapResultFactory<?> makeBitmapResultFactory(BitmapFactory factory)
   {
     return new DefaultBitmapResultFactory(factory);
diff --git a/processing/src/main/java/org/apache/druid/query/QueryContexts.java b/processing/src/main/java/org/apache/druid/query/QueryContexts.java
index a192218..d5003e2 100644
--- a/processing/src/main/java/org/apache/druid/query/QueryContexts.java
+++ b/processing/src/main/java/org/apache/druid/query/QueryContexts.java
@@ -23,7 +23,10 @@ import com.google.common.base.Preconditions;
 import com.google.common.collect.ImmutableMap;
 import org.apache.druid.guice.annotations.PublicApi;
 import org.apache.druid.java.util.common.IAE;
+import org.apache.druid.java.util.common.ISE;
 import org.apache.druid.java.util.common.Numbers;
+import org.apache.druid.java.util.common.StringUtils;
+import org.apache.druid.segment.QueryableIndexStorageAdapter;
 
 import java.util.concurrent.TimeUnit;
 
@@ -43,11 +46,44 @@ public class QueryContexts
   public static final boolean DEFAULT_USE_CACHE = true;
   public static final boolean DEFAULT_POPULATE_RESULTLEVEL_CACHE = true;
   public static final boolean DEFAULT_USE_RESULTLEVEL_CACHE = true;
+  public static final Vectorize DEFAULT_VECTORIZE = Vectorize.FALSE;
   public static final int DEFAULT_PRIORITY = 0;
   public static final int DEFAULT_UNCOVERED_INTERVALS_LIMIT = 0;
   public static final long DEFAULT_TIMEOUT_MILLIS = TimeUnit.MINUTES.toMillis(5);
   public static final long NO_TIMEOUT = 0;
 
+  @SuppressWarnings("unused") // Used by Jackson serialization
+  public enum Vectorize
+  {
+    FALSE {
+      @Override
+      public boolean shouldVectorize(final boolean canVectorize)
+      {
+        return false;
+      }
+    },
+    TRUE {
+      @Override
+      public boolean shouldVectorize(final boolean canVectorize)
+      {
+        return canVectorize;
+      }
+    },
+    FORCE {
+      @Override
+      public boolean shouldVectorize(final boolean canVectorize)
+      {
+        if (!canVectorize) {
+          throw new ISE("Cannot vectorize!");
+        }
+
+        return true;
+      }
+    };
+
+    public abstract boolean shouldVectorize(boolean canVectorize);
+  }
+
   public static <T> boolean isBySegment(Query<T> query)
   {
     return isBySegment(query, DEFAULT_BY_SEGMENT);
@@ -113,6 +149,16 @@ public class QueryContexts
     return parseBoolean(query, "serializeDateTimeAsLongInner", defaultValue);
   }
 
+  public static <T> Vectorize getVectorize(Query<T> query)
+  {
+    return parseEnum(query, "vectorize", Vectorize.class, DEFAULT_VECTORIZE);
+  }
+
+  public static <T> int getVectorSize(Query<T> query)
+  {
+    return parseInt(query, "vectorSize", QueryableIndexStorageAdapter.DEFAULT_VECTOR_SIZE);
+  }
+
   public static <T> int getUncoveredIntervalsLimit(Query<T> query)
   {
     return getUncoveredIntervalsLimit(query, DEFAULT_UNCOVERED_INTERVALS_LIMIT);
@@ -239,4 +285,19 @@ public class QueryContexts
   private QueryContexts()
   {
   }
+
+  static <T, E extends Enum<E>> E parseEnum(Query<T> query, String key, Class<E> clazz, E defaultValue)
+  {
+    Object val = query.getContextValue(key);
+    if (val == null) {
+      return defaultValue;
+    }
+    if (val instanceof String) {
+      return Enum.valueOf(clazz, StringUtils.toUpperCase((String) val));
+    } else if (val instanceof Boolean) {
+      return Enum.valueOf(clazz, StringUtils.toUpperCase(String.valueOf(val)));
+    } else {
+      throw new ISE("Unknown type [%s]. Cannot parse!", val.getClass());
+    }
+  }
 }
diff --git a/processing/src/main/java/org/apache/druid/query/QueryMetrics.java b/processing/src/main/java/org/apache/druid/query/QueryMetrics.java
index e797663..e34d661 100644
--- a/processing/src/main/java/org/apache/druid/query/QueryMetrics.java
+++ b/processing/src/main/java/org/apache/druid/query/QueryMetrics.java
@@ -237,6 +237,12 @@ public interface QueryMetrics<QueryType extends Query<?>>
   void identity(String identity);
 
   /**
+   * Sets whether or not a segment scan has been vectorized. Generally expected to only be attached to segment-level
+   * metrics, since at whole-query level we might have a mix of vectorized and non-vectorized segment scans.
+   */
+  void vectorized(boolean vectorized);
+
+  /**
    * Creates a {@link BitmapResultFactory} which may record some information along bitmap construction from {@link
    * #preFilters(List)}. The returned BitmapResultFactory may add some dimensions to this QueryMetrics from it's {@link
    * BitmapResultFactory#toImmutableBitmap(Object)} method. See {@link BitmapResultFactory} Javadoc for more
diff --git a/processing/src/main/java/org/apache/druid/query/QueryRunnerHelper.java b/processing/src/main/java/org/apache/druid/query/QueryRunnerHelper.java
index 53f281d..c99b6ae 100644
--- a/processing/src/main/java/org/apache/druid/query/QueryRunnerHelper.java
+++ b/processing/src/main/java/org/apache/druid/query/QueryRunnerHelper.java
@@ -21,7 +21,6 @@ package org.apache.druid.query;
 
 import com.google.common.base.Function;
 import com.google.common.base.Preconditions;
-import com.google.common.base.Predicates;
 import org.apache.druid.java.util.common.granularity.Granularity;
 import org.apache.druid.java.util.common.guava.Sequence;
 import org.apache.druid.java.util.common.guava.Sequences;
@@ -34,19 +33,19 @@ import org.joda.time.Interval;
 import java.io.Closeable;
 import java.util.List;
 import java.util.Map;
+import java.util.Objects;
 
 /**
  */
 public class QueryRunnerHelper
 {
-
   public static <T> Sequence<Result<T>> makeCursorBasedQuery(
       final StorageAdapter adapter,
-      List<Interval> queryIntervals,
-      Filter filter,
-      VirtualColumns virtualColumns,
-      boolean descending,
-      Granularity granularity,
+      final List<Interval> queryIntervals,
+      final Filter filter,
+      final VirtualColumns virtualColumns,
+      final boolean descending,
+      final Granularity granularity,
       final Function<Cursor, Result<T>> mapFn
   )
   {
@@ -57,16 +56,9 @@ public class QueryRunnerHelper
     return Sequences.filter(
         Sequences.map(
             adapter.makeCursors(filter, queryIntervals.get(0), virtualColumns, granularity, descending, null),
-            new Function<Cursor, Result<T>>()
-            {
-              @Override
-              public Result<T> apply(Cursor input)
-              {
-                return mapFn.apply(input);
-              }
-            }
+            mapFn
         ),
-        Predicates.notNull()
+        Objects::nonNull
     );
   }
 
diff --git a/processing/src/main/java/org/apache/druid/query/aggregation/AggregatorAdapters.java b/processing/src/main/java/org/apache/druid/query/aggregation/AggregatorAdapters.java
new file mode 100644
index 0000000..8ae7a33
--- /dev/null
+++ b/processing/src/main/java/org/apache/druid/query/aggregation/AggregatorAdapters.java
@@ -0,0 +1,389 @@
+/*
+ * 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.query.aggregation;
+
+import com.google.common.primitives.Ints;
+import org.apache.druid.java.util.common.ISE;
+import org.apache.druid.java.util.common.logger.Logger;
+import org.apache.druid.segment.ColumnSelectorFactory;
+import org.apache.druid.segment.vector.VectorColumnSelectorFactory;
+
+import javax.annotation.Nullable;
+import java.io.Closeable;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import java.util.List;
+import java.util.stream.Collectors;
+
+/**
+ * A class that helps query engines use Buffer- or VectorAggregators in a consistent way.
+ *
+ * The two main benefits this class provides are:
+ *
+ * (1) Query engines can treat BufferAggregators and VectorAggregators the same for operations that are equivalent
+ * across them, like "init", "get", "relocate", and "close".
+ * (2) Query engines are freed from the need to manage how much space each individual aggregator needs. They only
+ * need to allocate a block of size "spaceNeeded".
+ */
+public class AggregatorAdapters implements Closeable
+{
+  private static final Logger log = new Logger(AggregatorAdapters.class);
+
+  private final List<Adapter> adapters;
+  private final List<AggregatorFactory> factories;
+  private final int[] aggregatorPositions;
+  private final int spaceNeeded;
+
+  private AggregatorAdapters(final List<Adapter> adapters)
+  {
+    this.adapters = adapters;
+    this.factories = adapters.stream().map(Adapter::getFactory).collect(Collectors.toList());
+    this.aggregatorPositions = new int[adapters.size()];
+
+    long nextPosition = 0;
+    for (int i = 0; i < adapters.size(); i++) {
+      final AggregatorFactory aggregatorFactory = adapters.get(i).getFactory();
+      aggregatorPositions[i] = Ints.checkedCast(nextPosition);
+      nextPosition += aggregatorFactory.getMaxIntermediateSizeWithNulls();
+    }
+
+    this.spaceNeeded = Ints.checkedCast(nextPosition);
+  }
+
+  /**
+   * Create an adapters object based on {@link VectorAggregator}.
+   */
+  public static AggregatorAdapters factorizeVector(
+      final VectorColumnSelectorFactory columnSelectorFactory,
+      final List<AggregatorFactory> aggregatorFactories
+  )
+  {
+    final Adapter[] adapters = new Adapter[aggregatorFactories.size()];
+    for (int i = 0; i < aggregatorFactories.size(); i++) {
+      final AggregatorFactory aggregatorFactory = aggregatorFactories.get(i);
+      adapters[i] = new VectorAggregatorAdapter(
+          aggregatorFactory,
+          aggregatorFactory.factorizeVector(columnSelectorFactory)
+      );
+    }
+
+    return new AggregatorAdapters(Arrays.asList(adapters));
+  }
+
+  /**
+   * Create an adapters object based on {@link BufferAggregator}.
+   */
+  public static AggregatorAdapters factorizeBuffered(
+      final ColumnSelectorFactory columnSelectorFactory,
+      final List<AggregatorFactory> aggregatorFactories
+  )
+  {
+    final Adapter[] adapters = new Adapter[aggregatorFactories.size()];
+    for (int i = 0; i < aggregatorFactories.size(); i++) {
+      final AggregatorFactory aggregatorFactory = aggregatorFactories.get(i);
+      adapters[i] = new BufferAggregatorAdapter(
+          aggregatorFactory,
+          aggregatorFactory.factorizeBuffered(columnSelectorFactory)
+      );
+    }
+
+    return new AggregatorAdapters(Arrays.asList(adapters));
+  }
+
+  /**
+   * Return the amount of buffer bytes needed by all aggregators wrapped up in this object.
+   */
+  public int spaceNeeded()
+  {
+    return spaceNeeded;
+  }
+
+  /**
+   * Return the {@link AggregatorFactory} objects that were used to create this object.
+   */
+  public List<AggregatorFactory> factories()
+  {
+    return factories;
+  }
+
+  /**
+   * Return the individual positions of each aggregator within a hypothetical buffer of size {@link #spaceNeeded()}.
+   */
+  public int[] aggregatorPositions()
+  {
+    return aggregatorPositions;
+  }
+
+  /**
+   * Return the number of aggregators in this object.
+   */
+  public int size()
+  {
+    return adapters.size();
+  }
+
+  /**
+   * Initialize all aggregators.
+   *
+   * @param buf      aggregation buffer
+   * @param position position in buffer where our block of size {@link #spaceNeeded()} starts
+   */
+  public void init(final ByteBuffer buf, final int position)
+  {
+    for (int i = 0; i < adapters.size(); i++) {
+      adapters.get(i).init(buf, position + aggregatorPositions[i]);
+    }
+  }
+
+  /**
+   * Call {@link BufferAggregator#aggregate(ByteBuffer, int)} on all of our aggregators.
+   *
+   * This method is only valid if the underlying aggregators are {@link BufferAggregator}.
+   */
+  public void aggregateBuffered(final ByteBuffer buf, final int position)
+  {
+    for (int i = 0; i < adapters.size(); i++) {
+      final Adapter adapter = adapters.get(i);
+      adapter.asBufferAggregator().aggregate(buf, position + aggregatorPositions[i]);
+    }
+  }
+
+  /**
+   * Call {@link VectorAggregator#aggregate(ByteBuffer, int, int, int)} on all of our aggregators.
+   *
+   * This method is only valid if the underlying aggregators are {@link VectorAggregator}.
+   */
+  public void aggregateVector(
+      final ByteBuffer buf,
+      final int position,
+      final int start,
+      final int end
+  )
+  {
+    for (int i = 0; i < adapters.size(); i++) {
+      final Adapter adapter = adapters.get(i);
+      adapter.asVectorAggregator().aggregate(buf, position + aggregatorPositions[i], start, end);
+    }
+  }
+
+  /**
+   * Call {@link VectorAggregator#aggregate(ByteBuffer, int, int[], int[], int)} on all of our aggregators.
+   *
+   * This method is only valid if the underlying aggregators are {@link VectorAggregator}.
+   */
+  public void aggregateVector(
+      final ByteBuffer buf,
+      final int numRows,
+      final int[] positions,
+      @Nullable final int[] rows
+  )
+  {
+    for (int i = 0; i < adapters.size(); i++) {
+      final Adapter adapter = adapters.get(i);
+      adapter.asVectorAggregator().aggregate(buf, numRows, positions, rows, aggregatorPositions[i]);
+    }
+  }
+
+  /**
+   * Retrieve aggregation state from one of our aggregators.
+   *
+   * @param buf              aggregation buffer
+   * @param position         position in buffer where our block of size {@link #spaceNeeded()} starts
+   * @param aggregatorNumber which aggregator to retrieve state, from 0 to {@link #size()} - 1
+   */
+  @Nullable
+  public Object get(final ByteBuffer buf, final int position, final int aggregatorNumber)
+  {
+    return adapters.get(aggregatorNumber).get(buf, position + aggregatorPositions[aggregatorNumber]);
+  }
+
+  /**
+   * Inform all of our aggregators that they are being relocated.
+   */
+  public void relocate(int oldPosition, int newPosition, ByteBuffer oldBuffer, ByteBuffer newBuffer)
+  {
+    for (int i = 0; i < adapters.size(); i++) {
+      adapters.get(i).relocate(
+          oldPosition + aggregatorPositions[i],
+          newPosition + aggregatorPositions[i],
+          oldBuffer,
+          newBuffer
+      );
+    }
+  }
+
+  /**
+   * Close all of our aggregators.
+   */
+  @Override
+  public void close()
+  {
+    for (Adapter adapter : adapters) {
+      try {
+        adapter.close();
+      }
+      catch (Exception e) {
+        log.warn(e, "Could not close aggregator [%s], skipping.", adapter.getFactory().getName());
+      }
+    }
+  }
+
+  /**
+   * The interface that allows this class to achieve its goals of partially unifying handling of
+   * BufferAggregator and VectorAggregator. Private, since it doesn't escape this class and the
+   * only two implementations are private static classes below.
+   */
+  private interface Adapter extends Closeable
+  {
+    void init(ByteBuffer buf, int position);
+
+    @Nullable
+    Object get(ByteBuffer buf, int position);
+
+    void relocate(int oldPosition, int newPosition, ByteBuffer oldBuffer, ByteBuffer newBuffer);
+
+    @Override
+    void close();
+
+    AggregatorFactory getFactory();
+
+    BufferAggregator asBufferAggregator();
+
+    VectorAggregator asVectorAggregator();
+  }
+
+  private static class VectorAggregatorAdapter implements Adapter
+  {
+    private final AggregatorFactory factory;
+    private final VectorAggregator aggregator;
+
+    VectorAggregatorAdapter(final AggregatorFactory factory, final VectorAggregator aggregator)
+    {
+      this.factory = factory;
+      this.aggregator = aggregator;
+    }
+
+    @Override
+    public void init(final ByteBuffer buf, final int position)
+    {
+      aggregator.init(buf, position);
+    }
+
+    @Override
+    public Object get(final ByteBuffer buf, final int position)
+    {
+      return aggregator.get(buf, position);
+    }
+
+    @Override
+    public void close()
+    {
+      aggregator.close();
+    }
+
+    @Override
+    public void relocate(
+        final int oldPosition,
+        final int newPosition,
+        final ByteBuffer oldBuffer,
+        final ByteBuffer newBuffer
+    )
+    {
+      aggregator.relocate(oldPosition, newPosition, oldBuffer, newBuffer);
+    }
+
+    @Override
+    public AggregatorFactory getFactory()
+    {
+      return factory;
+    }
+
+    @Override
+    public BufferAggregator asBufferAggregator()
+    {
+      throw new ISE("Not a BufferAggregator!");
+    }
+
+    @Override
+    public VectorAggregator asVectorAggregator()
+    {
+      return aggregator;
+    }
+  }
+
+  private static class BufferAggregatorAdapter implements Adapter
+  {
+    private final AggregatorFactory factory;
+    private final BufferAggregator aggregator;
+
+    BufferAggregatorAdapter(final AggregatorFactory factory, final BufferAggregator aggregator)
+    {
+      this.factory = factory;
+      this.aggregator = aggregator;
+    }
+
+    @Override
+    public void init(final ByteBuffer buf, final int position)
+    {
+      aggregator.init(buf, position);
+    }
+
+    @Override
+    public Object get(final ByteBuffer buf, final int position)
+    {
+      return aggregator.get(buf, position);
+    }
+
+    @Override
+    public void close()
+    {
+      aggregator.close();
+    }
+
+    @Override
+    public void relocate(
+        final int oldPosition,
+        final int newPosition,
+        final ByteBuffer oldBuffer,
+        final ByteBuffer newBuffer
+    )
+    {
+      aggregator.relocate(oldPosition, newPosition, oldBuffer, newBuffer);
+    }
+
+    @Override
+    public AggregatorFactory getFactory()
+    {
+      return factory;
+    }
+
+    @Override
+    public BufferAggregator asBufferAggregator()
+    {
+      return aggregator;
+    }
+
+    @Override
+    public VectorAggregator asVectorAggregator()
+    {
+      throw new ISE("Not a VectorAggregator!");
+    }
+  }
+}
diff --git a/processing/src/main/java/org/apache/druid/query/aggregation/AggregatorFactory.java b/processing/src/main/java/org/apache/druid/query/aggregation/AggregatorFactory.java
index 6b0f4a1..ced087b 100644
--- a/processing/src/main/java/org/apache/druid/query/aggregation/AggregatorFactory.java
+++ b/processing/src/main/java/org/apache/druid/query/aggregation/AggregatorFactory.java
@@ -25,6 +25,7 @@ import org.apache.druid.java.util.common.UOE;
 import org.apache.druid.java.util.common.logger.Logger;
 import org.apache.druid.query.PerSegmentQueryOptimizationContext;
 import org.apache.druid.segment.ColumnSelectorFactory;
+import org.apache.druid.segment.vector.VectorColumnSelectorFactory;
 
 import javax.annotation.Nullable;
 import java.util.Arrays;
@@ -49,6 +50,23 @@ public abstract class AggregatorFactory implements Cacheable
 
   public abstract BufferAggregator factorizeBuffered(ColumnSelectorFactory metricFactory);
 
+  /**
+   * Create a VectorAggregator based on the provided column selector factory. Will throw an exception if
+   * this aggregation class does not support vectorization: check "canVectorize" first.
+   */
+  public VectorAggregator factorizeVector(VectorColumnSelectorFactory selectorFactory)
+  {
+    throw new UOE("Aggregator[%s] cannot vectorize", getClass().getName());
+  }
+
+  /**
+   * Returns whether or not this aggregation class supports vectorization. The default implementation returns false.
+   */
+  public boolean canVectorize()
+  {
+    return false;
+  }
+
   public abstract Comparator getComparator();
 
   /**
diff --git a/processing/src/main/java/org/apache/druid/query/aggregation/BufferAggregator.java b/processing/src/main/java/org/apache/druid/query/aggregation/BufferAggregator.java
index ed77c91..9860854 100644
--- a/processing/src/main/java/org/apache/druid/query/aggregation/BufferAggregator.java
+++ b/processing/src/main/java/org/apache/druid/query/aggregation/BufferAggregator.java
@@ -24,6 +24,7 @@ import org.apache.druid.query.monomorphicprocessing.CalledFromHotLoop;
 import org.apache.druid.query.monomorphicprocessing.HotLoopCallee;
 import org.apache.druid.query.monomorphicprocessing.RuntimeShapeInspector;
 
+import javax.annotation.Nullable;
 import java.nio.ByteBuffer;
 
 /**
@@ -33,6 +34,8 @@ import java.nio.ByteBuffer;
  *
  * Thus, an Aggregator can be thought of as a closure over some other thing that is stateful and changes between calls
  * to aggregate(...).
+ *
+ * @see VectorAggregator, the vectorized version
  */
 @ExtensionPoint
 public interface BufferAggregator extends HotLoopCallee
@@ -87,6 +90,7 @@ public interface BufferAggregator extends HotLoopCallee
    * @param position offset within the byte buffer at which the aggregate value is stored
    * @return the Object representation of the aggregate
    */
+  @Nullable
   Object get(ByteBuffer buf, int position);
 
   /**
diff --git a/processing/src/main/java/org/apache/druid/query/aggregation/CountAggregatorFactory.java b/processing/src/main/java/org/apache/druid/query/aggregation/CountAggregatorFactory.java
index f4329fe..599a2c4 100644
--- a/processing/src/main/java/org/apache/druid/query/aggregation/CountAggregatorFactory.java
+++ b/processing/src/main/java/org/apache/druid/query/aggregation/CountAggregatorFactory.java
@@ -24,6 +24,7 @@ import com.fasterxml.jackson.annotation.JsonProperty;
 import com.google.common.base.Preconditions;
 import com.google.common.collect.ImmutableList;
 import org.apache.druid.segment.ColumnSelectorFactory;
+import org.apache.druid.segment.vector.VectorColumnSelectorFactory;
 
 import javax.annotation.Nullable;
 import java.util.Collections;
@@ -59,12 +60,24 @@ public class CountAggregatorFactory extends AggregatorFactory
   }
 
   @Override
+  public VectorAggregator factorizeVector(final VectorColumnSelectorFactory selectorFactory)
+  {
+    return new CountVectorAggregator();
+  }
+
+  @Override
   public Comparator getComparator()
   {
     return CountAggregator.COMPARATOR;
   }
 
   @Override
+  public boolean canVectorize()
+  {
+    return true;
+  }
+
+  @Override
   public Object combine(Object lhs, Object rhs)
   {
     return CountAggregator.combineValues(lhs, rhs);
diff --git a/processing/src/main/java/org/apache/druid/segment/QueryableIndexSegment.java b/processing/src/main/java/org/apache/druid/query/aggregation/CountVectorAggregator.java
similarity index 50%
copy from processing/src/main/java/org/apache/druid/segment/QueryableIndexSegment.java
copy to processing/src/main/java/org/apache/druid/query/aggregation/CountVectorAggregator.java
index 8f68178..fec4793 100644
--- a/processing/src/main/java/org/apache/druid/segment/QueryableIndexSegment.java
+++ b/processing/src/main/java/org/apache/druid/query/aggregation/CountVectorAggregator.java
@@ -17,52 +17,50 @@
  * under the License.
  */
 
-package org.apache.druid.segment;
+package org.apache.druid.query.aggregation;
 
-import org.apache.druid.timeline.SegmentId;
-import org.joda.time.Interval;
+import javax.annotation.Nullable;
+import java.nio.ByteBuffer;
 
-/**
- */
-public class QueryableIndexSegment extends AbstractSegment
+public class CountVectorAggregator implements VectorAggregator
 {
-  private final QueryableIndex index;
-  private final SegmentId segmentId;
-
-  public QueryableIndexSegment(QueryableIndex index, final SegmentId segmentId)
-  {
-    this.index = index;
-    this.segmentId = segmentId;
-  }
-
   @Override
-  public SegmentId getId()
+  public void init(final ByteBuffer buf, final int position)
   {
-    return segmentId;
+    buf.putLong(position, 0);
   }
 
   @Override
-  public Interval getDataInterval()
+  public void aggregate(final ByteBuffer buf, final int position, final int startRow, final int endRow)
   {
-    return index.getDataInterval();
+    final int delta = endRow - startRow;
+    buf.putLong(position, buf.getLong(position) + delta);
   }
 
   @Override
-  public QueryableIndex asQueryableIndex()
+  public void aggregate(
+      final ByteBuffer buf,
+      final int numRows,
+      final int[] positions,
+      @Nullable final int[] rows,
+      final int positionOffset
+  )
   {
-    return index;
+    for (int i = 0; i < numRows; i++) {
+      final int position = positions[i] + positionOffset;
+      buf.putLong(position, buf.getLong(position) + 1);
+    }
   }
 
   @Override
-  public StorageAdapter asStorageAdapter()
+  public Object get(final ByteBuffer buf, final int position)
   {
-    return new QueryableIndexStorageAdapter(index);
+    return buf.getLong(position);
   }
 
   @Override
   public void close()
   {
-    // this is kinda nasty
-    index.close();
+    // Nothing to close.
   }
 }
diff --git a/processing/src/main/java/org/apache/druid/query/aggregation/DoubleSumAggregatorFactory.java b/processing/src/main/java/org/apache/druid/query/aggregation/DoubleSumAggregatorFactory.java
index e5c8bf6..0292823 100644
--- a/processing/src/main/java/org/apache/druid/query/aggregation/DoubleSumAggregatorFactory.java
+++ b/processing/src/main/java/org/apache/druid/query/aggregation/DoubleSumAggregatorFactory.java
@@ -26,6 +26,8 @@ import org.apache.druid.java.util.common.StringUtils;
 import org.apache.druid.math.expr.ExprMacroTable;
 import org.apache.druid.segment.BaseDoubleColumnValueSelector;
 import org.apache.druid.segment.ColumnSelectorFactory;
+import org.apache.druid.segment.vector.VectorColumnSelectorFactory;
+import org.apache.druid.segment.vector.VectorValueSelector;
 
 import javax.annotation.Nullable;
 import java.nio.ByteBuffer;
@@ -62,12 +64,24 @@ public class DoubleSumAggregatorFactory extends SimpleDoubleAggregatorFactory
   }
 
   @Override
+  protected VectorValueSelector vectorSelector(VectorColumnSelectorFactory columnSelectorFactory)
+  {
+    return columnSelectorFactory.makeValueSelector(fieldName);
+  }
+
+  @Override
   protected Aggregator factorize(ColumnSelectorFactory metricFactory, BaseDoubleColumnValueSelector selector)
   {
     return new DoubleSumAggregator(selector);
   }
 
   @Override
+  public boolean canVectorize()
+  {
+    return expression == null;
+  }
+
+  @Override
   protected BufferAggregator factorizeBuffered(
       ColumnSelectorFactory metricFactory,
       BaseDoubleColumnValueSelector selector
@@ -77,6 +91,15 @@ public class DoubleSumAggregatorFactory extends SimpleDoubleAggregatorFactory
   }
 
   @Override
+  protected VectorAggregator factorizeVector(
+      VectorColumnSelectorFactory columnSelectorFactory,
+      VectorValueSelector selector
+  )
+  {
+    return new DoubleSumVectorAggregator(selector);
+  }
+
+  @Override
   @Nullable
   public Object combine(@Nullable Object lhs, @Nullable Object rhs)
   {
diff --git a/processing/src/main/java/org/apache/druid/query/aggregation/DoubleSumVectorAggregator.java b/processing/src/main/java/org/apache/druid/query/aggregation/DoubleSumVectorAggregator.java
new file mode 100644
index 0000000..f66a4f4
--- /dev/null
+++ b/processing/src/main/java/org/apache/druid/query/aggregation/DoubleSumVectorAggregator.java
@@ -0,0 +1,83 @@
+/*
+ * 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.query.aggregation;
+
+import org.apache.druid.segment.vector.VectorValueSelector;
+
+import javax.annotation.Nullable;
+import java.nio.ByteBuffer;
+
+public class DoubleSumVectorAggregator implements VectorAggregator
+{
+  private final VectorValueSelector selector;
+
+  public DoubleSumVectorAggregator(final VectorValueSelector selector)
+  {
+    this.selector = selector;
+  }
+
+  @Override
+  public void init(final ByteBuffer buf, final int position)
+  {
+    buf.putDouble(position, 0);
+  }
+
+  @Override
+  public void aggregate(final ByteBuffer buf, final int position, final int startRow, final int endRow)
+  {
+    final double[] vector = selector.getDoubleVector();
+
+    double sum = 0;
+    for (int i = startRow; i < endRow; i++) {
+      sum += vector[i];
+    }
+
+    buf.putDouble(position, buf.getDouble(position) + sum);
+  }
+
+  @Override
+  public void aggregate(
+      final ByteBuffer buf,
+      final int numRows,
+      final int[] positions,
+      @Nullable final int[] rows,
+      final int positionOffset
+  )
+  {
+    final double[] vector = selector.getDoubleVector();
+
+    for (int i = 0; i < numRows; i++) {
+      final int position = positions[i] + positionOffset;
+      buf.putDouble(position, buf.getDouble(position) + vector[rows != null ? rows[i] : i]);
+    }
+  }
+
+  @Override
+  public Object get(final ByteBuffer buf, final int position)
+  {
+    return buf.getDouble(position);
+  }
+
+  @Override
+  public void close()
+  {
+    // Nothing to close.
+  }
+}
diff --git a/processing/src/main/java/org/apache/druid/query/aggregation/FilteredAggregatorFactory.java b/processing/src/main/java/org/apache/druid/query/aggregation/FilteredAggregatorFactory.java
index 52f072b..47d1755 100644
--- a/processing/src/main/java/org/apache/druid/query/aggregation/FilteredAggregatorFactory.java
+++ b/processing/src/main/java/org/apache/druid/query/aggregation/FilteredAggregatorFactory.java
@@ -25,11 +25,13 @@ import com.google.common.base.Preconditions;
 import com.google.common.base.Strings;
 import org.apache.druid.query.PerSegmentQueryOptimizationContext;
 import org.apache.druid.query.filter.DimFilter;
+import org.apache.druid.query.filter.Filter;
 import org.apache.druid.query.filter.IntervalDimFilter;
 import org.apache.druid.query.filter.ValueMatcher;
+import org.apache.druid.query.filter.vector.VectorValueMatcher;
 import org.apache.druid.segment.ColumnSelectorFactory;
 import org.apache.druid.segment.column.ColumnHolder;
-import org.apache.druid.segment.filter.Filters;
+import org.apache.druid.segment.vector.VectorColumnSelectorFactory;
 import org.joda.time.Interval;
 
 import javax.annotation.Nullable;
@@ -42,7 +44,10 @@ import java.util.Objects;
 public class FilteredAggregatorFactory extends AggregatorFactory
 {
   private final AggregatorFactory delegate;
-  private final DimFilter filter;
+  private final DimFilter dimFilter;
+  private final Filter filter;
+
+  @Nullable
   private final String name;
 
   // Constructor for backwards compat only
@@ -57,22 +62,23 @@ public class FilteredAggregatorFactory extends AggregatorFactory
   @JsonCreator
   public FilteredAggregatorFactory(
       @JsonProperty("aggregator") AggregatorFactory delegate,
-      @JsonProperty("filter") DimFilter filter,
-      @JsonProperty("name") String name
+      @JsonProperty("filter") DimFilter dimFilter,
+      @Nullable @JsonProperty("name") String name
   )
   {
-    Preconditions.checkNotNull(delegate);
-    Preconditions.checkNotNull(filter);
+    Preconditions.checkNotNull(delegate, "aggregator");
+    Preconditions.checkNotNull(dimFilter, "filter");
 
     this.delegate = delegate;
-    this.filter = filter;
+    this.dimFilter = dimFilter;
+    this.filter = dimFilter.toFilter();
     this.name = name;
   }
 
   @Override
   public Aggregator factorize(ColumnSelectorFactory columnSelectorFactory)
   {
-    final ValueMatcher valueMatcher = Filters.toFilter(filter).makeMatcher(columnSelectorFactory);
+    final ValueMatcher valueMatcher = filter.makeMatcher(columnSelectorFactory);
     return new FilteredAggregator(
         valueMatcher,
         delegate.factorize(columnSelectorFactory)
@@ -82,7 +88,7 @@ public class FilteredAggregatorFactory extends AggregatorFactory
   @Override
   public BufferAggregator factorizeBuffered(ColumnSelectorFactory columnSelectorFactory)
   {
-    final ValueMatcher valueMatcher = Filters.toFilter(filter).makeMatcher(columnSelectorFactory);
+    final ValueMatcher valueMatcher = filter.makeMatcher(columnSelectorFactory);
     return new FilteredBufferAggregator(
         valueMatcher,
         delegate.factorizeBuffered(columnSelectorFactory)
@@ -90,6 +96,23 @@ public class FilteredAggregatorFactory extends AggregatorFactory
   }
 
   @Override
+  public VectorAggregator factorizeVector(VectorColumnSelectorFactory columnSelectorFactory)
+  {
+    Preconditions.checkState(canVectorize(), "Cannot vectorize");
+    final VectorValueMatcher valueMatcher = filter.makeVectorMatcher(columnSelectorFactory);
+    return new FilteredVectorAggregator(
+        valueMatcher,
+        delegate.factorizeVector(columnSelectorFactory)
+    );
+  }
+
+  @Override
+  public boolean canVectorize()
+  {
+    return delegate.canVectorize() && filter.canVectorizeMatcher();
+  }
+
+  @Override
   public Comparator getComparator()
   {
     return delegate.getComparator();
@@ -147,7 +170,7 @@ public class FilteredAggregatorFactory extends AggregatorFactory
   @Override
   public byte[] getCacheKey()
   {
-    byte[] filterCacheKey = filter.getCacheKey();
+    byte[] filterCacheKey = dimFilter.getCacheKey();
     byte[] aggregatorCacheKey = delegate.getCacheKey();
     return ByteBuffer.allocate(1 + filterCacheKey.length + aggregatorCacheKey.length)
                      .put(AggregatorUtil.FILTERED_AGG_CACHE_TYPE_ID)
@@ -171,8 +194,8 @@ public class FilteredAggregatorFactory extends AggregatorFactory
   @Override
   public AggregatorFactory optimizeForSegment(PerSegmentQueryOptimizationContext optimizationContext)
   {
-    if (filter instanceof IntervalDimFilter) {
-      IntervalDimFilter intervalDimFilter = ((IntervalDimFilter) filter);
+    if (dimFilter instanceof IntervalDimFilter) {
+      IntervalDimFilter intervalDimFilter = ((IntervalDimFilter) dimFilter);
       if (intervalDimFilter.getExtractionFn() != null) {
         // no support for extraction functions right now
         return this;
@@ -238,7 +261,7 @@ public class FilteredAggregatorFactory extends AggregatorFactory
   @JsonProperty
   public DimFilter getFilter()
   {
-    return filter;
+    return dimFilter;
   }
 
   @Override
@@ -248,7 +271,7 @@ public class FilteredAggregatorFactory extends AggregatorFactory
   }
 
   @Override
-  public boolean equals(Object o)
+  public boolean equals(final Object o)
   {
     if (this == o) {
       return true;
@@ -256,16 +279,17 @@ public class FilteredAggregatorFactory extends AggregatorFactory
     if (o == null || getClass() != o.getClass()) {
       return false;
     }
-    FilteredAggregatorFactory that = (FilteredAggregatorFactory) o;
+    final FilteredAggregatorFactory that = (FilteredAggregatorFactory) o;
     return Objects.equals(delegate, that.delegate) &&
-           Objects.equals(filter, that.filter) &&
+           Objects.equals(dimFilter, that.dimFilter) &&
            Objects.equals(name, that.name);
   }
 
   @Override
   public int hashCode()
   {
-    return Objects.hash(delegate, filter, name);
+
+    return Objects.hash(delegate, dimFilter, name);
   }
 
   @Override
@@ -273,7 +297,7 @@ public class FilteredAggregatorFactory extends AggregatorFactory
   {
     return "FilteredAggregatorFactory{" +
            "delegate=" + delegate +
-           ", filter=" + filter +
+           ", dimFilter=" + dimFilter +
            ", name='" + name + '\'' +
            '}';
   }
diff --git a/processing/src/main/java/org/apache/druid/query/aggregation/FilteredVectorAggregator.java b/processing/src/main/java/org/apache/druid/query/aggregation/FilteredVectorAggregator.java
new file mode 100644
index 0000000..40c0490
--- /dev/null
+++ b/processing/src/main/java/org/apache/druid/query/aggregation/FilteredVectorAggregator.java
@@ -0,0 +1,153 @@
+/*
+ * 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.query.aggregation;
+
+import org.apache.druid.java.util.common.ISE;
+import org.apache.druid.query.filter.vector.ReadableVectorMatch;
+import org.apache.druid.query.filter.vector.VectorMatch;
+import org.apache.druid.query.filter.vector.VectorValueMatcher;
+
+import javax.annotation.Nullable;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+
+public class FilteredVectorAggregator implements VectorAggregator
+{
+  private final VectorValueMatcher matcher;
+  private final VectorAggregator delegate;
+  private final int[] delegatePositions;
+
+  @Nullable
+  private VectorMatch maskScratch = null;
+
+  public FilteredVectorAggregator(
+      final VectorValueMatcher matcher,
+      final VectorAggregator delegate
+  )
+  {
+    this.matcher = matcher;
+    this.delegate = delegate;
+    this.delegatePositions = new int[matcher.getMaxVectorSize()];
+  }
+
+  @Override
+  public void init(final ByteBuffer buf, final int position)
+  {
+    delegate.init(buf, position);
+  }
+
+  @Override
+  public void aggregate(final ByteBuffer buf, final int position, final int startRow, final int endRow)
+  {
+    final ReadableVectorMatch mask;
+
+    if (startRow == 0) {
+      mask = VectorMatch.allTrue(endRow);
+    } else {
+      if (maskScratch == null) {
+        maskScratch = VectorMatch.wrap(new int[matcher.getMaxVectorSize()]);
+      }
+
+      final int maskSize = endRow - startRow;
+      final int[] maskArray = maskScratch.getSelection();
+      for (int i = 0; i < maskSize; i++) {
+        maskArray[i] = startRow + i;
+      }
+
+      maskScratch.setSelectionSize(maskSize);
+      mask = maskScratch;
+    }
+
+    final ReadableVectorMatch match = matcher.match(mask);
+
+    if (match.isAllTrue(matcher.getCurrentVectorSize())) {
+      delegate.aggregate(buf, position, startRow, endRow);
+    } else if (!match.isAllFalse()) {
+      Arrays.fill(delegatePositions, 0, match.getSelectionSize(), position);
+      delegate.aggregate(buf, match.getSelectionSize(), delegatePositions, match.getSelection(), 0);
+    }
+  }
+
+  @Override
+  public void aggregate(
+      final ByteBuffer buf,
+      final int numRows,
+      final int[] positions,
+      @Nullable final int[] rows,
+      final int positionOffset
+  )
+  {
+    final ReadableVectorMatch match0;
+
+    if (rows == null) {
+      match0 = VectorMatch.allTrue(numRows);
+    } else {
+      match0 = VectorMatch.wrap(rows).setSelectionSize(numRows);
+    }
+
+    final ReadableVectorMatch match = matcher.match(match0);
+    final int[] selection = match.getSelection();
+
+    if (rows == null) {
+      for (int i = 0; i < match.getSelectionSize(); i++) {
+        delegatePositions[i] = positions[selection[i]];
+      }
+    } else {
+      // i iterates over the match; j iterates over the "rows" array
+      for (int i = 0, j = 0; i < match.getSelectionSize(); i++) {
+        for (; rows[j] < selection[i]; j++) {
+          // Do nothing; the for loop is doing the work of incrementing j.
+        }
+
+        if (rows[j] != selection[i]) {
+          throw new ISE("Selection contained phantom row[%d]", selection[i]);
+        }
+
+        delegatePositions[i] = positions[j];
+      }
+    }
+
+    delegate.aggregate(buf, match.getSelectionSize(), delegatePositions, selection, positionOffset);
+  }
+
+  @Override
+  public Object get(final ByteBuffer buf, final int position)
+  {
+    return delegate.get(buf, position);
+  }
+
+  @Override
+  public void close()
+  {
+    delegate.close();
+    maskScratch = null;
+  }
+
+  @Override
+  public void relocate(
+      final int oldPosition,
+      final int newPosition,
+      final ByteBuffer oldBuffer,
+      final ByteBuffer newBuffer
+  )
+  {
+    delegate.relocate(oldPosition, newPosition, oldBuffer, newBuffer);
+  }
+}
diff --git a/processing/src/main/java/org/apache/druid/query/aggregation/FloatSumAggregatorFactory.java b/processing/src/main/java/org/apache/druid/query/aggregation/FloatSumAggregatorFactory.java
index 17a9f9d..debe44d 100644
--- a/processing/src/main/java/org/apache/druid/query/aggregation/FloatSumAggregatorFactory.java
+++ b/processing/src/main/java/org/apache/druid/query/aggregation/FloatSumAggregatorFactory.java
@@ -26,6 +26,8 @@ import org.apache.druid.java.util.common.StringUtils;
 import org.apache.druid.math.expr.ExprMacroTable;
 import org.apache.druid.segment.BaseFloatColumnValueSelector;
 import org.apache.druid.segment.ColumnSelectorFactory;
+import org.apache.druid.segment.vector.VectorColumnSelectorFactory;
+import org.apache.druid.segment.vector.VectorValueSelector;
 
 import javax.annotation.Nullable;
 import java.nio.ByteBuffer;
@@ -62,12 +64,24 @@ public class FloatSumAggregatorFactory extends SimpleFloatAggregatorFactory
   }
 
   @Override
+  protected VectorValueSelector vectorSelector(VectorColumnSelectorFactory columnSelectorFactory)
+  {
+    return columnSelectorFactory.makeValueSelector(fieldName);
+  }
+
+  @Override
   protected Aggregator factorize(ColumnSelectorFactory metricFactory, BaseFloatColumnValueSelector selector)
   {
     return new FloatSumAggregator(selector);
   }
 
   @Override
+  public boolean canVectorize()
+  {
+    return expression == null;
+  }
+
+  @Override
   protected BufferAggregator factorizeBuffered(
       ColumnSelectorFactory metricFactory,
       BaseFloatColumnValueSelector selector
@@ -77,6 +91,15 @@ public class FloatSumAggregatorFactory extends SimpleFloatAggregatorFactory
   }
 
   @Override
+  protected VectorAggregator factorizeVector(
+      VectorColumnSelectorFactory columnSelectorFactory,
+      VectorValueSelector selector
+  )
+  {
+    return new FloatSumVectorAggregator(selector);
+  }
+
+  @Override
   @Nullable
   public Object combine(@Nullable Object lhs, @Nullable Object rhs)
   {
diff --git a/processing/src/main/java/org/apache/druid/query/aggregation/FloatSumVectorAggregator.java b/processing/src/main/java/org/apache/druid/query/aggregation/FloatSumVectorAggregator.java
new file mode 100644
index 0000000..d5bd54e
--- /dev/null
+++ b/processing/src/main/java/org/apache/druid/query/aggregation/FloatSumVectorAggregator.java
@@ -0,0 +1,84 @@
+/*
+ * 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.query.aggregation;
+
+import org.apache.druid.segment.vector.VectorValueSelector;
+
+import javax.annotation.Nullable;
+import java.nio.ByteBuffer;
+
+public class FloatSumVectorAggregator implements VectorAggregator
+{
+  private final VectorValueSelector selector;
+
+  public FloatSumVectorAggregator(final VectorValueSelector selector)
+  {
+    this.selector = selector;
+  }
+
+  @Override
+  public void init(final ByteBuffer buf, final int position)
+  {
+    buf.putFloat(position, 0);
+  }
+
+  @Override
+  public void aggregate(final ByteBuffer buf, final int position, final int startRow, final int endRow)
+  {
+    final float[] vector = selector.getFloatVector();
+
+    float sum = 0;
+    for (int i = startRow; i < endRow; i++) {
+      sum += vector[i];
+    }
+
+    buf.putFloat(position, buf.getFloat(position) + sum);
+  }
+
+
+  @Override
+  public void aggregate(
+      final ByteBuffer buf,
+      final int numPositions,
+      final int[] positions,
+      @Nullable final int[] rows,
+      final int positionOffset
+  )
+  {
+    final float[] vector = selector.getFloatVector();
+
+    for (int i = 0; i < numPositions; i++) {
+      final int position = positions[i] + positionOffset;
+      buf.putFloat(position, buf.getFloat(position) + vector[rows != null ? rows[i] : i]);
+    }
+  }
+
+  @Override
+  public Object get(final ByteBuffer buf, final int position)
+  {
+    return buf.getFloat(position);
+  }
+
+  @Override
+  public void close()
+  {
+    // Nothing to close.
+  }
+}
diff --git a/processing/src/main/java/org/apache/druid/query/aggregation/LongSumAggregatorFactory.java b/processing/src/main/java/org/apache/druid/query/aggregation/LongSumAggregatorFactory.java
index 0fc2c69..accfc78 100644
--- a/processing/src/main/java/org/apache/druid/query/aggregation/LongSumAggregatorFactory.java
+++ b/processing/src/main/java/org/apache/druid/query/aggregation/LongSumAggregatorFactory.java
@@ -26,6 +26,8 @@ import org.apache.druid.java.util.common.StringUtils;
 import org.apache.druid.math.expr.ExprMacroTable;
 import org.apache.druid.segment.BaseLongColumnValueSelector;
 import org.apache.druid.segment.ColumnSelectorFactory;
+import org.apache.druid.segment.vector.VectorColumnSelectorFactory;
+import org.apache.druid.segment.vector.VectorValueSelector;
 
 import javax.annotation.Nullable;
 import java.nio.ByteBuffer;
@@ -62,9 +64,9 @@ public class LongSumAggregatorFactory extends SimpleLongAggregatorFactory
   }
 
   @Override
-  protected Aggregator factorize(ColumnSelectorFactory metricFactory, BaseLongColumnValueSelector selector)
+  protected VectorValueSelector vectorSelector(VectorColumnSelectorFactory columnSelectorFactory)
   {
-    return new LongSumAggregator(selector);
+    return columnSelectorFactory.makeValueSelector(fieldName);
   }
 
   @Override
@@ -77,6 +79,27 @@ public class LongSumAggregatorFactory extends SimpleLongAggregatorFactory
   }
 
   @Override
+  protected VectorAggregator factorizeVector(
+      VectorColumnSelectorFactory columnSelectorFactory,
+      VectorValueSelector selector
+  )
+  {
+    return new LongSumVectorAggregator(selector);
+  }
+
+  @Override
+  protected Aggregator factorize(ColumnSelectorFactory metricFactory, BaseLongColumnValueSelector selector)
+  {
+    return new LongSumAggregator(selector);
+  }
+
+  @Override
+  public boolean canVectorize()
+  {
+    return expression == null;
+  }
+
+  @Override
   @Nullable
   public Object combine(@Nullable Object lhs, @Nullable Object rhs)
   {
diff --git a/processing/src/main/java/org/apache/druid/query/aggregation/LongSumVectorAggregator.java b/processing/src/main/java/org/apache/druid/query/aggregation/LongSumVectorAggregator.java
new file mode 100644
index 0000000..24496c7
--- /dev/null
+++ b/processing/src/main/java/org/apache/druid/query/aggregation/LongSumVectorAggregator.java
@@ -0,0 +1,83 @@
+/*
+ * 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.query.aggregation;
+
+import org.apache.druid.segment.vector.VectorValueSelector;
+
+import javax.annotation.Nullable;
+import java.nio.ByteBuffer;
+
+public class LongSumVectorAggregator implements VectorAggregator
+{
+  private final VectorValueSelector selector;
+
+  public LongSumVectorAggregator(final VectorValueSelector selector)
+  {
+    this.selector = selector;
+  }
+
+  @Override
+  public void init(final ByteBuffer buf, final int position)
+  {
+    buf.putLong(position, 0);
+  }
+
+  @Override
+  public void aggregate(final ByteBuffer buf, final int position, final int startRow, final int endRow)
+  {
+    final long[] vector = selector.getLongVector();
+
+    long sum = 0;
+    for (int i = startRow; i < endRow; i++) {
+      sum += vector[i];
+    }
+
+    buf.putLong(position, buf.getLong(position) + sum);
+  }
+
+  @Override
+  public void aggregate(
+      final ByteBuffer buf,
+      final int numRows,
+      final int[] positions,
+      @Nullable final int[] rows,
+      final int positionOffset
+  )
+  {
+    final long[] vector = selector.getLongVector();
+
+    for (int i = 0; i < numRows; i++) {
+      final int position = positions[i] + positionOffset;
+      buf.putLong(position, buf.getLong(position) + vector[rows != null ? rows[i] : i]);
+    }
+  }
+
+  @Override
+  public Object get(final ByteBuffer buf, final int position)
+  {
+    return buf.getLong(position);
+  }
+
+  @Override
+  public void close()
+  {
+    // Nothing to close.
+  }
+}
diff --git a/processing/src/main/java/org/apache/druid/segment/QueryableIndexSegment.java b/processing/src/main/java/org/apache/druid/query/aggregation/NoopVectorAggregator.java
similarity index 52%
copy from processing/src/main/java/org/apache/druid/segment/QueryableIndexSegment.java
copy to processing/src/main/java/org/apache/druid/query/aggregation/NoopVectorAggregator.java
index 8f68178..a57f7a9 100644
--- a/processing/src/main/java/org/apache/druid/segment/QueryableIndexSegment.java
+++ b/processing/src/main/java/org/apache/druid/query/aggregation/NoopVectorAggregator.java
@@ -17,52 +17,58 @@
  * under the License.
  */
 
-package org.apache.druid.segment;
+package org.apache.druid.query.aggregation;
 
-import org.apache.druid.timeline.SegmentId;
-import org.joda.time.Interval;
+import javax.annotation.Nullable;
+import java.nio.ByteBuffer;
 
-/**
- */
-public class QueryableIndexSegment extends AbstractSegment
+public class NoopVectorAggregator implements VectorAggregator
 {
-  private final QueryableIndex index;
-  private final SegmentId segmentId;
+  private static final NoopVectorAggregator INSTANCE = new NoopVectorAggregator();
+
+  public static NoopVectorAggregator instance()
+  {
+    return INSTANCE;
+  }
 
-  public QueryableIndexSegment(QueryableIndex index, final SegmentId segmentId)
+  private NoopVectorAggregator()
   {
-    this.index = index;
-    this.segmentId = segmentId;
+    // Singleton.
   }
 
   @Override
-  public SegmentId getId()
+  public void init(ByteBuffer buf, int position)
   {
-    return segmentId;
+    // Do nothing.
   }
 
   @Override
-  public Interval getDataInterval()
+  public void aggregate(final ByteBuffer buf, final int position, final int startRow, final int endRow)
   {
-    return index.getDataInterval();
+    // Do nothing.
   }
 
   @Override
-  public QueryableIndex asQueryableIndex()
+  public void aggregate(
+      final ByteBuffer buf,
+      final int numRows,
+      final int[] positions,
+      @Nullable final int[] rows,
+      final int positionOffset
+  )
   {
-    return index;
+    // Do nothing.
   }
 
   @Override
-  public StorageAdapter asStorageAdapter()
+  public Object get(ByteBuffer buf, int position)
   {
-    return new QueryableIndexStorageAdapter(index);
+    return null;
   }
 
   @Override
   public void close()
   {
-    // this is kinda nasty
-    index.close();
+    // Do nothing.
   }
 }
diff --git a/processing/src/main/java/org/apache/druid/query/aggregation/NullableAggregatorFactory.java b/processing/src/main/java/org/apache/druid/query/aggregation/NullableAggregatorFactory.java
index 1c7d4b2..e88eac3 100644
--- a/processing/src/main/java/org/apache/druid/query/aggregation/NullableAggregatorFactory.java
+++ b/processing/src/main/java/org/apache/druid/query/aggregation/NullableAggregatorFactory.java
@@ -20,11 +20,14 @@
 package org.apache.druid.query.aggregation;
 
 
+import com.google.common.base.Preconditions;
 import org.apache.druid.common.config.NullHandling;
 import org.apache.druid.guice.annotations.ExtensionPoint;
 import org.apache.druid.segment.BaseNullableColumnValueSelector;
 import org.apache.druid.segment.ColumnSelectorFactory;
 import org.apache.druid.segment.ColumnValueSelector;
+import org.apache.druid.segment.vector.VectorColumnSelectorFactory;
+import org.apache.druid.segment.vector.VectorValueSelector;
 
 /**
  * Abstract class with functionality to wrap {@link Aggregator}, {@link BufferAggregator} and {@link AggregateCombiner}
@@ -35,22 +38,31 @@ import org.apache.druid.segment.ColumnValueSelector;
 public abstract class NullableAggregatorFactory<T extends BaseNullableColumnValueSelector> extends AggregatorFactory
 {
   @Override
-  public final Aggregator factorize(ColumnSelectorFactory metricFactory)
+  public final Aggregator factorize(ColumnSelectorFactory columnSelectorFactory)
   {
-    T selector = selector(metricFactory);
-    Aggregator aggregator = factorize(metricFactory, selector);
+    T selector = selector(columnSelectorFactory);
+    Aggregator aggregator = factorize(columnSelectorFactory, selector);
     return NullHandling.replaceWithDefault() ? aggregator : new NullableAggregator(aggregator, selector);
   }
 
   @Override
-  public final BufferAggregator factorizeBuffered(ColumnSelectorFactory metricFactory)
+  public final BufferAggregator factorizeBuffered(ColumnSelectorFactory columnSelectorFactory)
   {
-    T selector = selector(metricFactory);
-    BufferAggregator aggregator = factorizeBuffered(metricFactory, selector);
+    T selector = selector(columnSelectorFactory);
+    BufferAggregator aggregator = factorizeBuffered(columnSelectorFactory, selector);
     return NullHandling.replaceWithDefault() ? aggregator : new NullableBufferAggregator(aggregator, selector);
   }
 
   @Override
+  public final VectorAggregator factorizeVector(VectorColumnSelectorFactory columnSelectorFactory)
+  {
+    Preconditions.checkState(canVectorize(), "Cannot vectorize");
+    VectorValueSelector selector = vectorSelector(columnSelectorFactory);
+    VectorAggregator aggregator = factorizeVector(columnSelectorFactory, selector);
+    return NullHandling.replaceWithDefault() ? aggregator : new NullableVectorAggregator(aggregator, selector);
+  }
+
+  @Override
   public final AggregateCombiner makeNullableAggregateCombiner()
   {
     AggregateCombiner combiner = makeAggregateCombiner();
@@ -70,26 +82,59 @@ public abstract class NullableAggregatorFactory<T extends BaseNullableColumnValu
    *
    * @see ColumnValueSelector
    */
-  protected abstract T selector(ColumnSelectorFactory metricFactory);
+  protected abstract T selector(ColumnSelectorFactory columnSelectorFactory);
+
+  /**
+   * Creates a {@link VectorValueSelector} for the aggregated column.
+   *
+   * @see VectorValueSelector
+   */
+  protected VectorValueSelector vectorSelector(VectorColumnSelectorFactory columnSelectorFactory)
+  {
+    throw new UnsupportedOperationException("Cannot vectorize");
+  }
 
   /**
    * Creates an {@link Aggregator} to aggregate values from several rows, by using the provided selector.
-   * @param metricFactory metricFactory
-   * @param selector {@link ColumnValueSelector} for the column to aggregate.
+   *
+   * @param columnSelectorFactory metricFactory
+   * @param selector              {@link ColumnValueSelector} for the column to aggregate.
    *
    * @see Aggregator
    */
-  protected abstract Aggregator factorize(ColumnSelectorFactory metricFactory, T selector);
+  protected abstract Aggregator factorize(ColumnSelectorFactory columnSelectorFactory, T selector);
 
   /**
    * Creates an {@link BufferAggregator} to aggregate values from several rows into a ByteBuffer.
-   * @param metricFactory metricFactory
-   * @param selector {@link ColumnValueSelector} for the column to aggregate.
+   *
+   * @param columnSelectorFactory columnSelectorFactory in case any other columns are needed.
+   * @param selector              {@link ColumnValueSelector} for the column to aggregate.
    *
    * @see BufferAggregator
    */
   protected abstract BufferAggregator factorizeBuffered(
-      ColumnSelectorFactory metricFactory,
+      ColumnSelectorFactory columnSelectorFactory,
       T selector
   );
+
+  /**
+   * Creates a {@link VectorAggregator} to aggregate values from several rows into a ByteBuffer.
+   *
+   * @param columnSelectorFactory columnSelectorFactory in case any other columns are needed.
+   * @param selector              {@link VectorValueSelector} for the column to aggregate.
+   *
+   * @see BufferAggregator
+   */
+  protected VectorAggregator factorizeVector(
+      // Not used by current aggregators, but here for parity with "factorizeBuffered".
+      @SuppressWarnings("unused") VectorColumnSelectorFactory columnSelectorFactory,
+      VectorValueSelector selector
+  )
+  {
+    if (!canVectorize()) {
+      throw new UnsupportedOperationException("Cannot vectorize");
+    } else {
+      throw new UnsupportedOperationException("canVectorize returned true but 'factorizeVector' is not implemented");
+    }
+  }
 }
diff --git a/processing/src/main/java/org/apache/druid/query/aggregation/NullableBufferAggregator.java b/processing/src/main/java/org/apache/druid/query/aggregation/NullableBufferAggregator.java
index e176a32..0e0d169 100644
--- a/processing/src/main/java/org/apache/druid/query/aggregation/NullableBufferAggregator.java
+++ b/processing/src/main/java/org/apache/druid/query/aggregation/NullableBufferAggregator.java
@@ -28,14 +28,18 @@ import javax.annotation.Nullable;
 import java.nio.ByteBuffer;
 
 /**
- * The result of a NullableBufferAggregator will be null if all the values to be aggregated are null values or no values
- * are aggregated at all. If any of the value is non-null, the result would be the aggregated value of the delegate
- * aggregator. Note that the delegate aggregator is not required to perform check for
- * {@link BaseNullableColumnValueSelector#isNull()} on the selector as only non-null values will be passed to the
- * delegate aggregator. This class is only used when SQL compatible null handling is enabled.
- * When writing aggregated result to buffer, it will write an additional byte to store the nullability of the
- * aggregated result.
- * Buffer Layout - 1 byte for storing nullability + delegate storage bytes.
+ * A wrapper around a non-null-aware BufferAggregator that makes it null-aware. This removes the need for each
+ * aggregator class to handle nulls on its own.
+ *
+ * The result of this aggregator will be null if all the values to be aggregated are null values or no values are
+ * aggregated at all. If any of the values are non-null, the result will be the aggregated value of the delegate
+ * aggregator.
+ *
+ * When wrapped by this class, the underlying aggregator's required storage space is increased by one byte. The extra
+ * byte is a boolean that stores whether or not any non-null values have been seen. The extra byte is placed before
+ * the underlying aggregator's normal state. (Buffer layout = [nullability byte] [delegate storage bytes])
+ *
+ * @see NullableVectorAggregator, the vectorized version.
  */
 @PublicApi
 public final class NullableBufferAggregator implements BufferAggregator
@@ -112,6 +116,12 @@ public final class NullableBufferAggregator implements BufferAggregator
   }
 
   @Override
+  public void relocate(int oldPosition, int newPosition, ByteBuffer oldBuffer, ByteBuffer newBuffer)
+  {
+    delegate.relocate(oldPosition + Byte.BYTES, newPosition + Byte.BYTES, oldBuffer, newBuffer);
+  }
+
+  @Override
   public void close()
   {
     delegate.close();
diff --git a/processing/src/main/java/org/apache/druid/query/aggregation/NullableVectorAggregator.java b/processing/src/main/java/org/apache/druid/query/aggregation/NullableVectorAggregator.java
new file mode 100644
index 0000000..bb8faed
--- /dev/null
+++ b/processing/src/main/java/org/apache/druid/query/aggregation/NullableVectorAggregator.java
@@ -0,0 +1,165 @@
+/*
+ * 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.query.aggregation;
+
+import org.apache.druid.common.config.NullHandling;
+import org.apache.druid.java.util.common.ISE;
+import org.apache.druid.segment.vector.VectorValueSelector;
+
+import javax.annotation.Nullable;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+
+/**
+ * A wrapper around a non-null-aware VectorAggregator that makes it null-aware. This removes the need for each
+ * aggregator class to handle nulls on its own. This class only makes sense as a wrapper for "primitive" aggregators,
+ * i.e., ones that take {@link VectorValueSelector} as input.
+ *
+ * The result of this aggregator will be null if all the values to be aggregated are null values or no values are
+ * aggregated at all. If any of the values are non-null, the result will be the aggregated value of the delegate
+ * aggregator.
+ *
+ * When wrapped by this class, the underlying aggregator's required storage space is increased by one byte. The extra
+ * byte is a boolean that stores whether or not any non-null values have been seen. The extra byte is placed before
+ * the underlying aggregator's normal state. (Buffer layout = [nullability byte] [delegate storage bytes])
+ *
+ * @see NullableBufferAggregator, the vectorized version.
+ */
+public class NullableVectorAggregator implements VectorAggregator
+{
+  private final VectorAggregator delegate;
+  private final VectorValueSelector selector;
+
+  @Nullable
+  private int[] vAggregationPositions = null;
+
+  @Nullable
+  private int[] vAggregationRows = null;
+
+  NullableVectorAggregator(VectorAggregator delegate, VectorValueSelector selector)
+  {
+    this.delegate = delegate;
+    this.selector = selector;
+  }
+
+  @Override
+  public void init(ByteBuffer buf, int position)
+  {
+    buf.put(position, NullHandling.IS_NULL_BYTE);
+    delegate.init(buf, position + Byte.BYTES);
+  }
+
+  @Override
+  public void aggregate(ByteBuffer buf, int position, int startRow, int endRow)
+  {
+    final boolean[] nullVector = selector.getNullVector();
+    if (nullVector != null) {
+      // Deferred initialization, since vAggregationPositions and vAggregationRows are only needed if nulls
+      // actually occur.
+      if (vAggregationPositions == null) {
+        vAggregationPositions = new int[selector.getMaxVectorSize()];
+        vAggregationRows = new int[selector.getMaxVectorSize()];
+      }
+
+      int j = 0;
+      for (int i = startRow; i < endRow; i++) {
+        if (!nullVector[i]) {
+          vAggregationRows[j++] = i;
+        }
+      }
+
+      Arrays.fill(vAggregationPositions, 0, j, position);
+
+      doAggregate(buf, j, vAggregationPositions, vAggregationRows, 0);
+    } else {
+      doAggregate(buf, position, startRow, endRow);
+    }
+  }
+
+  @Override
+  public void aggregate(ByteBuffer buf, int numRows, int[] positions, @Nullable int[] rows, int positionOffset)
+  {
+    final boolean[] nullVector = selector.getNullVector();
+    if (nullVector != null) {
+      // Deferred initialization, since vAggregationPositions and vAggregationRows are only needed if nulls
+      // actually occur.
+      if (vAggregationPositions == null) {
+        vAggregationPositions = new int[selector.getMaxVectorSize()];
+        vAggregationRows = new int[selector.getMaxVectorSize()];
+      }
+
+      int j = 0;
+      for (int i = 0; i < numRows; i++) {
+        final int rowNum = rows == null ? i : rows[i];
+        if (!nullVector[rowNum]) {
+          vAggregationPositions[j] = positions[i];
+          vAggregationRows[j] = rowNum;
+          j++;
+        }
+      }
+
+      doAggregate(buf, j, vAggregationPositions, vAggregationRows, positionOffset);
+    } else {
+      doAggregate(buf, numRows, positions, rows, positionOffset);
+    }
+  }
+
+  @Override
+  @Nullable
+  public Object get(ByteBuffer buf, int position)
+  {
+    switch (buf.get(position)) {
+      case NullHandling.IS_NULL_BYTE:
+        return null;
+      case NullHandling.IS_NOT_NULL_BYTE:
+        return delegate.get(buf, position + Byte.BYTES);
+      default:
+        // Corrupted byte?
+        throw new ISE("Bad null-marker byte, delegate class[%s]", delegate.getClass().getName());
+    }
+  }
+
+  @Override
+  public void relocate(int oldPosition, int newPosition, ByteBuffer oldBuffer, ByteBuffer newBuffer)
+  {
+    delegate.relocate(oldPosition + Byte.BYTES, newPosition + Byte.BYTES, oldBuffer, newBuffer);
+  }
+
+  @Override
+  public void close()
+  {
+    delegate.close();
+  }
+
+  private void doAggregate(ByteBuffer buf, int position, int start, int end)
+  {
+    buf.put(position, NullHandling.IS_NOT_NULL_BYTE);
+    delegate.aggregate(buf, position + Byte.BYTES, start, end);
+  }
+
+  private void doAggregate(ByteBuffer buf, int numRows, int[] positions, @Nullable int[] rows, int positionOffset)
+  {
+    for (int i = 0; i < numRows; i++) {
+      buf.put(positions[i] + positionOffset, NullHandling.IS_NOT_NULL_BYTE);
+    }
+
+    delegate.aggregate(buf, numRows, positions, rows, positionOffset + Byte.BYTES);
+  }
+}
diff --git a/processing/src/main/java/org/apache/druid/query/aggregation/SuppressedAggregatorFactory.java b/processing/src/main/java/org/apache/druid/query/aggregation/SuppressedAggregatorFactory.java
index c19eb58..35e7938 100644
--- a/processing/src/main/java/org/apache/druid/query/aggregation/SuppressedAggregatorFactory.java
+++ b/processing/src/main/java/org/apache/druid/query/aggregation/SuppressedAggregatorFactory.java
@@ -23,6 +23,7 @@ import org.apache.druid.query.PerSegmentQueryOptimizationContext;
 import org.apache.druid.query.cache.CacheKeyBuilder;
 import org.apache.druid.query.monomorphicprocessing.RuntimeShapeInspector;
 import org.apache.druid.segment.ColumnSelectorFactory;
+import org.apache.druid.segment.vector.VectorColumnSelectorFactory;
 
 import javax.annotation.Nullable;
 import java.nio.ByteBuffer;
@@ -63,6 +64,18 @@ public class SuppressedAggregatorFactory extends AggregatorFactory
   }
 
   @Override
+  public VectorAggregator factorizeVector(VectorColumnSelectorFactory columnSelectorFactory)
+  {
+    return new SuppressedVectorAggregator(delegate.factorizeVector(columnSelectorFactory));
+  }
+
+  @Override
+  public boolean canVectorize()
+  {
+    return delegate.canVectorize();
+  }
+
+  @Override
   public Comparator getComparator()
   {
     return delegate.getComparator();
@@ -136,6 +149,12 @@ public class SuppressedAggregatorFactory extends AggregatorFactory
   }
 
   @Override
+  public int getMaxIntermediateSizeWithNulls()
+  {
+    return delegate.getMaxIntermediateSizeWithNulls();
+  }
+
+  @Override
   public AggregatorFactory optimizeForSegment(PerSegmentQueryOptimizationContext optimizationContext)
   {
     // we are already the result of an optimizeForSegment() call
@@ -372,4 +391,78 @@ public class SuppressedAggregatorFactory extends AggregatorFactory
       return delegate;
     }
   }
+
+  public static class SuppressedVectorAggregator implements VectorAggregator
+  {
+    private final VectorAggregator delegate;
+
+    public SuppressedVectorAggregator(VectorAggregator delegate)
+    {
+      this.delegate = delegate;
+    }
+
+    @Override
+    public void init(ByteBuffer buf, int position)
+    {
+      delegate.init(buf, position);
+    }
+
+    @Override
+    public void aggregate(ByteBuffer buf, int position, int startRow, int endRow)
+    {
+      // no-op
+    }
+
+    @Override
+    public void aggregate(ByteBuffer buf, int numRows, int[] positions, @Nullable int[] rows, int positionOffset)
+    {
+      // no-op
+    }
+
+    @Nullable
+    @Override
+    public Object get(ByteBuffer buf, int position)
+    {
+      return delegate.get(buf, position);
+    }
+
+    @Override
+    public void relocate(int oldPosition, int newPosition, ByteBuffer oldBuffer, ByteBuffer newBuffer)
+    {
+      delegate.relocate(oldPosition, newPosition, oldBuffer, newBuffer);
+    }
+
+    @Override
+    public void close()
+    {
+      delegate.close();
+    }
+
+    @Override
+    public boolean equals(Object o)
+    {
+      if (this == o) {
+        return true;
+      }
+      if (o == null || getClass() != o.getClass()) {
+        return false;
+      }
+      SuppressedVectorAggregator that = (SuppressedVectorAggregator) o;
+      return Objects.equals(delegate, that.delegate);
+    }
+
+    @Override
+    public int hashCode()
+    {
+      return Objects.hash(delegate);
+    }
+
+    @Override
+    public String toString()
+    {
+      return "SuppressedVectorAggregator{" +
+             "delegate=" + delegate +
+             '}';
+    }
+  }
 }
diff --git a/processing/src/main/java/org/apache/druid/query/aggregation/VectorAggregator.java b/processing/src/main/java/org/apache/druid/query/aggregation/VectorAggregator.java
new file mode 100644
index 0000000..9da781d
--- /dev/null
+++ b/processing/src/main/java/org/apache/druid/query/aggregation/VectorAggregator.java
@@ -0,0 +1,86 @@
+/*
+ * 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.query.aggregation;
+
+import javax.annotation.Nullable;
+import java.nio.ByteBuffer;
+
+/**
+ * An object that can aggregate metrics into a ByteBuffer, from vectorized column selectors. Its aggregation-related
+ * methods (namely, "aggregate" and "get") do not take the actual input values to aggregate, because it is assumed that
+ * the VectorAggregator was given something that it can use to get at the current batch of data.
+ *
+ * None of the methods in this class are annotated with
+ * {@link org.apache.druid.query.monomorphicprocessing.CalledFromHotLoop} because vectorized query engines do not use
+ * monomorphic-processing-style specialization.
+ *
+ * @see BufferAggregator, the vectorized version.
+ */
+public interface VectorAggregator
+{
+  /**
+   * Same as {@link BufferAggregator#init}.
+   */
+  void init(ByteBuffer buf, int position);
+
+  /**
+   * Aggregate a range of rows into a single aggregation slot.
+   *
+   * <b>Implementations must not change the position, limit or mark of the given buffer</b>
+   *
+   * @param buf      byte buffer storing the byte array representation of the aggregate
+   * @param position offset within the byte buffer at which the current aggregate value is stored
+   * @param startRow first row of the range within the current batch to aggregate (inclusive)
+   * @param endRow   end row of the range (exclusive)
+   */
+  void aggregate(ByteBuffer buf, int position, int startRow, int endRow);
+
+  /**
+   * Aggregate a list of rows ("rows") into a list of aggregation slots ("positions").
+   *
+   * <b>Implementations must not change the position, limit or mark of the given buffer</b>
+   *
+   * @param buf            byte buffer storing the byte array representation of the aggregate
+   * @param numRows        number of rows to aggregate
+   * @param positions      array of aggregate value positions within the buffer; must be at least as long as "numRows"
+   * @param rows           array of row numbers within the current row batch; must be at least as long as "numRows". If
+   *                       null, the aggregator will aggregate rows from 0 (inclusive) to numRows (exclusive).
+   * @param positionOffset an offset to apply to each value from "positions"
+   */
+  void aggregate(ByteBuffer buf, int numRows, int positions[], @Nullable int[] rows, int positionOffset);
+
+  /**
+   * Same as {@link BufferAggregator#get}.
+   */
+  @Nullable
+  Object get(ByteBuffer buf, int position);
+
+  /**
+   * Same as {@link BufferAggregator#relocate}.
+   */
+  default void relocate(int oldPosition, int newPosition, ByteBuffer oldBuffer, ByteBuffer newBuffer)
+  {
+  }
+
+  /**
+   * Release any resources used by the aggregator.
+   */
+  void close();
+}
diff --git a/processing/src/main/java/org/apache/druid/query/aggregation/cardinality/CardinalityBufferAggregator.java b/processing/src/main/java/org/apache/druid/query/aggregation/cardinality/CardinalityBufferAggregator.java
index 875cd3d..64d70d4 100644
--- a/processing/src/main/java/org/apache/druid/query/aggregation/cardinality/CardinalityBufferAggregator.java
+++ b/processing/src/main/java/org/apache/druid/query/aggregation/cardinality/CardinalityBufferAggregator.java
@@ -24,6 +24,7 @@ import org.apache.druid.java.util.common.StringUtils;
 import org.apache.druid.query.ColumnSelectorPlus;
 import org.apache.druid.query.aggregation.BufferAggregator;
 import org.apache.druid.query.aggregation.cardinality.types.CardinalityAggregatorColumnSelectorStrategy;
+import org.apache.druid.query.aggregation.hyperloglog.HyperUniquesBufferAggregator;
 import org.apache.druid.query.monomorphicprocessing.RuntimeShapeInspector;
 
 import java.nio.ByteBuffer;
@@ -33,8 +34,6 @@ public class CardinalityBufferAggregator implements BufferAggregator
   private final ColumnSelectorPlus<CardinalityAggregatorColumnSelectorStrategy>[] selectorPluses;
   private final boolean byRow;
 
-  private static final byte[] EMPTY_BYTES = HyperLogLogCollector.makeEmptyVersionedByteArray();
-
   CardinalityBufferAggregator(
       ColumnSelectorPlus<CardinalityAggregatorColumnSelectorStrategy>[] selectorPluses,
       boolean byRow
@@ -47,9 +46,7 @@ public class CardinalityBufferAggregator implements BufferAggregator
   @Override
   public void init(ByteBuffer buf, int position)
   {
-    final ByteBuffer mutationBuffer = buf.duplicate();
-    mutationBuffer.position(position);
-    mutationBuffer.put(EMPTY_BYTES);
+    HyperUniquesBufferAggregator.doInit(buf, position);
   }
 
   @Override
@@ -78,11 +75,7 @@ public class CardinalityBufferAggregator implements BufferAggregator
   @Override
   public Object get(ByteBuffer buf, int position)
   {
-    ByteBuffer dataCopyBuffer = ByteBuffer.allocate(HyperLogLogCollector.getLatestNumBytesForDenseStorage());
-    ByteBuffer mutationBuffer = buf.duplicate();
-    mutationBuffer.position(position);
-    mutationBuffer.get(dataCopyBuffer.array());
-    return HyperLogLogCollector.makeCollector(dataCopyBuffer);
+    return HyperUniquesBufferAggregator.doGet(buf, position);
   }
 
   @Override
diff --git a/processing/src/main/java/org/apache/druid/query/aggregation/hyperloglog/HyperUniquesAggregatorFactory.java b/processing/src/main/java/org/apache/druid/query/aggregation/hyperloglog/HyperUniquesAggregatorFactory.java
index 27f1650..b3ab8c3 100644
--- a/processing/src/main/java/org/apache/druid/query/aggregation/hyperloglog/HyperUniquesAggregatorFactory.java
+++ b/processing/src/main/java/org/apache/druid/query/aggregation/hyperloglog/HyperUniquesAggregatorFactory.java
@@ -33,11 +33,16 @@ import org.apache.druid.query.aggregation.AggregatorUtil;
 import org.apache.druid.query.aggregation.BufferAggregator;
 import org.apache.druid.query.aggregation.NoopAggregator;
 import org.apache.druid.query.aggregation.NoopBufferAggregator;
+import org.apache.druid.query.aggregation.NoopVectorAggregator;
+import org.apache.druid.query.aggregation.VectorAggregator;
 import org.apache.druid.query.aggregation.cardinality.HyperLogLogCollectorAggregateCombiner;
 import org.apache.druid.query.cache.CacheKeyBuilder;
 import org.apache.druid.segment.BaseObjectColumnValueSelector;
 import org.apache.druid.segment.ColumnSelectorFactory;
 import org.apache.druid.segment.NilColumnValueSelector;
+import org.apache.druid.segment.column.ColumnCapabilities;
+import org.apache.druid.segment.column.ValueType;
+import org.apache.druid.segment.vector.VectorColumnSelectorFactory;
 
 import javax.annotation.Nullable;
 import java.nio.ByteBuffer;
@@ -124,6 +129,23 @@ public class HyperUniquesAggregatorFactory extends AggregatorFactory
   }
 
   @Override
+  public VectorAggregator factorizeVector(final VectorColumnSelectorFactory selectorFactory)
+  {
+    final ColumnCapabilities capabilities = selectorFactory.getColumnCapabilities(fieldName);
+    if (capabilities == null || capabilities.getType() != ValueType.COMPLEX) {
+      return NoopVectorAggregator.instance();
+    } else {
+      return new HyperUniquesVectorAggregator(selectorFactory.makeObjectSelector(fieldName));
+    }
+  }
+
+  @Override
+  public boolean canVectorize()
+  {
+    return true;
+  }
+
+  @Override
   public Comparator getComparator()
   {
     return Comparators.naturalNullsFirst();
diff --git a/processing/src/main/java/org/apache/druid/query/aggregation/hyperloglog/HyperUniquesBufferAggregator.java b/processing/src/main/java/org/apache/druid/query/aggregation/hyperloglog/HyperUniquesBufferAggregator.java
index c7f31f4..3eba440 100644
--- a/processing/src/main/java/org/apache/druid/query/aggregation/hyperloglog/HyperUniquesBufferAggregator.java
+++ b/processing/src/main/java/org/apache/druid/query/aggregation/hyperloglog/HyperUniquesBufferAggregator.java
@@ -38,14 +38,31 @@ public class HyperUniquesBufferAggregator implements BufferAggregator
     this.selector = selector;
   }
 
-  @Override
-  public void init(ByteBuffer buf, int position)
+  public static void doInit(ByteBuffer buf, int position)
   {
     final ByteBuffer mutationBuffer = buf.duplicate();
     mutationBuffer.position(position);
     mutationBuffer.put(EMPTY_BYTES);
   }
 
+  public static HyperLogLogCollector doGet(ByteBuffer buf, int position)
+  {
+    final int size = HyperLogLogCollector.getLatestNumBytesForDenseStorage();
+    ByteBuffer dataCopyBuffer = ByteBuffer.allocate(size);
+    ByteBuffer mutationBuffer = buf.duplicate();
+    mutationBuffer.position(position);
+    mutationBuffer.limit(position + size);
+    dataCopyBuffer.put(mutationBuffer);
+    dataCopyBuffer.rewind();
+    return HyperLogLogCollector.makeCollector(dataCopyBuffer);
+  }
+
+  @Override
+  public void init(ByteBuffer buf, int position)
+  {
+    doInit(buf, position);
+  }
+
   @Override
   public void aggregate(ByteBuffer buf, int position)
   {
@@ -73,14 +90,7 @@ public class HyperUniquesBufferAggregator implements BufferAggregator
   @Override
   public Object get(ByteBuffer buf, int position)
   {
-    final int size = HyperLogLogCollector.getLatestNumBytesForDenseStorage();
-    ByteBuffer dataCopyBuffer = ByteBuffer.allocate(size);
-    ByteBuffer mutationBuffer = buf.duplicate();
-    mutationBuffer.position(position);
-    mutationBuffer.limit(position + size);
-    dataCopyBuffer.put(mutationBuffer);
-    dataCopyBuffer.rewind();
-    return HyperLogLogCollector.makeCollector(dataCopyBuffer);
+    return doGet(buf, position);
   }
 
   @Override
diff --git a/processing/src/main/java/org/apache/druid/query/aggregation/hyperloglog/HyperUniquesVectorAggregator.java b/processing/src/main/java/org/apache/druid/query/aggregation/hyperloglog/HyperUniquesVectorAggregator.java
new file mode 100644
index 0000000..616b9de
--- /dev/null
+++ b/processing/src/main/java/org/apache/druid/query/aggregation/hyperloglog/HyperUniquesVectorAggregator.java
@@ -0,0 +1,116 @@
+/*
+ * 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.query.aggregation.hyperloglog;
+
+import com.google.common.base.Preconditions;
+import org.apache.druid.hll.HyperLogLogCollector;
+import org.apache.druid.query.aggregation.VectorAggregator;
+import org.apache.druid.segment.vector.VectorObjectSelector;
+
+import javax.annotation.Nullable;
+import java.nio.ByteBuffer;
+
+public class HyperUniquesVectorAggregator implements VectorAggregator
+{
+  private final VectorObjectSelector selector;
+
+  public HyperUniquesVectorAggregator(final VectorObjectSelector selector)
+  {
+    this.selector = Preconditions.checkNotNull(selector, "selector");
+  }
+
+  @Override
+  public void init(final ByteBuffer buf, final int position)
+  {
+    HyperUniquesBufferAggregator.doInit(buf, position);
+  }
+
+  @Override
+  public void aggregate(final ByteBuffer buf, final int position, final int startRow, final int endRow)
+  {
+    // Save position, limit and restore later instead of allocating a new ByteBuffer object
+    final int oldPosition = buf.position();
+    final int oldLimit = buf.limit();
+    buf.limit(position + HyperLogLogCollector.getLatestNumBytesForDenseStorage());
+    buf.position(position);
+
+    try {
+      final HyperLogLogCollector collector = HyperLogLogCollector.makeCollector(buf);
+      final Object[] vector = selector.getObjectVector();
+      for (int i = startRow; i < endRow; i++) {
+        final HyperLogLogCollector otherCollector = (HyperLogLogCollector) vector[i];
+        if (otherCollector != null) {
+          collector.fold(otherCollector);
+        }
+      }
+    }
+    finally {
+      buf.limit(oldLimit);
+      buf.position(oldPosition);
+    }
+  }
+
+  @Override
+  public void aggregate(
+      final ByteBuffer buf,
+      final int numRows,
+      final int[] positions,
+      @Nullable final int[] rows,
+      final int positionOffset
+  )
+  {
+    final Object[] vector = selector.getObjectVector();
+
+    for (int i = 0; i < numRows; i++) {
+      final HyperLogLogCollector otherCollector = (HyperLogLogCollector) vector[rows != null ? rows[i] : i];
+      if (otherCollector == null) {
+        continue;
+      }
+
+      final int position = positions[i] + positionOffset;
+
+      // Save position, limit and restore later instead of allocating a new ByteBuffer object
+      final int oldPosition = buf.position();
+      final int oldLimit = buf.limit();
+      buf.limit(position + HyperLogLogCollector.getLatestNumBytesForDenseStorage());
+      buf.position(position);
+
+      try {
+        HyperLogLogCollector.makeCollector(buf).fold(otherCollector);
+      }
+      finally {
+        buf.limit(oldLimit);
+        buf.position(oldPosition);
+      }
+    }
+  }
+
+  @Override
+  public Object get(final ByteBuffer buf, final int position)
+  {
+    return HyperUniquesBufferAggregator.doGet(buf, position);
+  }
+
+  @Override
+  public void close()
+  {
+    // Nothing to close.
+  }
+}
diff --git a/processing/src/main/java/org/apache/druid/query/dimension/DefaultDimensionSpec.java b/processing/src/main/java/org/apache/druid/query/dimension/DefaultDimensionSpec.java
index 921402c..ddc499c 100644
--- a/processing/src/main/java/org/apache/druid/query/dimension/DefaultDimensionSpec.java
+++ b/processing/src/main/java/org/apache/druid/query/dimension/DefaultDimensionSpec.java
@@ -25,6 +25,8 @@ import org.apache.druid.query.cache.CacheKeyBuilder;
 import org.apache.druid.query.extraction.ExtractionFn;
 import org.apache.druid.segment.DimensionSelector;
 import org.apache.druid.segment.column.ValueType;
+import org.apache.druid.segment.vector.MultiValueDimensionVectorSelector;
+import org.apache.druid.segment.vector.SingleValueDimensionVectorSelector;
 
 import javax.annotation.Nullable;
 import java.util.Arrays;
@@ -111,6 +113,24 @@ public class DefaultDimensionSpec implements DimensionSpec
   }
 
   @Override
+  public SingleValueDimensionVectorSelector decorate(final SingleValueDimensionVectorSelector selector)
+  {
+    return selector;
+  }
+
+  @Override
+  public MultiValueDimensionVectorSelector decorate(final MultiValueDimensionVectorSelector selector)
+  {
+    return selector;
+  }
+
+  @Override
+  public boolean canVectorize()
+  {
+    return true;
+  }
+
+  @Override
   public boolean mustDecorate()
   {
     return false;
diff --git a/processing/src/main/java/org/apache/druid/query/dimension/DimensionSpec.java b/processing/src/main/java/org/apache/druid/query/dimension/DimensionSpec.java
index bbf970e..9f3eb2a 100644
--- a/processing/src/main/java/org/apache/druid/query/dimension/DimensionSpec.java
+++ b/processing/src/main/java/org/apache/druid/query/dimension/DimensionSpec.java
@@ -22,9 +22,12 @@ package org.apache.druid.query.dimension;
 import com.fasterxml.jackson.annotation.JsonSubTypes;
 import com.fasterxml.jackson.annotation.JsonTypeInfo;
 import org.apache.druid.java.util.common.Cacheable;
+import org.apache.druid.java.util.common.UOE;
 import org.apache.druid.query.extraction.ExtractionFn;
 import org.apache.druid.segment.DimensionSelector;
 import org.apache.druid.segment.column.ValueType;
+import org.apache.druid.segment.vector.MultiValueDimensionVectorSelector;
+import org.apache.druid.segment.vector.SingleValueDimensionVectorSelector;
 
 import javax.annotation.Nullable;
 
@@ -55,10 +58,29 @@ public interface DimensionSpec extends Cacheable
 
   DimensionSelector decorate(DimensionSelector selector);
 
+  default SingleValueDimensionVectorSelector decorate(SingleValueDimensionVectorSelector selector)
+  {
+    throw new UOE("DimensionSpec[%s] cannot vectorize", getClass().getName());
+  }
+
+  default MultiValueDimensionVectorSelector decorate(MultiValueDimensionVectorSelector selector)
+  {
+    throw new UOE("DimensionSpec[%s] cannot vectorize", getClass().getName());
+  }
+
   /**
    * Does this DimensionSpec require that decorate() be called to produce correct results?
    */
   boolean mustDecorate();
 
+  /**
+   * Does this DimensionSpec have working {@link #decorate(SingleValueDimensionVectorSelector)} and
+   * {@link #decorate(MultiValueDimensionVectorSelector)} methods?
+   */
+  default boolean canVectorize()
+  {
+    return false;
+  }
+
   boolean preservesOrdering();
 }
diff --git a/processing/src/main/java/org/apache/druid/segment/data/ReadableOffset.java b/processing/src/main/java/org/apache/druid/query/dimension/VectorColumnStrategizer.java
similarity index 51%
copy from processing/src/main/java/org/apache/druid/segment/data/ReadableOffset.java
copy to processing/src/main/java/org/apache/druid/query/dimension/VectorColumnStrategizer.java
index e116aeb..06338d4 100644
--- a/processing/src/main/java/org/apache/druid/segment/data/ReadableOffset.java
+++ b/processing/src/main/java/org/apache/druid/query/dimension/VectorColumnStrategizer.java
@@ -17,21 +17,25 @@
  * under the License.
  */
 
-package org.apache.druid.segment.data;
+package org.apache.druid.query.dimension;
 
-import org.apache.druid.query.monomorphicprocessing.CalledFromHotLoop;
-import org.apache.druid.query.monomorphicprocessing.HotLoopCallee;
+import org.apache.druid.segment.vector.MultiValueDimensionVectorSelector;
+import org.apache.druid.segment.vector.SingleValueDimensionVectorSelector;
+import org.apache.druid.segment.vector.VectorValueSelector;
 
 /**
- * A ReadableOffset is an object that provides an integer offset, ostensibly as an index into an array.
- *
- * See the companion class Offset, for more context on how this could be useful.  A ReadableOffset should be
- * given to classes (e.g. FloatColumnSelector objects) by something which keeps a reference to the base Offset object
- * and increments it.
+ * Class that encapsulates knowledge about how to create vector column processors. Used by
+ * {@link org.apache.druid.segment.DimensionHandlerUtils#makeVectorProcessor}.
  */
-public interface ReadableOffset extends HotLoopCallee
+public interface VectorColumnStrategizer<T>
 {
-  @CalledFromHotLoop
-  int getOffset();
-}
+  T makeSingleValueDimensionStrategy(SingleValueDimensionVectorSelector selector);
+
+  T makeMultiValueDimensionStrategy(MultiValueDimensionVectorSelector selector);
 
+  T makeFloatStrategy(VectorValueSelector selector);
+
+  T makeDoubleStrategy(VectorValueSelector selector);
+
+  T makeLongStrategy(VectorValueSelector selector);
+}
diff --git a/processing/src/main/java/org/apache/druid/query/filter/Filter.java b/processing/src/main/java/org/apache/druid/query/filter/Filter.java
index a8b7c7f..120358f 100644
--- a/processing/src/main/java/org/apache/druid/query/filter/Filter.java
+++ b/processing/src/main/java/org/apache/druid/query/filter/Filter.java
@@ -20,10 +20,13 @@
 package org.apache.druid.query.filter;
 
 import org.apache.druid.collections.bitmap.ImmutableBitmap;
+import org.apache.druid.java.util.common.UOE;
 import org.apache.druid.query.BitmapResultFactory;
 import org.apache.druid.query.DefaultBitmapResultFactory;
+import org.apache.druid.query.filter.vector.VectorValueMatcher;
 import org.apache.druid.segment.ColumnSelector;
 import org.apache.druid.segment.ColumnSelectorFactory;
+import org.apache.druid.segment.vector.VectorColumnSelectorFactory;
 
 public interface Filter
 {
@@ -84,6 +87,17 @@ public interface Filter
    */
   ValueMatcher makeMatcher(ColumnSelectorFactory factory);
 
+  /**
+   * Get a VectorValueMatcher that applies this filter to row vectors.
+   *
+   * @param factory Object used to create ValueMatchers
+   *
+   * @return VectorValueMatcher that applies this filter to row vectors.
+   */
+  default VectorValueMatcher makeVectorMatcher(VectorColumnSelectorFactory factory)
+  {
+    throw new UOE("Filter[%s] cannot vectorize", getClass().getName());
+  }
 
   /**
    * Indicates whether this filter can return a bitmap index for filtering, based on
@@ -107,4 +121,12 @@ public interface Filter
    * @return true if this Filter supports selectivity estimation, false otherwise.
    */
   boolean supportsSelectivityEstimation(ColumnSelector columnSelector, BitmapIndexSelector indexSelector);
+
+  /**
+   * Returns true if this filter can produce a vectorized matcher from its "makeVectorMatcher" method.
+   */
+  default boolean canVectorizeMatcher()
+  {
+    return false;
+  }
 }
diff --git a/processing/src/main/java/org/apache/druid/query/filter/IntervalDimFilter.java b/processing/src/main/java/org/apache/druid/query/filter/IntervalDimFilter.java
index c2d8511..eec5cca 100644
--- a/processing/src/main/java/org/apache/druid/query/filter/IntervalDimFilter.java
+++ b/processing/src/main/java/org/apache/druid/query/filter/IntervalDimFilter.java
@@ -163,6 +163,12 @@ public class IntervalDimFilter implements DimFilter
     return result;
   }
 
+  @Override
+  public String toString()
+  {
+    return convertedFilter.toString();
+  }
+
   private List<Pair<Long, Long>> makeIntervalLongs()
   {
     List<Pair<Long, Long>> intervalLongs = new ArrayList<>();
diff --git a/processing/src/main/java/org/apache/druid/query/filter/StringValueMatcherColumnSelectorStrategy.java b/processing/src/main/java/org/apache/druid/query/filter/StringValueMatcherColumnSelectorStrategy.java
index e509275..73100c6 100644
--- a/processing/src/main/java/org/apache/druid/query/filter/StringValueMatcherColumnSelectorStrategy.java
+++ b/processing/src/main/java/org/apache/druid/query/filter/StringValueMatcherColumnSelectorStrategy.java
@@ -20,20 +20,72 @@
 package org.apache.druid.query.filter;
 
 import com.google.common.base.Predicate;
+import org.apache.druid.common.config.NullHandling;
+import org.apache.druid.segment.DimensionDictionarySelector;
 import org.apache.druid.segment.DimensionSelector;
 import org.apache.druid.segment.data.IndexedInts;
 import org.apache.druid.segment.filter.BooleanValueMatcher;
 
+import javax.annotation.Nullable;
+import java.util.Objects;
+
 public class StringValueMatcherColumnSelectorStrategy implements ValueMatcherColumnSelectorStrategy<DimensionSelector>
 {
   private static final String[] NULL_VALUE = new String[]{null};
   private static final ValueGetter NULL_VALUE_GETTER = () -> NULL_VALUE;
 
-  @Override
-  public ValueMatcher makeValueMatcher(final DimensionSelector selector, String value)
+  private final boolean hasMultipleValues;
+
+  public StringValueMatcherColumnSelectorStrategy(final boolean hasMultipleValues)
+  {
+    this.hasMultipleValues = hasMultipleValues;
+  }
+
+  @Nullable
+  public static Boolean toBooleanIfPossible(
+      final DimensionDictionarySelector selector,
+      final boolean hasMultipleValues,
+      final Predicate<String> predicate
+  )
   {
     if (selector.getValueCardinality() == 0) {
-      return BooleanValueMatcher.of(value == null);
+      // Column has no values (it doesn't exist, or it's all empty arrays).
+      // Match if and only if "predicate" matches null.
+      return predicate.apply(null);
+    } else if (!hasMultipleValues && selector.getValueCardinality() == 1 && selector.nameLookupPossibleInAdvance()) {
+      // Every row has the same value. Match if and only if "predicate" matches the possible value.
+      return predicate.apply(selector.lookupName(0));
+    } else {
+      return null;
+    }
+  }
+
+  @Nullable
+  private static ValueMatcher toBooleanMatcherIfPossible(
+      final DimensionSelector selector,
+      final boolean hasMultipleValues,
+      final Predicate<String> predicate
+  )
+  {
+    final Boolean booleanValue = StringValueMatcherColumnSelectorStrategy.toBooleanIfPossible(
+        selector,
+        hasMultipleValues,
+        predicate
+    );
+    return booleanValue == null ? null : BooleanValueMatcher.of(booleanValue);
+  }
+
+  @Override
+  public ValueMatcher makeValueMatcher(final DimensionSelector selector, final String value)
+  {
+    final ValueMatcher booleanMatcher = toBooleanMatcherIfPossible(
+        selector,
+        hasMultipleValues,
+        s -> Objects.equals(s, NullHandling.emptyToNullIfNeeded(value))
+    );
+
+    if (booleanMatcher != null) {
+      return booleanMatcher;
     } else {
       return selector.makeValueMatcher(value);
     }
@@ -46,8 +98,10 @@ public class StringValueMatcherColumnSelectorStrategy implements ValueMatcherCol
   )
   {
     final Predicate<String> predicate = predicateFactory.makeStringPredicate();
-    if (selector.getValueCardinality() == 0) {
-      return BooleanValueMatcher.of(predicate.apply(null));
+    final ValueMatcher booleanMatcher = toBooleanMatcherIfPossible(selector, hasMultipleValues, predicate);
+
+    if (booleanMatcher != null) {
+      return booleanMatcher;
     } else {
       return selector.makeValueMatcher(predicate);
     }
diff --git a/processing/src/main/java/org/apache/druid/query/filter/ValueMatcher.java b/processing/src/main/java/org/apache/druid/query/filter/ValueMatcher.java
index be36c4d..a800ec7 100644
--- a/processing/src/main/java/org/apache/druid/query/filter/ValueMatcher.java
+++ b/processing/src/main/java/org/apache/druid/query/filter/ValueMatcher.java
@@ -25,6 +25,11 @@ import org.apache.druid.query.monomorphicprocessing.RuntimeShapeInspector;
 import org.apache.druid.segment.BaseNullableColumnValueSelector;
 
 /**
+ * An object that returns a boolean indicating if the "current" row should be selected or not. The most prominent use
+ * of this interface is that it is returned by the {@link Filter} "makeMatcher" method, where it is used to identify
+ * selected rows for filtered cursors and filtered aggregators.
+ *
+ * @see org.apache.druid.query.filter.vector.VectorValueMatcher, the vectorized version
  */
 public interface ValueMatcher extends HotLoopCallee
 {
diff --git a/processing/src/main/java/org/apache/druid/query/filter/ValueMatcherColumnSelectorStrategyFactory.java b/processing/src/main/java/org/apache/druid/query/filter/ValueMatcherColumnSelectorStrategyFactory.java
index edecbbe..2797f08 100644
--- a/processing/src/main/java/org/apache/druid/query/filter/ValueMatcherColumnSelectorStrategyFactory.java
+++ b/processing/src/main/java/org/apache/druid/query/filter/ValueMatcherColumnSelectorStrategyFactory.java
@@ -49,7 +49,7 @@ public class ValueMatcherColumnSelectorStrategyFactory
     ValueType type = capabilities.getType();
     switch (type) {
       case STRING:
-        return new StringValueMatcherColumnSelectorStrategy();
+        return new StringValueMatcherColumnSelectorStrategy(capabilities.hasMultipleValues());
       case LONG:
         return new LongValueMatcherColumnSelectorStrategy();
       case FLOAT:
diff --git a/processing/src/main/java/org/apache/druid/segment/data/ColumnarMultiInts.java b/processing/src/main/java/org/apache/druid/query/filter/vector/BaseVectorValueMatcher.java
similarity index 61%
copy from processing/src/main/java/org/apache/druid/segment/data/ColumnarMultiInts.java
copy to processing/src/main/java/org/apache/druid/query/filter/vector/BaseVectorValueMatcher.java
index 8392e13..aea2f14 100644
--- a/processing/src/main/java/org/apache/druid/segment/data/ColumnarMultiInts.java
+++ b/processing/src/main/java/org/apache/druid/query/filter/vector/BaseVectorValueMatcher.java
@@ -17,16 +17,28 @@
  * under the License.
  */
 
-package org.apache.druid.segment.data;
+package org.apache.druid.query.filter.vector;
 
-import java.io.Closeable;
+import org.apache.druid.segment.vector.VectorSizeInspector;
 
-/**
- * Resource that provides random access to a packed array of short arrays of ints ({@link IndexedInts}). Backs up
- * multi-valued {@link org.apache.druid.segment.column.DictionaryEncodedColumn}s.
- */
-public interface ColumnarMultiInts extends Indexed<IndexedInts>, Closeable
+public abstract class BaseVectorValueMatcher implements VectorValueMatcher
 {
+  private final VectorSizeInspector selector;
+
+  public BaseVectorValueMatcher(final VectorSizeInspector selector)
+  {
+    this.selector = selector;
+  }
+
+  @Override
+  public int getCurrentVectorSize()
+  {
+    return selector.getCurrentVectorSize();
+  }
+
   @Override
-  IndexedInts get(int index);
+  public int getMaxVectorSize()
+  {
+    return selector.getMaxVectorSize();
+  }
 }
diff --git a/processing/src/main/java/org/apache/druid/query/filter/vector/BooleanVectorValueMatcher.java b/processing/src/main/java/org/apache/druid/query/filter/vector/BooleanVectorValueMatcher.java
new file mode 100644
index 0000000..65af27b
--- /dev/null
+++ b/processing/src/main/java/org/apache/druid/query/filter/vector/BooleanVectorValueMatcher.java
@@ -0,0 +1,63 @@
+/*
+ * 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.query.filter.vector;
+
+import org.apache.druid.segment.vector.VectorSizeInspector;
+
+public class BooleanVectorValueMatcher extends BaseVectorValueMatcher
+{
+  private final VectorSizeInspector selector;
+  private final boolean matches;
+
+  private BooleanVectorValueMatcher(final VectorSizeInspector selector, final boolean matches)
+  {
+    super(selector);
+    this.selector = selector;
+    this.matches = matches;
+  }
+
+  public static BooleanVectorValueMatcher of(final VectorSizeInspector selector, final boolean matches)
+  {
+    return new BooleanVectorValueMatcher(selector, matches);
+  }
+
+  @Override
+  public int getCurrentVectorSize()
+  {
+    return selector.getCurrentVectorSize();
+  }
+
+  @Override
+  public int getMaxVectorSize()
+  {
+    return selector.getCurrentVectorSize();
+  }
+
+  @Override
+  public ReadableVectorMatch match(final ReadableVectorMatch mask)
+  {
+    if (matches) {
+      assert mask.isValid(mask);
+      return mask;
+    } else {
+      return VectorMatch.allFalse();
+    }
+  }
+}
diff --git a/processing/src/main/java/org/apache/druid/query/filter/vector/DoubleVectorValueMatcher.java b/processing/src/main/java/org/apache/druid/query/filter/vector/DoubleVectorValueMatcher.java
new file mode 100644
index 0000000..c304a11
--- /dev/null
+++ b/processing/src/main/java/org/apache/druid/query/filter/vector/DoubleVectorValueMatcher.java
@@ -0,0 +1,105 @@
+/*
+ * 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.query.filter.vector;
+
+import org.apache.druid.query.filter.DruidDoublePredicate;
+import org.apache.druid.query.filter.DruidPredicateFactory;
+import org.apache.druid.segment.DimensionHandlerUtils;
+import org.apache.druid.segment.vector.VectorValueSelector;
+
+import javax.annotation.Nullable;
+
+public class DoubleVectorValueMatcher implements VectorValueMatcherFactory
+{
+  private final VectorValueSelector selector;
+
+  public DoubleVectorValueMatcher(final VectorValueSelector selector)
+  {
+    this.selector = selector;
+  }
+
+  @Override
+  public VectorValueMatcher makeMatcher(@Nullable final String value)
+  {
+    final Double matchVal = DimensionHandlerUtils.convertObjectToDouble(value);
+
+    if (matchVal == null) {
+      return BooleanVectorValueMatcher.of(selector, false);
+    }
+
+    final double matchValDouble = matchVal;
+
+    return new BaseVectorValueMatcher(selector)
+    {
+      final VectorMatch match = VectorMatch.wrap(new int[selector.getMaxVectorSize()]);
+
+      @Override
+      public ReadableVectorMatch match(final ReadableVectorMatch mask)
+      {
+        final double[] vector = selector.getDoubleVector();
+        final int[] selection = match.getSelection();
+
+        int numRows = 0;
+
+        for (int i = 0; i < mask.getSelectionSize(); i++) {
+          final int rowNum = mask.getSelection()[i];
+          if (vector[rowNum] == matchValDouble) {
+            selection[numRows++] = rowNum;
+          }
+        }
+
+        match.setSelectionSize(numRows);
+        assert match.isValid(mask);
+        return match;
+      }
+    };
+  }
+
+  @Override
+  public VectorValueMatcher makeMatcher(final DruidPredicateFactory predicateFactory)
+  {
+    final DruidDoublePredicate predicate = predicateFactory.makeDoublePredicate();
+
+    return new BaseVectorValueMatcher(selector)
+    {
+      final VectorMatch match = VectorMatch.wrap(new int[selector.getMaxVectorSize()]);
+
+      @Override
+      public ReadableVectorMatch match(final ReadableVectorMatch mask)
+      {
+        final double[] vector = selector.getDoubleVector();
+        final int[] selection = match.getSelection();
+
+        int numRows = 0;
+
+        for (int i = 0; i < mask.getSelectionSize(); i++) {
+          final int rowNum = mask.getSelection()[i];
+          if (predicate.applyDouble(vector[rowNum])) {
+            selection[numRows++] = rowNum;
+          }
+        }
+
+        match.setSelectionSize(numRows);
+        assert match.isValid(mask);
+        return match;
+      }
+    };
+  }
+}
diff --git a/processing/src/main/java/org/apache/druid/query/filter/vector/FloatVectorValueMatcher.java b/processing/src/main/java/org/apache/druid/query/filter/vector/FloatVectorValueMatcher.java
new file mode 100644
index 0000000..4ea33ae
--- /dev/null
+++ b/processing/src/main/java/org/apache/druid/query/filter/vector/FloatVectorValueMatcher.java
@@ -0,0 +1,105 @@
+/*
+ * 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.query.filter.vector;
+
+import org.apache.druid.query.filter.DruidFloatPredicate;
+import org.apache.druid.query.filter.DruidPredicateFactory;
+import org.apache.druid.segment.DimensionHandlerUtils;
+import org.apache.druid.segment.vector.VectorValueSelector;
+
+import javax.annotation.Nullable;
+
+public class FloatVectorValueMatcher implements VectorValueMatcherFactory
+{
+  private final VectorValueSelector selector;
+
+  public FloatVectorValueMatcher(final VectorValueSelector selector)
+  {
+    this.selector = selector;
+  }
+
+  @Override
+  public VectorValueMatcher makeMatcher(@Nullable final String value)
+  {
+    final Float matchVal = DimensionHandlerUtils.convertObjectToFloat(value);
+
+    if (matchVal == null) {
+      return BooleanVectorValueMatcher.of(selector, false);
+    }
+
+    final float matchValFloat = matchVal;
+
+    return new BaseVectorValueMatcher(selector)
+    {
+      final VectorMatch match = VectorMatch.wrap(new int[selector.getMaxVectorSize()]);
+
+      @Override
+      public ReadableVectorMatch match(final ReadableVectorMatch mask)
+      {
+        final float[] vector = selector.getFloatVector();
+        final int[] selection = match.getSelection();
+
+        int numRows = 0;
+
+        for (int i = 0; i < mask.getSelectionSize(); i++) {
+          final int rowNum = mask.getSelection()[i];
+          if (vector[rowNum] == matchValFloat) {
+            selection[numRows++] = rowNum;
+          }
+        }
+
+        match.setSelectionSize(numRows);
+        assert match.isValid(mask);
+        return match;
+      }
+    };
+  }
+
+  @Override
+  public VectorValueMatcher makeMatcher(final DruidPredicateFactory predicateFactory)
+  {
+    final DruidFloatPredicate predicate = predicateFactory.makeFloatPredicate();
+
+    return new BaseVectorValueMatcher(selector)
+    {
+      final VectorMatch match = VectorMatch.wrap(new int[selector.getMaxVectorSize()]);
+
+      @Override
+      public ReadableVectorMatch match(final ReadableVectorMatch mask)
+      {
+        final float[] vector = selector.getFloatVector();
+        final int[] selection = match.getSelection();
+
+        int numRows = 0;
+
+        for (int i = 0; i < mask.getSelectionSize(); i++) {
+          final int rowNum = mask.getSelection()[i];
+          if (predicate.applyFloat(vector[rowNum])) {
+            selection[numRows++] = rowNum;
+          }
+        }
+
+        match.setSelectionSize(numRows);
+        assert match.isValid(mask);
+        return match;
+      }
+    };
+  }
+}
diff --git a/processing/src/main/java/org/apache/druid/query/filter/vector/LongVectorValueMatcher.java b/processing/src/main/java/org/apache/druid/query/filter/vector/LongVectorValueMatcher.java
new file mode 100644
index 0000000..a07f9ba
--- /dev/null
+++ b/processing/src/main/java/org/apache/druid/query/filter/vector/LongVectorValueMatcher.java
@@ -0,0 +1,105 @@
+/*
+ * 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.query.filter.vector;
+
+import org.apache.druid.query.filter.DruidLongPredicate;
+import org.apache.druid.query.filter.DruidPredicateFactory;
+import org.apache.druid.segment.DimensionHandlerUtils;
+import org.apache.druid.segment.vector.VectorValueSelector;
+
+import javax.annotation.Nullable;
+
+public class LongVectorValueMatcher implements VectorValueMatcherFactory
+{
+  private final VectorValueSelector selector;
+
+  public LongVectorValueMatcher(final VectorValueSelector selector)
+  {
+    this.selector = selector;
+  }
+
+  @Override
+  public VectorValueMatcher makeMatcher(@Nullable final String value)
+  {
+    final Long matchVal = DimensionHandlerUtils.convertObjectToLong(value);
+
+    if (matchVal == null) {
+      return BooleanVectorValueMatcher.of(selector, false);
+    }
+
+    final long matchValLong = matchVal;
+
+    return new BaseVectorValueMatcher(selector)
+    {
+      final VectorMatch match = VectorMatch.wrap(new int[selector.getMaxVectorSize()]);
+
+      @Override
+      public ReadableVectorMatch match(final ReadableVectorMatch mask)
+      {
+        final long[] vector = selector.getLongVector();
+        final int[] selection = match.getSelection();
+
+        int numRows = 0;
+
+        for (int i = 0; i < mask.getSelectionSize(); i++) {
+          final int rowNum = mask.getSelection()[i];
+          if (vector[rowNum] == matchValLong) {
+            selection[numRows++] = rowNum;
+          }
+        }
+
+        match.setSelectionSize(numRows);
+        assert match.isValid(mask);
+        return match;
+      }
+    };
+  }
+
+  @Override
+  public VectorValueMatcher makeMatcher(final DruidPredicateFactory predicateFactory)
+  {
+    final DruidLongPredicate predicate = predicateFactory.makeLongPredicate();
+
+    return new BaseVectorValueMatcher(selector)
+    {
+      final VectorMatch match = VectorMatch.wrap(new int[selector.getMaxVectorSize()]);
+
+      @Override
+      public ReadableVectorMatch match(final ReadableVectorMatch mask)
+      {
+        final long[] vector = selector.getLongVector();
+        final int[] selection = match.getSelection();
+
+        int numRows = 0;
+
+        for (int i = 0; i < mask.getSelectionSize(); i++) {
+          final int rowNum = mask.getSelection()[i];
+          if (predicate.applyLong(vector[rowNum])) {
+            selection[numRows++] = rowNum;
+          }
+        }
+
+        match.setSelectionSize(numRows);
+        assert match.isValid(mask);
+        return match;
+      }
+    };
+  }
+}
diff --git a/processing/src/main/java/org/apache/druid/query/filter/vector/MultiValueStringVectorValueMatcher.java b/processing/src/main/java/org/apache/druid/query/filter/vector/MultiValueStringVectorValueMatcher.java
new file mode 100644
index 0000000..d192673
--- /dev/null
+++ b/processing/src/main/java/org/apache/druid/query/filter/vector/MultiValueStringVectorValueMatcher.java
@@ -0,0 +1,208 @@
+/*
+ * 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.query.filter.vector;
+
+import com.google.common.base.Predicate;
+import org.apache.druid.common.config.NullHandling;
+import org.apache.druid.query.filter.DruidPredicateFactory;
+import org.apache.druid.segment.IdLookup;
+import org.apache.druid.segment.data.IndexedInts;
+import org.apache.druid.segment.vector.MultiValueDimensionVectorSelector;
+
+import javax.annotation.Nullable;
+import java.util.BitSet;
+import java.util.Objects;
+
+public class MultiValueStringVectorValueMatcher implements VectorValueMatcherFactory
+{
+  private final MultiValueDimensionVectorSelector selector;
+
+  public MultiValueStringVectorValueMatcher(final MultiValueDimensionVectorSelector selector)
+  {
+    this.selector = selector;
+  }
+
+  @Override
+  public VectorValueMatcher makeMatcher(@Nullable final String value)
+  {
+    final String etnValue = NullHandling.emptyToNullIfNeeded(NullHandling.emptyToNullIfNeeded(value));
+    final IdLookup idLookup = selector.idLookup();
+    final int id;
+
+    if (idLookup != null) {
+      // Optimization when names can be looked up to IDs ahead of time.
+      id = idLookup.lookupId(etnValue);
+
+      if (id < 0) {
+        // Value doesn't exist in this column.
+        return BooleanVectorValueMatcher.of(selector, false);
+      }
+
+      // Check for "id".
+      return new BaseVectorValueMatcher(selector)
+      {
+        final VectorMatch match = VectorMatch.wrap(new int[selector.getMaxVectorSize()]);
+
+        @Override
+        public ReadableVectorMatch match(final ReadableVectorMatch mask)
+        {
+          final IndexedInts[] vector = selector.getRowVector();
+          final int[] selection = match.getSelection();
+
+          int numRows = 0;
+
+          for (int i = 0; i < mask.getSelectionSize(); i++) {
+            final int rowNum = mask.getSelection()[i];
+            final IndexedInts ints = vector[rowNum];
+            final int n = ints.size();
+
+            if (n == 0) {
+              // null should match empty rows in multi-value columns
+              if (etnValue == null) {
+                selection[numRows++] = rowNum;
+              }
+            } else {
+              for (int j = 0; j < n; j++) {
+                if (ints.get(j) == id) {
+                  selection[numRows++] = rowNum;
+                  break;
+                }
+              }
+            }
+          }
+
+          match.setSelectionSize(numRows);
+          assert match.isValid(mask);
+          return match;
+        }
+      };
+    } else {
+      return makeMatcher(s -> Objects.equals(s, etnValue));
+    }
+  }
+
+  @Override
+  public VectorValueMatcher makeMatcher(final DruidPredicateFactory predicateFactory)
+  {
+    return makeMatcher(predicateFactory.makeStringPredicate());
+  }
+
+  private VectorValueMatcher makeMatcher(final Predicate<String> predicate)
+  {
+    final boolean matchNull = predicate.apply(null);
+
+    if (selector.getValueCardinality() > 0) {
+      final BitSet checkedIds = new BitSet(selector.getValueCardinality());
+      final BitSet matchingIds = new BitSet(selector.getValueCardinality());
+
+      // Lazy matcher; only check an id if matches() is called.
+      return new BaseVectorValueMatcher(selector)
+      {
+        private final VectorMatch match = VectorMatch.wrap(new int[selector.getMaxVectorSize()]);
+
+        @Override
+        public ReadableVectorMatch match(final ReadableVectorMatch mask)
+        {
+          final IndexedInts[] vector = selector.getRowVector();
+          final int[] selection = match.getSelection();
+
+          int numRows = 0;
+
+          for (int i = 0; i < mask.getSelectionSize(); i++) {
+            final int rowNum = mask.getSelection()[i];
+            final IndexedInts ints = vector[rowNum];
+            final int n = ints.size();
+
+            if (n == 0) {
+              // null should match empty rows in multi-value columns
+              if (matchNull) {
+                selection[numRows++] = rowNum;
+              }
+            } else {
+              for (int j = 0; j < n; j++) {
+                final int id = ints.get(j);
+                final boolean matches;
+
+                if (checkedIds.get(id)) {
+                  matches = matchingIds.get(id);
+                } else {
+                  matches = predicate.apply(selector.lookupName(id));
+                  checkedIds.set(id);
+                  if (matches) {
+                    matchingIds.set(id);
+                  }
+                }
+
+                if (matches) {
+                  selection[numRows++] = rowNum;
+                  break;
+                }
+              }
+            }
+          }
+
+          match.setSelectionSize(numRows);
+          assert match.isValid(mask);
+          return match;
+        }
+      };
+    } else {
+      // Evaluate "lookupName" and "predicate" on every row.
+      return new BaseVectorValueMatcher(selector)
+      {
+        final VectorMatch match = VectorMatch.wrap(new int[selector.getMaxVectorSize()]);
+
+        @Override
+        public ReadableVectorMatch match(final ReadableVectorMatch mask)
+        {
+          final IndexedInts[] vector = selector.getRowVector();
+          final int[] selection = match.getSelection();
+
+          int numRows = 0;
+
+          for (int i = 0; i < mask.getSelectionSize(); i++) {
+            final int rowNum = mask.getSelection()[i];
+            final IndexedInts ints = vector[rowNum];
+            final int n = ints.size();
+
+            if (n == 0) {
+              // null should match empty rows in multi-value columns
+              if (matchNull) {
+                selection[numRows++] = rowNum;
+              }
+            } else {
+              for (int j = 0; j < n; j++) {
+                final int id = ints.get(j);
+                if (predicate.apply(selector.lookupName(id))) {
+                  selection[numRows++] = rowNum;
+                  break;
+                }
+              }
+            }
+          }
+
+          match.setSelectionSize(numRows);
+          assert match.isValid(mask);
+          return match;
+        }
+      };
+    }
+  }
+}
diff --git a/processing/src/main/java/org/apache/druid/query/filter/vector/ReadableVectorMatch.java b/processing/src/main/java/org/apache/druid/query/filter/vector/ReadableVectorMatch.java
new file mode 100644
index 0000000..88cbbf3
--- /dev/null
+++ b/processing/src/main/java/org/apache/druid/query/filter/vector/ReadableVectorMatch.java
@@ -0,0 +1,68 @@
+/*
+ * 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.query.filter.vector;
+
+import javax.annotation.Nullable;
+
+/**
+ * The result of calling {@link VectorValueMatcher#match}.
+ *
+ * @see VectorMatch, the implementation, which also adds some extra mutation methods.
+ */
+public interface ReadableVectorMatch
+{
+  /**
+   * Returns an array of indexes into the current batch. Only the first "getSelectionSize" are valid.
+   *
+   * Even though this array is technically mutable, it is very poor form to mutate it if you are not the owner of the
+   * VectorMatch object. The reason we use a mutable array here instead of positional getter methods, by the way, is in
+   * the hopes of keeping access to the selection vector as low-level and optimizable as possible. Potential
+   * optimizations could include making it easier for the JVM to use CPU-level vectorization, avoid method calls, etc.
+   */
+  int[] getSelection();
+
+  /**
+   * Returns the number of valid values in the array from "getSelection".
+   */
+  int getSelectionSize();
+
+  /**
+   * Checks if this match has accepted every row in the vector.
+   *
+   * @param vectorSize the current vector size; must be passed in since VectorMatch objects do not "know" the size
+   *                   of the vector they came from.
+   */
+  boolean isAllTrue(int vectorSize);
+
+  /**
+   * Checks if this match has accepted *nothing*.
+   */
+  boolean isAllFalse();
+
+  /**
+   * Checks if this match is valid (increasing row numbers, no out-of-range row numbers). Can additionally verify
+   * that the match is a subset of a provided "mask".
+   *
+   * Used by assertions and tests.
+   *
+   * @param mask if provided, checks if this match is a subset of the mask.
+   */
+  boolean isValid(@Nullable ReadableVectorMatch mask);
+}
diff --git a/processing/src/main/java/org/apache/druid/query/filter/vector/SingleValueStringVectorValueMatcher.java b/processing/src/main/java/org/apache/druid/query/filter/vector/SingleValueStringVectorValueMatcher.java
new file mode 100644
index 0000000..6ed7c16
--- /dev/null
+++ b/processing/src/main/java/org/apache/druid/query/filter/vector/SingleValueStringVectorValueMatcher.java
@@ -0,0 +1,192 @@
+/*
+ * 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.query.filter.vector;
+
+import com.google.common.base.Predicate;
+import org.apache.druid.common.config.NullHandling;
+import org.apache.druid.query.filter.DruidPredicateFactory;
+import org.apache.druid.query.filter.StringValueMatcherColumnSelectorStrategy;
+import org.apache.druid.segment.IdLookup;
+import org.apache.druid.segment.vector.SingleValueDimensionVectorSelector;
+
+import javax.annotation.Nullable;
+import java.util.BitSet;
+import java.util.Objects;
+
+public class SingleValueStringVectorValueMatcher implements VectorValueMatcherFactory
+{
+  private final SingleValueDimensionVectorSelector selector;
+
+  public SingleValueStringVectorValueMatcher(final SingleValueDimensionVectorSelector selector)
+  {
+    this.selector = selector;
+  }
+
+  @Nullable
+  private static BooleanVectorValueMatcher toBooleanMatcherIfPossible(
+      final SingleValueDimensionVectorSelector selector,
+      final Predicate<String> predicate
+  )
+  {
+    final Boolean booleanValue = StringValueMatcherColumnSelectorStrategy.toBooleanIfPossible(
+        selector,
+        false,
+        predicate
+    );
+
+    return booleanValue == null ? null : BooleanVectorValueMatcher.of(selector, booleanValue);
+  }
+
+  @Override
+  public VectorValueMatcher makeMatcher(@Nullable final String value)
+  {
+    final String etnValue = NullHandling.emptyToNullIfNeeded(value);
+
+    final VectorValueMatcher booleanMatcher = toBooleanMatcherIfPossible(selector, s -> Objects.equals(s, etnValue));
+    if (booleanMatcher != null) {
+      return booleanMatcher;
+    }
+
+    final IdLookup idLookup = selector.idLookup();
+    final int id;
+
+    if (idLookup != null) {
+      // Optimization when names can be looked up to IDs ahead of time.
+      id = idLookup.lookupId(etnValue);
+
+      if (id < 0) {
+        // Value doesn't exist in this column.
+        return BooleanVectorValueMatcher.of(selector, false);
+      }
+
+      // Check for "id".
+      return new BaseVectorValueMatcher(selector)
+      {
+        final VectorMatch match = VectorMatch.wrap(new int[selector.getMaxVectorSize()]);
+
+        @Override
+        public ReadableVectorMatch match(final ReadableVectorMatch mask)
+        {
+          final int[] vector = selector.getRowVector();
+          final int[] selection = match.getSelection();
+
+          int numRows = 0;
+
+          for (int i = 0; i < mask.getSelectionSize(); i++) {
+            final int rowNum = mask.getSelection()[i];
+            if (vector[rowNum] == id) {
+              selection[numRows++] = rowNum;
+            }
+          }
+
+          match.setSelectionSize(numRows);
+          assert match.isValid(mask);
+          return match;
+        }
+      };
+    } else {
+      return makeMatcher(s -> Objects.equals(s, etnValue));
+    }
+  }
+
+  @Override
+  public VectorValueMatcher makeMatcher(final DruidPredicateFactory predicateFactory)
+  {
+    return makeMatcher(predicateFactory.makeStringPredicate());
+  }
+
+  private VectorValueMatcher makeMatcher(final Predicate<String> predicate)
+  {
+    final VectorValueMatcher booleanMatcher = toBooleanMatcherIfPossible(selector, predicate);
+    if (booleanMatcher != null) {
+      return booleanMatcher;
+    }
+
+    if (selector.getValueCardinality() > 0) {
+      final BitSet checkedIds = new BitSet(selector.getValueCardinality());
+      final BitSet matchingIds = new BitSet(selector.getValueCardinality());
+
+      // Lazy matcher; only check an id if matches() is called.
+      return new BaseVectorValueMatcher(selector)
+      {
+        private final VectorMatch match = VectorMatch.wrap(new int[selector.getMaxVectorSize()]);
+
+        @Override
+        public ReadableVectorMatch match(final ReadableVectorMatch mask)
+        {
+          final int[] vector = selector.getRowVector();
+          final int[] selection = match.getSelection();
+
+          int numRows = 0;
+
+          for (int i = 0; i < mask.getSelectionSize(); i++) {
+            final int rowNum = mask.getSelection()[i];
+            final int id = vector[rowNum];
+            final boolean matches;
+
+            if (checkedIds.get(id)) {
+              matches = matchingIds.get(id);
+            } else {
+              matches = predicate.apply(selector.lookupName(id));
+              checkedIds.set(id);
+              if (matches) {
+                matchingIds.set(id);
+              }
+            }
+
+            if (matches) {
+              selection[numRows++] = rowNum;
+            }
+          }
+
+          match.setSelectionSize(numRows);
+          assert match.isValid(mask);
+          return match;
+        }
+      };
+    } else {
+      // Evaluate "lookupName" and "predicate" on every row.
+      return new BaseVectorValueMatcher(selector)
+      {
+        final VectorMatch match = VectorMatch.wrap(new int[selector.getMaxVectorSize()]);
+
+        @Override
+        public ReadableVectorMatch match(final ReadableVectorMatch mask)
+        {
+          final int[] vector = selector.getRowVector();
+          final int[] selection = match.getSelection();
+
+          int numRows = 0;
+
+          for (int i = 0; i < mask.getSelectionSize(); i++) {
+            final int rowNum = mask.getSelection()[i];
+            if (predicate.apply(selector.lookupName(vector[rowNum]))) {
+              selection[numRows++] = rowNum;
+            }
+          }
+
+          match.setSelectionSize(numRows);
+          assert match.isValid(mask);
+          return match;
+        }
+      };
+    }
+  }
+}
diff --git a/processing/src/main/java/org/apache/druid/query/filter/vector/VectorMatch.java b/processing/src/main/java/org/apache/druid/query/filter/vector/VectorMatch.java
new file mode 100644
index 0000000..54a6947
--- /dev/null
+++ b/processing/src/main/java/org/apache/druid/query/filter/vector/VectorMatch.java
@@ -0,0 +1,267 @@
+/*
+ * 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.query.filter.vector;
+
+import com.google.common.base.Preconditions;
+import org.apache.druid.segment.QueryableIndexStorageAdapter;
+
+import javax.annotation.Nullable;
+
+/**
+ * Implementation class for ReadableVectorMatch.
+ *
+ * Also adds some useful methods, like "addAll", "removeAll", and "copyFrom".
+ */
+public class VectorMatch implements ReadableVectorMatch
+{
+  private static final int[] DEFAULT_ALL_TRUE_VECTOR = new int[QueryableIndexStorageAdapter.DEFAULT_VECTOR_SIZE];
+
+  private static final VectorMatch ALL_FALSE = new VectorMatch(new int[0], 0);
+
+  static {
+    for (int i = 0; i < DEFAULT_ALL_TRUE_VECTOR.length; i++) {
+      DEFAULT_ALL_TRUE_VECTOR[i] = i;
+    }
+  }
+
+  private final int[] selection;
+  private int selectionSize;
+
+  private VectorMatch(final int[] selection, final int selectionSize)
+  {
+    this.selection = selection;
+    this.selectionSize = selectionSize;
+  }
+
+  /**
+   * Creates a match that matches everything up to "numRows". This will often be the current vector size, but
+   * does not necessarily have to be.
+   */
+  public static ReadableVectorMatch allTrue(final int numRows)
+  {
+    if (numRows <= DEFAULT_ALL_TRUE_VECTOR.length) {
+      return new VectorMatch(DEFAULT_ALL_TRUE_VECTOR, numRows);
+    } else {
+      final int[] selection = new int[numRows];
+
+      for (int i = 0; i < numRows; i++) {
+        selection[i] = i;
+      }
+
+      return new VectorMatch(selection, numRows);
+    }
+  }
+
+  /**
+   * Creates a match that matches nothing.
+   */
+  public static ReadableVectorMatch allFalse()
+  {
+    return ALL_FALSE;
+  }
+
+  /**
+   * Creates a new match object with selectionSize = 0, and the provided array as a backing array.
+   */
+  public static VectorMatch wrap(final int[] selection)
+  {
+    return new VectorMatch(selection, 0);
+  }
+
+  @Override
+  public boolean isAllTrue(final int vectorSize)
+  {
+    return selectionSize == vectorSize;
+  }
+
+  @Override
+  public boolean isAllFalse()
+  {
+    return selectionSize == 0;
+  }
+
+  @Override
+  public boolean isValid(@Nullable final ReadableVectorMatch mask)
+  {
+    if (mask != null && !mask.isValid(null)) {
+      // Invalid mask.
+      return false;
+    }
+
+    // row numbers must be increasing.
+    int rowNum = -1;
+    for (int i = 0; i < selectionSize; i++) {
+      if (selection[i] > rowNum) {
+        rowNum = selection[i];
+      } else {
+        return false;
+      }
+    }
+
+    // row number cannot be larger than the max length of the selection vector.
+    if (rowNum > selection.length) {
+      return false;
+    }
+
+    // row numbers must all be present in the mask, if it exists.
+    if (mask != null) {
+      final int[] maskArray = mask.getSelection();
+      for (int i = 0, j = 0; i < selectionSize; i++) {
+        while (j < mask.getSelectionSize() && selection[i] > maskArray[j]) {
+          j++;
+        }
+
+        if (j >= mask.getSelectionSize() || selection[i] != maskArray[j]) {
+          return false;
+        }
+      }
+    }
+
+    return true;
+  }
+
+  /**
+   * Removes all rows from this object that occur in "other", in place, and returns a reference to this object. Does
+   * not modify "other".
+   */
+  public VectorMatch removeAll(final ReadableVectorMatch other)
+  {
+    //noinspection ObjectEquality
+    Preconditions.checkState(this != other, "'other' must be a different instance from 'this'");
+
+    int i = 0; // reading position in this.selection
+    int j = 0; // writing position in this.selection
+    int p = 0; // position in otherSelection
+    final int[] otherSelection = other.getSelection();
+    for (; i < selectionSize; i++) {
+      while (p < other.getSelectionSize() && otherSelection[p] < selection[i]) {
+        // Other value < selection[i], keep reading in other so we can see if selection[i] should be preserved or not.
+        p++;
+      }
+
+      if (!(p < other.getSelectionSize() && otherSelection[p] == selection[i])) {
+        // Preserve selection[i].
+        selection[j++] = selection[i];
+      }
+    }
+    selectionSize = j;
+    assert isValid(null);
+    return this;
+  }
+
+  /**
+   * Adds all rows from "other" to this object, using "scratch" as scratch space if needed. Does not modify "other".
+   * Returns a reference to this object.
+   */
+  public VectorMatch addAll(final ReadableVectorMatch other, final VectorMatch scratch)
+  {
+    //noinspection ObjectEquality
+    Preconditions.checkState(this != scratch, "'scratch' must be a different instance from 'this'");
+    //noinspection ObjectEquality
+    Preconditions.checkState(other != scratch, "'scratch' must be a different instance from 'other'");
+
+    final int[] scratchSelection = scratch.getSelection();
+    final int[] otherSelection = other.getSelection();
+
+    int i = 0; // this.selection pointer
+    int j = 0; // otherSelection pointer
+    int k = 0; // scratchSelection pointer
+
+    for (; i < selectionSize; i++) {
+      while (j < other.getSelectionSize() && otherSelection[j] < selection[i]) {
+        scratchSelection[k++] = otherSelection[j++];
+      }
+
+      scratchSelection[k++] = selection[i];
+
+      if (j < other.getSelectionSize() && otherSelection[j] == selection[i]) {
+        j++;
+      }
+    }
+
+    while (j < other.getSelectionSize()) {
+      scratchSelection[k++] = otherSelection[j++];
+    }
+
+    scratch.setSelectionSize(k);
+    copyFrom(scratch);
+    assert isValid(null);
+    return this;
+  }
+
+  /**
+   * Copies "other" into this object, and returns a reference to this object. Does not modify "other".
+   */
+  public VectorMatch copyFrom(final ReadableVectorMatch other)
+  {
+    Preconditions.checkState(
+        selection.length >= other.getSelectionSize(),
+        "Capacity[%s] cannot fit other match's selectionSize[%s]",
+        selection.length,
+        other.getSelectionSize()
+    );
+    System.arraycopy(other.getSelection(), 0, selection, 0, other.getSelectionSize());
+    selectionSize = other.getSelectionSize();
+    assert isValid(null);
+    return this;
+  }
+
+  @Override
+  public int[] getSelection()
+  {
+    return selection;
+  }
+
+  @Override
+  public int getSelectionSize()
+  {
+    return selectionSize;
+  }
+
+  /**
+   * Sets the valid selectionSize, and returns a reference to this object.
+   */
+  public VectorMatch setSelectionSize(final int newSelectionSize)
+  {
+    Preconditions.checkArgument(
+        newSelectionSize <= selection.length,
+        "Oops! Cannot setSelectionSize[%s] > selection.length[%s].",
+        newSelectionSize,
+        selection.length
+    );
+    this.selectionSize = newSelectionSize;
+    assert isValid(null);
+    return this;
+  }
+
+  @Override
+  public String toString()
+  {
+    final StringBuilder retVal = new StringBuilder("[");
+    for (int i = 0; i < selectionSize; i++) {
+      if (i > 0) {
+        retVal.append(", ");
+      }
+      retVal.append(selection[i]);
+    }
+    retVal.append("]");
+    return retVal.toString();
+  }
+}
diff --git a/processing/src/main/java/org/apache/druid/query/filter/vector/VectorValueMatcher.java b/processing/src/main/java/org/apache/druid/query/filter/vector/VectorValueMatcher.java
new file mode 100644
index 0000000..2421661
--- /dev/null
+++ b/processing/src/main/java/org/apache/druid/query/filter/vector/VectorValueMatcher.java
@@ -0,0 +1,42 @@
+/*
+ * 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.query.filter.vector;
+
+import org.apache.druid.query.filter.Filter;
+import org.apache.druid.segment.vector.VectorSizeInspector;
+
+/**
+ * An object that returns a boolean indicating if the "current" row should be selected or not. The most prominent use
+ * of this interface is that it is returned by the {@link Filter} "makeVectorMatcher" method, where it is used to
+ * identify selected rows for filtered cursors and filtered aggregators.
+ *
+ * @see org.apache.druid.query.filter.ValueMatcher, the non-vectorized version
+ */
+public interface VectorValueMatcher extends VectorSizeInspector
+{
+  /**
+   * Examine the current vector and return a match indicating what is accepted.
+   *
+   * @param mask must not be null; use {@link VectorMatch#allTrue} if you don't need a mask.
+   *
+   * @return the subset of "mask" that this value matcher accepts
+   */
+  ReadableVectorMatch match(ReadableVectorMatch mask);
+}
diff --git a/processing/src/main/java/org/apache/druid/query/filter/vector/VectorValueMatcherColumnStrategizer.java b/processing/src/main/java/org/apache/druid/query/filter/vector/VectorValueMatcherColumnStrategizer.java
new file mode 100644
index 0000000..7b970e2
--- /dev/null
+++ b/processing/src/main/java/org/apache/druid/query/filter/vector/VectorValueMatcherColumnStrategizer.java
@@ -0,0 +1,74 @@
+/*
+ * 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.query.filter.vector;
+
+import org.apache.druid.query.dimension.VectorColumnStrategizer;
+import org.apache.druid.segment.vector.MultiValueDimensionVectorSelector;
+import org.apache.druid.segment.vector.SingleValueDimensionVectorSelector;
+import org.apache.druid.segment.vector.VectorValueSelector;
+
+public class VectorValueMatcherColumnStrategizer implements VectorColumnStrategizer<VectorValueMatcherFactory>
+{
+  private static final VectorValueMatcherColumnStrategizer INSTANCE = new VectorValueMatcherColumnStrategizer();
+
+  private VectorValueMatcherColumnStrategizer()
+  {
+    // Singleton.
+  }
+
+  public static VectorValueMatcherColumnStrategizer instance()
+  {
+    return INSTANCE;
+  }
+
+  @Override
+  public VectorValueMatcherFactory makeSingleValueDimensionStrategy(
+      final SingleValueDimensionVectorSelector selector
+  )
+  {
+    return new SingleValueStringVectorValueMatcher(selector);
+  }
+
+  @Override
+  public VectorValueMatcherFactory makeMultiValueDimensionStrategy(
+      final MultiValueDimensionVectorSelector selector
+  )
+  {
+    return new MultiValueStringVectorValueMatcher(selector);
+  }
+
+  @Override
+  public VectorValueMatcherFactory makeFloatStrategy(final VectorValueSelector selector)
+  {
+    return new FloatVectorValueMatcher(selector);
+  }
+
+  @Override
+  public VectorValueMatcherFactory makeDoubleStrategy(final VectorValueSelector selector)
+  {
+    return new DoubleVectorValueMatcher(selector);
+  }
+
+  @Override
+  public VectorValueMatcherFactory makeLongStrategy(final VectorValueSelector selector)
+  {
+    return new LongVectorValueMatcher(selector);
+  }
+}
diff --git a/processing/src/main/java/org/apache/druid/segment/column/BaseColumn.java b/processing/src/main/java/org/apache/druid/query/filter/vector/VectorValueMatcherFactory.java
similarity index 72%
copy from processing/src/main/java/org/apache/druid/segment/column/BaseColumn.java
copy to processing/src/main/java/org/apache/druid/query/filter/vector/VectorValueMatcherFactory.java
index d692725..a7971eb 100644
--- a/processing/src/main/java/org/apache/druid/segment/column/BaseColumn.java
+++ b/processing/src/main/java/org/apache/druid/query/filter/vector/VectorValueMatcherFactory.java
@@ -17,14 +17,15 @@
  * under the License.
  */
 
-package org.apache.druid.segment.column;
+package org.apache.druid.query.filter.vector;
 
-import org.apache.druid.segment.ColumnValueSelector;
-import org.apache.druid.segment.data.ReadableOffset;
+import org.apache.druid.query.filter.DruidPredicateFactory;
 
-import java.io.Closeable;
+import javax.annotation.Nullable;
 
-public interface BaseColumn extends Closeable
+public interface VectorValueMatcherFactory
 {
-  ColumnValueSelector<?> makeColumnValueSelector(ReadableOffset offset);
+  VectorValueMatcher makeMatcher(@Nullable String value);
+
+  VectorValueMatcher makeMatcher(DruidPredicateFactory predicateFactory);
 }
diff --git a/processing/src/main/java/org/apache/druid/query/groupby/GroupByQuery.java b/processing/src/main/java/org/apache/druid/query/groupby/GroupByQuery.java
index 8ab4601..29ec5ec 100644
--- a/processing/src/main/java/org/apache/druid/query/groupby/GroupByQuery.java
+++ b/processing/src/main/java/org/apache/druid/query/groupby/GroupByQuery.java
@@ -1077,6 +1077,7 @@ public class GroupByQuery extends BaseQuery<Row>
            ", aggregatorSpecs=" + aggregatorSpecs +
            ", postAggregatorSpecs=" + postAggregatorSpecs +
            ", havingSpec=" + havingSpec +
+           ", context=" + getContext() +
            '}';
   }
 
diff --git a/processing/src/main/java/org/apache/druid/query/groupby/GroupByQueryConfig.java b/processing/src/main/java/org/apache/druid/query/groupby/GroupByQueryConfig.java
index 2ec0025..13107fb 100644
--- a/processing/src/main/java/org/apache/druid/query/groupby/GroupByQueryConfig.java
+++ b/processing/src/main/java/org/apache/druid/query/groupby/GroupByQueryConfig.java
@@ -42,6 +42,7 @@ public class GroupByQueryConfig
   private static final String CTX_KEY_FORCE_HASH_AGGREGATION = "forceHashAggregation";
   private static final String CTX_KEY_INTERMEDIATE_COMBINE_DEGREE = "intermediateCombineDegree";
   private static final String CTX_KEY_NUM_PARALLEL_COMBINE_THREADS = "numParallelCombineThreads";
+  private static final String CTX_KEY_VECTORIZE = "vectorize";
 
   @JsonProperty
   private String defaultStrategy = GroupByStrategySelector.STRATEGY_V2;
@@ -88,6 +89,9 @@ public class GroupByQueryConfig
   @JsonProperty
   private int numParallelCombineThreads = 1;
 
+  @JsonProperty
+  private boolean vectorize = false;
+
   public String getDefaultStrategy()
   {
     return defaultStrategy;
@@ -168,6 +172,11 @@ public class GroupByQueryConfig
     return numParallelCombineThreads;
   }
 
+  public boolean isVectorize()
+  {
+    return vectorize;
+  }
+
   public boolean isForcePushDownNestedQuery()
   {
     return forcePushDownNestedQuery;
@@ -203,7 +212,10 @@ public class GroupByQueryConfig
         getMaxOnDiskStorage()
     );
     newConfig.maxMergingDictionarySize = Math.min(
-        ((Number) query.getContextValue(CTX_KEY_MAX_MERGING_DICTIONARY_SIZE, getMaxMergingDictionarySize())).longValue(),
+        ((Number) query.getContextValue(
+            CTX_KEY_MAX_MERGING_DICTIONARY_SIZE,
+            getMaxMergingDictionarySize()
+        )).longValue(),
         getMaxMergingDictionarySize()
     );
     newConfig.forcePushDownLimit = query.getContextBoolean(CTX_KEY_FORCE_LIMIT_PUSH_DOWN, isForcePushDownLimit());
@@ -217,6 +229,7 @@ public class GroupByQueryConfig
         CTX_KEY_NUM_PARALLEL_COMBINE_THREADS,
         getNumParallelCombineThreads()
     );
+    newConfig.vectorize = query.getContextBoolean(CTX_KEY_VECTORIZE, isVectorize());
     return newConfig;
   }
 
@@ -237,6 +250,7 @@ public class GroupByQueryConfig
            ", forceHashAggregation=" + forceHashAggregation +
            ", intermediateCombineDegree=" + intermediateCombineDegree +
            ", numParallelCombineThreads=" + numParallelCombineThreads +
+           ", vectorize=" + vectorize +
            ", forcePushDownNestedQuery=" + forcePushDownNestedQuery +
            '}';
   }
diff --git a/processing/src/main/java/org/apache/druid/query/groupby/GroupByQueryEngine.java b/processing/src/main/java/org/apache/druid/query/groupby/GroupByQueryEngine.java
index c8dcef3..1383b20 100644
--- a/processing/src/main/java/org/apache/druid/query/groupby/GroupByQueryEngine.java
+++ b/processing/src/main/java/org/apache/druid/query/groupby/GroupByQueryEngine.java
@@ -44,6 +44,7 @@ import org.apache.druid.query.aggregation.PostAggregator;
 import org.apache.druid.query.dimension.DimensionSpec;
 import org.apache.druid.query.filter.Filter;
 import org.apache.druid.segment.Cursor;
+import org.apache.druid.segment.DimensionDictionarySelector;
 import org.apache.druid.segment.DimensionSelector;
 import org.apache.druid.segment.StorageAdapter;
 import org.apache.druid.segment.column.ValueType;
@@ -331,7 +332,7 @@ public class GroupByQueryEngine
         }
 
         final DimensionSelector selector = cursor.getColumnSelectorFactory().makeDimensionSelector(dimSpec);
-        if (selector.getValueCardinality() == DimensionSelector.CARDINALITY_UNKNOWN) {
+        if (selector.getValueCardinality() == DimensionDictionarySelector.CARDINALITY_UNKNOWN) {
           throw new UnsupportedOperationException(
               "GroupBy v1 does not support dimension selectors with unknown cardinality.");
         }
diff --git a/processing/src/main/java/org/apache/druid/query/groupby/RowBasedColumnSelectorFactory.java b/processing/src/main/java/org/apache/druid/query/groupby/RowBasedColumnSelectorFactory.java
index 081b481..34e5596 100644
--- a/processing/src/main/java/org/apache/druid/query/groupby/RowBasedColumnSelectorFactory.java
+++ b/processing/src/main/java/org/apache/druid/query/groupby/RowBasedColumnSelectorFactory.java
@@ -31,6 +31,7 @@ import org.apache.druid.query.monomorphicprocessing.RuntimeShapeInspector;
 import org.apache.druid.segment.BaseSingleValueDimensionSelector;
 import org.apache.druid.segment.ColumnSelectorFactory;
 import org.apache.druid.segment.ColumnValueSelector;
+import org.apache.druid.segment.DimensionDictionarySelector;
 import org.apache.druid.segment.DimensionHandlerUtils;
 import org.apache.druid.segment.DimensionSelector;
 import org.apache.druid.segment.IdLookup;
@@ -242,7 +243,7 @@ public class RowBasedColumnSelectorFactory implements ColumnSelectorFactory
         @Override
         public int getValueCardinality()
         {
-          return DimensionSelector.CARDINALITY_UNKNOWN;
+          return DimensionDictionarySelector.CARDINALITY_UNKNOWN;
         }
 
         @Override
diff --git a/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/AbstractBufferHashGrouper.java b/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/AbstractBufferHashGrouper.java
index 8ca48eb..aae2e4b 100644
--- a/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/AbstractBufferHashGrouper.java
+++ b/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/AbstractBufferHashGrouper.java
@@ -22,8 +22,7 @@ package org.apache.druid.query.groupby.epinephelinae;
 import com.google.common.base.Supplier;
 import org.apache.druid.java.util.common.IAE;
 import org.apache.druid.java.util.common.logger.Logger;
-import org.apache.druid.query.aggregation.AggregatorFactory;
-import org.apache.druid.query.aggregation.BufferAggregator;
+import org.apache.druid.query.aggregation.AggregatorAdapters;
 
 import java.nio.ByteBuffer;
 
@@ -35,8 +34,8 @@ public abstract class AbstractBufferHashGrouper<KeyType> implements Grouper<KeyT
   protected final Supplier<ByteBuffer> bufferSupplier;
   protected final KeySerde<KeyType> keySerde;
   protected final int keySize;
-  protected final BufferAggregator[] aggregators;
-  protected final int[] aggregatorOffsets;
+  protected final AggregatorAdapters aggregators;
+  protected final int baseAggregatorOffset;
   protected final int bufferGrouperMaxSize; // Integer.MAX_VALUE in production, only used for unit tests
 
   // The load factor and bucket configurations are not final, to allow subclasses to set their own values
@@ -53,15 +52,16 @@ public abstract class AbstractBufferHashGrouper<KeyType> implements Grouper<KeyT
       // the buffer returned from the below supplier can have dirty bits and should be cleared during initialization
       final Supplier<ByteBuffer> bufferSupplier,
       final KeySerde<KeyType> keySerde,
-      final AggregatorFactory[] aggregatorFactories,
+      final AggregatorAdapters aggregators,
+      final int baseAggregatorOffset,
       final int bufferGrouperMaxSize
   )
   {
     this.bufferSupplier = bufferSupplier;
     this.keySerde = keySerde;
     this.keySize = keySerde.keySize();
-    this.aggregators = new BufferAggregator[aggregatorFactories.length];
-    this.aggregatorOffsets = new int[aggregatorFactories.length];
+    this.aggregators = aggregators;
+    this.baseAggregatorOffset = baseAggregatorOffset;
     this.bufferGrouperMaxSize = bufferGrouperMaxSize;
   }
 
@@ -77,8 +77,9 @@ public abstract class AbstractBufferHashGrouper<KeyType> implements Grouper<KeyT
    * Called to check if it's possible to skip aggregation for a row.
    *
    * @param bucketWasUsed Was the row a new entry in the hash table?
-   * @param bucketOffset Offset of the bucket containing this row's entry in the hash table,
-   *                     within the buffer returned by hashTable.getTableBuffer()
+   * @param bucketOffset  Offset of the bucket containing this row's entry in the hash table,
+   *                      within the buffer returned by hashTable.getTableBuffer()
+   *
    * @return true if aggregation can be skipped, false otherwise.
    */
   public abstract boolean canSkipAggregate(boolean bucketWasUsed, int bucketOffset);
@@ -123,7 +124,7 @@ public abstract class AbstractBufferHashGrouper<KeyType> implements Grouper<KeyT
     if (keyBuffer == null) {
       // This may just trigger a spill and get ignored, which is ok. If it bubbles up to the user, the message will
       // be correct.
-      return Groupers.DICTIONARY_FULL;
+      return Groupers.dictionaryFull(0);
     }
 
     if (keyBuffer.remaining() != keySize) {
@@ -135,11 +136,11 @@ public abstract class AbstractBufferHashGrouper<KeyType> implements Grouper<KeyT
     }
 
     // find and try to expand if table is full and find again
-    int bucket = hashTable.findBucketWithAutoGrowth(keyBuffer, keyHash);
+    int bucket = hashTable.findBucketWithAutoGrowth(keyBuffer, keyHash, () -> {});
     if (bucket < 0) {
       // This may just trigger a spill and get ignored, which is ok. If it bubbles up to the user, the message will
       // be correct.
-      return Groupers.HASH_TABLE_FULL;
+      return Groupers.hashTableFull(0);
     }
 
     final int bucketStartOffset = hashTable.getOffsetForBucket(bucket);
@@ -149,10 +150,7 @@ public abstract class AbstractBufferHashGrouper<KeyType> implements Grouper<KeyT
     // Set up key and initialize the aggs if this is a new bucket.
     if (!bucketWasUsed) {
       hashTable.initializeNewBucketKey(bucket, keyBuffer, keyHash);
-      for (int i = 0; i < aggregators.length; i++) {
-        aggregators[i].init(tableBuffer, bucketStartOffset + aggregatorOffsets[i]);
-      }
-
+      aggregators.init(tableBuffer, bucketStartOffset + baseAggregatorOffset);
       newBucketHook(bucketStartOffset);
     }
 
@@ -161,9 +159,7 @@ public abstract class AbstractBufferHashGrouper<KeyType> implements Grouper<KeyT
     }
 
     // Aggregate the current row.
-    for (int i = 0; i < aggregators.length; i++) {
-      aggregators[i].aggregate(tableBuffer, bucketStartOffset + aggregatorOffsets[i]);
-    }
+    aggregators.aggregateBuffered(tableBuffer, bucketStartOffset + baseAggregatorOffset);
 
     afterAggregateHook(bucketStartOffset);
 
@@ -173,23 +169,16 @@ public abstract class AbstractBufferHashGrouper<KeyType> implements Grouper<KeyT
   @Override
   public void close()
   {
-    for (BufferAggregator aggregator : aggregators) {
-      try {
-        aggregator.close();
-      }
-      catch (Exception e) {
-        log.warn(e, "Could not close aggregator [%s], skipping.", aggregator);
-      }
-    }
+    aggregators.close();
   }
 
   protected Entry<KeyType> bucketEntryForOffset(final int bucketOffset)
   {
     final ByteBuffer tableBuffer = hashTable.getTableBuffer();
     final KeyType key = keySerde.fromByteBuffer(tableBuffer, bucketOffset + HASH_SIZE);
-    final Object[] values = new Object[aggregators.length];
-    for (int i = 0; i < aggregators.length; i++) {
-      values[i] = aggregators[i].get(tableBuffer, bucketOffset + aggregatorOffsets[i]);
+    final Object[] values = new Object[aggregators.size()];
+    for (int i = 0; i < aggregators.size(); i++) {
+      values[i] = aggregators.get(tableBuffer, bucketOffset + baseAggregatorOffset, i);
     }
 
     return new Entry<>(key, values);
diff --git a/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/AggregateResult.java b/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/AggregateResult.java
index f1bd493..7ddd552 100644
--- a/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/AggregateResult.java
+++ b/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/AggregateResult.java
@@ -19,13 +19,19 @@
 
 package org.apache.druid.query.groupby.epinephelinae;
 
+import com.google.common.base.Preconditions;
+import org.apache.druid.java.util.common.ISE;
+
+import javax.annotation.Nullable;
 import java.util.Objects;
 
 public class AggregateResult
 {
-  private static final AggregateResult OK = new AggregateResult(true, null);
+  private static final AggregateResult OK = new AggregateResult(0, null);
+
+  private final int count;
 
-  private final boolean ok;
+  @Nullable
   private final String reason;
 
   public static AggregateResult ok()
@@ -33,29 +39,47 @@ public class AggregateResult
     return OK;
   }
 
-  public static AggregateResult failure(final String reason)
+  public static AggregateResult partial(final int count, final String reason)
   {
-    return new AggregateResult(false, reason);
+    return new AggregateResult(count, Preconditions.checkNotNull(reason, "reason"));
   }
 
-  private AggregateResult(final boolean ok, final String reason)
+  private AggregateResult(final int count, @Nullable final String reason)
   {
-    this.ok = ok;
+    Preconditions.checkArgument(count >= 0, "count >= 0");
+    this.count = count;
     this.reason = reason;
   }
 
+  /**
+   * True if all rows have been processed.
+   */
   public boolean isOk()
   {
-    return ok;
+    return reason == null;
   }
 
+  public int getCount()
+  {
+    if (isOk()) {
+      throw new ISE("Cannot call getCount when isOk = true");
+    }
+
+    return count;
+  }
+
+  @Nullable
   public String getReason()
   {
+    if (isOk()) {
+      throw new ISE("Cannot call getReason when isOk = true");
+    }
+
     return reason;
   }
 
   @Override
-  public boolean equals(final Object o)
+  public boolean equals(Object o)
   {
     if (this == o) {
       return true;
@@ -63,22 +87,22 @@ public class AggregateResult
     if (o == null || getClass() != o.getClass()) {
       return false;
     }
-    final AggregateResult that = (AggregateResult) o;
-    return ok == that.ok &&
+    AggregateResult that = (AggregateResult) o;
+    return count == that.count &&
            Objects.equals(reason, that.reason);
   }
 
   @Override
   public int hashCode()
   {
-    return Objects.hash(ok, reason);
+    return Objects.hash(count, reason);
   }
 
   @Override
   public String toString()
   {
     return "AggregateResult{" +
-           "ok=" + ok +
+           "count=" + count +
            ", reason='" + reason + '\'' +
            '}';
   }
diff --git a/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/BufferArrayGrouper.java b/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/BufferArrayGrouper.java
index 7994996..7fd34bf 100644
--- a/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/BufferArrayGrouper.java
+++ b/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/BufferArrayGrouper.java
@@ -21,14 +21,14 @@ package org.apache.druid.query.groupby.epinephelinae;
 
 import com.google.common.base.Preconditions;
 import com.google.common.base.Supplier;
+import org.apache.druid.java.util.common.IAE;
 import org.apache.druid.java.util.common.ISE;
-import org.apache.druid.java.util.common.logger.Logger;
 import org.apache.druid.java.util.common.parsers.CloseableIterator;
+import org.apache.druid.query.aggregation.AggregatorAdapters;
 import org.apache.druid.query.aggregation.AggregatorFactory;
-import org.apache.druid.query.aggregation.BufferAggregator;
 import org.apache.druid.query.groupby.epinephelinae.column.GroupByColumnSelectorStrategy;
-import org.apache.druid.segment.ColumnSelectorFactory;
 
+import java.io.IOException;
 import java.nio.ByteBuffer;
 import java.util.Arrays;
 import java.util.NoSuchElementException;
@@ -47,13 +47,10 @@ import java.util.NoSuchElementException;
  * different segments cannot be currently retrieved, this grouper can be used only when performing per-segment query
  * execution.
  */
-public class BufferArrayGrouper implements IntGrouper
+public class BufferArrayGrouper implements VectorGrouper, IntGrouper
 {
-  private static final Logger LOG = new Logger(BufferArrayGrouper.class);
-
   private final Supplier<ByteBuffer> bufferSupplier;
-  private final BufferAggregator[] aggregators;
-  private final int[] aggregatorOffsets;
+  private final AggregatorAdapters aggregators;
   private final int cardinalityWithMissingValue;
   private final int recordSize; // size of all aggregated values
 
@@ -61,6 +58,10 @@ public class BufferArrayGrouper implements IntGrouper
   private ByteBuffer usedFlagBuffer;
   private ByteBuffer valBuffer;
 
+  // Scratch objects used by aggregateVector(). Only set if initVectorized() is called.
+  private int[] vAggregationPositions = null;
+  private int[] vAggregationRows = null;
+
   static long requiredBufferCapacity(
       int cardinality,
       AggregatorFactory[] aggregatorFactories
@@ -72,7 +73,7 @@ public class BufferArrayGrouper implements IntGrouper
                                  .sum();
 
     return getUsedFlagBufferCapacity(cardinalityWithMissingValue) +  // total used flags size
-        (long) cardinalityWithMissingValue * recordSize;                 // total values size
+           (long) cardinalityWithMissingValue * recordSize;                 // total values size
   }
 
   /**
@@ -86,26 +87,17 @@ public class BufferArrayGrouper implements IntGrouper
   public BufferArrayGrouper(
       // the buffer returned from the below supplier can have dirty bits and should be cleared during initialization
       final Supplier<ByteBuffer> bufferSupplier,
-      final ColumnSelectorFactory columnSelectorFactory,
-      final AggregatorFactory[] aggregatorFactories,
+      final AggregatorAdapters aggregators,
       final int cardinality
   )
   {
-    Preconditions.checkNotNull(aggregatorFactories, "aggregatorFactories");
+    Preconditions.checkNotNull(aggregators, "aggregators");
     Preconditions.checkArgument(cardinality > 0, "Cardinality must a non-zero positive number");
 
     this.bufferSupplier = Preconditions.checkNotNull(bufferSupplier, "bufferSupplier");
-    this.aggregators = new BufferAggregator[aggregatorFactories.length];
-    this.aggregatorOffsets = new int[aggregatorFactories.length];
+    this.aggregators = aggregators;
     this.cardinalityWithMissingValue = cardinality + 1;
-
-    int offset = 0;
-    for (int i = 0; i < aggregatorFactories.length; i++) {
-      aggregators[i] = aggregatorFactories[i].factorizeBuffered(columnSelectorFactory);
-      aggregatorOffsets[i] = offset;
-      offset += aggregatorFactories[i].getMaxIntermediateSizeWithNulls();
-    }
-    recordSize = offset;
+    this.recordSize = aggregators.spaceNeeded();
   }
 
   @Override
@@ -115,6 +107,20 @@ public class BufferArrayGrouper implements IntGrouper
       final ByteBuffer buffer = bufferSupplier.get();
 
       final int usedFlagBufferEnd = getUsedFlagBufferCapacity(cardinalityWithMissingValue);
+
+      // Sanity check on buffer capacity.
+      if (usedFlagBufferEnd + (long) cardinalityWithMissingValue * recordSize > buffer.capacity()) {
+        // Should not happen in production, since we should only select array-based aggregation if we have
+        // enough scratch space.
+        throw new ISE(
+            "Records of size[%,d] and possible cardinality[%,d] exceeds the buffer capacity[%,d].",
+            recordSize,
+            cardinalityWithMissingValue,
+            valBuffer.capacity()
+        );
+      }
+
+      // Slice up the buffer.
       buffer.position(0);
       buffer.limit(usedFlagBufferEnd);
       usedFlagBuffer = buffer.slice();
@@ -130,13 +136,22 @@ public class BufferArrayGrouper implements IntGrouper
   }
 
   @Override
+  public void initVectorized(final int maxVectorSize)
+  {
+    init();
+
+    this.vAggregationPositions = new int[maxVectorSize];
+    this.vAggregationRows = new int[maxVectorSize];
+  }
+
+  @Override
   public boolean isInitialized()
   {
     return initialized;
   }
 
   @Override
-  public AggregateResult aggregateKeyHash(int dimIndex)
+  public AggregateResult aggregateKeyHash(final int dimIndex)
   {
     Preconditions.checkArgument(
         dimIndex >= 0 && dimIndex < cardinalityWithMissingValue,
@@ -144,39 +159,62 @@ public class BufferArrayGrouper implements IntGrouper
         dimIndex
     );
 
-    final int recordOffset = dimIndex * recordSize;
+    initializeSlotIfNeeded(dimIndex);
+    aggregators.aggregateBuffered(valBuffer, dimIndex * recordSize);
+    return AggregateResult.ok();
+  }
+
+  @Override
+  public AggregateResult aggregateVector(int[] keySpace, int startRow, int endRow)
+  {
+    if (keySpace.length == 0) {
+      // Empty key space, assume keys are all zeroes.
+      final int dimIndex = 1;
+
+      initializeSlotIfNeeded(dimIndex);
 
-    if (recordOffset + recordSize > valBuffer.capacity()) {
-      // This error cannot be recoverd, and the query must fail
-      throw new ISE(
-          "A record of size [%d] cannot be written to the array buffer at offset[%d] "
-          + "because it exceeds the buffer capacity[%d]. Try increasing druid.processing.buffer.sizeBytes",
-          recordSize,
-          recordOffset,
-          valBuffer.capacity()
+      aggregators.aggregateVector(
+          valBuffer,
+          dimIndex * recordSize,
+          startRow,
+          endRow
       );
-    }
+    } else {
+      final int numRows = endRow - startRow;
 
-    if (!isUsedSlot(dimIndex)) {
-      initializeSlot(dimIndex);
-    }
+      for (int i = 0; i < numRows; i++) {
+        // +1 matches what hashFunction() would do.
+        final int dimIndex = keySpace[i] + 1;
+
+        if (dimIndex < 0 || dimIndex >= cardinalityWithMissingValue) {
+          throw new IAE("Invalid dimIndex[%s]", dimIndex);
+        }
+
+        vAggregationPositions[i] = dimIndex * recordSize;
 
-    for (int i = 0; i < aggregators.length; i++) {
-      aggregators[i].aggregate(valBuffer, recordOffset + aggregatorOffsets[i]);
+        initializeSlotIfNeeded(dimIndex);
+      }
+
+      aggregators.aggregateVector(
+          valBuffer,
+          numRows,
+          vAggregationPositions,
+          Groupers.writeAggregationRows(vAggregationRows, startRow, endRow)
+      );
     }
 
     return AggregateResult.ok();
   }
 
-  private void initializeSlot(int dimIndex)
+  private void initializeSlotIfNeeded(int dimIndex)
   {
     final int index = dimIndex / Byte.SIZE;
     final int extraIndex = dimIndex % Byte.SIZE;
-    usedFlagBuffer.put(index, (byte) (usedFlagBuffer.get(index) | (1 << extraIndex)));
+    final int usedFlagByte = 1 << extraIndex;
 
-    final int recordOffset = dimIndex * recordSize;
-    for (int i = 0; i < aggregators.length; i++) {
-      aggregators[i].init(valBuffer, recordOffset + aggregatorOffsets[i]);
+    if ((usedFlagBuffer.get(index) & usedFlagByte) == 0) {
+      usedFlagBuffer.put(index, (byte) (usedFlagBuffer.get(index) | (1 << extraIndex)));
+      aggregators.init(valBuffer, dimIndex * recordSize);
     }
   }
 
@@ -185,6 +223,7 @@ public class BufferArrayGrouper implements IntGrouper
     final int index = dimIndex / Byte.SIZE;
     final int extraIndex = dimIndex % Byte.SIZE;
     final int usedFlagByte = 1 << extraIndex;
+
     return (usedFlagBuffer.get(index) & usedFlagByte) != 0;
   }
 
@@ -214,14 +253,36 @@ public class BufferArrayGrouper implements IntGrouper
   @Override
   public void close()
   {
-    for (BufferAggregator aggregator : aggregators) {
-      try {
-        aggregator.close();
+    aggregators.close();
+  }
+
+  @Override
+  public CloseableIterator<Entry<ByteBuffer>> iterator()
+  {
+    final CloseableIterator<Entry<Integer>> iterator = iterator(false);
+    final ByteBuffer keyBuffer = ByteBuffer.allocate(Integer.BYTES);
+    return new CloseableIterator<Entry<ByteBuffer>>()
+    {
+      @Override
+      public boolean hasNext()
+      {
+        return iterator.hasNext();
       }
-      catch (Exception e) {
-        LOG.warn(e, "Could not close aggregator [%s], skipping.", aggregator);
+
+      @Override
+      public Entry<ByteBuffer> next()
+      {
+        final Entry<Integer> integerEntry = iterator.next();
+        keyBuffer.putInt(0, integerEntry.getKey());
+        return new Entry<>(keyBuffer, integerEntry.getValues());
       }
-    }
+
+      @Override
+      public void close() throws IOException
+      {
+        iterator.close();
+      }
+    };
   }
 
   @Override
@@ -252,10 +313,10 @@ public class BufferArrayGrouper implements IntGrouper
         final int current = next;
         next = findNext(current);
 
-        final Object[] values = new Object[aggregators.length];
+        final Object[] values = new Object[aggregators.size()];
         final int recordOffset = current * recordSize;
-        for (int i = 0; i < aggregators.length; i++) {
-          values[i] = aggregators[i].get(valBuffer, recordOffset + aggregatorOffsets[i]);
+        for (int i = 0; i < aggregators.size(); i++) {
+          values[i] = aggregators.get(valBuffer, recordOffset, i);
         }
         // shift by -1 since values are initially shifted by +1 so they are all positive and
         // GroupByColumnSelectorStrategy.GROUP_BY_MISSING_VALUE is -1
diff --git a/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/BufferHashGrouper.java b/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/BufferHashGrouper.java
index 5f54fad..1799579 100644
--- a/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/BufferHashGrouper.java
+++ b/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/BufferHashGrouper.java
@@ -20,11 +20,13 @@
 package org.apache.druid.query.groupby.epinephelinae;
 
 import com.google.common.base.Supplier;
+import org.apache.commons.lang.mutable.MutableInt;
 import org.apache.druid.java.util.common.CloseableIterators;
 import org.apache.druid.java.util.common.IAE;
+import org.apache.druid.java.util.common.ISE;
 import org.apache.druid.java.util.common.parsers.CloseableIterator;
+import org.apache.druid.query.aggregation.AggregatorAdapters;
 import org.apache.druid.query.aggregation.AggregatorFactory;
-import org.apache.druid.segment.ColumnSelectorFactory;
 
 import java.nio.ByteBuffer;
 import java.util.AbstractList;
@@ -32,14 +34,14 @@ import java.util.Collections;
 import java.util.Comparator;
 import java.util.List;
 import java.util.NoSuchElementException;
+import java.util.function.ToIntFunction;
 
-public class BufferHashGrouper<KeyType> extends AbstractBufferHashGrouper<KeyType>
+public class BufferHashGrouper<KeyType> extends AbstractBufferHashGrouper<KeyType> implements VectorGrouper
 {
   private static final int MIN_INITIAL_BUCKETS = 4;
   private static final int DEFAULT_INITIAL_BUCKETS = 1024;
   private static final float DEFAULT_MAX_LOAD_FACTOR = 0.7f;
 
-  private final AggregatorFactory[] aggregatorFactories;
   private ByteBuffer buffer;
   private boolean initialized = false;
 
@@ -58,19 +60,23 @@ public class BufferHashGrouper<KeyType> extends AbstractBufferHashGrouper<KeyTyp
   private ByteBuffer offsetListBuffer;
   private ByteBufferIntList offsetList;
 
+  // Scratch objects used by aggregateVector(). Only set if initVectorized() is called.
+  private ByteBuffer vKeyBuffer = null;
+  private int[] vKeyHashCodes = null;
+  private int[] vAggregationPositions = null;
+  private int[] vAggregationRows = null;
+
   public BufferHashGrouper(
       final Supplier<ByteBuffer> bufferSupplier,
       final KeySerde<KeyType> keySerde,
-      final ColumnSelectorFactory columnSelectorFactory,
-      final AggregatorFactory[] aggregatorFactories,
+      final AggregatorAdapters aggregators,
       final int bufferGrouperMaxSize,
       final float maxLoadFactor,
       final int initialBuckets,
       final boolean useDefaultSorting
   )
   {
-    super(bufferSupplier, keySerde, aggregatorFactories, bufferGrouperMaxSize);
-    this.aggregatorFactories = aggregatorFactories;
+    super(bufferSupplier, keySerde, aggregators, HASH_SIZE + keySerde.keySize(), bufferGrouperMaxSize);
 
     this.maxLoadFactor = maxLoadFactor > 0 ? maxLoadFactor : DEFAULT_MAX_LOAD_FACTOR;
     this.initialBuckets = initialBuckets > 0 ? Math.max(MIN_INITIAL_BUCKETS, initialBuckets) : DEFAULT_INITIAL_BUCKETS;
@@ -79,14 +85,7 @@ public class BufferHashGrouper<KeyType> extends AbstractBufferHashGrouper<KeyTyp
       throw new IAE("Invalid maxLoadFactor[%f], must be < 1.0", maxLoadFactor);
     }
 
-    int offset = HASH_SIZE + keySize;
-    for (int i = 0; i < aggregatorFactories.length; i++) {
-      aggregators[i] = aggregatorFactories[i].factorizeBuffered(columnSelectorFactory);
-      aggregatorOffsets[i] = offset;
-      offset += aggregatorFactories[i].getMaxIntermediateSizeWithNulls();
-    }
-
-    this.bucketSize = offset;
+    this.bucketSize = HASH_SIZE + keySerde.keySize() + aggregators.spaceNeeded();
     this.useDefaultSorting = useDefaultSorting;
   }
 
@@ -133,14 +132,111 @@ public class BufferHashGrouper<KeyType> extends AbstractBufferHashGrouper<KeyTyp
   }
 
   @Override
+  public void initVectorized(final int maxVectorSize)
+  {
+    if (!ByteBuffer.class.equals(keySerde.keyClazz())) {
+      throw new ISE("keyClazz[%s] must be ByteBuffer", keySerde.keyClazz());
+    }
+
+    if (keySize % Integer.BYTES != 0) {
+      throw new ISE("keySize[%s] must be a multiple of[%s]", keySize, Integer.BYTES);
+    }
+
+    init();
+
+    this.vKeyBuffer = ByteBuffer.allocate(keySize);
+    this.vKeyHashCodes = new int[maxVectorSize];
+    this.vAggregationPositions = new int[maxVectorSize];
+    this.vAggregationRows = new int[maxVectorSize];
+  }
+
+  @Override
+  public AggregateResult aggregateVector(final int[] keySpace, final int startRow, final int endRow)
+  {
+    final int keyIntSize = keySize / Integer.BYTES;
+    final int numRows = endRow - startRow;
+
+    // Initialize vKeyHashCodes: one int per key.
+    // Does *not* use hashFunction(). This is okay because the API of VectorGrouper does not expose any way of messing
+    // about with hash codes.
+    for (int i = 0, rowStart = 0; i < numRows; i++, rowStart += keyIntSize) {
+      vKeyHashCodes[i] = Groupers.hashIntArray(keySpace, rowStart, keyIntSize);
+    }
+
+    final MutableInt aggregationStartRow = new MutableInt(startRow);
+    final MutableInt aggregationNumRows = new MutableInt(0);
+
+    for (int rowNum = 0, keySpacePosition = 0; rowNum < numRows; rowNum++, keySpacePosition += keyIntSize) {
+      // Copy current key into keyBuffer.
+      vKeyBuffer.rewind();
+      for (int i = 0; i < keyIntSize; i++) {
+        vKeyBuffer.putInt(keySpace[keySpacePosition + i]);
+      }
+      vKeyBuffer.rewind();
+
+      // Find, and if the table is full, expand and find again.
+      int bucket = hashTable.findBucketWithAutoGrowth(
+          vKeyBuffer,
+          vKeyHashCodes[rowNum],
+          () -> {
+            if (aggregationNumRows.intValue() > 0) {
+              doAggregateVector(aggregationStartRow.intValue(), aggregationNumRows.intValue());
+              aggregationStartRow.setValue(aggregationStartRow.intValue() + aggregationNumRows.intValue());
+              aggregationNumRows.setValue(0);
+            }
+          }
+      );
+
+      if (bucket < 0) {
+        // This may just trigger a spill and get ignored, which is ok. If it bubbles up to the user, the message will
+        // be correct.
+
+        // Aggregate any remaining rows.
+        if (aggregationNumRows.intValue() > 0) {
+          doAggregateVector(aggregationStartRow.intValue(), aggregationNumRows.intValue());
+        }
+
+        return Groupers.hashTableFull(rowNum);
+      }
+
+      final int bucketStartOffset = hashTable.getOffsetForBucket(bucket);
+      final boolean bucketWasUsed = hashTable.isBucketUsed(bucket);
+
+      // Set up key and initialize the aggs if this is a new bucket.
+      if (!bucketWasUsed) {
+        hashTable.initializeNewBucketKey(bucket, vKeyBuffer, vKeyHashCodes[rowNum]);
+        aggregators.init(hashTable.getTableBuffer(), bucketStartOffset + baseAggregatorOffset);
+      }
+
+      // Schedule the current row for aggregation.
+      vAggregationPositions[aggregationNumRows.intValue()] = bucketStartOffset + Integer.BYTES + keySize;
+      aggregationNumRows.increment();
+    }
+
+    // Aggregate any remaining rows.
+    if (aggregationNumRows.intValue() > 0) {
+      doAggregateVector(aggregationStartRow.intValue(), aggregationNumRows.intValue());
+    }
+
+    return AggregateResult.ok();
+  }
+
+  @Override
   public boolean isInitialized()
   {
     return initialized;
   }
 
   @Override
+  public ToIntFunction<KeyType> hashFunction()
+  {
+    return Groupers::hashObject;
+  }
+
+  @Override
   public void newBucketHook(int bucketOffset)
   {
+    // Nothing needed.
   }
 
   @Override
@@ -152,7 +248,7 @@ public class BufferHashGrouper<KeyType> extends AbstractBufferHashGrouper<KeyTyp
   @Override
   public void afterAggregateHook(int bucketOffset)
   {
-
+    // Nothing needed.
   }
 
   @Override
@@ -164,6 +260,15 @@ public class BufferHashGrouper<KeyType> extends AbstractBufferHashGrouper<KeyTyp
   }
 
   @Override
+  @SuppressWarnings("unchecked")
+  public CloseableIterator<Entry<ByteBuffer>> iterator()
+  {
+    // Unchecked cast, since this method is only called through the VectorGrouper interface, which uses
+    // ByteBuffer keys (and this is verified in initVectorized).
+    return (CloseableIterator) iterator(false);
+  }
+
+  @Override
   public CloseableIterator<Entry<KeyType>> iterator(boolean sorted)
   {
     if (!initialized) {
@@ -201,7 +306,10 @@ public class BufferHashGrouper<KeyType> extends AbstractBufferHashGrouper<KeyTyp
       if (useDefaultSorting) {
         comparator = keySerde.bufferComparator();
       } else {
-        comparator = keySerde.bufferComparatorWithAggregators(aggregatorFactories, aggregatorOffsets);
+        comparator = keySerde.bufferComparatorWithAggregators(
+            aggregators.factories().toArray(new AggregatorFactory[0]),
+            aggregators.aggregatorPositions()
+        );
       }
 
       // Sort offsets in-place.
@@ -296,6 +404,16 @@ public class BufferHashGrouper<KeyType> extends AbstractBufferHashGrouper<KeyTyp
     }
   }
 
+  private void doAggregateVector(final int startRow, final int numRows)
+  {
+    aggregators.aggregateVector(
+        hashTable.getTableBuffer(),
+        numRows,
+        vAggregationPositions,
+        Groupers.writeAggregationRows(vAggregationRows, startRow, startRow + numRows)
+    );
+  }
+
   private class BufferGrouperBucketUpdateHandler implements ByteBufferHashTable.BucketUpdateHandler
   {
     @Override
@@ -314,14 +432,12 @@ public class BufferHashGrouper<KeyType> extends AbstractBufferHashGrouper<KeyTyp
     public void handleBucketMove(int oldBucketOffset, int newBucketOffset, ByteBuffer oldBuffer, ByteBuffer newBuffer)
     {
       // relocate aggregators (see https://github.com/apache/incubator-druid/pull/4071)
-      for (int i = 0; i < aggregators.length; i++) {
-        aggregators[i].relocate(
-            oldBucketOffset + aggregatorOffsets[i],
-            newBucketOffset + aggregatorOffsets[i],
-            oldBuffer,
-            newBuffer
-        );
-      }
+      aggregators.relocate(
+          oldBucketOffset + baseAggregatorOffset,
+          newBucketOffset + baseAggregatorOffset,
+          oldBuffer,
+          newBuffer
+      );
 
       offsetList.add(newBucketOffset);
     }
diff --git a/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/ByteBufferHashTable.java b/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/ByteBufferHashTable.java
index 0148279..6ba201e 100644
--- a/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/ByteBufferHashTable.java
+++ b/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/ByteBufferHashTable.java
@@ -75,7 +75,6 @@ public class ByteBufferHashTable
   protected int growthCount;
 
 
-
   protected BucketUpdateHandler bucketUpdateHandler;
 
   public ByteBufferHashTable(
@@ -251,21 +250,25 @@ public class ByteBufferHashTable
   }
 
   /**
-   * Find a bucket for a key, attempting to resize the table with adjustTableWhenFull() if possible.
+   * Find a bucket for a key, attempting to grow the table with adjustTableWhenFull() if possible.
+   *
+   * @param keyBuffer              buffer containing the key
+   * @param keyHash                hash of the key
+   * @param preTableGrowthRunnable runnable that executes before the table grows
    *
-   * @param keyBuffer buffer containing the key
-   * @param keyHash hash of the key
    * @return bucket number of the found bucket or -1 if a bucket could not be allocated after resizing.
    */
   protected int findBucketWithAutoGrowth(
       final ByteBuffer keyBuffer,
-      final int keyHash
+      final int keyHash,
+      final Runnable preTableGrowthRunnable
   )
   {
     int bucket = findBucket(canAllowNewBucket(), maxBuckets, tableBuffer, keyBuffer, keyHash);
 
     if (bucket < 0) {
       if (size < maxSizeForTesting) {
+        preTableGrowthRunnable.run();
         adjustTableWhenFull();
         bucket = findBucket(size < regrowthThreshold, maxBuckets, tableBuffer, keyBuffer, keyHash);
       }
@@ -277,7 +280,7 @@ public class ByteBufferHashTable
   /**
    * Finds the bucket into which we should insert a key.
    *
-   * @param keyBuffer key, must have exactly keySize bytes remaining. Will not be modified.
+   * @param keyBuffer         key, must have exactly keySize bytes remaining. Will not be modified.
    * @param targetTableBuffer Need selectable buffer, since when resizing hash table,
    *                          findBucket() is used on the newly allocated table buffer
    *
@@ -379,7 +382,9 @@ public class ByteBufferHashTable
   public interface BucketUpdateHandler
   {
     void handleNewBucket(int bucketOffset);
+
     void handlePreTableSwap();
+
     void handleBucketMove(int oldBucketOffset, int newBucketOffset, ByteBuffer oldBuffer, ByteBuffer newBuffer);
   }
 }
diff --git a/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/ByteBufferKeySerde.java b/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/ByteBufferKeySerde.java
new file mode 100644
index 0000000..f840c01
--- /dev/null
+++ b/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/ByteBufferKeySerde.java
@@ -0,0 +1,91 @@
+/*
+ * 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.query.groupby.epinephelinae;
+
+import com.google.common.collect.ImmutableList;
+import org.apache.druid.query.aggregation.AggregatorFactory;
+
+import java.nio.ByteBuffer;
+import java.util.List;
+
+public class ByteBufferKeySerde implements Grouper.KeySerde<ByteBuffer>
+{
+  private final int keySize;
+
+  public ByteBufferKeySerde(final int keySize)
+  {
+    this.keySize = keySize;
+  }
+
+  @Override
+  public int keySize()
+  {
+    return keySize;
+  }
+
+  @Override
+  public Class<ByteBuffer> keyClazz()
+  {
+    return ByteBuffer.class;
+  }
+
+  @Override
+  public List<String> getDictionary()
+  {
+    return ImmutableList.of();
+  }
+
+  @Override
+  public ByteBuffer toByteBuffer(ByteBuffer key)
+  {
+    return key;
+  }
+
+  @Override
+  public ByteBuffer fromByteBuffer(ByteBuffer buffer, int position)
+  {
+    final ByteBuffer dup = buffer.duplicate();
+    dup.position(position).limit(position + keySize);
+    return dup.slice();
+  }
+
+  @Override
+  public Grouper.BufferComparator bufferComparator()
+  {
+    // This class is used by segment processing engines, where bufferComparator will not be called.
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public Grouper.BufferComparator bufferComparatorWithAggregators(
+      AggregatorFactory[] aggregatorFactories,
+      int[] aggregatorOffsets
+  )
+  {
+    // This class is used by segment processing engines, where bufferComparatorWithAggregators will not be called.
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public void reset()
+  {
+    // No state, nothing to reset
+  }
+}
diff --git a/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/CloseableGrouperIterator.java b/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/CloseableGrouperIterator.java
index 43f3bcb..aabb167 100644
--- a/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/CloseableGrouperIterator.java
+++ b/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/CloseableGrouperIterator.java
@@ -34,14 +34,13 @@ public class CloseableGrouperIterator<KeyType, T> implements CloseableIterator<T
   private final Closer closer;
 
   public CloseableGrouperIterator(
-      final Grouper<KeyType> grouper,
-      final boolean sorted,
+      final CloseableIterator<Entry<KeyType>> iterator,
       final Function<Grouper.Entry<KeyType>, T> transformer,
       final Closeable closeable
   )
   {
     this.transformer = transformer;
-    this.iterator = grouper.iterator(sorted);
+    this.iterator = iterator;
     this.closer = Closer.create();
 
     closer.register(iterator);
diff --git a/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/ConcurrentGrouper.java b/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/ConcurrentGrouper.java
index 4bf116e..f5ff7ba 100644
--- a/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/ConcurrentGrouper.java
+++ b/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/ConcurrentGrouper.java
@@ -264,9 +264,12 @@ public class ConcurrentGrouper<KeyType> implements Grouper<KeyType>
 
       synchronized (hashBasedGrouper) {
         if (!spilling) {
-          if (hashBasedGrouper.aggregate(key, keyHash).isOk()) {
+          final AggregateResult aggregateResult = hashBasedGrouper.aggregate(key, keyHash);
+          if (aggregateResult.isOk()) {
             return AggregateResult.ok();
           } else {
+            // Expecting all-or-nothing behavior.
+            assert aggregateResult.getCount() == 0;
             spilling = true;
           }
         }
diff --git a/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/GroupByQueryEngineV2.java b/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/GroupByQueryEngineV2.java
index 7594b90..a39441d 100644
--- a/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/GroupByQueryEngineV2.java
+++ b/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/GroupByQueryEngineV2.java
@@ -22,6 +22,7 @@ package org.apache.druid.query.groupby.epinephelinae;
 import com.google.common.base.Preconditions;
 import com.google.common.base.Suppliers;
 import com.google.common.collect.ImmutableList;
+import com.google.common.collect.Iterables;
 import com.google.common.collect.Maps;
 import org.apache.druid.collections.NonBlockingPool;
 import org.apache.druid.collections.ResourceHolder;
@@ -34,9 +35,12 @@ import org.apache.druid.java.util.common.ISE;
 import org.apache.druid.java.util.common.guava.BaseSequence;
 import org.apache.druid.java.util.common.guava.Sequence;
 import org.apache.druid.query.ColumnSelectorPlus;
+import org.apache.druid.query.QueryContexts;
+import org.apache.druid.query.aggregation.AggregatorAdapters;
 import org.apache.druid.query.aggregation.AggregatorFactory;
 import org.apache.druid.query.dimension.ColumnSelectorStrategyFactory;
 import org.apache.druid.query.dimension.DimensionSpec;
+import org.apache.druid.query.filter.Filter;
 import org.apache.druid.query.groupby.GroupByQuery;
 import org.apache.druid.query.groupby.GroupByQueryConfig;
 import org.apache.druid.query.groupby.epinephelinae.column.DictionaryBuildingStringGroupByColumnSelectorStrategy;
@@ -47,6 +51,7 @@ import org.apache.druid.query.groupby.epinephelinae.column.GroupByColumnSelector
 import org.apache.druid.query.groupby.epinephelinae.column.LongGroupByColumnSelectorStrategy;
 import org.apache.druid.query.groupby.epinephelinae.column.NullableValueGroupByColumnSelectorStrategy;
 import org.apache.druid.query.groupby.epinephelinae.column.StringGroupByColumnSelectorStrategy;
+import org.apache.druid.query.groupby.epinephelinae.vector.VectorGroupByEngine;
 import org.apache.druid.query.groupby.strategy.GroupByStrategyV2;
 import org.apache.druid.segment.ColumnSelectorFactory;
 import org.apache.druid.segment.ColumnValueSelector;
@@ -54,7 +59,6 @@ import org.apache.druid.segment.Cursor;
 import org.apache.druid.segment.DimensionHandlerUtils;
 import org.apache.druid.segment.DimensionSelector;
 import org.apache.druid.segment.StorageAdapter;
-import org.apache.druid.segment.VirtualColumns;
 import org.apache.druid.segment.column.ColumnCapabilities;
 import org.apache.druid.segment.column.ValueType;
 import org.apache.druid.segment.data.IndexedInts;
@@ -69,6 +73,7 @@ import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
 import java.util.NoSuchElementException;
+import java.util.function.Function;
 
 public class GroupByQueryEngineV2
 {
@@ -108,15 +113,6 @@ public class GroupByQueryEngineV2
       throw new IAE("Should only have one interval, got[%s]", intervals);
     }
 
-    final Sequence<Cursor> cursors = storageAdapter.makeCursors(
-        Filters.toFilter(query.getDimFilter()),
-        intervals.get(0),
-        query.getVirtualColumns(),
-        query.getGranularity(),
-        false,
-        null
-    );
-
     final ResourceHolder<ByteBuffer> bufferHolder = intermediateResultsBufferPool.take();
 
     final String fudgeTimestampString = NullHandling.emptyToNullIfNeeded(
@@ -127,6 +123,59 @@ public class GroupByQueryEngineV2
                                     ? null
                                     : DateTimes.utc(Long.parseLong(fudgeTimestampString));
 
+    final Filter filter = Filters.convertToCNFFromQueryContext(query, Filters.toFilter(query.getFilter()));
+    final Interval interval = Iterables.getOnlyElement(query.getIntervals());
+
+    final boolean doVectorize = QueryContexts.getVectorize(query).shouldVectorize(
+        VectorGroupByEngine.canVectorize(query, storageAdapter, filter)
+    );
+
+    final Sequence<Row> result;
+
+    if (doVectorize) {
+      result = VectorGroupByEngine.process(
+          query,
+          storageAdapter,
+          bufferHolder.get(),
+          fudgeTimestamp,
+          filter,
+          interval,
+          querySpecificConfig
+      );
+    } else {
+      result = processNonVectorized(
+          query,
+          storageAdapter,
+          bufferHolder.get(),
+          fudgeTimestamp,
+          querySpecificConfig,
+          filter,
+          interval
+      );
+    }
+
+    return result.withBaggage(bufferHolder);
+  }
+
+  private static Sequence<Row> processNonVectorized(
+      final GroupByQuery query,
+      final StorageAdapter storageAdapter,
+      final ByteBuffer processingBuffer,
+      @Nullable final DateTime fudgeTimestamp,
+      final GroupByQueryConfig querySpecificConfig,
+      @Nullable final Filter filter,
+      final Interval interval
+  )
+  {
+    final Sequence<Cursor> cursors = storageAdapter.makeCursors(
+        filter,
+        interval,
+        query.getVirtualColumns(),
+        query.getGranularity(),
+        false,
+        null
+    );
+
     return cursors.flatMap(
         cursor -> new BaseSequence<>(
             new BaseSequence.IteratorMaker<Row, GroupByEngineIterator<?>>()
@@ -135,57 +184,42 @@ public class GroupByQueryEngineV2
               public GroupByEngineIterator make()
               {
                 final ColumnSelectorFactory columnSelectorFactory = cursor.getColumnSelectorFactory();
-                final boolean allSingleValueDims = query
-                    .getDimensions()
-                    .stream()
-                    .allMatch(dimension -> {
-                      final ColumnCapabilities columnCapabilities = columnSelectorFactory.getColumnCapabilities(
-                          dimension.getDimension()
-                      );
-                      return columnCapabilities != null && !columnCapabilities.hasMultipleValues();
-                    });
-
-                ColumnSelectorPlus<GroupByColumnSelectorStrategy>[] selectorPlus = DimensionHandlerUtils
+                final ColumnSelectorPlus<GroupByColumnSelectorStrategy>[] selectorPlus = DimensionHandlerUtils
                     .createColumnSelectorPluses(
                         STRATEGY_FACTORY,
                         query.getDimensions(),
                         columnSelectorFactory
                     );
-                GroupByColumnSelectorPlus[] dims = createGroupBySelectorPlus(selectorPlus);
 
-                final ByteBuffer buffer = bufferHolder.get();
+                final GroupByColumnSelectorPlus[] dims = createGroupBySelectorPlus(selectorPlus);
 
-                // Check if array-based aggregation is applicable
-                final boolean useArrayAggregation = isArrayAggregateApplicable(
+                final int cardinalityForArrayAggregation = getCardinalityForArrayAggregation(
                     querySpecificConfig,
                     query,
-                    dims,
                     storageAdapter,
-                    query.getVirtualColumns(),
-                    buffer
+                    processingBuffer
                 );
 
-                if (useArrayAggregation) {
+                if (cardinalityForArrayAggregation >= 0) {
                   return new ArrayAggregateIterator(
                       query,
                       querySpecificConfig,
                       cursor,
-                      buffer,
+                      processingBuffer,
                       fudgeTimestamp,
                       dims,
-                      allSingleValueDims,
-                      // There must be 0 or 1 dimension if isArrayAggregateApplicable() is true
-                      dims.length == 0 ? 1 : storageAdapter.getDimensionCardinality(dims[0].getName())
+                      isAllSingleValueDims(columnSelectorFactory::getColumnCapabilities, query.getDimensions()),
+                      cardinalityForArrayAggregation
                   );
                 } else {
                   return new HashAggregateIterator(
                       query,
                       querySpecificConfig,
                       cursor,
-                      buffer,
+                      processingBuffer,
                       fudgeTimestamp,
                       dims,
-                      allSingleValueDims
+                      isAllSingleValueDims(columnSelectorFactory::getColumnCapabilities, query.getDimensions())
                   );
                 }
               }
@@ -197,65 +231,91 @@ public class GroupByQueryEngineV2
               }
             }
         )
-    ).withBaggage(bufferHolder);
+    );
   }
 
-  private static boolean isArrayAggregateApplicable(
+  /**
+   * Returns the cardinality of array needed to do array-based aggregation, or -1 if array-based aggregation
+   * is impossible.
+   */
+  public static int getCardinalityForArrayAggregation(
       GroupByQueryConfig querySpecificConfig,
       GroupByQuery query,
-      GroupByColumnSelectorPlus[] dims,
       StorageAdapter storageAdapter,
-      VirtualColumns virtualColumns,
       ByteBuffer buffer
   )
   {
     if (querySpecificConfig.isForceHashAggregation()) {
-      return false;
+      return -1;
     }
 
+    final List<DimensionSpec> dimensions = query.getDimensions();
     final ColumnCapabilities columnCapabilities;
     final int cardinality;
 
     // Find cardinality
-    if (dims.length == 0) {
+    if (dimensions.isEmpty()) {
       columnCapabilities = null;
       cardinality = 1;
-    } else if (dims.length == 1) {
+    } else if (dimensions.size() == 1) {
       // Only real columns can use array-based aggregation, since virtual columns cannot currently report their
       // cardinality. We need to check if a virtual column exists with the same name, since virtual columns can shadow
       // real columns, and we might miss that since we're going directly to the StorageAdapter (which only knows about
       // real columns).
-      if (virtualColumns.exists(dims[0].getName())) {
-        return false;
+      if (query.getVirtualColumns().exists(Iterables.getOnlyElement(dimensions).getDimension())) {
+        return -1;
       }
 
-      columnCapabilities = storageAdapter.getColumnCapabilities(dims[0].getName());
-      cardinality = storageAdapter.getDimensionCardinality(dims[0].getName());
+      final String columnName = Iterables.getOnlyElement(dimensions).getDimension();
+      columnCapabilities = storageAdapter.getColumnCapabilities(columnName);
+      cardinality = storageAdapter.getDimensionCardinality(columnName);
     } else {
       // Cannot use array-based aggregation with more than one dimension.
-      return false;
+      return -1;
     }
 
-    // Choose array-based aggregation if the grouping key is a single string dimension of a
-    // known cardinality
-    if ((columnCapabilities == null || columnCapabilities.getType().equals(ValueType.STRING))
-        && cardinality > 0) {
-      final AggregatorFactory[] aggregatorFactories = query
-          .getAggregatorSpecs()
-          .toArray(new AggregatorFactory[0]);
+    // Choose array-based aggregation if the grouping key is a single string dimension of a known cardinality
+    if (columnCapabilities != null && columnCapabilities.getType().equals(ValueType.STRING) && cardinality > 0) {
+      final AggregatorFactory[] aggregatorFactories = query.getAggregatorSpecs().toArray(new AggregatorFactory[0]);
       final long requiredBufferCapacity = BufferArrayGrouper.requiredBufferCapacity(
           cardinality,
           aggregatorFactories
       );
 
       // Check that all keys and aggregated values can be contained in the buffer
-      return requiredBufferCapacity <= buffer.capacity();
+      return requiredBufferCapacity <= buffer.capacity() ? cardinality : -1;
     } else {
-      return false;
+      return -1;
     }
   }
 
-  private static class GroupByStrategyFactory implements ColumnSelectorStrategyFactory<GroupByColumnSelectorStrategy>
+  /**
+   * Checks whether all "dimensions" are either single-valued or nonexistent (which is just as good as single-valued,
+   * since their selectors will show up as full of nulls).
+   */
+  public static boolean isAllSingleValueDims(
+      final Function<String, ColumnCapabilities> capabilitiesFunction,
+      final List<DimensionSpec> dimensions
+  )
+  {
+    return dimensions
+        .stream()
+        .allMatch(
+            dimension -> {
+              if (dimension.mustDecorate()) {
+                // DimensionSpecs that decorate may turn singly-valued columns into multi-valued selectors.
+                // To be safe, we must return false here.
+                return false;
+              }
+
+              // Now check column capabilities.
+              final ColumnCapabilities columnCapabilities = capabilitiesFunction.apply(dimension.getDimension());
+              return columnCapabilities == null || !columnCapabilities.hasMultipleValues();
+            });
+  }
+
+  private static class GroupByStrategyFactory
+      implements ColumnSelectorStrategyFactory<GroupByColumnSelectorStrategy>
   {
     @Override
     public GroupByColumnSelectorStrategy makeColumnSelectorStrategy(
@@ -311,7 +371,7 @@ public class GroupByQueryEngineV2
         final GroupByQueryConfig querySpecificConfig,
         final Cursor cursor,
         final ByteBuffer buffer,
-        final DateTime fudgeTimestamp,
+        @Nullable final DateTime fudgeTimestamp,
         final GroupByColumnSelectorPlus[] dims,
         final boolean allSingleValueDims
     )
@@ -340,8 +400,7 @@ public class GroupByQueryEngineV2
       }
 
       return new CloseableGrouperIterator<>(
-          grouper,
-          false,
+          grouper.iterator(false),
           entry -> {
             Map<String, Object> theMap = Maps.newLinkedHashMap();
 
@@ -448,7 +507,7 @@ public class GroupByQueryEngineV2
         GroupByQueryConfig querySpecificConfig,
         Cursor cursor,
         ByteBuffer buffer,
-        DateTime fudgeTimestamp,
+        @Nullable DateTime fudgeTimestamp,
         GroupByColumnSelectorPlus[] dims,
         boolean allSingleValueDims
     )
@@ -467,9 +526,10 @@ public class GroupByQueryEngineV2
       return new BufferHashGrouper<>(
           Suppliers.ofInstance(buffer),
           keySerde,
-          cursor.getColumnSelectorFactory(),
-          query.getAggregatorSpecs()
-               .toArray(new AggregatorFactory[0]),
+          AggregatorAdapters.factorizeBuffered(
+              cursor.getColumnSelectorFactory(),
+              query.getAggregatorSpecs()
+          ),
           querySpecificConfig.getBufferGrouperMaxSize(),
           querySpecificConfig.getBufferGrouperMaxLoadFactor(),
           querySpecificConfig.getBufferGrouperInitialBuckets(),
@@ -600,7 +660,7 @@ public class GroupByQueryEngineV2
         GroupByQueryConfig querySpecificConfig,
         Cursor cursor,
         ByteBuffer buffer,
-        DateTime fudgeTimestamp,
+        @Nullable DateTime fudgeTimestamp,
         GroupByColumnSelectorPlus[] dims,
         boolean allSingleValueDims,
         int cardinality
@@ -622,9 +682,7 @@ public class GroupByQueryEngineV2
     {
       return new BufferArrayGrouper(
           Suppliers.ofInstance(buffer),
-          cursor.getColumnSelectorFactory(),
-          query.getAggregatorSpecs()
-               .toArray(new AggregatorFactory[0]),
+          AggregatorAdapters.factorizeBuffered(cursor.getColumnSelectorFactory(), query.getAggregatorSpecs()),
           cardinality
       );
     }
@@ -698,7 +756,7 @@ public class GroupByQueryEngineV2
     protected void putToMap(Integer key, Map<String, Object> map)
     {
       if (dim != null) {
-        if (key != -1) {
+        if (key != GroupByColumnSelectorStrategy.GROUP_BY_MISSING_VALUE) {
           map.put(
               dim.getOutputName(),
               ((DimensionSelector) dim.getSelector()).lookupName(key)
@@ -710,7 +768,7 @@ public class GroupByQueryEngineV2
     }
   }
 
-  private static void convertRowTypesToOutputTypes(List<DimensionSpec> dimensionSpecs, Map<String, Object> rowMap)
+  public static void convertRowTypesToOutputTypes(List<DimensionSpec> dimensionSpecs, Map<String, Object> rowMap)
   {
     for (DimensionSpec dimSpec : dimensionSpecs) {
       final ValueType outputType = dimSpec.getOutputType();
diff --git a/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/Grouper.java b/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/Grouper.java
index 05664f5..596254e 100644
--- a/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/Grouper.java
+++ b/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/Grouper.java
@@ -25,6 +25,7 @@ import com.google.common.base.Preconditions;
 import org.apache.druid.java.util.common.parsers.CloseableIterator;
 import org.apache.druid.query.aggregation.AggregatorFactory;
 
+import javax.annotation.Nullable;
 import java.io.Closeable;
 import java.nio.ByteBuffer;
 import java.util.Arrays;
@@ -36,10 +37,12 @@ import java.util.function.ToIntFunction;
  * Groupers aggregate metrics from rows that they typically get from a ColumnSelectorFactory, under
  * grouping keys that some outside driver is passing in. They can also iterate over the grouped
  * rows after the aggregation is done.
- * <p>
+ *
  * They work sort of like a map of KeyType to aggregated values, except they don't support
  * random lookups.
  *
+ * See {@link VectorGrouper} for a vectorized version.
+ *
  * @param <KeyType> type of the key that will be passed in
  */
 public interface Grouper<KeyType> extends Closeable
@@ -89,7 +92,7 @@ public interface Grouper<KeyType> extends Closeable
 
   default ToIntFunction<KeyType> hashFunction()
   {
-    return Groupers::hash;
+    return Groupers::hashObject;
   }
 
   /**
@@ -247,6 +250,7 @@ public interface Grouper<KeyType> extends Closeable
      *
      * @return serialized key, or null if we are unable to serialize more keys due to resource limits
      */
+    @Nullable
     ByteBuffer toByteBuffer(T key);
 
     /**
diff --git a/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/Groupers.java b/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/Groupers.java
index d173b04..a1d8dbf 100644
--- a/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/Groupers.java
+++ b/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/Groupers.java
@@ -19,6 +19,7 @@
 
 package org.apache.druid.query.groupby.epinephelinae;
 
+import javax.annotation.Nullable;
 import java.nio.ByteBuffer;
 
 public class Groupers
@@ -28,17 +29,22 @@ public class Groupers
     // No instantiation
   }
 
-  static final AggregateResult DICTIONARY_FULL = AggregateResult.failure(
+  private static final AggregateResult DICTIONARY_FULL_ZERO_COUNT = AggregateResult.partial(
+      0,
       "Not enough dictionary space to execute this query. Try increasing "
       + "druid.query.groupBy.maxMergingDictionarySize or enable disk spilling by setting "
       + "druid.query.groupBy.maxOnDiskStorage to a positive number."
   );
-  static final AggregateResult HASH_TABLE_FULL = AggregateResult.failure(
+
+  private static final AggregateResult HASH_TABLE_FULL_ZERO_COUNT = AggregateResult.partial(
+      0,
       "Not enough aggregation buffer space to execute this query. Try increasing "
       + "druid.processing.buffer.sizeBytes or enable disk spilling by setting "
       + "druid.query.groupBy.maxOnDiskStorage to a positive number."
   );
 
+  private static final int USED_FLAG_MASK = 0x7fffffff;
+
   private static final int C1 = 0xcc9e2d51;
   private static final int C2 = 0x1b873593;
 
@@ -50,18 +56,46 @@ public class Groupers
    * MurmurHash3 was written by Austin Appleby, and is placed in the public domain. The author
    * hereby disclaims copyright to this source code.
    */
-  static int smear(int hashCode)
+  private static int smear(int hashCode)
   {
     return C2 * Integer.rotateLeft(hashCode * C1, 15);
   }
 
-  public static int hash(final Object obj)
+  public static AggregateResult dictionaryFull(final int count)
+  {
+    if (count == 0) {
+      return DICTIONARY_FULL_ZERO_COUNT;
+    } else {
+      return AggregateResult.partial(count, DICTIONARY_FULL_ZERO_COUNT.getReason());
+    }
+  }
+
+  public static AggregateResult hashTableFull(final int count)
+  {
+    if (count == 0) {
+      return HASH_TABLE_FULL_ZERO_COUNT;
+    } else {
+      return AggregateResult.partial(count, HASH_TABLE_FULL_ZERO_COUNT.getReason());
+    }
+  }
+
+  public static int hashObject(final Object obj)
   {
     // Mask off the high bit so we can use that to determine if a bucket is used or not.
-    // Also apply the smear function, to improve distribution.
-    final int code = obj.hashCode();
-    return smear(code) & 0x7fffffff;
+    // Also apply the "smear" function, to improve distribution.
+    return smear(obj.hashCode()) & USED_FLAG_MASK;
+  }
 
+  public static int hashIntArray(final int[] ints, final int start, final int length)
+  {
+    // Similar to what Arrays.hashCode would do.
+    // Also apply the "smear" function, to improve distribution.
+    int hashCode = 1;
+    for (int i = 0; i < length; i++) {
+      hashCode = 31 * hashCode + ints[start + i];
+    }
+
+    return smear(hashCode) & USED_FLAG_MASK;
   }
 
   static int getUsedFlag(int keyHash)
@@ -76,4 +110,22 @@ public class Groupers
     slice.limit(slice.position() + sliceSize);
     return slice.slice();
   }
+
+  /**
+   * Write ints from "start" to "end" into "scratch", if start != 0. Otherwise, return null.
+   */
+  @Nullable
+  public static int[] writeAggregationRows(final int[] scratch, final int start, final int end)
+  {
+    if (start == 0) {
+      return null;
+    } else {
+      final int numRows = end - start;
+      for (int i = 0; i < numRows; i++) {
+        scratch[i] = start + i;
+      }
+
+      return scratch;
+    }
+  }
 }
diff --git a/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/LimitedBufferHashGrouper.java b/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/LimitedBufferHashGrouper.java
index c8d97ea..4c71c31 100644
--- a/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/LimitedBufferHashGrouper.java
+++ b/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/LimitedBufferHashGrouper.java
@@ -24,8 +24,8 @@ import org.apache.druid.java.util.common.CloseableIterators;
 import org.apache.druid.java.util.common.IAE;
 import org.apache.druid.java.util.common.ISE;
 import org.apache.druid.java.util.common.parsers.CloseableIterator;
+import org.apache.druid.query.aggregation.AggregatorAdapters;
 import org.apache.druid.query.aggregation.AggregatorFactory;
-import org.apache.druid.segment.ColumnSelectorFactory;
 
 import java.nio.ByteBuffer;
 import java.util.AbstractList;
@@ -40,8 +40,6 @@ public class LimitedBufferHashGrouper<KeyType> extends AbstractBufferHashGrouper
   private static final int DEFAULT_INITIAL_BUCKETS = 1024;
   private static final float DEFAULT_MAX_LOAD_FACTOR = 0.7f;
 
-  private final AggregatorFactory[] aggregatorFactories;
-
   // Limit to apply to results.
   private int limit;
 
@@ -66,8 +64,7 @@ public class LimitedBufferHashGrouper<KeyType> extends AbstractBufferHashGrouper
   public LimitedBufferHashGrouper(
       final Supplier<ByteBuffer> bufferSupplier,
       final Grouper.KeySerde<KeyType> keySerde,
-      final ColumnSelectorFactory columnSelectorFactory,
-      final AggregatorFactory[] aggregatorFactories,
+      final AggregatorAdapters aggregators,
       final int bufferGrouperMaxSize,
       final float maxLoadFactor,
       final int initialBuckets,
@@ -75,7 +72,7 @@ public class LimitedBufferHashGrouper<KeyType> extends AbstractBufferHashGrouper
       final boolean sortHasNonGroupingFields
   )
   {
-    super(bufferSupplier, keySerde, aggregatorFactories, bufferGrouperMaxSize);
+    super(bufferSupplier, keySerde, aggregators, HASH_SIZE + keySerde.keySize(), bufferGrouperMaxSize);
     this.maxLoadFactor = maxLoadFactor > 0 ? maxLoadFactor : DEFAULT_MAX_LOAD_FACTOR;
     this.initialBuckets = initialBuckets > 0 ? Math.max(MIN_INITIAL_BUCKETS, initialBuckets) : DEFAULT_INITIAL_BUCKETS;
     this.limit = limit;
@@ -85,18 +82,9 @@ public class LimitedBufferHashGrouper<KeyType> extends AbstractBufferHashGrouper
       throw new IAE("Invalid maxLoadFactor[%f], must be < 1.0", maxLoadFactor);
     }
 
-    int offset = HASH_SIZE + keySize;
-    this.aggregatorFactories = aggregatorFactories;
-    for (int i = 0; i < aggregatorFactories.length; i++) {
-      aggregators[i] = aggregatorFactories[i].factorizeBuffered(columnSelectorFactory);
-      aggregatorOffsets[i] = offset;
-      offset += aggregatorFactories[i].getMaxIntermediateSizeWithNulls();
-    }
-
     // For each bucket, store an extra field indicating the bucket's current index within the heap when
-    // pushing down limits
-    offset += Integer.BYTES;
-    this.bucketSize = offset;
+    // pushing down limits (size Integer.BYTES).
+    this.bucketSize = HASH_SIZE + keySerde.keySize() + Integer.BYTES + aggregators.spaceNeeded();
   }
 
   @Override
@@ -374,8 +362,8 @@ public class LimitedBufferHashGrouper<KeyType> extends AbstractBufferHashGrouper
     return new Comparator<Integer>()
     {
       final BufferComparator bufferComparator = keySerde.bufferComparatorWithAggregators(
-          aggregatorFactories,
-          aggregatorOffsets
+          aggregators.factories().toArray(new AggregatorFactory[0]),
+          aggregators.aggregatorPositions()
       );
 
       @Override
@@ -511,14 +499,12 @@ public class LimitedBufferHashGrouper<KeyType> extends AbstractBufferHashGrouper
           offsetHeap.setAt(i, newBucketOffset);
 
           // relocate aggregators (see https://github.com/apache/incubator-druid/pull/4071)
-          for (int j = 0; j < aggregators.length; j++) {
-            aggregators[j].relocate(
-                oldBucketOffset + aggregatorOffsets[j],
-                newBucketOffset + aggregatorOffsets[j],
-                tableBuffer,
-                newTableBuffer
-            );
-          }
+          aggregators.relocate(
+              oldBucketOffset + baseAggregatorOffset,
+              newBucketOffset + baseAggregatorOffset,
+              tableBuffer,
+              newTableBuffer
+          );
         }
       }
 
diff --git a/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/RowBasedGrouperHelper.java b/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/RowBasedGrouperHelper.java
index bd41d08..de5da62 100644
--- a/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/RowBasedGrouperHelper.java
+++ b/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/RowBasedGrouperHelper.java
@@ -436,8 +436,7 @@ public class RowBasedGrouperHelper
     final boolean includeTimestamp = GroupByStrategyV2.getUniversalTimestamp(query) == null;
 
     return new CloseableGrouperIterator<>(
-        grouper,
-        true,
+        grouper.iterator(true),
         new Function<Grouper.Entry<RowBasedKey>, Row>()
         {
           @Override
@@ -833,7 +832,10 @@ public class RowBasedGrouperHelper
             @Override
             public int compare(Grouper.Entry<RowBasedKey> entry1, Grouper.Entry<RowBasedKey> entry2)
             {
-              final int timeCompare = Longs.compare((long) entry1.getKey().getKey()[0], (long) entry2.getKey().getKey()[0]);
+              final int timeCompare = Longs.compare(
+                  (long) entry1.getKey().getKey()[0],
+                  (long) entry2.getKey().getKey()[0]
+              );
 
               if (timeCompare != 0) {
                 return timeCompare;
@@ -930,8 +932,10 @@ public class RowBasedGrouperHelper
           // use natural comparison
           cmp = Comparators.<Comparable>naturalNullsFirst().compare(lhs, rhs);
         } else {
-          cmp = comparator.compare(DimensionHandlerUtils.convertObjectToString(lhs),
-                                   DimensionHandlerUtils.convertObjectToString(rhs));
+          cmp = comparator.compare(
+              DimensionHandlerUtils.convertObjectToString(lhs),
+              DimensionHandlerUtils.convertObjectToString(rhs)
+          );
         }
 
         if (cmp != 0) {
@@ -1637,7 +1641,8 @@ public class RowBasedGrouperHelper
       FloatRowBasedKeySerdeHelper(
           int keyBufferPosition,
           boolean pushLimitDown,
-          @Nullable StringComparator stringComparator)
+          @Nullable StringComparator stringComparator
+      )
       {
         this.keyBufferPosition = keyBufferPosition;
         if (isPrimitiveComparable(pushLimitDown, stringComparator)) {
diff --git a/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/SpillingGrouper.java b/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/SpillingGrouper.java
index 2ef2eb9..249060a 100644
--- a/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/SpillingGrouper.java
+++ b/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/SpillingGrouper.java
@@ -32,6 +32,7 @@ import org.apache.druid.java.util.common.io.Closer;
 import org.apache.druid.java.util.common.logger.Logger;
 import org.apache.druid.java.util.common.parsers.CloseableIterator;
 import org.apache.druid.query.BaseQuery;
+import org.apache.druid.query.aggregation.AggregatorAdapters;
 import org.apache.druid.query.aggregation.AggregatorFactory;
 import org.apache.druid.query.groupby.orderby.DefaultLimitSpec;
 import org.apache.druid.segment.ColumnSelectorFactory;
@@ -41,6 +42,7 @@ import java.io.FileInputStream;
 import java.io.IOException;
 import java.nio.ByteBuffer;
 import java.util.ArrayList;
+import java.util.Arrays;
 import java.util.Comparator;
 import java.util.HashSet;
 import java.util.Iterator;
@@ -55,11 +57,12 @@ import java.util.Set;
 public class SpillingGrouper<KeyType> implements Grouper<KeyType>
 {
   private static final Logger log = new Logger(SpillingGrouper.class);
-
-  private final Grouper<KeyType> grouper;
-  private static final AggregateResult DISK_FULL = AggregateResult.failure(
+  private static final AggregateResult DISK_FULL = AggregateResult.partial(
+      0,
       "Not enough disk space to execute this query. Try raising druid.query.groupBy.maxOnDiskStorage."
   );
+
+  private final Grouper<KeyType> grouper;
   private final KeySerde<KeyType> keySerde;
   private final LimitedTemporaryStorage temporaryStorage;
   private final ObjectMapper spillMapper;
@@ -96,8 +99,7 @@ public class SpillingGrouper<KeyType> implements Grouper<KeyType>
       LimitedBufferHashGrouper<KeyType> limitGrouper = new LimitedBufferHashGrouper<>(
           bufferSupplier,
           keySerde,
-          columnSelectorFactory,
-          aggregatorFactories,
+          AggregatorAdapters.factorizeBuffered(columnSelectorFactory, Arrays.asList(aggregatorFactories)),
           bufferGrouperMaxSize,
           bufferGrouperMaxLoadFactor,
           bufferGrouperInitialBuckets,
@@ -119,8 +121,7 @@ public class SpillingGrouper<KeyType> implements Grouper<KeyType>
         this.grouper = new BufferHashGrouper<>(
             bufferSupplier,
             keySerde,
-            columnSelectorFactory,
-            aggregatorFactories,
+            AggregatorAdapters.factorizeBuffered(columnSelectorFactory, Arrays.asList(aggregatorFactories)),
             bufferGrouperMaxSize,
             bufferGrouperMaxLoadFactor,
             bufferGrouperInitialBuckets,
@@ -133,8 +134,7 @@ public class SpillingGrouper<KeyType> implements Grouper<KeyType>
       this.grouper = new BufferHashGrouper<>(
           bufferSupplier,
           keySerde,
-          columnSelectorFactory,
-          aggregatorFactories,
+          AggregatorAdapters.factorizeBuffered(columnSelectorFactory, Arrays.asList(aggregatorFactories)),
           bufferGrouperMaxSize,
           bufferGrouperMaxLoadFactor,
           bufferGrouperInitialBuckets,
@@ -168,6 +168,9 @@ public class SpillingGrouper<KeyType> implements Grouper<KeyType>
     if (result.isOk() || !spillingAllowed || temporaryStorage.maxSize() <= 0) {
       return result;
     } else {
+      // Expecting all-or-nothing behavior.
+      assert result.getCount() == 0;
+
       // Warning: this can potentially block up a processing thread for a while.
       try {
         spill();
diff --git a/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/VectorGrouper.java b/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/VectorGrouper.java
new file mode 100644
index 0000000..1da43f8
--- /dev/null
+++ b/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/VectorGrouper.java
@@ -0,0 +1,77 @@
+/*
+ * 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.query.groupby.epinephelinae;
+
+import org.apache.druid.java.util.common.parsers.CloseableIterator;
+
+import java.io.Closeable;
+import java.nio.ByteBuffer;
+
+/**
+ * Like a {@link Grouper}, but vectorized. Keys are always int arrays, so there is no generic type parameter KeyType.
+ * <p>
+ * This interface is designed such that an implementation can implement both Grouper and VectorGrouper. Of course,
+ * it would generally only make sense for a particular instance to be called with one set of functionality or the
+ * other.
+ */
+public interface VectorGrouper extends Closeable
+{
+  /**
+   * Initialize the grouper. This method needs to be called before calling {@link #aggregateVector}.
+   */
+  void initVectorized(int maxVectorSize);
+
+  /**
+   * Aggregate the current vector of rows from "startVectorOffset" to "endVectorOffset" using the provided keys.
+   *
+   * @param keySpace array holding keys, chunked into ints. First (endVectorOffset - startVectorOffset) keys
+   *                 must be valid.
+   * @param startRow row to start at (inclusive).
+   * @param endRow   row to end at (exclusive).
+   *
+   * @return result that indicates how many keys were aggregated (may be partial due to resource limits)
+   */
+  AggregateResult aggregateVector(int[] keySpace, int startRow, int endRow);
+
+  /**
+   * Reset the grouper to its initial state.
+   */
+  void reset();
+
+  /**
+   * Close the grouper and release associated resources.
+   */
+  @Override
+  void close();
+
+  /**
+   * Iterate through entries.
+   * <p>
+   * Some implementations allow writes even after this method is called.  After you are done with the iterator
+   * returned by this method, you should either call {@link #close()} (if you are done with the VectorGrouper) or
+   * {@link #reset()} (if you want to reuse it).
+   * <p>
+   * Callers must process and discard the returned {@link Grouper.Entry}s immediately, because the keys may
+   * be reused.
+   *
+   * @return entry iterator
+   */
+  CloseableIterator<Grouper.Entry<ByteBuffer>> iterator();
+}
diff --git a/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/vector/DoubleGroupByVectorColumnSelector.java b/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/vector/DoubleGroupByVectorColumnSelector.java
new file mode 100644
index 0000000..2802e3a
--- /dev/null
+++ b/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/vector/DoubleGroupByVectorColumnSelector.java
@@ -0,0 +1,71 @@
+/*
+ * 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.query.groupby.epinephelinae.vector;
+
+import org.apache.druid.segment.vector.VectorValueSelector;
+
+import java.nio.ByteBuffer;
+import java.util.Map;
+
+public class DoubleGroupByVectorColumnSelector implements GroupByVectorColumnSelector
+{
+  private final VectorValueSelector selector;
+
+  DoubleGroupByVectorColumnSelector(final VectorValueSelector selector)
+  {
+    this.selector = selector;
+  }
+
+  @Override
+  public int getGroupingKeySize()
+  {
+    return 2;
+  }
+
+  @Override
+  public void writeKeys(
+      final int[] keySpace,
+      final int keySize,
+      final int keyOffset,
+      final int startRow,
+      final int endRow
+  )
+  {
+    final double[] vector = selector.getDoubleVector();
+
+    for (int i = startRow, j = keyOffset; i < endRow; i++, j += keySize) {
+      final long longValue = Double.doubleToLongBits(vector[i]);
+      keySpace[j] = (int) (longValue >>> 32);
+      keySpace[j + 1] = (int) (longValue & 0xffffffffL);
+    }
+  }
+
+  @Override
+  public void writeKeyToResultRow(
+      final String outputName,
+      final ByteBuffer keyBuffer,
+      final int keyOffset,
+      final Map<String, Object> resultMap
+  )
+  {
+    final double value = keyBuffer.getDouble(keyOffset * Integer.BYTES);
+    resultMap.put(outputName, value);
+  }
+}
diff --git a/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/vector/FloatGroupByVectorColumnSelector.java b/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/vector/FloatGroupByVectorColumnSelector.java
new file mode 100644
index 0000000..5adbdb1
--- /dev/null
+++ b/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/vector/FloatGroupByVectorColumnSelector.java
@@ -0,0 +1,69 @@
+/*
+ * 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.query.groupby.epinephelinae.vector;
+
+import org.apache.druid.segment.vector.VectorValueSelector;
+
+import java.nio.ByteBuffer;
+import java.util.Map;
+
+public class FloatGroupByVectorColumnSelector implements GroupByVectorColumnSelector
+{
+  private final VectorValueSelector selector;
+
+  FloatGroupByVectorColumnSelector(final VectorValueSelector selector)
+  {
+    this.selector = selector;
+  }
+
+  @Override
+  public int getGroupingKeySize()
+  {
+    return 1;
+  }
+
+  @Override
+  public void writeKeys(
+      final int[] keySpace,
+      final int keySize,
+      final int keyOffset,
+      final int startRow,
+      final int endRow
+  )
+  {
+    final float[] vector = selector.getFloatVector();
+
+    for (int i = startRow, j = keyOffset; i < endRow; i++, j += keySize) {
+      keySpace[j] = Float.floatToIntBits(vector[i]);
+    }
+  }
+
+  @Override
+  public void writeKeyToResultRow(
+      final String outputName,
+      final ByteBuffer keyBuffer,
+      final int keyOffset,
+      final Map<String, Object> resultMap
+  )
+  {
+    final float value = Float.intBitsToFloat(keyBuffer.getInt(keyOffset * Integer.BYTES));
+    resultMap.put(outputName, value);
+  }
+}
diff --git a/processing/src/main/java/org/apache/druid/segment/column/BaseColumn.java b/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/vector/GroupByVectorColumnSelector.java
similarity index 66%
copy from processing/src/main/java/org/apache/druid/segment/column/BaseColumn.java
copy to processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/vector/GroupByVectorColumnSelector.java
index d692725..3cc4153 100644
--- a/processing/src/main/java/org/apache/druid/segment/column/BaseColumn.java
+++ b/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/vector/GroupByVectorColumnSelector.java
@@ -17,14 +17,21 @@
  * under the License.
  */
 
-package org.apache.druid.segment.column;
+package org.apache.druid.query.groupby.epinephelinae.vector;
 
-import org.apache.druid.segment.ColumnValueSelector;
-import org.apache.druid.segment.data.ReadableOffset;
+import java.nio.ByteBuffer;
+import java.util.Map;
 
-import java.io.Closeable;
-
-public interface BaseColumn extends Closeable
+public interface GroupByVectorColumnSelector
 {
-  ColumnValueSelector<?> makeColumnValueSelector(ReadableOffset offset);
+  int getGroupingKeySize();
+
+  void writeKeys(int[] keySpace, int keySize, int keyOffset, int startRow, int endRow);
+
+  void writeKeyToResultRow(
+      String outputName,
+      ByteBuffer keyBuffer,
+      int keyOffset,
+      Map<String, Object> resultMap
+  );
 }
diff --git a/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/vector/GroupByVectorColumnStrategizer.java b/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/vector/GroupByVectorColumnStrategizer.java
new file mode 100644
index 0000000..c14041c
--- /dev/null
+++ b/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/vector/GroupByVectorColumnStrategizer.java
@@ -0,0 +1,70 @@
+/*
+ * 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.query.groupby.epinephelinae.vector;
+
+import org.apache.druid.query.dimension.VectorColumnStrategizer;
+import org.apache.druid.segment.vector.MultiValueDimensionVectorSelector;
+import org.apache.druid.segment.vector.SingleValueDimensionVectorSelector;
+import org.apache.druid.segment.vector.VectorValueSelector;
+
+public class GroupByVectorColumnStrategizer implements VectorColumnStrategizer<GroupByVectorColumnSelector>
+{
+  private static final GroupByVectorColumnStrategizer INSTANCE = new GroupByVectorColumnStrategizer();
+
+  private GroupByVectorColumnStrategizer()
+  {
+    // Singleton.
+  }
+
+  public static GroupByVectorColumnStrategizer instance()
+  {
+    return INSTANCE;
+  }
+
+  @Override
+  public GroupByVectorColumnSelector makeSingleValueDimensionStrategy(final SingleValueDimensionVectorSelector selector)
+  {
+    return new SingleValueStringGroupByVectorColumnSelector(selector);
+  }
+
+  @Override
+  public GroupByVectorColumnSelector makeMultiValueDimensionStrategy(final MultiValueDimensionVectorSelector selector)
+  {
+    throw new UnsupportedOperationException("Multi-value dimensions not yet implemented for vectorized groupBys");
+  }
+
+  @Override
+  public GroupByVectorColumnSelector makeFloatStrategy(final VectorValueSelector selector)
+  {
+    return new FloatGroupByVectorColumnSelector(selector);
+  }
+
+  @Override
+  public GroupByVectorColumnSelector makeDoubleStrategy(final VectorValueSelector selector)
+  {
+    return new DoubleGroupByVectorColumnSelector(selector);
+  }
+
+  @Override
+  public GroupByVectorColumnSelector makeLongStrategy(final VectorValueSelector selector)
+  {
+    return new LongGroupByVectorColumnSelector(selector);
+  }
+}
diff --git a/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/vector/LongGroupByVectorColumnSelector.java b/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/vector/LongGroupByVectorColumnSelector.java
new file mode 100644
index 0000000..6ddbd99
--- /dev/null
+++ b/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/vector/LongGroupByVectorColumnSelector.java
@@ -0,0 +1,70 @@
+/*
+ * 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.query.groupby.epinephelinae.vector;
+
+import org.apache.druid.segment.vector.VectorValueSelector;
+
+import java.nio.ByteBuffer;
+import java.util.Map;
+
+public class LongGroupByVectorColumnSelector implements GroupByVectorColumnSelector
+{
+  private final VectorValueSelector selector;
+
+  LongGroupByVectorColumnSelector(final VectorValueSelector selector)
+  {
+    this.selector = selector;
+  }
+
+  @Override
+  public int getGroupingKeySize()
+  {
+    return 2;
+  }
+
+  @Override
+  public void writeKeys(
+      final int[] keySpace,
+      final int keySize,
+      final int keyOffset,
+      final int startRow,
+      final int endRow
+  )
+  {
+    final long[] vector = selector.getLongVector();
+
+    for (int i = startRow, j = keyOffset; i < endRow; i++, j += keySize) {
+      keySpace[j] = (int) (vector[i] >>> 32);
+      keySpace[j + 1] = (int) (vector[i] & 0xffffffffL);
+    }
+  }
+
+  @Override
+  public void writeKeyToResultRow(
+      final String outputName,
+      final ByteBuffer keyBuffer,
+      final int keyOffset,
+      final Map<String, Object> resultMap
+  )
+  {
+    final long value = keyBuffer.getLong(keyOffset * Integer.BYTES);
+    resultMap.put(outputName, value);
+  }
+}
diff --git a/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/vector/SingleValueStringGroupByVectorColumnSelector.java b/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/vector/SingleValueStringGroupByVectorColumnSelector.java
new file mode 100644
index 0000000..6a9b428
--- /dev/null
+++ b/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/vector/SingleValueStringGroupByVectorColumnSelector.java
@@ -0,0 +1,69 @@
+/*
+ * 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.query.groupby.epinephelinae.vector;
+
+import org.apache.druid.segment.vector.SingleValueDimensionVectorSelector;
+
+import java.nio.ByteBuffer;
+import java.util.Map;
+
+public class SingleValueStringGroupByVectorColumnSelector implements GroupByVectorColumnSelector
+{
+  private final SingleValueDimensionVectorSelector selector;
+
+  SingleValueStringGroupByVectorColumnSelector(final SingleValueDimensionVectorSelector selector)
+  {
+    this.selector = selector;
+  }
+
+  @Override
+  public int getGroupingKeySize()
+  {
+    return 1;
+  }
+
+  @Override
+  public void writeKeys(
+      final int[] keySpace,
+      final int keySize,
+      final int keyOffset,
+      final int startRow,
+      final int endRow
+  )
+  {
+    final int[] rowVector = selector.getRowVector();
+
+    for (int i = startRow, j = keyOffset; i < endRow; i++, j += keySize) {
+      keySpace[j] = rowVector[i];
+    }
+  }
+
+  @Override
+  public void writeKeyToResultRow(
+      final String outputName,
+      final ByteBuffer keyBuffer,
+      final int keyOffset,
+      final Map<String, Object> resultMap
+  )
+  {
+    final int id = keyBuffer.getInt(keyOffset * Integer.BYTES);
+    resultMap.put(outputName, selector.lookupName(id));
+  }
+}
diff --git a/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/vector/VectorGroupByEngine.java b/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/vector/VectorGroupByEngine.java
new file mode 100644
index 0000000..96b9988
--- /dev/null
+++ b/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/vector/VectorGroupByEngine.java
@@ -0,0 +1,436 @@
+/*
+ * 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.query.groupby.epinephelinae.vector;
+
+import com.google.common.base.Suppliers;
+import org.apache.druid.data.input.MapBasedRow;
+import org.apache.druid.data.input.Row;
+import org.apache.druid.java.util.common.ISE;
+import org.apache.druid.java.util.common.guava.BaseSequence;
+import org.apache.druid.java.util.common.guava.Sequence;
+import org.apache.druid.java.util.common.parsers.CloseableIterator;
+import org.apache.druid.query.QueryContexts;
+import org.apache.druid.query.aggregation.AggregatorAdapters;
+import org.apache.druid.query.aggregation.AggregatorFactory;
+import org.apache.druid.query.dimension.DimensionSpec;
+import org.apache.druid.query.filter.Filter;
+import org.apache.druid.query.groupby.GroupByQuery;
+import org.apache.druid.query.groupby.GroupByQueryConfig;
+import org.apache.druid.query.groupby.epinephelinae.AggregateResult;
+import org.apache.druid.query.groupby.epinephelinae.BufferArrayGrouper;
+import org.apache.druid.query.groupby.epinephelinae.BufferHashGrouper;
+import org.apache.druid.query.groupby.epinephelinae.ByteBufferKeySerde;
+import org.apache.druid.query.groupby.epinephelinae.CloseableGrouperIterator;
+import org.apache.druid.query.groupby.epinephelinae.GroupByQueryEngineV2;
+import org.apache.druid.query.groupby.epinephelinae.Grouper;
+import org.apache.druid.query.groupby.epinephelinae.VectorGrouper;
+import org.apache.druid.query.vector.VectorCursorGranularizer;
+import org.apache.druid.segment.DimensionHandlerUtils;
+import org.apache.druid.segment.StorageAdapter;
+import org.apache.druid.segment.filter.Filters;
+import org.apache.druid.segment.vector.VectorColumnSelectorFactory;
+import org.apache.druid.segment.vector.VectorCursor;
+import org.joda.time.DateTime;
+import org.joda.time.Interval;
+
+import javax.annotation.Nullable;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.NoSuchElementException;
+import java.util.stream.Collectors;
+
+public class VectorGroupByEngine
+{
+  private VectorGroupByEngine()
+  {
+    // No instantiation.
+  }
+
+  public static boolean canVectorize(
+      final GroupByQuery query,
+      final StorageAdapter adapter,
+      @Nullable final Filter filter
+  )
+  {
+    // Multi-value dimensions are not yet supported.
+    //
+    // Two notes here about how we're handling this check:
+    //   1) After multi-value dimensions are supported, we could alter "GroupByQueryEngineV2.isAllSingleValueDims"
+    //      to accept a ColumnSelectorFactory, which makes more sense than using a StorageAdapter (see #8013).
+    //   2) Technically using StorageAdapter here is bad since it only looks at real columns, but they might
+    //      be shadowed by virtual columns (again, see #8013). But it's fine for now since adapter.canVectorize
+    //      always returns false if there are any virtual columns.
+    //
+    // This situation should sort itself out pretty well once this engine supports multi-valued columns. Then we
+    // won't have to worry about having this all-single-value-dims check here.
+
+    return GroupByQueryEngineV2.isAllSingleValueDims(adapter::getColumnCapabilities, query.getDimensions())
+           && query.getDimensions().stream().allMatch(DimensionSpec::canVectorize)
+           && query.getAggregatorSpecs().stream().allMatch(AggregatorFactory::canVectorize)
+           && adapter.canVectorize(filter, query.getVirtualColumns(), false);
+  }
+
+  public static Sequence<Row> process(
+      final GroupByQuery query,
+      final StorageAdapter storageAdapter,
+      final ByteBuffer processingBuffer,
+      @Nullable final DateTime fudgeTimestamp,
+      @Nullable final Filter filter,
+      final Interval interval,
+      final GroupByQueryConfig config
+  )
+  {
+    if (!canVectorize(query, storageAdapter, filter)) {
+      throw new ISE("Cannot vectorize");
+    }
+
+    return new BaseSequence<>(
+        new BaseSequence.IteratorMaker<Row, CloseableIterator<Row>>()
+        {
+          @Override
+          public CloseableIterator<Row> make()
+          {
+            final VectorCursor cursor = storageAdapter.makeVectorCursor(
+                Filters.toFilter(query.getDimFilter()),
+                interval,
+                query.getVirtualColumns(),
+                false,
+                QueryContexts.getVectorSize(query),
+                null
+            );
+
+            if (cursor == null) {
+              // Return empty iterator.
+              return new CloseableIterator<Row>()
+              {
+                @Override
+                public boolean hasNext()
+                {
+                  return false;
+                }
+
+                @Override
+                public Row next()
+                {
+                  throw new NoSuchElementException();
+                }
+
+                @Override
+                public void close()
+                {
+                  // Nothing to do.
+                }
+              };
+            }
+
+            try {
+              final VectorColumnSelectorFactory columnSelectorFactory = cursor.getColumnSelectorFactory();
+              final List<GroupByVectorColumnSelector> dimensions = query.getDimensions().stream().map(
+                  dimensionSpec ->
+                      DimensionHandlerUtils.makeVectorProcessor(
+                          dimensionSpec,
+                          GroupByVectorColumnStrategizer.instance(),
+                          columnSelectorFactory
+                      )
+              ).collect(Collectors.toList());
+
+              return new VectorGroupByEngineIterator(
+                  query,
+                  config,
+                  storageAdapter,
+                  cursor,
+                  interval,
+                  dimensions,
+                  processingBuffer,
+                  fudgeTimestamp
+              );
+            }
+            catch (Throwable e) {
+              try {
+                cursor.close();
+              }
+              catch (Throwable e2) {
+                e.addSuppressed(e2);
+              }
+              throw e;
+            }
+          }
+
+          @Override
+          public void cleanup(CloseableIterator<Row> iterFromMake)
+          {
+            try {
+              iterFromMake.close();
+            }
+            catch (IOException e) {
+              throw new RuntimeException(e);
+            }
+          }
+        }
+    );
+  }
+
+  private static class VectorGroupByEngineIterator implements CloseableIterator<Row>
+  {
+    private final GroupByQuery query;
+    private final GroupByQueryConfig querySpecificConfig;
+    private final StorageAdapter storageAdapter;
+    private final VectorCursor cursor;
+    private final List<GroupByVectorColumnSelector> selectors;
+    private final ByteBuffer processingBuffer;
+    private final DateTime fudgeTimestamp;
+    private final int keySize;
+    private final int[] keySpace;
+    private final Grouper.KeySerde<ByteBuffer> keySerde;
+    private final VectorGrouper vectorGrouper;
+
+    @Nullable
+    private final VectorCursorGranularizer granulizer;
+
+    // Granularity-bucket iterator and current bucket.
+    private final Iterator<Interval> bucketIterator;
+
+    @Nullable
+    private Interval bucketInterval;
+
+    private int partiallyAggregatedRows = -1;
+
+    @Nullable
+    private CloseableGrouperIterator<ByteBuffer, Row> delegate = null;
+
+    VectorGroupByEngineIterator(
+        final GroupByQuery query,
+        final GroupByQueryConfig config,
+        final StorageAdapter storageAdapter,
+        final VectorCursor cursor,
+        final Interval queryInterval,
+        final List<GroupByVectorColumnSelector> selectors,
+        final ByteBuffer processingBuffer,
+        @Nullable final DateTime fudgeTimestamp
+    )
+    {
+      this.query = query;
+      this.querySpecificConfig = config;
+      this.storageAdapter = storageAdapter;
+      this.cursor = cursor;
+      this.selectors = selectors;
+      this.processingBuffer = processingBuffer;
+      this.fudgeTimestamp = fudgeTimestamp;
+      this.keySize = selectors.stream().mapToInt(GroupByVectorColumnSelector::getGroupingKeySize).sum();
+      this.keySpace = new int[keySize * cursor.getMaxVectorSize()];
+      this.keySerde = new ByteBufferKeySerde(keySize * Integer.BYTES);
+      this.vectorGrouper = makeGrouper();
+      this.granulizer = VectorCursorGranularizer.create(storageAdapter, cursor, query.getGranularity(), queryInterval);
+
+      if (granulizer != null) {
+        this.bucketIterator = granulizer.getBucketIterable().iterator();
+      } else {
+        this.bucketIterator = Collections.emptyIterator();
+      }
+
+      this.bucketInterval = this.bucketIterator.hasNext() ? this.bucketIterator.next() : null;
+    }
+
+    @Override
+    public Row next()
+    {
+      if (delegate == null || !delegate.hasNext()) {
+        throw new NoSuchElementException();
+      }
+
+      return delegate.next();
+    }
+
+    @Override
+    public boolean hasNext()
+    {
+      if (delegate != null && delegate.hasNext()) {
+        return true;
+      } else {
+        final boolean moreToRead = !cursor.isDone() || partiallyAggregatedRows >= 0;
+
+        if (bucketInterval != null && moreToRead) {
+          while (delegate == null || !delegate.hasNext()) {
+            if (delegate != null) {
+              delegate.close();
+              vectorGrouper.reset();
+            }
+
+            delegate = initNewDelegate();
+          }
+          return true;
+        } else {
+          return false;
+        }
+      }
+    }
+
+    @Override
+    public void remove()
+    {
+      throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public void close()
+    {
+      cursor.close();
+
+      if (delegate != null) {
+        delegate.close();
+      }
+    }
+
+    private VectorGrouper makeGrouper()
+    {
+      final VectorGrouper grouper;
+
+      final int cardinalityForArrayAggregation = GroupByQueryEngineV2.getCardinalityForArrayAggregation(
+          querySpecificConfig,
+          query,
+          storageAdapter,
+          processingBuffer
+      );
+
+      if (cardinalityForArrayAggregation >= 0) {
+        grouper = new BufferArrayGrouper(
+            Suppliers.ofInstance(processingBuffer),
+            AggregatorAdapters.factorizeVector(
+                cursor.getColumnSelectorFactory(),
+                query.getAggregatorSpecs()
+            ),
+            cardinalityForArrayAggregation
+        );
+      } else {
+        grouper = new BufferHashGrouper<>(
+            Suppliers.ofInstance(processingBuffer),
+            keySerde,
+            AggregatorAdapters.factorizeVector(
+                cursor.getColumnSelectorFactory(),
+                query.getAggregatorSpecs()
+            ),
+            querySpecificConfig.getBufferGrouperMaxSize(),
+            querySpecificConfig.getBufferGrouperMaxLoadFactor(),
+            querySpecificConfig.getBufferGrouperInitialBuckets(),
+            true
+        );
+      }
+
+      grouper.initVectorized(cursor.getMaxVectorSize());
+
+      return grouper;
+    }
+
+    private CloseableGrouperIterator<ByteBuffer, Row> initNewDelegate()
+    {
+      // Method must not be called unless there's a current bucketInterval.
+      assert bucketInterval != null;
+
+      final DateTime timestamp = fudgeTimestamp != null
+                                 ? fudgeTimestamp
+                                 : query.getGranularity().toDateTime(bucketInterval.getStartMillis());
+
+      while (!cursor.isDone()) {
+        final int startOffset;
+
+        if (partiallyAggregatedRows < 0) {
+          granulizer.setCurrentOffsets(bucketInterval);
+          startOffset = granulizer.getStartOffset();
+        } else {
+          startOffset = granulizer.getStartOffset() + partiallyAggregatedRows;
+        }
+
+        if (granulizer.getEndOffset() > startOffset) {
+          // Write keys to the keySpace.
+          int keyOffset = 0;
+          for (final GroupByVectorColumnSelector selector : selectors) {
+            selector.writeKeys(keySpace, keySize, keyOffset, startOffset, granulizer.getEndOffset());
+            keyOffset += selector.getGroupingKeySize();
+          }
+
+          // Aggregate this vector.
+          final AggregateResult result = vectorGrouper.aggregateVector(
+              keySpace,
+              startOffset,
+              granulizer.getEndOffset()
+          );
+
+          if (result.isOk()) {
+            partiallyAggregatedRows = -1;
+          } else {
+            if (partiallyAggregatedRows < 0) {
+              partiallyAggregatedRows = result.getCount();
+            } else {
+              partiallyAggregatedRows += result.getCount();
+            }
+          }
+        } else {
+          partiallyAggregatedRows = -1;
+        }
+
+        if (partiallyAggregatedRows >= 0) {
+          break;
+        } else if (!granulizer.advanceCursorWithinBucket()) {
+          // Advance bucketInterval.
+          bucketInterval = bucketIterator.hasNext() ? bucketIterator.next() : null;
+          break;
+        }
+      }
+
+      return new CloseableGrouperIterator<>(
+          vectorGrouper.iterator(),
+          entry -> {
+            Map<String, Object> theMap = new LinkedHashMap<>();
+
+            // Add dimensions.
+            int keyOffset = 0;
+            for (int i = 0; i < selectors.size(); i++) {
+              final GroupByVectorColumnSelector selector = selectors.get(i);
+
+              selector.writeKeyToResultRow(
+                  query.getDimensions().get(i).getOutputName(),
+                  entry.getKey(),
+                  keyOffset,
+                  theMap
+              );
+
+              keyOffset += selector.getGroupingKeySize();
+            }
+
+            // Convert dimension values to desired output types, possibly.
+            GroupByQueryEngineV2.convertRowTypesToOutputTypes(query.getDimensions(), theMap);
+
+            // Add aggregations.
+            for (int i = 0; i < entry.getValues().length; i++) {
+              theMap.put(query.getAggregatorSpecs().get(i).getName(), entry.getValues()[i]);
+            }
+
+            return new MapBasedRow(timestamp, theMap);
+          },
+          vectorGrouper
+      );
+    }
+  }
+}
diff --git a/processing/src/main/java/org/apache/druid/query/search/DefaultSearchQueryMetrics.java b/processing/src/main/java/org/apache/druid/query/search/DefaultSearchQueryMetrics.java
index 494c261..ca1fe18 100644
--- a/processing/src/main/java/org/apache/druid/query/search/DefaultSearchQueryMetrics.java
+++ b/processing/src/main/java/org/apache/druid/query/search/DefaultSearchQueryMetrics.java
@@ -161,6 +161,12 @@ public class DefaultSearchQueryMetrics implements SearchQueryMetrics
   }
 
   @Override
+  public void vectorized(final boolean vectorized)
+  {
+    delegateQueryMetrics.vectorized(vectorized);
+  }
+
+  @Override
   public BitmapResultFactory<?> makeBitmapResultFactory(BitmapFactory factory)
   {
     return delegateQueryMetrics.makeBitmapResultFactory(factory);
diff --git a/processing/src/main/java/org/apache/druid/query/select/DefaultSelectQueryMetrics.java b/processing/src/main/java/org/apache/druid/query/select/DefaultSelectQueryMetrics.java
index c522987..80d9245 100644
--- a/processing/src/main/java/org/apache/druid/query/select/DefaultSelectQueryMetrics.java
+++ b/processing/src/main/java/org/apache/druid/query/select/DefaultSelectQueryMetrics.java
@@ -160,6 +160,12 @@ public class DefaultSelectQueryMetrics implements SelectQueryMetrics
   }
 
   @Override
+  public void vectorized(final boolean vectorized)
+  {
+    delegateQueryMetrics.vectorized(vectorized);
+  }
+
+  @Override
   public BitmapResultFactory<?> makeBitmapResultFactory(BitmapFactory factory)
   {
     return delegateQueryMetrics.makeBitmapResultFactory(factory);
diff --git a/processing/src/main/java/org/apache/druid/query/timeseries/TimeseriesQueryEngine.java b/processing/src/main/java/org/apache/druid/query/timeseries/TimeseriesQueryEngine.java
index 0de74bb..87c24ff 100644
--- a/processing/src/main/java/org/apache/druid/query/timeseries/TimeseriesQueryEngine.java
+++ b/processing/src/main/java/org/apache/druid/query/timeseries/TimeseriesQueryEngine.java
@@ -19,24 +19,65 @@
 
 package org.apache.druid.query.timeseries;
 
-import com.google.common.base.Function;
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.collect.Iterables;
+import com.google.inject.Inject;
+import org.apache.druid.collections.NonBlockingPool;
+import org.apache.druid.collections.ResourceHolder;
+import org.apache.druid.collections.StupidPool;
+import org.apache.druid.guice.annotations.Global;
+import org.apache.druid.java.util.common.ISE;
+import org.apache.druid.java.util.common.granularity.Granularity;
 import org.apache.druid.java.util.common.guava.Sequence;
+import org.apache.druid.java.util.common.guava.Sequences;
+import org.apache.druid.java.util.common.io.Closer;
+import org.apache.druid.query.QueryContexts;
 import org.apache.druid.query.QueryRunnerHelper;
 import org.apache.druid.query.Result;
 import org.apache.druid.query.aggregation.Aggregator;
+import org.apache.druid.query.aggregation.AggregatorAdapters;
 import org.apache.druid.query.aggregation.AggregatorFactory;
 import org.apache.druid.query.filter.Filter;
-import org.apache.druid.segment.Cursor;
+import org.apache.druid.query.vector.VectorCursorGranularizer;
 import org.apache.druid.segment.SegmentMissingException;
 import org.apache.druid.segment.StorageAdapter;
 import org.apache.druid.segment.filter.Filters;
+import org.apache.druid.segment.vector.VectorColumnSelectorFactory;
+import org.apache.druid.segment.vector.VectorCursor;
+import org.joda.time.Interval;
 
+import javax.annotation.Nullable;
+import java.nio.ByteBuffer;
+import java.util.Collections;
 import java.util.List;
+import java.util.Objects;
 
 /**
+ *
  */
 public class TimeseriesQueryEngine
 {
+  private final NonBlockingPool<ByteBuffer> bufferPool;
+
+  /**
+   * Constructor for tests. In production, the @Inject constructor is used instead.
+   */
+  @VisibleForTesting
+  public TimeseriesQueryEngine()
+  {
+    this.bufferPool = new StupidPool<>("dummy", () -> ByteBuffer.allocate(1000000));
+  }
+
+  @Inject
+  public TimeseriesQueryEngine(final @Global NonBlockingPool<ByteBuffer> bufferPool)
+  {
+    this.bufferPool = bufferPool;
+  }
+
+  /**
+   * Run a single-segment, single-interval timeseries query on a particular adapter. The query must have been
+   * scoped down to a single interval before calling this method.
+   */
   public Sequence<Result<TimeseriesResultValue>> process(final TimeseriesQuery query, final StorageAdapter adapter)
   {
     if (adapter == null) {
@@ -45,65 +86,210 @@ public class TimeseriesQueryEngine
       );
     }
 
-    final Filter filter = Filters.convertToCNFFromQueryContext(query, Filters.toFilter(query.getDimensionsFilter()));
+    final Filter filter = Filters.convertToCNFFromQueryContext(query, Filters.toFilter(query.getFilter()));
+    final Interval interval = Iterables.getOnlyElement(query.getIntervals());
+    final Granularity gran = query.getGranularity();
+    final boolean descending = query.isDescending();
+
+    final boolean doVectorize = QueryContexts.getVectorize(query).shouldVectorize(
+        adapter.canVectorize(filter, query.getVirtualColumns(), descending)
+        && query.getAggregatorSpecs().stream().allMatch(AggregatorFactory::canVectorize)
+    );
+
+    final Sequence<Result<TimeseriesResultValue>> result;
+
+    if (doVectorize) {
+      result = processVectorized(query, adapter, filter, interval, gran, descending);
+    } else {
+      result = processNonVectorized(query, adapter, filter, interval, gran, descending);
+    }
+
     final int limit = query.getLimit();
-    Sequence<Result<TimeseriesResultValue>> result = generateTimeseriesResult(adapter, query, filter);
     if (limit < Integer.MAX_VALUE) {
       return result.limit(limit);
+    } else {
+      return result;
+    }
+  }
+
+  private Sequence<Result<TimeseriesResultValue>> processVectorized(
+      final TimeseriesQuery query,
+      final StorageAdapter adapter,
+      @Nullable final Filter filter,
+      final Interval queryInterval,
+      final Granularity gran,
+      final boolean descending
+  )
+  {
+    final boolean skipEmptyBuckets = query.isSkipEmptyBuckets();
+    final List<AggregatorFactory> aggregatorSpecs = query.getAggregatorSpecs();
+
+    final VectorCursor cursor = adapter.makeVectorCursor(
+        filter,
+        queryInterval,
+        query.getVirtualColumns(),
+        descending,
+        QueryContexts.getVectorSize(query),
+        null
+    );
+
+    if (cursor == null) {
+      return Sequences.empty();
+    }
+
+    final Closer closer = Closer.create();
+    closer.register(cursor);
+
+    try {
+      final VectorCursorGranularizer granularizer = VectorCursorGranularizer.create(
+          adapter,
+          cursor,
+          gran,
+          queryInterval
+      );
+
+      if (granularizer == null) {
+        return Sequences.empty();
+      }
+
+      final VectorColumnSelectorFactory columnSelectorFactory = cursor.getColumnSelectorFactory();
+      final AggregatorAdapters aggregators = closer.register(
+          AggregatorAdapters.factorizeVector(columnSelectorFactory, query.getAggregatorSpecs())
+      );
+
+      final ResourceHolder<ByteBuffer> bufferHolder = closer.register(bufferPool.take());
+
+      final ByteBuffer buffer = bufferHolder.get();
+
+      if (aggregators.spaceNeeded() > buffer.remaining()) {
+        throw new ISE(
+            "Not enough space for aggregators, needed [%,d] bytes but have only [%,d].",
+            aggregators.spaceNeeded(),
+            buffer.remaining()
+        );
+      }
+
+      return Sequences.withBaggage(
+          Sequences
+              .simple(granularizer.getBucketIterable())
+              .map(
+                  bucketInterval -> {
+                    // Whether or not the current bucket is empty
+                    boolean emptyBucket = true;
+
+                    while (!cursor.isDone()) {
+                      granularizer.setCurrentOffsets(bucketInterval);
+
+                      if (granularizer.getEndOffset() > granularizer.getStartOffset()) {
+                        if (emptyBucket) {
+                          aggregators.init(buffer, 0);
+                        }
+
+                        aggregators.aggregateVector(
+                            buffer,
+                            0,
+                            granularizer.getStartOffset(),
+                            granularizer.getEndOffset()
+                        );
+
+                        emptyBucket = false;
+                      }
+
+                      if (!granularizer.advanceCursorWithinBucket()) {
+                        break;
+                      }
+                    }
+
+                    if (emptyBucket && skipEmptyBuckets) {
+                      // Return null, will get filtered out later by the Objects::nonNull filter.
+                      return null;
+                    }
+
+                    final TimeseriesResultBuilder bob = new TimeseriesResultBuilder(
+                        gran.toDateTime(bucketInterval.getStartMillis())
+                    );
+
+                    if (emptyBucket) {
+                      aggregators.init(buffer, 0);
+                    }
+
+                    for (int i = 0; i < aggregatorSpecs.size(); i++) {
+                      bob.addMetric(
+                          aggregatorSpecs.get(i).getName(),
+                          aggregators.get(buffer, 0, i)
+                      );
+                    }
+
+                    return bob.build();
+                  }
+              )
+              .filter(Objects::nonNull),
+          closer
+      );
+    }
+    catch (Throwable t1) {
+      try {
+        closer.close();
+      }
+      catch (Throwable t2) {
+        t1.addSuppressed(t2);
+      }
+      throw t1;
     }
-    return result;
   }
 
-  private Sequence<Result<TimeseriesResultValue>> generateTimeseriesResult(StorageAdapter adapter, TimeseriesQuery query, Filter filter)
+  private Sequence<Result<TimeseriesResultValue>> processNonVectorized(
+      final TimeseriesQuery query,
+      final StorageAdapter adapter,
+      @Nullable final Filter filter,
+      final Interval queryInterval,
+      final Granularity gran,
+      final boolean descending
+  )
   {
+    final boolean skipEmptyBuckets = query.isSkipEmptyBuckets();
+    final List<AggregatorFactory> aggregatorSpecs = query.getAggregatorSpecs();
+
     return QueryRunnerHelper.makeCursorBasedQuery(
         adapter,
-        query.getQuerySegmentSpec().getIntervals(),
+        Collections.singletonList(queryInterval),
         filter,
         query.getVirtualColumns(),
-        query.isDescending(),
-        query.getGranularity(),
-        new Function<Cursor, Result<TimeseriesResultValue>>()
-        {
-          private final boolean skipEmptyBuckets = query.isSkipEmptyBuckets();
-          private final List<AggregatorFactory> aggregatorSpecs = query.getAggregatorSpecs();
-
-          @Override
-          public Result<TimeseriesResultValue> apply(Cursor cursor)
-          {
-            if (skipEmptyBuckets && cursor.isDone()) {
-              return null;
-            }
+        descending,
+        gran,
+        cursor -> {
+          if (skipEmptyBuckets && cursor.isDone()) {
+            return null;
+          }
 
-            Aggregator[] aggregators = new Aggregator[aggregatorSpecs.size()];
-            String[] aggregatorNames = new String[aggregatorSpecs.size()];
+          Aggregator[] aggregators = new Aggregator[aggregatorSpecs.size()];
+          String[] aggregatorNames = new String[aggregatorSpecs.size()];
 
-            for (int i = 0; i < aggregatorSpecs.size(); i++) {
-              aggregators[i] = aggregatorSpecs.get(i).factorize(cursor.getColumnSelectorFactory());
-              aggregatorNames[i] = aggregatorSpecs.get(i).getName();
-            }
+          for (int i = 0; i < aggregatorSpecs.size(); i++) {
+            aggregators[i] = aggregatorSpecs.get(i).factorize(cursor.getColumnSelectorFactory());
+            aggregatorNames[i] = aggregatorSpecs.get(i).getName();
+          }
 
-            try {
-              while (!cursor.isDone()) {
-                for (Aggregator aggregator : aggregators) {
-                  aggregator.aggregate();
-                }
-                cursor.advance();
+          try {
+            while (!cursor.isDone()) {
+              for (Aggregator aggregator : aggregators) {
+                aggregator.aggregate();
               }
-              TimeseriesResultBuilder bob = new TimeseriesResultBuilder(cursor.getTime());
+              cursor.advance();
+            }
 
-              for (int i = 0; i < aggregatorSpecs.size(); i++) {
-                bob.addMetric(aggregatorNames[i], aggregators[i]);
-              }
+            TimeseriesResultBuilder bob = new TimeseriesResultBuilder(cursor.getTime());
 
-              Result<TimeseriesResultValue> retVal = bob.build();
-              return retVal;
+            for (int i = 0; i < aggregatorSpecs.size(); i++) {
+              bob.addMetric(aggregatorNames[i], aggregators[i].get());
             }
-            finally {
-              // cleanup
-              for (Aggregator agg : aggregators) {
-                agg.close();
-              }
+
+            return bob.build();
+          }
+          finally {
+            // cleanup
+            for (Aggregator agg : aggregators) {
+              agg.close();
             }
           }
         }
diff --git a/processing/src/main/java/org/apache/druid/query/timeseries/TimeseriesQueryQueryToolChest.java b/processing/src/main/java/org/apache/druid/query/timeseries/TimeseriesQueryQueryToolChest.java
index 0ae9a70..686ed68 100644
--- a/processing/src/main/java/org/apache/druid/query/timeseries/TimeseriesQueryQueryToolChest.java
+++ b/processing/src/main/java/org/apache/druid/query/timeseries/TimeseriesQueryQueryToolChest.java
@@ -227,7 +227,7 @@ public class TimeseriesQueryQueryToolChest extends QueryToolChest<Result<Timeser
     final DateTime start = query.getIntervals().isEmpty() ? DateTimes.EPOCH : query.getIntervals().get(0).getStart();
     TimeseriesResultBuilder bob = new TimeseriesResultBuilder(start);
     for (int i = 0; i < aggregatorSpecs.size(); i++) {
-      bob.addMetric(aggregatorNames[i], aggregators[i]);
+      bob.addMetric(aggregatorNames[i], aggregators[i].get());
       aggregators[i].close();
     }
     return bob.build();
diff --git a/processing/src/main/java/org/apache/druid/query/timeseries/TimeseriesResultBuilder.java b/processing/src/main/java/org/apache/druid/query/timeseries/TimeseriesResultBuilder.java
index bd389cd..2332f53 100644
--- a/processing/src/main/java/org/apache/druid/query/timeseries/TimeseriesResultBuilder.java
+++ b/processing/src/main/java/org/apache/druid/query/timeseries/TimeseriesResultBuilder.java
@@ -20,7 +20,6 @@
 package org.apache.druid.query.timeseries;
 
 import org.apache.druid.query.Result;
-import org.apache.druid.query.aggregation.Aggregator;
 import org.joda.time.DateTime;
 
 import java.util.HashMap;
@@ -41,15 +40,15 @@ public class TimeseriesResultBuilder
     this.timestamp = timestamp;
   }
 
-  public TimeseriesResultBuilder addMetric(String name, Aggregator aggregator)
+  public TimeseriesResultBuilder addMetric(String name, Object value)
   {
-    metricValues.put(name, aggregator.get());
+    metricValues.put(name, value);
     return this;
   }
 
   public Result<TimeseriesResultValue> build()
   {
-    return new Result<TimeseriesResultValue>(
+    return new Result<>(
         timestamp,
         new TimeseriesResultValue(metricValues)
     );
diff --git a/processing/src/main/java/org/apache/druid/query/topn/types/StringTopNColumnSelectorStrategy.java b/processing/src/main/java/org/apache/druid/query/topn/types/StringTopNColumnSelectorStrategy.java
index dd30c36..f5e838d 100644
--- a/processing/src/main/java/org/apache/druid/query/topn/types/StringTopNColumnSelectorStrategy.java
+++ b/processing/src/main/java/org/apache/druid/query/topn/types/StringTopNColumnSelectorStrategy.java
@@ -25,6 +25,7 @@ import org.apache.druid.query.topn.TopNParams;
 import org.apache.druid.query.topn.TopNQuery;
 import org.apache.druid.query.topn.TopNResultBuilder;
 import org.apache.druid.segment.Cursor;
+import org.apache.druid.segment.DimensionDictionarySelector;
 import org.apache.druid.segment.DimensionHandlerUtils;
 import org.apache.druid.segment.DimensionSelector;
 import org.apache.druid.segment.StorageAdapter;
@@ -87,7 +88,7 @@ public class StringTopNColumnSelectorStrategy
       Map<Comparable<?>, Aggregator[]> aggregatesStore
   )
   {
-    if (selector.getValueCardinality() != DimensionSelector.CARDINALITY_UNKNOWN) {
+    if (selector.getValueCardinality() != DimensionDictionarySelector.CARDINALITY_UNKNOWN) {
       return dimExtractionScanAndAggregateWithCardinalityKnown(query, cursor, selector, rowSelector, aggregatesStore);
     } else {
       return dimExtractionScanAndAggregateWithCardinalityUnknown(query, cursor, selector, aggregatesStore);
diff --git a/processing/src/main/java/org/apache/druid/query/vector/VectorCursorGranularizer.java b/processing/src/main/java/org/apache/druid/query/vector/VectorCursorGranularizer.java
new file mode 100644
index 0000000..163befc
--- /dev/null
+++ b/processing/src/main/java/org/apache/druid/query/vector/VectorCursorGranularizer.java
@@ -0,0 +1,173 @@
+/*
+ * 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.query.vector;
+
+import com.google.common.collect.Iterables;
+import org.apache.druid.java.util.common.granularity.Granularity;
+import org.apache.druid.segment.StorageAdapter;
+import org.apache.druid.segment.column.ColumnHolder;
+import org.apache.druid.segment.vector.VectorCursor;
+import org.apache.druid.segment.vector.VectorValueSelector;
+import org.joda.time.DateTime;
+import org.joda.time.Interval;
+
+import javax.annotation.Nullable;
+
+/**
+ * Class that helps vectorized query engines handle "granularity" parameters. Nonvectorized engines have it handled
+ * for them by the StorageAdapter. Vectorized engines don't, because they can get efficiency gains by pushing
+ * granularity handling into the engine layer.
+ */
+public class VectorCursorGranularizer
+{
+  // And a cursor that has been made from it.
+  private final VectorCursor cursor;
+
+  // Iterable that iterates over time buckets.
+  private final Iterable<Interval> bucketIterable;
+
+  // Vector selector for the "__time" column.
+  @Nullable
+  private final VectorValueSelector timeSelector;
+
+  // Current time vector.
+  @Nullable
+  private long[] timestamps = null;
+
+  // Offset into the vector that we should start reading from.
+  private int startOffset = 0;
+
+  // Offset into the vector that is one past the last one we should read.
+  private int endOffset = 0;
+
+  private VectorCursorGranularizer(
+      VectorCursor cursor,
+      Iterable<Interval> bucketIterable,
+      @Nullable VectorValueSelector timeSelector
+  )
+  {
+    this.cursor = cursor;
+    this.bucketIterable = bucketIterable;
+    this.timeSelector = timeSelector;
+  }
+
+  @Nullable
+  public static VectorCursorGranularizer create(
+      final StorageAdapter storageAdapter,
+      final VectorCursor cursor,
+      final Granularity granularity,
+      final Interval queryInterval
+  )
+  {
+    final DateTime minTime = storageAdapter.getMinTime();
+    final DateTime maxTime = storageAdapter.getMaxTime();
+
+    final Interval storageAdapterInterval = new Interval(minTime, granularity.bucketEnd(maxTime));
+    final Interval clippedQueryInterval = queryInterval.overlap(storageAdapterInterval);
+
+    if (clippedQueryInterval == null) {
+      return null;
+    }
+
+    final Iterable<Interval> bucketIterable = granularity.getIterable(clippedQueryInterval);
+    final Interval firstBucket = granularity.bucket(clippedQueryInterval.getStart());
+
+    final VectorValueSelector timeSelector;
+    if (firstBucket.contains(clippedQueryInterval)) {
+      // Only one bucket, no need to read the time column.
+      assert Iterables.size(bucketIterable) == 1;
+      timeSelector = null;
+    } else {
+      // Multiple buckets, need to read the time column to know when we move from one to the next.
+      timeSelector = cursor.getColumnSelectorFactory().makeValueSelector(ColumnHolder.TIME_COLUMN_NAME);
+    }
+
+    return new VectorCursorGranularizer(cursor, bucketIterable, timeSelector);
+  }
+
+  public void setCurrentOffsets(final Interval bucketInterval)
+  {
+    final long timeStart = bucketInterval.getStartMillis();
+    final long timeEnd = bucketInterval.getEndMillis();
+
+    int vectorSize = cursor.getCurrentVectorSize();
+    endOffset = 0;
+
+    if (timeSelector != null) {
+      if (timestamps == null) {
+        timestamps = timeSelector.getLongVector();
+      }
+
+      // Skip "offset" to start of bucketInterval.
+      while (startOffset < vectorSize && timestamps[startOffset] < timeStart) {
+        startOffset++;
+      }
+
+      // Find end of bucketInterval.
+      for (endOffset = vectorSize - 1;
+           endOffset >= startOffset && timestamps[endOffset] >= timeEnd;
+           endOffset--) {
+        // nothing needed, "for" is doing the work.
+      }
+
+      // Adjust: endOffset is now pointing at the last row to aggregate, but we want it
+      // to be one _past_ the last row.
+      endOffset++;
+    } else {
+      endOffset = vectorSize;
+    }
+  }
+
+  /**
+   * Return true, and advances the cursor, if it can be advanced within the current time bucket. Otherwise, returns
+   * false and does nothing else.
+   */
+  public boolean advanceCursorWithinBucket()
+  {
+    if (endOffset == cursor.getCurrentVectorSize()) {
+      cursor.advance();
+
+      if (timeSelector != null && !cursor.isDone()) {
+        timestamps = timeSelector.getLongVector();
+      }
+
+      startOffset = 0;
+
+      return true;
+    } else {
+      return false;
+    }
+  }
+
+  public Iterable<Interval> getBucketIterable()
+  {
+    return bucketIterable;
+  }
+
+  public int getStartOffset()
+  {
+    return startOffset;
+  }
+
+  public int getEndOffset()
+  {
+    return endOffset;
+  }
+}
diff --git a/processing/src/main/java/org/apache/druid/segment/ColumnSelectorFactory.java b/processing/src/main/java/org/apache/druid/segment/ColumnSelectorFactory.java
index 3d51c27..07e66a6 100644
--- a/processing/src/main/java/org/apache/druid/segment/ColumnSelectorFactory.java
+++ b/processing/src/main/java/org/apache/druid/segment/ColumnSelectorFactory.java
@@ -27,6 +27,8 @@ import javax.annotation.Nullable;
 
 /**
  * Factory class for MetricSelectors
+ *
+ * @see org.apache.druid.segment.vector.VectorColumnSelectorFactory, the vectorized version
  */
 @PublicApi
 public interface ColumnSelectorFactory
diff --git a/processing/src/main/java/org/apache/druid/segment/Cursor.java b/processing/src/main/java/org/apache/druid/segment/Cursor.java
index 7964485..645caee 100644
--- a/processing/src/main/java/org/apache/druid/segment/Cursor.java
+++ b/processing/src/main/java/org/apache/druid/segment/Cursor.java
@@ -23,13 +23,15 @@ import org.joda.time.DateTime;
 
 /**
  * Cursor is an interface for iteration over a range of data points, used during query execution. {@link
- * QueryableIndexStorageAdapter.QueryableIndexCursor} is an implementation for historical segments, and {@link
+ * QueryableIndexCursorSequenceBuilder.QueryableIndexCursor} is an implementation for historical segments, and {@link
  * org.apache.druid.segment.incremental.IncrementalIndexStorageAdapter.IncrementalIndexCursor} is an implementation for {@link
  * org.apache.druid.segment.incremental.IncrementalIndex}.
  *
  * Cursor is conceptually similar to {@link TimeAndDimsPointer}, but the latter is used for historical segment creation
  * rather than query execution (as Cursor). If those abstractions could be collapsed (and if it is worthwhile) is yet to
  * be determined.
+ *
+ * @see org.apache.druid.segment.vector.VectorCursor, the vectorized version
  */
 public interface Cursor
 {
diff --git a/processing/src/main/java/org/apache/druid/segment/CursorFactory.java b/processing/src/main/java/org/apache/druid/segment/CursorFactory.java
index c3e5503..809e7b8 100644
--- a/processing/src/main/java/org/apache/druid/segment/CursorFactory.java
+++ b/processing/src/main/java/org/apache/druid/segment/CursorFactory.java
@@ -23,14 +23,36 @@ import org.apache.druid.java.util.common.granularity.Granularity;
 import org.apache.druid.java.util.common.guava.Sequence;
 import org.apache.druid.query.QueryMetrics;
 import org.apache.druid.query.filter.Filter;
+import org.apache.druid.segment.vector.VectorCursor;
 import org.joda.time.Interval;
 
 import javax.annotation.Nullable;
 
 /**
+ * Interface extended by {@link StorageAdapter}, which gives them the power to create cursors.
+ *
+ * @see StorageAdapter
  */
 public interface CursorFactory
 {
+  /**
+   * Returns true if the provided combination of parameters can be handled by "makeVectorCursor".
+   *
+   * Query engines should use this before running in vectorized mode, and be prepared to fall back to non-vectorized
+   * mode if this method returns false.
+   */
+  default boolean canVectorize(
+      @Nullable Filter filter,
+      VirtualColumns virtualColumns,
+      boolean descending
+  )
+  {
+    return false;
+  }
+
+  /**
+   * Creates a sequence of Cursors, one for each time-granular bucket (based on the provided Granularity).
+   */
   Sequence<Cursor> makeCursors(
       @Nullable Filter filter,
       Interval interval,
@@ -39,4 +61,25 @@ public interface CursorFactory
       boolean descending,
       @Nullable QueryMetrics<?> queryMetrics
   );
+
+  /**
+   * Creates a VectorCursor. Unlike the Cursor returned by "makeCursor", there is just one of these. Hence, this method
+   * does not take a "granularity" parameter. Before calling this method, check "canVectorize" to see if the call you
+   * are about to make will throw an error or not.
+   *
+   * Returns null if there is no data to walk over (for example, if the "interval" does not overlap the data interval
+   * of this segment).
+   */
+  @Nullable
+  default VectorCursor makeVectorCursor(
+      @Nullable Filter filter,
+      Interval interval,
+      VirtualColumns virtualColumns,
+      boolean descending,
+      int vectorSize,
+      @Nullable QueryMetrics<?> queryMetrics
+  )
+  {
+    throw new UnsupportedOperationException("Cannot vectorize. Check 'canVectorize' before calling 'makeVectorCursor'.");
+  }
 }
diff --git a/processing/src/main/java/org/apache/druid/segment/DimensionDictionarySelector.java b/processing/src/main/java/org/apache/druid/segment/DimensionDictionarySelector.java
new file mode 100644
index 0000000..a02c025
--- /dev/null
+++ b/processing/src/main/java/org/apache/druid/segment/DimensionDictionarySelector.java
@@ -0,0 +1,104 @@
+/*
+ * 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;
+
+import org.apache.druid.query.monomorphicprocessing.CalledFromHotLoop;
+
+import javax.annotation.Nullable;
+
+/**
+ * Interface containing dictionary-related methods common to {@link DimensionSelector},
+ * {@link org.apache.druid.segment.vector.SingleValueDimensionVectorSelector}, and
+ * {@link org.apache.druid.segment.vector.MultiValueDimensionVectorSelector}.
+ */
+public interface DimensionDictionarySelector
+{
+  int CARDINALITY_UNKNOWN = -1;
+
+  /**
+   * Value cardinality is the cardinality of the different occurring values.  If there were 4 rows:
+   *
+   * A,B
+   * A
+   * B
+   * A
+   *
+   * Value cardinality would be 2.
+   *
+   * Cardinality may be unknown (e.g. the selector used by IncrementalIndex while reading input rows),
+   * in which case this method will return -1. If cardinality is unknown, you should assume this
+   * dimension selector has no dictionary, and avoid storing ids, calling "lookupId", or calling "lookupName"
+   * outside of the context of operating on a single row.
+   *
+   * @return the value cardinality, or -1 if unknown.
+   */
+  int getValueCardinality();
+
+  /**
+   * The Name is the String name of the actual field.  It is assumed that storage layers convert names
+   * into id values which can then be used to get the string value.  For example
+   *
+   * A,B
+   * A
+   * A,B
+   * B
+   *
+   * getRow() would return
+   *
+   * getRow(0) =&gt; [0 1]
+   * getRow(1) =&gt; [0]
+   * getRow(2) =&gt; [0 1]
+   * getRow(3) =&gt; [1]
+   *
+   * and then lookupName would return:
+   *
+   * lookupName(0) =&gt; A
+   * lookupName(1) =&gt; B
+   *
+   * @param id id to lookup the field name for
+   *
+   * @return the field name for the given id
+   */
+  @CalledFromHotLoop
+  @Nullable
+  String lookupName(int id);
+
+  /**
+   * Returns true if it is possible to {@link #lookupName(int)} by ids from 0 to {@link #getValueCardinality()}
+   * before the rows with those ids are returned.
+   *
+   * <p>Returns false if {@link #lookupName(int)} could be called with ids, returned from the most recent row (or row
+   * vector) returned by this DimensionSelector, but not earlier. If {@link #getValueCardinality()} of this
+   * selector additionally returns {@link #CARDINALITY_UNKNOWN}, {@code lookupName()} couldn't be called with
+   * ids, returned by not the most recent row (or row vector), i. e. names for ids couldn't be looked up "later". If
+   * {@link #getValueCardinality()} returns a non-negative number, {@code lookupName()} could be called with any ids,
+   * returned from rows (or row vectors) returned since the creation of this DimensionSelector.
+   *
+   * <p>If {@link #lookupName(int)} is called with an ineligible id, result is undefined: exception could be thrown, or
+   * null returned, or some other random value.
+   */
+  boolean nameLookupPossibleInAdvance();
+
+  /**
+   * Returns {@link IdLookup} if available for this DimensionSelector, or null.
+   */
+  @Nullable
+  IdLookup idLookup();
+}
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 73f2f18..da4232d 100644
--- a/processing/src/main/java/org/apache/druid/segment/DimensionHandlerUtils.java
+++ b/processing/src/main/java/org/apache/druid/segment/DimensionHandlerUtils.java
@@ -26,15 +26,19 @@ import com.google.common.primitives.Floats;
 import org.apache.druid.common.guava.GuavaUtils;
 import org.apache.druid.data.input.impl.DimensionSchema.MultiValueHandling;
 import org.apache.druid.java.util.common.IAE;
+import org.apache.druid.java.util.common.ISE;
 import org.apache.druid.java.util.common.guava.Comparators;
 import org.apache.druid.java.util.common.parsers.ParseException;
 import org.apache.druid.query.ColumnSelectorPlus;
 import org.apache.druid.query.dimension.ColumnSelectorStrategy;
 import org.apache.druid.query.dimension.ColumnSelectorStrategyFactory;
+import org.apache.druid.query.dimension.DefaultDimensionSpec;
 import org.apache.druid.query.dimension.DimensionSpec;
+import org.apache.druid.query.dimension.VectorColumnStrategizer;
 import org.apache.druid.segment.column.ColumnCapabilities;
 import org.apache.druid.segment.column.ColumnCapabilitiesImpl;
 import org.apache.druid.segment.column.ValueType;
+import org.apache.druid.segment.vector.VectorColumnSelectorFactory;
 
 import javax.annotation.Nullable;
 import java.math.BigDecimal;
@@ -242,6 +246,80 @@ public final class DimensionHandlerUtils
     return strategyFactory.makeColumnSelectorStrategy(capabilities, selector);
   }
 
+  /**
+   * Equivalent to calling makeVectorProcessor(DefaultDimensionSpec.of(column), strategyFactory, selectorFactory).
+   *
+   * @see #makeVectorProcessor(DimensionSpec, VectorColumnStrategizer, VectorColumnSelectorFactory)
+   */
+  public static <T> T makeVectorProcessor(
+      final String column,
+      final VectorColumnStrategizer<T> strategyFactory,
+      final VectorColumnSelectorFactory selectorFactory
+  )
+  {
+    return makeVectorProcessor(DefaultDimensionSpec.of(column), strategyFactory, selectorFactory);
+  }
+
+  /**
+   * Creates "vector processors", which are objects that wrap a single vectorized input column and provide some
+   * functionality on top of it. Used by things like query engines and filter matchers.
+   *
+   * Supports the basic types STRING, LONG, DOUBLE, and FLOAT.
+   *
+   * @param dimensionSpec   dimensionSpec for the input to the processor
+   * @param strategyFactory object that encapsulates the knowledge about how to create processors
+   * @param selectorFactory column selector factory used for creating the vector processor
+   */
+  public static <T> T makeVectorProcessor(
+      final DimensionSpec dimensionSpec,
+      final VectorColumnStrategizer<T> strategyFactory,
+      final VectorColumnSelectorFactory selectorFactory
+  )
+  {
+    final ColumnCapabilities capabilities = getEffectiveCapabilities(
+        dimensionSpec,
+        selectorFactory.getColumnCapabilities(dimensionSpec.getDimension())
+    );
+
+    final ValueType type = capabilities.getType();
+
+    if (type == ValueType.STRING) {
+      if (capabilities.hasMultipleValues()) {
+        return strategyFactory.makeMultiValueDimensionStrategy(
+            selectorFactory.makeMultiValueDimensionSelector(dimensionSpec)
+        );
+      } else {
+        return strategyFactory.makeSingleValueDimensionStrategy(
+            selectorFactory.makeSingleValueDimensionSelector(dimensionSpec)
+        );
+      }
+    } else {
+      Preconditions.checkState(
+          dimensionSpec.getExtractionFn() == null && !dimensionSpec.mustDecorate(),
+          "Uh oh, was about to try to make a value selector for type[%s] with a dimensionSpec of class[%s] that "
+          + "requires decoration. Possible bug.",
+          type,
+          dimensionSpec.getClass().getName()
+      );
+
+      if (type == ValueType.LONG) {
+        return strategyFactory.makeLongStrategy(
+            selectorFactory.makeValueSelector(dimensionSpec.getDimension())
+        );
+      } else if (type == ValueType.FLOAT) {
+        return strategyFactory.makeFloatStrategy(
+            selectorFactory.makeValueSelector(dimensionSpec.getDimension())
+        );
+      } else if (type == ValueType.DOUBLE) {
+        return strategyFactory.makeDoubleStrategy(
+            selectorFactory.makeValueSelector(dimensionSpec.getDimension())
+        );
+      } else {
+        throw new ISE("Unsupported type[%s]", capabilities.getType());
+      }
+    }
+  }
+
   @Nullable
   public static String convertObjectToString(@Nullable Object valObj)
   {
diff --git a/processing/src/main/java/org/apache/druid/segment/DimensionSelector.java b/processing/src/main/java/org/apache/druid/segment/DimensionSelector.java
index 1c9bf97..8a0f2a7 100644
--- a/processing/src/main/java/org/apache/druid/segment/DimensionSelector.java
+++ b/processing/src/main/java/org/apache/druid/segment/DimensionSelector.java
@@ -36,12 +36,15 @@ import javax.annotation.Nullable;
 import java.util.Arrays;
 
 /**
+ * Selector for a string-typed column, either single- or multi-valued. This is named a "dimension" selector for legacy
+ * reasons: in the past, all Druid dimensions were string-typed.
+ *
+ * @see org.apache.druid.segment.vector.SingleValueDimensionVectorSelector, a vectorized version
+ * @see org.apache.druid.segment.vector.MultiValueDimensionVectorSelector, another vectorized version
  */
 @PublicApi
-public interface DimensionSelector extends ColumnValueSelector<Object>, HotLoopCallee
+public interface DimensionSelector extends ColumnValueSelector<Object>, DimensionDictionarySelector, HotLoopCallee
 {
-  int CARDINALITY_UNKNOWN = -1;
-
   /**
    * Returns the indexed values at the current position in this DimensionSelector.
    *
@@ -64,75 +67,6 @@ public interface DimensionSelector extends ColumnValueSelector<Object>, HotLoopC
   ValueMatcher makeValueMatcher(Predicate<String> predicate);
 
   /**
-   * Value cardinality is the cardinality of the different occurring values.  If there were 4 rows:
-   *
-   * A,B
-   * A
-   * B
-   * A
-   *
-   * Value cardinality would be 2.
-   *
-   * Cardinality may be unknown (e.g. the selector used by IncrementalIndex while reading input rows),
-   * in which case this method will return -1. If cardinality is unknown, you should assume this
-   * dimension selector has no dictionary, and avoid storing ids, calling "lookupId", or calling "lookupName"
-   * outside of the context of operating on a single row.
-   *
-   * @return the value cardinality, or -1 if unknown.
-   */
-  int getValueCardinality();
-
-  /**
-   * The Name is the String name of the actual field.  It is assumed that storage layers convert names
-   * into id values which can then be used to get the string value.  For example
-   *
-   * A,B
-   * A
-   * A,B
-   * B
-   *
-   * getRow() would return
-   *
-   * getRow(0) =&gt; [0 1]
-   * getRow(1) =&gt; [0]
-   * getRow(2) =&gt; [0 1]
-   * getRow(3) =&gt; [1]
-   *
-   * and then lookupName would return:
-   *
-   * lookupName(0) =&gt; A
-   * lookupName(1) =&gt; B
-   *
-   * @param id id to lookup the field name for
-   * @return the field name for the given id
-   */
-  @CalledFromHotLoop
-  @Nullable
-  String lookupName(int id);
-
-  /**
-   * Returns true if it is possible to {@link #lookupName(int)} by ids from 0 to {@link #getValueCardinality()}
-   * before the rows with those ids are returned.
-   *
-   * <p>Returns false if {@link #lookupName(int)} could be called with ids, returned from the most recent call of {@link
-   * #getRow()} on this DimensionSelector, but not earlier. If {@link #getValueCardinality()} of this DimensionSelector
-   * additionally returns {@link #CARDINALITY_UNKNOWN}, {@code lookupName()} couldn't be called with ids, returned by
-   * not the most recent call of {@link #getRow()}, i. e. names for ids couldn't be looked up "later". If {@link
-   * #getValueCardinality()} returns a non-negative number, {@code lookupName()} could be called with any ids, returned
-   * from {@code #getRow()} since the creation of this DimensionSelector.
-   *
-   * <p>If {@link #lookupName(int)} is called with an ineligible id, result is undefined: exception could be thrown, or
-   * null returned, or some other random value.
-   */
-  boolean nameLookupPossibleInAdvance();
-
-  /**
-   * Returns {@link IdLookup} if available for this DimensionSelector, or null.
-   */
-  @Nullable
-  IdLookup idLookup();
-
-  /**
    * @deprecated This method is marked as deprecated in DimensionSelector to minimize the probability of accidental
    * calling. "Polymorphism" of DimensionSelector should be used only when operating on {@link ColumnValueSelector}
    * objects.
diff --git a/processing/src/main/java/org/apache/druid/segment/DoubleDimensionIndexer.java b/processing/src/main/java/org/apache/druid/segment/DoubleDimensionIndexer.java
index 0ddacfb..55c0a2c 100644
--- a/processing/src/main/java/org/apache/druid/segment/DoubleDimensionIndexer.java
+++ b/processing/src/main/java/org/apache/druid/segment/DoubleDimensionIndexer.java
@@ -80,7 +80,7 @@ public class DoubleDimensionIndexer implements DimensionIndexer<Double, Double,
   @Override
   public int getCardinality()
   {
-    return DimensionSelector.CARDINALITY_UNKNOWN;
+    return DimensionDictionarySelector.CARDINALITY_UNKNOWN;
   }
 
   @Override
diff --git a/processing/src/main/java/org/apache/druid/segment/FloatDimensionIndexer.java b/processing/src/main/java/org/apache/druid/segment/FloatDimensionIndexer.java
index a17ebea..d688563 100644
--- a/processing/src/main/java/org/apache/druid/segment/FloatDimensionIndexer.java
+++ b/processing/src/main/java/org/apache/druid/segment/FloatDimensionIndexer.java
@@ -81,7 +81,7 @@ public class FloatDimensionIndexer implements DimensionIndexer<Float, Float, Flo
   @Override
   public int getCardinality()
   {
-    return DimensionSelector.CARDINALITY_UNKNOWN;
+    return DimensionDictionarySelector.CARDINALITY_UNKNOWN;
   }
 
   @Override
diff --git a/processing/src/main/java/org/apache/druid/segment/LongDimensionIndexer.java b/processing/src/main/java/org/apache/druid/segment/LongDimensionIndexer.java
index 6c69735..c43945a 100644
--- a/processing/src/main/java/org/apache/druid/segment/LongDimensionIndexer.java
+++ b/processing/src/main/java/org/apache/druid/segment/LongDimensionIndexer.java
@@ -81,7 +81,7 @@ public class LongDimensionIndexer implements DimensionIndexer<Long, Long, Long>
   @Override
   public int getCardinality()
   {
-    return DimensionSelector.CARDINALITY_UNKNOWN;
+    return DimensionDictionarySelector.CARDINALITY_UNKNOWN;
   }
 
   @Override
diff --git a/processing/src/main/java/org/apache/druid/segment/QueryableIndexCursorSequenceBuilder.java b/processing/src/main/java/org/apache/druid/segment/QueryableIndexCursorSequenceBuilder.java
new file mode 100644
index 0000000..ba30649
--- /dev/null
+++ b/processing/src/main/java/org/apache/druid/segment/QueryableIndexCursorSequenceBuilder.java
@@ -0,0 +1,602 @@
+/*
+ * 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
... 10924 lines suppressed ...


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