You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@pinot.apache.org by "klsince (via GitHub)" <gi...@apache.org> on 2023/05/08 23:46:16 UTC

[GitHub] [pinot] klsince commented on a diff in pull request #10687: Implement mutable index using index SPI

klsince commented on code in PR #10687:
URL: https://github.com/apache/pinot/pull/10687#discussion_r1187969432


##########
pinot-segment-local/src/main/java/org/apache/pinot/segment/local/indexsegment/mutable/MutableSegmentImpl.java:
##########
@@ -287,111 +282,67 @@ public boolean isMutableSegment() {
         partitions.add(config.getPartitionId());
       }
 
+      // TODO (mutable-index-spi): The comment above was here, but no check was done.
+      //  It seems the code that apply that check was removed around 2020. Should we remove the comment?
       // Check whether to generate raw index for the column while consuming
       // Only support generating raw index on single-value columns that do not have inverted index while
       // consuming. After consumption completes and the segment is built, all single-value columns can have raw index
-      DataType storedType = fieldSpec.getDataType().getStoredType();
-      boolean isFixedWidthColumn = storedType.isFixedWidth();
-      MutableIndexProvider indexProvider = IndexingOverrides.getMutableIndexProvider();
-      MutableForwardIndex forwardIndex = indexProvider.newForwardIndex(context.forForwardIndex(avgNumMultiValues));
 
       // Dictionary-encoded column
-      MutableDictionary dictionary = null;
+      MutableDictionary dictionary;
       if (isDictionary) {
-        int dictionaryColumnSize =
-            isFixedWidthColumn ? storedType.size() : _statsHistory.getEstimatedAvgColSize(column);
-        // NOTE: preserve 10% buffer for cardinality to reduce the chance of re-sizing the dictionary
-        int estimatedCardinality = (int) (_statsHistory.getEstimatedCardinality(column) * 1.1);
-        dictionary = indexProvider.newDictionary(context.forDictionary(dictionaryColumnSize, estimatedCardinality));
-        // Even though the column is defined as 'no-dictionary' in the config, we did create dictionary for consuming
-        // segment.
-        noDictionaryColumns.remove(column);
-      }
-
-      // Inverted index
-      MutableInvertedIndex invertedIndexReader =
-          invertedIndexColumns.contains(column) ? indexProvider.newInvertedIndex(context.forInvertedIndex()) : null;
-
-      MutableTextIndex fstIndex = null;
-      // FST Index
-      if (_fieldConfigList != null && fstIndexColumns.contains(column)) {
-        for (FieldConfig fieldConfig : _fieldConfigList) {
-          if (fieldConfig.getName().equals(column)) {
-            Map<String, String> properties = fieldConfig.getProperties();
-            if (TextIndexUtils.isFstTypeNative(properties)) {
-              fstIndex = new NativeMutableFSTIndex(column);
-            }
-          }
+        DictionaryIndexConfig dictionaryIndexConfig = indexConfigs.getConfig(StandardIndexes.dictionary());
+        if (dictionaryIndexConfig.isDisabled()) {
+          dictionaryIndexConfig = DictionaryIndexConfig.DEFAULT;

Review Comment:
   why still assign DEFAULT to dictionaryIndexConfig when it's disabled? I see the DEFAULT is `disabled` too. 



##########
pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/index/mutable/provider/MutableIndexContext.java:
##########
@@ -18,36 +18,97 @@
  */
 package org.apache.pinot.segment.spi.index.mutable.provider;
 
+import java.io.File;
 import java.util.Objects;
+import javax.annotation.Nullable;
 import org.apache.pinot.segment.spi.memory.PinotDataBufferMemoryManager;
-import org.apache.pinot.spi.config.table.JsonIndexConfig;
 import org.apache.pinot.spi.data.FieldSpec;
 
 
-public interface MutableIndexContext {
-  PinotDataBufferMemoryManager getMemoryManager();
+public class MutableIndexContext {

Review Comment:
   pardon me, what's the new fixes you pushed? got a link to them?



##########
pinot-segment-local/src/main/java/org/apache/pinot/segment/local/indexsegment/mutable/MutableSegmentImpl.java:
##########
@@ -637,23 +595,23 @@ private void addNewRow(int docId, GenericRow row) {
         // Update numValues info
         indexContainer._valuesInfo.updateSVNumValues();
 
-        MutableForwardIndex forwardIndex = indexContainer._forwardIndex;
+        MutableIndex forwardIndex = indexContainer._mutableIndexes.get(StandardIndexes.forward());
         FieldSpec fieldSpec = indexContainer._fieldSpec;
 
         DataType dataType = fieldSpec.getDataType();
         value = indexContainer._valueAggregator.getInitialAggregatedValue(value);
         switch (dataType.getStoredType()) {

Review Comment:
   got it. this detail can use some comments here.



-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org

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