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/09 01:23:16 UTC

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

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