add vectorizeVirtualColumns query context parameter (#10432)

* add vectorizeVirtualColumns query context parameter

* oops

* spelling

* default to false, more docs

* fix test

* fix spelling
diff --git a/benchmarks/src/test/java/org/apache/druid/benchmark/FilteredAggregatorBenchmark.java b/benchmarks/src/test/java/org/apache/druid/benchmark/FilteredAggregatorBenchmark.java
index 47b5317..560148b 100644
--- a/benchmarks/src/test/java/org/apache/druid/benchmark/FilteredAggregatorBenchmark.java
+++ b/benchmarks/src/test/java/org/apache/druid/benchmark/FilteredAggregatorBenchmark.java
@@ -32,6 +32,7 @@
 import org.apache.druid.query.Druids;
 import org.apache.druid.query.FinalizeResultsQueryRunner;
 import org.apache.druid.query.Query;
+import org.apache.druid.query.QueryContexts;
 import org.apache.druid.query.QueryPlus;
 import org.apache.druid.query.QueryRunner;
 import org.apache.druid.query.QueryRunnerFactory;
@@ -239,7 +240,12 @@
     );
 
     final QueryPlus<T> queryToRun = QueryPlus.wrap(
-        query.withOverriddenContext(ImmutableMap.of("vectorize", vectorize))
+        query.withOverriddenContext(
+            ImmutableMap.of(
+                QueryContexts.VECTORIZE_KEY, vectorize,
+                QueryContexts.VECTORIZE_VIRTUAL_COLUMNS_KEY, vectorize
+            )
+        )
     );
     Sequence<T> queryResult = theRunner.run(queryToRun, ResponseContext.createEmpty());
     return queryResult.toList();
diff --git a/benchmarks/src/test/java/org/apache/druid/benchmark/query/SqlBenchmark.java b/benchmarks/src/test/java/org/apache/druid/benchmark/query/SqlBenchmark.java
index 55dc74c..38b5c3a 100644
--- a/benchmarks/src/test/java/org/apache/druid/benchmark/query/SqlBenchmark.java
+++ b/benchmarks/src/test/java/org/apache/druid/benchmark/query/SqlBenchmark.java
@@ -27,6 +27,7 @@
 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.QueryContexts;
 import org.apache.druid.query.QueryRunnerFactoryConglomerate;
 import org.apache.druid.segment.QueryableIndex;
 import org.apache.druid.segment.generator.GeneratorBasicSchemas;
@@ -434,7 +435,10 @@
   @OutputTimeUnit(TimeUnit.MILLISECONDS)
   public void querySql(Blackhole blackhole) throws Exception
   {
-    final Map<String, Object> context = ImmutableMap.of("vectorize", vectorize);
+    final Map<String, Object> context = ImmutableMap.of(
+        QueryContexts.VECTORIZE_KEY, vectorize,
+        QueryContexts.VECTORIZE_VIRTUAL_COLUMNS_KEY, vectorize
+    );
     final AuthenticationResult authenticationResult = NoopEscalator.getInstance()
                                                                    .createEscalatedAuthenticationResult();
     try (final DruidPlanner planner = plannerFactory.createPlanner(context, ImmutableList.of(), authenticationResult)) {
@@ -450,7 +454,10 @@
   @OutputTimeUnit(TimeUnit.MILLISECONDS)
   public void planSql(Blackhole blackhole) throws Exception
   {
-    final Map<String, Object> context = ImmutableMap.of("vectorize", vectorize);
+    final Map<String, Object> context = ImmutableMap.of(
+        QueryContexts.VECTORIZE_KEY, vectorize,
+        QueryContexts.VECTORIZE_VIRTUAL_COLUMNS_KEY, vectorize
+    );
     final AuthenticationResult authenticationResult = NoopEscalator.getInstance()
                                                                    .createEscalatedAuthenticationResult();
     try (final DruidPlanner planner = plannerFactory.createPlanner(context, ImmutableList.of(), authenticationResult)) {
diff --git a/benchmarks/src/test/java/org/apache/druid/benchmark/query/SqlExpressionBenchmark.java b/benchmarks/src/test/java/org/apache/druid/benchmark/query/SqlExpressionBenchmark.java
index dba8bd5..fedd75a 100644
--- a/benchmarks/src/test/java/org/apache/druid/benchmark/query/SqlExpressionBenchmark.java
+++ b/benchmarks/src/test/java/org/apache/druid/benchmark/query/SqlExpressionBenchmark.java
@@ -28,6 +28,7 @@
 import org.apache.druid.java.util.common.io.Closer;
 import org.apache.druid.java.util.common.logger.Logger;
 import org.apache.druid.query.DruidProcessingConfig;
+import org.apache.druid.query.QueryContexts;
 import org.apache.druid.query.QueryRunnerFactoryConglomerate;
 import org.apache.druid.segment.QueryableIndex;
 import org.apache.druid.segment.generator.GeneratorBasicSchemas;
@@ -285,7 +286,10 @@
   @OutputTimeUnit(TimeUnit.MILLISECONDS)
   public void querySql(Blackhole blackhole) throws Exception
   {
-    final Map<String, Object> context = ImmutableMap.of("vectorize", vectorize);
+    final Map<String, Object> context = ImmutableMap.of(
+        QueryContexts.VECTORIZE_KEY, vectorize,
+        QueryContexts.VECTORIZE_VIRTUAL_COLUMNS_KEY, vectorize
+    );
     final AuthenticationResult authenticationResult = NoopEscalator.getInstance()
                                                                    .createEscalatedAuthenticationResult();
     try (final DruidPlanner planner = plannerFactory.createPlanner(context, ImmutableList.of(), authenticationResult)) {
diff --git a/docs/misc/math-expr.md b/docs/misc/math-expr.md
index 3867fcb..d3ef3b7 100644
--- a/docs/misc/math-expr.md
+++ b/docs/misc/math-expr.md
@@ -189,7 +189,7 @@
 | all(lambda,arr) | returns 1 if all elements in the array matches the lambda expression, else 0 |
 
 
-### Reduction functions
+## Reduction functions
 
 Reduction functions operate on zero or more expressions and return a single expression. If no expressions are passed as
 arguments, then the result is `NULL`. The expressions must all be convertible to a common data type, which will be the
@@ -214,3 +214,16 @@
 | ipv4_match(address, subnet) | Returns 1 if the `address` belongs to the `subnet` literal, else 0. If `address` is not a valid IPv4 address, then 0 is returned. This function is more efficient if `address` is a long instead of a string.|
 | ipv4_parse(address) | Parses `address` into an IPv4 address stored as a long. If `address` is a long that is a valid IPv4 address, then it is passed through. Returns null if `address` cannot be represented as an IPv4 address. |
 | ipv4_stringify(address) | Converts `address` into an IPv4 address dotted-decimal string. If `address` is a string that is a valid IPv4 address, then it is passed through. Returns null if `address` cannot be represented as an IPv4 address.|
+
+
+## Vectorization Support
+A number of expressions support ['vectorized' query engines](../querying/query-context.md#vectorization-parameters)
+
+supported features:
+* constants and identifiers are supported for any column type
+* `cast` is supported for numeric and string types
+* math operators: `+`,`-`,`*`,`/`,`%`,`^` are supported for numeric types
+* comparison operators: `=`, `!=`, `>`, `>=`, `<`, `<=` are supported for numeric types
+* math functions: `abs`, `acos`, `asin`, `atan`, `cbrt`, `ceil`, `cos`, `cosh`, `cot`, `exp`, `expm1`, `floor`, `getExponent`, `log`, `log10`, `log1p`, `nextUp`, `rint`, `signum`, `sin`, `sinh`, `sqrt`, `tan`, `tanh`, `toDegrees`, `toRadians`, `ulp`, `atan2`, `copySign`, `div`, `hypot`, `max`, `min`, `nextAfter`,  `pow`, `remainder`, `scalb` are supported for numeric types
+* time functions: `timestamp_floor` (with constant granularity argument) is supported for numeric types
+* other: `parse_long` is supported for numeric and string types
diff --git a/docs/querying/query-context.md b/docs/querying/query-context.md
index 355a820..7bb912c 100644
--- a/docs/querying/query-context.md
+++ b/docs/querying/query-context.md
@@ -93,7 +93,7 @@
 - All aggregators must offer vectorized implementations. These include "count", "doubleSum", "floatSum", "longSum", "longMin",
  "longMax", "doubleMin", "doubleMax", "floatMin", "floatMax", "longAny", "doubleAny", "floatAny", "stringAny",
  "hyperUnique", "filtered", "approxHistogram", "approxHistogramFold", and "fixedBucketsHistogram" (with numerical input). 
-- No virtual columns.
+- All virtual columns must offer vectorized implementations. Currently for expression virtual columns, support for vectorization is decided on a per expression basis, depending on the type of input and the functions used by the expression. See the currently supported list in the [expression documentation](../misc/math-expr.md#vectorization-support).
 - 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.
@@ -107,3 +107,4 @@
 |--------|-------|------------|
 |vectorize|`true`|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). This will override `druid.query.default.context.vectorize` if it's set.|
 |vectorSize|`512`|Sets the row batching size for a particular query. This will override `druid.query.default.context.vectorSize` if it's set.|
+|vectorizeVirtualColumns|`false`|Enables or disables vectorized query processing of queries with virtual columns, layered on top of `vectorize` (`vectorize` must also be set to true for a query to utilize vectorization). Possible values are `false` (disabled), `true` (enabled if possible, disabled otherwise, on a per-segment basis), and `force` (enabled, and groupBy or timeseries queries with virtual columns that cannot be vectorized will fail). The `"force"` setting is meant to aid in testing, and is not generally useful in production. This will override `druid.query.default.context.vectorizeVirtualColumns` if it's set.|
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 5737e78..1a1e544 100644
--- a/processing/src/main/java/org/apache/druid/query/QueryContexts.java
+++ b/processing/src/main/java/org/apache/druid/query/QueryContexts.java
@@ -47,6 +47,7 @@
   public static final String BROKER_PARALLEL_MERGE_SMALL_BATCH_ROWS_KEY = "parallelMergeSmallBatchRows";
   public static final String BROKER_PARALLELISM = "parallelMergeParallelism";
   public static final String VECTORIZE_KEY = "vectorize";
+  public static final String VECTORIZE_VIRTUAL_COLUMNS_KEY = "vectorizeVirtualColumns";
   public static final String VECTOR_SIZE_KEY = "vectorSize";
   public static final String MAX_SUBQUERY_ROWS_KEY = "maxSubqueryRows";
   public static final String JOIN_FILTER_PUSH_DOWN_KEY = "enableJoinFilterPushDown";
@@ -65,6 +66,7 @@
   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.TRUE;
+  public static final Vectorize DEFAULT_VECTORIZE_VIRTUAL_COLUMN = 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);
@@ -197,6 +199,16 @@
     return parseEnum(query, VECTORIZE_KEY, Vectorize.class, defaultValue);
   }
 
+  public static <T> Vectorize getVectorizeVirtualColumns(Query<T> query)
+  {
+    return getVectorizeVirtualColumns(query, QueryContexts.DEFAULT_VECTORIZE_VIRTUAL_COLUMN);
+  }
+
+  public static <T> Vectorize getVectorizeVirtualColumns(Query<T> query, Vectorize defaultValue)
+  {
+    return parseEnum(query, VECTORIZE_VIRTUAL_COLUMNS_KEY, Vectorize.class, defaultValue);
+  }
+
   public static <T> int getVectorSize(Query<T> query)
   {
     return getVectorSize(query, QueryableIndexStorageAdapter.DEFAULT_VECTOR_SIZE);
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 f9945e9..7bf2d0d 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
@@ -20,6 +20,7 @@
 package org.apache.druid.query.groupby;
 
 import com.fasterxml.jackson.annotation.JsonProperty;
+import org.apache.druid.query.QueryContexts;
 import org.apache.druid.query.groupby.strategy.GroupByStrategySelector;
 
 /**
@@ -44,7 +45,6 @@
   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";
-  public static final String CTX_KEY_VECTORIZE = "vectorize";
 
   @JsonProperty
   private String defaultStrategy = GroupByStrategySelector.STRATEGY_V2;
@@ -243,7 +243,7 @@
         CTX_KEY_NUM_PARALLEL_COMBINE_THREADS,
         getNumParallelCombineThreads()
     );
-    newConfig.vectorize = query.getContextBoolean(CTX_KEY_VECTORIZE, isVectorize());
+    newConfig.vectorize = query.getContextBoolean(QueryContexts.VECTORIZE_KEY, isVectorize());
     return newConfig;
   }
 
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
index ff982c7..8a9991c 100644
--- 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
@@ -42,6 +42,7 @@
 import org.apache.druid.query.vector.VectorCursorGranularizer;
 import org.apache.druid.segment.DimensionHandlerUtils;
 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.filter.Filters;
@@ -79,6 +80,7 @@
     return canVectorizeDimensions(capabilitiesFunction, query.getDimensions())
            && query.getDimensions().stream().allMatch(DimensionSpec::canVectorize)
            && query.getAggregatorSpecs().stream().allMatch(aggregatorFactory -> aggregatorFactory.canVectorize(adapter))
+           && VirtualColumns.shouldVectorize(query, query.getVirtualColumns(), adapter)
            && adapter.canVectorize(filter, query.getVirtualColumns(), false);
   }
 
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 e2437a3..75cb498 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
@@ -41,6 +41,7 @@
 import org.apache.druid.query.vector.VectorCursorGranularizer;
 import org.apache.druid.segment.SegmentMissingException;
 import org.apache.druid.segment.StorageAdapter;
+import org.apache.druid.segment.VirtualColumns;
 import org.apache.druid.segment.filter.Filters;
 import org.apache.druid.segment.vector.VectorColumnSelectorFactory;
 import org.apache.druid.segment.vector.VectorCursor;
@@ -94,8 +95,9 @@
     final boolean descending = query.isDescending();
 
     final boolean doVectorize = QueryContexts.getVectorize(query).shouldVectorize(
-        adapter.canVectorize(filter, query.getVirtualColumns(), descending)
-        && query.getAggregatorSpecs().stream().allMatch(aggregatorFactory -> aggregatorFactory.canVectorize(adapter))
+        query.getAggregatorSpecs().stream().allMatch(aggregatorFactory -> aggregatorFactory.canVectorize(adapter))
+        && VirtualColumns.shouldVectorize(query, query.getVirtualColumns(), adapter)
+        && adapter.canVectorize(filter, query.getVirtualColumns(), descending)
     );
 
     final Sequence<Result<TimeseriesResultValue>> result;
diff --git a/processing/src/main/java/org/apache/druid/segment/QueryableIndexStorageAdapter.java b/processing/src/main/java/org/apache/druid/segment/QueryableIndexStorageAdapter.java
index 381fcf7..fcd541f 100644
--- a/processing/src/main/java/org/apache/druid/segment/QueryableIndexStorageAdapter.java
+++ b/processing/src/main/java/org/apache/druid/segment/QueryableIndexStorageAdapter.java
@@ -220,7 +220,7 @@
     }
 
     // vector cursors can't iterate backwards yet
-    return virtualColumns.canVectorize(this) && !descending;
+    return !descending;
   }
 
   @Override
diff --git a/processing/src/main/java/org/apache/druid/segment/VirtualColumns.java b/processing/src/main/java/org/apache/druid/segment/VirtualColumns.java
index 7419579..3b141d4 100644
--- a/processing/src/main/java/org/apache/druid/segment/VirtualColumns.java
+++ b/processing/src/main/java/org/apache/druid/segment/VirtualColumns.java
@@ -29,6 +29,8 @@
 import org.apache.druid.java.util.common.Cacheable;
 import org.apache.druid.java.util.common.IAE;
 import org.apache.druid.java.util.common.Pair;
+import org.apache.druid.query.Query;
+import org.apache.druid.query.QueryContexts;
 import org.apache.druid.query.cache.CacheKeyBuilder;
 import org.apache.druid.query.dimension.DimensionSpec;
 import org.apache.druid.segment.column.BitmapIndex;
@@ -113,6 +115,15 @@
     return virtualColumns == null ? EMPTY : virtualColumns;
   }
 
+  public static boolean shouldVectorize(Query<?> query, VirtualColumns virtualColumns, ColumnInspector inspector)
+  {
+    if (virtualColumns.getVirtualColumns().length > 0) {
+      return QueryContexts.getVectorizeVirtualColumns(query).shouldVectorize(virtualColumns.canVectorize(inspector));
+    } else {
+      return true;
+    }
+  }
+
   private VirtualColumns(
       List<VirtualColumn> virtualColumns,
       Map<String, VirtualColumn> withDotSupport,
@@ -473,5 +484,4 @@
   {
     return virtualColumns.toString();
   }
-
 }
diff --git a/processing/src/test/java/org/apache/druid/query/aggregation/mean/DoubleMeanAggregationTest.java b/processing/src/test/java/org/apache/druid/query/aggregation/mean/DoubleMeanAggregationTest.java
index cf14019..746fdbd 100644
--- a/processing/src/test/java/org/apache/druid/query/aggregation/mean/DoubleMeanAggregationTest.java
+++ b/processing/src/test/java/org/apache/druid/query/aggregation/mean/DoubleMeanAggregationTest.java
@@ -127,7 +127,7 @@
         .setAggregatorSpecs(
             new DoubleMeanAggregatorFactory("meanOnDouble", SimpleTestIndex.DOUBLE_COL)
         )
-        .setContext(Collections.singletonMap(GroupByQueryConfig.CTX_KEY_VECTORIZE, doVectorize))
+        .setContext(Collections.singletonMap(QueryContexts.VECTORIZE_KEY, doVectorize))
         .build();
 
     // do json serialization and deserialization of query to ensure there are no serde issues
diff --git a/processing/src/test/java/org/apache/druid/query/groupby/GroupByQueryRunnerTest.java b/processing/src/test/java/org/apache/druid/query/groupby/GroupByQueryRunnerTest.java
index 0f8b690..a4fc11d 100644
--- a/processing/src/test/java/org/apache/druid/query/groupby/GroupByQueryRunnerTest.java
+++ b/processing/src/test/java/org/apache/druid/query/groupby/GroupByQueryRunnerTest.java
@@ -10727,7 +10727,8 @@
   private Map<String, Object> makeContext()
   {
     return ImmutableMap.<String, Object>builder()
-        .put("vectorize", vectorize ? "force" : "false")
+        .put(QueryContexts.VECTORIZE_KEY, vectorize ? "force" : "false")
+        .put(QueryContexts.VECTORIZE_VIRTUAL_COLUMNS_KEY, vectorize ? "force" : "false")
         .put("vectorSize", 16) // Small vector size to ensure we use more than one.
         .build();
   }
diff --git a/processing/src/test/java/org/apache/druid/query/timeseries/TimeseriesQueryRunnerTest.java b/processing/src/test/java/org/apache/druid/query/timeseries/TimeseriesQueryRunnerTest.java
index ae5aa1e..c1ae6d4 100644
--- a/processing/src/test/java/org/apache/druid/query/timeseries/TimeseriesQueryRunnerTest.java
+++ b/processing/src/test/java/org/apache/druid/query/timeseries/TimeseriesQueryRunnerTest.java
@@ -34,6 +34,7 @@
 import org.apache.druid.java.util.common.guava.Sequence;
 import org.apache.druid.query.Druids;
 import org.apache.druid.query.FinalizeResultsQueryRunner;
+import org.apache.druid.query.QueryContexts;
 import org.apache.druid.query.QueryPlus;
 import org.apache.druid.query.QueryRunner;
 import org.apache.druid.query.QueryRunnerTestHelper;
@@ -2942,8 +2943,9 @@
   private Map<String, Object> makeContext(final Map<String, Object> myContext)
   {
     final Map<String, Object> context = new HashMap<>();
-    context.put("vectorize", vectorize ? "force" : "false");
-    context.put("vectorSize", 16); // Small vector size to ensure we use more than one.
+    context.put(QueryContexts.VECTORIZE_KEY, vectorize ? "force" : "false");
+    context.put(QueryContexts.VECTORIZE_VIRTUAL_COLUMNS_KEY, vectorize ? "force" : "false");
+    context.put(QueryContexts.VECTOR_SIZE_KEY, 16); // Small vector size to ensure we use more than one.
     context.putAll(myContext);
     return context;
   }
diff --git a/processing/src/test/java/org/apache/druid/segment/virtual/AlwaysTwoCounterAggregatorFactory.java b/processing/src/test/java/org/apache/druid/segment/virtual/AlwaysTwoCounterAggregatorFactory.java
index f441143..56f62cb 100644
--- a/processing/src/test/java/org/apache/druid/segment/virtual/AlwaysTwoCounterAggregatorFactory.java
+++ b/processing/src/test/java/org/apache/druid/segment/virtual/AlwaysTwoCounterAggregatorFactory.java
@@ -59,13 +59,13 @@
   @Override
   public Aggregator factorize(ColumnSelectorFactory metricFactory)
   {
-    throw new IllegalStateException("don't call this");
+    throw new IllegalStateException(AlwaysTwoVectorizedVirtualColumn.DONT_CALL_THIS);
   }
 
   @Override
   public BufferAggregator factorizeBuffered(ColumnSelectorFactory metricFactory)
   {
-    throw new IllegalStateException("don't call this");
+    throw new IllegalStateException(AlwaysTwoVectorizedVirtualColumn.DONT_CALL_THIS);
   }
 
   @Override
diff --git a/processing/src/test/java/org/apache/druid/segment/virtual/AlwaysTwoVectorizedVirtualColumn.java b/processing/src/test/java/org/apache/druid/segment/virtual/AlwaysTwoVectorizedVirtualColumn.java
index 78d031b..6c17032 100644
--- a/processing/src/test/java/org/apache/druid/segment/virtual/AlwaysTwoVectorizedVirtualColumn.java
+++ b/processing/src/test/java/org/apache/druid/segment/virtual/AlwaysTwoVectorizedVirtualColumn.java
@@ -48,26 +48,38 @@
  */
 public class AlwaysTwoVectorizedVirtualColumn implements VirtualColumn
 {
+  static final String DONT_CALL_THIS = "don't call this";
   private final String outputName;
   private final ColumnCapabilities capabilities;
   private final boolean dictionaryEncoded;
+  private final boolean canVectorize;
 
   public AlwaysTwoVectorizedVirtualColumn(
       String name,
       ColumnCapabilities capabilites
   )
   {
+    this(name, capabilites, true);
+  }
+
+  public AlwaysTwoVectorizedVirtualColumn(
+      String name,
+      ColumnCapabilities capabilites,
+      boolean canVectorize
+  )
+  {
     this.outputName = name;
     this.capabilities = capabilites;
     this.dictionaryEncoded = capabilites.isDictionaryEncoded().isTrue() &&
                              capabilites.areDictionaryValuesUnique().isTrue();
+    this.canVectorize = canVectorize;
   }
 
   @Override
   public boolean canVectorize(ColumnInspector inspector)
   {
     Assert.assertNotNull(inspector);
-    return true;
+    return canVectorize;
   }
 
   @Override
@@ -79,13 +91,13 @@
   @Override
   public DimensionSelector makeDimensionSelector(DimensionSpec dimensionSpec, ColumnSelectorFactory factory)
   {
-    throw new IllegalStateException("don't call this");
+    throw new IllegalStateException(DONT_CALL_THIS);
   }
 
   @Override
   public ColumnValueSelector<?> makeColumnValueSelector(String columnName, ColumnSelectorFactory factory)
   {
-    throw new IllegalStateException("don't call this");
+    throw new IllegalStateException(DONT_CALL_THIS);
   }
 
   @Override
diff --git a/processing/src/test/java/org/apache/druid/segment/virtual/VectorizedVirtualColumnTest.java b/processing/src/test/java/org/apache/druid/segment/virtual/VectorizedVirtualColumnTest.java
index ed1a573..4d8d3ef 100644
--- a/processing/src/test/java/org/apache/druid/segment/virtual/VectorizedVirtualColumnTest.java
+++ b/processing/src/test/java/org/apache/druid/segment/virtual/VectorizedVirtualColumnTest.java
@@ -31,6 +31,7 @@
 import org.apache.druid.query.QueryRunnerTestHelper;
 import org.apache.druid.query.Result;
 import org.apache.druid.query.aggregation.AggregationTestHelper;
+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.query.groupby.GroupByQueryConfig;
@@ -60,7 +61,31 @@
 {
   private static final String ALWAYS_TWO = "two";
   private static final String COUNT = "count";
-  private static final Map<String, Object> CONTEXT = ImmutableMap.of(QueryContexts.VECTORIZE_KEY, "force");
+  private static final Map<String, Object> CONTEXT_USE_DEFAULTS = ImmutableMap.of();
+  private static final Map<String, Object> CONTEXT_VECTORIZE_FORCE = ImmutableMap.of(
+      QueryContexts.VECTORIZE_KEY,
+      "force",
+      QueryContexts.VECTORIZE_VIRTUAL_COLUMNS_KEY,
+      "force"
+  );
+  private static final Map<String, Object> CONTEXT_VECTORIZE_TRUE_VIRTUAL_FORCE = ImmutableMap.of(
+      QueryContexts.VECTORIZE_KEY,
+      "true",
+      QueryContexts.VECTORIZE_VIRTUAL_COLUMNS_KEY,
+      "force"
+  );
+  private static final Map<String, Object> CONTEXT_CONTRADICTION_VECTORIZE_FALSE_VIRTUAL_FORCE = ImmutableMap.of(
+      QueryContexts.VECTORIZE_KEY,
+      "false",
+      QueryContexts.VECTORIZE_VIRTUAL_COLUMNS_KEY,
+      "force"
+  );
+  private static final Map<String, Object> CONTEXT_CONTRADICTION_VECTORIZE_FORCE_VIRTUAL_FALSE = ImmutableMap.of(
+      QueryContexts.VECTORIZE_KEY,
+      "force",
+      QueryContexts.VECTORIZE_VIRTUAL_COLUMNS_KEY,
+      "false"
+  );
 
   @Rule
   public final TemporaryFolder tmpFolder = new TemporaryFolder();
@@ -256,15 +281,195 @@
     testTimeseries(ColumnCapabilitiesImpl.createSimpleNumericColumnCapabilities(ValueType.FLOAT));
   }
 
+  @Test
+  public void testTimeseriesForceContextCannotVectorize()
+  {
+    cannotVectorize();
+    testTimeseries(
+        ColumnCapabilitiesImpl.createSimpleNumericColumnCapabilities(ValueType.FLOAT),
+        CONTEXT_VECTORIZE_FORCE,
+        false
+    );
+  }
+
+  @Test
+  public void testTimeseriesForceVirtualContextCannotVectorize()
+  {
+    cannotVectorize();
+    testTimeseries(
+        ColumnCapabilitiesImpl.createSimpleNumericColumnCapabilities(ValueType.FLOAT),
+        CONTEXT_VECTORIZE_TRUE_VIRTUAL_FORCE,
+        false
+    );
+  }
+
+  @Test
+  public void testTimeseriesTrueVirtualContextCannotVectorize()
+  {
+    expectNonvectorized();
+    testTimeseries(
+        ColumnCapabilitiesImpl.createSimpleNumericColumnCapabilities(ValueType.FLOAT),
+        CONTEXT_USE_DEFAULTS,
+        true
+    );
+  }
+
+  @Test
+  public void testTimeseriesContradictionVectorizeFalseVirtualForce()
+  {
+    expectNonvectorized();
+    testTimeseries(
+        ColumnCapabilitiesImpl.createSimpleNumericColumnCapabilities(ValueType.FLOAT),
+        CONTEXT_CONTRADICTION_VECTORIZE_FALSE_VIRTUAL_FORCE,
+        true
+    );
+  }
+
+  @Test
+  public void testTimeseriesContradictionVectorizeForceVirtualFalse()
+  {
+    cannotVectorize();
+    testTimeseries(
+        ColumnCapabilitiesImpl.createSimpleNumericColumnCapabilities(ValueType.FLOAT),
+        CONTEXT_CONTRADICTION_VECTORIZE_FORCE_VIRTUAL_FALSE,
+        true
+    );
+  }
+
+  @Test
+  public void testTimeseriesContradictionVectorizeFalseVirtualForceNoVirtualColumns()
+  {
+    testTimeseriesNoVirtual(
+        ColumnCapabilitiesImpl.createSimpleNumericColumnCapabilities(ValueType.FLOAT),
+        CONTEXT_CONTRADICTION_VECTORIZE_FALSE_VIRTUAL_FORCE
+    );
+  }
+  @Test
+  public void testTimeseriesContradictionVectorizeForceVirtualFalseNoVirtual()
+  {
+    testTimeseriesNoVirtual(
+        ColumnCapabilitiesImpl.createSimpleNumericColumnCapabilities(ValueType.FLOAT),
+        CONTEXT_CONTRADICTION_VECTORIZE_FORCE_VIRTUAL_FALSE
+    );
+  }
+
+  @Test
+  public void testTimeseriesForceDoestAffectWhenNoVirtualColumns()
+  {
+    testTimeseriesNoVirtual(
+        ColumnCapabilitiesImpl.createSimpleNumericColumnCapabilities(ValueType.FLOAT),
+        CONTEXT_VECTORIZE_TRUE_VIRTUAL_FORCE
+    );
+  }
+
+  @Test
+  public void testGroupByForceContextCannotVectorize()
+  {
+    cannotVectorize();
+    testGroupBy(
+        ColumnCapabilitiesImpl.createSimpleNumericColumnCapabilities(ValueType.FLOAT),
+        CONTEXT_VECTORIZE_FORCE,
+        false
+    );
+  }
+
+  @Test
+  public void testGroupByForceVirtualContextCannotVectorize()
+  {
+    cannotVectorize();
+    testGroupBy(
+        new ColumnCapabilitiesImpl()
+            .setType(ValueType.STRING)
+            .setDictionaryEncoded(true)
+            .setDictionaryValuesUnique(true)
+            .setHasMultipleValues(false),
+        CONTEXT_VECTORIZE_TRUE_VIRTUAL_FORCE,
+        false
+    );
+  }
+
+  @Test
+  public void testGroupByTrueVirtualContextCannotVectorize()
+  {
+    expectNonvectorized();
+    testGroupBy(
+        ColumnCapabilitiesImpl.createSimpleNumericColumnCapabilities(ValueType.FLOAT),
+        CONTEXT_USE_DEFAULTS,
+        false
+    );
+  }
+
+  @Test
+  public void testGroupByContradictionVectorizeFalseVirtualForce()
+  {
+    expectNonvectorized();
+    testGroupBy(
+        ColumnCapabilitiesImpl.createSimpleNumericColumnCapabilities(ValueType.FLOAT),
+        CONTEXT_CONTRADICTION_VECTORIZE_FALSE_VIRTUAL_FORCE,
+        true
+    );
+  }
+
+  @Test
+  public void testGroupByContradictionVectorizeForceVirtualFalse()
+  {
+    cannotVectorize();
+    testGroupBy(
+        ColumnCapabilitiesImpl.createSimpleNumericColumnCapabilities(ValueType.FLOAT),
+        CONTEXT_CONTRADICTION_VECTORIZE_FORCE_VIRTUAL_FALSE,
+        true
+    );
+  }
+
+  @Test
+  public void testGroupByContradictionVectorizeFalseVirtualForceNoVirtual()
+  {
+    testGroupByNoVirtual(
+        new ColumnCapabilitiesImpl()
+            .setType(ValueType.STRING)
+            .setDictionaryEncoded(true)
+            .setDictionaryValuesUnique(true)
+            .setHasMultipleValues(false),
+        CONTEXT_CONTRADICTION_VECTORIZE_FALSE_VIRTUAL_FORCE
+    );
+  }
+
+  @Test
+  public void testGroupByContradictionVectorizeForceVirtualFalseNoVirtual()
+  {
+    testGroupByNoVirtual(
+        new ColumnCapabilitiesImpl()
+            .setType(ValueType.STRING)
+            .setDictionaryEncoded(true)
+            .setDictionaryValuesUnique(true)
+            .setHasMultipleValues(false),
+        CONTEXT_CONTRADICTION_VECTORIZE_FORCE_VIRTUAL_FALSE
+    );
+  }
+
+  @Test
+  public void testGroupByForceDoestAffectWhenNoVirtualColumns()
+  {
+    testGroupByNoVirtual(
+        new ColumnCapabilitiesImpl()
+            .setType(ValueType.STRING)
+            .setDictionaryEncoded(true)
+            .setDictionaryValuesUnique(true)
+            .setHasMultipleValues(false),
+        CONTEXT_VECTORIZE_TRUE_VIRTUAL_FORCE
+    );
+  }
+
   private void testTimeseries(ColumnCapabilities capabilities)
   {
+    testTimeseries(capabilities, CONTEXT_VECTORIZE_FORCE, true);
     TimeseriesQuery query = Druids.newTimeseriesQueryBuilder()
                                   .intervals("2000/2030")
                                   .dataSource(QueryRunnerTestHelper.DATA_SOURCE)
                                   .granularity(Granularities.ALL)
                                   .virtualColumns(new AlwaysTwoVectorizedVirtualColumn(ALWAYS_TWO, capabilities))
                                   .aggregators(new AlwaysTwoCounterAggregatorFactory(COUNT, ALWAYS_TWO))
-                                  .context(CONTEXT)
+                                  .context(CONTEXT_VECTORIZE_FORCE)
                                   .build();
 
     Sequence seq = timeseriesTestHelper.runQueryOnSegmentsObjs(segments, query);
@@ -281,18 +486,73 @@
     TestHelper.assertExpectedObjects(expectedResults, seq.toList(), "failed");
   }
 
+  private void testTimeseries(ColumnCapabilities capabilities, Map<String, Object> context, boolean canVectorize)
+  {
+    TimeseriesQuery query = Druids.newTimeseriesQueryBuilder()
+                                  .intervals("2000/2030")
+                                  .dataSource(QueryRunnerTestHelper.DATA_SOURCE)
+                                  .granularity(Granularities.ALL)
+                                  .virtualColumns(new AlwaysTwoVectorizedVirtualColumn(ALWAYS_TWO, capabilities, canVectorize))
+                                  .aggregators(new AlwaysTwoCounterAggregatorFactory(COUNT, ALWAYS_TWO))
+                                  .context(context)
+                                  .build();
+
+    Sequence seq = timeseriesTestHelper.runQueryOnSegmentsObjs(segments, query);
+
+    List<Result<TimeseriesResultValue>> expectedResults = ImmutableList.of(
+        new Result<>(
+            DateTimes.of("2011-01-12T00:00:00.000Z"),
+            new TimeseriesResultValue(
+                ImmutableMap.of(COUNT, getCount(capabilities))
+            )
+        )
+    );
+
+    TestHelper.assertExpectedObjects(expectedResults, seq.toList(), "failed");
+  }
+
+  private void testTimeseriesNoVirtual(ColumnCapabilities capabilities, Map<String, Object> context)
+  {
+    TimeseriesQuery query = Druids.newTimeseriesQueryBuilder()
+                                  .intervals("2000/2030")
+                                  .dataSource(QueryRunnerTestHelper.DATA_SOURCE)
+                                  .granularity(Granularities.ALL)
+                                  .virtualColumns()
+                                  .aggregators(new CountAggregatorFactory(COUNT))
+                                  .context(context)
+                                  .build();
+
+    Sequence seq = timeseriesTestHelper.runQueryOnSegmentsObjs(segments, query);
+
+    List<Result<TimeseriesResultValue>> expectedResults = ImmutableList.of(
+        new Result<>(
+            DateTimes.of("2011-01-12T00:00:00.000Z"),
+            new TimeseriesResultValue(
+                ImmutableMap.of(COUNT, 2418L)
+            )
+        )
+    );
+
+    TestHelper.assertExpectedObjects(expectedResults, seq.toList(), "failed");
+  }
+
   private void testGroupBy(ColumnCapabilities capabilities)
   {
+    testGroupBy(capabilities, CONTEXT_VECTORIZE_FORCE, true);
+  }
+
+  private void testGroupBy(ColumnCapabilities capabilities, Map<String, Object> context, boolean canVectorize)
+  {
     GroupByQuery query = new GroupByQuery.Builder()
         .setDataSource(QueryRunnerTestHelper.DATA_SOURCE)
         .setGranularity(Granularities.ALL)
         .setVirtualColumns(
-            new AlwaysTwoVectorizedVirtualColumn(ALWAYS_TWO, capabilities)
+            new AlwaysTwoVectorizedVirtualColumn(ALWAYS_TWO, capabilities, canVectorize)
         )
         .addDimension(new DefaultDimensionSpec(ALWAYS_TWO, ALWAYS_TWO, capabilities.getType()))
         .setAggregatorSpecs(new AlwaysTwoCounterAggregatorFactory(COUNT, ALWAYS_TWO))
         .setInterval("2000/2030")
-        .setContext(CONTEXT)
+        .setContext(context)
         .addOrderByColumn(ALWAYS_TWO)
         .build();
 
@@ -312,6 +572,34 @@
     TestHelper.assertExpectedObjects(expectedRows, rows, "failed");
   }
 
+  private void testGroupByNoVirtual(ColumnCapabilities capabilities, Map<String, Object> context)
+  {
+    GroupByQuery query = new GroupByQuery.Builder()
+        .setDataSource(QueryRunnerTestHelper.DATA_SOURCE)
+        .setGranularity(Granularities.ALL)
+        .setVirtualColumns()
+        .addDimension(new DefaultDimensionSpec("placement", "placement", capabilities.getType()))
+        .setAggregatorSpecs(new CountAggregatorFactory(COUNT))
+        .setInterval("2000/2030")
+        .setContext(context)
+        .build();
+
+    List<ResultRow> rows = groupByTestHelper.runQueryOnSegmentsObjs(segments, query).toList();
+
+    List<ResultRow> expectedRows = Collections.singletonList(
+        GroupByQueryRunnerTestHelper.createExpectedRow(
+            query,
+            "2000",
+            COUNT,
+            2418L,
+            "placement",
+            "preferred"
+        )
+    );
+
+    TestHelper.assertExpectedObjects(expectedRows, rows, "failed");
+  }
+
   private long getCount(ColumnCapabilities capabilities)
   {
     long modifier = 1L;
@@ -339,6 +627,12 @@
     }
   }
 
+  private void expectNonvectorized()
+  {
+    expectedException.expect(RuntimeException.class);
+    expectedException.expectMessage(AlwaysTwoVectorizedVirtualColumn.DONT_CALL_THIS);
+  }
+
   private void cannotVectorize()
   {
     expectedException.expect(RuntimeException.class);
diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/BaseCalciteQueryTest.java b/sql/src/test/java/org/apache/druid/sql/calcite/BaseCalciteQueryTest.java
index 7c475e5..8cb5d35 100644
--- a/sql/src/test/java/org/apache/druid/sql/calcite/BaseCalciteQueryTest.java
+++ b/sql/src/test/java/org/apache/druid/sql/calcite/BaseCalciteQueryTest.java
@@ -623,10 +623,11 @@
       queryLogHook.clearRecordedQueries();
 
       final Map<String, Object> theQueryContext = new HashMap<>(queryContext);
-      theQueryContext.put("vectorize", vectorize);
+      theQueryContext.put(QueryContexts.VECTORIZE_KEY, vectorize);
+      theQueryContext.put(QueryContexts.VECTORIZE_VIRTUAL_COLUMNS_KEY, vectorize);
 
       if (!"false".equals(vectorize)) {
-        theQueryContext.put("vectorSize", 2); // Small vector size to ensure we use more than one.
+        theQueryContext.put(QueryContexts.VECTOR_SIZE_KEY, 2); // Small vector size to ensure we use more than one.
       }
 
       final List<Query> theQueries = new ArrayList<>();
diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/CalciteQueryTest.java b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteQueryTest.java
index d1c75ee..c232aa8 100644
--- a/sql/src/test/java/org/apache/druid/sql/calcite/CalciteQueryTest.java
+++ b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteQueryTest.java
@@ -1060,7 +1060,7 @@
         ImmutableList.of(),
         ImmutableList.of(
             new Object[]{
-                "DruidQueryRel(query=[{\"queryType\":\"scan\",\"dataSource\":{\"type\":\"table\",\"name\":\"foo\"},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"virtualColumns\":[],\"resultFormat\":\"compactedList\",\"batchSize\":20480,\"order\":\"none\",\"filter\":null,\"columns\":[\"__time\",\"cnt\",\"dim1\",\"dim2\",\"dim3\",\"m1\",\"m2\",\"unique_dim1\"],\"legacy\":false,\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlQueryId\":\"dummy\",\"vectorize\":\"false\"},\"descending\":false,\"granularity\":{\"type\":\"all\"}}], signature=[{__time:LONG, cnt:LONG, dim1:STRING, dim2:STRING, dim3:STRING, m1:FLOAT, m2:DOUBLE, unique_dim1:COMPLEX}])\n"
+                "DruidQueryRel(query=[{\"queryType\":\"scan\",\"dataSource\":{\"type\":\"table\",\"name\":\"foo\"},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"virtualColumns\":[],\"resultFormat\":\"compactedList\",\"batchSize\":20480,\"order\":\"none\",\"filter\":null,\"columns\":[\"__time\",\"cnt\",\"dim1\",\"dim2\",\"dim3\",\"m1\",\"m2\",\"unique_dim1\"],\"legacy\":false,\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlQueryId\":\"dummy\",\"vectorize\":\"false\",\"vectorizeVirtualColumns\":\"false\"},\"descending\":false,\"granularity\":{\"type\":\"all\"}}], signature=[{__time:LONG, cnt:LONG, dim1:STRING, dim2:STRING, dim3:STRING, m1:FLOAT, m2:DOUBLE, unique_dim1:COMPLEX}])\n"
             }
         )
     );
@@ -5133,7 +5133,7 @@
         + "\"aggregations\":[{\"type\":\"count\",\"name\":\"a0\"}],"
         + "\"postAggregations\":[],"
         + "\"limit\":2147483647,"
-        + "\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"skipEmptyBuckets\":true,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlQueryId\":\"dummy\",\"vectorize\":\"false\"}}]"
+        + "\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"skipEmptyBuckets\":true,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlQueryId\":\"dummy\",\"vectorize\":\"false\",\"vectorizeVirtualColumns\":\"false\"}}]"
         + ", signature=[{a0:LONG}])\n";
 
     testQuery(
@@ -7984,10 +7984,10 @@
     skipVectorize();
 
     final String explanation =
-        "DruidOuterQueryRel(query=[{\"queryType\":\"timeseries\",\"dataSource\":{\"type\":\"table\",\"name\":\"__subquery__\"},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"descending\":false,\"virtualColumns\":[],\"filter\":null,\"granularity\":{\"type\":\"all\"},\"aggregations\":[{\"type\":\"count\",\"name\":\"a0\"}],\"postAggregations\":[],\"limit\":2147483647,\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"skipEmptyBuckets\":true,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlQueryId\":\"dummy\",\"vectorize\":\"false\"}}], signature=[{a0:LONG}])\n"
-        + "  DruidJoinQueryRel(condition=[=(SUBSTRING($3, 1, 1), $8)], joinType=[inner], query=[{\"queryType\":\"groupBy\",\"dataSource\":{\"type\":\"table\",\"name\":\"__join__\"},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"virtualColumns\":[],\"filter\":null,\"granularity\":{\"type\":\"all\"},\"dimensions\":[{\"type\":\"default\",\"dimension\":\"dim2\",\"outputName\":\"d0\",\"outputType\":\"STRING\"}],\"aggregations\":[],\"postAggregations\":[],\"having\":null,\"limitSpec\":{\"type\":\"NoopLimitSpec\"},\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlQueryId\":\"dummy\",\"vectorize\":\"false\"},\"descending\":false}], signature=[{d0:STRING}])\n"
-        + "    DruidQueryRel(query=[{\"queryType\":\"scan\",\"dataSource\":{\"type\":\"table\",\"name\":\"foo\"},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"virtualColumns\":[],\"resultFormat\":\"compactedList\",\"batchSize\":20480,\"order\":\"none\",\"filter\":null,\"columns\":[\"__time\",\"cnt\",\"dim1\",\"dim2\",\"dim3\",\"m1\",\"m2\",\"unique_dim1\"],\"legacy\":false,\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlQueryId\":\"dummy\",\"vectorize\":\"false\"},\"descending\":false,\"granularity\":{\"type\":\"all\"}}], signature=[{__time:LONG, cnt:LONG, dim1:STRING, dim2:STRING, dim3:STRING, m1:FLOAT, m2:DOUBLE, unique_dim1:COMPLEX}])\n"
-        + "    DruidQueryRel(query=[{\"queryType\":\"groupBy\",\"dataSource\":{\"type\":\"table\",\"name\":\"foo\"},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"virtualColumns\":[],\"filter\":{\"type\":\"not\",\"field\":{\"type\":\"selector\",\"dimension\":\"dim1\",\"value\":null,\"extractionFn\":null}},\"granularity\":{\"type\":\"all\"},\"dimensions\":[{\"type\":\"extraction\",\"dimension\":\"dim1\",\"outputName\":\"d0\",\"outputType\":\"STRING\",\"extractionFn\":{\"type\":\"substring\",\"index\":0,\"length\":1}}],\"aggregations\":[],\"postAggregations\":[],\"having\":null,\"limitSpec\":{\"type\":\"NoopLimitSpec\"},\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlQueryId\":\"dummy\",\"vectorize\":\"false\"},\"descending\":false}], signature=[{d0:STRING}])\n";
+        "DruidOuterQueryRel(query=[{\"queryType\":\"timeseries\",\"dataSource\":{\"type\":\"table\",\"name\":\"__subquery__\"},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"descending\":false,\"virtualColumns\":[],\"filter\":null,\"granularity\":{\"type\":\"all\"},\"aggregations\":[{\"type\":\"count\",\"name\":\"a0\"}],\"postAggregations\":[],\"limit\":2147483647,\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"skipEmptyBuckets\":true,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlQueryId\":\"dummy\",\"vectorize\":\"false\",\"vectorizeVirtualColumns\":\"false\"}}], signature=[{a0:LONG}])\n"
+        + "  DruidJoinQueryRel(condition=[=(SUBSTRING($3, 1, 1), $8)], joinType=[inner], query=[{\"queryType\":\"groupBy\",\"dataSource\":{\"type\":\"table\",\"name\":\"__join__\"},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"virtualColumns\":[],\"filter\":null,\"granularity\":{\"type\":\"all\"},\"dimensions\":[{\"type\":\"default\",\"dimension\":\"dim2\",\"outputName\":\"d0\",\"outputType\":\"STRING\"}],\"aggregations\":[],\"postAggregations\":[],\"having\":null,\"limitSpec\":{\"type\":\"NoopLimitSpec\"},\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlQueryId\":\"dummy\",\"vectorize\":\"false\",\"vectorizeVirtualColumns\":\"false\"},\"descending\":false}], signature=[{d0:STRING}])\n"
+        + "    DruidQueryRel(query=[{\"queryType\":\"scan\",\"dataSource\":{\"type\":\"table\",\"name\":\"foo\"},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"virtualColumns\":[],\"resultFormat\":\"compactedList\",\"batchSize\":20480,\"order\":\"none\",\"filter\":null,\"columns\":[\"__time\",\"cnt\",\"dim1\",\"dim2\",\"dim3\",\"m1\",\"m2\",\"unique_dim1\"],\"legacy\":false,\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlQueryId\":\"dummy\",\"vectorize\":\"false\",\"vectorizeVirtualColumns\":\"false\"},\"descending\":false,\"granularity\":{\"type\":\"all\"}}], signature=[{__time:LONG, cnt:LONG, dim1:STRING, dim2:STRING, dim3:STRING, m1:FLOAT, m2:DOUBLE, unique_dim1:COMPLEX}])\n"
+        + "    DruidQueryRel(query=[{\"queryType\":\"groupBy\",\"dataSource\":{\"type\":\"table\",\"name\":\"foo\"},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"virtualColumns\":[],\"filter\":{\"type\":\"not\",\"field\":{\"type\":\"selector\",\"dimension\":\"dim1\",\"value\":null,\"extractionFn\":null}},\"granularity\":{\"type\":\"all\"},\"dimensions\":[{\"type\":\"extraction\",\"dimension\":\"dim1\",\"outputName\":\"d0\",\"outputType\":\"STRING\",\"extractionFn\":{\"type\":\"substring\",\"index\":0,\"length\":1}}],\"aggregations\":[],\"postAggregations\":[],\"having\":null,\"limitSpec\":{\"type\":\"NoopLimitSpec\"},\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlQueryId\":\"dummy\",\"vectorize\":\"false\",\"vectorizeVirtualColumns\":\"false\"},\"descending\":false}], signature=[{d0:STRING}])\n";
 
     testQuery(
         "EXPLAIN PLAN FOR SELECT COUNT(*)\n"
diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/SqlVectorizedExpressionSanityTest.java b/sql/src/test/java/org/apache/druid/sql/calcite/SqlVectorizedExpressionSanityTest.java
index aae98c2..874e745 100644
--- a/sql/src/test/java/org/apache/druid/sql/calcite/SqlVectorizedExpressionSanityTest.java
+++ b/sql/src/test/java/org/apache/druid/sql/calcite/SqlVectorizedExpressionSanityTest.java
@@ -33,6 +33,7 @@
 import org.apache.druid.java.util.common.guava.Yielders;
 import org.apache.druid.java.util.common.io.Closer;
 import org.apache.druid.java.util.common.logger.Logger;
+import org.apache.druid.query.QueryContexts;
 import org.apache.druid.query.QueryRunnerFactoryConglomerate;
 import org.apache.druid.segment.QueryableIndex;
 import org.apache.druid.segment.generator.GeneratorBasicSchemas;
@@ -179,8 +180,14 @@
   public static void sanityTestVectorizedSqlQueries(PlannerFactory plannerFactory, String query)
       throws ValidationException, RelConversionException, SqlParseException
   {
-    final Map<String, Object> vector = ImmutableMap.of("vectorize", "force");
-    final Map<String, Object> nonvector = ImmutableMap.of("vectorize", "false");
+    final Map<String, Object> vector = ImmutableMap.of(
+        QueryContexts.VECTORIZE_KEY, "force",
+        QueryContexts.VECTORIZE_VIRTUAL_COLUMNS_KEY, "force"
+    );
+    final Map<String, Object> nonvector = ImmutableMap.of(
+        QueryContexts.VECTORIZE_KEY, "false",
+        QueryContexts.VECTORIZE_VIRTUAL_COLUMNS_KEY, "false"
+    );
     final AuthenticationResult authenticationResult = NoopEscalator.getInstance()
                                                                    .createEscalatedAuthenticationResult();
 
diff --git a/website/.spelling b/website/.spelling
index c2cea43..dcf6140 100644
--- a/website/.spelling
+++ b/website/.spelling
@@ -413,6 +413,7 @@
 validator
 vectorizable
 vectorize
+vectorizeVirtualColumns
 versioning
 w.r.t.
 whitelist