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/09/07 18:03:08 UTC

[GitHub] [pinot] jackjlli commented on a diff in pull request #9333: Add an option to disable the creation of the forward index for a column

jackjlli commented on code in PR #9333:
URL: https://github.com/apache/pinot/pull/9333#discussion_r965125709


##########
pinot-segment-local/src/test/resources/data/newColumnsSchemaWithForwardIndexDisabled.json:
##########
@@ -0,0 +1,77 @@
+{
+  "schemaName": "testDataMV",
+  "dimensionFieldSpecs": [
+    {
+      "name": "column1",
+      "dataType": "INT"
+    },
+    {
+      "name": "column2",
+      "dataType": "INT"
+    },
+    {
+      "name": "column3",
+      "dataType": "STRING"
+    },
+    {
+      "name": "column4",
+      "dataType": "STRING"
+    },
+    {
+      "name": "column5",
+      "dataType": "STRING"
+    },
+    {
+      "name": "newForwardIndexDisabledColumnSV",
+      "dataType": "STRING"
+    },
+    {
+      "name": "newForwardIndexDisabledColumnMV",
+      "dataType": "STRING",
+      "singleValueField": false
+    },
+    {
+      "name": "column6",
+      "dataType": "INT",
+      "singleValueField": false
+    },
+    {
+      "name": "column7",
+      "dataType": "INT",
+      "singleValueField": false
+    },
+    {
+      "name": "column8",
+      "dataType": "INT"
+    },
+    {
+      "name": "column9",
+      "dataType": "INT"
+    },
+    {
+      "name": "column10",
+      "dataType": "INT"
+    },
+    {
+      "name": "column13",
+      "dataType": "INT"
+    },
+    {
+      "name": "weeksSinceEpochSunday",
+      "dataType": "INT"
+    }
+  ],
+  "metricFieldSpecs": [
+    {
+      "name": "count",
+      "dataType": "INT"
+    }
+  ],
+  "timeFieldSpec": {
+    "incomingGranularitySpec": {
+      "timeType": "DAYS",
+      "dataType": "INT",
+      "name": "daysSinceEpoch"
+    }
+  }
+}

Review Comment:
   nit: missing a tail empty line.



##########
pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/loader/invertedindex/H3IndexHandler.java:
##########
@@ -155,18 +155,30 @@ private void handleDictionaryBasedColumn(SegmentDirectory.Writer segmentWriter,
       throws IOException {
     File indexDir = _segmentMetadata.getIndexDir();
     String columnName = columnMetadata.getColumnName();
-    try (ForwardIndexReader forwardIndexReader = LoaderUtils.getForwardIndexReader(segmentWriter, columnMetadata);
-        ForwardIndexReaderContext readerContext = forwardIndexReader.createContext();
-        Dictionary dictionary = LoaderUtils.getDictionary(segmentWriter, columnMetadata);
+    int numDocs = columnMetadata.getTotalDocs();
+    try (Dictionary dictionary = LoaderUtils.getDictionary(segmentWriter, columnMetadata);
         GeoSpatialIndexCreator h3IndexCreator = indexCreatorProvider.newGeoSpatialIndexCreator(
             IndexCreationContext.builder().withIndexDir(indexDir).withColumnMetadata(columnMetadata)
                 .build().forGeospatialIndex(_h3Configs.get(columnName)))) {
-      int numDocs = columnMetadata.getTotalDocs();
-      for (int i = 0; i < numDocs; i++) {
-        int dictId = forwardIndexReader.getDictId(i, readerContext);
-        h3IndexCreator.add(GeometrySerializer.deserialize(dictionary.getBytesValue(dictId)));
+      if (columnMetadata.forwardIndexDisabled()) {
+        // Create the h3 index if the dictionary length is 1 as this is for a default column (i.e. newly added
+        // column). For existing columns it is not possible to create the h3 index without forward index
+        Preconditions.checkState(dictionary.length() == 1,
+            "Creating h3 index for forward index disabled default column, dictionary size must be 1");

Review Comment:
   Same here, print the column name in the message.



##########
pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/creator/IndexCreationContext.java:
##########
@@ -77,6 +79,7 @@ final class Builder {
     private boolean _hasDictionary = true;
     private Comparable<?> _minValue;
     private Comparable<?> _maxValue;
+    private boolean _forwardIndexDisabled = false;

Review Comment:
   IIRC, instance and class variables don't require us to initialize them. The default value of a boolean variable is false. So you don't need to assign a `false` here.



##########
pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/creator/impl/SegmentColumnarIndexCreator.java:
##########
@@ -196,6 +204,14 @@ public void init(SegmentGeneratorConfig segmentCreationSpec, SegmentIndexCreatio
       Preconditions.checkState(dictEnabledColumn || !invertedIndexColumns.contains(columnName),
           "Cannot create inverted index for raw index column: %s", columnName);
 
+      boolean forwardIndexDisabled = forwardIndexDisabledColumns.contains(columnName);
+      if (forwardIndexDisabled) {
+        Preconditions.checkState(dictEnabledColumn && invertedIndexColumns.contains(columnName)
+            && !columnIndexCreationInfo.isSorted() && (!rangeIndexColumns.contains(columnName)
+                || rangeIndexVersion == BitSlicedRangeIndexCreator.VERSION), "Cannot disable forward index for column "
+            + "without dictionary and inverted index or which is sorted or which has range index with version < 2");

Review Comment:
   nit: it'd be good to print the column name in this message?



##########
pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/datasource/MutableDataSource.java:
##########
@@ -106,6 +106,12 @@ public int getMaxNumValuesPerMVEntry() {
       return _maxNumValuesPerMVEntry;
     }
 
+    @Override
+    public boolean forwardIndexDisabled() {
+      // Disabling the forward index is only supported for offline segments.

Review Comment:
   > for offline segments
   
   Do you mean immutable segments including offline table segments and realtime completed segments, or just offline table segments?



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