Copilot commented on code in PR #17269:
URL: https://github.com/apache/pinot/pull/17269#discussion_r2608914511


##########
pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/loader/ForwardIndexHandler.java:
##########
@@ -162,13 +167,19 @@ public void updateIndices(SegmentDirectory.Writer 
segmentWriter)
                 
FieldIndexConfigsUtil.columnsWithIndexDisabled(_fieldIndexConfigs.keySet(), 
StandardIndexes.forward(),
                     _fieldIndexConfigs);
             if (newForwardIndexDisabledColumns.contains(column)) {
-              removeDictionaryFromForwardIndexDisabledColumn(column, 
segmentWriter);
+              disableDictionary(column, segmentWriter, "Disable dictionary 
when only no forward index exists");
               if (segmentWriter.hasIndexFor(column, 
StandardIndexes.dictionary())) {
                 throw new IllegalStateException(
                     String.format("Dictionary should not exist after disabling 
dictionary for column: %s", column));
               }
             } else {
-              disableDictionaryAndCreateRawForwardIndex(column, segmentWriter);
+              FieldIndexConfigs fieldIndexConfig = 
_fieldIndexConfigs.get(column);
+              ForwardIndexConfig forwardIndexConfig = 
fieldIndexConfig.getConfig(StandardIndexes.forward());
+              if (forwardIndexConfig.isRawEncoding()) {
+                disableDictionary(column, segmentWriter, "Disable dictionary 
when only raw forward index exists");
+              } else {
+                disableDictionaryAndCreateRawForwardIndex(column, 
segmentWriter);
+              }
             }

Review Comment:
   The nested if-else chain starting at line 169 is complex and could benefit 
from early returns or extraction into separate methods. The logic handling 
DISABLE_DICTIONARY has three distinct paths (no forward index, raw forward 
index, and dictionary forward index) that would be clearer with guard clauses.
   ```suggestion
                 break;
               }
   
               FieldIndexConfigs fieldIndexConfig = 
_fieldIndexConfigs.get(column);
               ForwardIndexConfig forwardIndexConfig = 
fieldIndexConfig.getConfig(StandardIndexes.forward());
               if (forwardIndexConfig.isRawEncoding()) {
                 disableDictionary(column, segmentWriter, "Disable dictionary 
when only raw forward index exists");
                 break;
               }
   
               disableDictionaryAndCreateRawForwardIndex(column, segmentWriter);
   ```



##########
pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/creator/impl/SegmentColumnarIndexCreator.java:
##########
@@ -61,6 +64,28 @@ public void init(SegmentGeneratorConfig segmentCreationSpec, 
SegmentIndexCreatio
         segmentCreationSpec.getIndexConfigsByColName().size());
     initializeCommon(segmentCreationSpec, segmentIndexCreationInfo, 
indexCreationInfoMap,
         schema, outDir, colIndexes, immutableToMutableIdMap, instanceType);
+
+    // Although NullValueVector is implemented as an index, it needs to be 
treated in a different way than other indexes
+    // Process all field specs (not just indexed columns) for null value 
vectors
+    for (FieldSpec fieldSpec : schema.getAllFieldSpecs()) {
+      String columnName = fieldSpec.getName();
+      // Create null value vector for non-indexed nullable columns
+      if (!_colIndexes.containsKey(columnName)) {
+        if (isNullable(fieldSpec, schema, segmentCreationSpec)) {
+          // Initialize Null value vector map
+          LOGGER.info("Column: {} is nullable", columnName);
+          _colIndexes.put(columnName,
+              new ColumnIndexCreators(columnName, fieldSpec, null, List.of(),
+                  new NullValueVectorCreator(outDir, columnName)));
+        } else {
+          LOGGER.info("Column: {} is not nullable", columnName);

Review Comment:
   The added null value vector handling logs both nullable and non-nullable 
columns at INFO level. This could create excessive logging for tables with many 
columns. Consider logging only nullable columns, or reducing non-nullable 
logging to DEBUG level.
   ```suggestion
             LOGGER.debug("Column: {} is not nullable", columnName);
   ```



##########
pinot-segment-local/src/test/java/org/apache/pinot/segment/local/segment/index/loader/SegmentPreProcessorTest.java:
##########
@@ -581,13 +584,17 @@ public void testDisableDictAndOtherIndexesSV()
     long oldRangeIndexSize =
         new 
SegmentMetadataImpl(INDEX_DIR).getColumnMetadataFor(COLUMN10_NAME).getIndexSizeFor(StandardIndexes.range());
     _noDictionaryColumns.add(COLUMN10_NAME);
-    checkForwardIndexCreation(COLUMN10_NAME, 3960, 12, _schema, false, false, 
false, 0, ChunkCompressionType.LZ4, true,
-        0, DataType.INT, 100000);
-    validateIndex(StandardIndexes.range(), COLUMN10_NAME, 3960, 12, false, 
false, false, 0, true, 0,
-        ChunkCompressionType.LZ4, false, DataType.INT, 100000);
-    long newRangeIndexSize =
-        new 
SegmentMetadataImpl(INDEX_DIR).getColumnMetadataFor(COLUMN10_NAME).getIndexSizeFor(StandardIndexes.range());
-    assertNotEquals(oldRangeIndexSize, newRangeIndexSize);
+    try {
+      checkForwardIndexCreation(COLUMN10_NAME, 3960, 12, _schema, false, 
false, false, 0, ChunkCompressionType.LZ4,
+          true, 0, DataType.INT, 100000);
+      validateIndex(StandardIndexes.range(), COLUMN10_NAME, 3960, 12, false, 
false, false, 0, true, 0,
+          ChunkCompressionType.LZ4, false, DataType.INT, 100000);
+      long newRangeIndexSize = new 
SegmentMetadataImpl(INDEX_DIR).getColumnMetadataFor(COLUMN10_NAME)
+          .getIndexSizeFor(StandardIndexes.range());
+      assertNotEquals(oldRangeIndexSize, newRangeIndexSize);
+    } catch (RuntimeException e) {
+      // Forward-index-disabled columns cannot rebuild range index without an 
existing forward index; accept failure.
+    }

Review Comment:
   Empty catch block swallows RuntimeException without logging or assertion. 
This makes test failures silent and hard to debug. Consider either asserting 
specific exception conditions, logging the caught exception, or restructuring 
the test to explicitly handle both success and failure paths with clear 
assertions.



##########
pinot-core/src/test/java/org/apache/pinot/queries/ForwardIndexHandlerReloadQueriesTest.java:
##########
@@ -621,40 +661,41 @@ public void testRangeIndexAfterReload()
     validateBeforeAfterQueryResults(resultRows1, resultRows2);
   }
 
+  @Test
+  public void testRawForwardColumnIndexAddAndRemove()
+      throws Exception {
+    String filterValue = escapeSingleQuotes(getMostFrequentColumn5Value());
+    String filterQuery = String.format("SELECT COUNT(*) FROM %s WHERE column5 
= '%s'", RAW_TABLE_NAME, filterValue);
+
+    BrokerResponseNative baselineResponse = getBrokerResponse(filterQuery);
+    assertTrue(baselineResponse.getExceptions() == null || 
baselineResponse.getExceptions().size() == 0);
+    ResultTable baselineResultTable = baselineResponse.getResultTable();
+    assertNotNull(baselineResultTable);
+    List<Object[]> baselineRows = baselineResultTable.getRows();
+    long baselineEntriesScanned = 
baselineResponse.getNumEntriesScannedInFilter();
+    assertTrue(baselineEntriesScanned > 0);
+  }

Review Comment:
   The test method testRawForwardColumnIndexAddAndRemove() establishes a 
baseline but doesn't perform any index add/remove operations or validate 
behavior changes. The test appears incomplete and should either be removed or 
expanded to test the full add/remove cycle it claims to test.



##########
pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/forward/ForwardIndexType.java:
##########
@@ -235,19 +246,22 @@ protected IndexReaderFactory<ForwardIndexReader> 
createReaderFactory() {
     return ForwardIndexReaderFactory.getInstance();
   }
 
-  public String getFileExtension(ColumnMetadata columnMetadata) {
+  public List<String> getFileExtension(ColumnMetadata columnMetadata) {
     if (columnMetadata.isSingleValue()) {
       if (!columnMetadata.hasDictionary()) {
-        return V1Constants.Indexes.RAW_SV_FORWARD_INDEX_FILE_EXTENSION;
+        return 
List.of(V1Constants.Indexes.RAW_SV_FORWARD_INDEX_FILE_EXTENSION);
       } else if (columnMetadata.isSorted()) {
-        return V1Constants.Indexes.SORTED_SV_FORWARD_INDEX_FILE_EXTENSION;
+        return List.of(V1Constants.Indexes.RAW_SV_FORWARD_INDEX_FILE_EXTENSION,
+            V1Constants.Indexes.SORTED_SV_FORWARD_INDEX_FILE_EXTENSION);
       } else {
-        return V1Constants.Indexes.UNSORTED_SV_FORWARD_INDEX_FILE_EXTENSION;
+        return List.of(V1Constants.Indexes.RAW_SV_FORWARD_INDEX_FILE_EXTENSION,
+            V1Constants.Indexes.UNSORTED_SV_FORWARD_INDEX_FILE_EXTENSION);
       }
     } else if (!columnMetadata.hasDictionary()) {
-      return V1Constants.Indexes.RAW_MV_FORWARD_INDEX_FILE_EXTENSION;
+      return List.of(V1Constants.Indexes.RAW_MV_FORWARD_INDEX_FILE_EXTENSION);
     } else {
-      return V1Constants.Indexes.UNSORTED_MV_FORWARD_INDEX_FILE_EXTENSION;
+      return List.of(V1Constants.Indexes.RAW_MV_FORWARD_INDEX_FILE_EXTENSION,
+          V1Constants.Indexes.UNSORTED_MV_FORWARD_INDEX_FILE_EXTENSION);
     }

Review Comment:
   The logic for returning file extensions now includes 
RAW_SV_FORWARD_INDEX_FILE_EXTENSION in all dictionary-enabled single-value 
cases (lines 254-258), even when sorted or unsorted. This seems inconsistent 
with the multi-value logic (lines 260-264). Verify that RAW extensions should 
indeed be included for dictionary-encoded columns, as this appears to 
contradict the raw-encoding concept.



##########
pinot-segment-local/src/test/java/org/apache/pinot/segment/local/segment/index/loader/SegmentPreProcessorTest.java:
##########
@@ -341,30 +342,32 @@ public SegmentVersion[][] bothV1AndV3() {
    * (1) Default column handler to add forward index and dictionary
    * (2) Text index handler to add text index
    */
-  @Test(dataProvider = "bothV1AndV3")
+  @Test(dataProvider = "bothV1AndV3", expectedExceptions = 
RuntimeException.class)
   public void testEnableTextIndexOnNewColumnRaw(SegmentVersion segmentVersion)
       throws Exception {
     buildSegment(segmentVersion);
     _fieldConfigMap.put(NEWLY_ADDED_STRING_COL_RAW,
         new FieldConfig(NEWLY_ADDED_STRING_COL_RAW, 
FieldConfig.EncodingType.RAW, List.of(FieldConfig.IndexType.TEXT),
             null, null));
-    _fieldConfigMap.put(NEWLY_ADDED_STRING_MV_COL_RAW,
-        new FieldConfig(NEWLY_ADDED_STRING_MV_COL_RAW, 
FieldConfig.EncodingType.RAW,
-            List.of(FieldConfig.IndexType.TEXT), null, null));
     checkTextIndexCreation(NEWLY_ADDED_STRING_COL_RAW, 1, 1, 
_newColumnsSchemaWithText, true, true, true, 4);

Review Comment:
   Test expects RuntimeException but doesn't verify the exception message or 
root cause. The test should either validate the specific exception type/message 
or document why any RuntimeException is acceptable. Additionally, removed test 
coverage for NEWLY_ADDED_STRING_MV_COL_RAW should be restored or its removal 
justified.



##########
pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/custom/RawForwardIndexInvertedIndexTest.java:
##########
@@ -0,0 +1,173 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pinot.integration.tests.custom;
+
+import com.fasterxml.jackson.databind.JsonNode;
+import java.io.File;
+import java.io.IOException;
+import java.util.List;
+import java.util.Random;
+import org.apache.avro.SchemaBuilder;
+import org.apache.avro.file.DataFileWriter;
+import org.apache.avro.generic.GenericData;
+import org.apache.avro.generic.GenericDatumWriter;
+import org.apache.pinot.spi.config.table.TableConfig;
+import org.apache.pinot.spi.config.table.TableType;
+import org.apache.pinot.spi.data.FieldSpec;
+import org.apache.pinot.spi.data.Schema;
+import org.apache.pinot.spi.utils.JsonUtils;
+import org.apache.pinot.spi.utils.builder.TableConfigBuilder;
+import org.testng.annotations.Test;
+
+import static org.testng.Assert.assertEquals;
+import static org.testng.Assert.assertNotNull;
+import static org.testng.Assert.assertTrue;
+
+
+/**
+ * Integration test that verifies inverted indexes can be built for 
raw-forward columns (no-dictionary forward index)
+ * and that queries can leverage those indexes.
+ */
+@Test(suiteName = "CustomClusterIntegrationTest")
+public class RawForwardIndexInvertedIndexTest extends 
CustomDataQueryClusterIntegrationTest {
+  private static final String TABLE_NAME = "RawForwardIndexInvertedIndexTest";
+  private static final String RAW_DIMENSION = "rawDim";
+  private static final String METRIC_COLUMN = "metric";
+  private static final int ROW_COUNT = 500;
+  private static final int UNIQUE_DIMENSION_VALUES = 10;
+  private static final String FILTER_VALUE = RAW_DIMENSION + "-3";
+
+  @Override
+  protected long getCountStarResult() {
+    return ROW_COUNT;
+  }
+
+  @Override
+  public String getTableName() {
+    return TABLE_NAME;
+  }
+
+  @Override
+  public TableConfig createOfflineTableConfig() {
+    return new 
TableConfigBuilder(TableType.OFFLINE).setTableName(getTableName())
+        
.setTimeColumnName(getTimeColumnName()).setSortedColumn(getSortedColumn())
+        .setInvertedIndexColumns(getInvertedIndexColumns())
+        
.setCreateInvertedIndexDuringSegmentGeneration(isCreateInvertedIndexDuringSegmentGeneration())
+        
.setNoDictionaryColumns(getNoDictionaryColumns()).setRangeIndexColumns(getRangeIndexColumns())
+        
.setBloomFilterColumns(getBloomFilterColumns()).setFieldConfigList(getFieldConfigs())
+        
.setNumReplicas(getNumReplicas()).setSegmentVersion(getSegmentVersion()).setLoadMode(getLoadMode())
+        
.setTaskConfig(getTaskConfig()).setBrokerTenant(getBrokerTenant()).setServerTenant(getServerTenant())
+        
.setIngestionConfig(getIngestionConfig()).setQueryConfig(getQueryConfig())
+        
.setNullHandlingEnabled(getNullHandlingEnabled()).setSegmentPartitionConfig(getSegmentPartitionConfig())
+        .setOptimizeNoDictStatsCollection(true).build();
+  }
+
+  @Override
+  public Schema createSchema() {
+    return new Schema.SchemaBuilder().setSchemaName(getTableName())
+        .addSingleValueDimension(RAW_DIMENSION, FieldSpec.DataType.STRING)
+        .addMetric(METRIC_COLUMN, FieldSpec.DataType.LONG)
+        .addDateTime(TIMESTAMP_FIELD_NAME, FieldSpec.DataType.LONG, 
"1:MILLISECONDS:EPOCH", "1:MILLISECONDS").build();
+  }
+
+  @Override
+  public List<File> createAvroFiles()
+      throws Exception {
+    org.apache.avro.Schema avroSchema = 
SchemaBuilder.record("RawForwardIndexRecord").fields()
+        .requiredString(RAW_DIMENSION)
+        .requiredLong(METRIC_COLUMN)
+        .requiredLong(TIMESTAMP_FIELD_NAME)
+        .endRecord();
+
+    File avroFile = new File(_tempDir, "raw-forward-inverted-index.avro");
+    try (DataFileWriter<GenericData.Record> fileWriter = new 
DataFileWriter<>(new GenericDatumWriter<>(avroSchema))) {
+      fileWriter.create(avroSchema, avroFile);
+      Random random = new Random(1234);
+      long currentTimeMillis = System.currentTimeMillis();
+      for (int i = 0; i < ROW_COUNT; i++) {
+        GenericData.Record record = new GenericData.Record(avroSchema);
+        record.put(RAW_DIMENSION, RAW_DIMENSION + "-" + (i % 
UNIQUE_DIMENSION_VALUES));
+        record.put(METRIC_COLUMN, random.nextInt(10_000));
+        record.put(TIMESTAMP_FIELD_NAME, currentTimeMillis + i);
+        fileWriter.append(record);
+      }
+    }
+    return List.of(avroFile);
+  }
+
+  @Override
+  protected List<String> getNoDictionaryColumns() {
+    return List.of(RAW_DIMENSION);
+  }
+
+  @Override
+  protected List<String> getInvertedIndexColumns() {
+    return List.of(RAW_DIMENSION);
+  }
+
+  @Override
+  protected boolean isCreateInvertedIndexDuringSegmentGeneration() {
+    return true;
+  }
+
+  @Test(dataProvider = "useBothQueryEngines")
+  public void testInvertedIndexOnRawForwardColumn(boolean 
useMultiStageQueryEngine)
+      throws Exception {
+    setUseMultiStageQueryEngine(useMultiStageQueryEngine);
+    String query =
+        String.format("SELECT COUNT(*) FROM %s WHERE %s = '%s'", 
getTableName(), RAW_DIMENSION, FILTER_VALUE);
+    JsonNode response = postQuery(query);
+    long matchedCount = 
response.get("resultTable").get("rows").get(0).get(0).asLong();
+    assertEquals(matchedCount, ROW_COUNT / UNIQUE_DIMENSION_VALUES);
+    assertEquals(response.get("numEntriesScannedInFilter").asLong(), 0L,
+        "Inverted index should avoid scanning filters even for raw forward 
index");

Review Comment:
   The assertion verifies that numEntriesScannedInFilter is 0, expecting the 
inverted index to eliminate filter scanning. However, the test doesn't verify 
that the inverted index was actually used. Consider adding an assertion to 
check query plan or execution statistics that confirm inverted index usage.



-- 
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: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]


---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to