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/11/07 08:33:12 UTC

(pinot) 02/03: Try lucene-backward-codecs package

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

commit 00cf7a829540814e4623e77f5cc1f4540520e411
Author: Xiang Fu <xi...@gmail.com>
AuthorDate: Tue Oct 31 15:09:15 2023 -0700

    Try lucene-backward-codecs package
---
 compatibility-verifier/compCheck.sh                |  6 ++++
 pinot-core/pom.xml                                 | 10 +++++--
 pinot-segment-local/pom.xml                        | 11 ++++++--
 .../impl/inv/text/LuceneFSTIndexCreator.java       |  2 +-
 .../creator/impl/text/LuceneTextIndexCreator.java  |  2 +-
 .../converter/SegmentV1V2ToV3FormatConverter.java  |  6 ++--
 .../local/segment/index/fst/FstIndexType.java      |  5 ++--
 .../loader/invertedindex/FSTIndexHandler.java      |  4 +--
 .../index/readers/text/LuceneTextIndexReader.java  | 20 ++++++++------
 .../local/segment/index/text/TextIndexType.java    |  3 +-
 .../local/segment/store/TextIndexUtils.java        | 11 ++++++--
 .../utils/nativefst/NativeFSTIndexCreator.java     |  2 +-
 .../index/creator/LuceneFSTIndexCreatorTest.java   |  4 +--
 .../index/creator/NativeFSTIndexCreatorTest.java   |  4 +--
 .../local/segment/index/loader/LoaderTest.java     | 32 +++++++++++-----------
 .../index/loader/SegmentPreProcessorTest.java      |  4 +--
 .../segment/store/FilePerIndexDirectoryTest.java   |  9 +++---
 .../store/SingleFileIndexDirectoryTest.java        |  9 +++---
 .../org/apache/pinot/segment/spi/V1Constants.java  |  9 ++++--
 .../segment/spi/store/SegmentDirectoryPaths.java   | 27 ++++++++++++++----
 .../fineFoodReviews_offline_table_config.json      | 12 ++++++++
 pom.xml                                            | 24 ++++++++++++++++
 22 files changed, 148 insertions(+), 68 deletions(-)

diff --git a/compatibility-verifier/compCheck.sh b/compatibility-verifier/compCheck.sh
index 0803a06b9c..8ed924c6c8 100755
--- a/compatibility-verifier/compCheck.sh
+++ b/compatibility-verifier/compCheck.sh
@@ -219,6 +219,12 @@ function stopServices() {
   stopService server
   stopService zookeeper
   stopService kafka
+  echo "Controller logs:"
+  cat ${LOG_DIR}/controller.*.log
+  echo "Broker logs:"
+  cat ${LOG_DIR}/broker.*.log
+  echo "Server logs:"
+  cat ${LOG_DIR}/server.*.log
   echo "Cluster stopped."
 }
 
diff --git a/pinot-core/pom.xml b/pinot-core/pom.xml
index 89ced53e82..815bf75291 100644
--- a/pinot-core/pom.xml
+++ b/pinot-core/pom.xml
@@ -246,21 +246,25 @@
       <version>${project.version}</version>
       <scope>test</scope>
     </dependency>
+
+    <!-- Lucene dependencies -->
+    <dependency>
+      <groupId>org.apache.lucene</groupId>
+      <artifactId>lucene-backward-codecs</artifactId>
+    </dependency>
     <dependency>
       <groupId>org.apache.lucene</groupId>
       <artifactId>lucene-core</artifactId>
-      <version>${lucene.version}</version>
     </dependency>
     <dependency>
       <groupId>org.apache.lucene</groupId>
       <artifactId>lucene-queryparser</artifactId>
-      <version>${lucene.version}</version>
     </dependency>
     <dependency>
       <groupId>org.apache.lucene</groupId>
       <artifactId>lucene-analysis-common</artifactId>
-      <version>${lucene.version}</version>
     </dependency>
+    <!-- Lucene dependencies end -->
   </dependencies>
   <profiles>
     <profile>
diff --git a/pinot-segment-local/pom.xml b/pinot-segment-local/pom.xml
index 2d90fbed46..5b2e5e24c7 100644
--- a/pinot-segment-local/pom.xml
+++ b/pinot-segment-local/pom.xml
@@ -57,21 +57,26 @@
       <groupId>com.uber</groupId>
       <artifactId>h3</artifactId>
     </dependency>
+
+    <!-- Lucene dependencies -->
+    <dependency>
+      <groupId>org.apache.lucene</groupId>
+      <artifactId>lucene-backward-codecs</artifactId>
+    </dependency>
     <dependency>
       <groupId>org.apache.lucene</groupId>
       <artifactId>lucene-core</artifactId>
-      <version>${lucene.version}</version>
     </dependency>
     <dependency>
       <groupId>org.apache.lucene</groupId>
       <artifactId>lucene-queryparser</artifactId>
-      <version>${lucene.version}</version>
     </dependency>
     <dependency>
       <groupId>org.apache.lucene</groupId>
       <artifactId>lucene-analysis-common</artifactId>
-      <version>${lucene.version}</version>
     </dependency>
+    <!-- Lucene dependencies end -->
+
     <dependency>
       <groupId>org.apache.datasketches</groupId>
       <artifactId>datasketches-java</artifactId>
diff --git a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/creator/impl/inv/text/LuceneFSTIndexCreator.java b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/creator/impl/inv/text/LuceneFSTIndexCreator.java
index 695ce4b1ea..335449fe96 100644
--- a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/creator/impl/inv/text/LuceneFSTIndexCreator.java
+++ b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/creator/impl/inv/text/LuceneFSTIndexCreator.java
@@ -56,7 +56,7 @@ public class LuceneFSTIndexCreator implements FSTIndexCreator {
    */
   public LuceneFSTIndexCreator(File indexDir, String columnName, String[] sortedEntries)
       throws IOException {
-    _fstIndexFile = new File(indexDir, columnName + V1Constants.Indexes.FST_INDEX_FILE_EXTENSION);
+    _fstIndexFile = new File(indexDir, columnName + V1Constants.Indexes.LUCENE_V9_FST_INDEX_FILE_EXTENSION);
 
     _fstBuilder = new FSTBuilder();
     _dictId = 0;
diff --git a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/creator/impl/text/LuceneTextIndexCreator.java b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/creator/impl/text/LuceneTextIndexCreator.java
index 3d96f9b226..771b42da3f 100644
--- a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/creator/impl/text/LuceneTextIndexCreator.java
+++ b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/creator/impl/text/LuceneTextIndexCreator.java
@@ -196,7 +196,7 @@ public class LuceneTextIndexCreator extends AbstractTextIndexCreator {
   }
 
   private File getV1TextIndexFile(File indexDir) {
-    String luceneIndexDirectory = _textColumn + V1Constants.Indexes.LUCENE_TEXT_INDEX_FILE_EXTENSION;
+    String luceneIndexDirectory = _textColumn + V1Constants.Indexes.LUCENE_V9_TEXT_INDEX_FILE_EXTENSION;
     return new File(indexDir, luceneIndexDirectory);
   }
 }
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 f8c6819efb..d6cc8b1515 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
@@ -110,7 +110,7 @@ public class SegmentV1V2ToV3FormatConverter implements SegmentFormatConverter {
       if (file.isFile() && file.exists()) {
         FileUtils.deleteQuietly(file);
       }
-      if (file.isDirectory() && file.getName().endsWith(V1Constants.Indexes.LUCENE_TEXT_INDEX_FILE_EXTENSION)) {
+      if (file.isDirectory() && file.getName().endsWith(V1Constants.Indexes.LUCENE_V9_TEXT_INDEX_FILE_EXTENSION)) {
         FileUtils.deleteDirectory(file);
       }
     }
@@ -222,7 +222,7 @@ public class SegmentV1V2ToV3FormatConverter implements SegmentFormatConverter {
   private void copyLuceneTextIndexIfExists(File segmentDirectory, File v3Dir)
       throws IOException {
     // TODO: see if this can be done by reusing some existing methods
-    String suffix = V1Constants.Indexes.LUCENE_TEXT_INDEX_FILE_EXTENSION;
+    String suffix = V1Constants.Indexes.LUCENE_V9_TEXT_INDEX_FILE_EXTENSION;
     File[] textIndexFiles = segmentDirectory.listFiles(new FilenameFilter() {
       @Override
       public boolean accept(File dir, String name) {
@@ -243,7 +243,7 @@ public class SegmentV1V2ToV3FormatConverter implements SegmentFormatConverter {
     // to moving the lucene text index files, we need to move the
     // docID mapping/cache file created by us in v1/v2 during an earlier
     // load of the segment.
-    String docIDFileSuffix = V1Constants.Indexes.LUCENE_TEXT_INDEX_DOCID_MAPPING_FILE_EXTENSION;
+    String docIDFileSuffix = V1Constants.Indexes.LUCENE_V9_TEXT_INDEX_DOCID_MAPPING_FILE_EXTENSION;
     File[] textIndexDocIdMappingFiles = segmentDirectory.listFiles(new FilenameFilter() {
       @Override
       public boolean accept(File dir, String name) {
diff --git a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/fst/FstIndexType.java b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/fst/FstIndexType.java
index 683227031f..d04ce7bc97 100644
--- a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/fst/FstIndexType.java
+++ b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/fst/FstIndexType.java
@@ -20,8 +20,8 @@
 package org.apache.pinot.segment.local.segment.index.fst;
 
 import com.google.common.base.Preconditions;
+import com.google.common.collect.ImmutableList;
 import java.io.IOException;
-import java.util.Collections;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
@@ -64,7 +64,8 @@ public class FstIndexType extends AbstractIndexType<FstIndexConfig, TextIndexRea
     implements ConfigurableFromIndexLoadingConfig<FstIndexConfig> {
   public static final String INDEX_DISPLAY_NAME = "fst";
   private static final List<String> EXTENSIONS =
-      Collections.singletonList(V1Constants.Indexes.FST_INDEX_FILE_EXTENSION);
+      ImmutableList.of(V1Constants.Indexes.LUCENE_FST_INDEX_FILE_EXTENSION,
+          V1Constants.Indexes.LUCENE_V9_FST_INDEX_FILE_EXTENSION);
 
   protected FstIndexType() {
     super(StandardIndexes.FST_ID);
diff --git a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/loader/invertedindex/FSTIndexHandler.java b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/loader/invertedindex/FSTIndexHandler.java
index e387ba3929..778e92db0a 100644
--- a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/loader/invertedindex/FSTIndexHandler.java
+++ b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/loader/invertedindex/FSTIndexHandler.java
@@ -44,7 +44,7 @@ import org.apache.pinot.spi.data.FieldSpec;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import static org.apache.pinot.segment.spi.V1Constants.Indexes.FST_INDEX_FILE_EXTENSION;
+import static org.apache.pinot.segment.spi.V1Constants.Indexes.LUCENE_V9_FST_INDEX_FILE_EXTENSION;
 
 
 /**
@@ -157,7 +157,7 @@ public class FSTIndexHandler extends BaseIndexHandler {
     String segmentName = _segmentDirectory.getSegmentMetadata().getName();
     String columnName = columnMetadata.getColumnName();
     File inProgress = new File(indexDir, columnName + ".fst.inprogress");
-    File fstIndexFile = new File(indexDir, columnName + FST_INDEX_FILE_EXTENSION);
+    File fstIndexFile = new File(indexDir, columnName + LUCENE_V9_FST_INDEX_FILE_EXTENSION);
 
     if (!inProgress.exists()) {
       // Create a marker file.
diff --git a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/readers/text/LuceneTextIndexReader.java b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/readers/text/LuceneTextIndexReader.java
index f1b5789fd5..0d4646e1da 100644
--- a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/readers/text/LuceneTextIndexReader.java
+++ b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/readers/text/LuceneTextIndexReader.java
@@ -84,9 +84,9 @@ public class LuceneTextIndexReader implements TextIndexReader {
       // 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 DocIdTranslator(indexDir, _column, numDocs, _indexSearcher);
-      _standardAnalyzer = TextIndexUtils.getStandardAnalyzerWithCustomizedStopWords(
-          config.getStopWordsInclude(), config.getStopWordsExclude()
-      );
+      _standardAnalyzer = TextIndexUtils.getStandardAnalyzerWithCustomizedStopWords(config.getStopWordsInclude(),
+          config.getStopWordsExclude());
+      LOGGER.info("Successfully read lucene index for {} from {}", _column, indexDir);
     } catch (Exception e) {
       LOGGER.error("Failed to instantiate Lucene text index reader for column {}, exception {}", column,
           e.getMessage());
@@ -99,15 +99,15 @@ public class LuceneTextIndexReader implements TextIndexReader {
    * we load the text index (per column if it exists) and store
    * the reference in {@link PhysicalColumnIndexContainer}
    * similar to how it is done for other types of indexes.
-   * @param column column name
+   *
+   * @param column   column name
    * @param indexDir segment index directory
-   * @param numDocs number of documents in the segment
+   * @param numDocs  number of documents in the segment
    */
   public LuceneTextIndexReader(String column, File indexDir, int numDocs,
       @Nullable Map<String, String> textIndexProperties) {
-    this(column, indexDir, numDocs, new TextIndexConfigBuilder(FSTType.LUCENE)
-        .withProperties(textIndexProperties)
-        .build());
+    this(column, indexDir, numDocs,
+        new TextIndexConfigBuilder(FSTType.LUCENE).withProperties(textIndexProperties).build());
   }
 
   /**
@@ -120,6 +120,7 @@ public class LuceneTextIndexReader implements TextIndexReader {
    * 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
    */
@@ -163,6 +164,7 @@ public class LuceneTextIndexReader implements TextIndexReader {
    * 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
@@ -188,7 +190,7 @@ public class LuceneTextIndexReader implements TextIndexReader {
         throws Exception {
       int length = Integer.BYTES * numDocs;
       File docIdMappingFile = new File(SegmentDirectoryPaths.findSegmentDirectory(segmentIndexDir),
-          column + V1Constants.Indexes.LUCENE_TEXT_INDEX_DOCID_MAPPING_FILE_EXTENSION);
+          column + V1Constants.Indexes.LUCENE_V9_TEXT_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.
diff --git a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/text/TextIndexType.java b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/text/TextIndexType.java
index bfd1732439..4ad6eb3a94 100644
--- a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/text/TextIndexType.java
+++ b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/text/TextIndexType.java
@@ -74,7 +74,8 @@ public class TextIndexType extends AbstractIndexType<TextIndexConfig, TextIndexR
   // TODO: Should V1Constants.Indexes.LUCENE_TEXT_INDEX_DOCID_MAPPING_FILE_EXTENSION be added here?
   private static final List<String> EXTENSIONS = Lists.newArrayList(
       V1Constants.Indexes.LUCENE_TEXT_INDEX_FILE_EXTENSION,
-      V1Constants.Indexes.NATIVE_TEXT_INDEX_FILE_EXTENSION
+      V1Constants.Indexes.NATIVE_TEXT_INDEX_FILE_EXTENSION,
+      V1Constants.Indexes.LUCENE_V9_TEXT_INDEX_FILE_EXTENSION
   );
 
   protected TextIndexType() {
diff --git a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/store/TextIndexUtils.java b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/store/TextIndexUtils.java
index aa474743c6..2d0510c486 100644
--- a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/store/TextIndexUtils.java
+++ b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/store/TextIndexUtils.java
@@ -47,6 +47,10 @@ public class TextIndexUtils {
     FileUtils.deleteQuietly(luceneIndexFile);
     File luceneMappingFile = new File(segDir, column + Indexes.LUCENE_TEXT_INDEX_DOCID_MAPPING_FILE_EXTENSION);
     FileUtils.deleteQuietly(luceneMappingFile);
+    File luceneV9IndexFile = new File(segDir, column + Indexes.LUCENE_V9_TEXT_INDEX_FILE_EXTENSION);
+    FileUtils.deleteQuietly(luceneV9IndexFile);
+    File luceneV9MappingFile = new File(segDir, column + Indexes.LUCENE_V9_TEXT_INDEX_DOCID_MAPPING_FILE_EXTENSION);
+    FileUtils.deleteQuietly(luceneV9MappingFile);
 
     // Remove the native index file
     File nativeIndexFile = new File(segDir, column + Indexes.NATIVE_TEXT_INDEX_FILE_EXTENSION);
@@ -54,8 +58,9 @@ public class TextIndexUtils {
   }
 
   static boolean hasTextIndex(File segDir, String column) {
-    return new File(segDir, column + Indexes.LUCENE_TEXT_INDEX_FILE_EXTENSION).exists() || new File(segDir,
-        column + Indexes.NATIVE_TEXT_INDEX_FILE_EXTENSION).exists();
+    return new File(segDir, column + Indexes.LUCENE_TEXT_INDEX_FILE_EXTENSION).exists()
+        || new File(segDir, column + Indexes.LUCENE_V9_TEXT_INDEX_FILE_EXTENSION).exists()
+        || new File(segDir, column + Indexes.NATIVE_TEXT_INDEX_FILE_EXTENSION).exists();
   }
 
   public static boolean isFstTypeNative(@Nullable Map<String, String> textIndexProperties) {
@@ -108,7 +113,7 @@ public class TextIndexUtils {
   }
 
   public static StandardAnalyzer getStandardAnalyzerWithCustomizedStopWords(@Nullable List<String> stopWordsInclude,
-     @Nullable List<String> stopWordsExclude) {
+      @Nullable List<String> stopWordsExclude) {
     HashSet<String> stopWordSet = LuceneTextIndexCreator.getDefaultEnglishStopWordsSet();
     if (stopWordsInclude != null) {
       stopWordSet.addAll(stopWordsInclude);
diff --git a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/utils/nativefst/NativeFSTIndexCreator.java b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/utils/nativefst/NativeFSTIndexCreator.java
index 1fb58b56bc..1f69bc50f1 100644
--- a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/utils/nativefst/NativeFSTIndexCreator.java
+++ b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/utils/nativefst/NativeFSTIndexCreator.java
@@ -47,7 +47,7 @@ public class NativeFSTIndexCreator implements FSTIndexCreator {
    * @param sortedEntries Sorted entries of the unique values of the column.
    */
   public NativeFSTIndexCreator(File indexDir, String columnName, String[] sortedEntries) {
-    _fstIndexFile = new File(indexDir, columnName + V1Constants.Indexes.FST_INDEX_FILE_EXTENSION);
+    _fstIndexFile = new File(indexDir, columnName + V1Constants.Indexes.LUCENE_V9_FST_INDEX_FILE_EXTENSION);
 
     _fstBuilder = new FSTBuilder();
     _dictId = 0;
diff --git a/pinot-segment-local/src/test/java/org/apache/pinot/segment/local/segment/index/creator/LuceneFSTIndexCreatorTest.java b/pinot-segment-local/src/test/java/org/apache/pinot/segment/local/segment/index/creator/LuceneFSTIndexCreatorTest.java
index c12a969037..b9c568000e 100644
--- a/pinot-segment-local/src/test/java/org/apache/pinot/segment/local/segment/index/creator/LuceneFSTIndexCreatorTest.java
+++ b/pinot-segment-local/src/test/java/org/apache/pinot/segment/local/segment/index/creator/LuceneFSTIndexCreatorTest.java
@@ -32,7 +32,7 @@ import org.testng.annotations.AfterClass;
 import org.testng.annotations.BeforeClass;
 import org.testng.annotations.Test;
 
-import static org.apache.pinot.segment.spi.V1Constants.Indexes.FST_INDEX_FILE_EXTENSION;
+import static org.apache.pinot.segment.spi.V1Constants.Indexes.LUCENE_V9_FST_INDEX_FILE_EXTENSION;
 
 
 public class LuceneFSTIndexCreatorTest {
@@ -62,7 +62,7 @@ public class LuceneFSTIndexCreatorTest {
     LuceneFSTIndexCreator creator = new LuceneFSTIndexCreator(
         INDEX_DIR, "testFSTColumn", uniqueValues);
     creator.seal();
-    File fstFile = new File(INDEX_DIR, "testFSTColumn" + FST_INDEX_FILE_EXTENSION);
+    File fstFile = new File(INDEX_DIR, "testFSTColumn" + LUCENE_V9_FST_INDEX_FILE_EXTENSION);
     PinotDataBuffer pinotDataBuffer =
         PinotDataBuffer.mapFile(fstFile, true, 0, fstFile.length(), ByteOrder.BIG_ENDIAN, "fstIndexFile");
     LuceneFSTIndexReader reader = new LuceneFSTIndexReader(pinotDataBuffer);
diff --git a/pinot-segment-local/src/test/java/org/apache/pinot/segment/local/segment/index/creator/NativeFSTIndexCreatorTest.java b/pinot-segment-local/src/test/java/org/apache/pinot/segment/local/segment/index/creator/NativeFSTIndexCreatorTest.java
index cc4f153d8e..d77268ef28 100644
--- a/pinot-segment-local/src/test/java/org/apache/pinot/segment/local/segment/index/creator/NativeFSTIndexCreatorTest.java
+++ b/pinot-segment-local/src/test/java/org/apache/pinot/segment/local/segment/index/creator/NativeFSTIndexCreatorTest.java
@@ -29,7 +29,7 @@ import org.testng.annotations.AfterClass;
 import org.testng.annotations.BeforeClass;
 import org.testng.annotations.Test;
 
-import static org.apache.pinot.segment.spi.V1Constants.Indexes.FST_INDEX_FILE_EXTENSION;
+import static org.apache.pinot.segment.spi.V1Constants.Indexes.LUCENE_V9_FST_INDEX_FILE_EXTENSION;
 
 
 public class NativeFSTIndexCreatorTest {
@@ -59,7 +59,7 @@ public class NativeFSTIndexCreatorTest {
       creator.seal();
     }
 
-    File fstFile = new File(INDEX_DIR, "testFSTColumn" + FST_INDEX_FILE_EXTENSION);
+    File fstFile = new File(INDEX_DIR, "testFSTColumn" + LUCENE_V9_FST_INDEX_FILE_EXTENSION);
     try (PinotDataBuffer dataBuffer = PinotDataBuffer.mapReadOnlyBigEndianFile(fstFile);
         NativeFSTIndexReader reader = new NativeFSTIndexReader(dataBuffer)) {
 
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 c9db1e3148..bc64d539b0 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
@@ -63,7 +63,7 @@ import org.testng.annotations.BeforeClass;
 import org.testng.annotations.Test;
 import org.testng.collections.Lists;
 
-import static org.apache.pinot.segment.spi.V1Constants.Indexes.FST_INDEX_FILE_EXTENSION;
+import static org.apache.pinot.segment.spi.V1Constants.Indexes.LUCENE_V9_FST_INDEX_FILE_EXTENSION;
 import static org.testng.Assert.assertFalse;
 import static org.testng.Assert.assertTrue;
 
@@ -330,7 +330,7 @@ public class LoaderTest {
     fstIndexFile = SegmentDirectoryPaths.findFSTIndexIndexFile(_indexDir, FST_INDEX_COL_NAME);
     Assert.assertNotNull(fstIndexFile);
     Assert.assertFalse(fstIndexFile.isDirectory());
-    Assert.assertEquals(fstIndexFile.getName(), FST_INDEX_COL_NAME + FST_INDEX_FILE_EXTENSION);
+    Assert.assertEquals(fstIndexFile.getName(), FST_INDEX_COL_NAME + LUCENE_V9_FST_INDEX_FILE_EXTENSION);
     Assert.assertEquals(fstIndexFile.getParentFile().getName(), new SegmentMetadataImpl(_indexDir).getName());
     indexSegment.destroy();
 
@@ -349,7 +349,7 @@ public class LoaderTest {
     fstIndexFile = SegmentDirectoryPaths.findFSTIndexIndexFile(_indexDir, FST_INDEX_COL_NAME);
     Assert.assertNotNull(fstIndexFile);
     Assert.assertFalse(fstIndexFile.isDirectory());
-    Assert.assertEquals(fstIndexFile.getName(), FST_INDEX_COL_NAME + FST_INDEX_FILE_EXTENSION);
+    Assert.assertEquals(fstIndexFile.getName(), FST_INDEX_COL_NAME + LUCENE_V9_FST_INDEX_FILE_EXTENSION);
     Assert.assertEquals(fstIndexFile.getParentFile().getName(), new SegmentMetadataImpl(_indexDir).getName());
     indexSegment.destroy();
 
@@ -366,7 +366,7 @@ public class LoaderTest {
     fstIndexFile = SegmentDirectoryPaths.findFSTIndexIndexFile(_indexDir, FST_INDEX_COL_NAME);
     Assert.assertNotNull(fstIndexFile);
     Assert.assertFalse(fstIndexFile.isDirectory());
-    Assert.assertEquals(fstIndexFile.getName(), FST_INDEX_COL_NAME + FST_INDEX_FILE_EXTENSION);
+    Assert.assertEquals(fstIndexFile.getName(), FST_INDEX_COL_NAME + LUCENE_V9_FST_INDEX_FILE_EXTENSION);
     Assert.assertEquals(fstIndexFile.getParentFile().getName(), new SegmentMetadataImpl(_indexDir).getName());
     indexSegment.destroy();
 
@@ -579,7 +579,7 @@ public class LoaderTest {
     Assert.assertNotNull(textIndexFile);
     Assert.assertTrue(textIndexFile.isDirectory());
     Assert.assertEquals(textIndexFile.getName(),
-        TEXT_INDEX_COL_NAME + V1Constants.Indexes.LUCENE_TEXT_INDEX_FILE_EXTENSION);
+        TEXT_INDEX_COL_NAME + V1Constants.Indexes.LUCENE_V9_TEXT_INDEX_FILE_EXTENSION);
     Assert.assertEquals(textIndexFile.getParentFile().getName(), SegmentDirectoryPaths.V3_SUBDIRECTORY_NAME);
 
     // CASE 1: don't set the segment version to load in IndexLoadingConfig
@@ -605,10 +605,10 @@ public class LoaderTest {
     Assert.assertTrue(textIndexFile.isDirectory());
     Assert.assertFalse(textIndexDocIdMappingFile.isDirectory());
     Assert.assertEquals(textIndexFile.getName(),
-        TEXT_INDEX_COL_NAME + V1Constants.Indexes.LUCENE_TEXT_INDEX_FILE_EXTENSION);
+        TEXT_INDEX_COL_NAME + V1Constants.Indexes.LUCENE_V9_TEXT_INDEX_FILE_EXTENSION);
     Assert.assertEquals(textIndexFile.getParentFile().getName(), SegmentDirectoryPaths.V3_SUBDIRECTORY_NAME);
     Assert.assertEquals(textIndexDocIdMappingFile.getName(),
-        TEXT_INDEX_COL_NAME + V1Constants.Indexes.LUCENE_TEXT_INDEX_DOCID_MAPPING_FILE_EXTENSION);
+        TEXT_INDEX_COL_NAME + V1Constants.Indexes.LUCENE_V9_TEXT_INDEX_DOCID_MAPPING_FILE_EXTENSION);
     Assert.assertEquals(textIndexDocIdMappingFile.getParentFile().getName(),
         SegmentDirectoryPaths.V3_SUBDIRECTORY_NAME);
     indexSegment.destroy();
@@ -633,10 +633,10 @@ public class LoaderTest {
     Assert.assertTrue(textIndexFile.isDirectory());
     Assert.assertFalse(textIndexDocIdMappingFile.isDirectory());
     Assert.assertEquals(textIndexFile.getName(),
-        TEXT_INDEX_COL_NAME + V1Constants.Indexes.LUCENE_TEXT_INDEX_FILE_EXTENSION);
+        TEXT_INDEX_COL_NAME + V1Constants.Indexes.LUCENE_V9_TEXT_INDEX_FILE_EXTENSION);
     Assert.assertEquals(textIndexFile.getParentFile().getName(), SegmentDirectoryPaths.V3_SUBDIRECTORY_NAME);
     Assert.assertEquals(textIndexDocIdMappingFile.getName(),
-        TEXT_INDEX_COL_NAME + V1Constants.Indexes.LUCENE_TEXT_INDEX_DOCID_MAPPING_FILE_EXTENSION);
+        TEXT_INDEX_COL_NAME + V1Constants.Indexes.LUCENE_V9_TEXT_INDEX_DOCID_MAPPING_FILE_EXTENSION);
     Assert.assertEquals(textIndexDocIdMappingFile.getParentFile().getName(),
         SegmentDirectoryPaths.V3_SUBDIRECTORY_NAME);
     indexSegment.destroy();
@@ -660,7 +660,7 @@ public class LoaderTest {
     Assert.assertTrue(textIndexFile.isDirectory());
     Assert.assertFalse(textIndexDocIdMappingFile.isDirectory());
     Assert.assertEquals(textIndexFile.getName(),
-        TEXT_INDEX_COL_NAME + V1Constants.Indexes.LUCENE_TEXT_INDEX_FILE_EXTENSION);
+        TEXT_INDEX_COL_NAME + V1Constants.Indexes.LUCENE_V9_TEXT_INDEX_FILE_EXTENSION);
     Assert.assertEquals(textIndexFile.getParentFile().getName(), new SegmentMetadataImpl(_indexDir).getName());
 
     // CASE 1: don't set the segment version to load in IndexLoadingConfig
@@ -683,10 +683,10 @@ public class LoaderTest {
     Assert.assertNotNull(textIndexDocIdMappingFile);
     Assert.assertTrue(textIndexFile.isDirectory());
     Assert.assertEquals(textIndexFile.getName(),
-        TEXT_INDEX_COL_NAME + V1Constants.Indexes.LUCENE_TEXT_INDEX_FILE_EXTENSION);
+        TEXT_INDEX_COL_NAME + V1Constants.Indexes.LUCENE_V9_TEXT_INDEX_FILE_EXTENSION);
     Assert.assertEquals(textIndexFile.getParentFile().getName(), new SegmentMetadataImpl(_indexDir).getName());
     Assert.assertEquals(textIndexDocIdMappingFile.getName(),
-        TEXT_INDEX_COL_NAME + V1Constants.Indexes.LUCENE_TEXT_INDEX_DOCID_MAPPING_FILE_EXTENSION);
+        TEXT_INDEX_COL_NAME + V1Constants.Indexes.LUCENE_V9_TEXT_INDEX_DOCID_MAPPING_FILE_EXTENSION);
     Assert.assertEquals(textIndexDocIdMappingFile.getParentFile().getName(),
         new SegmentMetadataImpl(_indexDir).getName());
     indexSegment.destroy();
@@ -709,10 +709,10 @@ public class LoaderTest {
     Assert.assertNotNull(textIndexDocIdMappingFile);
     Assert.assertTrue(textIndexFile.isDirectory());
     Assert.assertEquals(textIndexFile.getName(),
-        TEXT_INDEX_COL_NAME + V1Constants.Indexes.LUCENE_TEXT_INDEX_FILE_EXTENSION);
+        TEXT_INDEX_COL_NAME + V1Constants.Indexes.LUCENE_V9_TEXT_INDEX_FILE_EXTENSION);
     Assert.assertEquals(textIndexFile.getParentFile().getName(), new SegmentMetadataImpl(_indexDir).getName());
     Assert.assertEquals(textIndexDocIdMappingFile.getName(),
-        TEXT_INDEX_COL_NAME + V1Constants.Indexes.LUCENE_TEXT_INDEX_DOCID_MAPPING_FILE_EXTENSION);
+        TEXT_INDEX_COL_NAME + V1Constants.Indexes.LUCENE_V9_TEXT_INDEX_DOCID_MAPPING_FILE_EXTENSION);
     Assert.assertEquals(textIndexDocIdMappingFile.getParentFile().getName(),
         new SegmentMetadataImpl(_indexDir).getName());
     indexSegment.destroy();
@@ -735,10 +735,10 @@ public class LoaderTest {
     Assert.assertNotNull(textIndexDocIdMappingFile);
     Assert.assertTrue(textIndexFile.isDirectory());
     Assert.assertEquals(textIndexFile.getName(),
-        TEXT_INDEX_COL_NAME + V1Constants.Indexes.LUCENE_TEXT_INDEX_FILE_EXTENSION);
+        TEXT_INDEX_COL_NAME + V1Constants.Indexes.LUCENE_V9_TEXT_INDEX_FILE_EXTENSION);
     Assert.assertEquals(textIndexFile.getParentFile().getName(), SegmentDirectoryPaths.V3_SUBDIRECTORY_NAME);
     Assert.assertEquals(textIndexDocIdMappingFile.getName(),
-        TEXT_INDEX_COL_NAME + V1Constants.Indexes.LUCENE_TEXT_INDEX_DOCID_MAPPING_FILE_EXTENSION);
+        TEXT_INDEX_COL_NAME + V1Constants.Indexes.LUCENE_V9_TEXT_INDEX_DOCID_MAPPING_FILE_EXTENSION);
     Assert.assertEquals(textIndexDocIdMappingFile.getParentFile().getName(),
         SegmentDirectoryPaths.V3_SUBDIRECTORY_NAME);
     indexSegment.destroy();
diff --git a/pinot-segment-local/src/test/java/org/apache/pinot/segment/local/segment/index/loader/SegmentPreProcessorTest.java b/pinot-segment-local/src/test/java/org/apache/pinot/segment/local/segment/index/loader/SegmentPreProcessorTest.java
index cfe3a9aadd..8e549d2ba3 100644
--- a/pinot-segment-local/src/test/java/org/apache/pinot/segment/local/segment/index/loader/SegmentPreProcessorTest.java
+++ b/pinot-segment-local/src/test/java/org/apache/pinot/segment/local/segment/index/loader/SegmentPreProcessorTest.java
@@ -1415,8 +1415,8 @@ public class SegmentPreProcessorTest {
     // V1 use separate file for each column index.
     File iiFile = new File(_indexDir, strColumn + V1Constants.Indexes.BITMAP_INVERTED_INDEX_FILE_EXTENSION);
     File rgFile = new File(_indexDir, strColumn + V1Constants.Indexes.BITMAP_RANGE_INDEX_FILE_EXTENSION);
-    File txtFile = new File(_indexDir, strColumn + V1Constants.Indexes.LUCENE_TEXT_INDEX_FILE_EXTENSION);
-    File fstFile = new File(_indexDir, strColumn + V1Constants.Indexes.FST_INDEX_FILE_EXTENSION);
+    File txtFile = new File(_indexDir, strColumn + V1Constants.Indexes.LUCENE_V9_TEXT_INDEX_FILE_EXTENSION);
+    File fstFile = new File(_indexDir, strColumn + V1Constants.Indexes.LUCENE_V9_FST_INDEX_FILE_EXTENSION);
     File bfFile = new File(_indexDir, strColumn + V1Constants.Indexes.BLOOM_FILTER_FILE_EXTENSION);
 
     assertFalse(iiFile.exists());
diff --git a/pinot-segment-local/src/test/java/org/apache/pinot/segment/local/segment/store/FilePerIndexDirectoryTest.java b/pinot-segment-local/src/test/java/org/apache/pinot/segment/local/segment/store/FilePerIndexDirectoryTest.java
index 7cc4b7c70c..e115414b3e 100644
--- a/pinot-segment-local/src/test/java/org/apache/pinot/segment/local/segment/store/FilePerIndexDirectoryTest.java
+++ b/pinot-segment-local/src/test/java/org/apache/pinot/segment/local/segment/store/FilePerIndexDirectoryTest.java
@@ -230,12 +230,13 @@ public class FilePerIndexDirectoryTest {
 
       // Both files for TextIndex should be removed.
       fpi.removeIndex("foo", StandardIndexes.text());
-      assertFalse(new File(TEMP_DIR, "foo" + V1Constants.Indexes.LUCENE_TEXT_INDEX_FILE_EXTENSION).exists());
+      assertFalse(new File(TEMP_DIR, "foo" + V1Constants.Indexes.LUCENE_V9_TEXT_INDEX_FILE_EXTENSION).exists());
       assertFalse(
-          new File(TEMP_DIR, "foo" + V1Constants.Indexes.LUCENE_TEXT_INDEX_DOCID_MAPPING_FILE_EXTENSION).exists());
+          new File(TEMP_DIR, "foo" + V1Constants.Indexes.LUCENE_V9_TEXT_INDEX_DOCID_MAPPING_FILE_EXTENSION).exists());
     }
-    assertTrue(new File(TEMP_DIR, "bar" + V1Constants.Indexes.LUCENE_TEXT_INDEX_FILE_EXTENSION).exists());
-    assertTrue(new File(TEMP_DIR, "bar" + V1Constants.Indexes.LUCENE_TEXT_INDEX_DOCID_MAPPING_FILE_EXTENSION).exists());
+    assertTrue(new File(TEMP_DIR, "bar" + V1Constants.Indexes.LUCENE_V9_TEXT_INDEX_FILE_EXTENSION).exists());
+    assertTrue(
+        new File(TEMP_DIR, "bar" + V1Constants.Indexes.LUCENE_V9_TEXT_INDEX_DOCID_MAPPING_FILE_EXTENSION).exists());
 
     // Read indices back and check the content.
     try (FilePerIndexDirectory fpi = new FilePerIndexDirectory(TEMP_DIR, _segmentMetadata, ReadMode.mmap)) {
diff --git a/pinot-segment-local/src/test/java/org/apache/pinot/segment/local/segment/store/SingleFileIndexDirectoryTest.java b/pinot-segment-local/src/test/java/org/apache/pinot/segment/local/segment/store/SingleFileIndexDirectoryTest.java
index 32feaa0177..79778c8104 100644
--- a/pinot-segment-local/src/test/java/org/apache/pinot/segment/local/segment/store/SingleFileIndexDirectoryTest.java
+++ b/pinot-segment-local/src/test/java/org/apache/pinot/segment/local/segment/store/SingleFileIndexDirectoryTest.java
@@ -263,12 +263,13 @@ public class SingleFileIndexDirectoryTest {
 
       // Both files for TextIndex should be removed.
       sfd.removeIndex("foo", StandardIndexes.text());
-      assertFalse(new File(TEMP_DIR, "foo" + V1Constants.Indexes.LUCENE_TEXT_INDEX_FILE_EXTENSION).exists());
+      assertFalse(new File(TEMP_DIR, "foo" + V1Constants.Indexes.LUCENE_V9_TEXT_INDEX_FILE_EXTENSION).exists());
       assertFalse(
-          new File(TEMP_DIR, "foo" + V1Constants.Indexes.LUCENE_TEXT_INDEX_DOCID_MAPPING_FILE_EXTENSION).exists());
+          new File(TEMP_DIR, "foo" + V1Constants.Indexes.LUCENE_V9_TEXT_INDEX_DOCID_MAPPING_FILE_EXTENSION).exists());
     }
-    assertTrue(new File(TEMP_DIR, "bar" + V1Constants.Indexes.LUCENE_TEXT_INDEX_FILE_EXTENSION).exists());
-    assertTrue(new File(TEMP_DIR, "bar" + V1Constants.Indexes.LUCENE_TEXT_INDEX_DOCID_MAPPING_FILE_EXTENSION).exists());
+    assertTrue(new File(TEMP_DIR, "bar" + V1Constants.Indexes.LUCENE_V9_TEXT_INDEX_FILE_EXTENSION).exists());
+    assertTrue(
+        new File(TEMP_DIR, "bar" + V1Constants.Indexes.LUCENE_V9_TEXT_INDEX_DOCID_MAPPING_FILE_EXTENSION).exists());
 
     // Read indices back and check the content.
     try (SingleFileIndexDirectory sfd = new SingleFileIndexDirectory(TEMP_DIR, _segmentMetadata, ReadMode.mmap)) {
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 8900510ab6..218d174614 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
@@ -44,14 +44,17 @@ public class V1Constants {
     public static final String UNSORTED_MV_FORWARD_INDEX_FILE_EXTENSION = ".mv.fwd";
     public static final String BITMAP_INVERTED_INDEX_FILE_EXTENSION = ".bitmap.inv";
     public static final String BITMAP_RANGE_INDEX_FILE_EXTENSION = ".bitmap.range";
-    public static final String FST_INDEX_FILE_EXTENSION = ".lucene.v9.fst";
     public static final String JSON_INDEX_FILE_EXTENSION = ".json.idx";
     public static final String NATIVE_TEXT_INDEX_FILE_EXTENSION = ".nativetext.idx";
     public static final String H3_INDEX_FILE_EXTENSION = ".h3.idx";
     public static final String BLOOM_FILTER_FILE_EXTENSION = ".bloom";
     public static final String NULLVALUE_VECTOR_FILE_EXTENSION = ".bitmap.nullvalue";
-    public static final String LUCENE_TEXT_INDEX_DOCID_MAPPING_FILE_EXTENSION = ".lucene.v9.mapping";
-    public static final String LUCENE_TEXT_INDEX_FILE_EXTENSION = ".lucene.v9.index";
+    public static final String LUCENE_FST_INDEX_FILE_EXTENSION = ".lucene.fst";
+    public static final String LUCENE_TEXT_INDEX_DOCID_MAPPING_FILE_EXTENSION = ".lucene.mapping";
+    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_DOCID_MAPPING_FILE_EXTENSION = ".lucene.v9.mapping";
+    public static final String LUCENE_V9_TEXT_INDEX_FILE_EXTENSION = ".lucene.v9.index";
   }
 
   public static class MetadataKeys {
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 61686c38f7..98b60df91e 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
@@ -79,8 +79,13 @@ public class SegmentDirectoryPaths {
    */
   @Nullable
   public static File findTextIndexIndexFile(File indexDir, String column) {
-    String luceneIndexDirectory = column + V1Constants.Indexes.LUCENE_TEXT_INDEX_FILE_EXTENSION;
-    return findFormatFile(indexDir, luceneIndexDirectory);
+    String luceneIndexDirectory = column + V1Constants.Indexes.LUCENE_V9_TEXT_INDEX_FILE_EXTENSION;
+    File indexFormatFile = findFormatFile(indexDir, luceneIndexDirectory);
+    if (indexFormatFile == null) {
+      luceneIndexDirectory = column + V1Constants.Indexes.LUCENE_TEXT_INDEX_FILE_EXTENSION;
+      indexFormatFile = findFormatFile(indexDir, luceneIndexDirectory);
+    }
+    return indexFormatFile;
   }
 
   /**
@@ -96,15 +101,25 @@ public class SegmentDirectoryPaths {
   }
 
   public static File findFSTIndexIndexFile(File indexDir, String column) {
-    String luceneIndexDirectory = column + V1Constants.Indexes.FST_INDEX_FILE_EXTENSION;
-    return findFormatFile(indexDir, luceneIndexDirectory);
+    String luceneIndexDirectory = column + V1Constants.Indexes.LUCENE_V9_FST_INDEX_FILE_EXTENSION;
+    File formatFile = findFormatFile(indexDir, luceneIndexDirectory);
+    if (formatFile == null) {
+      luceneIndexDirectory = column + V1Constants.Indexes.LUCENE_FST_INDEX_FILE_EXTENSION;
+      formatFile = findFormatFile(indexDir, luceneIndexDirectory);
+    }
+    return formatFile;
   }
 
   @Nullable
   @VisibleForTesting
   public static File findTextIndexDocIdMappingFile(File indexDir, String column) {
-    String file = column + V1Constants.Indexes.LUCENE_TEXT_INDEX_DOCID_MAPPING_FILE_EXTENSION;
-    return findFormatFile(indexDir, file);
+    String file = column + V1Constants.Indexes.LUCENE_V9_TEXT_INDEX_DOCID_MAPPING_FILE_EXTENSION;
+    File formatFile = findFormatFile(indexDir, file);
+    if (formatFile == null) {
+      file = column + V1Constants.Indexes.LUCENE_TEXT_INDEX_DOCID_MAPPING_FILE_EXTENSION;
+      formatFile = findFormatFile(indexDir, file);
+    }
+    return formatFile;
   }
 
   /**
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 886c70ea80..acc50ae301 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,4 +1,15 @@
 {
+  "fieldConfigList": [
+    {
+      "encodingType": "RAW",
+      "indexType": "TEXT",
+      "name": "Text",
+      "properties": {
+        "deriveNumDocsPerChunkForRawIndex": "true",
+        "rawIndexWriterVersion": "3"
+      }
+    }
+  ],
   "tableName": "fineFoodReviews",
   "tableType": "OFFLINE",
   "segmentsConfig": {
@@ -11,6 +22,7 @@
   },
   "tableIndexConfig": {
     "loadMode": "MMAP",
+    "noDictionaryColumns": ["Text"],
     "invertedIndexColumns": [
     ]
   },
diff --git a/pom.xml b/pom.xml
index 29a739be48..fac14ba080 100644
--- a/pom.xml
+++ b/pom.xml
@@ -1403,6 +1403,30 @@
         <artifactId>woodstox-core</artifactId>
         <version>${woodstox.version}</version>
       </dependency>
+
+      <!-- Lucene dependencies -->
+      <dependency>
+        <groupId>org.apache.lucene</groupId>
+        <artifactId>lucene-backward-codecs</artifactId>
+        <version>${lucene.version}</version>
+      </dependency>
+      <dependency>
+        <groupId>org.apache.lucene</groupId>
+        <artifactId>lucene-core</artifactId>
+        <version>${lucene.version}</version>
+      </dependency>
+      <dependency>
+        <groupId>org.apache.lucene</groupId>
+        <artifactId>lucene-queryparser</artifactId>
+        <version>${lucene.version}</version>
+      </dependency>
+      <dependency>
+        <groupId>org.apache.lucene</groupId>
+        <artifactId>lucene-analysis-common</artifactId>
+        <version>${lucene.version}</version>
+      </dependency>
+      <!-- Lucene dependencies end -->
+
     </dependencies>
   </dependencyManagement>
   <build>


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