You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@pinot.apache.org by ja...@apache.org on 2023/12/05 22:09:40 UTC

(pinot) branch master updated: Configurable Lucene analyzer (#12027)

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

jackie 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 67cdb27721 Configurable Lucene analyzer (#12027)
67cdb27721 is described below

commit 67cdb27721aa156882d8e39cf18b317bb27c92bc
Author: Jack Luo <ja...@mail.utoronto.ca>
AuthorDate: Tue Dec 5 17:09:33 2023 -0500

    Configurable Lucene analyzer (#12027)
---
 .../invertedindex/RealtimeLuceneTextIndex.java     | 15 ++++-----
 .../creator/impl/text/LuceneTextIndexCreator.java  | 37 +++++++++++++---------
 .../index/readers/text/LuceneTextIndexReader.java  | 12 ++++---
 .../segment/index/text/TextIndexConfigBuilder.java |  3 ++
 .../local/segment/index/text/TextIndexType.java    |  3 +-
 .../local/segment/store/TextIndexUtils.java        |  7 ++++
 .../invertedindex/LuceneMutableTextIndexTest.java  |  5 ++-
 .../NativeAndLuceneMutableTextIndexTest.java       |  7 ++--
 .../segment/store/FilePerIndexDirectoryTest.java   | 17 +++++-----
 .../store/SingleFileIndexDirectoryTest.java        | 17 +++++-----
 .../pinot/segment/spi/index/TextIndexConfig.java   | 28 +++++++++++++---
 .../apache/pinot/spi/config/table/FieldConfig.java |  3 ++
 12 files changed, 102 insertions(+), 52 deletions(-)

diff --git a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/realtime/impl/invertedindex/RealtimeLuceneTextIndex.java b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/realtime/impl/invertedindex/RealtimeLuceneTextIndex.java
index b43d05ec0b..2a35b2da60 100644
--- a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/realtime/impl/invertedindex/RealtimeLuceneTextIndex.java
+++ b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/realtime/impl/invertedindex/RealtimeLuceneTextIndex.java
@@ -19,10 +19,9 @@
 package org.apache.pinot.segment.local.realtime.impl.invertedindex;
 
 import java.io.File;
-import java.util.List;
 import java.util.concurrent.Callable;
 import java.util.concurrent.Future;
-import org.apache.lucene.analysis.standard.StandardAnalyzer;
+import org.apache.lucene.analysis.Analyzer;
 import org.apache.lucene.document.Document;
 import org.apache.lucene.index.IndexWriter;
 import org.apache.lucene.queryparser.classic.QueryParser;
@@ -31,6 +30,7 @@ import org.apache.lucene.search.Query;
 import org.apache.lucene.search.SearcherManager;
 import org.apache.pinot.segment.local.indexsegment.mutable.MutableSegmentImpl;
 import org.apache.pinot.segment.local.segment.creator.impl.text.LuceneTextIndexCreator;
+import org.apache.pinot.segment.spi.index.TextIndexConfig;
 import org.apache.pinot.segment.spi.index.mutable.MutableTextIndex;
 import org.roaringbitmap.IntIterator;
 import org.roaringbitmap.buffer.ImmutableRoaringBitmap;
@@ -50,7 +50,7 @@ public class RealtimeLuceneTextIndex implements MutableTextIndex {
       RealtimeLuceneTextIndexSearcherPool.getInstance();
   private final LuceneTextIndexCreator _indexCreator;
   private SearcherManager _searcherManager;
-  private final StandardAnalyzer _analyzer = new StandardAnalyzer();
+  private Analyzer _analyzer;
   private final String _column;
   private final String _segmentName;
 
@@ -60,11 +60,9 @@ public class RealtimeLuceneTextIndex implements MutableTextIndex {
    * @param column column name
    * @param segmentIndexDir realtime segment consumer dir
    * @param segmentName realtime segment name
-   * @param stopWordsInclude the words to include in addition to the default stop word list
-   * @param stopWordsExclude stop words to exclude from default stop words
+   * @param config the table index config
    */
-  public RealtimeLuceneTextIndex(String column, File segmentIndexDir, String segmentName,
-      List<String> stopWordsInclude, List<String> stopWordsExclude, boolean useCompoundFile, int maxBufferSizeMB) {
+  public RealtimeLuceneTextIndex(String column, File segmentIndexDir, String segmentName, TextIndexConfig config) {
     _column = column;
     _segmentName = segmentName;
     try {
@@ -78,9 +76,10 @@ public class RealtimeLuceneTextIndex implements MutableTextIndex {
       // for realtime
       _indexCreator =
           new LuceneTextIndexCreator(column, new File(segmentIndexDir.getAbsolutePath() + "/" + segmentName),
-              false /* commitOnClose */, stopWordsInclude, stopWordsExclude, useCompoundFile, maxBufferSizeMB);
+              false /* commitOnClose */, config);
       IndexWriter indexWriter = _indexCreator.getIndexWriter();
       _searcherManager = new SearcherManager(indexWriter, false, false, null);
+      _analyzer = _indexCreator.getIndexWriter().getConfig().getAnalyzer();
     } catch (Exception e) {
       LOGGER.error("Failed to instantiate realtime Lucene index reader for column {}, exception {}", column,
           e.getMessage());
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 771b42da3f..59d1297f7e 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
@@ -22,8 +22,7 @@ import java.io.File;
 import java.io.IOException;
 import java.util.Arrays;
 import java.util.HashSet;
-import java.util.List;
-import javax.annotation.Nullable;
+import org.apache.lucene.analysis.Analyzer;
 import org.apache.lucene.analysis.CharArraySet;
 import org.apache.lucene.analysis.standard.StandardAnalyzer;
 import org.apache.lucene.document.Document;
@@ -42,6 +41,8 @@ import org.apache.pinot.segment.spi.V1Constants;
 import org.apache.pinot.segment.spi.creator.IndexCreationContext;
 import org.apache.pinot.segment.spi.index.TextIndexConfig;
 import org.apache.pinot.segment.spi.index.creator.DictionaryBasedInvertedIndexCreator;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 
 /**
@@ -50,6 +51,7 @@ import org.apache.pinot.segment.spi.index.creator.DictionaryBasedInvertedIndexCr
  * and realtime from {@link RealtimeLuceneTextIndex}
  */
 public class LuceneTextIndexCreator extends AbstractTextIndexCreator {
+  private static final Logger LOGGER = LoggerFactory.getLogger(LuceneTextIndexCreator.class);
   public static final String LUCENE_INDEX_DOC_ID_COLUMN_NAME = "DocID";
 
   private final String _textColumn;
@@ -89,26 +91,33 @@ public class LuceneTextIndexCreator extends AbstractTextIndexCreator {
    *               no need to commit the index from the realtime side. So when the realtime segment
    *               is destroyed (which is after the realtime segment has been committed and converted
    *               to offline), we close this lucene index writer to release resources but don't commit.
-   * @param stopWordsInclude the words to include in addition to the default stop word list
-   * @param stopWordsExclude the words to exclude from the default stop word list
+   * @param config the text index config
    */
-  public LuceneTextIndexCreator(String column, File segmentIndexDir, boolean commit,
-      @Nullable List<String> stopWordsInclude, @Nullable List<String> stopWordsExclude, boolean useCompoundFile,
-      int maxBufferSizeMB) {
+  public LuceneTextIndexCreator(String column, File segmentIndexDir, boolean commit, TextIndexConfig config) {
     _textColumn = column;
+    String luceneAnalyzerClass = config.getLuceneAnalyzerClass();
     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 = getV1TextIndexFile(segmentIndexDir);
       _indexDirectory = FSDirectory.open(indexFile.toPath());
 
-      StandardAnalyzer standardAnalyzer =
-          TextIndexUtils.getStandardAnalyzerWithCustomizedStopWords(stopWordsInclude, stopWordsExclude);
-      IndexWriterConfig indexWriterConfig = new IndexWriterConfig(standardAnalyzer);
-      indexWriterConfig.setRAMBufferSizeMB(maxBufferSizeMB);
+      Analyzer luceneAnalyzer;
+      if (luceneAnalyzerClass.isEmpty() || luceneAnalyzerClass.equals(StandardAnalyzer.class.getName())) {
+        luceneAnalyzer = TextIndexUtils.getStandardAnalyzerWithCustomizedStopWords(
+            config.getStopWordsInclude(), config.getStopWordsExclude());
+      } else {
+        luceneAnalyzer = TextIndexUtils.getAnalyzerFromFQCN(luceneAnalyzerClass);
+      }
+
+      IndexWriterConfig indexWriterConfig = new IndexWriterConfig(luceneAnalyzer);
+      indexWriterConfig.setRAMBufferSizeMB(config.getLuceneMaxBufferSizeMB());
       indexWriterConfig.setCommitOnClose(commit);
-      indexWriterConfig.setUseCompoundFile(useCompoundFile);
+      indexWriterConfig.setUseCompoundFile(config.isLuceneUseCompoundFile());
       _indexWriter = new IndexWriter(_indexDirectory, indexWriterConfig);
+    } catch (ReflectiveOperationException e) {
+      throw new RuntimeException(
+          "Failed to instantiate " + luceneAnalyzerClass + " lucene analyzer for column: " + column, e);
     } catch (Exception e) {
       throw new RuntimeException(
           "Caught exception while instantiating the LuceneTextIndexCreator for column: " + column, e);
@@ -116,9 +125,7 @@ public class LuceneTextIndexCreator extends AbstractTextIndexCreator {
   }
 
   public LuceneTextIndexCreator(IndexCreationContext context, TextIndexConfig indexConfig) {
-    this(context.getFieldSpec().getName(), context.getIndexDir(), context.isTextCommitOnClose(),
-        indexConfig.getStopWordsInclude(), indexConfig.getStopWordsExclude(), indexConfig.isLuceneUseCompoundFile(),
-        indexConfig.getLuceneMaxBufferSizeMB());
+    this(context.getFieldSpec().getName(), context.getIndexDir(), context.isTextCommitOnClose(), indexConfig);
   }
 
   public IndexWriter getIndexWriter() {
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 e249412e6b..abe56e6736 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
@@ -24,6 +24,7 @@ import java.io.IOException;
 import java.nio.ByteOrder;
 import java.util.Map;
 import javax.annotation.Nullable;
+import org.apache.lucene.analysis.Analyzer;
 import org.apache.lucene.analysis.standard.StandardAnalyzer;
 import org.apache.lucene.document.Document;
 import org.apache.lucene.index.DirectoryReader;
@@ -63,7 +64,7 @@ public class LuceneTextIndexReader implements TextIndexReader {
   private final IndexSearcher _indexSearcher;
   private final String _column;
   private final DocIdTranslator _docIdTranslator;
-  private final StandardAnalyzer _standardAnalyzer;
+  private final Analyzer _analyzer;
   private boolean _useANDForMultiTermQueries = false;
 
   public LuceneTextIndexReader(String column, File indexDir, int numDocs, TextIndexConfig config) {
@@ -84,8 +85,11 @@ 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());
+      String luceneAnalyzerClass = config.getLuceneAnalyzerClass();
+      _analyzer = luceneAnalyzerClass.equals(StandardAnalyzer.class.getName())
+              ? TextIndexUtils.getStandardAnalyzerWithCustomizedStopWords(
+              config.getStopWordsInclude(), config.getStopWordsExclude())
+              : TextIndexUtils.getAnalyzerFromFQCN(luceneAnalyzerClass);
       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,
@@ -146,7 +150,7 @@ public class LuceneTextIndexReader implements TextIndexReader {
       // 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, _standardAnalyzer);
+      QueryParser parser = new QueryParser(_column, _analyzer);
       if (_useANDForMultiTermQueries) {
         parser.setDefaultOperator(QueryParser.Operator.AND);
       }
diff --git a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/text/TextIndexConfigBuilder.java b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/text/TextIndexConfigBuilder.java
index 980cb04ccc..1c14226c0b 100644
--- a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/text/TextIndexConfigBuilder.java
+++ b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/text/TextIndexConfigBuilder.java
@@ -61,6 +61,9 @@ public class TextIndexConfigBuilder extends TextIndexConfig.AbstractBuilder {
             Integer.parseInt(textIndexProperties.get(FieldConfig.TEXT_INDEX_LUCENE_MAX_BUFFER_SIZE_MB));
       }
 
+      if (textIndexProperties.get(FieldConfig.TEXT_INDEX_LUCENE_ANALYZER_CLASS) != null) {
+        _luceneAnalyzerClass = textIndexProperties.get(FieldConfig.TEXT_INDEX_LUCENE_ANALYZER_CLASS);
+      }
 
       for (Map.Entry<String, String> entry : textIndexProperties.entrySet()) {
         if (entry.getKey().equalsIgnoreCase(FieldConfig.TEXT_FST_TYPE)) {
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 4ad6eb3a94..596380d81b 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
@@ -197,7 +197,6 @@ public class TextIndexType extends AbstractIndexType<TextIndexConfig, TextIndexR
       throw new IllegalArgumentException("A consumer directory is required");
     }
     return new RealtimeLuceneTextIndex(context.getFieldSpec().getName(), context.getConsumerDir(),
-        context.getSegmentName(), config.getStopWordsInclude(), config.getStopWordsExclude(),
-        config.isLuceneUseCompoundFile(), config.getLuceneMaxBufferSizeMB());
+        context.getSegmentName(), config);
   }
 }
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 235025acde..883ad47c4d 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
@@ -28,6 +28,7 @@ import java.util.stream.Collectors;
 import javax.annotation.Nonnull;
 import javax.annotation.Nullable;
 import org.apache.commons.io.FileUtils;
+import org.apache.lucene.analysis.Analyzer;
 import org.apache.lucene.analysis.CharArraySet;
 import org.apache.lucene.analysis.standard.StandardAnalyzer;
 import org.apache.pinot.segment.local.segment.creator.impl.text.LuceneTextIndexCreator;
@@ -112,6 +113,12 @@ public class TextIndexUtils {
         .map(String::trim).collect(Collectors.toList());
   }
 
+  public static Analyzer getAnalyzerFromClassName(String luceneAnalyzerClass) throws
+      ReflectiveOperationException {
+    // Support instantiation with default constructor for now unless customized
+    return (Analyzer) Class.forName(luceneAnalyzerFQCN).getConstructor().newInstance();
+  }
+
   public static StandardAnalyzer getStandardAnalyzerWithCustomizedStopWords(@Nullable List<String> stopWordsInclude,
       @Nullable List<String> stopWordsExclude) {
     HashSet<String> stopWordSet = LuceneTextIndexCreator.getDefaultEnglishStopWordsSet();
diff --git a/pinot-segment-local/src/test/java/org/apache/pinot/segment/local/realtime/impl/invertedindex/LuceneMutableTextIndexTest.java b/pinot-segment-local/src/test/java/org/apache/pinot/segment/local/realtime/impl/invertedindex/LuceneMutableTextIndexTest.java
index e7046f051a..e8066bb9e7 100644
--- a/pinot-segment-local/src/test/java/org/apache/pinot/segment/local/realtime/impl/invertedindex/LuceneMutableTextIndexTest.java
+++ b/pinot-segment-local/src/test/java/org/apache/pinot/segment/local/realtime/impl/invertedindex/LuceneMutableTextIndexTest.java
@@ -25,6 +25,7 @@ import java.util.concurrent.Executors;
 import java.util.concurrent.Future;
 import org.apache.commons.io.FileUtils;
 import org.apache.lucene.search.SearcherManager;
+import org.apache.pinot.segment.spi.index.TextIndexConfig;
 import org.roaringbitmap.buffer.ImmutableRoaringBitmap;
 import org.roaringbitmap.buffer.MutableRoaringBitmap;
 import org.testng.annotations.AfterClass;
@@ -57,8 +58,10 @@ public class LuceneMutableTextIndexTest {
   @BeforeClass
   public void setUp()
       throws Exception {
+    TextIndexConfig config =
+            new TextIndexConfig(false, null, null, false, false, null, null, true, 500, null);
     _realtimeLuceneTextIndex =
-        new RealtimeLuceneTextIndex(TEXT_COLUMN_NAME, INDEX_DIR, "fooBar", null, null, true, 500);
+        new RealtimeLuceneTextIndex(TEXT_COLUMN_NAME, INDEX_DIR, "fooBar", config);
     String[][] documents = getTextData();
     String[][] repeatedDocuments = getRepeatedData();
 
diff --git a/pinot-segment-local/src/test/java/org/apache/pinot/segment/local/realtime/impl/invertedindex/NativeAndLuceneMutableTextIndexTest.java b/pinot-segment-local/src/test/java/org/apache/pinot/segment/local/realtime/impl/invertedindex/NativeAndLuceneMutableTextIndexTest.java
index a7c85c6b8d..211614b5b2 100644
--- a/pinot-segment-local/src/test/java/org/apache/pinot/segment/local/realtime/impl/invertedindex/NativeAndLuceneMutableTextIndexTest.java
+++ b/pinot-segment-local/src/test/java/org/apache/pinot/segment/local/realtime/impl/invertedindex/NativeAndLuceneMutableTextIndexTest.java
@@ -23,6 +23,7 @@ import java.util.ArrayList;
 import java.util.List;
 import org.apache.commons.io.FileUtils;
 import org.apache.lucene.search.SearcherManager;
+import org.apache.pinot.segment.spi.index.TextIndexConfig;
 import org.testng.annotations.AfterClass;
 import org.testng.annotations.BeforeClass;
 import org.testng.annotations.Test;
@@ -70,12 +71,14 @@ public class NativeAndLuceneMutableTextIndexTest {
   @BeforeClass
   public void setUp()
       throws Exception {
+    TextIndexConfig config =
+        new TextIndexConfig(false, null, null, false, false, null, null, true, 500, null);
     _realtimeLuceneTextIndex =
-        new RealtimeLuceneTextIndex(TEXT_COLUMN_NAME, INDEX_DIR, "fooBar", null, null, true, 500);
+        new RealtimeLuceneTextIndex(TEXT_COLUMN_NAME, INDEX_DIR, "fooBar", config);
     _nativeMutableTextIndex = new NativeMutableTextIndex(TEXT_COLUMN_NAME);
 
     _realtimeLuceneMVTextIndex =
-        new RealtimeLuceneTextIndex(MV_TEXT_COLUMN_NAME, INDEX_DIR, "fooBar", null, null, true, 500);
+        new RealtimeLuceneTextIndex(MV_TEXT_COLUMN_NAME, INDEX_DIR, "fooBar", config);
     _nativeMutableMVTextIndex = new NativeMutableTextIndex(MV_TEXT_COLUMN_NAME);
 
     String[] documents = getTextData();
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 ca933e01b3..61fa9f0319 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
@@ -32,6 +32,7 @@ import org.apache.pinot.segment.local.segment.index.readers.text.LuceneTextIndex
 import org.apache.pinot.segment.spi.V1Constants;
 import org.apache.pinot.segment.spi.creator.SegmentVersion;
 import org.apache.pinot.segment.spi.index.StandardIndexes;
+import org.apache.pinot.segment.spi.index.TextIndexConfig;
 import org.apache.pinot.segment.spi.index.metadata.SegmentMetadataImpl;
 import org.apache.pinot.segment.spi.memory.PinotDataBuffer;
 import org.apache.pinot.segment.spi.store.ColumnIndexDirectory;
@@ -200,11 +201,11 @@ public class FilePerIndexDirectoryTest {
   @Test
   public void testRemoveTextIndices()
       throws IOException {
+    TextIndexConfig config =
+            new TextIndexConfig(false, null, null, false, false, null, null, true, 500, null);
     try (FilePerIndexDirectory fpi = new FilePerIndexDirectory(TEMP_DIR, _segmentMetadata, ReadMode.mmap);
-        LuceneTextIndexCreator fooCreator = new LuceneTextIndexCreator("foo", TEMP_DIR, true,
-            null, null, true, 500);
-        LuceneTextIndexCreator barCreator = new LuceneTextIndexCreator("bar", TEMP_DIR, true,
-            null, null, true, 500)) {
+         LuceneTextIndexCreator fooCreator = new LuceneTextIndexCreator("foo", TEMP_DIR, true, config);
+        LuceneTextIndexCreator barCreator = new LuceneTextIndexCreator("bar", TEMP_DIR, true, config)) {
       PinotDataBuffer buf = fpi.newBuffer("col1", StandardIndexes.forward(), 1024);
       buf.putInt(0, 1);
 
@@ -263,12 +264,12 @@ public class FilePerIndexDirectoryTest {
   @Test
   public void testGetColumnIndices()
       throws IOException {
+    TextIndexConfig config =
+            new TextIndexConfig(false, null, null, false, false, null, null, true, 500, null);
     // Write sth to buffers and flush them to index files on disk
     try (FilePerIndexDirectory fpi = new FilePerIndexDirectory(TEMP_DIR, _segmentMetadata, ReadMode.mmap);
-        LuceneTextIndexCreator fooCreator = new LuceneTextIndexCreator("foo", TEMP_DIR, true,
-            null, null, true, 500);
-        LuceneTextIndexCreator barCreator = new LuceneTextIndexCreator("bar", TEMP_DIR, true,
-            null, null, true, 500)) {
+        LuceneTextIndexCreator fooCreator = new LuceneTextIndexCreator("foo", TEMP_DIR, true, config);
+        LuceneTextIndexCreator barCreator = new LuceneTextIndexCreator("bar", TEMP_DIR, true, config)) {
       PinotDataBuffer buf = fpi.newBuffer("col1", StandardIndexes.forward(), 1024);
       buf.putInt(0, 111);
       buf = fpi.newBuffer("col2", StandardIndexes.dictionary(), 1024);
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 3c8ba99688..7943edcde7 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
@@ -39,6 +39,7 @@ import org.apache.pinot.segment.local.segment.index.readers.text.LuceneTextIndex
 import org.apache.pinot.segment.spi.V1Constants;
 import org.apache.pinot.segment.spi.creator.SegmentVersion;
 import org.apache.pinot.segment.spi.index.StandardIndexes;
+import org.apache.pinot.segment.spi.index.TextIndexConfig;
 import org.apache.pinot.segment.spi.index.metadata.SegmentMetadataImpl;
 import org.apache.pinot.segment.spi.memory.PinotDataBuffer;
 import org.apache.pinot.segment.spi.store.ColumnIndexDirectory;
@@ -233,11 +234,11 @@ public class SingleFileIndexDirectoryTest {
   @Test
   public void testRemoveTextIndices()
       throws IOException, ConfigurationException {
+    TextIndexConfig config =
+            new TextIndexConfig(false, null, null, false, false, null, null, true, 500, null);
     try (SingleFileIndexDirectory sfd = new SingleFileIndexDirectory(TEMP_DIR, _segmentMetadata, ReadMode.mmap);
-        LuceneTextIndexCreator fooCreator = new LuceneTextIndexCreator("foo", TEMP_DIR, true,
-            null, null, true, 500);
-        LuceneTextIndexCreator barCreator = new LuceneTextIndexCreator("bar", TEMP_DIR, true,
-            null, null, true, 500)) {
+        LuceneTextIndexCreator fooCreator = new LuceneTextIndexCreator("foo", TEMP_DIR, true, config);
+        LuceneTextIndexCreator barCreator = new LuceneTextIndexCreator("bar", TEMP_DIR, true, config)) {
       PinotDataBuffer buf = sfd.newBuffer("col1", StandardIndexes.forward(), 1024);
       buf.putInt(0, 1);
 
@@ -339,11 +340,11 @@ public class SingleFileIndexDirectoryTest {
   @Test
   public void testGetColumnIndices()
       throws Exception {
+    TextIndexConfig config =
+            new TextIndexConfig(false, null, null, false, false, null, null, true, 500, null);
     try (SingleFileIndexDirectory sfd = new SingleFileIndexDirectory(TEMP_DIR, _segmentMetadata, ReadMode.mmap);
-        LuceneTextIndexCreator fooCreator = new LuceneTextIndexCreator("foo", TEMP_DIR, true,
-            null, null, true, 500);
-        LuceneTextIndexCreator barCreator = new LuceneTextIndexCreator("bar", TEMP_DIR, true,
-            null, null, true, 500)) {
+        LuceneTextIndexCreator fooCreator = new LuceneTextIndexCreator("foo", TEMP_DIR, true, config);
+        LuceneTextIndexCreator barCreator = new LuceneTextIndexCreator("bar", TEMP_DIR, true, config)) {
       PinotDataBuffer buf = sfd.newBuffer("col1", StandardIndexes.forward(), 1024);
       buf.putInt(0, 111);
       buf = sfd.newBuffer("col2", StandardIndexes.dictionary(), 1024);
diff --git a/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/index/TextIndexConfig.java b/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/index/TextIndexConfig.java
index 474bfd67e1..ff8e851084 100644
--- a/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/index/TextIndexConfig.java
+++ b/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/index/TextIndexConfig.java
@@ -28,6 +28,7 @@ import java.util.Map;
 import java.util.Objects;
 import javax.annotation.Nullable;
 import org.apache.pinot.spi.config.table.FSTType;
+import org.apache.pinot.spi.config.table.FieldConfig;
 import org.apache.pinot.spi.config.table.IndexConfig;
 
 
@@ -36,7 +37,7 @@ public class TextIndexConfig extends IndexConfig {
   private static final boolean LUCENE_INDEX_DEFAULT_USE_COMPOUND_FILE = true;
   public static final TextIndexConfig DISABLED =
       new TextIndexConfig(true, null, null, false, false, Collections.emptyList(), Collections.emptyList(), false,
-          LUCENE_INDEX_DEFAULT_MAX_BUFFER_SIZE_MB);
+          LUCENE_INDEX_DEFAULT_MAX_BUFFER_SIZE_MB, null);
   private final FSTType _fstType;
   @Nullable
   private final Object _rawValueForTextIndex;
@@ -46,6 +47,7 @@ public class TextIndexConfig extends IndexConfig {
   private final List<String> _stopWordsExclude;
   private final boolean _luceneUseCompoundFile;
   private final int _luceneMaxBufferSizeMB;
+  private final String _luceneAnalyzerClass;
 
   @JsonCreator
   public TextIndexConfig(
@@ -57,7 +59,9 @@ public class TextIndexConfig extends IndexConfig {
       @JsonProperty("stopWordsInclude") List<String> stopWordsInclude,
       @JsonProperty("stopWordsExclude") List<String> stopWordsExclude,
       @JsonProperty("luceneUseCompoundFile") Boolean luceneUseCompoundFile,
-      @JsonProperty("luceneMaxBufferSizeMB") Integer luceneMaxBufferSizeMB) {
+      @JsonProperty("luceneMaxBufferSizeMB") Integer luceneMaxBufferSizeMB,
+      @JsonProperty("luceneAnalyzerClass") String luceneAnalyzerClass
+      ) {
     super(disabled);
     _fstType = fstType;
     _rawValueForTextIndex = rawValueForTextIndex;
@@ -69,6 +73,8 @@ public class TextIndexConfig extends IndexConfig {
         luceneUseCompoundFile == null ? LUCENE_INDEX_DEFAULT_USE_COMPOUND_FILE : luceneUseCompoundFile;
     _luceneMaxBufferSizeMB =
         luceneMaxBufferSizeMB == null ? LUCENE_INDEX_DEFAULT_MAX_BUFFER_SIZE_MB : luceneMaxBufferSizeMB;
+    _luceneAnalyzerClass = (luceneAnalyzerClass == null || luceneAnalyzerClass.isEmpty())
+            ? FieldConfig.TEXT_INDEX_DEFAULT_LUCENE_ANALYZER_CLASS : luceneAnalyzerClass;
   }
 
   public FSTType getFstType() {
@@ -115,6 +121,13 @@ public class TextIndexConfig extends IndexConfig {
     return _luceneMaxBufferSizeMB;
   }
 
+  /**
+   * Lucene analyzer fully qualified class name specifying which analyzer class to use for indexing
+   */
+  public String getLuceneAnalyzerClass() {
+    return _luceneAnalyzerClass;
+  }
+
   public static abstract class AbstractBuilder {
     @Nullable
     protected FSTType _fstType;
@@ -126,6 +139,7 @@ public class TextIndexConfig extends IndexConfig {
     protected List<String> _stopWordsExclude = new ArrayList<>();
     protected boolean _luceneUseCompoundFile = LUCENE_INDEX_DEFAULT_USE_COMPOUND_FILE;
     protected int _luceneMaxBufferSizeMB = LUCENE_INDEX_DEFAULT_MAX_BUFFER_SIZE_MB;
+    protected String _luceneAnalyzerClass = FieldConfig.TEXT_INDEX_DEFAULT_LUCENE_ANALYZER_CLASS;
 
     public AbstractBuilder(@Nullable FSTType fstType) {
       _fstType = fstType;
@@ -139,11 +153,12 @@ public class TextIndexConfig extends IndexConfig {
       _stopWordsExclude = new ArrayList<>(other._stopWordsExclude);
       _luceneUseCompoundFile = other._luceneUseCompoundFile;
       _luceneMaxBufferSizeMB = other._luceneMaxBufferSizeMB;
+      _luceneAnalyzerClass = other._luceneAnalyzerClass;
     }
 
     public TextIndexConfig build() {
       return new TextIndexConfig(false, _fstType, _rawValueForTextIndex, _enableQueryCache, _useANDForMultiTermQueries,
-          _stopWordsInclude, _stopWordsExclude, _luceneUseCompoundFile, _luceneMaxBufferSizeMB);
+          _stopWordsInclude, _stopWordsExclude, _luceneUseCompoundFile, _luceneMaxBufferSizeMB, _luceneAnalyzerClass);
     }
 
     public abstract AbstractBuilder withProperties(@Nullable Map<String, String> textIndexProperties);
@@ -172,6 +187,11 @@ public class TextIndexConfig extends IndexConfig {
       _luceneMaxBufferSizeMB = maxBufferSizeMB;
       return this;
     }
+
+    public AbstractBuilder withLuceneAnalyzerClass(String luceneAnalyzerClass) {
+      _luceneAnalyzerClass = luceneAnalyzerFQCN;
+      return this;
+    }
   }
 
   @Override
@@ -197,6 +217,6 @@ public class TextIndexConfig extends IndexConfig {
   public int hashCode() {
     return Objects.hash(super.hashCode(), _fstType, _rawValueForTextIndex, _enableQueryCache,
         _useANDForMultiTermQueries, _stopWordsInclude, _stopWordsExclude, _luceneUseCompoundFile,
-        _luceneMaxBufferSizeMB);
+        _luceneMaxBufferSizeMB, _luceneAnalyzerClass);
   }
 }
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 47a846d878..145f9dd65c 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
@@ -51,6 +51,9 @@ public class FieldConfig extends BaseJsonConfig {
   public static final String TEXT_INDEX_STOP_WORD_EXCLUDE_KEY = "stopWordExclude";
   public static final String TEXT_INDEX_LUCENE_USE_COMPOUND_FILE = "luceneUseCompoundFile";
   public static final String TEXT_INDEX_LUCENE_MAX_BUFFER_SIZE_MB = "luceneMaxBufferSizeMB";
+  public static final String TEXT_INDEX_LUCENE_ANALYZER_CLASS = "luceneAnalyzerClass";
+  public static final String TEXT_INDEX_DEFAULT_LUCENE_ANALYZER_CLASS
+          = "org.apache.lucene.analysis.standard.StandardAnalyzer";
   public static final String TEXT_INDEX_STOP_WORD_SEPERATOR = ",";
   // "native" for native, default is Lucene
   public static final String TEXT_FST_TYPE = "fstType";


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