You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@pinot.apache.org by GitBox <gi...@apache.org> on 2020/07/08 07:56:51 UTC

[GitHub] [incubator-pinot] siddharthteotia opened a new pull request #5667: Support co-existence of text index with other indexes

siddharthteotia opened a new pull request #5667:
URL: https://github.com/apache/incubator-pinot/pull/5667


   Support co-existence of text index with other indexes and structures
   
   - Remove the constraint that text index should be on raw columns
   - Remove the constraint that text index should be on SV columns
       
   Note: there has been a pending cleanup for both range index and text index (which followed the InvertedIndexReader) interface. I'd like to cleanup the same for text index in this PR. 


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [incubator-pinot] siddharthteotia commented on a change in pull request #5667: Support co-existence of text index with other indexes

Posted by GitBox <gi...@apache.org>.
siddharthteotia commented on a change in pull request #5667:
URL: https://github.com/apache/incubator-pinot/pull/5667#discussion_r451354267



##########
File path: pinot-core/src/main/java/org/apache/pinot/core/common/DataSource.java
##########
@@ -60,6 +60,13 @@
   @Nullable
   InvertedIndexReader<?> getRangeIndex();
 
+  /**
+   * Returns the text index for the column if exists, or {@code null} if not.
+   * <p>TODO: Have a separate interface for text index.
+   */
+  @Nullable
+  InvertedIndexReader<?> getTextIndex();

Review comment:
       I'd like to do the cleanup for TextIndex in this PR itself. Both text and range followed the InvertedIndexReader interface.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [incubator-pinot] siddharthteotia commented on a change in pull request #5667: Support text index on dictionary encoded columns

Posted by GitBox <gi...@apache.org>.
siddharthteotia commented on a change in pull request #5667:
URL: https://github.com/apache/incubator-pinot/pull/5667#discussion_r455326416



##########
File path: pinot-core/src/main/java/org/apache/pinot/core/segment/index/loader/defaultcolumn/BaseDefaultColumnHandler.java
##########
@@ -325,77 +323,92 @@ protected void removeColumnV1Indices(String column)
   }
 
   /**
-   * Right now the text index is supported on RAW (non-dictionary encoded)
+   * Right now the text index is supported on RAW and dictionary encoded
    * single-value STRING columns. Eventually we will relax the constraints
    * step by step.
    * For example, later on user should be able to create text index on
-   * a dictionary encoded STRING column that also has native Pinot's inverted
-   * index. We can also support it on BYTE columns later.
+   * a MV column
    * @param column column name
    * @param indexLoadingConfig index loading config
    * @param fieldSpec field spec
    */
   private void checkUnsupportedOperationsForTextIndex(String column, IndexLoadingConfig indexLoadingConfig,
       FieldSpec fieldSpec) {
-    if (!indexLoadingConfig.getNoDictionaryColumns().contains(column)) {
-      throw new UnsupportedOperationException(
-          "Text index is currently not supported on dictionary encoded column: " + column);
-    }
-
-    Set<String> sortedColumns = new HashSet<>(indexLoadingConfig.getSortedColumns());
-    if (sortedColumns.contains(column)) {
-      // since Pinot's current implementation doesn't support raw sorted columns,
-      // we need to check for this too
-      throw new UnsupportedOperationException("Text index is currently not supported on sorted column: " + column);
-    }
-
     if (!fieldSpec.isSingleValueField()) {
       throw new UnsupportedOperationException("Text index is currently not supported on multi-value column: " + column);
     }
-
     if (fieldSpec.getDataType() != DataType.STRING) {
       throw new UnsupportedOperationException("Text index is currently only supported on STRING column:" + column);
     }
   }
 
   void createV1ForwardIndexForTextIndex(String column, IndexLoadingConfig indexLoadingConfig)

Review comment:
       It seems to me that existing function createColumnV1Indices() has a bug which I want to investigate further and fix before I reuse that method for text index. Meanwhile, this PR is complete and I can refactor this portion next.
   
   Consider that case a new column gets added with inverted index. Two things happen:
   
   (1) First, default column handler creates dictionary and forward index. Note that it doesn't create bit encoded forward index. It creates sorted forward index.
   (2) Later during segment load, InvertedIndexHandler reads the forward index to create inverted index. This code when trying to create inverted index first gets a forward index reader. It gets the forward index buffer and creates the bit encoded forward index reader. This seems wrong to me since for newly added column, the forward index buffer had sorted index format and wasn't bit encoded.
   
   I will look into this in detail and as part of that cleanup the existing code first before reusing it for text.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [incubator-pinot] siddharthteotia merged pull request #5667: Support text index on dictionary encoded columns

Posted by GitBox <gi...@apache.org>.
siddharthteotia merged pull request #5667:
URL: https://github.com/apache/incubator-pinot/pull/5667


   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [incubator-pinot] mcvsubbu commented on a change in pull request #5667: Support text index on dictionary encoded columns

Posted by GitBox <gi...@apache.org>.
mcvsubbu commented on a change in pull request #5667:
URL: https://github.com/apache/incubator-pinot/pull/5667#discussion_r453821384



##########
File path: pinot-core/src/main/java/org/apache/pinot/core/segment/index/loader/defaultcolumn/BaseDefaultColumnHandler.java
##########
@@ -325,77 +323,92 @@ protected void removeColumnV1Indices(String column)
   }
 
   /**
-   * Right now the text index is supported on RAW (non-dictionary encoded)
+   * Right now the text index is supported on RAW and dictionary encoded

Review comment:
       Are there any more constraints to relax? If not, we can modify this comment?

##########
File path: pinot-core/src/main/java/org/apache/pinot/core/segment/index/loader/invertedindex/TextIndexHandler.java
##########
@@ -111,35 +117,22 @@ public void createTextIndexesOnSegmentLoad()
   }
 
   /**
-   * Right now the text index is supported on RAW (non-dictionary encoded)
+   * Right now the text index is supported on RAW and dictionary encoded

Review comment:
       same here. capture the constraints correctly in comments

##########
File path: pinot-core/src/main/java/org/apache/pinot/core/segment/index/loader/invertedindex/TextIndexHandler.java
##########
@@ -151,30 +144,68 @@ private void createTextIndexForColumn(ColumnMetadata columnMetadata)
       return;
     }
     int numDocs = columnMetadata.getTotalDocs();
-    LOGGER.info("Creating new text index for column: {} in segment: {}", column, _segmentName);
+    boolean hasDictionary = columnMetadata.hasDictionary();
+    LOGGER.info("Creating new text index for column: {} in segment: {}, hasDictionary: {}", column, _segmentName, hasDictionary);
     File segmentDirectory = SegmentDirectoryPaths.segmentDirectoryFor(_indexDir, _segmentVersion);
     // The handlers are always invoked by the preprocessor. Before this ImmutableSegmentLoader would have already
     // up-converted the segment from v1/v2 -> v3 (if needed). So based on the segmentVersion, whatever segment
     // segmentDirectory is indicated to us by SegmentDirectoryPaths, we create lucene index there. There is no
     // further need to move around the lucene index directory since it is created with correct directory structure
     // based on segmentVersion.
-    try (LuceneTextIndexCreator textIndexCreator = new LuceneTextIndexCreator(column, segmentDirectory, true);
-        VarByteChunkSVForwardIndexReader forwardIndexReader = getForwardIndexReader(columnMetadata);
-        ChunkReaderContext readerContext = forwardIndexReader.createContext()) {
-      for (int docId = 0; docId < numDocs; docId++) {
-        textIndexCreator.addDoc(forwardIndexReader.getString(docId, readerContext), docId);
+    try (ForwardIndexReader forwardIndexReader = getForwardIndexReader(columnMetadata);
+        ForwardIndexReaderContext readerContext = forwardIndexReader.createContext();
+        LuceneTextIndexCreator textIndexCreator = new LuceneTextIndexCreator(column, segmentDirectory, true)) {
+      if (!hasDictionary) {
+        // text index on raw column, just read the raw forward index
+        VarByteChunkSVForwardIndexReader rawIndexReader = (VarByteChunkSVForwardIndexReader)forwardIndexReader;
+        ChunkReaderContext chunkReaderContext = (ChunkReaderContext)readerContext;
+        for (int docId = 0; docId < numDocs; docId++) {
+          textIndexCreator.addDoc(rawIndexReader.getString(docId, chunkReaderContext), docId);
+        }
+      } else {
+        // text index on dictionary encoded SV column
+        // read forward index to get dictId
+        // read the raw value from dictionary using dictId
+        try (BaseImmutableDictionary dictionary = getDictionaryReader(columnMetadata)) {
+          if (columnMetadata.isSingleValue()) {
+            for (int docId = 0; docId < numDocs; docId++) {
+              int dictId = forwardIndexReader.getDictId(docId, readerContext);
+              String value = dictionary.getStringValue(dictId);
+              textIndexCreator.addDoc(value, docId);
+            }
+          }
+        }
       }
       textIndexCreator.seal();
     }
+
     LOGGER.info("Created text index for column: {} in segment: {}", column, _segmentName);
     PropertiesConfiguration properties = SegmentMetadataImpl.getPropertiesConfiguration(_indexDir);
     properties.setProperty(getKeyFor(column, TEXT_INDEX_TYPE), TextIndexType.LUCENE.name());
     properties.save();
   }
 
-  private VarByteChunkSVForwardIndexReader getForwardIndexReader(ColumnMetadata columnMetadata)
+  private ForwardIndexReader<?> getForwardIndexReader(ColumnMetadata columnMetadata)
+      throws IOException {
+    if (!columnMetadata.hasDictionary()) {
+      // raw index
+      PinotDataBuffer buffer = _segmentWriter.getIndexFor(columnMetadata.getColumnName(), ColumnIndexType.FORWARD_INDEX);
+      return new VarByteChunkSVForwardIndexReader(buffer, DataType.STRING);
+    } else {
+      PinotDataBuffer buffer = _segmentWriter.getIndexFor(columnMetadata.getColumnName(), ColumnIndexType.FORWARD_INDEX);

Review comment:
       Should it be `ColumnIndexType.DICTIONARY`?




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [incubator-pinot] siddharthteotia commented on a change in pull request #5667: Support text index on dictionary encoded columns

Posted by GitBox <gi...@apache.org>.
siddharthteotia commented on a change in pull request #5667:
URL: https://github.com/apache/incubator-pinot/pull/5667#discussion_r455983567



##########
File path: pinot-core/src/main/java/org/apache/pinot/core/segment/index/loader/defaultcolumn/V3DefaultColumnHandler.java
##########
@@ -67,10 +67,19 @@ protected void updateDefaultColumn(String column, DefaultColumnAction action, In
     Set<String> textIndexColumns = indexLoadingConfig.getTextIndexColumns();

Review comment:
       done

##########
File path: pinot-core/src/main/java/org/apache/pinot/core/segment/index/loader/defaultcolumn/BaseDefaultColumnHandler.java
##########
@@ -325,77 +323,92 @@ protected void removeColumnV1Indices(String column)
   }
 
   /**
-   * Right now the text index is supported on RAW (non-dictionary encoded)
+   * Right now the text index is supported on RAW and dictionary encoded
    * single-value STRING columns. Eventually we will relax the constraints
    * step by step.
    * For example, later on user should be able to create text index on
-   * a dictionary encoded STRING column that also has native Pinot's inverted
-   * index. We can also support it on BYTE columns later.
+   * a MV column
    * @param column column name
    * @param indexLoadingConfig index loading config
    * @param fieldSpec field spec
    */
   private void checkUnsupportedOperationsForTextIndex(String column, IndexLoadingConfig indexLoadingConfig,
       FieldSpec fieldSpec) {
-    if (!indexLoadingConfig.getNoDictionaryColumns().contains(column)) {
-      throw new UnsupportedOperationException(
-          "Text index is currently not supported on dictionary encoded column: " + column);
-    }
-
-    Set<String> sortedColumns = new HashSet<>(indexLoadingConfig.getSortedColumns());
-    if (sortedColumns.contains(column)) {
-      // since Pinot's current implementation doesn't support raw sorted columns,
-      // we need to check for this too
-      throw new UnsupportedOperationException("Text index is currently not supported on sorted column: " + column);
-    }
-
     if (!fieldSpec.isSingleValueField()) {
       throw new UnsupportedOperationException("Text index is currently not supported on multi-value column: " + column);
     }
-
     if (fieldSpec.getDataType() != DataType.STRING) {
       throw new UnsupportedOperationException("Text index is currently only supported on STRING column:" + column);
     }
   }
 
   void createV1ForwardIndexForTextIndex(String column, IndexLoadingConfig indexLoadingConfig)

Review comment:
       Cleaned up the code to reuse existing




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [incubator-pinot] siddharthteotia commented on a change in pull request #5667: Support text index on dictionary encoded columns

Posted by GitBox <gi...@apache.org>.
siddharthteotia commented on a change in pull request #5667:
URL: https://github.com/apache/incubator-pinot/pull/5667#discussion_r455315663



##########
File path: pinot-core/src/main/java/org/apache/pinot/core/segment/creator/impl/SegmentColumnarIndexCreator.java
##########
@@ -379,6 +370,13 @@ public void indexRow(GenericRow row) {
           _nullValueVectorCreatorMap.get(columnName).setNull(docIdCounter);
         }
       }
+
+      // text-search enabled column

Review comment:
       done

##########
File path: pinot-core/src/main/java/org/apache/pinot/core/segment/creator/impl/SegmentColumnarIndexCreator.java
##########
@@ -199,16 +200,17 @@ public void init(SegmentGeneratorConfig segmentCreationSpec, SegmentIndexCreatio
         _forwardIndexCreatorMap.put(columnName,
             getRawIndexCreatorForColumn(_indexDir, compressionType, columnName, fieldSpec.getDataType(), totalDocs,
                 indexCreationInfo.getLengthOfLongestEntry(), deriveNumDocsPerChunk, writerVersion));
+      }
 
+      if (_textIndexColumns.contains(columnName)) {
         // Initialize text index creator
-        if (_textIndexColumns.contains(columnName)) {
-          _invertedIndexCreatorMap
-              .put(columnName, new LuceneTextIndexCreator(columnName, _indexDir, true /* commitOnClose */));
-        }
+        Preconditions.checkState(fieldSpec.isSingleValueField(),

Review comment:
       Yeah, missed that earlier

##########
File path: pinot-core/src/main/java/org/apache/pinot/core/segment/index/loader/defaultcolumn/BaseDefaultColumnHandler.java
##########
@@ -325,77 +323,92 @@ protected void removeColumnV1Indices(String column)
   }
 
   /**
-   * Right now the text index is supported on RAW (non-dictionary encoded)
+   * Right now the text index is supported on RAW and dictionary encoded

Review comment:
       done

##########
File path: pinot-core/src/main/java/org/apache/pinot/core/segment/index/loader/invertedindex/TextIndexHandler.java
##########
@@ -111,35 +117,22 @@ public void createTextIndexesOnSegmentLoad()
   }
 
   /**
-   * Right now the text index is supported on RAW (non-dictionary encoded)
+   * Right now the text index is supported on RAW and dictionary encoded

Review comment:
       done

##########
File path: pinot-core/src/main/java/org/apache/pinot/core/segment/index/loader/invertedindex/TextIndexHandler.java
##########
@@ -151,30 +144,68 @@ private void createTextIndexForColumn(ColumnMetadata columnMetadata)
       return;
     }
     int numDocs = columnMetadata.getTotalDocs();
-    LOGGER.info("Creating new text index for column: {} in segment: {}", column, _segmentName);
+    boolean hasDictionary = columnMetadata.hasDictionary();
+    LOGGER.info("Creating new text index for column: {} in segment: {}, hasDictionary: {}", column, _segmentName, hasDictionary);
     File segmentDirectory = SegmentDirectoryPaths.segmentDirectoryFor(_indexDir, _segmentVersion);
     // The handlers are always invoked by the preprocessor. Before this ImmutableSegmentLoader would have already
     // up-converted the segment from v1/v2 -> v3 (if needed). So based on the segmentVersion, whatever segment
     // segmentDirectory is indicated to us by SegmentDirectoryPaths, we create lucene index there. There is no
     // further need to move around the lucene index directory since it is created with correct directory structure
     // based on segmentVersion.
-    try (LuceneTextIndexCreator textIndexCreator = new LuceneTextIndexCreator(column, segmentDirectory, true);
-        VarByteChunkSVForwardIndexReader forwardIndexReader = getForwardIndexReader(columnMetadata);
-        ChunkReaderContext readerContext = forwardIndexReader.createContext()) {
-      for (int docId = 0; docId < numDocs; docId++) {
-        textIndexCreator.addDoc(forwardIndexReader.getString(docId, readerContext), docId);
+    try (ForwardIndexReader forwardIndexReader = getForwardIndexReader(columnMetadata);
+        ForwardIndexReaderContext readerContext = forwardIndexReader.createContext();
+        LuceneTextIndexCreator textIndexCreator = new LuceneTextIndexCreator(column, segmentDirectory, true)) {
+      if (!hasDictionary) {
+        // text index on raw column, just read the raw forward index
+        VarByteChunkSVForwardIndexReader rawIndexReader = (VarByteChunkSVForwardIndexReader)forwardIndexReader;
+        ChunkReaderContext chunkReaderContext = (ChunkReaderContext)readerContext;
+        for (int docId = 0; docId < numDocs; docId++) {
+          textIndexCreator.addDoc(rawIndexReader.getString(docId, chunkReaderContext), docId);
+        }
+      } else {
+        // text index on dictionary encoded SV column
+        // read forward index to get dictId
+        // read the raw value from dictionary using dictId
+        try (BaseImmutableDictionary dictionary = getDictionaryReader(columnMetadata)) {
+          if (columnMetadata.isSingleValue()) {
+            for (int docId = 0; docId < numDocs; docId++) {
+              int dictId = forwardIndexReader.getDictId(docId, readerContext);
+              String value = dictionary.getStringValue(dictId);
+              textIndexCreator.addDoc(value, docId);
+            }
+          }
+        }
       }
       textIndexCreator.seal();
     }
+
     LOGGER.info("Created text index for column: {} in segment: {}", column, _segmentName);
     PropertiesConfiguration properties = SegmentMetadataImpl.getPropertiesConfiguration(_indexDir);
     properties.setProperty(getKeyFor(column, TEXT_INDEX_TYPE), TextIndexType.LUCENE.name());
     properties.save();
   }
 
-  private VarByteChunkSVForwardIndexReader getForwardIndexReader(ColumnMetadata columnMetadata)
+  private ForwardIndexReader<?> getForwardIndexReader(ColumnMetadata columnMetadata)
+      throws IOException {
+    if (!columnMetadata.hasDictionary()) {
+      // raw index
+      PinotDataBuffer buffer = _segmentWriter.getIndexFor(columnMetadata.getColumnName(), ColumnIndexType.FORWARD_INDEX);
+      return new VarByteChunkSVForwardIndexReader(buffer, DataType.STRING);
+    } else {
+      PinotDataBuffer buffer = _segmentWriter.getIndexFor(columnMetadata.getColumnName(), ColumnIndexType.FORWARD_INDEX);

Review comment:
       No




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [incubator-pinot] Jackie-Jiang commented on a change in pull request #5667: Support text index on dictionary encoded columns

Posted by GitBox <gi...@apache.org>.
Jackie-Jiang commented on a change in pull request #5667:
URL: https://github.com/apache/incubator-pinot/pull/5667#discussion_r456002900



##########
File path: pinot-core/src/main/java/org/apache/pinot/core/segment/index/loader/invertedindex/TextIndexHandler.java
##########
@@ -151,30 +143,76 @@ private void createTextIndexForColumn(ColumnMetadata columnMetadata)
       return;
     }
     int numDocs = columnMetadata.getTotalDocs();
-    LOGGER.info("Creating new text index for column: {} in segment: {}", column, _segmentName);
+    boolean hasDictionary = columnMetadata.hasDictionary();
+    LOGGER.info("Creating new text index for column: {} in segment: {}, hasDictionary: {}", column, _segmentName, hasDictionary);
     File segmentDirectory = SegmentDirectoryPaths.segmentDirectoryFor(_indexDir, _segmentVersion);
     // The handlers are always invoked by the preprocessor. Before this ImmutableSegmentLoader would have already
     // up-converted the segment from v1/v2 -> v3 (if needed). So based on the segmentVersion, whatever segment
     // segmentDirectory is indicated to us by SegmentDirectoryPaths, we create lucene index there. There is no
     // further need to move around the lucene index directory since it is created with correct directory structure
     // based on segmentVersion.
-    try (LuceneTextIndexCreator textIndexCreator = new LuceneTextIndexCreator(column, segmentDirectory, true);
-        VarByteChunkSVForwardIndexReader forwardIndexReader = getForwardIndexReader(columnMetadata);
-        ChunkReaderContext readerContext = forwardIndexReader.createContext()) {
-      for (int docId = 0; docId < numDocs; docId++) {
-        textIndexCreator.addDoc(forwardIndexReader.getString(docId, readerContext), docId);
+    try (ForwardIndexReader forwardIndexReader = getForwardIndexReader(columnMetadata);
+        ForwardIndexReaderContext readerContext = forwardIndexReader.createContext();
+        LuceneTextIndexCreator textIndexCreator = new LuceneTextIndexCreator(column, segmentDirectory, true)) {
+      if (!hasDictionary) {
+        // text index on raw column, just read the raw forward index
+        VarByteChunkSVForwardIndexReader rawIndexReader = (VarByteChunkSVForwardIndexReader)forwardIndexReader;
+        ChunkReaderContext chunkReaderContext = (ChunkReaderContext)readerContext;
+        for (int docId = 0; docId < numDocs; docId++) {
+          textIndexCreator.addDoc(rawIndexReader.getString(docId, chunkReaderContext), docId);
+        }
+      } else {
+        // text index on dictionary encoded SV column
+        // read forward index to get dictId
+        // read the raw value from dictionary using dictId
+        try (BaseImmutableDictionary dictionary = getDictionaryReader(columnMetadata)) {
+          if (columnMetadata.isSingleValue()) {

Review comment:
       Remove this check (or replace with checkState)

##########
File path: pinot-core/src/main/java/org/apache/pinot/core/segment/index/loader/defaultcolumn/BaseDefaultColumnHandler.java
##########
@@ -321,88 +321,33 @@ protected void removeColumnV1Indices(String column)
   }
 
   /**
-   * Right now the text index is supported on RAW (non-dictionary encoded)
-   * single-value STRING columns. Eventually we will relax the constraints
-   * step by step.
-   * For example, later on user should be able to create text index on
-   * a dictionary encoded STRING column that also has native Pinot's inverted
-   * index. We can also support it on BYTE columns later.
+   * Right now the text index is supported on RAW and dictionary encoded
+   * single-value STRING columns. Later we can add support for text index
+   * on multi-value columns and BYTE type columns
    * @param column column name
-   * @param indexLoadingConfig index loading config
    * @param fieldSpec field spec
    */
-  private void checkUnsupportedOperationsForTextIndex(String column, IndexLoadingConfig indexLoadingConfig,
-      FieldSpec fieldSpec) {
-    if (!indexLoadingConfig.getNoDictionaryColumns().contains(column)) {
-      throw new UnsupportedOperationException(
-          "Text index is currently not supported on dictionary encoded column: " + column);
-    }
-
-    Set<String> sortedColumns = new HashSet<>(indexLoadingConfig.getSortedColumns());
-    if (sortedColumns.contains(column)) {
-      // since Pinot's current implementation doesn't support raw sorted columns,
-      // we need to check for this too
-      throw new UnsupportedOperationException("Text index is currently not supported on sorted column: " + column);
-    }
-
+  private void checkUnsupportedOperationsForTextIndex(String column, FieldSpec fieldSpec) {

Review comment:
       Let's remove this check as it will be checked in `TextIndexHandler`. Try not to couple different modules together.  




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [incubator-pinot] siddharthteotia commented on a change in pull request #5667: Support text index on dictionary encoded columns

Posted by GitBox <gi...@apache.org>.
siddharthteotia commented on a change in pull request #5667:
URL: https://github.com/apache/incubator-pinot/pull/5667#discussion_r451884394



##########
File path: pinot-core/src/main/java/org/apache/pinot/core/segment/creator/impl/inv/text/LuceneTextIndexCreator.java
##########
@@ -115,14 +115,31 @@ public IndexWriter getIndexWriter() {
 
   @Override
   public void addDoc(Object document, int docIdCounter) {
-    Document docToIndex = new Document();
-    docToIndex.add(new TextField(_textColumn, document.toString(), Field.Store.NO));
-    docToIndex.add(new StoredField(LUCENE_INDEX_DOC_ID_COLUMN_NAME, docIdCounter));
-    try {
-      _indexWriter.addDocument(docToIndex);
-    } catch (Exception e) {
-      LOGGER.error("Failure while adding a new document to index for column {}, exception {}", _textColumn, e.getMessage());
-      throw new RuntimeException(e);
+    if (!(document instanceof Object[])) {
+      // text index on SV column
+      Document docToIndex = new Document();
+      docToIndex.add(new TextField(_textColumn, document.toString(), Field.Store.NO));
+      docToIndex.add(new StoredField(LUCENE_INDEX_DOC_ID_COLUMN_NAME, docIdCounter));
+      try {
+        _indexWriter.addDocument(docToIndex);
+      } catch (Exception e) {
+        LOGGER.error("Failure while adding a new document to index for column {}, exception {}", _textColumn, e.getMessage());
+        throw new RuntimeException(e);
+      }
+    } else {
+      // text index on MV column
+      Object[] values = (Object[])document;
+      for (Object value : values) {
+        Document docToIndex = new Document();

Review comment:
       In order to fix this issue https://github.com/apache/incubator-pinot/issues/5666 sooner, I am only relaxing the constraint of text index on raw columns in this PR. The constraint of text index on SV columns is still there and will address in a follow-up. Need to do some digging into Lucene as well to see how arrays can be stored in a single document etc. 




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [incubator-pinot] Jackie-Jiang commented on a change in pull request #5667: Support co-existence of text index with other indexes

Posted by GitBox <gi...@apache.org>.
Jackie-Jiang commented on a change in pull request #5667:
URL: https://github.com/apache/incubator-pinot/pull/5667#discussion_r451711537



##########
File path: pinot-core/src/main/java/org/apache/pinot/core/segment/creator/impl/inv/text/LuceneTextIndexCreator.java
##########
@@ -115,14 +115,31 @@ public IndexWriter getIndexWriter() {
 
   @Override
   public void addDoc(Object document, int docIdCounter) {
-    Document docToIndex = new Document();
-    docToIndex.add(new TextField(_textColumn, document.toString(), Field.Store.NO));
-    docToIndex.add(new StoredField(LUCENE_INDEX_DOC_ID_COLUMN_NAME, docIdCounter));
-    try {
-      _indexWriter.addDocument(docToIndex);
-    } catch (Exception e) {
-      LOGGER.error("Failure while adding a new document to index for column {}, exception {}", _textColumn, e.getMessage());
-      throw new RuntimeException(e);
+    if (!(document instanceof Object[])) {
+      // text index on SV column
+      Document docToIndex = new Document();
+      docToIndex.add(new TextField(_textColumn, document.toString(), Field.Store.NO));
+      docToIndex.add(new StoredField(LUCENE_INDEX_DOC_ID_COLUMN_NAME, docIdCounter));
+      try {
+        _indexWriter.addDocument(docToIndex);
+      } catch (Exception e) {
+        LOGGER.error("Failure while adding a new document to index for column {}, exception {}", _textColumn, e.getMessage());
+        throw new RuntimeException(e);
+      }
+    } else {
+      // text index on MV column
+      Object[] values = (Object[])document;
+      for (Object value : values) {
+        Document docToIndex = new Document();

Review comment:
       Will this cause multiple text document map to the same pinot document? How do you maintain the map? We need to figure out the semantic of text index on MV columns




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [incubator-pinot] siddharthteotia commented on pull request #5667: Support text index on dictionary encoded columns

Posted by GitBox <gi...@apache.org>.
siddharthteotia commented on pull request #5667:
URL: https://github.com/apache/incubator-pinot/pull/5667#issuecomment-658996765


   @Jackie-Jiang, addressed the review comments.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [incubator-pinot] kishoreg commented on pull request #5667: Support text index on dictionary encoded columns

Posted by GitBox <gi...@apache.org>.
kishoreg commented on pull request #5667:
URL: https://github.com/apache/incubator-pinot/pull/5667#issuecomment-657182217


   Anything pending here? @siddharthteotia @Jackie-Jiang 


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [incubator-pinot] siddharthteotia commented on a change in pull request #5667: Support text index on dictionary encoded columns

Posted by GitBox <gi...@apache.org>.
siddharthteotia commented on a change in pull request #5667:
URL: https://github.com/apache/incubator-pinot/pull/5667#discussion_r451354267



##########
File path: pinot-core/src/main/java/org/apache/pinot/core/common/DataSource.java
##########
@@ -60,6 +60,13 @@
   @Nullable
   InvertedIndexReader<?> getRangeIndex();
 
+  /**
+   * Returns the text index for the column if exists, or {@code null} if not.
+   * <p>TODO: Have a separate interface for text index.
+   */
+  @Nullable
+  InvertedIndexReader<?> getTextIndex();

Review comment:
       I'd like to do the cleanup for TextIndex in this PR itself. Both text and range followed the InvertedIndexReader interface.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [incubator-pinot] Jackie-Jiang commented on a change in pull request #5667: Support text index on dictionary encoded columns

Posted by GitBox <gi...@apache.org>.
Jackie-Jiang commented on a change in pull request #5667:
URL: https://github.com/apache/incubator-pinot/pull/5667#discussion_r451904412



##########
File path: pinot-core/src/main/java/org/apache/pinot/core/segment/creator/impl/SegmentColumnarIndexCreator.java
##########
@@ -379,6 +370,13 @@ public void indexRow(GenericRow row) {
           _nullValueVectorCreatorMap.get(columnName).setNull(docIdCounter);
         }
       }
+
+      // text-search enabled column

Review comment:
       Move this into the branch of single value column

##########
File path: pinot-core/src/main/java/org/apache/pinot/core/segment/creator/impl/SegmentColumnarIndexCreator.java
##########
@@ -199,16 +200,17 @@ public void init(SegmentGeneratorConfig segmentCreationSpec, SegmentIndexCreatio
         _forwardIndexCreatorMap.put(columnName,
             getRawIndexCreatorForColumn(_indexDir, compressionType, columnName, fieldSpec.getDataType(), totalDocs,
                 indexCreationInfo.getLengthOfLongestEntry(), deriveNumDocsPerChunk, writerVersion));
+      }
 
+      if (_textIndexColumns.contains(columnName)) {
         // Initialize text index creator
-        if (_textIndexColumns.contains(columnName)) {
-          _invertedIndexCreatorMap
-              .put(columnName, new LuceneTextIndexCreator(columnName, _indexDir, true /* commitOnClose */));
-        }
+        Preconditions.checkState(fieldSpec.isSingleValueField(),

Review comment:
       Also check that column is STRING type?

##########
File path: pinot-core/src/main/java/org/apache/pinot/core/segment/index/loader/defaultcolumn/V3DefaultColumnHandler.java
##########
@@ -67,10 +67,19 @@ protected void updateDefaultColumn(String column, DefaultColumnAction action, In
     Set<String> textIndexColumns = indexLoadingConfig.getTextIndexColumns();

Review comment:
       Since text column supports both raw & dictionary-encoded index, remove the branch for text column

##########
File path: pinot-core/src/main/java/org/apache/pinot/core/segment/index/loader/defaultcolumn/BaseDefaultColumnHandler.java
##########
@@ -325,77 +323,92 @@ protected void removeColumnV1Indices(String column)
   }
 
   /**
-   * Right now the text index is supported on RAW (non-dictionary encoded)
+   * Right now the text index is supported on RAW and dictionary encoded
    * single-value STRING columns. Eventually we will relax the constraints
    * step by step.
    * For example, later on user should be able to create text index on
-   * a dictionary encoded STRING column that also has native Pinot's inverted
-   * index. We can also support it on BYTE columns later.
+   * a MV column
    * @param column column name
    * @param indexLoadingConfig index loading config
    * @param fieldSpec field spec
    */
   private void checkUnsupportedOperationsForTextIndex(String column, IndexLoadingConfig indexLoadingConfig,
       FieldSpec fieldSpec) {
-    if (!indexLoadingConfig.getNoDictionaryColumns().contains(column)) {
-      throw new UnsupportedOperationException(
-          "Text index is currently not supported on dictionary encoded column: " + column);
-    }
-
-    Set<String> sortedColumns = new HashSet<>(indexLoadingConfig.getSortedColumns());
-    if (sortedColumns.contains(column)) {
-      // since Pinot's current implementation doesn't support raw sorted columns,
-      // we need to check for this too
-      throw new UnsupportedOperationException("Text index is currently not supported on sorted column: " + column);
-    }
-
     if (!fieldSpec.isSingleValueField()) {
       throw new UnsupportedOperationException("Text index is currently not supported on multi-value column: " + column);
     }
-
     if (fieldSpec.getDataType() != DataType.STRING) {
       throw new UnsupportedOperationException("Text index is currently only supported on STRING column:" + column);
     }
   }
 
   void createV1ForwardIndexForTextIndex(String column, IndexLoadingConfig indexLoadingConfig)

Review comment:
       I think you can just remove this method and always use the default one. We should always create dictionary-encoded index for default column because it is much more efficient.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [incubator-pinot] siddharthteotia commented on a change in pull request #5667: Support text index on dictionary encoded columns

Posted by GitBox <gi...@apache.org>.
siddharthteotia commented on a change in pull request #5667:
URL: https://github.com/apache/incubator-pinot/pull/5667#discussion_r455335735



##########
File path: pinot-core/src/main/java/org/apache/pinot/core/segment/index/loader/defaultcolumn/BaseDefaultColumnHandler.java
##########
@@ -325,77 +323,92 @@ protected void removeColumnV1Indices(String column)
   }
 
   /**
-   * Right now the text index is supported on RAW (non-dictionary encoded)
+   * Right now the text index is supported on RAW and dictionary encoded
    * single-value STRING columns. Eventually we will relax the constraints
    * step by step.
    * For example, later on user should be able to create text index on
-   * a dictionary encoded STRING column that also has native Pinot's inverted
-   * index. We can also support it on BYTE columns later.
+   * a MV column
    * @param column column name
    * @param indexLoadingConfig index loading config
    * @param fieldSpec field spec
    */
   private void checkUnsupportedOperationsForTextIndex(String column, IndexLoadingConfig indexLoadingConfig,
       FieldSpec fieldSpec) {
-    if (!indexLoadingConfig.getNoDictionaryColumns().contains(column)) {
-      throw new UnsupportedOperationException(
-          "Text index is currently not supported on dictionary encoded column: " + column);
-    }
-
-    Set<String> sortedColumns = new HashSet<>(indexLoadingConfig.getSortedColumns());
-    if (sortedColumns.contains(column)) {
-      // since Pinot's current implementation doesn't support raw sorted columns,
-      // we need to check for this too
-      throw new UnsupportedOperationException("Text index is currently not supported on sorted column: " + column);
-    }
-
     if (!fieldSpec.isSingleValueField()) {
       throw new UnsupportedOperationException("Text index is currently not supported on multi-value column: " + column);
     }
-
     if (fieldSpec.getDataType() != DataType.STRING) {
       throw new UnsupportedOperationException("Text index is currently only supported on STRING column:" + column);
     }
   }
 
   void createV1ForwardIndexForTextIndex(String column, IndexLoadingConfig indexLoadingConfig)

Review comment:
       Never mind. This particular code handles the problem I was talking about
   
   `// Only create inverted index on dictionary-encoded unsorted columns
       for (String column : indexLoadingConfig.getInvertedIndexColumns()) {
         ColumnMetadata columnMetadata = segmentMetadata.getColumnMetadataFor(column);
         if (columnMetadata != null && !columnMetadata.isSorted() && columnMetadata.hasDictionary()) {
           _invertedIndexColumns.add(columnMetadata);
         }
       }`




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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