You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@pinot.apache.org by xi...@apache.org on 2023/12/07 08:58:23 UTC

(pinot) branch master updated: Support Vector index and HNSW as the first implementation (#11977)

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

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


The following commit(s) were added to refs/heads/master by this push:
     new 47cfcb382c Support Vector index and HNSW as the first implementation (#11977)
47cfcb382c is described below

commit 47cfcb382c0335bd9b5963c50c1f8dc57c017f41
Author: Xiang Fu <xi...@gmail.com>
AuthorDate: Thu Dec 7 00:58:17 2023 -0800

    Support Vector index and HNSW as the first implementation (#11977)
    
    * Support Vector index and HNSW as the first implementation
    
    * make distance function configurable
    
    * make hnsw codec configurable
    
    * make mutable index using hnsw as well
    
    * Delegate the Lucene95 codec and format implementation
---
 .../pinot/common/function/FunctionRegistry.java    |   5 +
 .../common/function/TransformFunctionType.java     |   3 +
 .../common/function/scalar/VectorFunctions.java    |  32 ++
 .../request/context/RequestContextUtils.java       |  42 ++
 .../request/context/predicate/Predicate.java       |   3 +-
 .../predicate/VectorSimilarityPredicate.java       |  82 ++++
 .../main/java/org/apache/pinot/sql/FilterKind.java |   3 +-
 .../apache/pinot/sql/parsers/CalciteSqlParser.java |  20 +
 .../rewriter/PredicateComparisonRewriter.java      |  33 +-
 .../filter/VectorSimilarityFilterOperator.java     | 118 ++++++
 .../org/apache/pinot/core/plan/FilterPlanNode.java |   8 +
 .../realtime/impl/vector/MutableVectorIndex.java   | 154 +++++++
 .../impl/vector/HnswVectorIndexCreator.java        | 120 ++++++
 .../creator/impl/vector/XKnnFloatVectorField.java  |  68 +++
 .../creator/impl/vector/lucene95/HnswCodec.java    | 139 +++++++
 .../impl/vector/lucene95/HnswVectorsFormat.java    |  91 ++++
 .../converter/SegmentV1V2ToV3FormatConverter.java  |  27 +-
 .../segment/index/datasource/BaseDataSource.java   |   7 +
 .../segment/index/loader/IndexLoadingConfig.java   |  25 ++
 .../loader/invertedindex/VectorIndexHandler.java   | 215 ++++++++++
 .../index/readers/vector/HnswDocIdCollector.java   |  78 ++++
 .../readers/vector/HnswVectorIndexReader.java      | 194 +++++++++
 .../segment/index/vector/VectorIndexPlugin.java    |  42 +-
 .../segment/index/vector/VectorIndexType.java      | 174 ++++++++
 .../local/segment/store/FilePerIndexDirectory.java |   2 +
 .../segment/store/SingleFileIndexDirectory.java    |   7 +
 .../local/segment/store/VectorIndexUtils.java      |  98 +++++
 .../local/segment/index/loader/LoaderTest.java     | 186 +++++++++
 .../src/test/resources/data/test_vector_data.avro  | Bin 0 -> 6219422 bytes
 .../org/apache/pinot/segment/spi/V1Constants.java  |   3 +
 .../pinot/segment/spi/datasource/DataSource.java   |   7 +
 .../pinot/segment/spi/index/StandardIndexes.java   |   9 +
 .../spi/index/creator/VectorIndexConfig.java       | 126 ++++++
 .../spi/index/creator/VectorIndexCreator.java      |  55 ++-
 .../spi/index/reader/VectorIndexReader.java        |  42 +-
 .../segment/spi/store/SegmentDirectoryPaths.java   |   7 +
 .../apache/pinot/spi/config/table/FieldConfig.java |   2 +-
 .../pinot/spi/config/table/IndexingConfig.java     |  28 ++
 .../org/apache/pinot/tools/QuickStartBase.java     | 460 ++++++++++++++++++++-
 .../java/org/apache/pinot/tools/Quickstart.java    | 202 +--------
 .../org/apache/pinot/tools/RealtimeQuickStart.java |   2 +
 .../fineFoodReviews_offline_table_config.json      |  35 +-
 .../fineFoodReviews_realtime_table_config.json     |  73 ++++
 .../fineFoodReviews/fineFoodReviews_schema.json    |  48 +++
 .../fine_food_reviews_with_embeddings_1k.json.gz   | Bin 0 -> 8101775 bytes
 45 files changed, 2760 insertions(+), 315 deletions(-)

diff --git a/pinot-common/src/main/java/org/apache/pinot/common/function/FunctionRegistry.java b/pinot-common/src/main/java/org/apache/pinot/common/function/FunctionRegistry.java
index 1fc583bd6a..97fa972bee 100644
--- a/pinot-common/src/main/java/org/apache/pinot/common/function/FunctionRegistry.java
+++ b/pinot-common/src/main/java/org/apache/pinot/common/function/FunctionRegistry.java
@@ -173,5 +173,10 @@ public class FunctionRegistry {
     public static boolean jsonMatch(String text, String pattern) {
       throw new UnsupportedOperationException("Placeholder scalar function, should not reach here");
     }
+
+    @ScalarFunction(names = {"vectorSimilarity", "vector_similarity"}, isPlaceholder = true)
+    public static double vectorSimilarity(float[] vector1, float[] vector2) {
+      throw new UnsupportedOperationException("Placeholder scalar function, should not reach here");
+    }
   }
 }
diff --git a/pinot-common/src/main/java/org/apache/pinot/common/function/TransformFunctionType.java b/pinot-common/src/main/java/org/apache/pinot/common/function/TransformFunctionType.java
index 330890a2f9..c2d7d3e240 100644
--- a/pinot-common/src/main/java/org/apache/pinot/common/function/TransformFunctionType.java
+++ b/pinot-common/src/main/java/org/apache/pinot/common/function/TransformFunctionType.java
@@ -264,6 +264,9 @@ public enum TransformFunctionType {
       OperandTypes.family(ImmutableList.of(SqlTypeFamily.ARRAY)), "vector_dims"),
   VECTOR_NORM("vectorNorm", ReturnTypes.explicit(SqlTypeName.DOUBLE),
       OperandTypes.family(ImmutableList.of(SqlTypeFamily.ARRAY)), "vector_norm"),
+  VECTOR_SIMILARITY("vectorSimilarity", ReturnTypes.explicit(SqlTypeName.DOUBLE),
+      OperandTypes.family(ImmutableList.of(SqlTypeFamily.ARRAY, SqlTypeFamily.ARRAY, SqlTypeFamily.CHARACTER),
+          ordinal -> ordinal > 1 && ordinal < 4), "vector_similarity"),
 
   ARRAY_VALUE_CONSTRUCTOR("arrayValueConstructor", "array_value_constructor"),
 
diff --git a/pinot-common/src/main/java/org/apache/pinot/common/function/scalar/VectorFunctions.java b/pinot-common/src/main/java/org/apache/pinot/common/function/scalar/VectorFunctions.java
index 3c1ab75478..2de34d15c9 100644
--- a/pinot-common/src/main/java/org/apache/pinot/common/function/scalar/VectorFunctions.java
+++ b/pinot-common/src/main/java/org/apache/pinot/common/function/scalar/VectorFunctions.java
@@ -116,6 +116,38 @@ public class VectorFunctions {
     return distance;
   }
 
+  /**
+   * Returns the Euclidean distance between two vectors
+   * @param vector1 vector1
+   * @param vector2 vector2
+   * @return Euclidean distance
+   */
+  @ScalarFunction(names = {"euclideandistance", "euclidean_distance"})
+  public static double euclideanDistance(float[] vector1, float[] vector2) {
+    validateVectors(vector1, vector2);
+    double distance = 0;
+    for (int i = 0; i < vector1.length; i++) {
+      distance += Math.pow(vector1[i] - vector2[i], 2);
+    }
+    return distance;
+  }
+
+  /**
+   * Returns the dot product between two vectors
+   * @param vector1 vector1
+   * @param vector2 vector2
+   * @return dot product
+   */
+  @ScalarFunction(names = {"dotproduct", "dot_product"})
+  public static double dotProduct(float[] vector1, float[] vector2) {
+    validateVectors(vector1, vector2);
+    double dotProduct = 0.0;
+    for (int i = 0; i < vector1.length; i++) {
+      dotProduct += vector1[i] * vector2[i];
+    }
+    return dotProduct;
+  }
+
   /**
    * Returns the number of dimensions in a vector
    * @param vector input vector
diff --git a/pinot-common/src/main/java/org/apache/pinot/common/request/context/RequestContextUtils.java b/pinot-common/src/main/java/org/apache/pinot/common/request/context/RequestContextUtils.java
index 933f022bfb..0369b29f59 100644
--- a/pinot-common/src/main/java/org/apache/pinot/common/request/context/RequestContextUtils.java
+++ b/pinot-common/src/main/java/org/apache/pinot/common/request/context/RequestContextUtils.java
@@ -37,6 +37,7 @@ import org.apache.pinot.common.request.context.predicate.RangePredicate;
 import org.apache.pinot.common.request.context.predicate.RegexpLikePredicate;
 import org.apache.pinot.common.request.context.predicate.TextContainsPredicate;
 import org.apache.pinot.common.request.context.predicate.TextMatchPredicate;
+import org.apache.pinot.common.request.context.predicate.VectorSimilarityPredicate;
 import org.apache.pinot.common.utils.RegexpPatternConverterUtils;
 import org.apache.pinot.common.utils.request.RequestUtils;
 import org.apache.pinot.segment.spi.AggregationFunctionType;
@@ -241,6 +242,14 @@ public class RequestContextUtils {
       case JSON_MATCH:
         return FilterContext.forPredicate(
             new JsonMatchPredicate(getExpression(operands.get(0)), getStringValue(operands.get(1))));
+      case VECTOR_SIMILARITY:
+        ExpressionContext lhs = getExpression(operands.get(0));
+        float[] vectorValue = getVectorValue(operands.get(1));
+        int topK = VectorSimilarityPredicate.DEFAULT_TOP_K;
+        if (operands.size() == 3) {
+          topK = (int) operands.get(2).getLiteral().getLongValue();
+        }
+        return FilterContext.forPredicate(new VectorSimilarityPredicate(lhs, vectorValue, topK));
       case IS_NULL:
         return FilterContext.forPredicate(new IsNullPredicate(getExpression(operands.get(0))));
       case IS_NOT_NULL:
@@ -395,6 +404,13 @@ public class RequestContextUtils {
         return FilterContext.forPredicate(new TextMatchPredicate(operands.get(0), getStringValue(operands.get(1))));
       case JSON_MATCH:
         return FilterContext.forPredicate(new JsonMatchPredicate(operands.get(0), getStringValue(operands.get(1))));
+      case VECTOR_SIMILARITY:
+        int topK = VectorSimilarityPredicate.DEFAULT_TOP_K;
+        if (operands.size() == 3) {
+          topK = (int) operands.get(2).getLiteral().getLongValue();
+        }
+        return FilterContext.forPredicate(
+            new VectorSimilarityPredicate(operands.get(0), getVectorValue(operands.get(1)), topK));
       case IS_NULL:
         return FilterContext.forPredicate(new IsNullPredicate(operands.get(0)));
       case IS_NOT_NULL:
@@ -414,4 +430,30 @@ public class RequestContextUtils {
     }
     return expressionContext.getLiteral().getStringValue();
   }
+
+  private static float[] getVectorValue(ExpressionContext expressionContext) {
+    if (expressionContext.getType() != ExpressionContext.Type.FUNCTION) {
+      throw new BadQueryRequestException(
+          "Pinot does not support column or function on the right-hand side of the predicate");
+    }
+    float[] vector = new float[expressionContext.getFunction().getArguments().size()];
+    for (int i = 0; i < expressionContext.getFunction().getArguments().size(); i++) {
+      vector[i] =
+          Float.parseFloat(expressionContext.getFunction().getArguments().get(i).getLiteral().getValue().toString());
+    }
+    return vector;
+  }
+
+  private static float[] getVectorValue(Expression thriftExpression) {
+    if (thriftExpression.getType() != ExpressionType.FUNCTION) {
+      throw new BadQueryRequestException(
+          "Pinot does not support column or function on the right-hand side of the predicate");
+    }
+    float[] vector = new float[thriftExpression.getFunctionCall().getOperandsSize()];
+    for (int i = 0; i < thriftExpression.getFunctionCall().getOperandsSize(); i++) {
+      vector[i] = Float.parseFloat(
+          Double.toString(thriftExpression.getFunctionCall().getOperands().get(i).getLiteral().getDoubleValue()));
+    }
+    return vector;
+  }
 }
diff --git a/pinot-common/src/main/java/org/apache/pinot/common/request/context/predicate/Predicate.java b/pinot-common/src/main/java/org/apache/pinot/common/request/context/predicate/Predicate.java
index 724bc81a19..db88ed5c11 100644
--- a/pinot-common/src/main/java/org/apache/pinot/common/request/context/predicate/Predicate.java
+++ b/pinot-common/src/main/java/org/apache/pinot/common/request/context/predicate/Predicate.java
@@ -38,7 +38,8 @@ public interface Predicate {
     TEXT_MATCH,
     JSON_MATCH,
     IS_NULL,
-    IS_NOT_NULL(true);
+    IS_NOT_NULL(true),
+    VECTOR_SIMILARITY;
 
     private final boolean _exclusive;
 
diff --git a/pinot-common/src/main/java/org/apache/pinot/common/request/context/predicate/VectorSimilarityPredicate.java b/pinot-common/src/main/java/org/apache/pinot/common/request/context/predicate/VectorSimilarityPredicate.java
new file mode 100644
index 0000000000..b8281a1a08
--- /dev/null
+++ b/pinot-common/src/main/java/org/apache/pinot/common/request/context/predicate/VectorSimilarityPredicate.java
@@ -0,0 +1,82 @@
+/**
+ * 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.pinot.common.request.context.predicate;
+
+import java.util.Arrays;
+import java.util.Objects;
+import org.apache.pinot.common.request.context.ExpressionContext;
+
+
+/**
+ * Predicate for vector similarity search.
+ * NOTE: Currently, we only support vector similarity search on float array column.
+ * Example:
+ *   {
+ *   "type": "vectorSimilarity",
+ *   "leftValue": "embedding",
+ *   "rightValue": [1.0, 2.0, 3.0],
+ *   "topK": 10
+ *   }
+ */
+public class VectorSimilarityPredicate extends BasePredicate {
+  public static final int DEFAULT_TOP_K = 10;
+  private final float[] _value;
+  private final int _topK;
+
+  public VectorSimilarityPredicate(ExpressionContext lhs, float[] value, int topK) {
+    super(lhs);
+    _value = value;
+    _topK = topK;
+  }
+
+  @Override
+  public Type getType() {
+    return Type.VECTOR_SIMILARITY;
+  }
+
+  public float[] getValue() {
+    return _value;
+  }
+
+  public int getTopK() {
+    return _topK;
+  }
+
+  @Override
+  public boolean equals(Object o) {
+    if (this == o) {
+      return true;
+    }
+    if (!(o instanceof VectorSimilarityPredicate)) {
+      return false;
+    }
+    VectorSimilarityPredicate that = (VectorSimilarityPredicate) o;
+    return Objects.equals(_lhs, that._lhs) && Arrays.equals(_value, that._value) && _topK == that._topK;
+  }
+
+  @Override
+  public int hashCode() {
+    return Objects.hash(_lhs, Arrays.hashCode(_value), _topK);
+  }
+
+  @Override
+  public String toString() {
+    return "vector_similarity(" + _lhs + ",'" + Arrays.toString(_value) + "'," + _topK + ")";
+  }
+}
diff --git a/pinot-common/src/main/java/org/apache/pinot/sql/FilterKind.java b/pinot-common/src/main/java/org/apache/pinot/sql/FilterKind.java
index 0c2a8dca75..fdf47e7239 100644
--- a/pinot-common/src/main/java/org/apache/pinot/sql/FilterKind.java
+++ b/pinot-common/src/main/java/org/apache/pinot/sql/FilterKind.java
@@ -38,7 +38,8 @@ public enum FilterKind {
   TEXT_MATCH,
   JSON_MATCH,
   IS_NULL,
-  IS_NOT_NULL;
+  IS_NOT_NULL,
+  VECTOR_SIMILARITY;
 
   /**
    * Helper method that returns true if the enum maps to a Range.
diff --git a/pinot-common/src/main/java/org/apache/pinot/sql/parsers/CalciteSqlParser.java b/pinot-common/src/main/java/org/apache/pinot/sql/parsers/CalciteSqlParser.java
index 5b28ce238f..63058fd8c3 100644
--- a/pinot-common/src/main/java/org/apache/pinot/sql/parsers/CalciteSqlParser.java
+++ b/pinot-common/src/main/java/org/apache/pinot/sql/parsers/CalciteSqlParser.java
@@ -289,6 +289,26 @@ public class CalciteSqlParser {
       for (Expression filter : filterExpression.getFunctionCall().getOperands()) {
         validateFilter(filter);
       }
+    } else if (operator.equals(FilterKind.VECTOR_SIMILARITY.name())) {
+      Expression vectorIdentifier = filterExpression.getFunctionCall().getOperands().get(0);
+      if (!vectorIdentifier.isSetIdentifier()) {
+        throw new IllegalStateException("The first argument of VECTOR_SIMILARITY must be an identifier of float array, "
+            + "the signature is VECTOR_SIMILARITY(float[], float[], int).");
+      }
+      Expression vectorLiteral = filterExpression.getFunctionCall().getOperands().get(1);
+      // Array Literal is a function of type 'ARRAYVALUECONSTRUCTOR' with operands of Float/Double Literals
+      if (!vectorLiteral.isSetFunctionCall() || !vectorLiteral.getFunctionCall().getOperator().equalsIgnoreCase(
+          "arrayvalueconstructor")) {
+        throw new IllegalStateException("The second argument of VECTOR_SIMILARITY must be a float array literal, "
+            + "the signature is VECTOR_SIMILARITY(float[], float[], int).");
+      }
+      if (filterExpression.getFunctionCall().getOperands().size() == 3) {
+        Expression topK = filterExpression.getFunctionCall().getOperands().get(2);
+        if (!topK.isSetLiteral()) {
+          throw new IllegalStateException("The third argument of VECTOR_SIMILARITY must be an integer literal, "
+              + "the signature is VECTOR_SIMILARITY(float[], float[], int).");
+        }
+      }
     } else {
       List<Expression> operands = filterExpression.getFunctionCall().getOperands();
       for (int i = 1; i < operands.size(); i++) {
diff --git a/pinot-common/src/main/java/org/apache/pinot/sql/parsers/rewriter/PredicateComparisonRewriter.java b/pinot-common/src/main/java/org/apache/pinot/sql/parsers/rewriter/PredicateComparisonRewriter.java
index f787ae2eaf..1917e37abc 100644
--- a/pinot-common/src/main/java/org/apache/pinot/sql/parsers/rewriter/PredicateComparisonRewriter.java
+++ b/pinot-common/src/main/java/org/apache/pinot/sql/parsers/rewriter/PredicateComparisonRewriter.java
@@ -18,6 +18,7 @@
  */
 package org.apache.pinot.sql.parsers.rewriter;
 
+import com.google.common.base.Preconditions;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.List;
@@ -128,6 +129,23 @@ public class PredicateComparisonRewriter implements QueryRewriter {
             break;
           }
           break;
+        case VECTOR_SIMILARITY: {
+          Preconditions.checkArgument(operands.size() >= 2 && operands.size() <= 3,
+              "For %s predicate, the number of operands must be at either 2 or 3, got: %s", filterKind, expression);
+          // Array Literal is a function of type 'ARRAYVALUECONSTRUCTOR' with operands of Float/Double Literals
+          if (operands.get(1).getFunctionCall() == null || !operands.get(1).getFunctionCall().getOperator()
+              .equalsIgnoreCase("arrayvalueconstructor")) {
+            throw new SqlCompilationException(
+                String.format("For %s predicate, the second operand must be a float array literal, got: %s", filterKind,
+                    expression));
+          }
+          if (operands.size() == 3 && operands.get(2).getLiteral() == null) {
+            throw new SqlCompilationException(
+                String.format("For %s predicate, the third operand must be a literal, got: %s", filterKind,
+                    expression));
+          }
+          break;
+        }
         default:
           int numOperands = operands.size();
           for (int i = 1; i < numOperands; i++) {
@@ -150,18 +168,19 @@ public class PredicateComparisonRewriter implements QueryRewriter {
    *                "select * from table where col1 = true"
    *     Example2: "select * from table where startsWith(col1, 'str')" converts to
    *               "select * from table where startsWith(col1, 'str') = true"
+   *
    * @param expression Expression
    * @return Rewritten expression
    */
   private static Expression convertPredicateToEqualsBooleanExpression(Expression expression) {
-      Expression newExpression;
-      newExpression = RequestUtils.getFunctionExpression(FilterKind.EQUALS.name());
-      List<Expression> operands = new ArrayList<>();
-      operands.add(expression);
-      operands.add(RequestUtils.getLiteralExpression(true));
-      newExpression.getFunctionCall().setOperands(operands);
+    Expression newExpression;
+    newExpression = RequestUtils.getFunctionExpression(FilterKind.EQUALS.name());
+    List<Expression> operands = new ArrayList<>();
+    operands.add(expression);
+    operands.add(RequestUtils.getLiteralExpression(true));
+    newExpression.getFunctionCall().setOperands(operands);
 
-      return newExpression;
+    return newExpression;
   }
 
   /**
diff --git a/pinot-core/src/main/java/org/apache/pinot/core/operator/filter/VectorSimilarityFilterOperator.java b/pinot-core/src/main/java/org/apache/pinot/core/operator/filter/VectorSimilarityFilterOperator.java
new file mode 100644
index 0000000000..633418560e
--- /dev/null
+++ b/pinot-core/src/main/java/org/apache/pinot/core/operator/filter/VectorSimilarityFilterOperator.java
@@ -0,0 +1,118 @@
+/**
+ * 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.pinot.core.operator.filter;
+
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+import org.apache.pinot.common.request.context.predicate.VectorSimilarityPredicate;
+import org.apache.pinot.core.common.BlockDocIdSet;
+import org.apache.pinot.core.common.Operator;
+import org.apache.pinot.core.operator.docidsets.BitmapDocIdSet;
+import org.apache.pinot.segment.spi.index.reader.VectorIndexReader;
+import org.apache.pinot.spi.data.FieldSpec;
+import org.apache.pinot.spi.trace.FilterType;
+import org.apache.pinot.spi.trace.InvocationRecording;
+import org.apache.pinot.spi.trace.Tracing;
+import org.roaringbitmap.buffer.ImmutableRoaringBitmap;
+
+
+/**
+ * Operator for Vector Search query.
+ * <p>Currently, we only support vector similarity search on float array column.
+ * Example:
+ * {
+ *  "type": "vectorSimilarity",
+ *  "leftValue": "embedding",
+ *  "rightValue": [1.0, 2.0, 3.0],
+ *  "topK": 10
+ *  }
+ *
+ */
+public class VectorSimilarityFilterOperator extends BaseFilterOperator {
+  private static final String EXPLAIN_NAME = "VECTOR_SIMILARITY_INDEX";
+
+  private final VectorIndexReader _vectorIndexReader;
+  private final VectorSimilarityPredicate _predicate;
+  private ImmutableRoaringBitmap _matches;
+
+  public VectorSimilarityFilterOperator(VectorIndexReader vectorIndexReader, VectorSimilarityPredicate predicate,
+      int numDocs) {
+    super(numDocs, false);
+    _vectorIndexReader = vectorIndexReader;
+    _predicate = predicate;
+    _matches = null;
+  }
+
+  @Override
+  protected BlockDocIdSet getTrues() {
+    if (_matches == null) {
+      _matches = _vectorIndexReader.getDocIds(_predicate.getValue(), _predicate.getTopK());
+    }
+    return new BitmapDocIdSet(_matches, _numDocs);
+  }
+
+  @Override
+  public int getNumMatchingDocs() {
+    if (_matches == null) {
+      _matches = _vectorIndexReader.getDocIds(_predicate.getValue(), _predicate.getTopK());
+    }
+    return _matches.getCardinality();
+  }
+
+  @Override
+  public boolean canProduceBitmaps() {
+    return true;
+  }
+
+  @Override
+  public BitmapCollection getBitmaps() {
+    if (_matches == null) {
+      _matches = _vectorIndexReader.getDocIds(_predicate.getValue(), _predicate.getTopK());
+    }
+    record(_matches);
+    return new BitmapCollection(_numDocs, false, _matches);
+  }
+
+  @Override
+  public List<Operator> getChildOperators() {
+    return Collections.emptyList();
+  }
+
+  @Override
+  public String toExplainString() {
+    return EXPLAIN_NAME + "(indexLookUp:vector_index"
+        + ", operator:" + _predicate.getType()
+        + ", vector identifier:" + _predicate.getLhs().getIdentifier()
+        + ", vector literal:" + Arrays.toString(_predicate.getValue())
+        + ", topK to search:" + _predicate.getTopK()
+        + ')';
+  }
+
+  private void record(ImmutableRoaringBitmap matches) {
+    InvocationRecording recording = Tracing.activeRecording();
+    if (recording.isEnabled()) {
+      recording.setNumDocsMatchingAfterFilter(matches.getCardinality());
+      recording.setColumnName(_predicate.getLhs().getIdentifier());
+      recording.setFilter(FilterType.INDEX, "VECTOR_SIMILARITY");
+      recording.setInputDataType(FieldSpec.DataType.FLOAT, false);
+      recording.setNumDocsMatchingAfterFilter(matches.getCardinality());
+    }
+  }
+}
diff --git a/pinot-core/src/main/java/org/apache/pinot/core/plan/FilterPlanNode.java b/pinot-core/src/main/java/org/apache/pinot/core/plan/FilterPlanNode.java
index d1cef2ef26..84c53374e2 100644
--- a/pinot-core/src/main/java/org/apache/pinot/core/plan/FilterPlanNode.java
+++ b/pinot-core/src/main/java/org/apache/pinot/core/plan/FilterPlanNode.java
@@ -32,6 +32,7 @@ import org.apache.pinot.common.request.context.predicate.Predicate;
 import org.apache.pinot.common.request.context.predicate.RegexpLikePredicate;
 import org.apache.pinot.common.request.context.predicate.TextContainsPredicate;
 import org.apache.pinot.common.request.context.predicate.TextMatchPredicate;
+import org.apache.pinot.common.request.context.predicate.VectorSimilarityPredicate;
 import org.apache.pinot.core.geospatial.transform.function.StDistanceFunction;
 import org.apache.pinot.core.operator.filter.BaseFilterOperator;
 import org.apache.pinot.core.operator.filter.BitmapBasedFilterOperator;
@@ -44,6 +45,7 @@ import org.apache.pinot.core.operator.filter.JsonMatchFilterOperator;
 import org.apache.pinot.core.operator.filter.MatchAllFilterOperator;
 import org.apache.pinot.core.operator.filter.TextContainsFilterOperator;
 import org.apache.pinot.core.operator.filter.TextMatchFilterOperator;
+import org.apache.pinot.core.operator.filter.VectorSimilarityFilterOperator;
 import org.apache.pinot.core.operator.filter.predicate.FSTBasedRegexpPredicateEvaluatorFactory;
 import org.apache.pinot.core.operator.filter.predicate.PredicateEvaluator;
 import org.apache.pinot.core.operator.filter.predicate.PredicateEvaluatorProvider;
@@ -56,6 +58,7 @@ import org.apache.pinot.segment.spi.index.mutable.ThreadSafeMutableRoaringBitmap
 import org.apache.pinot.segment.spi.index.reader.JsonIndexReader;
 import org.apache.pinot.segment.spi.index.reader.NullValueVectorReader;
 import org.apache.pinot.segment.spi.index.reader.TextIndexReader;
+import org.apache.pinot.segment.spi.index.reader.VectorIndexReader;
 import org.apache.pinot.spi.exception.BadQueryRequestException;
 import org.roaringbitmap.buffer.MutableRoaringBitmap;
 
@@ -289,6 +292,11 @@ public class FilterPlanNode implements PlanNode {
               Preconditions.checkState(jsonIndex != null, "Cannot apply JSON_MATCH on column: %s without json index",
                   column);
               return new JsonMatchFilterOperator(jsonIndex, (JsonMatchPredicate) predicate, numDocs);
+            case VECTOR_SIMILARITY:
+              VectorIndexReader vectorIndex = dataSource.getVectorIndex();
+              Preconditions.checkState(vectorIndex != null,
+                  "Cannot apply VECTOR_SIMILARITY on column: %s without vector index", column);
+              return new VectorSimilarityFilterOperator(vectorIndex, (VectorSimilarityPredicate) predicate, numDocs);
             case IS_NULL:
               NullValueVectorReader nullValueVector = dataSource.getNullValueVector();
               if (nullValueVector != null) {
diff --git a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/realtime/impl/vector/MutableVectorIndex.java b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/realtime/impl/vector/MutableVectorIndex.java
new file mode 100644
index 0000000000..a591650be4
--- /dev/null
+++ b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/realtime/impl/vector/MutableVectorIndex.java
@@ -0,0 +1,154 @@
+/**
+ * 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.pinot.segment.local.realtime.impl.vector;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.Arrays;
+import javax.annotation.Nonnull;
+import javax.annotation.Nullable;
+import org.apache.commons.io.FileUtils;
+import org.apache.lucene.document.Document;
+import org.apache.lucene.document.StoredField;
+import org.apache.lucene.index.DirectoryReader;
+import org.apache.lucene.index.IndexWriter;
+import org.apache.lucene.index.VectorSimilarityFunction;
+import org.apache.lucene.search.IndexSearcher;
+import org.apache.lucene.search.KnnFloatVectorQuery;
+import org.apache.lucene.search.Query;
+import org.apache.lucene.search.TopDocs;
+import org.apache.lucene.store.FSDirectory;
+import org.apache.pinot.segment.local.segment.creator.impl.vector.XKnnFloatVectorField;
+import org.apache.pinot.segment.local.segment.store.VectorIndexUtils;
+import org.apache.pinot.segment.spi.V1Constants;
+import org.apache.pinot.segment.spi.index.creator.VectorIndexConfig;
+import org.apache.pinot.segment.spi.index.mutable.MutableIndex;
+import org.apache.pinot.segment.spi.index.reader.VectorIndexReader;
+import org.roaringbitmap.buffer.MutableRoaringBitmap;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * A Vector index reader for the real-time Vector index values on the fly.
+ * Since there is no good mutable vector index implementation for topK search, we just do brute force search.
+ * <p>This class is thread-safe for single writer multiple readers.
+ */
+public class MutableVectorIndex implements VectorIndexReader, MutableIndex {
+  private static final Logger LOGGER = LoggerFactory.getLogger(MutableVectorIndex.class);
+  public static final String VECTOR_INDEX_DOC_ID_COLUMN_NAME = "DocID";
+  public static final long DEFAULT_COMMIT_INTERVAL_MS = 10_000L;
+  public static final long DEFAULT_COMMIT_DOCS = 1000L;
+  private final int _vectorDimension;
+  private final VectorSimilarityFunction _vectorSimilarityFunction;
+  private final IndexWriter _indexWriter;
+  private final String _vectorColumn;
+  private final String _segmentName;
+  private final long _commitIntervalMs;
+  private final long _commitDocs;
+  private final File _indexDir;
+
+  private final FSDirectory _indexDirectory;
+  private int _nextDocId;
+
+  private long _lastCommitTime;
+
+  public MutableVectorIndex(String segmentName, String vectorColumn, VectorIndexConfig vectorIndexConfig) {
+    _vectorColumn = vectorColumn;
+    _vectorDimension = vectorIndexConfig.getVectorDimension();
+    _segmentName = segmentName;
+    _commitIntervalMs = Long.parseLong(
+        vectorIndexConfig.getProperties().getOrDefault("commitIntervalMs", String.valueOf(DEFAULT_COMMIT_INTERVAL_MS)));
+    _commitDocs = Long.parseLong(
+        vectorIndexConfig.getProperties().getOrDefault("commitDocs", String.valueOf(DEFAULT_COMMIT_DOCS)));
+    _vectorSimilarityFunction = VectorIndexUtils.toSimilarityFunction(vectorIndexConfig.getVectorDistanceFunction());
+    try {
+      // segment generation is always in V1 and later we convert (as part of post creation processing)
+      // to V3 if segmentVersion is set to V3 in SegmentGeneratorConfig.
+      _indexDir = new File(FileUtils.getTempDirectory(), segmentName);
+      _indexDirectory = FSDirectory.open(
+          new File(_indexDir, _vectorColumn + V1Constants.Indexes.VECTOR_HNSW_INDEX_FILE_EXTENSION).toPath());
+      LOGGER.info("Creating mutable HNSW index for segment: {}, column: {} at path: {} with {}", segmentName,
+          vectorColumn, _indexDir.getAbsolutePath(), vectorIndexConfig.getProperties());
+      _indexWriter = new IndexWriter(_indexDirectory, VectorIndexUtils.getIndexWriterConfig(vectorIndexConfig));
+      _indexWriter.commit();
+      _lastCommitTime = System.currentTimeMillis();
+    } catch (Exception e) {
+      throw new RuntimeException(
+          "Caught exception while instantiating the LuceneTextIndexCreator for column: " + vectorColumn, e);
+    }
+  }
+
+  @Override
+  public void add(@Nonnull Object value, int dictId, int docId) {
+    throw new UnsupportedOperationException("Mutable Vector indexes are not supported for single-valued columns");
+  }
+
+  @Override
+  public void add(@Nonnull Object[] values, @Nullable int[] dictIds, int docId) {
+    float[] floatValues = new float[_vectorDimension];
+    for (int i = 0; i < values.length; i++) {
+      floatValues[i] = (Float) values[i];
+    }
+    Document docToIndex = new Document();
+    XKnnFloatVectorField xKnnFloatVectorField =
+        new XKnnFloatVectorField(_vectorColumn, floatValues, _vectorSimilarityFunction);
+    docToIndex.add(xKnnFloatVectorField);
+    docToIndex.add(new StoredField(VECTOR_INDEX_DOC_ID_COLUMN_NAME, _nextDocId++));
+    try {
+      _indexWriter.addDocument(docToIndex);
+      if ((_lastCommitTime + _commitIntervalMs < System.currentTimeMillis()) || (_nextDocId % _commitDocs == 0)) {
+        _indexWriter.commit();
+        _lastCommitTime = System.currentTimeMillis();
+        LOGGER.debug("Committed index for column: {}, segment: {}", _vectorColumn, _segmentName);
+      }
+    } catch (Exception e) {
+      throw new RuntimeException(
+          "Caught exception while adding a new document to the Lucene index for column: " + _vectorColumn, e);
+    }
+  }
+
+  @Override
+  public MutableRoaringBitmap getDocIds(float[] vector, int topK) {
+    MutableRoaringBitmap docIds;
+    try {
+      IndexSearcher indexSearcher = new IndexSearcher(DirectoryReader.open(_indexDirectory));
+      Query query = new KnnFloatVectorQuery(_vectorColumn, vector, topK);
+      docIds = new MutableRoaringBitmap();
+      TopDocs search = indexSearcher.search(query, topK);
+      Arrays.stream(search.scoreDocs).map(scoreDoc -> scoreDoc.doc).forEach(docIds::add);
+    } catch (IOException e) {
+      throw new RuntimeException(e);
+    }
+    return docIds;
+  }
+
+  @Override
+  public void close() {
+    try {
+      _indexWriter.commit();
+      _indexWriter.close();
+    } catch (IOException e) {
+      throw new RuntimeException(e);
+    } finally {
+      // Delete the temporary index directory.
+      FileUtils.deleteQuietly(_indexDir);
+    }
+  }
+}
diff --git a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/creator/impl/vector/HnswVectorIndexCreator.java b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/creator/impl/vector/HnswVectorIndexCreator.java
new file mode 100644
index 0000000000..d13b450397
--- /dev/null
+++ b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/creator/impl/vector/HnswVectorIndexCreator.java
@@ -0,0 +1,120 @@
+/**
+ * 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.pinot.segment.local.segment.creator.impl.vector;
+
+import java.io.File;
+import java.io.IOException;
+import javax.annotation.Nonnull;
+import javax.annotation.Nullable;
+import org.apache.lucene.document.Document;
+import org.apache.lucene.document.StoredField;
+import org.apache.lucene.index.IndexWriter;
+import org.apache.lucene.index.VectorSimilarityFunction;
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.store.FSDirectory;
+import org.apache.pinot.segment.local.segment.creator.impl.SegmentColumnarIndexCreator;
+import org.apache.pinot.segment.local.segment.store.VectorIndexUtils;
+import org.apache.pinot.segment.spi.V1Constants;
+import org.apache.pinot.segment.spi.index.creator.VectorIndexConfig;
+import org.apache.pinot.segment.spi.index.creator.VectorIndexCreator;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * This is used to create Lucene based HNSW index.
+ * Used for both offline from {@link SegmentColumnarIndexCreator}
+ * and realtime from {@link org.apache.pinot.segment.local.realtime.impl.vector.MutableVectorIndex}
+ */
+public class HnswVectorIndexCreator implements VectorIndexCreator {
+  private static final Logger LOGGER = LoggerFactory.getLogger(HnswVectorIndexCreator.class);
+  public static final String VECTOR_INDEX_DOC_ID_COLUMN_NAME = "DocID";
+
+  private final Directory _indexDirectory;
+  private final IndexWriter _indexWriter;
+  private final String _vectorColumn;
+  private final VectorSimilarityFunction _vectorSimilarityFunction;
+  private final int _vectorDimension;
+
+  private int _nextDocId = 0;
+
+  public HnswVectorIndexCreator(String column, File segmentIndexDir, VectorIndexConfig vectorIndexConfig) {
+    _vectorColumn = column;
+    _vectorDimension = vectorIndexConfig.getVectorDimension();
+    _vectorSimilarityFunction = VectorIndexUtils.toSimilarityFunction(vectorIndexConfig.getVectorDistanceFunction());
+    try {
+      // segment generation is always in V1 and later we convert (as part of post creation processing)
+      // to V3 if segmentVersion is set to V3 in SegmentGeneratorConfig.
+      File indexFile = new File(segmentIndexDir, _vectorColumn + V1Constants.Indexes.VECTOR_HNSW_INDEX_FILE_EXTENSION);
+      _indexDirectory = FSDirectory.open(indexFile.toPath());
+      LOGGER.info("Creating HNSW index for column: {} at path: {} with {} for segment: {}", column,
+          indexFile.getAbsolutePath(), vectorIndexConfig.getProperties(), segmentIndexDir.getAbsolutePath());
+      _indexWriter = new IndexWriter(_indexDirectory, VectorIndexUtils.getIndexWriterConfig(vectorIndexConfig));
+    } catch (Exception e) {
+      throw new RuntimeException(
+          "Caught exception while instantiating the HnswVectorIndexCreator for column: " + column, e);
+    }
+  }
+
+  @Override
+  public void add(@Nonnull Object[] values, @Nullable int[] dictIds) {
+    float[] floatValues = new float[_vectorDimension];
+    for (int i = 0; i < values.length; i++) {
+      floatValues[i] = (Float) values[i];
+    }
+    add(floatValues);
+  }
+
+  @Override
+  public void add(float[] document) {
+    Document docToIndex = new Document();
+    XKnnFloatVectorField xKnnFloatVectorField =
+        new XKnnFloatVectorField(_vectorColumn, document, _vectorSimilarityFunction);
+    docToIndex.add(xKnnFloatVectorField);
+    docToIndex.add(new StoredField(VECTOR_INDEX_DOC_ID_COLUMN_NAME, _nextDocId++));
+    try {
+      _indexWriter.addDocument(docToIndex);
+    } catch (Exception e) {
+      throw new RuntimeException(
+          "Caught exception while adding a new document to the HNSW index for column: " + _vectorColumn, e);
+    }
+  }
+
+  @Override
+  public void seal() {
+    try {
+      LOGGER.info("Sealing HNSW index for column: " + _vectorColumn);
+      _indexWriter.forceMerge(1);
+    } catch (Exception e) {
+      throw new RuntimeException("Caught exception while sealing the HNSW index for column: " + _vectorColumn, e);
+    }
+  }
+
+  @Override
+  public void close()
+      throws IOException {
+    try {
+      // based on the commit flag set in IndexWriterConfig, this will decide to commit or not
+      _indexWriter.close();
+      _indexDirectory.close();
+    } catch (Exception e) {
+      throw new RuntimeException("Caught exception while closing the HNSW index for column: " + _vectorColumn, e);
+    }
+  }
+}
diff --git a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/creator/impl/vector/XKnnFloatVectorField.java b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/creator/impl/vector/XKnnFloatVectorField.java
new file mode 100644
index 0000000000..ed1b5e707c
--- /dev/null
+++ b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/creator/impl/vector/XKnnFloatVectorField.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.pinot.segment.local.segment.creator.impl.vector;
+
+import org.apache.lucene.document.FieldType;
+import org.apache.lucene.document.KnnFloatVectorField;
+import org.apache.lucene.index.VectorEncoding;
+import org.apache.lucene.index.VectorSimilarityFunction;
+
+
+public class XKnnFloatVectorField extends KnnFloatVectorField {
+  public static final int MAX_DIMS_COUNT = 2048;
+
+  private static FieldType createType(float[] v, VectorSimilarityFunction similarityFunction) {
+    if (v == null) {
+      throw new IllegalArgumentException("vector value must not be null");
+    }
+    int dimension = v.length;
+    if (dimension == 0) {
+      throw new IllegalArgumentException("cannot index an empty vector");
+    }
+    if (dimension > MAX_DIMS_COUNT) {
+      throw new IllegalArgumentException("cannot index vectors with dimension greater than " + MAX_DIMS_COUNT);
+    }
+    if (similarityFunction == null) {
+      throw new IllegalArgumentException("similarity function must not be null");
+    }
+    FieldType type = new FieldType() {
+      @Override
+      public int vectorDimension() {
+        return dimension;
+      }
+
+      @Override
+      public VectorEncoding vectorEncoding() {
+        return VectorEncoding.FLOAT32;
+      }
+
+      @Override
+      public VectorSimilarityFunction vectorSimilarityFunction() {
+        return similarityFunction;
+      }
+    };
+    type.freeze();
+
+    return type;
+  }
+
+  public XKnnFloatVectorField(String name, float[] vector, VectorSimilarityFunction similarityFunction) {
+    super(name, vector, createType(vector, similarityFunction));
+  }
+}
diff --git a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/creator/impl/vector/lucene95/HnswCodec.java b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/creator/impl/vector/lucene95/HnswCodec.java
new file mode 100644
index 0000000000..ee7cf560df
--- /dev/null
+++ b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/creator/impl/vector/lucene95/HnswCodec.java
@@ -0,0 +1,139 @@
+/**
+ * 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.pinot.segment.local.segment.creator.impl.vector.lucene95;
+
+import org.apache.lucene.codecs.DocValuesFormat;
+import org.apache.lucene.codecs.FilterCodec;
+import org.apache.lucene.codecs.KnnVectorsFormat;
+import org.apache.lucene.codecs.PointsFormat;
+import org.apache.lucene.codecs.PostingsFormat;
+import org.apache.lucene.codecs.lucene90.Lucene90DocValuesFormat;
+import org.apache.lucene.codecs.lucene90.Lucene90PointsFormat;
+import org.apache.lucene.codecs.lucene90.Lucene90PostingsFormat;
+import org.apache.lucene.codecs.lucene95.Lucene95Codec;
+import org.apache.lucene.codecs.perfield.PerFieldDocValuesFormat;
+import org.apache.lucene.codecs.perfield.PerFieldKnnVectorsFormat;
+import org.apache.lucene.codecs.perfield.PerFieldPostingsFormat;
+
+
+/**
+ * Extend the Lucene 9.5 index format
+ * The major change here is to allow custom: @link{org.apache.lucene.codecs.KnnVectorsFormat}
+ *
+ * @see org.apache.lucene.codecs.lucene95 package documentation for file format details.
+ */
+public class HnswCodec extends FilterCodec {
+
+  private final PostingsFormat _defaultPostingsFormat;
+  private final PostingsFormat _postingsFormat =
+      new PerFieldPostingsFormat() {
+        @Override
+        public PostingsFormat getPostingsFormatForField(String field) {
+          return HnswCodec.this.getPostingsFormatForField(field);
+        }
+      };
+
+  private final DocValuesFormat _defaultDVFormat;
+  private final DocValuesFormat _docValuesFormat =
+      new PerFieldDocValuesFormat() {
+        @Override
+        public DocValuesFormat getDocValuesFormatForField(String field) {
+          return HnswCodec.this.getDocValuesFormatForField(field);
+        }
+      };
+
+  private final KnnVectorsFormat _knnVectorsFormat =
+      new PerFieldKnnVectorsFormat() {
+        @Override
+        public KnnVectorsFormat getKnnVectorsFormatForField(String field) {
+          return HnswCodec.this.getKnnVectorsFormatForField(field);
+        }
+      };
+
+  private final KnnVectorsFormat _defaultKnnVectorsFormat;
+
+  /**
+   * Instantiates a new codec, specifying the stored fields compression mode to use.
+   *
+   * @param mode stored fields compression mode to use for newly flushed/merged segments.
+   */
+  public HnswCodec(Lucene95Codec.Mode mode, KnnVectorsFormat defaultKnnVectorsFormat) {
+    super("Lucene95", new Lucene95Codec(mode));
+    _defaultKnnVectorsFormat = defaultKnnVectorsFormat;
+    _defaultPostingsFormat = new Lucene90PostingsFormat();
+    _defaultDVFormat = new Lucene90DocValuesFormat();
+  }
+
+  @Override
+  public final PostingsFormat postingsFormat() {
+    return _postingsFormat;
+  }
+
+  @Override
+  public final PointsFormat pointsFormat() {
+    return new Lucene90PointsFormat();
+  }
+
+  @Override
+  public final KnnVectorsFormat knnVectorsFormat() {
+    return _knnVectorsFormat;
+  }
+
+  /**
+   * Returns the postings format that should be used for writing new segments of <code>field</code>.
+   *
+   * <p>The default implementation always returns "Lucene90".
+   *
+   * <p><b>WARNING:</b> if you subclass, you are responsible for index backwards compatibility:
+   * future version of Lucene are only guaranteed to be able to read the default implementation,
+   */
+  public PostingsFormat getPostingsFormatForField(String field) {
+    return _defaultPostingsFormat;
+  }
+
+  /**
+   * Returns the docvalues format that should be used for writing new segments of <code>field</code>
+   * .
+   *
+   * <p>The default implementation always returns "Lucene90".
+   *
+   * <p><b>WARNING:</b> if you subclass, you are responsible for index backwards compatibility:
+   * future version of Lucene are only guaranteed to be able to read the default implementation.
+   */
+  public DocValuesFormat getDocValuesFormatForField(String field) {
+    return _defaultDVFormat;
+  }
+
+  /**
+   * Returns the vectors format that should be used for writing new segments of <code>field</code>
+   *
+   * <p>The default implementation always returns "Lucene95".
+   *
+   * <p><b>WARNING:</b> if you subclass, you are responsible for index backwards compatibility:
+   * future version of Lucene are only guaranteed to be able to read the default implementation.
+   */
+  public KnnVectorsFormat getKnnVectorsFormatForField(String field) {
+    return _defaultKnnVectorsFormat;
+  }
+
+  @Override
+  public final DocValuesFormat docValuesFormat() {
+    return _docValuesFormat;
+  }
+}
diff --git a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/creator/impl/vector/lucene95/HnswVectorsFormat.java b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/creator/impl/vector/lucene95/HnswVectorsFormat.java
new file mode 100644
index 0000000000..b3f356c51e
--- /dev/null
+++ b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/creator/impl/vector/lucene95/HnswVectorsFormat.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.pinot.segment.local.segment.creator.impl.vector.lucene95;
+
+import java.io.IOException;
+import org.apache.lucene.codecs.KnnVectorsFormat;
+import org.apache.lucene.codecs.KnnVectorsReader;
+import org.apache.lucene.codecs.KnnVectorsWriter;
+import org.apache.lucene.codecs.lucene95.Lucene95HnswVectorsFormat;
+import org.apache.lucene.index.SegmentReadState;
+import org.apache.lucene.index.SegmentWriteState;
+import org.apache.lucene.util.hnsw.HnswGraph;
+
+
+/**
+ * Extend Lucene 9.5 vector format to support HNSW graph
+ * The major change here is to allow higher dimension vectors
+ */
+public final class HnswVectorsFormat extends KnnVectorsFormat {
+  /**
+   * The maximum number of dimensions supported by this format. This is a limitation of the
+   * underlying implementation of {@link HnswGraph} which uses a fixed size array to store the
+   * vector values.
+   */
+  public static final int DEFAULT_MAX_DIMENSIONS = 2048;
+
+  private final int _maxDimensions;
+  private final Lucene95HnswVectorsFormat _delegate;
+
+  /**
+   * Constructs a format using the given graph construction parameters.
+   *
+   * @param maxConn       the maximum number of connections to a node in the HNSW graph
+   * @param beamWidth     the size of the queue maintained during graph construction.
+   * @param maxDimensions the maximum number of dimensions supported by this format
+   */
+  public HnswVectorsFormat(int maxConn, int beamWidth, int maxDimensions) {
+    super("Lucene95HnswVectorsFormat");
+    if (maxDimensions <= 0 || maxDimensions > DEFAULT_MAX_DIMENSIONS) {
+      throw new IllegalArgumentException(
+          "maxDimensions must be postive and less than or equal to"
+              + DEFAULT_MAX_DIMENSIONS
+              + "; maxDimensions="
+              + maxDimensions);
+    }
+    _delegate = new Lucene95HnswVectorsFormat(maxConn, beamWidth);
+    _maxDimensions = maxDimensions;
+  }
+
+  @Override
+  public KnnVectorsWriter fieldsWriter(SegmentWriteState state)
+      throws IOException {
+    return _delegate.fieldsWriter(state);
+  }
+
+  @Override
+  public KnnVectorsReader fieldsReader(SegmentReadState state)
+      throws IOException {
+    return _delegate.fieldsReader(state);
+  }
+
+  @Override
+  public int getMaxDimensions(String fieldName) {
+    return _maxDimensions;
+  }
+
+  @Override
+  public String toString() {
+    return "HnswVectorsFormat(name=HnswVectorsFormat, maxDimensions="
+        + _maxDimensions
+        + ", delegate="
+        + _delegate
+        + ")";
+  }
+}
diff --git a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/converter/SegmentV1V2ToV3FormatConverter.java b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/converter/SegmentV1V2ToV3FormatConverter.java
index a65b58fd32..3ee1b3912a 100644
--- a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/converter/SegmentV1V2ToV3FormatConverter.java
+++ b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/converter/SegmentV1V2ToV3FormatConverter.java
@@ -113,6 +113,9 @@ public class SegmentV1V2ToV3FormatConverter implements SegmentFormatConverter {
       if (file.isDirectory() && file.getName().endsWith(V1Constants.Indexes.LUCENE_V9_TEXT_INDEX_FILE_EXTENSION)) {
         FileUtils.deleteDirectory(file);
       }
+      if (file.isDirectory() && file.getName().endsWith(V1Constants.Indexes.VECTOR_HNSW_INDEX_FILE_EXTENSION)) {
+        FileUtils.deleteDirectory(file);
+      }
     }
   }
 
@@ -154,7 +157,7 @@ public class SegmentV1V2ToV3FormatConverter implements SegmentFormatConverter {
         for (String column : v2Metadata.getAllColumns()) {
           for (IndexType<?, ?, ?> indexType : sortedIndexTypes()) {
             // NOTE: Text index is copied separately
-            if (indexType != StandardIndexes.text()) {
+            if (indexType != StandardIndexes.text() && indexType != StandardIndexes.vector()) {
               copyIndexIfExists(v2DataReader, v3DataWriter, column, indexType);
             }
           }
@@ -163,6 +166,7 @@ public class SegmentV1V2ToV3FormatConverter implements SegmentFormatConverter {
       }
     }
     copyLuceneTextIndexIfExists(v2Directory, v3Directory);
+    copyVectorIndexIfExists(v2Directory, v3Directory);
     copyStarTreeV2(v2Directory, v3Directory);
   }
 
@@ -256,6 +260,27 @@ public class SegmentV1V2ToV3FormatConverter implements SegmentFormatConverter {
     }
   }
 
+  private void copyVectorIndexIfExists(File segmentDirectory, File v3Dir)
+      throws IOException {
+    // TODO: see if this can be done by reusing some existing methods
+    String suffix = V1Constants.Indexes.VECTOR_HNSW_INDEX_FILE_EXTENSION;
+    File[] vectorIndexFiles = segmentDirectory.listFiles(new FilenameFilter() {
+      @Override
+      public boolean accept(File dir, String name) {
+        return name.endsWith(suffix);
+      }
+    });
+    for (File vectorIndexFile : vectorIndexFiles) {
+      File[] indexFiles = vectorIndexFile.listFiles();
+      File v3VectorIndexDir = new File(v3Dir, vectorIndexFile.getName());
+      v3VectorIndexDir.mkdir();
+      for (File indexFile : indexFiles) {
+        File v3VectorIndexFile = new File(v3VectorIndexDir, indexFile.getName());
+        Files.copy(indexFile.toPath(), v3VectorIndexFile.toPath());
+      }
+    }
+  }
+
   private void deleteStaleConversionDirectories(File segmentDirectory) {
     final String prefix = segmentDirectory.getName() + V3_TEMP_DIR_SUFFIX;
     File[] files = segmentDirectory.listFiles(new FilenameFilter() {
diff --git a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/datasource/BaseDataSource.java b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/datasource/BaseDataSource.java
index dfd355ac3a..ce66bcce59 100644
--- a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/datasource/BaseDataSource.java
+++ b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/datasource/BaseDataSource.java
@@ -34,6 +34,7 @@ import org.apache.pinot.segment.spi.index.reader.JsonIndexReader;
 import org.apache.pinot.segment.spi.index.reader.NullValueVectorReader;
 import org.apache.pinot.segment.spi.index.reader.RangeIndexReader;
 import org.apache.pinot.segment.spi.index.reader.TextIndexReader;
+import org.apache.pinot.segment.spi.index.reader.VectorIndexReader;
 
 
 public abstract class BaseDataSource implements DataSource {
@@ -113,4 +114,10 @@ public abstract class BaseDataSource implements DataSource {
   public NullValueVectorReader getNullValueVector() {
     return getIndex(StandardIndexes.nullValueVector());
   }
+
+  @Nullable
+  @Override
+  public VectorIndexReader getVectorIndex() {
+    return getIndex(StandardIndexes.vector());
+  }
 }
diff --git a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/loader/IndexLoadingConfig.java b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/loader/IndexLoadingConfig.java
index f7f185cc65..b711a899c3 100644
--- a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/loader/IndexLoadingConfig.java
+++ b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/loader/IndexLoadingConfig.java
@@ -43,6 +43,7 @@ import org.apache.pinot.segment.spi.index.IndexConfigDeserializer;
 import org.apache.pinot.segment.spi.index.IndexType;
 import org.apache.pinot.segment.spi.index.RangeIndexConfig;
 import org.apache.pinot.segment.spi.index.creator.H3IndexConfig;
+import org.apache.pinot.segment.spi.index.creator.VectorIndexConfig;
 import org.apache.pinot.segment.spi.loader.SegmentDirectoryLoaderRegistry;
 import org.apache.pinot.spi.config.instance.InstanceDataManagerConfig;
 import org.apache.pinot.spi.config.table.BloomFilterConfig;
@@ -84,6 +85,7 @@ public class IndexLoadingConfig {
   private FSTType _fstIndexType = FSTType.LUCENE;
   private Map<String, JsonIndexConfig> _jsonIndexConfigs = new HashMap<>();
   private Map<String, H3IndexConfig> _h3IndexConfigs = new HashMap<>();
+  private Map<String, VectorIndexConfig> _vectorIndexConfigs = new HashMap<>();
   private Set<String> _noDictionaryColumns = new HashSet<>(); // TODO: replace this by _noDictionaryConfig.
   private final Map<String, String> _noDictionaryConfig = new HashMap<>();
   private final Set<String> _varLengthDictionaryColumns = new HashSet<>();
@@ -217,6 +219,7 @@ public class IndexLoadingConfig {
     extractTextIndexColumnsFromTableConfig(tableConfig);
     extractFSTIndexColumnsFromTableConfig(tableConfig);
     extractH3IndexConfigsFromTableConfig(tableConfig);
+    extractVectorIndexConfigsFromTableConfig(tableConfig);
     extractForwardIndexDisabledColumnsFromTableConfig(tableConfig);
 
     Map<String, String> noDictionaryConfig = indexingConfig.getNoDictionaryConfig();
@@ -401,6 +404,18 @@ public class IndexLoadingConfig {
     }
   }
 
+  private void extractVectorIndexConfigsFromTableConfig(TableConfig tableConfig) {
+    List<FieldConfig> fieldConfigList = tableConfig.getFieldConfigList();
+    if (fieldConfigList != null) {
+      for (FieldConfig fieldConfig : fieldConfigList) {
+        if (fieldConfig.getIndexType() == FieldConfig.IndexType.VECTOR) {
+          //noinspection ConstantConditions
+          _vectorIndexConfigs.put(fieldConfig.getName(), new VectorIndexConfig(fieldConfig.getProperties()));
+        }
+      }
+    }
+  }
+
   private void extractFromInstanceConfig(InstanceDataManagerConfig instanceDataManagerConfig) {
     if (instanceDataManagerConfig == null) {
       return;
@@ -531,6 +546,10 @@ public class IndexLoadingConfig {
     return unmodifiable(_h3IndexConfigs);
   }
 
+  public Map<String, VectorIndexConfig> getVectorIndexConfigs() {
+    return unmodifiable(_vectorIndexConfigs);
+  }
+
   public Map<String, Map<String, String>> getColumnProperties() {
     return unmodifiable(_columnProperties);
   }
@@ -703,6 +722,12 @@ public class IndexLoadingConfig {
     _dirty = true;
   }
 
+  @VisibleForTesting
+  public void setVectorIndexConfigs(Map<String, VectorIndexConfig> vectorIndexConfigs) {
+    _vectorIndexConfigs = new HashMap<>(vectorIndexConfigs);
+    _dirty = true;
+  }
+
   @VisibleForTesting
   public void setBloomFilterConfigs(Map<String, BloomFilterConfig> bloomFilterConfigs) {
     _bloomFilterConfigs = new HashMap<>(bloomFilterConfigs);
diff --git a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/loader/invertedindex/VectorIndexHandler.java b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/loader/invertedindex/VectorIndexHandler.java
new file mode 100644
index 0000000000..584d4be1c3
--- /dev/null
+++ b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/loader/invertedindex/VectorIndexHandler.java
@@ -0,0 +1,215 @@
+/**
+ * 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.pinot.segment.local.segment.index.loader.invertedindex;
+
+import com.google.common.base.Preconditions;
+import java.io.File;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+import javax.annotation.Nullable;
+import org.apache.commons.io.FileUtils;
+import org.apache.pinot.segment.local.segment.index.forward.ForwardIndexType;
+import org.apache.pinot.segment.local.segment.index.loader.BaseIndexHandler;
+import org.apache.pinot.segment.spi.ColumnMetadata;
+import org.apache.pinot.segment.spi.V1Constants;
+import org.apache.pinot.segment.spi.creator.IndexCreationContext;
+import org.apache.pinot.segment.spi.index.FieldIndexConfigs;
+import org.apache.pinot.segment.spi.index.FieldIndexConfigsUtil;
+import org.apache.pinot.segment.spi.index.StandardIndexes;
+import org.apache.pinot.segment.spi.index.creator.VectorIndexConfig;
+import org.apache.pinot.segment.spi.index.creator.VectorIndexCreator;
+import org.apache.pinot.segment.spi.index.reader.Dictionary;
+import org.apache.pinot.segment.spi.index.reader.ForwardIndexReader;
+import org.apache.pinot.segment.spi.index.reader.ForwardIndexReaderContext;
+import org.apache.pinot.segment.spi.store.SegmentDirectory;
+import org.apache.pinot.segment.spi.store.SegmentDirectoryPaths;
+import org.apache.pinot.spi.config.table.TableConfig;
+import org.apache.pinot.spi.data.FieldSpec;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+public class VectorIndexHandler extends BaseIndexHandler {
+  private static final Logger LOGGER = LoggerFactory.getLogger(VectorIndexHandler.class);
+
+  private final Map<String, VectorIndexConfig> _vectorConfigs;
+
+  public VectorIndexHandler(SegmentDirectory segmentDirectory, Map<String, FieldIndexConfigs> fieldIndexConfigs,
+      @Nullable TableConfig tableConfig) {
+    super(segmentDirectory, fieldIndexConfigs, tableConfig);
+    _vectorConfigs = FieldIndexConfigsUtil.enableConfigByColumn(StandardIndexes.vector(), _fieldIndexConfigs);
+  }
+
+  @Override
+  public boolean needUpdateIndices(SegmentDirectory.Reader segmentReader) {
+    String segmentName = _segmentDirectory.getSegmentMetadata().getName();
+    Set<String> columnsToAddIdx = new HashSet<>(_vectorConfigs.keySet());
+    Set<String> existingColumns = segmentReader.toSegmentDirectory().getColumnsWithIndex(StandardIndexes.vector());
+    // Check if any existing index need to be removed.
+    for (String column : existingColumns) {
+      if (!columnsToAddIdx.remove(column)) {
+        LOGGER.info("Need to remove existing Vector index from segment: {}, column: {}", segmentName, column);
+        return true;
+      }
+    }
+    // Check if any new index need to be added.
+    for (String column : columnsToAddIdx) {
+      ColumnMetadata columnMetadata = _segmentDirectory.getSegmentMetadata().getColumnMetadataFor(column);
+      if (shouldCreateVectorIndex(columnMetadata)) {
+        LOGGER.info("Need to create new Vector index for segment: {}, column: {}", segmentName, column);
+        return true;
+      }
+    }
+    return false;
+  }
+
+  @Override
+  public void updateIndices(SegmentDirectory.Writer segmentWriter)
+      throws Exception {
+    Set<String> columnsToAddIdx = new HashSet<>(_vectorConfigs.keySet());
+    // Remove indices not set in table config any more
+    String segmentName = _segmentDirectory.getSegmentMetadata().getName();
+    Set<String> existingColumns = segmentWriter.toSegmentDirectory().getColumnsWithIndex(StandardIndexes.vector());
+    for (String column : existingColumns) {
+      if (!columnsToAddIdx.remove(column)) {
+        LOGGER.info("Removing existing Vector index from segment: {}, column: {}", segmentName, column);
+        segmentWriter.removeIndex(column, StandardIndexes.vector());
+        LOGGER.info("Removed existing Vector index from segment: {}, column: {}", segmentName, column);
+      }
+    }
+    for (String column : columnsToAddIdx) {
+      ColumnMetadata columnMetadata = _segmentDirectory.getSegmentMetadata().getColumnMetadataFor(column);
+      if (shouldCreateVectorIndex(columnMetadata)) {
+        createVectorIndexForColumn(segmentWriter, columnMetadata);
+      }
+    }
+  }
+
+  private boolean shouldCreateVectorIndex(ColumnMetadata columnMetadata) {
+    return columnMetadata != null;
+  }
+
+  private void createVectorIndexForColumn(SegmentDirectory.Writer segmentWriter, ColumnMetadata columnMetadata)
+      throws Exception {
+    File indexDir = _segmentDirectory.getSegmentMetadata().getIndexDir();
+    String segmentName = _segmentDirectory.getSegmentMetadata().getName();
+    File segmentDirectory = SegmentDirectoryPaths.segmentDirectoryFor(indexDir,
+        _segmentDirectory.getSegmentMetadata().getVersion());
+
+    String columnName = columnMetadata.getColumnName();
+    File inProgress =
+        new File(segmentDirectory, columnName + V1Constants.Indexes.VECTOR_HNSW_INDEX_FILE_EXTENSION + ".inprogress");
+    File vectorIndexFile =
+        new File(segmentDirectory, columnName + V1Constants.Indexes.VECTOR_HNSW_INDEX_FILE_EXTENSION);
+
+    if (!inProgress.exists()) {
+      // Marker file does not exist, which means last run ended normally.
+      // Create a marker file.
+      FileUtils.touch(inProgress);
+    } else {
+      // Marker file exists, which means last run gets interrupted.
+      // Remove Vector index if exists.
+      // For v1 and v2, it's the actual Vector index. For v3, it's the temporary Vector index.
+      FileUtils.deleteQuietly(vectorIndexFile);
+    }
+
+    // Create a temporary forward index if it is disabled and does not exist
+    columnMetadata = createForwardIndexIfNeeded(segmentWriter, columnName, true);
+
+    // Create new Vector index for the column.
+    LOGGER.info("Creating new Vector index for segment: {}, column: {}", segmentName, columnName);
+    Preconditions.checkState(columnMetadata.getDataType() == FieldSpec.DataType.FLOAT,
+        "VECTOR index can only be applied to Float Array columns");
+    if (columnMetadata.hasDictionary()) {
+      handleDictionaryBasedColumn(segmentWriter, columnMetadata);
+    } else {
+      handleNonDictionaryBasedColumn(segmentWriter, columnMetadata);
+    }
+
+//    // For v3, write the generated Vector index file into the single file and remove it.
+//    if (_segmentDirectory.getSegmentMetadata().getVersion() == SegmentVersion.v3) {
+//      LoaderUtils.writeIndexToV3Format(segmentWriter, columnName, vectorIndexFile, StandardIndexes.vector());
+//    }
+
+    // Delete the marker file.
+    FileUtils.deleteQuietly(inProgress);
+
+    LOGGER.info("Created Vector index for segment: {}, column: {}", segmentName, columnName);
+  }
+
+  private void handleDictionaryBasedColumn(SegmentDirectory.Writer segmentWriter, ColumnMetadata columnMetadata)
+      throws Exception {
+    File indexDir = _segmentDirectory.getSegmentMetadata().getIndexDir();
+    String columnName = columnMetadata.getColumnName();
+    File segmentDirectory = SegmentDirectoryPaths.segmentDirectoryFor(indexDir,
+        _segmentDirectory.getSegmentMetadata().getVersion());
+
+    FieldIndexConfigs colIndexConf = _fieldIndexConfigs.get(columnName);
+
+    IndexCreationContext context = IndexCreationContext.builder()
+        .withIndexDir(segmentDirectory)
+        .withColumnMetadata(columnMetadata)
+        .build();
+    VectorIndexConfig config = colIndexConf.getConfig(StandardIndexes.vector());
+
+    try (ForwardIndexReader forwardIndexReader = StandardIndexes.forward().getReaderFactory()
+        .createIndexReader(segmentWriter, colIndexConf, columnMetadata);
+        ForwardIndexReaderContext readerContext = forwardIndexReader.createContext();
+        Dictionary dictionary = StandardIndexes.dictionary().getReaderFactory()
+            .createIndexReader(segmentWriter, colIndexConf, columnMetadata);
+        VectorIndexCreator vectorIndexCreator = StandardIndexes.vector().createIndexCreator(context, config)) {
+      int numDocs = columnMetadata.getTotalDocs();
+      float[] vector = new float[columnMetadata.getMaxNumberOfMultiValues()];
+      int[] dictIds = new int[columnMetadata.getMaxNumberOfMultiValues()];
+      for (int i = 0; i < numDocs; i++) {
+        forwardIndexReader.getDictIdMV(i, dictIds, readerContext);
+        for (int j = 0; j < dictIds.length; j++) {
+          vector[j] = dictionary.getFloatValue(dictIds[j]);
+        }
+        vectorIndexCreator.add(vector);
+      }
+      vectorIndexCreator.seal();
+    }
+  }
+
+  private void handleNonDictionaryBasedColumn(SegmentDirectory.Writer segmentWriter, ColumnMetadata columnMetadata)
+      throws Exception {
+    File indexDir = _segmentDirectory.getSegmentMetadata().getIndexDir();
+    String columnName = columnMetadata.getColumnName();
+    File segmentDirectory = SegmentDirectoryPaths.segmentDirectoryFor(indexDir,
+        _segmentDirectory.getSegmentMetadata().getVersion());
+
+    IndexCreationContext context = IndexCreationContext.builder()
+        .withIndexDir(segmentDirectory)
+        .withColumnMetadata(columnMetadata)
+        .build();
+    VectorIndexConfig config = _fieldIndexConfigs.get(columnName).getConfig(StandardIndexes.vector());
+    try (ForwardIndexReader forwardIndexReader = ForwardIndexType.read(segmentWriter, columnMetadata);
+        ForwardIndexReaderContext readerContext = forwardIndexReader.createContext();
+        VectorIndexCreator vectorIndexCreator = StandardIndexes.vector().createIndexCreator(context, config)) {
+      int numDocs = columnMetadata.getTotalDocs();
+      float[] vector = new float[columnMetadata.getMaxNumberOfMultiValues()];
+      for (int i = 0; i < numDocs; i++) {
+        forwardIndexReader.getFloatMV(i, vector, readerContext);
+      }
+      vectorIndexCreator.seal();
+    }
+  }
+}
diff --git a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/readers/vector/HnswDocIdCollector.java b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/readers/vector/HnswDocIdCollector.java
new file mode 100644
index 0000000000..788e6f6dc8
--- /dev/null
+++ b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/readers/vector/HnswDocIdCollector.java
@@ -0,0 +1,78 @@
+/**
+ * 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.pinot.segment.local.segment.index.readers.vector;
+
+import java.io.IOException;
+import org.apache.lucene.index.LeafReaderContext;
+import org.apache.lucene.search.Collector;
+import org.apache.lucene.search.LeafCollector;
+import org.apache.lucene.search.Scorable;
+import org.apache.lucene.search.ScoreMode;
+import org.roaringbitmap.buffer.MutableRoaringBitmap;
+
+
+/**
+ * A simple collector created to bypass all the heap heavy process
+ * of collecting the results in Lucene. Lucene by default will
+ * create a {@link org.apache.lucene.search.TopScoreDocCollector}
+ * which internally uses a {@link org.apache.lucene.search.TopDocsCollector}
+ * and uses a PriorityQueue to maintain the top results. From the heap usage
+ * experiments (please see the design doc), we found out that this was
+ * substantially contributing to heap whereas we currently don't need any
+ * scoring or top doc collecting.
+ * Every time Lucene finds a matching document for the text search query,
+ * a callback is invoked into this collector that simply collects the
+ * matching doc's docID. We store the docID in a bitmap to be traversed later
+ * as part of doc id iteration etc.
+ */
+public class HnswDocIdCollector implements Collector {
+
+  private final MutableRoaringBitmap _docIds;
+  private final HnswVectorIndexReader.DocIdTranslator _docIdTranslator;
+
+  public HnswDocIdCollector(MutableRoaringBitmap docIds, HnswVectorIndexReader.DocIdTranslator docIdTranslator) {
+    _docIds = docIds;
+    _docIdTranslator = docIdTranslator;
+  }
+
+  @Override
+  public ScoreMode scoreMode() {
+    return ScoreMode.COMPLETE_NO_SCORES;
+  }
+
+  @Override
+  public LeafCollector getLeafCollector(LeafReaderContext context) {
+    return new LeafCollector() {
+
+      @Override
+      public void setScorer(Scorable scorer)
+          throws IOException {
+        // we don't use scoring, so this is NO-OP
+      }
+
+      @Override
+      public void collect(int doc)
+          throws IOException {
+        // Compute the absolute lucene docID across
+        // sub-indexes because that's how the lookup table in docIdTranslator is built
+        _docIds.add(_docIdTranslator.getPinotDocId(context.docBase + doc));
+      }
+    };
+  }
+}
diff --git a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/readers/vector/HnswVectorIndexReader.java b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/readers/vector/HnswVectorIndexReader.java
new file mode 100644
index 0000000000..079a4a5ce2
--- /dev/null
+++ b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/readers/vector/HnswVectorIndexReader.java
@@ -0,0 +1,194 @@
+/**
+ * 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.pinot.segment.local.segment.index.readers.vector;
+
+import java.io.Closeable;
+import java.io.File;
+import java.io.IOException;
+import java.nio.ByteOrder;
+import org.apache.lucene.document.Document;
+import org.apache.lucene.index.DirectoryReader;
+import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.queryparser.classic.QueryParser;
+import org.apache.lucene.search.Collector;
+import org.apache.lucene.search.IndexSearcher;
+import org.apache.lucene.search.KnnFloatVectorQuery;
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.store.FSDirectory;
+import org.apache.pinot.segment.local.indexsegment.immutable.ImmutableSegmentLoader;
+import org.apache.pinot.segment.local.segment.creator.impl.vector.HnswVectorIndexCreator;
+import org.apache.pinot.segment.local.segment.index.readers.text.LuceneTextIndexReader;
+import org.apache.pinot.segment.spi.V1Constants;
+import org.apache.pinot.segment.spi.index.creator.VectorIndexConfig;
+import org.apache.pinot.segment.spi.index.reader.VectorIndexReader;
+import org.apache.pinot.segment.spi.memory.PinotDataBuffer;
+import org.apache.pinot.segment.spi.store.SegmentDirectoryPaths;
+import org.roaringbitmap.buffer.MutableRoaringBitmap;
+import org.slf4j.LoggerFactory;
+
+
+public class HnswVectorIndexReader implements VectorIndexReader {
+
+  private static final org.slf4j.Logger LOGGER = LoggerFactory.getLogger(LuceneTextIndexReader.class);
+
+  private final IndexReader _indexReader;
+  private final Directory _indexDirectory;
+  private final IndexSearcher _indexSearcher;
+  private final String _column;
+  private final HnswVectorIndexReader.DocIdTranslator _docIdTranslator;
+  private boolean _useANDForMultiTermQueries = false;
+
+  public HnswVectorIndexReader(String column, File indexDir, int numDocs, VectorIndexConfig config) {
+    _column = column;
+    try {
+      File indexFile = getVectorIndexFile(indexDir);
+      _indexDirectory = FSDirectory.open(indexFile.toPath());
+      _indexReader = DirectoryReader.open(_indexDirectory);
+      _indexSearcher = new IndexSearcher(_indexReader);
+
+//      if (!config.isEnableQueryCache()) {
+//        // Disable Lucene query result cache. While it helps a lot with performance for
+//        // repeated queries, on the downside it cause heap issues.
+//        _indexSearcher.setQueryCache(null);
+//      }
+//      if (config.isUseANDForMultiTermQueries()) {
+//        _useANDForMultiTermQueries = true;
+//      }
+      // TODO: consider using a threshold of num docs per segment to decide between building
+      // mapping file upfront on segment load v/s on-the-fly during query processing
+      _docIdTranslator = new HnswVectorIndexReader.DocIdTranslator(indexDir, _column, numDocs, _indexSearcher);
+    } catch (Exception e) {
+      LOGGER.error("Failed to instantiate Lucene text index reader for column {}, exception {}", column,
+          e.getMessage());
+      throw new RuntimeException(e);
+    }
+  }
+
+  /**
+   * CASE 1: If IndexLoadingConfig specifies a segment version to load and if it is different then
+   * the on-disk version of the segment, then {@link ImmutableSegmentLoader}
+   * will take care of up-converting the on-disk segment to v3 before load. The converter
+   * already has support for converting v1 text index to v3. So the text index can be
+   * loaded from segmentIndexDir/v3/ since v3 sub-directory would have already been created
+   *
+   * CASE 2: However, if IndexLoadingConfig doesn't specify the segment version to load or if the specified
+   * version is same as the on-disk version of the segment, then ImmutableSegmentLoader will load
+   * whatever the version of segment is on disk.
+   * @param segmentIndexDir top-level segment index directory
+   * @return text index file
+   */
+  private File getVectorIndexFile(File segmentIndexDir) {
+    // will return null if file does not exist
+    File file = SegmentDirectoryPaths.findVectorIndexIndexFile(segmentIndexDir, _column);
+    if (file == null) {
+      throw new IllegalStateException("Failed to find text index file for column: " + _column);
+    }
+    return file;
+  }
+
+  @Override
+  public MutableRoaringBitmap getDocIds(float[] searchQuery, int topK) {
+    MutableRoaringBitmap docIds = new MutableRoaringBitmap();
+    Collector docIDCollector = new HnswDocIdCollector(docIds, _docIdTranslator);
+    try {
+      // Lucene Query Parser is JavaCC based. It is stateful and should
+      // be instantiated per query. Analyzer on the other hand is stateless
+      // and can be created upfront.
+      QueryParser parser = new QueryParser(_column, null);
+
+      if (_useANDForMultiTermQueries) {
+        parser.setDefaultOperator(QueryParser.Operator.AND);
+      }
+      KnnFloatVectorQuery knnFloatVectorQuery = new KnnFloatVectorQuery(_column, searchQuery, topK);
+      _indexSearcher.search(knnFloatVectorQuery, docIDCollector);
+      return docIds;
+    } catch (Exception e) {
+      String msg =
+          "Caught excepttion while searching the text index for column:" + _column + " search query:" + searchQuery;
+      throw new RuntimeException(msg, e);
+    }
+  }
+
+  /**
+   * When we destroy the loaded ImmutableSegment, all the indexes
+   * (for each column) are destroyed and as part of that
+   * we release the text index
+   * @throws IOException
+   */
+  @Override
+  public void close()
+      throws IOException {
+    _indexReader.close();
+    _indexDirectory.close();
+    _docIdTranslator.close();
+  }
+
+  /**
+   * Lucene docIDs are not same as pinot docIDs. The internal implementation
+   * of Lucene can change the docIds and they are not guaranteed to be the
+   * same as how we expect -- strictly increasing docIDs as the documents
+   * are ingested during segment/index creation.
+   * This class is used to map the luceneDocId (returned by the search query
+   * to the collector) to corresponding pinotDocId.
+   */
+  static class DocIdTranslator implements Closeable {
+    final PinotDataBuffer _buffer;
+
+    DocIdTranslator(File segmentIndexDir, String column, int numDocs, IndexSearcher indexSearcher)
+        throws Exception {
+      int length = Integer.BYTES * numDocs;
+      File docIdMappingFile = new File(SegmentDirectoryPaths.findSegmentDirectory(segmentIndexDir),
+          column + V1Constants.Indexes.VECTOR_HNSW_INDEX_DOCID_MAPPING_FILE_EXTENSION);
+      // The mapping is local to a segment. It is created on the server during segment load.
+      // Unless we are running Pinot on Solaris/SPARC, the underlying architecture is
+      // LITTLE_ENDIAN (Linux/x86). So use that as byte order.
+      String desc = "Text index docId mapping buffer: " + column;
+      if (docIdMappingFile.exists()) {
+        // we will be here for segment reload and server restart
+        // for refresh, we will not be here since segment is deleted/replaced
+        // TODO: see if we can prefetch the pages
+        _buffer =
+            PinotDataBuffer.mapFile(docIdMappingFile, /* readOnly */ true, 0, length, ByteOrder.LITTLE_ENDIAN, desc);
+      } else {
+        _buffer =
+            PinotDataBuffer.mapFile(docIdMappingFile, /* readOnly */ false, 0, length, ByteOrder.LITTLE_ENDIAN, desc);
+        for (int i = 0; i < numDocs; i++) {
+          try {
+            Document document = indexSearcher.doc(i);
+            int pinotDocId = Integer.parseInt(document.get(HnswVectorIndexCreator.VECTOR_INDEX_DOC_ID_COLUMN_NAME));
+            _buffer.putInt(i * Integer.BYTES, pinotDocId);
+          } catch (Exception e) {
+            throw new RuntimeException(
+                "Caught exception while building doc id mapping for text index column: " + column, e);
+          }
+        }
+      }
+    }
+
+    int getPinotDocId(int luceneDocId) {
+      return _buffer.getInt(luceneDocId * Integer.BYTES);
+    }
+
+    @Override
+    public void close()
+        throws IOException {
+      _buffer.close();
+    }
+  }
+}
diff --git a/pinot-common/src/main/java/org/apache/pinot/sql/FilterKind.java b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/vector/VectorIndexPlugin.java
similarity index 56%
copy from pinot-common/src/main/java/org/apache/pinot/sql/FilterKind.java
copy to pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/vector/VectorIndexPlugin.java
index 0c2a8dca75..9f6c47ffc2 100644
--- a/pinot-common/src/main/java/org/apache/pinot/sql/FilterKind.java
+++ b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/vector/VectorIndexPlugin.java
@@ -16,37 +16,19 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.pinot.sql;
+package org.apache.pinot.segment.local.segment.index.vector;
 
-public enum FilterKind {
-  AND,
-  OR,
-  NOT,
-  EQUALS,
-  NOT_EQUALS,
-  GREATER_THAN,
-  GREATER_THAN_OR_EQUAL,
-  LESS_THAN,
-  LESS_THAN_OR_EQUAL,
-  BETWEEN,
-  RANGE,
-  IN,
-  NOT_IN,
-  LIKE,
-  REGEXP_LIKE,
-  TEXT_CONTAINS,
-  TEXT_MATCH,
-  JSON_MATCH,
-  IS_NULL,
-  IS_NOT_NULL;
+import com.google.auto.service.AutoService;
+import org.apache.pinot.segment.spi.index.IndexPlugin;
 
-  /**
-   * Helper method that returns true if the enum maps to a Range.
-   *
-   * @return True if the enum is of Range type, false otherwise.
-   */
-  public boolean isRange() {
-    return this == GREATER_THAN || this == GREATER_THAN_OR_EQUAL || this == LESS_THAN || this == LESS_THAN_OR_EQUAL
-        || this == BETWEEN || this == RANGE;
+
+@AutoService(IndexPlugin.class)
+public class VectorIndexPlugin implements IndexPlugin<VectorIndexType> {
+
+  private static final VectorIndexType INSTANCE = new VectorIndexType();
+
+  @Override
+  public VectorIndexType getIndexType() {
+    return INSTANCE;
   }
 }
diff --git a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/vector/VectorIndexType.java b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/vector/VectorIndexType.java
new file mode 100644
index 0000000000..59faf4c880
--- /dev/null
+++ b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/vector/VectorIndexType.java
@@ -0,0 +1,174 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pinot.segment.local.segment.index.vector;
+
+import com.clearspring.analytics.util.Preconditions;
+import java.io.File;
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import javax.annotation.Nullable;
+import org.apache.pinot.segment.local.realtime.impl.vector.MutableVectorIndex;
+import org.apache.pinot.segment.local.segment.creator.impl.vector.HnswVectorIndexCreator;
+import org.apache.pinot.segment.local.segment.index.loader.ConfigurableFromIndexLoadingConfig;
+import org.apache.pinot.segment.local.segment.index.loader.IndexLoadingConfig;
+import org.apache.pinot.segment.local.segment.index.loader.invertedindex.VectorIndexHandler;
+import org.apache.pinot.segment.local.segment.index.readers.vector.HnswVectorIndexReader;
+import org.apache.pinot.segment.spi.ColumnMetadata;
+import org.apache.pinot.segment.spi.V1Constants;
+import org.apache.pinot.segment.spi.creator.IndexCreationContext;
+import org.apache.pinot.segment.spi.index.AbstractIndexType;
+import org.apache.pinot.segment.spi.index.ColumnConfigDeserializer;
+import org.apache.pinot.segment.spi.index.FieldIndexConfigs;
+import org.apache.pinot.segment.spi.index.IndexConfigDeserializer;
+import org.apache.pinot.segment.spi.index.IndexHandler;
+import org.apache.pinot.segment.spi.index.IndexReaderConstraintException;
+import org.apache.pinot.segment.spi.index.IndexReaderFactory;
+import org.apache.pinot.segment.spi.index.StandardIndexes;
+import org.apache.pinot.segment.spi.index.creator.VectorIndexConfig;
+import org.apache.pinot.segment.spi.index.creator.VectorIndexCreator;
+import org.apache.pinot.segment.spi.index.mutable.MutableIndex;
+import org.apache.pinot.segment.spi.index.mutable.provider.MutableIndexContext;
+import org.apache.pinot.segment.spi.index.reader.VectorIndexReader;
+import org.apache.pinot.segment.spi.store.SegmentDirectory;
+import org.apache.pinot.spi.config.table.TableConfig;
+import org.apache.pinot.spi.data.FieldSpec;
+import org.apache.pinot.spi.data.Schema;
+
+
+/**
+ * Index type for vector columns.
+ * Currently only supports for float array columns and the supported vector index type is: HNSW.
+ *
+ */
+public class VectorIndexType extends AbstractIndexType<VectorIndexConfig, VectorIndexReader, VectorIndexCreator>
+    implements ConfigurableFromIndexLoadingConfig<VectorIndexConfig> {
+  public static final String INDEX_DISPLAY_NAME = "vector";
+
+  protected VectorIndexType() {
+    super(StandardIndexes.VECTOR_ID);
+  }
+
+  @Override
+  public Class<VectorIndexConfig> getIndexConfigClass() {
+    return VectorIndexConfig.class;
+  }
+
+  @Override
+  public Map<String, VectorIndexConfig> fromIndexLoadingConfig(IndexLoadingConfig indexLoadingConfig) {
+    return indexLoadingConfig.getVectorIndexConfigs();
+  }
+
+  @Override
+  public VectorIndexConfig getDefaultConfig() {
+    return VectorIndexConfig.DISABLED;
+  }
+
+  @Override
+  public String getPrettyName() {
+    return INDEX_DISPLAY_NAME;
+  }
+
+  @Override
+  public ColumnConfigDeserializer<VectorIndexConfig> createDeserializer() {
+
+    // reads tableConfig.indexingConfig.jsonIndexColumns
+    ColumnConfigDeserializer<VectorIndexConfig> fromVectorIndexCols =
+        IndexConfigDeserializer.fromCollection(
+            tableConfig -> tableConfig.getIndexingConfig().getVectorIndexColumns(),
+            (accum, column) -> accum.put(column, new VectorIndexConfig(new HashMap<>())));
+
+    return IndexConfigDeserializer.fromIndexes(getPrettyName(), getIndexConfigClass())
+        .withExclusiveAlternative(
+            IndexConfigDeserializer.ifIndexingConfig(fromVectorIndexCols));
+  }
+
+  @Override
+  public VectorIndexCreator createIndexCreator(IndexCreationContext context, VectorIndexConfig indexConfig)
+      throws IOException {
+    Preconditions.checkState(context.getFieldSpec().getDataType() == FieldSpec.DataType.FLOAT
+            && !context.getFieldSpec().isSingleValueField(),
+        "Vector index is currently only supported on float array columns");
+
+    switch (IndexType.valueOf(indexConfig.getVectorIndexType())) {
+      case HNSW:
+        return new HnswVectorIndexCreator(context.getFieldSpec().getName(), context.getIndexDir(), indexConfig);
+      // TODO: Support more vector index types.
+      default:
+        throw new UnsupportedOperationException("Unsupported vector index type: " + indexConfig.getVectorIndexType());
+    }
+  }
+
+  @Override
+  protected IndexReaderFactory<VectorIndexReader> createReaderFactory() {
+    return ReaderFactory.INSTANCE;
+  }
+
+  @Override
+  public IndexHandler createIndexHandler(SegmentDirectory segmentDirectory, Map<String, FieldIndexConfigs> configsByCol,
+      @Nullable Schema schema, @Nullable TableConfig tableConfig) {
+    return new VectorIndexHandler(segmentDirectory, configsByCol, tableConfig);
+  }
+
+  @Override
+  public List<String> getFileExtensions(@Nullable ColumnMetadata columnMetadata) {
+    return List.of(V1Constants.Indexes.VECTOR_INDEX_FILE_EXTENSION);
+  }
+
+  private static class ReaderFactory implements IndexReaderFactory<VectorIndexReader> {
+
+    public static final VectorIndexType.ReaderFactory INSTANCE = new VectorIndexType.ReaderFactory();
+
+    private ReaderFactory() {
+    }
+
+    @Override
+    public VectorIndexReader createIndexReader(SegmentDirectory.Reader segmentReader,
+        FieldIndexConfigs fieldIndexConfigs, ColumnMetadata metadata)
+        throws IOException, IndexReaderConstraintException {
+      if (metadata.getDataType() != FieldSpec.DataType.FLOAT || metadata.getFieldSpec().isSingleValueField()) {
+        throw new IndexReaderConstraintException(metadata.getColumnName(), StandardIndexes.vector(),
+            "HNSW Vector index is currently only supported on float array type columns");
+      }
+      File segmentDir = segmentReader.toSegmentDirectory().getPath().toFile();
+
+      VectorIndexConfig indexConfig = fieldIndexConfigs.getConfig(StandardIndexes.vector());
+      return new HnswVectorIndexReader(metadata.getColumnName(), segmentDir, metadata.getTotalDocs(), indexConfig);
+    }
+  }
+
+  @Nullable
+  @Override
+  public MutableIndex createMutableIndex(MutableIndexContext context, VectorIndexConfig config) {
+    if (config.isDisabled()) {
+      return null;
+    }
+    if (!context.getFieldSpec().getDataType().equals(FieldSpec.DataType.FLOAT) || context.getFieldSpec()
+        .isSingleValueField()) {
+      return null;
+    }
+
+    return new MutableVectorIndex(context.getSegmentName(), context.getFieldSpec().getName(), config);
+  }
+
+  public enum IndexType {
+    HNSW
+  }
+}
diff --git a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/store/FilePerIndexDirectory.java b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/store/FilePerIndexDirectory.java
index 89fe759c55..21fa74950a 100644
--- a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/store/FilePerIndexDirectory.java
+++ b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/store/FilePerIndexDirectory.java
@@ -102,6 +102,8 @@ class FilePerIndexDirectory extends ColumnIndexDirectory {
     _indexBuffers.remove(new IndexKey(columnName, indexType));
     if (indexType == StandardIndexes.text()) {
       TextIndexUtils.cleanupTextIndex(_segmentDirectory, columnName);
+    } else if (indexType == StandardIndexes.vector()) {
+      VectorIndexUtils.cleanupVectorIndex(_segmentDirectory, columnName);
     } else {
       getFilesFor(columnName, indexType).forEach(FileUtils::deleteQuietly);
     }
diff --git a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/store/SingleFileIndexDirectory.java b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/store/SingleFileIndexDirectory.java
index a8af853be3..3d79e00de8 100644
--- a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/store/SingleFileIndexDirectory.java
+++ b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/store/SingleFileIndexDirectory.java
@@ -143,6 +143,9 @@ class SingleFileIndexDirectory extends ColumnIndexDirectory {
     if (type == StandardIndexes.text()) {
       return TextIndexUtils.hasTextIndex(_segmentDirectory, column);
     }
+    if (type == StandardIndexes.vector()) {
+      return VectorIndexUtils.hasVectorIndex(_segmentDirectory, column);
+    }
     IndexKey key = new IndexKey(column, type);
     return _columnEntries.containsKey(key);
   }
@@ -363,6 +366,10 @@ class SingleFileIndexDirectory extends ColumnIndexDirectory {
       TextIndexUtils.cleanupTextIndex(_segmentDirectory, columnName);
       return;
     }
+    if (indexType == StandardIndexes.vector()) {
+      VectorIndexUtils.cleanupVectorIndex(_segmentDirectory, columnName);
+      return;
+    }
     // Only remember to cleanup indices upon close(), if any existing
     // index gets marked for removal.
     if (_columnEntries.remove(new IndexKey(columnName, indexType)) != null) {
diff --git a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/store/VectorIndexUtils.java b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/store/VectorIndexUtils.java
new file mode 100644
index 0000000000..15de36cf5f
--- /dev/null
+++ b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/store/VectorIndexUtils.java
@@ -0,0 +1,98 @@
+/**
+ * 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.pinot.segment.local.segment.store;
+
+import java.io.File;
+import org.apache.commons.io.FileUtils;
+import org.apache.lucene.codecs.lucene95.Lucene95Codec;
+import org.apache.lucene.codecs.lucene95.Lucene95HnswVectorsFormat;
+import org.apache.lucene.index.IndexWriterConfig;
+import org.apache.lucene.index.VectorSimilarityFunction;
+import org.apache.pinot.segment.local.segment.creator.impl.vector.lucene95.HnswCodec;
+import org.apache.pinot.segment.local.segment.creator.impl.vector.lucene95.HnswVectorsFormat;
+import org.apache.pinot.segment.spi.V1Constants.Indexes;
+import org.apache.pinot.segment.spi.index.creator.VectorIndexConfig;
+
+
+public class VectorIndexUtils {
+  private VectorIndexUtils() {
+  }
+
+  static void cleanupVectorIndex(File segDir, String column) {
+    // Remove the lucene index file and potentially the docId mapping file.
+    File luceneIndexFile = new File(segDir, column + Indexes.VECTOR_HNSW_INDEX_FILE_EXTENSION);
+    FileUtils.deleteQuietly(luceneIndexFile);
+    File luceneMappingFile = new File(segDir, column + Indexes.VECTOR_HNSW_INDEX_DOCID_MAPPING_FILE_EXTENSION);
+    FileUtils.deleteQuietly(luceneMappingFile);
+
+    // Remove the native index file
+    File nativeIndexFile = new File(segDir, column + Indexes.VECTOR_INDEX_FILE_EXTENSION);
+    FileUtils.deleteQuietly(nativeIndexFile);
+  }
+
+  static boolean hasVectorIndex(File segDir, String column) {
+    return new File(segDir, column + Indexes.VECTOR_HNSW_INDEX_FILE_EXTENSION).exists() || new File(segDir,
+        column + Indexes.VECTOR_INDEX_FILE_EXTENSION).exists();
+  }
+
+  public static VectorSimilarityFunction toSimilarityFunction(
+      VectorIndexConfig.VectorDistanceFunction distanceFunction) {
+    switch (distanceFunction) {
+      case COSINE:
+        return VectorSimilarityFunction.COSINE;
+      case INNER_PRODUCT:
+        return VectorSimilarityFunction.MAXIMUM_INNER_PRODUCT;
+      case EUCLIDEAN:
+        return VectorSimilarityFunction.EUCLIDEAN;
+      case DOT_PRODUCT:
+        return VectorSimilarityFunction.DOT_PRODUCT;
+      default:
+        throw new IllegalArgumentException("Unknown distance function: " + distanceFunction);
+    }
+  }
+
+  public static IndexWriterConfig getIndexWriterConfig(VectorIndexConfig vectorIndexConfig) {
+    IndexWriterConfig indexWriterConfig = new IndexWriterConfig();
+
+    double maxBufferSizeMB = Double.parseDouble(vectorIndexConfig.getProperties()
+        .getOrDefault("maxBufferSizeMB", String.valueOf(IndexWriterConfig.DEFAULT_RAM_BUFFER_SIZE_MB)));
+    boolean commit = Boolean.parseBoolean(vectorIndexConfig.getProperties()
+        .getOrDefault("commit", String.valueOf(IndexWriterConfig.DEFAULT_COMMIT_ON_CLOSE)));
+    boolean useCompoundFile = Boolean.parseBoolean(vectorIndexConfig.getProperties()
+        .getOrDefault("useCompoundFile", String.valueOf(IndexWriterConfig.DEFAULT_USE_COMPOUND_FILE_SYSTEM)));
+    indexWriterConfig.setRAMBufferSizeMB(maxBufferSizeMB);
+    indexWriterConfig.setCommitOnClose(commit);
+    indexWriterConfig.setUseCompoundFile(useCompoundFile);
+
+    int maxCon = Integer.parseInt(vectorIndexConfig.getProperties()
+        .getOrDefault("maxCon", String.valueOf(Lucene95HnswVectorsFormat.DEFAULT_MAX_CONN)));
+    int beamWidth = Integer.parseInt(vectorIndexConfig.getProperties()
+        .getOrDefault("beamWidth", String.valueOf(Lucene95HnswVectorsFormat.DEFAULT_BEAM_WIDTH)));
+    int maxDimensions = Integer.parseInt(vectorIndexConfig.getProperties()
+        .getOrDefault("maxDimensions", String.valueOf(HnswVectorsFormat.DEFAULT_MAX_DIMENSIONS)));
+
+    HnswVectorsFormat knnVectorsFormat =
+        new HnswVectorsFormat(maxCon, beamWidth, maxDimensions);
+
+    Lucene95Codec.Mode mode = Lucene95Codec.Mode.valueOf(vectorIndexConfig.getProperties()
+        .getOrDefault("mode", Lucene95Codec.Mode.BEST_SPEED.name()));
+    indexWriterConfig.setCodec(new HnswCodec(mode, knnVectorsFormat));
+    return indexWriterConfig;
+  }
+}
diff --git a/pinot-segment-local/src/test/java/org/apache/pinot/segment/local/segment/index/loader/LoaderTest.java b/pinot-segment-local/src/test/java/org/apache/pinot/segment/local/segment/index/loader/LoaderTest.java
index 6c73f20e17..87a1e5db25 100644
--- a/pinot-segment-local/src/test/java/org/apache/pinot/segment/local/segment/index/loader/LoaderTest.java
+++ b/pinot-segment-local/src/test/java/org/apache/pinot/segment/local/segment/index/loader/LoaderTest.java
@@ -18,6 +18,7 @@
  */
 package org.apache.pinot.segment.local.segment.index.loader;
 
+import com.google.common.collect.ImmutableMap;
 import java.io.File;
 import java.net.URL;
 import java.util.Arrays;
@@ -42,6 +43,7 @@ import org.apache.pinot.segment.spi.index.ForwardIndexConfig;
 import org.apache.pinot.segment.spi.index.FstIndexConfig;
 import org.apache.pinot.segment.spi.index.StandardIndexes;
 import org.apache.pinot.segment.spi.index.TextIndexConfig;
+import org.apache.pinot.segment.spi.index.creator.VectorIndexConfig;
 import org.apache.pinot.segment.spi.index.metadata.SegmentMetadataImpl;
 import org.apache.pinot.segment.spi.loader.SegmentDirectoryLoader;
 import org.apache.pinot.segment.spi.loader.SegmentDirectoryLoaderContext;
@@ -71,12 +73,18 @@ import static org.testng.Assert.assertTrue;
 public class LoaderTest {
   private static final File INDEX_DIR = new File(LoaderTest.class.getName());
   private static final String AVRO_DATA = "data/test_data-mv.avro";
+  private static final String VECTOR_AVRO_DATA = "data/test_vector_data.avro";
 
   private static final String TEXT_INDEX_COL_NAME = "column5";
   private static final String FST_INDEX_COL_NAME = "column5";
   private static final String NO_FORWARD_INDEX_COL_NAME = "column4";
 
+  private static final String VECTOR_INDEX_COL_NAME = "vector1";
+  private static final int VECTOR_DIM_SIZE = 512;
+
   private File _avroFile;
+
+  private File _vectorAvroFile;
   private File _indexDir;
   private IndexLoadingConfig _v1IndexLoadingConfig;
   private IndexLoadingConfig _v3IndexLoadingConfig;
@@ -91,6 +99,7 @@ public class LoaderTest {
     URL resourceUrl = getClass().getClassLoader().getResource(AVRO_DATA);
     Assert.assertNotNull(resourceUrl);
     _avroFile = new File(resourceUrl.getFile());
+    _vectorAvroFile = new File(getClass().getClassLoader().getResource(VECTOR_AVRO_DATA).getFile());
     Map<String, Object> props = new HashMap<>();
     props.put(IndexLoadingConfig.READ_MODE_KEY, ReadMode.heap.toString());
     _pinotConfiguration = new PinotConfiguration(props);
@@ -98,10 +107,12 @@ public class LoaderTest {
     _v1IndexLoadingConfig = new IndexLoadingConfig();
     _v1IndexLoadingConfig.setReadMode(ReadMode.mmap);
     _v1IndexLoadingConfig.setSegmentVersion(SegmentVersion.v1);
+    _v1IndexLoadingConfig.setVectorIndexConfigs(new HashMap<>());
 
     _v3IndexLoadingConfig = new IndexLoadingConfig();
     _v3IndexLoadingConfig.setReadMode(ReadMode.mmap);
     _v3IndexLoadingConfig.setSegmentVersion(SegmentVersion.v3);
+    _v3IndexLoadingConfig.setVectorIndexConfigs(new HashMap<>());
 
     _localSegmentDirectoryLoader = SegmentDirectoryLoaderRegistry.getDefaultSegmentDirectoryLoader();
   }
@@ -744,6 +755,181 @@ public class LoaderTest {
     indexSegment.destroy();
   }
 
+  @Test
+  public void testVectorIndexLoad()
+      throws Exception {
+    // Tests for scenarios by creating on-disk segment in V3 and then loading
+    // the segment with and without specifying segmentVersion in IndexLoadingConfig
+
+    // create on-disk segment in V3
+    // this generates the segment in V1 but converts to V3 as part of post-creation processing
+    constructSegmentWithVectorIndex(SegmentVersion.v3);
+
+    // check that segment on-disk version is V3 after creation
+    Assert.assertEquals(new SegmentMetadataImpl(_indexDir).getVersion(), SegmentVersion.v3);
+    // check that V3 index sub-dir exists
+    Assert.assertTrue(SegmentDirectoryPaths.segmentDirectoryFor(_indexDir, SegmentVersion.v3).exists());
+    // check that index dir is not in V1 format (the only subdir it should have is V3)
+    verifyIndexDirIsV3(_indexDir);
+    // check that vector index exists under V3 subdir.
+    File vectorIndexFile = SegmentDirectoryPaths.findVectorIndexIndexFile(_indexDir, VECTOR_INDEX_COL_NAME);
+    Assert.assertNotNull(vectorIndexFile);
+    Assert.assertTrue(vectorIndexFile.isDirectory());
+    Assert.assertEquals(vectorIndexFile.getName(),
+        VECTOR_INDEX_COL_NAME + V1Constants.Indexes.VECTOR_HNSW_INDEX_FILE_EXTENSION);
+    Assert.assertEquals(vectorIndexFile.getParentFile().getName(), SegmentDirectoryPaths.V3_SUBDIRECTORY_NAME);
+
+    // CASE 1: don't set the segment version to load in IndexLoadingConfig
+    // there should be no conversion done by ImmutableSegmentLoader and it should
+    // be able to create vector index reader with on-disk version V3
+    IndexLoadingConfig indexLoadingConfig = new IndexLoadingConfig();
+    indexLoadingConfig.setVectorIndexConfigs(
+        ImmutableMap.of(VECTOR_INDEX_COL_NAME, new VectorIndexConfig(ImmutableMap.of(
+            "vectorDimension", String.valueOf(VECTOR_DIM_SIZE),
+            "vectorIndexType", "HNSW"
+        ))));
+    indexLoadingConfig.setReadMode(ReadMode.mmap);
+    IndexSegment indexSegment = ImmutableSegmentLoader.load(_indexDir, indexLoadingConfig);
+    // check that loaded segment version is v3
+    Assert.assertEquals(indexSegment.getSegmentMetadata().getVersion(), SegmentVersion.v3);
+    // no change/conversion should have happened in indexDir
+    Assert.assertTrue(SegmentDirectoryPaths.segmentDirectoryFor(_indexDir, SegmentVersion.v3).exists());
+    // check that index dir is not in V1 format (the only subdir it should have is V3)
+    verifyIndexDirIsV3(_indexDir);
+    // no change/conversion should have happened for vectorIndex dir
+    vectorIndexFile = SegmentDirectoryPaths.findVectorIndexIndexFile(_indexDir, VECTOR_INDEX_COL_NAME);
+    Assert.assertNotNull(vectorIndexFile);
+    Assert.assertTrue(vectorIndexFile.isDirectory());
+    Assert.assertEquals(vectorIndexFile.getName(),
+        VECTOR_INDEX_COL_NAME + V1Constants.Indexes.VECTOR_HNSW_INDEX_FILE_EXTENSION);
+    Assert.assertEquals(vectorIndexFile.getParentFile().getName(), SegmentDirectoryPaths.V3_SUBDIRECTORY_NAME);
+    indexSegment.destroy();
+
+    // CASE 2: set the segment version to load in IndexLoadingConfig as V3
+    // there should be no conversion done by ImmutableSegmentLoader since the segmentVersionToLoad
+    // is same as the version of segment on disk (V3)
+    indexLoadingConfig.setSegmentVersion(SegmentVersion.v3);
+    indexSegment = ImmutableSegmentLoader.load(_indexDir, indexLoadingConfig);
+    // check that loaded segment version is v3
+    Assert.assertEquals(indexSegment.getSegmentMetadata().getVersion(), SegmentVersion.v3);
+    // no change/conversion should have happened in indexDir
+    Assert.assertTrue(SegmentDirectoryPaths.segmentDirectoryFor(_indexDir, SegmentVersion.v3).exists());
+    // check that index dir is not in V1 format (the only subdir it should have is V3)
+    verifyIndexDirIsV3(_indexDir);
+    // no change/conversion should have happened for vectorIndex dir
+    vectorIndexFile = SegmentDirectoryPaths.findVectorIndexIndexFile(_indexDir, VECTOR_INDEX_COL_NAME);
+    Assert.assertNotNull(vectorIndexFile);
+    Assert.assertTrue(vectorIndexFile.isDirectory());
+    Assert.assertEquals(vectorIndexFile.getName(),
+        VECTOR_INDEX_COL_NAME + V1Constants.Indexes.VECTOR_HNSW_INDEX_FILE_EXTENSION);
+    Assert.assertEquals(vectorIndexFile.getParentFile().getName(), SegmentDirectoryPaths.V3_SUBDIRECTORY_NAME);
+    indexSegment.destroy();
+
+    // Test for scenarios by creating on-disk segment in V1 and then loading
+    // the segment with and without specifying segmentVersion in IndexLoadingConfig
+
+    // create on-disk segment in V1
+    // this generates the segment in V1 and does not convert to V3 as part of post-creation processing
+    constructSegmentWithVectorIndex(SegmentVersion.v1);
+
+    // check that segment on-disk version is V1 after creation
+    Assert.assertEquals(new SegmentMetadataImpl(_indexDir).getVersion(), SegmentVersion.v1);
+    // check that segment v1 dir exists
+    Assert.assertTrue(SegmentDirectoryPaths.segmentDirectoryFor(_indexDir, SegmentVersion.v1).exists());
+    // check that v3 index sub-dir does not exist
+    Assert.assertFalse(SegmentDirectoryPaths.segmentDirectoryFor(_indexDir, SegmentVersion.v3).exists());
+    // check that vector index exists directly under indexDir (V1). it should exist and should be a subdir
+    vectorIndexFile = SegmentDirectoryPaths.findVectorIndexIndexFile(_indexDir, VECTOR_INDEX_COL_NAME);
+    Assert.assertNotNull(vectorIndexFile);
+    Assert.assertTrue(vectorIndexFile.isDirectory());
+    Assert.assertEquals(vectorIndexFile.getName(),
+        VECTOR_INDEX_COL_NAME + V1Constants.Indexes.VECTOR_HNSW_INDEX_FILE_EXTENSION);
+    Assert.assertEquals(vectorIndexFile.getParentFile().getName(), new SegmentMetadataImpl(_indexDir).getName());
+
+    // CASE 1: don't set the segment version to load in IndexLoadingConfig
+    // there should be no conversion done by ImmutableSegmentLoader and it should
+    // be able to create vector index reader with on-disk version V1
+    indexLoadingConfig = new IndexLoadingConfig();
+    indexLoadingConfig.setVectorIndexConfigs(
+        ImmutableMap.of(VECTOR_INDEX_COL_NAME, new VectorIndexConfig(ImmutableMap.of(
+            "vectorDimension", String.valueOf(VECTOR_DIM_SIZE),
+            "vectorIndexType", "HNSW"
+        ))));
+    indexLoadingConfig.setReadMode(ReadMode.mmap);
+    indexSegment = ImmutableSegmentLoader.load(_indexDir, indexLoadingConfig);
+    // check that loaded segment version is v1
+    Assert.assertEquals(indexSegment.getSegmentMetadata().getVersion(), SegmentVersion.v1);
+    // no change/conversion should have happened in indexDir
+    Assert.assertTrue(SegmentDirectoryPaths.segmentDirectoryFor(_indexDir, SegmentVersion.v1).exists());
+    Assert.assertFalse(SegmentDirectoryPaths.segmentDirectoryFor(_indexDir, SegmentVersion.v3).exists());
+    // no change/conversion should have happened in vector index Dir
+    vectorIndexFile = SegmentDirectoryPaths.findVectorIndexIndexFile(_indexDir, VECTOR_INDEX_COL_NAME);
+    Assert.assertNotNull(vectorIndexFile);
+    Assert.assertTrue(vectorIndexFile.isDirectory());
+    Assert.assertEquals(vectorIndexFile.getName(),
+        VECTOR_INDEX_COL_NAME + V1Constants.Indexes.VECTOR_HNSW_INDEX_FILE_EXTENSION);
+    Assert.assertEquals(vectorIndexFile.getParentFile().getName(), new SegmentMetadataImpl(_indexDir).getName());
+    indexSegment.destroy();
+
+    // CASE 2: set the segment version to load in IndexLoadingConfig to V1
+    // there should be no conversion done by ImmutableSegmentLoader since the segmentVersionToLoad
+    // is same as the version of segment on fisk
+    indexLoadingConfig.setSegmentVersion(SegmentVersion.v1);
+    indexSegment = ImmutableSegmentLoader.load(_indexDir, indexLoadingConfig);
+    // check that loaded segment version is v1
+    Assert.assertEquals(indexSegment.getSegmentMetadata().getVersion(), SegmentVersion.v1);
+    // no change/conversion should have happened in indexDir
+    Assert.assertTrue(SegmentDirectoryPaths.segmentDirectoryFor(_indexDir, SegmentVersion.v1).exists());
+    Assert.assertFalse(SegmentDirectoryPaths.segmentDirectoryFor(_indexDir, SegmentVersion.v3).exists());
+    // no change/conversion should have happened in vector index Dir
+    vectorIndexFile = SegmentDirectoryPaths.findVectorIndexIndexFile(_indexDir, VECTOR_INDEX_COL_NAME);
+    Assert.assertNotNull(vectorIndexFile);
+    Assert.assertTrue(vectorIndexFile.isDirectory());
+    Assert.assertEquals(vectorIndexFile.getName(),
+        VECTOR_INDEX_COL_NAME + V1Constants.Indexes.VECTOR_HNSW_INDEX_FILE_EXTENSION);
+    Assert.assertEquals(vectorIndexFile.getParentFile().getName(), new SegmentMetadataImpl(_indexDir).getName());
+    indexSegment.destroy();
+
+    // CASE 3: set the segment version to load in IndexLoadingConfig to V3
+    // there should be conversion done by ImmutableSegmentLoader since the segmentVersionToLoad
+    // is different than the version of segment on disk
+    indexLoadingConfig.setSegmentVersion(SegmentVersion.v3);
+    indexSegment = ImmutableSegmentLoader.load(_indexDir, indexLoadingConfig);
+    // check that loaded segment version is v3
+    Assert.assertEquals(indexSegment.getSegmentMetadata().getVersion(), SegmentVersion.v3);
+    // the index dir should exist in v3 format due to conversion
+    Assert.assertTrue(SegmentDirectoryPaths.segmentDirectoryFor(_indexDir, SegmentVersion.v3).exists());
+    verifyIndexDirIsV3(_indexDir);
+    // check that vector index exists under V3 subdir. It should exist and should be a subdir
+    vectorIndexFile = SegmentDirectoryPaths.findVectorIndexIndexFile(_indexDir, VECTOR_INDEX_COL_NAME);
+    Assert.assertNotNull(vectorIndexFile);
+    Assert.assertTrue(vectorIndexFile.isDirectory());
+    Assert.assertEquals(vectorIndexFile.getName(),
+        VECTOR_INDEX_COL_NAME + V1Constants.Indexes.VECTOR_HNSW_INDEX_FILE_EXTENSION);
+    Assert.assertEquals(vectorIndexFile.getParentFile().getName(), SegmentDirectoryPaths.V3_SUBDIRECTORY_NAME);
+    indexSegment.destroy();
+  }
+
+  private void constructSegmentWithVectorIndex(SegmentVersion segmentVersion)
+      throws Exception {
+    FileUtils.deleteQuietly(INDEX_DIR);
+    SegmentGeneratorConfig segmentGeneratorConfig =
+        SegmentTestUtils.getSegmentGeneratorConfigWithoutTimeColumn(_vectorAvroFile, INDEX_DIR, "testTable");
+    SegmentIndexCreationDriver driver = SegmentCreationDriverFactory.get(null);
+    List<String> rawIndexCreationColumns = Lists.newArrayList(VECTOR_INDEX_COL_NAME);
+    VectorIndexConfig vectorIndexConfig = new VectorIndexConfig(ImmutableMap.of(
+        "vectorDimension", String.valueOf(VECTOR_DIM_SIZE),
+        "vectorIndexType", "HNSW"
+    ));
+    segmentGeneratorConfig.setIndexOn(StandardIndexes.vector(), vectorIndexConfig, VECTOR_INDEX_COL_NAME);
+    segmentGeneratorConfig.setRawIndexCreationColumns(rawIndexCreationColumns);
+    segmentGeneratorConfig.setSegmentVersion(segmentVersion);
+    driver.init(segmentGeneratorConfig);
+    driver.build();
+
+    _indexDir = new File(INDEX_DIR, driver.getSegmentName());
+  }
+
   private void verifyIndexDirIsV3(File indexDir) {
     File[] files = indexDir.listFiles();
     Assert.assertEquals(files.length, 1);
diff --git a/pinot-segment-local/src/test/resources/data/test_vector_data.avro b/pinot-segment-local/src/test/resources/data/test_vector_data.avro
new file mode 100644
index 0000000000..d35b7a5a58
Binary files /dev/null and b/pinot-segment-local/src/test/resources/data/test_vector_data.avro differ
diff --git a/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/V1Constants.java b/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/V1Constants.java
index 887bd87ff5..25ded5fa30 100644
--- a/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/V1Constants.java
+++ b/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/V1Constants.java
@@ -54,6 +54,9 @@ public class V1Constants {
     public static final String LUCENE_TEXT_INDEX_FILE_EXTENSION = ".lucene.index";
     public static final String LUCENE_V9_FST_INDEX_FILE_EXTENSION = ".lucene.v9.fst";
     public static final String LUCENE_V9_TEXT_INDEX_FILE_EXTENSION = ".lucene.v9.index";
+    public static final String VECTOR_INDEX_FILE_EXTENSION = ".vector.index";
+    public static final String VECTOR_HNSW_INDEX_FILE_EXTENSION = ".vector.hnsw.index";
+    public static final String VECTOR_HNSW_INDEX_DOCID_MAPPING_FILE_EXTENSION = ".vector.hnsw.mapping";
   }
 
   public static class MetadataKeys {
diff --git a/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/datasource/DataSource.java b/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/datasource/DataSource.java
index 821cfc6726..8775c14e12 100644
--- a/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/datasource/DataSource.java
+++ b/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/datasource/DataSource.java
@@ -30,6 +30,7 @@ import org.apache.pinot.segment.spi.index.reader.JsonIndexReader;
 import org.apache.pinot.segment.spi.index.reader.NullValueVectorReader;
 import org.apache.pinot.segment.spi.index.reader.RangeIndexReader;
 import org.apache.pinot.segment.spi.index.reader.TextIndexReader;
+import org.apache.pinot.segment.spi.index.reader.VectorIndexReader;
 
 
 /**
@@ -103,4 +104,10 @@ public interface DataSource {
    */
   @Nullable
   NullValueVectorReader getNullValueVector();
+
+  /**
+   * Returns the vector index for the column if exists, or {@code null} if not.
+   */
+  @Nullable
+  VectorIndexReader getVectorIndex();
 }
diff --git a/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/index/StandardIndexes.java b/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/index/StandardIndexes.java
index 4c61cb6d0a..f97ee5252f 100644
--- a/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/index/StandardIndexes.java
+++ b/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/index/StandardIndexes.java
@@ -28,6 +28,8 @@ import org.apache.pinot.segment.spi.index.creator.GeoSpatialIndexCreator;
 import org.apache.pinot.segment.spi.index.creator.H3IndexConfig;
 import org.apache.pinot.segment.spi.index.creator.JsonIndexCreator;
 import org.apache.pinot.segment.spi.index.creator.TextIndexCreator;
+import org.apache.pinot.segment.spi.index.creator.VectorIndexConfig;
+import org.apache.pinot.segment.spi.index.creator.VectorIndexCreator;
 import org.apache.pinot.segment.spi.index.reader.BloomFilterReader;
 import org.apache.pinot.segment.spi.index.reader.Dictionary;
 import org.apache.pinot.segment.spi.index.reader.ForwardIndexReader;
@@ -37,6 +39,7 @@ import org.apache.pinot.segment.spi.index.reader.JsonIndexReader;
 import org.apache.pinot.segment.spi.index.reader.NullValueVectorReader;
 import org.apache.pinot.segment.spi.index.reader.RangeIndexReader;
 import org.apache.pinot.segment.spi.index.reader.TextIndexReader;
+import org.apache.pinot.segment.spi.index.reader.VectorIndexReader;
 import org.apache.pinot.spi.config.table.BloomFilterConfig;
 import org.apache.pinot.spi.config.table.IndexConfig;
 import org.apache.pinot.spi.config.table.JsonIndexConfig;
@@ -74,6 +77,7 @@ public class StandardIndexes {
   public static final String RANGE_ID = "range_index";
   public static final String TEXT_ID = "text_index";
   public static final String H3_ID = "h3_index";
+  public static final String VECTOR_ID = "vector_index";
 
   private StandardIndexes() {
   }
@@ -127,4 +131,9 @@ public class StandardIndexes {
     return (IndexType<H3IndexConfig, H3IndexReader, GeoSpatialIndexCreator>)
         IndexService.getInstance().get(H3_ID);
   }
+
+  public static IndexType<VectorIndexConfig, VectorIndexReader, VectorIndexCreator> vector() {
+    return (IndexType<VectorIndexConfig, VectorIndexReader, VectorIndexCreator>)
+        IndexService.getInstance().get(VECTOR_ID);
+  }
 }
diff --git a/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/index/creator/VectorIndexConfig.java b/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/index/creator/VectorIndexConfig.java
new file mode 100644
index 0000000000..65efd841bb
--- /dev/null
+++ b/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/index/creator/VectorIndexConfig.java
@@ -0,0 +1,126 @@
+/**
+ * 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.pinot.segment.spi.index.creator;
+
+import com.google.common.base.Preconditions;
+import java.util.Map;
+import javax.annotation.Nullable;
+import org.apache.pinot.spi.config.table.IndexConfig;
+
+
+/**
+ * Config for vector index. Since this is generic configs for vector index, the only common fields are version,
+ * vectorIndexType, vectorDimension, distance function. All the other configs are specific to the index type should
+ * be put in properties.
+ */
+public class VectorIndexConfig extends IndexConfig {
+  public static final VectorIndexConfig DISABLED = new VectorIndexConfig(true);
+  private static final String VECTOR_INDEX_TYPE = "vectorIndexType";
+  private static final String VECTOR_DIMENSION = "vectorDimension";
+  private static final String VECTOR_DISTANCE_FUNCTION = "vectorDistanceFunction";
+  private static final String VERSION = "version";
+  private static final String DEFAULT_VERSION = "1";
+  private static final VectorDistanceFunction DEFAULT_VECTOR_DISTANCE_FUNCTION =
+      VectorDistanceFunction.COSINE;
+
+  private String _vectorIndexType;
+  private int _vectorDimension;
+  private int _version;
+  private VectorDistanceFunction _vectorDistanceFunction;
+  private Map<String, String> _properties;
+
+  /**
+   * @param disabled whether the config is disabled. Null is considered enabled.
+   */
+  public VectorIndexConfig(Boolean disabled) {
+    super(disabled);
+  }
+
+  // Used to read from older configs
+  public VectorIndexConfig(@Nullable Map<String, String> properties) {
+    super(false);
+    Preconditions.checkArgument(properties != null, "Properties must not be null");
+    Preconditions.checkArgument(properties.containsKey(VECTOR_INDEX_TYPE),
+        "Properties must contain vector index type");
+    _vectorIndexType = properties.get(VECTOR_INDEX_TYPE);
+    Preconditions.checkArgument(properties.containsKey(VECTOR_DIMENSION),
+        "Properties must contain vector dimension");
+    _vectorDimension = Integer.parseInt(properties.get(VECTOR_DIMENSION));
+    _vectorDistanceFunction = properties.containsKey(VECTOR_DISTANCE_FUNCTION) ? VectorDistanceFunction.valueOf(
+        properties.get(VECTOR_DISTANCE_FUNCTION)) : DEFAULT_VECTOR_DISTANCE_FUNCTION;
+    _version = Integer.parseInt(properties.getOrDefault(VERSION, DEFAULT_VERSION));
+    _properties = properties;
+  }
+
+  public String getVectorIndexType() {
+    return _vectorIndexType;
+  }
+
+  public VectorIndexConfig setVectorIndexType(String vectorIndexType) {
+    _vectorIndexType = vectorIndexType;
+    return this;
+  }
+
+  public int getVectorDimension() {
+    return _vectorDimension;
+  }
+
+  public VectorIndexConfig setVectorDimension(int vectorDimension) {
+    _vectorDimension = vectorDimension;
+    return this;
+  }
+
+  public VectorDistanceFunction getVectorDistanceFunction() {
+    return _vectorDistanceFunction;
+  }
+
+  public VectorIndexConfig setVectorDistanceFunction(
+      VectorDistanceFunction vectorDistanceFunction) {
+    _vectorDistanceFunction = vectorDistanceFunction;
+    return this;
+  }
+
+  public int getVersion() {
+    return _version;
+  }
+
+  public VectorIndexConfig setVersion(int version) {
+    _version = version;
+    return this;
+  }
+
+  public Map<String, String> getProperties() {
+    return _properties;
+  }
+
+  public VectorIndexConfig setProperties(Map<String, String> properties) {
+    _properties = properties;
+    return this;
+  }
+
+  public String toString() {
+    return "VectorIndexConfig{" + "_vectorIndexType='" + _vectorIndexType + "', _vectorDimension="
+        + _vectorDimension + ", _version=" + _version + ", _vectorDistanceFunction="
+        + _vectorDistanceFunction + ", _properties=" + _properties + '}';
+  }
+
+  public enum VectorDistanceFunction {
+    COSINE, INNER_PRODUCT, EUCLIDEAN, DOT_PRODUCT;
+  }
+}
diff --git a/pinot-common/src/main/java/org/apache/pinot/sql/FilterKind.java b/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/index/creator/VectorIndexCreator.java
similarity index 53%
copy from pinot-common/src/main/java/org/apache/pinot/sql/FilterKind.java
copy to pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/index/creator/VectorIndexCreator.java
index 0c2a8dca75..030f250bd4 100644
--- a/pinot-common/src/main/java/org/apache/pinot/sql/FilterKind.java
+++ b/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/index/creator/VectorIndexCreator.java
@@ -16,37 +16,32 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.pinot.sql;
+package org.apache.pinot.segment.spi.index.creator;
 
-public enum FilterKind {
-  AND,
-  OR,
-  NOT,
-  EQUALS,
-  NOT_EQUALS,
-  GREATER_THAN,
-  GREATER_THAN_OR_EQUAL,
-  LESS_THAN,
-  LESS_THAN_OR_EQUAL,
-  BETWEEN,
-  RANGE,
-  IN,
-  NOT_IN,
-  LIKE,
-  REGEXP_LIKE,
-  TEXT_CONTAINS,
-  TEXT_MATCH,
-  JSON_MATCH,
-  IS_NULL,
-  IS_NOT_NULL;
+import java.io.IOException;
+import javax.annotation.Nonnull;
+import javax.annotation.Nullable;
+import org.apache.pinot.segment.spi.index.IndexCreator;
 
-  /**
-   * Helper method that returns true if the enum maps to a Range.
-   *
-   * @return True if the enum is of Range type, false otherwise.
-   */
-  public boolean isRange() {
-    return this == GREATER_THAN || this == GREATER_THAN_OR_EQUAL || this == LESS_THAN || this == LESS_THAN_OR_EQUAL
-        || this == BETWEEN || this == RANGE;
+
+public interface VectorIndexCreator extends IndexCreator {
+  @Override
+  default void add(@Nonnull Object value, int dictId)
+      throws IOException {
+    throw new UnsupportedOperationException("Mutable Vector indexes are not supported for single-valued columns");
+  }
+
+  @Override
+  default void add(@Nonnull Object[] values, @Nullable int[] dictIds) {
+  }
+
+  void add(float[] document);
+
+  void seal()
+      throws IOException;
+
+  @Override
+  default void close()
+      throws IOException {
   }
 }
diff --git a/pinot-common/src/main/java/org/apache/pinot/sql/FilterKind.java b/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/index/reader/VectorIndexReader.java
similarity index 56%
copy from pinot-common/src/main/java/org/apache/pinot/sql/FilterKind.java
copy to pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/index/reader/VectorIndexReader.java
index 0c2a8dca75..7a906fc648 100644
--- a/pinot-common/src/main/java/org/apache/pinot/sql/FilterKind.java
+++ b/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/index/reader/VectorIndexReader.java
@@ -16,37 +16,21 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.pinot.sql;
+package org.apache.pinot.segment.spi.index.reader;
 
-public enum FilterKind {
-  AND,
-  OR,
-  NOT,
-  EQUALS,
-  NOT_EQUALS,
-  GREATER_THAN,
-  GREATER_THAN_OR_EQUAL,
-  LESS_THAN,
-  LESS_THAN_OR_EQUAL,
-  BETWEEN,
-  RANGE,
-  IN,
-  NOT_IN,
-  LIKE,
-  REGEXP_LIKE,
-  TEXT_CONTAINS,
-  TEXT_MATCH,
-  JSON_MATCH,
-  IS_NULL,
-  IS_NOT_NULL;
+import org.apache.pinot.segment.spi.index.IndexReader;
+import org.roaringbitmap.buffer.ImmutableRoaringBitmap;
 
+
+/**
+ * Reader for vector index.
+ */
+public interface VectorIndexReader extends IndexReader {
   /**
-   * Helper method that returns true if the enum maps to a Range.
-   *
-   * @return True if the enum is of Range type, false otherwise.
+   * Returns the bitmap of top k closest vectors from the given vector.
+   * @param vector vector to search
+   * @param topK number of closest vectors to return
+   * @return bitmap of top k closest vectors
    */
-  public boolean isRange() {
-    return this == GREATER_THAN || this == GREATER_THAN_OR_EQUAL || this == LESS_THAN || this == LESS_THAN_OR_EQUAL
-        || this == BETWEEN || this == RANGE;
-  }
+  ImmutableRoaringBitmap getDocIds(float[] vector, int topK);
 }
diff --git a/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/store/SegmentDirectoryPaths.java b/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/store/SegmentDirectoryPaths.java
index 2f1cf67b81..b8c09a0329 100644
--- a/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/store/SegmentDirectoryPaths.java
+++ b/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/store/SegmentDirectoryPaths.java
@@ -117,6 +117,13 @@ public class SegmentDirectoryPaths {
     return findFormatFile(indexDir, file);
   }
 
+  @Nullable
+  @VisibleForTesting
+  public static File findVectorIndexIndexFile(File segmentIndexDir, String column) {
+    String vectorIndexDirectory = column + V1Constants.Indexes.VECTOR_HNSW_INDEX_FILE_EXTENSION;
+    return findFormatFile(segmentIndexDir, vectorIndexDirectory);
+  }
+
   /**
    * Find a file in any segment version.
    * <p>Index directory passed in should be top level segment directory.
diff --git a/pinot-spi/src/main/java/org/apache/pinot/spi/config/table/FieldConfig.java b/pinot-spi/src/main/java/org/apache/pinot/spi/config/table/FieldConfig.java
index 145f9dd65c..e1b554e54a 100644
--- a/pinot-spi/src/main/java/org/apache/pinot/spi/config/table/FieldConfig.java
+++ b/pinot-spi/src/main/java/org/apache/pinot/spi/config/table/FieldConfig.java
@@ -118,7 +118,7 @@ public class FieldConfig extends BaseJsonConfig {
 
   // If null, there won't be any index
   public enum IndexType {
-    INVERTED, SORTED, TEXT, FST, H3, JSON, TIMESTAMP, RANGE
+    INVERTED, SORTED, TEXT, FST, H3, JSON, TIMESTAMP, VECTOR, RANGE
   }
 
   public enum CompressionCodec {
diff --git a/pinot-spi/src/main/java/org/apache/pinot/spi/config/table/IndexingConfig.java b/pinot-spi/src/main/java/org/apache/pinot/spi/config/table/IndexingConfig.java
index 9b7930cd74..a433c845ca 100644
--- a/pinot-spi/src/main/java/org/apache/pinot/spi/config/table/IndexingConfig.java
+++ b/pinot-spi/src/main/java/org/apache/pinot/spi/config/table/IndexingConfig.java
@@ -42,6 +42,7 @@ public class IndexingConfig extends BaseJsonConfig {
   private List<String> _jsonIndexColumns;
   private Map<String, JsonIndexConfig> _jsonIndexConfigs;
   private List<String> _h3IndexColumns;
+  private List<String> _vectorIndexColumns;
   private List<String> _sortedColumn;
   private List<String> _bloomFilterColumns;
   private Map<String, BloomFilterConfig> _bloomFilterConfigs;
@@ -156,6 +157,33 @@ public class IndexingConfig extends BaseJsonConfig {
     _createInvertedIndexDuringSegmentGeneration = createInvertedIndexDuringSegmentGeneration;
   }
 
+  public List<String> getH3IndexColumns() {
+    return _h3IndexColumns;
+  }
+
+  public IndexingConfig setH3IndexColumns(List<String> h3IndexColumns) {
+    _h3IndexColumns = h3IndexColumns;
+    return this;
+  }
+
+  public List<String> getVectorIndexColumns() {
+    return _vectorIndexColumns;
+  }
+
+  public IndexingConfig setVectorIndexColumns(List<String> vectorIndexColumns) {
+    _vectorIndexColumns = vectorIndexColumns;
+    return this;
+  }
+
+  public FSTType getFstTypeForFSTIndex() {
+    return _fstTypeForFSTIndex;
+  }
+
+  public IndexingConfig setFstTypeForFSTIndex(FSTType fstTypeForFSTIndex) {
+    _fstTypeForFSTIndex = fstTypeForFSTIndex;
+    return this;
+  }
+
   @Nullable
   public List<String> getSortedColumn() {
     return _sortedColumn;
diff --git a/pinot-tools/src/main/java/org/apache/pinot/tools/QuickStartBase.java b/pinot-tools/src/main/java/org/apache/pinot/tools/QuickStartBase.java
index d9127ed4b5..e3942e76e7 100644
--- a/pinot-tools/src/main/java/org/apache/pinot/tools/QuickStartBase.java
+++ b/pinot-tools/src/main/java/org/apache/pinot/tools/QuickStartBase.java
@@ -23,6 +23,8 @@ import com.fasterxml.jackson.databind.node.ArrayNode;
 import com.google.common.base.Preconditions;
 import com.google.common.collect.ImmutableMap;
 import java.io.File;
+import java.io.FileInputStream;
+import java.io.FileOutputStream;
 import java.io.IOException;
 import java.net.URL;
 import java.nio.charset.StandardCharsets;
@@ -31,6 +33,7 @@ import java.util.ArrayList;
 import java.util.List;
 import java.util.Map;
 import java.util.Properties;
+import java.util.zip.GZIPInputStream;
 import org.apache.commons.configuration2.ex.ConfigurationException;
 import org.apache.commons.io.FileUtils;
 import org.apache.commons.io.LineIterator;
@@ -86,6 +89,7 @@ public abstract class QuickStartBase {
       .put("upsertMeetupRsvp", "examples/stream/upsertMeetupRsvp")
       .put("upsertJsonMeetupRsvp", "examples/stream/upsertJsonMeetupRsvp")
       .put("upsertPartialMeetupRsvp", "examples/stream/upsertPartialMeetupRsvp")
+      .put("fineFoodReviews", "examples/stream/fineFoodReviews")
       .build();
 
   protected File _dataDir = FileUtils.getTempDirectory();
@@ -332,14 +336,18 @@ public abstract class QuickStartBase {
       throws Exception {
     switch (tableName) {
       case "githubEvents":
-        publishGithubEventsToKafka("githubEvents", new File(dataDir, "/rawdata/2021-07-21-few-hours.json"));
+        publishLineSplitFileToKafka("githubEvents", new File(dataDir, "/rawdata/2021-07-21-few-hours.json"));
+        break;
+      case "fineFoodReviews":
+        publishLineSplitFileToKafka("fineFoodReviews",
+            new File(dataDir, "/rawdata/fine_food_reviews_with_embeddings_1k.json.gz"));
         break;
       default:
         break;
     }
   }
 
-  protected static void publishGithubEventsToKafka(String topicName, File dataFile)
+  protected static void publishLineSplitFileToKafka(String topicName, File dataFile)
       throws Exception {
     Properties properties = new Properties();
     properties.put("metadata.broker.list", KafkaStarterUtils.DEFAULT_KAFKA_BROKER);
@@ -348,6 +356,14 @@ public abstract class QuickStartBase {
     StreamDataProducer producer =
         StreamDataProvider.getStreamDataProducer(KafkaStarterUtils.KAFKA_PRODUCER_CLASS_NAME, properties);
     try {
+      if (dataFile.getName().endsWith(".gz")) {
+        File unzippedFile = new File(dataFile.getParentFile(), dataFile.getName().replace(".gz", ""));
+        if (unzippedFile.exists()) {
+          FileUtils.deleteQuietly(unzippedFile);
+        }
+        unGzipFile(dataFile.getAbsolutePath(), unzippedFile.getAbsolutePath());
+        dataFile = unzippedFile;
+      }
       LineIterator dataStream = FileUtils.lineIterator(dataFile);
 
       while (dataStream.hasNext()) {
@@ -358,6 +374,19 @@ public abstract class QuickStartBase {
     }
   }
 
+  public static void unGzipFile(String gzipFile, String outputFile) {
+    byte[] buffer = new byte[1024];
+    try (GZIPInputStream gzis = new GZIPInputStream(new FileInputStream(gzipFile));
+        FileOutputStream out = new FileOutputStream(outputFile)) {
+      int len;
+      while ((len = gzis.read(buffer)) > 0) {
+        out.write(buffer, 0, len);
+      }
+    } catch (IOException e) {
+      e.printStackTrace();
+    }
+  }
+
   protected void startKafka() {
     printStatus(Quickstart.Color.CYAN, "***** Starting Kafka *****");
     _zookeeperInstance = ZkStarter.startLocalZkServer();
@@ -489,9 +518,436 @@ public abstract class QuickStartBase {
           printStatus(Quickstart.Color.CYAN, "***** Starting githubEvents data stream and publishing to Kafka *****");
           publishStreamDataToKafka("githubEvents", new File(quickstartTmpDir, "githubEvents"));
           break;
+        case "fineFoodReviews":
+          kafkaStarter.createTopic("fineFoodReviews", KafkaStarterUtils.getTopicCreationProps(2));
+          printStatus(Quickstart.Color.CYAN,
+              "***** Starting fineFoodReviews data stream and publishing to Kafka *****");
+          publishStreamDataToKafka("fineFoodReviews", new File(quickstartTmpDir, "fineFoodReviews"));
+          break;
         default:
           throw new UnsupportedOperationException("Unknown stream name: " + streamName);
       }
     }
   }
+
+  protected void runVectorQueryExamples(QuickstartRunner runner)
+      throws Exception {
+    // The following embedding is the text: "tomato soup" generated by model "text-embedding-ada-002" from OpenAI.
+    // The Python code snippet is:
+    //    import openai
+    //    from openai.embeddings_utils import get_embedding
+    //    openai.api_key = "sk-xxxx"
+    //    response = openai.Embedding.create(
+    //      model="text-embedding-ada-002",  # You can choose different models
+    //      input="tomato soup"
+    //    )
+    //    search_embedding = response['data'][0]['embedding']
+    //    print(search_embedding)
+    String vectorArrayLiteral =
+        "ARRAY[0.010218413546681404, -0.015945643186569214, -0.008374051190912724, -0.011700374074280262, -0"
+            + ".009357711300253868, 0.012360461987555027, -0.022675946354866028, -0.01687753200531006, -0"
+            + ".009700697846710682, -0.030778197571635246, 0.023672549054026604, 0.009280053898692131, -0"
+            + ".02050154097378254, 0.008736452087759972, -0.02531629614531994, 0.02303834818303585, 0"
+            + ".02785310335457325, -0.007241548039019108, 0.0252386387437582, -0.033392660319805145, 0"
+            + ".009545383043587208, 0.007493934594094753, 0.016916360706090927, -0.018482450395822525, -0"
+            + ".011881574988365173, -0.012748748995363712, 0.0005990132340230048, -0.0218864306807518, 0"
+            + ".009584211744368076, -0.0014229092048481107, 0.01646335795521736, -0.022417088970541954, -0"
+            + ".011389745399355888, -0.017201103270053864, 0.02022974006831646, 0.01375829428434372, -0"
+            + ".010710243135690689, -0.0032632267102599144, 0.022404145449399948, -0.020113253965973854, 0"
+            + ".010425499640405178, 0.003106294199824333, -0.001236855168826878, -0.01876719295978546, -0"
+            + ".03303026035428047, -0.007804563734680414, 0.008069893345236778, -0.011150301434099674, 0"
+            + ".0042679188773036, -0.009642454795539379, 0.03919107839465141, -0.0018686300609260798, -0"
+            + ".0029153863433748484, -0.0007668663747608662, 0.006801489740610123, 0.01792590506374836, -0"
+            + ".015816213563084602, 0.0005670604878105223, -0.01019899919629097, -0.004623848013579845, 0"
+            + ".01546675618737936, 0.01832713559269905, -0.034712836146354675, 0.0126193193718791, -0"
+            + ".006685003638267517, -0.014288952574133873, -0.010846143588423729, -0.013913609087467194, 0"
+            + ".004083482548594475, 0.008386993780732155, 0.03939816355705261, 0.03409157693386078, 0"
+            + ".007739849388599396, -0.00590519467368722, 0.007422748487442732, 0.0013323089806362987, -0"
+            + ".012004532851278782, -0.003077172674238682, -0.004138489719480276, -0.01057434268295765, 0"
+            + ".023594891652464867, -0.036576613783836365, -0.005892251618206501, 0.014327781274914742, 0"
+            + ".02456560917198658, 0.004869763273745775, -0.019116651266813278, 0.03924284875392914, -0"
+            + ".006503803189843893, -0.01236693374812603, -0.012036889791488647, 0.011169715784490108, 0"
+            + ".026895329356193542, 0.022274717688560486, -0.01748584769666195, 0.018831908702850342, 0"
+            + ".007370977196842432, 0.03442809358239174, -0.0034007448703050613, -0.03771558776497841, -0"
+            + ".012108075432479382, -0.010677886195480824, 0.0013954055029898882, -0.012392818927764893, -0"
+            + ".013201749883592129, -0.021304000169038773, 0.0320466011762619, 0.01057434268295765, -0"
+            + ".0013776090927422047, -0.0093706538900733, -0.010697300545871258, 0.032745517790317535, -0"
+            + ".010393142700195312, -0.03331500291824341, 0.013667694292962551, -0.009778355248272419, 0"
+            + ".012444591149687767, -0.030208710581064224, -0.024267923086881638, 0.0015814596554264426, 0"
+            + ".0012069246731698513, 0.006659117992967367, 0.031632427126169205, -0.023504292592406273, 0"
+            + ".011939818039536476, -0.00034339111880399287, -0.0053551215678453445, 0.0013816537102684379, -0"
+            + ".028370819985866547, -0.010956157930195332, 0.055343806743621826, 0.01859893649816513, 0"
+            + ".01748584769666195, -0.0018977515865117311, -0.03279728814959526, 0.018391849473118782, -0"
+            + ".03269374370574951, -0.0241902656853199, -0.027594245970249176, -0.02551043964922428, 0"
+            + ".0007822360494174063, 0.02676590159535408, 0.0021177807357162237, 0.0027891932986676693, -0"
+            + ".006186702288687229, -0.001595211448147893, 0.031114712357521057, 0.0033198518212884665, -0"
+            + ".0068597327917814255, 0.00025016182917170227, 0.0053810072131454945, -0.031321797519922256, 0"
+            + ".014987869188189507, -0.010237827897071838, -0.0005529041518457234, -0.00018635742890182883, 0"
+            + ".010004855692386627, 0.013926551677286625, -0.009066496044397354, 0.011273259297013283, -0"
+            + ".01053551398217678, -0.004853584337979555, 0.0063873170875012875, -0.018793080002069473, -0"
+            + ".006125223822891712, 0.03398803621530533, 0.02818961814045906, 0.018275363370776176, -0"
+            + ".011855688877403736, -0.018145935609936714, 0.013861837796866894, 0.03445398062467575, -0"
+            + ".024345580488443375, 0.01409480907022953, -0.013848894275724888, 0.013848894275724888, 0"
+            + ".0012360461987555027, 0.0056398650631308556, -0.004358518868684769, -0.018443621695041656, -0"
+            + ".04133959859609604, 0.008147550746798515, 0.008852938190102577, 0.044575318694114685, -0"
+            + ".010050156153738499, -0.018922507762908936, -6.577618478331715e-05, -0.0027228610124439, -0"
+            + ".022857148200273514, -0.016088014468550682, 0.019867340102791786, 0.024889182299375534, 0"
+            + ".02171817235648632, -0.004326161462813616, -0.6846272349357605, -0.035489410161972046, -0"
+            + ".023840807378292084, -0.025562211871147156, 0.021562857553362846, 0.011713317595422268, -0"
+            + ".0012514159316197038, -0.016476301476359367, -0.024591494351625443, -0.01519495528191328, -0"
+            + ".013538265600800514, -0.002554603386670351, 0.004112604074180126, -0.00669147539883852, 0"
+            + ".013460608199238777, -0.0364471860229969, 0.004009061027318239, -0.007383919786661863, -0"
+            + ".012133961543440819, 0.03523055091500282, -0.027335388585925102, 0.0014496039366349578, -0"
+            + ".004788869991898537, -0.0027228610124439, 0.02551043964922428, -0.0047791628167033195, 0"
+            + ".0036725455429404974, -0.012832877226173878, -0.012910534627735615, 0.006526453420519829, -0"
+            + ".023051291704177856, 0.008704095147550106, 0.02548455446958542, -0.007694549392908812, 0"
+            + ".057518213987350464, 0.013577093370258808, -0.02079922705888748, 0.027956647798419, 0"
+            + ".02381492219865322, 0.03194305673241615, -0.014211295172572136, -0.012729334644973278, 0"
+            + ".017602333799004555, -0.006235238164663315, -0.016657501459121704, -0.0005674649146385491, 0"
+            + ".0196861382573843, 0.01102734450250864, -0.005704579874873161, -0.022067630663514137, 0"
+            + ".035592954605817795, 0.006021680776029825, -0.020410940051078796, 0.0018330371240153909, 0"
+            + ".022442974150180817, 0.017226990312337875, 0.014392496086657047, -0.023569006472826004, -0"
+            + ".023504292592406273, 0.012897592037916183, -0.00891118124127388, 0.01667044498026371, -0"
+            + ".018003562465310097, -0.041831426322460175, 0.015673842281103134, 0.0109949866309762, -0"
+            + ".009170038625597954, 0.014081866480410099, 0.01890956610441208, -0.015622070990502834, 0"
+            + ".020915713161230087, 0.026921216398477554, 0.0017036081990227103, -0.012399290688335896, 0"
+            + ".028552019968628883, 0.016748102381825447, -0.0013193660415709019, 0.011868632398545742, 0"
+            + ".008891766890883446, -0.006028152070939541, 0.01173920277506113, -0.005186864174902439, -0"
+            + ".003462223568931222, -0.013874780386686325, 0.05881250277161598, -0.00857466645538807, -0"
+            + ".03919107839465141, 0.006057273596525192, 0.00021193985594436526, -0.010134284384548664, -0"
+            + ".006270831450819969, 0.022753603756427765, -0.025665754452347755, -0.022857148200273514, -0"
+            + ".02406083606183529, 0.0010791136883199215, -0.02531629614531994, 0.005093027837574482, 0"
+            + ".006002266425639391, -0.03365151956677437, -0.012211618945002556, 0.002769778948277235, 0"
+            + ".019608480855822563, 0.0037275529466569424, 0.006519982125610113, 0.011111472733318806, -0"
+            + ".008820581249892712, 0.019090766087174416, 0.015039640478789806, -0.03880279138684273, -0"
+            + ".0010427117813378572, 0.0033295589964836836, 0.009558325633406639, -0.03784501552581787, 0"
+            + ".017602333799004555, -0.013357064686715603, 0.019375508651137352, -0.00658146059140563, 0"
+            + ".008755866438150406, -0.021912315860390663, -0.0010718333069235086, -0.0019252551719546318, 0"
+            + ".007578063290566206, 0.017356418073177338, 0.008477594703435898, 0.01047080010175705, 8"
+            + ".66567061166279e-05, -0.02395729348063469, -0.020177969709038734, -0.02446206659078598, -0"
+            + ".0017909726593643427, 0.007442162837833166, -0.0020045305136591196, -0.01721404679119587, 0"
+            + ".005044492427259684, 0.004730626940727234, 0.018107106909155846, -0.014224238693714142, 0"
+            + ".0209416002035141, 0.0018831908237189054, -0.0025028318632394075, -0.0024009065236896276, -0"
+            + ".006545867770910263, 0.0210451427847147, 0.005093027837574482, -0.025199810042977333, -0"
+            + ".014405438676476479, -0.015026697888970375, -0.012476948089897633, 0.0077980924397706985, 0"
+            + ".010496685281395912, -0.017796477302908897, -0.0037728529423475266, -0.005571914836764336, 0"
+            + ".019776739180088043, -0.014185409992933273, 0.00698916194960475, -0.020436827093362808, -0"
+            + ".006086395122110844, -0.0017214046092703938, 0.002803754061460495, 0.027464816346764565, -0"
+            + ".0019721731077879667, 0.006510274950414896, -0.011383273638784885, -0.028344932943582535, -0"
+            + ".015842100605368614, 0.009318882599473, -0.011519174091517925, -0.02456560917198658, 0"
+            + ".010024270042777061, -0.02327132038772106, 0.0037178457714617252, 0.01941433735191822, -0"
+            + ".02619641274213791, 0.025937555357813835, -0.008445236831903458, -0.00918298214673996, 0"
+            + ".032305460423231125, 0.00472739152610302, 0.0014568843180313706, -0.00973952654749155, -0"
+            + ".00973952654749155, -0.0014528395840898156, 0.019634367898106575, -7.376437133643776e-05, -0"
+            + ".0024834175128489733, 0.016489244997501373, -0.01463841088116169, 0.0376897007226944, -0"
+            + ".018987223505973816, -0.004083482548594475, 0.007849863730370998, 0.015635013580322266, 0"
+            + ".02531629614531994, -0.019323738291859627, -0.01067141443490982, 0.022274717688560486, 0"
+            + ".021744059398770332, -0.008607023395597935, 0.012580491602420807, -0.007474520243704319, 0"
+            + ".019802624359726906, -0.010826729238033295, 0.016916360706090927, -0.014250123873353004, 0"
+            + ".004937713500112295, 0.0015806506853550673, 0.010988515801727772, 0.007662191987037659, 0"
+            + ".006659117992967367, -0.021459314972162247, -0.027438931167125702, -0.002679178724065423, 0"
+            + ".0036240098997950554, 0.02852613478899002, 0.013499436900019646, -0.00530982157215476, -0"
+            + ".016049185767769814, 0.0013711376814171672, -0.004565604962408543, 0.0009828509064391255, -0"
+            + ".005264521576464176, -0.008160493336617947, -0.0196861382573843, 0.0004695843090303242, 0"
+            + ".012341047637164593, 0.03435043618083, -0.0004930432769469917, -0.020177969709038734, -0"
+            + ".022753603756427765, 0.014327781274914742, 0.01985439658164978, 0.00773337809368968, 0"
+            + ".015052583068609238, 0.004371461924165487, 0.0165410153567791, -0.01025077048689127, 0"
+            + ".021938202902674675, 0.015285555273294449, -0.0032761695329099894, 0.004368226043879986, 0"
+            + ".017188161611557007, 5.50578479305841e-05, 0.013020549900829792, -0.017058731988072395, 0"
+            + ".028008418157696724, -0.006976218894124031, -0.0093706538900733, 0.0018346549477428198, 0"
+            + ".004284097347408533, -0.0030496688559651375, -0.037793245166540146, 0.0076168919913470745, 0"
+            + ".0067561897449195385, -0.011150301434099674, -0.004352047573775053, 0.013602979481220245, 0"
+            + ".03587769716978073, 0.033289119601249695, 0.025148039683699608, -0.0011017636861652136, -0"
+            + ".012153375893831253, -0.013357064686715603, -0.0019026051741093397, -0.0013541501248255372, 0"
+            + ".005050963722169399, -0.0024284101091325283, -0.021705230697989464, 0.017679991200566292, -0"
+            + ".01985439658164978, -0.006956804543733597, 0.013564150780439377, -0.028914421796798706, -0"
+            + ".0023556062951684, -0.004724155645817518, 0.019323738291859627, 0.025536326691508293, -0"
+            + ".018728364259004593, -0.020812170580029488, -0.006723832339048386, -0.037793245166540146, 0"
+            + ".005358357448130846, 0.00885940995067358, 0.0005941596464253962, 0.01080731488764286, -0"
+            + ".012496362440288067, -0.006503803189843893, 0.004960363265126944, -0.008529365994036198, 0"
+            + ".016217444092035294, -0.005918137263506651, 0.0011236048303544521, 0.019233137369155884, -0"
+            + ".001074260100722313, -0.004911827389150858, -0.004863291513174772, -0.01485844049602747, 0"
+            + ".0006196409813128412, 0.029225049540400505, 0.0034557522740215063, -0.01426306739449501, -0"
+            + ".0018799550598487258, -0.01153858844190836, 0.03359974920749664, 0.008238150738179684, 0"
+            + ".0018928979989141226, -0.02894030697643757, -0.021407542750239372, -0.00821873638778925, -2"
+            + ".3256759504874935e-06, 0.009642454795539379, 0.012664619833230972, -0.018676593899726868, -0"
+            + ".009532440453767776, 0.005067142192274332, -0.0015256433980539441, -0.00651351036503911, 0"
+            + ".03533409535884857, 0.010677886195480824, -0.004002589266747236, -0.027568360790610313, -0"
+            + ".015285555273294449, 0.02117457240819931, 0.05881250277161598, 0.02611875720322132, -0"
+            + ".0014342342037707567, 0.0028086076490581036, 0.016722217202186584, -0.021731115877628326, -0"
+            + ".018417734652757645, -0.003449280746281147, 0.01679987460374832, -0.004698270000517368, 0"
+            + ".011493287980556488, -0.0012878177221864462, 0.012360461987555027, 0.0037178457714617252, -0"
+            + ".011221487075090408, 0.004309982992708683, -0.00970716867595911, -0.00343957357108593, -0"
+            + ".009545383043587208, -0.018443621695041656, -0.0028668507002294064, 0.0025740177370607853, -0"
+            + ".00690503278747201, 0.005442486144602299, 0.03533409535884857, 0.003517230972647667, 0"
+            + ".02395729348063469, 0.029639223590493202, 0.02873321995139122, -0.01519495528191328, -0"
+            + ".02148520015180111, -0.0011947907041758299, -0.018340077251195908, 0.0021646986715495586, 0"
+            + ".010509628802537918, -0.02002265490591526, 0.027361273765563965, 0.028215505182743073, 0"
+            + ".008865880779922009, -0.025083325803279877, 0.0058534229174256325, 0.0016259507974609733, 0"
+            + ".017239931970834732, -0.011519174091517925, 0.010800843127071857, -0.011053229682147503, -0"
+            + ".006866204086691141, 0.022650061175227165, -0.0010548457503318787, -0.0374826155602932, 0"
+            + ".02290891855955124, 0.02310306206345558, -0.023983178660273552, 0.0010928654810413718, -0"
+            + ".0033263233490288258, 0.023025404661893845, -0.0028959722258150578, 0.001330691040493548, -0"
+            + ".012308690696954727, 0.0019187837606295943, -0.020074425265192986, -0.023633720353245735, 0"
+            + ".01758939027786255, 0.011092058382928371, -0.014780783094465733, -0.023323090746998787, -0"
+            + ".013473550789058208, -0.0109949866309762, -0.011331502348184586, -0.0008012459147721529, 0"
+            + ".0020514484494924545, -0.0008149977657012641, -0.025432782247662544, 0.006801489740610123, 0"
+            + ".03277140110731125, 0.02148520015180111, -0.014897269196808338, -0.0030205475632101297, -0"
+            + ".013175863772630692, -0.0013654751237481833, -0.0173823032528162, -0.02175700105726719, 0"
+            + ".0024623852223157883, -0.02151108719408512, -0.012645205482840538, -0.0009529204107820988, 4"
+            + ".231719140079804e-05, -0.007882221601903439, 2.79333908110857e-05, 0.009487139992415905, 0"
+            + ".009215339086949825, 0.01670927368104458, 0.005711051169782877, -0.01795179210603237, -0"
+            + ".0004018363542854786, -0.006099337711930275, -0.006594403646886349, 0.0003846465842798352, 0"
+            + ".0014957130188122392, 0.007526291534304619, 0.00036947912303730845, 0.0013590037124231458, -0"
+            + ".013512379489839077, -0.006814432796090841, -0.002059537684544921, 0.0005241871112957597, 0"
+            + ".016282157972455025, 0.022921862080693245, -0.013861837796866894, -0.013590036891400814, 0"
+            + ".05353180319070816, -0.00641967449337244, 0.008011650294065475, -0.012936420738697052, -0"
+            + ".0006669634021818638, 0.022481802850961685, 0.009745997376739979, 0.009745997376739979, -0"
+            + ".0140300951898098, -0.011117944493889809, -0.019259022548794746, -0.02351723425090313, 0"
+            + ".002206763019785285, 0.00877528078854084, 0.0007785958587191999, -0.007869278080761433, 0"
+            + ".026442328467965126, -0.034816380590200424, -0.01006309874355793, -0.013447664678096771, 0"
+            + ".0076168919913470745, 0.00959068350493908, -0.008406408131122589, -0.008568194694817066, -0"
+            + ".01972496695816517, -0.01731758937239647, 0.006293481215834618, 0.024707980453968048, 0"
+            + ".018728364259004593, -0.03145122900605202, -0.0010823493357747793, 0.010503157041966915, -0"
+            + ".0037631457671523094, -0.004335868638008833, -0.0010208706371486187, -0.021459314972162247, 0"
+            + ".00268888589926064, -0.01343472208827734, 0.0014164377935230732, 0.010600228793919086, 0"
+            + ".008781752549111843, -0.0032389587722718716, 0.0035819453187286854, 0.02066979929804802, 0"
+            + ".01016664132475853, -0.023336034268140793, -0.027024758979678154, 0.016592787578701973, 0"
+            + ".009648925624787807, 0.013201749883592129, 0.03862158954143524, 0.0009537293808534741, 0"
+            + ".02029445394873619, 0.005840479861944914, -0.013900666497647762, 0.010134284384548664, -0"
+            + ".004309982992708683, 0.004876234568655491, -0.006982690189033747, 0.022753603756427765, 0"
+            + ".05267757177352905, 0.031968943774700165, -0.005830772686749697, -0.0040220036171376705, 0"
+            + ".01809416338801384, 0.00843876600265503, 0.003915224689990282, -0.015233783982694149, -0"
+            + ".007869278080761433, -0.027180073782801628, 0.003604595549404621, -0.0017828834243118763, 0"
+            + ".002671089256182313, 0.022857148200273514, 0.03303026035428047, -0.003607831196859479, 0"
+            + ".014767839573323727, 0.015660898759961128, 0.0069632758386433125, -0.006079923361539841, 0"
+            + ".023012463003396988, -0.022248830646276474, -0.005736936815083027, -0.021083971485495567, 0"
+            + ".0028247861191630363, -0.025264525786042213, 0.0033878020476549864, 0.0034881094470620155, -0"
+            + ".030519340187311172, 0.010315485298633575, 0.0021388130262494087, 0.005649572238326073, 0"
+            + ".003154829842969775, 0.004487948026508093, 0.0006726259016431868, 0.023711377754807472, -0"
+            + ".011260315775871277, -0.025044497102499008, 0.004468533676117659, -0.016100957989692688, -0"
+            + ".029561566188931465, -0.0055266148410737514, -0.03561883792281151, -0.02483741007745266, 0"
+            + ".002624171320348978, -0.010762014426290989, -0.008820581249892712, 0.021666401997208595, -0"
+            + ".03375506401062012, -0.015842100605368614, 0.005756351165473461, -0.0005359166534617543, 0"
+            + ".026610586792230606, 0.006639703642576933, 0.02762013114988804, -0.0018751014722511172, -0"
+            + ".024384409189224243, -0.010516099631786346, 0.02008736878633499, 0.005921373143792152, -0"
+            + ".007383919786661863, -0.005623686593025923, 0.01572561450302601, 0.015156126581132412, -0"
+            + ".01989322528243065, 0.014625468291342258, 0.025432782247662544, 0.007713963743299246, -0"
+            + ".0018249477725476027, 0.02538101188838482, -0.004750041291117668, 0.017434075474739075, -0"
+            + ".03212425857782364, -0.04035593941807747, -0.018107106909155846, 0.02598932757973671, -0"
+            + ".01241870503872633, 0.002530335448682308, 0.013732408173382282, -0.013195278123021126, -0"
+            + ".03005339577794075, 0.04656852409243584, -0.006380845792591572, 0.00274874665774405, 0"
+            + ".002815078943967819, -0.02131694369018078, 0.021524028852581978, -0.016113901510834694, -0"
+            + ".0117262601852417, 0.014806668274104595, -0.01591975800693035, 0.04045948013663292, 0"
+            + ".00212586997076869, 0.030234595760703087, 0.011583888903260231, -0.004552662372589111, -0"
+            + ".029147392138838768, 0.004714448470622301, 0.0010823493357747793, 0.020346226170659065, 0"
+            + ".01802944950759411, -0.03551529720425606, -0.0035107594449073076, -0.010283127427101135, 0"
+            + ".022520631551742554, -0.01551852747797966, -0.005005663726478815, -0.0002287251700181514, -0"
+            + ".014703125692903996, -0.013719465583562851, 0.004542955197393894, 0.017537618055939674, -0"
+            + ".018042391166090965, 0.008626437745988369, 0.004400583449751139, -0.009972498752176762, -0"
+            + ".01742113195359707, -0.0256528127938509, -0.014729010872542858, -0.00984306912869215, -0"
+            + ".023426635190844536, -0.010108399204909801, 0.006633232347667217, 0.011460931040346622, 0"
+            + ".005426307674497366, -0.02829316258430481, 0.016424529254436493, 0.022740662097930908, -0"
+            + ".008924123831093311, 0.025872841477394104, -0.023556062951683998, 0.0021744058467447758, -0"
+            + ".010639057494699955, -0.0002372189483139664, -0.005290407221764326, -0.008930595591664314, 0"
+            + ".025497497990727425, -0.031218254938721657, -0.015971528366208076, -0.021679343655705452, 0"
+            + ".0055686794221401215, 0.014120695181190968, 0.04025239497423172, -0.006827375385910273, 0"
+            + ".040744222700595856, -0.0015757970977574587, 0.0196861382573843, -0.015635013580322266, -0"
+            + ".011661545373499393, 0.022455917671322823, -0.013913609087467194, 0.012017475441098213, 0"
+            + ".009021195583045483, 0.009163567796349525, 0.015479698777198792, 0.01880602166056633, 0"
+            + ".020527426153421402, 0.01859893649816513, -0.009292996488511562, 0.002499595982953906, 0"
+            + ".005264521576464176, 0.031011169776320457, -0.003924931865185499, 5.596789560513571e-05, -0"
+            + ".00411583948880434, -0.004811520222574472, -0.008794695138931274, -0.011053229682147503, -0"
+            + ".013926551677286625, -0.0027503645978868008, 0.007506877649575472, 0.023491349071264267, -0"
+            + ".02571752667427063, 0.007701020687818527, 0.0042032040655612946, -0.002611228497698903, -0"
+            + ".027568360790610313, -0.004407054744660854, -0.006775604095309973, -0.026248184964060783, -0"
+            + ".011648602783679962, 0.007817506790161133, -0.0016340401489287615, -0.020579198375344276, -0"
+            + ".00379873882047832, -0.0024785639252513647, -0.03608478233218193, -0.008561722934246063, -0"
+            + ".0015127004589885473, 0.0055266148410737514, 0.019194308668375015, -0.008283451199531555, -0"
+            + ".02354312129318714, 0.03093351237475872, -0.02364666387438774, 0.008607023395597935, 0"
+            + ".0013598125660791993, 0.010101927444338799, -0.0011341209756210446, 0.00411583948880434, 0"
+            + ".01181038934737444, 0.00021173762797843665, -0.0013808448566123843, -0.013861837796866894, 0"
+            + ".011176187545061111, -0.006723832339048386, -0.0013994502369314432, -0.0058113583363592625, -0"
+            + ".005953730549663305, -0.0037243172992020845, -0.018314192071557045, -0.00193334452342242, -0"
+            + ".011862160637974739, 0.0027066823095083237, 0.013823009096086025, -0.0053551215678453445, -0"
+            + ".01287817768752575, 0.01121501624584198, 0.007086233235895634, -0.017123445868492126, -0"
+            + ".005581622011959553, -0.007066818885505199, 0.00024409485922660679, 0.013525322079658508, -0"
+            + ".024850353598594666, -0.027801332995295525, 0.004371461924165487, -0.02008736878633499, 0"
+            + ".01918136700987816, 0.012949363328516483, -0.013667694292962551, 0.006510274950414896, -0"
+            + ".003643424017354846, 0.01126678753644228, 0.003403980517759919, -0.015065526589751244, -0"
+            + ".03463517874479294, -0.02324543334543705, 0.00326969800516963, -0.010787900537252426, -0"
+            + ".010347842238843441, -0.00946772564202547, 0.014457210898399353, -0.02527746744453907, 0"
+            + ".02341369166970253, 0.0035819453187286854, -0.017900019884109497, -0.004827698692679405, 0"
+            + ".00445559062063694, 0.013525322079658508, -0.013234106823801994, 0.015065526589751244, 0"
+            + ".00026816054014489055, 0.00773337809368968, -0.01233457587659359, 0.003009222447872162, -0"
+            + ".01496198307722807, -0.0028344932943582535, 0.006749718450009823, 0.019841453060507774, -0"
+            + ".00646173907443881, -0.02886264957487583, -0.024177322164177895, -0.013117620721459389, -0"
+            + ".002773014595732093, 0.024863295257091522, 0.19590361416339874, 0.009144153445959091, -0"
+            + ".0008736452437005937, 0.045791953802108765, 0.012127489782869816, 0.012101603671908379, 0"
+            + ".009021195583045483, 0.0149231543764472, -0.006749718450009823, 0.031528886407613754, 0"
+            + ".003030254505574703, 0.008102250285446644, -0.009092382155358791, 0.00015268567949533463, -0"
+            + ".004678855650126934, -0.024047894403338432, -0.021083971485495567, -0.030312253162264824, -0"
+            + ".023607835173606873, -0.03567061200737953, -0.01436660997569561, 0.011305616237223148, -0"
+            + ".006549103185534477, -0.013693579472601414, 0.03787090256810188, 0.006730304099619389, -0"
+            + ".0023944349959492683, -0.0047112125903368, 0.03359974920749664, -0.00479534175246954, -0"
+            + ".00189936941023916, -0.0004400583275128156, -0.0063193668611347675, 0.0013735644752159715, 0"
+            + ".003776088822633028, -0.0008542308933101594, -0.011803917586803436, -0.023439576849341393, 0"
+            + ".019440224394202232, -0.0037049029488116503, -0.021187514066696167, -0.011758617125451565, -0"
+            + ".002996279625222087, 0.0013169392477720976, 0.002907297108322382, 0.003245430300012231, 0"
+            + ".00034925586078315973, 0.00038788228994235396, 0.020048540085554123, 0.023931408300995827, -0"
+            + ".013900666497647762, -0.014612524770200253, 0.019608480855822563, 0.0303640253841877, -0"
+            + ".006917975842952728, 0.019466109573841095, 0.01822359301149845, 0.0037243172992020845, -0"
+            + ".010477270931005478, -0.01274227723479271, -0.005102735012769699, 0.040951311588287354, -0"
+            + ".011525645852088928, 0.02463032305240631, -0.0008493773057125509, 0.005063906311988831, -0"
+            + ".007765735499560833, -0.010580814443528652, 0.009053553454577923, -0.02764601819217205, 0"
+            + ".014974926598370075, -0.030415795743465424, -0.0025254818610846996, 0.012036889791488647, -0"
+            + ".016916360706090927, -0.006769132800400257, 0.02209351770579815, 0.019776739180088043, 0"
+            + ".0052774641662836075, 0.03743084520101547, -0.017058731988072395, -0.0010613171616569161, -0"
+            + ".0007567547145299613, -0.009797769598662853, 0.004468533676117659, -0.03605889901518822, -0"
+            + ".0005945641314610839, 0.0012902445159852505, 0.0014722539344802499, 0.006775604095309973, 0"
+            + ".019259022548794746, -0.013913609087467194, 0.007468048948794603, -0.003462223568931222, 0"
+            + ".01849539205431938, 0.03290083259344101, -0.015065526589751244, 0.028759106993675232, -0"
+            + ".009124739095568657, 7.280377030838281e-05, -0.023193662986159325, -0.03378094732761383, 0"
+            + ".012121018022298813, 0.0034913450945168734, -0.005329235922545195, -0.014004209078848362, 0"
+            + ".007746321149170399, -0.007817506790161133, 0.008930595591664314, -0.00823167897760868, 0"
+            + ".005067142192274332, -0.0015321148093789816, -0.0002521841670386493, 0.011700374074280262, 0"
+            + ".00343957357108593, -0.00014985442976467311, -0.01057434268295765, -0.012483419850468636, 0"
+            + ".003504288149997592, 0.005041256546974182, 0.013499436900019646, -0.0241902656853199, -0"
+            + ".010684357024729252, 0.01182333193719387, -0.029587451368570328, -0.023426635190844536, -0"
+            + ".016786931082606316, 0.0051739211194217205, 0.009558325633406639, -0.03577415272593498, 0"
+            + ".014457210898399353, 0.005474843550473452, 0.0173823032528162, 0.005468371789902449, -0"
+            + ".006186702288687229, -0.008386993780732155, 0.0007887074607424438, 0.02798253297805786, -0"
+            + ".02710241638123989, 0.019530823454260826, -0.010600228793919086, -0.02582106925547123, 0"
+            + ".004206439945846796, 0.00326969800516963, -0.004733862821012735, -0.008762338198721409, 0"
+            + ".011525645852088928, -0.014301896095275879, -0.0353858657181263, -0.02008736878633499, -0"
+            + ".015867985785007477, -0.021692287176847458, 0.007280376739799976, 0.009040609933435917, 0"
+            + ".01223103329539299, -0.04297040030360222, -0.043384574353694916, -0.034169234335422516, 0"
+            + ".015803271904587746, 0.02818961814045906, -0.034816380590200424, 0.003921696450561285, 0"
+            + ".03846627473831177, -0.0061025735922157764, -0.01646335795521736, 0.0006079114391468465, -0"
+            + ".165461927652359, 0.02029445394873619, 0.008328750729560852, -0.0025934320874512196, 0"
+            + ".012988192029297352, -0.0010710243368521333, 0.02950979396700859, 0.01572561450302601, -0"
+            + ".011732731945812702, 0.005966673139482737, 0.03082996979355812, -0.0022698596585541964, -0"
+            + ".037016671150922775, -0.004119075369089842, 0.014133637771010399, -0.012852291576564312, -0"
+            + ".0028215504717081785, 0.014677239581942558, 0.010548457503318787, 0.008374051190912724, 0"
+            + ".02883676439523697, -0.00891118124127388, 0.028137847781181335, -0.02002265490591526, 0"
+            + ".0020676269195973873, 0.01302702073007822, 0.022455917671322823, 0.018780136480927467, -0"
+            + ".010017798282206059, -0.0010653617791831493, -0.004898884799331427, -0.011234430596232414, 0"
+            + ".021083971485495567, 0.016657501459121704, -0.008755866438150406, -0.0033327946439385414, -0"
+            + ".004177318420261145, 0.020100312307476997, -0.013266464695334435, 0.006807961035519838, 0"
+            + ".017511732876300812, 0.012638733722269535, 0.028655562549829483, -0.006982690189033747, -0"
+            + ".02653292939066887, 0.014172466471791267, 0.00811519380658865, -0.024306751787662506, -0"
+            + ".01975085400044918, -0.010354313999414444, 0.012004532851278782, -0.029017964377999306, -0"
+            + ".012496362440288067, 0.014444267377257347, 0.011894517578184605, 0.0030334903858602047, -0"
+            + ".00038080415106378496, 0.006008737720549107, 0.014625468291342258, 0.012968777678906918, -0"
+            + ".0035463524982333183, -0.005232164170593023, 0.01859893649816513, 0.004737098701298237, -0"
+            + ".016644559800624847, -0.025199810042977333, -0.008555252104997635, 0.005044492427259684, -0"
+            + ".013007606379687786, 0.017874134704470634, -0.02096748538315296, -0.0018475978868082166, -0"
+            + ".013208221644163132, -0.026455271989107132, 0.009868955239653587, 0.03442809358239174, -0"
+            + ".014871383085846901, 0.009610097855329514, 0.01426306739449501, -8.205591439036652e-05, -0"
+            + ".001627568737603724, 0.012192204594612122, -0.02971688099205494, 0.004788869991898537, -0"
+            + ".015039640478789806, 0.00997896958142519, -0.018961336463689804, 0.003753438824787736, 0"
+            + ".0043488116934895515, -0.017641162499785423, 0.03380683436989784, -0.016890473663806915, -0"
+            + ".011473873630166054, -0.002499595982953906, 0.006066980771720409, 0.009881897829473019, 0"
+            + ".002724478719756007, 0.015065526589751244, -0.008509951643645763, -0.014444267377257347, 0"
+            + ".007940464653074741, -0.01769293285906315, -0.02127811498939991, 0.004468533676117659, 0"
+            + ".02741304598748684, 0.029742766171693802, 0.003403980517759919, 0.017679991200566292, 0"
+            + ".024591494351625443, 0.011816860176622868, 0.007299791090190411, 0.01121501624584198, 0"
+            + ".025536326691508293, 0.015816213563084602, -0.00438764039427042, 0.02012619748711586, 0"
+            + ".004099661018699408, 0.013590036891400814, 0.005183628294616938, 0.005160978063941002, 0"
+            + ".02883676439523697, 0.004264682997018099, -9.242034138878807e-05, 0.01704578846693039, -0"
+            + ".007642777636647224, -0.004979777615517378, -0.10121341794729233, -0.012082190252840519, 0"
+            + ".021860545501112938, 0.029690993949770927, -0.013952437788248062, 0.0320466011762619, 0"
+            + ".010820257477462292, 0.0048827058635652065, -0.023970237001776695, 0.01575149968266487, -0"
+            + ".00946772564202547, -0.02873321995139122, 0.0029995152726769447, -0.023025404661893845, 0"
+            + ".03409157693386078, 0.015324383974075317, -0.015932699665427208, 0.006342017091810703, -0"
+            + ".003801974467933178, 0.017874134704470634, 0.004853584337979555, -0.004830934572964907, -0"
+            + ".009027667343616486, -0.007578063290566206, -0.0010386670473963022, -0.007565120235085487, -0"
+            + ".019543766975402832, 0.0218864306807518, 0.0019495231099426746, 0.01761527545750141, 0"
+            + ".014004209078848362, 0.002963922219350934, 0.0008574665989726782, -0.045791953802108765, -0"
+            + ".0023669314105063677, 0.004669148474931717, -0.01941433735191822, -0.0011106618912890553, 0"
+            + ".001112279831431806, -0.015091411769390106, -0.003130562137812376, 0.014301896095275879, 0"
+            + ".01214690413326025, -0.03530820831656456, 0.034376323223114014, -0.04967482015490532, -0"
+            + ".03541175276041031, 0.004630319774150848, 0.01236693374812603, -0.01758939027786255, -0"
+            + ".007688078097999096, 0.006047566421329975, -0.025393953546881676, 0.0056042722426354885, 0"
+            + ".017667047679424286, -0.0012344283750280738, 0.00757159199565649, -0.0032502838876098394, -0"
+            + ".02527746744453907, -0.010652000084519386, -0.022378260269761086, -0.010645528323948383, 0"
+            + ".002430028049275279, 0.0195825956761837, 0.01612684316933155, -0.00029121508123353124, -0"
+            + ".012800520285964012, -0.025225697085261345, 0.01670927368104458, -0.033936262130737305, 0"
+            + ".001469018287025392, -0.017770590260624886, -0.01910370960831642, 0.002533571096137166, -0"
+            + ".013370007276535034, -0.015635013580322266, -0.0042711542919278145, -0.01546675618737936, 0"
+            + ".012185732834041119, 0.003701667068526149, -0.008878824301064014, -0.0351787805557251, -0"
+            + ".005575150717049837, -0.006584696471691132, 0.018132992088794708, 0.010742600075900555, 0"
+            + ".013848894275724888, 0.00773337809368968, -0.009616568684577942, -0.0280601903796196, 0"
+            + ".019440224394202232, 0.0023895814083516598, 0.005141563713550568, -0.026584699749946594, -0"
+            + ".019297851249575615, 0.020385054871439934, 1.1445105883467477e-05, -0.0014908594312146306, 0"
+            + ".013590036891400814, -0.009940140880644321, -0.00803106464445591, -0.012632262893021107, -0"
+            + ".046464983373880386, 0.016748102381825447, -0.0008696005679666996, -0.006189938168972731, 0"
+            + ".025562211871147156, -0.004060832317918539, -0.002705064369365573, -0.0069373901933431625, -0"
+            + ".0016777224373072386, 0.01795179210603237, -0.022688889876008034, 0.006134930998086929, 0"
+            + ".0009933669352903962, 0.00017836922779679298, -0.004303511697798967, -0.029535679146647453, 0"
+            + ".025225697085261345, -0.0010030741104856133, 0.010554928332567215, 0.012153375893831253, 0"
+            + ".014949040487408638, 0.014444267377257347, 0.014250123873353004, 0.0014787254622206092, -0"
+            + ".010904386639595032, 0.0029558329842984676, -0.005358357448130846, 0.040433596819639206, -0"
+            + ".017744705080986023, -0.01799062080681324, 0.010820257477462292, -0.022779490798711777, -0"
+            + ".0018136227736249566, 0.017162274569272995, -0.01025077048689127, -0.018353020772337914, -0"
+            + ".006542631890624762, 0.005924609024077654, 0.01704578846693039, -0.0001178005404653959, -0"
+            + ".023840807378292084, -0.0065329247154295444, 0.03574826940894127, -0.0005876881768926978, -0"
+            + ".012133961543440819, 0.002150137908756733, -0.007591006346046925, 0.007079761940985918, 0"
+            + ".008548780344426632, 0.0008574665989726782, 0.037353187799453735, 0.01742113195359707, 0"
+            + ".015285555273294449, -0.023387806490063667, -0.012800520285964012, -0.014288952574133873, 0"
+            + ".013305293396115303, 0.0003504692576825619, 0.00816696509718895, -0.02008736878633499, 0"
+            + ".0017667047213762999, 0.030881740152835846, -0.01302702073007822, -0.00582106551155448, -0"
+            + ".016515130177140236, -0.004934477619826794, -0.012580491602420807, -0.020281512290239334, 0"
+            + ".00553955789655447, -0.05446368828415871, -0.01190746109932661, -0.0008158066775649786, 0"
+            + ".019983826205134392, -0.0036660742480307817, 0.016968131065368652, -0.0066785323433578014, 0"
+            + ".0019204015843570232, -0.012457533739507198, -0.0053195287473499775, 0.032072488218545914, 0"
+            + ".009157096035778522, 0.012774634175002575, -0.02548455446958542, 0.015932699665427208, 0"
+            + ".026338785886764526, 0.03831095993518829, -0.02950979396700859, 0.016994018107652664, 0"
+            + ".00551367225125432, 0.0033780948724597692, -0.02270183339715004, -0.005005663726478815, 0"
+            + ".0210451427847147, -0.024164380505681038, 0.011693903245031834, 0.04058890789747238, -0"
+            + ".01656690239906311, 0.018275363370776176, -0.003336030524224043, 0.010341370478272438, 0"
+            + ".02117457240819931, 0.004086717963218689, -0.025199810042977333, -0.02852613478899002, -0"
+            + ".021627573296427727, -0.02287008985877037, -0.0318395160138607, -0.02337486296892166, -0"
+            + ".01381006557494402, 0.012069246731698513, 0.0162303876131773, -0.007882221601903439, 0"
+            + ".011234430596232414, 0.025458669289946556, -0.030182823538780212, 0.0210451427847147, -0"
+            + ".019336679950356483, -0.03838861733675003, 0.004530012141913176, 0.010211941786110401, 0"
+            + ".042711544781923294, 0.0055686794221401215, 0.03178774192929268, 0.008056950755417347, 0"
+            + ".0007288465858437121, 0.0068338471464812756, 0.01047080010175705, -0.01463841088116169, -0"
+            + ".003911989275366068, 0.019673196598887444, -0.009377125650644302, -0.004973306320607662, 0"
+            + ".003627245547249913, -0.021200457587838173, -0.0007296555559150875, -0.01060670055449009, 0"
+            + ".036809585988521576, 0.04794047400355339, -0.020268568769097328, 0.03655072674155235, -0"
+            + ".011059701442718506, 0.007248019799590111, -0.01260637678205967, -0.0026079928502440453, 0"
+            + ".00228765606880188, -0.0036498955450952053, 0.0039864107966423035, -0.012845820747315884, -0"
+            + ".022287659347057343, -0.002739039482548833, 0.003133797785267234, 0.01761527545750141, -0"
+            + ".02839670516550541, -0.010412557050585747, 0.004322926048189402, -0.013149978592991829, -0"
+            + ".012353990226984024, -0.011525645852088928, 0.015013755299150944, 0.0022601524833589792, 0"
+            + ".013253521174192429, 0.002197055844590068, 0.013421779498457909, -0.010833200998604298, -0"
+            + ".0075133489444851875, 0.020475655794143677, -0.004141725599765778, -0.006164052523672581, -0"
+            + ".03717198595404625, 0.0025659282691776752, 0.011363859288394451, -0.04061479493975639, -0"
+            + ".024034950882196426, 0.006477917544543743, 0.015156126581132412, -0.017058731988072395, -0"
+            + ".0019204015843570232, 0.029147392138838768, 0.01043844223022461, -0.022947747260332108, 0"
+            + ".02435852214694023, -0.007791621144860983, -0.005927844438701868, 0.006183466874063015, -0"
+            + ".020385054871439934, -0.001414010999724269, -0.018391849473118782, -0.004025239497423172]";
+    String q6 =
+        "select ProductId, UserId, l2_distance(embedding, " + vectorArrayLiteral + ") as l2_dist, n_tokens, combined "
+            + "from fineFoodReviews "
+            + "order by l2_dist ASC "
+            + "limit 10";
+    printStatus(Quickstart.Color.YELLOW, "Search the most relevant review with the embedding of tomato soup");
+    printStatus(Quickstart.Color.CYAN, "Query : " + q6);
+    printStatus(Quickstart.Color.YELLOW, prettyPrintResponse(runner.runQuery(q6)));
+    printStatus(Quickstart.Color.GREEN, "***************************************************");
+
+    String q7 =
+        "select ProductId, UserId, l2_distance(embedding, " + vectorArrayLiteral + ") as l2_dist, n_tokens, combined "
+            + "from fineFoodReviews "
+            + "where VECTOR_SIMILARITY(embedding," + vectorArrayLiteral + ", 5) "
+            + "order by l2_dist ASC "
+            + "limit 5";
+    printStatus(Quickstart.Color.YELLOW,
+        "Search the top 5 most relevant review with the embedding of tomato soup using HNSW index");
+    printStatus(Quickstart.Color.CYAN, "Query : " + q7);
+    printStatus(Quickstart.Color.YELLOW, prettyPrintResponse(runner.runQuery(q7)));
+    printStatus(Quickstart.Color.GREEN, "***************************************************");
+  }
 }
diff --git a/pinot-tools/src/main/java/org/apache/pinot/tools/Quickstart.java b/pinot-tools/src/main/java/org/apache/pinot/tools/Quickstart.java
index e134cf0bf3..4e528b33b4 100644
--- a/pinot-tools/src/main/java/org/apache/pinot/tools/Quickstart.java
+++ b/pinot-tools/src/main/java/org/apache/pinot/tools/Quickstart.java
@@ -138,207 +138,7 @@ public class Quickstart extends QuickStartBase {
     printStatus(Color.YELLOW, prettyPrintResponse(runner.runQuery(q5)));
     printStatus(Color.GREEN, "***************************************************");
 
-    String q6 =
-        "select ProductId, UserId, l2_distance(embedding, ARRAY[-0.0013143676,-0.011042999,-0.015155246,-0"
-            + ".0085545285,-0.0030792344,0.0115776565,0.000080961916,-0.042006962,-0.019419309,-0.016145352,0"
-            + ".028119052,0.002938969,-0.025373155,-0.028990347,-0.025663586,0.030178476,0.024805494,-0.0019752644,0"
-            + ".0016897834,-0.012541361,0.004617201,0.0038911225,0.023234522,0.006405995,-0.00057591253,-0.009439025,"
-            + "0.010257513,-0.01675262,-0.0031254394,0.006254179,0.012911001,-0.018759236,-0.026785707,-0.0065413103,"
-            + "-0.0036237934,0.0032541533,-0.014521576,-0.012382944,0.02459427,-0.034429338,0.021478731,-0"
-            + ".00054950966,0.008224493,-0.015168447,-0.02154474,0.0039043238,0.0062310765,-0.006435699,0.0024703182,"
-            + "-0.003976932,0.043881565,0.013320246,-0.00006471796,-0.00064687023,0.016607404,0.011128808,-0"
-            + ".011452244,0.0031023368,0.023274127,-0.028594304,0.017531503,0.016924238,-0.04385516,0.0040297373,-0"
-            + ".01611895,-0.011042999,-0.022231214,-0.004438982,0.02155794,-0.008811956,0.019749343,0.042218182,-0"
-            + ".015062835,0.00083870353,0.028435888,-0.021848371,-0.013082621,0.007049565,0.02191438,-0.011089204,0"
-            + ".010488539,-0.027036535,-0.011016596,0.033214808,0.014547979,0.014165138,-0.006151868,0.042059764,-0"
-            + ".010825175,-0.015062835,0.024858298,0.0071815797,0.01225093,0.013142027,-0.033029985,0.020132186,0"
-            + ".007947262,0.035063006,-0.0016790573,-0.02975603,-0.0047657173,-0.0129968105,0.0059901504,-0.01685823,"
-            + "-0.025346752,-0.006844943,0.019049669,0.022350026,0.019551322,0.0054389904,-0.0070627667,0.017439093,"
-            + "-0.0010602401,-0.02377578,-0.013861504,-0.010534744,0.03363725,-0.038468976,-0.01860082,-0.0012772387,"
-            + "0.0030049763,0.020871466,0.020198192,-0.026627291,0.0066370205,0.0041122464,-0.03049531,-0.0068779467,"
-            + "0.027379772,-0.010310319,0.03324121,0.023683371,-0.002339954,0.007425806,-0.0060099526,0.016039742,-0"
-            + ".0359343,0.008178288,0.010026488,-0.009505032,0.012323538,0.016620604,-0.00866014,-0.006320186,-0"
-            + ".017927546,0.018798841,-0.00635649,0.039498687,0.0021138794,0.0074390075,0.013901109,-0.02165035,-0"
-            + ".0043069674,0.002587481,0.028356679,0.021584343,0.005386185,0.021742761,-0.02238963,-0.0042013563,0"
-            + ".015155246,-0.002570979,-0.018019956,-0.0063036843,0.0130496165,0.014415965,0.043327104,-0.016818626,"
-            + "-0.00021452329,-0.013716289,-0.006405995,0.019472115,-0.035775885,0.00948523,0.0050528487,-0"
-            + ".005396086,0.0075380183,0.02228402,-0.03350524,-0.016514992,-0.009934079,0.011036398,0.023551356,0"
-            + ".02312891,-0.009802064,-0.03794092,0.003703002,-0.005703019,0.0038812214,-0.009973682,-0.005950546,0"
-            + ".024845097,0.0014414315,-0.008250896,-0.6353586,-0.03556466,0.001391101,-0.03860099,0.03765049,0"
-            + ".029597614,0.009868071,-0.005287174,0.0044158795,-0.003739306,-0.015762512,0.029518405,-0.009775661,-0"
-            + ".0146535905,0.031049771,-0.030257685,0.04578257,-0.0023284028,-0.007075968,0.02044902,-0.008184888,-0"
-            + ".0023713075,-0.010495139,-0.021373121,0.028251067,0.004072642,0.027643802,-0.012270732,-0.0019010063,0"
-            + ".009604042,-0.021518337,0.015326864,0.029254375,0.016184958,0.052330483,-0.0061188643,-0.009148593,-0"
-            + ".0054950966,0.017795531,0.030996965,-0.041426096,-0.010574348,0.027274162,-0.016620604,-0.012957207,0"
-            + ".010165104,0.028171858,0.0068515437,-0.0033383125,-0.008429115,0.02183517,0.006465402,-0.040317178,-0"
-            + ".013663483,0.019274093,-0.005353181,0.044462427,-0.014033124,-0.0046403036,0.0004826774,-0.006399395,0"
-            + ".011590858,-0.018310389,-0.030970562,-0.008448917,0.011769078,-0.019089272,-0.004033038,0.02275927,-0"
-            + ".010019888,-0.000750419,0.026521679,-0.010151902,-0.009359816,0.028039845,0.01668661,0.012198124,0"
-            + ".016066143,0.008125482,0.002726096,0.023340134,-0.022165205,0.020884667,-0.039551493,0.03527423,0"
-            + ".009326813,-0.013953915,0.008336705,-0.013808699,-0.028805528,0.0017788932,0.018785639,0.0028614106,-0"
-            + ".034323726,-0.014746001,-0.012633772,-0.02532035,0.0036237934,0.0026270852,-0.019551322,-0.0031254394,"
-            + "0.006468702,0.023287328,0.021689955,-0.00047648922,0.010884582,-0.030284088,0.015115641,0.033214808,-0"
-            + ".023260925,0.0059934505,-0.010013287,-0.0027425978,-0.04087164,-0.0066007166,-0.017359884,0.009188198,"
-            + "-0.0085545285,-0.009201399,-0.003019828,0.008726147,-0.0015132143,-0.007953864,0.014561181,0"
-            + ".018046359,0.01887805,0.0133400485,-0.003630394,-0.022152005,-0.010495139,-0.0045742965,0.02458107,-0"
-            + ".001815197,-0.026204845,0.0017392888,0.023155313,0.012567764,-0.0132608395,0.0058086305,-0.005630411,0"
-            + ".019168481,-0.009075985,0.012099113,-0.015036432,-0.019313697,-0.025095925,-0.008363108,0.009478629,-0"
-            + ".01704305,-0.0018845046,0.0028944141,-0.011590858,-0.0006802864,0.016052943,-0.01040273,-0.031551424,"
-            + "-0.001594073,-0.045993794,-0.00930701,0.00764363,0.0067558335,0.04670667,-0.0013671734,0.009571039,-0"
-            + ".002679891,0.0014719598,-0.008580931,0.011689869,-0.009036381,-0.027036535,0.007122173,-0.031181784,-0"
-            + ".0018085963,0.04044919,-0.015551289,-0.0029983756,-0.018904453,-0.015128843,0.01685823,0.0066436213,0"
-            + ".009009978,0.00022751845,-0.010072693,-0.003445574,0.022046393,-0.013452261,0.0014331805,0.03234351,-0"
-            + ".012534761,0.033742864,0.012911001,0.024198227,0.010864779,0.006528109,-0.009174996,-0.02091107,0"
-            + ".004709611,0.021967186,0.011274024,-0.012567764,0.021227904,0.0006778111,0.005488496,-0.013650282,0"
-            + ".031076174,-0.0146271875,0.005538001,-0.013307044,0.008145284,0.037544876,0.008831759,-0.008633737,-0"
-            + ".014191541,-0.0018267484,0.007075968,0.02201999,0.018046359,0.005049548,-0.020052977,-0.018191574,0"
-            + ".0005932394,0.0015767461,-0.008316902,-0.014957224,-0.02377578,0.0065710135,-0.00045132398,0"
-            + ".012858196,-0.007663432,-0.018838445,0.0046006995,0.014851612,0.013300444,0.02819826,0.018349992,-0"
-            + ".0008003369,0.012772387,-0.004838325,0.022072796,-0.004392777,0.0032261002,0.011300427,0.045254514,0"
-            + ".000018358243,0.0067855367,-0.017518302,0.028488692,0.0011831784,-0.004650205,0.0030462306,-0"
-            + ".010085895,-0.010013287,-0.028620707,-0.008653539,0.010429132,0.017557906,-0.019788949,-0.012508358,0"
-            + ".02938639,0.023630565,0.00755122,-0.0018960559,-0.006871346,0.017676719,0.011663466,-0.014983627,-0"
-            + ".014547979,-0.012950606,-0.019379703,0.008409313,-0.015630497,-0.017755928,0.019221287,-0.023379738,0"
-            + ".005554503,0.003976932,-0.006844943,0.026825313,-0.028224664,-0.014508375,-0.03720164,-0.024369845,0"
-            + ".019696537,0.01675262,0.01778233,0.02633686,0.0031072872,0.014112332,0.0006584215,-0.002432364,0"
-            + ".0174919,-0.006924152,0.0016138752,0.030917756,0.0015156895,0.017557906,0.0088713635,-0.028251067,-0"
-            + ".00039439282,0.013274041,0.017254272,-0.018310389,-0.0035148815,0.008488522,0.032713152,-0.0022112401,"
-            + "-0.024765888,-0.00755122,-0.0026501878,-0.018825244,0.0041485503,0.005607309,-0.013003412,-0.03556466,"
-            + "0.023709774,0.0076898346,-0.015696503,-0.012013304,0.010627153,0.01298361,0.005369683,-0.03825775,-0"
-            + ".0008230269,0.02745898,0.082060106,0.013432458,-0.007108972,0.008646939,0.029465599,-0.011102405,-0"
-            + ".027485384,-0.009069385,-0.005600708,-0.005013244,-0.01695064,0.00045091144,0.0064522005,0.002506622,"
-            + "-0.013676684,-0.005970348,-0.0099472795,0.0026105833,-0.008396111,-0.009062784,0.0024389648,0"
-            + ".00985487,0.00017460958,-0.007201382,0.03923466,-0.006376292,0.01889125,0.041003652,0.029544808,-0"
-            + ".015274058,-0.00967005,-0.0018861548,0.0035049806,-0.010455535,-0.011379635,-0.0011625512,0.033188403,"
-            + "0.018165171,0.010930787,-0.0051848628,0.004752516,0.024792291,0.011947297,-0.0074390075,-0.0088449605,"
-            + "-0.006392794,-0.0012722882,0.030468907,-0.010085895,-0.015788915,0.026086032,-0.0013407705,-0"
-            + ".014112332,-0.008594133,0.019379703,0.0068515437,-0.026363263,0.011848286,-0.033558045,-0.003851518,0"
-            + ".009069385,-0.009155194,-0.009201399,-0.0025181733,-0.013221235,-0.04301027,-0.011485247,0.0033383125,"
-            + "-0.017650316,-0.00672283,0.022033192,0.00022029891,-0.010528143,0.00060272793,0.029043153,0.010666758,"
-            + "-0.013181631,0.0020759255,0.004686509,0.015340066,0.0052541704,-0.032554734,0.007174979,-0.007174979,"
-            + "-0.008026471,0.017373087,0.020739451,-0.007940662,0.0050924527,0.022270817,0.008541327,0.024211429,0"
-            + ".007168378,0.004864728,-0.0012178322,0.0051518595,0.0144687705,0.012884599,0.0061089634,-0.01059415,-0"
-            + ".0134984655,0.024937507,-0.005234368,-0.015049634,0.002311901,0.003805313,-0.012653573,-0.0008106505,0"
-            + ".0026056329,-0.029650418,0.04844926,0.00008384973,-0.01335985,-0.019406106,0.011491847,0.02525434,-0"
-            + ".008283899,-0.0013712989,-0.024237832,-0.017729525,-0.034772575,-0.021610746,0.013056218,0.006485204,"
-            + "-0.0044224802,0.014389562,0.012600768,-0.029597614,-0.017795531,-0.022363227,-0.0042013563,-0"
-            + ".00433007,-0.022970494,0.0032624041,-0.018587617,-0.017993553,-0.0033762665,0.021676753,0.01225753,-0"
-            + ".013320246,-0.023089306,0.018006755,-0.022785673,-0.011432441,0.004188155,-0.027881427,-0.0071881805,"
-            + "-0.022072796,-0.010303719,0.009953881,0.018534811,0.0033861676,0.0037789103,0.01639618,0.003693101,-0"
-            + ".02782862,-0.015696503,-0.0049505373,0.016699813,0.032290705,0.030020058,-0.014336756,-0.008693144,0"
-            + ".015617295,-0.005445591,-0.0017673419,-0.0018432501,-0.0105017405,0.009505032,0.0147856055,0.03160423,"
-            + "0.014904418,-0.00396043,0.00598685,0.022350026,-0.006003352,0.0046931095,-0.032211497,-0.013531469,-0"
-            + ".029043153,0.0285679,-0.022996897,-0.018455604,0.02017179,0.020686645,0.0029719726,0.04359113,0"
-            + ".0044752858,0.018548014,-0.0047030104,0.03519502,0.017095856,-0.0053564813,-0.01889125,-0.005739323,-0"
-            + ".025003515,0.00023597562,-0.027643802,-0.030706534,0.02192758,-0.0003046643,0.010541344,0.02525434,0"
-            + ".002569329,-0.004874629,0.016462186,-0.019881358,-0.010112298,-0.0021963886,-0.018957257,-0.0082905,-0"
-            + ".007887856,-0.043195087,-0.021399522,-0.005141958,-0.023538155,-0.01262717,0.0058383336,-0.024752688,"
-            + "-0.03572308,0.0010354874,-0.008330104,0.030680131,0.011069402,0.009049582,0.020977078,-0.013003412,-0"
-            + ".0054290895,-0.013980318,0.00773604,-0.019564524,0.0018201476,0.032396317,0.0121057145,-0.038310558,-0"
-            + ".0043564728,-0.0076502305,0.0117624765,-0.002072625,0.01631697,0.01279879,0.014217944,-0.022891285,-0"
-            + ".008580931,-0.023709774,0.009709654,-0.006105663,0.020145386,-0.0026947425,-0.012699779,-0.0063795927,"
-            + "0.037439264,-0.037333652,0.021161897,0.004670007,-0.010924186,-0.016792223,-0.029175168,-0.004557795,0"
-            + ".012587567,0.0063795927,0.0033828672,-0.009128791,0.039762717,0.022917688,-0.010138701,0.001454633,0"
-            + ".011742675,0.0071881805,0.02562398,0.022534847,-0.032211497,0.023590961,-0.014244346,0.011538053,-0"
-            + ".03530063,-0.01354467,0.006844943,-0.017148662,-0.020515027,0.008759151,-0.0016650307,-0.015854921,-0"
-            + ".017241072,0.012277333,-0.012600768,-0.012198124,-0.026548082,0.005326778,-0.010990193,-0.023208119,-0"
-            + ".00023680071,0.0006023154,-0.010620553,0.018957257,-0.0020610737,0.0033746164,0.027247759,-0"
-            + ".008547928,0.021742761,0.007049565,-0.008356507,-0.006772335,0.007194781,-0.020581035,-0.01861402,0"
-            + ".012792189,-0.011986901,-0.02330053,-0.022970494,0.004874629,0.009168396,0.008785554,-0.00080446235,0"
-            + ".008442316,0.007174979,0.023524953,-0.006336688,-0.01225753,0.0029356687,-0.023683371,-0.002617184,0"
-            + ".030891353,-0.0073531982,0.007023162,0.031445812,0.013584275,-0.00783505,0.009788862,-0.008924169,-0"
-            + ".03242272,0.024541464,-0.0119737,-0.015221252,-0.023815386,-0.007709637,-0.016739417,0.0019934163,0"
-            + ".0037492071,-0.025808802,0.0156569,0.043036673,-0.0007768218,0.008805356,0.005353181,-0.02165035,-0"
-            + ".034693368,0.009795464,0.0034092702,-0.011300427,-0.0037162034,0.011637064,-0.008396111,-0.03260754,-0"
-            + ".008435716,-0.011650265,-0.039657105,-0.016158555,-0.030891353,-0.006798738,0.018363193,-0.017425891,"
-            + "-0.011742675,0.0506671,-0.010990193,0.020581035,0.015590892,-0.009214601,-0.0014934121,0.010739366,0"
-            + ".01713546,-0.0027855025,-0.010825175,-0.0041386494,0.007907659,0.0037690091,0.009775661,0.0006827616,0"
-            + ".0040231366,0.019458912,-0.01510244,-0.014693195,-0.008580931,-0.008864762,0.022587651,-0.008719547,-0"
-            + ".0043531726,0.008613935,0.014178339,-0.002782202,-0.009518233,0.0010627153,0.007762443,0.023841789,-0"
-            + ".027775815,-0.009544636,-0.01262717,-0.012336739,0.006957155,0.014151936,-0.016224561,0.006891148,-0"
-            + ".026099233,-0.021874774,-0.01372949,-0.00837631,-0.029175168,-0.03186826,0.014812008,-0.024726285,-0"
-            + ".002356456,0.0013432459,0.016594201,0.0144423675,-0.015247655,-0.0156701,-0.0076766335,-0.017109057,-0"
-            + ".006336688,0.0020742752,-0.009399421,-0.006650222,-0.004957138,0.011274024,0.0054587927,-0.008244295,"
-            + "-0.00071700284,-0.022561248,-0.022112401,-0.0051287566,-0.022363227,-0.047868397,-0.011386236,-0"
-            + ".020250998,-0.016224561,0.026231248,0.20298524,0.011643664,-0.0029521706,0.056185298,0.033188403,0"
-            + ".004901032,0.010798772,0.01510244,-0.031049771,0.014191541,0.028752722,0.023419343,-0.018825244,-0"
-            + ".006501706,-0.008145284,-0.04649545,-0.03305639,-0.021901177,-0.010250913,-0.03508941,-0.008726147,-0"
-            + ".0043564728,0.003917525,-0.003445574,0.021769164,0.0075116153,-0.001519815,0.0010973691,0.026270851,-0"
-            + ".008052874,-0.007993468,-0.009795464,0.017003445,0.0022590952,0.0067921374,0.0016732817,-0.00838291,-0"
-            + ".009703053,0.017069453,-0.0008943971,-0.012369743,-0.013940713,0.010026488,-0.007148576,-0.020277401,0"
-            + ".015247655,0.000884496,-0.0031023368,0.014930821,0.020858264,-0.02505632,-0.02542596,0.016581,0"
-            + ".016990244,0.009551237,0.021478731,0.014455569,0.0023201518,-0.023062903,-0.011386236,-0.011128808,0"
-            + ".037069622,0.0013292193,0.028224664,-0.005683217,0.0029802236,-0.011254222,-0.0075050145,0.011643664,"
-            + "-0.013505066,0.017148662,-0.02200679,-0.007571022,-0.004498388,-0.02274607,-0.0076832343,0.02790783,0"
-            + ".01372949,0.009505032,0.03049531,-0.001925759,0.0004139887,-0.022178408,-0.00052846986,-0.013029815,-0"
-            + ".027300565,0.021478731,-0.007432407,-0.007148576,0.0025016717,0.019445712,-0.02644247,-0.0005177437,-0"
-            + ".013584275,0.015920928,0.007102371,-0.017993553,0.007201382,-0.010455535,-0.013280641,-0.0072541875,0"
-            + ".01409913,-0.0063894936,0.0034983798,-0.035696674,-0.0012648624,0.004561095,0.0016831828,0.0033960687,"
-            + "-0.017161863,-0.001694734,-0.01638298,-0.0230365,0.00018162285,0.008990176,0.038178544,-0.010283916,-0"
-            + ".02091107,0.011109006,-0.0056898175,0.02533355,-0.002229392,-0.008957173,0.018059561,-0.03371646,-0"
-            + ".012660175,0.0019571125,0.023828587,0.0060825604,-0.048211634,-0.0061485674,0.028251067,-0.0010478637,"
-            + "0.021016682,-0.0069109504,-0.012264132,0.0004579247,0.018178374,-0.019128876,0.0041980557,-0"
-            + ".025439162,-0.0043234695,0.012112315,-0.011953898,0.03382207,-0.010838376,0.0006811114,-0.0036435956,"
-            + "-0.030680131,-0.035907898,-0.0022161906,-0.0023053002,-0.0070957704,0.032554734,0.019762546,-0"
-            + ".02128071,-0.05137998,-0.019775746,0.0034488745,0.026772507,-0.0396043,0.0023086006,0.043485522,-0"
-            + ".005204665,-0.01188129,-0.0074192053,-0.16602123,0.023168515,0.0076502305,-0.009128791,0.013148627,0"
-            + ".005600708,0.02901675,0.010937387,0.00064150715,0.0007083394,0.023445746,0.0060462565,-0.0060462565,-0"
-            + ".023894593,-0.00045132398,-0.012297135,-0.024264233,0.008693144,0.012369743,0.0004123385,0.008686543,0"
-            + ".011194815,0.023696572,-0.021439128,0.0014744351,0.014349958,0.034666963,0.013663483,0.01759751,0"
-            + ".009531435,0.011755876,-0.0009645297,0.015604094,0.007174979,0.0072475867,-0.0018250982,-0.0022887986,"
-            + "-0.002681541,-0.0048845303,0.012382944,0.009095787,-0.01906287,0.017373087,-0.0021864874,0.0022310421,"
-            + "0.0037294049,-0.0029901245,-0.046733074,-0.0010519892,-0.01815197,0.031260993,-0.037835307,-0"
-            + ".034138907,0.015326864,-0.010297118,0.0044554835,0.003554486,0.01391431,0.0019092573,0.0052211666,-0"
-            + ".012369743,-0.014363159,0.00874595,0.009729456,-0.033584446,-0.019841755,-0.0004967039,-0.0009843318,"
-            + "-0.014402764,0.0043069674,-0.027934233,0.008963773,0.0072079827,0.024290636,0.005277273,0.02340614,-0"
-            + ".017650316,-0.0090561835,0.027802218,-0.0070693674,-0.012673376,-0.021043085,-0.056132495,0.02790783,"
-            + "-0.026825313,0.011214618,-0.015709706,-0.009835068,0.0044224802,0.0019356601,0.029676821,-0.006594116,"
-            + "-0.022614054,-0.0028449087,0.0033201603,0.034191713,0.0033696657,0.0008902716,-0.009201399,-0"
-            + ".008864762,0.013148627,0.006495105,-0.022891285,0.024818694,0.011274024,0.033135597,-0.007577623,0"
-            + ".0039670304,0.017214669,-0.028515095,-0.01695064,0.009498431,0.021254307,0.025175134,0.0057096197,0"
-            + ".027643802,0.004181554,-0.013861504,0.034138907,-0.013821901,0.01594733,0.0060924613,0.0002652663,0"
-            + ".0147856055,-0.0033350121,-0.009953881,-0.1077237,-0.022072796,0.010151902,0.02340614,-0.035168618,0"
-            + ".007293792,-0.011465444,0.004577597,-0.012468753,0.008699744,0.01206611,-0.033452433,0.013452261,-0"
-            + ".020132186,0.022336826,-0.007102371,-0.030653728,-0.01611895,-0.025637183,-0.005174962,-0.0021402824,"
-            + "-0.018442402,-0.0017359884,0.0040792427,0.012000103,-0.026323657,-0.005333379,-0.0029241175,0"
-            + ".021676753,0.022442436,0.0018085963,-0.0059571466,0.008303702,-0.055762853,0.00847532,0.030548116,-0"
-            + ".015683303,0.006316886,0.010613952,-0.0034125706,-0.0034521748,-0.009610644,0.009973682,-0.027432578,0"
-            + ".027855024,-0.023683371,-0.025412759,0.005729422,0.005511598,-0.018495208,-0.016356576,0.007056166,-0"
-            + ".014072727,-0.0023630566,0.021346718,0.01786154,0.024765888,0.0002749611,-0.021425925,-0.015802115,-0"
-            + ".03160423,-0.017214669,0.0004111009,0.012732782,0.018402798,0.019894559,-0.018217977,-0.009062784,-0"
-            + ".0010676659,-0.032290705,0.0041980557,-0.013676684,-0.0130232135,0.009346615,-0.01639618,0.006917551,"
-            + "-0.013419257,-0.027274162,0.008897766,0.019683337,-0.0078086476,-0.0313138,-0.012415948,-0.016079346,0"
-            + ".01391431,-0.003141941,-0.012884599,0.013861504,0.005571005,-0.043696743,-0.008508324,0.018191574,0"
-            + ".014746001,0.00819809,-0.031102577,0.0015288909,0.0025924314,0.0030973863,0.04427761,-0.034904588,-0"
-            + ".015802115,-0.004871329,-0.06305005,0.027643802,0.004716212,0.013689886,0.0037558076,-0.03279236,0"
-            + ".0033663656,-0.00016821513,0.029043153,0.018666826,-0.018812042,0.006366391,-0.005633712,-0.014429166,"
-            + "-0.012693178,-0.032924373,0.02007938,0.0052277674,0.024198227,0.02154474,0.0148912165,0.005604008,-0"
-            + ".006405995,0.016805425,0.0037228041,0.018732833,-0.0312874,0.023736177,-0.0025099225,-0.015920928,0"
-            + ".014138735,-0.008052874,0.013782296,0.011472045,-0.025755996,0.016145352,-0.024185026,0.003950529,0"
-            + ".01638298,-0.01391431,-0.009828467,0.00414525,0.027353369,-0.012693178,-0.017689921,-0.023168515,-0"
-            + ".027115744,0.009722856,0.030627325,0.00027227955,0.04578257,0.01225753,0.006392794,-0.027115744,-0"
-            + ".005656814,-0.010646956,0.0146535905,0.01639618,-0.000047803627,-0.0071287737,0.018442402,-0"
-            + ".001787144,-0.00016336773,-0.01639618,0.0050363466,0.011590858,-0.023815386,-0.013109023,0.012224527,"
-            + "-0.03057452,-0.018006755,0.0082905,0.012825192,-0.012191524,0.023287328,-0.015617295,-0.0011741024,-0"
-            + ".015432475,0.002699693,0.03437653,-0.01602654,0.029043153,-0.014481972,0.046917893,0.007359799,0"
-            + ".034033295,-0.019696537,0.014244346,-0.006531409,-0.012448952,-0.024831897,-0.005019845,0.0010206358,0"
-            + ".0073267953,0.04541293,0.028066248,-0.012092513,0.0068185404,0.004151851,0.019155279,0.008217892,0"
-            + ".0076568313,-0.0067393314,-0.03371646,-0.016871432,-0.002082526,-0.042059764,-0.04414559,-0.014257547,"
-            + "0.02165035,-0.0027954034,-0.029835239,0.005699719,0.006620519,-0.030257685,0.04042279,-0.029597614,-0"
-            + ".009524834,-0.0029406191,0.023049703,0.014481972,0.015960533,0.0062343767,0.012422549,0.009465427,0"
-            + ".027221356,0.0087393485,-0.008297101,0.009049582,0.020422617,0.004495088,-0.0011476995,0.022152005,-0"
-            + ".015089238,-0.0042508612,-0.02330053,0.026732903,0.058825586,-0.018653626,0.05840314,0.0047393143,0"
-            + ".03276596,-0.0011460495,0.0032739553,0.009518233,0.0014868114,0.000076011376,0.020871466,0.0063036843,"
-            + "-0.013597476,-0.005359782,0.034402933,-0.011016596,-0.009650247,0.021399522,-0.012950606,-0"
-            + ".0130760195,-0.012567764,0.002938969,0.027274162,-0.009934079,-0.004521491,0.0026237848,-0.02000017,0"
-            + ".0068317414,0.00967005,-0.0070891697,-0.0026485375,-0.02568999,0.013003412,0.01898366,-0.04448883,-0"
-            + ".021663552,0.014534778,0.0013894509,-0.027062938,0.036171928,0.019419309,0.0066040168,0.0072343852,0"
-            + ".025465565,-0.012316938,0.015155246,0.012356541,-0.037439264,0.0020330206,-0.049690194,-0.0174919]) as"
-            + " l2_dist, n_tokens, combined from fineFoodReviews order by l2_dist ASC limit 10";
-    printStatus(Color.YELLOW, "Search the most relevant review with the embedding of tomato soup");
-    printStatus(Color.CYAN, "Query : " + q6);
-    printStatus(Color.YELLOW, prettyPrintResponse(runner.runQuery(q6)));
-    printStatus(Color.GREEN, "***************************************************");
+    runVectorQueryExamples(runner);
   }
 
   public static void main(String[] args)
diff --git a/pinot-tools/src/main/java/org/apache/pinot/tools/RealtimeQuickStart.java b/pinot-tools/src/main/java/org/apache/pinot/tools/RealtimeQuickStart.java
index b40722eea4..67bea9c6bd 100644
--- a/pinot-tools/src/main/java/org/apache/pinot/tools/RealtimeQuickStart.java
+++ b/pinot-tools/src/main/java/org/apache/pinot/tools/RealtimeQuickStart.java
@@ -77,6 +77,8 @@ public class RealtimeQuickStart extends QuickStartBase {
     printStatus(Color.CYAN, "Query : " + q5);
     printStatus(Color.YELLOW, prettyPrintResponse(runner.runQuery(q5)));
     printStatus(Color.GREEN, "***************************************************");
+
+    runVectorQueryExamples(runner);
   }
 
   public void execute()
diff --git a/pinot-tools/src/main/resources/examples/batch/fineFoodReviews/fineFoodReviews_offline_table_config.json b/pinot-tools/src/main/resources/examples/batch/fineFoodReviews/fineFoodReviews_offline_table_config.json
index acc50ae301..ad082e3612 100644
--- a/pinot-tools/src/main/resources/examples/batch/fineFoodReviews/fineFoodReviews_offline_table_config.json
+++ b/pinot-tools/src/main/resources/examples/batch/fineFoodReviews/fineFoodReviews_offline_table_config.json
@@ -1,15 +1,4 @@
 {
-  "fieldConfigList": [
-    {
-      "encodingType": "RAW",
-      "indexType": "TEXT",
-      "name": "Text",
-      "properties": {
-        "deriveNumDocsPerChunkForRawIndex": "true",
-        "rawIndexWriterVersion": "3"
-      }
-    }
-  ],
   "tableName": "fineFoodReviews",
   "tableType": "OFFLINE",
   "segmentsConfig": {
@@ -29,5 +18,27 @@
   "metadata": {
     "customConfigs": {
     }
-  }
+  },
+  "fieldConfigList": [
+    {
+      "encodingType": "RAW",
+      "indexType": "VECTOR",
+      "name": "embedding",
+      "properties": {
+        "vectorIndexType": "HNSW",
+        "vectorDimension": 1536,
+        "vectorDistanceFunction": "COSINE",
+        "version": 1
+      }
+    },
+    {
+      "encodingType": "RAW",
+      "indexType": "TEXT",
+      "name": "Text",
+      "properties": {
+        "deriveNumDocsPerChunkForRawIndex": "true",
+        "rawIndexWriterVersion": "3"
+      }
+    }
+  ]
 }
diff --git a/pinot-tools/src/main/resources/examples/stream/fineFoodReviews/fineFoodReviews_realtime_table_config.json b/pinot-tools/src/main/resources/examples/stream/fineFoodReviews/fineFoodReviews_realtime_table_config.json
new file mode 100644
index 0000000000..dd9d551e19
--- /dev/null
+++ b/pinot-tools/src/main/resources/examples/stream/fineFoodReviews/fineFoodReviews_realtime_table_config.json
@@ -0,0 +1,73 @@
+{
+  "tableName": "fineFoodReviews",
+  "tableType": "REALTIME",
+  "segmentsConfig": {
+    "segmentPushType": "APPEND",
+    "segmentAssignmentStrategy": "BalanceNumSegmentAssignmentStrategy",
+    "timeColumnName": "ts",
+    "retentionTimeUnit": "DAYS",
+    "retentionTimeValue": "5",
+    "replication": "1"
+  },
+  "tenants": {
+  },
+  "tableIndexConfig": {
+    "loadMode": "MMAP",
+    "noDictionaryColumns": ["Text"]
+  },
+  "routing": {
+    "segmentPrunerTypes": [
+      "time"
+    ]
+  },
+  "ingestionConfig": {
+    "streamIngestionConfig": {
+      "streamConfigMaps": [
+        {
+          "streamType": "kafka",
+          "stream.kafka.topic.name": "fineFoodReviews",
+          "stream.kafka.decoder.class.name": "org.apache.pinot.plugin.stream.kafka.KafkaJSONMessageDecoder",
+          "stream.kafka.consumer.factory.class.name": "org.apache.pinot.plugin.stream.kafka20.KafkaConsumerFactory",
+          "stream.kafka.consumer.prop.auto.offset.reset": "smallest",
+          "stream.kafka.zk.broker.url": "localhost:2191/kafka",
+          "stream.kafka.broker.list": "localhost:19092",
+          "realtime.segment.flush.threshold.time": "3600000",
+          "realtime.segment.flush.threshold.size": "50000"
+        }
+      ]
+    },
+    "transformConfigs": [
+      {
+        "columnName": "ts",
+        "transformFunction": "now()"
+      }
+    ]
+  },
+  "metadata": {
+    "customConfigs": {
+    }
+  },
+  "fieldConfigList": [
+    {
+      "encodingType": "RAW",
+      "indexType": "VECTOR",
+      "name": "embedding",
+      "properties": {
+        "vectorIndexType": "HNSW",
+        "vectorDimension": 1536,
+        "vectorDistanceFunction": "COSINE",
+        "version": 1,
+        "commitDocs": "1"
+      }
+    },
+    {
+      "encodingType": "RAW",
+      "indexType": "TEXT",
+      "name": "Text",
+      "properties": {
+        "deriveNumDocsPerChunkForRawIndex": "true",
+        "rawIndexWriterVersion": "3"
+      }
+    }
+  ]
+}
diff --git a/pinot-tools/src/main/resources/examples/stream/fineFoodReviews/fineFoodReviews_schema.json b/pinot-tools/src/main/resources/examples/stream/fineFoodReviews/fineFoodReviews_schema.json
new file mode 100644
index 0000000000..e1a67730f9
--- /dev/null
+++ b/pinot-tools/src/main/resources/examples/stream/fineFoodReviews/fineFoodReviews_schema.json
@@ -0,0 +1,48 @@
+{
+  "metricFieldSpecs": [
+  ],
+  "dimensionFieldSpecs": [
+    {
+      "dataType": "STRING",
+      "name": "ProductId"
+    },
+    {
+      "dataType": "STRING",
+      "name": "UserId"
+    },
+    {
+      "dataType": "INT",
+      "name": "Score"
+    },
+    {
+      "dataType": "STRING",
+      "name": "Summary"
+    },
+    {
+      "dataType": "STRING",
+      "name": "Text"
+    },
+    {
+      "dataType": "STRING",
+      "name": "combined"
+    },
+    {
+      "dataType": "INT",
+      "name": "n_tokens"
+    },
+    {
+      "dataType": "FLOAT",
+      "name": "embedding",
+      "singleValueField": false
+    }
+  ],
+  "dateTimeFieldSpecs": [
+    {
+      "name": "ts",
+      "dataType": "TIMESTAMP",
+      "format": "1:MILLISECONDS:TIMESTAMP",
+      "granularity": "1:SECONDS"
+    }
+  ],
+  "schemaName": "fineFoodReviews"
+}
diff --git a/pinot-tools/src/main/resources/examples/stream/fineFoodReviews/rawdata/fine_food_reviews_with_embeddings_1k.json.gz b/pinot-tools/src/main/resources/examples/stream/fineFoodReviews/rawdata/fine_food_reviews_with_embeddings_1k.json.gz
new file mode 100644
index 0000000000..62413ba0c1
Binary files /dev/null and b/pinot-tools/src/main/resources/examples/stream/fineFoodReviews/rawdata/fine_food_reviews_with_embeddings_1k.json.gz differ


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