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 2022/10/28 06:47:57 UTC

[GitHub] [pinot] vvivekiyer opened a new pull request, #9678: Enable dictionary code changes

vvivekiyer opened a new pull request, #9678:
URL: https://github.com/apache/pinot/pull/9678

   OSS issue: https://github.com/apache/pinot/issues/9348
   Label: Feature
   [Document](https://docs.google.com/document/d/1Gai0DHBnyR4joG_8AcoR-27_exEBTpVfTFU04HyPdd8/edit#heading=h.liwwtls82n1z) 
   
   With this PR, we add support enable dictionary on a raw column for immutable segments. This support is added for both SV and MV columns.
    
   I'm adding more exhaustive unit tests. Will update 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.

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


[GitHub] [pinot] siddharthteotia commented on a diff in pull request #9678: Support creating dictionary at runtime for an existing column

Posted by GitBox <gi...@apache.org>.
siddharthteotia commented on code in PR #9678:
URL: https://github.com/apache/pinot/pull/9678#discussion_r1012264245


##########
pinot-core/src/test/java/org/apache/pinot/queries/RangeQueriesTest.java:
##########
@@ -223,6 +223,42 @@ public void testSelectionOverRangeFilter(String query, int min, int max, boolean
     }
   }
 
+  @Test(dataProvider = "selectionTestCases")
+  public void testSelectionOverRangeFilterAfterReload(String query, int min, int max, boolean inclusive)

Review Comment:
   Query execution tests seem to be focused on range index only. Any reason ?



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


[GitHub] [pinot] siddharthteotia commented on a diff in pull request #9678: Support creating dictionary at runtime for an existing column

Posted by GitBox <gi...@apache.org>.
siddharthteotia commented on code in PR #9678:
URL: https://github.com/apache/pinot/pull/9678#discussion_r1012270227


##########
pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/loader/ForwardIndexHandler.java:
##########
@@ -271,118 +304,295 @@ private void rewriteRawSVForwardIndex(String column, ColumnMetadata existingColM
               .forForwardIndex(newCompressionType, _indexLoadingConfig.getColumnProperties());
 
       try (ForwardIndexCreator creator = indexCreatorProvider.newForwardIndexCreator(context)) {
-        // If creator stored type and the reader stored type do not match, throw an exception.
         if (!reader.getStoredType().equals(creator.getValueType())) {
+          // Creator stored type should match reader stored type for raw columns. We do not support changing datatypes.
           String failureMsg =
               "Unsupported operation to change datatype for column=" + column + " from " + reader.getStoredType()
                   .toString() + " to " + creator.getValueType().toString();
           throw new UnsupportedOperationException(failureMsg);
         }
 
         int numDocs = existingColMetadata.getTotalDocs();
-        forwardIndexWriterHelper(column, reader, creator, numDocs);
+        forwardIndexWriterHelper(column, existingColMetadata, reader, creator, numDocs, null);
       }
     }
   }
 
-  private void forwardIndexWriterHelper(String column, ForwardIndexReader reader, ForwardIndexCreator creator,
-      int numDocs) {
-    // If creator stored type should match reader stored type. We do not support changing datatypes.
-    if (!reader.getStoredType().equals(creator.getValueType())) {
-      String failureMsg =
-          "Unsupported operation to change datatype for column=" + column + " from " + reader.getStoredType().toString()
-              + " to " + creator.getValueType().toString();
-      throw new UnsupportedOperationException(failureMsg);
-    }
-
+  private void forwardIndexWriterHelper(String column, ColumnMetadata existingColumnMetadata, ForwardIndexReader reader,
+      ForwardIndexCreator creator, int numDocs, @Nullable SegmentDictionaryCreator dictionaryCreator) {
     ForwardIndexReaderContext readerContext = reader.createContext();
     boolean isSVColumn = reader.isSingleValue();
 
-    switch (reader.getStoredType()) {
-      // JSON fields are either stored as string or bytes. No special handling is needed because we make this
-      // decision based on the storedType of the reader.
-      case INT: {
-        for (int i = 0; i < numDocs; i++) {
-          if (isSVColumn) {
-            int val = reader.getInt(i, readerContext);
-            creator.putInt(val);
-          } else {
-            int[] ints = reader.getIntMV(i, readerContext);
-            creator.putIntMV(ints);
-          }
+    if (dictionaryCreator != null) {
+      int maxNumValuesPerEntry = existingColumnMetadata.getMaxNumberOfMultiValues();
+      PinotSegmentColumnReader columnReader = new PinotSegmentColumnReader(reader, null, null, maxNumValuesPerEntry);
+
+      for (int i = 0; i < numDocs; i++) {
+        Object obj = columnReader.getValue(i);
+
+        if (isSVColumn) {
+          int dictId = dictionaryCreator.indexOfSV(obj);
+          creator.putDictId(dictId);
+        } else {
+          int[] dictIds = dictionaryCreator.indexOfMV(obj);
+          creator.putDictIdMV(dictIds);
         }
-        break;
       }
-      case LONG: {
-        for (int i = 0; i < numDocs; i++) {
-          if (isSVColumn) {
-            long val = reader.getLong(i, readerContext);
-            creator.putLong(val);
-          } else {
-            long[] longs = reader.getLongMV(i, readerContext);
-            creator.putLongMV(longs);
+    } else {
+      switch (reader.getStoredType()) {
+        // JSON fields are either stored as string or bytes. No special handling is needed because we make this
+        // decision based on the storedType of the reader.
+        case INT: {
+          for (int i = 0; i < numDocs; i++) {
+            if (isSVColumn) {
+              int val = reader.getInt(i, readerContext);
+              creator.putInt(val);
+            } else {
+              int[] ints = reader.getIntMV(i, readerContext);
+              creator.putIntMV(ints);
+            }
           }
+          break;
         }
-        break;
-      }
-      case FLOAT: {
-        for (int i = 0; i < numDocs; i++) {
-          if (isSVColumn) {
-            float val = reader.getFloat(i, readerContext);
-            creator.putFloat(val);
-          } else {
-            float[] floats = reader.getFloatMV(i, readerContext);
-            creator.putFloatMV(floats);
+        case LONG: {
+          for (int i = 0; i < numDocs; i++) {
+            if (isSVColumn) {
+              long val = reader.getLong(i, readerContext);
+              creator.putLong(val);
+            } else {
+              long[] longs = reader.getLongMV(i, readerContext);
+              creator.putLongMV(longs);
+            }
           }
+          break;
         }
-        break;
-      }
-      case DOUBLE: {
-        for (int i = 0; i < numDocs; i++) {
-          if (isSVColumn) {
-            double val = reader.getDouble(i, readerContext);
-            creator.putDouble(val);
-          } else {
-            double[] doubles = reader.getDoubleMV(i, readerContext);
-            creator.putDoubleMV(doubles);
+        case FLOAT: {
+          for (int i = 0; i < numDocs; i++) {
+            if (isSVColumn) {
+              float val = reader.getFloat(i, readerContext);
+              creator.putFloat(val);
+            } else {
+              float[] floats = reader.getFloatMV(i, readerContext);
+              creator.putFloatMV(floats);
+            }
           }
+          break;
         }
-        break;
-      }
-      case STRING: {
-        for (int i = 0; i < numDocs; i++) {
-          if (isSVColumn) {
-            String val = reader.getString(i, readerContext);
-            creator.putString(val);
-          } else {
-            String[] strings = reader.getStringMV(i, readerContext);
-            creator.putStringMV(strings);
+        case DOUBLE: {
+          for (int i = 0; i < numDocs; i++) {
+            if (isSVColumn) {
+              double val = reader.getDouble(i, readerContext);
+              creator.putDouble(val);
+            } else {
+              double[] doubles = reader.getDoubleMV(i, readerContext);
+              creator.putDoubleMV(doubles);
+            }
           }
+          break;
         }
-        break;
-      }
-      case BYTES: {
-        for (int i = 0; i < numDocs; i++) {
-          if (isSVColumn) {
-            byte[] val = reader.getBytes(i, readerContext);
-            creator.putBytes(val);
-          } else {
-            byte[][] bytesArray = reader.getBytesMV(i, readerContext);
-            creator.putBytesMV(bytesArray);
+        case STRING: {
+          for (int i = 0; i < numDocs; i++) {
+            if (isSVColumn) {
+              String val = reader.getString(i, readerContext);
+              creator.putString(val);
+            } else {
+              String[] strings = reader.getStringMV(i, readerContext);
+              creator.putStringMV(strings);
+            }
           }
+          break;
         }
-        break;
-      }
-      case BIG_DECIMAL: {
-        for (int i = 0; i < numDocs; i++) {
+        case BYTES: {
+          for (int i = 0; i < numDocs; i++) {
+            if (isSVColumn) {
+              byte[] val = reader.getBytes(i, readerContext);
+              creator.putBytes(val);
+            } else {
+              byte[][] bytesArray = reader.getBytesMV(i, readerContext);
+              creator.putBytesMV(bytesArray);
+            }
+          }
+          break;
+        }
+        case BIG_DECIMAL: {
           Preconditions.checkState(isSVColumn, "BigDecimal is not supported for MV columns");
-          BigDecimal val = reader.getBigDecimal(i, readerContext);
-          creator.putBigDecimal(val);
+          for (int i = 0; i < numDocs; i++) {
+            BigDecimal val = reader.getBigDecimal(i, readerContext);
+            creator.putBigDecimal(val);
+          }
+          break;
         }
-        break;
+        default:
+          throw new IllegalStateException();
+      }
+    }
+  }
+
+  private void enableDictionary(String column, SegmentDirectory.Writer segmentWriter,
+      IndexCreatorProvider indexCreatorProvider)
+      throws Exception {
+    Preconditions.checkState(_segmentMetadata.getVersion() == SegmentVersion.v3);
+    ColumnMetadata existingColMetadata = _segmentMetadata.getColumnMetadataFor(column);
+    Preconditions.checkState(!existingColMetadata.hasDictionary(),
+        "Cannot rewrite dictionary enabled forward index. Dictionary already exists for column:" + column);
+    boolean isSingleValue = existingColMetadata.isSingleValue();
+
+    File indexDir = _segmentMetadata.getIndexDir();
+    String segmentName = _segmentMetadata.getName();
+    File inProgress = new File(indexDir, column + ".dict.inprogress");
+    File dictionaryFile = new File(indexDir, column + V1Constants.Dict.FILE_EXTENSION);
+    String fwdIndexFileExtension;
+    if (isSingleValue) {
+      fwdIndexFileExtension =
+          existingColMetadata.isSorted() ? V1Constants.Indexes.SORTED_SV_FORWARD_INDEX_FILE_EXTENSION

Review Comment:
   Discussed offline. @vvivekiyer  you may want to update this thread



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


[GitHub] [pinot] vvivekiyer commented on a diff in pull request #9678: Support creating dictionary at runtime for an existing column

Posted by GitBox <gi...@apache.org>.
vvivekiyer commented on code in PR #9678:
URL: https://github.com/apache/pinot/pull/9678#discussion_r1011006485


##########
pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/creator/impl/SegmentIndexCreationDriverImpl.java:
##########
@@ -416,15 +416,11 @@ void buildIndexCreationInfo()
       String columnName = fieldSpec.getName();
       DataType storedType = fieldSpec.getDataType().getStoredType();
       ColumnStatistics columnProfile = _segmentStats.getColumnProfileFor(columnName);
-      boolean useVarLengthDictionary = varLengthDictionaryColumns.contains(columnName);
+      boolean useVarLengthDictionary =
+          varLengthDictionaryColumns.contains(columnName) || shouldUseVarLengthDictionary(storedType, columnProfile);

Review Comment:
   Done.



##########
pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/loader/ForwardIndexHandler.java:
##########
@@ -26,23 +26,43 @@
 import java.util.HashSet;
 import java.util.Map;
 import java.util.Set;
+import javax.annotation.Nullable;
+import org.apache.commons.configuration.PropertiesConfiguration;
 import org.apache.commons.io.FileUtils;
+import org.apache.pinot.segment.local.segment.creator.impl.SegmentDictionaryCreator;
+import org.apache.pinot.segment.local.segment.creator.impl.SegmentIndexCreationDriverImpl;
 import org.apache.pinot.segment.local.segment.creator.impl.fwd.MultiValueVarByteRawIndexCreator;
+import org.apache.pinot.segment.local.segment.creator.impl.stats.AbstractColumnStatisticsCollector;
+import org.apache.pinot.segment.local.segment.creator.impl.stats.BigDecimalColumnPreIndexStatsCollector;
+import org.apache.pinot.segment.local.segment.creator.impl.stats.BytesColumnPredIndexStatsCollector;
+import org.apache.pinot.segment.local.segment.creator.impl.stats.DoubleColumnPreIndexStatsCollector;
+import org.apache.pinot.segment.local.segment.creator.impl.stats.FloatColumnPreIndexStatsCollector;
+import org.apache.pinot.segment.local.segment.creator.impl.stats.IntColumnPreIndexStatsCollector;
+import org.apache.pinot.segment.local.segment.creator.impl.stats.LongColumnPreIndexStatsCollector;
+import org.apache.pinot.segment.local.segment.creator.impl.stats.StringColumnPreIndexStatsCollector;
+import org.apache.pinot.segment.local.segment.readers.PinotSegmentColumnReader;
 import org.apache.pinot.segment.spi.ColumnMetadata;
 import org.apache.pinot.segment.spi.SegmentMetadata;
 import org.apache.pinot.segment.spi.V1Constants;
 import org.apache.pinot.segment.spi.compression.ChunkCompressionType;
 import org.apache.pinot.segment.spi.creator.IndexCreationContext;
 import org.apache.pinot.segment.spi.creator.IndexCreatorProvider;
 import org.apache.pinot.segment.spi.creator.SegmentVersion;
+import org.apache.pinot.segment.spi.creator.StatsCollectorConfig;
 import org.apache.pinot.segment.spi.index.creator.ForwardIndexCreator;
 import org.apache.pinot.segment.spi.index.reader.ForwardIndexReader;
 import org.apache.pinot.segment.spi.index.reader.ForwardIndexReaderContext;
 import org.apache.pinot.segment.spi.store.ColumnIndexType;
 import org.apache.pinot.segment.spi.store.SegmentDirectory;
+import org.apache.pinot.segment.spi.store.SegmentDirectoryPaths;
+import org.apache.pinot.spi.data.Schema;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import static org.apache.pinot.segment.spi.V1Constants.MetadataKeys.Column.DICTIONARY_ELEMENT_SIZE;

Review Comment:
   Done



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


[GitHub] [pinot] siddharthteotia commented on a diff in pull request #9678: Support creating dictionary at runtime for an existing column

Posted by GitBox <gi...@apache.org>.
siddharthteotia commented on code in PR #9678:
URL: https://github.com/apache/pinot/pull/9678#discussion_r1012322511


##########
pinot-segment-local/src/test/java/org/apache/pinot/segment/local/segment/index/loader/SegmentPreProcessorTest.java:
##########
@@ -321,15 +348,106 @@ public void testEnableFSTIndexOnExistingColumnDictEncoded()
     assertNotNull(columnMetadata);
     checkFSTIndexCreation(EXISTING_STRING_COL_DICT, 9, 4, _newColumnsSchemaWithFST, false, false, 26);
 
-    constructV1Segment();
+    constructV1Segment(Collections.emptyList(), Collections.emptyList(), Collections.emptyList());
     segmentMetadata = new SegmentMetadataImpl(_indexDir);
     columnMetadata = segmentMetadata.getColumnMetadataFor(EXISTING_STRING_COL_DICT);
     assertNotNull(columnMetadata);
     checkFSTIndexCreation(EXISTING_STRING_COL_DICT, 9, 4, _newColumnsSchemaWithFST, false, false, 26);
   }
 
   @Test
-  public void testForwardIndexHandler()
+  public void testForwardIndexHandlerEnableDictionary()
+      throws Exception {
+    // Add raw columns in indexingConfig so that the ForwardIndexHandler doesn't end up converting them to dictionary
+    // enabled columns
+    _indexLoadingConfig.getNoDictionaryColumns().add(EXISTING_INT_COL_RAW_MV);
+    _indexLoadingConfig.getNoDictionaryColumns().add(EXISTING_INT_COL_RAW);
+    _indexLoadingConfig.getNoDictionaryColumns().add(EXISTING_STRING_COL_RAW);
+
+    // TEST 1. Check running forwardIndexHandler on a V1 segment. No-op for all existing raw columns.
+    constructV1Segment(Collections.emptyList(), Collections.emptyList(), Collections.emptyList());
+    checkForwardIndexCreation(EXISTING_STRING_COL_DICT, 9, 4, _schema, false, true, false, 26, null, true, 0,
+        DataType.STRING, 100000);
+    validateIndex(ColumnIndexType.FORWARD_INDEX, EXISTING_STRING_COL_RAW, 5, 3, _schema, false, false, false, 0, true,
+        0, ChunkCompressionType.LZ4, false, DataType.STRING, 100000);
+    validateIndex(ColumnIndexType.FORWARD_INDEX, EXISTING_INT_COL_RAW_MV, 18499, 15, _schema, false, false, false, 0,
+        false, 13, ChunkCompressionType.LZ4, false, DataType.INT, 106688);
+    validateIndex(ColumnIndexType.FORWARD_INDEX, EXISTING_INT_COL_RAW, 42242, 16, _schema, false, false, false, 0, true,
+        0, ChunkCompressionType.LZ4, false, DataType.INT, 100000);
+
+    // Convert the segment to V3.
+    new SegmentV1V2ToV3FormatConverter().convert(_indexDir);
+
+    // TEST 2: Run reload with no-changes.
+    checkForwardIndexCreation(EXISTING_STRING_COL_DICT, 9, 4, _schema, false, true, false, 26, null, true, 0,
+        DataType.STRING, 100000);
+
+    // TEST 3: EXISTING_STRING_COL_RAW. Enable dictionary. Also add inverted index and text index. Reload code path
+    // will create dictionary, inverted index and text index.
+    _indexLoadingConfig.getNoDictionaryColumns().remove(EXISTING_STRING_COL_RAW);
+    _indexLoadingConfig.getInvertedIndexColumns().add(EXISTING_STRING_COL_RAW);
+    _indexLoadingConfig.getTextIndexColumns().add(EXISTING_STRING_COL_RAW);
+    checkForwardIndexCreation(EXISTING_STRING_COL_RAW, 5, 3, _schema, false, true, false, 4, null, true, 0,
+        DataType.STRING, 100000);
+    validateIndex(ColumnIndexType.INVERTED_INDEX, EXISTING_STRING_COL_RAW, 5, 3, _schema, false, true, false, 4, true,
+        0, null, false, DataType.STRING, 100000);
+    validateIndex(ColumnIndexType.TEXT_INDEX, EXISTING_STRING_COL_RAW, 5, 3, _schema, false, true, false, 4, true, 0,
+        null, false, DataType.STRING, 100000);
+
+    // TEST4: EXISTING_STRING_COL_RAW. Enable dictionary on a raw column that already has text index.
+    List<String> textIndexCols = new ArrayList<>();
+    textIndexCols.add(EXISTING_STRING_COL_RAW);
+    constructV1Segment(Collections.emptyList(), textIndexCols, Collections.emptyList());
+    new SegmentV1V2ToV3FormatConverter().convert(_indexDir);
+    validateIndex(ColumnIndexType.TEXT_INDEX, EXISTING_STRING_COL_RAW, 5, 3, _schema, false, false, false, 0, true, 0,
+        null, false, DataType.STRING, 100000);
+
+    // At this point, the segment has text index. Now, the reload path should create a dictionary.
+    checkForwardIndexCreation(EXISTING_STRING_COL_RAW, 5, 3, _schema, false, true, false, 4, null, true, 0,
+        DataType.STRING, 100000);
+    validateIndex(ColumnIndexType.TEXT_INDEX, EXISTING_STRING_COL_RAW, 5, 3, _schema, false, true, false, 4, true, 0,
+        null, false, DataType.STRING, 100000);
+    // Add it back so that this column is not rewritten for the other tests below.
+    _indexLoadingConfig.getNoDictionaryColumns().add(EXISTING_STRING_COL_RAW);
+
+    // TEST 5: EXISTING_INT_COL_RAW. Enable dictionary on a column that already has range index.
+    List<String> rangeIndexCols = new ArrayList<>();
+    rangeIndexCols.add(EXISTING_INT_COL_RAW);
+    constructV1Segment(Collections.emptyList(), Collections.emptyList(), rangeIndexCols);
+    new SegmentV1V2ToV3FormatConverter().convert(_indexDir);
+    validateIndex(ColumnIndexType.RANGE_INDEX, EXISTING_INT_COL_RAW, 42242, 16, _schema, false, false, false, 0, true,
+        0, ChunkCompressionType.LZ4, false, DataType.INT, 100000);
+    // At this point, the segment has range index. Now the reload path should create a dictionary and rewrite the
+    // range index.
+    _indexLoadingConfig.getNoDictionaryColumns().remove(EXISTING_INT_COL_RAW);
+    _indexLoadingConfig.getRangeIndexColumns().add(EXISTING_INT_COL_RAW);
+    checkForwardIndexCreation(EXISTING_INT_COL_RAW, 42242, 16, _schema, false, true, false, 0, null, true, 0,
+        DataType.INT, 100000);
+    validateIndex(ColumnIndexType.RANGE_INDEX, EXISTING_INT_COL_RAW, 42242, 16, _schema, false, true, false, 0, true, 0,
+        null, false, DataType.INT, 100000);
+    // Add it back so that this column is not rewritten for the other tests below.
+    _indexLoadingConfig.getNoDictionaryColumns().add(EXISTING_INT_COL_RAW);
+
+    // TEST 6: EXISTING_INT_COL_RAW_MV. Enable dictionary for an MV column. Also enable inverted index and range index.
+    constructV1Segment(Collections.emptyList(), Collections.emptyList(), Collections.emptyList());
+    new SegmentV1V2ToV3FormatConverter().convert(_indexDir);
+    validateIndex(ColumnIndexType.FORWARD_INDEX, EXISTING_INT_COL_RAW_MV, 18499, 15, _schema, false, false, false, 0,
+        false, 13, ChunkCompressionType.LZ4, false, DataType.INT, 106688);
+
+    // Enable dictionary and inverted index.

Review Comment:
   (nit) should be enable dict, inv index and range index



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


[GitHub] [pinot] siddharthteotia merged pull request #9678: Support creating dictionary at runtime for an existing column

Posted by GitBox <gi...@apache.org>.
siddharthteotia merged PR #9678:
URL: https://github.com/apache/pinot/pull/9678


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


[GitHub] [pinot] somandal commented on a diff in pull request #9678: Support creating dictionary at runtime for an existing column

Posted by GitBox <gi...@apache.org>.
somandal commented on code in PR #9678:
URL: https://github.com/apache/pinot/pull/9678#discussion_r1012149014


##########
pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/loader/ForwardIndexHandler.java:
##########
@@ -271,118 +306,293 @@ private void rewriteRawSVForwardIndex(String column, ColumnMetadata existingColM
               .forForwardIndex(newCompressionType, _indexLoadingConfig.getColumnProperties());
 
       try (ForwardIndexCreator creator = indexCreatorProvider.newForwardIndexCreator(context)) {
-        // If creator stored type and the reader stored type do not match, throw an exception.
         if (!reader.getStoredType().equals(creator.getValueType())) {
+          // Creator stored type should match reader stored type for raw columns. We do not support changing datatypes.
           String failureMsg =
               "Unsupported operation to change datatype for column=" + column + " from " + reader.getStoredType()
                   .toString() + " to " + creator.getValueType().toString();
           throw new UnsupportedOperationException(failureMsg);
         }
 
         int numDocs = existingColMetadata.getTotalDocs();
-        forwardIndexWriterHelper(column, reader, creator, numDocs);
+        forwardIndexWriterHelper(column, existingColMetadata, reader, creator, numDocs, null);
       }
     }
   }
 
-  private void forwardIndexWriterHelper(String column, ForwardIndexReader reader, ForwardIndexCreator creator,
-      int numDocs) {
-    // If creator stored type should match reader stored type. We do not support changing datatypes.
-    if (!reader.getStoredType().equals(creator.getValueType())) {
-      String failureMsg =
-          "Unsupported operation to change datatype for column=" + column + " from " + reader.getStoredType().toString()
-              + " to " + creator.getValueType().toString();
-      throw new UnsupportedOperationException(failureMsg);
-    }
-
+  private void forwardIndexWriterHelper(String column, ColumnMetadata existingColumnMetadata, ForwardIndexReader reader,
+      ForwardIndexCreator creator, int numDocs, @Nullable SegmentDictionaryCreator dictionaryCreator) {
     ForwardIndexReaderContext readerContext = reader.createContext();
     boolean isSVColumn = reader.isSingleValue();
 
-    switch (reader.getStoredType()) {
-      // JSON fields are either stored as string or bytes. No special handling is needed because we make this
-      // decision based on the storedType of the reader.
-      case INT: {
-        for (int i = 0; i < numDocs; i++) {
-          if (isSVColumn) {
-            int val = reader.getInt(i, readerContext);
-            creator.putInt(val);
-          } else {
-            int[] ints = reader.getIntMV(i, readerContext);
-            creator.putIntMV(ints);
-          }
+    if (dictionaryCreator != null) {
+      int maxNumValuesPerEntry = existingColumnMetadata.getMaxNumberOfMultiValues();
+      PinotSegmentColumnReader columnReader = new PinotSegmentColumnReader(reader, null, null, maxNumValuesPerEntry);
+
+      for (int i = 0; i < numDocs; i++) {
+        Object obj = columnReader.getValue(i);
+
+        if (isSVColumn) {
+          int dictId = dictionaryCreator.indexOfSV(obj);
+          creator.putDictId(dictId);
+        } else {
+          int[] dictIds = dictionaryCreator.indexOfMV(obj);
+          creator.putDictIdMV(dictIds);
         }
-        break;
       }
-      case LONG: {
-        for (int i = 0; i < numDocs; i++) {
-          if (isSVColumn) {
-            long val = reader.getLong(i, readerContext);
-            creator.putLong(val);
-          } else {
-            long[] longs = reader.getLongMV(i, readerContext);
-            creator.putLongMV(longs);
+    } else {
+      switch (reader.getStoredType()) {
+        // JSON fields are either stored as string or bytes. No special handling is needed because we make this
+        // decision based on the storedType of the reader.
+        case INT: {
+          for (int i = 0; i < numDocs; i++) {
+            if (isSVColumn) {
+              int val = reader.getInt(i, readerContext);
+              creator.putInt(val);
+            } else {
+              int[] ints = reader.getIntMV(i, readerContext);
+              creator.putIntMV(ints);
+            }
           }
+          break;
         }
-        break;
-      }
-      case FLOAT: {
-        for (int i = 0; i < numDocs; i++) {
-          if (isSVColumn) {
-            float val = reader.getFloat(i, readerContext);
-            creator.putFloat(val);
-          } else {
-            float[] floats = reader.getFloatMV(i, readerContext);
-            creator.putFloatMV(floats);
+        case LONG: {
+          for (int i = 0; i < numDocs; i++) {
+            if (isSVColumn) {
+              long val = reader.getLong(i, readerContext);
+              creator.putLong(val);
+            } else {
+              long[] longs = reader.getLongMV(i, readerContext);
+              creator.putLongMV(longs);
+            }
           }
+          break;
         }
-        break;
-      }
-      case DOUBLE: {
-        for (int i = 0; i < numDocs; i++) {
-          if (isSVColumn) {
-            double val = reader.getDouble(i, readerContext);
-            creator.putDouble(val);
-          } else {
-            double[] doubles = reader.getDoubleMV(i, readerContext);
-            creator.putDoubleMV(doubles);
+        case FLOAT: {
+          for (int i = 0; i < numDocs; i++) {
+            if (isSVColumn) {
+              float val = reader.getFloat(i, readerContext);
+              creator.putFloat(val);
+            } else {
+              float[] floats = reader.getFloatMV(i, readerContext);
+              creator.putFloatMV(floats);
+            }
           }
+          break;
         }
-        break;
-      }
-      case STRING: {
-        for (int i = 0; i < numDocs; i++) {
-          if (isSVColumn) {
-            String val = reader.getString(i, readerContext);
-            creator.putString(val);
-          } else {
-            String[] strings = reader.getStringMV(i, readerContext);
-            creator.putStringMV(strings);
+        case DOUBLE: {
+          for (int i = 0; i < numDocs; i++) {
+            if (isSVColumn) {
+              double val = reader.getDouble(i, readerContext);
+              creator.putDouble(val);
+            } else {
+              double[] doubles = reader.getDoubleMV(i, readerContext);
+              creator.putDoubleMV(doubles);
+            }
           }
+          break;
         }
-        break;
-      }
-      case BYTES: {
-        for (int i = 0; i < numDocs; i++) {
-          if (isSVColumn) {
-            byte[] val = reader.getBytes(i, readerContext);
-            creator.putBytes(val);
-          } else {
-            byte[][] bytesArray = reader.getBytesMV(i, readerContext);
-            creator.putBytesMV(bytesArray);
+        case STRING: {
+          for (int i = 0; i < numDocs; i++) {
+            if (isSVColumn) {
+              String val = reader.getString(i, readerContext);
+              creator.putString(val);
+            } else {
+              String[] strings = reader.getStringMV(i, readerContext);
+              creator.putStringMV(strings);
+            }
           }
+          break;
         }
-        break;
-      }
-      case BIG_DECIMAL: {
-        for (int i = 0; i < numDocs; i++) {
+        case BYTES: {
+          for (int i = 0; i < numDocs; i++) {
+            if (isSVColumn) {
+              byte[] val = reader.getBytes(i, readerContext);
+              creator.putBytes(val);
+            } else {
+              byte[][] bytesArray = reader.getBytesMV(i, readerContext);
+              creator.putBytesMV(bytesArray);
+            }
+          }
+          break;
+        }
+        case BIG_DECIMAL: {
           Preconditions.checkState(isSVColumn, "BigDecimal is not supported for MV columns");
-          BigDecimal val = reader.getBigDecimal(i, readerContext);
-          creator.putBigDecimal(val);
+          for (int i = 0; i < numDocs; i++) {
+            BigDecimal val = reader.getBigDecimal(i, readerContext);
+            creator.putBigDecimal(val);
+          }
+          break;
         }
-        break;
+        default:
+          throw new IllegalStateException();
+      }
+    }
+  }
+
+  private void enableDictionary(String column, SegmentDirectory.Writer segmentWriter,
+      IndexCreatorProvider indexCreatorProvider)
+      throws Exception {
+    Preconditions.checkState(_segmentMetadata.getVersion() == SegmentVersion.v3);

Review Comment:
   This is a private function though, so tests shouldn't directly be able to access these anyways. 



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


[GitHub] [pinot] siddharthteotia commented on pull request #9678: Support creating dictionary at runtime for an existing column

Posted by GitBox <gi...@apache.org>.
siddharthteotia commented on PR #9678:
URL: https://github.com/apache/pinot/pull/9678#issuecomment-1299407339

   Let's add e2e query execution tests as well (need not be in integration but one of the subclasses of BaseQueriesTest)


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


[GitHub] [pinot] somandal commented on a diff in pull request #9678: Support creating dictionary at runtime for an existing column

Posted by GitBox <gi...@apache.org>.
somandal commented on code in PR #9678:
URL: https://github.com/apache/pinot/pull/9678#discussion_r1012139502


##########
pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/loader/ForwardIndexHandler.java:
##########
@@ -26,23 +26,43 @@
 import java.util.HashSet;
 import java.util.Map;
 import java.util.Set;
+import javax.annotation.Nullable;
+import org.apache.commons.configuration.PropertiesConfiguration;
 import org.apache.commons.io.FileUtils;
+import org.apache.pinot.segment.local.segment.creator.impl.SegmentDictionaryCreator;
+import org.apache.pinot.segment.local.segment.creator.impl.SegmentIndexCreationDriverImpl;
 import org.apache.pinot.segment.local.segment.creator.impl.fwd.MultiValueVarByteRawIndexCreator;
+import org.apache.pinot.segment.local.segment.creator.impl.stats.AbstractColumnStatisticsCollector;
+import org.apache.pinot.segment.local.segment.creator.impl.stats.BigDecimalColumnPreIndexStatsCollector;
+import org.apache.pinot.segment.local.segment.creator.impl.stats.BytesColumnPredIndexStatsCollector;
+import org.apache.pinot.segment.local.segment.creator.impl.stats.DoubleColumnPreIndexStatsCollector;
+import org.apache.pinot.segment.local.segment.creator.impl.stats.FloatColumnPreIndexStatsCollector;
+import org.apache.pinot.segment.local.segment.creator.impl.stats.IntColumnPreIndexStatsCollector;
+import org.apache.pinot.segment.local.segment.creator.impl.stats.LongColumnPreIndexStatsCollector;
+import org.apache.pinot.segment.local.segment.creator.impl.stats.StringColumnPreIndexStatsCollector;
+import org.apache.pinot.segment.local.segment.readers.PinotSegmentColumnReader;
 import org.apache.pinot.segment.spi.ColumnMetadata;
 import org.apache.pinot.segment.spi.SegmentMetadata;
 import org.apache.pinot.segment.spi.V1Constants;
 import org.apache.pinot.segment.spi.compression.ChunkCompressionType;
 import org.apache.pinot.segment.spi.creator.IndexCreationContext;
 import org.apache.pinot.segment.spi.creator.IndexCreatorProvider;
 import org.apache.pinot.segment.spi.creator.SegmentVersion;
+import org.apache.pinot.segment.spi.creator.StatsCollectorConfig;
 import org.apache.pinot.segment.spi.index.creator.ForwardIndexCreator;
 import org.apache.pinot.segment.spi.index.reader.ForwardIndexReader;
 import org.apache.pinot.segment.spi.index.reader.ForwardIndexReaderContext;
 import org.apache.pinot.segment.spi.store.ColumnIndexType;
 import org.apache.pinot.segment.spi.store.SegmentDirectory;
+import org.apache.pinot.segment.spi.store.SegmentDirectoryPaths;
+import org.apache.pinot.spi.data.Schema;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import static org.apache.pinot.segment.spi.V1Constants.MetadataKeys.Column.DICTIONARY_ELEMENT_SIZE;

Review Comment:
   oops, my bad. What about adding a `*` so that we don't need to update this for every new key we need to use?
   
   ```
   import static org.apache.pinot.segment.spi.V1Constants.MetadataKeys.Column.*;
   ```
   
   I see it done this way in `SegmentColumnarIndexCreator`



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


[GitHub] [pinot] vvivekiyer commented on a diff in pull request #9678: Support creating dictionary at runtime for an existing column

Posted by GitBox <gi...@apache.org>.
vvivekiyer commented on code in PR #9678:
URL: https://github.com/apache/pinot/pull/9678#discussion_r1012178239


##########
pinot-segment-local/src/test/java/org/apache/pinot/segment/local/segment/index/loader/ForwardIndexHandlerTest.java:
##########
@@ -387,7 +464,7 @@ public void testComputeOperation()
     segmentLocalFSDirectory.close();
   }
 
-  @Test
+  @Test(priority = 1)

Review Comment:
   Because we are changing raw columns to dict enabled columns in some newer tests. We want these enable dictionary tests to run after all the test for changing compression types (which work on raw columns) have run.



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


[GitHub] [pinot] siddharthteotia commented on a diff in pull request #9678: Support creating dictionary at runtime for an existing column

Posted by GitBox <gi...@apache.org>.
siddharthteotia commented on code in PR #9678:
URL: https://github.com/apache/pinot/pull/9678#discussion_r1012323342


##########
pinot-segment-local/src/test/java/org/apache/pinot/segment/local/segment/index/loader/SegmentPreProcessorTest.java:
##########
@@ -321,15 +348,106 @@ public void testEnableFSTIndexOnExistingColumnDictEncoded()
     assertNotNull(columnMetadata);
     checkFSTIndexCreation(EXISTING_STRING_COL_DICT, 9, 4, _newColumnsSchemaWithFST, false, false, 26);
 
-    constructV1Segment();
+    constructV1Segment(Collections.emptyList(), Collections.emptyList(), Collections.emptyList());
     segmentMetadata = new SegmentMetadataImpl(_indexDir);
     columnMetadata = segmentMetadata.getColumnMetadataFor(EXISTING_STRING_COL_DICT);
     assertNotNull(columnMetadata);
     checkFSTIndexCreation(EXISTING_STRING_COL_DICT, 9, 4, _newColumnsSchemaWithFST, false, false, 26);
   }
 
   @Test
-  public void testForwardIndexHandler()
+  public void testForwardIndexHandlerEnableDictionary()
+      throws Exception {
+    // Add raw columns in indexingConfig so that the ForwardIndexHandler doesn't end up converting them to dictionary
+    // enabled columns
+    _indexLoadingConfig.getNoDictionaryColumns().add(EXISTING_INT_COL_RAW_MV);
+    _indexLoadingConfig.getNoDictionaryColumns().add(EXISTING_INT_COL_RAW);
+    _indexLoadingConfig.getNoDictionaryColumns().add(EXISTING_STRING_COL_RAW);
+
+    // TEST 1. Check running forwardIndexHandler on a V1 segment. No-op for all existing raw columns.
+    constructV1Segment(Collections.emptyList(), Collections.emptyList(), Collections.emptyList());
+    checkForwardIndexCreation(EXISTING_STRING_COL_DICT, 9, 4, _schema, false, true, false, 26, null, true, 0,
+        DataType.STRING, 100000);
+    validateIndex(ColumnIndexType.FORWARD_INDEX, EXISTING_STRING_COL_RAW, 5, 3, _schema, false, false, false, 0, true,
+        0, ChunkCompressionType.LZ4, false, DataType.STRING, 100000);
+    validateIndex(ColumnIndexType.FORWARD_INDEX, EXISTING_INT_COL_RAW_MV, 18499, 15, _schema, false, false, false, 0,
+        false, 13, ChunkCompressionType.LZ4, false, DataType.INT, 106688);
+    validateIndex(ColumnIndexType.FORWARD_INDEX, EXISTING_INT_COL_RAW, 42242, 16, _schema, false, false, false, 0, true,
+        0, ChunkCompressionType.LZ4, false, DataType.INT, 100000);
+
+    // Convert the segment to V3.
+    new SegmentV1V2ToV3FormatConverter().convert(_indexDir);
+
+    // TEST 2: Run reload with no-changes.
+    checkForwardIndexCreation(EXISTING_STRING_COL_DICT, 9, 4, _schema, false, true, false, 26, null, true, 0,
+        DataType.STRING, 100000);
+
+    // TEST 3: EXISTING_STRING_COL_RAW. Enable dictionary. Also add inverted index and text index. Reload code path
+    // will create dictionary, inverted index and text index.
+    _indexLoadingConfig.getNoDictionaryColumns().remove(EXISTING_STRING_COL_RAW);
+    _indexLoadingConfig.getInvertedIndexColumns().add(EXISTING_STRING_COL_RAW);
+    _indexLoadingConfig.getTextIndexColumns().add(EXISTING_STRING_COL_RAW);
+    checkForwardIndexCreation(EXISTING_STRING_COL_RAW, 5, 3, _schema, false, true, false, 4, null, true, 0,
+        DataType.STRING, 100000);
+    validateIndex(ColumnIndexType.INVERTED_INDEX, EXISTING_STRING_COL_RAW, 5, 3, _schema, false, true, false, 4, true,
+        0, null, false, DataType.STRING, 100000);
+    validateIndex(ColumnIndexType.TEXT_INDEX, EXISTING_STRING_COL_RAW, 5, 3, _schema, false, true, false, 4, true, 0,
+        null, false, DataType.STRING, 100000);
+
+    // TEST4: EXISTING_STRING_COL_RAW. Enable dictionary on a raw column that already has text index.
+    List<String> textIndexCols = new ArrayList<>();
+    textIndexCols.add(EXISTING_STRING_COL_RAW);
+    constructV1Segment(Collections.emptyList(), textIndexCols, Collections.emptyList());
+    new SegmentV1V2ToV3FormatConverter().convert(_indexDir);
+    validateIndex(ColumnIndexType.TEXT_INDEX, EXISTING_STRING_COL_RAW, 5, 3, _schema, false, false, false, 0, true, 0,
+        null, false, DataType.STRING, 100000);
+
+    // At this point, the segment has text index. Now, the reload path should create a dictionary.
+    checkForwardIndexCreation(EXISTING_STRING_COL_RAW, 5, 3, _schema, false, true, false, 4, null, true, 0,
+        DataType.STRING, 100000);
+    validateIndex(ColumnIndexType.TEXT_INDEX, EXISTING_STRING_COL_RAW, 5, 3, _schema, false, true, false, 4, true, 0,
+        null, false, DataType.STRING, 100000);
+    // Add it back so that this column is not rewritten for the other tests below.
+    _indexLoadingConfig.getNoDictionaryColumns().add(EXISTING_STRING_COL_RAW);
+
+    // TEST 5: EXISTING_INT_COL_RAW. Enable dictionary on a column that already has range index.
+    List<String> rangeIndexCols = new ArrayList<>();
+    rangeIndexCols.add(EXISTING_INT_COL_RAW);
+    constructV1Segment(Collections.emptyList(), Collections.emptyList(), rangeIndexCols);
+    new SegmentV1V2ToV3FormatConverter().convert(_indexDir);
+    validateIndex(ColumnIndexType.RANGE_INDEX, EXISTING_INT_COL_RAW, 42242, 16, _schema, false, false, false, 0, true,
+        0, ChunkCompressionType.LZ4, false, DataType.INT, 100000);
+    // At this point, the segment has range index. Now the reload path should create a dictionary and rewrite the
+    // range index.
+    _indexLoadingConfig.getNoDictionaryColumns().remove(EXISTING_INT_COL_RAW);
+    _indexLoadingConfig.getRangeIndexColumns().add(EXISTING_INT_COL_RAW);
+    checkForwardIndexCreation(EXISTING_INT_COL_RAW, 42242, 16, _schema, false, true, false, 0, null, true, 0,
+        DataType.INT, 100000);
+    validateIndex(ColumnIndexType.RANGE_INDEX, EXISTING_INT_COL_RAW, 42242, 16, _schema, false, true, false, 0, true, 0,

Review Comment:
   Does this actually validate that range index got correctly rewritten to be dict based ? I guess that's where query exec tests will be useful



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


[GitHub] [pinot] vvivekiyer commented on a diff in pull request #9678: Support creating dictionary at runtime for an existing column

Posted by GitBox <gi...@apache.org>.
vvivekiyer commented on code in PR #9678:
URL: https://github.com/apache/pinot/pull/9678#discussion_r1012136265


##########
pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/loader/ForwardIndexHandler.java:
##########
@@ -127,7 +151,15 @@ Map<String, Operation> computeOperation(SegmentDirectory.Reader segmentReader)
     Set<String> newNoDictColumns = _indexLoadingConfig.getNoDictionaryColumns();
 
     for (String column : existingAllColumns) {
-      if (existingNoDictColumns.contains(column) && newNoDictColumns.contains(column)) {
+      if (existingNoDictColumns.contains(column) && !newNoDictColumns.contains(column)) {
+        if (_schema == null || _indexLoadingConfig.getTableConfig() == null) {
+          // This can only happen in tests.

Review Comment:
   There are some tests that call SegmentPreProcessor with null schema. I added this condition because those tests could end up failing. 



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


[GitHub] [pinot] siddharthteotia commented on a diff in pull request #9678: Support creating dictionary at runtime for an existing column

Posted by GitBox <gi...@apache.org>.
siddharthteotia commented on code in PR #9678:
URL: https://github.com/apache/pinot/pull/9678#discussion_r1011046909


##########
pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/loader/ForwardIndexHandler.java:
##########
@@ -271,118 +304,295 @@ private void rewriteRawSVForwardIndex(String column, ColumnMetadata existingColM
               .forForwardIndex(newCompressionType, _indexLoadingConfig.getColumnProperties());
 
       try (ForwardIndexCreator creator = indexCreatorProvider.newForwardIndexCreator(context)) {
-        // If creator stored type and the reader stored type do not match, throw an exception.
         if (!reader.getStoredType().equals(creator.getValueType())) {
+          // Creator stored type should match reader stored type for raw columns. We do not support changing datatypes.
           String failureMsg =
               "Unsupported operation to change datatype for column=" + column + " from " + reader.getStoredType()
                   .toString() + " to " + creator.getValueType().toString();
           throw new UnsupportedOperationException(failureMsg);
         }
 
         int numDocs = existingColMetadata.getTotalDocs();
-        forwardIndexWriterHelper(column, reader, creator, numDocs);
+        forwardIndexWriterHelper(column, existingColMetadata, reader, creator, numDocs, null);
       }
     }
   }
 
-  private void forwardIndexWriterHelper(String column, ForwardIndexReader reader, ForwardIndexCreator creator,
-      int numDocs) {
-    // If creator stored type should match reader stored type. We do not support changing datatypes.
-    if (!reader.getStoredType().equals(creator.getValueType())) {
-      String failureMsg =
-          "Unsupported operation to change datatype for column=" + column + " from " + reader.getStoredType().toString()
-              + " to " + creator.getValueType().toString();
-      throw new UnsupportedOperationException(failureMsg);
-    }
-
+  private void forwardIndexWriterHelper(String column, ColumnMetadata existingColumnMetadata, ForwardIndexReader reader,
+      ForwardIndexCreator creator, int numDocs, @Nullable SegmentDictionaryCreator dictionaryCreator) {
     ForwardIndexReaderContext readerContext = reader.createContext();
     boolean isSVColumn = reader.isSingleValue();
 
-    switch (reader.getStoredType()) {
-      // JSON fields are either stored as string or bytes. No special handling is needed because we make this
-      // decision based on the storedType of the reader.
-      case INT: {
-        for (int i = 0; i < numDocs; i++) {
-          if (isSVColumn) {
-            int val = reader.getInt(i, readerContext);
-            creator.putInt(val);
-          } else {
-            int[] ints = reader.getIntMV(i, readerContext);
-            creator.putIntMV(ints);
-          }
+    if (dictionaryCreator != null) {
+      int maxNumValuesPerEntry = existingColumnMetadata.getMaxNumberOfMultiValues();
+      PinotSegmentColumnReader columnReader = new PinotSegmentColumnReader(reader, null, null, maxNumValuesPerEntry);
+
+      for (int i = 0; i < numDocs; i++) {
+        Object obj = columnReader.getValue(i);
+
+        if (isSVColumn) {
+          int dictId = dictionaryCreator.indexOfSV(obj);
+          creator.putDictId(dictId);
+        } else {
+          int[] dictIds = dictionaryCreator.indexOfMV(obj);
+          creator.putDictIdMV(dictIds);
         }
-        break;
       }
-      case LONG: {
-        for (int i = 0; i < numDocs; i++) {
-          if (isSVColumn) {
-            long val = reader.getLong(i, readerContext);
-            creator.putLong(val);
-          } else {
-            long[] longs = reader.getLongMV(i, readerContext);
-            creator.putLongMV(longs);
+    } else {
+      switch (reader.getStoredType()) {
+        // JSON fields are either stored as string or bytes. No special handling is needed because we make this
+        // decision based on the storedType of the reader.
+        case INT: {
+          for (int i = 0; i < numDocs; i++) {
+            if (isSVColumn) {
+              int val = reader.getInt(i, readerContext);
+              creator.putInt(val);
+            } else {
+              int[] ints = reader.getIntMV(i, readerContext);
+              creator.putIntMV(ints);
+            }
           }
+          break;
         }
-        break;
-      }
-      case FLOAT: {
-        for (int i = 0; i < numDocs; i++) {
-          if (isSVColumn) {
-            float val = reader.getFloat(i, readerContext);
-            creator.putFloat(val);
-          } else {
-            float[] floats = reader.getFloatMV(i, readerContext);
-            creator.putFloatMV(floats);
+        case LONG: {
+          for (int i = 0; i < numDocs; i++) {
+            if (isSVColumn) {
+              long val = reader.getLong(i, readerContext);
+              creator.putLong(val);
+            } else {
+              long[] longs = reader.getLongMV(i, readerContext);
+              creator.putLongMV(longs);
+            }
           }
+          break;
         }
-        break;
-      }
-      case DOUBLE: {
-        for (int i = 0; i < numDocs; i++) {
-          if (isSVColumn) {
-            double val = reader.getDouble(i, readerContext);
-            creator.putDouble(val);
-          } else {
-            double[] doubles = reader.getDoubleMV(i, readerContext);
-            creator.putDoubleMV(doubles);
+        case FLOAT: {
+          for (int i = 0; i < numDocs; i++) {
+            if (isSVColumn) {
+              float val = reader.getFloat(i, readerContext);
+              creator.putFloat(val);
+            } else {
+              float[] floats = reader.getFloatMV(i, readerContext);
+              creator.putFloatMV(floats);
+            }
           }
+          break;
         }
-        break;
-      }
-      case STRING: {
-        for (int i = 0; i < numDocs; i++) {
-          if (isSVColumn) {
-            String val = reader.getString(i, readerContext);
-            creator.putString(val);
-          } else {
-            String[] strings = reader.getStringMV(i, readerContext);
-            creator.putStringMV(strings);
+        case DOUBLE: {
+          for (int i = 0; i < numDocs; i++) {
+            if (isSVColumn) {
+              double val = reader.getDouble(i, readerContext);
+              creator.putDouble(val);
+            } else {
+              double[] doubles = reader.getDoubleMV(i, readerContext);
+              creator.putDoubleMV(doubles);
+            }
           }
+          break;
         }
-        break;
-      }
-      case BYTES: {
-        for (int i = 0; i < numDocs; i++) {
-          if (isSVColumn) {
-            byte[] val = reader.getBytes(i, readerContext);
-            creator.putBytes(val);
-          } else {
-            byte[][] bytesArray = reader.getBytesMV(i, readerContext);
-            creator.putBytesMV(bytesArray);
+        case STRING: {
+          for (int i = 0; i < numDocs; i++) {
+            if (isSVColumn) {
+              String val = reader.getString(i, readerContext);
+              creator.putString(val);
+            } else {
+              String[] strings = reader.getStringMV(i, readerContext);
+              creator.putStringMV(strings);
+            }
           }
+          break;
         }
-        break;
-      }
-      case BIG_DECIMAL: {
-        for (int i = 0; i < numDocs; i++) {
+        case BYTES: {
+          for (int i = 0; i < numDocs; i++) {
+            if (isSVColumn) {
+              byte[] val = reader.getBytes(i, readerContext);
+              creator.putBytes(val);
+            } else {
+              byte[][] bytesArray = reader.getBytesMV(i, readerContext);
+              creator.putBytesMV(bytesArray);
+            }
+          }
+          break;
+        }
+        case BIG_DECIMAL: {
           Preconditions.checkState(isSVColumn, "BigDecimal is not supported for MV columns");
-          BigDecimal val = reader.getBigDecimal(i, readerContext);
-          creator.putBigDecimal(val);
+          for (int i = 0; i < numDocs; i++) {
+            BigDecimal val = reader.getBigDecimal(i, readerContext);
+            creator.putBigDecimal(val);
+          }
+          break;
         }
-        break;
+        default:
+          throw new IllegalStateException();
+      }
+    }
+  }
+
+  private void enableDictionary(String column, SegmentDirectory.Writer segmentWriter,
+      IndexCreatorProvider indexCreatorProvider)
+      throws Exception {
+    Preconditions.checkState(_segmentMetadata.getVersion() == SegmentVersion.v3);
+    ColumnMetadata existingColMetadata = _segmentMetadata.getColumnMetadataFor(column);
+    Preconditions.checkState(!existingColMetadata.hasDictionary(),
+        "Cannot rewrite dictionary enabled forward index. Dictionary already exists for column:" + column);
+    boolean isSingleValue = existingColMetadata.isSingleValue();
+
+    File indexDir = _segmentMetadata.getIndexDir();
+    String segmentName = _segmentMetadata.getName();
+    File inProgress = new File(indexDir, column + ".dict.inprogress");
+    File dictionaryFile = new File(indexDir, column + V1Constants.Dict.FILE_EXTENSION);
+    String fwdIndexFileExtension;
+    if (isSingleValue) {
+      fwdIndexFileExtension =
+          existingColMetadata.isSorted() ? V1Constants.Indexes.SORTED_SV_FORWARD_INDEX_FILE_EXTENSION
+              : V1Constants.Indexes.UNSORTED_SV_FORWARD_INDEX_FILE_EXTENSION;
+    } else {
+      fwdIndexFileExtension = V1Constants.Indexes.UNSORTED_MV_FORWARD_INDEX_FILE_EXTENSION;
+    }
+    File fwdIndexFile = new File(indexDir, column + fwdIndexFileExtension);
+
+    if (!inProgress.exists()) {
+      // Marker file does not exist, which means last run ended normally.
+      // Create a marker file.
+      FileUtils.touch(inProgress);
+    } else {
+      // Marker file exists, which means last run was interrupted.
+      // Remove forward index and dictionary files if they exist.
+      FileUtils.deleteQuietly(fwdIndexFile);
+      FileUtils.deleteQuietly(dictionaryFile);
+    }
+
+    LOGGER.info("Creating a new dictionary for segment={} and column={}", segmentName, column);
+    SegmentDictionaryCreator dictionaryCreator = buildDictionary(column, existingColMetadata, segmentWriter);
+    LoaderUtils.writeIndexToV3Format(segmentWriter, column, dictionaryFile, ColumnIndexType.DICTIONARY);
+
+    LOGGER.info("Built dictionary. Rewriting dictionary enabled forward index for segment={} and column={}",
+        segmentName, column);
+    writeDictEnabledForwardIndex(column, existingColMetadata, segmentWriter, indexDir, indexCreatorProvider,
+        dictionaryCreator);
+    // We used the existing forward index to generate a new forward index. The existing forward index will be in V3
+    // format and the new forward index will be in V1 format. Remove the existing forward index as it is not needed
+    // anymore. Note that removeIndex() will only mark an index for removal and remove the in-memory state. The
+    // actual cleanup from columns.psf file will happen when singleFileIndexDirectory.cleanupRemovedIndices() is
+    // called during segmentWriter.close().
+    segmentWriter.removeIndex(column, ColumnIndexType.FORWARD_INDEX);
+    LoaderUtils.writeIndexToV3Format(segmentWriter, column, fwdIndexFile, ColumnIndexType.FORWARD_INDEX);
+
+    LOGGER.info("Created forwardIndex. Updating metadata properties for segment={} and column={}", segmentName, column);
+    Map<String, String> metadataProperties = new HashMap<>();
+    metadataProperties.put(getKeyFor(column, HAS_DICTIONARY), String.valueOf(true));
+    metadataProperties.put(getKeyFor(column, DICTIONARY_ELEMENT_SIZE),
+        String.valueOf(dictionaryCreator.getNumBytesPerEntry()));
+    updateMetadataProperties(indexDir, metadataProperties);
+
+    // We remove indexes that have to be rewritten when a dictEnabled is toggled. Note that the respective index
+    // handler will take care of recreating the index.
+    removeDictRelatedIndexes(column, segmentWriter);
+
+    // Delete the marker file.
+    FileUtils.deleteQuietly(inProgress);
+
+    LOGGER.info("Created dictionary based forward index for segment: {}, column: {}", segmentName, column);
+  }
+
+  private SegmentDictionaryCreator buildDictionary(String column, ColumnMetadata existingColMetadata,
+      SegmentDirectory.Writer segmentWriter)
+      throws Exception {
+    int numDocs = existingColMetadata.getTotalDocs();
+    // SegmentPartitionConfig is not relevant for rewrites.
+    StatsCollectorConfig statsCollectorConfig =
+        new StatsCollectorConfig(_indexLoadingConfig.getTableConfig(), _schema, null);
+    AbstractColumnStatisticsCollector statsCollector;
+
+    try (ForwardIndexReader reader = LoaderUtils.getForwardIndexReader(segmentWriter, existingColMetadata)) {
+      boolean isSVColumn = reader.isSingleValue();
+
+      switch (reader.getStoredType()) {
+        case INT:
+          statsCollector = new IntColumnPreIndexStatsCollector(column, statsCollectorConfig);
+          break;
+        case LONG:
+          statsCollector = new LongColumnPreIndexStatsCollector(column, statsCollectorConfig);
+          break;
+        case FLOAT:
+          statsCollector = new FloatColumnPreIndexStatsCollector(column, statsCollectorConfig);
+          break;
+        case DOUBLE:
+          statsCollector = new DoubleColumnPreIndexStatsCollector(column, statsCollectorConfig);
+          break;
+        case STRING:
+          statsCollector = new StringColumnPreIndexStatsCollector(column, statsCollectorConfig);
+          break;
+        case BYTES:
+          statsCollector = new BytesColumnPredIndexStatsCollector(column, statsCollectorConfig);
+          break;
+        case BIG_DECIMAL:
+          Preconditions.checkState(isSVColumn, "BigDecimal is not supported for MV columns");
+          statsCollector = new BigDecimalColumnPreIndexStatsCollector(column, statsCollectorConfig);
+          break;
+        default:
+          throw new IllegalStateException();
+      }
+
+      Preconditions.checkState(statsCollector != null);

Review Comment:
   (nit) no need to check again here I guess . Exception would have already been thrown at line 534



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


[GitHub] [pinot] siddharthteotia commented on a diff in pull request #9678: Support creating dictionary at runtime for an existing column

Posted by GitBox <gi...@apache.org>.
siddharthteotia commented on code in PR #9678:
URL: https://github.com/apache/pinot/pull/9678#discussion_r1011061918


##########
pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/loader/ForwardIndexHandler.java:
##########
@@ -271,118 +304,295 @@ private void rewriteRawSVForwardIndex(String column, ColumnMetadata existingColM
               .forForwardIndex(newCompressionType, _indexLoadingConfig.getColumnProperties());
 
       try (ForwardIndexCreator creator = indexCreatorProvider.newForwardIndexCreator(context)) {
-        // If creator stored type and the reader stored type do not match, throw an exception.
         if (!reader.getStoredType().equals(creator.getValueType())) {
+          // Creator stored type should match reader stored type for raw columns. We do not support changing datatypes.
           String failureMsg =
               "Unsupported operation to change datatype for column=" + column + " from " + reader.getStoredType()
                   .toString() + " to " + creator.getValueType().toString();
           throw new UnsupportedOperationException(failureMsg);
         }
 
         int numDocs = existingColMetadata.getTotalDocs();
-        forwardIndexWriterHelper(column, reader, creator, numDocs);
+        forwardIndexWriterHelper(column, existingColMetadata, reader, creator, numDocs, null);
       }
     }
   }
 
-  private void forwardIndexWriterHelper(String column, ForwardIndexReader reader, ForwardIndexCreator creator,
-      int numDocs) {
-    // If creator stored type should match reader stored type. We do not support changing datatypes.
-    if (!reader.getStoredType().equals(creator.getValueType())) {
-      String failureMsg =
-          "Unsupported operation to change datatype for column=" + column + " from " + reader.getStoredType().toString()
-              + " to " + creator.getValueType().toString();
-      throw new UnsupportedOperationException(failureMsg);
-    }
-
+  private void forwardIndexWriterHelper(String column, ColumnMetadata existingColumnMetadata, ForwardIndexReader reader,
+      ForwardIndexCreator creator, int numDocs, @Nullable SegmentDictionaryCreator dictionaryCreator) {
     ForwardIndexReaderContext readerContext = reader.createContext();
     boolean isSVColumn = reader.isSingleValue();
 
-    switch (reader.getStoredType()) {
-      // JSON fields are either stored as string or bytes. No special handling is needed because we make this
-      // decision based on the storedType of the reader.
-      case INT: {
-        for (int i = 0; i < numDocs; i++) {
-          if (isSVColumn) {
-            int val = reader.getInt(i, readerContext);
-            creator.putInt(val);
-          } else {
-            int[] ints = reader.getIntMV(i, readerContext);
-            creator.putIntMV(ints);
-          }
+    if (dictionaryCreator != null) {
+      int maxNumValuesPerEntry = existingColumnMetadata.getMaxNumberOfMultiValues();
+      PinotSegmentColumnReader columnReader = new PinotSegmentColumnReader(reader, null, null, maxNumValuesPerEntry);
+
+      for (int i = 0; i < numDocs; i++) {
+        Object obj = columnReader.getValue(i);
+
+        if (isSVColumn) {
+          int dictId = dictionaryCreator.indexOfSV(obj);
+          creator.putDictId(dictId);
+        } else {
+          int[] dictIds = dictionaryCreator.indexOfMV(obj);
+          creator.putDictIdMV(dictIds);
         }
-        break;
       }
-      case LONG: {
-        for (int i = 0; i < numDocs; i++) {
-          if (isSVColumn) {
-            long val = reader.getLong(i, readerContext);
-            creator.putLong(val);
-          } else {
-            long[] longs = reader.getLongMV(i, readerContext);
-            creator.putLongMV(longs);
+    } else {
+      switch (reader.getStoredType()) {
+        // JSON fields are either stored as string or bytes. No special handling is needed because we make this
+        // decision based on the storedType of the reader.
+        case INT: {
+          for (int i = 0; i < numDocs; i++) {
+            if (isSVColumn) {
+              int val = reader.getInt(i, readerContext);
+              creator.putInt(val);
+            } else {
+              int[] ints = reader.getIntMV(i, readerContext);
+              creator.putIntMV(ints);
+            }
           }
+          break;
         }
-        break;
-      }
-      case FLOAT: {
-        for (int i = 0; i < numDocs; i++) {
-          if (isSVColumn) {
-            float val = reader.getFloat(i, readerContext);
-            creator.putFloat(val);
-          } else {
-            float[] floats = reader.getFloatMV(i, readerContext);
-            creator.putFloatMV(floats);
+        case LONG: {
+          for (int i = 0; i < numDocs; i++) {
+            if (isSVColumn) {
+              long val = reader.getLong(i, readerContext);
+              creator.putLong(val);
+            } else {
+              long[] longs = reader.getLongMV(i, readerContext);
+              creator.putLongMV(longs);
+            }
           }
+          break;
         }
-        break;
-      }
-      case DOUBLE: {
-        for (int i = 0; i < numDocs; i++) {
-          if (isSVColumn) {
-            double val = reader.getDouble(i, readerContext);
-            creator.putDouble(val);
-          } else {
-            double[] doubles = reader.getDoubleMV(i, readerContext);
-            creator.putDoubleMV(doubles);
+        case FLOAT: {
+          for (int i = 0; i < numDocs; i++) {
+            if (isSVColumn) {
+              float val = reader.getFloat(i, readerContext);
+              creator.putFloat(val);
+            } else {
+              float[] floats = reader.getFloatMV(i, readerContext);
+              creator.putFloatMV(floats);
+            }
           }
+          break;
         }
-        break;
-      }
-      case STRING: {
-        for (int i = 0; i < numDocs; i++) {
-          if (isSVColumn) {
-            String val = reader.getString(i, readerContext);
-            creator.putString(val);
-          } else {
-            String[] strings = reader.getStringMV(i, readerContext);
-            creator.putStringMV(strings);
+        case DOUBLE: {
+          for (int i = 0; i < numDocs; i++) {
+            if (isSVColumn) {
+              double val = reader.getDouble(i, readerContext);
+              creator.putDouble(val);
+            } else {
+              double[] doubles = reader.getDoubleMV(i, readerContext);
+              creator.putDoubleMV(doubles);
+            }
           }
+          break;
         }
-        break;
-      }
-      case BYTES: {
-        for (int i = 0; i < numDocs; i++) {
-          if (isSVColumn) {
-            byte[] val = reader.getBytes(i, readerContext);
-            creator.putBytes(val);
-          } else {
-            byte[][] bytesArray = reader.getBytesMV(i, readerContext);
-            creator.putBytesMV(bytesArray);
+        case STRING: {
+          for (int i = 0; i < numDocs; i++) {
+            if (isSVColumn) {
+              String val = reader.getString(i, readerContext);
+              creator.putString(val);
+            } else {
+              String[] strings = reader.getStringMV(i, readerContext);
+              creator.putStringMV(strings);
+            }
           }
+          break;
         }
-        break;
-      }
-      case BIG_DECIMAL: {
-        for (int i = 0; i < numDocs; i++) {
+        case BYTES: {
+          for (int i = 0; i < numDocs; i++) {
+            if (isSVColumn) {
+              byte[] val = reader.getBytes(i, readerContext);
+              creator.putBytes(val);
+            } else {
+              byte[][] bytesArray = reader.getBytesMV(i, readerContext);
+              creator.putBytesMV(bytesArray);
+            }
+          }
+          break;
+        }
+        case BIG_DECIMAL: {
           Preconditions.checkState(isSVColumn, "BigDecimal is not supported for MV columns");
-          BigDecimal val = reader.getBigDecimal(i, readerContext);
-          creator.putBigDecimal(val);
+          for (int i = 0; i < numDocs; i++) {
+            BigDecimal val = reader.getBigDecimal(i, readerContext);
+            creator.putBigDecimal(val);
+          }
+          break;
         }
-        break;
+        default:
+          throw new IllegalStateException();
+      }
+    }
+  }
+
+  private void enableDictionary(String column, SegmentDirectory.Writer segmentWriter,
+      IndexCreatorProvider indexCreatorProvider)
+      throws Exception {
+    Preconditions.checkState(_segmentMetadata.getVersion() == SegmentVersion.v3);
+    ColumnMetadata existingColMetadata = _segmentMetadata.getColumnMetadataFor(column);
+    Preconditions.checkState(!existingColMetadata.hasDictionary(),
+        "Cannot rewrite dictionary enabled forward index. Dictionary already exists for column:" + column);
+    boolean isSingleValue = existingColMetadata.isSingleValue();
+
+    File indexDir = _segmentMetadata.getIndexDir();
+    String segmentName = _segmentMetadata.getName();
+    File inProgress = new File(indexDir, column + ".dict.inprogress");
+    File dictionaryFile = new File(indexDir, column + V1Constants.Dict.FILE_EXTENSION);
+    String fwdIndexFileExtension;
+    if (isSingleValue) {
+      fwdIndexFileExtension =
+          existingColMetadata.isSorted() ? V1Constants.Indexes.SORTED_SV_FORWARD_INDEX_FILE_EXTENSION
+              : V1Constants.Indexes.UNSORTED_SV_FORWARD_INDEX_FILE_EXTENSION;
+    } else {
+      fwdIndexFileExtension = V1Constants.Indexes.UNSORTED_MV_FORWARD_INDEX_FILE_EXTENSION;
+    }
+    File fwdIndexFile = new File(indexDir, column + fwdIndexFileExtension);
+
+    if (!inProgress.exists()) {
+      // Marker file does not exist, which means last run ended normally.
+      // Create a marker file.
+      FileUtils.touch(inProgress);
+    } else {
+      // Marker file exists, which means last run was interrupted.
+      // Remove forward index and dictionary files if they exist.
+      FileUtils.deleteQuietly(fwdIndexFile);
+      FileUtils.deleteQuietly(dictionaryFile);
+    }
+
+    LOGGER.info("Creating a new dictionary for segment={} and column={}", segmentName, column);
+    SegmentDictionaryCreator dictionaryCreator = buildDictionary(column, existingColMetadata, segmentWriter);
+    LoaderUtils.writeIndexToV3Format(segmentWriter, column, dictionaryFile, ColumnIndexType.DICTIONARY);
+
+    LOGGER.info("Built dictionary. Rewriting dictionary enabled forward index for segment={} and column={}",

Review Comment:
   So if we reach this point and then fail, the on-disk v3 file would have been appended with the dictionary (and metadata will reflect that dictionary exists) but forward index will not have been rewritten to be dictionary based at this point.  
   
   Is this handled as part of the overall failure handling framework during reload / load ?



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


[GitHub] [pinot] siddharthteotia commented on a diff in pull request #9678: Support creating dictionary at runtime for an existing column

Posted by GitBox <gi...@apache.org>.
siddharthteotia commented on code in PR #9678:
URL: https://github.com/apache/pinot/pull/9678#discussion_r1012298557


##########
pinot-segment-local/src/test/java/org/apache/pinot/segment/local/segment/index/loader/ForwardIndexHandlerTest.java:
##########
@@ -469,18 +553,131 @@ public void testRewriteRawForwardIndexForMultipleColumns()
 
     IndexLoadingConfig indexLoadingConfig = new IndexLoadingConfig(null, tableConfig);
     IndexCreatorProvider indexCreatorProvider = IndexingOverrides.getIndexCreatorProvider();
-    ForwardIndexHandler fwdIndexHandler = new ForwardIndexHandler(existingSegmentMetadata, indexLoadingConfig);
+    ForwardIndexHandler fwdIndexHandler = new ForwardIndexHandler(existingSegmentMetadata, indexLoadingConfig, null);
     fwdIndexHandler.updateIndices(writer, indexCreatorProvider);
 
     // Tear down before validation. Because columns.psf and index map cleanup happens at segmentDirectory.close()
     segmentLocalFSDirectory.close();
 
-    validateIndexMap();
+    validateIndexMap(column1, false);
     validateForwardIndex(column1, newCompressionType);
+    // Validate metadata properties. Nothing should change when a forwardIndex is rewritten for compressionType
+    // change.
+    ColumnMetadata metadata = existingSegmentMetadata.getColumnMetadataFor(column1);
+    validateMetadataProperties(column1, metadata.hasDictionary(), metadata.getColumnMaxLength(),
+        metadata.getCardinality(), metadata.getTotalDocs(), metadata.getDataType(), metadata.getFieldType(),
+        metadata.isSorted(), metadata.isSingleValue(), metadata.getMaxNumberOfMultiValues(),
+        metadata.getTotalNumberOfEntries(), metadata.isAutoGenerated(), metadata.getMinValue(), metadata.getMaxValue());
+
+    validateIndexMap(column2, false);
     validateForwardIndex(column2, newCompressionType);
+    metadata = existingSegmentMetadata.getColumnMetadataFor(column2);
+    validateMetadataProperties(column2, metadata.hasDictionary(), metadata.getColumnMaxLength(),
+        metadata.getCardinality(), metadata.getTotalDocs(), metadata.getDataType(), metadata.getFieldType(),
+        metadata.isSorted(), metadata.isSingleValue(), metadata.getMaxNumberOfMultiValues(),
+        metadata.getTotalNumberOfEntries(), metadata.isAutoGenerated(), metadata.getMinValue(), metadata.getMaxValue());
+  }
+
+  @Test(priority = 3)
+  public void testEnableDictionaryForMultipleColumns()
+      throws Exception {
+    SegmentMetadataImpl existingSegmentMetadata = new SegmentMetadataImpl(_segmentDirectory);
+    SegmentDirectory segmentLocalFSDirectory =
+        new SegmentLocalFSDirectory(_segmentDirectory, existingSegmentMetadata, ReadMode.mmap);
+    SegmentDirectory.Writer writer = segmentLocalFSDirectory.createWriter();
+    IndexLoadingConfig indexLoadingConfig = new IndexLoadingConfig(null, _tableConfig);
+
+    Random rand = new Random();
+    String col1 = _noDictionaryColumns.get(rand.nextInt(_noDictionaryColumns.size()));
+    indexLoadingConfig.getNoDictionaryColumns().remove(col1);
+    String col2 = _noDictionaryColumns.get(rand.nextInt(_noDictionaryColumns.size()));
+    indexLoadingConfig.getNoDictionaryColumns().remove(col2);
+
+    ForwardIndexHandler fwdIndexHandler = new ForwardIndexHandler(existingSegmentMetadata, indexLoadingConfig, _schema);
+    IndexCreatorProvider indexCreatorProvider = IndexingOverrides.getIndexCreatorProvider();
+    fwdIndexHandler.updateIndices(writer, indexCreatorProvider);
+
+    // Tear down before validation. Because columns.psf and index map cleanup happens at segmentDirectory.close()
+    segmentLocalFSDirectory.close();
+
+    // Col1 validation.
+    validateIndexMap(col1, true);
+    validateForwardIndex(col1, null);
+    // In column metadata, nothing other than hasDictionary and dictionaryElementSize should change.
+    int dictionaryElementSize = 0;
+    ColumnMetadata metadata = existingSegmentMetadata.getColumnMetadataFor(col1);
+    FieldSpec.DataType dataType = metadata.getDataType();
+    if (dataType == FieldSpec.DataType.STRING || dataType == FieldSpec.DataType.BYTES) {
+      // This value is based on the rows in createTestData().
+      dictionaryElementSize = 7;
+    } else if (dataType == FieldSpec.DataType.BIG_DECIMAL) {
+      dictionaryElementSize = 11;
+    }
+    validateMetadataProperties(col1, true, dictionaryElementSize, metadata.getCardinality(), metadata.getTotalDocs(),
+        dataType, metadata.getFieldType(), metadata.isSorted(), metadata.isSingleValue(),
+        metadata.getMaxNumberOfMultiValues(), metadata.getTotalNumberOfEntries(), metadata.isAutoGenerated(),
+        metadata.getMinValue(), metadata.getMaxValue());
+
+    // Col2 validation.
+    validateIndexMap(col2, true);
+    validateForwardIndex(col2, null);
+    // In column metadata, nothing other than hasDictionary and dictionaryElementSize should change.
+    dictionaryElementSize = 0;
+    metadata = existingSegmentMetadata.getColumnMetadataFor(col2);
+    dataType = metadata.getDataType();
+    if (dataType == FieldSpec.DataType.STRING || dataType == FieldSpec.DataType.BYTES) {
+      // This value is based on the rows in createTestData().
+      dictionaryElementSize = 7;
+    } else if (dataType == FieldSpec.DataType.BIG_DECIMAL) {
+      dictionaryElementSize = 11;
+    }
+    validateMetadataProperties(col2, true, dictionaryElementSize, metadata.getCardinality(), metadata.getTotalDocs(),
+        dataType, metadata.getFieldType(), metadata.isSorted(), metadata.isSingleValue(),
+        metadata.getMaxNumberOfMultiValues(), metadata.getTotalNumberOfEntries(), metadata.isAutoGenerated(),
+        metadata.getMinValue(), metadata.getMaxValue());
+  }
+
+  @Test(priority = 4)
+  public void testEnableDictionaryForSingleColumn()
+      throws Exception {
+    for (int i = 0; i < _noDictionaryColumns.size(); i++) {
+      SegmentMetadataImpl existingSegmentMetadata = new SegmentMetadataImpl(_segmentDirectory);
+      SegmentDirectory segmentLocalFSDirectory =
+          new SegmentLocalFSDirectory(_segmentDirectory, existingSegmentMetadata, ReadMode.mmap);
+      SegmentDirectory.Writer writer = segmentLocalFSDirectory.createWriter();
+
+      IndexLoadingConfig indexLoadingConfig = new IndexLoadingConfig(null, _tableConfig);
+      String column = _noDictionaryColumns.get(i);
+      indexLoadingConfig.getNoDictionaryColumns().remove(column);
+      ForwardIndexHandler fwdIndexHandler =
+          new ForwardIndexHandler(existingSegmentMetadata, indexLoadingConfig, _schema);
+      IndexCreatorProvider indexCreatorProvider = IndexingOverrides.getIndexCreatorProvider();
+      fwdIndexHandler.updateIndices(writer, indexCreatorProvider);
+
+      // Tear down before validation. Because columns.psf and index map cleanup happens at segmentDirectory.close()
+      segmentLocalFSDirectory.close();
+
+      validateIndexMap(column, true);
+      validateForwardIndex(column, null);

Review Comment:
   Can we also check if ColumnIndexEntry exists / dictionary index buffer exists by calling `hasIndex()` ?



##########
pinot-segment-local/src/test/java/org/apache/pinot/segment/local/segment/index/loader/ForwardIndexHandlerTest.java:
##########
@@ -469,18 +553,131 @@ public void testRewriteRawForwardIndexForMultipleColumns()
 
     IndexLoadingConfig indexLoadingConfig = new IndexLoadingConfig(null, tableConfig);
     IndexCreatorProvider indexCreatorProvider = IndexingOverrides.getIndexCreatorProvider();
-    ForwardIndexHandler fwdIndexHandler = new ForwardIndexHandler(existingSegmentMetadata, indexLoadingConfig);
+    ForwardIndexHandler fwdIndexHandler = new ForwardIndexHandler(existingSegmentMetadata, indexLoadingConfig, null);
     fwdIndexHandler.updateIndices(writer, indexCreatorProvider);
 
     // Tear down before validation. Because columns.psf and index map cleanup happens at segmentDirectory.close()
     segmentLocalFSDirectory.close();
 
-    validateIndexMap();
+    validateIndexMap(column1, false);
     validateForwardIndex(column1, newCompressionType);
+    // Validate metadata properties. Nothing should change when a forwardIndex is rewritten for compressionType
+    // change.
+    ColumnMetadata metadata = existingSegmentMetadata.getColumnMetadataFor(column1);
+    validateMetadataProperties(column1, metadata.hasDictionary(), metadata.getColumnMaxLength(),
+        metadata.getCardinality(), metadata.getTotalDocs(), metadata.getDataType(), metadata.getFieldType(),
+        metadata.isSorted(), metadata.isSingleValue(), metadata.getMaxNumberOfMultiValues(),
+        metadata.getTotalNumberOfEntries(), metadata.isAutoGenerated(), metadata.getMinValue(), metadata.getMaxValue());
+
+    validateIndexMap(column2, false);
     validateForwardIndex(column2, newCompressionType);
+    metadata = existingSegmentMetadata.getColumnMetadataFor(column2);
+    validateMetadataProperties(column2, metadata.hasDictionary(), metadata.getColumnMaxLength(),
+        metadata.getCardinality(), metadata.getTotalDocs(), metadata.getDataType(), metadata.getFieldType(),
+        metadata.isSorted(), metadata.isSingleValue(), metadata.getMaxNumberOfMultiValues(),
+        metadata.getTotalNumberOfEntries(), metadata.isAutoGenerated(), metadata.getMinValue(), metadata.getMaxValue());
+  }
+
+  @Test(priority = 3)
+  public void testEnableDictionaryForMultipleColumns()
+      throws Exception {
+    SegmentMetadataImpl existingSegmentMetadata = new SegmentMetadataImpl(_segmentDirectory);
+    SegmentDirectory segmentLocalFSDirectory =
+        new SegmentLocalFSDirectory(_segmentDirectory, existingSegmentMetadata, ReadMode.mmap);
+    SegmentDirectory.Writer writer = segmentLocalFSDirectory.createWriter();
+    IndexLoadingConfig indexLoadingConfig = new IndexLoadingConfig(null, _tableConfig);
+
+    Random rand = new Random();
+    String col1 = _noDictionaryColumns.get(rand.nextInt(_noDictionaryColumns.size()));
+    indexLoadingConfig.getNoDictionaryColumns().remove(col1);
+    String col2 = _noDictionaryColumns.get(rand.nextInt(_noDictionaryColumns.size()));
+    indexLoadingConfig.getNoDictionaryColumns().remove(col2);
+
+    ForwardIndexHandler fwdIndexHandler = new ForwardIndexHandler(existingSegmentMetadata, indexLoadingConfig, _schema);
+    IndexCreatorProvider indexCreatorProvider = IndexingOverrides.getIndexCreatorProvider();
+    fwdIndexHandler.updateIndices(writer, indexCreatorProvider);
+
+    // Tear down before validation. Because columns.psf and index map cleanup happens at segmentDirectory.close()
+    segmentLocalFSDirectory.close();
+
+    // Col1 validation.
+    validateIndexMap(col1, true);
+    validateForwardIndex(col1, null);
+    // In column metadata, nothing other than hasDictionary and dictionaryElementSize should change.
+    int dictionaryElementSize = 0;
+    ColumnMetadata metadata = existingSegmentMetadata.getColumnMetadataFor(col1);
+    FieldSpec.DataType dataType = metadata.getDataType();
+    if (dataType == FieldSpec.DataType.STRING || dataType == FieldSpec.DataType.BYTES) {
+      // This value is based on the rows in createTestData().
+      dictionaryElementSize = 7;
+    } else if (dataType == FieldSpec.DataType.BIG_DECIMAL) {
+      dictionaryElementSize = 11;
+    }
+    validateMetadataProperties(col1, true, dictionaryElementSize, metadata.getCardinality(), metadata.getTotalDocs(),
+        dataType, metadata.getFieldType(), metadata.isSorted(), metadata.isSingleValue(),
+        metadata.getMaxNumberOfMultiValues(), metadata.getTotalNumberOfEntries(), metadata.isAutoGenerated(),
+        metadata.getMinValue(), metadata.getMaxValue());
+
+    // Col2 validation.
+    validateIndexMap(col2, true);
+    validateForwardIndex(col2, null);
+    // In column metadata, nothing other than hasDictionary and dictionaryElementSize should change.
+    dictionaryElementSize = 0;
+    metadata = existingSegmentMetadata.getColumnMetadataFor(col2);
+    dataType = metadata.getDataType();
+    if (dataType == FieldSpec.DataType.STRING || dataType == FieldSpec.DataType.BYTES) {
+      // This value is based on the rows in createTestData().
+      dictionaryElementSize = 7;
+    } else if (dataType == FieldSpec.DataType.BIG_DECIMAL) {
+      dictionaryElementSize = 11;
+    }
+    validateMetadataProperties(col2, true, dictionaryElementSize, metadata.getCardinality(), metadata.getTotalDocs(),
+        dataType, metadata.getFieldType(), metadata.isSorted(), metadata.isSingleValue(),
+        metadata.getMaxNumberOfMultiValues(), metadata.getTotalNumberOfEntries(), metadata.isAutoGenerated(),
+        metadata.getMinValue(), metadata.getMaxValue());
+  }
+
+  @Test(priority = 4)
+  public void testEnableDictionaryForSingleColumn()
+      throws Exception {
+    for (int i = 0; i < _noDictionaryColumns.size(); i++) {
+      SegmentMetadataImpl existingSegmentMetadata = new SegmentMetadataImpl(_segmentDirectory);
+      SegmentDirectory segmentLocalFSDirectory =
+          new SegmentLocalFSDirectory(_segmentDirectory, existingSegmentMetadata, ReadMode.mmap);
+      SegmentDirectory.Writer writer = segmentLocalFSDirectory.createWriter();
+
+      IndexLoadingConfig indexLoadingConfig = new IndexLoadingConfig(null, _tableConfig);
+      String column = _noDictionaryColumns.get(i);
+      indexLoadingConfig.getNoDictionaryColumns().remove(column);
+      ForwardIndexHandler fwdIndexHandler =
+          new ForwardIndexHandler(existingSegmentMetadata, indexLoadingConfig, _schema);
+      IndexCreatorProvider indexCreatorProvider = IndexingOverrides.getIndexCreatorProvider();
+      fwdIndexHandler.updateIndices(writer, indexCreatorProvider);
+
+      // Tear down before validation. Because columns.psf and index map cleanup happens at segmentDirectory.close()
+      segmentLocalFSDirectory.close();
+
+      validateIndexMap(column, true);
+      validateForwardIndex(column, null);

Review Comment:
   Can we also check if `ColumnIndexEntry` exists / dictionary index buffer exists by calling `hasIndex()` ?



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


[GitHub] [pinot] vvivekiyer commented on a diff in pull request #9678: Support creating dictionary at runtime for an existing column

Posted by GitBox <gi...@apache.org>.
vvivekiyer commented on code in PR #9678:
URL: https://github.com/apache/pinot/pull/9678#discussion_r1011006727


##########
pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/loader/ForwardIndexHandler.java:
##########
@@ -244,16 +279,16 @@ private void rewriteRawMVForwardIndex(String column, ColumnMetadata existingColM
               .forForwardIndex(newCompressionType, _indexLoadingConfig.getColumnProperties());
 
       try (ForwardIndexCreator creator = indexCreatorProvider.newForwardIndexCreator(context)) {
-        // If creator stored type and the reader stored type do not match, throw an exception.
         if (!reader.getStoredType().equals(creator.getValueType())) {
+          // Creator stored type should match reader stored type for raw columns. We do not support changing datatypes.
           String failureMsg =
               "Unsupported operation to change datatype for column=" + column + " from " + reader.getStoredType()
                   .toString() + " to " + creator.getValueType().toString();
           throw new UnsupportedOperationException(failureMsg);
         }

Review Comment:
   For dict enabled columns, this validation is not right. 
   Ofcourse, I could check for dictCreator == null before performing the validation. But decided to extract it out as it would be more cleaner. 



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


[GitHub] [pinot] siddharthteotia commented on a diff in pull request #9678: Support creating dictionary at runtime for an existing column

Posted by GitBox <gi...@apache.org>.
siddharthteotia commented on code in PR #9678:
URL: https://github.com/apache/pinot/pull/9678#discussion_r1011078687


##########
pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/loader/ForwardIndexHandler.java:
##########
@@ -92,10 +112,14 @@ public void updateIndices(SegmentDirectory.Writer segmentWriter, IndexCreatorPro
       Operation operation = entry.getValue();
 
       switch (operation) {
-        case CHANGE_RAW_INDEX_COMPRESSION_TYPE:
+        case CHANGE_RAW_INDEX_COMPRESSION_TYPE: {
           rewriteRawForwardIndex(column, segmentWriter, indexCreatorProvider);
           break;
-        // TODO: Add other operations here.
+        }
+        case ENABLE_DICTIONARY: {
+          enableDictionary(column, segmentWriter, indexCreatorProvider);

Review Comment:
   (nit) `createDictBasedForwardIndex` or `rewriteDictBasedForwardIndex` ?



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


[GitHub] [pinot] somandal commented on a diff in pull request #9678: Support creating dictionary at runtime for an existing column

Posted by GitBox <gi...@apache.org>.
somandal commented on code in PR #9678:
URL: https://github.com/apache/pinot/pull/9678#discussion_r1012142411


##########
pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/loader/ForwardIndexHandler.java:
##########
@@ -271,118 +306,293 @@ private void rewriteRawSVForwardIndex(String column, ColumnMetadata existingColM
               .forForwardIndex(newCompressionType, _indexLoadingConfig.getColumnProperties());
 
       try (ForwardIndexCreator creator = indexCreatorProvider.newForwardIndexCreator(context)) {
-        // If creator stored type and the reader stored type do not match, throw an exception.
         if (!reader.getStoredType().equals(creator.getValueType())) {
+          // Creator stored type should match reader stored type for raw columns. We do not support changing datatypes.
           String failureMsg =
               "Unsupported operation to change datatype for column=" + column + " from " + reader.getStoredType()
                   .toString() + " to " + creator.getValueType().toString();
           throw new UnsupportedOperationException(failureMsg);
         }
 
         int numDocs = existingColMetadata.getTotalDocs();
-        forwardIndexWriterHelper(column, reader, creator, numDocs);
+        forwardIndexWriterHelper(column, existingColMetadata, reader, creator, numDocs, null);
       }
     }
   }
 
-  private void forwardIndexWriterHelper(String column, ForwardIndexReader reader, ForwardIndexCreator creator,
-      int numDocs) {
-    // If creator stored type should match reader stored type. We do not support changing datatypes.
-    if (!reader.getStoredType().equals(creator.getValueType())) {
-      String failureMsg =
-          "Unsupported operation to change datatype for column=" + column + " from " + reader.getStoredType().toString()
-              + " to " + creator.getValueType().toString();
-      throw new UnsupportedOperationException(failureMsg);
-    }
-
+  private void forwardIndexWriterHelper(String column, ColumnMetadata existingColumnMetadata, ForwardIndexReader reader,
+      ForwardIndexCreator creator, int numDocs, @Nullable SegmentDictionaryCreator dictionaryCreator) {
     ForwardIndexReaderContext readerContext = reader.createContext();
     boolean isSVColumn = reader.isSingleValue();
 
-    switch (reader.getStoredType()) {
-      // JSON fields are either stored as string or bytes. No special handling is needed because we make this
-      // decision based on the storedType of the reader.
-      case INT: {
-        for (int i = 0; i < numDocs; i++) {
-          if (isSVColumn) {
-            int val = reader.getInt(i, readerContext);
-            creator.putInt(val);
-          } else {
-            int[] ints = reader.getIntMV(i, readerContext);
-            creator.putIntMV(ints);
-          }
+    if (dictionaryCreator != null) {
+      int maxNumValuesPerEntry = existingColumnMetadata.getMaxNumberOfMultiValues();
+      PinotSegmentColumnReader columnReader = new PinotSegmentColumnReader(reader, null, null, maxNumValuesPerEntry);
+
+      for (int i = 0; i < numDocs; i++) {
+        Object obj = columnReader.getValue(i);
+
+        if (isSVColumn) {
+          int dictId = dictionaryCreator.indexOfSV(obj);
+          creator.putDictId(dictId);
+        } else {
+          int[] dictIds = dictionaryCreator.indexOfMV(obj);
+          creator.putDictIdMV(dictIds);
         }
-        break;
       }
-      case LONG: {
-        for (int i = 0; i < numDocs; i++) {
-          if (isSVColumn) {
-            long val = reader.getLong(i, readerContext);
-            creator.putLong(val);
-          } else {
-            long[] longs = reader.getLongMV(i, readerContext);
-            creator.putLongMV(longs);
+    } else {
+      switch (reader.getStoredType()) {
+        // JSON fields are either stored as string or bytes. No special handling is needed because we make this
+        // decision based on the storedType of the reader.
+        case INT: {
+          for (int i = 0; i < numDocs; i++) {
+            if (isSVColumn) {
+              int val = reader.getInt(i, readerContext);
+              creator.putInt(val);
+            } else {
+              int[] ints = reader.getIntMV(i, readerContext);
+              creator.putIntMV(ints);
+            }
           }
+          break;
         }
-        break;
-      }
-      case FLOAT: {
-        for (int i = 0; i < numDocs; i++) {
-          if (isSVColumn) {
-            float val = reader.getFloat(i, readerContext);
-            creator.putFloat(val);
-          } else {
-            float[] floats = reader.getFloatMV(i, readerContext);
-            creator.putFloatMV(floats);
+        case LONG: {
+          for (int i = 0; i < numDocs; i++) {
+            if (isSVColumn) {
+              long val = reader.getLong(i, readerContext);
+              creator.putLong(val);
+            } else {
+              long[] longs = reader.getLongMV(i, readerContext);
+              creator.putLongMV(longs);
+            }
           }
+          break;
         }
-        break;
-      }
-      case DOUBLE: {
-        for (int i = 0; i < numDocs; i++) {
-          if (isSVColumn) {
-            double val = reader.getDouble(i, readerContext);
-            creator.putDouble(val);
-          } else {
-            double[] doubles = reader.getDoubleMV(i, readerContext);
-            creator.putDoubleMV(doubles);
+        case FLOAT: {
+          for (int i = 0; i < numDocs; i++) {
+            if (isSVColumn) {
+              float val = reader.getFloat(i, readerContext);
+              creator.putFloat(val);
+            } else {
+              float[] floats = reader.getFloatMV(i, readerContext);
+              creator.putFloatMV(floats);
+            }
           }
+          break;
         }
-        break;
-      }
-      case STRING: {
-        for (int i = 0; i < numDocs; i++) {
-          if (isSVColumn) {
-            String val = reader.getString(i, readerContext);
-            creator.putString(val);
-          } else {
-            String[] strings = reader.getStringMV(i, readerContext);
-            creator.putStringMV(strings);
+        case DOUBLE: {
+          for (int i = 0; i < numDocs; i++) {
+            if (isSVColumn) {
+              double val = reader.getDouble(i, readerContext);
+              creator.putDouble(val);
+            } else {
+              double[] doubles = reader.getDoubleMV(i, readerContext);
+              creator.putDoubleMV(doubles);
+            }
           }
+          break;
         }
-        break;
-      }
-      case BYTES: {
-        for (int i = 0; i < numDocs; i++) {
-          if (isSVColumn) {
-            byte[] val = reader.getBytes(i, readerContext);
-            creator.putBytes(val);
-          } else {
-            byte[][] bytesArray = reader.getBytesMV(i, readerContext);
-            creator.putBytesMV(bytesArray);
+        case STRING: {
+          for (int i = 0; i < numDocs; i++) {
+            if (isSVColumn) {
+              String val = reader.getString(i, readerContext);
+              creator.putString(val);
+            } else {
+              String[] strings = reader.getStringMV(i, readerContext);
+              creator.putStringMV(strings);
+            }
           }
+          break;
         }
-        break;
-      }
-      case BIG_DECIMAL: {
-        for (int i = 0; i < numDocs; i++) {
+        case BYTES: {
+          for (int i = 0; i < numDocs; i++) {
+            if (isSVColumn) {
+              byte[] val = reader.getBytes(i, readerContext);
+              creator.putBytes(val);
+            } else {
+              byte[][] bytesArray = reader.getBytesMV(i, readerContext);
+              creator.putBytesMV(bytesArray);
+            }
+          }
+          break;
+        }
+        case BIG_DECIMAL: {
           Preconditions.checkState(isSVColumn, "BigDecimal is not supported for MV columns");
-          BigDecimal val = reader.getBigDecimal(i, readerContext);
-          creator.putBigDecimal(val);
+          for (int i = 0; i < numDocs; i++) {
+            BigDecimal val = reader.getBigDecimal(i, readerContext);
+            creator.putBigDecimal(val);
+          }
+          break;
         }
-        break;
+        default:
+          throw new IllegalStateException();
+      }
+    }
+  }
+
+  private void enableDictionary(String column, SegmentDirectory.Writer segmentWriter,
+      IndexCreatorProvider indexCreatorProvider)
+      throws Exception {
+    Preconditions.checkState(_segmentMetadata.getVersion() == SegmentVersion.v3);
+    ColumnMetadata existingColMetadata = _segmentMetadata.getColumnMetadataFor(column);
+    Preconditions.checkState(!existingColMetadata.hasDictionary(), "Existing column already has dictionary.");
+    boolean isSingleValue = existingColMetadata.isSingleValue();
+
+    File indexDir = _segmentMetadata.getIndexDir();
+    String segmentName = _segmentMetadata.getName();
+    File inProgress = new File(indexDir, column + ".dict.inprogress");
+    File dictionaryFile = new File(indexDir, column + V1Constants.Dict.FILE_EXTENSION);
+    String fwdIndexFileExtension;
+    if (isSingleValue) {
+      if (existingColMetadata.isSorted()) {
+        fwdIndexFileExtension = V1Constants.Indexes.SORTED_SV_FORWARD_INDEX_FILE_EXTENSION;
+      } else {
+        fwdIndexFileExtension = V1Constants.Indexes.UNSORTED_SV_FORWARD_INDEX_FILE_EXTENSION;
+      }
+    } else {
+      fwdIndexFileExtension = V1Constants.Indexes.UNSORTED_MV_FORWARD_INDEX_FILE_EXTENSION;
+    }
+    File fwdIndexFile = new File(indexDir, column + fwdIndexFileExtension);
+
+    if (!inProgress.exists()) {
+      // Marker file does not exist, which means last run ended normally.
+      // Create a marker file.
+      FileUtils.touch(inProgress);
+    } else {
+      // Marker file exists, which means last run was interrupted.
+      // Remove forward index and dictionary files if they exist.
+      FileUtils.deleteQuietly(fwdIndexFile);
+      FileUtils.deleteQuietly(dictionaryFile);
+    }
+
+    LOGGER.info("Creating a new dictionary for segment={} and column={}", segmentName, column);
+    SegmentDictionaryCreator dictionaryCreator = buildDictionary(column, existingColMetadata, segmentWriter);
+    LoaderUtils.writeIndexToV3Format(segmentWriter, column, dictionaryFile, ColumnIndexType.DICTIONARY);
+
+    LOGGER.info("Built dictionary. Rewriting dictionary enabled forward index for segment={} and column={}",
+        segmentName, column);
+    writeDictEnabledForwardIndex(column, existingColMetadata, segmentWriter, indexDir, indexCreatorProvider,
+        dictionaryCreator);
+    // We used the existing forward index to generate a new forward index. The existing forward index will be in V3
+    // format and the new forward index will be in V1 format. Remove the existing forward index as it is not needed
+    // anymore. Note that removeIndex() will only mark an index for removal and remove the in-memory state. The
+    // actual cleanup from columns.psf file will happen when singleFileIndexDirectory.cleanupRemovedIndices() is
+    // called during segmentWriter.close().
+    segmentWriter.removeIndex(column, ColumnIndexType.FORWARD_INDEX);
+    LoaderUtils.writeIndexToV3Format(segmentWriter, column, fwdIndexFile, ColumnIndexType.FORWARD_INDEX);
+
+    LOGGER.info("Created forwardIndex. Updating metadata properties for segment={} and column={}", segmentName, column);
+    updateMetadataProperties(indexDir, column, dictionaryCreator);
+
+    // We remove indexes that have to be rewritten when a dictEnabled is toggled. Note that the respective index
+    // handler will take care of recreating the index.
+    removeDictRelatedIndexes(column, segmentWriter);
+
+    // Delete the marker file.
+    FileUtils.deleteQuietly(inProgress);
+
+    LOGGER.info("Created dictionary based forward index for segment: {}, column: {}", segmentName, column);
+  }
+
+  private SegmentDictionaryCreator buildDictionary(String column, ColumnMetadata existingColMetadata,
+      SegmentDirectory.Writer segmentWriter)
+      throws Exception {
+    int numDocs = existingColMetadata.getTotalDocs();
+    // SegmentPartitionConfig is not relevant for rewrites.
+    Preconditions.checkState(_indexLoadingConfig.getTableConfig() != null);
+    StatsCollectorConfig statsCollectorConfig =
+        new StatsCollectorConfig(_indexLoadingConfig.getTableConfig(), _schema, null);
+    AbstractColumnStatisticsCollector statsCollector;
+
+    try (ForwardIndexReader reader = LoaderUtils.getForwardIndexReader(segmentWriter, existingColMetadata)) {
+      boolean isSVColumn = reader.isSingleValue();
+
+      switch (reader.getStoredType()) {
+        case INT:
+          statsCollector = new IntColumnPreIndexStatsCollector(column, statsCollectorConfig);
+          break;
+        case LONG:
+          statsCollector = new LongColumnPreIndexStatsCollector(column, statsCollectorConfig);
+          break;
+        case FLOAT:
+          statsCollector = new FloatColumnPreIndexStatsCollector(column, statsCollectorConfig);
+          break;
+        case DOUBLE:
+          statsCollector = new DoubleColumnPreIndexStatsCollector(column, statsCollectorConfig);
+          break;
+        case STRING:
+          statsCollector = new StringColumnPreIndexStatsCollector(column, statsCollectorConfig);
+          break;
+        case BYTES:
+          statsCollector = new BytesColumnPredIndexStatsCollector(column, statsCollectorConfig);
+          break;
+        case BIG_DECIMAL:
+          Preconditions.checkState(isSVColumn, "BigDecimal is not supported for MV columns");
+          statsCollector = new BigDecimalColumnPreIndexStatsCollector(column, statsCollectorConfig);
+          break;
+        default:
+          throw new IllegalStateException();
+      }
+
+      Preconditions.checkState(statsCollector != null);
+      // Note: Special Null handling is not necessary here. This is because, the existing default null value in the
+      // raw forwardIndex will be retained as such while created the dictionary and dict-based forward index. Also,
+      // null value vectors maintain a bitmap of docIds. No handling is necessary there.
+      PinotSegmentColumnReader columnReader =
+          new PinotSegmentColumnReader(reader, null, null, existingColMetadata.getMaxNumberOfMultiValues());
+      for (int i = 0; i < numDocs; i++) {
+        Object obj = columnReader.getValue(i);
+        statsCollector.collect(obj);
+      }
+      statsCollector.seal();
+
+      boolean useVarLength = _indexLoadingConfig.getVarLengthDictionaryColumns().contains(column)
+          || SegmentIndexCreationDriverImpl.shouldUseVarLengthDictionary(reader.getStoredType(), statsCollector);
+      SegmentDictionaryCreator dictionaryCreator =
+          new SegmentDictionaryCreator(existingColMetadata.getFieldSpec(), _segmentMetadata.getIndexDir(),
+              useVarLength);
+
+      dictionaryCreator.build(statsCollector.getUniqueValuesSet());
+      return dictionaryCreator;
+    }
+  }
+
+  private void writeDictEnabledForwardIndex(String column, ColumnMetadata existingColMetadata,
+      SegmentDirectory.Writer segmentWriter, File indexDir, IndexCreatorProvider indexCreatorProvider,
+      SegmentDictionaryCreator dictionaryCreator)
+      throws Exception {
+    try (ForwardIndexReader reader = LoaderUtils.getForwardIndexReader(segmentWriter, existingColMetadata)) {
+      int lengthOfLongestEntry = reader.getLengthOfLongestEntry();
+      IndexCreationContext.Builder builder =
+          IndexCreationContext.builder().withIndexDir(indexDir).withColumnMetadata(existingColMetadata)
+              .withLengthOfLongestEntry(lengthOfLongestEntry);
+      // existingColMetadata has dictEnable=false. Overwrite the value.
+      builder.withDictionary(true);
+      IndexCreationContext.Forward context =
+          builder.build().forForwardIndex(null, _indexLoadingConfig.getColumnProperties());
+
+      try (ForwardIndexCreator creator = indexCreatorProvider.newForwardIndexCreator(context)) {
+        int numDocs = existingColMetadata.getTotalDocs();
+        forwardIndexWriterHelper(column, existingColMetadata, reader, creator, numDocs, dictionaryCreator);
       }
-      default:
-        throw new IllegalStateException();
     }
   }
+
+  private void removeDictRelatedIndexes(String column, SegmentDirectory.Writer segmentWriter) {
+    // TODO: Move this logic as a static function in each index creator.
+    segmentWriter.removeIndex(column, ColumnIndexType.RANGE_INDEX);
+  }
+
+  private void updateMetadataProperties(File indexDir, String column, SegmentDictionaryCreator dictionaryCreator)

Review Comment:
   Yes this looks good to me, thanks for addressing this!



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


[GitHub] [pinot] siddharthteotia commented on a diff in pull request #9678: Support creating dictionary at runtime for an existing column

Posted by GitBox <gi...@apache.org>.
siddharthteotia commented on code in PR #9678:
URL: https://github.com/apache/pinot/pull/9678#discussion_r1012268092


##########
pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/loader/SegmentPreProcessor.java:
##########
@@ -104,8 +104,15 @@ public void process()
       // Update single-column indices, like inverted index, json index etc.
       IndexCreatorProvider indexCreatorProvider = IndexingOverrides.getIndexCreatorProvider();
       for (ColumnIndexType type : ColumnIndexType.values()) {
-        IndexHandler handler = IndexHandlerFactory.getIndexHandler(type, _segmentMetadata, _indexLoadingConfig);
+        IndexHandler handler =
+            IndexHandlerFactory.getIndexHandler(type, _segmentMetadata, _indexLoadingConfig, _schema);
         handler.updateIndices(segmentWriter, indexCreatorProvider);
+        if (type == ColumnIndexType.FORWARD_INDEX) {

Review Comment:
   Yes TODO is fine. 



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


[GitHub] [pinot] siddharthteotia commented on a diff in pull request #9678: Support creating dictionary at runtime for an existing column

Posted by GitBox <gi...@apache.org>.
siddharthteotia commented on code in PR #9678:
URL: https://github.com/apache/pinot/pull/9678#discussion_r1012329439


##########
pinot-core/src/test/java/org/apache/pinot/queries/RangeQueriesTest.java:
##########
@@ -232,4 +268,38 @@ public void testCountOverRangeFilter(String query, int expectedCount) {
     assertEquals(aggregationResult.size(), 1);
     assertEquals(((Number) aggregationResult.get(0)).intValue(), expectedCount, query);
   }
+
+  @Test(dataProvider = "countTestCases")
+  public void testCountOverRangeFilterAfterReload(String query, int expectedCount)

Review Comment:
   - SELECT <particularColumn> FROM foo LIMIT <everything> -- this will test out that dict is correct and fwd index got rewritten
   - Try the same in GROUP BY as GROUP BY uses dict. 
   - Try MIN and MAX aggregations -- they can be answered from dict. 
   - WHERE clause like mentioned in another commend



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


[GitHub] [pinot] siddharthteotia commented on a diff in pull request #9678: Support creating dictionary at runtime for an existing column

Posted by GitBox <gi...@apache.org>.
siddharthteotia commented on code in PR #9678:
URL: https://github.com/apache/pinot/pull/9678#discussion_r1012328273


##########
pinot-core/src/test/java/org/apache/pinot/queries/RangeQueriesTest.java:
##########
@@ -232,4 +268,38 @@ public void testCountOverRangeFilter(String query, int expectedCount) {
     assertEquals(aggregationResult.size(), 1);
     assertEquals(((Number) aggregationResult.get(0)).intValue(), expectedCount, query);
   }
+
+  @Test(dataProvider = "countTestCases")
+  public void testCountOverRangeFilterAfterReload(String query, int expectedCount)

Review Comment:
   Best for us to control the query. Not really sure if the query being sent here by the provider is indeed using the concerned column in parts where we want



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


[GitHub] [pinot] vvivekiyer commented on a diff in pull request #9678: Support creating dictionary at runtime for an existing column

Posted by GitBox <gi...@apache.org>.
vvivekiyer commented on code in PR #9678:
URL: https://github.com/apache/pinot/pull/9678#discussion_r1012165235


##########
pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/loader/ForwardIndexHandler.java:
##########
@@ -271,118 +304,295 @@ private void rewriteRawSVForwardIndex(String column, ColumnMetadata existingColM
               .forForwardIndex(newCompressionType, _indexLoadingConfig.getColumnProperties());
 
       try (ForwardIndexCreator creator = indexCreatorProvider.newForwardIndexCreator(context)) {
-        // If creator stored type and the reader stored type do not match, throw an exception.
         if (!reader.getStoredType().equals(creator.getValueType())) {
+          // Creator stored type should match reader stored type for raw columns. We do not support changing datatypes.
           String failureMsg =
               "Unsupported operation to change datatype for column=" + column + " from " + reader.getStoredType()
                   .toString() + " to " + creator.getValueType().toString();
           throw new UnsupportedOperationException(failureMsg);
         }
 
         int numDocs = existingColMetadata.getTotalDocs();
-        forwardIndexWriterHelper(column, reader, creator, numDocs);
+        forwardIndexWriterHelper(column, existingColMetadata, reader, creator, numDocs, null);
       }
     }
   }
 
-  private void forwardIndexWriterHelper(String column, ForwardIndexReader reader, ForwardIndexCreator creator,
-      int numDocs) {
-    // If creator stored type should match reader stored type. We do not support changing datatypes.
-    if (!reader.getStoredType().equals(creator.getValueType())) {
-      String failureMsg =
-          "Unsupported operation to change datatype for column=" + column + " from " + reader.getStoredType().toString()
-              + " to " + creator.getValueType().toString();
-      throw new UnsupportedOperationException(failureMsg);
-    }
-
+  private void forwardIndexWriterHelper(String column, ColumnMetadata existingColumnMetadata, ForwardIndexReader reader,
+      ForwardIndexCreator creator, int numDocs, @Nullable SegmentDictionaryCreator dictionaryCreator) {
     ForwardIndexReaderContext readerContext = reader.createContext();
     boolean isSVColumn = reader.isSingleValue();
 
-    switch (reader.getStoredType()) {
-      // JSON fields are either stored as string or bytes. No special handling is needed because we make this
-      // decision based on the storedType of the reader.
-      case INT: {
-        for (int i = 0; i < numDocs; i++) {
-          if (isSVColumn) {
-            int val = reader.getInt(i, readerContext);
-            creator.putInt(val);
-          } else {
-            int[] ints = reader.getIntMV(i, readerContext);
-            creator.putIntMV(ints);
-          }
+    if (dictionaryCreator != null) {
+      int maxNumValuesPerEntry = existingColumnMetadata.getMaxNumberOfMultiValues();
+      PinotSegmentColumnReader columnReader = new PinotSegmentColumnReader(reader, null, null, maxNumValuesPerEntry);
+
+      for (int i = 0; i < numDocs; i++) {
+        Object obj = columnReader.getValue(i);
+
+        if (isSVColumn) {
+          int dictId = dictionaryCreator.indexOfSV(obj);
+          creator.putDictId(dictId);
+        } else {
+          int[] dictIds = dictionaryCreator.indexOfMV(obj);
+          creator.putDictIdMV(dictIds);
         }
-        break;
       }
-      case LONG: {
-        for (int i = 0; i < numDocs; i++) {
-          if (isSVColumn) {
-            long val = reader.getLong(i, readerContext);
-            creator.putLong(val);
-          } else {
-            long[] longs = reader.getLongMV(i, readerContext);
-            creator.putLongMV(longs);
+    } else {
+      switch (reader.getStoredType()) {
+        // JSON fields are either stored as string or bytes. No special handling is needed because we make this
+        // decision based on the storedType of the reader.
+        case INT: {
+          for (int i = 0; i < numDocs; i++) {
+            if (isSVColumn) {
+              int val = reader.getInt(i, readerContext);
+              creator.putInt(val);
+            } else {
+              int[] ints = reader.getIntMV(i, readerContext);
+              creator.putIntMV(ints);
+            }
           }
+          break;
         }
-        break;
-      }
-      case FLOAT: {
-        for (int i = 0; i < numDocs; i++) {
-          if (isSVColumn) {
-            float val = reader.getFloat(i, readerContext);
-            creator.putFloat(val);
-          } else {
-            float[] floats = reader.getFloatMV(i, readerContext);
-            creator.putFloatMV(floats);
+        case LONG: {
+          for (int i = 0; i < numDocs; i++) {
+            if (isSVColumn) {
+              long val = reader.getLong(i, readerContext);
+              creator.putLong(val);
+            } else {
+              long[] longs = reader.getLongMV(i, readerContext);
+              creator.putLongMV(longs);
+            }
           }
+          break;
         }
-        break;
-      }
-      case DOUBLE: {
-        for (int i = 0; i < numDocs; i++) {
-          if (isSVColumn) {
-            double val = reader.getDouble(i, readerContext);
-            creator.putDouble(val);
-          } else {
-            double[] doubles = reader.getDoubleMV(i, readerContext);
-            creator.putDoubleMV(doubles);
+        case FLOAT: {
+          for (int i = 0; i < numDocs; i++) {
+            if (isSVColumn) {
+              float val = reader.getFloat(i, readerContext);
+              creator.putFloat(val);
+            } else {
+              float[] floats = reader.getFloatMV(i, readerContext);
+              creator.putFloatMV(floats);
+            }
           }
+          break;
         }
-        break;
-      }
-      case STRING: {
-        for (int i = 0; i < numDocs; i++) {
-          if (isSVColumn) {
-            String val = reader.getString(i, readerContext);
-            creator.putString(val);
-          } else {
-            String[] strings = reader.getStringMV(i, readerContext);
-            creator.putStringMV(strings);
+        case DOUBLE: {
+          for (int i = 0; i < numDocs; i++) {
+            if (isSVColumn) {
+              double val = reader.getDouble(i, readerContext);
+              creator.putDouble(val);
+            } else {
+              double[] doubles = reader.getDoubleMV(i, readerContext);
+              creator.putDoubleMV(doubles);
+            }
           }
+          break;
         }
-        break;
-      }
-      case BYTES: {
-        for (int i = 0; i < numDocs; i++) {
-          if (isSVColumn) {
-            byte[] val = reader.getBytes(i, readerContext);
-            creator.putBytes(val);
-          } else {
-            byte[][] bytesArray = reader.getBytesMV(i, readerContext);
-            creator.putBytesMV(bytesArray);
+        case STRING: {
+          for (int i = 0; i < numDocs; i++) {
+            if (isSVColumn) {
+              String val = reader.getString(i, readerContext);
+              creator.putString(val);
+            } else {
+              String[] strings = reader.getStringMV(i, readerContext);
+              creator.putStringMV(strings);
+            }
           }
+          break;
         }
-        break;
-      }
-      case BIG_DECIMAL: {
-        for (int i = 0; i < numDocs; i++) {
+        case BYTES: {
+          for (int i = 0; i < numDocs; i++) {
+            if (isSVColumn) {
+              byte[] val = reader.getBytes(i, readerContext);
+              creator.putBytes(val);
+            } else {
+              byte[][] bytesArray = reader.getBytesMV(i, readerContext);
+              creator.putBytesMV(bytesArray);
+            }
+          }
+          break;
+        }
+        case BIG_DECIMAL: {
           Preconditions.checkState(isSVColumn, "BigDecimal is not supported for MV columns");
-          BigDecimal val = reader.getBigDecimal(i, readerContext);
-          creator.putBigDecimal(val);
+          for (int i = 0; i < numDocs; i++) {
+            BigDecimal val = reader.getBigDecimal(i, readerContext);
+            creator.putBigDecimal(val);
+          }
+          break;
         }
-        break;
+        default:
+          throw new IllegalStateException();
+      }
+    }
+  }
+
+  private void enableDictionary(String column, SegmentDirectory.Writer segmentWriter,
+      IndexCreatorProvider indexCreatorProvider)
+      throws Exception {
+    Preconditions.checkState(_segmentMetadata.getVersion() == SegmentVersion.v3);
+    ColumnMetadata existingColMetadata = _segmentMetadata.getColumnMetadataFor(column);
+    Preconditions.checkState(!existingColMetadata.hasDictionary(),
+        "Cannot rewrite dictionary enabled forward index. Dictionary already exists for column:" + column);
+    boolean isSingleValue = existingColMetadata.isSingleValue();
+
+    File indexDir = _segmentMetadata.getIndexDir();
+    String segmentName = _segmentMetadata.getName();
+    File inProgress = new File(indexDir, column + ".dict.inprogress");
+    File dictionaryFile = new File(indexDir, column + V1Constants.Dict.FILE_EXTENSION);
+    String fwdIndexFileExtension;
+    if (isSingleValue) {
+      fwdIndexFileExtension =
+          existingColMetadata.isSorted() ? V1Constants.Indexes.SORTED_SV_FORWARD_INDEX_FILE_EXTENSION
+              : V1Constants.Indexes.UNSORTED_SV_FORWARD_INDEX_FILE_EXTENSION;
+    } else {
+      fwdIndexFileExtension = V1Constants.Indexes.UNSORTED_MV_FORWARD_INDEX_FILE_EXTENSION;
+    }
+    File fwdIndexFile = new File(indexDir, column + fwdIndexFileExtension);
+
+    if (!inProgress.exists()) {
+      // Marker file does not exist, which means last run ended normally.
+      // Create a marker file.
+      FileUtils.touch(inProgress);
+    } else {
+      // Marker file exists, which means last run was interrupted.
+      // Remove forward index and dictionary files if they exist.
+      FileUtils.deleteQuietly(fwdIndexFile);
+      FileUtils.deleteQuietly(dictionaryFile);
+    }
+
+    LOGGER.info("Creating a new dictionary for segment={} and column={}", segmentName, column);
+    SegmentDictionaryCreator dictionaryCreator = buildDictionary(column, existingColMetadata, segmentWriter);
+    LoaderUtils.writeIndexToV3Format(segmentWriter, column, dictionaryFile, ColumnIndexType.DICTIONARY);
+
+    LOGGER.info("Built dictionary. Rewriting dictionary enabled forward index for segment={} and column={}",
+        segmentName, column);
+    writeDictEnabledForwardIndex(column, existingColMetadata, segmentWriter, indexDir, indexCreatorProvider,
+        dictionaryCreator);
+    // We used the existing forward index to generate a new forward index. The existing forward index will be in V3
+    // format and the new forward index will be in V1 format. Remove the existing forward index as it is not needed
+    // anymore. Note that removeIndex() will only mark an index for removal and remove the in-memory state. The
+    // actual cleanup from columns.psf file will happen when singleFileIndexDirectory.cleanupRemovedIndices() is
+    // called during segmentWriter.close().
+    segmentWriter.removeIndex(column, ColumnIndexType.FORWARD_INDEX);
+    LoaderUtils.writeIndexToV3Format(segmentWriter, column, fwdIndexFile, ColumnIndexType.FORWARD_INDEX);
+
+    LOGGER.info("Created forwardIndex. Updating metadata properties for segment={} and column={}", segmentName, column);
+    Map<String, String> metadataProperties = new HashMap<>();
+    metadataProperties.put(getKeyFor(column, HAS_DICTIONARY), String.valueOf(true));
+    metadataProperties.put(getKeyFor(column, DICTIONARY_ELEMENT_SIZE),
+        String.valueOf(dictionaryCreator.getNumBytesPerEntry()));
+    updateMetadataProperties(indexDir, metadataProperties);
+
+    // We remove indexes that have to be rewritten when a dictEnabled is toggled. Note that the respective index
+    // handler will take care of recreating the index.
+    removeDictRelatedIndexes(column, segmentWriter);
+
+    // Delete the marker file.
+    FileUtils.deleteQuietly(inProgress);
+
+    LOGGER.info("Created dictionary based forward index for segment: {}, column: {}", segmentName, column);
+  }
+
+  private SegmentDictionaryCreator buildDictionary(String column, ColumnMetadata existingColMetadata,
+      SegmentDirectory.Writer segmentWriter)
+      throws Exception {
+    int numDocs = existingColMetadata.getTotalDocs();
+    // SegmentPartitionConfig is not relevant for rewrites.
+    StatsCollectorConfig statsCollectorConfig =
+        new StatsCollectorConfig(_indexLoadingConfig.getTableConfig(), _schema, null);
+    AbstractColumnStatisticsCollector statsCollector;
+
+    try (ForwardIndexReader reader = LoaderUtils.getForwardIndexReader(segmentWriter, existingColMetadata)) {
+      boolean isSVColumn = reader.isSingleValue();
+
+      switch (reader.getStoredType()) {
+        case INT:
+          statsCollector = new IntColumnPreIndexStatsCollector(column, statsCollectorConfig);
+          break;
+        case LONG:
+          statsCollector = new LongColumnPreIndexStatsCollector(column, statsCollectorConfig);
+          break;
+        case FLOAT:
+          statsCollector = new FloatColumnPreIndexStatsCollector(column, statsCollectorConfig);
+          break;
+        case DOUBLE:
+          statsCollector = new DoubleColumnPreIndexStatsCollector(column, statsCollectorConfig);
+          break;
+        case STRING:
+          statsCollector = new StringColumnPreIndexStatsCollector(column, statsCollectorConfig);
+          break;
+        case BYTES:
+          statsCollector = new BytesColumnPredIndexStatsCollector(column, statsCollectorConfig);
+          break;
+        case BIG_DECIMAL:
+          Preconditions.checkState(isSVColumn, "BigDecimal is not supported for MV columns");
+          statsCollector = new BigDecimalColumnPreIndexStatsCollector(column, statsCollectorConfig);
+          break;
+        default:
+          throw new IllegalStateException();
+      }
+
+      Preconditions.checkState(statsCollector != null);
+      // Note: Special Null handling is not necessary here. This is because, the existing default null value in the
+      // raw forwardIndex will be retained as such while created the dictionary and dict-based forward index. Also,
+      // null value vectors maintain a bitmap of docIds. No handling is necessary there.
+      PinotSegmentColumnReader columnReader =
+          new PinotSegmentColumnReader(reader, null, null, existingColMetadata.getMaxNumberOfMultiValues());
+      for (int i = 0; i < numDocs; i++) {
+        Object obj = columnReader.getValue(i);
+        statsCollector.collect(obj);
+      }
+      statsCollector.seal();
+
+      boolean useVarLength = SegmentIndexCreationDriverImpl.shouldUseVarLengthDictionary(column,
+          _indexLoadingConfig.getVarLengthDictionaryColumns(), reader.getStoredType(), statsCollector);
+      SegmentDictionaryCreator dictionaryCreator =
+          new SegmentDictionaryCreator(existingColMetadata.getFieldSpec(), _segmentMetadata.getIndexDir(),
+              useVarLength);
+
+      dictionaryCreator.build(statsCollector.getUniqueValuesSet());
+      return dictionaryCreator;
+    }
+  }
+
+  private void writeDictEnabledForwardIndex(String column, ColumnMetadata existingColMetadata,
+      SegmentDirectory.Writer segmentWriter, File indexDir, IndexCreatorProvider indexCreatorProvider,
+      SegmentDictionaryCreator dictionaryCreator)
+      throws Exception {
+    try (ForwardIndexReader reader = LoaderUtils.getForwardIndexReader(segmentWriter, existingColMetadata)) {
+      int lengthOfLongestEntry = reader.getLengthOfLongestEntry();
+      IndexCreationContext.Builder builder =
+          IndexCreationContext.builder().withIndexDir(indexDir).withColumnMetadata(existingColMetadata)
+              .withLengthOfLongestEntry(lengthOfLongestEntry);
+      // existingColMetadata has dictEnable=false. Overwrite the value.
+      builder.withDictionary(true);
+      IndexCreationContext.Forward context =
+          builder.build().forForwardIndex(null, _indexLoadingConfig.getColumnProperties());
+
+      try (ForwardIndexCreator creator = indexCreatorProvider.newForwardIndexCreator(context)) {
+        int numDocs = existingColMetadata.getTotalDocs();
+        forwardIndexWriterHelper(column, existingColMetadata, reader, creator, numDocs, dictionaryCreator);
       }
-      default:
-        throw new IllegalStateException();
     }
   }
+
+  private void removeDictRelatedIndexes(String column, SegmentDirectory.Writer segmentWriter) {
+    // TODO: Move this logic as a static function in each index creator.
+    segmentWriter.removeIndex(column, ColumnIndexType.RANGE_INDEX);
+  }
+
+  private void updateMetadataProperties(File indexDir, Map<String, String> metadataProperties)
+      throws Exception {
+    File v3Dir = SegmentDirectoryPaths.segmentDirectoryFor(indexDir, SegmentVersion.v3);
+    File metadataFile = new File(v3Dir, V1Constants.MetadataKeys.METADATA_FILE_NAME);
+    PropertiesConfiguration properties = new PropertiesConfiguration(metadataFile);
+
+    for (Map.Entry<String, String> entry : metadataProperties.entrySet()) {
+      properties.setProperty(entry.getKey(), entry.getValue());
+    }
+
+    properties.save();

Review Comment:
   `save()` is just a wrapper on  `function save(OutputStream out)`



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


[GitHub] [pinot] siddharthteotia commented on a diff in pull request #9678: Support creating dictionary at runtime for an existing column

Posted by GitBox <gi...@apache.org>.
siddharthteotia commented on code in PR #9678:
URL: https://github.com/apache/pinot/pull/9678#discussion_r1011038492


##########
pinot-segment-local/src/test/java/org/apache/pinot/segment/local/segment/index/loader/ForwardIndexHandlerTest.java:
##########
@@ -387,7 +464,7 @@ public void testComputeOperation()
     segmentLocalFSDirectory.close();
   }
 
-  @Test
+  @Test(priority = 1)

Review Comment:
   Why is priority needed ?



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


[GitHub] [pinot] siddharthteotia commented on a diff in pull request #9678: Support creating dictionary at runtime for an existing column

Posted by GitBox <gi...@apache.org>.
siddharthteotia commented on code in PR #9678:
URL: https://github.com/apache/pinot/pull/9678#discussion_r1012268600


##########
pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/loader/ForwardIndexHandler.java:
##########
@@ -271,118 +304,295 @@ private void rewriteRawSVForwardIndex(String column, ColumnMetadata existingColM
               .forForwardIndex(newCompressionType, _indexLoadingConfig.getColumnProperties());
 
       try (ForwardIndexCreator creator = indexCreatorProvider.newForwardIndexCreator(context)) {
-        // If creator stored type and the reader stored type do not match, throw an exception.
         if (!reader.getStoredType().equals(creator.getValueType())) {
+          // Creator stored type should match reader stored type for raw columns. We do not support changing datatypes.
           String failureMsg =
               "Unsupported operation to change datatype for column=" + column + " from " + reader.getStoredType()
                   .toString() + " to " + creator.getValueType().toString();
           throw new UnsupportedOperationException(failureMsg);
         }
 
         int numDocs = existingColMetadata.getTotalDocs();
-        forwardIndexWriterHelper(column, reader, creator, numDocs);
+        forwardIndexWriterHelper(column, existingColMetadata, reader, creator, numDocs, null);
       }
     }
   }
 
-  private void forwardIndexWriterHelper(String column, ForwardIndexReader reader, ForwardIndexCreator creator,
-      int numDocs) {
-    // If creator stored type should match reader stored type. We do not support changing datatypes.
-    if (!reader.getStoredType().equals(creator.getValueType())) {
-      String failureMsg =
-          "Unsupported operation to change datatype for column=" + column + " from " + reader.getStoredType().toString()
-              + " to " + creator.getValueType().toString();
-      throw new UnsupportedOperationException(failureMsg);
-    }
-
+  private void forwardIndexWriterHelper(String column, ColumnMetadata existingColumnMetadata, ForwardIndexReader reader,
+      ForwardIndexCreator creator, int numDocs, @Nullable SegmentDictionaryCreator dictionaryCreator) {
     ForwardIndexReaderContext readerContext = reader.createContext();
     boolean isSVColumn = reader.isSingleValue();
 
-    switch (reader.getStoredType()) {
-      // JSON fields are either stored as string or bytes. No special handling is needed because we make this
-      // decision based on the storedType of the reader.
-      case INT: {
-        for (int i = 0; i < numDocs; i++) {
-          if (isSVColumn) {
-            int val = reader.getInt(i, readerContext);
-            creator.putInt(val);
-          } else {
-            int[] ints = reader.getIntMV(i, readerContext);
-            creator.putIntMV(ints);
-          }
+    if (dictionaryCreator != null) {
+      int maxNumValuesPerEntry = existingColumnMetadata.getMaxNumberOfMultiValues();
+      PinotSegmentColumnReader columnReader = new PinotSegmentColumnReader(reader, null, null, maxNumValuesPerEntry);
+
+      for (int i = 0; i < numDocs; i++) {
+        Object obj = columnReader.getValue(i);
+
+        if (isSVColumn) {
+          int dictId = dictionaryCreator.indexOfSV(obj);
+          creator.putDictId(dictId);
+        } else {
+          int[] dictIds = dictionaryCreator.indexOfMV(obj);
+          creator.putDictIdMV(dictIds);
         }
-        break;
       }
-      case LONG: {
-        for (int i = 0; i < numDocs; i++) {
-          if (isSVColumn) {
-            long val = reader.getLong(i, readerContext);
-            creator.putLong(val);
-          } else {
-            long[] longs = reader.getLongMV(i, readerContext);
-            creator.putLongMV(longs);
+    } else {
+      switch (reader.getStoredType()) {
+        // JSON fields are either stored as string or bytes. No special handling is needed because we make this
+        // decision based on the storedType of the reader.
+        case INT: {
+          for (int i = 0; i < numDocs; i++) {
+            if (isSVColumn) {
+              int val = reader.getInt(i, readerContext);
+              creator.putInt(val);
+            } else {
+              int[] ints = reader.getIntMV(i, readerContext);
+              creator.putIntMV(ints);
+            }
           }
+          break;
         }
-        break;
-      }
-      case FLOAT: {
-        for (int i = 0; i < numDocs; i++) {
-          if (isSVColumn) {
-            float val = reader.getFloat(i, readerContext);
-            creator.putFloat(val);
-          } else {
-            float[] floats = reader.getFloatMV(i, readerContext);
-            creator.putFloatMV(floats);
+        case LONG: {
+          for (int i = 0; i < numDocs; i++) {
+            if (isSVColumn) {
+              long val = reader.getLong(i, readerContext);
+              creator.putLong(val);
+            } else {
+              long[] longs = reader.getLongMV(i, readerContext);
+              creator.putLongMV(longs);
+            }
           }
+          break;
         }
-        break;
-      }
-      case DOUBLE: {
-        for (int i = 0; i < numDocs; i++) {
-          if (isSVColumn) {
-            double val = reader.getDouble(i, readerContext);
-            creator.putDouble(val);
-          } else {
-            double[] doubles = reader.getDoubleMV(i, readerContext);
-            creator.putDoubleMV(doubles);
+        case FLOAT: {
+          for (int i = 0; i < numDocs; i++) {
+            if (isSVColumn) {
+              float val = reader.getFloat(i, readerContext);
+              creator.putFloat(val);
+            } else {
+              float[] floats = reader.getFloatMV(i, readerContext);
+              creator.putFloatMV(floats);
+            }
           }
+          break;
         }
-        break;
-      }
-      case STRING: {
-        for (int i = 0; i < numDocs; i++) {
-          if (isSVColumn) {
-            String val = reader.getString(i, readerContext);
-            creator.putString(val);
-          } else {
-            String[] strings = reader.getStringMV(i, readerContext);
-            creator.putStringMV(strings);
+        case DOUBLE: {
+          for (int i = 0; i < numDocs; i++) {
+            if (isSVColumn) {
+              double val = reader.getDouble(i, readerContext);
+              creator.putDouble(val);
+            } else {
+              double[] doubles = reader.getDoubleMV(i, readerContext);
+              creator.putDoubleMV(doubles);
+            }
           }
+          break;
         }
-        break;
-      }
-      case BYTES: {
-        for (int i = 0; i < numDocs; i++) {
-          if (isSVColumn) {
-            byte[] val = reader.getBytes(i, readerContext);
-            creator.putBytes(val);
-          } else {
-            byte[][] bytesArray = reader.getBytesMV(i, readerContext);
-            creator.putBytesMV(bytesArray);
+        case STRING: {
+          for (int i = 0; i < numDocs; i++) {
+            if (isSVColumn) {
+              String val = reader.getString(i, readerContext);
+              creator.putString(val);
+            } else {
+              String[] strings = reader.getStringMV(i, readerContext);
+              creator.putStringMV(strings);
+            }
           }
+          break;
         }
-        break;
-      }
-      case BIG_DECIMAL: {
-        for (int i = 0; i < numDocs; i++) {
+        case BYTES: {
+          for (int i = 0; i < numDocs; i++) {
+            if (isSVColumn) {
+              byte[] val = reader.getBytes(i, readerContext);
+              creator.putBytes(val);
+            } else {
+              byte[][] bytesArray = reader.getBytesMV(i, readerContext);
+              creator.putBytesMV(bytesArray);
+            }
+          }
+          break;
+        }
+        case BIG_DECIMAL: {
           Preconditions.checkState(isSVColumn, "BigDecimal is not supported for MV columns");
-          BigDecimal val = reader.getBigDecimal(i, readerContext);
-          creator.putBigDecimal(val);
+          for (int i = 0; i < numDocs; i++) {
+            BigDecimal val = reader.getBigDecimal(i, readerContext);
+            creator.putBigDecimal(val);
+          }
+          break;
         }
-        break;
+        default:
+          throw new IllegalStateException();
+      }
+    }
+  }
+
+  private void enableDictionary(String column, SegmentDirectory.Writer segmentWriter,
+      IndexCreatorProvider indexCreatorProvider)
+      throws Exception {
+    Preconditions.checkState(_segmentMetadata.getVersion() == SegmentVersion.v3);
+    ColumnMetadata existingColMetadata = _segmentMetadata.getColumnMetadataFor(column);
+    Preconditions.checkState(!existingColMetadata.hasDictionary(),
+        "Cannot rewrite dictionary enabled forward index. Dictionary already exists for column:" + column);
+    boolean isSingleValue = existingColMetadata.isSingleValue();
+
+    File indexDir = _segmentMetadata.getIndexDir();
+    String segmentName = _segmentMetadata.getName();
+    File inProgress = new File(indexDir, column + ".dict.inprogress");
+    File dictionaryFile = new File(indexDir, column + V1Constants.Dict.FILE_EXTENSION);
+    String fwdIndexFileExtension;
+    if (isSingleValue) {
+      fwdIndexFileExtension =
+          existingColMetadata.isSorted() ? V1Constants.Indexes.SORTED_SV_FORWARD_INDEX_FILE_EXTENSION
+              : V1Constants.Indexes.UNSORTED_SV_FORWARD_INDEX_FILE_EXTENSION;
+    } else {
+      fwdIndexFileExtension = V1Constants.Indexes.UNSORTED_MV_FORWARD_INDEX_FILE_EXTENSION;
+    }
+    File fwdIndexFile = new File(indexDir, column + fwdIndexFileExtension);
+
+    if (!inProgress.exists()) {
+      // Marker file does not exist, which means last run ended normally.
+      // Create a marker file.
+      FileUtils.touch(inProgress);
+    } else {
+      // Marker file exists, which means last run was interrupted.
+      // Remove forward index and dictionary files if they exist.
+      FileUtils.deleteQuietly(fwdIndexFile);
+      FileUtils.deleteQuietly(dictionaryFile);
+    }
+
+    LOGGER.info("Creating a new dictionary for segment={} and column={}", segmentName, column);
+    SegmentDictionaryCreator dictionaryCreator = buildDictionary(column, existingColMetadata, segmentWriter);
+    LoaderUtils.writeIndexToV3Format(segmentWriter, column, dictionaryFile, ColumnIndexType.DICTIONARY);
+
+    LOGGER.info("Built dictionary. Rewriting dictionary enabled forward index for segment={} and column={}",
+        segmentName, column);
+    writeDictEnabledForwardIndex(column, existingColMetadata, segmentWriter, indexDir, indexCreatorProvider,
+        dictionaryCreator);
+    // We used the existing forward index to generate a new forward index. The existing forward index will be in V3
+    // format and the new forward index will be in V1 format. Remove the existing forward index as it is not needed
+    // anymore. Note that removeIndex() will only mark an index for removal and remove the in-memory state. The
+    // actual cleanup from columns.psf file will happen when singleFileIndexDirectory.cleanupRemovedIndices() is
+    // called during segmentWriter.close().
+    segmentWriter.removeIndex(column, ColumnIndexType.FORWARD_INDEX);
+    LoaderUtils.writeIndexToV3Format(segmentWriter, column, fwdIndexFile, ColumnIndexType.FORWARD_INDEX);
+
+    LOGGER.info("Created forwardIndex. Updating metadata properties for segment={} and column={}", segmentName, column);
+    Map<String, String> metadataProperties = new HashMap<>();
+    metadataProperties.put(getKeyFor(column, HAS_DICTIONARY), String.valueOf(true));
+    metadataProperties.put(getKeyFor(column, DICTIONARY_ELEMENT_SIZE),
+        String.valueOf(dictionaryCreator.getNumBytesPerEntry()));
+    updateMetadataProperties(indexDir, metadataProperties);
+
+    // We remove indexes that have to be rewritten when a dictEnabled is toggled. Note that the respective index
+    // handler will take care of recreating the index.
+    removeDictRelatedIndexes(column, segmentWriter);
+
+    // Delete the marker file.
+    FileUtils.deleteQuietly(inProgress);
+
+    LOGGER.info("Created dictionary based forward index for segment: {}, column: {}", segmentName, column);
+  }
+
+  private SegmentDictionaryCreator buildDictionary(String column, ColumnMetadata existingColMetadata,
+      SegmentDirectory.Writer segmentWriter)
+      throws Exception {
+    int numDocs = existingColMetadata.getTotalDocs();
+    // SegmentPartitionConfig is not relevant for rewrites.
+    StatsCollectorConfig statsCollectorConfig =
+        new StatsCollectorConfig(_indexLoadingConfig.getTableConfig(), _schema, null);
+    AbstractColumnStatisticsCollector statsCollector;
+
+    try (ForwardIndexReader reader = LoaderUtils.getForwardIndexReader(segmentWriter, existingColMetadata)) {
+      boolean isSVColumn = reader.isSingleValue();
+
+      switch (reader.getStoredType()) {
+        case INT:
+          statsCollector = new IntColumnPreIndexStatsCollector(column, statsCollectorConfig);
+          break;
+        case LONG:
+          statsCollector = new LongColumnPreIndexStatsCollector(column, statsCollectorConfig);
+          break;
+        case FLOAT:
+          statsCollector = new FloatColumnPreIndexStatsCollector(column, statsCollectorConfig);
+          break;
+        case DOUBLE:
+          statsCollector = new DoubleColumnPreIndexStatsCollector(column, statsCollectorConfig);
+          break;
+        case STRING:
+          statsCollector = new StringColumnPreIndexStatsCollector(column, statsCollectorConfig);
+          break;
+        case BYTES:
+          statsCollector = new BytesColumnPredIndexStatsCollector(column, statsCollectorConfig);
+          break;
+        case BIG_DECIMAL:
+          Preconditions.checkState(isSVColumn, "BigDecimal is not supported for MV columns");
+          statsCollector = new BigDecimalColumnPreIndexStatsCollector(column, statsCollectorConfig);
+          break;
+        default:
+          throw new IllegalStateException();
+      }
+
+      Preconditions.checkState(statsCollector != null);
+      // Note: Special Null handling is not necessary here. This is because, the existing default null value in the
+      // raw forwardIndex will be retained as such while created the dictionary and dict-based forward index. Also,
+      // null value vectors maintain a bitmap of docIds. No handling is necessary there.
+      PinotSegmentColumnReader columnReader =

Review Comment:
   Not a blocker for me. Ok with existing code as well 



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


[GitHub] [pinot] siddharthteotia commented on a diff in pull request #9678: Support creating dictionary at runtime for an existing column

Posted by GitBox <gi...@apache.org>.
siddharthteotia commented on code in PR #9678:
URL: https://github.com/apache/pinot/pull/9678#discussion_r1012326965


##########
pinot-segment-local/src/test/java/org/apache/pinot/segment/local/segment/index/loader/SegmentPreProcessorTest.java:
##########
@@ -321,15 +348,106 @@ public void testEnableFSTIndexOnExistingColumnDictEncoded()
     assertNotNull(columnMetadata);
     checkFSTIndexCreation(EXISTING_STRING_COL_DICT, 9, 4, _newColumnsSchemaWithFST, false, false, 26);
 
-    constructV1Segment();
+    constructV1Segment(Collections.emptyList(), Collections.emptyList(), Collections.emptyList());
     segmentMetadata = new SegmentMetadataImpl(_indexDir);
     columnMetadata = segmentMetadata.getColumnMetadataFor(EXISTING_STRING_COL_DICT);
     assertNotNull(columnMetadata);
     checkFSTIndexCreation(EXISTING_STRING_COL_DICT, 9, 4, _newColumnsSchemaWithFST, false, false, 26);
   }
 
   @Test
-  public void testForwardIndexHandler()
+  public void testForwardIndexHandlerEnableDictionary()
+      throws Exception {
+    // Add raw columns in indexingConfig so that the ForwardIndexHandler doesn't end up converting them to dictionary
+    // enabled columns
+    _indexLoadingConfig.getNoDictionaryColumns().add(EXISTING_INT_COL_RAW_MV);
+    _indexLoadingConfig.getNoDictionaryColumns().add(EXISTING_INT_COL_RAW);
+    _indexLoadingConfig.getNoDictionaryColumns().add(EXISTING_STRING_COL_RAW);
+
+    // TEST 1. Check running forwardIndexHandler on a V1 segment. No-op for all existing raw columns.
+    constructV1Segment(Collections.emptyList(), Collections.emptyList(), Collections.emptyList());
+    checkForwardIndexCreation(EXISTING_STRING_COL_DICT, 9, 4, _schema, false, true, false, 26, null, true, 0,
+        DataType.STRING, 100000);
+    validateIndex(ColumnIndexType.FORWARD_INDEX, EXISTING_STRING_COL_RAW, 5, 3, _schema, false, false, false, 0, true,
+        0, ChunkCompressionType.LZ4, false, DataType.STRING, 100000);
+    validateIndex(ColumnIndexType.FORWARD_INDEX, EXISTING_INT_COL_RAW_MV, 18499, 15, _schema, false, false, false, 0,
+        false, 13, ChunkCompressionType.LZ4, false, DataType.INT, 106688);
+    validateIndex(ColumnIndexType.FORWARD_INDEX, EXISTING_INT_COL_RAW, 42242, 16, _schema, false, false, false, 0, true,
+        0, ChunkCompressionType.LZ4, false, DataType.INT, 100000);
+
+    // Convert the segment to V3.
+    new SegmentV1V2ToV3FormatConverter().convert(_indexDir);
+
+    // TEST 2: Run reload with no-changes.
+    checkForwardIndexCreation(EXISTING_STRING_COL_DICT, 9, 4, _schema, false, true, false, 26, null, true, 0,
+        DataType.STRING, 100000);
+
+    // TEST 3: EXISTING_STRING_COL_RAW. Enable dictionary. Also add inverted index and text index. Reload code path

Review Comment:
   (nit) you can potentially reorg tests to make them more readable
   
   - simple enable dict on SV, no other change
   - simple enable dict on MV, no other change
   - enable dict + <something else> on SV
   - enable dict + <something else> on MV



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


[GitHub] [pinot] somandal commented on a diff in pull request #9678: Support creating dictionary at runtime for an existing column

Posted by GitBox <gi...@apache.org>.
somandal commented on code in PR #9678:
URL: https://github.com/apache/pinot/pull/9678#discussion_r1012155150


##########
pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/loader/ForwardIndexHandler.java:
##########
@@ -271,118 +304,295 @@ private void rewriteRawSVForwardIndex(String column, ColumnMetadata existingColM
               .forForwardIndex(newCompressionType, _indexLoadingConfig.getColumnProperties());
 
       try (ForwardIndexCreator creator = indexCreatorProvider.newForwardIndexCreator(context)) {
-        // If creator stored type and the reader stored type do not match, throw an exception.
         if (!reader.getStoredType().equals(creator.getValueType())) {
+          // Creator stored type should match reader stored type for raw columns. We do not support changing datatypes.
           String failureMsg =
               "Unsupported operation to change datatype for column=" + column + " from " + reader.getStoredType()
                   .toString() + " to " + creator.getValueType().toString();
           throw new UnsupportedOperationException(failureMsg);
         }
 
         int numDocs = existingColMetadata.getTotalDocs();
-        forwardIndexWriterHelper(column, reader, creator, numDocs);
+        forwardIndexWriterHelper(column, existingColMetadata, reader, creator, numDocs, null);
       }
     }
   }
 
-  private void forwardIndexWriterHelper(String column, ForwardIndexReader reader, ForwardIndexCreator creator,
-      int numDocs) {
-    // If creator stored type should match reader stored type. We do not support changing datatypes.
-    if (!reader.getStoredType().equals(creator.getValueType())) {
-      String failureMsg =
-          "Unsupported operation to change datatype for column=" + column + " from " + reader.getStoredType().toString()
-              + " to " + creator.getValueType().toString();
-      throw new UnsupportedOperationException(failureMsg);
-    }
-
+  private void forwardIndexWriterHelper(String column, ColumnMetadata existingColumnMetadata, ForwardIndexReader reader,
+      ForwardIndexCreator creator, int numDocs, @Nullable SegmentDictionaryCreator dictionaryCreator) {
     ForwardIndexReaderContext readerContext = reader.createContext();
     boolean isSVColumn = reader.isSingleValue();
 
-    switch (reader.getStoredType()) {
-      // JSON fields are either stored as string or bytes. No special handling is needed because we make this
-      // decision based on the storedType of the reader.
-      case INT: {
-        for (int i = 0; i < numDocs; i++) {
-          if (isSVColumn) {
-            int val = reader.getInt(i, readerContext);
-            creator.putInt(val);
-          } else {
-            int[] ints = reader.getIntMV(i, readerContext);
-            creator.putIntMV(ints);
-          }
+    if (dictionaryCreator != null) {
+      int maxNumValuesPerEntry = existingColumnMetadata.getMaxNumberOfMultiValues();
+      PinotSegmentColumnReader columnReader = new PinotSegmentColumnReader(reader, null, null, maxNumValuesPerEntry);
+
+      for (int i = 0; i < numDocs; i++) {
+        Object obj = columnReader.getValue(i);
+
+        if (isSVColumn) {
+          int dictId = dictionaryCreator.indexOfSV(obj);
+          creator.putDictId(dictId);
+        } else {
+          int[] dictIds = dictionaryCreator.indexOfMV(obj);
+          creator.putDictIdMV(dictIds);
         }
-        break;
       }
-      case LONG: {
-        for (int i = 0; i < numDocs; i++) {
-          if (isSVColumn) {
-            long val = reader.getLong(i, readerContext);
-            creator.putLong(val);
-          } else {
-            long[] longs = reader.getLongMV(i, readerContext);
-            creator.putLongMV(longs);
+    } else {
+      switch (reader.getStoredType()) {
+        // JSON fields are either stored as string or bytes. No special handling is needed because we make this
+        // decision based on the storedType of the reader.
+        case INT: {
+          for (int i = 0; i < numDocs; i++) {
+            if (isSVColumn) {
+              int val = reader.getInt(i, readerContext);
+              creator.putInt(val);
+            } else {
+              int[] ints = reader.getIntMV(i, readerContext);
+              creator.putIntMV(ints);
+            }
           }
+          break;
         }
-        break;
-      }
-      case FLOAT: {
-        for (int i = 0; i < numDocs; i++) {
-          if (isSVColumn) {
-            float val = reader.getFloat(i, readerContext);
-            creator.putFloat(val);
-          } else {
-            float[] floats = reader.getFloatMV(i, readerContext);
-            creator.putFloatMV(floats);
+        case LONG: {
+          for (int i = 0; i < numDocs; i++) {
+            if (isSVColumn) {
+              long val = reader.getLong(i, readerContext);
+              creator.putLong(val);
+            } else {
+              long[] longs = reader.getLongMV(i, readerContext);
+              creator.putLongMV(longs);
+            }
           }
+          break;
         }
-        break;
-      }
-      case DOUBLE: {
-        for (int i = 0; i < numDocs; i++) {
-          if (isSVColumn) {
-            double val = reader.getDouble(i, readerContext);
-            creator.putDouble(val);
-          } else {
-            double[] doubles = reader.getDoubleMV(i, readerContext);
-            creator.putDoubleMV(doubles);
+        case FLOAT: {
+          for (int i = 0; i < numDocs; i++) {
+            if (isSVColumn) {
+              float val = reader.getFloat(i, readerContext);
+              creator.putFloat(val);
+            } else {
+              float[] floats = reader.getFloatMV(i, readerContext);
+              creator.putFloatMV(floats);
+            }
           }
+          break;
         }
-        break;
-      }
-      case STRING: {
-        for (int i = 0; i < numDocs; i++) {
-          if (isSVColumn) {
-            String val = reader.getString(i, readerContext);
-            creator.putString(val);
-          } else {
-            String[] strings = reader.getStringMV(i, readerContext);
-            creator.putStringMV(strings);
+        case DOUBLE: {
+          for (int i = 0; i < numDocs; i++) {
+            if (isSVColumn) {
+              double val = reader.getDouble(i, readerContext);
+              creator.putDouble(val);
+            } else {
+              double[] doubles = reader.getDoubleMV(i, readerContext);
+              creator.putDoubleMV(doubles);
+            }
           }
+          break;
         }
-        break;
-      }
-      case BYTES: {
-        for (int i = 0; i < numDocs; i++) {
-          if (isSVColumn) {
-            byte[] val = reader.getBytes(i, readerContext);
-            creator.putBytes(val);
-          } else {
-            byte[][] bytesArray = reader.getBytesMV(i, readerContext);
-            creator.putBytesMV(bytesArray);
+        case STRING: {
+          for (int i = 0; i < numDocs; i++) {
+            if (isSVColumn) {
+              String val = reader.getString(i, readerContext);
+              creator.putString(val);
+            } else {
+              String[] strings = reader.getStringMV(i, readerContext);
+              creator.putStringMV(strings);
+            }
           }
+          break;
         }
-        break;
-      }
-      case BIG_DECIMAL: {
-        for (int i = 0; i < numDocs; i++) {
+        case BYTES: {
+          for (int i = 0; i < numDocs; i++) {
+            if (isSVColumn) {
+              byte[] val = reader.getBytes(i, readerContext);
+              creator.putBytes(val);
+            } else {
+              byte[][] bytesArray = reader.getBytesMV(i, readerContext);
+              creator.putBytesMV(bytesArray);
+            }
+          }
+          break;
+        }
+        case BIG_DECIMAL: {
           Preconditions.checkState(isSVColumn, "BigDecimal is not supported for MV columns");
-          BigDecimal val = reader.getBigDecimal(i, readerContext);
-          creator.putBigDecimal(val);
+          for (int i = 0; i < numDocs; i++) {
+            BigDecimal val = reader.getBigDecimal(i, readerContext);
+            creator.putBigDecimal(val);
+          }
+          break;
         }
-        break;
+        default:
+          throw new IllegalStateException();
+      }
+    }
+  }
+
+  private void enableDictionary(String column, SegmentDirectory.Writer segmentWriter,
+      IndexCreatorProvider indexCreatorProvider)
+      throws Exception {
+    Preconditions.checkState(_segmentMetadata.getVersion() == SegmentVersion.v3);
+    ColumnMetadata existingColMetadata = _segmentMetadata.getColumnMetadataFor(column);
+    Preconditions.checkState(!existingColMetadata.hasDictionary(),
+        "Cannot rewrite dictionary enabled forward index. Dictionary already exists for column:" + column);
+    boolean isSingleValue = existingColMetadata.isSingleValue();
+
+    File indexDir = _segmentMetadata.getIndexDir();
+    String segmentName = _segmentMetadata.getName();
+    File inProgress = new File(indexDir, column + ".dict.inprogress");
+    File dictionaryFile = new File(indexDir, column + V1Constants.Dict.FILE_EXTENSION);
+    String fwdIndexFileExtension;
+    if (isSingleValue) {
+      fwdIndexFileExtension =
+          existingColMetadata.isSorted() ? V1Constants.Indexes.SORTED_SV_FORWARD_INDEX_FILE_EXTENSION
+              : V1Constants.Indexes.UNSORTED_SV_FORWARD_INDEX_FILE_EXTENSION;
+    } else {
+      fwdIndexFileExtension = V1Constants.Indexes.UNSORTED_MV_FORWARD_INDEX_FILE_EXTENSION;
+    }
+    File fwdIndexFile = new File(indexDir, column + fwdIndexFileExtension);
+
+    if (!inProgress.exists()) {
+      // Marker file does not exist, which means last run ended normally.
+      // Create a marker file.
+      FileUtils.touch(inProgress);
+    } else {
+      // Marker file exists, which means last run was interrupted.
+      // Remove forward index and dictionary files if they exist.
+      FileUtils.deleteQuietly(fwdIndexFile);
+      FileUtils.deleteQuietly(dictionaryFile);
+    }
+
+    LOGGER.info("Creating a new dictionary for segment={} and column={}", segmentName, column);
+    SegmentDictionaryCreator dictionaryCreator = buildDictionary(column, existingColMetadata, segmentWriter);
+    LoaderUtils.writeIndexToV3Format(segmentWriter, column, dictionaryFile, ColumnIndexType.DICTIONARY);
+
+    LOGGER.info("Built dictionary. Rewriting dictionary enabled forward index for segment={} and column={}",
+        segmentName, column);
+    writeDictEnabledForwardIndex(column, existingColMetadata, segmentWriter, indexDir, indexCreatorProvider,
+        dictionaryCreator);
+    // We used the existing forward index to generate a new forward index. The existing forward index will be in V3
+    // format and the new forward index will be in V1 format. Remove the existing forward index as it is not needed
+    // anymore. Note that removeIndex() will only mark an index for removal and remove the in-memory state. The
+    // actual cleanup from columns.psf file will happen when singleFileIndexDirectory.cleanupRemovedIndices() is
+    // called during segmentWriter.close().
+    segmentWriter.removeIndex(column, ColumnIndexType.FORWARD_INDEX);
+    LoaderUtils.writeIndexToV3Format(segmentWriter, column, fwdIndexFile, ColumnIndexType.FORWARD_INDEX);
+
+    LOGGER.info("Created forwardIndex. Updating metadata properties for segment={} and column={}", segmentName, column);
+    Map<String, String> metadataProperties = new HashMap<>();
+    metadataProperties.put(getKeyFor(column, HAS_DICTIONARY), String.valueOf(true));
+    metadataProperties.put(getKeyFor(column, DICTIONARY_ELEMENT_SIZE),
+        String.valueOf(dictionaryCreator.getNumBytesPerEntry()));
+    updateMetadataProperties(indexDir, metadataProperties);
+
+    // We remove indexes that have to be rewritten when a dictEnabled is toggled. Note that the respective index
+    // handler will take care of recreating the index.
+    removeDictRelatedIndexes(column, segmentWriter);
+
+    // Delete the marker file.
+    FileUtils.deleteQuietly(inProgress);
+
+    LOGGER.info("Created dictionary based forward index for segment: {}, column: {}", segmentName, column);
+  }
+
+  private SegmentDictionaryCreator buildDictionary(String column, ColumnMetadata existingColMetadata,
+      SegmentDirectory.Writer segmentWriter)
+      throws Exception {
+    int numDocs = existingColMetadata.getTotalDocs();
+    // SegmentPartitionConfig is not relevant for rewrites.
+    StatsCollectorConfig statsCollectorConfig =
+        new StatsCollectorConfig(_indexLoadingConfig.getTableConfig(), _schema, null);
+    AbstractColumnStatisticsCollector statsCollector;
+
+    try (ForwardIndexReader reader = LoaderUtils.getForwardIndexReader(segmentWriter, existingColMetadata)) {
+      boolean isSVColumn = reader.isSingleValue();
+
+      switch (reader.getStoredType()) {
+        case INT:
+          statsCollector = new IntColumnPreIndexStatsCollector(column, statsCollectorConfig);
+          break;
+        case LONG:
+          statsCollector = new LongColumnPreIndexStatsCollector(column, statsCollectorConfig);
+          break;
+        case FLOAT:
+          statsCollector = new FloatColumnPreIndexStatsCollector(column, statsCollectorConfig);
+          break;
+        case DOUBLE:
+          statsCollector = new DoubleColumnPreIndexStatsCollector(column, statsCollectorConfig);
+          break;
+        case STRING:
+          statsCollector = new StringColumnPreIndexStatsCollector(column, statsCollectorConfig);
+          break;
+        case BYTES:
+          statsCollector = new BytesColumnPredIndexStatsCollector(column, statsCollectorConfig);
+          break;
+        case BIG_DECIMAL:
+          Preconditions.checkState(isSVColumn, "BigDecimal is not supported for MV columns");
+          statsCollector = new BigDecimalColumnPreIndexStatsCollector(column, statsCollectorConfig);
+          break;
+        default:
+          throw new IllegalStateException();

Review Comment:
   nit: returning a meaningful exception message? include column name and perhaps the type?



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


[GitHub] [pinot] somandal commented on a diff in pull request #9678: Support creating dictionary at runtime for an existing column

Posted by GitBox <gi...@apache.org>.
somandal commented on code in PR #9678:
URL: https://github.com/apache/pinot/pull/9678#discussion_r1012148150


##########
pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/loader/ForwardIndexHandler.java:
##########
@@ -271,118 +304,295 @@ private void rewriteRawSVForwardIndex(String column, ColumnMetadata existingColM
               .forForwardIndex(newCompressionType, _indexLoadingConfig.getColumnProperties());
 
       try (ForwardIndexCreator creator = indexCreatorProvider.newForwardIndexCreator(context)) {
-        // If creator stored type and the reader stored type do not match, throw an exception.
         if (!reader.getStoredType().equals(creator.getValueType())) {
+          // Creator stored type should match reader stored type for raw columns. We do not support changing datatypes.
           String failureMsg =
               "Unsupported operation to change datatype for column=" + column + " from " + reader.getStoredType()
                   .toString() + " to " + creator.getValueType().toString();
           throw new UnsupportedOperationException(failureMsg);
         }
 
         int numDocs = existingColMetadata.getTotalDocs();
-        forwardIndexWriterHelper(column, reader, creator, numDocs);
+        forwardIndexWriterHelper(column, existingColMetadata, reader, creator, numDocs, null);
       }
     }
   }
 
-  private void forwardIndexWriterHelper(String column, ForwardIndexReader reader, ForwardIndexCreator creator,
-      int numDocs) {
-    // If creator stored type should match reader stored type. We do not support changing datatypes.
-    if (!reader.getStoredType().equals(creator.getValueType())) {
-      String failureMsg =
-          "Unsupported operation to change datatype for column=" + column + " from " + reader.getStoredType().toString()
-              + " to " + creator.getValueType().toString();
-      throw new UnsupportedOperationException(failureMsg);
-    }
-
+  private void forwardIndexWriterHelper(String column, ColumnMetadata existingColumnMetadata, ForwardIndexReader reader,
+      ForwardIndexCreator creator, int numDocs, @Nullable SegmentDictionaryCreator dictionaryCreator) {
     ForwardIndexReaderContext readerContext = reader.createContext();
     boolean isSVColumn = reader.isSingleValue();
 
-    switch (reader.getStoredType()) {
-      // JSON fields are either stored as string or bytes. No special handling is needed because we make this
-      // decision based on the storedType of the reader.
-      case INT: {
-        for (int i = 0; i < numDocs; i++) {
-          if (isSVColumn) {
-            int val = reader.getInt(i, readerContext);
-            creator.putInt(val);
-          } else {
-            int[] ints = reader.getIntMV(i, readerContext);
-            creator.putIntMV(ints);
-          }
+    if (dictionaryCreator != null) {
+      int maxNumValuesPerEntry = existingColumnMetadata.getMaxNumberOfMultiValues();
+      PinotSegmentColumnReader columnReader = new PinotSegmentColumnReader(reader, null, null, maxNumValuesPerEntry);
+
+      for (int i = 0; i < numDocs; i++) {
+        Object obj = columnReader.getValue(i);
+
+        if (isSVColumn) {
+          int dictId = dictionaryCreator.indexOfSV(obj);
+          creator.putDictId(dictId);
+        } else {
+          int[] dictIds = dictionaryCreator.indexOfMV(obj);
+          creator.putDictIdMV(dictIds);
         }
-        break;
       }
-      case LONG: {
-        for (int i = 0; i < numDocs; i++) {
-          if (isSVColumn) {
-            long val = reader.getLong(i, readerContext);
-            creator.putLong(val);
-          } else {
-            long[] longs = reader.getLongMV(i, readerContext);
-            creator.putLongMV(longs);
+    } else {
+      switch (reader.getStoredType()) {
+        // JSON fields are either stored as string or bytes. No special handling is needed because we make this
+        // decision based on the storedType of the reader.
+        case INT: {
+          for (int i = 0; i < numDocs; i++) {
+            if (isSVColumn) {
+              int val = reader.getInt(i, readerContext);
+              creator.putInt(val);
+            } else {
+              int[] ints = reader.getIntMV(i, readerContext);
+              creator.putIntMV(ints);
+            }
           }
+          break;
         }
-        break;
-      }
-      case FLOAT: {
-        for (int i = 0; i < numDocs; i++) {
-          if (isSVColumn) {
-            float val = reader.getFloat(i, readerContext);
-            creator.putFloat(val);
-          } else {
-            float[] floats = reader.getFloatMV(i, readerContext);
-            creator.putFloatMV(floats);
+        case LONG: {
+          for (int i = 0; i < numDocs; i++) {
+            if (isSVColumn) {
+              long val = reader.getLong(i, readerContext);
+              creator.putLong(val);
+            } else {
+              long[] longs = reader.getLongMV(i, readerContext);
+              creator.putLongMV(longs);
+            }
           }
+          break;
         }
-        break;
-      }
-      case DOUBLE: {
-        for (int i = 0; i < numDocs; i++) {
-          if (isSVColumn) {
-            double val = reader.getDouble(i, readerContext);
-            creator.putDouble(val);
-          } else {
-            double[] doubles = reader.getDoubleMV(i, readerContext);
-            creator.putDoubleMV(doubles);
+        case FLOAT: {
+          for (int i = 0; i < numDocs; i++) {
+            if (isSVColumn) {
+              float val = reader.getFloat(i, readerContext);
+              creator.putFloat(val);
+            } else {
+              float[] floats = reader.getFloatMV(i, readerContext);
+              creator.putFloatMV(floats);
+            }
           }
+          break;
         }
-        break;
-      }
-      case STRING: {
-        for (int i = 0; i < numDocs; i++) {
-          if (isSVColumn) {
-            String val = reader.getString(i, readerContext);
-            creator.putString(val);
-          } else {
-            String[] strings = reader.getStringMV(i, readerContext);
-            creator.putStringMV(strings);
+        case DOUBLE: {
+          for (int i = 0; i < numDocs; i++) {
+            if (isSVColumn) {
+              double val = reader.getDouble(i, readerContext);
+              creator.putDouble(val);
+            } else {
+              double[] doubles = reader.getDoubleMV(i, readerContext);
+              creator.putDoubleMV(doubles);
+            }
           }
+          break;
         }
-        break;
-      }
-      case BYTES: {
-        for (int i = 0; i < numDocs; i++) {
-          if (isSVColumn) {
-            byte[] val = reader.getBytes(i, readerContext);
-            creator.putBytes(val);
-          } else {
-            byte[][] bytesArray = reader.getBytesMV(i, readerContext);
-            creator.putBytesMV(bytesArray);
+        case STRING: {
+          for (int i = 0; i < numDocs; i++) {
+            if (isSVColumn) {
+              String val = reader.getString(i, readerContext);
+              creator.putString(val);
+            } else {
+              String[] strings = reader.getStringMV(i, readerContext);
+              creator.putStringMV(strings);
+            }
           }
+          break;
         }
-        break;
-      }
-      case BIG_DECIMAL: {
-        for (int i = 0; i < numDocs; i++) {
+        case BYTES: {
+          for (int i = 0; i < numDocs; i++) {
+            if (isSVColumn) {
+              byte[] val = reader.getBytes(i, readerContext);
+              creator.putBytes(val);
+            } else {
+              byte[][] bytesArray = reader.getBytesMV(i, readerContext);
+              creator.putBytesMV(bytesArray);
+            }
+          }
+          break;
+        }
+        case BIG_DECIMAL: {
           Preconditions.checkState(isSVColumn, "BigDecimal is not supported for MV columns");
-          BigDecimal val = reader.getBigDecimal(i, readerContext);
-          creator.putBigDecimal(val);
+          for (int i = 0; i < numDocs; i++) {
+            BigDecimal val = reader.getBigDecimal(i, readerContext);
+            creator.putBigDecimal(val);
+          }
+          break;
         }
-        break;
+        default:
+          throw new IllegalStateException();
+      }
+    }
+  }
+
+  private void enableDictionary(String column, SegmentDirectory.Writer segmentWriter,
+      IndexCreatorProvider indexCreatorProvider)
+      throws Exception {
+    Preconditions.checkState(_segmentMetadata.getVersion() == SegmentVersion.v3);
+    ColumnMetadata existingColMetadata = _segmentMetadata.getColumnMetadataFor(column);
+    Preconditions.checkState(!existingColMetadata.hasDictionary(),
+        "Cannot rewrite dictionary enabled forward index. Dictionary already exists for column:" + column);

Review Comment:
   This is a private function right? Why would tests have direct access to this function?



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


[GitHub] [pinot] somandal commented on a diff in pull request #9678: Support creating dictionary at runtime for an existing column

Posted by GitBox <gi...@apache.org>.
somandal commented on code in PR #9678:
URL: https://github.com/apache/pinot/pull/9678#discussion_r1010661628


##########
pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/loader/ForwardIndexHandler.java:
##########
@@ -271,118 +306,293 @@ private void rewriteRawSVForwardIndex(String column, ColumnMetadata existingColM
               .forForwardIndex(newCompressionType, _indexLoadingConfig.getColumnProperties());
 
       try (ForwardIndexCreator creator = indexCreatorProvider.newForwardIndexCreator(context)) {
-        // If creator stored type and the reader stored type do not match, throw an exception.
         if (!reader.getStoredType().equals(creator.getValueType())) {
+          // Creator stored type should match reader stored type for raw columns. We do not support changing datatypes.
           String failureMsg =
               "Unsupported operation to change datatype for column=" + column + " from " + reader.getStoredType()
                   .toString() + " to " + creator.getValueType().toString();
           throw new UnsupportedOperationException(failureMsg);
         }
 
         int numDocs = existingColMetadata.getTotalDocs();
-        forwardIndexWriterHelper(column, reader, creator, numDocs);
+        forwardIndexWriterHelper(column, existingColMetadata, reader, creator, numDocs, null);
       }
     }
   }
 
-  private void forwardIndexWriterHelper(String column, ForwardIndexReader reader, ForwardIndexCreator creator,
-      int numDocs) {
-    // If creator stored type should match reader stored type. We do not support changing datatypes.
-    if (!reader.getStoredType().equals(creator.getValueType())) {
-      String failureMsg =
-          "Unsupported operation to change datatype for column=" + column + " from " + reader.getStoredType().toString()
-              + " to " + creator.getValueType().toString();
-      throw new UnsupportedOperationException(failureMsg);
-    }
-
+  private void forwardIndexWriterHelper(String column, ColumnMetadata existingColumnMetadata, ForwardIndexReader reader,
+      ForwardIndexCreator creator, int numDocs, @Nullable SegmentDictionaryCreator dictionaryCreator) {
     ForwardIndexReaderContext readerContext = reader.createContext();
     boolean isSVColumn = reader.isSingleValue();
 
-    switch (reader.getStoredType()) {
-      // JSON fields are either stored as string or bytes. No special handling is needed because we make this
-      // decision based on the storedType of the reader.
-      case INT: {
-        for (int i = 0; i < numDocs; i++) {
-          if (isSVColumn) {
-            int val = reader.getInt(i, readerContext);
-            creator.putInt(val);
-          } else {
-            int[] ints = reader.getIntMV(i, readerContext);
-            creator.putIntMV(ints);
-          }
+    if (dictionaryCreator != null) {
+      int maxNumValuesPerEntry = existingColumnMetadata.getMaxNumberOfMultiValues();
+      PinotSegmentColumnReader columnReader = new PinotSegmentColumnReader(reader, null, null, maxNumValuesPerEntry);
+
+      for (int i = 0; i < numDocs; i++) {
+        Object obj = columnReader.getValue(i);
+
+        if (isSVColumn) {
+          int dictId = dictionaryCreator.indexOfSV(obj);
+          creator.putDictId(dictId);
+        } else {
+          int[] dictIds = dictionaryCreator.indexOfMV(obj);
+          creator.putDictIdMV(dictIds);
         }
-        break;
       }
-      case LONG: {
-        for (int i = 0; i < numDocs; i++) {
-          if (isSVColumn) {
-            long val = reader.getLong(i, readerContext);
-            creator.putLong(val);
-          } else {
-            long[] longs = reader.getLongMV(i, readerContext);
-            creator.putLongMV(longs);
+    } else {
+      switch (reader.getStoredType()) {
+        // JSON fields are either stored as string or bytes. No special handling is needed because we make this
+        // decision based on the storedType of the reader.
+        case INT: {
+          for (int i = 0; i < numDocs; i++) {
+            if (isSVColumn) {
+              int val = reader.getInt(i, readerContext);
+              creator.putInt(val);
+            } else {
+              int[] ints = reader.getIntMV(i, readerContext);
+              creator.putIntMV(ints);
+            }
           }
+          break;
         }
-        break;
-      }
-      case FLOAT: {
-        for (int i = 0; i < numDocs; i++) {
-          if (isSVColumn) {
-            float val = reader.getFloat(i, readerContext);
-            creator.putFloat(val);
-          } else {
-            float[] floats = reader.getFloatMV(i, readerContext);
-            creator.putFloatMV(floats);
+        case LONG: {
+          for (int i = 0; i < numDocs; i++) {
+            if (isSVColumn) {
+              long val = reader.getLong(i, readerContext);
+              creator.putLong(val);
+            } else {
+              long[] longs = reader.getLongMV(i, readerContext);
+              creator.putLongMV(longs);
+            }
           }
+          break;
         }
-        break;
-      }
-      case DOUBLE: {
-        for (int i = 0; i < numDocs; i++) {
-          if (isSVColumn) {
-            double val = reader.getDouble(i, readerContext);
-            creator.putDouble(val);
-          } else {
-            double[] doubles = reader.getDoubleMV(i, readerContext);
-            creator.putDoubleMV(doubles);
+        case FLOAT: {
+          for (int i = 0; i < numDocs; i++) {
+            if (isSVColumn) {
+              float val = reader.getFloat(i, readerContext);
+              creator.putFloat(val);
+            } else {
+              float[] floats = reader.getFloatMV(i, readerContext);
+              creator.putFloatMV(floats);
+            }
           }
+          break;
         }
-        break;
-      }
-      case STRING: {
-        for (int i = 0; i < numDocs; i++) {
-          if (isSVColumn) {
-            String val = reader.getString(i, readerContext);
-            creator.putString(val);
-          } else {
-            String[] strings = reader.getStringMV(i, readerContext);
-            creator.putStringMV(strings);
+        case DOUBLE: {
+          for (int i = 0; i < numDocs; i++) {
+            if (isSVColumn) {
+              double val = reader.getDouble(i, readerContext);
+              creator.putDouble(val);
+            } else {
+              double[] doubles = reader.getDoubleMV(i, readerContext);
+              creator.putDoubleMV(doubles);
+            }
           }
+          break;
         }
-        break;
-      }
-      case BYTES: {
-        for (int i = 0; i < numDocs; i++) {
-          if (isSVColumn) {
-            byte[] val = reader.getBytes(i, readerContext);
-            creator.putBytes(val);
-          } else {
-            byte[][] bytesArray = reader.getBytesMV(i, readerContext);
-            creator.putBytesMV(bytesArray);
+        case STRING: {
+          for (int i = 0; i < numDocs; i++) {
+            if (isSVColumn) {
+              String val = reader.getString(i, readerContext);
+              creator.putString(val);
+            } else {
+              String[] strings = reader.getStringMV(i, readerContext);
+              creator.putStringMV(strings);
+            }
           }
+          break;
         }
-        break;
-      }
-      case BIG_DECIMAL: {
-        for (int i = 0; i < numDocs; i++) {
+        case BYTES: {
+          for (int i = 0; i < numDocs; i++) {
+            if (isSVColumn) {
+              byte[] val = reader.getBytes(i, readerContext);
+              creator.putBytes(val);
+            } else {
+              byte[][] bytesArray = reader.getBytesMV(i, readerContext);
+              creator.putBytesMV(bytesArray);
+            }
+          }
+          break;
+        }
+        case BIG_DECIMAL: {
           Preconditions.checkState(isSVColumn, "BigDecimal is not supported for MV columns");
-          BigDecimal val = reader.getBigDecimal(i, readerContext);
-          creator.putBigDecimal(val);
+          for (int i = 0; i < numDocs; i++) {
+            BigDecimal val = reader.getBigDecimal(i, readerContext);
+            creator.putBigDecimal(val);
+          }
+          break;
         }
-        break;
+        default:
+          throw new IllegalStateException();
+      }
+    }
+  }
+
+  private void enableDictionary(String column, SegmentDirectory.Writer segmentWriter,
+      IndexCreatorProvider indexCreatorProvider)
+      throws Exception {
+    Preconditions.checkState(_segmentMetadata.getVersion() == SegmentVersion.v3);
+    ColumnMetadata existingColMetadata = _segmentMetadata.getColumnMetadataFor(column);
+    Preconditions.checkState(!existingColMetadata.hasDictionary(), "Existing column already has dictionary.");
+    boolean isSingleValue = existingColMetadata.isSingleValue();
+
+    File indexDir = _segmentMetadata.getIndexDir();
+    String segmentName = _segmentMetadata.getName();
+    File inProgress = new File(indexDir, column + ".dict.inprogress");
+    File dictionaryFile = new File(indexDir, column + V1Constants.Dict.FILE_EXTENSION);
+    String fwdIndexFileExtension;
+    if (isSingleValue) {
+      if (existingColMetadata.isSorted()) {
+        fwdIndexFileExtension = V1Constants.Indexes.SORTED_SV_FORWARD_INDEX_FILE_EXTENSION;
+      } else {
+        fwdIndexFileExtension = V1Constants.Indexes.UNSORTED_SV_FORWARD_INDEX_FILE_EXTENSION;
+      }
+    } else {
+      fwdIndexFileExtension = V1Constants.Indexes.UNSORTED_MV_FORWARD_INDEX_FILE_EXTENSION;
+    }
+    File fwdIndexFile = new File(indexDir, column + fwdIndexFileExtension);
+
+    if (!inProgress.exists()) {
+      // Marker file does not exist, which means last run ended normally.
+      // Create a marker file.
+      FileUtils.touch(inProgress);
+    } else {
+      // Marker file exists, which means last run was interrupted.
+      // Remove forward index and dictionary files if they exist.
+      FileUtils.deleteQuietly(fwdIndexFile);
+      FileUtils.deleteQuietly(dictionaryFile);
+    }
+
+    LOGGER.info("Creating a new dictionary for segment={} and column={}", segmentName, column);
+    SegmentDictionaryCreator dictionaryCreator = buildDictionary(column, existingColMetadata, segmentWriter);
+    LoaderUtils.writeIndexToV3Format(segmentWriter, column, dictionaryFile, ColumnIndexType.DICTIONARY);
+
+    LOGGER.info("Built dictionary. Rewriting dictionary enabled forward index for segment={} and column={}",
+        segmentName, column);
+    writeDictEnabledForwardIndex(column, existingColMetadata, segmentWriter, indexDir, indexCreatorProvider,
+        dictionaryCreator);
+    // We used the existing forward index to generate a new forward index. The existing forward index will be in V3
+    // format and the new forward index will be in V1 format. Remove the existing forward index as it is not needed
+    // anymore. Note that removeIndex() will only mark an index for removal and remove the in-memory state. The
+    // actual cleanup from columns.psf file will happen when singleFileIndexDirectory.cleanupRemovedIndices() is
+    // called during segmentWriter.close().
+    segmentWriter.removeIndex(column, ColumnIndexType.FORWARD_INDEX);
+    LoaderUtils.writeIndexToV3Format(segmentWriter, column, fwdIndexFile, ColumnIndexType.FORWARD_INDEX);
+
+    LOGGER.info("Created forwardIndex. Updating metadata properties for segment={} and column={}", segmentName, column);
+    updateMetadataProperties(indexDir, column, dictionaryCreator);
+
+    // We remove indexes that have to be rewritten when a dictEnabled is toggled. Note that the respective index
+    // handler will take care of recreating the index.
+    removeDictRelatedIndexes(column, segmentWriter);
+
+    // Delete the marker file.
+    FileUtils.deleteQuietly(inProgress);
+
+    LOGGER.info("Created dictionary based forward index for segment: {}, column: {}", segmentName, column);
+  }
+
+  private SegmentDictionaryCreator buildDictionary(String column, ColumnMetadata existingColMetadata,
+      SegmentDirectory.Writer segmentWriter)
+      throws Exception {
+    int numDocs = existingColMetadata.getTotalDocs();
+    // SegmentPartitionConfig is not relevant for rewrites.
+    Preconditions.checkState(_indexLoadingConfig.getTableConfig() != null);
+    StatsCollectorConfig statsCollectorConfig =
+        new StatsCollectorConfig(_indexLoadingConfig.getTableConfig(), _schema, null);
+    AbstractColumnStatisticsCollector statsCollector;
+
+    try (ForwardIndexReader reader = LoaderUtils.getForwardIndexReader(segmentWriter, existingColMetadata)) {
+      boolean isSVColumn = reader.isSingleValue();
+
+      switch (reader.getStoredType()) {
+        case INT:
+          statsCollector = new IntColumnPreIndexStatsCollector(column, statsCollectorConfig);
+          break;
+        case LONG:
+          statsCollector = new LongColumnPreIndexStatsCollector(column, statsCollectorConfig);
+          break;
+        case FLOAT:
+          statsCollector = new FloatColumnPreIndexStatsCollector(column, statsCollectorConfig);
+          break;
+        case DOUBLE:
+          statsCollector = new DoubleColumnPreIndexStatsCollector(column, statsCollectorConfig);
+          break;
+        case STRING:
+          statsCollector = new StringColumnPreIndexStatsCollector(column, statsCollectorConfig);
+          break;
+        case BYTES:
+          statsCollector = new BytesColumnPredIndexStatsCollector(column, statsCollectorConfig);
+          break;
+        case BIG_DECIMAL:
+          Preconditions.checkState(isSVColumn, "BigDecimal is not supported for MV columns");
+          statsCollector = new BigDecimalColumnPreIndexStatsCollector(column, statsCollectorConfig);
+          break;
+        default:
+          throw new IllegalStateException();
+      }
+
+      Preconditions.checkState(statsCollector != null);
+      // Note: Special Null handling is not necessary here. This is because, the existing default null value in the
+      // raw forwardIndex will be retained as such while created the dictionary and dict-based forward index. Also,
+      // null value vectors maintain a bitmap of docIds. No handling is necessary there.
+      PinotSegmentColumnReader columnReader =
+          new PinotSegmentColumnReader(reader, null, null, existingColMetadata.getMaxNumberOfMultiValues());
+      for (int i = 0; i < numDocs; i++) {
+        Object obj = columnReader.getValue(i);
+        statsCollector.collect(obj);
+      }
+      statsCollector.seal();
+
+      boolean useVarLength = _indexLoadingConfig.getVarLengthDictionaryColumns().contains(column)
+          || SegmentIndexCreationDriverImpl.shouldUseVarLengthDictionary(reader.getStoredType(), statsCollector);
+      SegmentDictionaryCreator dictionaryCreator =
+          new SegmentDictionaryCreator(existingColMetadata.getFieldSpec(), _segmentMetadata.getIndexDir(),
+              useVarLength);
+
+      dictionaryCreator.build(statsCollector.getUniqueValuesSet());
+      return dictionaryCreator;
+    }
+  }
+
+  private void writeDictEnabledForwardIndex(String column, ColumnMetadata existingColMetadata,
+      SegmentDirectory.Writer segmentWriter, File indexDir, IndexCreatorProvider indexCreatorProvider,
+      SegmentDictionaryCreator dictionaryCreator)
+      throws Exception {
+    try (ForwardIndexReader reader = LoaderUtils.getForwardIndexReader(segmentWriter, existingColMetadata)) {
+      int lengthOfLongestEntry = reader.getLengthOfLongestEntry();
+      IndexCreationContext.Builder builder =
+          IndexCreationContext.builder().withIndexDir(indexDir).withColumnMetadata(existingColMetadata)
+              .withLengthOfLongestEntry(lengthOfLongestEntry);
+      // existingColMetadata has dictEnable=false. Overwrite the value.
+      builder.withDictionary(true);
+      IndexCreationContext.Forward context =
+          builder.build().forForwardIndex(null, _indexLoadingConfig.getColumnProperties());
+
+      try (ForwardIndexCreator creator = indexCreatorProvider.newForwardIndexCreator(context)) {
+        int numDocs = existingColMetadata.getTotalDocs();
+        forwardIndexWriterHelper(column, existingColMetadata, reader, creator, numDocs, dictionaryCreator);
       }
-      default:
-        throw new IllegalStateException();
     }
   }
+
+  private void removeDictRelatedIndexes(String column, SegmentDirectory.Writer segmentWriter) {
+    // TODO: Move this logic as a static function in each index creator.
+    segmentWriter.removeIndex(column, ColumnIndexType.RANGE_INDEX);
+  }
+
+  private void updateMetadataProperties(File indexDir, String column, SegmentDictionaryCreator dictionaryCreator)

Review Comment:
   I recommend making this more generic. I too will have a need to update the metadata for enabling forward index on a forward index disabled column. I will want to set up different properties. Perhaps have it take up a set of input properties and set them all in a loop?
   
   Though feel free to leave this as is. I can also take up this change when I get to that 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.

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


[GitHub] [pinot] somandal commented on a diff in pull request #9678: Support creating dictionary at runtime for an existing column

Posted by GitBox <gi...@apache.org>.
somandal commented on code in PR #9678:
URL: https://github.com/apache/pinot/pull/9678#discussion_r1010667763


##########
pinot-segment-local/src/test/java/org/apache/pinot/segment/local/segment/index/loader/ForwardIndexHandlerTest.java:
##########
@@ -302,34 +351,54 @@ public void testComputeOperation()
 
     // TEST1 : Validate with zero changes. ForwardIndexHandler should be a No-Op.
     IndexLoadingConfig indexLoadingConfig = new IndexLoadingConfig(null, _tableConfig);
-    ForwardIndexHandler fwdIndexHandler = new ForwardIndexHandler(existingSegmentMetadata, indexLoadingConfig);
+    ForwardIndexHandler fwdIndexHandler = new ForwardIndexHandler(existingSegmentMetadata, indexLoadingConfig, null);
     Map<String, ForwardIndexHandler.Operation> operationMap = new HashMap<>();
     operationMap = fwdIndexHandler.computeOperation(writer);
     assertEquals(operationMap, Collections.EMPTY_MAP);
 
-    // TEST2: Enable dictionary for a RAW_ZSTANDARD_INDEX_COLUMN. ForwardIndexHandler should be a No-Op.
+    // TEST2: Enable dictionary for a RAW_ZSTANDARD_INDEX_COLUMN.
     indexLoadingConfig = new IndexLoadingConfig(null, _tableConfig);
     indexLoadingConfig.getNoDictionaryColumns().remove(DIM_ZSTANDARD_STRING);
-    fwdIndexHandler = new ForwardIndexHandler(existingSegmentMetadata, indexLoadingConfig);
+    fwdIndexHandler = new ForwardIndexHandler(existingSegmentMetadata, indexLoadingConfig, _schema);
+    operationMap = fwdIndexHandler.computeOperation(writer);
+    assertEquals(operationMap.get(DIM_ZSTANDARD_STRING), ForwardIndexHandler.Operation.ENABLE_DICTIONARY);
+
+    // TEST3: Enable dictionary for an MV column.
+    indexLoadingConfig = new IndexLoadingConfig(null, _tableConfig);
+    indexLoadingConfig.getNoDictionaryColumns().remove(DIM_MV_PASS_THROUGH_STRING);
+    fwdIndexHandler = new ForwardIndexHandler(existingSegmentMetadata, indexLoadingConfig, _schema);
+    operationMap = fwdIndexHandler.computeOperation(writer);
+    assertEquals(operationMap.get(DIM_MV_PASS_THROUGH_STRING), ForwardIndexHandler.Operation.ENABLE_DICTIONARY);
+
+    // TEST4: Enable dictionary for a sorted column.
+    indexLoadingConfig = new IndexLoadingConfig(null, _tableConfig);
+    indexLoadingConfig.getNoDictionaryColumns().remove(DIM_PASS_THROUGH_SORTED_LONG);
+    fwdIndexHandler = new ForwardIndexHandler(existingSegmentMetadata, indexLoadingConfig, _schema);
+    operationMap = fwdIndexHandler.computeOperation(writer);
+    assertEquals(operationMap.get(DIM_PASS_THROUGH_SORTED_LONG), ForwardIndexHandler.Operation.ENABLE_DICTIONARY);
+
+    // TEST5: Enable dictionary for a dict column. Should be a No-op.
+    indexLoadingConfig = new IndexLoadingConfig(null, _tableConfig);
+    fwdIndexHandler = new ForwardIndexHandler(existingSegmentMetadata, indexLoadingConfig, _schema);
     operationMap = fwdIndexHandler.computeOperation(writer);
     assertEquals(operationMap, Collections.EMPTY_MAP);
 
-    // TEST3: Disable dictionary. ForwardIndexHandler should be a No-Op.
+    // TEST6: Disable dictionary. Should be a No-op.
     indexLoadingConfig = new IndexLoadingConfig(null, _tableConfig);
     indexLoadingConfig.getNoDictionaryColumns().add(DIM_DICT_INTEGER);
-    fwdIndexHandler = new ForwardIndexHandler(existingSegmentMetadata, indexLoadingConfig);
+    fwdIndexHandler = new ForwardIndexHandler(existingSegmentMetadata, indexLoadingConfig, null);
     operationMap = fwdIndexHandler.computeOperation(writer);
     assertEquals(operationMap, Collections.EMPTY_MAP);
 
-    // TEST4: Add an additional text index. ForwardIndexHandler should be a No-Op.
+    // TEST7: Add an additional text index. ForwardIndexHandler should be a No-Op.
     indexLoadingConfig = new IndexLoadingConfig(null, _tableConfig);
     indexLoadingConfig.getTextIndexColumns().add(DIM_DICT_INTEGER);
     indexLoadingConfig.getTextIndexColumns().add(DIM_LZ4_INTEGER);
-    fwdIndexHandler = new ForwardIndexHandler(existingSegmentMetadata, indexLoadingConfig);
+    fwdIndexHandler = new ForwardIndexHandler(existingSegmentMetadata, indexLoadingConfig, null);

Review Comment:
   did you test adding a range index and enabling dictionary at the same time?



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


[GitHub] [pinot] vvivekiyer commented on a diff in pull request #9678: Support creating dictionary at runtime for an existing column

Posted by GitBox <gi...@apache.org>.
vvivekiyer commented on code in PR #9678:
URL: https://github.com/apache/pinot/pull/9678#discussion_r1011009045


##########
pinot-segment-local/src/test/java/org/apache/pinot/segment/local/segment/index/loader/ForwardIndexHandlerTest.java:
##########
@@ -302,34 +351,54 @@ public void testComputeOperation()
 
     // TEST1 : Validate with zero changes. ForwardIndexHandler should be a No-Op.
     IndexLoadingConfig indexLoadingConfig = new IndexLoadingConfig(null, _tableConfig);
-    ForwardIndexHandler fwdIndexHandler = new ForwardIndexHandler(existingSegmentMetadata, indexLoadingConfig);
+    ForwardIndexHandler fwdIndexHandler = new ForwardIndexHandler(existingSegmentMetadata, indexLoadingConfig, null);
     Map<String, ForwardIndexHandler.Operation> operationMap = new HashMap<>();
     operationMap = fwdIndexHandler.computeOperation(writer);
     assertEquals(operationMap, Collections.EMPTY_MAP);
 
-    // TEST2: Enable dictionary for a RAW_ZSTANDARD_INDEX_COLUMN. ForwardIndexHandler should be a No-Op.
+    // TEST2: Enable dictionary for a RAW_ZSTANDARD_INDEX_COLUMN.
     indexLoadingConfig = new IndexLoadingConfig(null, _tableConfig);
     indexLoadingConfig.getNoDictionaryColumns().remove(DIM_ZSTANDARD_STRING);
-    fwdIndexHandler = new ForwardIndexHandler(existingSegmentMetadata, indexLoadingConfig);
+    fwdIndexHandler = new ForwardIndexHandler(existingSegmentMetadata, indexLoadingConfig, _schema);
+    operationMap = fwdIndexHandler.computeOperation(writer);
+    assertEquals(operationMap.get(DIM_ZSTANDARD_STRING), ForwardIndexHandler.Operation.ENABLE_DICTIONARY);
+
+    // TEST3: Enable dictionary for an MV column.
+    indexLoadingConfig = new IndexLoadingConfig(null, _tableConfig);
+    indexLoadingConfig.getNoDictionaryColumns().remove(DIM_MV_PASS_THROUGH_STRING);
+    fwdIndexHandler = new ForwardIndexHandler(existingSegmentMetadata, indexLoadingConfig, _schema);
+    operationMap = fwdIndexHandler.computeOperation(writer);
+    assertEquals(operationMap.get(DIM_MV_PASS_THROUGH_STRING), ForwardIndexHandler.Operation.ENABLE_DICTIONARY);
+
+    // TEST4: Enable dictionary for a sorted column.
+    indexLoadingConfig = new IndexLoadingConfig(null, _tableConfig);
+    indexLoadingConfig.getNoDictionaryColumns().remove(DIM_PASS_THROUGH_SORTED_LONG);
+    fwdIndexHandler = new ForwardIndexHandler(existingSegmentMetadata, indexLoadingConfig, _schema);
+    operationMap = fwdIndexHandler.computeOperation(writer);
+    assertEquals(operationMap.get(DIM_PASS_THROUGH_SORTED_LONG), ForwardIndexHandler.Operation.ENABLE_DICTIONARY);
+
+    // TEST5: Enable dictionary for a dict column. Should be a No-op.
+    indexLoadingConfig = new IndexLoadingConfig(null, _tableConfig);
+    fwdIndexHandler = new ForwardIndexHandler(existingSegmentMetadata, indexLoadingConfig, _schema);
     operationMap = fwdIndexHandler.computeOperation(writer);
     assertEquals(operationMap, Collections.EMPTY_MAP);
 
-    // TEST3: Disable dictionary. ForwardIndexHandler should be a No-Op.
+    // TEST6: Disable dictionary. Should be a No-op.
     indexLoadingConfig = new IndexLoadingConfig(null, _tableConfig);
     indexLoadingConfig.getNoDictionaryColumns().add(DIM_DICT_INTEGER);
-    fwdIndexHandler = new ForwardIndexHandler(existingSegmentMetadata, indexLoadingConfig);
+    fwdIndexHandler = new ForwardIndexHandler(existingSegmentMetadata, indexLoadingConfig, null);
     operationMap = fwdIndexHandler.computeOperation(writer);
     assertEquals(operationMap, Collections.EMPTY_MAP);
 
-    // TEST4: Add an additional text index. ForwardIndexHandler should be a No-Op.
+    // TEST7: Add an additional text index. ForwardIndexHandler should be a No-Op.
     indexLoadingConfig = new IndexLoadingConfig(null, _tableConfig);
     indexLoadingConfig.getTextIndexColumns().add(DIM_DICT_INTEGER);
     indexLoadingConfig.getTextIndexColumns().add(DIM_LZ4_INTEGER);
-    fwdIndexHandler = new ForwardIndexHandler(existingSegmentMetadata, indexLoadingConfig);
+    fwdIndexHandler = new ForwardIndexHandler(existingSegmentMetadata, indexLoadingConfig, null);

Review Comment:
   Tested in SegmentPreProcessorTest. Also added a test 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


[GitHub] [pinot] siddharthteotia commented on a diff in pull request #9678: Support creating dictionary at runtime for an existing column

Posted by GitBox <gi...@apache.org>.
siddharthteotia commented on code in PR #9678:
URL: https://github.com/apache/pinot/pull/9678#discussion_r1012286332


##########
pinot-core/src/test/java/org/apache/pinot/queries/RangeQueriesTest.java:
##########
@@ -223,6 +223,42 @@ public void testSelectionOverRangeFilter(String query, int min, int max, boolean
     }
   }
 
+  @Test(dataProvider = "selectionTestCases")
+  public void testSelectionOverRangeFilterAfterReload(String query, int min, int max, boolean inclusive)

Review Comment:
   Couple of types for both SV and MV should be considered



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


[GitHub] [pinot] siddharthteotia commented on a diff in pull request #9678: Support creating dictionary at runtime for an existing column

Posted by GitBox <gi...@apache.org>.
siddharthteotia commented on code in PR #9678:
URL: https://github.com/apache/pinot/pull/9678#discussion_r1011041747


##########
pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/loader/ForwardIndexHandler.java:
##########
@@ -127,7 +151,15 @@ Map<String, Operation> computeOperation(SegmentDirectory.Reader segmentReader)
     Set<String> newNoDictColumns = _indexLoadingConfig.getNoDictionaryColumns();
 
     for (String column : existingAllColumns) {
-      if (existingNoDictColumns.contains(column) && newNoDictColumns.contains(column)) {
+      if (existingNoDictColumns.contains(column) && !newNoDictColumns.contains(column)) {
+        if (_schema == null || _indexLoadingConfig.getTableConfig() == null) {
+          // This can only happen in tests.

Review Comment:
   why should this happen in tests ? I feel this should be a preconditions check 



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


[GitHub] [pinot] somandal commented on a diff in pull request #9678: Support creating dictionary at runtime for an existing column

Posted by GitBox <gi...@apache.org>.
somandal commented on code in PR #9678:
URL: https://github.com/apache/pinot/pull/9678#discussion_r1012192867


##########
pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/loader/ForwardIndexHandler.java:
##########
@@ -271,118 +304,295 @@ private void rewriteRawSVForwardIndex(String column, ColumnMetadata existingColM
               .forForwardIndex(newCompressionType, _indexLoadingConfig.getColumnProperties());
 
       try (ForwardIndexCreator creator = indexCreatorProvider.newForwardIndexCreator(context)) {
-        // If creator stored type and the reader stored type do not match, throw an exception.
         if (!reader.getStoredType().equals(creator.getValueType())) {
+          // Creator stored type should match reader stored type for raw columns. We do not support changing datatypes.
           String failureMsg =
               "Unsupported operation to change datatype for column=" + column + " from " + reader.getStoredType()
                   .toString() + " to " + creator.getValueType().toString();
           throw new UnsupportedOperationException(failureMsg);
         }
 
         int numDocs = existingColMetadata.getTotalDocs();
-        forwardIndexWriterHelper(column, reader, creator, numDocs);
+        forwardIndexWriterHelper(column, existingColMetadata, reader, creator, numDocs, null);
       }
     }
   }
 
-  private void forwardIndexWriterHelper(String column, ForwardIndexReader reader, ForwardIndexCreator creator,
-      int numDocs) {
-    // If creator stored type should match reader stored type. We do not support changing datatypes.
-    if (!reader.getStoredType().equals(creator.getValueType())) {
-      String failureMsg =
-          "Unsupported operation to change datatype for column=" + column + " from " + reader.getStoredType().toString()
-              + " to " + creator.getValueType().toString();
-      throw new UnsupportedOperationException(failureMsg);
-    }
-
+  private void forwardIndexWriterHelper(String column, ColumnMetadata existingColumnMetadata, ForwardIndexReader reader,
+      ForwardIndexCreator creator, int numDocs, @Nullable SegmentDictionaryCreator dictionaryCreator) {
     ForwardIndexReaderContext readerContext = reader.createContext();
     boolean isSVColumn = reader.isSingleValue();
 
-    switch (reader.getStoredType()) {
-      // JSON fields are either stored as string or bytes. No special handling is needed because we make this
-      // decision based on the storedType of the reader.
-      case INT: {
-        for (int i = 0; i < numDocs; i++) {
-          if (isSVColumn) {
-            int val = reader.getInt(i, readerContext);
-            creator.putInt(val);
-          } else {
-            int[] ints = reader.getIntMV(i, readerContext);
-            creator.putIntMV(ints);
-          }
+    if (dictionaryCreator != null) {
+      int maxNumValuesPerEntry = existingColumnMetadata.getMaxNumberOfMultiValues();
+      PinotSegmentColumnReader columnReader = new PinotSegmentColumnReader(reader, null, null, maxNumValuesPerEntry);
+
+      for (int i = 0; i < numDocs; i++) {
+        Object obj = columnReader.getValue(i);
+
+        if (isSVColumn) {
+          int dictId = dictionaryCreator.indexOfSV(obj);
+          creator.putDictId(dictId);
+        } else {
+          int[] dictIds = dictionaryCreator.indexOfMV(obj);
+          creator.putDictIdMV(dictIds);
         }
-        break;
       }
-      case LONG: {
-        for (int i = 0; i < numDocs; i++) {
-          if (isSVColumn) {
-            long val = reader.getLong(i, readerContext);
-            creator.putLong(val);
-          } else {
-            long[] longs = reader.getLongMV(i, readerContext);
-            creator.putLongMV(longs);
+    } else {
+      switch (reader.getStoredType()) {
+        // JSON fields are either stored as string or bytes. No special handling is needed because we make this
+        // decision based on the storedType of the reader.
+        case INT: {
+          for (int i = 0; i < numDocs; i++) {
+            if (isSVColumn) {
+              int val = reader.getInt(i, readerContext);
+              creator.putInt(val);
+            } else {
+              int[] ints = reader.getIntMV(i, readerContext);
+              creator.putIntMV(ints);
+            }
           }
+          break;
         }
-        break;
-      }
-      case FLOAT: {
-        for (int i = 0; i < numDocs; i++) {
-          if (isSVColumn) {
-            float val = reader.getFloat(i, readerContext);
-            creator.putFloat(val);
-          } else {
-            float[] floats = reader.getFloatMV(i, readerContext);
-            creator.putFloatMV(floats);
+        case LONG: {
+          for (int i = 0; i < numDocs; i++) {
+            if (isSVColumn) {
+              long val = reader.getLong(i, readerContext);
+              creator.putLong(val);
+            } else {
+              long[] longs = reader.getLongMV(i, readerContext);
+              creator.putLongMV(longs);
+            }
           }
+          break;
         }
-        break;
-      }
-      case DOUBLE: {
-        for (int i = 0; i < numDocs; i++) {
-          if (isSVColumn) {
-            double val = reader.getDouble(i, readerContext);
-            creator.putDouble(val);
-          } else {
-            double[] doubles = reader.getDoubleMV(i, readerContext);
-            creator.putDoubleMV(doubles);
+        case FLOAT: {
+          for (int i = 0; i < numDocs; i++) {
+            if (isSVColumn) {
+              float val = reader.getFloat(i, readerContext);
+              creator.putFloat(val);
+            } else {
+              float[] floats = reader.getFloatMV(i, readerContext);
+              creator.putFloatMV(floats);
+            }
           }
+          break;
         }
-        break;
-      }
-      case STRING: {
-        for (int i = 0; i < numDocs; i++) {
-          if (isSVColumn) {
-            String val = reader.getString(i, readerContext);
-            creator.putString(val);
-          } else {
-            String[] strings = reader.getStringMV(i, readerContext);
-            creator.putStringMV(strings);
+        case DOUBLE: {
+          for (int i = 0; i < numDocs; i++) {
+            if (isSVColumn) {
+              double val = reader.getDouble(i, readerContext);
+              creator.putDouble(val);
+            } else {
+              double[] doubles = reader.getDoubleMV(i, readerContext);
+              creator.putDoubleMV(doubles);
+            }
           }
+          break;
         }
-        break;
-      }
-      case BYTES: {
-        for (int i = 0; i < numDocs; i++) {
-          if (isSVColumn) {
-            byte[] val = reader.getBytes(i, readerContext);
-            creator.putBytes(val);
-          } else {
-            byte[][] bytesArray = reader.getBytesMV(i, readerContext);
-            creator.putBytesMV(bytesArray);
+        case STRING: {
+          for (int i = 0; i < numDocs; i++) {
+            if (isSVColumn) {
+              String val = reader.getString(i, readerContext);
+              creator.putString(val);
+            } else {
+              String[] strings = reader.getStringMV(i, readerContext);
+              creator.putStringMV(strings);
+            }
           }
+          break;
         }
-        break;
-      }
-      case BIG_DECIMAL: {
-        for (int i = 0; i < numDocs; i++) {
+        case BYTES: {
+          for (int i = 0; i < numDocs; i++) {
+            if (isSVColumn) {
+              byte[] val = reader.getBytes(i, readerContext);
+              creator.putBytes(val);
+            } else {
+              byte[][] bytesArray = reader.getBytesMV(i, readerContext);
+              creator.putBytesMV(bytesArray);
+            }
+          }
+          break;
+        }
+        case BIG_DECIMAL: {
           Preconditions.checkState(isSVColumn, "BigDecimal is not supported for MV columns");
-          BigDecimal val = reader.getBigDecimal(i, readerContext);
-          creator.putBigDecimal(val);
+          for (int i = 0; i < numDocs; i++) {
+            BigDecimal val = reader.getBigDecimal(i, readerContext);
+            creator.putBigDecimal(val);
+          }
+          break;
         }
-        break;
+        default:
+          throw new IllegalStateException();
+      }
+    }
+  }
+
+  private void enableDictionary(String column, SegmentDirectory.Writer segmentWriter,
+      IndexCreatorProvider indexCreatorProvider)
+      throws Exception {
+    Preconditions.checkState(_segmentMetadata.getVersion() == SegmentVersion.v3);
+    ColumnMetadata existingColMetadata = _segmentMetadata.getColumnMetadataFor(column);
+    Preconditions.checkState(!existingColMetadata.hasDictionary(),
+        "Cannot rewrite dictionary enabled forward index. Dictionary already exists for column:" + column);
+    boolean isSingleValue = existingColMetadata.isSingleValue();
+
+    File indexDir = _segmentMetadata.getIndexDir();
+    String segmentName = _segmentMetadata.getName();
+    File inProgress = new File(indexDir, column + ".dict.inprogress");
+    File dictionaryFile = new File(indexDir, column + V1Constants.Dict.FILE_EXTENSION);
+    String fwdIndexFileExtension;
+    if (isSingleValue) {
+      fwdIndexFileExtension =
+          existingColMetadata.isSorted() ? V1Constants.Indexes.SORTED_SV_FORWARD_INDEX_FILE_EXTENSION

Review Comment:
   What's the behavior for sorted columns on the segment creation code path? How does it interact with the noDict columns list? The behavior should be consistent IMO. So if that code path allows creation of raw forward index for sorted columns then this code path should do. If not, I'd recommend tacking adding support for raw forward index for sorted columns as a separate change and tackle both segment creation and reload code paths. Wdyt?



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


[GitHub] [pinot] siddharthteotia commented on pull request #9678: Support creating dictionary at runtime for an existing column

Posted by GitBox <gi...@apache.org>.
siddharthteotia commented on PR #9678:
URL: https://github.com/apache/pinot/pull/9678#issuecomment-1301212549

   Decided to merge this in the interest of unblocking @somandal 's reload support work on https://github.com/apache/pinot/issues/6473
   
   Main work is done here. @vvivekiyer is going to follow-up immediately with
   
   - More tests (Called out above)
   - Sorted column handling check


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


[GitHub] [pinot] vvivekiyer commented on a diff in pull request #9678: Support creating dictionary at runtime for an existing column

Posted by GitBox <gi...@apache.org>.
vvivekiyer commented on code in PR #9678:
URL: https://github.com/apache/pinot/pull/9678#discussion_r1011006867


##########
pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/loader/ForwardIndexHandler.java:
##########
@@ -271,118 +306,293 @@ private void rewriteRawSVForwardIndex(String column, ColumnMetadata existingColM
               .forForwardIndex(newCompressionType, _indexLoadingConfig.getColumnProperties());
 
       try (ForwardIndexCreator creator = indexCreatorProvider.newForwardIndexCreator(context)) {
-        // If creator stored type and the reader stored type do not match, throw an exception.
         if (!reader.getStoredType().equals(creator.getValueType())) {
+          // Creator stored type should match reader stored type for raw columns. We do not support changing datatypes.
           String failureMsg =
               "Unsupported operation to change datatype for column=" + column + " from " + reader.getStoredType()
                   .toString() + " to " + creator.getValueType().toString();
           throw new UnsupportedOperationException(failureMsg);
         }
 
         int numDocs = existingColMetadata.getTotalDocs();
-        forwardIndexWriterHelper(column, reader, creator, numDocs);
+        forwardIndexWriterHelper(column, existingColMetadata, reader, creator, numDocs, null);
       }
     }
   }
 
-  private void forwardIndexWriterHelper(String column, ForwardIndexReader reader, ForwardIndexCreator creator,
-      int numDocs) {
-    // If creator stored type should match reader stored type. We do not support changing datatypes.
-    if (!reader.getStoredType().equals(creator.getValueType())) {
-      String failureMsg =
-          "Unsupported operation to change datatype for column=" + column + " from " + reader.getStoredType().toString()
-              + " to " + creator.getValueType().toString();
-      throw new UnsupportedOperationException(failureMsg);
-    }
-
+  private void forwardIndexWriterHelper(String column, ColumnMetadata existingColumnMetadata, ForwardIndexReader reader,
+      ForwardIndexCreator creator, int numDocs, @Nullable SegmentDictionaryCreator dictionaryCreator) {
     ForwardIndexReaderContext readerContext = reader.createContext();
     boolean isSVColumn = reader.isSingleValue();
 
-    switch (reader.getStoredType()) {
-      // JSON fields are either stored as string or bytes. No special handling is needed because we make this
-      // decision based on the storedType of the reader.
-      case INT: {
-        for (int i = 0; i < numDocs; i++) {
-          if (isSVColumn) {
-            int val = reader.getInt(i, readerContext);
-            creator.putInt(val);
-          } else {
-            int[] ints = reader.getIntMV(i, readerContext);
-            creator.putIntMV(ints);
-          }
+    if (dictionaryCreator != null) {
+      int maxNumValuesPerEntry = existingColumnMetadata.getMaxNumberOfMultiValues();
+      PinotSegmentColumnReader columnReader = new PinotSegmentColumnReader(reader, null, null, maxNumValuesPerEntry);
+
+      for (int i = 0; i < numDocs; i++) {
+        Object obj = columnReader.getValue(i);
+
+        if (isSVColumn) {
+          int dictId = dictionaryCreator.indexOfSV(obj);
+          creator.putDictId(dictId);
+        } else {
+          int[] dictIds = dictionaryCreator.indexOfMV(obj);
+          creator.putDictIdMV(dictIds);
         }
-        break;
       }
-      case LONG: {
-        for (int i = 0; i < numDocs; i++) {
-          if (isSVColumn) {
-            long val = reader.getLong(i, readerContext);
-            creator.putLong(val);
-          } else {
-            long[] longs = reader.getLongMV(i, readerContext);
-            creator.putLongMV(longs);
+    } else {
+      switch (reader.getStoredType()) {
+        // JSON fields are either stored as string or bytes. No special handling is needed because we make this
+        // decision based on the storedType of the reader.
+        case INT: {
+          for (int i = 0; i < numDocs; i++) {
+            if (isSVColumn) {
+              int val = reader.getInt(i, readerContext);
+              creator.putInt(val);
+            } else {
+              int[] ints = reader.getIntMV(i, readerContext);
+              creator.putIntMV(ints);
+            }
           }
+          break;
         }
-        break;
-      }
-      case FLOAT: {
-        for (int i = 0; i < numDocs; i++) {
-          if (isSVColumn) {
-            float val = reader.getFloat(i, readerContext);
-            creator.putFloat(val);
-          } else {
-            float[] floats = reader.getFloatMV(i, readerContext);
-            creator.putFloatMV(floats);
+        case LONG: {
+          for (int i = 0; i < numDocs; i++) {
+            if (isSVColumn) {
+              long val = reader.getLong(i, readerContext);
+              creator.putLong(val);
+            } else {
+              long[] longs = reader.getLongMV(i, readerContext);
+              creator.putLongMV(longs);
+            }
           }
+          break;
         }
-        break;
-      }
-      case DOUBLE: {
-        for (int i = 0; i < numDocs; i++) {
-          if (isSVColumn) {
-            double val = reader.getDouble(i, readerContext);
-            creator.putDouble(val);
-          } else {
-            double[] doubles = reader.getDoubleMV(i, readerContext);
-            creator.putDoubleMV(doubles);
+        case FLOAT: {
+          for (int i = 0; i < numDocs; i++) {
+            if (isSVColumn) {
+              float val = reader.getFloat(i, readerContext);
+              creator.putFloat(val);
+            } else {
+              float[] floats = reader.getFloatMV(i, readerContext);
+              creator.putFloatMV(floats);
+            }
           }
+          break;
         }
-        break;
-      }
-      case STRING: {
-        for (int i = 0; i < numDocs; i++) {
-          if (isSVColumn) {
-            String val = reader.getString(i, readerContext);
-            creator.putString(val);
-          } else {
-            String[] strings = reader.getStringMV(i, readerContext);
-            creator.putStringMV(strings);
+        case DOUBLE: {
+          for (int i = 0; i < numDocs; i++) {
+            if (isSVColumn) {
+              double val = reader.getDouble(i, readerContext);
+              creator.putDouble(val);
+            } else {
+              double[] doubles = reader.getDoubleMV(i, readerContext);
+              creator.putDoubleMV(doubles);
+            }
           }
+          break;
         }
-        break;
-      }
-      case BYTES: {
-        for (int i = 0; i < numDocs; i++) {
-          if (isSVColumn) {
-            byte[] val = reader.getBytes(i, readerContext);
-            creator.putBytes(val);
-          } else {
-            byte[][] bytesArray = reader.getBytesMV(i, readerContext);
-            creator.putBytesMV(bytesArray);
+        case STRING: {
+          for (int i = 0; i < numDocs; i++) {
+            if (isSVColumn) {
+              String val = reader.getString(i, readerContext);
+              creator.putString(val);
+            } else {
+              String[] strings = reader.getStringMV(i, readerContext);
+              creator.putStringMV(strings);
+            }
           }
+          break;
         }
-        break;
-      }
-      case BIG_DECIMAL: {
-        for (int i = 0; i < numDocs; i++) {
+        case BYTES: {
+          for (int i = 0; i < numDocs; i++) {
+            if (isSVColumn) {
+              byte[] val = reader.getBytes(i, readerContext);
+              creator.putBytes(val);
+            } else {
+              byte[][] bytesArray = reader.getBytesMV(i, readerContext);
+              creator.putBytesMV(bytesArray);
+            }
+          }
+          break;
+        }
+        case BIG_DECIMAL: {
           Preconditions.checkState(isSVColumn, "BigDecimal is not supported for MV columns");
-          BigDecimal val = reader.getBigDecimal(i, readerContext);
-          creator.putBigDecimal(val);
+          for (int i = 0; i < numDocs; i++) {
+            BigDecimal val = reader.getBigDecimal(i, readerContext);
+            creator.putBigDecimal(val);
+          }
+          break;
         }
-        break;
+        default:
+          throw new IllegalStateException();
+      }
+    }
+  }
+
+  private void enableDictionary(String column, SegmentDirectory.Writer segmentWriter,
+      IndexCreatorProvider indexCreatorProvider)
+      throws Exception {
+    Preconditions.checkState(_segmentMetadata.getVersion() == SegmentVersion.v3);

Review Comment:
   We already have an "if" condition where we avoid computing a valid operation if segment version is < v3. 
   
   I have these checks only in the entry functions: `rewriteRawForwardIndex`, `enableDictionary` and `disableDictionary`. My reasoning for keeping it is 2 fold:
   1. Looking at the check, it's immediately evident for the reader that this function works only for v3.
   2. If someone decides to write a test and directly call this function, this would immediately fire.



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


[GitHub] [pinot] siddharthteotia commented on a diff in pull request #9678: Support creating dictionary at runtime for an existing column

Posted by GitBox <gi...@apache.org>.
siddharthteotia commented on code in PR #9678:
URL: https://github.com/apache/pinot/pull/9678#discussion_r1011047656


##########
pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/loader/ForwardIndexHandler.java:
##########
@@ -271,118 +304,295 @@ private void rewriteRawSVForwardIndex(String column, ColumnMetadata existingColM
               .forForwardIndex(newCompressionType, _indexLoadingConfig.getColumnProperties());
 
       try (ForwardIndexCreator creator = indexCreatorProvider.newForwardIndexCreator(context)) {
-        // If creator stored type and the reader stored type do not match, throw an exception.
         if (!reader.getStoredType().equals(creator.getValueType())) {
+          // Creator stored type should match reader stored type for raw columns. We do not support changing datatypes.
           String failureMsg =
               "Unsupported operation to change datatype for column=" + column + " from " + reader.getStoredType()
                   .toString() + " to " + creator.getValueType().toString();
           throw new UnsupportedOperationException(failureMsg);
         }
 
         int numDocs = existingColMetadata.getTotalDocs();
-        forwardIndexWriterHelper(column, reader, creator, numDocs);
+        forwardIndexWriterHelper(column, existingColMetadata, reader, creator, numDocs, null);
       }
     }
   }
 
-  private void forwardIndexWriterHelper(String column, ForwardIndexReader reader, ForwardIndexCreator creator,
-      int numDocs) {
-    // If creator stored type should match reader stored type. We do not support changing datatypes.
-    if (!reader.getStoredType().equals(creator.getValueType())) {
-      String failureMsg =
-          "Unsupported operation to change datatype for column=" + column + " from " + reader.getStoredType().toString()
-              + " to " + creator.getValueType().toString();
-      throw new UnsupportedOperationException(failureMsg);
-    }
-
+  private void forwardIndexWriterHelper(String column, ColumnMetadata existingColumnMetadata, ForwardIndexReader reader,
+      ForwardIndexCreator creator, int numDocs, @Nullable SegmentDictionaryCreator dictionaryCreator) {
     ForwardIndexReaderContext readerContext = reader.createContext();
     boolean isSVColumn = reader.isSingleValue();
 
-    switch (reader.getStoredType()) {
-      // JSON fields are either stored as string or bytes. No special handling is needed because we make this
-      // decision based on the storedType of the reader.
-      case INT: {
-        for (int i = 0; i < numDocs; i++) {
-          if (isSVColumn) {
-            int val = reader.getInt(i, readerContext);
-            creator.putInt(val);
-          } else {
-            int[] ints = reader.getIntMV(i, readerContext);
-            creator.putIntMV(ints);
-          }
+    if (dictionaryCreator != null) {
+      int maxNumValuesPerEntry = existingColumnMetadata.getMaxNumberOfMultiValues();
+      PinotSegmentColumnReader columnReader = new PinotSegmentColumnReader(reader, null, null, maxNumValuesPerEntry);
+
+      for (int i = 0; i < numDocs; i++) {
+        Object obj = columnReader.getValue(i);
+
+        if (isSVColumn) {
+          int dictId = dictionaryCreator.indexOfSV(obj);
+          creator.putDictId(dictId);
+        } else {
+          int[] dictIds = dictionaryCreator.indexOfMV(obj);
+          creator.putDictIdMV(dictIds);
         }
-        break;
       }
-      case LONG: {
-        for (int i = 0; i < numDocs; i++) {
-          if (isSVColumn) {
-            long val = reader.getLong(i, readerContext);
-            creator.putLong(val);
-          } else {
-            long[] longs = reader.getLongMV(i, readerContext);
-            creator.putLongMV(longs);
+    } else {
+      switch (reader.getStoredType()) {
+        // JSON fields are either stored as string or bytes. No special handling is needed because we make this
+        // decision based on the storedType of the reader.
+        case INT: {
+          for (int i = 0; i < numDocs; i++) {
+            if (isSVColumn) {
+              int val = reader.getInt(i, readerContext);
+              creator.putInt(val);
+            } else {
+              int[] ints = reader.getIntMV(i, readerContext);
+              creator.putIntMV(ints);
+            }
           }
+          break;
         }
-        break;
-      }
-      case FLOAT: {
-        for (int i = 0; i < numDocs; i++) {
-          if (isSVColumn) {
-            float val = reader.getFloat(i, readerContext);
-            creator.putFloat(val);
-          } else {
-            float[] floats = reader.getFloatMV(i, readerContext);
-            creator.putFloatMV(floats);
+        case LONG: {
+          for (int i = 0; i < numDocs; i++) {
+            if (isSVColumn) {
+              long val = reader.getLong(i, readerContext);
+              creator.putLong(val);
+            } else {
+              long[] longs = reader.getLongMV(i, readerContext);
+              creator.putLongMV(longs);
+            }
           }
+          break;
         }
-        break;
-      }
-      case DOUBLE: {
-        for (int i = 0; i < numDocs; i++) {
-          if (isSVColumn) {
-            double val = reader.getDouble(i, readerContext);
-            creator.putDouble(val);
-          } else {
-            double[] doubles = reader.getDoubleMV(i, readerContext);
-            creator.putDoubleMV(doubles);
+        case FLOAT: {
+          for (int i = 0; i < numDocs; i++) {
+            if (isSVColumn) {
+              float val = reader.getFloat(i, readerContext);
+              creator.putFloat(val);
+            } else {
+              float[] floats = reader.getFloatMV(i, readerContext);
+              creator.putFloatMV(floats);
+            }
           }
+          break;
         }
-        break;
-      }
-      case STRING: {
-        for (int i = 0; i < numDocs; i++) {
-          if (isSVColumn) {
-            String val = reader.getString(i, readerContext);
-            creator.putString(val);
-          } else {
-            String[] strings = reader.getStringMV(i, readerContext);
-            creator.putStringMV(strings);
+        case DOUBLE: {
+          for (int i = 0; i < numDocs; i++) {
+            if (isSVColumn) {
+              double val = reader.getDouble(i, readerContext);
+              creator.putDouble(val);
+            } else {
+              double[] doubles = reader.getDoubleMV(i, readerContext);
+              creator.putDoubleMV(doubles);
+            }
           }
+          break;
         }
-        break;
-      }
-      case BYTES: {
-        for (int i = 0; i < numDocs; i++) {
-          if (isSVColumn) {
-            byte[] val = reader.getBytes(i, readerContext);
-            creator.putBytes(val);
-          } else {
-            byte[][] bytesArray = reader.getBytesMV(i, readerContext);
-            creator.putBytesMV(bytesArray);
+        case STRING: {
+          for (int i = 0; i < numDocs; i++) {
+            if (isSVColumn) {
+              String val = reader.getString(i, readerContext);
+              creator.putString(val);
+            } else {
+              String[] strings = reader.getStringMV(i, readerContext);
+              creator.putStringMV(strings);
+            }
           }
+          break;
         }
-        break;
-      }
-      case BIG_DECIMAL: {
-        for (int i = 0; i < numDocs; i++) {
+        case BYTES: {
+          for (int i = 0; i < numDocs; i++) {
+            if (isSVColumn) {
+              byte[] val = reader.getBytes(i, readerContext);
+              creator.putBytes(val);
+            } else {
+              byte[][] bytesArray = reader.getBytesMV(i, readerContext);
+              creator.putBytesMV(bytesArray);
+            }
+          }
+          break;
+        }
+        case BIG_DECIMAL: {
           Preconditions.checkState(isSVColumn, "BigDecimal is not supported for MV columns");
-          BigDecimal val = reader.getBigDecimal(i, readerContext);
-          creator.putBigDecimal(val);
+          for (int i = 0; i < numDocs; i++) {
+            BigDecimal val = reader.getBigDecimal(i, readerContext);
+            creator.putBigDecimal(val);
+          }
+          break;
         }
-        break;
+        default:
+          throw new IllegalStateException();
+      }
+    }
+  }
+
+  private void enableDictionary(String column, SegmentDirectory.Writer segmentWriter,
+      IndexCreatorProvider indexCreatorProvider)
+      throws Exception {
+    Preconditions.checkState(_segmentMetadata.getVersion() == SegmentVersion.v3);
+    ColumnMetadata existingColMetadata = _segmentMetadata.getColumnMetadataFor(column);
+    Preconditions.checkState(!existingColMetadata.hasDictionary(),
+        "Cannot rewrite dictionary enabled forward index. Dictionary already exists for column:" + column);
+    boolean isSingleValue = existingColMetadata.isSingleValue();
+
+    File indexDir = _segmentMetadata.getIndexDir();
+    String segmentName = _segmentMetadata.getName();
+    File inProgress = new File(indexDir, column + ".dict.inprogress");
+    File dictionaryFile = new File(indexDir, column + V1Constants.Dict.FILE_EXTENSION);
+    String fwdIndexFileExtension;
+    if (isSingleValue) {
+      fwdIndexFileExtension =
+          existingColMetadata.isSorted() ? V1Constants.Indexes.SORTED_SV_FORWARD_INDEX_FILE_EXTENSION
+              : V1Constants.Indexes.UNSORTED_SV_FORWARD_INDEX_FILE_EXTENSION;
+    } else {
+      fwdIndexFileExtension = V1Constants.Indexes.UNSORTED_MV_FORWARD_INDEX_FILE_EXTENSION;
+    }
+    File fwdIndexFile = new File(indexDir, column + fwdIndexFileExtension);
+
+    if (!inProgress.exists()) {
+      // Marker file does not exist, which means last run ended normally.
+      // Create a marker file.
+      FileUtils.touch(inProgress);
+    } else {
+      // Marker file exists, which means last run was interrupted.
+      // Remove forward index and dictionary files if they exist.
+      FileUtils.deleteQuietly(fwdIndexFile);
+      FileUtils.deleteQuietly(dictionaryFile);
+    }
+
+    LOGGER.info("Creating a new dictionary for segment={} and column={}", segmentName, column);
+    SegmentDictionaryCreator dictionaryCreator = buildDictionary(column, existingColMetadata, segmentWriter);
+    LoaderUtils.writeIndexToV3Format(segmentWriter, column, dictionaryFile, ColumnIndexType.DICTIONARY);
+
+    LOGGER.info("Built dictionary. Rewriting dictionary enabled forward index for segment={} and column={}",
+        segmentName, column);
+    writeDictEnabledForwardIndex(column, existingColMetadata, segmentWriter, indexDir, indexCreatorProvider,
+        dictionaryCreator);
+    // We used the existing forward index to generate a new forward index. The existing forward index will be in V3
+    // format and the new forward index will be in V1 format. Remove the existing forward index as it is not needed
+    // anymore. Note that removeIndex() will only mark an index for removal and remove the in-memory state. The
+    // actual cleanup from columns.psf file will happen when singleFileIndexDirectory.cleanupRemovedIndices() is
+    // called during segmentWriter.close().
+    segmentWriter.removeIndex(column, ColumnIndexType.FORWARD_INDEX);
+    LoaderUtils.writeIndexToV3Format(segmentWriter, column, fwdIndexFile, ColumnIndexType.FORWARD_INDEX);
+
+    LOGGER.info("Created forwardIndex. Updating metadata properties for segment={} and column={}", segmentName, column);
+    Map<String, String> metadataProperties = new HashMap<>();
+    metadataProperties.put(getKeyFor(column, HAS_DICTIONARY), String.valueOf(true));
+    metadataProperties.put(getKeyFor(column, DICTIONARY_ELEMENT_SIZE),
+        String.valueOf(dictionaryCreator.getNumBytesPerEntry()));
+    updateMetadataProperties(indexDir, metadataProperties);
+
+    // We remove indexes that have to be rewritten when a dictEnabled is toggled. Note that the respective index
+    // handler will take care of recreating the index.
+    removeDictRelatedIndexes(column, segmentWriter);
+
+    // Delete the marker file.
+    FileUtils.deleteQuietly(inProgress);
+
+    LOGGER.info("Created dictionary based forward index for segment: {}, column: {}", segmentName, column);
+  }
+
+  private SegmentDictionaryCreator buildDictionary(String column, ColumnMetadata existingColMetadata,
+      SegmentDirectory.Writer segmentWriter)
+      throws Exception {
+    int numDocs = existingColMetadata.getTotalDocs();
+    // SegmentPartitionConfig is not relevant for rewrites.
+    StatsCollectorConfig statsCollectorConfig =
+        new StatsCollectorConfig(_indexLoadingConfig.getTableConfig(), _schema, null);
+    AbstractColumnStatisticsCollector statsCollector;
+
+    try (ForwardIndexReader reader = LoaderUtils.getForwardIndexReader(segmentWriter, existingColMetadata)) {
+      boolean isSVColumn = reader.isSingleValue();
+
+      switch (reader.getStoredType()) {
+        case INT:
+          statsCollector = new IntColumnPreIndexStatsCollector(column, statsCollectorConfig);
+          break;
+        case LONG:
+          statsCollector = new LongColumnPreIndexStatsCollector(column, statsCollectorConfig);
+          break;
+        case FLOAT:
+          statsCollector = new FloatColumnPreIndexStatsCollector(column, statsCollectorConfig);
+          break;
+        case DOUBLE:
+          statsCollector = new DoubleColumnPreIndexStatsCollector(column, statsCollectorConfig);
+          break;
+        case STRING:
+          statsCollector = new StringColumnPreIndexStatsCollector(column, statsCollectorConfig);
+          break;
+        case BYTES:
+          statsCollector = new BytesColumnPredIndexStatsCollector(column, statsCollectorConfig);
+          break;
+        case BIG_DECIMAL:
+          Preconditions.checkState(isSVColumn, "BigDecimal is not supported for MV columns");
+          statsCollector = new BigDecimalColumnPreIndexStatsCollector(column, statsCollectorConfig);
+          break;
+        default:
+          throw new IllegalStateException();
+      }
+
+      Preconditions.checkState(statsCollector != null);
+      // Note: Special Null handling is not necessary here. This is because, the existing default null value in the
+      // raw forwardIndex will be retained as such while created the dictionary and dict-based forward index. Also,
+      // null value vectors maintain a bitmap of docIds. No handling is necessary there.
+      PinotSegmentColumnReader columnReader =

Review Comment:
   We have already initialized the ForwardIndexReader at line 507. We should not need to init another reader here since this will also internally create another forward index reader. 
   
   I suggest we continue to use reader at line 507 to read the values out of forward index and gather stats / build 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.

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


[GitHub] [pinot] siddharthteotia commented on a diff in pull request #9678: Support creating dictionary at runtime for an existing column

Posted by GitBox <gi...@apache.org>.
siddharthteotia commented on code in PR #9678:
URL: https://github.com/apache/pinot/pull/9678#discussion_r1011043306


##########
pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/loader/ForwardIndexHandler.java:
##########
@@ -271,118 +304,295 @@ private void rewriteRawSVForwardIndex(String column, ColumnMetadata existingColM
               .forForwardIndex(newCompressionType, _indexLoadingConfig.getColumnProperties());
 
       try (ForwardIndexCreator creator = indexCreatorProvider.newForwardIndexCreator(context)) {
-        // If creator stored type and the reader stored type do not match, throw an exception.
         if (!reader.getStoredType().equals(creator.getValueType())) {
+          // Creator stored type should match reader stored type for raw columns. We do not support changing datatypes.
           String failureMsg =
               "Unsupported operation to change datatype for column=" + column + " from " + reader.getStoredType()
                   .toString() + " to " + creator.getValueType().toString();
           throw new UnsupportedOperationException(failureMsg);
         }
 
         int numDocs = existingColMetadata.getTotalDocs();
-        forwardIndexWriterHelper(column, reader, creator, numDocs);
+        forwardIndexWriterHelper(column, existingColMetadata, reader, creator, numDocs, null);
       }
     }
   }
 
-  private void forwardIndexWriterHelper(String column, ForwardIndexReader reader, ForwardIndexCreator creator,
-      int numDocs) {
-    // If creator stored type should match reader stored type. We do not support changing datatypes.
-    if (!reader.getStoredType().equals(creator.getValueType())) {
-      String failureMsg =
-          "Unsupported operation to change datatype for column=" + column + " from " + reader.getStoredType().toString()
-              + " to " + creator.getValueType().toString();
-      throw new UnsupportedOperationException(failureMsg);
-    }
-
+  private void forwardIndexWriterHelper(String column, ColumnMetadata existingColumnMetadata, ForwardIndexReader reader,
+      ForwardIndexCreator creator, int numDocs, @Nullable SegmentDictionaryCreator dictionaryCreator) {
     ForwardIndexReaderContext readerContext = reader.createContext();
     boolean isSVColumn = reader.isSingleValue();
 
-    switch (reader.getStoredType()) {
-      // JSON fields are either stored as string or bytes. No special handling is needed because we make this
-      // decision based on the storedType of the reader.
-      case INT: {
-        for (int i = 0; i < numDocs; i++) {
-          if (isSVColumn) {
-            int val = reader.getInt(i, readerContext);
-            creator.putInt(val);
-          } else {
-            int[] ints = reader.getIntMV(i, readerContext);
-            creator.putIntMV(ints);
-          }
+    if (dictionaryCreator != null) {
+      int maxNumValuesPerEntry = existingColumnMetadata.getMaxNumberOfMultiValues();
+      PinotSegmentColumnReader columnReader = new PinotSegmentColumnReader(reader, null, null, maxNumValuesPerEntry);
+
+      for (int i = 0; i < numDocs; i++) {
+        Object obj = columnReader.getValue(i);
+
+        if (isSVColumn) {
+          int dictId = dictionaryCreator.indexOfSV(obj);
+          creator.putDictId(dictId);
+        } else {
+          int[] dictIds = dictionaryCreator.indexOfMV(obj);
+          creator.putDictIdMV(dictIds);
         }
-        break;
       }
-      case LONG: {
-        for (int i = 0; i < numDocs; i++) {
-          if (isSVColumn) {
-            long val = reader.getLong(i, readerContext);
-            creator.putLong(val);
-          } else {
-            long[] longs = reader.getLongMV(i, readerContext);
-            creator.putLongMV(longs);
+    } else {
+      switch (reader.getStoredType()) {
+        // JSON fields are either stored as string or bytes. No special handling is needed because we make this
+        // decision based on the storedType of the reader.
+        case INT: {
+          for (int i = 0; i < numDocs; i++) {
+            if (isSVColumn) {
+              int val = reader.getInt(i, readerContext);
+              creator.putInt(val);
+            } else {
+              int[] ints = reader.getIntMV(i, readerContext);
+              creator.putIntMV(ints);
+            }
           }
+          break;
         }
-        break;
-      }
-      case FLOAT: {
-        for (int i = 0; i < numDocs; i++) {
-          if (isSVColumn) {
-            float val = reader.getFloat(i, readerContext);
-            creator.putFloat(val);
-          } else {
-            float[] floats = reader.getFloatMV(i, readerContext);
-            creator.putFloatMV(floats);
+        case LONG: {
+          for (int i = 0; i < numDocs; i++) {
+            if (isSVColumn) {
+              long val = reader.getLong(i, readerContext);
+              creator.putLong(val);
+            } else {
+              long[] longs = reader.getLongMV(i, readerContext);
+              creator.putLongMV(longs);
+            }
           }
+          break;
         }
-        break;
-      }
-      case DOUBLE: {
-        for (int i = 0; i < numDocs; i++) {
-          if (isSVColumn) {
-            double val = reader.getDouble(i, readerContext);
-            creator.putDouble(val);
-          } else {
-            double[] doubles = reader.getDoubleMV(i, readerContext);
-            creator.putDoubleMV(doubles);
+        case FLOAT: {
+          for (int i = 0; i < numDocs; i++) {
+            if (isSVColumn) {
+              float val = reader.getFloat(i, readerContext);
+              creator.putFloat(val);
+            } else {
+              float[] floats = reader.getFloatMV(i, readerContext);
+              creator.putFloatMV(floats);
+            }
           }
+          break;
         }
-        break;
-      }
-      case STRING: {
-        for (int i = 0; i < numDocs; i++) {
-          if (isSVColumn) {
-            String val = reader.getString(i, readerContext);
-            creator.putString(val);
-          } else {
-            String[] strings = reader.getStringMV(i, readerContext);
-            creator.putStringMV(strings);
+        case DOUBLE: {
+          for (int i = 0; i < numDocs; i++) {
+            if (isSVColumn) {
+              double val = reader.getDouble(i, readerContext);
+              creator.putDouble(val);
+            } else {
+              double[] doubles = reader.getDoubleMV(i, readerContext);
+              creator.putDoubleMV(doubles);
+            }
           }
+          break;
         }
-        break;
-      }
-      case BYTES: {
-        for (int i = 0; i < numDocs; i++) {
-          if (isSVColumn) {
-            byte[] val = reader.getBytes(i, readerContext);
-            creator.putBytes(val);
-          } else {
-            byte[][] bytesArray = reader.getBytesMV(i, readerContext);
-            creator.putBytesMV(bytesArray);
+        case STRING: {
+          for (int i = 0; i < numDocs; i++) {
+            if (isSVColumn) {
+              String val = reader.getString(i, readerContext);
+              creator.putString(val);
+            } else {
+              String[] strings = reader.getStringMV(i, readerContext);
+              creator.putStringMV(strings);
+            }
           }
+          break;
         }
-        break;
-      }
-      case BIG_DECIMAL: {
-        for (int i = 0; i < numDocs; i++) {
+        case BYTES: {
+          for (int i = 0; i < numDocs; i++) {
+            if (isSVColumn) {
+              byte[] val = reader.getBytes(i, readerContext);
+              creator.putBytes(val);
+            } else {
+              byte[][] bytesArray = reader.getBytesMV(i, readerContext);
+              creator.putBytesMV(bytesArray);
+            }
+          }
+          break;
+        }
+        case BIG_DECIMAL: {
           Preconditions.checkState(isSVColumn, "BigDecimal is not supported for MV columns");
-          BigDecimal val = reader.getBigDecimal(i, readerContext);
-          creator.putBigDecimal(val);
+          for (int i = 0; i < numDocs; i++) {
+            BigDecimal val = reader.getBigDecimal(i, readerContext);
+            creator.putBigDecimal(val);
+          }
+          break;
         }
-        break;
+        default:
+          throw new IllegalStateException();
+      }
+    }
+  }
+
+  private void enableDictionary(String column, SegmentDirectory.Writer segmentWriter,
+      IndexCreatorProvider indexCreatorProvider)
+      throws Exception {
+    Preconditions.checkState(_segmentMetadata.getVersion() == SegmentVersion.v3);
+    ColumnMetadata existingColMetadata = _segmentMetadata.getColumnMetadataFor(column);
+    Preconditions.checkState(!existingColMetadata.hasDictionary(),
+        "Cannot rewrite dictionary enabled forward index. Dictionary already exists for column:" + column);

Review Comment:
   Not sure I understand why we need to fail. This should be treated as a NO-OP.



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


[GitHub] [pinot] siddharthteotia commented on a diff in pull request #9678: Support creating dictionary at runtime for an existing column

Posted by GitBox <gi...@apache.org>.
siddharthteotia commented on code in PR #9678:
URL: https://github.com/apache/pinot/pull/9678#discussion_r1011073256


##########
pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/loader/SegmentPreProcessor.java:
##########
@@ -104,8 +104,15 @@ public void process()
       // Update single-column indices, like inverted index, json index etc.
       IndexCreatorProvider indexCreatorProvider = IndexingOverrides.getIndexCreatorProvider();
       for (ColumnIndexType type : ColumnIndexType.values()) {
-        IndexHandler handler = IndexHandlerFactory.getIndexHandler(type, _segmentMetadata, _indexLoadingConfig);
+        IndexHandler handler =
+            IndexHandlerFactory.getIndexHandler(type, _segmentMetadata, _indexLoadingConfig, _schema);
         handler.updateIndices(segmentWriter, indexCreatorProvider);
+        if (type == ColumnIndexType.FORWARD_INDEX) {

Review Comment:
   Rather than relying on ColumnIndexType enum entry order, I feel this code should explicitly ensure that FORWARD_INDEX handler is the first handler to be invoked and then everyone else in the loop
   
   We have seen cases (in the recent past) where either people disregarded or just simply forgot to honor the javadoc guideline regarding enum entry order. So, code should be more resilient against such cases



##########
pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/loader/SegmentPreProcessor.java:
##########
@@ -104,8 +104,15 @@ public void process()
       // Update single-column indices, like inverted index, json index etc.
       IndexCreatorProvider indexCreatorProvider = IndexingOverrides.getIndexCreatorProvider();
       for (ColumnIndexType type : ColumnIndexType.values()) {
-        IndexHandler handler = IndexHandlerFactory.getIndexHandler(type, _segmentMetadata, _indexLoadingConfig);
+        IndexHandler handler =
+            IndexHandlerFactory.getIndexHandler(type, _segmentMetadata, _indexLoadingConfig, _schema);
         handler.updateIndices(segmentWriter, indexCreatorProvider);
+        if (type == ColumnIndexType.FORWARD_INDEX) {

Review Comment:
   Rather than relying on `ColumnIndexType` enum entry order, I feel this code should explicitly ensure that `FORWARD_INDEX` handler is the first handler to be invoked and then everyone else in the loop
   
   We have seen cases (in the recent past) where either people disregarded or just simply forgot to honor the javadoc guideline regarding enum entry order. So, code should be more resilient against such cases



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


[GitHub] [pinot] vvivekiyer commented on a diff in pull request #9678: Support creating dictionary at runtime for an existing column

Posted by GitBox <gi...@apache.org>.
vvivekiyer commented on code in PR #9678:
URL: https://github.com/apache/pinot/pull/9678#discussion_r1012211022


##########
pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/loader/ForwardIndexHandler.java:
##########
@@ -271,118 +304,295 @@ private void rewriteRawSVForwardIndex(String column, ColumnMetadata existingColM
               .forForwardIndex(newCompressionType, _indexLoadingConfig.getColumnProperties());
 
       try (ForwardIndexCreator creator = indexCreatorProvider.newForwardIndexCreator(context)) {
-        // If creator stored type and the reader stored type do not match, throw an exception.
         if (!reader.getStoredType().equals(creator.getValueType())) {
+          // Creator stored type should match reader stored type for raw columns. We do not support changing datatypes.
           String failureMsg =
               "Unsupported operation to change datatype for column=" + column + " from " + reader.getStoredType()
                   .toString() + " to " + creator.getValueType().toString();
           throw new UnsupportedOperationException(failureMsg);
         }
 
         int numDocs = existingColMetadata.getTotalDocs();
-        forwardIndexWriterHelper(column, reader, creator, numDocs);
+        forwardIndexWriterHelper(column, existingColMetadata, reader, creator, numDocs, null);
       }
     }
   }
 
-  private void forwardIndexWriterHelper(String column, ForwardIndexReader reader, ForwardIndexCreator creator,
-      int numDocs) {
-    // If creator stored type should match reader stored type. We do not support changing datatypes.
-    if (!reader.getStoredType().equals(creator.getValueType())) {
-      String failureMsg =
-          "Unsupported operation to change datatype for column=" + column + " from " + reader.getStoredType().toString()
-              + " to " + creator.getValueType().toString();
-      throw new UnsupportedOperationException(failureMsg);
-    }
-
+  private void forwardIndexWriterHelper(String column, ColumnMetadata existingColumnMetadata, ForwardIndexReader reader,
+      ForwardIndexCreator creator, int numDocs, @Nullable SegmentDictionaryCreator dictionaryCreator) {
     ForwardIndexReaderContext readerContext = reader.createContext();
     boolean isSVColumn = reader.isSingleValue();
 
-    switch (reader.getStoredType()) {
-      // JSON fields are either stored as string or bytes. No special handling is needed because we make this
-      // decision based on the storedType of the reader.
-      case INT: {
-        for (int i = 0; i < numDocs; i++) {
-          if (isSVColumn) {
-            int val = reader.getInt(i, readerContext);
-            creator.putInt(val);
-          } else {
-            int[] ints = reader.getIntMV(i, readerContext);
-            creator.putIntMV(ints);
-          }
+    if (dictionaryCreator != null) {
+      int maxNumValuesPerEntry = existingColumnMetadata.getMaxNumberOfMultiValues();
+      PinotSegmentColumnReader columnReader = new PinotSegmentColumnReader(reader, null, null, maxNumValuesPerEntry);
+
+      for (int i = 0; i < numDocs; i++) {
+        Object obj = columnReader.getValue(i);
+
+        if (isSVColumn) {
+          int dictId = dictionaryCreator.indexOfSV(obj);
+          creator.putDictId(dictId);
+        } else {
+          int[] dictIds = dictionaryCreator.indexOfMV(obj);
+          creator.putDictIdMV(dictIds);
         }
-        break;
       }
-      case LONG: {
-        for (int i = 0; i < numDocs; i++) {
-          if (isSVColumn) {
-            long val = reader.getLong(i, readerContext);
-            creator.putLong(val);
-          } else {
-            long[] longs = reader.getLongMV(i, readerContext);
-            creator.putLongMV(longs);
+    } else {
+      switch (reader.getStoredType()) {
+        // JSON fields are either stored as string or bytes. No special handling is needed because we make this
+        // decision based on the storedType of the reader.
+        case INT: {
+          for (int i = 0; i < numDocs; i++) {
+            if (isSVColumn) {
+              int val = reader.getInt(i, readerContext);
+              creator.putInt(val);
+            } else {
+              int[] ints = reader.getIntMV(i, readerContext);
+              creator.putIntMV(ints);
+            }
           }
+          break;
         }
-        break;
-      }
-      case FLOAT: {
-        for (int i = 0; i < numDocs; i++) {
-          if (isSVColumn) {
-            float val = reader.getFloat(i, readerContext);
-            creator.putFloat(val);
-          } else {
-            float[] floats = reader.getFloatMV(i, readerContext);
-            creator.putFloatMV(floats);
+        case LONG: {
+          for (int i = 0; i < numDocs; i++) {
+            if (isSVColumn) {
+              long val = reader.getLong(i, readerContext);
+              creator.putLong(val);
+            } else {
+              long[] longs = reader.getLongMV(i, readerContext);
+              creator.putLongMV(longs);
+            }
           }
+          break;
         }
-        break;
-      }
-      case DOUBLE: {
-        for (int i = 0; i < numDocs; i++) {
-          if (isSVColumn) {
-            double val = reader.getDouble(i, readerContext);
-            creator.putDouble(val);
-          } else {
-            double[] doubles = reader.getDoubleMV(i, readerContext);
-            creator.putDoubleMV(doubles);
+        case FLOAT: {
+          for (int i = 0; i < numDocs; i++) {
+            if (isSVColumn) {
+              float val = reader.getFloat(i, readerContext);
+              creator.putFloat(val);
+            } else {
+              float[] floats = reader.getFloatMV(i, readerContext);
+              creator.putFloatMV(floats);
+            }
           }
+          break;
         }
-        break;
-      }
-      case STRING: {
-        for (int i = 0; i < numDocs; i++) {
-          if (isSVColumn) {
-            String val = reader.getString(i, readerContext);
-            creator.putString(val);
-          } else {
-            String[] strings = reader.getStringMV(i, readerContext);
-            creator.putStringMV(strings);
+        case DOUBLE: {
+          for (int i = 0; i < numDocs; i++) {
+            if (isSVColumn) {
+              double val = reader.getDouble(i, readerContext);
+              creator.putDouble(val);
+            } else {
+              double[] doubles = reader.getDoubleMV(i, readerContext);
+              creator.putDoubleMV(doubles);
+            }
           }
+          break;
         }
-        break;
-      }
-      case BYTES: {
-        for (int i = 0; i < numDocs; i++) {
-          if (isSVColumn) {
-            byte[] val = reader.getBytes(i, readerContext);
-            creator.putBytes(val);
-          } else {
-            byte[][] bytesArray = reader.getBytesMV(i, readerContext);
-            creator.putBytesMV(bytesArray);
+        case STRING: {
+          for (int i = 0; i < numDocs; i++) {
+            if (isSVColumn) {
+              String val = reader.getString(i, readerContext);
+              creator.putString(val);
+            } else {
+              String[] strings = reader.getStringMV(i, readerContext);
+              creator.putStringMV(strings);
+            }
           }
+          break;
         }
-        break;
-      }
-      case BIG_DECIMAL: {
-        for (int i = 0; i < numDocs; i++) {
+        case BYTES: {
+          for (int i = 0; i < numDocs; i++) {
+            if (isSVColumn) {
+              byte[] val = reader.getBytes(i, readerContext);
+              creator.putBytes(val);
+            } else {
+              byte[][] bytesArray = reader.getBytesMV(i, readerContext);
+              creator.putBytesMV(bytesArray);
+            }
+          }
+          break;
+        }
+        case BIG_DECIMAL: {
           Preconditions.checkState(isSVColumn, "BigDecimal is not supported for MV columns");
-          BigDecimal val = reader.getBigDecimal(i, readerContext);
-          creator.putBigDecimal(val);
+          for (int i = 0; i < numDocs; i++) {
+            BigDecimal val = reader.getBigDecimal(i, readerContext);
+            creator.putBigDecimal(val);
+          }
+          break;
         }
-        break;
+        default:
+          throw new IllegalStateException();
+      }
+    }
+  }
+
+  private void enableDictionary(String column, SegmentDirectory.Writer segmentWriter,
+      IndexCreatorProvider indexCreatorProvider)
+      throws Exception {
+    Preconditions.checkState(_segmentMetadata.getVersion() == SegmentVersion.v3);
+    ColumnMetadata existingColMetadata = _segmentMetadata.getColumnMetadataFor(column);
+    Preconditions.checkState(!existingColMetadata.hasDictionary(),
+        "Cannot rewrite dictionary enabled forward index. Dictionary already exists for column:" + column);
+    boolean isSingleValue = existingColMetadata.isSingleValue();
+
+    File indexDir = _segmentMetadata.getIndexDir();
+    String segmentName = _segmentMetadata.getName();
+    File inProgress = new File(indexDir, column + ".dict.inprogress");
+    File dictionaryFile = new File(indexDir, column + V1Constants.Dict.FILE_EXTENSION);
+    String fwdIndexFileExtension;
+    if (isSingleValue) {
+      fwdIndexFileExtension =
+          existingColMetadata.isSorted() ? V1Constants.Indexes.SORTED_SV_FORWARD_INDEX_FILE_EXTENSION
+              : V1Constants.Indexes.UNSORTED_SV_FORWARD_INDEX_FILE_EXTENSION;
+    } else {
+      fwdIndexFileExtension = V1Constants.Indexes.UNSORTED_MV_FORWARD_INDEX_FILE_EXTENSION;
+    }
+    File fwdIndexFile = new File(indexDir, column + fwdIndexFileExtension);
+
+    if (!inProgress.exists()) {
+      // Marker file does not exist, which means last run ended normally.
+      // Create a marker file.
+      FileUtils.touch(inProgress);
+    } else {
+      // Marker file exists, which means last run was interrupted.
+      // Remove forward index and dictionary files if they exist.
+      FileUtils.deleteQuietly(fwdIndexFile);
+      FileUtils.deleteQuietly(dictionaryFile);
+    }
+
+    LOGGER.info("Creating a new dictionary for segment={} and column={}", segmentName, column);
+    SegmentDictionaryCreator dictionaryCreator = buildDictionary(column, existingColMetadata, segmentWriter);
+    LoaderUtils.writeIndexToV3Format(segmentWriter, column, dictionaryFile, ColumnIndexType.DICTIONARY);
+
+    LOGGER.info("Built dictionary. Rewriting dictionary enabled forward index for segment={} and column={}",

Review Comment:
   That's right.



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


[GitHub] [pinot] vvivekiyer commented on a diff in pull request #9678: Support creating dictionary at runtime for an existing column

Posted by GitBox <gi...@apache.org>.
vvivekiyer commented on code in PR #9678:
URL: https://github.com/apache/pinot/pull/9678#discussion_r1012173646


##########
pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/loader/ForwardIndexHandler.java:
##########
@@ -271,118 +304,295 @@ private void rewriteRawSVForwardIndex(String column, ColumnMetadata existingColM
               .forForwardIndex(newCompressionType, _indexLoadingConfig.getColumnProperties());
 
       try (ForwardIndexCreator creator = indexCreatorProvider.newForwardIndexCreator(context)) {
-        // If creator stored type and the reader stored type do not match, throw an exception.
         if (!reader.getStoredType().equals(creator.getValueType())) {
+          // Creator stored type should match reader stored type for raw columns. We do not support changing datatypes.
           String failureMsg =
               "Unsupported operation to change datatype for column=" + column + " from " + reader.getStoredType()
                   .toString() + " to " + creator.getValueType().toString();
           throw new UnsupportedOperationException(failureMsg);
         }
 
         int numDocs = existingColMetadata.getTotalDocs();
-        forwardIndexWriterHelper(column, reader, creator, numDocs);
+        forwardIndexWriterHelper(column, existingColMetadata, reader, creator, numDocs, null);
       }
     }
   }
 
-  private void forwardIndexWriterHelper(String column, ForwardIndexReader reader, ForwardIndexCreator creator,
-      int numDocs) {
-    // If creator stored type should match reader stored type. We do not support changing datatypes.
-    if (!reader.getStoredType().equals(creator.getValueType())) {
-      String failureMsg =
-          "Unsupported operation to change datatype for column=" + column + " from " + reader.getStoredType().toString()
-              + " to " + creator.getValueType().toString();
-      throw new UnsupportedOperationException(failureMsg);
-    }
-
+  private void forwardIndexWriterHelper(String column, ColumnMetadata existingColumnMetadata, ForwardIndexReader reader,
+      ForwardIndexCreator creator, int numDocs, @Nullable SegmentDictionaryCreator dictionaryCreator) {
     ForwardIndexReaderContext readerContext = reader.createContext();
     boolean isSVColumn = reader.isSingleValue();
 
-    switch (reader.getStoredType()) {
-      // JSON fields are either stored as string or bytes. No special handling is needed because we make this
-      // decision based on the storedType of the reader.
-      case INT: {
-        for (int i = 0; i < numDocs; i++) {
-          if (isSVColumn) {
-            int val = reader.getInt(i, readerContext);
-            creator.putInt(val);
-          } else {
-            int[] ints = reader.getIntMV(i, readerContext);
-            creator.putIntMV(ints);
-          }
+    if (dictionaryCreator != null) {
+      int maxNumValuesPerEntry = existingColumnMetadata.getMaxNumberOfMultiValues();
+      PinotSegmentColumnReader columnReader = new PinotSegmentColumnReader(reader, null, null, maxNumValuesPerEntry);
+
+      for (int i = 0; i < numDocs; i++) {
+        Object obj = columnReader.getValue(i);
+
+        if (isSVColumn) {
+          int dictId = dictionaryCreator.indexOfSV(obj);
+          creator.putDictId(dictId);
+        } else {
+          int[] dictIds = dictionaryCreator.indexOfMV(obj);
+          creator.putDictIdMV(dictIds);
         }
-        break;
       }
-      case LONG: {
-        for (int i = 0; i < numDocs; i++) {
-          if (isSVColumn) {
-            long val = reader.getLong(i, readerContext);
-            creator.putLong(val);
-          } else {
-            long[] longs = reader.getLongMV(i, readerContext);
-            creator.putLongMV(longs);
+    } else {
+      switch (reader.getStoredType()) {
+        // JSON fields are either stored as string or bytes. No special handling is needed because we make this
+        // decision based on the storedType of the reader.
+        case INT: {
+          for (int i = 0; i < numDocs; i++) {
+            if (isSVColumn) {
+              int val = reader.getInt(i, readerContext);
+              creator.putInt(val);
+            } else {
+              int[] ints = reader.getIntMV(i, readerContext);
+              creator.putIntMV(ints);
+            }
           }
+          break;
         }
-        break;
-      }
-      case FLOAT: {
-        for (int i = 0; i < numDocs; i++) {
-          if (isSVColumn) {
-            float val = reader.getFloat(i, readerContext);
-            creator.putFloat(val);
-          } else {
-            float[] floats = reader.getFloatMV(i, readerContext);
-            creator.putFloatMV(floats);
+        case LONG: {
+          for (int i = 0; i < numDocs; i++) {
+            if (isSVColumn) {
+              long val = reader.getLong(i, readerContext);
+              creator.putLong(val);
+            } else {
+              long[] longs = reader.getLongMV(i, readerContext);
+              creator.putLongMV(longs);
+            }
           }
+          break;
         }
-        break;
-      }
-      case DOUBLE: {
-        for (int i = 0; i < numDocs; i++) {
-          if (isSVColumn) {
-            double val = reader.getDouble(i, readerContext);
-            creator.putDouble(val);
-          } else {
-            double[] doubles = reader.getDoubleMV(i, readerContext);
-            creator.putDoubleMV(doubles);
+        case FLOAT: {
+          for (int i = 0; i < numDocs; i++) {
+            if (isSVColumn) {
+              float val = reader.getFloat(i, readerContext);
+              creator.putFloat(val);
+            } else {
+              float[] floats = reader.getFloatMV(i, readerContext);
+              creator.putFloatMV(floats);
+            }
           }
+          break;
         }
-        break;
-      }
-      case STRING: {
-        for (int i = 0; i < numDocs; i++) {
-          if (isSVColumn) {
-            String val = reader.getString(i, readerContext);
-            creator.putString(val);
-          } else {
-            String[] strings = reader.getStringMV(i, readerContext);
-            creator.putStringMV(strings);
+        case DOUBLE: {
+          for (int i = 0; i < numDocs; i++) {
+            if (isSVColumn) {
+              double val = reader.getDouble(i, readerContext);
+              creator.putDouble(val);
+            } else {
+              double[] doubles = reader.getDoubleMV(i, readerContext);
+              creator.putDoubleMV(doubles);
+            }
           }
+          break;
         }
-        break;
-      }
-      case BYTES: {
-        for (int i = 0; i < numDocs; i++) {
-          if (isSVColumn) {
-            byte[] val = reader.getBytes(i, readerContext);
-            creator.putBytes(val);
-          } else {
-            byte[][] bytesArray = reader.getBytesMV(i, readerContext);
-            creator.putBytesMV(bytesArray);
+        case STRING: {
+          for (int i = 0; i < numDocs; i++) {
+            if (isSVColumn) {
+              String val = reader.getString(i, readerContext);
+              creator.putString(val);
+            } else {
+              String[] strings = reader.getStringMV(i, readerContext);
+              creator.putStringMV(strings);
+            }
           }
+          break;
         }
-        break;
-      }
-      case BIG_DECIMAL: {
-        for (int i = 0; i < numDocs; i++) {
+        case BYTES: {
+          for (int i = 0; i < numDocs; i++) {
+            if (isSVColumn) {
+              byte[] val = reader.getBytes(i, readerContext);
+              creator.putBytes(val);
+            } else {
+              byte[][] bytesArray = reader.getBytesMV(i, readerContext);
+              creator.putBytesMV(bytesArray);
+            }
+          }
+          break;
+        }
+        case BIG_DECIMAL: {
           Preconditions.checkState(isSVColumn, "BigDecimal is not supported for MV columns");
-          BigDecimal val = reader.getBigDecimal(i, readerContext);
-          creator.putBigDecimal(val);
+          for (int i = 0; i < numDocs; i++) {
+            BigDecimal val = reader.getBigDecimal(i, readerContext);
+            creator.putBigDecimal(val);
+          }
+          break;
         }
-        break;
+        default:
+          throw new IllegalStateException();
+      }
+    }
+  }
+
+  private void enableDictionary(String column, SegmentDirectory.Writer segmentWriter,
+      IndexCreatorProvider indexCreatorProvider)
+      throws Exception {
+    Preconditions.checkState(_segmentMetadata.getVersion() == SegmentVersion.v3);
+    ColumnMetadata existingColMetadata = _segmentMetadata.getColumnMetadataFor(column);
+    Preconditions.checkState(!existingColMetadata.hasDictionary(),
+        "Cannot rewrite dictionary enabled forward index. Dictionary already exists for column:" + column);

Review Comment:
   Future proofing - if tests are added in the future :)  In my mind, I thought it made it easier to read code as the reader would instantly know this function only works for v3 version and if dictionary is already disabled.
   
   But thinking about it, I don't have a strong preference. I'll remove it.



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


[GitHub] [pinot] vvivekiyer commented on a diff in pull request #9678: Support creating dictionary at runtime for an existing column

Posted by GitBox <gi...@apache.org>.
vvivekiyer commented on code in PR #9678:
URL: https://github.com/apache/pinot/pull/9678#discussion_r1011006549


##########
pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/loader/ForwardIndexHandler.java:
##########
@@ -26,23 +26,43 @@
 import java.util.HashSet;
 import java.util.Map;
 import java.util.Set;
+import javax.annotation.Nullable;
+import org.apache.commons.configuration.PropertiesConfiguration;
 import org.apache.commons.io.FileUtils;
+import org.apache.pinot.segment.local.segment.creator.impl.SegmentDictionaryCreator;
+import org.apache.pinot.segment.local.segment.creator.impl.SegmentIndexCreationDriverImpl;
 import org.apache.pinot.segment.local.segment.creator.impl.fwd.MultiValueVarByteRawIndexCreator;
+import org.apache.pinot.segment.local.segment.creator.impl.stats.AbstractColumnStatisticsCollector;
+import org.apache.pinot.segment.local.segment.creator.impl.stats.BigDecimalColumnPreIndexStatsCollector;
+import org.apache.pinot.segment.local.segment.creator.impl.stats.BytesColumnPredIndexStatsCollector;
+import org.apache.pinot.segment.local.segment.creator.impl.stats.DoubleColumnPreIndexStatsCollector;
+import org.apache.pinot.segment.local.segment.creator.impl.stats.FloatColumnPreIndexStatsCollector;
+import org.apache.pinot.segment.local.segment.creator.impl.stats.IntColumnPreIndexStatsCollector;
+import org.apache.pinot.segment.local.segment.creator.impl.stats.LongColumnPreIndexStatsCollector;
+import org.apache.pinot.segment.local.segment.creator.impl.stats.StringColumnPreIndexStatsCollector;
+import org.apache.pinot.segment.local.segment.readers.PinotSegmentColumnReader;
 import org.apache.pinot.segment.spi.ColumnMetadata;
 import org.apache.pinot.segment.spi.SegmentMetadata;
 import org.apache.pinot.segment.spi.V1Constants;
 import org.apache.pinot.segment.spi.compression.ChunkCompressionType;
 import org.apache.pinot.segment.spi.creator.IndexCreationContext;
 import org.apache.pinot.segment.spi.creator.IndexCreatorProvider;
 import org.apache.pinot.segment.spi.creator.SegmentVersion;
+import org.apache.pinot.segment.spi.creator.StatsCollectorConfig;
 import org.apache.pinot.segment.spi.index.creator.ForwardIndexCreator;
 import org.apache.pinot.segment.spi.index.reader.ForwardIndexReader;
 import org.apache.pinot.segment.spi.index.reader.ForwardIndexReaderContext;
 import org.apache.pinot.segment.spi.store.ColumnIndexType;
 import org.apache.pinot.segment.spi.store.SegmentDirectory;
+import org.apache.pinot.segment.spi.store.SegmentDirectoryPaths;
+import org.apache.pinot.spi.data.Schema;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import static org.apache.pinot.segment.spi.V1Constants.MetadataKeys.Column.DICTIONARY_ELEMENT_SIZE;

Review Comment:
   Done



##########
pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/creator/impl/SegmentIndexCreationDriverImpl.java:
##########
@@ -416,15 +416,11 @@ void buildIndexCreationInfo()
       String columnName = fieldSpec.getName();
       DataType storedType = fieldSpec.getDataType().getStoredType();
       ColumnStatistics columnProfile = _segmentStats.getColumnProfileFor(columnName);
-      boolean useVarLengthDictionary = varLengthDictionaryColumns.contains(columnName);
+      boolean useVarLengthDictionary =
+          varLengthDictionaryColumns.contains(columnName) || shouldUseVarLengthDictionary(storedType, columnProfile);

Review Comment:
   Done.



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


[GitHub] [pinot] somandal commented on a diff in pull request #9678: Support creating dictionary at runtime for an existing column

Posted by GitBox <gi...@apache.org>.
somandal commented on code in PR #9678:
URL: https://github.com/apache/pinot/pull/9678#discussion_r1010639412


##########
pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/creator/impl/SegmentIndexCreationDriverImpl.java:
##########
@@ -416,15 +416,11 @@ void buildIndexCreationInfo()
       String columnName = fieldSpec.getName();
       DataType storedType = fieldSpec.getDataType().getStoredType();
       ColumnStatistics columnProfile = _segmentStats.getColumnProfileFor(columnName);
-      boolean useVarLengthDictionary = varLengthDictionaryColumns.contains(columnName);
+      boolean useVarLengthDictionary =
+          varLengthDictionaryColumns.contains(columnName) || shouldUseVarLengthDictionary(storedType, columnProfile);

Review Comment:
   how about you move the first part of the OR, `varLengthDictionaryColumns.contains(columnName)`, to the function too?



##########
pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/loader/ForwardIndexHandler.java:
##########
@@ -26,23 +26,43 @@
 import java.util.HashSet;
 import java.util.Map;
 import java.util.Set;
+import javax.annotation.Nullable;
+import org.apache.commons.configuration.PropertiesConfiguration;
 import org.apache.commons.io.FileUtils;
+import org.apache.pinot.segment.local.segment.creator.impl.SegmentDictionaryCreator;
+import org.apache.pinot.segment.local.segment.creator.impl.SegmentIndexCreationDriverImpl;
 import org.apache.pinot.segment.local.segment.creator.impl.fwd.MultiValueVarByteRawIndexCreator;
+import org.apache.pinot.segment.local.segment.creator.impl.stats.AbstractColumnStatisticsCollector;
+import org.apache.pinot.segment.local.segment.creator.impl.stats.BigDecimalColumnPreIndexStatsCollector;
+import org.apache.pinot.segment.local.segment.creator.impl.stats.BytesColumnPredIndexStatsCollector;
+import org.apache.pinot.segment.local.segment.creator.impl.stats.DoubleColumnPreIndexStatsCollector;
+import org.apache.pinot.segment.local.segment.creator.impl.stats.FloatColumnPreIndexStatsCollector;
+import org.apache.pinot.segment.local.segment.creator.impl.stats.IntColumnPreIndexStatsCollector;
+import org.apache.pinot.segment.local.segment.creator.impl.stats.LongColumnPreIndexStatsCollector;
+import org.apache.pinot.segment.local.segment.creator.impl.stats.StringColumnPreIndexStatsCollector;
+import org.apache.pinot.segment.local.segment.readers.PinotSegmentColumnReader;
 import org.apache.pinot.segment.spi.ColumnMetadata;
 import org.apache.pinot.segment.spi.SegmentMetadata;
 import org.apache.pinot.segment.spi.V1Constants;
 import org.apache.pinot.segment.spi.compression.ChunkCompressionType;
 import org.apache.pinot.segment.spi.creator.IndexCreationContext;
 import org.apache.pinot.segment.spi.creator.IndexCreatorProvider;
 import org.apache.pinot.segment.spi.creator.SegmentVersion;
+import org.apache.pinot.segment.spi.creator.StatsCollectorConfig;
 import org.apache.pinot.segment.spi.index.creator.ForwardIndexCreator;
 import org.apache.pinot.segment.spi.index.reader.ForwardIndexReader;
 import org.apache.pinot.segment.spi.index.reader.ForwardIndexReaderContext;
 import org.apache.pinot.segment.spi.store.ColumnIndexType;
 import org.apache.pinot.segment.spi.store.SegmentDirectory;
+import org.apache.pinot.segment.spi.store.SegmentDirectoryPaths;
+import org.apache.pinot.spi.data.Schema;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import static org.apache.pinot.segment.spi.V1Constants.MetadataKeys.Column.DICTIONARY_ELEMENT_SIZE;

Review Comment:
   nit: just add import for `MetadataKeys` or `MetadataKeys.Column`?



##########
pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/loader/ForwardIndexHandler.java:
##########
@@ -271,118 +306,293 @@ private void rewriteRawSVForwardIndex(String column, ColumnMetadata existingColM
               .forForwardIndex(newCompressionType, _indexLoadingConfig.getColumnProperties());
 
       try (ForwardIndexCreator creator = indexCreatorProvider.newForwardIndexCreator(context)) {
-        // If creator stored type and the reader stored type do not match, throw an exception.
         if (!reader.getStoredType().equals(creator.getValueType())) {
+          // Creator stored type should match reader stored type for raw columns. We do not support changing datatypes.
           String failureMsg =
               "Unsupported operation to change datatype for column=" + column + " from " + reader.getStoredType()
                   .toString() + " to " + creator.getValueType().toString();
           throw new UnsupportedOperationException(failureMsg);
         }
 
         int numDocs = existingColMetadata.getTotalDocs();
-        forwardIndexWriterHelper(column, reader, creator, numDocs);
+        forwardIndexWriterHelper(column, existingColMetadata, reader, creator, numDocs, null);
       }
     }
   }
 
-  private void forwardIndexWriterHelper(String column, ForwardIndexReader reader, ForwardIndexCreator creator,
-      int numDocs) {
-    // If creator stored type should match reader stored type. We do not support changing datatypes.
-    if (!reader.getStoredType().equals(creator.getValueType())) {
-      String failureMsg =
-          "Unsupported operation to change datatype for column=" + column + " from " + reader.getStoredType().toString()
-              + " to " + creator.getValueType().toString();
-      throw new UnsupportedOperationException(failureMsg);
-    }
-
+  private void forwardIndexWriterHelper(String column, ColumnMetadata existingColumnMetadata, ForwardIndexReader reader,
+      ForwardIndexCreator creator, int numDocs, @Nullable SegmentDictionaryCreator dictionaryCreator) {
     ForwardIndexReaderContext readerContext = reader.createContext();
     boolean isSVColumn = reader.isSingleValue();
 
-    switch (reader.getStoredType()) {
-      // JSON fields are either stored as string or bytes. No special handling is needed because we make this
-      // decision based on the storedType of the reader.
-      case INT: {
-        for (int i = 0; i < numDocs; i++) {
-          if (isSVColumn) {
-            int val = reader.getInt(i, readerContext);
-            creator.putInt(val);
-          } else {
-            int[] ints = reader.getIntMV(i, readerContext);
-            creator.putIntMV(ints);
-          }
+    if (dictionaryCreator != null) {
+      int maxNumValuesPerEntry = existingColumnMetadata.getMaxNumberOfMultiValues();
+      PinotSegmentColumnReader columnReader = new PinotSegmentColumnReader(reader, null, null, maxNumValuesPerEntry);
+
+      for (int i = 0; i < numDocs; i++) {
+        Object obj = columnReader.getValue(i);
+
+        if (isSVColumn) {
+          int dictId = dictionaryCreator.indexOfSV(obj);
+          creator.putDictId(dictId);
+        } else {
+          int[] dictIds = dictionaryCreator.indexOfMV(obj);
+          creator.putDictIdMV(dictIds);
         }
-        break;
       }
-      case LONG: {
-        for (int i = 0; i < numDocs; i++) {
-          if (isSVColumn) {
-            long val = reader.getLong(i, readerContext);
-            creator.putLong(val);
-          } else {
-            long[] longs = reader.getLongMV(i, readerContext);
-            creator.putLongMV(longs);
+    } else {
+      switch (reader.getStoredType()) {
+        // JSON fields are either stored as string or bytes. No special handling is needed because we make this
+        // decision based on the storedType of the reader.
+        case INT: {
+          for (int i = 0; i < numDocs; i++) {
+            if (isSVColumn) {
+              int val = reader.getInt(i, readerContext);
+              creator.putInt(val);
+            } else {
+              int[] ints = reader.getIntMV(i, readerContext);
+              creator.putIntMV(ints);
+            }
           }
+          break;
         }
-        break;
-      }
-      case FLOAT: {
-        for (int i = 0; i < numDocs; i++) {
-          if (isSVColumn) {
-            float val = reader.getFloat(i, readerContext);
-            creator.putFloat(val);
-          } else {
-            float[] floats = reader.getFloatMV(i, readerContext);
-            creator.putFloatMV(floats);
+        case LONG: {
+          for (int i = 0; i < numDocs; i++) {
+            if (isSVColumn) {
+              long val = reader.getLong(i, readerContext);
+              creator.putLong(val);
+            } else {
+              long[] longs = reader.getLongMV(i, readerContext);
+              creator.putLongMV(longs);
+            }
           }
+          break;
         }
-        break;
-      }
-      case DOUBLE: {
-        for (int i = 0; i < numDocs; i++) {
-          if (isSVColumn) {
-            double val = reader.getDouble(i, readerContext);
-            creator.putDouble(val);
-          } else {
-            double[] doubles = reader.getDoubleMV(i, readerContext);
-            creator.putDoubleMV(doubles);
+        case FLOAT: {
+          for (int i = 0; i < numDocs; i++) {
+            if (isSVColumn) {
+              float val = reader.getFloat(i, readerContext);
+              creator.putFloat(val);
+            } else {
+              float[] floats = reader.getFloatMV(i, readerContext);
+              creator.putFloatMV(floats);
+            }
           }
+          break;
         }
-        break;
-      }
-      case STRING: {
-        for (int i = 0; i < numDocs; i++) {
-          if (isSVColumn) {
-            String val = reader.getString(i, readerContext);
-            creator.putString(val);
-          } else {
-            String[] strings = reader.getStringMV(i, readerContext);
-            creator.putStringMV(strings);
+        case DOUBLE: {
+          for (int i = 0; i < numDocs; i++) {
+            if (isSVColumn) {
+              double val = reader.getDouble(i, readerContext);
+              creator.putDouble(val);
+            } else {
+              double[] doubles = reader.getDoubleMV(i, readerContext);
+              creator.putDoubleMV(doubles);
+            }
           }
+          break;
         }
-        break;
-      }
-      case BYTES: {
-        for (int i = 0; i < numDocs; i++) {
-          if (isSVColumn) {
-            byte[] val = reader.getBytes(i, readerContext);
-            creator.putBytes(val);
-          } else {
-            byte[][] bytesArray = reader.getBytesMV(i, readerContext);
-            creator.putBytesMV(bytesArray);
+        case STRING: {
+          for (int i = 0; i < numDocs; i++) {
+            if (isSVColumn) {
+              String val = reader.getString(i, readerContext);
+              creator.putString(val);
+            } else {
+              String[] strings = reader.getStringMV(i, readerContext);
+              creator.putStringMV(strings);
+            }
           }
+          break;
         }
-        break;
-      }
-      case BIG_DECIMAL: {
-        for (int i = 0; i < numDocs; i++) {
+        case BYTES: {
+          for (int i = 0; i < numDocs; i++) {
+            if (isSVColumn) {
+              byte[] val = reader.getBytes(i, readerContext);
+              creator.putBytes(val);
+            } else {
+              byte[][] bytesArray = reader.getBytesMV(i, readerContext);
+              creator.putBytesMV(bytesArray);
+            }
+          }
+          break;
+        }
+        case BIG_DECIMAL: {
           Preconditions.checkState(isSVColumn, "BigDecimal is not supported for MV columns");
-          BigDecimal val = reader.getBigDecimal(i, readerContext);
-          creator.putBigDecimal(val);
+          for (int i = 0; i < numDocs; i++) {
+            BigDecimal val = reader.getBigDecimal(i, readerContext);
+            creator.putBigDecimal(val);
+          }
+          break;
         }
-        break;
+        default:
+          throw new IllegalStateException();
+      }
+    }
+  }
+
+  private void enableDictionary(String column, SegmentDirectory.Writer segmentWriter,
+      IndexCreatorProvider indexCreatorProvider)
+      throws Exception {
+    Preconditions.checkState(_segmentMetadata.getVersion() == SegmentVersion.v3);

Review Comment:
   nit: This check needs to be done for every function called to take action. Move it outside so that once we add support for V1, we don't need to remove the check from too many places?



##########
pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/loader/ForwardIndexHandler.java:
##########
@@ -271,118 +306,293 @@ private void rewriteRawSVForwardIndex(String column, ColumnMetadata existingColM
               .forForwardIndex(newCompressionType, _indexLoadingConfig.getColumnProperties());
 
       try (ForwardIndexCreator creator = indexCreatorProvider.newForwardIndexCreator(context)) {
-        // If creator stored type and the reader stored type do not match, throw an exception.
         if (!reader.getStoredType().equals(creator.getValueType())) {
+          // Creator stored type should match reader stored type for raw columns. We do not support changing datatypes.
           String failureMsg =
               "Unsupported operation to change datatype for column=" + column + " from " + reader.getStoredType()
                   .toString() + " to " + creator.getValueType().toString();
           throw new UnsupportedOperationException(failureMsg);
         }
 
         int numDocs = existingColMetadata.getTotalDocs();
-        forwardIndexWriterHelper(column, reader, creator, numDocs);
+        forwardIndexWriterHelper(column, existingColMetadata, reader, creator, numDocs, null);
       }
     }
   }
 
-  private void forwardIndexWriterHelper(String column, ForwardIndexReader reader, ForwardIndexCreator creator,
-      int numDocs) {
-    // If creator stored type should match reader stored type. We do not support changing datatypes.
-    if (!reader.getStoredType().equals(creator.getValueType())) {
-      String failureMsg =
-          "Unsupported operation to change datatype for column=" + column + " from " + reader.getStoredType().toString()
-              + " to " + creator.getValueType().toString();
-      throw new UnsupportedOperationException(failureMsg);
-    }
-
+  private void forwardIndexWriterHelper(String column, ColumnMetadata existingColumnMetadata, ForwardIndexReader reader,
+      ForwardIndexCreator creator, int numDocs, @Nullable SegmentDictionaryCreator dictionaryCreator) {
     ForwardIndexReaderContext readerContext = reader.createContext();
     boolean isSVColumn = reader.isSingleValue();
 
-    switch (reader.getStoredType()) {
-      // JSON fields are either stored as string or bytes. No special handling is needed because we make this
-      // decision based on the storedType of the reader.
-      case INT: {
-        for (int i = 0; i < numDocs; i++) {
-          if (isSVColumn) {
-            int val = reader.getInt(i, readerContext);
-            creator.putInt(val);
-          } else {
-            int[] ints = reader.getIntMV(i, readerContext);
-            creator.putIntMV(ints);
-          }
+    if (dictionaryCreator != null) {
+      int maxNumValuesPerEntry = existingColumnMetadata.getMaxNumberOfMultiValues();
+      PinotSegmentColumnReader columnReader = new PinotSegmentColumnReader(reader, null, null, maxNumValuesPerEntry);
+
+      for (int i = 0; i < numDocs; i++) {
+        Object obj = columnReader.getValue(i);
+
+        if (isSVColumn) {
+          int dictId = dictionaryCreator.indexOfSV(obj);
+          creator.putDictId(dictId);
+        } else {
+          int[] dictIds = dictionaryCreator.indexOfMV(obj);
+          creator.putDictIdMV(dictIds);
         }
-        break;
       }
-      case LONG: {
-        for (int i = 0; i < numDocs; i++) {
-          if (isSVColumn) {
-            long val = reader.getLong(i, readerContext);
-            creator.putLong(val);
-          } else {
-            long[] longs = reader.getLongMV(i, readerContext);
-            creator.putLongMV(longs);
+    } else {
+      switch (reader.getStoredType()) {
+        // JSON fields are either stored as string or bytes. No special handling is needed because we make this
+        // decision based on the storedType of the reader.
+        case INT: {
+          for (int i = 0; i < numDocs; i++) {
+            if (isSVColumn) {
+              int val = reader.getInt(i, readerContext);
+              creator.putInt(val);
+            } else {
+              int[] ints = reader.getIntMV(i, readerContext);
+              creator.putIntMV(ints);
+            }
           }
+          break;
         }
-        break;
-      }
-      case FLOAT: {
-        for (int i = 0; i < numDocs; i++) {
-          if (isSVColumn) {
-            float val = reader.getFloat(i, readerContext);
-            creator.putFloat(val);
-          } else {
-            float[] floats = reader.getFloatMV(i, readerContext);
-            creator.putFloatMV(floats);
+        case LONG: {
+          for (int i = 0; i < numDocs; i++) {
+            if (isSVColumn) {
+              long val = reader.getLong(i, readerContext);
+              creator.putLong(val);
+            } else {
+              long[] longs = reader.getLongMV(i, readerContext);
+              creator.putLongMV(longs);
+            }
           }
+          break;
         }
-        break;
-      }
-      case DOUBLE: {
-        for (int i = 0; i < numDocs; i++) {
-          if (isSVColumn) {
-            double val = reader.getDouble(i, readerContext);
-            creator.putDouble(val);
-          } else {
-            double[] doubles = reader.getDoubleMV(i, readerContext);
-            creator.putDoubleMV(doubles);
+        case FLOAT: {
+          for (int i = 0; i < numDocs; i++) {
+            if (isSVColumn) {
+              float val = reader.getFloat(i, readerContext);
+              creator.putFloat(val);
+            } else {
+              float[] floats = reader.getFloatMV(i, readerContext);
+              creator.putFloatMV(floats);
+            }
           }
+          break;
         }
-        break;
-      }
-      case STRING: {
-        for (int i = 0; i < numDocs; i++) {
-          if (isSVColumn) {
-            String val = reader.getString(i, readerContext);
-            creator.putString(val);
-          } else {
-            String[] strings = reader.getStringMV(i, readerContext);
-            creator.putStringMV(strings);
+        case DOUBLE: {
+          for (int i = 0; i < numDocs; i++) {
+            if (isSVColumn) {
+              double val = reader.getDouble(i, readerContext);
+              creator.putDouble(val);
+            } else {
+              double[] doubles = reader.getDoubleMV(i, readerContext);
+              creator.putDoubleMV(doubles);
+            }
           }
+          break;
         }
-        break;
-      }
-      case BYTES: {
-        for (int i = 0; i < numDocs; i++) {
-          if (isSVColumn) {
-            byte[] val = reader.getBytes(i, readerContext);
-            creator.putBytes(val);
-          } else {
-            byte[][] bytesArray = reader.getBytesMV(i, readerContext);
-            creator.putBytesMV(bytesArray);
+        case STRING: {
+          for (int i = 0; i < numDocs; i++) {
+            if (isSVColumn) {
+              String val = reader.getString(i, readerContext);
+              creator.putString(val);
+            } else {
+              String[] strings = reader.getStringMV(i, readerContext);
+              creator.putStringMV(strings);
+            }
           }
+          break;
         }
-        break;
-      }
-      case BIG_DECIMAL: {
-        for (int i = 0; i < numDocs; i++) {
+        case BYTES: {
+          for (int i = 0; i < numDocs; i++) {
+            if (isSVColumn) {
+              byte[] val = reader.getBytes(i, readerContext);
+              creator.putBytes(val);
+            } else {
+              byte[][] bytesArray = reader.getBytesMV(i, readerContext);
+              creator.putBytesMV(bytesArray);
+            }
+          }
+          break;
+        }
+        case BIG_DECIMAL: {
           Preconditions.checkState(isSVColumn, "BigDecimal is not supported for MV columns");
-          BigDecimal val = reader.getBigDecimal(i, readerContext);
-          creator.putBigDecimal(val);
+          for (int i = 0; i < numDocs; i++) {
+            BigDecimal val = reader.getBigDecimal(i, readerContext);
+            creator.putBigDecimal(val);
+          }
+          break;
         }
-        break;
+        default:
+          throw new IllegalStateException();
+      }
+    }
+  }
+
+  private void enableDictionary(String column, SegmentDirectory.Writer segmentWriter,
+      IndexCreatorProvider indexCreatorProvider)
+      throws Exception {
+    Preconditions.checkState(_segmentMetadata.getVersion() == SegmentVersion.v3);
+    ColumnMetadata existingColMetadata = _segmentMetadata.getColumnMetadataFor(column);
+    Preconditions.checkState(!existingColMetadata.hasDictionary(), "Existing column already has dictionary.");
+    boolean isSingleValue = existingColMetadata.isSingleValue();
+
+    File indexDir = _segmentMetadata.getIndexDir();
+    String segmentName = _segmentMetadata.getName();
+    File inProgress = new File(indexDir, column + ".dict.inprogress");
+    File dictionaryFile = new File(indexDir, column + V1Constants.Dict.FILE_EXTENSION);
+    String fwdIndexFileExtension;
+    if (isSingleValue) {
+      if (existingColMetadata.isSorted()) {
+        fwdIndexFileExtension = V1Constants.Indexes.SORTED_SV_FORWARD_INDEX_FILE_EXTENSION;
+      } else {
+        fwdIndexFileExtension = V1Constants.Indexes.UNSORTED_SV_FORWARD_INDEX_FILE_EXTENSION;
+      }

Review Comment:
   nit: simplify using conditionals?
   
   ```
   fwdIndexFileExtension = existingColMetadata.isSorted() ? V1Constants.Indexes.SORTED_SV_FORWARD_INDEX_FILE_EXTENSION : V1Constants.Indexes.SORTED_SV_FORWARD_INDEX_FILE_EXTENSION;
   ```



##########
pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/loader/ForwardIndexHandler.java:
##########
@@ -271,118 +306,293 @@ private void rewriteRawSVForwardIndex(String column, ColumnMetadata existingColM
               .forForwardIndex(newCompressionType, _indexLoadingConfig.getColumnProperties());
 
       try (ForwardIndexCreator creator = indexCreatorProvider.newForwardIndexCreator(context)) {
-        // If creator stored type and the reader stored type do not match, throw an exception.
         if (!reader.getStoredType().equals(creator.getValueType())) {
+          // Creator stored type should match reader stored type for raw columns. We do not support changing datatypes.
           String failureMsg =
               "Unsupported operation to change datatype for column=" + column + " from " + reader.getStoredType()
                   .toString() + " to " + creator.getValueType().toString();
           throw new UnsupportedOperationException(failureMsg);
         }
 
         int numDocs = existingColMetadata.getTotalDocs();
-        forwardIndexWriterHelper(column, reader, creator, numDocs);
+        forwardIndexWriterHelper(column, existingColMetadata, reader, creator, numDocs, null);
       }
     }
   }
 
-  private void forwardIndexWriterHelper(String column, ForwardIndexReader reader, ForwardIndexCreator creator,
-      int numDocs) {
-    // If creator stored type should match reader stored type. We do not support changing datatypes.
-    if (!reader.getStoredType().equals(creator.getValueType())) {
-      String failureMsg =
-          "Unsupported operation to change datatype for column=" + column + " from " + reader.getStoredType().toString()
-              + " to " + creator.getValueType().toString();
-      throw new UnsupportedOperationException(failureMsg);
-    }
-
+  private void forwardIndexWriterHelper(String column, ColumnMetadata existingColumnMetadata, ForwardIndexReader reader,
+      ForwardIndexCreator creator, int numDocs, @Nullable SegmentDictionaryCreator dictionaryCreator) {
     ForwardIndexReaderContext readerContext = reader.createContext();
     boolean isSVColumn = reader.isSingleValue();
 
-    switch (reader.getStoredType()) {
-      // JSON fields are either stored as string or bytes. No special handling is needed because we make this
-      // decision based on the storedType of the reader.
-      case INT: {
-        for (int i = 0; i < numDocs; i++) {
-          if (isSVColumn) {
-            int val = reader.getInt(i, readerContext);
-            creator.putInt(val);
-          } else {
-            int[] ints = reader.getIntMV(i, readerContext);
-            creator.putIntMV(ints);
-          }
+    if (dictionaryCreator != null) {
+      int maxNumValuesPerEntry = existingColumnMetadata.getMaxNumberOfMultiValues();
+      PinotSegmentColumnReader columnReader = new PinotSegmentColumnReader(reader, null, null, maxNumValuesPerEntry);
+
+      for (int i = 0; i < numDocs; i++) {
+        Object obj = columnReader.getValue(i);
+
+        if (isSVColumn) {
+          int dictId = dictionaryCreator.indexOfSV(obj);
+          creator.putDictId(dictId);
+        } else {
+          int[] dictIds = dictionaryCreator.indexOfMV(obj);
+          creator.putDictIdMV(dictIds);
         }
-        break;
       }
-      case LONG: {
-        for (int i = 0; i < numDocs; i++) {
-          if (isSVColumn) {
-            long val = reader.getLong(i, readerContext);
-            creator.putLong(val);
-          } else {
-            long[] longs = reader.getLongMV(i, readerContext);
-            creator.putLongMV(longs);
+    } else {
+      switch (reader.getStoredType()) {
+        // JSON fields are either stored as string or bytes. No special handling is needed because we make this
+        // decision based on the storedType of the reader.
+        case INT: {
+          for (int i = 0; i < numDocs; i++) {
+            if (isSVColumn) {
+              int val = reader.getInt(i, readerContext);
+              creator.putInt(val);
+            } else {
+              int[] ints = reader.getIntMV(i, readerContext);
+              creator.putIntMV(ints);
+            }
           }
+          break;
         }
-        break;
-      }
-      case FLOAT: {
-        for (int i = 0; i < numDocs; i++) {
-          if (isSVColumn) {
-            float val = reader.getFloat(i, readerContext);
-            creator.putFloat(val);
-          } else {
-            float[] floats = reader.getFloatMV(i, readerContext);
-            creator.putFloatMV(floats);
+        case LONG: {
+          for (int i = 0; i < numDocs; i++) {
+            if (isSVColumn) {
+              long val = reader.getLong(i, readerContext);
+              creator.putLong(val);
+            } else {
+              long[] longs = reader.getLongMV(i, readerContext);
+              creator.putLongMV(longs);
+            }
           }
+          break;
         }
-        break;
-      }
-      case DOUBLE: {
-        for (int i = 0; i < numDocs; i++) {
-          if (isSVColumn) {
-            double val = reader.getDouble(i, readerContext);
-            creator.putDouble(val);
-          } else {
-            double[] doubles = reader.getDoubleMV(i, readerContext);
-            creator.putDoubleMV(doubles);
+        case FLOAT: {
+          for (int i = 0; i < numDocs; i++) {
+            if (isSVColumn) {
+              float val = reader.getFloat(i, readerContext);
+              creator.putFloat(val);
+            } else {
+              float[] floats = reader.getFloatMV(i, readerContext);
+              creator.putFloatMV(floats);
+            }
           }
+          break;
         }
-        break;
-      }
-      case STRING: {
-        for (int i = 0; i < numDocs; i++) {
-          if (isSVColumn) {
-            String val = reader.getString(i, readerContext);
-            creator.putString(val);
-          } else {
-            String[] strings = reader.getStringMV(i, readerContext);
-            creator.putStringMV(strings);
+        case DOUBLE: {
+          for (int i = 0; i < numDocs; i++) {
+            if (isSVColumn) {
+              double val = reader.getDouble(i, readerContext);
+              creator.putDouble(val);
+            } else {
+              double[] doubles = reader.getDoubleMV(i, readerContext);
+              creator.putDoubleMV(doubles);
+            }
           }
+          break;
         }
-        break;
-      }
-      case BYTES: {
-        for (int i = 0; i < numDocs; i++) {
-          if (isSVColumn) {
-            byte[] val = reader.getBytes(i, readerContext);
-            creator.putBytes(val);
-          } else {
-            byte[][] bytesArray = reader.getBytesMV(i, readerContext);
-            creator.putBytesMV(bytesArray);
+        case STRING: {
+          for (int i = 0; i < numDocs; i++) {
+            if (isSVColumn) {
+              String val = reader.getString(i, readerContext);
+              creator.putString(val);
+            } else {
+              String[] strings = reader.getStringMV(i, readerContext);
+              creator.putStringMV(strings);
+            }
           }
+          break;
         }
-        break;
-      }
-      case BIG_DECIMAL: {
-        for (int i = 0; i < numDocs; i++) {
+        case BYTES: {
+          for (int i = 0; i < numDocs; i++) {
+            if (isSVColumn) {
+              byte[] val = reader.getBytes(i, readerContext);
+              creator.putBytes(val);
+            } else {
+              byte[][] bytesArray = reader.getBytesMV(i, readerContext);
+              creator.putBytesMV(bytesArray);
+            }
+          }
+          break;
+        }
+        case BIG_DECIMAL: {
           Preconditions.checkState(isSVColumn, "BigDecimal is not supported for MV columns");
-          BigDecimal val = reader.getBigDecimal(i, readerContext);
-          creator.putBigDecimal(val);
+          for (int i = 0; i < numDocs; i++) {
+            BigDecimal val = reader.getBigDecimal(i, readerContext);
+            creator.putBigDecimal(val);
+          }
+          break;
         }
-        break;
+        default:
+          throw new IllegalStateException();
+      }
+    }
+  }
+
+  private void enableDictionary(String column, SegmentDirectory.Writer segmentWriter,
+      IndexCreatorProvider indexCreatorProvider)
+      throws Exception {
+    Preconditions.checkState(_segmentMetadata.getVersion() == SegmentVersion.v3);
+    ColumnMetadata existingColMetadata = _segmentMetadata.getColumnMetadataFor(column);
+    Preconditions.checkState(!existingColMetadata.hasDictionary(), "Existing column already has dictionary.");

Review Comment:
   nit: Modify the error message to indicate what operation we are doing? e.g. "Cannot rewrite dictionary enabled forward index, dictionary already exists for column: {}"



##########
pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/loader/ForwardIndexHandler.java:
##########
@@ -271,118 +306,293 @@ private void rewriteRawSVForwardIndex(String column, ColumnMetadata existingColM
               .forForwardIndex(newCompressionType, _indexLoadingConfig.getColumnProperties());
 
       try (ForwardIndexCreator creator = indexCreatorProvider.newForwardIndexCreator(context)) {
-        // If creator stored type and the reader stored type do not match, throw an exception.
         if (!reader.getStoredType().equals(creator.getValueType())) {
+          // Creator stored type should match reader stored type for raw columns. We do not support changing datatypes.
           String failureMsg =
               "Unsupported operation to change datatype for column=" + column + " from " + reader.getStoredType()
                   .toString() + " to " + creator.getValueType().toString();
           throw new UnsupportedOperationException(failureMsg);
         }
 
         int numDocs = existingColMetadata.getTotalDocs();
-        forwardIndexWriterHelper(column, reader, creator, numDocs);
+        forwardIndexWriterHelper(column, existingColMetadata, reader, creator, numDocs, null);
       }
     }
   }
 
-  private void forwardIndexWriterHelper(String column, ForwardIndexReader reader, ForwardIndexCreator creator,
-      int numDocs) {
-    // If creator stored type should match reader stored type. We do not support changing datatypes.
-    if (!reader.getStoredType().equals(creator.getValueType())) {
-      String failureMsg =
-          "Unsupported operation to change datatype for column=" + column + " from " + reader.getStoredType().toString()
-              + " to " + creator.getValueType().toString();
-      throw new UnsupportedOperationException(failureMsg);
-    }
-
+  private void forwardIndexWriterHelper(String column, ColumnMetadata existingColumnMetadata, ForwardIndexReader reader,
+      ForwardIndexCreator creator, int numDocs, @Nullable SegmentDictionaryCreator dictionaryCreator) {
     ForwardIndexReaderContext readerContext = reader.createContext();
     boolean isSVColumn = reader.isSingleValue();
 
-    switch (reader.getStoredType()) {
-      // JSON fields are either stored as string or bytes. No special handling is needed because we make this
-      // decision based on the storedType of the reader.
-      case INT: {
-        for (int i = 0; i < numDocs; i++) {
-          if (isSVColumn) {
-            int val = reader.getInt(i, readerContext);
-            creator.putInt(val);
-          } else {
-            int[] ints = reader.getIntMV(i, readerContext);
-            creator.putIntMV(ints);
-          }
+    if (dictionaryCreator != null) {
+      int maxNumValuesPerEntry = existingColumnMetadata.getMaxNumberOfMultiValues();
+      PinotSegmentColumnReader columnReader = new PinotSegmentColumnReader(reader, null, null, maxNumValuesPerEntry);
+
+      for (int i = 0; i < numDocs; i++) {
+        Object obj = columnReader.getValue(i);
+
+        if (isSVColumn) {
+          int dictId = dictionaryCreator.indexOfSV(obj);
+          creator.putDictId(dictId);
+        } else {
+          int[] dictIds = dictionaryCreator.indexOfMV(obj);
+          creator.putDictIdMV(dictIds);
         }
-        break;
       }
-      case LONG: {
-        for (int i = 0; i < numDocs; i++) {
-          if (isSVColumn) {
-            long val = reader.getLong(i, readerContext);
-            creator.putLong(val);
-          } else {
-            long[] longs = reader.getLongMV(i, readerContext);
-            creator.putLongMV(longs);
+    } else {
+      switch (reader.getStoredType()) {
+        // JSON fields are either stored as string or bytes. No special handling is needed because we make this
+        // decision based on the storedType of the reader.
+        case INT: {
+          for (int i = 0; i < numDocs; i++) {
+            if (isSVColumn) {
+              int val = reader.getInt(i, readerContext);
+              creator.putInt(val);
+            } else {
+              int[] ints = reader.getIntMV(i, readerContext);
+              creator.putIntMV(ints);
+            }
           }
+          break;
         }
-        break;
-      }
-      case FLOAT: {
-        for (int i = 0; i < numDocs; i++) {
-          if (isSVColumn) {
-            float val = reader.getFloat(i, readerContext);
-            creator.putFloat(val);
-          } else {
-            float[] floats = reader.getFloatMV(i, readerContext);
-            creator.putFloatMV(floats);
+        case LONG: {
+          for (int i = 0; i < numDocs; i++) {
+            if (isSVColumn) {
+              long val = reader.getLong(i, readerContext);
+              creator.putLong(val);
+            } else {
+              long[] longs = reader.getLongMV(i, readerContext);
+              creator.putLongMV(longs);
+            }
           }
+          break;
         }
-        break;
-      }
-      case DOUBLE: {
-        for (int i = 0; i < numDocs; i++) {
-          if (isSVColumn) {
-            double val = reader.getDouble(i, readerContext);
-            creator.putDouble(val);
-          } else {
-            double[] doubles = reader.getDoubleMV(i, readerContext);
-            creator.putDoubleMV(doubles);
+        case FLOAT: {
+          for (int i = 0; i < numDocs; i++) {
+            if (isSVColumn) {
+              float val = reader.getFloat(i, readerContext);
+              creator.putFloat(val);
+            } else {
+              float[] floats = reader.getFloatMV(i, readerContext);
+              creator.putFloatMV(floats);
+            }
           }
+          break;
         }
-        break;
-      }
-      case STRING: {
-        for (int i = 0; i < numDocs; i++) {
-          if (isSVColumn) {
-            String val = reader.getString(i, readerContext);
-            creator.putString(val);
-          } else {
-            String[] strings = reader.getStringMV(i, readerContext);
-            creator.putStringMV(strings);
+        case DOUBLE: {
+          for (int i = 0; i < numDocs; i++) {
+            if (isSVColumn) {
+              double val = reader.getDouble(i, readerContext);
+              creator.putDouble(val);
+            } else {
+              double[] doubles = reader.getDoubleMV(i, readerContext);
+              creator.putDoubleMV(doubles);
+            }
           }
+          break;
         }
-        break;
-      }
-      case BYTES: {
-        for (int i = 0; i < numDocs; i++) {
-          if (isSVColumn) {
-            byte[] val = reader.getBytes(i, readerContext);
-            creator.putBytes(val);
-          } else {
-            byte[][] bytesArray = reader.getBytesMV(i, readerContext);
-            creator.putBytesMV(bytesArray);
+        case STRING: {
+          for (int i = 0; i < numDocs; i++) {
+            if (isSVColumn) {
+              String val = reader.getString(i, readerContext);
+              creator.putString(val);
+            } else {
+              String[] strings = reader.getStringMV(i, readerContext);
+              creator.putStringMV(strings);
+            }
           }
+          break;
         }
-        break;
-      }
-      case BIG_DECIMAL: {
-        for (int i = 0; i < numDocs; i++) {
+        case BYTES: {
+          for (int i = 0; i < numDocs; i++) {
+            if (isSVColumn) {
+              byte[] val = reader.getBytes(i, readerContext);
+              creator.putBytes(val);
+            } else {
+              byte[][] bytesArray = reader.getBytesMV(i, readerContext);
+              creator.putBytesMV(bytesArray);
+            }
+          }
+          break;
+        }
+        case BIG_DECIMAL: {
           Preconditions.checkState(isSVColumn, "BigDecimal is not supported for MV columns");
-          BigDecimal val = reader.getBigDecimal(i, readerContext);
-          creator.putBigDecimal(val);
+          for (int i = 0; i < numDocs; i++) {
+            BigDecimal val = reader.getBigDecimal(i, readerContext);
+            creator.putBigDecimal(val);
+          }
+          break;
         }
-        break;
+        default:
+          throw new IllegalStateException();
+      }
+    }
+  }
+
+  private void enableDictionary(String column, SegmentDirectory.Writer segmentWriter,
+      IndexCreatorProvider indexCreatorProvider)
+      throws Exception {
+    Preconditions.checkState(_segmentMetadata.getVersion() == SegmentVersion.v3);
+    ColumnMetadata existingColMetadata = _segmentMetadata.getColumnMetadataFor(column);
+    Preconditions.checkState(!existingColMetadata.hasDictionary(), "Existing column already has dictionary.");
+    boolean isSingleValue = existingColMetadata.isSingleValue();
+
+    File indexDir = _segmentMetadata.getIndexDir();
+    String segmentName = _segmentMetadata.getName();
+    File inProgress = new File(indexDir, column + ".dict.inprogress");
+    File dictionaryFile = new File(indexDir, column + V1Constants.Dict.FILE_EXTENSION);
+    String fwdIndexFileExtension;
+    if (isSingleValue) {
+      if (existingColMetadata.isSorted()) {
+        fwdIndexFileExtension = V1Constants.Indexes.SORTED_SV_FORWARD_INDEX_FILE_EXTENSION;
+      } else {
+        fwdIndexFileExtension = V1Constants.Indexes.UNSORTED_SV_FORWARD_INDEX_FILE_EXTENSION;
+      }
+    } else {
+      fwdIndexFileExtension = V1Constants.Indexes.UNSORTED_MV_FORWARD_INDEX_FILE_EXTENSION;
+    }
+    File fwdIndexFile = new File(indexDir, column + fwdIndexFileExtension);
+
+    if (!inProgress.exists()) {
+      // Marker file does not exist, which means last run ended normally.
+      // Create a marker file.
+      FileUtils.touch(inProgress);
+    } else {
+      // Marker file exists, which means last run was interrupted.
+      // Remove forward index and dictionary files if they exist.
+      FileUtils.deleteQuietly(fwdIndexFile);
+      FileUtils.deleteQuietly(dictionaryFile);
+    }
+
+    LOGGER.info("Creating a new dictionary for segment={} and column={}", segmentName, column);
+    SegmentDictionaryCreator dictionaryCreator = buildDictionary(column, existingColMetadata, segmentWriter);
+    LoaderUtils.writeIndexToV3Format(segmentWriter, column, dictionaryFile, ColumnIndexType.DICTIONARY);
+
+    LOGGER.info("Built dictionary. Rewriting dictionary enabled forward index for segment={} and column={}",
+        segmentName, column);
+    writeDictEnabledForwardIndex(column, existingColMetadata, segmentWriter, indexDir, indexCreatorProvider,
+        dictionaryCreator);
+    // We used the existing forward index to generate a new forward index. The existing forward index will be in V3
+    // format and the new forward index will be in V1 format. Remove the existing forward index as it is not needed
+    // anymore. Note that removeIndex() will only mark an index for removal and remove the in-memory state. The
+    // actual cleanup from columns.psf file will happen when singleFileIndexDirectory.cleanupRemovedIndices() is
+    // called during segmentWriter.close().
+    segmentWriter.removeIndex(column, ColumnIndexType.FORWARD_INDEX);
+    LoaderUtils.writeIndexToV3Format(segmentWriter, column, fwdIndexFile, ColumnIndexType.FORWARD_INDEX);
+
+    LOGGER.info("Created forwardIndex. Updating metadata properties for segment={} and column={}", segmentName, column);
+    updateMetadataProperties(indexDir, column, dictionaryCreator);
+
+    // We remove indexes that have to be rewritten when a dictEnabled is toggled. Note that the respective index
+    // handler will take care of recreating the index.
+    removeDictRelatedIndexes(column, segmentWriter);
+
+    // Delete the marker file.
+    FileUtils.deleteQuietly(inProgress);
+
+    LOGGER.info("Created dictionary based forward index for segment: {}, column: {}", segmentName, column);
+  }
+
+  private SegmentDictionaryCreator buildDictionary(String column, ColumnMetadata existingColMetadata,
+      SegmentDirectory.Writer segmentWriter)
+      throws Exception {
+    int numDocs = existingColMetadata.getTotalDocs();
+    // SegmentPartitionConfig is not relevant for rewrites.
+    Preconditions.checkState(_indexLoadingConfig.getTableConfig() != null);
+    StatsCollectorConfig statsCollectorConfig =
+        new StatsCollectorConfig(_indexLoadingConfig.getTableConfig(), _schema, null);
+    AbstractColumnStatisticsCollector statsCollector;
+
+    try (ForwardIndexReader reader = LoaderUtils.getForwardIndexReader(segmentWriter, existingColMetadata)) {
+      boolean isSVColumn = reader.isSingleValue();
+
+      switch (reader.getStoredType()) {
+        case INT:
+          statsCollector = new IntColumnPreIndexStatsCollector(column, statsCollectorConfig);
+          break;
+        case LONG:
+          statsCollector = new LongColumnPreIndexStatsCollector(column, statsCollectorConfig);
+          break;
+        case FLOAT:
+          statsCollector = new FloatColumnPreIndexStatsCollector(column, statsCollectorConfig);
+          break;
+        case DOUBLE:
+          statsCollector = new DoubleColumnPreIndexStatsCollector(column, statsCollectorConfig);
+          break;
+        case STRING:
+          statsCollector = new StringColumnPreIndexStatsCollector(column, statsCollectorConfig);
+          break;
+        case BYTES:
+          statsCollector = new BytesColumnPredIndexStatsCollector(column, statsCollectorConfig);
+          break;
+        case BIG_DECIMAL:
+          Preconditions.checkState(isSVColumn, "BigDecimal is not supported for MV columns");
+          statsCollector = new BigDecimalColumnPreIndexStatsCollector(column, statsCollectorConfig);
+          break;
+        default:
+          throw new IllegalStateException();
+      }
+
+      Preconditions.checkState(statsCollector != null);
+      // Note: Special Null handling is not necessary here. This is because, the existing default null value in the
+      // raw forwardIndex will be retained as such while created the dictionary and dict-based forward index. Also,
+      // null value vectors maintain a bitmap of docIds. No handling is necessary there.
+      PinotSegmentColumnReader columnReader =
+          new PinotSegmentColumnReader(reader, null, null, existingColMetadata.getMaxNumberOfMultiValues());
+      for (int i = 0; i < numDocs; i++) {
+        Object obj = columnReader.getValue(i);
+        statsCollector.collect(obj);
+      }
+      statsCollector.seal();
+
+      boolean useVarLength = _indexLoadingConfig.getVarLengthDictionaryColumns().contains(column)
+          || SegmentIndexCreationDriverImpl.shouldUseVarLengthDictionary(reader.getStoredType(), statsCollector);
+      SegmentDictionaryCreator dictionaryCreator =
+          new SegmentDictionaryCreator(existingColMetadata.getFieldSpec(), _segmentMetadata.getIndexDir(),
+              useVarLength);
+
+      dictionaryCreator.build(statsCollector.getUniqueValuesSet());

Review Comment:
   just curious, can't the data collected by the `statsCollector` be used to set up the `IndexCreationContext` in the next step? Just wondering if there is a need to pull any of the other stats to create the forward index. I know we did some ad-hoc calculations for the RAW forward index scenario, which may have benefitted from running the stats collector and using these values, right?



##########
pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/loader/SegmentPreProcessor.java:
##########
@@ -104,8 +104,14 @@ public void process()
       // Update single-column indices, like inverted index, json index etc.
       IndexCreatorProvider indexCreatorProvider = IndexingOverrides.getIndexCreatorProvider();
       for (ColumnIndexType type : ColumnIndexType.values()) {
-        IndexHandler handler = IndexHandlerFactory.getIndexHandler(type, _segmentMetadata, _indexLoadingConfig);
+        IndexHandler handler =
+            IndexHandlerFactory.getIndexHandler(type, _segmentMetadata, _indexLoadingConfig, _schema);
         handler.updateIndices(segmentWriter, indexCreatorProvider);
+        if (type == ColumnIndexType.FORWARD_INDEX) {
+          // ForwardIndexHandler may modify the segment metadata.

Review Comment:
   Suggest enhancing the comment to mention why you need to do this here and not outside the index handling loop. The metadata update will be needed to construct the other indexes such as Range index, right?



##########
pinot-segment-local/src/test/java/org/apache/pinot/segment/local/segment/index/loader/ForwardIndexHandlerTest.java:
##########
@@ -302,34 +351,54 @@ public void testComputeOperation()
 
     // TEST1 : Validate with zero changes. ForwardIndexHandler should be a No-Op.
     IndexLoadingConfig indexLoadingConfig = new IndexLoadingConfig(null, _tableConfig);
-    ForwardIndexHandler fwdIndexHandler = new ForwardIndexHandler(existingSegmentMetadata, indexLoadingConfig);
+    ForwardIndexHandler fwdIndexHandler = new ForwardIndexHandler(existingSegmentMetadata, indexLoadingConfig, null);
     Map<String, ForwardIndexHandler.Operation> operationMap = new HashMap<>();
     operationMap = fwdIndexHandler.computeOperation(writer);
     assertEquals(operationMap, Collections.EMPTY_MAP);
 
-    // TEST2: Enable dictionary for a RAW_ZSTANDARD_INDEX_COLUMN. ForwardIndexHandler should be a No-Op.
+    // TEST2: Enable dictionary for a RAW_ZSTANDARD_INDEX_COLUMN.
     indexLoadingConfig = new IndexLoadingConfig(null, _tableConfig);
     indexLoadingConfig.getNoDictionaryColumns().remove(DIM_ZSTANDARD_STRING);
-    fwdIndexHandler = new ForwardIndexHandler(existingSegmentMetadata, indexLoadingConfig);
+    fwdIndexHandler = new ForwardIndexHandler(existingSegmentMetadata, indexLoadingConfig, _schema);
+    operationMap = fwdIndexHandler.computeOperation(writer);
+    assertEquals(operationMap.get(DIM_ZSTANDARD_STRING), ForwardIndexHandler.Operation.ENABLE_DICTIONARY);
+
+    // TEST3: Enable dictionary for an MV column.
+    indexLoadingConfig = new IndexLoadingConfig(null, _tableConfig);
+    indexLoadingConfig.getNoDictionaryColumns().remove(DIM_MV_PASS_THROUGH_STRING);
+    fwdIndexHandler = new ForwardIndexHandler(existingSegmentMetadata, indexLoadingConfig, _schema);
+    operationMap = fwdIndexHandler.computeOperation(writer);
+    assertEquals(operationMap.get(DIM_MV_PASS_THROUGH_STRING), ForwardIndexHandler.Operation.ENABLE_DICTIONARY);
+
+    // TEST4: Enable dictionary for a sorted column.
+    indexLoadingConfig = new IndexLoadingConfig(null, _tableConfig);
+    indexLoadingConfig.getNoDictionaryColumns().remove(DIM_PASS_THROUGH_SORTED_LONG);
+    fwdIndexHandler = new ForwardIndexHandler(existingSegmentMetadata, indexLoadingConfig, _schema);
+    operationMap = fwdIndexHandler.computeOperation(writer);
+    assertEquals(operationMap.get(DIM_PASS_THROUGH_SORTED_LONG), ForwardIndexHandler.Operation.ENABLE_DICTIONARY);
+
+    // TEST5: Enable dictionary for a dict column. Should be a No-op.
+    indexLoadingConfig = new IndexLoadingConfig(null, _tableConfig);
+    fwdIndexHandler = new ForwardIndexHandler(existingSegmentMetadata, indexLoadingConfig, _schema);
     operationMap = fwdIndexHandler.computeOperation(writer);
     assertEquals(operationMap, Collections.EMPTY_MAP);
 
-    // TEST3: Disable dictionary. ForwardIndexHandler should be a No-Op.
+    // TEST6: Disable dictionary. Should be a No-op.
     indexLoadingConfig = new IndexLoadingConfig(null, _tableConfig);
     indexLoadingConfig.getNoDictionaryColumns().add(DIM_DICT_INTEGER);
-    fwdIndexHandler = new ForwardIndexHandler(existingSegmentMetadata, indexLoadingConfig);
+    fwdIndexHandler = new ForwardIndexHandler(existingSegmentMetadata, indexLoadingConfig, null);
     operationMap = fwdIndexHandler.computeOperation(writer);
     assertEquals(operationMap, Collections.EMPTY_MAP);
 
-    // TEST4: Add an additional text index. ForwardIndexHandler should be a No-Op.
+    // TEST7: Add an additional text index. ForwardIndexHandler should be a No-Op.
     indexLoadingConfig = new IndexLoadingConfig(null, _tableConfig);
     indexLoadingConfig.getTextIndexColumns().add(DIM_DICT_INTEGER);
     indexLoadingConfig.getTextIndexColumns().add(DIM_LZ4_INTEGER);
-    fwdIndexHandler = new ForwardIndexHandler(existingSegmentMetadata, indexLoadingConfig);
+    fwdIndexHandler = new ForwardIndexHandler(existingSegmentMetadata, indexLoadingConfig, null);

Review Comment:
   did you test adding a range index and enabling forward index at the same time?



##########
pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/loader/ForwardIndexHandler.java:
##########
@@ -271,118 +306,293 @@ private void rewriteRawSVForwardIndex(String column, ColumnMetadata existingColM
               .forForwardIndex(newCompressionType, _indexLoadingConfig.getColumnProperties());
 
       try (ForwardIndexCreator creator = indexCreatorProvider.newForwardIndexCreator(context)) {
-        // If creator stored type and the reader stored type do not match, throw an exception.
         if (!reader.getStoredType().equals(creator.getValueType())) {
+          // Creator stored type should match reader stored type for raw columns. We do not support changing datatypes.
           String failureMsg =
               "Unsupported operation to change datatype for column=" + column + " from " + reader.getStoredType()
                   .toString() + " to " + creator.getValueType().toString();
           throw new UnsupportedOperationException(failureMsg);
         }
 
         int numDocs = existingColMetadata.getTotalDocs();
-        forwardIndexWriterHelper(column, reader, creator, numDocs);
+        forwardIndexWriterHelper(column, existingColMetadata, reader, creator, numDocs, null);
       }
     }
   }
 
-  private void forwardIndexWriterHelper(String column, ForwardIndexReader reader, ForwardIndexCreator creator,
-      int numDocs) {
-    // If creator stored type should match reader stored type. We do not support changing datatypes.
-    if (!reader.getStoredType().equals(creator.getValueType())) {
-      String failureMsg =
-          "Unsupported operation to change datatype for column=" + column + " from " + reader.getStoredType().toString()
-              + " to " + creator.getValueType().toString();
-      throw new UnsupportedOperationException(failureMsg);
-    }
-
+  private void forwardIndexWriterHelper(String column, ColumnMetadata existingColumnMetadata, ForwardIndexReader reader,
+      ForwardIndexCreator creator, int numDocs, @Nullable SegmentDictionaryCreator dictionaryCreator) {
     ForwardIndexReaderContext readerContext = reader.createContext();
     boolean isSVColumn = reader.isSingleValue();
 
-    switch (reader.getStoredType()) {
-      // JSON fields are either stored as string or bytes. No special handling is needed because we make this
-      // decision based on the storedType of the reader.
-      case INT: {
-        for (int i = 0; i < numDocs; i++) {
-          if (isSVColumn) {
-            int val = reader.getInt(i, readerContext);
-            creator.putInt(val);
-          } else {
-            int[] ints = reader.getIntMV(i, readerContext);
-            creator.putIntMV(ints);
-          }
+    if (dictionaryCreator != null) {
+      int maxNumValuesPerEntry = existingColumnMetadata.getMaxNumberOfMultiValues();
+      PinotSegmentColumnReader columnReader = new PinotSegmentColumnReader(reader, null, null, maxNumValuesPerEntry);
+
+      for (int i = 0; i < numDocs; i++) {
+        Object obj = columnReader.getValue(i);
+
+        if (isSVColumn) {
+          int dictId = dictionaryCreator.indexOfSV(obj);
+          creator.putDictId(dictId);
+        } else {
+          int[] dictIds = dictionaryCreator.indexOfMV(obj);
+          creator.putDictIdMV(dictIds);
         }
-        break;
       }
-      case LONG: {
-        for (int i = 0; i < numDocs; i++) {
-          if (isSVColumn) {
-            long val = reader.getLong(i, readerContext);
-            creator.putLong(val);
-          } else {
-            long[] longs = reader.getLongMV(i, readerContext);
-            creator.putLongMV(longs);
+    } else {
+      switch (reader.getStoredType()) {
+        // JSON fields are either stored as string or bytes. No special handling is needed because we make this
+        // decision based on the storedType of the reader.
+        case INT: {
+          for (int i = 0; i < numDocs; i++) {
+            if (isSVColumn) {
+              int val = reader.getInt(i, readerContext);
+              creator.putInt(val);
+            } else {
+              int[] ints = reader.getIntMV(i, readerContext);
+              creator.putIntMV(ints);
+            }
           }
+          break;
         }
-        break;
-      }
-      case FLOAT: {
-        for (int i = 0; i < numDocs; i++) {
-          if (isSVColumn) {
-            float val = reader.getFloat(i, readerContext);
-            creator.putFloat(val);
-          } else {
-            float[] floats = reader.getFloatMV(i, readerContext);
-            creator.putFloatMV(floats);
+        case LONG: {
+          for (int i = 0; i < numDocs; i++) {
+            if (isSVColumn) {
+              long val = reader.getLong(i, readerContext);
+              creator.putLong(val);
+            } else {
+              long[] longs = reader.getLongMV(i, readerContext);
+              creator.putLongMV(longs);
+            }
           }
+          break;
         }
-        break;
-      }
-      case DOUBLE: {
-        for (int i = 0; i < numDocs; i++) {
-          if (isSVColumn) {
-            double val = reader.getDouble(i, readerContext);
-            creator.putDouble(val);
-          } else {
-            double[] doubles = reader.getDoubleMV(i, readerContext);
-            creator.putDoubleMV(doubles);
+        case FLOAT: {
+          for (int i = 0; i < numDocs; i++) {
+            if (isSVColumn) {
+              float val = reader.getFloat(i, readerContext);
+              creator.putFloat(val);
+            } else {
+              float[] floats = reader.getFloatMV(i, readerContext);
+              creator.putFloatMV(floats);
+            }
           }
+          break;
         }
-        break;
-      }
-      case STRING: {
-        for (int i = 0; i < numDocs; i++) {
-          if (isSVColumn) {
-            String val = reader.getString(i, readerContext);
-            creator.putString(val);
-          } else {
-            String[] strings = reader.getStringMV(i, readerContext);
-            creator.putStringMV(strings);
+        case DOUBLE: {
+          for (int i = 0; i < numDocs; i++) {
+            if (isSVColumn) {
+              double val = reader.getDouble(i, readerContext);
+              creator.putDouble(val);
+            } else {
+              double[] doubles = reader.getDoubleMV(i, readerContext);
+              creator.putDoubleMV(doubles);
+            }
           }
+          break;
         }
-        break;
-      }
-      case BYTES: {
-        for (int i = 0; i < numDocs; i++) {
-          if (isSVColumn) {
-            byte[] val = reader.getBytes(i, readerContext);
-            creator.putBytes(val);
-          } else {
-            byte[][] bytesArray = reader.getBytesMV(i, readerContext);
-            creator.putBytesMV(bytesArray);
+        case STRING: {
+          for (int i = 0; i < numDocs; i++) {
+            if (isSVColumn) {
+              String val = reader.getString(i, readerContext);
+              creator.putString(val);
+            } else {
+              String[] strings = reader.getStringMV(i, readerContext);
+              creator.putStringMV(strings);
+            }
           }
+          break;
         }
-        break;
-      }
-      case BIG_DECIMAL: {
-        for (int i = 0; i < numDocs; i++) {
+        case BYTES: {
+          for (int i = 0; i < numDocs; i++) {
+            if (isSVColumn) {
+              byte[] val = reader.getBytes(i, readerContext);
+              creator.putBytes(val);
+            } else {
+              byte[][] bytesArray = reader.getBytesMV(i, readerContext);
+              creator.putBytesMV(bytesArray);
+            }
+          }
+          break;
+        }
+        case BIG_DECIMAL: {
           Preconditions.checkState(isSVColumn, "BigDecimal is not supported for MV columns");
-          BigDecimal val = reader.getBigDecimal(i, readerContext);
-          creator.putBigDecimal(val);
+          for (int i = 0; i < numDocs; i++) {
+            BigDecimal val = reader.getBigDecimal(i, readerContext);
+            creator.putBigDecimal(val);
+          }
+          break;
         }
-        break;
+        default:
+          throw new IllegalStateException();
+      }
+    }
+  }
+
+  private void enableDictionary(String column, SegmentDirectory.Writer segmentWriter,
+      IndexCreatorProvider indexCreatorProvider)
+      throws Exception {
+    Preconditions.checkState(_segmentMetadata.getVersion() == SegmentVersion.v3);
+    ColumnMetadata existingColMetadata = _segmentMetadata.getColumnMetadataFor(column);
+    Preconditions.checkState(!existingColMetadata.hasDictionary(), "Existing column already has dictionary.");
+    boolean isSingleValue = existingColMetadata.isSingleValue();
+
+    File indexDir = _segmentMetadata.getIndexDir();
+    String segmentName = _segmentMetadata.getName();
+    File inProgress = new File(indexDir, column + ".dict.inprogress");
+    File dictionaryFile = new File(indexDir, column + V1Constants.Dict.FILE_EXTENSION);
+    String fwdIndexFileExtension;
+    if (isSingleValue) {
+      if (existingColMetadata.isSorted()) {
+        fwdIndexFileExtension = V1Constants.Indexes.SORTED_SV_FORWARD_INDEX_FILE_EXTENSION;
+      } else {
+        fwdIndexFileExtension = V1Constants.Indexes.UNSORTED_SV_FORWARD_INDEX_FILE_EXTENSION;
+      }
+    } else {
+      fwdIndexFileExtension = V1Constants.Indexes.UNSORTED_MV_FORWARD_INDEX_FILE_EXTENSION;
+    }
+    File fwdIndexFile = new File(indexDir, column + fwdIndexFileExtension);
+
+    if (!inProgress.exists()) {
+      // Marker file does not exist, which means last run ended normally.
+      // Create a marker file.
+      FileUtils.touch(inProgress);
+    } else {
+      // Marker file exists, which means last run was interrupted.
+      // Remove forward index and dictionary files if they exist.
+      FileUtils.deleteQuietly(fwdIndexFile);
+      FileUtils.deleteQuietly(dictionaryFile);
+    }
+
+    LOGGER.info("Creating a new dictionary for segment={} and column={}", segmentName, column);
+    SegmentDictionaryCreator dictionaryCreator = buildDictionary(column, existingColMetadata, segmentWriter);
+    LoaderUtils.writeIndexToV3Format(segmentWriter, column, dictionaryFile, ColumnIndexType.DICTIONARY);
+
+    LOGGER.info("Built dictionary. Rewriting dictionary enabled forward index for segment={} and column={}",
+        segmentName, column);
+    writeDictEnabledForwardIndex(column, existingColMetadata, segmentWriter, indexDir, indexCreatorProvider,
+        dictionaryCreator);
+    // We used the existing forward index to generate a new forward index. The existing forward index will be in V3
+    // format and the new forward index will be in V1 format. Remove the existing forward index as it is not needed
+    // anymore. Note that removeIndex() will only mark an index for removal and remove the in-memory state. The
+    // actual cleanup from columns.psf file will happen when singleFileIndexDirectory.cleanupRemovedIndices() is
+    // called during segmentWriter.close().
+    segmentWriter.removeIndex(column, ColumnIndexType.FORWARD_INDEX);
+    LoaderUtils.writeIndexToV3Format(segmentWriter, column, fwdIndexFile, ColumnIndexType.FORWARD_INDEX);
+
+    LOGGER.info("Created forwardIndex. Updating metadata properties for segment={} and column={}", segmentName, column);
+    updateMetadataProperties(indexDir, column, dictionaryCreator);
+
+    // We remove indexes that have to be rewritten when a dictEnabled is toggled. Note that the respective index
+    // handler will take care of recreating the index.
+    removeDictRelatedIndexes(column, segmentWriter);
+
+    // Delete the marker file.
+    FileUtils.deleteQuietly(inProgress);
+
+    LOGGER.info("Created dictionary based forward index for segment: {}, column: {}", segmentName, column);
+  }
+
+  private SegmentDictionaryCreator buildDictionary(String column, ColumnMetadata existingColMetadata,
+      SegmentDirectory.Writer segmentWriter)
+      throws Exception {
+    int numDocs = existingColMetadata.getTotalDocs();
+    // SegmentPartitionConfig is not relevant for rewrites.
+    Preconditions.checkState(_indexLoadingConfig.getTableConfig() != null);
+    StatsCollectorConfig statsCollectorConfig =
+        new StatsCollectorConfig(_indexLoadingConfig.getTableConfig(), _schema, null);
+    AbstractColumnStatisticsCollector statsCollector;
+
+    try (ForwardIndexReader reader = LoaderUtils.getForwardIndexReader(segmentWriter, existingColMetadata)) {
+      boolean isSVColumn = reader.isSingleValue();
+
+      switch (reader.getStoredType()) {
+        case INT:
+          statsCollector = new IntColumnPreIndexStatsCollector(column, statsCollectorConfig);
+          break;
+        case LONG:
+          statsCollector = new LongColumnPreIndexStatsCollector(column, statsCollectorConfig);
+          break;
+        case FLOAT:
+          statsCollector = new FloatColumnPreIndexStatsCollector(column, statsCollectorConfig);
+          break;
+        case DOUBLE:
+          statsCollector = new DoubleColumnPreIndexStatsCollector(column, statsCollectorConfig);
+          break;
+        case STRING:
+          statsCollector = new StringColumnPreIndexStatsCollector(column, statsCollectorConfig);
+          break;
+        case BYTES:
+          statsCollector = new BytesColumnPredIndexStatsCollector(column, statsCollectorConfig);
+          break;
+        case BIG_DECIMAL:
+          Preconditions.checkState(isSVColumn, "BigDecimal is not supported for MV columns");
+          statsCollector = new BigDecimalColumnPreIndexStatsCollector(column, statsCollectorConfig);
+          break;
+        default:
+          throw new IllegalStateException();
+      }
+
+      Preconditions.checkState(statsCollector != null);
+      // Note: Special Null handling is not necessary here. This is because, the existing default null value in the
+      // raw forwardIndex will be retained as such while created the dictionary and dict-based forward index. Also,
+      // null value vectors maintain a bitmap of docIds. No handling is necessary there.
+      PinotSegmentColumnReader columnReader =
+          new PinotSegmentColumnReader(reader, null, null, existingColMetadata.getMaxNumberOfMultiValues());
+      for (int i = 0; i < numDocs; i++) {
+        Object obj = columnReader.getValue(i);
+        statsCollector.collect(obj);
+      }
+      statsCollector.seal();
+
+      boolean useVarLength = _indexLoadingConfig.getVarLengthDictionaryColumns().contains(column)
+          || SegmentIndexCreationDriverImpl.shouldUseVarLengthDictionary(reader.getStoredType(), statsCollector);
+      SegmentDictionaryCreator dictionaryCreator =
+          new SegmentDictionaryCreator(existingColMetadata.getFieldSpec(), _segmentMetadata.getIndexDir(),
+              useVarLength);
+
+      dictionaryCreator.build(statsCollector.getUniqueValuesSet());
+      return dictionaryCreator;
+    }
+  }
+
+  private void writeDictEnabledForwardIndex(String column, ColumnMetadata existingColMetadata,
+      SegmentDirectory.Writer segmentWriter, File indexDir, IndexCreatorProvider indexCreatorProvider,
+      SegmentDictionaryCreator dictionaryCreator)
+      throws Exception {
+    try (ForwardIndexReader reader = LoaderUtils.getForwardIndexReader(segmentWriter, existingColMetadata)) {
+      int lengthOfLongestEntry = reader.getLengthOfLongestEntry();
+      IndexCreationContext.Builder builder =
+          IndexCreationContext.builder().withIndexDir(indexDir).withColumnMetadata(existingColMetadata)
+              .withLengthOfLongestEntry(lengthOfLongestEntry);
+      // existingColMetadata has dictEnable=false. Overwrite the value.
+      builder.withDictionary(true);
+      IndexCreationContext.Forward context =
+          builder.build().forForwardIndex(null, _indexLoadingConfig.getColumnProperties());
+
+      try (ForwardIndexCreator creator = indexCreatorProvider.newForwardIndexCreator(context)) {
+        int numDocs = existingColMetadata.getTotalDocs();
+        forwardIndexWriterHelper(column, existingColMetadata, reader, creator, numDocs, dictionaryCreator);
       }
-      default:
-        throw new IllegalStateException();
     }
   }
+
+  private void removeDictRelatedIndexes(String column, SegmentDirectory.Writer segmentWriter) {
+    // TODO: Move this logic as a static function in each index creator.
+    segmentWriter.removeIndex(column, ColumnIndexType.RANGE_INDEX);
+  }
+
+  private void updateMetadataProperties(File indexDir, String column, SegmentDictionaryCreator dictionaryCreator)

Review Comment:
   I recommend making this more generic. I too will have a need to update the metadata for enabling forward index on a forward index disabled column. I will want to set up different properties. Perhaps have it take up a set of input properties and set them all in a loop?



##########
pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/loader/ForwardIndexHandler.java:
##########
@@ -244,16 +279,16 @@ private void rewriteRawMVForwardIndex(String column, ColumnMetadata existingColM
               .forForwardIndex(newCompressionType, _indexLoadingConfig.getColumnProperties());
 
       try (ForwardIndexCreator creator = indexCreatorProvider.newForwardIndexCreator(context)) {
-        // If creator stored type and the reader stored type do not match, throw an exception.
         if (!reader.getStoredType().equals(creator.getValueType())) {
+          // Creator stored type should match reader stored type for raw columns. We do not support changing datatypes.
           String failureMsg =
               "Unsupported operation to change datatype for column=" + column + " from " + reader.getStoredType()
                   .toString() + " to " + creator.getValueType().toString();
           throw new UnsupportedOperationException(failureMsg);
         }

Review Comment:
   Curious, since this check is done in both functions before calling `forwardIndexWriterHelper`, can you move it into `forwardIndexWriterHelper` so that all callers will get this check?



##########
pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/loader/ForwardIndexHandler.java:
##########
@@ -271,118 +306,293 @@ private void rewriteRawSVForwardIndex(String column, ColumnMetadata existingColM
               .forForwardIndex(newCompressionType, _indexLoadingConfig.getColumnProperties());
 
       try (ForwardIndexCreator creator = indexCreatorProvider.newForwardIndexCreator(context)) {
-        // If creator stored type and the reader stored type do not match, throw an exception.
         if (!reader.getStoredType().equals(creator.getValueType())) {
+          // Creator stored type should match reader stored type for raw columns. We do not support changing datatypes.
           String failureMsg =
               "Unsupported operation to change datatype for column=" + column + " from " + reader.getStoredType()
                   .toString() + " to " + creator.getValueType().toString();
           throw new UnsupportedOperationException(failureMsg);
         }
 
         int numDocs = existingColMetadata.getTotalDocs();
-        forwardIndexWriterHelper(column, reader, creator, numDocs);
+        forwardIndexWriterHelper(column, existingColMetadata, reader, creator, numDocs, null);
       }
     }
   }
 
-  private void forwardIndexWriterHelper(String column, ForwardIndexReader reader, ForwardIndexCreator creator,
-      int numDocs) {
-    // If creator stored type should match reader stored type. We do not support changing datatypes.
-    if (!reader.getStoredType().equals(creator.getValueType())) {
-      String failureMsg =
-          "Unsupported operation to change datatype for column=" + column + " from " + reader.getStoredType().toString()
-              + " to " + creator.getValueType().toString();
-      throw new UnsupportedOperationException(failureMsg);
-    }
-
+  private void forwardIndexWriterHelper(String column, ColumnMetadata existingColumnMetadata, ForwardIndexReader reader,
+      ForwardIndexCreator creator, int numDocs, @Nullable SegmentDictionaryCreator dictionaryCreator) {
     ForwardIndexReaderContext readerContext = reader.createContext();
     boolean isSVColumn = reader.isSingleValue();
 
-    switch (reader.getStoredType()) {
-      // JSON fields are either stored as string or bytes. No special handling is needed because we make this
-      // decision based on the storedType of the reader.
-      case INT: {
-        for (int i = 0; i < numDocs; i++) {
-          if (isSVColumn) {
-            int val = reader.getInt(i, readerContext);
-            creator.putInt(val);
-          } else {
-            int[] ints = reader.getIntMV(i, readerContext);
-            creator.putIntMV(ints);
-          }
+    if (dictionaryCreator != null) {
+      int maxNumValuesPerEntry = existingColumnMetadata.getMaxNumberOfMultiValues();
+      PinotSegmentColumnReader columnReader = new PinotSegmentColumnReader(reader, null, null, maxNumValuesPerEntry);
+
+      for (int i = 0; i < numDocs; i++) {
+        Object obj = columnReader.getValue(i);
+
+        if (isSVColumn) {
+          int dictId = dictionaryCreator.indexOfSV(obj);
+          creator.putDictId(dictId);
+        } else {
+          int[] dictIds = dictionaryCreator.indexOfMV(obj);
+          creator.putDictIdMV(dictIds);
         }
-        break;
       }
-      case LONG: {
-        for (int i = 0; i < numDocs; i++) {
-          if (isSVColumn) {
-            long val = reader.getLong(i, readerContext);
-            creator.putLong(val);
-          } else {
-            long[] longs = reader.getLongMV(i, readerContext);
-            creator.putLongMV(longs);
+    } else {
+      switch (reader.getStoredType()) {
+        // JSON fields are either stored as string or bytes. No special handling is needed because we make this
+        // decision based on the storedType of the reader.
+        case INT: {
+          for (int i = 0; i < numDocs; i++) {
+            if (isSVColumn) {
+              int val = reader.getInt(i, readerContext);
+              creator.putInt(val);
+            } else {
+              int[] ints = reader.getIntMV(i, readerContext);
+              creator.putIntMV(ints);
+            }
           }
+          break;
         }
-        break;
-      }
-      case FLOAT: {
-        for (int i = 0; i < numDocs; i++) {
-          if (isSVColumn) {
-            float val = reader.getFloat(i, readerContext);
-            creator.putFloat(val);
-          } else {
-            float[] floats = reader.getFloatMV(i, readerContext);
-            creator.putFloatMV(floats);
+        case LONG: {
+          for (int i = 0; i < numDocs; i++) {
+            if (isSVColumn) {
+              long val = reader.getLong(i, readerContext);
+              creator.putLong(val);
+            } else {
+              long[] longs = reader.getLongMV(i, readerContext);
+              creator.putLongMV(longs);
+            }
           }
+          break;
         }
-        break;
-      }
-      case DOUBLE: {
-        for (int i = 0; i < numDocs; i++) {
-          if (isSVColumn) {
-            double val = reader.getDouble(i, readerContext);
-            creator.putDouble(val);
-          } else {
-            double[] doubles = reader.getDoubleMV(i, readerContext);
-            creator.putDoubleMV(doubles);
+        case FLOAT: {
+          for (int i = 0; i < numDocs; i++) {
+            if (isSVColumn) {
+              float val = reader.getFloat(i, readerContext);
+              creator.putFloat(val);
+            } else {
+              float[] floats = reader.getFloatMV(i, readerContext);
+              creator.putFloatMV(floats);
+            }
           }
+          break;
         }
-        break;
-      }
-      case STRING: {
-        for (int i = 0; i < numDocs; i++) {
-          if (isSVColumn) {
-            String val = reader.getString(i, readerContext);
-            creator.putString(val);
-          } else {
-            String[] strings = reader.getStringMV(i, readerContext);
-            creator.putStringMV(strings);
+        case DOUBLE: {
+          for (int i = 0; i < numDocs; i++) {
+            if (isSVColumn) {
+              double val = reader.getDouble(i, readerContext);
+              creator.putDouble(val);
+            } else {
+              double[] doubles = reader.getDoubleMV(i, readerContext);
+              creator.putDoubleMV(doubles);
+            }
           }
+          break;
         }
-        break;
-      }
-      case BYTES: {
-        for (int i = 0; i < numDocs; i++) {
-          if (isSVColumn) {
-            byte[] val = reader.getBytes(i, readerContext);
-            creator.putBytes(val);
-          } else {
-            byte[][] bytesArray = reader.getBytesMV(i, readerContext);
-            creator.putBytesMV(bytesArray);
+        case STRING: {
+          for (int i = 0; i < numDocs; i++) {
+            if (isSVColumn) {
+              String val = reader.getString(i, readerContext);
+              creator.putString(val);
+            } else {
+              String[] strings = reader.getStringMV(i, readerContext);
+              creator.putStringMV(strings);
+            }
           }
+          break;
         }
-        break;
-      }
-      case BIG_DECIMAL: {
-        for (int i = 0; i < numDocs; i++) {
+        case BYTES: {
+          for (int i = 0; i < numDocs; i++) {
+            if (isSVColumn) {
+              byte[] val = reader.getBytes(i, readerContext);
+              creator.putBytes(val);
+            } else {
+              byte[][] bytesArray = reader.getBytesMV(i, readerContext);
+              creator.putBytesMV(bytesArray);
+            }
+          }
+          break;
+        }
+        case BIG_DECIMAL: {
           Preconditions.checkState(isSVColumn, "BigDecimal is not supported for MV columns");
-          BigDecimal val = reader.getBigDecimal(i, readerContext);
-          creator.putBigDecimal(val);
+          for (int i = 0; i < numDocs; i++) {
+            BigDecimal val = reader.getBigDecimal(i, readerContext);
+            creator.putBigDecimal(val);
+          }
+          break;
         }
-        break;
+        default:
+          throw new IllegalStateException();
+      }
+    }
+  }
+
+  private void enableDictionary(String column, SegmentDirectory.Writer segmentWriter,
+      IndexCreatorProvider indexCreatorProvider)
+      throws Exception {
+    Preconditions.checkState(_segmentMetadata.getVersion() == SegmentVersion.v3);
+    ColumnMetadata existingColMetadata = _segmentMetadata.getColumnMetadataFor(column);
+    Preconditions.checkState(!existingColMetadata.hasDictionary(), "Existing column already has dictionary.");
+    boolean isSingleValue = existingColMetadata.isSingleValue();
+
+    File indexDir = _segmentMetadata.getIndexDir();
+    String segmentName = _segmentMetadata.getName();
+    File inProgress = new File(indexDir, column + ".dict.inprogress");
+    File dictionaryFile = new File(indexDir, column + V1Constants.Dict.FILE_EXTENSION);
+    String fwdIndexFileExtension;
+    if (isSingleValue) {
+      if (existingColMetadata.isSorted()) {
+        fwdIndexFileExtension = V1Constants.Indexes.SORTED_SV_FORWARD_INDEX_FILE_EXTENSION;
+      } else {
+        fwdIndexFileExtension = V1Constants.Indexes.UNSORTED_SV_FORWARD_INDEX_FILE_EXTENSION;
+      }
+    } else {
+      fwdIndexFileExtension = V1Constants.Indexes.UNSORTED_MV_FORWARD_INDEX_FILE_EXTENSION;
+    }
+    File fwdIndexFile = new File(indexDir, column + fwdIndexFileExtension);
+
+    if (!inProgress.exists()) {
+      // Marker file does not exist, which means last run ended normally.
+      // Create a marker file.
+      FileUtils.touch(inProgress);
+    } else {
+      // Marker file exists, which means last run was interrupted.
+      // Remove forward index and dictionary files if they exist.
+      FileUtils.deleteQuietly(fwdIndexFile);
+      FileUtils.deleteQuietly(dictionaryFile);
+    }
+
+    LOGGER.info("Creating a new dictionary for segment={} and column={}", segmentName, column);
+    SegmentDictionaryCreator dictionaryCreator = buildDictionary(column, existingColMetadata, segmentWriter);
+    LoaderUtils.writeIndexToV3Format(segmentWriter, column, dictionaryFile, ColumnIndexType.DICTIONARY);
+
+    LOGGER.info("Built dictionary. Rewriting dictionary enabled forward index for segment={} and column={}",
+        segmentName, column);
+    writeDictEnabledForwardIndex(column, existingColMetadata, segmentWriter, indexDir, indexCreatorProvider,
+        dictionaryCreator);
+    // We used the existing forward index to generate a new forward index. The existing forward index will be in V3
+    // format and the new forward index will be in V1 format. Remove the existing forward index as it is not needed
+    // anymore. Note that removeIndex() will only mark an index for removal and remove the in-memory state. The
+    // actual cleanup from columns.psf file will happen when singleFileIndexDirectory.cleanupRemovedIndices() is
+    // called during segmentWriter.close().
+    segmentWriter.removeIndex(column, ColumnIndexType.FORWARD_INDEX);
+    LoaderUtils.writeIndexToV3Format(segmentWriter, column, fwdIndexFile, ColumnIndexType.FORWARD_INDEX);
+
+    LOGGER.info("Created forwardIndex. Updating metadata properties for segment={} and column={}", segmentName, column);
+    updateMetadataProperties(indexDir, column, dictionaryCreator);
+
+    // We remove indexes that have to be rewritten when a dictEnabled is toggled. Note that the respective index
+    // handler will take care of recreating the index.
+    removeDictRelatedIndexes(column, segmentWriter);
+
+    // Delete the marker file.
+    FileUtils.deleteQuietly(inProgress);
+
+    LOGGER.info("Created dictionary based forward index for segment: {}, column: {}", segmentName, column);
+  }
+
+  private SegmentDictionaryCreator buildDictionary(String column, ColumnMetadata existingColMetadata,
+      SegmentDirectory.Writer segmentWriter)
+      throws Exception {
+    int numDocs = existingColMetadata.getTotalDocs();
+    // SegmentPartitionConfig is not relevant for rewrites.
+    Preconditions.checkState(_indexLoadingConfig.getTableConfig() != null);

Review Comment:
   should this check be done much earlier in the code flow? is it valid to have null TableConfig for other scenarios?



##########
pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/loader/ForwardIndexHandler.java:
##########
@@ -271,118 +306,293 @@ private void rewriteRawSVForwardIndex(String column, ColumnMetadata existingColM
               .forForwardIndex(newCompressionType, _indexLoadingConfig.getColumnProperties());
 
       try (ForwardIndexCreator creator = indexCreatorProvider.newForwardIndexCreator(context)) {
-        // If creator stored type and the reader stored type do not match, throw an exception.
         if (!reader.getStoredType().equals(creator.getValueType())) {
+          // Creator stored type should match reader stored type for raw columns. We do not support changing datatypes.
           String failureMsg =
               "Unsupported operation to change datatype for column=" + column + " from " + reader.getStoredType()
                   .toString() + " to " + creator.getValueType().toString();
           throw new UnsupportedOperationException(failureMsg);
         }
 
         int numDocs = existingColMetadata.getTotalDocs();
-        forwardIndexWriterHelper(column, reader, creator, numDocs);
+        forwardIndexWriterHelper(column, existingColMetadata, reader, creator, numDocs, null);
       }
     }
   }
 
-  private void forwardIndexWriterHelper(String column, ForwardIndexReader reader, ForwardIndexCreator creator,
-      int numDocs) {
-    // If creator stored type should match reader stored type. We do not support changing datatypes.
-    if (!reader.getStoredType().equals(creator.getValueType())) {
-      String failureMsg =
-          "Unsupported operation to change datatype for column=" + column + " from " + reader.getStoredType().toString()
-              + " to " + creator.getValueType().toString();
-      throw new UnsupportedOperationException(failureMsg);
-    }
-
+  private void forwardIndexWriterHelper(String column, ColumnMetadata existingColumnMetadata, ForwardIndexReader reader,
+      ForwardIndexCreator creator, int numDocs, @Nullable SegmentDictionaryCreator dictionaryCreator) {
     ForwardIndexReaderContext readerContext = reader.createContext();
     boolean isSVColumn = reader.isSingleValue();
 
-    switch (reader.getStoredType()) {
-      // JSON fields are either stored as string or bytes. No special handling is needed because we make this
-      // decision based on the storedType of the reader.
-      case INT: {
-        for (int i = 0; i < numDocs; i++) {
-          if (isSVColumn) {
-            int val = reader.getInt(i, readerContext);
-            creator.putInt(val);
-          } else {
-            int[] ints = reader.getIntMV(i, readerContext);
-            creator.putIntMV(ints);
-          }
+    if (dictionaryCreator != null) {
+      int maxNumValuesPerEntry = existingColumnMetadata.getMaxNumberOfMultiValues();
+      PinotSegmentColumnReader columnReader = new PinotSegmentColumnReader(reader, null, null, maxNumValuesPerEntry);
+
+      for (int i = 0; i < numDocs; i++) {
+        Object obj = columnReader.getValue(i);
+
+        if (isSVColumn) {
+          int dictId = dictionaryCreator.indexOfSV(obj);
+          creator.putDictId(dictId);
+        } else {
+          int[] dictIds = dictionaryCreator.indexOfMV(obj);
+          creator.putDictIdMV(dictIds);
         }
-        break;
       }
-      case LONG: {
-        for (int i = 0; i < numDocs; i++) {
-          if (isSVColumn) {
-            long val = reader.getLong(i, readerContext);
-            creator.putLong(val);
-          } else {
-            long[] longs = reader.getLongMV(i, readerContext);
-            creator.putLongMV(longs);
+    } else {
+      switch (reader.getStoredType()) {
+        // JSON fields are either stored as string or bytes. No special handling is needed because we make this
+        // decision based on the storedType of the reader.
+        case INT: {
+          for (int i = 0; i < numDocs; i++) {
+            if (isSVColumn) {
+              int val = reader.getInt(i, readerContext);
+              creator.putInt(val);
+            } else {
+              int[] ints = reader.getIntMV(i, readerContext);
+              creator.putIntMV(ints);
+            }
           }
+          break;
         }
-        break;
-      }
-      case FLOAT: {
-        for (int i = 0; i < numDocs; i++) {
-          if (isSVColumn) {
-            float val = reader.getFloat(i, readerContext);
-            creator.putFloat(val);
-          } else {
-            float[] floats = reader.getFloatMV(i, readerContext);
-            creator.putFloatMV(floats);
+        case LONG: {
+          for (int i = 0; i < numDocs; i++) {
+            if (isSVColumn) {
+              long val = reader.getLong(i, readerContext);
+              creator.putLong(val);
+            } else {
+              long[] longs = reader.getLongMV(i, readerContext);
+              creator.putLongMV(longs);
+            }
           }
+          break;
         }
-        break;
-      }
-      case DOUBLE: {
-        for (int i = 0; i < numDocs; i++) {
-          if (isSVColumn) {
-            double val = reader.getDouble(i, readerContext);
-            creator.putDouble(val);
-          } else {
-            double[] doubles = reader.getDoubleMV(i, readerContext);
-            creator.putDoubleMV(doubles);
+        case FLOAT: {
+          for (int i = 0; i < numDocs; i++) {
+            if (isSVColumn) {
+              float val = reader.getFloat(i, readerContext);
+              creator.putFloat(val);
+            } else {
+              float[] floats = reader.getFloatMV(i, readerContext);
+              creator.putFloatMV(floats);
+            }
           }
+          break;
         }
-        break;
-      }
-      case STRING: {
-        for (int i = 0; i < numDocs; i++) {
-          if (isSVColumn) {
-            String val = reader.getString(i, readerContext);
-            creator.putString(val);
-          } else {
-            String[] strings = reader.getStringMV(i, readerContext);
-            creator.putStringMV(strings);
+        case DOUBLE: {
+          for (int i = 0; i < numDocs; i++) {
+            if (isSVColumn) {
+              double val = reader.getDouble(i, readerContext);
+              creator.putDouble(val);
+            } else {
+              double[] doubles = reader.getDoubleMV(i, readerContext);
+              creator.putDoubleMV(doubles);
+            }
           }
+          break;
         }
-        break;
-      }
-      case BYTES: {
-        for (int i = 0; i < numDocs; i++) {
-          if (isSVColumn) {
-            byte[] val = reader.getBytes(i, readerContext);
-            creator.putBytes(val);
-          } else {
-            byte[][] bytesArray = reader.getBytesMV(i, readerContext);
-            creator.putBytesMV(bytesArray);
+        case STRING: {
+          for (int i = 0; i < numDocs; i++) {
+            if (isSVColumn) {
+              String val = reader.getString(i, readerContext);
+              creator.putString(val);
+            } else {
+              String[] strings = reader.getStringMV(i, readerContext);
+              creator.putStringMV(strings);
+            }
           }
+          break;
         }
-        break;
-      }
-      case BIG_DECIMAL: {
-        for (int i = 0; i < numDocs; i++) {
+        case BYTES: {
+          for (int i = 0; i < numDocs; i++) {
+            if (isSVColumn) {
+              byte[] val = reader.getBytes(i, readerContext);
+              creator.putBytes(val);
+            } else {
+              byte[][] bytesArray = reader.getBytesMV(i, readerContext);
+              creator.putBytesMV(bytesArray);
+            }
+          }
+          break;
+        }
+        case BIG_DECIMAL: {
           Preconditions.checkState(isSVColumn, "BigDecimal is not supported for MV columns");
-          BigDecimal val = reader.getBigDecimal(i, readerContext);
-          creator.putBigDecimal(val);
+          for (int i = 0; i < numDocs; i++) {
+            BigDecimal val = reader.getBigDecimal(i, readerContext);
+            creator.putBigDecimal(val);
+          }
+          break;
         }
-        break;
+        default:
+          throw new IllegalStateException();
+      }
+    }
+  }
+
+  private void enableDictionary(String column, SegmentDirectory.Writer segmentWriter,
+      IndexCreatorProvider indexCreatorProvider)
+      throws Exception {
+    Preconditions.checkState(_segmentMetadata.getVersion() == SegmentVersion.v3);
+    ColumnMetadata existingColMetadata = _segmentMetadata.getColumnMetadataFor(column);
+    Preconditions.checkState(!existingColMetadata.hasDictionary(), "Existing column already has dictionary.");
+    boolean isSingleValue = existingColMetadata.isSingleValue();
+
+    File indexDir = _segmentMetadata.getIndexDir();
+    String segmentName = _segmentMetadata.getName();
+    File inProgress = new File(indexDir, column + ".dict.inprogress");
+    File dictionaryFile = new File(indexDir, column + V1Constants.Dict.FILE_EXTENSION);
+    String fwdIndexFileExtension;
+    if (isSingleValue) {
+      if (existingColMetadata.isSorted()) {
+        fwdIndexFileExtension = V1Constants.Indexes.SORTED_SV_FORWARD_INDEX_FILE_EXTENSION;
+      } else {
+        fwdIndexFileExtension = V1Constants.Indexes.UNSORTED_SV_FORWARD_INDEX_FILE_EXTENSION;
+      }
+    } else {
+      fwdIndexFileExtension = V1Constants.Indexes.UNSORTED_MV_FORWARD_INDEX_FILE_EXTENSION;
+    }
+    File fwdIndexFile = new File(indexDir, column + fwdIndexFileExtension);
+
+    if (!inProgress.exists()) {
+      // Marker file does not exist, which means last run ended normally.
+      // Create a marker file.
+      FileUtils.touch(inProgress);
+    } else {
+      // Marker file exists, which means last run was interrupted.
+      // Remove forward index and dictionary files if they exist.
+      FileUtils.deleteQuietly(fwdIndexFile);
+      FileUtils.deleteQuietly(dictionaryFile);
+    }
+
+    LOGGER.info("Creating a new dictionary for segment={} and column={}", segmentName, column);
+    SegmentDictionaryCreator dictionaryCreator = buildDictionary(column, existingColMetadata, segmentWriter);
+    LoaderUtils.writeIndexToV3Format(segmentWriter, column, dictionaryFile, ColumnIndexType.DICTIONARY);
+
+    LOGGER.info("Built dictionary. Rewriting dictionary enabled forward index for segment={} and column={}",
+        segmentName, column);
+    writeDictEnabledForwardIndex(column, existingColMetadata, segmentWriter, indexDir, indexCreatorProvider,
+        dictionaryCreator);
+    // We used the existing forward index to generate a new forward index. The existing forward index will be in V3
+    // format and the new forward index will be in V1 format. Remove the existing forward index as it is not needed
+    // anymore. Note that removeIndex() will only mark an index for removal and remove the in-memory state. The
+    // actual cleanup from columns.psf file will happen when singleFileIndexDirectory.cleanupRemovedIndices() is
+    // called during segmentWriter.close().
+    segmentWriter.removeIndex(column, ColumnIndexType.FORWARD_INDEX);
+    LoaderUtils.writeIndexToV3Format(segmentWriter, column, fwdIndexFile, ColumnIndexType.FORWARD_INDEX);
+
+    LOGGER.info("Created forwardIndex. Updating metadata properties for segment={} and column={}", segmentName, column);
+    updateMetadataProperties(indexDir, column, dictionaryCreator);
+
+    // We remove indexes that have to be rewritten when a dictEnabled is toggled. Note that the respective index
+    // handler will take care of recreating the index.
+    removeDictRelatedIndexes(column, segmentWriter);
+
+    // Delete the marker file.
+    FileUtils.deleteQuietly(inProgress);
+
+    LOGGER.info("Created dictionary based forward index for segment: {}, column: {}", segmentName, column);
+  }
+
+  private SegmentDictionaryCreator buildDictionary(String column, ColumnMetadata existingColMetadata,
+      SegmentDirectory.Writer segmentWriter)
+      throws Exception {
+    int numDocs = existingColMetadata.getTotalDocs();
+    // SegmentPartitionConfig is not relevant for rewrites.
+    Preconditions.checkState(_indexLoadingConfig.getTableConfig() != null);
+    StatsCollectorConfig statsCollectorConfig =
+        new StatsCollectorConfig(_indexLoadingConfig.getTableConfig(), _schema, null);
+    AbstractColumnStatisticsCollector statsCollector;
+
+    try (ForwardIndexReader reader = LoaderUtils.getForwardIndexReader(segmentWriter, existingColMetadata)) {
+      boolean isSVColumn = reader.isSingleValue();
+
+      switch (reader.getStoredType()) {
+        case INT:
+          statsCollector = new IntColumnPreIndexStatsCollector(column, statsCollectorConfig);
+          break;
+        case LONG:
+          statsCollector = new LongColumnPreIndexStatsCollector(column, statsCollectorConfig);
+          break;
+        case FLOAT:
+          statsCollector = new FloatColumnPreIndexStatsCollector(column, statsCollectorConfig);
+          break;
+        case DOUBLE:
+          statsCollector = new DoubleColumnPreIndexStatsCollector(column, statsCollectorConfig);
+          break;
+        case STRING:
+          statsCollector = new StringColumnPreIndexStatsCollector(column, statsCollectorConfig);
+          break;
+        case BYTES:
+          statsCollector = new BytesColumnPredIndexStatsCollector(column, statsCollectorConfig);
+          break;
+        case BIG_DECIMAL:
+          Preconditions.checkState(isSVColumn, "BigDecimal is not supported for MV columns");
+          statsCollector = new BigDecimalColumnPreIndexStatsCollector(column, statsCollectorConfig);
+          break;
+        default:
+          throw new IllegalStateException();
+      }
+
+      Preconditions.checkState(statsCollector != null);
+      // Note: Special Null handling is not necessary here. This is because, the existing default null value in the
+      // raw forwardIndex will be retained as such while created the dictionary and dict-based forward index. Also,
+      // null value vectors maintain a bitmap of docIds. No handling is necessary there.
+      PinotSegmentColumnReader columnReader =
+          new PinotSegmentColumnReader(reader, null, null, existingColMetadata.getMaxNumberOfMultiValues());
+      for (int i = 0; i < numDocs; i++) {
+        Object obj = columnReader.getValue(i);
+        statsCollector.collect(obj);
+      }
+      statsCollector.seal();
+
+      boolean useVarLength = _indexLoadingConfig.getVarLengthDictionaryColumns().contains(column)

Review Comment:
   as mentioned earlier better to move the first part of the OR check into the function. Have the function take the `varLengthDictionaryColumns` as input. This will help ensure that all callers will get both checks and not accidentally miss out on the first.



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


[GitHub] [pinot] siddharthteotia commented on a diff in pull request #9678: Support creating dictionary at runtime for an existing column

Posted by GitBox <gi...@apache.org>.
siddharthteotia commented on code in PR #9678:
URL: https://github.com/apache/pinot/pull/9678#discussion_r1011043808


##########
pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/loader/ForwardIndexHandler.java:
##########
@@ -271,118 +304,295 @@ private void rewriteRawSVForwardIndex(String column, ColumnMetadata existingColM
               .forForwardIndex(newCompressionType, _indexLoadingConfig.getColumnProperties());
 
       try (ForwardIndexCreator creator = indexCreatorProvider.newForwardIndexCreator(context)) {
-        // If creator stored type and the reader stored type do not match, throw an exception.
         if (!reader.getStoredType().equals(creator.getValueType())) {
+          // Creator stored type should match reader stored type for raw columns. We do not support changing datatypes.
           String failureMsg =
               "Unsupported operation to change datatype for column=" + column + " from " + reader.getStoredType()
                   .toString() + " to " + creator.getValueType().toString();
           throw new UnsupportedOperationException(failureMsg);
         }
 
         int numDocs = existingColMetadata.getTotalDocs();
-        forwardIndexWriterHelper(column, reader, creator, numDocs);
+        forwardIndexWriterHelper(column, existingColMetadata, reader, creator, numDocs, null);
       }
     }
   }
 
-  private void forwardIndexWriterHelper(String column, ForwardIndexReader reader, ForwardIndexCreator creator,
-      int numDocs) {
-    // If creator stored type should match reader stored type. We do not support changing datatypes.
-    if (!reader.getStoredType().equals(creator.getValueType())) {
-      String failureMsg =
-          "Unsupported operation to change datatype for column=" + column + " from " + reader.getStoredType().toString()
-              + " to " + creator.getValueType().toString();
-      throw new UnsupportedOperationException(failureMsg);
-    }
-
+  private void forwardIndexWriterHelper(String column, ColumnMetadata existingColumnMetadata, ForwardIndexReader reader,
+      ForwardIndexCreator creator, int numDocs, @Nullable SegmentDictionaryCreator dictionaryCreator) {
     ForwardIndexReaderContext readerContext = reader.createContext();
     boolean isSVColumn = reader.isSingleValue();
 
-    switch (reader.getStoredType()) {
-      // JSON fields are either stored as string or bytes. No special handling is needed because we make this
-      // decision based on the storedType of the reader.
-      case INT: {
-        for (int i = 0; i < numDocs; i++) {
-          if (isSVColumn) {
-            int val = reader.getInt(i, readerContext);
-            creator.putInt(val);
-          } else {
-            int[] ints = reader.getIntMV(i, readerContext);
-            creator.putIntMV(ints);
-          }
+    if (dictionaryCreator != null) {
+      int maxNumValuesPerEntry = existingColumnMetadata.getMaxNumberOfMultiValues();
+      PinotSegmentColumnReader columnReader = new PinotSegmentColumnReader(reader, null, null, maxNumValuesPerEntry);
+
+      for (int i = 0; i < numDocs; i++) {
+        Object obj = columnReader.getValue(i);
+
+        if (isSVColumn) {
+          int dictId = dictionaryCreator.indexOfSV(obj);
+          creator.putDictId(dictId);
+        } else {
+          int[] dictIds = dictionaryCreator.indexOfMV(obj);
+          creator.putDictIdMV(dictIds);
         }
-        break;
       }
-      case LONG: {
-        for (int i = 0; i < numDocs; i++) {
-          if (isSVColumn) {
-            long val = reader.getLong(i, readerContext);
-            creator.putLong(val);
-          } else {
-            long[] longs = reader.getLongMV(i, readerContext);
-            creator.putLongMV(longs);
+    } else {
+      switch (reader.getStoredType()) {
+        // JSON fields are either stored as string or bytes. No special handling is needed because we make this
+        // decision based on the storedType of the reader.
+        case INT: {
+          for (int i = 0; i < numDocs; i++) {
+            if (isSVColumn) {
+              int val = reader.getInt(i, readerContext);
+              creator.putInt(val);
+            } else {
+              int[] ints = reader.getIntMV(i, readerContext);
+              creator.putIntMV(ints);
+            }
           }
+          break;
         }
-        break;
-      }
-      case FLOAT: {
-        for (int i = 0; i < numDocs; i++) {
-          if (isSVColumn) {
-            float val = reader.getFloat(i, readerContext);
-            creator.putFloat(val);
-          } else {
-            float[] floats = reader.getFloatMV(i, readerContext);
-            creator.putFloatMV(floats);
+        case LONG: {
+          for (int i = 0; i < numDocs; i++) {
+            if (isSVColumn) {
+              long val = reader.getLong(i, readerContext);
+              creator.putLong(val);
+            } else {
+              long[] longs = reader.getLongMV(i, readerContext);
+              creator.putLongMV(longs);
+            }
           }
+          break;
         }
-        break;
-      }
-      case DOUBLE: {
-        for (int i = 0; i < numDocs; i++) {
-          if (isSVColumn) {
-            double val = reader.getDouble(i, readerContext);
-            creator.putDouble(val);
-          } else {
-            double[] doubles = reader.getDoubleMV(i, readerContext);
-            creator.putDoubleMV(doubles);
+        case FLOAT: {
+          for (int i = 0; i < numDocs; i++) {
+            if (isSVColumn) {
+              float val = reader.getFloat(i, readerContext);
+              creator.putFloat(val);
+            } else {
+              float[] floats = reader.getFloatMV(i, readerContext);
+              creator.putFloatMV(floats);
+            }
           }
+          break;
         }
-        break;
-      }
-      case STRING: {
-        for (int i = 0; i < numDocs; i++) {
-          if (isSVColumn) {
-            String val = reader.getString(i, readerContext);
-            creator.putString(val);
-          } else {
-            String[] strings = reader.getStringMV(i, readerContext);
-            creator.putStringMV(strings);
+        case DOUBLE: {
+          for (int i = 0; i < numDocs; i++) {
+            if (isSVColumn) {
+              double val = reader.getDouble(i, readerContext);
+              creator.putDouble(val);
+            } else {
+              double[] doubles = reader.getDoubleMV(i, readerContext);
+              creator.putDoubleMV(doubles);
+            }
           }
+          break;
         }
-        break;
-      }
-      case BYTES: {
-        for (int i = 0; i < numDocs; i++) {
-          if (isSVColumn) {
-            byte[] val = reader.getBytes(i, readerContext);
-            creator.putBytes(val);
-          } else {
-            byte[][] bytesArray = reader.getBytesMV(i, readerContext);
-            creator.putBytesMV(bytesArray);
+        case STRING: {
+          for (int i = 0; i < numDocs; i++) {
+            if (isSVColumn) {
+              String val = reader.getString(i, readerContext);
+              creator.putString(val);
+            } else {
+              String[] strings = reader.getStringMV(i, readerContext);
+              creator.putStringMV(strings);
+            }
           }
+          break;
         }
-        break;
-      }
-      case BIG_DECIMAL: {
-        for (int i = 0; i < numDocs; i++) {
+        case BYTES: {
+          for (int i = 0; i < numDocs; i++) {
+            if (isSVColumn) {
+              byte[] val = reader.getBytes(i, readerContext);
+              creator.putBytes(val);
+            } else {
+              byte[][] bytesArray = reader.getBytesMV(i, readerContext);
+              creator.putBytesMV(bytesArray);
+            }
+          }
+          break;
+        }
+        case BIG_DECIMAL: {
           Preconditions.checkState(isSVColumn, "BigDecimal is not supported for MV columns");
-          BigDecimal val = reader.getBigDecimal(i, readerContext);
-          creator.putBigDecimal(val);
+          for (int i = 0; i < numDocs; i++) {
+            BigDecimal val = reader.getBigDecimal(i, readerContext);
+            creator.putBigDecimal(val);
+          }
+          break;
         }
-        break;
+        default:
+          throw new IllegalStateException();
+      }
+    }
+  }
+
+  private void enableDictionary(String column, SegmentDirectory.Writer segmentWriter,
+      IndexCreatorProvider indexCreatorProvider)
+      throws Exception {
+    Preconditions.checkState(_segmentMetadata.getVersion() == SegmentVersion.v3);
+    ColumnMetadata existingColMetadata = _segmentMetadata.getColumnMetadataFor(column);
+    Preconditions.checkState(!existingColMetadata.hasDictionary(),
+        "Cannot rewrite dictionary enabled forward index. Dictionary already exists for column:" + column);
+    boolean isSingleValue = existingColMetadata.isSingleValue();
+
+    File indexDir = _segmentMetadata.getIndexDir();
+    String segmentName = _segmentMetadata.getName();
+    File inProgress = new File(indexDir, column + ".dict.inprogress");
+    File dictionaryFile = new File(indexDir, column + V1Constants.Dict.FILE_EXTENSION);
+    String fwdIndexFileExtension;
+    if (isSingleValue) {
+      fwdIndexFileExtension =
+          existingColMetadata.isSorted() ? V1Constants.Indexes.SORTED_SV_FORWARD_INDEX_FILE_EXTENSION

Review Comment:
   This case will never be hit. If the column is sorted then dictionary should have already existed. Sorted forward index is always dictionary encoded / dictionary based. 



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


[GitHub] [pinot] vvivekiyer commented on a diff in pull request #9678: Support creating dictionary at runtime for an existing column

Posted by GitBox <gi...@apache.org>.
vvivekiyer commented on code in PR #9678:
URL: https://github.com/apache/pinot/pull/9678#discussion_r1012169149


##########
pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/loader/ForwardIndexHandler.java:
##########
@@ -26,23 +26,43 @@
 import java.util.HashSet;
 import java.util.Map;
 import java.util.Set;
+import javax.annotation.Nullable;
+import org.apache.commons.configuration.PropertiesConfiguration;
 import org.apache.commons.io.FileUtils;
+import org.apache.pinot.segment.local.segment.creator.impl.SegmentDictionaryCreator;
+import org.apache.pinot.segment.local.segment.creator.impl.SegmentIndexCreationDriverImpl;
 import org.apache.pinot.segment.local.segment.creator.impl.fwd.MultiValueVarByteRawIndexCreator;
+import org.apache.pinot.segment.local.segment.creator.impl.stats.AbstractColumnStatisticsCollector;
+import org.apache.pinot.segment.local.segment.creator.impl.stats.BigDecimalColumnPreIndexStatsCollector;
+import org.apache.pinot.segment.local.segment.creator.impl.stats.BytesColumnPredIndexStatsCollector;
+import org.apache.pinot.segment.local.segment.creator.impl.stats.DoubleColumnPreIndexStatsCollector;
+import org.apache.pinot.segment.local.segment.creator.impl.stats.FloatColumnPreIndexStatsCollector;
+import org.apache.pinot.segment.local.segment.creator.impl.stats.IntColumnPreIndexStatsCollector;
+import org.apache.pinot.segment.local.segment.creator.impl.stats.LongColumnPreIndexStatsCollector;
+import org.apache.pinot.segment.local.segment.creator.impl.stats.StringColumnPreIndexStatsCollector;
+import org.apache.pinot.segment.local.segment.readers.PinotSegmentColumnReader;
 import org.apache.pinot.segment.spi.ColumnMetadata;
 import org.apache.pinot.segment.spi.SegmentMetadata;
 import org.apache.pinot.segment.spi.V1Constants;
 import org.apache.pinot.segment.spi.compression.ChunkCompressionType;
 import org.apache.pinot.segment.spi.creator.IndexCreationContext;
 import org.apache.pinot.segment.spi.creator.IndexCreatorProvider;
 import org.apache.pinot.segment.spi.creator.SegmentVersion;
+import org.apache.pinot.segment.spi.creator.StatsCollectorConfig;
 import org.apache.pinot.segment.spi.index.creator.ForwardIndexCreator;
 import org.apache.pinot.segment.spi.index.reader.ForwardIndexReader;
 import org.apache.pinot.segment.spi.index.reader.ForwardIndexReaderContext;
 import org.apache.pinot.segment.spi.store.ColumnIndexType;
 import org.apache.pinot.segment.spi.store.SegmentDirectory;
+import org.apache.pinot.segment.spi.store.SegmentDirectoryPaths;
+import org.apache.pinot.spi.data.Schema;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import static org.apache.pinot.segment.spi.V1Constants.MetadataKeys.Column.DICTIONARY_ELEMENT_SIZE;

Review Comment:
   Tried this as well. That also threw the same error.



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


[GitHub] [pinot] vvivekiyer commented on a diff in pull request #9678: Support creating dictionary at runtime for an existing column

Posted by GitBox <gi...@apache.org>.
vvivekiyer commented on code in PR #9678:
URL: https://github.com/apache/pinot/pull/9678#discussion_r1011013773


##########
pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/loader/ForwardIndexHandler.java:
##########
@@ -26,23 +26,43 @@
 import java.util.HashSet;
 import java.util.Map;
 import java.util.Set;
+import javax.annotation.Nullable;
+import org.apache.commons.configuration.PropertiesConfiguration;
 import org.apache.commons.io.FileUtils;
+import org.apache.pinot.segment.local.segment.creator.impl.SegmentDictionaryCreator;
+import org.apache.pinot.segment.local.segment.creator.impl.SegmentIndexCreationDriverImpl;
 import org.apache.pinot.segment.local.segment.creator.impl.fwd.MultiValueVarByteRawIndexCreator;
+import org.apache.pinot.segment.local.segment.creator.impl.stats.AbstractColumnStatisticsCollector;
+import org.apache.pinot.segment.local.segment.creator.impl.stats.BigDecimalColumnPreIndexStatsCollector;
+import org.apache.pinot.segment.local.segment.creator.impl.stats.BytesColumnPredIndexStatsCollector;
+import org.apache.pinot.segment.local.segment.creator.impl.stats.DoubleColumnPreIndexStatsCollector;
+import org.apache.pinot.segment.local.segment.creator.impl.stats.FloatColumnPreIndexStatsCollector;
+import org.apache.pinot.segment.local.segment.creator.impl.stats.IntColumnPreIndexStatsCollector;
+import org.apache.pinot.segment.local.segment.creator.impl.stats.LongColumnPreIndexStatsCollector;
+import org.apache.pinot.segment.local.segment.creator.impl.stats.StringColumnPreIndexStatsCollector;
+import org.apache.pinot.segment.local.segment.readers.PinotSegmentColumnReader;
 import org.apache.pinot.segment.spi.ColumnMetadata;
 import org.apache.pinot.segment.spi.SegmentMetadata;
 import org.apache.pinot.segment.spi.V1Constants;
 import org.apache.pinot.segment.spi.compression.ChunkCompressionType;
 import org.apache.pinot.segment.spi.creator.IndexCreationContext;
 import org.apache.pinot.segment.spi.creator.IndexCreatorProvider;
 import org.apache.pinot.segment.spi.creator.SegmentVersion;
+import org.apache.pinot.segment.spi.creator.StatsCollectorConfig;
 import org.apache.pinot.segment.spi.index.creator.ForwardIndexCreator;
 import org.apache.pinot.segment.spi.index.reader.ForwardIndexReader;
 import org.apache.pinot.segment.spi.index.reader.ForwardIndexReaderContext;
 import org.apache.pinot.segment.spi.store.ColumnIndexType;
 import org.apache.pinot.segment.spi.store.SegmentDirectory;
+import org.apache.pinot.segment.spi.store.SegmentDirectoryPaths;
+import org.apache.pinot.spi.data.Schema;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import static org.apache.pinot.segment.spi.V1Constants.MetadataKeys.Column.DICTIONARY_ELEMENT_SIZE;

Review Comment:
   Checkstyle violation is thrown if we use MetadataKeys.Column or MetadataKeys.



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


[GitHub] [pinot] vvivekiyer commented on a diff in pull request #9678: Support creating dictionary at runtime for an existing column

Posted by GitBox <gi...@apache.org>.
vvivekiyer commented on code in PR #9678:
URL: https://github.com/apache/pinot/pull/9678#discussion_r1012138285


##########
pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/loader/ForwardIndexHandler.java:
##########
@@ -271,118 +304,295 @@ private void rewriteRawSVForwardIndex(String column, ColumnMetadata existingColM
               .forForwardIndex(newCompressionType, _indexLoadingConfig.getColumnProperties());
 
       try (ForwardIndexCreator creator = indexCreatorProvider.newForwardIndexCreator(context)) {
-        // If creator stored type and the reader stored type do not match, throw an exception.
         if (!reader.getStoredType().equals(creator.getValueType())) {
+          // Creator stored type should match reader stored type for raw columns. We do not support changing datatypes.
           String failureMsg =
               "Unsupported operation to change datatype for column=" + column + " from " + reader.getStoredType()
                   .toString() + " to " + creator.getValueType().toString();
           throw new UnsupportedOperationException(failureMsg);
         }
 
         int numDocs = existingColMetadata.getTotalDocs();
-        forwardIndexWriterHelper(column, reader, creator, numDocs);
+        forwardIndexWriterHelper(column, existingColMetadata, reader, creator, numDocs, null);
       }
     }
   }
 
-  private void forwardIndexWriterHelper(String column, ForwardIndexReader reader, ForwardIndexCreator creator,
-      int numDocs) {
-    // If creator stored type should match reader stored type. We do not support changing datatypes.
-    if (!reader.getStoredType().equals(creator.getValueType())) {
-      String failureMsg =
-          "Unsupported operation to change datatype for column=" + column + " from " + reader.getStoredType().toString()
-              + " to " + creator.getValueType().toString();
-      throw new UnsupportedOperationException(failureMsg);
-    }
-
+  private void forwardIndexWriterHelper(String column, ColumnMetadata existingColumnMetadata, ForwardIndexReader reader,
+      ForwardIndexCreator creator, int numDocs, @Nullable SegmentDictionaryCreator dictionaryCreator) {
     ForwardIndexReaderContext readerContext = reader.createContext();
     boolean isSVColumn = reader.isSingleValue();
 
-    switch (reader.getStoredType()) {
-      // JSON fields are either stored as string or bytes. No special handling is needed because we make this
-      // decision based on the storedType of the reader.
-      case INT: {
-        for (int i = 0; i < numDocs; i++) {
-          if (isSVColumn) {
-            int val = reader.getInt(i, readerContext);
-            creator.putInt(val);
-          } else {
-            int[] ints = reader.getIntMV(i, readerContext);
-            creator.putIntMV(ints);
-          }
+    if (dictionaryCreator != null) {
+      int maxNumValuesPerEntry = existingColumnMetadata.getMaxNumberOfMultiValues();
+      PinotSegmentColumnReader columnReader = new PinotSegmentColumnReader(reader, null, null, maxNumValuesPerEntry);
+
+      for (int i = 0; i < numDocs; i++) {
+        Object obj = columnReader.getValue(i);
+
+        if (isSVColumn) {
+          int dictId = dictionaryCreator.indexOfSV(obj);
+          creator.putDictId(dictId);
+        } else {
+          int[] dictIds = dictionaryCreator.indexOfMV(obj);
+          creator.putDictIdMV(dictIds);
         }
-        break;
       }
-      case LONG: {
-        for (int i = 0; i < numDocs; i++) {
-          if (isSVColumn) {
-            long val = reader.getLong(i, readerContext);
-            creator.putLong(val);
-          } else {
-            long[] longs = reader.getLongMV(i, readerContext);
-            creator.putLongMV(longs);
+    } else {
+      switch (reader.getStoredType()) {
+        // JSON fields are either stored as string or bytes. No special handling is needed because we make this
+        // decision based on the storedType of the reader.
+        case INT: {
+          for (int i = 0; i < numDocs; i++) {
+            if (isSVColumn) {
+              int val = reader.getInt(i, readerContext);
+              creator.putInt(val);
+            } else {
+              int[] ints = reader.getIntMV(i, readerContext);
+              creator.putIntMV(ints);
+            }
           }
+          break;
         }
-        break;
-      }
-      case FLOAT: {
-        for (int i = 0; i < numDocs; i++) {
-          if (isSVColumn) {
-            float val = reader.getFloat(i, readerContext);
-            creator.putFloat(val);
-          } else {
-            float[] floats = reader.getFloatMV(i, readerContext);
-            creator.putFloatMV(floats);
+        case LONG: {
+          for (int i = 0; i < numDocs; i++) {
+            if (isSVColumn) {
+              long val = reader.getLong(i, readerContext);
+              creator.putLong(val);
+            } else {
+              long[] longs = reader.getLongMV(i, readerContext);
+              creator.putLongMV(longs);
+            }
           }
+          break;
         }
-        break;
-      }
-      case DOUBLE: {
-        for (int i = 0; i < numDocs; i++) {
-          if (isSVColumn) {
-            double val = reader.getDouble(i, readerContext);
-            creator.putDouble(val);
-          } else {
-            double[] doubles = reader.getDoubleMV(i, readerContext);
-            creator.putDoubleMV(doubles);
+        case FLOAT: {
+          for (int i = 0; i < numDocs; i++) {
+            if (isSVColumn) {
+              float val = reader.getFloat(i, readerContext);
+              creator.putFloat(val);
+            } else {
+              float[] floats = reader.getFloatMV(i, readerContext);
+              creator.putFloatMV(floats);
+            }
           }
+          break;
         }
-        break;
-      }
-      case STRING: {
-        for (int i = 0; i < numDocs; i++) {
-          if (isSVColumn) {
-            String val = reader.getString(i, readerContext);
-            creator.putString(val);
-          } else {
-            String[] strings = reader.getStringMV(i, readerContext);
-            creator.putStringMV(strings);
+        case DOUBLE: {
+          for (int i = 0; i < numDocs; i++) {
+            if (isSVColumn) {
+              double val = reader.getDouble(i, readerContext);
+              creator.putDouble(val);
+            } else {
+              double[] doubles = reader.getDoubleMV(i, readerContext);
+              creator.putDoubleMV(doubles);
+            }
           }
+          break;
         }
-        break;
-      }
-      case BYTES: {
-        for (int i = 0; i < numDocs; i++) {
-          if (isSVColumn) {
-            byte[] val = reader.getBytes(i, readerContext);
-            creator.putBytes(val);
-          } else {
-            byte[][] bytesArray = reader.getBytesMV(i, readerContext);
-            creator.putBytesMV(bytesArray);
+        case STRING: {
+          for (int i = 0; i < numDocs; i++) {
+            if (isSVColumn) {
+              String val = reader.getString(i, readerContext);
+              creator.putString(val);
+            } else {
+              String[] strings = reader.getStringMV(i, readerContext);
+              creator.putStringMV(strings);
+            }
           }
+          break;
         }
-        break;
-      }
-      case BIG_DECIMAL: {
-        for (int i = 0; i < numDocs; i++) {
+        case BYTES: {
+          for (int i = 0; i < numDocs; i++) {
+            if (isSVColumn) {
+              byte[] val = reader.getBytes(i, readerContext);
+              creator.putBytes(val);
+            } else {
+              byte[][] bytesArray = reader.getBytesMV(i, readerContext);
+              creator.putBytesMV(bytesArray);
+            }
+          }
+          break;
+        }
+        case BIG_DECIMAL: {
           Preconditions.checkState(isSVColumn, "BigDecimal is not supported for MV columns");
-          BigDecimal val = reader.getBigDecimal(i, readerContext);
-          creator.putBigDecimal(val);
+          for (int i = 0; i < numDocs; i++) {
+            BigDecimal val = reader.getBigDecimal(i, readerContext);
+            creator.putBigDecimal(val);
+          }
+          break;
         }
-        break;
+        default:
+          throw new IllegalStateException();
+      }
+    }
+  }
+
+  private void enableDictionary(String column, SegmentDirectory.Writer segmentWriter,
+      IndexCreatorProvider indexCreatorProvider)
+      throws Exception {
+    Preconditions.checkState(_segmentMetadata.getVersion() == SegmentVersion.v3);
+    ColumnMetadata existingColMetadata = _segmentMetadata.getColumnMetadataFor(column);
+    Preconditions.checkState(!existingColMetadata.hasDictionary(),
+        "Cannot rewrite dictionary enabled forward index. Dictionary already exists for column:" + column);

Review Comment:
   It will actually be a no-op if version < v3 || !existingColMetadata.hasDictionary(). We check for this and return NO_OP in computeOperation.
   
   This precondition check is just to make sure that other callers from test don't call this function when dictionary is already enabled.



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


[GitHub] [pinot] siddharthteotia commented on a diff in pull request #9678: Support creating dictionary at runtime for an existing column

Posted by GitBox <gi...@apache.org>.
siddharthteotia commented on code in PR #9678:
URL: https://github.com/apache/pinot/pull/9678#discussion_r1011068710


##########
pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/loader/ForwardIndexHandler.java:
##########
@@ -271,118 +304,295 @@ private void rewriteRawSVForwardIndex(String column, ColumnMetadata existingColM
               .forForwardIndex(newCompressionType, _indexLoadingConfig.getColumnProperties());
 
       try (ForwardIndexCreator creator = indexCreatorProvider.newForwardIndexCreator(context)) {
-        // If creator stored type and the reader stored type do not match, throw an exception.
         if (!reader.getStoredType().equals(creator.getValueType())) {
+          // Creator stored type should match reader stored type for raw columns. We do not support changing datatypes.
           String failureMsg =
               "Unsupported operation to change datatype for column=" + column + " from " + reader.getStoredType()
                   .toString() + " to " + creator.getValueType().toString();
           throw new UnsupportedOperationException(failureMsg);
         }
 
         int numDocs = existingColMetadata.getTotalDocs();
-        forwardIndexWriterHelper(column, reader, creator, numDocs);
+        forwardIndexWriterHelper(column, existingColMetadata, reader, creator, numDocs, null);
       }
     }
   }
 
-  private void forwardIndexWriterHelper(String column, ForwardIndexReader reader, ForwardIndexCreator creator,
-      int numDocs) {
-    // If creator stored type should match reader stored type. We do not support changing datatypes.
-    if (!reader.getStoredType().equals(creator.getValueType())) {
-      String failureMsg =
-          "Unsupported operation to change datatype for column=" + column + " from " + reader.getStoredType().toString()
-              + " to " + creator.getValueType().toString();
-      throw new UnsupportedOperationException(failureMsg);
-    }
-
+  private void forwardIndexWriterHelper(String column, ColumnMetadata existingColumnMetadata, ForwardIndexReader reader,
+      ForwardIndexCreator creator, int numDocs, @Nullable SegmentDictionaryCreator dictionaryCreator) {
     ForwardIndexReaderContext readerContext = reader.createContext();
     boolean isSVColumn = reader.isSingleValue();
 
-    switch (reader.getStoredType()) {
-      // JSON fields are either stored as string or bytes. No special handling is needed because we make this
-      // decision based on the storedType of the reader.
-      case INT: {
-        for (int i = 0; i < numDocs; i++) {
-          if (isSVColumn) {
-            int val = reader.getInt(i, readerContext);
-            creator.putInt(val);
-          } else {
-            int[] ints = reader.getIntMV(i, readerContext);
-            creator.putIntMV(ints);
-          }
+    if (dictionaryCreator != null) {
+      int maxNumValuesPerEntry = existingColumnMetadata.getMaxNumberOfMultiValues();
+      PinotSegmentColumnReader columnReader = new PinotSegmentColumnReader(reader, null, null, maxNumValuesPerEntry);
+
+      for (int i = 0; i < numDocs; i++) {
+        Object obj = columnReader.getValue(i);
+
+        if (isSVColumn) {
+          int dictId = dictionaryCreator.indexOfSV(obj);
+          creator.putDictId(dictId);
+        } else {
+          int[] dictIds = dictionaryCreator.indexOfMV(obj);
+          creator.putDictIdMV(dictIds);
         }
-        break;
       }
-      case LONG: {
-        for (int i = 0; i < numDocs; i++) {
-          if (isSVColumn) {
-            long val = reader.getLong(i, readerContext);
-            creator.putLong(val);
-          } else {
-            long[] longs = reader.getLongMV(i, readerContext);
-            creator.putLongMV(longs);
+    } else {
+      switch (reader.getStoredType()) {
+        // JSON fields are either stored as string or bytes. No special handling is needed because we make this
+        // decision based on the storedType of the reader.
+        case INT: {
+          for (int i = 0; i < numDocs; i++) {
+            if (isSVColumn) {
+              int val = reader.getInt(i, readerContext);
+              creator.putInt(val);
+            } else {
+              int[] ints = reader.getIntMV(i, readerContext);
+              creator.putIntMV(ints);
+            }
           }
+          break;
         }
-        break;
-      }
-      case FLOAT: {
-        for (int i = 0; i < numDocs; i++) {
-          if (isSVColumn) {
-            float val = reader.getFloat(i, readerContext);
-            creator.putFloat(val);
-          } else {
-            float[] floats = reader.getFloatMV(i, readerContext);
-            creator.putFloatMV(floats);
+        case LONG: {
+          for (int i = 0; i < numDocs; i++) {
+            if (isSVColumn) {
+              long val = reader.getLong(i, readerContext);
+              creator.putLong(val);
+            } else {
+              long[] longs = reader.getLongMV(i, readerContext);
+              creator.putLongMV(longs);
+            }
           }
+          break;
         }
-        break;
-      }
-      case DOUBLE: {
-        for (int i = 0; i < numDocs; i++) {
-          if (isSVColumn) {
-            double val = reader.getDouble(i, readerContext);
-            creator.putDouble(val);
-          } else {
-            double[] doubles = reader.getDoubleMV(i, readerContext);
-            creator.putDoubleMV(doubles);
+        case FLOAT: {
+          for (int i = 0; i < numDocs; i++) {
+            if (isSVColumn) {
+              float val = reader.getFloat(i, readerContext);
+              creator.putFloat(val);
+            } else {
+              float[] floats = reader.getFloatMV(i, readerContext);
+              creator.putFloatMV(floats);
+            }
           }
+          break;
         }
-        break;
-      }
-      case STRING: {
-        for (int i = 0; i < numDocs; i++) {
-          if (isSVColumn) {
-            String val = reader.getString(i, readerContext);
-            creator.putString(val);
-          } else {
-            String[] strings = reader.getStringMV(i, readerContext);
-            creator.putStringMV(strings);
+        case DOUBLE: {
+          for (int i = 0; i < numDocs; i++) {
+            if (isSVColumn) {
+              double val = reader.getDouble(i, readerContext);
+              creator.putDouble(val);
+            } else {
+              double[] doubles = reader.getDoubleMV(i, readerContext);
+              creator.putDoubleMV(doubles);
+            }
           }
+          break;
         }
-        break;
-      }
-      case BYTES: {
-        for (int i = 0; i < numDocs; i++) {
-          if (isSVColumn) {
-            byte[] val = reader.getBytes(i, readerContext);
-            creator.putBytes(val);
-          } else {
-            byte[][] bytesArray = reader.getBytesMV(i, readerContext);
-            creator.putBytesMV(bytesArray);
+        case STRING: {
+          for (int i = 0; i < numDocs; i++) {
+            if (isSVColumn) {
+              String val = reader.getString(i, readerContext);
+              creator.putString(val);
+            } else {
+              String[] strings = reader.getStringMV(i, readerContext);
+              creator.putStringMV(strings);
+            }
           }
+          break;
         }
-        break;
-      }
-      case BIG_DECIMAL: {
-        for (int i = 0; i < numDocs; i++) {
+        case BYTES: {
+          for (int i = 0; i < numDocs; i++) {
+            if (isSVColumn) {
+              byte[] val = reader.getBytes(i, readerContext);
+              creator.putBytes(val);
+            } else {
+              byte[][] bytesArray = reader.getBytesMV(i, readerContext);
+              creator.putBytesMV(bytesArray);
+            }
+          }
+          break;
+        }
+        case BIG_DECIMAL: {
           Preconditions.checkState(isSVColumn, "BigDecimal is not supported for MV columns");
-          BigDecimal val = reader.getBigDecimal(i, readerContext);
-          creator.putBigDecimal(val);
+          for (int i = 0; i < numDocs; i++) {
+            BigDecimal val = reader.getBigDecimal(i, readerContext);
+            creator.putBigDecimal(val);
+          }
+          break;
         }
-        break;
+        default:
+          throw new IllegalStateException();
+      }
+    }
+  }
+
+  private void enableDictionary(String column, SegmentDirectory.Writer segmentWriter,
+      IndexCreatorProvider indexCreatorProvider)
+      throws Exception {
+    Preconditions.checkState(_segmentMetadata.getVersion() == SegmentVersion.v3);
+    ColumnMetadata existingColMetadata = _segmentMetadata.getColumnMetadataFor(column);
+    Preconditions.checkState(!existingColMetadata.hasDictionary(),
+        "Cannot rewrite dictionary enabled forward index. Dictionary already exists for column:" + column);
+    boolean isSingleValue = existingColMetadata.isSingleValue();
+
+    File indexDir = _segmentMetadata.getIndexDir();
+    String segmentName = _segmentMetadata.getName();
+    File inProgress = new File(indexDir, column + ".dict.inprogress");
+    File dictionaryFile = new File(indexDir, column + V1Constants.Dict.FILE_EXTENSION);
+    String fwdIndexFileExtension;
+    if (isSingleValue) {
+      fwdIndexFileExtension =
+          existingColMetadata.isSorted() ? V1Constants.Indexes.SORTED_SV_FORWARD_INDEX_FILE_EXTENSION
+              : V1Constants.Indexes.UNSORTED_SV_FORWARD_INDEX_FILE_EXTENSION;
+    } else {
+      fwdIndexFileExtension = V1Constants.Indexes.UNSORTED_MV_FORWARD_INDEX_FILE_EXTENSION;
+    }
+    File fwdIndexFile = new File(indexDir, column + fwdIndexFileExtension);
+
+    if (!inProgress.exists()) {
+      // Marker file does not exist, which means last run ended normally.
+      // Create a marker file.
+      FileUtils.touch(inProgress);
+    } else {
+      // Marker file exists, which means last run was interrupted.
+      // Remove forward index and dictionary files if they exist.
+      FileUtils.deleteQuietly(fwdIndexFile);
+      FileUtils.deleteQuietly(dictionaryFile);
+    }
+
+    LOGGER.info("Creating a new dictionary for segment={} and column={}", segmentName, column);
+    SegmentDictionaryCreator dictionaryCreator = buildDictionary(column, existingColMetadata, segmentWriter);
+    LoaderUtils.writeIndexToV3Format(segmentWriter, column, dictionaryFile, ColumnIndexType.DICTIONARY);
+
+    LOGGER.info("Built dictionary. Rewriting dictionary enabled forward index for segment={} and column={}",
+        segmentName, column);
+    writeDictEnabledForwardIndex(column, existingColMetadata, segmentWriter, indexDir, indexCreatorProvider,
+        dictionaryCreator);
+    // We used the existing forward index to generate a new forward index. The existing forward index will be in V3
+    // format and the new forward index will be in V1 format. Remove the existing forward index as it is not needed
+    // anymore. Note that removeIndex() will only mark an index for removal and remove the in-memory state. The
+    // actual cleanup from columns.psf file will happen when singleFileIndexDirectory.cleanupRemovedIndices() is
+    // called during segmentWriter.close().
+    segmentWriter.removeIndex(column, ColumnIndexType.FORWARD_INDEX);
+    LoaderUtils.writeIndexToV3Format(segmentWriter, column, fwdIndexFile, ColumnIndexType.FORWARD_INDEX);
+
+    LOGGER.info("Created forwardIndex. Updating metadata properties for segment={} and column={}", segmentName, column);
+    Map<String, String> metadataProperties = new HashMap<>();
+    metadataProperties.put(getKeyFor(column, HAS_DICTIONARY), String.valueOf(true));
+    metadataProperties.put(getKeyFor(column, DICTIONARY_ELEMENT_SIZE),
+        String.valueOf(dictionaryCreator.getNumBytesPerEntry()));
+    updateMetadataProperties(indexDir, metadataProperties);
+
+    // We remove indexes that have to be rewritten when a dictEnabled is toggled. Note that the respective index
+    // handler will take care of recreating the index.
+    removeDictRelatedIndexes(column, segmentWriter);
+
+    // Delete the marker file.
+    FileUtils.deleteQuietly(inProgress);
+
+    LOGGER.info("Created dictionary based forward index for segment: {}, column: {}", segmentName, column);
+  }
+
+  private SegmentDictionaryCreator buildDictionary(String column, ColumnMetadata existingColMetadata,
+      SegmentDirectory.Writer segmentWriter)
+      throws Exception {
+    int numDocs = existingColMetadata.getTotalDocs();
+    // SegmentPartitionConfig is not relevant for rewrites.
+    StatsCollectorConfig statsCollectorConfig =
+        new StatsCollectorConfig(_indexLoadingConfig.getTableConfig(), _schema, null);
+    AbstractColumnStatisticsCollector statsCollector;
+
+    try (ForwardIndexReader reader = LoaderUtils.getForwardIndexReader(segmentWriter, existingColMetadata)) {
+      boolean isSVColumn = reader.isSingleValue();
+
+      switch (reader.getStoredType()) {
+        case INT:
+          statsCollector = new IntColumnPreIndexStatsCollector(column, statsCollectorConfig);
+          break;
+        case LONG:
+          statsCollector = new LongColumnPreIndexStatsCollector(column, statsCollectorConfig);
+          break;
+        case FLOAT:
+          statsCollector = new FloatColumnPreIndexStatsCollector(column, statsCollectorConfig);
+          break;
+        case DOUBLE:
+          statsCollector = new DoubleColumnPreIndexStatsCollector(column, statsCollectorConfig);
+          break;
+        case STRING:
+          statsCollector = new StringColumnPreIndexStatsCollector(column, statsCollectorConfig);
+          break;
+        case BYTES:
+          statsCollector = new BytesColumnPredIndexStatsCollector(column, statsCollectorConfig);
+          break;
+        case BIG_DECIMAL:
+          Preconditions.checkState(isSVColumn, "BigDecimal is not supported for MV columns");
+          statsCollector = new BigDecimalColumnPreIndexStatsCollector(column, statsCollectorConfig);
+          break;
+        default:
+          throw new IllegalStateException();
+      }
+
+      Preconditions.checkState(statsCollector != null);
+      // Note: Special Null handling is not necessary here. This is because, the existing default null value in the
+      // raw forwardIndex will be retained as such while created the dictionary and dict-based forward index. Also,
+      // null value vectors maintain a bitmap of docIds. No handling is necessary there.
+      PinotSegmentColumnReader columnReader =
+          new PinotSegmentColumnReader(reader, null, null, existingColMetadata.getMaxNumberOfMultiValues());
+      for (int i = 0; i < numDocs; i++) {
+        Object obj = columnReader.getValue(i);
+        statsCollector.collect(obj);
+      }
+      statsCollector.seal();
+
+      boolean useVarLength = SegmentIndexCreationDriverImpl.shouldUseVarLengthDictionary(column,
+          _indexLoadingConfig.getVarLengthDictionaryColumns(), reader.getStoredType(), statsCollector);
+      SegmentDictionaryCreator dictionaryCreator =
+          new SegmentDictionaryCreator(existingColMetadata.getFieldSpec(), _segmentMetadata.getIndexDir(),
+              useVarLength);
+
+      dictionaryCreator.build(statsCollector.getUniqueValuesSet());
+      return dictionaryCreator;
+    }
+  }
+
+  private void writeDictEnabledForwardIndex(String column, ColumnMetadata existingColMetadata,
+      SegmentDirectory.Writer segmentWriter, File indexDir, IndexCreatorProvider indexCreatorProvider,
+      SegmentDictionaryCreator dictionaryCreator)
+      throws Exception {
+    try (ForwardIndexReader reader = LoaderUtils.getForwardIndexReader(segmentWriter, existingColMetadata)) {
+      int lengthOfLongestEntry = reader.getLengthOfLongestEntry();
+      IndexCreationContext.Builder builder =
+          IndexCreationContext.builder().withIndexDir(indexDir).withColumnMetadata(existingColMetadata)
+              .withLengthOfLongestEntry(lengthOfLongestEntry);
+      // existingColMetadata has dictEnable=false. Overwrite the value.
+      builder.withDictionary(true);
+      IndexCreationContext.Forward context =
+          builder.build().forForwardIndex(null, _indexLoadingConfig.getColumnProperties());
+
+      try (ForwardIndexCreator creator = indexCreatorProvider.newForwardIndexCreator(context)) {
+        int numDocs = existingColMetadata.getTotalDocs();
+        forwardIndexWriterHelper(column, existingColMetadata, reader, creator, numDocs, dictionaryCreator);
       }
-      default:
-        throw new IllegalStateException();
     }
   }
+
+  private void removeDictRelatedIndexes(String column, SegmentDirectory.Writer segmentWriter) {
+    // TODO: Move this logic as a static function in each index creator.
+    segmentWriter.removeIndex(column, ColumnIndexType.RANGE_INDEX);
+  }
+
+  private void updateMetadataProperties(File indexDir, Map<String, String> metadataProperties)
+      throws Exception {
+    File v3Dir = SegmentDirectoryPaths.segmentDirectoryFor(indexDir, SegmentVersion.v3);
+    File metadataFile = new File(v3Dir, V1Constants.MetadataKeys.METADATA_FILE_NAME);
+    PropertiesConfiguration properties = new PropertiesConfiguration(metadataFile);
+
+    for (Map.Entry<String, String> entry : metadataProperties.entrySet()) {
+      properties.setProperty(entry.getKey(), entry.getValue());
+    }
+
+    properties.save();

Review Comment:
   Don't we need to pass in the output stream to save ?



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


[GitHub] [pinot] siddharthteotia commented on a diff in pull request #9678: Support creating dictionary at runtime for an existing column

Posted by GitBox <gi...@apache.org>.
siddharthteotia commented on code in PR #9678:
URL: https://github.com/apache/pinot/pull/9678#discussion_r1012267144


##########
pinot-core/src/test/java/org/apache/pinot/queries/RangeQueriesTest.java:
##########
@@ -223,6 +223,42 @@ public void testSelectionOverRangeFilter(String query, int min, int max, boolean
     }
   }
 
+  @Test(dataProvider = "selectionTestCases")
+  public void testSelectionOverRangeFilterAfterReload(String query, int min, int max, boolean inclusive)

Review Comment:
   We should add other tests like
   
   - Run query on a raw column using that in filter
   - Enable dict + reload
   - Run same query again and it should use dict based predicate evaluator and return same result
   
   Same goes for using the column in SELECT clause. It should correctly return the same result after doing dual lookup in rewriten fwd index and dict as it did before with raw fwd index. Things like that
   
   



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


[GitHub] [pinot] vvivekiyer commented on a diff in pull request #9678: Support creating dictionary at runtime for an existing column

Posted by GitBox <gi...@apache.org>.
vvivekiyer commented on code in PR #9678:
URL: https://github.com/apache/pinot/pull/9678#discussion_r1012180322


##########
pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/loader/ForwardIndexHandler.java:
##########
@@ -271,118 +304,295 @@ private void rewriteRawSVForwardIndex(String column, ColumnMetadata existingColM
               .forForwardIndex(newCompressionType, _indexLoadingConfig.getColumnProperties());
 
       try (ForwardIndexCreator creator = indexCreatorProvider.newForwardIndexCreator(context)) {
-        // If creator stored type and the reader stored type do not match, throw an exception.
         if (!reader.getStoredType().equals(creator.getValueType())) {
+          // Creator stored type should match reader stored type for raw columns. We do not support changing datatypes.
           String failureMsg =
               "Unsupported operation to change datatype for column=" + column + " from " + reader.getStoredType()
                   .toString() + " to " + creator.getValueType().toString();
           throw new UnsupportedOperationException(failureMsg);
         }
 
         int numDocs = existingColMetadata.getTotalDocs();
-        forwardIndexWriterHelper(column, reader, creator, numDocs);
+        forwardIndexWriterHelper(column, existingColMetadata, reader, creator, numDocs, null);
       }
     }
   }
 
-  private void forwardIndexWriterHelper(String column, ForwardIndexReader reader, ForwardIndexCreator creator,
-      int numDocs) {
-    // If creator stored type should match reader stored type. We do not support changing datatypes.
-    if (!reader.getStoredType().equals(creator.getValueType())) {
-      String failureMsg =
-          "Unsupported operation to change datatype for column=" + column + " from " + reader.getStoredType().toString()
-              + " to " + creator.getValueType().toString();
-      throw new UnsupportedOperationException(failureMsg);
-    }
-
+  private void forwardIndexWriterHelper(String column, ColumnMetadata existingColumnMetadata, ForwardIndexReader reader,
+      ForwardIndexCreator creator, int numDocs, @Nullable SegmentDictionaryCreator dictionaryCreator) {
     ForwardIndexReaderContext readerContext = reader.createContext();
     boolean isSVColumn = reader.isSingleValue();
 
-    switch (reader.getStoredType()) {
-      // JSON fields are either stored as string or bytes. No special handling is needed because we make this
-      // decision based on the storedType of the reader.
-      case INT: {
-        for (int i = 0; i < numDocs; i++) {
-          if (isSVColumn) {
-            int val = reader.getInt(i, readerContext);
-            creator.putInt(val);
-          } else {
-            int[] ints = reader.getIntMV(i, readerContext);
-            creator.putIntMV(ints);
-          }
+    if (dictionaryCreator != null) {
+      int maxNumValuesPerEntry = existingColumnMetadata.getMaxNumberOfMultiValues();
+      PinotSegmentColumnReader columnReader = new PinotSegmentColumnReader(reader, null, null, maxNumValuesPerEntry);
+
+      for (int i = 0; i < numDocs; i++) {
+        Object obj = columnReader.getValue(i);
+
+        if (isSVColumn) {
+          int dictId = dictionaryCreator.indexOfSV(obj);
+          creator.putDictId(dictId);
+        } else {
+          int[] dictIds = dictionaryCreator.indexOfMV(obj);
+          creator.putDictIdMV(dictIds);
         }
-        break;
       }
-      case LONG: {
-        for (int i = 0; i < numDocs; i++) {
-          if (isSVColumn) {
-            long val = reader.getLong(i, readerContext);
-            creator.putLong(val);
-          } else {
-            long[] longs = reader.getLongMV(i, readerContext);
-            creator.putLongMV(longs);
+    } else {
+      switch (reader.getStoredType()) {
+        // JSON fields are either stored as string or bytes. No special handling is needed because we make this
+        // decision based on the storedType of the reader.
+        case INT: {
+          for (int i = 0; i < numDocs; i++) {
+            if (isSVColumn) {
+              int val = reader.getInt(i, readerContext);
+              creator.putInt(val);
+            } else {
+              int[] ints = reader.getIntMV(i, readerContext);
+              creator.putIntMV(ints);
+            }
           }
+          break;
         }
-        break;
-      }
-      case FLOAT: {
-        for (int i = 0; i < numDocs; i++) {
-          if (isSVColumn) {
-            float val = reader.getFloat(i, readerContext);
-            creator.putFloat(val);
-          } else {
-            float[] floats = reader.getFloatMV(i, readerContext);
-            creator.putFloatMV(floats);
+        case LONG: {
+          for (int i = 0; i < numDocs; i++) {
+            if (isSVColumn) {
+              long val = reader.getLong(i, readerContext);
+              creator.putLong(val);
+            } else {
+              long[] longs = reader.getLongMV(i, readerContext);
+              creator.putLongMV(longs);
+            }
           }
+          break;
         }
-        break;
-      }
-      case DOUBLE: {
-        for (int i = 0; i < numDocs; i++) {
-          if (isSVColumn) {
-            double val = reader.getDouble(i, readerContext);
-            creator.putDouble(val);
-          } else {
-            double[] doubles = reader.getDoubleMV(i, readerContext);
-            creator.putDoubleMV(doubles);
+        case FLOAT: {
+          for (int i = 0; i < numDocs; i++) {
+            if (isSVColumn) {
+              float val = reader.getFloat(i, readerContext);
+              creator.putFloat(val);
+            } else {
+              float[] floats = reader.getFloatMV(i, readerContext);
+              creator.putFloatMV(floats);
+            }
           }
+          break;
         }
-        break;
-      }
-      case STRING: {
-        for (int i = 0; i < numDocs; i++) {
-          if (isSVColumn) {
-            String val = reader.getString(i, readerContext);
-            creator.putString(val);
-          } else {
-            String[] strings = reader.getStringMV(i, readerContext);
-            creator.putStringMV(strings);
+        case DOUBLE: {
+          for (int i = 0; i < numDocs; i++) {
+            if (isSVColumn) {
+              double val = reader.getDouble(i, readerContext);
+              creator.putDouble(val);
+            } else {
+              double[] doubles = reader.getDoubleMV(i, readerContext);
+              creator.putDoubleMV(doubles);
+            }
           }
+          break;
         }
-        break;
-      }
-      case BYTES: {
-        for (int i = 0; i < numDocs; i++) {
-          if (isSVColumn) {
-            byte[] val = reader.getBytes(i, readerContext);
-            creator.putBytes(val);
-          } else {
-            byte[][] bytesArray = reader.getBytesMV(i, readerContext);
-            creator.putBytesMV(bytesArray);
+        case STRING: {
+          for (int i = 0; i < numDocs; i++) {
+            if (isSVColumn) {
+              String val = reader.getString(i, readerContext);
+              creator.putString(val);
+            } else {
+              String[] strings = reader.getStringMV(i, readerContext);
+              creator.putStringMV(strings);
+            }
           }
+          break;
         }
-        break;
-      }
-      case BIG_DECIMAL: {
-        for (int i = 0; i < numDocs; i++) {
+        case BYTES: {
+          for (int i = 0; i < numDocs; i++) {
+            if (isSVColumn) {
+              byte[] val = reader.getBytes(i, readerContext);
+              creator.putBytes(val);
+            } else {
+              byte[][] bytesArray = reader.getBytesMV(i, readerContext);
+              creator.putBytesMV(bytesArray);
+            }
+          }
+          break;
+        }
+        case BIG_DECIMAL: {
           Preconditions.checkState(isSVColumn, "BigDecimal is not supported for MV columns");
-          BigDecimal val = reader.getBigDecimal(i, readerContext);
-          creator.putBigDecimal(val);
+          for (int i = 0; i < numDocs; i++) {
+            BigDecimal val = reader.getBigDecimal(i, readerContext);
+            creator.putBigDecimal(val);
+          }
+          break;
         }
-        break;
+        default:
+          throw new IllegalStateException();
+      }
+    }
+  }
+
+  private void enableDictionary(String column, SegmentDirectory.Writer segmentWriter,
+      IndexCreatorProvider indexCreatorProvider)
+      throws Exception {
+    Preconditions.checkState(_segmentMetadata.getVersion() == SegmentVersion.v3);
+    ColumnMetadata existingColMetadata = _segmentMetadata.getColumnMetadataFor(column);
+    Preconditions.checkState(!existingColMetadata.hasDictionary(),
+        "Cannot rewrite dictionary enabled forward index. Dictionary already exists for column:" + column);
+    boolean isSingleValue = existingColMetadata.isSingleValue();
+
+    File indexDir = _segmentMetadata.getIndexDir();
+    String segmentName = _segmentMetadata.getName();
+    File inProgress = new File(indexDir, column + ".dict.inprogress");
+    File dictionaryFile = new File(indexDir, column + V1Constants.Dict.FILE_EXTENSION);
+    String fwdIndexFileExtension;
+    if (isSingleValue) {
+      fwdIndexFileExtension =
+          existingColMetadata.isSorted() ? V1Constants.Indexes.SORTED_SV_FORWARD_INDEX_FILE_EXTENSION
+              : V1Constants.Indexes.UNSORTED_SV_FORWARD_INDEX_FILE_EXTENSION;
+    } else {
+      fwdIndexFileExtension = V1Constants.Indexes.UNSORTED_MV_FORWARD_INDEX_FILE_EXTENSION;
+    }
+    File fwdIndexFile = new File(indexDir, column + fwdIndexFileExtension);
+
+    if (!inProgress.exists()) {
+      // Marker file does not exist, which means last run ended normally.
+      // Create a marker file.
+      FileUtils.touch(inProgress);
+    } else {
+      // Marker file exists, which means last run was interrupted.
+      // Remove forward index and dictionary files if they exist.
+      FileUtils.deleteQuietly(fwdIndexFile);
+      FileUtils.deleteQuietly(dictionaryFile);
+    }
+
+    LOGGER.info("Creating a new dictionary for segment={} and column={}", segmentName, column);
+    SegmentDictionaryCreator dictionaryCreator = buildDictionary(column, existingColMetadata, segmentWriter);
+    LoaderUtils.writeIndexToV3Format(segmentWriter, column, dictionaryFile, ColumnIndexType.DICTIONARY);
+
+    LOGGER.info("Built dictionary. Rewriting dictionary enabled forward index for segment={} and column={}",

Review Comment:
   > and metadata will reflect that dictionary exists
   
   Actually, at this point, the metadata file will not be rewritten to contain dictionary. So, we're good.
   
   > Is this handled as part of the overall failure handling framework during reload / load ?
   
   Yes. It is. So, the failure handling framework creates a backup index directory and works on it. If there's an issue, the  only the backup directory is affected.



##########
pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/loader/SegmentPreProcessor.java:
##########
@@ -104,8 +104,15 @@ public void process()
       // Update single-column indices, like inverted index, json index etc.
       IndexCreatorProvider indexCreatorProvider = IndexingOverrides.getIndexCreatorProvider();
       for (ColumnIndexType type : ColumnIndexType.values()) {
-        IndexHandler handler = IndexHandlerFactory.getIndexHandler(type, _segmentMetadata, _indexLoadingConfig);
+        IndexHandler handler =
+            IndexHandlerFactory.getIndexHandler(type, _segmentMetadata, _indexLoadingConfig, _schema);
         handler.updateIndices(segmentWriter, indexCreatorProvider);
+        if (type == ColumnIndexType.FORWARD_INDEX) {

Review Comment:
   Ack. Can we take this up as a TODO? I have a few ideas and we can discuss this offline to refactor the code.



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


[GitHub] [pinot] vvivekiyer commented on a diff in pull request #9678: Support creating dictionary at runtime for an existing column

Posted by GitBox <gi...@apache.org>.
vvivekiyer commented on code in PR #9678:
URL: https://github.com/apache/pinot/pull/9678#discussion_r1012174268


##########
pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/loader/ForwardIndexHandler.java:
##########
@@ -271,118 +304,295 @@ private void rewriteRawSVForwardIndex(String column, ColumnMetadata existingColM
               .forForwardIndex(newCompressionType, _indexLoadingConfig.getColumnProperties());
 
       try (ForwardIndexCreator creator = indexCreatorProvider.newForwardIndexCreator(context)) {
-        // If creator stored type and the reader stored type do not match, throw an exception.
         if (!reader.getStoredType().equals(creator.getValueType())) {
+          // Creator stored type should match reader stored type for raw columns. We do not support changing datatypes.
           String failureMsg =
               "Unsupported operation to change datatype for column=" + column + " from " + reader.getStoredType()
                   .toString() + " to " + creator.getValueType().toString();
           throw new UnsupportedOperationException(failureMsg);
         }
 
         int numDocs = existingColMetadata.getTotalDocs();
-        forwardIndexWriterHelper(column, reader, creator, numDocs);
+        forwardIndexWriterHelper(column, existingColMetadata, reader, creator, numDocs, null);
       }
     }
   }
 
-  private void forwardIndexWriterHelper(String column, ForwardIndexReader reader, ForwardIndexCreator creator,
-      int numDocs) {
-    // If creator stored type should match reader stored type. We do not support changing datatypes.
-    if (!reader.getStoredType().equals(creator.getValueType())) {
-      String failureMsg =
-          "Unsupported operation to change datatype for column=" + column + " from " + reader.getStoredType().toString()
-              + " to " + creator.getValueType().toString();
-      throw new UnsupportedOperationException(failureMsg);
-    }
-
+  private void forwardIndexWriterHelper(String column, ColumnMetadata existingColumnMetadata, ForwardIndexReader reader,
+      ForwardIndexCreator creator, int numDocs, @Nullable SegmentDictionaryCreator dictionaryCreator) {
     ForwardIndexReaderContext readerContext = reader.createContext();
     boolean isSVColumn = reader.isSingleValue();
 
-    switch (reader.getStoredType()) {
-      // JSON fields are either stored as string or bytes. No special handling is needed because we make this
-      // decision based on the storedType of the reader.
-      case INT: {
-        for (int i = 0; i < numDocs; i++) {
-          if (isSVColumn) {
-            int val = reader.getInt(i, readerContext);
-            creator.putInt(val);
-          } else {
-            int[] ints = reader.getIntMV(i, readerContext);
-            creator.putIntMV(ints);
-          }
+    if (dictionaryCreator != null) {
+      int maxNumValuesPerEntry = existingColumnMetadata.getMaxNumberOfMultiValues();
+      PinotSegmentColumnReader columnReader = new PinotSegmentColumnReader(reader, null, null, maxNumValuesPerEntry);
+
+      for (int i = 0; i < numDocs; i++) {
+        Object obj = columnReader.getValue(i);
+
+        if (isSVColumn) {
+          int dictId = dictionaryCreator.indexOfSV(obj);
+          creator.putDictId(dictId);
+        } else {
+          int[] dictIds = dictionaryCreator.indexOfMV(obj);
+          creator.putDictIdMV(dictIds);
         }
-        break;
       }
-      case LONG: {
-        for (int i = 0; i < numDocs; i++) {
-          if (isSVColumn) {
-            long val = reader.getLong(i, readerContext);
-            creator.putLong(val);
-          } else {
-            long[] longs = reader.getLongMV(i, readerContext);
-            creator.putLongMV(longs);
+    } else {
+      switch (reader.getStoredType()) {
+        // JSON fields are either stored as string or bytes. No special handling is needed because we make this
+        // decision based on the storedType of the reader.
+        case INT: {
+          for (int i = 0; i < numDocs; i++) {
+            if (isSVColumn) {
+              int val = reader.getInt(i, readerContext);
+              creator.putInt(val);
+            } else {
+              int[] ints = reader.getIntMV(i, readerContext);
+              creator.putIntMV(ints);
+            }
           }
+          break;
         }
-        break;
-      }
-      case FLOAT: {
-        for (int i = 0; i < numDocs; i++) {
-          if (isSVColumn) {
-            float val = reader.getFloat(i, readerContext);
-            creator.putFloat(val);
-          } else {
-            float[] floats = reader.getFloatMV(i, readerContext);
-            creator.putFloatMV(floats);
+        case LONG: {
+          for (int i = 0; i < numDocs; i++) {
+            if (isSVColumn) {
+              long val = reader.getLong(i, readerContext);
+              creator.putLong(val);
+            } else {
+              long[] longs = reader.getLongMV(i, readerContext);
+              creator.putLongMV(longs);
+            }
           }
+          break;
         }
-        break;
-      }
-      case DOUBLE: {
-        for (int i = 0; i < numDocs; i++) {
-          if (isSVColumn) {
-            double val = reader.getDouble(i, readerContext);
-            creator.putDouble(val);
-          } else {
-            double[] doubles = reader.getDoubleMV(i, readerContext);
-            creator.putDoubleMV(doubles);
+        case FLOAT: {
+          for (int i = 0; i < numDocs; i++) {
+            if (isSVColumn) {
+              float val = reader.getFloat(i, readerContext);
+              creator.putFloat(val);
+            } else {
+              float[] floats = reader.getFloatMV(i, readerContext);
+              creator.putFloatMV(floats);
+            }
           }
+          break;
         }
-        break;
-      }
-      case STRING: {
-        for (int i = 0; i < numDocs; i++) {
-          if (isSVColumn) {
-            String val = reader.getString(i, readerContext);
-            creator.putString(val);
-          } else {
-            String[] strings = reader.getStringMV(i, readerContext);
-            creator.putStringMV(strings);
+        case DOUBLE: {
+          for (int i = 0; i < numDocs; i++) {
+            if (isSVColumn) {
+              double val = reader.getDouble(i, readerContext);
+              creator.putDouble(val);
+            } else {
+              double[] doubles = reader.getDoubleMV(i, readerContext);
+              creator.putDoubleMV(doubles);
+            }
           }
+          break;
         }
-        break;
-      }
-      case BYTES: {
-        for (int i = 0; i < numDocs; i++) {
-          if (isSVColumn) {
-            byte[] val = reader.getBytes(i, readerContext);
-            creator.putBytes(val);
-          } else {
-            byte[][] bytesArray = reader.getBytesMV(i, readerContext);
-            creator.putBytesMV(bytesArray);
+        case STRING: {
+          for (int i = 0; i < numDocs; i++) {
+            if (isSVColumn) {
+              String val = reader.getString(i, readerContext);
+              creator.putString(val);
+            } else {
+              String[] strings = reader.getStringMV(i, readerContext);
+              creator.putStringMV(strings);
+            }
           }
+          break;
         }
-        break;
-      }
-      case BIG_DECIMAL: {
-        for (int i = 0; i < numDocs; i++) {
+        case BYTES: {
+          for (int i = 0; i < numDocs; i++) {
+            if (isSVColumn) {
+              byte[] val = reader.getBytes(i, readerContext);
+              creator.putBytes(val);
+            } else {
+              byte[][] bytesArray = reader.getBytesMV(i, readerContext);
+              creator.putBytesMV(bytesArray);
+            }
+          }
+          break;
+        }
+        case BIG_DECIMAL: {
           Preconditions.checkState(isSVColumn, "BigDecimal is not supported for MV columns");
-          BigDecimal val = reader.getBigDecimal(i, readerContext);
-          creator.putBigDecimal(val);
+          for (int i = 0; i < numDocs; i++) {
+            BigDecimal val = reader.getBigDecimal(i, readerContext);
+            creator.putBigDecimal(val);
+          }
+          break;
         }
-        break;
+        default:
+          throw new IllegalStateException();
+      }
+    }
+  }
+
+  private void enableDictionary(String column, SegmentDirectory.Writer segmentWriter,
+      IndexCreatorProvider indexCreatorProvider)
+      throws Exception {
+    Preconditions.checkState(_segmentMetadata.getVersion() == SegmentVersion.v3);
+    ColumnMetadata existingColMetadata = _segmentMetadata.getColumnMetadataFor(column);
+    Preconditions.checkState(!existingColMetadata.hasDictionary(),
+        "Cannot rewrite dictionary enabled forward index. Dictionary already exists for column:" + column);
+    boolean isSingleValue = existingColMetadata.isSingleValue();
+
+    File indexDir = _segmentMetadata.getIndexDir();
+    String segmentName = _segmentMetadata.getName();
+    File inProgress = new File(indexDir, column + ".dict.inprogress");
+    File dictionaryFile = new File(indexDir, column + V1Constants.Dict.FILE_EXTENSION);
+    String fwdIndexFileExtension;
+    if (isSingleValue) {
+      fwdIndexFileExtension =
+          existingColMetadata.isSorted() ? V1Constants.Indexes.SORTED_SV_FORWARD_INDEX_FILE_EXTENSION

Review Comment:
   I was able to write a unit test (please see ForwardIndexHandlerTest) where I created a sorted column without dictionary. 
   I understand the comment. But I believe it still makes sense to have this support just incase someone decides to change the behavior? We're just covering all cases by having this support. Thoughts?



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


[GitHub] [pinot] vvivekiyer commented on a diff in pull request #9678: Support creating dictionary at runtime for an existing column

Posted by GitBox <gi...@apache.org>.
vvivekiyer commented on code in PR #9678:
URL: https://github.com/apache/pinot/pull/9678#discussion_r1011007394


##########
pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/loader/ForwardIndexHandler.java:
##########
@@ -271,118 +306,293 @@ private void rewriteRawSVForwardIndex(String column, ColumnMetadata existingColM
               .forForwardIndex(newCompressionType, _indexLoadingConfig.getColumnProperties());
 
       try (ForwardIndexCreator creator = indexCreatorProvider.newForwardIndexCreator(context)) {
-        // If creator stored type and the reader stored type do not match, throw an exception.
         if (!reader.getStoredType().equals(creator.getValueType())) {
+          // Creator stored type should match reader stored type for raw columns. We do not support changing datatypes.
           String failureMsg =
               "Unsupported operation to change datatype for column=" + column + " from " + reader.getStoredType()
                   .toString() + " to " + creator.getValueType().toString();
           throw new UnsupportedOperationException(failureMsg);
         }
 
         int numDocs = existingColMetadata.getTotalDocs();
-        forwardIndexWriterHelper(column, reader, creator, numDocs);
+        forwardIndexWriterHelper(column, existingColMetadata, reader, creator, numDocs, null);
       }
     }
   }
 
-  private void forwardIndexWriterHelper(String column, ForwardIndexReader reader, ForwardIndexCreator creator,
-      int numDocs) {
-    // If creator stored type should match reader stored type. We do not support changing datatypes.
-    if (!reader.getStoredType().equals(creator.getValueType())) {
-      String failureMsg =
-          "Unsupported operation to change datatype for column=" + column + " from " + reader.getStoredType().toString()
-              + " to " + creator.getValueType().toString();
-      throw new UnsupportedOperationException(failureMsg);
-    }
-
+  private void forwardIndexWriterHelper(String column, ColumnMetadata existingColumnMetadata, ForwardIndexReader reader,
+      ForwardIndexCreator creator, int numDocs, @Nullable SegmentDictionaryCreator dictionaryCreator) {
     ForwardIndexReaderContext readerContext = reader.createContext();
     boolean isSVColumn = reader.isSingleValue();
 
-    switch (reader.getStoredType()) {
-      // JSON fields are either stored as string or bytes. No special handling is needed because we make this
-      // decision based on the storedType of the reader.
-      case INT: {
-        for (int i = 0; i < numDocs; i++) {
-          if (isSVColumn) {
-            int val = reader.getInt(i, readerContext);
-            creator.putInt(val);
-          } else {
-            int[] ints = reader.getIntMV(i, readerContext);
-            creator.putIntMV(ints);
-          }
+    if (dictionaryCreator != null) {
+      int maxNumValuesPerEntry = existingColumnMetadata.getMaxNumberOfMultiValues();
+      PinotSegmentColumnReader columnReader = new PinotSegmentColumnReader(reader, null, null, maxNumValuesPerEntry);
+
+      for (int i = 0; i < numDocs; i++) {
+        Object obj = columnReader.getValue(i);
+
+        if (isSVColumn) {
+          int dictId = dictionaryCreator.indexOfSV(obj);
+          creator.putDictId(dictId);
+        } else {
+          int[] dictIds = dictionaryCreator.indexOfMV(obj);
+          creator.putDictIdMV(dictIds);
         }
-        break;
       }
-      case LONG: {
-        for (int i = 0; i < numDocs; i++) {
-          if (isSVColumn) {
-            long val = reader.getLong(i, readerContext);
-            creator.putLong(val);
-          } else {
-            long[] longs = reader.getLongMV(i, readerContext);
-            creator.putLongMV(longs);
+    } else {
+      switch (reader.getStoredType()) {
+        // JSON fields are either stored as string or bytes. No special handling is needed because we make this
+        // decision based on the storedType of the reader.
+        case INT: {
+          for (int i = 0; i < numDocs; i++) {
+            if (isSVColumn) {
+              int val = reader.getInt(i, readerContext);
+              creator.putInt(val);
+            } else {
+              int[] ints = reader.getIntMV(i, readerContext);
+              creator.putIntMV(ints);
+            }
           }
+          break;
         }
-        break;
-      }
-      case FLOAT: {
-        for (int i = 0; i < numDocs; i++) {
-          if (isSVColumn) {
-            float val = reader.getFloat(i, readerContext);
-            creator.putFloat(val);
-          } else {
-            float[] floats = reader.getFloatMV(i, readerContext);
-            creator.putFloatMV(floats);
+        case LONG: {
+          for (int i = 0; i < numDocs; i++) {
+            if (isSVColumn) {
+              long val = reader.getLong(i, readerContext);
+              creator.putLong(val);
+            } else {
+              long[] longs = reader.getLongMV(i, readerContext);
+              creator.putLongMV(longs);
+            }
           }
+          break;
         }
-        break;
-      }
-      case DOUBLE: {
-        for (int i = 0; i < numDocs; i++) {
-          if (isSVColumn) {
-            double val = reader.getDouble(i, readerContext);
-            creator.putDouble(val);
-          } else {
-            double[] doubles = reader.getDoubleMV(i, readerContext);
-            creator.putDoubleMV(doubles);
+        case FLOAT: {
+          for (int i = 0; i < numDocs; i++) {
+            if (isSVColumn) {
+              float val = reader.getFloat(i, readerContext);
+              creator.putFloat(val);
+            } else {
+              float[] floats = reader.getFloatMV(i, readerContext);
+              creator.putFloatMV(floats);
+            }
           }
+          break;
         }
-        break;
-      }
-      case STRING: {
-        for (int i = 0; i < numDocs; i++) {
-          if (isSVColumn) {
-            String val = reader.getString(i, readerContext);
-            creator.putString(val);
-          } else {
-            String[] strings = reader.getStringMV(i, readerContext);
-            creator.putStringMV(strings);
+        case DOUBLE: {
+          for (int i = 0; i < numDocs; i++) {
+            if (isSVColumn) {
+              double val = reader.getDouble(i, readerContext);
+              creator.putDouble(val);
+            } else {
+              double[] doubles = reader.getDoubleMV(i, readerContext);
+              creator.putDoubleMV(doubles);
+            }
           }
+          break;
         }
-        break;
-      }
-      case BYTES: {
-        for (int i = 0; i < numDocs; i++) {
-          if (isSVColumn) {
-            byte[] val = reader.getBytes(i, readerContext);
-            creator.putBytes(val);
-          } else {
-            byte[][] bytesArray = reader.getBytesMV(i, readerContext);
-            creator.putBytesMV(bytesArray);
+        case STRING: {
+          for (int i = 0; i < numDocs; i++) {
+            if (isSVColumn) {
+              String val = reader.getString(i, readerContext);
+              creator.putString(val);
+            } else {
+              String[] strings = reader.getStringMV(i, readerContext);
+              creator.putStringMV(strings);
+            }
           }
+          break;
         }
-        break;
-      }
-      case BIG_DECIMAL: {
-        for (int i = 0; i < numDocs; i++) {
+        case BYTES: {
+          for (int i = 0; i < numDocs; i++) {
+            if (isSVColumn) {
+              byte[] val = reader.getBytes(i, readerContext);
+              creator.putBytes(val);
+            } else {
+              byte[][] bytesArray = reader.getBytesMV(i, readerContext);
+              creator.putBytesMV(bytesArray);
+            }
+          }
+          break;
+        }
+        case BIG_DECIMAL: {
           Preconditions.checkState(isSVColumn, "BigDecimal is not supported for MV columns");
-          BigDecimal val = reader.getBigDecimal(i, readerContext);
-          creator.putBigDecimal(val);
+          for (int i = 0; i < numDocs; i++) {
+            BigDecimal val = reader.getBigDecimal(i, readerContext);
+            creator.putBigDecimal(val);
+          }
+          break;
         }
-        break;
+        default:
+          throw new IllegalStateException();
+      }
+    }
+  }
+
+  private void enableDictionary(String column, SegmentDirectory.Writer segmentWriter,
+      IndexCreatorProvider indexCreatorProvider)
+      throws Exception {
+    Preconditions.checkState(_segmentMetadata.getVersion() == SegmentVersion.v3);
+    ColumnMetadata existingColMetadata = _segmentMetadata.getColumnMetadataFor(column);
+    Preconditions.checkState(!existingColMetadata.hasDictionary(), "Existing column already has dictionary.");
+    boolean isSingleValue = existingColMetadata.isSingleValue();
+
+    File indexDir = _segmentMetadata.getIndexDir();
+    String segmentName = _segmentMetadata.getName();
+    File inProgress = new File(indexDir, column + ".dict.inprogress");
+    File dictionaryFile = new File(indexDir, column + V1Constants.Dict.FILE_EXTENSION);
+    String fwdIndexFileExtension;
+    if (isSingleValue) {
+      if (existingColMetadata.isSorted()) {
+        fwdIndexFileExtension = V1Constants.Indexes.SORTED_SV_FORWARD_INDEX_FILE_EXTENSION;
+      } else {
+        fwdIndexFileExtension = V1Constants.Indexes.UNSORTED_SV_FORWARD_INDEX_FILE_EXTENSION;
+      }
+    } else {
+      fwdIndexFileExtension = V1Constants.Indexes.UNSORTED_MV_FORWARD_INDEX_FILE_EXTENSION;
+    }
+    File fwdIndexFile = new File(indexDir, column + fwdIndexFileExtension);
+
+    if (!inProgress.exists()) {
+      // Marker file does not exist, which means last run ended normally.
+      // Create a marker file.
+      FileUtils.touch(inProgress);
+    } else {
+      // Marker file exists, which means last run was interrupted.
+      // Remove forward index and dictionary files if they exist.
+      FileUtils.deleteQuietly(fwdIndexFile);
+      FileUtils.deleteQuietly(dictionaryFile);
+    }
+
+    LOGGER.info("Creating a new dictionary for segment={} and column={}", segmentName, column);
+    SegmentDictionaryCreator dictionaryCreator = buildDictionary(column, existingColMetadata, segmentWriter);
+    LoaderUtils.writeIndexToV3Format(segmentWriter, column, dictionaryFile, ColumnIndexType.DICTIONARY);
+
+    LOGGER.info("Built dictionary. Rewriting dictionary enabled forward index for segment={} and column={}",
+        segmentName, column);
+    writeDictEnabledForwardIndex(column, existingColMetadata, segmentWriter, indexDir, indexCreatorProvider,
+        dictionaryCreator);
+    // We used the existing forward index to generate a new forward index. The existing forward index will be in V3
+    // format and the new forward index will be in V1 format. Remove the existing forward index as it is not needed
+    // anymore. Note that removeIndex() will only mark an index for removal and remove the in-memory state. The
+    // actual cleanup from columns.psf file will happen when singleFileIndexDirectory.cleanupRemovedIndices() is
+    // called during segmentWriter.close().
+    segmentWriter.removeIndex(column, ColumnIndexType.FORWARD_INDEX);
+    LoaderUtils.writeIndexToV3Format(segmentWriter, column, fwdIndexFile, ColumnIndexType.FORWARD_INDEX);
+
+    LOGGER.info("Created forwardIndex. Updating metadata properties for segment={} and column={}", segmentName, column);
+    updateMetadataProperties(indexDir, column, dictionaryCreator);
+
+    // We remove indexes that have to be rewritten when a dictEnabled is toggled. Note that the respective index
+    // handler will take care of recreating the index.
+    removeDictRelatedIndexes(column, segmentWriter);
+
+    // Delete the marker file.
+    FileUtils.deleteQuietly(inProgress);
+
+    LOGGER.info("Created dictionary based forward index for segment: {}, column: {}", segmentName, column);
+  }
+
+  private SegmentDictionaryCreator buildDictionary(String column, ColumnMetadata existingColMetadata,
+      SegmentDirectory.Writer segmentWriter)
+      throws Exception {
+    int numDocs = existingColMetadata.getTotalDocs();
+    // SegmentPartitionConfig is not relevant for rewrites.
+    Preconditions.checkState(_indexLoadingConfig.getTableConfig() != null);

Review Comment:
   Done. This can only be true in tests.



##########
pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/loader/ForwardIndexHandler.java:
##########
@@ -271,118 +306,293 @@ private void rewriteRawSVForwardIndex(String column, ColumnMetadata existingColM
               .forForwardIndex(newCompressionType, _indexLoadingConfig.getColumnProperties());
 
       try (ForwardIndexCreator creator = indexCreatorProvider.newForwardIndexCreator(context)) {
-        // If creator stored type and the reader stored type do not match, throw an exception.
         if (!reader.getStoredType().equals(creator.getValueType())) {
+          // Creator stored type should match reader stored type for raw columns. We do not support changing datatypes.
           String failureMsg =
               "Unsupported operation to change datatype for column=" + column + " from " + reader.getStoredType()
                   .toString() + " to " + creator.getValueType().toString();
           throw new UnsupportedOperationException(failureMsg);
         }
 
         int numDocs = existingColMetadata.getTotalDocs();
-        forwardIndexWriterHelper(column, reader, creator, numDocs);
+        forwardIndexWriterHelper(column, existingColMetadata, reader, creator, numDocs, null);
       }
     }
   }
 
-  private void forwardIndexWriterHelper(String column, ForwardIndexReader reader, ForwardIndexCreator creator,
-      int numDocs) {
-    // If creator stored type should match reader stored type. We do not support changing datatypes.
-    if (!reader.getStoredType().equals(creator.getValueType())) {
-      String failureMsg =
-          "Unsupported operation to change datatype for column=" + column + " from " + reader.getStoredType().toString()
-              + " to " + creator.getValueType().toString();
-      throw new UnsupportedOperationException(failureMsg);
-    }
-
+  private void forwardIndexWriterHelper(String column, ColumnMetadata existingColumnMetadata, ForwardIndexReader reader,
+      ForwardIndexCreator creator, int numDocs, @Nullable SegmentDictionaryCreator dictionaryCreator) {
     ForwardIndexReaderContext readerContext = reader.createContext();
     boolean isSVColumn = reader.isSingleValue();
 
-    switch (reader.getStoredType()) {
-      // JSON fields are either stored as string or bytes. No special handling is needed because we make this
-      // decision based on the storedType of the reader.
-      case INT: {
-        for (int i = 0; i < numDocs; i++) {
-          if (isSVColumn) {
-            int val = reader.getInt(i, readerContext);
-            creator.putInt(val);
-          } else {
-            int[] ints = reader.getIntMV(i, readerContext);
-            creator.putIntMV(ints);
-          }
+    if (dictionaryCreator != null) {
+      int maxNumValuesPerEntry = existingColumnMetadata.getMaxNumberOfMultiValues();
+      PinotSegmentColumnReader columnReader = new PinotSegmentColumnReader(reader, null, null, maxNumValuesPerEntry);
+
+      for (int i = 0; i < numDocs; i++) {
+        Object obj = columnReader.getValue(i);
+
+        if (isSVColumn) {
+          int dictId = dictionaryCreator.indexOfSV(obj);
+          creator.putDictId(dictId);
+        } else {
+          int[] dictIds = dictionaryCreator.indexOfMV(obj);
+          creator.putDictIdMV(dictIds);
         }
-        break;
       }
-      case LONG: {
-        for (int i = 0; i < numDocs; i++) {
-          if (isSVColumn) {
-            long val = reader.getLong(i, readerContext);
-            creator.putLong(val);
-          } else {
-            long[] longs = reader.getLongMV(i, readerContext);
-            creator.putLongMV(longs);
+    } else {
+      switch (reader.getStoredType()) {
+        // JSON fields are either stored as string or bytes. No special handling is needed because we make this
+        // decision based on the storedType of the reader.
+        case INT: {
+          for (int i = 0; i < numDocs; i++) {
+            if (isSVColumn) {
+              int val = reader.getInt(i, readerContext);
+              creator.putInt(val);
+            } else {
+              int[] ints = reader.getIntMV(i, readerContext);
+              creator.putIntMV(ints);
+            }
           }
+          break;
         }
-        break;
-      }
-      case FLOAT: {
-        for (int i = 0; i < numDocs; i++) {
-          if (isSVColumn) {
-            float val = reader.getFloat(i, readerContext);
-            creator.putFloat(val);
-          } else {
-            float[] floats = reader.getFloatMV(i, readerContext);
-            creator.putFloatMV(floats);
+        case LONG: {
+          for (int i = 0; i < numDocs; i++) {
+            if (isSVColumn) {
+              long val = reader.getLong(i, readerContext);
+              creator.putLong(val);
+            } else {
+              long[] longs = reader.getLongMV(i, readerContext);
+              creator.putLongMV(longs);
+            }
           }
+          break;
         }
-        break;
-      }
-      case DOUBLE: {
-        for (int i = 0; i < numDocs; i++) {
-          if (isSVColumn) {
-            double val = reader.getDouble(i, readerContext);
-            creator.putDouble(val);
-          } else {
-            double[] doubles = reader.getDoubleMV(i, readerContext);
-            creator.putDoubleMV(doubles);
+        case FLOAT: {
+          for (int i = 0; i < numDocs; i++) {
+            if (isSVColumn) {
+              float val = reader.getFloat(i, readerContext);
+              creator.putFloat(val);
+            } else {
+              float[] floats = reader.getFloatMV(i, readerContext);
+              creator.putFloatMV(floats);
+            }
           }
+          break;
         }
-        break;
-      }
-      case STRING: {
-        for (int i = 0; i < numDocs; i++) {
-          if (isSVColumn) {
-            String val = reader.getString(i, readerContext);
-            creator.putString(val);
-          } else {
-            String[] strings = reader.getStringMV(i, readerContext);
-            creator.putStringMV(strings);
+        case DOUBLE: {
+          for (int i = 0; i < numDocs; i++) {
+            if (isSVColumn) {
+              double val = reader.getDouble(i, readerContext);
+              creator.putDouble(val);
+            } else {
+              double[] doubles = reader.getDoubleMV(i, readerContext);
+              creator.putDoubleMV(doubles);
+            }
           }
+          break;
         }
-        break;
-      }
-      case BYTES: {
-        for (int i = 0; i < numDocs; i++) {
-          if (isSVColumn) {
-            byte[] val = reader.getBytes(i, readerContext);
-            creator.putBytes(val);
-          } else {
-            byte[][] bytesArray = reader.getBytesMV(i, readerContext);
-            creator.putBytesMV(bytesArray);
+        case STRING: {
+          for (int i = 0; i < numDocs; i++) {
+            if (isSVColumn) {
+              String val = reader.getString(i, readerContext);
+              creator.putString(val);
+            } else {
+              String[] strings = reader.getStringMV(i, readerContext);
+              creator.putStringMV(strings);
+            }
           }
+          break;
         }
-        break;
-      }
-      case BIG_DECIMAL: {
-        for (int i = 0; i < numDocs; i++) {
+        case BYTES: {
+          for (int i = 0; i < numDocs; i++) {
+            if (isSVColumn) {
+              byte[] val = reader.getBytes(i, readerContext);
+              creator.putBytes(val);
+            } else {
+              byte[][] bytesArray = reader.getBytesMV(i, readerContext);
+              creator.putBytesMV(bytesArray);
+            }
+          }
+          break;
+        }
+        case BIG_DECIMAL: {
           Preconditions.checkState(isSVColumn, "BigDecimal is not supported for MV columns");
-          BigDecimal val = reader.getBigDecimal(i, readerContext);
-          creator.putBigDecimal(val);
+          for (int i = 0; i < numDocs; i++) {
+            BigDecimal val = reader.getBigDecimal(i, readerContext);
+            creator.putBigDecimal(val);
+          }
+          break;
         }
-        break;
+        default:
+          throw new IllegalStateException();
+      }
+    }
+  }
+
+  private void enableDictionary(String column, SegmentDirectory.Writer segmentWriter,
+      IndexCreatorProvider indexCreatorProvider)
+      throws Exception {
+    Preconditions.checkState(_segmentMetadata.getVersion() == SegmentVersion.v3);
+    ColumnMetadata existingColMetadata = _segmentMetadata.getColumnMetadataFor(column);
+    Preconditions.checkState(!existingColMetadata.hasDictionary(), "Existing column already has dictionary.");
+    boolean isSingleValue = existingColMetadata.isSingleValue();
+
+    File indexDir = _segmentMetadata.getIndexDir();
+    String segmentName = _segmentMetadata.getName();
+    File inProgress = new File(indexDir, column + ".dict.inprogress");
+    File dictionaryFile = new File(indexDir, column + V1Constants.Dict.FILE_EXTENSION);
+    String fwdIndexFileExtension;
+    if (isSingleValue) {
+      if (existingColMetadata.isSorted()) {
+        fwdIndexFileExtension = V1Constants.Indexes.SORTED_SV_FORWARD_INDEX_FILE_EXTENSION;
+      } else {
+        fwdIndexFileExtension = V1Constants.Indexes.UNSORTED_SV_FORWARD_INDEX_FILE_EXTENSION;
+      }
+    } else {
+      fwdIndexFileExtension = V1Constants.Indexes.UNSORTED_MV_FORWARD_INDEX_FILE_EXTENSION;
+    }
+    File fwdIndexFile = new File(indexDir, column + fwdIndexFileExtension);
+
+    if (!inProgress.exists()) {
+      // Marker file does not exist, which means last run ended normally.
+      // Create a marker file.
+      FileUtils.touch(inProgress);
+    } else {
+      // Marker file exists, which means last run was interrupted.
+      // Remove forward index and dictionary files if they exist.
+      FileUtils.deleteQuietly(fwdIndexFile);
+      FileUtils.deleteQuietly(dictionaryFile);
+    }
+
+    LOGGER.info("Creating a new dictionary for segment={} and column={}", segmentName, column);
+    SegmentDictionaryCreator dictionaryCreator = buildDictionary(column, existingColMetadata, segmentWriter);
+    LoaderUtils.writeIndexToV3Format(segmentWriter, column, dictionaryFile, ColumnIndexType.DICTIONARY);
+
+    LOGGER.info("Built dictionary. Rewriting dictionary enabled forward index for segment={} and column={}",
+        segmentName, column);
+    writeDictEnabledForwardIndex(column, existingColMetadata, segmentWriter, indexDir, indexCreatorProvider,
+        dictionaryCreator);
+    // We used the existing forward index to generate a new forward index. The existing forward index will be in V3
+    // format and the new forward index will be in V1 format. Remove the existing forward index as it is not needed
+    // anymore. Note that removeIndex() will only mark an index for removal and remove the in-memory state. The
+    // actual cleanup from columns.psf file will happen when singleFileIndexDirectory.cleanupRemovedIndices() is
+    // called during segmentWriter.close().
+    segmentWriter.removeIndex(column, ColumnIndexType.FORWARD_INDEX);
+    LoaderUtils.writeIndexToV3Format(segmentWriter, column, fwdIndexFile, ColumnIndexType.FORWARD_INDEX);
+
+    LOGGER.info("Created forwardIndex. Updating metadata properties for segment={} and column={}", segmentName, column);
+    updateMetadataProperties(indexDir, column, dictionaryCreator);
+
+    // We remove indexes that have to be rewritten when a dictEnabled is toggled. Note that the respective index
+    // handler will take care of recreating the index.
+    removeDictRelatedIndexes(column, segmentWriter);
+
+    // Delete the marker file.
+    FileUtils.deleteQuietly(inProgress);
+
+    LOGGER.info("Created dictionary based forward index for segment: {}, column: {}", segmentName, column);
+  }
+
+  private SegmentDictionaryCreator buildDictionary(String column, ColumnMetadata existingColMetadata,
+      SegmentDirectory.Writer segmentWriter)
+      throws Exception {
+    int numDocs = existingColMetadata.getTotalDocs();
+    // SegmentPartitionConfig is not relevant for rewrites.
+    Preconditions.checkState(_indexLoadingConfig.getTableConfig() != null);
+    StatsCollectorConfig statsCollectorConfig =
+        new StatsCollectorConfig(_indexLoadingConfig.getTableConfig(), _schema, null);
+    AbstractColumnStatisticsCollector statsCollector;
+
+    try (ForwardIndexReader reader = LoaderUtils.getForwardIndexReader(segmentWriter, existingColMetadata)) {
+      boolean isSVColumn = reader.isSingleValue();
+
+      switch (reader.getStoredType()) {
+        case INT:
+          statsCollector = new IntColumnPreIndexStatsCollector(column, statsCollectorConfig);
+          break;
+        case LONG:
+          statsCollector = new LongColumnPreIndexStatsCollector(column, statsCollectorConfig);
+          break;
+        case FLOAT:
+          statsCollector = new FloatColumnPreIndexStatsCollector(column, statsCollectorConfig);
+          break;
+        case DOUBLE:
+          statsCollector = new DoubleColumnPreIndexStatsCollector(column, statsCollectorConfig);
+          break;
+        case STRING:
+          statsCollector = new StringColumnPreIndexStatsCollector(column, statsCollectorConfig);
+          break;
+        case BYTES:
+          statsCollector = new BytesColumnPredIndexStatsCollector(column, statsCollectorConfig);
+          break;
+        case BIG_DECIMAL:
+          Preconditions.checkState(isSVColumn, "BigDecimal is not supported for MV columns");
+          statsCollector = new BigDecimalColumnPreIndexStatsCollector(column, statsCollectorConfig);
+          break;
+        default:
+          throw new IllegalStateException();
+      }
+
+      Preconditions.checkState(statsCollector != null);
+      // Note: Special Null handling is not necessary here. This is because, the existing default null value in the
+      // raw forwardIndex will be retained as such while created the dictionary and dict-based forward index. Also,
+      // null value vectors maintain a bitmap of docIds. No handling is necessary there.
+      PinotSegmentColumnReader columnReader =
+          new PinotSegmentColumnReader(reader, null, null, existingColMetadata.getMaxNumberOfMultiValues());
+      for (int i = 0; i < numDocs; i++) {
+        Object obj = columnReader.getValue(i);
+        statsCollector.collect(obj);
+      }
+      statsCollector.seal();
+
+      boolean useVarLength = _indexLoadingConfig.getVarLengthDictionaryColumns().contains(column)

Review Comment:
   Done



##########
pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/loader/ForwardIndexHandler.java:
##########
@@ -271,118 +306,293 @@ private void rewriteRawSVForwardIndex(String column, ColumnMetadata existingColM
               .forForwardIndex(newCompressionType, _indexLoadingConfig.getColumnProperties());
 
       try (ForwardIndexCreator creator = indexCreatorProvider.newForwardIndexCreator(context)) {
-        // If creator stored type and the reader stored type do not match, throw an exception.
         if (!reader.getStoredType().equals(creator.getValueType())) {
+          // Creator stored type should match reader stored type for raw columns. We do not support changing datatypes.
           String failureMsg =
               "Unsupported operation to change datatype for column=" + column + " from " + reader.getStoredType()
                   .toString() + " to " + creator.getValueType().toString();
           throw new UnsupportedOperationException(failureMsg);
         }
 
         int numDocs = existingColMetadata.getTotalDocs();
-        forwardIndexWriterHelper(column, reader, creator, numDocs);
+        forwardIndexWriterHelper(column, existingColMetadata, reader, creator, numDocs, null);
       }
     }
   }
 
-  private void forwardIndexWriterHelper(String column, ForwardIndexReader reader, ForwardIndexCreator creator,
-      int numDocs) {
-    // If creator stored type should match reader stored type. We do not support changing datatypes.
-    if (!reader.getStoredType().equals(creator.getValueType())) {
-      String failureMsg =
-          "Unsupported operation to change datatype for column=" + column + " from " + reader.getStoredType().toString()
-              + " to " + creator.getValueType().toString();
-      throw new UnsupportedOperationException(failureMsg);
-    }
-
+  private void forwardIndexWriterHelper(String column, ColumnMetadata existingColumnMetadata, ForwardIndexReader reader,
+      ForwardIndexCreator creator, int numDocs, @Nullable SegmentDictionaryCreator dictionaryCreator) {
     ForwardIndexReaderContext readerContext = reader.createContext();
     boolean isSVColumn = reader.isSingleValue();
 
-    switch (reader.getStoredType()) {
-      // JSON fields are either stored as string or bytes. No special handling is needed because we make this
-      // decision based on the storedType of the reader.
-      case INT: {
-        for (int i = 0; i < numDocs; i++) {
-          if (isSVColumn) {
-            int val = reader.getInt(i, readerContext);
-            creator.putInt(val);
-          } else {
-            int[] ints = reader.getIntMV(i, readerContext);
-            creator.putIntMV(ints);
-          }
+    if (dictionaryCreator != null) {
+      int maxNumValuesPerEntry = existingColumnMetadata.getMaxNumberOfMultiValues();
+      PinotSegmentColumnReader columnReader = new PinotSegmentColumnReader(reader, null, null, maxNumValuesPerEntry);
+
+      for (int i = 0; i < numDocs; i++) {
+        Object obj = columnReader.getValue(i);
+
+        if (isSVColumn) {
+          int dictId = dictionaryCreator.indexOfSV(obj);
+          creator.putDictId(dictId);
+        } else {
+          int[] dictIds = dictionaryCreator.indexOfMV(obj);
+          creator.putDictIdMV(dictIds);
         }
-        break;
       }
-      case LONG: {
-        for (int i = 0; i < numDocs; i++) {
-          if (isSVColumn) {
-            long val = reader.getLong(i, readerContext);
-            creator.putLong(val);
-          } else {
-            long[] longs = reader.getLongMV(i, readerContext);
-            creator.putLongMV(longs);
+    } else {
+      switch (reader.getStoredType()) {
+        // JSON fields are either stored as string or bytes. No special handling is needed because we make this
+        // decision based on the storedType of the reader.
+        case INT: {
+          for (int i = 0; i < numDocs; i++) {
+            if (isSVColumn) {
+              int val = reader.getInt(i, readerContext);
+              creator.putInt(val);
+            } else {
+              int[] ints = reader.getIntMV(i, readerContext);
+              creator.putIntMV(ints);
+            }
           }
+          break;
         }
-        break;
-      }
-      case FLOAT: {
-        for (int i = 0; i < numDocs; i++) {
-          if (isSVColumn) {
-            float val = reader.getFloat(i, readerContext);
-            creator.putFloat(val);
-          } else {
-            float[] floats = reader.getFloatMV(i, readerContext);
-            creator.putFloatMV(floats);
+        case LONG: {
+          for (int i = 0; i < numDocs; i++) {
+            if (isSVColumn) {
+              long val = reader.getLong(i, readerContext);
+              creator.putLong(val);
+            } else {
+              long[] longs = reader.getLongMV(i, readerContext);
+              creator.putLongMV(longs);
+            }
           }
+          break;
         }
-        break;
-      }
-      case DOUBLE: {
-        for (int i = 0; i < numDocs; i++) {
-          if (isSVColumn) {
-            double val = reader.getDouble(i, readerContext);
-            creator.putDouble(val);
-          } else {
-            double[] doubles = reader.getDoubleMV(i, readerContext);
-            creator.putDoubleMV(doubles);
+        case FLOAT: {
+          for (int i = 0; i < numDocs; i++) {
+            if (isSVColumn) {
+              float val = reader.getFloat(i, readerContext);
+              creator.putFloat(val);
+            } else {
+              float[] floats = reader.getFloatMV(i, readerContext);
+              creator.putFloatMV(floats);
+            }
           }
+          break;
         }
-        break;
-      }
-      case STRING: {
-        for (int i = 0; i < numDocs; i++) {
-          if (isSVColumn) {
-            String val = reader.getString(i, readerContext);
-            creator.putString(val);
-          } else {
-            String[] strings = reader.getStringMV(i, readerContext);
-            creator.putStringMV(strings);
+        case DOUBLE: {
+          for (int i = 0; i < numDocs; i++) {
+            if (isSVColumn) {
+              double val = reader.getDouble(i, readerContext);
+              creator.putDouble(val);
+            } else {
+              double[] doubles = reader.getDoubleMV(i, readerContext);
+              creator.putDoubleMV(doubles);
+            }
           }
+          break;
         }
-        break;
-      }
-      case BYTES: {
-        for (int i = 0; i < numDocs; i++) {
-          if (isSVColumn) {
-            byte[] val = reader.getBytes(i, readerContext);
-            creator.putBytes(val);
-          } else {
-            byte[][] bytesArray = reader.getBytesMV(i, readerContext);
-            creator.putBytesMV(bytesArray);
+        case STRING: {
+          for (int i = 0; i < numDocs; i++) {
+            if (isSVColumn) {
+              String val = reader.getString(i, readerContext);
+              creator.putString(val);
+            } else {
+              String[] strings = reader.getStringMV(i, readerContext);
+              creator.putStringMV(strings);
+            }
           }
+          break;
         }
-        break;
-      }
-      case BIG_DECIMAL: {
-        for (int i = 0; i < numDocs; i++) {
+        case BYTES: {
+          for (int i = 0; i < numDocs; i++) {
+            if (isSVColumn) {
+              byte[] val = reader.getBytes(i, readerContext);
+              creator.putBytes(val);
+            } else {
+              byte[][] bytesArray = reader.getBytesMV(i, readerContext);
+              creator.putBytesMV(bytesArray);
+            }
+          }
+          break;
+        }
+        case BIG_DECIMAL: {
           Preconditions.checkState(isSVColumn, "BigDecimal is not supported for MV columns");
-          BigDecimal val = reader.getBigDecimal(i, readerContext);
-          creator.putBigDecimal(val);
+          for (int i = 0; i < numDocs; i++) {
+            BigDecimal val = reader.getBigDecimal(i, readerContext);
+            creator.putBigDecimal(val);
+          }
+          break;
         }
-        break;
+        default:
+          throw new IllegalStateException();
+      }
+    }
+  }
+
+  private void enableDictionary(String column, SegmentDirectory.Writer segmentWriter,
+      IndexCreatorProvider indexCreatorProvider)
+      throws Exception {
+    Preconditions.checkState(_segmentMetadata.getVersion() == SegmentVersion.v3);
+    ColumnMetadata existingColMetadata = _segmentMetadata.getColumnMetadataFor(column);
+    Preconditions.checkState(!existingColMetadata.hasDictionary(), "Existing column already has dictionary.");
+    boolean isSingleValue = existingColMetadata.isSingleValue();
+
+    File indexDir = _segmentMetadata.getIndexDir();
+    String segmentName = _segmentMetadata.getName();
+    File inProgress = new File(indexDir, column + ".dict.inprogress");
+    File dictionaryFile = new File(indexDir, column + V1Constants.Dict.FILE_EXTENSION);
+    String fwdIndexFileExtension;
+    if (isSingleValue) {
+      if (existingColMetadata.isSorted()) {
+        fwdIndexFileExtension = V1Constants.Indexes.SORTED_SV_FORWARD_INDEX_FILE_EXTENSION;
+      } else {
+        fwdIndexFileExtension = V1Constants.Indexes.UNSORTED_SV_FORWARD_INDEX_FILE_EXTENSION;
+      }
+    } else {
+      fwdIndexFileExtension = V1Constants.Indexes.UNSORTED_MV_FORWARD_INDEX_FILE_EXTENSION;
+    }
+    File fwdIndexFile = new File(indexDir, column + fwdIndexFileExtension);
+
+    if (!inProgress.exists()) {
+      // Marker file does not exist, which means last run ended normally.
+      // Create a marker file.
+      FileUtils.touch(inProgress);
+    } else {
+      // Marker file exists, which means last run was interrupted.
+      // Remove forward index and dictionary files if they exist.
+      FileUtils.deleteQuietly(fwdIndexFile);
+      FileUtils.deleteQuietly(dictionaryFile);
+    }
+
+    LOGGER.info("Creating a new dictionary for segment={} and column={}", segmentName, column);
+    SegmentDictionaryCreator dictionaryCreator = buildDictionary(column, existingColMetadata, segmentWriter);
+    LoaderUtils.writeIndexToV3Format(segmentWriter, column, dictionaryFile, ColumnIndexType.DICTIONARY);
+
+    LOGGER.info("Built dictionary. Rewriting dictionary enabled forward index for segment={} and column={}",
+        segmentName, column);
+    writeDictEnabledForwardIndex(column, existingColMetadata, segmentWriter, indexDir, indexCreatorProvider,
+        dictionaryCreator);
+    // We used the existing forward index to generate a new forward index. The existing forward index will be in V3
+    // format and the new forward index will be in V1 format. Remove the existing forward index as it is not needed
+    // anymore. Note that removeIndex() will only mark an index for removal and remove the in-memory state. The
+    // actual cleanup from columns.psf file will happen when singleFileIndexDirectory.cleanupRemovedIndices() is
+    // called during segmentWriter.close().
+    segmentWriter.removeIndex(column, ColumnIndexType.FORWARD_INDEX);
+    LoaderUtils.writeIndexToV3Format(segmentWriter, column, fwdIndexFile, ColumnIndexType.FORWARD_INDEX);
+
+    LOGGER.info("Created forwardIndex. Updating metadata properties for segment={} and column={}", segmentName, column);
+    updateMetadataProperties(indexDir, column, dictionaryCreator);
+
+    // We remove indexes that have to be rewritten when a dictEnabled is toggled. Note that the respective index
+    // handler will take care of recreating the index.
+    removeDictRelatedIndexes(column, segmentWriter);
+
+    // Delete the marker file.
+    FileUtils.deleteQuietly(inProgress);
+
+    LOGGER.info("Created dictionary based forward index for segment: {}, column: {}", segmentName, column);
+  }
+
+  private SegmentDictionaryCreator buildDictionary(String column, ColumnMetadata existingColMetadata,
+      SegmentDirectory.Writer segmentWriter)
+      throws Exception {
+    int numDocs = existingColMetadata.getTotalDocs();
+    // SegmentPartitionConfig is not relevant for rewrites.
+    Preconditions.checkState(_indexLoadingConfig.getTableConfig() != null);
+    StatsCollectorConfig statsCollectorConfig =
+        new StatsCollectorConfig(_indexLoadingConfig.getTableConfig(), _schema, null);
+    AbstractColumnStatisticsCollector statsCollector;
+
+    try (ForwardIndexReader reader = LoaderUtils.getForwardIndexReader(segmentWriter, existingColMetadata)) {
+      boolean isSVColumn = reader.isSingleValue();
+
+      switch (reader.getStoredType()) {
+        case INT:
+          statsCollector = new IntColumnPreIndexStatsCollector(column, statsCollectorConfig);
+          break;
+        case LONG:
+          statsCollector = new LongColumnPreIndexStatsCollector(column, statsCollectorConfig);
+          break;
+        case FLOAT:
+          statsCollector = new FloatColumnPreIndexStatsCollector(column, statsCollectorConfig);
+          break;
+        case DOUBLE:
+          statsCollector = new DoubleColumnPreIndexStatsCollector(column, statsCollectorConfig);
+          break;
+        case STRING:
+          statsCollector = new StringColumnPreIndexStatsCollector(column, statsCollectorConfig);
+          break;
+        case BYTES:
+          statsCollector = new BytesColumnPredIndexStatsCollector(column, statsCollectorConfig);
+          break;
+        case BIG_DECIMAL:
+          Preconditions.checkState(isSVColumn, "BigDecimal is not supported for MV columns");
+          statsCollector = new BigDecimalColumnPreIndexStatsCollector(column, statsCollectorConfig);
+          break;
+        default:
+          throw new IllegalStateException();
+      }
+
+      Preconditions.checkState(statsCollector != null);
+      // Note: Special Null handling is not necessary here. This is because, the existing default null value in the
+      // raw forwardIndex will be retained as such while created the dictionary and dict-based forward index. Also,
+      // null value vectors maintain a bitmap of docIds. No handling is necessary there.
+      PinotSegmentColumnReader columnReader =
+          new PinotSegmentColumnReader(reader, null, null, existingColMetadata.getMaxNumberOfMultiValues());
+      for (int i = 0; i < numDocs; i++) {
+        Object obj = columnReader.getValue(i);
+        statsCollector.collect(obj);
+      }
+      statsCollector.seal();
+
+      boolean useVarLength = _indexLoadingConfig.getVarLengthDictionaryColumns().contains(column)
+          || SegmentIndexCreationDriverImpl.shouldUseVarLengthDictionary(reader.getStoredType(), statsCollector);
+      SegmentDictionaryCreator dictionaryCreator =
+          new SegmentDictionaryCreator(existingColMetadata.getFieldSpec(), _segmentMetadata.getIndexDir(),
+              useVarLength);
+
+      dictionaryCreator.build(statsCollector.getUniqueValuesSet());

Review Comment:
   We could have gone this route for the raw forward index scenario. But we didn't because, StatsCollector maintains all the values in-memory. For rewriting a forward index for compression change, this felt like an unnecessary thing to do. 



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


[GitHub] [pinot] vvivekiyer commented on a diff in pull request #9678: Support creating dictionary at runtime for an existing column

Posted by GitBox <gi...@apache.org>.
vvivekiyer commented on code in PR #9678:
URL: https://github.com/apache/pinot/pull/9678#discussion_r1011007524


##########
pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/loader/ForwardIndexHandler.java:
##########
@@ -271,118 +306,293 @@ private void rewriteRawSVForwardIndex(String column, ColumnMetadata existingColM
               .forForwardIndex(newCompressionType, _indexLoadingConfig.getColumnProperties());
 
       try (ForwardIndexCreator creator = indexCreatorProvider.newForwardIndexCreator(context)) {
-        // If creator stored type and the reader stored type do not match, throw an exception.
         if (!reader.getStoredType().equals(creator.getValueType())) {
+          // Creator stored type should match reader stored type for raw columns. We do not support changing datatypes.
           String failureMsg =
               "Unsupported operation to change datatype for column=" + column + " from " + reader.getStoredType()
                   .toString() + " to " + creator.getValueType().toString();
           throw new UnsupportedOperationException(failureMsg);
         }
 
         int numDocs = existingColMetadata.getTotalDocs();
-        forwardIndexWriterHelper(column, reader, creator, numDocs);
+        forwardIndexWriterHelper(column, existingColMetadata, reader, creator, numDocs, null);
       }
     }
   }
 
-  private void forwardIndexWriterHelper(String column, ForwardIndexReader reader, ForwardIndexCreator creator,
-      int numDocs) {
-    // If creator stored type should match reader stored type. We do not support changing datatypes.
-    if (!reader.getStoredType().equals(creator.getValueType())) {
-      String failureMsg =
-          "Unsupported operation to change datatype for column=" + column + " from " + reader.getStoredType().toString()
-              + " to " + creator.getValueType().toString();
-      throw new UnsupportedOperationException(failureMsg);
-    }
-
+  private void forwardIndexWriterHelper(String column, ColumnMetadata existingColumnMetadata, ForwardIndexReader reader,
+      ForwardIndexCreator creator, int numDocs, @Nullable SegmentDictionaryCreator dictionaryCreator) {
     ForwardIndexReaderContext readerContext = reader.createContext();
     boolean isSVColumn = reader.isSingleValue();
 
-    switch (reader.getStoredType()) {
-      // JSON fields are either stored as string or bytes. No special handling is needed because we make this
-      // decision based on the storedType of the reader.
-      case INT: {
-        for (int i = 0; i < numDocs; i++) {
-          if (isSVColumn) {
-            int val = reader.getInt(i, readerContext);
-            creator.putInt(val);
-          } else {
-            int[] ints = reader.getIntMV(i, readerContext);
-            creator.putIntMV(ints);
-          }
+    if (dictionaryCreator != null) {
+      int maxNumValuesPerEntry = existingColumnMetadata.getMaxNumberOfMultiValues();
+      PinotSegmentColumnReader columnReader = new PinotSegmentColumnReader(reader, null, null, maxNumValuesPerEntry);
+
+      for (int i = 0; i < numDocs; i++) {
+        Object obj = columnReader.getValue(i);
+
+        if (isSVColumn) {
+          int dictId = dictionaryCreator.indexOfSV(obj);
+          creator.putDictId(dictId);
+        } else {
+          int[] dictIds = dictionaryCreator.indexOfMV(obj);
+          creator.putDictIdMV(dictIds);
         }
-        break;
       }
-      case LONG: {
-        for (int i = 0; i < numDocs; i++) {
-          if (isSVColumn) {
-            long val = reader.getLong(i, readerContext);
-            creator.putLong(val);
-          } else {
-            long[] longs = reader.getLongMV(i, readerContext);
-            creator.putLongMV(longs);
+    } else {
+      switch (reader.getStoredType()) {
+        // JSON fields are either stored as string or bytes. No special handling is needed because we make this
+        // decision based on the storedType of the reader.
+        case INT: {
+          for (int i = 0; i < numDocs; i++) {
+            if (isSVColumn) {
+              int val = reader.getInt(i, readerContext);
+              creator.putInt(val);
+            } else {
+              int[] ints = reader.getIntMV(i, readerContext);
+              creator.putIntMV(ints);
+            }
           }
+          break;
         }
-        break;
-      }
-      case FLOAT: {
-        for (int i = 0; i < numDocs; i++) {
-          if (isSVColumn) {
-            float val = reader.getFloat(i, readerContext);
-            creator.putFloat(val);
-          } else {
-            float[] floats = reader.getFloatMV(i, readerContext);
-            creator.putFloatMV(floats);
+        case LONG: {
+          for (int i = 0; i < numDocs; i++) {
+            if (isSVColumn) {
+              long val = reader.getLong(i, readerContext);
+              creator.putLong(val);
+            } else {
+              long[] longs = reader.getLongMV(i, readerContext);
+              creator.putLongMV(longs);
+            }
           }
+          break;
         }
-        break;
-      }
-      case DOUBLE: {
-        for (int i = 0; i < numDocs; i++) {
-          if (isSVColumn) {
-            double val = reader.getDouble(i, readerContext);
-            creator.putDouble(val);
-          } else {
-            double[] doubles = reader.getDoubleMV(i, readerContext);
-            creator.putDoubleMV(doubles);
+        case FLOAT: {
+          for (int i = 0; i < numDocs; i++) {
+            if (isSVColumn) {
+              float val = reader.getFloat(i, readerContext);
+              creator.putFloat(val);
+            } else {
+              float[] floats = reader.getFloatMV(i, readerContext);
+              creator.putFloatMV(floats);
+            }
           }
+          break;
         }
-        break;
-      }
-      case STRING: {
-        for (int i = 0; i < numDocs; i++) {
-          if (isSVColumn) {
-            String val = reader.getString(i, readerContext);
-            creator.putString(val);
-          } else {
-            String[] strings = reader.getStringMV(i, readerContext);
-            creator.putStringMV(strings);
+        case DOUBLE: {
+          for (int i = 0; i < numDocs; i++) {
+            if (isSVColumn) {
+              double val = reader.getDouble(i, readerContext);
+              creator.putDouble(val);
+            } else {
+              double[] doubles = reader.getDoubleMV(i, readerContext);
+              creator.putDoubleMV(doubles);
+            }
           }
+          break;
         }
-        break;
-      }
-      case BYTES: {
-        for (int i = 0; i < numDocs; i++) {
-          if (isSVColumn) {
-            byte[] val = reader.getBytes(i, readerContext);
-            creator.putBytes(val);
-          } else {
-            byte[][] bytesArray = reader.getBytesMV(i, readerContext);
-            creator.putBytesMV(bytesArray);
+        case STRING: {
+          for (int i = 0; i < numDocs; i++) {
+            if (isSVColumn) {
+              String val = reader.getString(i, readerContext);
+              creator.putString(val);
+            } else {
+              String[] strings = reader.getStringMV(i, readerContext);
+              creator.putStringMV(strings);
+            }
           }
+          break;
         }
-        break;
-      }
-      case BIG_DECIMAL: {
-        for (int i = 0; i < numDocs; i++) {
+        case BYTES: {
+          for (int i = 0; i < numDocs; i++) {
+            if (isSVColumn) {
+              byte[] val = reader.getBytes(i, readerContext);
+              creator.putBytes(val);
+            } else {
+              byte[][] bytesArray = reader.getBytesMV(i, readerContext);
+              creator.putBytesMV(bytesArray);
+            }
+          }
+          break;
+        }
+        case BIG_DECIMAL: {
           Preconditions.checkState(isSVColumn, "BigDecimal is not supported for MV columns");
-          BigDecimal val = reader.getBigDecimal(i, readerContext);
-          creator.putBigDecimal(val);
+          for (int i = 0; i < numDocs; i++) {
+            BigDecimal val = reader.getBigDecimal(i, readerContext);
+            creator.putBigDecimal(val);
+          }
+          break;
         }
-        break;
+        default:
+          throw new IllegalStateException();
+      }
+    }
+  }
+
+  private void enableDictionary(String column, SegmentDirectory.Writer segmentWriter,
+      IndexCreatorProvider indexCreatorProvider)
+      throws Exception {
+    Preconditions.checkState(_segmentMetadata.getVersion() == SegmentVersion.v3);
+    ColumnMetadata existingColMetadata = _segmentMetadata.getColumnMetadataFor(column);
+    Preconditions.checkState(!existingColMetadata.hasDictionary(), "Existing column already has dictionary.");
+    boolean isSingleValue = existingColMetadata.isSingleValue();
+
+    File indexDir = _segmentMetadata.getIndexDir();
+    String segmentName = _segmentMetadata.getName();
+    File inProgress = new File(indexDir, column + ".dict.inprogress");
+    File dictionaryFile = new File(indexDir, column + V1Constants.Dict.FILE_EXTENSION);
+    String fwdIndexFileExtension;
+    if (isSingleValue) {
+      if (existingColMetadata.isSorted()) {
+        fwdIndexFileExtension = V1Constants.Indexes.SORTED_SV_FORWARD_INDEX_FILE_EXTENSION;
+      } else {
+        fwdIndexFileExtension = V1Constants.Indexes.UNSORTED_SV_FORWARD_INDEX_FILE_EXTENSION;
+      }
+    } else {
+      fwdIndexFileExtension = V1Constants.Indexes.UNSORTED_MV_FORWARD_INDEX_FILE_EXTENSION;
+    }
+    File fwdIndexFile = new File(indexDir, column + fwdIndexFileExtension);
+
+    if (!inProgress.exists()) {
+      // Marker file does not exist, which means last run ended normally.
+      // Create a marker file.
+      FileUtils.touch(inProgress);
+    } else {
+      // Marker file exists, which means last run was interrupted.
+      // Remove forward index and dictionary files if they exist.
+      FileUtils.deleteQuietly(fwdIndexFile);
+      FileUtils.deleteQuietly(dictionaryFile);
+    }
+
+    LOGGER.info("Creating a new dictionary for segment={} and column={}", segmentName, column);
+    SegmentDictionaryCreator dictionaryCreator = buildDictionary(column, existingColMetadata, segmentWriter);
+    LoaderUtils.writeIndexToV3Format(segmentWriter, column, dictionaryFile, ColumnIndexType.DICTIONARY);
+
+    LOGGER.info("Built dictionary. Rewriting dictionary enabled forward index for segment={} and column={}",
+        segmentName, column);
+    writeDictEnabledForwardIndex(column, existingColMetadata, segmentWriter, indexDir, indexCreatorProvider,
+        dictionaryCreator);
+    // We used the existing forward index to generate a new forward index. The existing forward index will be in V3
+    // format and the new forward index will be in V1 format. Remove the existing forward index as it is not needed
+    // anymore. Note that removeIndex() will only mark an index for removal and remove the in-memory state. The
+    // actual cleanup from columns.psf file will happen when singleFileIndexDirectory.cleanupRemovedIndices() is
+    // called during segmentWriter.close().
+    segmentWriter.removeIndex(column, ColumnIndexType.FORWARD_INDEX);
+    LoaderUtils.writeIndexToV3Format(segmentWriter, column, fwdIndexFile, ColumnIndexType.FORWARD_INDEX);
+
+    LOGGER.info("Created forwardIndex. Updating metadata properties for segment={} and column={}", segmentName, column);
+    updateMetadataProperties(indexDir, column, dictionaryCreator);
+
+    // We remove indexes that have to be rewritten when a dictEnabled is toggled. Note that the respective index
+    // handler will take care of recreating the index.
+    removeDictRelatedIndexes(column, segmentWriter);
+
+    // Delete the marker file.
+    FileUtils.deleteQuietly(inProgress);
+
+    LOGGER.info("Created dictionary based forward index for segment: {}, column: {}", segmentName, column);
+  }
+
+  private SegmentDictionaryCreator buildDictionary(String column, ColumnMetadata existingColMetadata,
+      SegmentDirectory.Writer segmentWriter)
+      throws Exception {
+    int numDocs = existingColMetadata.getTotalDocs();
+    // SegmentPartitionConfig is not relevant for rewrites.
+    Preconditions.checkState(_indexLoadingConfig.getTableConfig() != null);
+    StatsCollectorConfig statsCollectorConfig =
+        new StatsCollectorConfig(_indexLoadingConfig.getTableConfig(), _schema, null);
+    AbstractColumnStatisticsCollector statsCollector;
+
+    try (ForwardIndexReader reader = LoaderUtils.getForwardIndexReader(segmentWriter, existingColMetadata)) {
+      boolean isSVColumn = reader.isSingleValue();
+
+      switch (reader.getStoredType()) {
+        case INT:
+          statsCollector = new IntColumnPreIndexStatsCollector(column, statsCollectorConfig);
+          break;
+        case LONG:
+          statsCollector = new LongColumnPreIndexStatsCollector(column, statsCollectorConfig);
+          break;
+        case FLOAT:
+          statsCollector = new FloatColumnPreIndexStatsCollector(column, statsCollectorConfig);
+          break;
+        case DOUBLE:
+          statsCollector = new DoubleColumnPreIndexStatsCollector(column, statsCollectorConfig);
+          break;
+        case STRING:
+          statsCollector = new StringColumnPreIndexStatsCollector(column, statsCollectorConfig);
+          break;
+        case BYTES:
+          statsCollector = new BytesColumnPredIndexStatsCollector(column, statsCollectorConfig);
+          break;
+        case BIG_DECIMAL:
+          Preconditions.checkState(isSVColumn, "BigDecimal is not supported for MV columns");
+          statsCollector = new BigDecimalColumnPreIndexStatsCollector(column, statsCollectorConfig);
+          break;
+        default:
+          throw new IllegalStateException();
+      }
+
+      Preconditions.checkState(statsCollector != null);
+      // Note: Special Null handling is not necessary here. This is because, the existing default null value in the
+      // raw forwardIndex will be retained as such while created the dictionary and dict-based forward index. Also,
+      // null value vectors maintain a bitmap of docIds. No handling is necessary there.
+      PinotSegmentColumnReader columnReader =
+          new PinotSegmentColumnReader(reader, null, null, existingColMetadata.getMaxNumberOfMultiValues());
+      for (int i = 0; i < numDocs; i++) {
+        Object obj = columnReader.getValue(i);
+        statsCollector.collect(obj);
+      }
+      statsCollector.seal();
+
+      boolean useVarLength = _indexLoadingConfig.getVarLengthDictionaryColumns().contains(column)
+          || SegmentIndexCreationDriverImpl.shouldUseVarLengthDictionary(reader.getStoredType(), statsCollector);
+      SegmentDictionaryCreator dictionaryCreator =
+          new SegmentDictionaryCreator(existingColMetadata.getFieldSpec(), _segmentMetadata.getIndexDir(),
+              useVarLength);
+
+      dictionaryCreator.build(statsCollector.getUniqueValuesSet());
+      return dictionaryCreator;
+    }
+  }
+
+  private void writeDictEnabledForwardIndex(String column, ColumnMetadata existingColMetadata,
+      SegmentDirectory.Writer segmentWriter, File indexDir, IndexCreatorProvider indexCreatorProvider,
+      SegmentDictionaryCreator dictionaryCreator)
+      throws Exception {
+    try (ForwardIndexReader reader = LoaderUtils.getForwardIndexReader(segmentWriter, existingColMetadata)) {
+      int lengthOfLongestEntry = reader.getLengthOfLongestEntry();
+      IndexCreationContext.Builder builder =
+          IndexCreationContext.builder().withIndexDir(indexDir).withColumnMetadata(existingColMetadata)
+              .withLengthOfLongestEntry(lengthOfLongestEntry);
+      // existingColMetadata has dictEnable=false. Overwrite the value.
+      builder.withDictionary(true);
+      IndexCreationContext.Forward context =
+          builder.build().forForwardIndex(null, _indexLoadingConfig.getColumnProperties());
+
+      try (ForwardIndexCreator creator = indexCreatorProvider.newForwardIndexCreator(context)) {
+        int numDocs = existingColMetadata.getTotalDocs();
+        forwardIndexWriterHelper(column, existingColMetadata, reader, creator, numDocs, dictionaryCreator);
       }
-      default:
-        throw new IllegalStateException();
     }
   }
+
+  private void removeDictRelatedIndexes(String column, SegmentDirectory.Writer segmentWriter) {
+    // TODO: Move this logic as a static function in each index creator.
+    segmentWriter.removeIndex(column, ColumnIndexType.RANGE_INDEX);
+  }
+
+  private void updateMetadataProperties(File indexDir, String column, SegmentDictionaryCreator dictionaryCreator)

Review Comment:
   Done. Please review and let me know if this would work for your implementation.



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


[GitHub] [pinot] vvivekiyer commented on a diff in pull request #9678: Support creating dictionary at runtime for an existing column

Posted by GitBox <gi...@apache.org>.
vvivekiyer commented on code in PR #9678:
URL: https://github.com/apache/pinot/pull/9678#discussion_r1012179882


##########
pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/loader/ForwardIndexHandler.java:
##########
@@ -271,118 +304,295 @@ private void rewriteRawSVForwardIndex(String column, ColumnMetadata existingColM
               .forForwardIndex(newCompressionType, _indexLoadingConfig.getColumnProperties());
 
       try (ForwardIndexCreator creator = indexCreatorProvider.newForwardIndexCreator(context)) {
-        // If creator stored type and the reader stored type do not match, throw an exception.
         if (!reader.getStoredType().equals(creator.getValueType())) {
+          // Creator stored type should match reader stored type for raw columns. We do not support changing datatypes.
           String failureMsg =
               "Unsupported operation to change datatype for column=" + column + " from " + reader.getStoredType()
                   .toString() + " to " + creator.getValueType().toString();
           throw new UnsupportedOperationException(failureMsg);
         }
 
         int numDocs = existingColMetadata.getTotalDocs();
-        forwardIndexWriterHelper(column, reader, creator, numDocs);
+        forwardIndexWriterHelper(column, existingColMetadata, reader, creator, numDocs, null);
       }
     }
   }
 
-  private void forwardIndexWriterHelper(String column, ForwardIndexReader reader, ForwardIndexCreator creator,
-      int numDocs) {
-    // If creator stored type should match reader stored type. We do not support changing datatypes.
-    if (!reader.getStoredType().equals(creator.getValueType())) {
-      String failureMsg =
-          "Unsupported operation to change datatype for column=" + column + " from " + reader.getStoredType().toString()
-              + " to " + creator.getValueType().toString();
-      throw new UnsupportedOperationException(failureMsg);
-    }
-
+  private void forwardIndexWriterHelper(String column, ColumnMetadata existingColumnMetadata, ForwardIndexReader reader,
+      ForwardIndexCreator creator, int numDocs, @Nullable SegmentDictionaryCreator dictionaryCreator) {
     ForwardIndexReaderContext readerContext = reader.createContext();
     boolean isSVColumn = reader.isSingleValue();
 
-    switch (reader.getStoredType()) {
-      // JSON fields are either stored as string or bytes. No special handling is needed because we make this
-      // decision based on the storedType of the reader.
-      case INT: {
-        for (int i = 0; i < numDocs; i++) {
-          if (isSVColumn) {
-            int val = reader.getInt(i, readerContext);
-            creator.putInt(val);
-          } else {
-            int[] ints = reader.getIntMV(i, readerContext);
-            creator.putIntMV(ints);
-          }
+    if (dictionaryCreator != null) {
+      int maxNumValuesPerEntry = existingColumnMetadata.getMaxNumberOfMultiValues();
+      PinotSegmentColumnReader columnReader = new PinotSegmentColumnReader(reader, null, null, maxNumValuesPerEntry);
+
+      for (int i = 0; i < numDocs; i++) {
+        Object obj = columnReader.getValue(i);
+
+        if (isSVColumn) {
+          int dictId = dictionaryCreator.indexOfSV(obj);
+          creator.putDictId(dictId);
+        } else {
+          int[] dictIds = dictionaryCreator.indexOfMV(obj);
+          creator.putDictIdMV(dictIds);
         }
-        break;
       }
-      case LONG: {
-        for (int i = 0; i < numDocs; i++) {
-          if (isSVColumn) {
-            long val = reader.getLong(i, readerContext);
-            creator.putLong(val);
-          } else {
-            long[] longs = reader.getLongMV(i, readerContext);
-            creator.putLongMV(longs);
+    } else {
+      switch (reader.getStoredType()) {
+        // JSON fields are either stored as string or bytes. No special handling is needed because we make this
+        // decision based on the storedType of the reader.
+        case INT: {
+          for (int i = 0; i < numDocs; i++) {
+            if (isSVColumn) {
+              int val = reader.getInt(i, readerContext);
+              creator.putInt(val);
+            } else {
+              int[] ints = reader.getIntMV(i, readerContext);
+              creator.putIntMV(ints);
+            }
           }
+          break;
         }
-        break;
-      }
-      case FLOAT: {
-        for (int i = 0; i < numDocs; i++) {
-          if (isSVColumn) {
-            float val = reader.getFloat(i, readerContext);
-            creator.putFloat(val);
-          } else {
-            float[] floats = reader.getFloatMV(i, readerContext);
-            creator.putFloatMV(floats);
+        case LONG: {
+          for (int i = 0; i < numDocs; i++) {
+            if (isSVColumn) {
+              long val = reader.getLong(i, readerContext);
+              creator.putLong(val);
+            } else {
+              long[] longs = reader.getLongMV(i, readerContext);
+              creator.putLongMV(longs);
+            }
           }
+          break;
         }
-        break;
-      }
-      case DOUBLE: {
-        for (int i = 0; i < numDocs; i++) {
-          if (isSVColumn) {
-            double val = reader.getDouble(i, readerContext);
-            creator.putDouble(val);
-          } else {
-            double[] doubles = reader.getDoubleMV(i, readerContext);
-            creator.putDoubleMV(doubles);
+        case FLOAT: {
+          for (int i = 0; i < numDocs; i++) {
+            if (isSVColumn) {
+              float val = reader.getFloat(i, readerContext);
+              creator.putFloat(val);
+            } else {
+              float[] floats = reader.getFloatMV(i, readerContext);
+              creator.putFloatMV(floats);
+            }
           }
+          break;
         }
-        break;
-      }
-      case STRING: {
-        for (int i = 0; i < numDocs; i++) {
-          if (isSVColumn) {
-            String val = reader.getString(i, readerContext);
-            creator.putString(val);
-          } else {
-            String[] strings = reader.getStringMV(i, readerContext);
-            creator.putStringMV(strings);
+        case DOUBLE: {
+          for (int i = 0; i < numDocs; i++) {
+            if (isSVColumn) {
+              double val = reader.getDouble(i, readerContext);
+              creator.putDouble(val);
+            } else {
+              double[] doubles = reader.getDoubleMV(i, readerContext);
+              creator.putDoubleMV(doubles);
+            }
           }
+          break;
         }
-        break;
-      }
-      case BYTES: {
-        for (int i = 0; i < numDocs; i++) {
-          if (isSVColumn) {
-            byte[] val = reader.getBytes(i, readerContext);
-            creator.putBytes(val);
-          } else {
-            byte[][] bytesArray = reader.getBytesMV(i, readerContext);
-            creator.putBytesMV(bytesArray);
+        case STRING: {
+          for (int i = 0; i < numDocs; i++) {
+            if (isSVColumn) {
+              String val = reader.getString(i, readerContext);
+              creator.putString(val);
+            } else {
+              String[] strings = reader.getStringMV(i, readerContext);
+              creator.putStringMV(strings);
+            }
           }
+          break;
         }
-        break;
-      }
-      case BIG_DECIMAL: {
-        for (int i = 0; i < numDocs; i++) {
+        case BYTES: {
+          for (int i = 0; i < numDocs; i++) {
+            if (isSVColumn) {
+              byte[] val = reader.getBytes(i, readerContext);
+              creator.putBytes(val);
+            } else {
+              byte[][] bytesArray = reader.getBytesMV(i, readerContext);
+              creator.putBytesMV(bytesArray);
+            }
+          }
+          break;
+        }
+        case BIG_DECIMAL: {
           Preconditions.checkState(isSVColumn, "BigDecimal is not supported for MV columns");
-          BigDecimal val = reader.getBigDecimal(i, readerContext);
-          creator.putBigDecimal(val);
+          for (int i = 0; i < numDocs; i++) {
+            BigDecimal val = reader.getBigDecimal(i, readerContext);
+            creator.putBigDecimal(val);
+          }
+          break;
         }
-        break;
+        default:
+          throw new IllegalStateException();
+      }
+    }
+  }
+
+  private void enableDictionary(String column, SegmentDirectory.Writer segmentWriter,
+      IndexCreatorProvider indexCreatorProvider)
+      throws Exception {
+    Preconditions.checkState(_segmentMetadata.getVersion() == SegmentVersion.v3);
+    ColumnMetadata existingColMetadata = _segmentMetadata.getColumnMetadataFor(column);
+    Preconditions.checkState(!existingColMetadata.hasDictionary(),
+        "Cannot rewrite dictionary enabled forward index. Dictionary already exists for column:" + column);
+    boolean isSingleValue = existingColMetadata.isSingleValue();
+
+    File indexDir = _segmentMetadata.getIndexDir();
+    String segmentName = _segmentMetadata.getName();
+    File inProgress = new File(indexDir, column + ".dict.inprogress");
+    File dictionaryFile = new File(indexDir, column + V1Constants.Dict.FILE_EXTENSION);
+    String fwdIndexFileExtension;
+    if (isSingleValue) {
+      fwdIndexFileExtension =
+          existingColMetadata.isSorted() ? V1Constants.Indexes.SORTED_SV_FORWARD_INDEX_FILE_EXTENSION
+              : V1Constants.Indexes.UNSORTED_SV_FORWARD_INDEX_FILE_EXTENSION;
+    } else {
+      fwdIndexFileExtension = V1Constants.Indexes.UNSORTED_MV_FORWARD_INDEX_FILE_EXTENSION;
+    }
+    File fwdIndexFile = new File(indexDir, column + fwdIndexFileExtension);
+
+    if (!inProgress.exists()) {
+      // Marker file does not exist, which means last run ended normally.
+      // Create a marker file.
+      FileUtils.touch(inProgress);
+    } else {
+      // Marker file exists, which means last run was interrupted.
+      // Remove forward index and dictionary files if they exist.
+      FileUtils.deleteQuietly(fwdIndexFile);
+      FileUtils.deleteQuietly(dictionaryFile);
+    }
+
+    LOGGER.info("Creating a new dictionary for segment={} and column={}", segmentName, column);
+    SegmentDictionaryCreator dictionaryCreator = buildDictionary(column, existingColMetadata, segmentWriter);
+    LoaderUtils.writeIndexToV3Format(segmentWriter, column, dictionaryFile, ColumnIndexType.DICTIONARY);
+
+    LOGGER.info("Built dictionary. Rewriting dictionary enabled forward index for segment={} and column={}",
+        segmentName, column);
+    writeDictEnabledForwardIndex(column, existingColMetadata, segmentWriter, indexDir, indexCreatorProvider,
+        dictionaryCreator);
+    // We used the existing forward index to generate a new forward index. The existing forward index will be in V3
+    // format and the new forward index will be in V1 format. Remove the existing forward index as it is not needed
+    // anymore. Note that removeIndex() will only mark an index for removal and remove the in-memory state. The
+    // actual cleanup from columns.psf file will happen when singleFileIndexDirectory.cleanupRemovedIndices() is
+    // called during segmentWriter.close().
+    segmentWriter.removeIndex(column, ColumnIndexType.FORWARD_INDEX);
+    LoaderUtils.writeIndexToV3Format(segmentWriter, column, fwdIndexFile, ColumnIndexType.FORWARD_INDEX);
+
+    LOGGER.info("Created forwardIndex. Updating metadata properties for segment={} and column={}", segmentName, column);
+    Map<String, String> metadataProperties = new HashMap<>();
+    metadataProperties.put(getKeyFor(column, HAS_DICTIONARY), String.valueOf(true));
+    metadataProperties.put(getKeyFor(column, DICTIONARY_ELEMENT_SIZE),
+        String.valueOf(dictionaryCreator.getNumBytesPerEntry()));
+    updateMetadataProperties(indexDir, metadataProperties);
+
+    // We remove indexes that have to be rewritten when a dictEnabled is toggled. Note that the respective index
+    // handler will take care of recreating the index.
+    removeDictRelatedIndexes(column, segmentWriter);
+
+    // Delete the marker file.
+    FileUtils.deleteQuietly(inProgress);
+
+    LOGGER.info("Created dictionary based forward index for segment: {}, column: {}", segmentName, column);
+  }
+
+  private SegmentDictionaryCreator buildDictionary(String column, ColumnMetadata existingColMetadata,
+      SegmentDirectory.Writer segmentWriter)
+      throws Exception {
+    int numDocs = existingColMetadata.getTotalDocs();
+    // SegmentPartitionConfig is not relevant for rewrites.
+    StatsCollectorConfig statsCollectorConfig =
+        new StatsCollectorConfig(_indexLoadingConfig.getTableConfig(), _schema, null);
+    AbstractColumnStatisticsCollector statsCollector;
+
+    try (ForwardIndexReader reader = LoaderUtils.getForwardIndexReader(segmentWriter, existingColMetadata)) {
+      boolean isSVColumn = reader.isSingleValue();
+
+      switch (reader.getStoredType()) {
+        case INT:
+          statsCollector = new IntColumnPreIndexStatsCollector(column, statsCollectorConfig);
+          break;
+        case LONG:
+          statsCollector = new LongColumnPreIndexStatsCollector(column, statsCollectorConfig);
+          break;
+        case FLOAT:
+          statsCollector = new FloatColumnPreIndexStatsCollector(column, statsCollectorConfig);
+          break;
+        case DOUBLE:
+          statsCollector = new DoubleColumnPreIndexStatsCollector(column, statsCollectorConfig);
+          break;
+        case STRING:
+          statsCollector = new StringColumnPreIndexStatsCollector(column, statsCollectorConfig);
+          break;
+        case BYTES:
+          statsCollector = new BytesColumnPredIndexStatsCollector(column, statsCollectorConfig);
+          break;
+        case BIG_DECIMAL:
+          Preconditions.checkState(isSVColumn, "BigDecimal is not supported for MV columns");
+          statsCollector = new BigDecimalColumnPreIndexStatsCollector(column, statsCollectorConfig);
+          break;
+        default:
+          throw new IllegalStateException();
+      }
+
+      Preconditions.checkState(statsCollector != null);
+      // Note: Special Null handling is not necessary here. This is because, the existing default null value in the
+      // raw forwardIndex will be retained as such while created the dictionary and dict-based forward index. Also,
+      // null value vectors maintain a bitmap of docIds. No handling is necessary there.
+      PinotSegmentColumnReader columnReader =

Review Comment:
   Just want to clarify that - PinotSegmentColumnReader doesn't initiate a new reader. It just uses the reader that is passed in the constructor. It provides a concise way to read from forward index without worry about dataTypes. Otherwise, we would write redundant code for each dataType.
   
   If you still feel we should not use it, I'm okay with using the existing reader. 



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


[GitHub] [pinot] somandal commented on a diff in pull request #9678: Support creating dictionary at runtime for an existing column

Posted by GitBox <gi...@apache.org>.
somandal commented on code in PR #9678:
URL: https://github.com/apache/pinot/pull/9678#discussion_r1012195565


##########
pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/loader/ForwardIndexHandler.java:
##########
@@ -271,118 +304,295 @@ private void rewriteRawSVForwardIndex(String column, ColumnMetadata existingColM
               .forForwardIndex(newCompressionType, _indexLoadingConfig.getColumnProperties());
 
       try (ForwardIndexCreator creator = indexCreatorProvider.newForwardIndexCreator(context)) {
-        // If creator stored type and the reader stored type do not match, throw an exception.
         if (!reader.getStoredType().equals(creator.getValueType())) {
+          // Creator stored type should match reader stored type for raw columns. We do not support changing datatypes.
           String failureMsg =
               "Unsupported operation to change datatype for column=" + column + " from " + reader.getStoredType()
                   .toString() + " to " + creator.getValueType().toString();
           throw new UnsupportedOperationException(failureMsg);
         }
 
         int numDocs = existingColMetadata.getTotalDocs();
-        forwardIndexWriterHelper(column, reader, creator, numDocs);
+        forwardIndexWriterHelper(column, existingColMetadata, reader, creator, numDocs, null);
       }
     }
   }
 
-  private void forwardIndexWriterHelper(String column, ForwardIndexReader reader, ForwardIndexCreator creator,
-      int numDocs) {
-    // If creator stored type should match reader stored type. We do not support changing datatypes.
-    if (!reader.getStoredType().equals(creator.getValueType())) {
-      String failureMsg =
-          "Unsupported operation to change datatype for column=" + column + " from " + reader.getStoredType().toString()
-              + " to " + creator.getValueType().toString();
-      throw new UnsupportedOperationException(failureMsg);
-    }
-
+  private void forwardIndexWriterHelper(String column, ColumnMetadata existingColumnMetadata, ForwardIndexReader reader,
+      ForwardIndexCreator creator, int numDocs, @Nullable SegmentDictionaryCreator dictionaryCreator) {
     ForwardIndexReaderContext readerContext = reader.createContext();
     boolean isSVColumn = reader.isSingleValue();
 
-    switch (reader.getStoredType()) {
-      // JSON fields are either stored as string or bytes. No special handling is needed because we make this
-      // decision based on the storedType of the reader.
-      case INT: {
-        for (int i = 0; i < numDocs; i++) {
-          if (isSVColumn) {
-            int val = reader.getInt(i, readerContext);
-            creator.putInt(val);
-          } else {
-            int[] ints = reader.getIntMV(i, readerContext);
-            creator.putIntMV(ints);
-          }
+    if (dictionaryCreator != null) {
+      int maxNumValuesPerEntry = existingColumnMetadata.getMaxNumberOfMultiValues();
+      PinotSegmentColumnReader columnReader = new PinotSegmentColumnReader(reader, null, null, maxNumValuesPerEntry);
+
+      for (int i = 0; i < numDocs; i++) {
+        Object obj = columnReader.getValue(i);
+
+        if (isSVColumn) {
+          int dictId = dictionaryCreator.indexOfSV(obj);
+          creator.putDictId(dictId);
+        } else {
+          int[] dictIds = dictionaryCreator.indexOfMV(obj);
+          creator.putDictIdMV(dictIds);
         }
-        break;
       }
-      case LONG: {
-        for (int i = 0; i < numDocs; i++) {
-          if (isSVColumn) {
-            long val = reader.getLong(i, readerContext);
-            creator.putLong(val);
-          } else {
-            long[] longs = reader.getLongMV(i, readerContext);
-            creator.putLongMV(longs);
+    } else {
+      switch (reader.getStoredType()) {
+        // JSON fields are either stored as string or bytes. No special handling is needed because we make this
+        // decision based on the storedType of the reader.
+        case INT: {
+          for (int i = 0; i < numDocs; i++) {
+            if (isSVColumn) {
+              int val = reader.getInt(i, readerContext);
+              creator.putInt(val);
+            } else {
+              int[] ints = reader.getIntMV(i, readerContext);
+              creator.putIntMV(ints);
+            }
           }
+          break;
         }
-        break;
-      }
-      case FLOAT: {
-        for (int i = 0; i < numDocs; i++) {
-          if (isSVColumn) {
-            float val = reader.getFloat(i, readerContext);
-            creator.putFloat(val);
-          } else {
-            float[] floats = reader.getFloatMV(i, readerContext);
-            creator.putFloatMV(floats);
+        case LONG: {
+          for (int i = 0; i < numDocs; i++) {
+            if (isSVColumn) {
+              long val = reader.getLong(i, readerContext);
+              creator.putLong(val);
+            } else {
+              long[] longs = reader.getLongMV(i, readerContext);
+              creator.putLongMV(longs);
+            }
           }
+          break;
         }
-        break;
-      }
-      case DOUBLE: {
-        for (int i = 0; i < numDocs; i++) {
-          if (isSVColumn) {
-            double val = reader.getDouble(i, readerContext);
-            creator.putDouble(val);
-          } else {
-            double[] doubles = reader.getDoubleMV(i, readerContext);
-            creator.putDoubleMV(doubles);
+        case FLOAT: {
+          for (int i = 0; i < numDocs; i++) {
+            if (isSVColumn) {
+              float val = reader.getFloat(i, readerContext);
+              creator.putFloat(val);
+            } else {
+              float[] floats = reader.getFloatMV(i, readerContext);
+              creator.putFloatMV(floats);
+            }
           }
+          break;
         }
-        break;
-      }
-      case STRING: {
-        for (int i = 0; i < numDocs; i++) {
-          if (isSVColumn) {
-            String val = reader.getString(i, readerContext);
-            creator.putString(val);
-          } else {
-            String[] strings = reader.getStringMV(i, readerContext);
-            creator.putStringMV(strings);
+        case DOUBLE: {
+          for (int i = 0; i < numDocs; i++) {
+            if (isSVColumn) {
+              double val = reader.getDouble(i, readerContext);
+              creator.putDouble(val);
+            } else {
+              double[] doubles = reader.getDoubleMV(i, readerContext);
+              creator.putDoubleMV(doubles);
+            }
           }
+          break;
         }
-        break;
-      }
-      case BYTES: {
-        for (int i = 0; i < numDocs; i++) {
-          if (isSVColumn) {
-            byte[] val = reader.getBytes(i, readerContext);
-            creator.putBytes(val);
-          } else {
-            byte[][] bytesArray = reader.getBytesMV(i, readerContext);
-            creator.putBytesMV(bytesArray);
+        case STRING: {
+          for (int i = 0; i < numDocs; i++) {
+            if (isSVColumn) {
+              String val = reader.getString(i, readerContext);
+              creator.putString(val);
+            } else {
+              String[] strings = reader.getStringMV(i, readerContext);
+              creator.putStringMV(strings);
+            }
           }
+          break;
         }
-        break;
-      }
-      case BIG_DECIMAL: {
-        for (int i = 0; i < numDocs; i++) {
+        case BYTES: {
+          for (int i = 0; i < numDocs; i++) {
+            if (isSVColumn) {
+              byte[] val = reader.getBytes(i, readerContext);
+              creator.putBytes(val);
+            } else {
+              byte[][] bytesArray = reader.getBytesMV(i, readerContext);
+              creator.putBytesMV(bytesArray);
+            }
+          }
+          break;
+        }
+        case BIG_DECIMAL: {
           Preconditions.checkState(isSVColumn, "BigDecimal is not supported for MV columns");
-          BigDecimal val = reader.getBigDecimal(i, readerContext);
-          creator.putBigDecimal(val);
+          for (int i = 0; i < numDocs; i++) {
+            BigDecimal val = reader.getBigDecimal(i, readerContext);
+            creator.putBigDecimal(val);
+          }
+          break;
         }
-        break;
+        default:
+          throw new IllegalStateException();
+      }
+    }
+  }
+
+  private void enableDictionary(String column, SegmentDirectory.Writer segmentWriter,
+      IndexCreatorProvider indexCreatorProvider)
+      throws Exception {
+    Preconditions.checkState(_segmentMetadata.getVersion() == SegmentVersion.v3);
+    ColumnMetadata existingColMetadata = _segmentMetadata.getColumnMetadataFor(column);
+    Preconditions.checkState(!existingColMetadata.hasDictionary(),
+        "Cannot rewrite dictionary enabled forward index. Dictionary already exists for column:" + column);
+    boolean isSingleValue = existingColMetadata.isSingleValue();
+
+    File indexDir = _segmentMetadata.getIndexDir();
+    String segmentName = _segmentMetadata.getName();
+    File inProgress = new File(indexDir, column + ".dict.inprogress");
+    File dictionaryFile = new File(indexDir, column + V1Constants.Dict.FILE_EXTENSION);
+    String fwdIndexFileExtension;
+    if (isSingleValue) {
+      fwdIndexFileExtension =
+          existingColMetadata.isSorted() ? V1Constants.Indexes.SORTED_SV_FORWARD_INDEX_FILE_EXTENSION
+              : V1Constants.Indexes.UNSORTED_SV_FORWARD_INDEX_FILE_EXTENSION;
+    } else {
+      fwdIndexFileExtension = V1Constants.Indexes.UNSORTED_MV_FORWARD_INDEX_FILE_EXTENSION;
+    }
+    File fwdIndexFile = new File(indexDir, column + fwdIndexFileExtension);
+
+    if (!inProgress.exists()) {
+      // Marker file does not exist, which means last run ended normally.
+      // Create a marker file.
+      FileUtils.touch(inProgress);
+    } else {
+      // Marker file exists, which means last run was interrupted.
+      // Remove forward index and dictionary files if they exist.
+      FileUtils.deleteQuietly(fwdIndexFile);
+      FileUtils.deleteQuietly(dictionaryFile);
+    }
+
+    LOGGER.info("Creating a new dictionary for segment={} and column={}", segmentName, column);
+    SegmentDictionaryCreator dictionaryCreator = buildDictionary(column, existingColMetadata, segmentWriter);
+    LoaderUtils.writeIndexToV3Format(segmentWriter, column, dictionaryFile, ColumnIndexType.DICTIONARY);
+
+    LOGGER.info("Built dictionary. Rewriting dictionary enabled forward index for segment={} and column={}",

Review Comment:
   Just to confirm the backup index directory also saves us in the scenario where: dict + forward index are created and written to the index, we fail before we can update metadata, right?



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


[GitHub] [pinot] somandal commented on a diff in pull request #9678: Support creating dictionary at runtime for an existing column

Posted by GitBox <gi...@apache.org>.
somandal commented on code in PR #9678:
URL: https://github.com/apache/pinot/pull/9678#discussion_r1012193537


##########
pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/loader/ForwardIndexHandler.java:
##########
@@ -26,23 +26,43 @@
 import java.util.HashSet;
 import java.util.Map;
 import java.util.Set;
+import javax.annotation.Nullable;
+import org.apache.commons.configuration.PropertiesConfiguration;
 import org.apache.commons.io.FileUtils;
+import org.apache.pinot.segment.local.segment.creator.impl.SegmentDictionaryCreator;
+import org.apache.pinot.segment.local.segment.creator.impl.SegmentIndexCreationDriverImpl;
 import org.apache.pinot.segment.local.segment.creator.impl.fwd.MultiValueVarByteRawIndexCreator;
+import org.apache.pinot.segment.local.segment.creator.impl.stats.AbstractColumnStatisticsCollector;
+import org.apache.pinot.segment.local.segment.creator.impl.stats.BigDecimalColumnPreIndexStatsCollector;
+import org.apache.pinot.segment.local.segment.creator.impl.stats.BytesColumnPredIndexStatsCollector;
+import org.apache.pinot.segment.local.segment.creator.impl.stats.DoubleColumnPreIndexStatsCollector;
+import org.apache.pinot.segment.local.segment.creator.impl.stats.FloatColumnPreIndexStatsCollector;
+import org.apache.pinot.segment.local.segment.creator.impl.stats.IntColumnPreIndexStatsCollector;
+import org.apache.pinot.segment.local.segment.creator.impl.stats.LongColumnPreIndexStatsCollector;
+import org.apache.pinot.segment.local.segment.creator.impl.stats.StringColumnPreIndexStatsCollector;
+import org.apache.pinot.segment.local.segment.readers.PinotSegmentColumnReader;
 import org.apache.pinot.segment.spi.ColumnMetadata;
 import org.apache.pinot.segment.spi.SegmentMetadata;
 import org.apache.pinot.segment.spi.V1Constants;
 import org.apache.pinot.segment.spi.compression.ChunkCompressionType;
 import org.apache.pinot.segment.spi.creator.IndexCreationContext;
 import org.apache.pinot.segment.spi.creator.IndexCreatorProvider;
 import org.apache.pinot.segment.spi.creator.SegmentVersion;
+import org.apache.pinot.segment.spi.creator.StatsCollectorConfig;
 import org.apache.pinot.segment.spi.index.creator.ForwardIndexCreator;
 import org.apache.pinot.segment.spi.index.reader.ForwardIndexReader;
 import org.apache.pinot.segment.spi.index.reader.ForwardIndexReaderContext;
 import org.apache.pinot.segment.spi.store.ColumnIndexType;
 import org.apache.pinot.segment.spi.store.SegmentDirectory;
+import org.apache.pinot.segment.spi.store.SegmentDirectoryPaths;
+import org.apache.pinot.spi.data.Schema;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import static org.apache.pinot.segment.spi.V1Constants.MetadataKeys.Column.DICTIONARY_ELEMENT_SIZE;

Review Comment:
   got it, thanks for confirming!



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


[GitHub] [pinot] vvivekiyer commented on a diff in pull request #9678: Support creating dictionary at runtime for an existing column

Posted by GitBox <gi...@apache.org>.
vvivekiyer commented on code in PR #9678:
URL: https://github.com/apache/pinot/pull/9678#discussion_r1011007064


##########
pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/loader/ForwardIndexHandler.java:
##########
@@ -271,118 +306,293 @@ private void rewriteRawSVForwardIndex(String column, ColumnMetadata existingColM
               .forForwardIndex(newCompressionType, _indexLoadingConfig.getColumnProperties());
 
       try (ForwardIndexCreator creator = indexCreatorProvider.newForwardIndexCreator(context)) {
-        // If creator stored type and the reader stored type do not match, throw an exception.
         if (!reader.getStoredType().equals(creator.getValueType())) {
+          // Creator stored type should match reader stored type for raw columns. We do not support changing datatypes.
           String failureMsg =
               "Unsupported operation to change datatype for column=" + column + " from " + reader.getStoredType()
                   .toString() + " to " + creator.getValueType().toString();
           throw new UnsupportedOperationException(failureMsg);
         }
 
         int numDocs = existingColMetadata.getTotalDocs();
-        forwardIndexWriterHelper(column, reader, creator, numDocs);
+        forwardIndexWriterHelper(column, existingColMetadata, reader, creator, numDocs, null);
       }
     }
   }
 
-  private void forwardIndexWriterHelper(String column, ForwardIndexReader reader, ForwardIndexCreator creator,
-      int numDocs) {
-    // If creator stored type should match reader stored type. We do not support changing datatypes.
-    if (!reader.getStoredType().equals(creator.getValueType())) {
-      String failureMsg =
-          "Unsupported operation to change datatype for column=" + column + " from " + reader.getStoredType().toString()
-              + " to " + creator.getValueType().toString();
-      throw new UnsupportedOperationException(failureMsg);
-    }
-
+  private void forwardIndexWriterHelper(String column, ColumnMetadata existingColumnMetadata, ForwardIndexReader reader,
+      ForwardIndexCreator creator, int numDocs, @Nullable SegmentDictionaryCreator dictionaryCreator) {
     ForwardIndexReaderContext readerContext = reader.createContext();
     boolean isSVColumn = reader.isSingleValue();
 
-    switch (reader.getStoredType()) {
-      // JSON fields are either stored as string or bytes. No special handling is needed because we make this
-      // decision based on the storedType of the reader.
-      case INT: {
-        for (int i = 0; i < numDocs; i++) {
-          if (isSVColumn) {
-            int val = reader.getInt(i, readerContext);
-            creator.putInt(val);
-          } else {
-            int[] ints = reader.getIntMV(i, readerContext);
-            creator.putIntMV(ints);
-          }
+    if (dictionaryCreator != null) {
+      int maxNumValuesPerEntry = existingColumnMetadata.getMaxNumberOfMultiValues();
+      PinotSegmentColumnReader columnReader = new PinotSegmentColumnReader(reader, null, null, maxNumValuesPerEntry);
+
+      for (int i = 0; i < numDocs; i++) {
+        Object obj = columnReader.getValue(i);
+
+        if (isSVColumn) {
+          int dictId = dictionaryCreator.indexOfSV(obj);
+          creator.putDictId(dictId);
+        } else {
+          int[] dictIds = dictionaryCreator.indexOfMV(obj);
+          creator.putDictIdMV(dictIds);
         }
-        break;
       }
-      case LONG: {
-        for (int i = 0; i < numDocs; i++) {
-          if (isSVColumn) {
-            long val = reader.getLong(i, readerContext);
-            creator.putLong(val);
-          } else {
-            long[] longs = reader.getLongMV(i, readerContext);
-            creator.putLongMV(longs);
+    } else {
+      switch (reader.getStoredType()) {
+        // JSON fields are either stored as string or bytes. No special handling is needed because we make this
+        // decision based on the storedType of the reader.
+        case INT: {
+          for (int i = 0; i < numDocs; i++) {
+            if (isSVColumn) {
+              int val = reader.getInt(i, readerContext);
+              creator.putInt(val);
+            } else {
+              int[] ints = reader.getIntMV(i, readerContext);
+              creator.putIntMV(ints);
+            }
           }
+          break;
         }
-        break;
-      }
-      case FLOAT: {
-        for (int i = 0; i < numDocs; i++) {
-          if (isSVColumn) {
-            float val = reader.getFloat(i, readerContext);
-            creator.putFloat(val);
-          } else {
-            float[] floats = reader.getFloatMV(i, readerContext);
-            creator.putFloatMV(floats);
+        case LONG: {
+          for (int i = 0; i < numDocs; i++) {
+            if (isSVColumn) {
+              long val = reader.getLong(i, readerContext);
+              creator.putLong(val);
+            } else {
+              long[] longs = reader.getLongMV(i, readerContext);
+              creator.putLongMV(longs);
+            }
           }
+          break;
         }
-        break;
-      }
-      case DOUBLE: {
-        for (int i = 0; i < numDocs; i++) {
-          if (isSVColumn) {
-            double val = reader.getDouble(i, readerContext);
-            creator.putDouble(val);
-          } else {
-            double[] doubles = reader.getDoubleMV(i, readerContext);
-            creator.putDoubleMV(doubles);
+        case FLOAT: {
+          for (int i = 0; i < numDocs; i++) {
+            if (isSVColumn) {
+              float val = reader.getFloat(i, readerContext);
+              creator.putFloat(val);
+            } else {
+              float[] floats = reader.getFloatMV(i, readerContext);
+              creator.putFloatMV(floats);
+            }
           }
+          break;
         }
-        break;
-      }
-      case STRING: {
-        for (int i = 0; i < numDocs; i++) {
-          if (isSVColumn) {
-            String val = reader.getString(i, readerContext);
-            creator.putString(val);
-          } else {
-            String[] strings = reader.getStringMV(i, readerContext);
-            creator.putStringMV(strings);
+        case DOUBLE: {
+          for (int i = 0; i < numDocs; i++) {
+            if (isSVColumn) {
+              double val = reader.getDouble(i, readerContext);
+              creator.putDouble(val);
+            } else {
+              double[] doubles = reader.getDoubleMV(i, readerContext);
+              creator.putDoubleMV(doubles);
+            }
           }
+          break;
         }
-        break;
-      }
-      case BYTES: {
-        for (int i = 0; i < numDocs; i++) {
-          if (isSVColumn) {
-            byte[] val = reader.getBytes(i, readerContext);
-            creator.putBytes(val);
-          } else {
-            byte[][] bytesArray = reader.getBytesMV(i, readerContext);
-            creator.putBytesMV(bytesArray);
+        case STRING: {
+          for (int i = 0; i < numDocs; i++) {
+            if (isSVColumn) {
+              String val = reader.getString(i, readerContext);
+              creator.putString(val);
+            } else {
+              String[] strings = reader.getStringMV(i, readerContext);
+              creator.putStringMV(strings);
+            }
           }
+          break;
         }
-        break;
-      }
-      case BIG_DECIMAL: {
-        for (int i = 0; i < numDocs; i++) {
+        case BYTES: {
+          for (int i = 0; i < numDocs; i++) {
+            if (isSVColumn) {
+              byte[] val = reader.getBytes(i, readerContext);
+              creator.putBytes(val);
+            } else {
+              byte[][] bytesArray = reader.getBytesMV(i, readerContext);
+              creator.putBytesMV(bytesArray);
+            }
+          }
+          break;
+        }
+        case BIG_DECIMAL: {
           Preconditions.checkState(isSVColumn, "BigDecimal is not supported for MV columns");
-          BigDecimal val = reader.getBigDecimal(i, readerContext);
-          creator.putBigDecimal(val);
+          for (int i = 0; i < numDocs; i++) {
+            BigDecimal val = reader.getBigDecimal(i, readerContext);
+            creator.putBigDecimal(val);
+          }
+          break;
         }
-        break;
+        default:
+          throw new IllegalStateException();
+      }
+    }
+  }
+
+  private void enableDictionary(String column, SegmentDirectory.Writer segmentWriter,
+      IndexCreatorProvider indexCreatorProvider)
+      throws Exception {
+    Preconditions.checkState(_segmentMetadata.getVersion() == SegmentVersion.v3);
+    ColumnMetadata existingColMetadata = _segmentMetadata.getColumnMetadataFor(column);
+    Preconditions.checkState(!existingColMetadata.hasDictionary(), "Existing column already has dictionary.");

Review Comment:
   Done



##########
pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/loader/ForwardIndexHandler.java:
##########
@@ -271,118 +306,293 @@ private void rewriteRawSVForwardIndex(String column, ColumnMetadata existingColM
               .forForwardIndex(newCompressionType, _indexLoadingConfig.getColumnProperties());
 
       try (ForwardIndexCreator creator = indexCreatorProvider.newForwardIndexCreator(context)) {
-        // If creator stored type and the reader stored type do not match, throw an exception.
         if (!reader.getStoredType().equals(creator.getValueType())) {
+          // Creator stored type should match reader stored type for raw columns. We do not support changing datatypes.
           String failureMsg =
               "Unsupported operation to change datatype for column=" + column + " from " + reader.getStoredType()
                   .toString() + " to " + creator.getValueType().toString();
           throw new UnsupportedOperationException(failureMsg);
         }
 
         int numDocs = existingColMetadata.getTotalDocs();
-        forwardIndexWriterHelper(column, reader, creator, numDocs);
+        forwardIndexWriterHelper(column, existingColMetadata, reader, creator, numDocs, null);
       }
     }
   }
 
-  private void forwardIndexWriterHelper(String column, ForwardIndexReader reader, ForwardIndexCreator creator,
-      int numDocs) {
-    // If creator stored type should match reader stored type. We do not support changing datatypes.
-    if (!reader.getStoredType().equals(creator.getValueType())) {
-      String failureMsg =
-          "Unsupported operation to change datatype for column=" + column + " from " + reader.getStoredType().toString()
-              + " to " + creator.getValueType().toString();
-      throw new UnsupportedOperationException(failureMsg);
-    }
-
+  private void forwardIndexWriterHelper(String column, ColumnMetadata existingColumnMetadata, ForwardIndexReader reader,
+      ForwardIndexCreator creator, int numDocs, @Nullable SegmentDictionaryCreator dictionaryCreator) {
     ForwardIndexReaderContext readerContext = reader.createContext();
     boolean isSVColumn = reader.isSingleValue();
 
-    switch (reader.getStoredType()) {
-      // JSON fields are either stored as string or bytes. No special handling is needed because we make this
-      // decision based on the storedType of the reader.
-      case INT: {
-        for (int i = 0; i < numDocs; i++) {
-          if (isSVColumn) {
-            int val = reader.getInt(i, readerContext);
-            creator.putInt(val);
-          } else {
-            int[] ints = reader.getIntMV(i, readerContext);
-            creator.putIntMV(ints);
-          }
+    if (dictionaryCreator != null) {
+      int maxNumValuesPerEntry = existingColumnMetadata.getMaxNumberOfMultiValues();
+      PinotSegmentColumnReader columnReader = new PinotSegmentColumnReader(reader, null, null, maxNumValuesPerEntry);
+
+      for (int i = 0; i < numDocs; i++) {
+        Object obj = columnReader.getValue(i);
+
+        if (isSVColumn) {
+          int dictId = dictionaryCreator.indexOfSV(obj);
+          creator.putDictId(dictId);
+        } else {
+          int[] dictIds = dictionaryCreator.indexOfMV(obj);
+          creator.putDictIdMV(dictIds);
         }
-        break;
       }
-      case LONG: {
-        for (int i = 0; i < numDocs; i++) {
-          if (isSVColumn) {
-            long val = reader.getLong(i, readerContext);
-            creator.putLong(val);
-          } else {
-            long[] longs = reader.getLongMV(i, readerContext);
-            creator.putLongMV(longs);
+    } else {
+      switch (reader.getStoredType()) {
+        // JSON fields are either stored as string or bytes. No special handling is needed because we make this
+        // decision based on the storedType of the reader.
+        case INT: {
+          for (int i = 0; i < numDocs; i++) {
+            if (isSVColumn) {
+              int val = reader.getInt(i, readerContext);
+              creator.putInt(val);
+            } else {
+              int[] ints = reader.getIntMV(i, readerContext);
+              creator.putIntMV(ints);
+            }
           }
+          break;
         }
-        break;
-      }
-      case FLOAT: {
-        for (int i = 0; i < numDocs; i++) {
-          if (isSVColumn) {
-            float val = reader.getFloat(i, readerContext);
-            creator.putFloat(val);
-          } else {
-            float[] floats = reader.getFloatMV(i, readerContext);
-            creator.putFloatMV(floats);
+        case LONG: {
+          for (int i = 0; i < numDocs; i++) {
+            if (isSVColumn) {
+              long val = reader.getLong(i, readerContext);
+              creator.putLong(val);
+            } else {
+              long[] longs = reader.getLongMV(i, readerContext);
+              creator.putLongMV(longs);
+            }
           }
+          break;
         }
-        break;
-      }
-      case DOUBLE: {
-        for (int i = 0; i < numDocs; i++) {
-          if (isSVColumn) {
-            double val = reader.getDouble(i, readerContext);
-            creator.putDouble(val);
-          } else {
-            double[] doubles = reader.getDoubleMV(i, readerContext);
-            creator.putDoubleMV(doubles);
+        case FLOAT: {
+          for (int i = 0; i < numDocs; i++) {
+            if (isSVColumn) {
+              float val = reader.getFloat(i, readerContext);
+              creator.putFloat(val);
+            } else {
+              float[] floats = reader.getFloatMV(i, readerContext);
+              creator.putFloatMV(floats);
+            }
           }
+          break;
         }
-        break;
-      }
-      case STRING: {
-        for (int i = 0; i < numDocs; i++) {
-          if (isSVColumn) {
-            String val = reader.getString(i, readerContext);
-            creator.putString(val);
-          } else {
-            String[] strings = reader.getStringMV(i, readerContext);
-            creator.putStringMV(strings);
+        case DOUBLE: {
+          for (int i = 0; i < numDocs; i++) {
+            if (isSVColumn) {
+              double val = reader.getDouble(i, readerContext);
+              creator.putDouble(val);
+            } else {
+              double[] doubles = reader.getDoubleMV(i, readerContext);
+              creator.putDoubleMV(doubles);
+            }
           }
+          break;
         }
-        break;
-      }
-      case BYTES: {
-        for (int i = 0; i < numDocs; i++) {
-          if (isSVColumn) {
-            byte[] val = reader.getBytes(i, readerContext);
-            creator.putBytes(val);
-          } else {
-            byte[][] bytesArray = reader.getBytesMV(i, readerContext);
-            creator.putBytesMV(bytesArray);
+        case STRING: {
+          for (int i = 0; i < numDocs; i++) {
+            if (isSVColumn) {
+              String val = reader.getString(i, readerContext);
+              creator.putString(val);
+            } else {
+              String[] strings = reader.getStringMV(i, readerContext);
+              creator.putStringMV(strings);
+            }
           }
+          break;
         }
-        break;
-      }
-      case BIG_DECIMAL: {
-        for (int i = 0; i < numDocs; i++) {
+        case BYTES: {
+          for (int i = 0; i < numDocs; i++) {
+            if (isSVColumn) {
+              byte[] val = reader.getBytes(i, readerContext);
+              creator.putBytes(val);
+            } else {
+              byte[][] bytesArray = reader.getBytesMV(i, readerContext);
+              creator.putBytesMV(bytesArray);
+            }
+          }
+          break;
+        }
+        case BIG_DECIMAL: {
           Preconditions.checkState(isSVColumn, "BigDecimal is not supported for MV columns");
-          BigDecimal val = reader.getBigDecimal(i, readerContext);
-          creator.putBigDecimal(val);
+          for (int i = 0; i < numDocs; i++) {
+            BigDecimal val = reader.getBigDecimal(i, readerContext);
+            creator.putBigDecimal(val);
+          }
+          break;
         }
-        break;
+        default:
+          throw new IllegalStateException();
+      }
+    }
+  }
+
+  private void enableDictionary(String column, SegmentDirectory.Writer segmentWriter,
+      IndexCreatorProvider indexCreatorProvider)
+      throws Exception {
+    Preconditions.checkState(_segmentMetadata.getVersion() == SegmentVersion.v3);
+    ColumnMetadata existingColMetadata = _segmentMetadata.getColumnMetadataFor(column);
+    Preconditions.checkState(!existingColMetadata.hasDictionary(), "Existing column already has dictionary.");
+    boolean isSingleValue = existingColMetadata.isSingleValue();
+
+    File indexDir = _segmentMetadata.getIndexDir();
+    String segmentName = _segmentMetadata.getName();
+    File inProgress = new File(indexDir, column + ".dict.inprogress");
+    File dictionaryFile = new File(indexDir, column + V1Constants.Dict.FILE_EXTENSION);
+    String fwdIndexFileExtension;
+    if (isSingleValue) {
+      if (existingColMetadata.isSorted()) {
+        fwdIndexFileExtension = V1Constants.Indexes.SORTED_SV_FORWARD_INDEX_FILE_EXTENSION;
+      } else {
+        fwdIndexFileExtension = V1Constants.Indexes.UNSORTED_SV_FORWARD_INDEX_FILE_EXTENSION;
+      }

Review Comment:
   Done



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


[GitHub] [pinot] somandal commented on a diff in pull request #9678: Support creating dictionary at runtime for an existing column

Posted by GitBox <gi...@apache.org>.
somandal commented on code in PR #9678:
URL: https://github.com/apache/pinot/pull/9678#discussion_r1012153508


##########
pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/loader/ForwardIndexHandler.java:
##########
@@ -271,118 +304,295 @@ private void rewriteRawSVForwardIndex(String column, ColumnMetadata existingColM
               .forForwardIndex(newCompressionType, _indexLoadingConfig.getColumnProperties());
 
       try (ForwardIndexCreator creator = indexCreatorProvider.newForwardIndexCreator(context)) {
-        // If creator stored type and the reader stored type do not match, throw an exception.
         if (!reader.getStoredType().equals(creator.getValueType())) {
+          // Creator stored type should match reader stored type for raw columns. We do not support changing datatypes.
           String failureMsg =
               "Unsupported operation to change datatype for column=" + column + " from " + reader.getStoredType()
                   .toString() + " to " + creator.getValueType().toString();
           throw new UnsupportedOperationException(failureMsg);
         }
 
         int numDocs = existingColMetadata.getTotalDocs();
-        forwardIndexWriterHelper(column, reader, creator, numDocs);
+        forwardIndexWriterHelper(column, existingColMetadata, reader, creator, numDocs, null);
       }
     }
   }
 
-  private void forwardIndexWriterHelper(String column, ForwardIndexReader reader, ForwardIndexCreator creator,
-      int numDocs) {
-    // If creator stored type should match reader stored type. We do not support changing datatypes.
-    if (!reader.getStoredType().equals(creator.getValueType())) {
-      String failureMsg =
-          "Unsupported operation to change datatype for column=" + column + " from " + reader.getStoredType().toString()
-              + " to " + creator.getValueType().toString();
-      throw new UnsupportedOperationException(failureMsg);
-    }
-
+  private void forwardIndexWriterHelper(String column, ColumnMetadata existingColumnMetadata, ForwardIndexReader reader,
+      ForwardIndexCreator creator, int numDocs, @Nullable SegmentDictionaryCreator dictionaryCreator) {
     ForwardIndexReaderContext readerContext = reader.createContext();
     boolean isSVColumn = reader.isSingleValue();
 
-    switch (reader.getStoredType()) {
-      // JSON fields are either stored as string or bytes. No special handling is needed because we make this
-      // decision based on the storedType of the reader.
-      case INT: {
-        for (int i = 0; i < numDocs; i++) {
-          if (isSVColumn) {
-            int val = reader.getInt(i, readerContext);
-            creator.putInt(val);
-          } else {
-            int[] ints = reader.getIntMV(i, readerContext);
-            creator.putIntMV(ints);
-          }
+    if (dictionaryCreator != null) {
+      int maxNumValuesPerEntry = existingColumnMetadata.getMaxNumberOfMultiValues();
+      PinotSegmentColumnReader columnReader = new PinotSegmentColumnReader(reader, null, null, maxNumValuesPerEntry);
+
+      for (int i = 0; i < numDocs; i++) {
+        Object obj = columnReader.getValue(i);
+
+        if (isSVColumn) {
+          int dictId = dictionaryCreator.indexOfSV(obj);
+          creator.putDictId(dictId);
+        } else {
+          int[] dictIds = dictionaryCreator.indexOfMV(obj);
+          creator.putDictIdMV(dictIds);
         }
-        break;
       }
-      case LONG: {
-        for (int i = 0; i < numDocs; i++) {
-          if (isSVColumn) {
-            long val = reader.getLong(i, readerContext);
-            creator.putLong(val);
-          } else {
-            long[] longs = reader.getLongMV(i, readerContext);
-            creator.putLongMV(longs);
+    } else {
+      switch (reader.getStoredType()) {
+        // JSON fields are either stored as string or bytes. No special handling is needed because we make this
+        // decision based on the storedType of the reader.
+        case INT: {
+          for (int i = 0; i < numDocs; i++) {
+            if (isSVColumn) {
+              int val = reader.getInt(i, readerContext);
+              creator.putInt(val);
+            } else {
+              int[] ints = reader.getIntMV(i, readerContext);
+              creator.putIntMV(ints);
+            }
           }
+          break;
         }
-        break;
-      }
-      case FLOAT: {
-        for (int i = 0; i < numDocs; i++) {
-          if (isSVColumn) {
-            float val = reader.getFloat(i, readerContext);
-            creator.putFloat(val);
-          } else {
-            float[] floats = reader.getFloatMV(i, readerContext);
-            creator.putFloatMV(floats);
+        case LONG: {
+          for (int i = 0; i < numDocs; i++) {
+            if (isSVColumn) {
+              long val = reader.getLong(i, readerContext);
+              creator.putLong(val);
+            } else {
+              long[] longs = reader.getLongMV(i, readerContext);
+              creator.putLongMV(longs);
+            }
           }
+          break;
         }
-        break;
-      }
-      case DOUBLE: {
-        for (int i = 0; i < numDocs; i++) {
-          if (isSVColumn) {
-            double val = reader.getDouble(i, readerContext);
-            creator.putDouble(val);
-          } else {
-            double[] doubles = reader.getDoubleMV(i, readerContext);
-            creator.putDoubleMV(doubles);
+        case FLOAT: {
+          for (int i = 0; i < numDocs; i++) {
+            if (isSVColumn) {
+              float val = reader.getFloat(i, readerContext);
+              creator.putFloat(val);
+            } else {
+              float[] floats = reader.getFloatMV(i, readerContext);
+              creator.putFloatMV(floats);
+            }
           }
+          break;
         }
-        break;
-      }
-      case STRING: {
-        for (int i = 0; i < numDocs; i++) {
-          if (isSVColumn) {
-            String val = reader.getString(i, readerContext);
-            creator.putString(val);
-          } else {
-            String[] strings = reader.getStringMV(i, readerContext);
-            creator.putStringMV(strings);
+        case DOUBLE: {
+          for (int i = 0; i < numDocs; i++) {
+            if (isSVColumn) {
+              double val = reader.getDouble(i, readerContext);
+              creator.putDouble(val);
+            } else {
+              double[] doubles = reader.getDoubleMV(i, readerContext);
+              creator.putDoubleMV(doubles);
+            }
           }
+          break;
         }
-        break;
-      }
-      case BYTES: {
-        for (int i = 0; i < numDocs; i++) {
-          if (isSVColumn) {
-            byte[] val = reader.getBytes(i, readerContext);
-            creator.putBytes(val);
-          } else {
-            byte[][] bytesArray = reader.getBytesMV(i, readerContext);
-            creator.putBytesMV(bytesArray);
+        case STRING: {
+          for (int i = 0; i < numDocs; i++) {
+            if (isSVColumn) {
+              String val = reader.getString(i, readerContext);
+              creator.putString(val);
+            } else {
+              String[] strings = reader.getStringMV(i, readerContext);
+              creator.putStringMV(strings);
+            }
           }
+          break;
         }
-        break;
-      }
-      case BIG_DECIMAL: {
-        for (int i = 0; i < numDocs; i++) {
+        case BYTES: {
+          for (int i = 0; i < numDocs; i++) {
+            if (isSVColumn) {
+              byte[] val = reader.getBytes(i, readerContext);
+              creator.putBytes(val);
+            } else {
+              byte[][] bytesArray = reader.getBytesMV(i, readerContext);
+              creator.putBytesMV(bytesArray);
+            }
+          }
+          break;
+        }
+        case BIG_DECIMAL: {
           Preconditions.checkState(isSVColumn, "BigDecimal is not supported for MV columns");
-          BigDecimal val = reader.getBigDecimal(i, readerContext);
-          creator.putBigDecimal(val);
+          for (int i = 0; i < numDocs; i++) {
+            BigDecimal val = reader.getBigDecimal(i, readerContext);
+            creator.putBigDecimal(val);
+          }
+          break;
         }
-        break;
+        default:
+          throw new IllegalStateException();
+      }
+    }
+  }
+
+  private void enableDictionary(String column, SegmentDirectory.Writer segmentWriter,
+      IndexCreatorProvider indexCreatorProvider)
+      throws Exception {
+    Preconditions.checkState(_segmentMetadata.getVersion() == SegmentVersion.v3);
+    ColumnMetadata existingColMetadata = _segmentMetadata.getColumnMetadataFor(column);
+    Preconditions.checkState(!existingColMetadata.hasDictionary(),
+        "Cannot rewrite dictionary enabled forward index. Dictionary already exists for column:" + column);
+    boolean isSingleValue = existingColMetadata.isSingleValue();
+
+    File indexDir = _segmentMetadata.getIndexDir();
+    String segmentName = _segmentMetadata.getName();
+    File inProgress = new File(indexDir, column + ".dict.inprogress");
+    File dictionaryFile = new File(indexDir, column + V1Constants.Dict.FILE_EXTENSION);
+    String fwdIndexFileExtension;
+    if (isSingleValue) {
+      fwdIndexFileExtension =
+          existingColMetadata.isSorted() ? V1Constants.Indexes.SORTED_SV_FORWARD_INDEX_FILE_EXTENSION

Review Comment:
   Just to understand, if someone had added a sorted column under the noDict list in the TableConfig what happens? For dict disable case, will this operation be allowed for sorted columns? Should this fail or be a no-op? Perhaps need to check this code path and add relevant Precondition checks / returns as no-op code.



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


[GitHub] [pinot] siddharthteotia commented on a diff in pull request #9678: Support creating dictionary at runtime for an existing column

Posted by GitBox <gi...@apache.org>.
siddharthteotia commented on code in PR #9678:
URL: https://github.com/apache/pinot/pull/9678#discussion_r1012294683


##########
pinot-segment-local/src/test/java/org/apache/pinot/segment/local/segment/index/loader/ForwardIndexHandlerTest.java:
##########
@@ -302,34 +351,62 @@ public void testComputeOperation()
 
     // TEST1 : Validate with zero changes. ForwardIndexHandler should be a No-Op.
     IndexLoadingConfig indexLoadingConfig = new IndexLoadingConfig(null, _tableConfig);
-    ForwardIndexHandler fwdIndexHandler = new ForwardIndexHandler(existingSegmentMetadata, indexLoadingConfig);
+    ForwardIndexHandler fwdIndexHandler = new ForwardIndexHandler(existingSegmentMetadata, indexLoadingConfig, null);
     Map<String, ForwardIndexHandler.Operation> operationMap = new HashMap<>();
     operationMap = fwdIndexHandler.computeOperation(writer);
     assertEquals(operationMap, Collections.EMPTY_MAP);
 
-    // TEST2: Enable dictionary for a RAW_ZSTANDARD_INDEX_COLUMN. ForwardIndexHandler should be a No-Op.
+    // TEST2: Enable dictionary for a RAW_ZSTANDARD_INDEX_COLUMN.
     indexLoadingConfig = new IndexLoadingConfig(null, _tableConfig);
     indexLoadingConfig.getNoDictionaryColumns().remove(DIM_ZSTANDARD_STRING);
-    fwdIndexHandler = new ForwardIndexHandler(existingSegmentMetadata, indexLoadingConfig);
+    fwdIndexHandler = new ForwardIndexHandler(existingSegmentMetadata, indexLoadingConfig, _schema);
+    operationMap = fwdIndexHandler.computeOperation(writer);
+    assertEquals(operationMap.get(DIM_ZSTANDARD_STRING), ForwardIndexHandler.Operation.ENABLE_DICTIONARY);
+
+    // TEST3: Enable dictionary for an MV column.
+    indexLoadingConfig = new IndexLoadingConfig(null, _tableConfig);
+    indexLoadingConfig.getNoDictionaryColumns().remove(DIM_MV_PASS_THROUGH_STRING);
+    fwdIndexHandler = new ForwardIndexHandler(existingSegmentMetadata, indexLoadingConfig, _schema);
+    operationMap = fwdIndexHandler.computeOperation(writer);
+    assertEquals(operationMap.get(DIM_MV_PASS_THROUGH_STRING), ForwardIndexHandler.Operation.ENABLE_DICTIONARY);
+
+    // TEST4: Enable dictionary for a sorted column.
+    indexLoadingConfig = new IndexLoadingConfig(null, _tableConfig);
+    indexLoadingConfig.getNoDictionaryColumns().remove(DIM_PASS_THROUGH_SORTED_LONG);
+    fwdIndexHandler = new ForwardIndexHandler(existingSegmentMetadata, indexLoadingConfig, _schema);
+    operationMap = fwdIndexHandler.computeOperation(writer);
+    assertEquals(operationMap.get(DIM_PASS_THROUGH_SORTED_LONG), ForwardIndexHandler.Operation.ENABLE_DICTIONARY);
+
+    // TEST5: Enable dictionary for a dict column. Should be a No-op.
+    indexLoadingConfig = new IndexLoadingConfig(null, _tableConfig);
+    fwdIndexHandler = new ForwardIndexHandler(existingSegmentMetadata, indexLoadingConfig, _schema);
     operationMap = fwdIndexHandler.computeOperation(writer);
     assertEquals(operationMap, Collections.EMPTY_MAP);
 
-    // TEST3: Disable dictionary. ForwardIndexHandler should be a No-Op.
+    // TEST6: Disable dictionary. Should be a No-op.
     indexLoadingConfig = new IndexLoadingConfig(null, _tableConfig);
     indexLoadingConfig.getNoDictionaryColumns().add(DIM_DICT_INTEGER);
-    fwdIndexHandler = new ForwardIndexHandler(existingSegmentMetadata, indexLoadingConfig);
+    fwdIndexHandler = new ForwardIndexHandler(existingSegmentMetadata, indexLoadingConfig, _schema);
     operationMap = fwdIndexHandler.computeOperation(writer);
     assertEquals(operationMap, Collections.EMPTY_MAP);
 
-    // TEST4: Add an additional text index. ForwardIndexHandler should be a No-Op.
+    // TEST7: Add an additional text index. ForwardIndexHandler should be a No-Op.
     indexLoadingConfig = new IndexLoadingConfig(null, _tableConfig);
     indexLoadingConfig.getTextIndexColumns().add(DIM_DICT_INTEGER);
     indexLoadingConfig.getTextIndexColumns().add(DIM_LZ4_INTEGER);
-    fwdIndexHandler = new ForwardIndexHandler(existingSegmentMetadata, indexLoadingConfig);
+    fwdIndexHandler = new ForwardIndexHandler(existingSegmentMetadata, indexLoadingConfig, _schema);
     operationMap = fwdIndexHandler.computeOperation(writer);
     assertEquals(operationMap, Collections.EMPTY_MAP);
 
-    // TEST5: Change compression
+    // TEST8: Add text index and disable forward index.

Review Comment:
   I think this comment is misleading. You mean to say `"Add text index and enable dictionary"`



##########
pinot-segment-local/src/test/java/org/apache/pinot/segment/local/segment/index/loader/ForwardIndexHandlerTest.java:
##########
@@ -302,34 +351,62 @@ public void testComputeOperation()
 
     // TEST1 : Validate with zero changes. ForwardIndexHandler should be a No-Op.
     IndexLoadingConfig indexLoadingConfig = new IndexLoadingConfig(null, _tableConfig);
-    ForwardIndexHandler fwdIndexHandler = new ForwardIndexHandler(existingSegmentMetadata, indexLoadingConfig);
+    ForwardIndexHandler fwdIndexHandler = new ForwardIndexHandler(existingSegmentMetadata, indexLoadingConfig, null);
     Map<String, ForwardIndexHandler.Operation> operationMap = new HashMap<>();
     operationMap = fwdIndexHandler.computeOperation(writer);
     assertEquals(operationMap, Collections.EMPTY_MAP);
 
-    // TEST2: Enable dictionary for a RAW_ZSTANDARD_INDEX_COLUMN. ForwardIndexHandler should be a No-Op.
+    // TEST2: Enable dictionary for a RAW_ZSTANDARD_INDEX_COLUMN.
     indexLoadingConfig = new IndexLoadingConfig(null, _tableConfig);
     indexLoadingConfig.getNoDictionaryColumns().remove(DIM_ZSTANDARD_STRING);
-    fwdIndexHandler = new ForwardIndexHandler(existingSegmentMetadata, indexLoadingConfig);
+    fwdIndexHandler = new ForwardIndexHandler(existingSegmentMetadata, indexLoadingConfig, _schema);
+    operationMap = fwdIndexHandler.computeOperation(writer);
+    assertEquals(operationMap.get(DIM_ZSTANDARD_STRING), ForwardIndexHandler.Operation.ENABLE_DICTIONARY);
+
+    // TEST3: Enable dictionary for an MV column.
+    indexLoadingConfig = new IndexLoadingConfig(null, _tableConfig);
+    indexLoadingConfig.getNoDictionaryColumns().remove(DIM_MV_PASS_THROUGH_STRING);
+    fwdIndexHandler = new ForwardIndexHandler(existingSegmentMetadata, indexLoadingConfig, _schema);
+    operationMap = fwdIndexHandler.computeOperation(writer);
+    assertEquals(operationMap.get(DIM_MV_PASS_THROUGH_STRING), ForwardIndexHandler.Operation.ENABLE_DICTIONARY);
+
+    // TEST4: Enable dictionary for a sorted column.
+    indexLoadingConfig = new IndexLoadingConfig(null, _tableConfig);
+    indexLoadingConfig.getNoDictionaryColumns().remove(DIM_PASS_THROUGH_SORTED_LONG);
+    fwdIndexHandler = new ForwardIndexHandler(existingSegmentMetadata, indexLoadingConfig, _schema);
+    operationMap = fwdIndexHandler.computeOperation(writer);
+    assertEquals(operationMap.get(DIM_PASS_THROUGH_SORTED_LONG), ForwardIndexHandler.Operation.ENABLE_DICTIONARY);
+
+    // TEST5: Enable dictionary for a dict column. Should be a No-op.
+    indexLoadingConfig = new IndexLoadingConfig(null, _tableConfig);
+    fwdIndexHandler = new ForwardIndexHandler(existingSegmentMetadata, indexLoadingConfig, _schema);
     operationMap = fwdIndexHandler.computeOperation(writer);
     assertEquals(operationMap, Collections.EMPTY_MAP);
 
-    // TEST3: Disable dictionary. ForwardIndexHandler should be a No-Op.
+    // TEST6: Disable dictionary. Should be a No-op.
     indexLoadingConfig = new IndexLoadingConfig(null, _tableConfig);
     indexLoadingConfig.getNoDictionaryColumns().add(DIM_DICT_INTEGER);
-    fwdIndexHandler = new ForwardIndexHandler(existingSegmentMetadata, indexLoadingConfig);
+    fwdIndexHandler = new ForwardIndexHandler(existingSegmentMetadata, indexLoadingConfig, _schema);
     operationMap = fwdIndexHandler.computeOperation(writer);
     assertEquals(operationMap, Collections.EMPTY_MAP);
 
-    // TEST4: Add an additional text index. ForwardIndexHandler should be a No-Op.
+    // TEST7: Add an additional text index. ForwardIndexHandler should be a No-Op.
     indexLoadingConfig = new IndexLoadingConfig(null, _tableConfig);
     indexLoadingConfig.getTextIndexColumns().add(DIM_DICT_INTEGER);
     indexLoadingConfig.getTextIndexColumns().add(DIM_LZ4_INTEGER);
-    fwdIndexHandler = new ForwardIndexHandler(existingSegmentMetadata, indexLoadingConfig);
+    fwdIndexHandler = new ForwardIndexHandler(existingSegmentMetadata, indexLoadingConfig, _schema);
     operationMap = fwdIndexHandler.computeOperation(writer);
     assertEquals(operationMap, Collections.EMPTY_MAP);
 
-    // TEST5: Change compression
+    // TEST8: Add text index and disable forward index.

Review Comment:
   I think this comment is misleading. You mean to say `"Add text index and enable 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.

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


[GitHub] [pinot] codecov-commenter commented on pull request #9678: Support creating dictionary at runtime for an existing column

Posted by GitBox <gi...@apache.org>.
codecov-commenter commented on PR #9678:
URL: https://github.com/apache/pinot/pull/9678#issuecomment-1297597493

   # [Codecov](https://codecov.io/gh/apache/pinot/pull/9678?src=pr&el=h1&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) Report
   > Merging [#9678](https://codecov.io/gh/apache/pinot/pull/9678?src=pr&el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (c2ba236) into [master](https://codecov.io/gh/apache/pinot/commit/63c6438d3cd3f31983edc8324325a0e268332f37?el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (63c6438) will **decrease** coverage by `0.30%`.
   > The diff coverage is `81.00%`.
   
   ```diff
   @@             Coverage Diff              @@
   ##             master    #9678      +/-   ##
   ============================================
   - Coverage     68.71%   68.41%   -0.31%     
   + Complexity     4971     4881      -90     
   ============================================
     Files          1939     1950      +11     
     Lines        103671   104417     +746     
     Branches      15727    15816      +89     
   ============================================
   + Hits          71242    71441     +199     
   - Misses        27350    27869     +519     
   - Partials       5079     5107      +28     
   ```
   
   | Flag | Coverage Δ | |
   |---|---|---|
   | integration1 | `25.29% <0.00%> (-0.50%)` | :arrow_down: |
   | unittests1 | `67.50% <81.00%> (+0.12%)` | :arrow_up: |
   | unittests2 | `15.68% <0.00%> (+0.08%)` | :arrow_up: |
   
   Flags with carried forward coverage won't be shown. [Click here](https://docs.codecov.io/docs/carryforward-flags?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#carryforward-flags-in-the-pull-request-comment) to find out more.
   
   | [Impacted Files](https://codecov.io/gh/apache/pinot/pull/9678?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | Coverage Δ | |
   |---|---|---|
   | [...ocal/segment/index/loader/ForwardIndexHandler.java](https://codecov.io/gh/apache/pinot/pull/9678/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3Qtc2VnbWVudC1sb2NhbC9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3Qvc2VnbWVudC9sb2NhbC9zZWdtZW50L2luZGV4L2xvYWRlci9Gb3J3YXJkSW5kZXhIYW5kbGVyLmphdmE=) | `83.14% <79.76%> (+12.14%)` | :arrow_up: |
   | [...t/creator/impl/SegmentIndexCreationDriverImpl.java](https://codecov.io/gh/apache/pinot/pull/9678/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3Qtc2VnbWVudC1sb2NhbC9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3Qvc2VnbWVudC9sb2NhbC9zZWdtZW50L2NyZWF0b3IvaW1wbC9TZWdtZW50SW5kZXhDcmVhdGlvbkRyaXZlckltcGwuamF2YQ==) | `81.52% <100.00%> (-0.20%)` | :arrow_down: |
   | [...ocal/segment/index/loader/IndexHandlerFactory.java](https://codecov.io/gh/apache/pinot/pull/9678/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3Qtc2VnbWVudC1sb2NhbC9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3Qvc2VnbWVudC9sb2NhbC9zZWdtZW50L2luZGV4L2xvYWRlci9JbmRleEhhbmRsZXJGYWN0b3J5LmphdmE=) | `100.00% <100.00%> (ø)` | |
   | [...ocal/segment/index/loader/SegmentPreProcessor.java](https://codecov.io/gh/apache/pinot/pull/9678/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3Qtc2VnbWVudC1sb2NhbC9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3Qvc2VnbWVudC9sb2NhbC9zZWdtZW50L2luZGV4L2xvYWRlci9TZWdtZW50UHJlUHJvY2Vzc29yLmphdmE=) | `83.49% <100.00%> (+0.66%)` | :arrow_up: |
   | [...pinot/core/data/manager/realtime/TimerService.java](https://codecov.io/gh/apache/pinot/pull/9678/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29yZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29yZS9kYXRhL21hbmFnZXIvcmVhbHRpbWUvVGltZXJTZXJ2aWNlLmphdmE=) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [...ent/creator/IntermediateSegmentStatsContainer.java](https://codecov.io/gh/apache/pinot/pull/9678/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3Qtc2VnbWVudC1sb2NhbC9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3Qvc2VnbWVudC9sb2NhbC9zZWdtZW50L2NyZWF0b3IvSW50ZXJtZWRpYXRlU2VnbWVudFN0YXRzQ29udGFpbmVyLmphdmE=) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [...lugin/stream/kafka20/KafkaStreamLevelConsumer.java](https://codecov.io/gh/apache/pinot/pull/9678/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtcGx1Z2lucy9waW5vdC1zdHJlYW0taW5nZXN0aW9uL3Bpbm90LWthZmthLTIuMC9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvcGx1Z2luL3N0cmVhbS9rYWZrYTIwL0thZmthU3RyZWFtTGV2ZWxDb25zdW1lci5qYXZh) | `0.00% <0.00%> (-88.34%)` | :arrow_down: |
   | [...data/manager/realtime/DefaultSegmentCommitter.java](https://codecov.io/gh/apache/pinot/pull/9678/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29yZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29yZS9kYXRhL21hbmFnZXIvcmVhbHRpbWUvRGVmYXVsdFNlZ21lbnRDb21taXR0ZXIuamF2YQ==) | `0.00% <0.00%> (-80.00%)` | :arrow_down: |
   | [...manager/realtime/HLRealtimeSegmentDataManager.java](https://codecov.io/gh/apache/pinot/pull/9678/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29yZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29yZS9kYXRhL21hbmFnZXIvcmVhbHRpbWUvSExSZWFsdGltZVNlZ21lbnREYXRhTWFuYWdlci5qYXZh) | `0.00% <0.00%> (-74.08%)` | :arrow_down: |
   | [...n/stream/kafka20/KafkaStreamLevelStreamConfig.java](https://codecov.io/gh/apache/pinot/pull/9678/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtcGx1Z2lucy9waW5vdC1zdHJlYW0taW5nZXN0aW9uL3Bpbm90LWthZmthLTIuMC9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvcGx1Z2luL3N0cmVhbS9rYWZrYTIwL0thZmthU3RyZWFtTGV2ZWxTdHJlYW1Db25maWcuamF2YQ==) | `0.00% <0.00%> (-68.75%)` | :arrow_down: |
   | ... and [169 more](https://codecov.io/gh/apache/pinot/pull/9678/diff?src=pr&el=tree-more&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | |
   
   :mega: We’re building smart automated test selection to slash your CI/CD build times. [Learn more](https://about.codecov.io/iterative-testing/?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   


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