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/03 00:05:03 UTC

[GitHub] [pinot] somandal opened a new pull request, #9333: Add an option to disable the creation of the forward index for a column

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

   This PR adds an option to disable the forward index for a given column via the FieldConfig properties list. This is a PR to solve issue https://github.com/apache/pinot/issues/6473. This PR does not allow the forward index to be disabled yet by adding a check in the `TableConfig`. This feature will be allowed once we address the reload code path to toggle the flag to disable the forward index.
   
   To disable the forward index, the following mandates have been added at the moment (depending on the usage of this feature we may decide to relax some of them over time):
   
   - The inverted index for the column must be enabled
   - The column must be a dictionary based column
   - The column cannot be sorted
   - The column cannot have a V1 type range index present
   
   The above checks have been added in the code in the following places:
   
   - TableConfig validator
   - Segment reload path (check when creating the default column)
   - Segment creation path
   
   
   The ability to disable the forward index is currently implemented for the following cases:
   
   - New immutable segments generated for a column (all refresh use cases and reload use cases for new segments)
   - Add a default value column with the forward index disabled
   
   This PR does **not** add support for disabling the forward index for:
   
   - Mutable segments
   - Enabling / disabling the disable forward index flag for an existing column -> we will add support for this reload case in a follow up PR shortly after this PR
   
   
   Validations with other indices can be broadly classified into the following paths:
   
   - Query path - ensure that none of the other indices rely on the forward index
       - We found the range index v1 resorts to a scan on the forward index, thus disabled this feature if a range index of version 1 is present
       - Sorted columns rely on the forward index as the inverted index as well. We disabled this feature if the column is sorted
   - Segment creation path does not rely on the forward index to create other indices. The `indexRow` function creates all the indices needed at the same time for a given row.
   - Segment reload path relies on the forward index to add new indices such as range index, text index, etc. This path cannot be supported until we address enabling/disabling this feature on the reload path and will be done as part of a future PR along with any additional validations needed.
   
   cc @siddharthteotia @Jackie-Jiang 


-- 
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 #9333: Add an option to disable the creation of the forward index for a column

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


##########
pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/column/PhysicalColumnIndexContainer.java:
##########
@@ -142,6 +145,8 @@ public PhysicalColumnIndexContainer(SegmentDirectory.Reader segmentReader, Colum
         // Single-value
         if (metadata.isSorted()) {
           // Sorted
+          // forwardIndexDisabled columns do not need to be handled here as forward index cannot be disabled on a

Review Comment:
   May be I am mis-reading the `if-else` block here. For an unsorted dict column with inv index, are we ok with line 157 being executed with null `fwdIndexBuffer` ?



-- 
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 #9333: Add an option to disable the creation of the forward index for a column

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


##########
pinot-segment-local/src/main/java/org/apache/pinot/segment/local/utils/TableConfigUtils.java:
##########
@@ -869,6 +871,25 @@ private static void validateFieldConfigList(@Nullable List<FieldConfig> fieldCon
           default:
             break;
         }
+
+        // Validate the forward index disabled compatibility if enabled for this column

Review Comment:
   (nit) put this in a helper method ?



-- 
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 #9333: Add an option to disable the creation of the forward index for a column

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


##########
pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/column/PhysicalColumnIndexContainer.java:
##########
@@ -133,7 +133,12 @@ public PhysicalColumnIndexContainer(SegmentDirectory.Reader segmentReader, Colum
       _rangeIndex = null;
     }
 
-    PinotDataBuffer fwdIndexBuffer = segmentReader.getIndexFor(columnName, ColumnIndexType.FORWARD_INDEX);
+    // Setting the 'fwdIndexBuffer' to null if forward index is enabled. No-op index readers will be setup for the
+    // forward index disabled columns which doesn't require the 'fwdIndexBuffer'.
+    boolean forwardIndexDisabled = !segmentReader.hasIndexFor(columnName, ColumnIndexType.FORWARD_INDEX);
+    PinotDataBuffer fwdIndexBuffer = forwardIndexDisabled ? null
+        : segmentReader.getIndexFor(columnName, ColumnIndexType.FORWARD_INDEX);

Review Comment:
   Can we replace the 2 calls with just 1 call to `getIndexFor` ?



-- 
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 #9333: Add an option to disable the creation of the forward index for a column

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


##########
pinot-tools/src/main/java/org/apache/pinot/tools/segment/converter/DictionaryToRawIndexConverter.java:
##########
@@ -296,6 +297,10 @@ private void convertOneColumn(IndexSegment segment, String column, File newSegme
       throws IOException {
     DataSource dataSource = segment.getDataSource(column);
     ForwardIndexReader reader = dataSource.getForwardIndex();
+    if (reader == null) {
+      throw new UnsupportedEncodingException(String.format("Forward index is disabled for column %s, cannot convert!",

Review Comment:
   oops, that was meant to be an `UnsupportedOperationException`. I've modified all of these to use the Preconditions.checkState() instead.



##########
pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/loader/IndexLoadingConfig.java:
##########
@@ -315,6 +341,19 @@ public List<String> getSortedColumns() {
     return _sortedColumns;
   }
 
+  /**
+   * For tests only.
+   */
+  @VisibleForTesting
+  public void setSortedColumns(String sortedColumn) {

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 #9333: Add an option to disable the creation of the forward index for a column

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


##########
pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/loader/defaultcolumn/BaseDefaultColumnHandler.java:
##########
@@ -388,8 +416,40 @@ protected boolean createColumnV1Indices(String column)
     return true;
   }
 
+  /**
+   * Validates the compatibility of the indexes if the column has the forward index disabled. Throws exceptions due to
+   * compatibility mismatch. The checks performed are:
+   *     - Validate dictionary is enabled.
+   *     - Validate inverted index is enabled.
+   *     - Validate that either no range index exists for column or the range index version is at least 2 and isn't a
+   *       multi-value column (since multi-value defaults to index v1).
+   */
+  protected void validateForwardIndexDisabledConfigsIfPresent(String column, boolean forwardIndexDisabled) {

Review Comment:
   As discussed on the other comment, added this as an additional safeguard. I can remove it though if you think this is unnecessary and the only way is to validate the table configs on the controller. e.g. sometimes we may directly change table config for incompatible schema changes in EI etc. For such cases will the controller catch it for sure?



-- 
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 #9333: Add an option to disable the creation of the forward index for a column

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


##########
pinot-core/src/test/java/org/apache/pinot/queries/ForwardIndexDisabledSingleValueQueriesTest.java:
##########
@@ -0,0 +1,1088 @@
+/**
+ * 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.queries;
+
+import java.io.File;
+import java.net.URL;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.TimeUnit;
+import org.apache.commons.io.FileUtils;
+import org.apache.pinot.common.response.broker.BrokerResponseNative;
+import org.apache.pinot.common.response.broker.ResultTable;
+import org.apache.pinot.common.utils.DataSchema;
+import org.apache.pinot.segment.local.indexsegment.immutable.ImmutableSegmentLoader;
+import org.apache.pinot.segment.local.segment.creator.impl.SegmentIndexCreationDriverImpl;
+import org.apache.pinot.segment.local.segment.index.loader.IndexLoadingConfig;
+import org.apache.pinot.segment.spi.ColumnMetadata;
+import org.apache.pinot.segment.spi.ImmutableSegment;
+import org.apache.pinot.segment.spi.IndexSegment;
+import org.apache.pinot.segment.spi.creator.SegmentGeneratorConfig;
+import org.apache.pinot.segment.spi.creator.SegmentIndexCreationDriver;
+import org.apache.pinot.spi.config.table.FieldConfig;
+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.data.TimeGranularitySpec;
+import org.apache.pinot.spi.utils.ReadMode;
+import org.apache.pinot.spi.utils.builder.TableConfigBuilder;
+import org.testng.Assert;
+import org.testng.annotations.AfterClass;
+import org.testng.annotations.BeforeClass;
+import org.testng.annotations.Test;
+
+import static org.testng.Assert.assertEquals;
+import static org.testng.Assert.assertNotNull;
+import static org.testng.Assert.assertNull;
+import static org.testng.Assert.assertTrue;
+
+
+/**
+ * The <code>ForwardIndexDisabledSingleValueQueriesTest</code> class sets up the index segment for the no forward
+ * index single-value queries test.
+ * <p>There are totally 18 columns, 30000 records inside the original Avro file where 11 columns are selected to build
+ * the index segment. Selected columns information are as following:
+ * <ul>
+ *   ColumnName, FieldType, DataType, Cardinality, IsSorted, HasInvertedIndex, FwdIndexDisabled: S1, S2, HasRangeIndex
+ *   <li>column1, METRIC, INT, 6582, F, F, F, F, F</li>
+ *   <li>column3, METRIC, INT, 21910, F, F, F, F, F</li>
+ *   <li>column5, DIMENSION, STRING, 1, T, F, F, F, F</li>
+ *   <li>column6, DIMENSION, INT, 608, F, T, T, T, T</li>
+ *   <li>column7, DIMENSION, INT, 146, F, T, T, F, F</li>
+ *   <li>column9, DIMENSION, INT, 1737, F, F, F, F, F</li>
+ *   <li>column11, DIMENSION, STRING, 5, F, T, F, F, F</li>
+ *   <li>column12, DIMENSION, STRING, 5, F, F, F, F, F</li>
+ *   <li>column17, METRIC, INT, 24, F, T, F, F, F</li>
+ *   <li>column18, METRIC, INT, 1440, F, T, F, F, F</li>
+ *   <li>daysSinceEpoch, TIME, INT, 2, T, F, F, F, F</li>
+ * </ul>
+ */
+public class ForwardIndexDisabledSingleValueQueriesTest extends BaseQueriesTest {
+  private static final String AVRO_DATA = "data" + File.separator + "test_data-sv.avro";
+  private static final String SEGMENT_NAME_1 = "testTable_126164076_167572857";
+  private static final String SEGMENT_NAME_2 = "testTable_126164076_167572858";
+  private static final File INDEX_DIR = new File(FileUtils.getTempDirectory(),
+      "ForwardIndexDisabledSingleValueQueriesTest");
+
+  private static final String SELECT_STAR_QUERY = "SELECT * FROM testTable";
+
+  // Hard-coded query filter.
+  private static final String FILTER = " WHERE column1 > 100000000"
+      + " AND column3 BETWEEN 20000000 AND 1000000000"
+      + " AND column5 = 'gFuH'"
+      + " AND (column6 < 500000000 OR column11 NOT IN ('t', 'P'))"
+      + " AND daysSinceEpoch = 126164076";
+
+  private IndexSegment _indexSegment;
+  // Contains 2 index segments, one with 2 columns with forward index disabled, and the other with just 1.
+  private List<IndexSegment> _indexSegments;
+
+  private TableConfig _tableConfig;
+  private List<String> _invertedIndexColumns;
+  private List<String> _forwardIndexDisabledColumns;
+
+  @BeforeClass
+  public void buildAndLoadSegment()
+      throws Exception {
+    FileUtils.deleteQuietly(INDEX_DIR);
+
+    // Get resource file path.
+    URL resource = getClass().getClassLoader().getResource(AVRO_DATA);
+    assertNotNull(resource);
+    String filePath = resource.getFile();
+
+    // Build the segment schema.
+    Schema schema = new Schema.SchemaBuilder().setSchemaName("testTable").addMetric("column1", FieldSpec.DataType.INT)
+        .addMetric("column3", FieldSpec.DataType.INT).addSingleValueDimension("column5", FieldSpec.DataType.STRING)
+        .addSingleValueDimension("column6", FieldSpec.DataType.INT)
+        .addSingleValueDimension("column7", FieldSpec.DataType.INT)
+        .addSingleValueDimension("column9", FieldSpec.DataType.INT)
+        .addSingleValueDimension("column11", FieldSpec.DataType.STRING)
+        .addSingleValueDimension("column12", FieldSpec.DataType.STRING).addMetric("column17", FieldSpec.DataType.INT)
+        .addMetric("column18", FieldSpec.DataType.INT)
+        .addTime(new TimeGranularitySpec(FieldSpec.DataType.INT, TimeUnit.DAYS, "daysSinceEpoch"), null).build();
+
+    createSegment(filePath, SEGMENT_NAME_1, schema);
+    createSegment(filePath, SEGMENT_NAME_2, schema);
+
+    ImmutableSegment immutableSegment1 = loadSegmentWithMetadataChecks(SEGMENT_NAME_1);
+    ImmutableSegment immutableSegment2 = loadSegmentWithMetadataChecks(SEGMENT_NAME_2);
+
+    _indexSegment = immutableSegment1;
+    _indexSegments = Arrays.asList(immutableSegment1, immutableSegment2);
+  }
+
+  private void createSegment(String filePath, String segmentName, Schema schema)
+      throws Exception {
+    // Create field configs for the no forward index columns
+    List<FieldConfig> fieldConfigList = new ArrayList<>();
+    fieldConfigList.add(new FieldConfig("column6", FieldConfig.EncodingType.DICTIONARY, Collections.emptyList(), null,
+        Collections.singletonMap(FieldConfig.FORWARD_INDEX_DISABLED, Boolean.TRUE.toString())));
+    if (segmentName.equals(SEGMENT_NAME_1)) {
+      fieldConfigList.add(new FieldConfig("column7", FieldConfig.EncodingType.DICTIONARY, Collections.emptyList(),
+          null, Collections.singletonMap(FieldConfig.FORWARD_INDEX_DISABLED, Boolean.TRUE.toString())));
+
+      // Build table config based on segment 1 as it contains both columns under no forward index
+      _tableConfig =
+          new TableConfigBuilder(TableType.OFFLINE).setTableName("testTable").setTimeColumnName("daysSinceEpoch")
+              .setFieldConfigList(fieldConfigList).setRangeIndexColumns(Arrays.asList("column6")).build();
+    }
+
+    // Create the segment generator config.
+    SegmentGeneratorConfig segmentGeneratorConfig = new SegmentGeneratorConfig(_tableConfig, schema);
+    segmentGeneratorConfig.setInputFilePath(filePath);
+    segmentGeneratorConfig.setTableName("testTable");
+    segmentGeneratorConfig.setOutDir(INDEX_DIR.getAbsolutePath());
+    segmentGeneratorConfig.setSegmentName(segmentName);
+    // The segment generation code in SegmentColumnarIndexCreator will throw
+    // exception if start and end time in time column are not in acceptable
+    // range. For this test, we first need to fix the input avro data
+    // to have the time column values in allowed range. Until then, the check
+    // is explicitly disabled
+    segmentGeneratorConfig.setSkipTimeValueCheck(true);
+    _invertedIndexColumns = Arrays.asList("column6", "column7", "column11", "column17", "column18");
+    segmentGeneratorConfig.setInvertedIndexCreationColumns(_invertedIndexColumns);
+
+    _forwardIndexDisabledColumns = new ArrayList<>(Arrays.asList("column6", "column7"));
+    segmentGeneratorConfig.setForwardIndexDisabledColumns(_forwardIndexDisabledColumns);
+    segmentGeneratorConfig.setRangeIndexCreationColumns(Arrays.asList("column6"));
+
+    // Build the index segment.
+    SegmentIndexCreationDriver driver = new SegmentIndexCreationDriverImpl();
+    driver.init(segmentGeneratorConfig);
+    driver.build();
+  }
+
+  private ImmutableSegment loadSegmentWithMetadataChecks(String segmentName)
+      throws Exception {
+    IndexLoadingConfig indexLoadingConfig = new IndexLoadingConfig();
+    indexLoadingConfig.setTableConfig(_tableConfig);
+    indexLoadingConfig.setInvertedIndexColumns(new HashSet<>(_invertedIndexColumns));
+    indexLoadingConfig.setForwardIndexDisabledColumns(new HashSet<>(_forwardIndexDisabledColumns));
+    indexLoadingConfig.setRangeIndexColumns(new HashSet<>(Arrays.asList("column6")));
+    indexLoadingConfig.setReadMode(ReadMode.heap);
+
+    ImmutableSegment immutableSegment = ImmutableSegmentLoader.load(new File(INDEX_DIR, segmentName),
+        indexLoadingConfig);
+
+    Map<String, ColumnMetadata> columnMetadataMap1 = immutableSegment.getSegmentMetadata().getColumnMetadataMap();
+    columnMetadataMap1.forEach((column, metadata) -> {
+      if (column.equals("column6") || column.equals("column7")) {
+        assertTrue(metadata.hasDictionary());
+        assertTrue(metadata.isSingleValue());
+        assertNull(immutableSegment.getForwardIndex(column));
+      } else {
+        assertNotNull(immutableSegment.getForwardIndex(column));
+      }
+    });
+
+    return immutableSegment;
+  }
+
+  @AfterClass
+  public void deleteAndDestroySegment() {
+    FileUtils.deleteQuietly(INDEX_DIR);
+    _indexSegments.forEach((IndexSegment::destroy));
+  }
+
+  @Override
+  protected String getFilter() {
+    return FILTER;
+  }
+
+  @Override
+  protected IndexSegment getIndexSegment() {
+    return _indexSegment;
+  }
+
+  @Override
+  protected List<IndexSegment> getIndexSegments() {
+    return _indexSegments;
+  }
+
+  @Test
+  public void testSelectStarQueries() {
+    // Select * without any filters
+    try {
+      getBrokerResponse(SELECT_STAR_QUERY);
+      Assert.fail("Select * query should fail since forwardIndexDisabled on a select column");
+    } catch (IllegalStateException e) {
+      assertTrue(e.getMessage().contains("Forward index disabled for column:")
+          && e.getMessage().contains("cannot create DataFetcher!"));
+    }
+
+    // Select * with filters
+    try {
+      getBrokerResponse(SELECT_STAR_QUERY + FILTER);
+      Assert.fail("Select * query should fail since forwardIndexDisabled on a select column");
+    } catch (IllegalStateException e) {
+      assertTrue(e.getMessage().contains("Forward index disabled for column:")
+          && e.getMessage().contains("cannot create DataFetcher!"));
+    }
+  }
+
+  @Test
+  public void testSelectQueries() {
+    {
+      // Selection query without filters including a column with forwardIndexDisabled enabled on both segments
+      String query = "SELECT column1, column5, column6, column9, column11 FROM testTable";
+      try {
+        getBrokerResponse(query);
+        Assert.fail("Query should fail since forwardIndexDisabled on a select column");
+      } catch (IllegalStateException e) {
+        assertTrue(e.getMessage().contains("Forward index disabled for column:")
+            && e.getMessage().contains("cannot create DataFetcher!"));
+      }
+    }
+    {
+      // Selection query without filters including a column with forwardIndexDisabled enabled on one segment
+      String query = "SELECT column1, column5, column7, column9, column11 FROM testTable";
+      try {
+        getBrokerResponse(query);
+        Assert.fail("Query should fail since forwardIndexDisabled on a select column");
+      } catch (IllegalStateException e) {
+        assertTrue(e.getMessage().contains("Forward index disabled for column:")
+            && e.getMessage().contains("cannot create DataFetcher!"));
+      }
+    }
+    {
+      // Selection query with filters including a column with forwardIndexDisabled enabled on both segments
+      String query = "SELECT column1, column5, column6, column9, column11 FROM testTable WHERE column6 = 2147458029";
+      try {
+        getBrokerResponse(query);
+        Assert.fail("Query should fail since forwardIndexDisabled on a select column");
+      } catch (IllegalStateException e) {
+        assertTrue(e.getMessage().contains("Forward index disabled for column:")
+            && e.getMessage().contains("cannot create DataFetcher!"));
+      }
+    }
+    {
+      // Selection query with filters including a column with forwardIndexDisabled enabled on one segment
+      String query = "SELECT column1, column5, column7, column9, column11 FROM testTable WHERE column7 = 675695";
+      try {
+        getBrokerResponse(query);
+        Assert.fail("Query should fail since forwardIndexDisabled on a select column");
+      } catch (IllegalStateException e) {
+        assertTrue(e.getMessage().contains("Forward index disabled for column:")
+            && e.getMessage().contains("cannot create DataFetcher!"));
+      }
+    }
+    {
+      // Selection query without filters and without columns with forwardIndexDisabled enabled on either segment
+      String query = "SELECT column1, column5, column9, column11 FROM testTable ORDER BY column1";
+      BrokerResponseNative brokerResponseNative = getBrokerResponse(query);
+      assertTrue(brokerResponseNative.getProcessingExceptions() == null
+          || brokerResponseNative.getProcessingExceptions().size() == 0);
+      ResultTable resultTable = brokerResponseNative.getResultTable();
+      assertEquals(brokerResponseNative.getNumRowsResultSet(), 10);
+      assertEquals(brokerResponseNative.getTotalDocs(), 120_000L);
+      assertEquals(brokerResponseNative.getNumDocsScanned(), 120_000L);
+      assertEquals(brokerResponseNative.getNumSegmentsProcessed(), 4L);
+      assertEquals(brokerResponseNative.getNumSegmentsMatched(), 4L);
+      assertEquals(brokerResponseNative.getNumEntriesScannedPostFilter(), 120_120L);
+      assertEquals(brokerResponseNative.getNumEntriesScannedInFilter(), 0L);
+      assertNotNull(brokerResponseNative.getProcessingExceptions());
+      assertEquals(brokerResponseNative.getProcessingExceptions().size(), 0);
+      DataSchema dataSchema = new DataSchema(new String[]{"column1", "column5", "column9", "column11"},
+        new DataSchema.ColumnDataType[]{DataSchema.ColumnDataType.INT, DataSchema.ColumnDataType.STRING,
+            DataSchema.ColumnDataType.INT, DataSchema.ColumnDataType.STRING});
+      assertEquals(resultTable.getDataSchema(), dataSchema);
+      List<Object[]> resultRows = resultTable.getRows();
+      int previousColumn1 = Integer.MIN_VALUE;
+      for (Object[] resultRow : resultRows) {
+        assertEquals(resultRow.length, 4);
+        assertEquals((String) resultRow[1], "gFuH");
+        // Column 1
+        assertTrue((Integer) resultRow[0] >= previousColumn1);
+        previousColumn1 = (Integer) resultRow[0];
+      }
+
+      Object[] firstRow = resultRows.get(0);
+      // Column 11
+      assertEquals((String) firstRow[3], "o");
+    }
+    {
+      // Transform function on a selection clause with a forwardIndexDisabled column in transform
+      String query = "SELECT CONCAT(column6, column9, '-') from testTable";

Review Comment:
   cool done then



-- 
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 #9333: Add an option to disable the creation of the forward index for a column

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


##########
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",

Review Comment:
   Do we have test for this feature with one of INT / FLOAT / LONG . This file seems to test only for STRING ?



-- 
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 #9333: Add an option to disable the creation of the forward index for a column

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


##########
pinot-spi/src/main/java/org/apache/pinot/spi/config/table/TableConfig.java:
##########
@@ -143,6 +143,16 @@ public TableConfig(@JsonProperty(value = TABLE_NAME_KEY, required = true) String
     Preconditions.checkArgument(indexingConfig != null, "'tableIndexConfig' must be configured");
     Preconditions.checkArgument(customConfig != null, "'metadata' must be configured");
 
+    if (fieldConfigList != null) {
+      for (FieldConfig fieldConfig : fieldConfigList) {
+        Map<String, String> fieldConfigProperties = fieldConfig.getProperties();
+        if (fieldConfigProperties != null) {
+          boolean forwardIndexDisabled = Boolean.parseBoolean(fieldConfigProperties.getOrDefault(

Review Comment:
   removed this, forgot to clean this up



-- 
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 #9333: Add an option to disable the creation of the forward index for a column

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


##########
pinot-core/src/test/java/org/apache/pinot/queries/ForwardIndexDisabledSingleValueQueriesTest.java:
##########
@@ -0,0 +1,1088 @@
+/**
+ * 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.queries;
+
+import java.io.File;
+import java.net.URL;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.TimeUnit;
+import org.apache.commons.io.FileUtils;
+import org.apache.pinot.common.response.broker.BrokerResponseNative;
+import org.apache.pinot.common.response.broker.ResultTable;
+import org.apache.pinot.common.utils.DataSchema;
+import org.apache.pinot.segment.local.indexsegment.immutable.ImmutableSegmentLoader;
+import org.apache.pinot.segment.local.segment.creator.impl.SegmentIndexCreationDriverImpl;
+import org.apache.pinot.segment.local.segment.index.loader.IndexLoadingConfig;
+import org.apache.pinot.segment.spi.ColumnMetadata;
+import org.apache.pinot.segment.spi.ImmutableSegment;
+import org.apache.pinot.segment.spi.IndexSegment;
+import org.apache.pinot.segment.spi.creator.SegmentGeneratorConfig;
+import org.apache.pinot.segment.spi.creator.SegmentIndexCreationDriver;
+import org.apache.pinot.spi.config.table.FieldConfig;
+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.data.TimeGranularitySpec;
+import org.apache.pinot.spi.utils.ReadMode;
+import org.apache.pinot.spi.utils.builder.TableConfigBuilder;
+import org.testng.Assert;
+import org.testng.annotations.AfterClass;
+import org.testng.annotations.BeforeClass;
+import org.testng.annotations.Test;
+
+import static org.testng.Assert.assertEquals;
+import static org.testng.Assert.assertNotNull;
+import static org.testng.Assert.assertNull;
+import static org.testng.Assert.assertTrue;
+
+
+/**
+ * The <code>ForwardIndexDisabledSingleValueQueriesTest</code> class sets up the index segment for the no forward
+ * index single-value queries test.
+ * <p>There are totally 18 columns, 30000 records inside the original Avro file where 11 columns are selected to build
+ * the index segment. Selected columns information are as following:
+ * <ul>
+ *   ColumnName, FieldType, DataType, Cardinality, IsSorted, HasInvertedIndex, FwdIndexDisabled: S1, S2, HasRangeIndex
+ *   <li>column1, METRIC, INT, 6582, F, F, F, F, F</li>
+ *   <li>column3, METRIC, INT, 21910, F, F, F, F, F</li>
+ *   <li>column5, DIMENSION, STRING, 1, T, F, F, F, F</li>
+ *   <li>column6, DIMENSION, INT, 608, F, T, T, T, T</li>
+ *   <li>column7, DIMENSION, INT, 146, F, T, T, F, F</li>
+ *   <li>column9, DIMENSION, INT, 1737, F, F, F, F, F</li>
+ *   <li>column11, DIMENSION, STRING, 5, F, T, F, F, F</li>
+ *   <li>column12, DIMENSION, STRING, 5, F, F, F, F, F</li>
+ *   <li>column17, METRIC, INT, 24, F, T, F, F, F</li>
+ *   <li>column18, METRIC, INT, 1440, F, T, F, F, F</li>
+ *   <li>daysSinceEpoch, TIME, INT, 2, T, F, F, F, F</li>
+ * </ul>
+ */
+public class ForwardIndexDisabledSingleValueQueriesTest extends BaseQueriesTest {
+  private static final String AVRO_DATA = "data" + File.separator + "test_data-sv.avro";
+  private static final String SEGMENT_NAME_1 = "testTable_126164076_167572857";
+  private static final String SEGMENT_NAME_2 = "testTable_126164076_167572858";
+  private static final File INDEX_DIR = new File(FileUtils.getTempDirectory(),
+      "ForwardIndexDisabledSingleValueQueriesTest");
+
+  private static final String SELECT_STAR_QUERY = "SELECT * FROM testTable";
+
+  // Hard-coded query filter.
+  private static final String FILTER = " WHERE column1 > 100000000"
+      + " AND column3 BETWEEN 20000000 AND 1000000000"
+      + " AND column5 = 'gFuH'"
+      + " AND (column6 < 500000000 OR column11 NOT IN ('t', 'P'))"
+      + " AND daysSinceEpoch = 126164076";
+
+  private IndexSegment _indexSegment;
+  // Contains 2 index segments, one with 2 columns with forward index disabled, and the other with just 1.
+  private List<IndexSegment> _indexSegments;
+
+  private TableConfig _tableConfig;
+  private List<String> _invertedIndexColumns;
+  private List<String> _forwardIndexDisabledColumns;
+
+  @BeforeClass
+  public void buildAndLoadSegment()
+      throws Exception {
+    FileUtils.deleteQuietly(INDEX_DIR);
+
+    // Get resource file path.
+    URL resource = getClass().getClassLoader().getResource(AVRO_DATA);
+    assertNotNull(resource);
+    String filePath = resource.getFile();
+
+    // Build the segment schema.
+    Schema schema = new Schema.SchemaBuilder().setSchemaName("testTable").addMetric("column1", FieldSpec.DataType.INT)
+        .addMetric("column3", FieldSpec.DataType.INT).addSingleValueDimension("column5", FieldSpec.DataType.STRING)
+        .addSingleValueDimension("column6", FieldSpec.DataType.INT)
+        .addSingleValueDimension("column7", FieldSpec.DataType.INT)
+        .addSingleValueDimension("column9", FieldSpec.DataType.INT)
+        .addSingleValueDimension("column11", FieldSpec.DataType.STRING)
+        .addSingleValueDimension("column12", FieldSpec.DataType.STRING).addMetric("column17", FieldSpec.DataType.INT)
+        .addMetric("column18", FieldSpec.DataType.INT)
+        .addTime(new TimeGranularitySpec(FieldSpec.DataType.INT, TimeUnit.DAYS, "daysSinceEpoch"), null).build();
+
+    createSegment(filePath, SEGMENT_NAME_1, schema);
+    createSegment(filePath, SEGMENT_NAME_2, schema);
+
+    ImmutableSegment immutableSegment1 = loadSegmentWithMetadataChecks(SEGMENT_NAME_1);
+    ImmutableSegment immutableSegment2 = loadSegmentWithMetadataChecks(SEGMENT_NAME_2);
+
+    _indexSegment = immutableSegment1;
+    _indexSegments = Arrays.asList(immutableSegment1, immutableSegment2);
+  }
+
+  private void createSegment(String filePath, String segmentName, Schema schema)
+      throws Exception {
+    // Create field configs for the no forward index columns
+    List<FieldConfig> fieldConfigList = new ArrayList<>();
+    fieldConfigList.add(new FieldConfig("column6", FieldConfig.EncodingType.DICTIONARY, Collections.emptyList(), null,
+        Collections.singletonMap(FieldConfig.FORWARD_INDEX_DISABLED, Boolean.TRUE.toString())));
+    if (segmentName.equals(SEGMENT_NAME_1)) {
+      fieldConfigList.add(new FieldConfig("column7", FieldConfig.EncodingType.DICTIONARY, Collections.emptyList(),
+          null, Collections.singletonMap(FieldConfig.FORWARD_INDEX_DISABLED, Boolean.TRUE.toString())));
+
+      // Build table config based on segment 1 as it contains both columns under no forward index
+      _tableConfig =
+          new TableConfigBuilder(TableType.OFFLINE).setTableName("testTable").setTimeColumnName("daysSinceEpoch")
+              .setFieldConfigList(fieldConfigList).setRangeIndexColumns(Arrays.asList("column6")).build();
+    }
+
+    // Create the segment generator config.
+    SegmentGeneratorConfig segmentGeneratorConfig = new SegmentGeneratorConfig(_tableConfig, schema);
+    segmentGeneratorConfig.setInputFilePath(filePath);
+    segmentGeneratorConfig.setTableName("testTable");
+    segmentGeneratorConfig.setOutDir(INDEX_DIR.getAbsolutePath());
+    segmentGeneratorConfig.setSegmentName(segmentName);
+    // The segment generation code in SegmentColumnarIndexCreator will throw
+    // exception if start and end time in time column are not in acceptable
+    // range. For this test, we first need to fix the input avro data
+    // to have the time column values in allowed range. Until then, the check
+    // is explicitly disabled
+    segmentGeneratorConfig.setSkipTimeValueCheck(true);
+    _invertedIndexColumns = Arrays.asList("column6", "column7", "column11", "column17", "column18");
+    segmentGeneratorConfig.setInvertedIndexCreationColumns(_invertedIndexColumns);
+
+    _forwardIndexDisabledColumns = new ArrayList<>(Arrays.asList("column6", "column7"));
+    segmentGeneratorConfig.setForwardIndexDisabledColumns(_forwardIndexDisabledColumns);
+    segmentGeneratorConfig.setRangeIndexCreationColumns(Arrays.asList("column6"));
+
+    // Build the index segment.
+    SegmentIndexCreationDriver driver = new SegmentIndexCreationDriverImpl();
+    driver.init(segmentGeneratorConfig);
+    driver.build();
+  }
+
+  private ImmutableSegment loadSegmentWithMetadataChecks(String segmentName)
+      throws Exception {
+    IndexLoadingConfig indexLoadingConfig = new IndexLoadingConfig();
+    indexLoadingConfig.setTableConfig(_tableConfig);
+    indexLoadingConfig.setInvertedIndexColumns(new HashSet<>(_invertedIndexColumns));
+    indexLoadingConfig.setForwardIndexDisabledColumns(new HashSet<>(_forwardIndexDisabledColumns));
+    indexLoadingConfig.setRangeIndexColumns(new HashSet<>(Arrays.asList("column6")));
+    indexLoadingConfig.setReadMode(ReadMode.heap);
+
+    ImmutableSegment immutableSegment = ImmutableSegmentLoader.load(new File(INDEX_DIR, segmentName),
+        indexLoadingConfig);
+
+    Map<String, ColumnMetadata> columnMetadataMap1 = immutableSegment.getSegmentMetadata().getColumnMetadataMap();
+    columnMetadataMap1.forEach((column, metadata) -> {
+      if (column.equals("column6") || column.equals("column7")) {
+        assertTrue(metadata.hasDictionary());
+        assertTrue(metadata.isSingleValue());
+        assertNull(immutableSegment.getForwardIndex(column));
+      } else {
+        assertNotNull(immutableSegment.getForwardIndex(column));
+      }
+    });
+
+    return immutableSegment;
+  }
+
+  @AfterClass
+  public void deleteAndDestroySegment() {
+    FileUtils.deleteQuietly(INDEX_DIR);
+    _indexSegments.forEach((IndexSegment::destroy));
+  }
+
+  @Override
+  protected String getFilter() {
+    return FILTER;
+  }
+
+  @Override
+  protected IndexSegment getIndexSegment() {
+    return _indexSegment;
+  }
+
+  @Override
+  protected List<IndexSegment> getIndexSegments() {
+    return _indexSegments;
+  }
+
+  @Test
+  public void testSelectStarQueries() {
+    // Select * without any filters
+    try {
+      getBrokerResponse(SELECT_STAR_QUERY);
+      Assert.fail("Select * query should fail since forwardIndexDisabled on a select column");
+    } catch (IllegalStateException e) {
+      assertTrue(e.getMessage().contains("Forward index disabled for column:")
+          && e.getMessage().contains("cannot create DataFetcher!"));
+    }
+
+    // Select * with filters
+    try {
+      getBrokerResponse(SELECT_STAR_QUERY + FILTER);
+      Assert.fail("Select * query should fail since forwardIndexDisabled on a select column");
+    } catch (IllegalStateException e) {
+      assertTrue(e.getMessage().contains("Forward index disabled for column:")
+          && e.getMessage().contains("cannot create DataFetcher!"));
+    }
+  }
+
+  @Test
+  public void testSelectQueries() {
+    {
+      // Selection query without filters including a column with forwardIndexDisabled enabled on both segments
+      String query = "SELECT column1, column5, column6, column9, column11 FROM testTable";
+      try {
+        getBrokerResponse(query);
+        Assert.fail("Query should fail since forwardIndexDisabled on a select column");
+      } catch (IllegalStateException e) {
+        assertTrue(e.getMessage().contains("Forward index disabled for column:")
+            && e.getMessage().contains("cannot create DataFetcher!"));
+      }
+    }
+    {
+      // Selection query without filters including a column with forwardIndexDisabled enabled on one segment
+      String query = "SELECT column1, column5, column7, column9, column11 FROM testTable";
+      try {
+        getBrokerResponse(query);
+        Assert.fail("Query should fail since forwardIndexDisabled on a select column");
+      } catch (IllegalStateException e) {
+        assertTrue(e.getMessage().contains("Forward index disabled for column:")
+            && e.getMessage().contains("cannot create DataFetcher!"));
+      }
+    }
+    {
+      // Selection query with filters including a column with forwardIndexDisabled enabled on both segments
+      String query = "SELECT column1, column5, column6, column9, column11 FROM testTable WHERE column6 = 2147458029";
+      try {
+        getBrokerResponse(query);
+        Assert.fail("Query should fail since forwardIndexDisabled on a select column");
+      } catch (IllegalStateException e) {
+        assertTrue(e.getMessage().contains("Forward index disabled for column:")
+            && e.getMessage().contains("cannot create DataFetcher!"));
+      }
+    }
+    {
+      // Selection query with filters including a column with forwardIndexDisabled enabled on one segment
+      String query = "SELECT column1, column5, column7, column9, column11 FROM testTable WHERE column7 = 675695";
+      try {
+        getBrokerResponse(query);
+        Assert.fail("Query should fail since forwardIndexDisabled on a select column");
+      } catch (IllegalStateException e) {
+        assertTrue(e.getMessage().contains("Forward index disabled for column:")
+            && e.getMessage().contains("cannot create DataFetcher!"));
+      }
+    }
+    {
+      // Selection query without filters and without columns with forwardIndexDisabled enabled on either segment
+      String query = "SELECT column1, column5, column9, column11 FROM testTable ORDER BY column1";
+      BrokerResponseNative brokerResponseNative = getBrokerResponse(query);
+      assertTrue(brokerResponseNative.getProcessingExceptions() == null
+          || brokerResponseNative.getProcessingExceptions().size() == 0);
+      ResultTable resultTable = brokerResponseNative.getResultTable();
+      assertEquals(brokerResponseNative.getNumRowsResultSet(), 10);
+      assertEquals(brokerResponseNative.getTotalDocs(), 120_000L);
+      assertEquals(brokerResponseNative.getNumDocsScanned(), 120_000L);
+      assertEquals(brokerResponseNative.getNumSegmentsProcessed(), 4L);
+      assertEquals(brokerResponseNative.getNumSegmentsMatched(), 4L);
+      assertEquals(brokerResponseNative.getNumEntriesScannedPostFilter(), 120_120L);
+      assertEquals(brokerResponseNative.getNumEntriesScannedInFilter(), 0L);
+      assertNotNull(brokerResponseNative.getProcessingExceptions());
+      assertEquals(brokerResponseNative.getProcessingExceptions().size(), 0);
+      DataSchema dataSchema = new DataSchema(new String[]{"column1", "column5", "column9", "column11"},
+        new DataSchema.ColumnDataType[]{DataSchema.ColumnDataType.INT, DataSchema.ColumnDataType.STRING,
+            DataSchema.ColumnDataType.INT, DataSchema.ColumnDataType.STRING});
+      assertEquals(resultTable.getDataSchema(), dataSchema);
+      List<Object[]> resultRows = resultTable.getRows();
+      int previousColumn1 = Integer.MIN_VALUE;
+      for (Object[] resultRow : resultRows) {
+        assertEquals(resultRow.length, 4);
+        assertEquals((String) resultRow[1], "gFuH");
+        // Column 1
+        assertTrue((Integer) resultRow[0] >= previousColumn1);
+        previousColumn1 = (Integer) resultRow[0];
+      }
+
+      Object[] firstRow = resultRows.get(0);
+      // Column 11
+      assertEquals((String) firstRow[3], "o");
+    }
+    {
+      // Transform function on a selection clause with a forwardIndexDisabled column in transform
+      String query = "SELECT CONCAT(column6, column9, '-') from testTable";
+      try {
+        getBrokerResponse(query);
+        Assert.fail("Query should fail since forwardIndexDisabled on a column in transform");
+      } catch (IllegalStateException e) {
+        assertTrue(e.getMessage().contains("Forward index disabled for column:")
+            && e.getMessage().contains("cannot create DataFetcher!"));
+      }
+    }
+    {
+      // Transform function on a selection clause without a forwardIndexDisabled column in transform
+      String query = "SELECT CONCAT(column5, column9, '-') from testTable ORDER BY column1";
+      BrokerResponseNative brokerResponseNative = getBrokerResponse(query);
+      assertTrue(brokerResponseNative.getProcessingExceptions() == null
+          || brokerResponseNative.getProcessingExceptions().size() == 0);
+      ResultTable resultTable = brokerResponseNative.getResultTable();
+      assertEquals(brokerResponseNative.getNumRowsResultSet(), 10);
+      assertEquals(brokerResponseNative.getTotalDocs(), 120_000L);
+      assertEquals(brokerResponseNative.getNumDocsScanned(), 120_000L);
+      assertEquals(brokerResponseNative.getNumSegmentsProcessed(), 4L);
+      assertEquals(brokerResponseNative.getNumSegmentsMatched(), 4L);
+      assertEquals(brokerResponseNative.getNumEntriesScannedPostFilter(), 120_080L);
+      assertEquals(brokerResponseNative.getNumEntriesScannedInFilter(), 0L);
+      assertNotNull(brokerResponseNative.getProcessingExceptions());
+      assertEquals(brokerResponseNative.getProcessingExceptions().size(), 0);
+      DataSchema dataSchema = new DataSchema(new String[]{"concat(column5,column9,'-')"},
+          new DataSchema.ColumnDataType[]{DataSchema.ColumnDataType.STRING});
+      assertEquals(resultTable.getDataSchema(), dataSchema);
+      List<Object[]> resultRows = resultTable.getRows();
+      for (Object[] resultRow : resultRows) {
+        assertEquals(resultRow.length, 1);
+        assertTrue(resultRow[0].toString().startsWith("gFuH-"));
+      }
+    }
+    {
+      // Selection query with filters (not including forwardIndexDisabled column) and without columns with
+      // forwardIndexDisabled enabled on either segment
+      String query = "SELECT column1, column5, column9, column11 FROM testTable WHERE column1 > 100000000"
+        + " AND column3 BETWEEN 20000000 AND 1000000000"
+        + " AND column5 = 'gFuH'"
+        + " AND daysSinceEpoch = 126164076 ORDER BY column1";
+      BrokerResponseNative brokerResponseNative = getBrokerResponse(query);
+      assertTrue(brokerResponseNative.getProcessingExceptions() == null
+          || brokerResponseNative.getProcessingExceptions().size() == 0);
+      ResultTable resultTable = brokerResponseNative.getResultTable();
+      assertEquals(brokerResponseNative.getNumRowsResultSet(), 10);
+      assertEquals(brokerResponseNative.getTotalDocs(), 120_000L);
+      assertEquals(brokerResponseNative.getNumDocsScanned(), 42_368L);
+      assertEquals(brokerResponseNative.getNumSegmentsProcessed(), 4L);
+      assertEquals(brokerResponseNative.getNumSegmentsMatched(), 4L);
+      assertEquals(brokerResponseNative.getNumEntriesScannedPostFilter(), 42_488);
+      assertEquals(brokerResponseNative.getNumEntriesScannedInFilter(), 192744L);
+      assertNotNull(brokerResponseNative.getProcessingExceptions());
+      assertEquals(brokerResponseNative.getProcessingExceptions().size(), 0);
+      DataSchema dataSchema = new DataSchema(new String[]{"column1", "column5", "column9", "column11"},
+          new DataSchema.ColumnDataType[]{DataSchema.ColumnDataType.INT, DataSchema.ColumnDataType.STRING,
+              DataSchema.ColumnDataType.INT, DataSchema.ColumnDataType.STRING});
+      assertEquals(resultTable.getDataSchema(), dataSchema);
+      List<Object[]> resultRows = resultTable.getRows();
+      int previousColumn1 = Integer.MIN_VALUE;
+      for (Object[] resultRow : resultRows) {
+        assertEquals(resultRow.length, 4);
+        assertEquals((String) resultRow[1], "gFuH");
+        assertTrue((Integer) resultRow[0] >= previousColumn1);
+        previousColumn1 = (Integer) resultRow[0];
+      }
+
+      Object[] firstRow = resultRows.get(0);
+      // Column 11
+      assertEquals((String) firstRow[3], "P");
+    }
+    {
+      // Selection query with supported filters (including forwardIndexDisabled column) and without columns with
+      // forwardIndexDisabled enabled on either segment
+      String query = "SELECT column1, column5, column9, column11 FROM testTable WHERE column6 = 2147458029 "
+          + "ORDER BY column1";
+      BrokerResponseNative brokerResponseNative = getBrokerResponse(query);
+      assertTrue(brokerResponseNative.getProcessingExceptions() == null
+          || brokerResponseNative.getProcessingExceptions().size() == 0);
+      ResultTable resultTable = brokerResponseNative.getResultTable();
+      assertEquals(brokerResponseNative.getNumRowsResultSet(), 10);
+      assertEquals(brokerResponseNative.getTotalDocs(), 120_000L);
+      assertEquals(brokerResponseNative.getNumDocsScanned(), 16L);
+      assertEquals(brokerResponseNative.getNumSegmentsProcessed(), 4L);
+      assertEquals(brokerResponseNative.getNumSegmentsMatched(), 4L);
+      assertEquals(brokerResponseNative.getNumEntriesScannedPostFilter(), 64L);
+      assertEquals(brokerResponseNative.getNumEntriesScannedInFilter(), 0L);
+      assertNotNull(brokerResponseNative.getProcessingExceptions());
+      assertEquals(brokerResponseNative.getProcessingExceptions().size(), 0);
+      DataSchema dataSchema = new DataSchema(new String[]{"column1", "column5", "column9", "column11"},
+          new DataSchema.ColumnDataType[]{DataSchema.ColumnDataType.INT, DataSchema.ColumnDataType.STRING,
+              DataSchema.ColumnDataType.INT, DataSchema.ColumnDataType.STRING});
+      assertEquals(resultTable.getDataSchema(), dataSchema);
+      List<Object[]> resultRows = resultTable.getRows();
+      for (Object[] resultRow : resultRows) {
+        assertEquals(resultRow.length, 4);
+        assertEquals(resultRow[0], 635553468);
+        assertEquals((String) resultRow[1], "gFuH");
+        assertEquals(resultRow[2], 705242697);
+        assertEquals((String) resultRow[3], "P");
+      }
+    }
+    {
+      // Selection query with supported filters (including forwardIndexDisabled column) and without columns with
+      // forwardIndexDisabled enabled on either segment
+      String query = "SELECT column1, column5, column9, column11 FROM testTable WHERE column7 != 675695 "
+          + "ORDER BY column1";
+      BrokerResponseNative brokerResponseNative = getBrokerResponse(query);
+      assertTrue(brokerResponseNative.getProcessingExceptions() == null
+          || brokerResponseNative.getProcessingExceptions().size() == 0);
+      ResultTable resultTable = brokerResponseNative.getResultTable();
+      assertEquals(brokerResponseNative.getNumRowsResultSet(), 10);
+      assertEquals(brokerResponseNative.getTotalDocs(), 120_000L);
+      assertEquals(brokerResponseNative.getNumDocsScanned(), 119_908L);
+      assertEquals(brokerResponseNative.getNumSegmentsProcessed(), 4L);
+      assertEquals(brokerResponseNative.getNumSegmentsMatched(), 4L);
+      assertEquals(brokerResponseNative.getNumEntriesScannedPostFilter(), 120_028L);
+      assertEquals(brokerResponseNative.getNumEntriesScannedInFilter(), 0L);
+      assertNotNull(brokerResponseNative.getProcessingExceptions());
+      assertEquals(brokerResponseNative.getProcessingExceptions().size(), 0);
+      DataSchema dataSchema = new DataSchema(new String[]{"column1", "column5", "column9", "column11"},
+          new DataSchema.ColumnDataType[]{DataSchema.ColumnDataType.INT, DataSchema.ColumnDataType.STRING,
+              DataSchema.ColumnDataType.INT, DataSchema.ColumnDataType.STRING});
+      assertEquals(resultTable.getDataSchema(), dataSchema);
+      List<Object[]> resultRows = resultTable.getRows();
+      int previousColumn1 = Integer.MIN_VALUE;
+      for (Object[] resultRow : resultRows) {
+        assertEquals(resultRow.length, 4);
+        assertEquals((String) resultRow[1], "gFuH");
+        assertTrue((Integer) resultRow[0] >= previousColumn1);
+        previousColumn1 = (Integer) resultRow[0];
+      }
+
+      Object[] firstRow = resultRows.get(0);
+      // Column 11
+      assertEquals((String) firstRow[3], "o");
+    }
+    {
+      // Selection query with supported filters (including forwardIndexDisabled column) and without columns with
+      // forwardIndexDisabled enabled on either segment
+      String query = "SELECT column1, column5, column9, column11 FROM testTable WHERE column7 IN (675695, 2137685743) "
+          + "ORDER BY column1";
+      BrokerResponseNative brokerResponseNative = getBrokerResponse(query);
+      assertTrue(brokerResponseNative.getProcessingExceptions() == null
+          || brokerResponseNative.getProcessingExceptions().size() == 0);
+      ResultTable resultTable = brokerResponseNative.getResultTable();
+      assertEquals(brokerResponseNative.getNumRowsResultSet(), 10);
+      assertEquals(brokerResponseNative.getTotalDocs(), 120_000L);
+      assertEquals(brokerResponseNative.getNumDocsScanned(), 828L);
+      assertEquals(brokerResponseNative.getNumSegmentsProcessed(), 4L);
+      assertEquals(brokerResponseNative.getNumSegmentsMatched(), 4L);
+      assertEquals(brokerResponseNative.getNumEntriesScannedPostFilter(), 948L);
+      assertEquals(brokerResponseNative.getNumEntriesScannedInFilter(), 0L);
+      assertNotNull(brokerResponseNative.getProcessingExceptions());
+      assertEquals(brokerResponseNative.getProcessingExceptions().size(), 0);
+      DataSchema dataSchema = new DataSchema(new String[]{"column1", "column5", "column9", "column11"},
+          new DataSchema.ColumnDataType[]{DataSchema.ColumnDataType.INT, DataSchema.ColumnDataType.STRING,
+              DataSchema.ColumnDataType.INT, DataSchema.ColumnDataType.STRING});
+      assertEquals(resultTable.getDataSchema(), dataSchema);
+      List<Object[]> resultRows = resultTable.getRows();
+      int previousColumn1 = Integer.MIN_VALUE;
+      for (Object[] resultRow : resultRows) {
+        assertEquals(resultRow.length, 4);
+        assertEquals((String) resultRow[1], "gFuH");
+        assertTrue((Integer) resultRow[0] >= previousColumn1);
+        previousColumn1 = (Integer) resultRow[0];
+      }
+
+      Object[] firstRow = resultRows.get(0);
+      // Column 11
+      assertEquals((String) firstRow[3], "P");
+    }
+    {
+      // Selection query with supported filters (including forwardIndexDisabled column) and without columns with
+      // forwardIndexDisabled enabled on either segment
+      String query = "SELECT column1, column5, column9, column11 FROM testTable WHERE column6 NOT IN "
+          + "(1689277, 2147458029) ORDER BY column1";
+      BrokerResponseNative brokerResponseNative = getBrokerResponse(query);
+      assertTrue(brokerResponseNative.getProcessingExceptions() == null
+          || brokerResponseNative.getProcessingExceptions().size() == 0);
+      ResultTable resultTable = brokerResponseNative.getResultTable();
+      assertEquals(brokerResponseNative.getNumRowsResultSet(), 10);
+      assertEquals(brokerResponseNative.getTotalDocs(), 120_000L);
+      assertEquals(brokerResponseNative.getNumDocsScanned(), 119_980L);
+      assertEquals(brokerResponseNative.getNumSegmentsProcessed(), 4L);
+      assertEquals(brokerResponseNative.getNumSegmentsMatched(), 4L);
+      assertEquals(brokerResponseNative.getNumEntriesScannedPostFilter(), 120_100L);
+      assertEquals(brokerResponseNative.getNumEntriesScannedInFilter(), 0L);
+      assertNotNull(brokerResponseNative.getProcessingExceptions());
+      assertEquals(brokerResponseNative.getProcessingExceptions().size(), 0);
+      DataSchema dataSchema = new DataSchema(new String[]{"column1", "column5", "column9", "column11"},
+          new DataSchema.ColumnDataType[]{DataSchema.ColumnDataType.INT, DataSchema.ColumnDataType.STRING,
+              DataSchema.ColumnDataType.INT, DataSchema.ColumnDataType.STRING});
+      assertEquals(resultTable.getDataSchema(), dataSchema);
+      List<Object[]> resultRows = resultTable.getRows();
+      int previousColumn1 = Integer.MIN_VALUE;
+      for (Object[] resultRow : resultRows) {
+        assertEquals(resultRow.length, 4);
+        assertEquals((String) resultRow[1], "gFuH");
+        assertTrue((Integer) resultRow[0] >= previousColumn1);
+        previousColumn1 = (Integer) resultRow[0];
+      }
+
+      Object[] firstRow = resultRows.get(0);
+      // Column 11
+      assertEquals((String) firstRow[3], "o");
+    }
+    {
+      // Query with literal only in SELECT
+      String query = "SELECT 'marvin' from testTable ORDER BY column1";
+      BrokerResponseNative brokerResponseNative = getBrokerResponse(query);
+      assertTrue(brokerResponseNative.getProcessingExceptions() == null
+          || brokerResponseNative.getProcessingExceptions().size() == 0);
+      ResultTable resultTable = brokerResponseNative.getResultTable();
+      assertEquals(brokerResponseNative.getNumRowsResultSet(), 10);
+      assertEquals(brokerResponseNative.getTotalDocs(), 120_000L);
+      assertEquals(brokerResponseNative.getNumDocsScanned(), 120_000L);
+      assertEquals(brokerResponseNative.getNumSegmentsProcessed(), 4L);
+      assertEquals(brokerResponseNative.getNumSegmentsMatched(), 4L);
+      assertEquals(brokerResponseNative.getNumEntriesScannedPostFilter(), 120_000L);
+      assertEquals(brokerResponseNative.getNumEntriesScannedInFilter(), 0L);
+      assertNotNull(brokerResponseNative.getProcessingExceptions());
+      assertEquals(brokerResponseNative.getProcessingExceptions().size(), 0);
+      assertEquals(resultTable.getDataSchema(), new DataSchema(new String[]{"'marvin'"},
+          new DataSchema.ColumnDataType[]{DataSchema.ColumnDataType.STRING}));
+      List<Object[]> resultRows = resultTable.getRows();
+      for (Object[] resultRow : resultRows) {
+        assertEquals(resultRow.length, 1);
+        assertEquals((String) resultRow[0], "marvin");
+      }
+    }
+    {
+      // Selection query with '<' filter on a forwardIndexDisabled column with range index available
+      String query = "SELECT column1, column5, column9, column11 FROM testTable WHERE column6 < 2147458029 AND "
+          + "column6 > 1699000 ORDER BY column1";
+      BrokerResponseNative brokerResponseNative = getBrokerResponse(query);
+      assertTrue(brokerResponseNative.getProcessingExceptions() == null
+          || brokerResponseNative.getProcessingExceptions().size() == 0);
+      ResultTable resultTable = brokerResponseNative.getResultTable();
+      assertEquals(brokerResponseNative.getNumRowsResultSet(), 10);
+      assertEquals(brokerResponseNative.getTotalDocs(), 120_000L);
+      assertEquals(brokerResponseNative.getNumDocsScanned(), 119_980L);
+      assertEquals(brokerResponseNative.getNumSegmentsProcessed(), 4L);
+      assertEquals(brokerResponseNative.getNumSegmentsMatched(), 4L);
+      assertEquals(brokerResponseNative.getNumEntriesScannedPostFilter(), 120_100L);
+      assertEquals(brokerResponseNative.getNumEntriesScannedInFilter(), 0L);
+      assertNotNull(brokerResponseNative.getProcessingExceptions());
+      assertEquals(brokerResponseNative.getProcessingExceptions().size(), 0);
+      DataSchema dataSchema = new DataSchema(new String[]{"column1", "column5", "column9", "column11"},
+          new DataSchema.ColumnDataType[]{DataSchema.ColumnDataType.INT, DataSchema.ColumnDataType.STRING,
+              DataSchema.ColumnDataType.INT, DataSchema.ColumnDataType.STRING});
+      assertEquals(resultTable.getDataSchema(), dataSchema);
+      List<Object[]> resultRows = resultTable.getRows();
+      int previousColumn1 = Integer.MIN_VALUE;
+      for (Object[] resultRow : resultRows) {
+        assertEquals(resultRow.length, 4);
+        assertEquals((String) resultRow[1], "gFuH");
+        assertTrue((Integer) resultRow[0] >= previousColumn1);
+        previousColumn1 = (Integer) resultRow[0];
+      }
+
+      Object[] firstRow = resultRows.get(0);
+      // Column 11
+      assertEquals((String) firstRow[3], "o");
+    }
+    {
+      // Selection query with '>=' filter on a forwardIndexDisabled column without range index available
+      String query = "SELECT column1, column5, column9, column11 FROM testTable WHERE column7 >= 676000";
+      try {
+        getBrokerResponse(query);
+        Assert.fail("Query should fail since forwardIndexDisabled on a range query column without range index");
+      } catch (IllegalStateException e) {
+        assertTrue(e.getMessage().contains("Forward index disabled for column:")
+            && e.getMessage().contains("creating ScanDocIdSet unsupported!"));
+      }
+    }
+    {
+      // Select query with a filter on a column which doesn't have forwardIndexDisabled enabled
+      String query = "SELECT column1, column5, column9 from testTable WHERE column9 < 50000 ORDER BY column1";
+      BrokerResponseNative brokerResponseNative = getBrokerResponse(query);
+      assertTrue(brokerResponseNative.getProcessingExceptions() == null
+          || brokerResponseNative.getProcessingExceptions().size() == 0);
+      ResultTable resultTable = brokerResponseNative.getResultTable();
+      assertEquals(brokerResponseNative.getNumRowsResultSet(), 4);
+      assertEquals(brokerResponseNative.getTotalDocs(), 120_000L);
+      assertEquals(brokerResponseNative.getNumDocsScanned(), 4L);
+      assertEquals(brokerResponseNative.getNumSegmentsProcessed(), 4L);
+      assertEquals(brokerResponseNative.getNumSegmentsMatched(), 4L);
+      assertEquals(brokerResponseNative.getNumEntriesScannedPostFilter(), 12L);
+      assertEquals(brokerResponseNative.getNumEntriesScannedInFilter(), 120000L);
+      assertNotNull(brokerResponseNative.getProcessingExceptions());
+      assertEquals(brokerResponseNative.getProcessingExceptions().size(), 0);
+      assertEquals(resultTable.getDataSchema(), new DataSchema(new String[]{"column1", "column5", "column9"},
+          new DataSchema.ColumnDataType[]{
+          DataSchema.ColumnDataType.INT, DataSchema.ColumnDataType.STRING, DataSchema.ColumnDataType.INT
+      }));
+      List<Object[]> resultRows = resultTable.getRows();
+      for (Object[] resultRow : resultRows) {
+        assertEquals(resultRow.length, 3);
+      }
+    }
+    {
+      // Transform function on a filter clause for forwardIndexDisabled column in transform
+      String query = "SELECT column1, column11 from testTable WHERE CONCAT(column6, column9, '-') = '1689277-11270'";
+      BrokerResponseNative brokerResponseNative = getBrokerResponse(query);
+      assertTrue(brokerResponseNative.getProcessingExceptions() != null
+          && brokerResponseNative.getProcessingExceptions().size() > 0);
+    }
+    {
+      // Transform function on a filter clause for a non-forwardIndexDisabled column in transform
+      String query = "SELECT column1, column11 from testTable WHERE CONCAT(column5, column9, '-') = 'gFuH-11270' "
+          + "ORDER BY column1";
+      BrokerResponseNative brokerResponseNative = getBrokerResponse(query);
+      assertTrue(brokerResponseNative.getProcessingExceptions() == null
+          || brokerResponseNative.getProcessingExceptions().size() == 0);
+      ResultTable resultTable = brokerResponseNative.getResultTable();
+      assertEquals(brokerResponseNative.getNumRowsResultSet(), 4);
+      assertEquals(brokerResponseNative.getTotalDocs(), 120_000L);
+      assertEquals(brokerResponseNative.getNumDocsScanned(), 4L);
+      assertEquals(brokerResponseNative.getNumSegmentsProcessed(), 4L);
+      assertEquals(brokerResponseNative.getNumSegmentsMatched(), 4L);
+      assertEquals(brokerResponseNative.getNumEntriesScannedPostFilter(), 8L);
+      assertEquals(brokerResponseNative.getNumEntriesScannedInFilter(), 120000L);
+      assertNotNull(brokerResponseNative.getProcessingExceptions());
+      assertEquals(brokerResponseNative.getProcessingExceptions().size(), 0);
+      assertEquals(resultTable.getDataSchema(), new DataSchema(new String[]{"column1", "column11"},
+          new DataSchema.ColumnDataType[]{
+              DataSchema.ColumnDataType.INT, DataSchema.ColumnDataType.STRING
+          }));
+      List<Object[]> resultRows = resultTable.getRows();
+      assertEquals(resultRows.size(), 4);
+      for (Object[] resultRow : resultRows) {
+        assertEquals(resultRow.length, 2);
+        assertEquals(resultRow[0], 815409257);
+        assertEquals(resultRow[1], "P");
+      }
+    }
+  }
+
+  @Test
+  public void testSelectWithDistinctQueries() {
+    {
+      // Select a mix of forwardIndexDisabled and non-forwardIndexDisabled columns with distinct
+      String query = "SELECT DISTINCT column1, column6, column9 FROM testTable LIMIT 10";
+      try {
+        getBrokerResponse(query);
+        Assert.fail("Query should fail since forwardIndexDisabled on a column in select distinct");
+      } catch (IllegalStateException e) {
+        assertTrue(e.getMessage().contains("Forward index disabled for column:")
+            && e.getMessage().contains("cannot create DataFetcher!"));
+      }
+    }
+    {
+      // Select non-forwardIndexDisabled columns with distinct
+      String query = "SELECT DISTINCT column1, column5, column9 FROM testTable ORDER BY column1 LIMIT 10";
+      BrokerResponseNative brokerResponseNative = getBrokerResponse(query);
+      assertTrue(brokerResponseNative.getProcessingExceptions() == null
+          || brokerResponseNative.getProcessingExceptions().size() == 0);
+      ResultTable resultTable = brokerResponseNative.getResultTable();
+      assertEquals(brokerResponseNative.getNumRowsResultSet(), 10);
+      assertEquals(brokerResponseNative.getTotalDocs(), 120_000L);
+      assertEquals(brokerResponseNative.getNumDocsScanned(), 120_000L);
+      assertEquals(brokerResponseNative.getNumSegmentsProcessed(), 4L);
+      assertEquals(brokerResponseNative.getNumSegmentsMatched(), 4L);
+      assertEquals(brokerResponseNative.getNumEntriesScannedPostFilter(), 360_000L);
+      assertEquals(brokerResponseNative.getNumEntriesScannedInFilter(), 0L);
+      assertNotNull(brokerResponseNative.getProcessingExceptions());
+      assertEquals(brokerResponseNative.getProcessingExceptions().size(), 0);
+      assertEquals(resultTable.getDataSchema(), new DataSchema(new String[]{"column1", "column5", "column9"},
+          new DataSchema.ColumnDataType[]{DataSchema.ColumnDataType.INT, DataSchema.ColumnDataType.STRING,
+              DataSchema.ColumnDataType.INT}));
+      List<Object[]> resultRows = resultTable.getRows();
+      int previousColumn1 = Integer.MIN_VALUE;
+      for (Object[] resultRow : resultRows) {
+        assertEquals(resultRow.length, 3);
+        assertEquals(resultRow[1], "gFuH");
+        assertTrue((Integer) resultRow[0] >= previousColumn1);
+        previousColumn1 = (Integer) resultRow[0];
+      }
+    }
+  }
+
+  @Test
+  public void testSelectWithGroupByOrderByQueries() {
+    {
+      // Select a mix of forwardIndexDisabled and non-forwardIndexDisabled columns with group by order by
+      String query = "SELECT column1, column6 FROM testTable GROUP BY column1, column6 ORDER BY column1, column6 "
+          + " LIMIT 10";
+      try {
+        getBrokerResponse(query);
+        Assert.fail("Query should fail since forwardIndexDisabled on a column in select group by order by");
+      } catch (IllegalStateException e) {
+        assertTrue(e.getMessage().contains("Forward index disabled for column:")
+            && e.getMessage().contains("cannot create DataFetcher!"));
+      }
+    }
+    {
+      // Select forwardIndexDisabled columns with group by order by
+      String query = "SELECT column7, column6 FROM testTable GROUP BY column7, column6 ORDER BY column7, column6 "
+          + " LIMIT 10";
+      try {
+        getBrokerResponse(query);
+        Assert.fail("Query should fail since forwardIndexDisabled on a column in select group by order by");
+      } catch (IllegalStateException e) {
+        assertTrue(e.getMessage().contains("Forward index disabled for column:")
+            && e.getMessage().contains("cannot create DataFetcher!"));
+      }
+    }
+    {
+      // Select non-forwardIndexDisabled columns with group by order by
+      String query = "SELECT column1, column5 FROM testTable GROUP BY column1, column5 ORDER BY column1, column5 "
+          + " LIMIT 10";
+      BrokerResponseNative brokerResponseNative = getBrokerResponse(query);
+      assertTrue(brokerResponseNative.getProcessingExceptions() == null
+          || brokerResponseNative.getProcessingExceptions().size() == 0);
+      ResultTable resultTable = brokerResponseNative.getResultTable();
+      assertEquals(brokerResponseNative.getNumRowsResultSet(), 10);
+      assertEquals(brokerResponseNative.getTotalDocs(), 120_000L);
+      assertEquals(brokerResponseNative.getNumDocsScanned(), 120000L);
+      assertEquals(brokerResponseNative.getNumSegmentsProcessed(), 4L);
+      assertEquals(brokerResponseNative.getNumSegmentsMatched(), 4L);
+      assertEquals(brokerResponseNative.getNumEntriesScannedPostFilter(), 240000L);
+      assertEquals(brokerResponseNative.getNumEntriesScannedInFilter(), 0L);
+      assertNotNull(brokerResponseNative.getProcessingExceptions());
+      assertEquals(brokerResponseNative.getProcessingExceptions().size(), 0);
+      assertEquals(resultTable.getDataSchema(), new DataSchema(new String[]{"column1", "column5"},
+          new DataSchema.ColumnDataType[]{DataSchema.ColumnDataType.INT, DataSchema.ColumnDataType.STRING}));
+      List<Object[]> resultRows = resultTable.getRows();
+      int previousVal = -1;
+      for (Object[] resultRow : resultRows) {
+        assertEquals(resultRow.length, 2);
+        assertEquals(resultRow[1], "gFuH");
+        assertTrue((int) resultRow[0] > previousVal);
+        previousVal = (int) resultRow[0];
+      }
+    }
+    {
+      // Select forwardIndexDisabled columns using transform with group by order by
+      String query = "SELECT CONCAT(column1, column6, '-') FROM testTable GROUP BY CONCAT(column1, column6, '-') "
+          + "ORDER BY CONCAT(column1, column6, '-') LIMIT 10";
+      try {
+        getBrokerResponse(query);
+        Assert.fail("Query should fail since forwardIndexDisabled on a transformed column in group by order by");
+      } catch (IllegalStateException e) {
+        assertTrue(e.getMessage().contains("Forward index disabled for column:")
+            && e.getMessage().contains("cannot create DataFetcher!"));
+      }
+    }
+    {
+      // Select non-forwardIndexDisabled columns using transform with group by order by
+      String query = "SELECT CONCAT(column1, column5, '-') FROM testTable GROUP BY CONCAT(column1, column5, '-') "
+          + "ORDER BY CONCAT(column1, column5, '-') LIMIT 10";
+      BrokerResponseNative brokerResponseNative = getBrokerResponse(query);
+      assertTrue(brokerResponseNative.getProcessingExceptions() == null
+          || brokerResponseNative.getProcessingExceptions().size() == 0);
+      ResultTable resultTable = brokerResponseNative.getResultTable();
+      assertEquals(brokerResponseNative.getNumRowsResultSet(), 10);
+      assertEquals(brokerResponseNative.getTotalDocs(), 120_000L);
+      assertEquals(brokerResponseNative.getNumDocsScanned(), 120000L);
+      assertEquals(brokerResponseNative.getNumSegmentsProcessed(), 4L);
+      assertEquals(brokerResponseNative.getNumSegmentsMatched(), 4L);
+      assertEquals(brokerResponseNative.getNumEntriesScannedPostFilter(), 240000L);
+      assertEquals(brokerResponseNative.getNumEntriesScannedInFilter(), 0L);
+      assertNotNull(brokerResponseNative.getProcessingExceptions());
+      assertEquals(brokerResponseNative.getProcessingExceptions().size(), 0);
+      assertEquals(resultTable.getDataSchema(), new DataSchema(new String[]{"concat(column1,column5,'-')"},
+          new DataSchema.ColumnDataType[]{DataSchema.ColumnDataType.STRING}));
+      List<Object[]> resultRows = resultTable.getRows();
+      for (Object[] resultRow : resultRows) {
+        assertEquals(resultRow.length, 1);
+        assertTrue(resultRow[0].toString().endsWith("-gFuH"));
+      }
+    }
+  }
+
+  @Test
+  public void testSelectWithAggregationQueries() {
+    {
+      // Allowed aggregation functions on forwardIndexDisabled columns
+      String query = "SELECT max(column7), min(column7), count(column7), minmaxrange(column7), "
+          + "distinctcount(column7), distinctcounthll(column6), distinctcountrawhll(column7), "

Review Comment:
   Can you add distinctcountbitmap as well to this list ?



-- 
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 #9333: Add an option to disable the creation of the forward index for a column

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


##########
pinot-core/src/test/java/org/apache/pinot/queries/ForwardIndexDisabledSingleValueQueriesTest.java:
##########
@@ -0,0 +1,1088 @@
+/**
+ * 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.queries;
+
+import java.io.File;
+import java.net.URL;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.TimeUnit;
+import org.apache.commons.io.FileUtils;
+import org.apache.pinot.common.response.broker.BrokerResponseNative;
+import org.apache.pinot.common.response.broker.ResultTable;
+import org.apache.pinot.common.utils.DataSchema;
+import org.apache.pinot.segment.local.indexsegment.immutable.ImmutableSegmentLoader;
+import org.apache.pinot.segment.local.segment.creator.impl.SegmentIndexCreationDriverImpl;
+import org.apache.pinot.segment.local.segment.index.loader.IndexLoadingConfig;
+import org.apache.pinot.segment.spi.ColumnMetadata;
+import org.apache.pinot.segment.spi.ImmutableSegment;
+import org.apache.pinot.segment.spi.IndexSegment;
+import org.apache.pinot.segment.spi.creator.SegmentGeneratorConfig;
+import org.apache.pinot.segment.spi.creator.SegmentIndexCreationDriver;
+import org.apache.pinot.spi.config.table.FieldConfig;
+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.data.TimeGranularitySpec;
+import org.apache.pinot.spi.utils.ReadMode;
+import org.apache.pinot.spi.utils.builder.TableConfigBuilder;
+import org.testng.Assert;
+import org.testng.annotations.AfterClass;
+import org.testng.annotations.BeforeClass;
+import org.testng.annotations.Test;
+
+import static org.testng.Assert.assertEquals;
+import static org.testng.Assert.assertNotNull;
+import static org.testng.Assert.assertNull;
+import static org.testng.Assert.assertTrue;
+
+
+/**
+ * The <code>ForwardIndexDisabledSingleValueQueriesTest</code> class sets up the index segment for the no forward
+ * index single-value queries test.
+ * <p>There are totally 18 columns, 30000 records inside the original Avro file where 11 columns are selected to build
+ * the index segment. Selected columns information are as following:
+ * <ul>
+ *   ColumnName, FieldType, DataType, Cardinality, IsSorted, HasInvertedIndex, FwdIndexDisabled: S1, S2, HasRangeIndex
+ *   <li>column1, METRIC, INT, 6582, F, F, F, F, F</li>
+ *   <li>column3, METRIC, INT, 21910, F, F, F, F, F</li>
+ *   <li>column5, DIMENSION, STRING, 1, T, F, F, F, F</li>
+ *   <li>column6, DIMENSION, INT, 608, F, T, T, T, T</li>
+ *   <li>column7, DIMENSION, INT, 146, F, T, T, F, F</li>
+ *   <li>column9, DIMENSION, INT, 1737, F, F, F, F, F</li>
+ *   <li>column11, DIMENSION, STRING, 5, F, T, F, F, F</li>
+ *   <li>column12, DIMENSION, STRING, 5, F, F, F, F, F</li>
+ *   <li>column17, METRIC, INT, 24, F, T, F, F, F</li>
+ *   <li>column18, METRIC, INT, 1440, F, T, F, F, F</li>
+ *   <li>daysSinceEpoch, TIME, INT, 2, T, F, F, F, F</li>
+ * </ul>
+ */
+public class ForwardIndexDisabledSingleValueQueriesTest extends BaseQueriesTest {
+  private static final String AVRO_DATA = "data" + File.separator + "test_data-sv.avro";
+  private static final String SEGMENT_NAME_1 = "testTable_126164076_167572857";
+  private static final String SEGMENT_NAME_2 = "testTable_126164076_167572858";
+  private static final File INDEX_DIR = new File(FileUtils.getTempDirectory(),
+      "ForwardIndexDisabledSingleValueQueriesTest");
+
+  private static final String SELECT_STAR_QUERY = "SELECT * FROM testTable";
+
+  // Hard-coded query filter.
+  private static final String FILTER = " WHERE column1 > 100000000"
+      + " AND column3 BETWEEN 20000000 AND 1000000000"
+      + " AND column5 = 'gFuH'"
+      + " AND (column6 < 500000000 OR column11 NOT IN ('t', 'P'))"
+      + " AND daysSinceEpoch = 126164076";
+
+  private IndexSegment _indexSegment;
+  // Contains 2 index segments, one with 2 columns with forward index disabled, and the other with just 1.
+  private List<IndexSegment> _indexSegments;
+
+  private TableConfig _tableConfig;
+  private List<String> _invertedIndexColumns;
+  private List<String> _forwardIndexDisabledColumns;
+
+  @BeforeClass
+  public void buildAndLoadSegment()
+      throws Exception {
+    FileUtils.deleteQuietly(INDEX_DIR);
+
+    // Get resource file path.
+    URL resource = getClass().getClassLoader().getResource(AVRO_DATA);
+    assertNotNull(resource);
+    String filePath = resource.getFile();
+
+    // Build the segment schema.
+    Schema schema = new Schema.SchemaBuilder().setSchemaName("testTable").addMetric("column1", FieldSpec.DataType.INT)
+        .addMetric("column3", FieldSpec.DataType.INT).addSingleValueDimension("column5", FieldSpec.DataType.STRING)
+        .addSingleValueDimension("column6", FieldSpec.DataType.INT)
+        .addSingleValueDimension("column7", FieldSpec.DataType.INT)
+        .addSingleValueDimension("column9", FieldSpec.DataType.INT)
+        .addSingleValueDimension("column11", FieldSpec.DataType.STRING)
+        .addSingleValueDimension("column12", FieldSpec.DataType.STRING).addMetric("column17", FieldSpec.DataType.INT)
+        .addMetric("column18", FieldSpec.DataType.INT)
+        .addTime(new TimeGranularitySpec(FieldSpec.DataType.INT, TimeUnit.DAYS, "daysSinceEpoch"), null).build();
+
+    createSegment(filePath, SEGMENT_NAME_1, schema);
+    createSegment(filePath, SEGMENT_NAME_2, schema);
+
+    ImmutableSegment immutableSegment1 = loadSegmentWithMetadataChecks(SEGMENT_NAME_1);
+    ImmutableSegment immutableSegment2 = loadSegmentWithMetadataChecks(SEGMENT_NAME_2);
+
+    _indexSegment = immutableSegment1;
+    _indexSegments = Arrays.asList(immutableSegment1, immutableSegment2);
+  }
+
+  private void createSegment(String filePath, String segmentName, Schema schema)
+      throws Exception {
+    // Create field configs for the no forward index columns
+    List<FieldConfig> fieldConfigList = new ArrayList<>();
+    fieldConfigList.add(new FieldConfig("column6", FieldConfig.EncodingType.DICTIONARY, Collections.emptyList(), null,
+        Collections.singletonMap(FieldConfig.FORWARD_INDEX_DISABLED, Boolean.TRUE.toString())));
+    if (segmentName.equals(SEGMENT_NAME_1)) {
+      fieldConfigList.add(new FieldConfig("column7", FieldConfig.EncodingType.DICTIONARY, Collections.emptyList(),
+          null, Collections.singletonMap(FieldConfig.FORWARD_INDEX_DISABLED, Boolean.TRUE.toString())));
+
+      // Build table config based on segment 1 as it contains both columns under no forward index
+      _tableConfig =
+          new TableConfigBuilder(TableType.OFFLINE).setTableName("testTable").setTimeColumnName("daysSinceEpoch")
+              .setFieldConfigList(fieldConfigList).setRangeIndexColumns(Arrays.asList("column6")).build();
+    }
+
+    // Create the segment generator config.
+    SegmentGeneratorConfig segmentGeneratorConfig = new SegmentGeneratorConfig(_tableConfig, schema);
+    segmentGeneratorConfig.setInputFilePath(filePath);
+    segmentGeneratorConfig.setTableName("testTable");
+    segmentGeneratorConfig.setOutDir(INDEX_DIR.getAbsolutePath());
+    segmentGeneratorConfig.setSegmentName(segmentName);
+    // The segment generation code in SegmentColumnarIndexCreator will throw
+    // exception if start and end time in time column are not in acceptable
+    // range. For this test, we first need to fix the input avro data
+    // to have the time column values in allowed range. Until then, the check
+    // is explicitly disabled
+    segmentGeneratorConfig.setSkipTimeValueCheck(true);
+    _invertedIndexColumns = Arrays.asList("column6", "column7", "column11", "column17", "column18");
+    segmentGeneratorConfig.setInvertedIndexCreationColumns(_invertedIndexColumns);
+
+    _forwardIndexDisabledColumns = new ArrayList<>(Arrays.asList("column6", "column7"));
+    segmentGeneratorConfig.setForwardIndexDisabledColumns(_forwardIndexDisabledColumns);
+    segmentGeneratorConfig.setRangeIndexCreationColumns(Arrays.asList("column6"));
+
+    // Build the index segment.
+    SegmentIndexCreationDriver driver = new SegmentIndexCreationDriverImpl();
+    driver.init(segmentGeneratorConfig);
+    driver.build();
+  }
+
+  private ImmutableSegment loadSegmentWithMetadataChecks(String segmentName)
+      throws Exception {
+    IndexLoadingConfig indexLoadingConfig = new IndexLoadingConfig();
+    indexLoadingConfig.setTableConfig(_tableConfig);
+    indexLoadingConfig.setInvertedIndexColumns(new HashSet<>(_invertedIndexColumns));
+    indexLoadingConfig.setForwardIndexDisabledColumns(new HashSet<>(_forwardIndexDisabledColumns));
+    indexLoadingConfig.setRangeIndexColumns(new HashSet<>(Arrays.asList("column6")));
+    indexLoadingConfig.setReadMode(ReadMode.heap);
+
+    ImmutableSegment immutableSegment = ImmutableSegmentLoader.load(new File(INDEX_DIR, segmentName),
+        indexLoadingConfig);
+
+    Map<String, ColumnMetadata> columnMetadataMap1 = immutableSegment.getSegmentMetadata().getColumnMetadataMap();
+    columnMetadataMap1.forEach((column, metadata) -> {
+      if (column.equals("column6") || column.equals("column7")) {
+        assertTrue(metadata.hasDictionary());
+        assertTrue(metadata.isSingleValue());
+        assertNull(immutableSegment.getForwardIndex(column));
+      } else {
+        assertNotNull(immutableSegment.getForwardIndex(column));
+      }
+    });
+
+    return immutableSegment;
+  }
+
+  @AfterClass
+  public void deleteAndDestroySegment() {
+    FileUtils.deleteQuietly(INDEX_DIR);
+    _indexSegments.forEach((IndexSegment::destroy));
+  }
+
+  @Override
+  protected String getFilter() {
+    return FILTER;
+  }
+
+  @Override
+  protected IndexSegment getIndexSegment() {
+    return _indexSegment;
+  }
+
+  @Override
+  protected List<IndexSegment> getIndexSegments() {
+    return _indexSegments;
+  }
+
+  @Test
+  public void testSelectStarQueries() {
+    // Select * without any filters
+    try {
+      getBrokerResponse(SELECT_STAR_QUERY);
+      Assert.fail("Select * query should fail since forwardIndexDisabled on a select column");
+    } catch (IllegalStateException e) {
+      assertTrue(e.getMessage().contains("Forward index disabled for column:")
+          && e.getMessage().contains("cannot create DataFetcher!"));
+    }
+
+    // Select * with filters
+    try {
+      getBrokerResponse(SELECT_STAR_QUERY + FILTER);
+      Assert.fail("Select * query should fail since forwardIndexDisabled on a select column");
+    } catch (IllegalStateException e) {
+      assertTrue(e.getMessage().contains("Forward index disabled for column:")
+          && e.getMessage().contains("cannot create DataFetcher!"));
+    }
+  }
+
+  @Test
+  public void testSelectQueries() {
+    {
+      // Selection query without filters including a column with forwardIndexDisabled enabled on both segments
+      String query = "SELECT column1, column5, column6, column9, column11 FROM testTable";
+      try {
+        getBrokerResponse(query);
+        Assert.fail("Query should fail since forwardIndexDisabled on a select column");
+      } catch (IllegalStateException e) {
+        assertTrue(e.getMessage().contains("Forward index disabled for column:")
+            && e.getMessage().contains("cannot create DataFetcher!"));
+      }
+    }
+    {
+      // Selection query without filters including a column with forwardIndexDisabled enabled on one segment
+      String query = "SELECT column1, column5, column7, column9, column11 FROM testTable";
+      try {
+        getBrokerResponse(query);
+        Assert.fail("Query should fail since forwardIndexDisabled on a select column");
+      } catch (IllegalStateException e) {
+        assertTrue(e.getMessage().contains("Forward index disabled for column:")
+            && e.getMessage().contains("cannot create DataFetcher!"));
+      }
+    }
+    {
+      // Selection query with filters including a column with forwardIndexDisabled enabled on both segments
+      String query = "SELECT column1, column5, column6, column9, column11 FROM testTable WHERE column6 = 2147458029";

Review Comment:
   Ohh looks like it is covered further down. Just saw. Ignore this..



##########
pinot-core/src/test/java/org/apache/pinot/queries/ForwardIndexDisabledSingleValueQueriesTest.java:
##########
@@ -0,0 +1,1088 @@
+/**
+ * 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.queries;
+
+import java.io.File;
+import java.net.URL;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.TimeUnit;
+import org.apache.commons.io.FileUtils;
+import org.apache.pinot.common.response.broker.BrokerResponseNative;
+import org.apache.pinot.common.response.broker.ResultTable;
+import org.apache.pinot.common.utils.DataSchema;
+import org.apache.pinot.segment.local.indexsegment.immutable.ImmutableSegmentLoader;
+import org.apache.pinot.segment.local.segment.creator.impl.SegmentIndexCreationDriverImpl;
+import org.apache.pinot.segment.local.segment.index.loader.IndexLoadingConfig;
+import org.apache.pinot.segment.spi.ColumnMetadata;
+import org.apache.pinot.segment.spi.ImmutableSegment;
+import org.apache.pinot.segment.spi.IndexSegment;
+import org.apache.pinot.segment.spi.creator.SegmentGeneratorConfig;
+import org.apache.pinot.segment.spi.creator.SegmentIndexCreationDriver;
+import org.apache.pinot.spi.config.table.FieldConfig;
+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.data.TimeGranularitySpec;
+import org.apache.pinot.spi.utils.ReadMode;
+import org.apache.pinot.spi.utils.builder.TableConfigBuilder;
+import org.testng.Assert;
+import org.testng.annotations.AfterClass;
+import org.testng.annotations.BeforeClass;
+import org.testng.annotations.Test;
+
+import static org.testng.Assert.assertEquals;
+import static org.testng.Assert.assertNotNull;
+import static org.testng.Assert.assertNull;
+import static org.testng.Assert.assertTrue;
+
+
+/**
+ * The <code>ForwardIndexDisabledSingleValueQueriesTest</code> class sets up the index segment for the no forward
+ * index single-value queries test.
+ * <p>There are totally 18 columns, 30000 records inside the original Avro file where 11 columns are selected to build
+ * the index segment. Selected columns information are as following:
+ * <ul>
+ *   ColumnName, FieldType, DataType, Cardinality, IsSorted, HasInvertedIndex, FwdIndexDisabled: S1, S2, HasRangeIndex
+ *   <li>column1, METRIC, INT, 6582, F, F, F, F, F</li>
+ *   <li>column3, METRIC, INT, 21910, F, F, F, F, F</li>
+ *   <li>column5, DIMENSION, STRING, 1, T, F, F, F, F</li>
+ *   <li>column6, DIMENSION, INT, 608, F, T, T, T, T</li>
+ *   <li>column7, DIMENSION, INT, 146, F, T, T, F, F</li>
+ *   <li>column9, DIMENSION, INT, 1737, F, F, F, F, F</li>
+ *   <li>column11, DIMENSION, STRING, 5, F, T, F, F, F</li>
+ *   <li>column12, DIMENSION, STRING, 5, F, F, F, F, F</li>
+ *   <li>column17, METRIC, INT, 24, F, T, F, F, F</li>
+ *   <li>column18, METRIC, INT, 1440, F, T, F, F, F</li>
+ *   <li>daysSinceEpoch, TIME, INT, 2, T, F, F, F, F</li>
+ * </ul>
+ */
+public class ForwardIndexDisabledSingleValueQueriesTest extends BaseQueriesTest {
+  private static final String AVRO_DATA = "data" + File.separator + "test_data-sv.avro";
+  private static final String SEGMENT_NAME_1 = "testTable_126164076_167572857";
+  private static final String SEGMENT_NAME_2 = "testTable_126164076_167572858";
+  private static final File INDEX_DIR = new File(FileUtils.getTempDirectory(),
+      "ForwardIndexDisabledSingleValueQueriesTest");
+
+  private static final String SELECT_STAR_QUERY = "SELECT * FROM testTable";
+
+  // Hard-coded query filter.
+  private static final String FILTER = " WHERE column1 > 100000000"
+      + " AND column3 BETWEEN 20000000 AND 1000000000"
+      + " AND column5 = 'gFuH'"
+      + " AND (column6 < 500000000 OR column11 NOT IN ('t', 'P'))"
+      + " AND daysSinceEpoch = 126164076";
+
+  private IndexSegment _indexSegment;
+  // Contains 2 index segments, one with 2 columns with forward index disabled, and the other with just 1.
+  private List<IndexSegment> _indexSegments;
+
+  private TableConfig _tableConfig;
+  private List<String> _invertedIndexColumns;
+  private List<String> _forwardIndexDisabledColumns;
+
+  @BeforeClass
+  public void buildAndLoadSegment()
+      throws Exception {
+    FileUtils.deleteQuietly(INDEX_DIR);
+
+    // Get resource file path.
+    URL resource = getClass().getClassLoader().getResource(AVRO_DATA);
+    assertNotNull(resource);
+    String filePath = resource.getFile();
+
+    // Build the segment schema.
+    Schema schema = new Schema.SchemaBuilder().setSchemaName("testTable").addMetric("column1", FieldSpec.DataType.INT)
+        .addMetric("column3", FieldSpec.DataType.INT).addSingleValueDimension("column5", FieldSpec.DataType.STRING)
+        .addSingleValueDimension("column6", FieldSpec.DataType.INT)
+        .addSingleValueDimension("column7", FieldSpec.DataType.INT)
+        .addSingleValueDimension("column9", FieldSpec.DataType.INT)
+        .addSingleValueDimension("column11", FieldSpec.DataType.STRING)
+        .addSingleValueDimension("column12", FieldSpec.DataType.STRING).addMetric("column17", FieldSpec.DataType.INT)
+        .addMetric("column18", FieldSpec.DataType.INT)
+        .addTime(new TimeGranularitySpec(FieldSpec.DataType.INT, TimeUnit.DAYS, "daysSinceEpoch"), null).build();
+
+    createSegment(filePath, SEGMENT_NAME_1, schema);
+    createSegment(filePath, SEGMENT_NAME_2, schema);
+
+    ImmutableSegment immutableSegment1 = loadSegmentWithMetadataChecks(SEGMENT_NAME_1);
+    ImmutableSegment immutableSegment2 = loadSegmentWithMetadataChecks(SEGMENT_NAME_2);
+
+    _indexSegment = immutableSegment1;
+    _indexSegments = Arrays.asList(immutableSegment1, immutableSegment2);
+  }
+
+  private void createSegment(String filePath, String segmentName, Schema schema)
+      throws Exception {
+    // Create field configs for the no forward index columns
+    List<FieldConfig> fieldConfigList = new ArrayList<>();
+    fieldConfigList.add(new FieldConfig("column6", FieldConfig.EncodingType.DICTIONARY, Collections.emptyList(), null,
+        Collections.singletonMap(FieldConfig.FORWARD_INDEX_DISABLED, Boolean.TRUE.toString())));
+    if (segmentName.equals(SEGMENT_NAME_1)) {
+      fieldConfigList.add(new FieldConfig("column7", FieldConfig.EncodingType.DICTIONARY, Collections.emptyList(),
+          null, Collections.singletonMap(FieldConfig.FORWARD_INDEX_DISABLED, Boolean.TRUE.toString())));
+
+      // Build table config based on segment 1 as it contains both columns under no forward index
+      _tableConfig =
+          new TableConfigBuilder(TableType.OFFLINE).setTableName("testTable").setTimeColumnName("daysSinceEpoch")
+              .setFieldConfigList(fieldConfigList).setRangeIndexColumns(Arrays.asList("column6")).build();
+    }
+
+    // Create the segment generator config.
+    SegmentGeneratorConfig segmentGeneratorConfig = new SegmentGeneratorConfig(_tableConfig, schema);
+    segmentGeneratorConfig.setInputFilePath(filePath);
+    segmentGeneratorConfig.setTableName("testTable");
+    segmentGeneratorConfig.setOutDir(INDEX_DIR.getAbsolutePath());
+    segmentGeneratorConfig.setSegmentName(segmentName);
+    // The segment generation code in SegmentColumnarIndexCreator will throw
+    // exception if start and end time in time column are not in acceptable
+    // range. For this test, we first need to fix the input avro data
+    // to have the time column values in allowed range. Until then, the check
+    // is explicitly disabled
+    segmentGeneratorConfig.setSkipTimeValueCheck(true);
+    _invertedIndexColumns = Arrays.asList("column6", "column7", "column11", "column17", "column18");
+    segmentGeneratorConfig.setInvertedIndexCreationColumns(_invertedIndexColumns);
+
+    _forwardIndexDisabledColumns = new ArrayList<>(Arrays.asList("column6", "column7"));
+    segmentGeneratorConfig.setForwardIndexDisabledColumns(_forwardIndexDisabledColumns);
+    segmentGeneratorConfig.setRangeIndexCreationColumns(Arrays.asList("column6"));
+
+    // Build the index segment.
+    SegmentIndexCreationDriver driver = new SegmentIndexCreationDriverImpl();
+    driver.init(segmentGeneratorConfig);
+    driver.build();
+  }
+
+  private ImmutableSegment loadSegmentWithMetadataChecks(String segmentName)
+      throws Exception {
+    IndexLoadingConfig indexLoadingConfig = new IndexLoadingConfig();
+    indexLoadingConfig.setTableConfig(_tableConfig);
+    indexLoadingConfig.setInvertedIndexColumns(new HashSet<>(_invertedIndexColumns));
+    indexLoadingConfig.setForwardIndexDisabledColumns(new HashSet<>(_forwardIndexDisabledColumns));
+    indexLoadingConfig.setRangeIndexColumns(new HashSet<>(Arrays.asList("column6")));
+    indexLoadingConfig.setReadMode(ReadMode.heap);
+
+    ImmutableSegment immutableSegment = ImmutableSegmentLoader.load(new File(INDEX_DIR, segmentName),
+        indexLoadingConfig);
+
+    Map<String, ColumnMetadata> columnMetadataMap1 = immutableSegment.getSegmentMetadata().getColumnMetadataMap();
+    columnMetadataMap1.forEach((column, metadata) -> {
+      if (column.equals("column6") || column.equals("column7")) {
+        assertTrue(metadata.hasDictionary());
+        assertTrue(metadata.isSingleValue());
+        assertNull(immutableSegment.getForwardIndex(column));
+      } else {
+        assertNotNull(immutableSegment.getForwardIndex(column));
+      }
+    });
+
+    return immutableSegment;
+  }
+
+  @AfterClass
+  public void deleteAndDestroySegment() {
+    FileUtils.deleteQuietly(INDEX_DIR);
+    _indexSegments.forEach((IndexSegment::destroy));
+  }
+
+  @Override
+  protected String getFilter() {
+    return FILTER;
+  }
+
+  @Override
+  protected IndexSegment getIndexSegment() {
+    return _indexSegment;
+  }
+
+  @Override
+  protected List<IndexSegment> getIndexSegments() {
+    return _indexSegments;
+  }
+
+  @Test
+  public void testSelectStarQueries() {
+    // Select * without any filters
+    try {
+      getBrokerResponse(SELECT_STAR_QUERY);
+      Assert.fail("Select * query should fail since forwardIndexDisabled on a select column");
+    } catch (IllegalStateException e) {
+      assertTrue(e.getMessage().contains("Forward index disabled for column:")
+          && e.getMessage().contains("cannot create DataFetcher!"));
+    }
+
+    // Select * with filters
+    try {
+      getBrokerResponse(SELECT_STAR_QUERY + FILTER);
+      Assert.fail("Select * query should fail since forwardIndexDisabled on a select column");
+    } catch (IllegalStateException e) {
+      assertTrue(e.getMessage().contains("Forward index disabled for column:")
+          && e.getMessage().contains("cannot create DataFetcher!"));
+    }
+  }
+
+  @Test
+  public void testSelectQueries() {
+    {
+      // Selection query without filters including a column with forwardIndexDisabled enabled on both segments
+      String query = "SELECT column1, column5, column6, column9, column11 FROM testTable";
+      try {
+        getBrokerResponse(query);
+        Assert.fail("Query should fail since forwardIndexDisabled on a select column");
+      } catch (IllegalStateException e) {
+        assertTrue(e.getMessage().contains("Forward index disabled for column:")
+            && e.getMessage().contains("cannot create DataFetcher!"));
+      }
+    }
+    {
+      // Selection query without filters including a column with forwardIndexDisabled enabled on one segment
+      String query = "SELECT column1, column5, column7, column9, column11 FROM testTable";
+      try {
+        getBrokerResponse(query);
+        Assert.fail("Query should fail since forwardIndexDisabled on a select column");
+      } catch (IllegalStateException e) {
+        assertTrue(e.getMessage().contains("Forward index disabled for column:")
+            && e.getMessage().contains("cannot create DataFetcher!"));
+      }
+    }
+    {
+      // Selection query with filters including a column with forwardIndexDisabled enabled on both segments
+      String query = "SELECT column1, column5, column6, column9, column11 FROM testTable WHERE column6 = 2147458029";
+      try {
+        getBrokerResponse(query);
+        Assert.fail("Query should fail since forwardIndexDisabled on a select column");
+      } catch (IllegalStateException e) {
+        assertTrue(e.getMessage().contains("Forward index disabled for column:")
+            && e.getMessage().contains("cannot create DataFetcher!"));
+      }
+    }
+    {
+      // Selection query with filters including a column with forwardIndexDisabled enabled on one segment
+      String query = "SELECT column1, column5, column7, column9, column11 FROM testTable WHERE column7 = 675695";

Review Comment:
   nvm.. looks like this is covered



-- 
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 #9333: Add an option to disable the creation of the forward index for a column

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


##########
pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/creator/impl/fwd/MultiValueNoOpForwardIndexCreator.java:
##########
@@ -0,0 +1,53 @@
+/**
+ * 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.segment.local.segment.creator.impl.fwd;
+
+import java.io.IOException;
+import org.apache.pinot.segment.spi.index.creator.ForwardIndexCreator;
+import org.apache.pinot.spi.data.FieldSpec;
+
+
+/**
+ * Forward index creator for dictionary-encoded multi-value column with forward index disabled. This is a no-op.
+ */
+public class MultiValueNoOpForwardIndexCreator implements ForwardIndexCreator {
+  @Override
+  public boolean isDictionaryEncoded() {
+    return true;
+  }
+
+  @Override
+  public boolean isSingleValue() {
+    return false;
+  }
+
+  @Override
+  public FieldSpec.DataType getValueType() {
+    return FieldSpec.DataType.INT;
+  }
+
+  @Override
+  public void putDictIdMV(int[] dictIds) {

Review Comment:
   Similar to NO-OP reader, do we need to throw specific UNSUP 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 #9333: Add an option to disable the creation of the forward index for a column

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


##########
pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/creator/impl/DefaultIndexCreatorProvider.java:
##########
@@ -88,17 +90,25 @@ public ForwardIndexCreator newForwardIndexCreator(IndexCreationContext.Forward c
             context.getMaxRowLengthInBytes());
       }
     } else {
-      if (context.getFieldSpec().isSingleValueField()) {
-        if (context.isSorted()) {
-          return new SingleValueSortedForwardIndexCreator(context.getIndexDir(), context.getFieldSpec().getName(),
-              context.getCardinality());
+      if (context.forwardIndexDisabled()) {

Review Comment:
   (nit) not related to this PR -- do you mind adding a comment under the else branch above line 93 ? something like `Dictionary enabled column`. Will make it bit more readable. Similarly above line 79 `NoDictionary column`
   
   We may also want to make this more readable by having separate helper method but separate PR for that is fine too



-- 
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 #9333: Add an option to disable the creation of the forward index for a column

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


##########
pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/readers/forward/NoOpMVForwardIndexReader.java:
##########
@@ -0,0 +1,66 @@
+/**
+ * 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.segment.local.segment.index.readers.forward;
+
+import java.io.IOException;
+import org.apache.pinot.segment.spi.index.reader.ForwardIndexReader;
+import org.apache.pinot.segment.spi.index.reader.ForwardIndexReaderContext;
+import org.apache.pinot.spi.data.FieldSpec;
+
+
+/**
+ * No-op forward index reader meant to be used by multi-value columns with forward index disabled. This is meant to
+ * provide clean exception handling for operations which need to read forward index when forward index is disabled.
+ */
+public class NoOpMVForwardIndexReader implements ForwardIndexReader<ForwardIndexReaderContext> {

Review Comment:
   I feel that will be quite tricky and less clean since that requires policing multiple areas in the code and then throw exception. IIRC, we had considered this option but NO-OP reader looked better / 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] somandal commented on a diff in pull request #9333: Add an option to disable the creation of the forward index for a column

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


##########
pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/creator/impl/SegmentColumnarIndexCreator.java:
##########
@@ -816,6 +835,7 @@ public static void addColumnMetadataInfo(PropertiesConfiguration properties, Str
         String.valueOf(columnIndexCreationInfo.getTotalNumberOfEntries()));
     properties.setProperty(getKeyFor(column, IS_AUTO_GENERATED),
         String.valueOf(columnIndexCreationInfo.isAutoGenerated()));
+    properties.setProperty(getKeyFor(column, FORWARD_INDEX_DISABLED), String.valueOf(forwardIndexDisabled));

Review Comment:
   done! remove 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] somandal commented on a diff in pull request #9333: Add an option to disable the creation of the forward index for a column

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


##########
pinot-core/src/main/java/org/apache/pinot/core/minion/RawIndexConverter.java:
##########
@@ -204,6 +204,11 @@ private void convertColumn(FieldSpec fieldSpec)
     // Create the raw index
     DataSource dataSource = _originalImmutableSegment.getDataSource(columnName);
     ForwardIndexReader reader = dataSource.getForwardIndex();
+    if (reader == null) {

Review Comment:
   This is a refactoring change (since the original variable was named `reader`). Is it alright if I do this as a follow up to keep this PR simpler? I can check for this in a bunch of other files too then.



##########
pinot-tools/src/main/java/org/apache/pinot/tools/segment/converter/DictionaryToRawIndexConverter.java:
##########
@@ -296,6 +297,10 @@ private void convertOneColumn(IndexSegment segment, String column, File newSegme
       throws IOException {
     DataSource dataSource = segment.getDataSource(column);
     ForwardIndexReader reader = dataSource.getForwardIndex();
+    if (reader == null) {

Review Comment:
   This is a refactoring change (since the original variable was named `reader`). Is it alright if I do this as a follow up to keep this PR simpler? I can check for this in a bunch of other files too then.



-- 
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] Jackie-Jiang commented on a diff in pull request #9333: Add an option to disable the creation of the forward index for a column

Posted by GitBox <gi...@apache.org>.
Jackie-Jiang commented on code in PR #9333:
URL: https://github.com/apache/pinot/pull/9333#discussion_r974673959


##########
pinot-core/src/main/java/org/apache/pinot/core/common/DataFetcher.java:
##########
@@ -68,9 +68,14 @@ public DataFetcher(Map<String, DataSource> dataSourceMap) {
       String column = entry.getKey();
       DataSource dataSource = entry.getValue();
       DataSourceMetadata dataSourceMetadata = dataSource.getDataSourceMetadata();
+      ForwardIndexReader<?> forwardIndexReader = dataSource.getForwardIndex();
+      if (forwardIndexReader == null) {
+        throw new UnsupportedOperationException(
+            String.format("Forward index disabled for column: %s, cannot create DataFetcher!",
+            dataSourceMetadata.getFieldSpec().getName()));
+      }

Review Comment:
   (minor) Can be simplified to
   ```suggestion
         Preconditions.checkState(forwardIndexReader != null, "Forward index disabled for column: %s, cannot create DataFetcher!", column);
   ```
   
   Note that we might want to throw `IllegalStateException` instead. Same for other places



##########
pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/column/PhysicalColumnIndexContainer.java:
##########
@@ -133,7 +133,12 @@ public PhysicalColumnIndexContainer(SegmentDirectory.Reader segmentReader, Colum
       _rangeIndex = null;
     }
 
-    PinotDataBuffer fwdIndexBuffer = segmentReader.getIndexFor(columnName, ColumnIndexType.FORWARD_INDEX);
+    // Setting the 'fwdIndexBuffer' to null if forward index is enabled. No-op index readers will be setup for the
+    // forward index disabled columns which doesn't require the 'fwdIndexBuffer'.
+    boolean forwardIndexDisabled = !segmentReader.hasIndexFor(columnName, ColumnIndexType.FORWARD_INDEX);

Review Comment:
   (minor) `forwardIndexDisabled` is redundant. We can check if `fwdIndexBuffer` is `null`



##########
pinot-spi/src/main/java/org/apache/pinot/spi/config/table/TableConfig.java:
##########
@@ -58,6 +58,11 @@ public class TableConfig extends BaseJsonConfig {
   // Double underscore is reserved for real-time segment name delimiter
   private static final String TABLE_NAME_FORBIDDEN_SUBSTRING = "__";
 
+  // TODO: Remove this flag once the reload path to create forward index from inverted index and dictionary is
+  //       added. This feature will be disabled until the reload path is updated to handle forward index enable ->
+  //       disable and forward index disable -> enable.
+  public static boolean _disallowForwardIndexDisabled = true;

Review Comment:
   IMO this is not needed. We can still enable this feature without ability to automatically generate the forward index



##########
pinot-tools/src/main/java/org/apache/pinot/tools/segment/converter/DictionaryToRawIndexConverter.java:
##########
@@ -296,6 +297,10 @@ private void convertOneColumn(IndexSegment segment, String column, File newSegme
       throws IOException {
     DataSource dataSource = segment.getDataSource(column);
     ForwardIndexReader reader = dataSource.getForwardIndex();
+    if (reader == null) {
+      throw new UnsupportedEncodingException(String.format("Forward index is disabled for column %s, cannot convert!",

Review Comment:
   Why this exception?



##########
pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/loader/IndexLoadingConfig.java:
##########
@@ -315,6 +341,19 @@ public List<String> getSortedColumns() {
     return _sortedColumns;
   }
 
+  /**
+   * For tests only.
+   */
+  @VisibleForTesting
+  public void setSortedColumns(String sortedColumn) {

Review Comment:
   ```suggestion
     public void setSortedColumn(String sortedColumn) {
   ```



##########
pinot-segment-local/src/main/java/org/apache/pinot/segment/local/utils/TableConfigUtils.java:
##########
@@ -897,6 +903,41 @@ private static void validateFieldConfigList(@Nullable List<FieldConfig> fieldCon
     }
   }
 
+  /**
+   * Validates the compatibility of the indexes if the column has the forward index disabled. Throws exceptions due to
+   * compatibility mismatch. The checks performed are:
+   *     - Validate dictionary is enabled.
+   *     - Validate inverted index is enabled.
+   *     - Validate that either no range index exists for column or the range index version is at least 2 and isn't a
+   *       multi-value column (since mulit-value defaults to index v1).
+   */
+  private static void validateForwardIndexDisabledIndexCompatibility(String columnName, FieldConfig fieldConfig,
+      IndexingConfig indexingConfigs, List<String> noDictionaryColumns, Schema schema) {
+    Map<String, String> fieldConfigProperties = fieldConfig.getProperties();
+    if (fieldConfigProperties == null) {
+      return;
+    }
+
+    boolean forwardIndexDisabled = Boolean.parseBoolean(fieldConfigProperties
+        .getOrDefault(FieldConfig.FORWARD_INDEX_DISABLED, FieldConfig.DEFAULT_FORWARD_INDEX_DISABLED));

Review Comment:
   (minor)
   ```suggestion
       boolean forwardIndexDisabled = Boolean.parseBoolean(fieldConfigProperties.get(FieldConfig.FORWARD_INDEX_DISABLED));
   ```



##########
pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/column/PhysicalColumnIndexContainer.java:
##########
@@ -149,7 +154,12 @@ public PhysicalColumnIndexContainer(SegmentDirectory.Reader segmentReader, Colum
           return;
         }
       }
-      _forwardIndex = indexReaderProvider.newForwardIndexReader(fwdIndexBuffer, metadata);
+      if (forwardIndexDisabled && !metadata.isSorted()) {
+        // Forward index disabled flag is a no-op for sorted columns
+        _forwardIndex = null;
+      } else {
+        _forwardIndex = indexReaderProvider.newForwardIndexReader(fwdIndexBuffer, metadata);
+      }

Review Comment:
   (minor) No need to check sorted here
   ```suggestion
         if (fwdIndexBuffer != null) {
           _forwardIndex = indexReaderProvider.newForwardIndexReader(fwdIndexBuffer, metadata);
         } else {
           _forwardIndex = null;
         }
   ```



##########
pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/loader/invertedindex/TextIndexHandler.java:
##########
@@ -174,18 +175,46 @@ private void createTextIndexForColumn(SegmentDirectory.Writer segmentWriter, Col
     // segmentDirectory is indicated to us by SegmentDirectoryPaths, we create lucene index there. There is no
     // further need to move around the lucene index directory since it is created with correct directory structure
     // based on segmentVersion.
-    try (ForwardIndexReader forwardIndexReader = LoaderUtils.getForwardIndexReader(segmentWriter, columnMetadata);
-        ForwardIndexReaderContext readerContext = forwardIndexReader.createContext();
-        TextIndexCreator textIndexCreator = indexCreatorProvider.newTextIndexCreator(IndexCreationContext.builder()
-            .withColumnMetadata(columnMetadata).withIndexDir(segmentDirectory).build().forTextIndex(_fstType, true))) {
-      if (columnMetadata.isSingleValue()) {
-        processSVField(segmentWriter, hasDictionary, forwardIndexReader, readerContext, textIndexCreator, numDocs,
-            columnMetadata);
+    try (TextIndexCreator textIndexCreator = indexCreatorProvider.newTextIndexCreator(
+        IndexCreationContext.builder().withColumnMetadata(columnMetadata).withIndexDir(segmentDirectory).build()
+            .forTextIndex(_fstType, true))) {
+      boolean forwardIndexDisabled = !segmentWriter.hasIndexFor(columnName, ColumnIndexType.FORWARD_INDEX);
+      if (forwardIndexDisabled) {
+        try (Dictionary dictionary = LoaderUtils.getDictionary(segmentWriter, columnMetadata)) {
+          // Create the text 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 text index without forward index
+          Preconditions.checkState(dictionary.length() == 1, String.format("Creating text index for forward index "

Review Comment:
   Suggest not adding this part but directly throwing exception. I don't see the point of special handling generating text index for a column with all same values. Same for other index handlers.



##########
pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/index/metadata/ColumnMetadataImpl.java:
##########
@@ -348,6 +348,7 @@ public static class Builder {
     private PartitionFunction _partitionFunction;
     private Set<Integer> _partitions;
     private boolean _autoGenerated;
+    private boolean _forwardIndexDisabled;

Review Comment:
   Revert the changes in this file?



-- 
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 #9333: Add an option to disable the creation of the forward index for a column

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


##########
pinot-tools/src/main/java/org/apache/pinot/tools/segment/converter/DictionaryToRawIndexConverter.java:
##########
@@ -296,6 +297,10 @@ private void convertOneColumn(IndexSegment segment, String column, File newSegme
       throws IOException {
     DataSource dataSource = segment.getDataSource(column);
     ForwardIndexReader reader = dataSource.getForwardIndex();
+    if (reader == null) {

Review Comment:
   done



##########
pinot-core/src/main/java/org/apache/pinot/core/minion/RawIndexConverter.java:
##########
@@ -204,6 +204,11 @@ private void convertColumn(FieldSpec fieldSpec)
     // Create the raw index
     DataSource dataSource = _originalImmutableSegment.getDataSource(columnName);
     ForwardIndexReader reader = dataSource.getForwardIndex();
+    if (reader == null) {

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 #9333: Add an option to disable the creation of the forward index for a column

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


##########
pinot-tools/src/main/java/org/apache/pinot/tools/segment/converter/DictionaryToRawIndexConverter.java:
##########
@@ -296,6 +297,10 @@ private void convertOneColumn(IndexSegment segment, String column, File newSegme
       throws IOException {
     DataSource dataSource = segment.getDataSource(column);
     ForwardIndexReader reader = dataSource.getForwardIndex();
+    if (reader == null) {
+      throw new UnsupportedEncodingException(String.format("Forward index is disabled for column %s, cannot convert!",

Review Comment:
   oops, that was meant to be an `UnsupportedOperationException`. I've modified all of these to use the `Preconditions.checkState()` instead.



-- 
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 #9333: Add an option to disable the creation of the forward index for a column

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


-- 
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 #9333: Add an option to disable the creation of the forward index for a column

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


##########
pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/creator/impl/fwd/SingleValueNoOpForwardIndexCreator.java:
##########
@@ -0,0 +1,53 @@
+/**
+ * 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.segment.local.segment.creator.impl.fwd;
+
+import java.io.IOException;
+import org.apache.pinot.segment.spi.index.creator.ForwardIndexCreator;
+import org.apache.pinot.spi.data.FieldSpec;
+
+
+/**
+ * Forward index creator for dictionary-encoded single-value column with forward index disabled. This is a no-op.
+ */
+public class SingleValueNoOpForwardIndexCreator implements ForwardIndexCreator {
+  @Override
+  public boolean isDictionaryEncoded() {
+    return true;
+  }
+
+  @Override
+  public boolean isSingleValue() {
+    return true;
+  }
+
+  @Override
+  public FieldSpec.DataType getValueType() {
+    return FieldSpec.DataType.INT;
+  }
+
+  @Override
+  public void putDictId(int dictId) {

Review Comment:
   Same 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] somandal commented on a diff in pull request #9333: Add an option to disable the creation of the forward index for a column

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


##########
pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/datasource/DataSourceMetadata.java:
##########
@@ -70,6 +70,11 @@ default boolean isSingleValue() {
    */
   int getMaxNumValuesPerMVEntry();
 
+  /**
+   * Returns whether forward index is disabled for this column or not, returns 'true' if disabled
+   */
+  boolean forwardIndexDisabled();

Review Comment:
   i removed this as it isn't used at the moment



-- 
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 #9333: Add an option to disable the creation of the forward index for a column

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


##########
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:
   yes meant immutable segments, done



##########
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:
   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] Jackie-Jiang commented on a diff in pull request #9333: Add an option to disable the creation of the forward index for a column

Posted by GitBox <gi...@apache.org>.
Jackie-Jiang commented on code in PR #9333:
URL: https://github.com/apache/pinot/pull/9333#discussion_r966414059


##########
pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/creator/impl/SegmentColumnarIndexCreator.java:
##########
@@ -816,6 +835,7 @@ public static void addColumnMetadataInfo(PropertiesConfiguration properties, Str
         String.valueOf(columnIndexCreationInfo.getTotalNumberOfEntries()));
     properties.setProperty(getKeyFor(column, IS_AUTO_GENERATED),
         String.valueOf(columnIndexCreationInfo.isAutoGenerated()));
+    properties.setProperty(getKeyFor(column, FORWARD_INDEX_DISABLED), String.valueOf(forwardIndexDisabled));

Review Comment:
   Not sure if we need this in the segment metadata. We determine whether the index exists by reading the index file (e.g. there is no metadata for `HAS_INVERTED_INDEX`, `HAS_RANGE_INDEX` etc.)



##########
pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/creator/impl/DefaultIndexCreatorProvider.java:
##########
@@ -88,17 +90,25 @@ public ForwardIndexCreator newForwardIndexCreator(IndexCreationContext.Forward c
             context.getMaxRowLengthInBytes());
       }
     } else {
-      if (context.getFieldSpec().isSingleValueField()) {
-        if (context.isSorted()) {
-          return new SingleValueSortedForwardIndexCreator(context.getIndexDir(), context.getFieldSpec().getName(),
-              context.getCardinality());
+      if (context.forwardIndexDisabled()) {
+        if (context.getFieldSpec().isSingleValueField()) {
+          return new SingleValueNoOpForwardIndexCreator();

Review Comment:
   Since the creator is no-op, no need to separate the SV and MV. Ideally we should skip the adding value to the creator step, which has unnecessary overhead, but this can be addressed in the future.



##########
pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/datasource/DataSourceMetadata.java:
##########
@@ -70,6 +70,11 @@ default boolean isSingleValue() {
    */
   int getMaxNumValuesPerMVEntry();
 
+  /**
+   * Returns whether forward index is disabled for this column or not, returns 'true' if disabled
+   */
+  boolean forwardIndexDisabled();

Review Comment:
   I don't think we need this in the metadata. Instead, we can make `DateSource.getForwardIndex()` nullable, and perform proper null checks when it is used. This is consistent with how we handle other indexes.



##########
pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/creator/impl/SegmentColumnarIndexCreator.java:
##########
@@ -196,6 +204,15 @@ 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)

Review Comment:
   We should allow sorted column



##########
pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/readers/forward/NoOpMVForwardIndexReader.java:
##########
@@ -0,0 +1,66 @@
+/**
+ * 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.segment.local.segment.index.readers.forward;
+
+import java.io.IOException;
+import org.apache.pinot.segment.spi.index.reader.ForwardIndexReader;
+import org.apache.pinot.segment.spi.index.reader.ForwardIndexReaderContext;
+import org.apache.pinot.spi.data.FieldSpec;
+
+
+/**
+ * No-op forward index reader meant to be used by multi-value columns with forward index disabled. This is meant to
+ * provide clean exception handling for operations which need to read forward index when forward index is disabled.
+ */
+public class NoOpMVForwardIndexReader implements ForwardIndexReader<ForwardIndexReaderContext> {

Review Comment:
   Suggest not using no-op reader as this is not consistent with how we handle other index types. Instead, make `DataSource.getForwardIndex()` return `null` when the forward index does not exist. When forward index is needed to solve the query, check if the forward index is `null` and throw exception accordingly



-- 
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 #9333: Add an option to disable the creation of the forward index for a column

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


##########
pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/readers/forward/NoOpMVForwardIndexReader.java:
##########
@@ -0,0 +1,66 @@
+/**
+ * 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.segment.local.segment.index.readers.forward;
+
+import java.io.IOException;
+import org.apache.pinot.segment.spi.index.reader.ForwardIndexReader;
+import org.apache.pinot.segment.spi.index.reader.ForwardIndexReaderContext;
+import org.apache.pinot.spi.data.FieldSpec;
+
+
+/**
+ * No-op forward index reader meant to be used by multi-value columns with forward index disabled. This is meant to
+ * provide clean exception handling for operations which need to read forward index when forward index is disabled.
+ */
+public class NoOpMVForwardIndexReader implements ForwardIndexReader<ForwardIndexReaderContext> {

Review Comment:
   we discussed and agreed to remove the no-op reader. I've made these changes. thanks for your patience to discuss this @Jackie-Jiang and @siddharthteotia !



-- 
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 #9333: Add an option to disable the creation of the forward index for a column

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


##########
pinot-segment-local/src/main/java/org/apache/pinot/segment/local/realtime/converter/stats/MutableColumnStatistics.java:
##########
@@ -145,6 +146,8 @@ public boolean isSorted() {
 
     // Iterate over all data to figure out whether or not it's in sorted order
     MutableForwardIndex mutableForwardIndex = (MutableForwardIndex) _dataSource.getForwardIndex();
+    Preconditions.checkState(mutableForwardIndex != null,

Review Comment:
   Thanks for the review!
   
   I've done that on purpose. Where I throw `UnsupportedOperationException` are places where the forward index is allowed to be null due to disabling it. The exception is just trying to indicate that the query isn't supported.
   
   On the other hand, we don't allow disabling the forward index for `Mutable` segments at the moment. So I added the `Precondition` more as an assert that the forward index absolutely should exist otherwise it means something is very wrong.
   
   Let me know if you still think these should be unified or if the above reasoning seems alright.



-- 
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] Jackie-Jiang commented on a diff in pull request #9333: Add an option to disable the creation of the forward index for a column

Posted by GitBox <gi...@apache.org>.
Jackie-Jiang commented on code in PR #9333:
URL: https://github.com/apache/pinot/pull/9333#discussion_r974849707


##########
pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/loader/invertedindex/TextIndexHandler.java:
##########
@@ -174,18 +175,46 @@ private void createTextIndexForColumn(SegmentDirectory.Writer segmentWriter, Col
     // segmentDirectory is indicated to us by SegmentDirectoryPaths, we create lucene index there. There is no
     // further need to move around the lucene index directory since it is created with correct directory structure
     // based on segmentVersion.
-    try (ForwardIndexReader forwardIndexReader = LoaderUtils.getForwardIndexReader(segmentWriter, columnMetadata);
-        ForwardIndexReaderContext readerContext = forwardIndexReader.createContext();
-        TextIndexCreator textIndexCreator = indexCreatorProvider.newTextIndexCreator(IndexCreationContext.builder()
-            .withColumnMetadata(columnMetadata).withIndexDir(segmentDirectory).build().forTextIndex(_fstType, true))) {
-      if (columnMetadata.isSingleValue()) {
-        processSVField(segmentWriter, hasDictionary, forwardIndexReader, readerContext, textIndexCreator, numDocs,
-            columnMetadata);
+    try (TextIndexCreator textIndexCreator = indexCreatorProvider.newTextIndexCreator(
+        IndexCreationContext.builder().withColumnMetadata(columnMetadata).withIndexDir(segmentDirectory).build()
+            .forTextIndex(_fstType, true))) {
+      boolean forwardIndexDisabled = !segmentWriter.hasIndexFor(columnName, ColumnIndexType.FORWARD_INDEX);
+      if (forwardIndexDisabled) {
+        try (Dictionary dictionary = LoaderUtils.getDictionary(segmentWriter, columnMetadata)) {
+          // Create the text 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 text index without forward index
+          Preconditions.checkState(dictionary.length() == 1, String.format("Creating text index for forward index "

Review Comment:
   Hmm.. That is unexpected. When the dictionary size is 1, the column should be sorted, and we shouldn't skip the forward 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 commented on a diff in pull request #9333: Add an option to disable the creation of the forward index for a column

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


##########
pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/column/PhysicalColumnIndexContainer.java:
##########
@@ -133,7 +133,11 @@ public PhysicalColumnIndexContainer(SegmentDirectory.Reader segmentReader, Colum
       _rangeIndex = null;
     }
 
-    PinotDataBuffer fwdIndexBuffer = segmentReader.getIndexFor(columnName, ColumnIndexType.FORWARD_INDEX);
+    // Setting the 'fwdIndexBuffer' to null if forward index is enabled. No-op index readers will be setup for the

Review Comment:
   (nit) `typo enabled -> disabled` 
   
   > No-op index readers will be setup for the forward index disabled columns ...
   
   This is no longer the case I guess since we changed approach and rely on throwing exceptions. So might want to change the comment. 



-- 
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 #9333: Add an option to disable the creation of the forward index for a column

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


##########
pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/loader/defaultcolumn/BaseDefaultColumnHandler.java:
##########
@@ -372,6 +394,12 @@ protected boolean createColumnV1Indices(String column)
             return false;
           }
 
+          // TODO: Support forward index disabled derived column
+          if (_indexLoadingConfig.getForwardIndexDisabledColumns().contains(column)) {
+            LOGGER.warn("Skip creating forward index disabled derived column: {}", column);

Review Comment:
   In this case it is to warn and not throw error since user asked for a feature and we silently ignored (for now). 
   
   But in previous case, error should be thrown imo to tell user to change derived column config's source column or refresh the segments first to generate forward index on source column and then generate target derivedColumn



-- 
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 #9333: Add an option to disable the creation of the forward index for a column

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


##########
pinot-core/src/test/java/org/apache/pinot/queries/ForwardIndexDisabledSingleValueQueriesTest.java:
##########
@@ -0,0 +1,1088 @@
+/**
+ * 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.queries;
+
+import java.io.File;
+import java.net.URL;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.TimeUnit;
+import org.apache.commons.io.FileUtils;
+import org.apache.pinot.common.response.broker.BrokerResponseNative;
+import org.apache.pinot.common.response.broker.ResultTable;
+import org.apache.pinot.common.utils.DataSchema;
+import org.apache.pinot.segment.local.indexsegment.immutable.ImmutableSegmentLoader;
+import org.apache.pinot.segment.local.segment.creator.impl.SegmentIndexCreationDriverImpl;
+import org.apache.pinot.segment.local.segment.index.loader.IndexLoadingConfig;
+import org.apache.pinot.segment.spi.ColumnMetadata;
+import org.apache.pinot.segment.spi.ImmutableSegment;
+import org.apache.pinot.segment.spi.IndexSegment;
+import org.apache.pinot.segment.spi.creator.SegmentGeneratorConfig;
+import org.apache.pinot.segment.spi.creator.SegmentIndexCreationDriver;
+import org.apache.pinot.spi.config.table.FieldConfig;
+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.data.TimeGranularitySpec;
+import org.apache.pinot.spi.utils.ReadMode;
+import org.apache.pinot.spi.utils.builder.TableConfigBuilder;
+import org.testng.Assert;
+import org.testng.annotations.AfterClass;
+import org.testng.annotations.BeforeClass;
+import org.testng.annotations.Test;
+
+import static org.testng.Assert.assertEquals;
+import static org.testng.Assert.assertNotNull;
+import static org.testng.Assert.assertNull;
+import static org.testng.Assert.assertTrue;
+
+
+/**
+ * The <code>ForwardIndexDisabledSingleValueQueriesTest</code> class sets up the index segment for the no forward
+ * index single-value queries test.
+ * <p>There are totally 18 columns, 30000 records inside the original Avro file where 11 columns are selected to build
+ * the index segment. Selected columns information are as following:
+ * <ul>
+ *   ColumnName, FieldType, DataType, Cardinality, IsSorted, HasInvertedIndex, FwdIndexDisabled: S1, S2, HasRangeIndex
+ *   <li>column1, METRIC, INT, 6582, F, F, F, F, F</li>
+ *   <li>column3, METRIC, INT, 21910, F, F, F, F, F</li>
+ *   <li>column5, DIMENSION, STRING, 1, T, F, F, F, F</li>
+ *   <li>column6, DIMENSION, INT, 608, F, T, T, T, T</li>
+ *   <li>column7, DIMENSION, INT, 146, F, T, T, F, F</li>
+ *   <li>column9, DIMENSION, INT, 1737, F, F, F, F, F</li>
+ *   <li>column11, DIMENSION, STRING, 5, F, T, F, F, F</li>
+ *   <li>column12, DIMENSION, STRING, 5, F, F, F, F, F</li>
+ *   <li>column17, METRIC, INT, 24, F, T, F, F, F</li>
+ *   <li>column18, METRIC, INT, 1440, F, T, F, F, F</li>
+ *   <li>daysSinceEpoch, TIME, INT, 2, T, F, F, F, F</li>
+ * </ul>
+ */
+public class ForwardIndexDisabledSingleValueQueriesTest extends BaseQueriesTest {
+  private static final String AVRO_DATA = "data" + File.separator + "test_data-sv.avro";
+  private static final String SEGMENT_NAME_1 = "testTable_126164076_167572857";
+  private static final String SEGMENT_NAME_2 = "testTable_126164076_167572858";
+  private static final File INDEX_DIR = new File(FileUtils.getTempDirectory(),
+      "ForwardIndexDisabledSingleValueQueriesTest");
+
+  private static final String SELECT_STAR_QUERY = "SELECT * FROM testTable";
+
+  // Hard-coded query filter.
+  private static final String FILTER = " WHERE column1 > 100000000"
+      + " AND column3 BETWEEN 20000000 AND 1000000000"
+      + " AND column5 = 'gFuH'"
+      + " AND (column6 < 500000000 OR column11 NOT IN ('t', 'P'))"
+      + " AND daysSinceEpoch = 126164076";
+
+  private IndexSegment _indexSegment;
+  // Contains 2 index segments, one with 2 columns with forward index disabled, and the other with just 1.
+  private List<IndexSegment> _indexSegments;
+
+  private TableConfig _tableConfig;
+  private List<String> _invertedIndexColumns;
+  private List<String> _forwardIndexDisabledColumns;
+
+  @BeforeClass
+  public void buildAndLoadSegment()
+      throws Exception {
+    FileUtils.deleteQuietly(INDEX_DIR);
+
+    // Get resource file path.
+    URL resource = getClass().getClassLoader().getResource(AVRO_DATA);
+    assertNotNull(resource);
+    String filePath = resource.getFile();
+
+    // Build the segment schema.
+    Schema schema = new Schema.SchemaBuilder().setSchemaName("testTable").addMetric("column1", FieldSpec.DataType.INT)
+        .addMetric("column3", FieldSpec.DataType.INT).addSingleValueDimension("column5", FieldSpec.DataType.STRING)
+        .addSingleValueDimension("column6", FieldSpec.DataType.INT)
+        .addSingleValueDimension("column7", FieldSpec.DataType.INT)
+        .addSingleValueDimension("column9", FieldSpec.DataType.INT)
+        .addSingleValueDimension("column11", FieldSpec.DataType.STRING)
+        .addSingleValueDimension("column12", FieldSpec.DataType.STRING).addMetric("column17", FieldSpec.DataType.INT)
+        .addMetric("column18", FieldSpec.DataType.INT)
+        .addTime(new TimeGranularitySpec(FieldSpec.DataType.INT, TimeUnit.DAYS, "daysSinceEpoch"), null).build();
+
+    createSegment(filePath, SEGMENT_NAME_1, schema);
+    createSegment(filePath, SEGMENT_NAME_2, schema);
+
+    ImmutableSegment immutableSegment1 = loadSegmentWithMetadataChecks(SEGMENT_NAME_1);
+    ImmutableSegment immutableSegment2 = loadSegmentWithMetadataChecks(SEGMENT_NAME_2);
+
+    _indexSegment = immutableSegment1;
+    _indexSegments = Arrays.asList(immutableSegment1, immutableSegment2);
+  }
+
+  private void createSegment(String filePath, String segmentName, Schema schema)
+      throws Exception {
+    // Create field configs for the no forward index columns
+    List<FieldConfig> fieldConfigList = new ArrayList<>();
+    fieldConfigList.add(new FieldConfig("column6", FieldConfig.EncodingType.DICTIONARY, Collections.emptyList(), null,
+        Collections.singletonMap(FieldConfig.FORWARD_INDEX_DISABLED, Boolean.TRUE.toString())));
+    if (segmentName.equals(SEGMENT_NAME_1)) {
+      fieldConfigList.add(new FieldConfig("column7", FieldConfig.EncodingType.DICTIONARY, Collections.emptyList(),
+          null, Collections.singletonMap(FieldConfig.FORWARD_INDEX_DISABLED, Boolean.TRUE.toString())));
+
+      // Build table config based on segment 1 as it contains both columns under no forward index
+      _tableConfig =
+          new TableConfigBuilder(TableType.OFFLINE).setTableName("testTable").setTimeColumnName("daysSinceEpoch")
+              .setFieldConfigList(fieldConfigList).setRangeIndexColumns(Arrays.asList("column6")).build();
+    }
+
+    // Create the segment generator config.
+    SegmentGeneratorConfig segmentGeneratorConfig = new SegmentGeneratorConfig(_tableConfig, schema);
+    segmentGeneratorConfig.setInputFilePath(filePath);
+    segmentGeneratorConfig.setTableName("testTable");
+    segmentGeneratorConfig.setOutDir(INDEX_DIR.getAbsolutePath());
+    segmentGeneratorConfig.setSegmentName(segmentName);
+    // The segment generation code in SegmentColumnarIndexCreator will throw
+    // exception if start and end time in time column are not in acceptable
+    // range. For this test, we first need to fix the input avro data
+    // to have the time column values in allowed range. Until then, the check
+    // is explicitly disabled
+    segmentGeneratorConfig.setSkipTimeValueCheck(true);
+    _invertedIndexColumns = Arrays.asList("column6", "column7", "column11", "column17", "column18");
+    segmentGeneratorConfig.setInvertedIndexCreationColumns(_invertedIndexColumns);
+
+    _forwardIndexDisabledColumns = new ArrayList<>(Arrays.asList("column6", "column7"));
+    segmentGeneratorConfig.setForwardIndexDisabledColumns(_forwardIndexDisabledColumns);
+    segmentGeneratorConfig.setRangeIndexCreationColumns(Arrays.asList("column6"));
+
+    // Build the index segment.
+    SegmentIndexCreationDriver driver = new SegmentIndexCreationDriverImpl();
+    driver.init(segmentGeneratorConfig);
+    driver.build();
+  }
+
+  private ImmutableSegment loadSegmentWithMetadataChecks(String segmentName)
+      throws Exception {
+    IndexLoadingConfig indexLoadingConfig = new IndexLoadingConfig();
+    indexLoadingConfig.setTableConfig(_tableConfig);
+    indexLoadingConfig.setInvertedIndexColumns(new HashSet<>(_invertedIndexColumns));
+    indexLoadingConfig.setForwardIndexDisabledColumns(new HashSet<>(_forwardIndexDisabledColumns));
+    indexLoadingConfig.setRangeIndexColumns(new HashSet<>(Arrays.asList("column6")));
+    indexLoadingConfig.setReadMode(ReadMode.heap);
+
+    ImmutableSegment immutableSegment = ImmutableSegmentLoader.load(new File(INDEX_DIR, segmentName),
+        indexLoadingConfig);
+
+    Map<String, ColumnMetadata> columnMetadataMap1 = immutableSegment.getSegmentMetadata().getColumnMetadataMap();
+    columnMetadataMap1.forEach((column, metadata) -> {
+      if (column.equals("column6") || column.equals("column7")) {
+        assertTrue(metadata.hasDictionary());
+        assertTrue(metadata.isSingleValue());
+        assertNull(immutableSegment.getForwardIndex(column));
+      } else {
+        assertNotNull(immutableSegment.getForwardIndex(column));
+      }
+    });
+
+    return immutableSegment;
+  }
+
+  @AfterClass
+  public void deleteAndDestroySegment() {
+    FileUtils.deleteQuietly(INDEX_DIR);
+    _indexSegments.forEach((IndexSegment::destroy));
+  }
+
+  @Override
+  protected String getFilter() {
+    return FILTER;
+  }
+
+  @Override
+  protected IndexSegment getIndexSegment() {
+    return _indexSegment;
+  }
+
+  @Override
+  protected List<IndexSegment> getIndexSegments() {
+    return _indexSegments;
+  }
+
+  @Test
+  public void testSelectStarQueries() {
+    // Select * without any filters
+    try {
+      getBrokerResponse(SELECT_STAR_QUERY);
+      Assert.fail("Select * query should fail since forwardIndexDisabled on a select column");
+    } catch (IllegalStateException e) {
+      assertTrue(e.getMessage().contains("Forward index disabled for column:")
+          && e.getMessage().contains("cannot create DataFetcher!"));
+    }
+
+    // Select * with filters
+    try {
+      getBrokerResponse(SELECT_STAR_QUERY + FILTER);
+      Assert.fail("Select * query should fail since forwardIndexDisabled on a select column");
+    } catch (IllegalStateException e) {
+      assertTrue(e.getMessage().contains("Forward index disabled for column:")
+          && e.getMessage().contains("cannot create DataFetcher!"));
+    }
+  }
+
+  @Test
+  public void testSelectQueries() {
+    {
+      // Selection query without filters including a column with forwardIndexDisabled enabled on both segments
+      String query = "SELECT column1, column5, column6, column9, column11 FROM testTable";
+      try {
+        getBrokerResponse(query);
+        Assert.fail("Query should fail since forwardIndexDisabled on a select column");
+      } catch (IllegalStateException e) {
+        assertTrue(e.getMessage().contains("Forward index disabled for column:")
+            && e.getMessage().contains("cannot create DataFetcher!"));
+      }
+    }
+    {
+      // Selection query without filters including a column with forwardIndexDisabled enabled on one segment
+      String query = "SELECT column1, column5, column7, column9, column11 FROM testTable";
+      try {
+        getBrokerResponse(query);
+        Assert.fail("Query should fail since forwardIndexDisabled on a select column");
+      } catch (IllegalStateException e) {
+        assertTrue(e.getMessage().contains("Forward index disabled for column:")
+            && e.getMessage().contains("cannot create DataFetcher!"));
+      }
+    }
+    {
+      // Selection query with filters including a column with forwardIndexDisabled enabled on both segments
+      String query = "SELECT column1, column5, column6, column9, column11 FROM testTable WHERE column6 = 2147458029";
+      try {
+        getBrokerResponse(query);
+        Assert.fail("Query should fail since forwardIndexDisabled on a select column");
+      } catch (IllegalStateException e) {
+        assertTrue(e.getMessage().contains("Forward index disabled for column:")
+            && e.getMessage().contains("cannot create DataFetcher!"));
+      }
+    }
+    {
+      // Selection query with filters including a column with forwardIndexDisabled enabled on one segment
+      String query = "SELECT column1, column5, column7, column9, column11 FROM testTable WHERE column7 = 675695";
+      try {
+        getBrokerResponse(query);
+        Assert.fail("Query should fail since forwardIndexDisabled on a select column");
+      } catch (IllegalStateException e) {
+        assertTrue(e.getMessage().contains("Forward index disabled for column:")
+            && e.getMessage().contains("cannot create DataFetcher!"));
+      }
+    }
+    {
+      // Selection query without filters and without columns with forwardIndexDisabled enabled on either segment
+      String query = "SELECT column1, column5, column9, column11 FROM testTable ORDER BY column1";
+      BrokerResponseNative brokerResponseNative = getBrokerResponse(query);
+      assertTrue(brokerResponseNative.getProcessingExceptions() == null
+          || brokerResponseNative.getProcessingExceptions().size() == 0);
+      ResultTable resultTable = brokerResponseNative.getResultTable();
+      assertEquals(brokerResponseNative.getNumRowsResultSet(), 10);
+      assertEquals(brokerResponseNative.getTotalDocs(), 120_000L);
+      assertEquals(brokerResponseNative.getNumDocsScanned(), 120_000L);
+      assertEquals(brokerResponseNative.getNumSegmentsProcessed(), 4L);
+      assertEquals(brokerResponseNative.getNumSegmentsMatched(), 4L);
+      assertEquals(brokerResponseNative.getNumEntriesScannedPostFilter(), 120_120L);
+      assertEquals(brokerResponseNative.getNumEntriesScannedInFilter(), 0L);
+      assertNotNull(brokerResponseNative.getProcessingExceptions());
+      assertEquals(brokerResponseNative.getProcessingExceptions().size(), 0);
+      DataSchema dataSchema = new DataSchema(new String[]{"column1", "column5", "column9", "column11"},
+        new DataSchema.ColumnDataType[]{DataSchema.ColumnDataType.INT, DataSchema.ColumnDataType.STRING,
+            DataSchema.ColumnDataType.INT, DataSchema.ColumnDataType.STRING});
+      assertEquals(resultTable.getDataSchema(), dataSchema);
+      List<Object[]> resultRows = resultTable.getRows();
+      int previousColumn1 = Integer.MIN_VALUE;
+      for (Object[] resultRow : resultRows) {
+        assertEquals(resultRow.length, 4);
+        assertEquals((String) resultRow[1], "gFuH");
+        // Column 1
+        assertTrue((Integer) resultRow[0] >= previousColumn1);
+        previousColumn1 = (Integer) resultRow[0];
+      }
+
+      Object[] firstRow = resultRows.get(0);
+      // Column 11
+      assertEquals((String) firstRow[3], "o");
+    }
+    {
+      // Transform function on a selection clause with a forwardIndexDisabled column in transform
+      String query = "SELECT CONCAT(column6, column9, '-') from testTable";
+      try {
+        getBrokerResponse(query);
+        Assert.fail("Query should fail since forwardIndexDisabled on a column in transform");
+      } catch (IllegalStateException e) {
+        assertTrue(e.getMessage().contains("Forward index disabled for column:")
+            && e.getMessage().contains("cannot create DataFetcher!"));
+      }
+    }
+    {
+      // Transform function on a selection clause without a forwardIndexDisabled column in transform
+      String query = "SELECT CONCAT(column5, column9, '-') from testTable ORDER BY column1";

Review Comment:
   Can we add test for one nested transform ?



-- 
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 #9333: Add an option to disable the creation of the forward index for a column

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


##########
pinot-segment-local/src/main/java/org/apache/pinot/segment/local/utils/TableConfigUtils.java:
##########
@@ -897,6 +903,40 @@ private static void validateFieldConfigList(@Nullable List<FieldConfig> fieldCon
     }
   }
 
+  /**
+   * Validates the compatibility of the indexes if the column has the forward index disabled. Throws exceptions due to
+   * compatibility mismatch. The checks performed are:
+   *     - Validate dictionary is enabled.
+   *     - Validate inverted index is enabled.
+   *     - Validate that either no range index exists for column or the range index version is at least 2 and isn't a
+   *       multi-value column (since mulit-value defaults to index v1).
+   */
+  private static void validateForwardIndexDisabledIndexCompatibility(String columnName, FieldConfig fieldConfig,

Review Comment:
   Ok to keep for now but once we do the reload follow-ups, we should just keep this one because it is just duplicate code / harder to maintain



-- 
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 #9333: Add an option to disable the creation of the forward index for a column

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

   # [Codecov](https://codecov.io/gh/apache/pinot/pull/9333?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 [#9333](https://codecov.io/gh/apache/pinot/pull/9333?src=pr&el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (f7b445c) into [master](https://codecov.io/gh/apache/pinot/commit/6047b06ac6f62ab7349e854a94f50199512ad973?el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (6047b06) will **decrease** coverage by `13.13%`.
   > The diff coverage is `0.00%`.
   
   ```diff
   @@              Coverage Diff              @@
   ##             master    #9333       +/-   ##
   =============================================
   - Coverage     28.37%   15.23%   -13.14%     
   - Complexity       53      168      +115     
   =============================================
     Files          1867     1826       -41     
     Lines         99608    97801     -1807     
     Branches      15167    15007      -160     
   =============================================
   - Hits          28260    14899    -13361     
   - Misses        68608    81779    +13171     
   + Partials       2740     1123     -1617     
   ```
   
   | Flag | Coverage Δ | |
   |---|---|---|
   | integration1 | `?` | |
   | integration2 | `?` | |
   | unittests2 | `15.23% <0.00%> (?)` | |
   
   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/9333?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | Coverage Δ | |
   |---|---|---|
   | [...ment/creator/impl/DefaultIndexCreatorProvider.java](https://codecov.io/gh/apache/pinot/pull/9333/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-cGlub3Qtc2VnbWVudC1sb2NhbC9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3Qvc2VnbWVudC9sb2NhbC9zZWdtZW50L2NyZWF0b3IvaW1wbC9EZWZhdWx0SW5kZXhDcmVhdG9yUHJvdmlkZXIuamF2YQ==) | `0.00% <0.00%> (ø)` | |
   | [...ment/creator/impl/SegmentColumnarIndexCreator.java](https://codecov.io/gh/apache/pinot/pull/9333/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-cGlub3Qtc2VnbWVudC1sb2NhbC9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3Qvc2VnbWVudC9sb2NhbC9zZWdtZW50L2NyZWF0b3IvaW1wbC9TZWdtZW50Q29sdW1uYXJJbmRleENyZWF0b3IuamF2YQ==) | `0.00% <0.00%> (ø)` | |
   | [...or/impl/fwd/MultiValueNoOpForwardIndexCreator.java](https://codecov.io/gh/apache/pinot/pull/9333/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-cGlub3Qtc2VnbWVudC1sb2NhbC9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3Qvc2VnbWVudC9sb2NhbC9zZWdtZW50L2NyZWF0b3IvaW1wbC9md2QvTXVsdGlWYWx1ZU5vT3BGb3J3YXJkSW5kZXhDcmVhdG9yLmphdmE=) | `0.00% <0.00%> (ø)` | |
   | [...r/impl/fwd/SingleValueNoOpForwardIndexCreator.java](https://codecov.io/gh/apache/pinot/pull/9333/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-cGlub3Qtc2VnbWVudC1sb2NhbC9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3Qvc2VnbWVudC9sb2NhbC9zZWdtZW50L2NyZWF0b3IvaW1wbC9md2QvU2luZ2xlVmFsdWVOb09wRm9yd2FyZEluZGV4Q3JlYXRvci5qYXZh) | `0.00% <0.00%> (ø)` | |
   | [...ent/index/column/PhysicalColumnIndexContainer.java](https://codecov.io/gh/apache/pinot/pull/9333/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-cGlub3Qtc2VnbWVudC1sb2NhbC9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3Qvc2VnbWVudC9sb2NhbC9zZWdtZW50L2luZGV4L2NvbHVtbi9QaHlzaWNhbENvbHVtbkluZGV4Q29udGFpbmVyLmphdmE=) | `0.00% <0.00%> (ø)` | |
   | [...ocal/segment/index/datasource/EmptyDataSource.java](https://codecov.io/gh/apache/pinot/pull/9333/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-cGlub3Qtc2VnbWVudC1sb2NhbC9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3Qvc2VnbWVudC9sb2NhbC9zZWdtZW50L2luZGV4L2RhdGFzb3VyY2UvRW1wdHlEYXRhU291cmNlLmphdmE=) | `0.00% <0.00%> (ø)` | |
   | [.../segment/index/datasource/ImmutableDataSource.java](https://codecov.io/gh/apache/pinot/pull/9333/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-cGlub3Qtc2VnbWVudC1sb2NhbC9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3Qvc2VnbWVudC9sb2NhbC9zZWdtZW50L2luZGV4L2RhdGFzb3VyY2UvSW1tdXRhYmxlRGF0YVNvdXJjZS5qYXZh) | `0.00% <0.00%> (ø)` | |
   | [...al/segment/index/datasource/MutableDataSource.java](https://codecov.io/gh/apache/pinot/pull/9333/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-cGlub3Qtc2VnbWVudC1sb2NhbC9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3Qvc2VnbWVudC9sb2NhbC9zZWdtZW50L2luZGV4L2RhdGFzb3VyY2UvTXV0YWJsZURhdGFTb3VyY2UuamF2YQ==) | `0.00% <0.00%> (ø)` | |
   | [...local/segment/index/loader/IndexLoadingConfig.java](https://codecov.io/gh/apache/pinot/pull/9333/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-cGlub3Qtc2VnbWVudC1sb2NhbC9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3Qvc2VnbWVudC9sb2NhbC9zZWdtZW50L2luZGV4L2xvYWRlci9JbmRleExvYWRpbmdDb25maWcuamF2YQ==) | `0.00% <0.00%> (ø)` | |
   | [...loader/defaultcolumn/BaseDefaultColumnHandler.java](https://codecov.io/gh/apache/pinot/pull/9333/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-cGlub3Qtc2VnbWVudC1sb2NhbC9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3Qvc2VnbWVudC9sb2NhbC9zZWdtZW50L2luZGV4L2xvYWRlci9kZWZhdWx0Y29sdW1uL0Jhc2VEZWZhdWx0Q29sdW1uSGFuZGxlci5qYXZh) | `0.00% <0.00%> (ø)` | |
   | ... and [977 more](https://codecov.io/gh/apache/pinot/pull/9333/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


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

Posted by GitBox <gi...@apache.org>.
Jackie-Jiang commented on code in PR #9333:
URL: https://github.com/apache/pinot/pull/9333#discussion_r969258008


##########
pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/readers/forward/NoOpMVForwardIndexReader.java:
##########
@@ -0,0 +1,66 @@
+/**
+ * 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.segment.local.segment.index.readers.forward;
+
+import java.io.IOException;
+import org.apache.pinot.segment.spi.index.reader.ForwardIndexReader;
+import org.apache.pinot.segment.spi.index.reader.ForwardIndexReaderContext;
+import org.apache.pinot.spi.data.FieldSpec;
+
+
+/**
+ * No-op forward index reader meant to be used by multi-value columns with forward index disabled. This is meant to
+ * provide clean exception handling for operations which need to read forward index when forward index is disabled.
+ */
+public class NoOpMVForwardIndexReader implements ForwardIndexReader<ForwardIndexReaderContext> {

Review Comment:
   @siddharthteotia More thoughts on this? Adding a dummy reader can simplify the implementation for now, but in the future we will very likely need to change it to know whether the forward index exist (e.g when supporting re-generating forward index from inverted index). One way to do it is adding it to the data source metadata, but that will make the index handling inconsistent.



-- 
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] Jackie-Jiang commented on a diff in pull request #9333: Add an option to disable the creation of the forward index for a column

Posted by GitBox <gi...@apache.org>.
Jackie-Jiang commented on code in PR #9333:
URL: https://github.com/apache/pinot/pull/9333#discussion_r976058172


##########
pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/loader/invertedindex/TextIndexHandler.java:
##########
@@ -174,18 +175,46 @@ private void createTextIndexForColumn(SegmentDirectory.Writer segmentWriter, Col
     // segmentDirectory is indicated to us by SegmentDirectoryPaths, we create lucene index there. There is no
     // further need to move around the lucene index directory since it is created with correct directory structure
     // based on segmentVersion.
-    try (ForwardIndexReader forwardIndexReader = LoaderUtils.getForwardIndexReader(segmentWriter, columnMetadata);
-        ForwardIndexReaderContext readerContext = forwardIndexReader.createContext();
-        TextIndexCreator textIndexCreator = indexCreatorProvider.newTextIndexCreator(IndexCreationContext.builder()
-            .withColumnMetadata(columnMetadata).withIndexDir(segmentDirectory).build().forTextIndex(_fstType, true))) {
-      if (columnMetadata.isSingleValue()) {
-        processSVField(segmentWriter, hasDictionary, forwardIndexReader, readerContext, textIndexCreator, numDocs,
-            columnMetadata);
+    try (TextIndexCreator textIndexCreator = indexCreatorProvider.newTextIndexCreator(
+        IndexCreationContext.builder().withColumnMetadata(columnMetadata).withIndexDir(segmentDirectory).build()
+            .forTextIndex(_fstType, true))) {
+      boolean forwardIndexDisabled = !segmentWriter.hasIndexFor(columnName, ColumnIndexType.FORWARD_INDEX);
+      if (forwardIndexDisabled) {
+        try (Dictionary dictionary = LoaderUtils.getDictionary(segmentWriter, columnMetadata)) {
+          // Create the text 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 text index without forward index
+          Preconditions.checkState(dictionary.length() == 1, String.format("Creating text index for forward index "

Review Comment:
   I don't follow the second approach. For the default column where all the values are the same for a column, we should create sorted index and a single entry dictionary. This is the same behavior as generating a new segment with a column of all the same values.
   IMO second approach will actually cause inconsistent behavior because it will have inverted index with only a single entry, and the segment generated during the reload or from the raw data without a column will be different.
   I strongly suggest generating sorted index + dictionary for the default column.



-- 
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 #9333: Add an option to disable the creation of the forward index for a column

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


##########
pinot-segment-local/src/main/java/org/apache/pinot/segment/local/utils/TableConfigUtils.java:
##########
@@ -897,6 +903,40 @@ private static void validateFieldConfigList(@Nullable List<FieldConfig> fieldCon
     }
   }
 
+  /**
+   * Validates the compatibility of the indexes if the column has the forward index disabled. Throws exceptions due to
+   * compatibility mismatch. The checks performed are:
+   *     - Validate dictionary is enabled.
+   *     - Validate inverted index is enabled.
+   *     - Validate that either no range index exists for column or the range index version is at least 2 and isn't a
+   *       multi-value column (since mulit-value defaults to index v1).
+   */
+  private static void validateForwardIndexDisabledIndexCompatibility(String columnName, FieldConfig fieldConfig,
+      IndexingConfig indexingConfigs, List<String> noDictionaryColumns, Schema schema) {
+    Map<String, String> fieldConfigProperties = fieldConfig.getProperties();
+    if (fieldConfigProperties == null) {
+      return;
+    }
+
+    boolean forwardIndexDisabled = Boolean.parseBoolean(fieldConfigProperties.get(FieldConfig.FORWARD_INDEX_DISABLED));
+    if (!forwardIndexDisabled) {
+      return;
+    }
+
+    FieldSpec fieldSpec = schema.getFieldSpecFor(columnName);
+    Preconditions.checkState(noDictionaryColumns == null || !noDictionaryColumns.contains(columnName),
+        String.format("Forward index disabled column %s must have dictionary enabled", columnName));
+    Preconditions.checkState(indexingConfigs.getInvertedIndexColumns() != null
+            && indexingConfigs.getInvertedIndexColumns().contains(columnName),
+        String.format("Forward index disabled column %s must have inverted index enabled", columnName));
+    Preconditions.checkState(indexingConfigs.getRangeIndexColumns() == null

Review Comment:
   (nit) This check can be cleaned up / broken into 2 checks to make code readable



-- 
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 #9333: Add an option to disable the creation of the forward index for a column

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


##########
pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/column/PhysicalColumnIndexContainer.java:
##########
@@ -133,7 +133,11 @@ public PhysicalColumnIndexContainer(SegmentDirectory.Reader segmentReader, Colum
       _rangeIndex = null;
     }
 
-    PinotDataBuffer fwdIndexBuffer = segmentReader.getIndexFor(columnName, ColumnIndexType.FORWARD_INDEX);
+    // Setting the 'fwdIndexBuffer' to null if forward index is enabled. No-op index readers will be setup for the

Review Comment:
   (nit) `typo enabled -> disabled` 
   
   > No-op index readers will be setup for the forward index disabled columns ...
   
   This is no longer the case I guess since we rely on throwing exceptions. So might want to change the comment. 



-- 
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 #9333: Add an option to disable the creation of the forward index for a column

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


##########
pinot-core/src/test/java/org/apache/pinot/queries/ForwardIndexDisabledSingleValueQueriesTest.java:
##########
@@ -0,0 +1,1088 @@
+/**
+ * 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.queries;
+
+import java.io.File;
+import java.net.URL;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.TimeUnit;
+import org.apache.commons.io.FileUtils;
+import org.apache.pinot.common.response.broker.BrokerResponseNative;
+import org.apache.pinot.common.response.broker.ResultTable;
+import org.apache.pinot.common.utils.DataSchema;
+import org.apache.pinot.segment.local.indexsegment.immutable.ImmutableSegmentLoader;
+import org.apache.pinot.segment.local.segment.creator.impl.SegmentIndexCreationDriverImpl;
+import org.apache.pinot.segment.local.segment.index.loader.IndexLoadingConfig;
+import org.apache.pinot.segment.spi.ColumnMetadata;
+import org.apache.pinot.segment.spi.ImmutableSegment;
+import org.apache.pinot.segment.spi.IndexSegment;
+import org.apache.pinot.segment.spi.creator.SegmentGeneratorConfig;
+import org.apache.pinot.segment.spi.creator.SegmentIndexCreationDriver;
+import org.apache.pinot.spi.config.table.FieldConfig;
+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.data.TimeGranularitySpec;
+import org.apache.pinot.spi.utils.ReadMode;
+import org.apache.pinot.spi.utils.builder.TableConfigBuilder;
+import org.testng.Assert;
+import org.testng.annotations.AfterClass;
+import org.testng.annotations.BeforeClass;
+import org.testng.annotations.Test;
+
+import static org.testng.Assert.assertEquals;
+import static org.testng.Assert.assertNotNull;
+import static org.testng.Assert.assertNull;
+import static org.testng.Assert.assertTrue;
+
+
+/**
+ * The <code>ForwardIndexDisabledSingleValueQueriesTest</code> class sets up the index segment for the no forward
+ * index single-value queries test.
+ * <p>There are totally 18 columns, 30000 records inside the original Avro file where 11 columns are selected to build
+ * the index segment. Selected columns information are as following:
+ * <ul>
+ *   ColumnName, FieldType, DataType, Cardinality, IsSorted, HasInvertedIndex, FwdIndexDisabled: S1, S2, HasRangeIndex
+ *   <li>column1, METRIC, INT, 6582, F, F, F, F, F</li>
+ *   <li>column3, METRIC, INT, 21910, F, F, F, F, F</li>
+ *   <li>column5, DIMENSION, STRING, 1, T, F, F, F, F</li>
+ *   <li>column6, DIMENSION, INT, 608, F, T, T, T, T</li>
+ *   <li>column7, DIMENSION, INT, 146, F, T, T, F, F</li>
+ *   <li>column9, DIMENSION, INT, 1737, F, F, F, F, F</li>
+ *   <li>column11, DIMENSION, STRING, 5, F, T, F, F, F</li>
+ *   <li>column12, DIMENSION, STRING, 5, F, F, F, F, F</li>
+ *   <li>column17, METRIC, INT, 24, F, T, F, F, F</li>
+ *   <li>column18, METRIC, INT, 1440, F, T, F, F, F</li>
+ *   <li>daysSinceEpoch, TIME, INT, 2, T, F, F, F, F</li>
+ * </ul>
+ */
+public class ForwardIndexDisabledSingleValueQueriesTest extends BaseQueriesTest {
+  private static final String AVRO_DATA = "data" + File.separator + "test_data-sv.avro";
+  private static final String SEGMENT_NAME_1 = "testTable_126164076_167572857";
+  private static final String SEGMENT_NAME_2 = "testTable_126164076_167572858";
+  private static final File INDEX_DIR = new File(FileUtils.getTempDirectory(),
+      "ForwardIndexDisabledSingleValueQueriesTest");
+
+  private static final String SELECT_STAR_QUERY = "SELECT * FROM testTable";
+
+  // Hard-coded query filter.
+  private static final String FILTER = " WHERE column1 > 100000000"
+      + " AND column3 BETWEEN 20000000 AND 1000000000"
+      + " AND column5 = 'gFuH'"
+      + " AND (column6 < 500000000 OR column11 NOT IN ('t', 'P'))"
+      + " AND daysSinceEpoch = 126164076";
+
+  private IndexSegment _indexSegment;
+  // Contains 2 index segments, one with 2 columns with forward index disabled, and the other with just 1.
+  private List<IndexSegment> _indexSegments;
+
+  private TableConfig _tableConfig;
+  private List<String> _invertedIndexColumns;
+  private List<String> _forwardIndexDisabledColumns;
+
+  @BeforeClass
+  public void buildAndLoadSegment()
+      throws Exception {
+    FileUtils.deleteQuietly(INDEX_DIR);
+
+    // Get resource file path.
+    URL resource = getClass().getClassLoader().getResource(AVRO_DATA);
+    assertNotNull(resource);
+    String filePath = resource.getFile();
+
+    // Build the segment schema.
+    Schema schema = new Schema.SchemaBuilder().setSchemaName("testTable").addMetric("column1", FieldSpec.DataType.INT)
+        .addMetric("column3", FieldSpec.DataType.INT).addSingleValueDimension("column5", FieldSpec.DataType.STRING)
+        .addSingleValueDimension("column6", FieldSpec.DataType.INT)
+        .addSingleValueDimension("column7", FieldSpec.DataType.INT)
+        .addSingleValueDimension("column9", FieldSpec.DataType.INT)
+        .addSingleValueDimension("column11", FieldSpec.DataType.STRING)
+        .addSingleValueDimension("column12", FieldSpec.DataType.STRING).addMetric("column17", FieldSpec.DataType.INT)
+        .addMetric("column18", FieldSpec.DataType.INT)
+        .addTime(new TimeGranularitySpec(FieldSpec.DataType.INT, TimeUnit.DAYS, "daysSinceEpoch"), null).build();
+
+    createSegment(filePath, SEGMENT_NAME_1, schema);
+    createSegment(filePath, SEGMENT_NAME_2, schema);
+
+    ImmutableSegment immutableSegment1 = loadSegmentWithMetadataChecks(SEGMENT_NAME_1);
+    ImmutableSegment immutableSegment2 = loadSegmentWithMetadataChecks(SEGMENT_NAME_2);
+
+    _indexSegment = immutableSegment1;
+    _indexSegments = Arrays.asList(immutableSegment1, immutableSegment2);
+  }
+
+  private void createSegment(String filePath, String segmentName, Schema schema)
+      throws Exception {
+    // Create field configs for the no forward index columns
+    List<FieldConfig> fieldConfigList = new ArrayList<>();
+    fieldConfigList.add(new FieldConfig("column6", FieldConfig.EncodingType.DICTIONARY, Collections.emptyList(), null,
+        Collections.singletonMap(FieldConfig.FORWARD_INDEX_DISABLED, Boolean.TRUE.toString())));
+    if (segmentName.equals(SEGMENT_NAME_1)) {
+      fieldConfigList.add(new FieldConfig("column7", FieldConfig.EncodingType.DICTIONARY, Collections.emptyList(),
+          null, Collections.singletonMap(FieldConfig.FORWARD_INDEX_DISABLED, Boolean.TRUE.toString())));
+
+      // Build table config based on segment 1 as it contains both columns under no forward index
+      _tableConfig =
+          new TableConfigBuilder(TableType.OFFLINE).setTableName("testTable").setTimeColumnName("daysSinceEpoch")
+              .setFieldConfigList(fieldConfigList).setRangeIndexColumns(Arrays.asList("column6")).build();
+    }
+
+    // Create the segment generator config.
+    SegmentGeneratorConfig segmentGeneratorConfig = new SegmentGeneratorConfig(_tableConfig, schema);
+    segmentGeneratorConfig.setInputFilePath(filePath);
+    segmentGeneratorConfig.setTableName("testTable");
+    segmentGeneratorConfig.setOutDir(INDEX_DIR.getAbsolutePath());
+    segmentGeneratorConfig.setSegmentName(segmentName);
+    // The segment generation code in SegmentColumnarIndexCreator will throw
+    // exception if start and end time in time column are not in acceptable
+    // range. For this test, we first need to fix the input avro data
+    // to have the time column values in allowed range. Until then, the check
+    // is explicitly disabled
+    segmentGeneratorConfig.setSkipTimeValueCheck(true);
+    _invertedIndexColumns = Arrays.asList("column6", "column7", "column11", "column17", "column18");
+    segmentGeneratorConfig.setInvertedIndexCreationColumns(_invertedIndexColumns);
+
+    _forwardIndexDisabledColumns = new ArrayList<>(Arrays.asList("column6", "column7"));
+    segmentGeneratorConfig.setForwardIndexDisabledColumns(_forwardIndexDisabledColumns);
+    segmentGeneratorConfig.setRangeIndexCreationColumns(Arrays.asList("column6"));
+
+    // Build the index segment.
+    SegmentIndexCreationDriver driver = new SegmentIndexCreationDriverImpl();
+    driver.init(segmentGeneratorConfig);
+    driver.build();
+  }
+
+  private ImmutableSegment loadSegmentWithMetadataChecks(String segmentName)
+      throws Exception {
+    IndexLoadingConfig indexLoadingConfig = new IndexLoadingConfig();
+    indexLoadingConfig.setTableConfig(_tableConfig);
+    indexLoadingConfig.setInvertedIndexColumns(new HashSet<>(_invertedIndexColumns));
+    indexLoadingConfig.setForwardIndexDisabledColumns(new HashSet<>(_forwardIndexDisabledColumns));
+    indexLoadingConfig.setRangeIndexColumns(new HashSet<>(Arrays.asList("column6")));
+    indexLoadingConfig.setReadMode(ReadMode.heap);
+
+    ImmutableSegment immutableSegment = ImmutableSegmentLoader.load(new File(INDEX_DIR, segmentName),
+        indexLoadingConfig);
+
+    Map<String, ColumnMetadata> columnMetadataMap1 = immutableSegment.getSegmentMetadata().getColumnMetadataMap();
+    columnMetadataMap1.forEach((column, metadata) -> {
+      if (column.equals("column6") || column.equals("column7")) {
+        assertTrue(metadata.hasDictionary());
+        assertTrue(metadata.isSingleValue());
+        assertNull(immutableSegment.getForwardIndex(column));
+      } else {
+        assertNotNull(immutableSegment.getForwardIndex(column));
+      }
+    });
+
+    return immutableSegment;
+  }
+
+  @AfterClass
+  public void deleteAndDestroySegment() {
+    FileUtils.deleteQuietly(INDEX_DIR);
+    _indexSegments.forEach((IndexSegment::destroy));
+  }
+
+  @Override
+  protected String getFilter() {
+    return FILTER;
+  }
+
+  @Override
+  protected IndexSegment getIndexSegment() {
+    return _indexSegment;
+  }
+
+  @Override
+  protected List<IndexSegment> getIndexSegments() {
+    return _indexSegments;
+  }
+
+  @Test
+  public void testSelectStarQueries() {
+    // Select * without any filters
+    try {
+      getBrokerResponse(SELECT_STAR_QUERY);
+      Assert.fail("Select * query should fail since forwardIndexDisabled on a select column");
+    } catch (IllegalStateException e) {
+      assertTrue(e.getMessage().contains("Forward index disabled for column:")
+          && e.getMessage().contains("cannot create DataFetcher!"));
+    }
+
+    // Select * with filters
+    try {
+      getBrokerResponse(SELECT_STAR_QUERY + FILTER);
+      Assert.fail("Select * query should fail since forwardIndexDisabled on a select column");
+    } catch (IllegalStateException e) {
+      assertTrue(e.getMessage().contains("Forward index disabled for column:")
+          && e.getMessage().contains("cannot create DataFetcher!"));
+    }
+  }
+
+  @Test
+  public void testSelectQueries() {
+    {
+      // Selection query without filters including a column with forwardIndexDisabled enabled on both segments
+      String query = "SELECT column1, column5, column6, column9, column11 FROM testTable";
+      try {
+        getBrokerResponse(query);
+        Assert.fail("Query should fail since forwardIndexDisabled on a select column");
+      } catch (IllegalStateException e) {
+        assertTrue(e.getMessage().contains("Forward index disabled for column:")
+            && e.getMessage().contains("cannot create DataFetcher!"));
+      }
+    }
+    {
+      // Selection query without filters including a column with forwardIndexDisabled enabled on one segment
+      String query = "SELECT column1, column5, column7, column9, column11 FROM testTable";
+      try {
+        getBrokerResponse(query);
+        Assert.fail("Query should fail since forwardIndexDisabled on a select column");
+      } catch (IllegalStateException e) {
+        assertTrue(e.getMessage().contains("Forward index disabled for column:")
+            && e.getMessage().contains("cannot create DataFetcher!"));
+      }
+    }
+    {
+      // Selection query with filters including a column with forwardIndexDisabled enabled on both segments
+      String query = "SELECT column1, column5, column6, column9, column11 FROM testTable WHERE column6 = 2147458029";
+      try {
+        getBrokerResponse(query);
+        Assert.fail("Query should fail since forwardIndexDisabled on a select column");
+      } catch (IllegalStateException e) {
+        assertTrue(e.getMessage().contains("Forward index disabled for column:")
+            && e.getMessage().contains("cannot create DataFetcher!"));
+      }
+    }
+    {
+      // Selection query with filters including a column with forwardIndexDisabled enabled on one segment
+      String query = "SELECT column1, column5, column7, column9, column11 FROM testTable WHERE column7 = 675695";
+      try {
+        getBrokerResponse(query);
+        Assert.fail("Query should fail since forwardIndexDisabled on a select column");
+      } catch (IllegalStateException e) {
+        assertTrue(e.getMessage().contains("Forward index disabled for column:")
+            && e.getMessage().contains("cannot create DataFetcher!"));
+      }
+    }
+    {
+      // Selection query without filters and without columns with forwardIndexDisabled enabled on either segment
+      String query = "SELECT column1, column5, column9, column11 FROM testTable ORDER BY column1";
+      BrokerResponseNative brokerResponseNative = getBrokerResponse(query);
+      assertTrue(brokerResponseNative.getProcessingExceptions() == null
+          || brokerResponseNative.getProcessingExceptions().size() == 0);
+      ResultTable resultTable = brokerResponseNative.getResultTable();
+      assertEquals(brokerResponseNative.getNumRowsResultSet(), 10);
+      assertEquals(brokerResponseNative.getTotalDocs(), 120_000L);
+      assertEquals(brokerResponseNative.getNumDocsScanned(), 120_000L);
+      assertEquals(brokerResponseNative.getNumSegmentsProcessed(), 4L);
+      assertEquals(brokerResponseNative.getNumSegmentsMatched(), 4L);
+      assertEquals(brokerResponseNative.getNumEntriesScannedPostFilter(), 120_120L);
+      assertEquals(brokerResponseNative.getNumEntriesScannedInFilter(), 0L);
+      assertNotNull(brokerResponseNative.getProcessingExceptions());
+      assertEquals(brokerResponseNative.getProcessingExceptions().size(), 0);
+      DataSchema dataSchema = new DataSchema(new String[]{"column1", "column5", "column9", "column11"},
+        new DataSchema.ColumnDataType[]{DataSchema.ColumnDataType.INT, DataSchema.ColumnDataType.STRING,
+            DataSchema.ColumnDataType.INT, DataSchema.ColumnDataType.STRING});
+      assertEquals(resultTable.getDataSchema(), dataSchema);
+      List<Object[]> resultRows = resultTable.getRows();
+      int previousColumn1 = Integer.MIN_VALUE;
+      for (Object[] resultRow : resultRows) {
+        assertEquals(resultRow.length, 4);
+        assertEquals((String) resultRow[1], "gFuH");
+        // Column 1
+        assertTrue((Integer) resultRow[0] >= previousColumn1);
+        previousColumn1 = (Integer) resultRow[0];
+      }
+
+      Object[] firstRow = resultRows.get(0);
+      // Column 11
+      assertEquals((String) firstRow[3], "o");
+    }
+    {
+      // Transform function on a selection clause with a forwardIndexDisabled column in transform
+      String query = "SELECT CONCAT(column6, column9, '-') from testTable";

Review Comment:
   Can we add test for one nested transform ?



-- 
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 #9333: Add an option to disable the creation of the forward index for a column

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


##########
pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/loader/defaultcolumn/BaseDefaultColumnHandler.java:
##########
@@ -363,6 +379,12 @@ protected boolean createColumnV1Indices(String column)
                   argument);
               return false;
             }
+            // TODO: Support creation of derived columns from forward index disabled columns
+            if (!_segmentWriter.hasIndexFor(argument, ColumnIndexType.FORWARD_INDEX)) {
+              LOGGER.warn("Skip creating derived column: {} because argument: {} does not have a forward index", column,

Review Comment:
   Throwing error is better to force the user to not create a target `derivedColumn` from source `noForwardIndex` column until reload code path supports everything (including derived column handling possibly) ?



-- 
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] Jackie-Jiang commented on pull request #9333: Add an option to disable the creation of the forward index for a column

Posted by GitBox <gi...@apache.org>.
Jackie-Jiang commented on PR #9333:
URL: https://github.com/apache/pinot/pull/9333#issuecomment-1241110458

   We should allow disabling forward index for sorted column, which is a no-op because the inverted index for sorted column can be used as forward index. I believe it is already allowed in the implementation because the default column is always sorted. No need to add this in the validation.


-- 
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 #9333: Add an option to disable the creation of the forward index for a column

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


##########
pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/creator/impl/SegmentColumnarIndexCreator.java:
##########
@@ -196,6 +204,15 @@ 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), String.format("Cannot disable forward "
+            + "index for column: %s without dictionary and inverted index or which is sorted or which has range "
+            + "index with version < 2", columnName));

Review Comment:
   good point, done!
   I have addressed all comments, but will update the PR after I add some multi-value query tests



-- 
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] Jackie-Jiang commented on a diff in pull request #9333: Add an option to disable the creation of the forward index for a column

Posted by GitBox <gi...@apache.org>.
Jackie-Jiang commented on code in PR #9333:
URL: https://github.com/apache/pinot/pull/9333#discussion_r966440470


##########
pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/readers/forward/NoOpMVForwardIndexReader.java:
##########
@@ -0,0 +1,66 @@
+/**
+ * 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.segment.local.segment.index.readers.forward;
+
+import java.io.IOException;
+import org.apache.pinot.segment.spi.index.reader.ForwardIndexReader;
+import org.apache.pinot.segment.spi.index.reader.ForwardIndexReaderContext;
+import org.apache.pinot.spi.data.FieldSpec;
+
+
+/**
+ * No-op forward index reader meant to be used by multi-value columns with forward index disabled. This is meant to
+ * provide clean exception handling for operations which need to read forward index when forward index is disabled.
+ */
+public class NoOpMVForwardIndexReader implements ForwardIndexReader<ForwardIndexReaderContext> {

Review Comment:
   We should treat forward index the same as other type of indexes, instead of specializing it and always put a dummy reader for it. Let's say in the future I want to support GROUP BY with inverted index only, my branching will be `if (dataSource,getForwardIndex() == null) ...`, similar to when we choose to use the scan based filter when inverted index doesn't exist.
   Conceptually we are able to support all operations with inverted index only, so I don't want to specialize forward index just for the exception message.



-- 
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 #9333: Add an option to disable the creation of the forward index for a column

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


##########
pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/column/PhysicalColumnIndexContainer.java:
##########
@@ -133,7 +133,12 @@ public PhysicalColumnIndexContainer(SegmentDirectory.Reader segmentReader, Colum
       _rangeIndex = null;
     }
 
-    PinotDataBuffer fwdIndexBuffer = segmentReader.getIndexFor(columnName, ColumnIndexType.FORWARD_INDEX);
+    // Setting the 'fwdIndexBuffer' to null if forward index is enabled. No-op index readers will be setup for the
+    // forward index disabled columns which doesn't require the 'fwdIndexBuffer'.
+    boolean forwardIndexDisabled = !segmentReader.hasIndexFor(columnName, ColumnIndexType.FORWARD_INDEX);
+    PinotDataBuffer fwdIndexBuffer = forwardIndexDisabled ? null
+        : segmentReader.getIndexFor(columnName, ColumnIndexType.FORWARD_INDEX);

Review Comment:
   I can only do this if I put the `getIndexFor` in a `try catch` block. The underlying implementation of the `ColumnIndexDirectory` throws an exception if the index is not present in the `getIndexFor` call. Do you want me to add this in a `try catch` block like this?:
   
   ```
   PinotDataBuffer fwdIndexBuffer;
   try {
       fwdIndexBuffer = segmentReader.getIndexFor(columnName, ColumnIndexType.FORWARD_INDEX);
   } catch (Exception e) {
       fwdIndexBuffer = null;
   }
   ```
   
   e.g. of the type of exception thrown down the line (`SingleFileIndexDirectory`):
   
   ```
     private PinotDataBuffer checkAndGetIndexBuffer(String column, ColumnIndexType type) {
       IndexKey key = new IndexKey(column, type);
       IndexEntry entry = _columnEntries.get(key);
       if (entry == null || entry._buffer == null) {
         throw new RuntimeException(
             "Could not find index for column: " + column + ", type: " + type + ", segment: " + _segmentDirectory
                 .toString());
       }
       return entry._buffer;
     }
   ```



##########
pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/loader/IndexLoadingConfig.java:
##########
@@ -315,6 +341,14 @@ public List<String> getSortedColumns() {
     return _sortedColumns;
   }
 
+  /**
+   * For tests only.
+   */
+  @VisibleForTesting
+  public void setSortedColumns(List<String> sortedColumns) {

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 #9333: Add an option to disable the creation of the forward index for a column

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


##########
pinot-core/src/test/java/org/apache/pinot/queries/ForwardIndexDisabledSingleValueQueriesTest.java:
##########
@@ -0,0 +1,1088 @@
+/**
+ * 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.queries;
+
+import java.io.File;
+import java.net.URL;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.TimeUnit;
+import org.apache.commons.io.FileUtils;
+import org.apache.pinot.common.response.broker.BrokerResponseNative;
+import org.apache.pinot.common.response.broker.ResultTable;
+import org.apache.pinot.common.utils.DataSchema;
+import org.apache.pinot.segment.local.indexsegment.immutable.ImmutableSegmentLoader;
+import org.apache.pinot.segment.local.segment.creator.impl.SegmentIndexCreationDriverImpl;
+import org.apache.pinot.segment.local.segment.index.loader.IndexLoadingConfig;
+import org.apache.pinot.segment.spi.ColumnMetadata;
+import org.apache.pinot.segment.spi.ImmutableSegment;
+import org.apache.pinot.segment.spi.IndexSegment;
+import org.apache.pinot.segment.spi.creator.SegmentGeneratorConfig;
+import org.apache.pinot.segment.spi.creator.SegmentIndexCreationDriver;
+import org.apache.pinot.spi.config.table.FieldConfig;
+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.data.TimeGranularitySpec;
+import org.apache.pinot.spi.utils.ReadMode;
+import org.apache.pinot.spi.utils.builder.TableConfigBuilder;
+import org.testng.Assert;
+import org.testng.annotations.AfterClass;
+import org.testng.annotations.BeforeClass;
+import org.testng.annotations.Test;
+
+import static org.testng.Assert.assertEquals;
+import static org.testng.Assert.assertNotNull;
+import static org.testng.Assert.assertNull;
+import static org.testng.Assert.assertTrue;
+
+
+/**
+ * The <code>ForwardIndexDisabledSingleValueQueriesTest</code> class sets up the index segment for the no forward
+ * index single-value queries test.
+ * <p>There are totally 18 columns, 30000 records inside the original Avro file where 11 columns are selected to build
+ * the index segment. Selected columns information are as following:
+ * <ul>
+ *   ColumnName, FieldType, DataType, Cardinality, IsSorted, HasInvertedIndex, FwdIndexDisabled: S1, S2, HasRangeIndex
+ *   <li>column1, METRIC, INT, 6582, F, F, F, F, F</li>
+ *   <li>column3, METRIC, INT, 21910, F, F, F, F, F</li>
+ *   <li>column5, DIMENSION, STRING, 1, T, F, F, F, F</li>
+ *   <li>column6, DIMENSION, INT, 608, F, T, T, T, T</li>
+ *   <li>column7, DIMENSION, INT, 146, F, T, T, F, F</li>
+ *   <li>column9, DIMENSION, INT, 1737, F, F, F, F, F</li>
+ *   <li>column11, DIMENSION, STRING, 5, F, T, F, F, F</li>
+ *   <li>column12, DIMENSION, STRING, 5, F, F, F, F, F</li>
+ *   <li>column17, METRIC, INT, 24, F, T, F, F, F</li>
+ *   <li>column18, METRIC, INT, 1440, F, T, F, F, F</li>
+ *   <li>daysSinceEpoch, TIME, INT, 2, T, F, F, F, F</li>
+ * </ul>
+ */
+public class ForwardIndexDisabledSingleValueQueriesTest extends BaseQueriesTest {
+  private static final String AVRO_DATA = "data" + File.separator + "test_data-sv.avro";
+  private static final String SEGMENT_NAME_1 = "testTable_126164076_167572857";
+  private static final String SEGMENT_NAME_2 = "testTable_126164076_167572858";
+  private static final File INDEX_DIR = new File(FileUtils.getTempDirectory(),
+      "ForwardIndexDisabledSingleValueQueriesTest");
+
+  private static final String SELECT_STAR_QUERY = "SELECT * FROM testTable";
+
+  // Hard-coded query filter.
+  private static final String FILTER = " WHERE column1 > 100000000"
+      + " AND column3 BETWEEN 20000000 AND 1000000000"
+      + " AND column5 = 'gFuH'"
+      + " AND (column6 < 500000000 OR column11 NOT IN ('t', 'P'))"
+      + " AND daysSinceEpoch = 126164076";
+
+  private IndexSegment _indexSegment;
+  // Contains 2 index segments, one with 2 columns with forward index disabled, and the other with just 1.
+  private List<IndexSegment> _indexSegments;
+
+  private TableConfig _tableConfig;
+  private List<String> _invertedIndexColumns;
+  private List<String> _forwardIndexDisabledColumns;
+
+  @BeforeClass
+  public void buildAndLoadSegment()
+      throws Exception {
+    FileUtils.deleteQuietly(INDEX_DIR);
+
+    // Get resource file path.
+    URL resource = getClass().getClassLoader().getResource(AVRO_DATA);
+    assertNotNull(resource);
+    String filePath = resource.getFile();
+
+    // Build the segment schema.
+    Schema schema = new Schema.SchemaBuilder().setSchemaName("testTable").addMetric("column1", FieldSpec.DataType.INT)
+        .addMetric("column3", FieldSpec.DataType.INT).addSingleValueDimension("column5", FieldSpec.DataType.STRING)
+        .addSingleValueDimension("column6", FieldSpec.DataType.INT)
+        .addSingleValueDimension("column7", FieldSpec.DataType.INT)
+        .addSingleValueDimension("column9", FieldSpec.DataType.INT)
+        .addSingleValueDimension("column11", FieldSpec.DataType.STRING)
+        .addSingleValueDimension("column12", FieldSpec.DataType.STRING).addMetric("column17", FieldSpec.DataType.INT)
+        .addMetric("column18", FieldSpec.DataType.INT)
+        .addTime(new TimeGranularitySpec(FieldSpec.DataType.INT, TimeUnit.DAYS, "daysSinceEpoch"), null).build();
+
+    createSegment(filePath, SEGMENT_NAME_1, schema);
+    createSegment(filePath, SEGMENT_NAME_2, schema);
+
+    ImmutableSegment immutableSegment1 = loadSegmentWithMetadataChecks(SEGMENT_NAME_1);
+    ImmutableSegment immutableSegment2 = loadSegmentWithMetadataChecks(SEGMENT_NAME_2);
+
+    _indexSegment = immutableSegment1;
+    _indexSegments = Arrays.asList(immutableSegment1, immutableSegment2);
+  }
+
+  private void createSegment(String filePath, String segmentName, Schema schema)
+      throws Exception {
+    // Create field configs for the no forward index columns
+    List<FieldConfig> fieldConfigList = new ArrayList<>();
+    fieldConfigList.add(new FieldConfig("column6", FieldConfig.EncodingType.DICTIONARY, Collections.emptyList(), null,
+        Collections.singletonMap(FieldConfig.FORWARD_INDEX_DISABLED, Boolean.TRUE.toString())));
+    if (segmentName.equals(SEGMENT_NAME_1)) {
+      fieldConfigList.add(new FieldConfig("column7", FieldConfig.EncodingType.DICTIONARY, Collections.emptyList(),
+          null, Collections.singletonMap(FieldConfig.FORWARD_INDEX_DISABLED, Boolean.TRUE.toString())));
+
+      // Build table config based on segment 1 as it contains both columns under no forward index
+      _tableConfig =
+          new TableConfigBuilder(TableType.OFFLINE).setTableName("testTable").setTimeColumnName("daysSinceEpoch")
+              .setFieldConfigList(fieldConfigList).setRangeIndexColumns(Arrays.asList("column6")).build();
+    }
+
+    // Create the segment generator config.
+    SegmentGeneratorConfig segmentGeneratorConfig = new SegmentGeneratorConfig(_tableConfig, schema);
+    segmentGeneratorConfig.setInputFilePath(filePath);
+    segmentGeneratorConfig.setTableName("testTable");
+    segmentGeneratorConfig.setOutDir(INDEX_DIR.getAbsolutePath());
+    segmentGeneratorConfig.setSegmentName(segmentName);
+    // The segment generation code in SegmentColumnarIndexCreator will throw
+    // exception if start and end time in time column are not in acceptable
+    // range. For this test, we first need to fix the input avro data
+    // to have the time column values in allowed range. Until then, the check
+    // is explicitly disabled
+    segmentGeneratorConfig.setSkipTimeValueCheck(true);
+    _invertedIndexColumns = Arrays.asList("column6", "column7", "column11", "column17", "column18");
+    segmentGeneratorConfig.setInvertedIndexCreationColumns(_invertedIndexColumns);
+
+    _forwardIndexDisabledColumns = new ArrayList<>(Arrays.asList("column6", "column7"));
+    segmentGeneratorConfig.setForwardIndexDisabledColumns(_forwardIndexDisabledColumns);
+    segmentGeneratorConfig.setRangeIndexCreationColumns(Arrays.asList("column6"));
+
+    // Build the index segment.
+    SegmentIndexCreationDriver driver = new SegmentIndexCreationDriverImpl();
+    driver.init(segmentGeneratorConfig);
+    driver.build();
+  }
+
+  private ImmutableSegment loadSegmentWithMetadataChecks(String segmentName)
+      throws Exception {
+    IndexLoadingConfig indexLoadingConfig = new IndexLoadingConfig();
+    indexLoadingConfig.setTableConfig(_tableConfig);
+    indexLoadingConfig.setInvertedIndexColumns(new HashSet<>(_invertedIndexColumns));
+    indexLoadingConfig.setForwardIndexDisabledColumns(new HashSet<>(_forwardIndexDisabledColumns));
+    indexLoadingConfig.setRangeIndexColumns(new HashSet<>(Arrays.asList("column6")));
+    indexLoadingConfig.setReadMode(ReadMode.heap);
+
+    ImmutableSegment immutableSegment = ImmutableSegmentLoader.load(new File(INDEX_DIR, segmentName),
+        indexLoadingConfig);
+
+    Map<String, ColumnMetadata> columnMetadataMap1 = immutableSegment.getSegmentMetadata().getColumnMetadataMap();
+    columnMetadataMap1.forEach((column, metadata) -> {
+      if (column.equals("column6") || column.equals("column7")) {
+        assertTrue(metadata.hasDictionary());
+        assertTrue(metadata.isSingleValue());
+        assertNull(immutableSegment.getForwardIndex(column));
+      } else {
+        assertNotNull(immutableSegment.getForwardIndex(column));
+      }
+    });
+
+    return immutableSegment;
+  }
+
+  @AfterClass
+  public void deleteAndDestroySegment() {
+    FileUtils.deleteQuietly(INDEX_DIR);
+    _indexSegments.forEach((IndexSegment::destroy));
+  }
+
+  @Override
+  protected String getFilter() {
+    return FILTER;
+  }
+
+  @Override
+  protected IndexSegment getIndexSegment() {
+    return _indexSegment;
+  }
+
+  @Override
+  protected List<IndexSegment> getIndexSegments() {
+    return _indexSegments;
+  }
+
+  @Test
+  public void testSelectStarQueries() {
+    // Select * without any filters
+    try {
+      getBrokerResponse(SELECT_STAR_QUERY);
+      Assert.fail("Select * query should fail since forwardIndexDisabled on a select column");
+    } catch (IllegalStateException e) {
+      assertTrue(e.getMessage().contains("Forward index disabled for column:")
+          && e.getMessage().contains("cannot create DataFetcher!"));
+    }
+
+    // Select * with filters
+    try {
+      getBrokerResponse(SELECT_STAR_QUERY + FILTER);
+      Assert.fail("Select * query should fail since forwardIndexDisabled on a select column");
+    } catch (IllegalStateException e) {
+      assertTrue(e.getMessage().contains("Forward index disabled for column:")
+          && e.getMessage().contains("cannot create DataFetcher!"));
+    }
+  }
+
+  @Test
+  public void testSelectQueries() {
+    {
+      // Selection query without filters including a column with forwardIndexDisabled enabled on both segments
+      String query = "SELECT column1, column5, column6, column9, column11 FROM testTable";
+      try {
+        getBrokerResponse(query);
+        Assert.fail("Query should fail since forwardIndexDisabled on a select column");
+      } catch (IllegalStateException e) {
+        assertTrue(e.getMessage().contains("Forward index disabled for column:")
+            && e.getMessage().contains("cannot create DataFetcher!"));
+      }
+    }
+    {
+      // Selection query without filters including a column with forwardIndexDisabled enabled on one segment
+      String query = "SELECT column1, column5, column7, column9, column11 FROM testTable";
+      try {
+        getBrokerResponse(query);
+        Assert.fail("Query should fail since forwardIndexDisabled on a select column");
+      } catch (IllegalStateException e) {
+        assertTrue(e.getMessage().contains("Forward index disabled for column:")
+            && e.getMessage().contains("cannot create DataFetcher!"));
+      }
+    }
+    {
+      // Selection query with filters including a column with forwardIndexDisabled enabled on both segments
+      String query = "SELECT column1, column5, column6, column9, column11 FROM testTable WHERE column6 = 2147458029";
+      try {
+        getBrokerResponse(query);
+        Assert.fail("Query should fail since forwardIndexDisabled on a select column");
+      } catch (IllegalStateException e) {
+        assertTrue(e.getMessage().contains("Forward index disabled for column:")
+            && e.getMessage().contains("cannot create DataFetcher!"));
+      }
+    }
+    {
+      // Selection query with filters including a column with forwardIndexDisabled enabled on one segment
+      String query = "SELECT column1, column5, column7, column9, column11 FROM testTable WHERE column7 = 675695";
+      try {
+        getBrokerResponse(query);
+        Assert.fail("Query should fail since forwardIndexDisabled on a select column");
+      } catch (IllegalStateException e) {
+        assertTrue(e.getMessage().contains("Forward index disabled for column:")
+            && e.getMessage().contains("cannot create DataFetcher!"));
+      }
+    }
+    {
+      // Selection query without filters and without columns with forwardIndexDisabled enabled on either segment
+      String query = "SELECT column1, column5, column9, column11 FROM testTable ORDER BY column1";
+      BrokerResponseNative brokerResponseNative = getBrokerResponse(query);
+      assertTrue(brokerResponseNative.getProcessingExceptions() == null
+          || brokerResponseNative.getProcessingExceptions().size() == 0);
+      ResultTable resultTable = brokerResponseNative.getResultTable();
+      assertEquals(brokerResponseNative.getNumRowsResultSet(), 10);
+      assertEquals(brokerResponseNative.getTotalDocs(), 120_000L);
+      assertEquals(brokerResponseNative.getNumDocsScanned(), 120_000L);
+      assertEquals(brokerResponseNative.getNumSegmentsProcessed(), 4L);
+      assertEquals(brokerResponseNative.getNumSegmentsMatched(), 4L);
+      assertEquals(brokerResponseNative.getNumEntriesScannedPostFilter(), 120_120L);
+      assertEquals(brokerResponseNative.getNumEntriesScannedInFilter(), 0L);
+      assertNotNull(brokerResponseNative.getProcessingExceptions());
+      assertEquals(brokerResponseNative.getProcessingExceptions().size(), 0);
+      DataSchema dataSchema = new DataSchema(new String[]{"column1", "column5", "column9", "column11"},
+        new DataSchema.ColumnDataType[]{DataSchema.ColumnDataType.INT, DataSchema.ColumnDataType.STRING,
+            DataSchema.ColumnDataType.INT, DataSchema.ColumnDataType.STRING});
+      assertEquals(resultTable.getDataSchema(), dataSchema);
+      List<Object[]> resultRows = resultTable.getRows();
+      int previousColumn1 = Integer.MIN_VALUE;
+      for (Object[] resultRow : resultRows) {
+        assertEquals(resultRow.length, 4);
+        assertEquals((String) resultRow[1], "gFuH");
+        // Column 1
+        assertTrue((Integer) resultRow[0] >= previousColumn1);
+        previousColumn1 = (Integer) resultRow[0];
+      }
+
+      Object[] firstRow = resultRows.get(0);
+      // Column 11
+      assertEquals((String) firstRow[3], "o");
+    }
+    {
+      // Transform function on a selection clause with a forwardIndexDisabled column in transform
+      String query = "SELECT CONCAT(column6, column9, '-') from testTable";

Review Comment:
   Yep, this is good. Just wanted coverage for transform inside a transform or transform inside an aggregate



-- 
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 #9333: Add an option to disable the creation of the forward index for a column

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


##########
pinot-segment-local/src/main/java/org/apache/pinot/segment/local/utils/TableConfigUtils.java:
##########
@@ -897,6 +903,44 @@ private static void validateFieldConfigList(@Nullable List<FieldConfig> fieldCon
     }
   }
 
+  /**
+   * Validates the compatibility of the indexes if the column has the forward index disabled. Throws exceptions due to
+   * compatibility mismatch. The checks performed are:
+   *     - Validate dictionary is enabled.
+   *     - Validate inverted index is enabled.
+   *     - Validate that either no range index exists for column or the range index version is at least 2 and isn't a
+   *       multi-value column (since mulit-value defaults to index v1).
+   */
+  private static void validateForwardIndexDisabledIndexCompatibility(String columnName, FieldConfig fieldConfig,
+      IndexingConfig indexingConfigs, List<String> noDictionaryColumns, Schema schema) {
+    Map<String, String> fieldConfigProperties = fieldConfig.getProperties();
+    if (fieldConfigProperties == null) {
+      return;
+    }
+
+    boolean forwardIndexDisabled = Boolean.parseBoolean(fieldConfigProperties.get(FieldConfig.FORWARD_INDEX_DISABLED));
+    if (!forwardIndexDisabled) {
+      return;
+    }
+
+    FieldSpec fieldSpec = schema.getFieldSpecFor(columnName);
+    Preconditions.checkState(fieldConfig.getEncodingType() == FieldConfig.EncodingType.DICTIONARY
+            || noDictionaryColumns == null || !noDictionaryColumns.contains(columnName),
+        String.format("Forward index disabled column %s must have dictionary enabled", columnName));

Review Comment:
   Sorry my bad. I misread the code :) Please ignore
   
   This is preconditions so you are looking for condition to evaluate to TRUE



-- 
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 #9333: Add an option to disable the creation of the forward index for a column

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


##########
pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/loader/defaultcolumn/BaseDefaultColumnHandler.java:
##########
@@ -372,6 +394,12 @@ protected boolean createColumnV1Indices(String column)
             return false;
           }
 
+          // TODO: Support forward index disabled derived column
+          if (_indexLoadingConfig.getForwardIndexDisabledColumns().contains(column)) {
+            LOGGER.warn("Skip creating forward index disabled derived column: {}", column);

Review Comment:
   sure updated the error message in the previous case to indicate next steps for user



-- 
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 #9333: Add an option to disable the creation of the forward index for a column

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


##########
pinot-segment-local/src/main/java/org/apache/pinot/segment/local/utils/TableConfigUtils.java:
##########
@@ -897,6 +903,44 @@ private static void validateFieldConfigList(@Nullable List<FieldConfig> fieldCon
     }
   }
 
+  /**
+   * Validates the compatibility of the indexes if the column has the forward index disabled. Throws exceptions due to
+   * compatibility mismatch. The checks performed are:
+   *     - Validate dictionary is enabled.
+   *     - Validate inverted index is enabled.
+   *     - Validate that either no range index exists for column or the range index version is at least 2 and isn't a
+   *       multi-value column (since mulit-value defaults to index v1).
+   */
+  private static void validateForwardIndexDisabledIndexCompatibility(String columnName, FieldConfig fieldConfig,
+      IndexingConfig indexingConfigs, List<String> noDictionaryColumns, Schema schema) {
+    Map<String, String> fieldConfigProperties = fieldConfig.getProperties();
+    if (fieldConfigProperties == null) {
+      return;
+    }
+
+    boolean forwardIndexDisabled = Boolean.parseBoolean(fieldConfigProperties.get(FieldConfig.FORWARD_INDEX_DISABLED));
+    if (!forwardIndexDisabled) {
+      return;
+    }
+
+    FieldSpec fieldSpec = schema.getFieldSpecFor(columnName);
+    Preconditions.checkState(fieldConfig.getEncodingType() == FieldConfig.EncodingType.DICTIONARY
+            || noDictionaryColumns == null || !noDictionaryColumns.contains(columnName),
+        String.format("Forward index disabled column %s must have dictionary enabled", columnName));

Review Comment:
   @somandal  - this is incorrect. 
   
   Error for this scenario should be thrown if column is marked noDict -- either `encodingType is RAW` or `noDictionaryColumns` contains the column.
   
   This code seems to be throwing error for acceptable scenario -- column marked with `DICTIONARY encodingType` or if it is not present in `noDictionaryColumns` unless I am misreading 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] somandal commented on a diff in pull request #9333: Add an option to disable the creation of the forward index for a column

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


##########
pinot-spi/src/main/java/org/apache/pinot/spi/config/table/TableConfig.java:
##########
@@ -58,6 +58,11 @@ public class TableConfig extends BaseJsonConfig {
   // Double underscore is reserved for real-time segment name delimiter
   private static final String TABLE_NAME_FORBIDDEN_SUBSTRING = "__";
 
+  // TODO: Remove this flag once the reload path to create forward index from inverted index and dictionary is
+  //       added. This feature will be disabled until the reload path is updated to handle forward index enable ->
+  //       disable and forward index disable -> enable.
+  public static boolean _disallowForwardIndexDisabled = true;

Review Comment:
   done



##########
pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/index/metadata/ColumnMetadataImpl.java:
##########
@@ -348,6 +348,7 @@ public static class Builder {
     private PartitionFunction _partitionFunction;
     private Set<Integer> _partitions;
     private boolean _autoGenerated;
+    private boolean _forwardIndexDisabled;

Review Comment:
   oops, good point. 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] jackjlli commented on a diff in pull request #9333: Add an option to disable the creation of the forward index for a column

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


##########
pinot-segment-local/src/main/java/org/apache/pinot/segment/local/realtime/converter/stats/MutableColumnStatistics.java:
##########
@@ -145,6 +146,8 @@ public boolean isSorted() {
 
     // Iterate over all data to figure out whether or not it's in sorted order
     MutableForwardIndex mutableForwardIndex = (MutableForwardIndex) _dataSource.getForwardIndex();
+    Preconditions.checkState(mutableForwardIndex != null,

Review Comment:
   I saw that in some places `UnsupportedOperationException` is thrown and in others the precondition is used. Should we consider unifying them?



##########
pinot-tools/src/main/java/org/apache/pinot/tools/segment/converter/DictionaryToRawIndexConverter.java:
##########
@@ -296,6 +297,10 @@ private void convertOneColumn(IndexSegment segment, String column, File newSegme
       throws IOException {
     DataSource dataSource = segment.getDataSource(column);
     ForwardIndexReader reader = dataSource.getForwardIndex();
+    if (reader == null) {

Review Comment:
   same here. Consider renaming it to `forwardIndexReader`.



##########
pinot-core/src/main/java/org/apache/pinot/core/minion/RawIndexConverter.java:
##########
@@ -204,6 +204,11 @@ private void convertColumn(FieldSpec fieldSpec)
     // Create the raw index
     DataSource dataSource = _originalImmutableSegment.getDataSource(columnName);
     ForwardIndexReader reader = dataSource.getForwardIndex();
+    if (reader == null) {

Review Comment:
   nit: rename it to `forwardIndexReader`



-- 
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 #9333: Add an option to disable the creation of the forward index for a column

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


##########
pinot-core/src/test/java/org/apache/pinot/queries/ForwardIndexDisabledSingleValueQueriesTest.java:
##########
@@ -0,0 +1,1088 @@
+/**
+ * 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.queries;
+
+import java.io.File;
+import java.net.URL;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.TimeUnit;
+import org.apache.commons.io.FileUtils;
+import org.apache.pinot.common.response.broker.BrokerResponseNative;
+import org.apache.pinot.common.response.broker.ResultTable;
+import org.apache.pinot.common.utils.DataSchema;
+import org.apache.pinot.segment.local.indexsegment.immutable.ImmutableSegmentLoader;
+import org.apache.pinot.segment.local.segment.creator.impl.SegmentIndexCreationDriverImpl;
+import org.apache.pinot.segment.local.segment.index.loader.IndexLoadingConfig;
+import org.apache.pinot.segment.spi.ColumnMetadata;
+import org.apache.pinot.segment.spi.ImmutableSegment;
+import org.apache.pinot.segment.spi.IndexSegment;
+import org.apache.pinot.segment.spi.creator.SegmentGeneratorConfig;
+import org.apache.pinot.segment.spi.creator.SegmentIndexCreationDriver;
+import org.apache.pinot.spi.config.table.FieldConfig;
+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.data.TimeGranularitySpec;
+import org.apache.pinot.spi.utils.ReadMode;
+import org.apache.pinot.spi.utils.builder.TableConfigBuilder;
+import org.testng.Assert;
+import org.testng.annotations.AfterClass;
+import org.testng.annotations.BeforeClass;
+import org.testng.annotations.Test;
+
+import static org.testng.Assert.assertEquals;
+import static org.testng.Assert.assertNotNull;
+import static org.testng.Assert.assertNull;
+import static org.testng.Assert.assertTrue;
+
+
+/**
+ * The <code>ForwardIndexDisabledSingleValueQueriesTest</code> class sets up the index segment for the no forward
+ * index single-value queries test.
+ * <p>There are totally 18 columns, 30000 records inside the original Avro file where 11 columns are selected to build
+ * the index segment. Selected columns information are as following:
+ * <ul>
+ *   ColumnName, FieldType, DataType, Cardinality, IsSorted, HasInvertedIndex, FwdIndexDisabled: S1, S2, HasRangeIndex
+ *   <li>column1, METRIC, INT, 6582, F, F, F, F, F</li>
+ *   <li>column3, METRIC, INT, 21910, F, F, F, F, F</li>
+ *   <li>column5, DIMENSION, STRING, 1, T, F, F, F, F</li>
+ *   <li>column6, DIMENSION, INT, 608, F, T, T, T, T</li>
+ *   <li>column7, DIMENSION, INT, 146, F, T, T, F, F</li>
+ *   <li>column9, DIMENSION, INT, 1737, F, F, F, F, F</li>
+ *   <li>column11, DIMENSION, STRING, 5, F, T, F, F, F</li>
+ *   <li>column12, DIMENSION, STRING, 5, F, F, F, F, F</li>
+ *   <li>column17, METRIC, INT, 24, F, T, F, F, F</li>
+ *   <li>column18, METRIC, INT, 1440, F, T, F, F, F</li>
+ *   <li>daysSinceEpoch, TIME, INT, 2, T, F, F, F, F</li>
+ * </ul>
+ */
+public class ForwardIndexDisabledSingleValueQueriesTest extends BaseQueriesTest {
+  private static final String AVRO_DATA = "data" + File.separator + "test_data-sv.avro";
+  private static final String SEGMENT_NAME_1 = "testTable_126164076_167572857";
+  private static final String SEGMENT_NAME_2 = "testTable_126164076_167572858";
+  private static final File INDEX_DIR = new File(FileUtils.getTempDirectory(),
+      "ForwardIndexDisabledSingleValueQueriesTest");
+
+  private static final String SELECT_STAR_QUERY = "SELECT * FROM testTable";
+
+  // Hard-coded query filter.
+  private static final String FILTER = " WHERE column1 > 100000000"
+      + " AND column3 BETWEEN 20000000 AND 1000000000"
+      + " AND column5 = 'gFuH'"
+      + " AND (column6 < 500000000 OR column11 NOT IN ('t', 'P'))"
+      + " AND daysSinceEpoch = 126164076";
+
+  private IndexSegment _indexSegment;
+  // Contains 2 index segments, one with 2 columns with forward index disabled, and the other with just 1.
+  private List<IndexSegment> _indexSegments;
+
+  private TableConfig _tableConfig;
+  private List<String> _invertedIndexColumns;
+  private List<String> _forwardIndexDisabledColumns;
+
+  @BeforeClass
+  public void buildAndLoadSegment()
+      throws Exception {
+    FileUtils.deleteQuietly(INDEX_DIR);
+
+    // Get resource file path.
+    URL resource = getClass().getClassLoader().getResource(AVRO_DATA);
+    assertNotNull(resource);
+    String filePath = resource.getFile();
+
+    // Build the segment schema.
+    Schema schema = new Schema.SchemaBuilder().setSchemaName("testTable").addMetric("column1", FieldSpec.DataType.INT)
+        .addMetric("column3", FieldSpec.DataType.INT).addSingleValueDimension("column5", FieldSpec.DataType.STRING)
+        .addSingleValueDimension("column6", FieldSpec.DataType.INT)
+        .addSingleValueDimension("column7", FieldSpec.DataType.INT)
+        .addSingleValueDimension("column9", FieldSpec.DataType.INT)
+        .addSingleValueDimension("column11", FieldSpec.DataType.STRING)
+        .addSingleValueDimension("column12", FieldSpec.DataType.STRING).addMetric("column17", FieldSpec.DataType.INT)
+        .addMetric("column18", FieldSpec.DataType.INT)
+        .addTime(new TimeGranularitySpec(FieldSpec.DataType.INT, TimeUnit.DAYS, "daysSinceEpoch"), null).build();
+
+    createSegment(filePath, SEGMENT_NAME_1, schema);
+    createSegment(filePath, SEGMENT_NAME_2, schema);
+
+    ImmutableSegment immutableSegment1 = loadSegmentWithMetadataChecks(SEGMENT_NAME_1);
+    ImmutableSegment immutableSegment2 = loadSegmentWithMetadataChecks(SEGMENT_NAME_2);
+
+    _indexSegment = immutableSegment1;
+    _indexSegments = Arrays.asList(immutableSegment1, immutableSegment2);
+  }
+
+  private void createSegment(String filePath, String segmentName, Schema schema)
+      throws Exception {
+    // Create field configs for the no forward index columns
+    List<FieldConfig> fieldConfigList = new ArrayList<>();
+    fieldConfigList.add(new FieldConfig("column6", FieldConfig.EncodingType.DICTIONARY, Collections.emptyList(), null,
+        Collections.singletonMap(FieldConfig.FORWARD_INDEX_DISABLED, Boolean.TRUE.toString())));
+    if (segmentName.equals(SEGMENT_NAME_1)) {
+      fieldConfigList.add(new FieldConfig("column7", FieldConfig.EncodingType.DICTIONARY, Collections.emptyList(),
+          null, Collections.singletonMap(FieldConfig.FORWARD_INDEX_DISABLED, Boolean.TRUE.toString())));
+
+      // Build table config based on segment 1 as it contains both columns under no forward index
+      _tableConfig =
+          new TableConfigBuilder(TableType.OFFLINE).setTableName("testTable").setTimeColumnName("daysSinceEpoch")
+              .setFieldConfigList(fieldConfigList).setRangeIndexColumns(Arrays.asList("column6")).build();
+    }
+
+    // Create the segment generator config.
+    SegmentGeneratorConfig segmentGeneratorConfig = new SegmentGeneratorConfig(_tableConfig, schema);
+    segmentGeneratorConfig.setInputFilePath(filePath);
+    segmentGeneratorConfig.setTableName("testTable");
+    segmentGeneratorConfig.setOutDir(INDEX_DIR.getAbsolutePath());
+    segmentGeneratorConfig.setSegmentName(segmentName);
+    // The segment generation code in SegmentColumnarIndexCreator will throw
+    // exception if start and end time in time column are not in acceptable
+    // range. For this test, we first need to fix the input avro data
+    // to have the time column values in allowed range. Until then, the check
+    // is explicitly disabled
+    segmentGeneratorConfig.setSkipTimeValueCheck(true);
+    _invertedIndexColumns = Arrays.asList("column6", "column7", "column11", "column17", "column18");
+    segmentGeneratorConfig.setInvertedIndexCreationColumns(_invertedIndexColumns);
+
+    _forwardIndexDisabledColumns = new ArrayList<>(Arrays.asList("column6", "column7"));
+    segmentGeneratorConfig.setForwardIndexDisabledColumns(_forwardIndexDisabledColumns);
+    segmentGeneratorConfig.setRangeIndexCreationColumns(Arrays.asList("column6"));
+
+    // Build the index segment.
+    SegmentIndexCreationDriver driver = new SegmentIndexCreationDriverImpl();
+    driver.init(segmentGeneratorConfig);
+    driver.build();
+  }
+
+  private ImmutableSegment loadSegmentWithMetadataChecks(String segmentName)
+      throws Exception {
+    IndexLoadingConfig indexLoadingConfig = new IndexLoadingConfig();
+    indexLoadingConfig.setTableConfig(_tableConfig);
+    indexLoadingConfig.setInvertedIndexColumns(new HashSet<>(_invertedIndexColumns));
+    indexLoadingConfig.setForwardIndexDisabledColumns(new HashSet<>(_forwardIndexDisabledColumns));
+    indexLoadingConfig.setRangeIndexColumns(new HashSet<>(Arrays.asList("column6")));
+    indexLoadingConfig.setReadMode(ReadMode.heap);
+
+    ImmutableSegment immutableSegment = ImmutableSegmentLoader.load(new File(INDEX_DIR, segmentName),
+        indexLoadingConfig);
+
+    Map<String, ColumnMetadata> columnMetadataMap1 = immutableSegment.getSegmentMetadata().getColumnMetadataMap();
+    columnMetadataMap1.forEach((column, metadata) -> {
+      if (column.equals("column6") || column.equals("column7")) {
+        assertTrue(metadata.hasDictionary());
+        assertTrue(metadata.isSingleValue());
+        assertNull(immutableSegment.getForwardIndex(column));
+      } else {
+        assertNotNull(immutableSegment.getForwardIndex(column));
+      }
+    });
+
+    return immutableSegment;
+  }
+
+  @AfterClass
+  public void deleteAndDestroySegment() {
+    FileUtils.deleteQuietly(INDEX_DIR);
+    _indexSegments.forEach((IndexSegment::destroy));
+  }
+
+  @Override
+  protected String getFilter() {
+    return FILTER;
+  }
+
+  @Override
+  protected IndexSegment getIndexSegment() {
+    return _indexSegment;
+  }
+
+  @Override
+  protected List<IndexSegment> getIndexSegments() {
+    return _indexSegments;
+  }
+
+  @Test
+  public void testSelectStarQueries() {
+    // Select * without any filters
+    try {
+      getBrokerResponse(SELECT_STAR_QUERY);
+      Assert.fail("Select * query should fail since forwardIndexDisabled on a select column");
+    } catch (IllegalStateException e) {
+      assertTrue(e.getMessage().contains("Forward index disabled for column:")
+          && e.getMessage().contains("cannot create DataFetcher!"));
+    }
+
+    // Select * with filters
+    try {
+      getBrokerResponse(SELECT_STAR_QUERY + FILTER);
+      Assert.fail("Select * query should fail since forwardIndexDisabled on a select column");
+    } catch (IllegalStateException e) {
+      assertTrue(e.getMessage().contains("Forward index disabled for column:")
+          && e.getMessage().contains("cannot create DataFetcher!"));
+    }
+  }
+
+  @Test
+  public void testSelectQueries() {
+    {
+      // Selection query without filters including a column with forwardIndexDisabled enabled on both segments
+      String query = "SELECT column1, column5, column6, column9, column11 FROM testTable";
+      try {
+        getBrokerResponse(query);
+        Assert.fail("Query should fail since forwardIndexDisabled on a select column");
+      } catch (IllegalStateException e) {
+        assertTrue(e.getMessage().contains("Forward index disabled for column:")
+            && e.getMessage().contains("cannot create DataFetcher!"));
+      }
+    }
+    {
+      // Selection query without filters including a column with forwardIndexDisabled enabled on one segment
+      String query = "SELECT column1, column5, column7, column9, column11 FROM testTable";
+      try {
+        getBrokerResponse(query);
+        Assert.fail("Query should fail since forwardIndexDisabled on a select column");
+      } catch (IllegalStateException e) {
+        assertTrue(e.getMessage().contains("Forward index disabled for column:")
+            && e.getMessage().contains("cannot create DataFetcher!"));
+      }
+    }
+    {
+      // Selection query with filters including a column with forwardIndexDisabled enabled on both segments
+      String query = "SELECT column1, column5, column6, column9, column11 FROM testTable WHERE column6 = 2147458029";
+      try {
+        getBrokerResponse(query);
+        Assert.fail("Query should fail since forwardIndexDisabled on a select column");
+      } catch (IllegalStateException e) {
+        assertTrue(e.getMessage().contains("Forward index disabled for column:")
+            && e.getMessage().contains("cannot create DataFetcher!"));
+      }
+    }
+    {
+      // Selection query with filters including a column with forwardIndexDisabled enabled on one segment
+      String query = "SELECT column1, column5, column7, column9, column11 FROM testTable WHERE column7 = 675695";
+      try {
+        getBrokerResponse(query);
+        Assert.fail("Query should fail since forwardIndexDisabled on a select column");
+      } catch (IllegalStateException e) {
+        assertTrue(e.getMessage().contains("Forward index disabled for column:")
+            && e.getMessage().contains("cannot create DataFetcher!"));
+      }
+    }
+    {
+      // Selection query without filters and without columns with forwardIndexDisabled enabled on either segment
+      String query = "SELECT column1, column5, column9, column11 FROM testTable ORDER BY column1";
+      BrokerResponseNative brokerResponseNative = getBrokerResponse(query);
+      assertTrue(brokerResponseNative.getProcessingExceptions() == null
+          || brokerResponseNative.getProcessingExceptions().size() == 0);
+      ResultTable resultTable = brokerResponseNative.getResultTable();
+      assertEquals(brokerResponseNative.getNumRowsResultSet(), 10);
+      assertEquals(brokerResponseNative.getTotalDocs(), 120_000L);
+      assertEquals(brokerResponseNative.getNumDocsScanned(), 120_000L);
+      assertEquals(brokerResponseNative.getNumSegmentsProcessed(), 4L);
+      assertEquals(brokerResponseNative.getNumSegmentsMatched(), 4L);
+      assertEquals(brokerResponseNative.getNumEntriesScannedPostFilter(), 120_120L);
+      assertEquals(brokerResponseNative.getNumEntriesScannedInFilter(), 0L);
+      assertNotNull(brokerResponseNative.getProcessingExceptions());
+      assertEquals(brokerResponseNative.getProcessingExceptions().size(), 0);
+      DataSchema dataSchema = new DataSchema(new String[]{"column1", "column5", "column9", "column11"},
+        new DataSchema.ColumnDataType[]{DataSchema.ColumnDataType.INT, DataSchema.ColumnDataType.STRING,
+            DataSchema.ColumnDataType.INT, DataSchema.ColumnDataType.STRING});
+      assertEquals(resultTable.getDataSchema(), dataSchema);
+      List<Object[]> resultRows = resultTable.getRows();
+      int previousColumn1 = Integer.MIN_VALUE;
+      for (Object[] resultRow : resultRows) {
+        assertEquals(resultRow.length, 4);
+        assertEquals((String) resultRow[1], "gFuH");
+        // Column 1
+        assertTrue((Integer) resultRow[0] >= previousColumn1);
+        previousColumn1 = (Integer) resultRow[0];
+      }
+
+      Object[] firstRow = resultRows.get(0);
+      // Column 11
+      assertEquals((String) firstRow[3], "o");
+    }
+    {
+      // Transform function on a selection clause with a forwardIndexDisabled column in transform
+      String query = "SELECT CONCAT(column6, column9, '-') from testTable";
+      try {
+        getBrokerResponse(query);
+        Assert.fail("Query should fail since forwardIndexDisabled on a column in transform");
+      } catch (IllegalStateException e) {
+        assertTrue(e.getMessage().contains("Forward index disabled for column:")
+            && e.getMessage().contains("cannot create DataFetcher!"));
+      }
+    }
+    {
+      // Transform function on a selection clause without a forwardIndexDisabled column in transform
+      String query = "SELECT CONCAT(column5, column9, '-') from testTable ORDER BY column1";
+      BrokerResponseNative brokerResponseNative = getBrokerResponse(query);
+      assertTrue(brokerResponseNative.getProcessingExceptions() == null
+          || brokerResponseNative.getProcessingExceptions().size() == 0);
+      ResultTable resultTable = brokerResponseNative.getResultTable();
+      assertEquals(brokerResponseNative.getNumRowsResultSet(), 10);
+      assertEquals(brokerResponseNative.getTotalDocs(), 120_000L);
+      assertEquals(brokerResponseNative.getNumDocsScanned(), 120_000L);
+      assertEquals(brokerResponseNative.getNumSegmentsProcessed(), 4L);
+      assertEquals(brokerResponseNative.getNumSegmentsMatched(), 4L);
+      assertEquals(brokerResponseNative.getNumEntriesScannedPostFilter(), 120_080L);
+      assertEquals(brokerResponseNative.getNumEntriesScannedInFilter(), 0L);
+      assertNotNull(brokerResponseNative.getProcessingExceptions());
+      assertEquals(brokerResponseNative.getProcessingExceptions().size(), 0);
+      DataSchema dataSchema = new DataSchema(new String[]{"concat(column5,column9,'-')"},
+          new DataSchema.ColumnDataType[]{DataSchema.ColumnDataType.STRING});
+      assertEquals(resultTable.getDataSchema(), dataSchema);
+      List<Object[]> resultRows = resultTable.getRows();
+      for (Object[] resultRow : resultRows) {
+        assertEquals(resultRow.length, 1);
+        assertTrue(resultRow[0].toString().startsWith("gFuH-"));
+      }
+    }
+    {
+      // Selection query with filters (not including forwardIndexDisabled column) and without columns with
+      // forwardIndexDisabled enabled on either segment
+      String query = "SELECT column1, column5, column9, column11 FROM testTable WHERE column1 > 100000000"
+        + " AND column3 BETWEEN 20000000 AND 1000000000"
+        + " AND column5 = 'gFuH'"
+        + " AND daysSinceEpoch = 126164076 ORDER BY column1";
+      BrokerResponseNative brokerResponseNative = getBrokerResponse(query);
+      assertTrue(brokerResponseNative.getProcessingExceptions() == null
+          || brokerResponseNative.getProcessingExceptions().size() == 0);
+      ResultTable resultTable = brokerResponseNative.getResultTable();
+      assertEquals(brokerResponseNative.getNumRowsResultSet(), 10);
+      assertEquals(brokerResponseNative.getTotalDocs(), 120_000L);
+      assertEquals(brokerResponseNative.getNumDocsScanned(), 42_368L);
+      assertEquals(brokerResponseNative.getNumSegmentsProcessed(), 4L);
+      assertEquals(brokerResponseNative.getNumSegmentsMatched(), 4L);
+      assertEquals(brokerResponseNative.getNumEntriesScannedPostFilter(), 42_488);
+      assertEquals(brokerResponseNative.getNumEntriesScannedInFilter(), 192744L);
+      assertNotNull(brokerResponseNative.getProcessingExceptions());
+      assertEquals(brokerResponseNative.getProcessingExceptions().size(), 0);
+      DataSchema dataSchema = new DataSchema(new String[]{"column1", "column5", "column9", "column11"},
+          new DataSchema.ColumnDataType[]{DataSchema.ColumnDataType.INT, DataSchema.ColumnDataType.STRING,
+              DataSchema.ColumnDataType.INT, DataSchema.ColumnDataType.STRING});
+      assertEquals(resultTable.getDataSchema(), dataSchema);
+      List<Object[]> resultRows = resultTable.getRows();
+      int previousColumn1 = Integer.MIN_VALUE;
+      for (Object[] resultRow : resultRows) {
+        assertEquals(resultRow.length, 4);
+        assertEquals((String) resultRow[1], "gFuH");
+        assertTrue((Integer) resultRow[0] >= previousColumn1);
+        previousColumn1 = (Integer) resultRow[0];
+      }
+
+      Object[] firstRow = resultRows.get(0);
+      // Column 11
+      assertEquals((String) firstRow[3], "P");
+    }
+    {
+      // Selection query with supported filters (including forwardIndexDisabled column) and without columns with
+      // forwardIndexDisabled enabled on either segment
+      String query = "SELECT column1, column5, column9, column11 FROM testTable WHERE column6 = 2147458029 "
+          + "ORDER BY column1";
+      BrokerResponseNative brokerResponseNative = getBrokerResponse(query);
+      assertTrue(brokerResponseNative.getProcessingExceptions() == null
+          || brokerResponseNative.getProcessingExceptions().size() == 0);
+      ResultTable resultTable = brokerResponseNative.getResultTable();
+      assertEquals(brokerResponseNative.getNumRowsResultSet(), 10);
+      assertEquals(brokerResponseNative.getTotalDocs(), 120_000L);
+      assertEquals(brokerResponseNative.getNumDocsScanned(), 16L);
+      assertEquals(brokerResponseNative.getNumSegmentsProcessed(), 4L);
+      assertEquals(brokerResponseNative.getNumSegmentsMatched(), 4L);
+      assertEquals(brokerResponseNative.getNumEntriesScannedPostFilter(), 64L);
+      assertEquals(brokerResponseNative.getNumEntriesScannedInFilter(), 0L);
+      assertNotNull(brokerResponseNative.getProcessingExceptions());
+      assertEquals(brokerResponseNative.getProcessingExceptions().size(), 0);
+      DataSchema dataSchema = new DataSchema(new String[]{"column1", "column5", "column9", "column11"},
+          new DataSchema.ColumnDataType[]{DataSchema.ColumnDataType.INT, DataSchema.ColumnDataType.STRING,
+              DataSchema.ColumnDataType.INT, DataSchema.ColumnDataType.STRING});
+      assertEquals(resultTable.getDataSchema(), dataSchema);
+      List<Object[]> resultRows = resultTable.getRows();
+      for (Object[] resultRow : resultRows) {
+        assertEquals(resultRow.length, 4);
+        assertEquals(resultRow[0], 635553468);
+        assertEquals((String) resultRow[1], "gFuH");
+        assertEquals(resultRow[2], 705242697);
+        assertEquals((String) resultRow[3], "P");
+      }
+    }
+    {
+      // Selection query with supported filters (including forwardIndexDisabled column) and without columns with
+      // forwardIndexDisabled enabled on either segment
+      String query = "SELECT column1, column5, column9, column11 FROM testTable WHERE column7 != 675695 "
+          + "ORDER BY column1";
+      BrokerResponseNative brokerResponseNative = getBrokerResponse(query);
+      assertTrue(brokerResponseNative.getProcessingExceptions() == null
+          || brokerResponseNative.getProcessingExceptions().size() == 0);
+      ResultTable resultTable = brokerResponseNative.getResultTable();
+      assertEquals(brokerResponseNative.getNumRowsResultSet(), 10);
+      assertEquals(brokerResponseNative.getTotalDocs(), 120_000L);
+      assertEquals(brokerResponseNative.getNumDocsScanned(), 119_908L);
+      assertEquals(brokerResponseNative.getNumSegmentsProcessed(), 4L);
+      assertEquals(brokerResponseNative.getNumSegmentsMatched(), 4L);
+      assertEquals(brokerResponseNative.getNumEntriesScannedPostFilter(), 120_028L);
+      assertEquals(brokerResponseNative.getNumEntriesScannedInFilter(), 0L);
+      assertNotNull(brokerResponseNative.getProcessingExceptions());
+      assertEquals(brokerResponseNative.getProcessingExceptions().size(), 0);
+      DataSchema dataSchema = new DataSchema(new String[]{"column1", "column5", "column9", "column11"},
+          new DataSchema.ColumnDataType[]{DataSchema.ColumnDataType.INT, DataSchema.ColumnDataType.STRING,
+              DataSchema.ColumnDataType.INT, DataSchema.ColumnDataType.STRING});
+      assertEquals(resultTable.getDataSchema(), dataSchema);
+      List<Object[]> resultRows = resultTable.getRows();
+      int previousColumn1 = Integer.MIN_VALUE;
+      for (Object[] resultRow : resultRows) {
+        assertEquals(resultRow.length, 4);
+        assertEquals((String) resultRow[1], "gFuH");
+        assertTrue((Integer) resultRow[0] >= previousColumn1);
+        previousColumn1 = (Integer) resultRow[0];
+      }
+
+      Object[] firstRow = resultRows.get(0);
+      // Column 11
+      assertEquals((String) firstRow[3], "o");
+    }
+    {
+      // Selection query with supported filters (including forwardIndexDisabled column) and without columns with
+      // forwardIndexDisabled enabled on either segment
+      String query = "SELECT column1, column5, column9, column11 FROM testTable WHERE column7 IN (675695, 2137685743) "
+          + "ORDER BY column1";
+      BrokerResponseNative brokerResponseNative = getBrokerResponse(query);
+      assertTrue(brokerResponseNative.getProcessingExceptions() == null
+          || brokerResponseNative.getProcessingExceptions().size() == 0);
+      ResultTable resultTable = brokerResponseNative.getResultTable();
+      assertEquals(brokerResponseNative.getNumRowsResultSet(), 10);
+      assertEquals(brokerResponseNative.getTotalDocs(), 120_000L);
+      assertEquals(brokerResponseNative.getNumDocsScanned(), 828L);
+      assertEquals(brokerResponseNative.getNumSegmentsProcessed(), 4L);
+      assertEquals(brokerResponseNative.getNumSegmentsMatched(), 4L);
+      assertEquals(brokerResponseNative.getNumEntriesScannedPostFilter(), 948L);
+      assertEquals(brokerResponseNative.getNumEntriesScannedInFilter(), 0L);
+      assertNotNull(brokerResponseNative.getProcessingExceptions());
+      assertEquals(brokerResponseNative.getProcessingExceptions().size(), 0);
+      DataSchema dataSchema = new DataSchema(new String[]{"column1", "column5", "column9", "column11"},
+          new DataSchema.ColumnDataType[]{DataSchema.ColumnDataType.INT, DataSchema.ColumnDataType.STRING,
+              DataSchema.ColumnDataType.INT, DataSchema.ColumnDataType.STRING});
+      assertEquals(resultTable.getDataSchema(), dataSchema);
+      List<Object[]> resultRows = resultTable.getRows();
+      int previousColumn1 = Integer.MIN_VALUE;
+      for (Object[] resultRow : resultRows) {
+        assertEquals(resultRow.length, 4);
+        assertEquals((String) resultRow[1], "gFuH");
+        assertTrue((Integer) resultRow[0] >= previousColumn1);
+        previousColumn1 = (Integer) resultRow[0];
+      }
+
+      Object[] firstRow = resultRows.get(0);
+      // Column 11
+      assertEquals((String) firstRow[3], "P");
+    }
+    {
+      // Selection query with supported filters (including forwardIndexDisabled column) and without columns with
+      // forwardIndexDisabled enabled on either segment
+      String query = "SELECT column1, column5, column9, column11 FROM testTable WHERE column6 NOT IN "
+          + "(1689277, 2147458029) ORDER BY column1";
+      BrokerResponseNative brokerResponseNative = getBrokerResponse(query);
+      assertTrue(brokerResponseNative.getProcessingExceptions() == null
+          || brokerResponseNative.getProcessingExceptions().size() == 0);
+      ResultTable resultTable = brokerResponseNative.getResultTable();
+      assertEquals(brokerResponseNative.getNumRowsResultSet(), 10);
+      assertEquals(brokerResponseNative.getTotalDocs(), 120_000L);
+      assertEquals(brokerResponseNative.getNumDocsScanned(), 119_980L);
+      assertEquals(brokerResponseNative.getNumSegmentsProcessed(), 4L);
+      assertEquals(brokerResponseNative.getNumSegmentsMatched(), 4L);
+      assertEquals(brokerResponseNative.getNumEntriesScannedPostFilter(), 120_100L);
+      assertEquals(brokerResponseNative.getNumEntriesScannedInFilter(), 0L);
+      assertNotNull(brokerResponseNative.getProcessingExceptions());
+      assertEquals(brokerResponseNative.getProcessingExceptions().size(), 0);
+      DataSchema dataSchema = new DataSchema(new String[]{"column1", "column5", "column9", "column11"},
+          new DataSchema.ColumnDataType[]{DataSchema.ColumnDataType.INT, DataSchema.ColumnDataType.STRING,
+              DataSchema.ColumnDataType.INT, DataSchema.ColumnDataType.STRING});
+      assertEquals(resultTable.getDataSchema(), dataSchema);
+      List<Object[]> resultRows = resultTable.getRows();
+      int previousColumn1 = Integer.MIN_VALUE;
+      for (Object[] resultRow : resultRows) {
+        assertEquals(resultRow.length, 4);
+        assertEquals((String) resultRow[1], "gFuH");
+        assertTrue((Integer) resultRow[0] >= previousColumn1);
+        previousColumn1 = (Integer) resultRow[0];
+      }
+
+      Object[] firstRow = resultRows.get(0);
+      // Column 11
+      assertEquals((String) firstRow[3], "o");
+    }
+    {
+      // Query with literal only in SELECT
+      String query = "SELECT 'marvin' from testTable ORDER BY column1";
+      BrokerResponseNative brokerResponseNative = getBrokerResponse(query);
+      assertTrue(brokerResponseNative.getProcessingExceptions() == null
+          || brokerResponseNative.getProcessingExceptions().size() == 0);
+      ResultTable resultTable = brokerResponseNative.getResultTable();
+      assertEquals(brokerResponseNative.getNumRowsResultSet(), 10);
+      assertEquals(brokerResponseNative.getTotalDocs(), 120_000L);
+      assertEquals(brokerResponseNative.getNumDocsScanned(), 120_000L);
+      assertEquals(brokerResponseNative.getNumSegmentsProcessed(), 4L);
+      assertEquals(brokerResponseNative.getNumSegmentsMatched(), 4L);
+      assertEquals(brokerResponseNative.getNumEntriesScannedPostFilter(), 120_000L);
+      assertEquals(brokerResponseNative.getNumEntriesScannedInFilter(), 0L);
+      assertNotNull(brokerResponseNative.getProcessingExceptions());
+      assertEquals(brokerResponseNative.getProcessingExceptions().size(), 0);
+      assertEquals(resultTable.getDataSchema(), new DataSchema(new String[]{"'marvin'"},
+          new DataSchema.ColumnDataType[]{DataSchema.ColumnDataType.STRING}));
+      List<Object[]> resultRows = resultTable.getRows();
+      for (Object[] resultRow : resultRows) {
+        assertEquals(resultRow.length, 1);
+        assertEquals((String) resultRow[0], "marvin");
+      }
+    }
+    {
+      // Selection query with '<' filter on a forwardIndexDisabled column with range index available
+      String query = "SELECT column1, column5, column9, column11 FROM testTable WHERE column6 < 2147458029 AND "
+          + "column6 > 1699000 ORDER BY column1";
+      BrokerResponseNative brokerResponseNative = getBrokerResponse(query);
+      assertTrue(brokerResponseNative.getProcessingExceptions() == null
+          || brokerResponseNative.getProcessingExceptions().size() == 0);
+      ResultTable resultTable = brokerResponseNative.getResultTable();
+      assertEquals(brokerResponseNative.getNumRowsResultSet(), 10);
+      assertEquals(brokerResponseNative.getTotalDocs(), 120_000L);
+      assertEquals(brokerResponseNative.getNumDocsScanned(), 119_980L);
+      assertEquals(brokerResponseNative.getNumSegmentsProcessed(), 4L);
+      assertEquals(brokerResponseNative.getNumSegmentsMatched(), 4L);
+      assertEquals(brokerResponseNative.getNumEntriesScannedPostFilter(), 120_100L);
+      assertEquals(brokerResponseNative.getNumEntriesScannedInFilter(), 0L);
+      assertNotNull(brokerResponseNative.getProcessingExceptions());
+      assertEquals(brokerResponseNative.getProcessingExceptions().size(), 0);
+      DataSchema dataSchema = new DataSchema(new String[]{"column1", "column5", "column9", "column11"},
+          new DataSchema.ColumnDataType[]{DataSchema.ColumnDataType.INT, DataSchema.ColumnDataType.STRING,
+              DataSchema.ColumnDataType.INT, DataSchema.ColumnDataType.STRING});
+      assertEquals(resultTable.getDataSchema(), dataSchema);
+      List<Object[]> resultRows = resultTable.getRows();
+      int previousColumn1 = Integer.MIN_VALUE;
+      for (Object[] resultRow : resultRows) {
+        assertEquals(resultRow.length, 4);
+        assertEquals((String) resultRow[1], "gFuH");
+        assertTrue((Integer) resultRow[0] >= previousColumn1);
+        previousColumn1 = (Integer) resultRow[0];
+      }
+
+      Object[] firstRow = resultRows.get(0);
+      // Column 11
+      assertEquals((String) firstRow[3], "o");
+    }
+    {
+      // Selection query with '>=' filter on a forwardIndexDisabled column without range index available
+      String query = "SELECT column1, column5, column9, column11 FROM testTable WHERE column7 >= 676000";
+      try {
+        getBrokerResponse(query);
+        Assert.fail("Query should fail since forwardIndexDisabled on a range query column without range index");
+      } catch (IllegalStateException e) {
+        assertTrue(e.getMessage().contains("Forward index disabled for column:")
+            && e.getMessage().contains("creating ScanDocIdSet unsupported!"));
+      }
+    }
+    {
+      // Select query with a filter on a column which doesn't have forwardIndexDisabled enabled
+      String query = "SELECT column1, column5, column9 from testTable WHERE column9 < 50000 ORDER BY column1";
+      BrokerResponseNative brokerResponseNative = getBrokerResponse(query);
+      assertTrue(brokerResponseNative.getProcessingExceptions() == null
+          || brokerResponseNative.getProcessingExceptions().size() == 0);
+      ResultTable resultTable = brokerResponseNative.getResultTable();
+      assertEquals(brokerResponseNative.getNumRowsResultSet(), 4);
+      assertEquals(brokerResponseNative.getTotalDocs(), 120_000L);
+      assertEquals(brokerResponseNative.getNumDocsScanned(), 4L);
+      assertEquals(brokerResponseNative.getNumSegmentsProcessed(), 4L);
+      assertEquals(brokerResponseNative.getNumSegmentsMatched(), 4L);
+      assertEquals(brokerResponseNative.getNumEntriesScannedPostFilter(), 12L);
+      assertEquals(brokerResponseNative.getNumEntriesScannedInFilter(), 120000L);
+      assertNotNull(brokerResponseNative.getProcessingExceptions());
+      assertEquals(brokerResponseNative.getProcessingExceptions().size(), 0);
+      assertEquals(resultTable.getDataSchema(), new DataSchema(new String[]{"column1", "column5", "column9"},
+          new DataSchema.ColumnDataType[]{
+          DataSchema.ColumnDataType.INT, DataSchema.ColumnDataType.STRING, DataSchema.ColumnDataType.INT
+      }));
+      List<Object[]> resultRows = resultTable.getRows();
+      for (Object[] resultRow : resultRows) {
+        assertEquals(resultRow.length, 3);
+      }
+    }
+    {
+      // Transform function on a filter clause for forwardIndexDisabled column in transform
+      String query = "SELECT column1, column11 from testTable WHERE CONCAT(column6, column9, '-') = '1689277-11270'";
+      BrokerResponseNative brokerResponseNative = getBrokerResponse(query);
+      assertTrue(brokerResponseNative.getProcessingExceptions() != null
+          && brokerResponseNative.getProcessingExceptions().size() > 0);
+    }
+    {
+      // Transform function on a filter clause for a non-forwardIndexDisabled column in transform
+      String query = "SELECT column1, column11 from testTable WHERE CONCAT(column5, column9, '-') = 'gFuH-11270' "
+          + "ORDER BY column1";
+      BrokerResponseNative brokerResponseNative = getBrokerResponse(query);
+      assertTrue(brokerResponseNative.getProcessingExceptions() == null
+          || brokerResponseNative.getProcessingExceptions().size() == 0);
+      ResultTable resultTable = brokerResponseNative.getResultTable();
+      assertEquals(brokerResponseNative.getNumRowsResultSet(), 4);
+      assertEquals(brokerResponseNative.getTotalDocs(), 120_000L);
+      assertEquals(brokerResponseNative.getNumDocsScanned(), 4L);
+      assertEquals(brokerResponseNative.getNumSegmentsProcessed(), 4L);
+      assertEquals(brokerResponseNative.getNumSegmentsMatched(), 4L);
+      assertEquals(brokerResponseNative.getNumEntriesScannedPostFilter(), 8L);
+      assertEquals(brokerResponseNative.getNumEntriesScannedInFilter(), 120000L);
+      assertNotNull(brokerResponseNative.getProcessingExceptions());
+      assertEquals(brokerResponseNative.getProcessingExceptions().size(), 0);
+      assertEquals(resultTable.getDataSchema(), new DataSchema(new String[]{"column1", "column11"},
+          new DataSchema.ColumnDataType[]{
+              DataSchema.ColumnDataType.INT, DataSchema.ColumnDataType.STRING
+          }));
+      List<Object[]> resultRows = resultTable.getRows();
+      assertEquals(resultRows.size(), 4);
+      for (Object[] resultRow : resultRows) {
+        assertEquals(resultRow.length, 2);
+        assertEquals(resultRow[0], 815409257);
+        assertEquals(resultRow[1], "P");
+      }
+    }
+  }
+
+  @Test
+  public void testSelectWithDistinctQueries() {
+    {
+      // Select a mix of forwardIndexDisabled and non-forwardIndexDisabled columns with distinct
+      String query = "SELECT DISTINCT column1, column6, column9 FROM testTable LIMIT 10";
+      try {
+        getBrokerResponse(query);
+        Assert.fail("Query should fail since forwardIndexDisabled on a column in select distinct");
+      } catch (IllegalStateException e) {
+        assertTrue(e.getMessage().contains("Forward index disabled for column:")
+            && e.getMessage().contains("cannot create DataFetcher!"));
+      }
+    }
+    {
+      // Select non-forwardIndexDisabled columns with distinct
+      String query = "SELECT DISTINCT column1, column5, column9 FROM testTable ORDER BY column1 LIMIT 10";
+      BrokerResponseNative brokerResponseNative = getBrokerResponse(query);
+      assertTrue(brokerResponseNative.getProcessingExceptions() == null
+          || brokerResponseNative.getProcessingExceptions().size() == 0);
+      ResultTable resultTable = brokerResponseNative.getResultTable();
+      assertEquals(brokerResponseNative.getNumRowsResultSet(), 10);
+      assertEquals(brokerResponseNative.getTotalDocs(), 120_000L);
+      assertEquals(brokerResponseNative.getNumDocsScanned(), 120_000L);
+      assertEquals(brokerResponseNative.getNumSegmentsProcessed(), 4L);
+      assertEquals(brokerResponseNative.getNumSegmentsMatched(), 4L);
+      assertEquals(brokerResponseNative.getNumEntriesScannedPostFilter(), 360_000L);
+      assertEquals(brokerResponseNative.getNumEntriesScannedInFilter(), 0L);
+      assertNotNull(brokerResponseNative.getProcessingExceptions());
+      assertEquals(brokerResponseNative.getProcessingExceptions().size(), 0);
+      assertEquals(resultTable.getDataSchema(), new DataSchema(new String[]{"column1", "column5", "column9"},
+          new DataSchema.ColumnDataType[]{DataSchema.ColumnDataType.INT, DataSchema.ColumnDataType.STRING,
+              DataSchema.ColumnDataType.INT}));
+      List<Object[]> resultRows = resultTable.getRows();
+      int previousColumn1 = Integer.MIN_VALUE;
+      for (Object[] resultRow : resultRows) {
+        assertEquals(resultRow.length, 3);
+        assertEquals(resultRow[1], "gFuH");
+        assertTrue((Integer) resultRow[0] >= previousColumn1);
+        previousColumn1 = (Integer) resultRow[0];
+      }
+    }
+  }
+
+  @Test
+  public void testSelectWithGroupByOrderByQueries() {
+    {
+      // Select a mix of forwardIndexDisabled and non-forwardIndexDisabled columns with group by order by
+      String query = "SELECT column1, column6 FROM testTable GROUP BY column1, column6 ORDER BY column1, column6 "
+          + " LIMIT 10";
+      try {
+        getBrokerResponse(query);
+        Assert.fail("Query should fail since forwardIndexDisabled on a column in select group by order by");
+      } catch (IllegalStateException e) {
+        assertTrue(e.getMessage().contains("Forward index disabled for column:")
+            && e.getMessage().contains("cannot create DataFetcher!"));
+      }
+    }
+    {
+      // Select forwardIndexDisabled columns with group by order by
+      String query = "SELECT column7, column6 FROM testTable GROUP BY column7, column6 ORDER BY column7, column6 "
+          + " LIMIT 10";
+      try {
+        getBrokerResponse(query);
+        Assert.fail("Query should fail since forwardIndexDisabled on a column in select group by order by");
+      } catch (IllegalStateException e) {
+        assertTrue(e.getMessage().contains("Forward index disabled for column:")
+            && e.getMessage().contains("cannot create DataFetcher!"));
+      }
+    }
+    {
+      // Select non-forwardIndexDisabled columns with group by order by
+      String query = "SELECT column1, column5 FROM testTable GROUP BY column1, column5 ORDER BY column1, column5 "
+          + " LIMIT 10";
+      BrokerResponseNative brokerResponseNative = getBrokerResponse(query);
+      assertTrue(brokerResponseNative.getProcessingExceptions() == null
+          || brokerResponseNative.getProcessingExceptions().size() == 0);
+      ResultTable resultTable = brokerResponseNative.getResultTable();
+      assertEquals(brokerResponseNative.getNumRowsResultSet(), 10);
+      assertEquals(brokerResponseNative.getTotalDocs(), 120_000L);
+      assertEquals(brokerResponseNative.getNumDocsScanned(), 120000L);
+      assertEquals(brokerResponseNative.getNumSegmentsProcessed(), 4L);
+      assertEquals(brokerResponseNative.getNumSegmentsMatched(), 4L);
+      assertEquals(brokerResponseNative.getNumEntriesScannedPostFilter(), 240000L);
+      assertEquals(brokerResponseNative.getNumEntriesScannedInFilter(), 0L);
+      assertNotNull(brokerResponseNative.getProcessingExceptions());
+      assertEquals(brokerResponseNative.getProcessingExceptions().size(), 0);
+      assertEquals(resultTable.getDataSchema(), new DataSchema(new String[]{"column1", "column5"},
+          new DataSchema.ColumnDataType[]{DataSchema.ColumnDataType.INT, DataSchema.ColumnDataType.STRING}));
+      List<Object[]> resultRows = resultTable.getRows();
+      int previousVal = -1;
+      for (Object[] resultRow : resultRows) {
+        assertEquals(resultRow.length, 2);
+        assertEquals(resultRow[1], "gFuH");
+        assertTrue((int) resultRow[0] > previousVal);
+        previousVal = (int) resultRow[0];
+      }
+    }
+    {
+      // Select forwardIndexDisabled columns using transform with group by order by
+      String query = "SELECT CONCAT(column1, column6, '-') FROM testTable GROUP BY CONCAT(column1, column6, '-') "
+          + "ORDER BY CONCAT(column1, column6, '-') LIMIT 10";
+      try {
+        getBrokerResponse(query);
+        Assert.fail("Query should fail since forwardIndexDisabled on a transformed column in group by order by");
+      } catch (IllegalStateException e) {
+        assertTrue(e.getMessage().contains("Forward index disabled for column:")
+            && e.getMessage().contains("cannot create DataFetcher!"));
+      }
+    }
+    {
+      // Select non-forwardIndexDisabled columns using transform with group by order by
+      String query = "SELECT CONCAT(column1, column5, '-') FROM testTable GROUP BY CONCAT(column1, column5, '-') "
+          + "ORDER BY CONCAT(column1, column5, '-') LIMIT 10";
+      BrokerResponseNative brokerResponseNative = getBrokerResponse(query);
+      assertTrue(brokerResponseNative.getProcessingExceptions() == null
+          || brokerResponseNative.getProcessingExceptions().size() == 0);
+      ResultTable resultTable = brokerResponseNative.getResultTable();
+      assertEquals(brokerResponseNative.getNumRowsResultSet(), 10);
+      assertEquals(brokerResponseNative.getTotalDocs(), 120_000L);
+      assertEquals(brokerResponseNative.getNumDocsScanned(), 120000L);
+      assertEquals(brokerResponseNative.getNumSegmentsProcessed(), 4L);
+      assertEquals(brokerResponseNative.getNumSegmentsMatched(), 4L);
+      assertEquals(brokerResponseNative.getNumEntriesScannedPostFilter(), 240000L);
+      assertEquals(brokerResponseNative.getNumEntriesScannedInFilter(), 0L);
+      assertNotNull(brokerResponseNative.getProcessingExceptions());
+      assertEquals(brokerResponseNative.getProcessingExceptions().size(), 0);
+      assertEquals(resultTable.getDataSchema(), new DataSchema(new String[]{"concat(column1,column5,'-')"},
+          new DataSchema.ColumnDataType[]{DataSchema.ColumnDataType.STRING}));
+      List<Object[]> resultRows = resultTable.getRows();
+      for (Object[] resultRow : resultRows) {
+        assertEquals(resultRow.length, 1);
+        assertTrue(resultRow[0].toString().endsWith("-gFuH"));
+      }
+    }
+  }
+
+  @Test
+  public void testSelectWithAggregationQueries() {
+    {
+      // Allowed aggregation functions on forwardIndexDisabled columns
+      String query = "SELECT max(column7), min(column7), count(column7), minmaxrange(column7), "
+          + "distinctcount(column7), distinctcounthll(column6), distinctcountrawhll(column7), "

Review Comment:
   Let's make sure to double check this once in follow-up. This PR is now too big to incorporate this 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 pull request #9333: Add an option to disable the creation of the forward index for a column

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

   Thanks for the contribution @somandal . Let's please also share the doc on follow-up changes like discussed in last meeting with OSS.


-- 
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 pull request #9333: Add an option to disable the creation of the forward index for a column

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

   > @somandal We want to disable the forward index to save the storage. For sorted index, we store the index in inverted format (map from dictId to doc range), and also use it as forward index. We should allow disabling the forward index for it in the table config, and simply ignore it when creating the index.
   
   Thanks for discussing this, 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 #9333: Add an option to disable the creation of the forward index for a column

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


##########
pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/loader/defaultcolumn/BaseDefaultColumnHandler.java:
##########
@@ -372,6 +394,12 @@ protected boolean createColumnV1Indices(String column)
             return false;
           }
 
+          // TODO: Support forward index disabled derived column
+          if (_indexLoadingConfig.getForwardIndexDisabledColumns().contains(column)) {
+            LOGGER.warn("Skip creating forward index disabled derived column: {}", column);

Review Comment:
   In this case it is to warn and not throw error since user asked for a feature and we silently ignored (for now). 
   
   But in previous case, error should be thrown imo to tell user to change `derivedColumn` config's `sourceColumn` or refresh/backfill the segments first to generate forward index on `sourceColumn` and then generate target `derivedColumn`



-- 
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 #9333: Add an option to disable the creation of the forward index for a column

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


##########
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",

Review Comment:
   cool then we should be good. as part of reload i can perhaps test out more types on reload path too



-- 
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 #9333: Add an option to disable the creation of the forward index for a column

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


##########
pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/readers/forward/NoOpMVForwardIndexReader.java:
##########
@@ -0,0 +1,66 @@
+/**
+ * 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.segment.local.segment.index.readers.forward;
+
+import java.io.IOException;
+import org.apache.pinot.segment.spi.index.reader.ForwardIndexReader;
+import org.apache.pinot.segment.spi.index.reader.ForwardIndexReaderContext;
+import org.apache.pinot.spi.data.FieldSpec;
+
+
+/**
+ * No-op forward index reader meant to be used by multi-value columns with forward index disabled. This is meant to
+ * provide clean exception handling for operations which need to read forward index when forward index is disabled.
+ */
+public class NoOpMVForwardIndexReader implements ForwardIndexReader<ForwardIndexReaderContext> {

Review Comment:
   GROUP BY
   Transform
   Aggregate
   SELECT
   SCAN based filter
   .....
   
   
   Whoever accesses forward index now needs to be aware of the non-existence of forward index. The NO-OP reader on the other hand does the exact same thing but fails just slightly later by letting the caller attempting to read forward index and then fail with UNSUP. This is much cleaner imo.



-- 
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 #9333: Add an option to disable the creation of the forward index for a column

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


##########
pinot-segment-local/src/main/java/org/apache/pinot/segment/local/utils/TableConfigUtils.java:
##########
@@ -869,6 +871,25 @@ private static void validateFieldConfigList(@Nullable List<FieldConfig> fieldCon
           default:
             break;
         }
+
+        // Validate the forward index disabled compatibility if enabled for this column
+        Map<String, String> fieldConfigProperties = fieldConfig.getProperties();
+        if (fieldConfigProperties != null) {
+          boolean forwardIndexDisabled = Boolean.parseBoolean(fieldConfigProperties
+              .getOrDefault(FieldConfig.FORWARD_INDEX_DISABLED, FieldConfig.DEFAULT_FORWARD_INDEX_DISABLED));
+          if (forwardIndexDisabled) {
+            Preconditions.checkState((noDictionaryColumns == null || !noDictionaryColumns.contains(columnName))

Review Comment:
   suggest splitting the check into multiple checks for clearer message



-- 
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] Jackie-Jiang commented on pull request #9333: Add an option to disable the creation of the forward index for a column

Posted by GitBox <gi...@apache.org>.
Jackie-Jiang commented on PR #9333:
URL: https://github.com/apache/pinot/pull/9333#issuecomment-1241352079

   @somandal We want to disable the forward index to save the storage. For sorted index, we store the index in inverted format (map from dictId to doc range), and also use it as forward index. We should allow disabling the forward index for it in the table config, and simply ignore it when creating the 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 commented on a diff in pull request #9333: Add an option to disable the creation of the forward index for a column

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


##########
pinot-segment-local/src/main/java/org/apache/pinot/segment/local/utils/TableConfigUtils.java:
##########
@@ -897,6 +903,40 @@ private static void validateFieldConfigList(@Nullable List<FieldConfig> fieldCon
     }
   }
 
+  /**
+   * Validates the compatibility of the indexes if the column has the forward index disabled. Throws exceptions due to
+   * compatibility mismatch. The checks performed are:
+   *     - Validate dictionary is enabled.
+   *     - Validate inverted index is enabled.
+   *     - Validate that either no range index exists for column or the range index version is at least 2 and isn't a
+   *       multi-value column (since mulit-value defaults to index v1).
+   */
+  private static void validateForwardIndexDisabledIndexCompatibility(String columnName, FieldConfig fieldConfig,

Review Comment:
   (nit) we should consider keeping the check either here or during segment generation in `SegmentColumnarIndexCreator`. For now it is fine to safeguard multiple places until feature gets used for a while. 
   
   I am guessing basic validation at the table config level will always be needed since it will come handy even during  enabling / disabling the feature on an existing column. So an unsupported combination will be caught when user attempts to update the table config in controller as opposed to detecting later on in the reload path or segment refresh / push path



##########
pinot-segment-local/src/main/java/org/apache/pinot/segment/local/utils/TableConfigUtils.java:
##########
@@ -897,6 +903,40 @@ private static void validateFieldConfigList(@Nullable List<FieldConfig> fieldCon
     }
   }
 
+  /**
+   * Validates the compatibility of the indexes if the column has the forward index disabled. Throws exceptions due to
+   * compatibility mismatch. The checks performed are:
+   *     - Validate dictionary is enabled.
+   *     - Validate inverted index is enabled.
+   *     - Validate that either no range index exists for column or the range index version is at least 2 and isn't a
+   *       multi-value column (since mulit-value defaults to index v1).
+   */
+  private static void validateForwardIndexDisabledIndexCompatibility(String columnName, FieldConfig fieldConfig,

Review Comment:
   (nit) we should consider keeping the check either here or during segment generation in `SegmentColumnarIndexCreator`. For now it is fine to safeguard multiple places until feature gets used for a while. 
   
   I am guessing basic validation at the table config level will always be needed since it will come handy even during  enabling / disabling the feature on an existing column. So an unsupported combination will be caught when user attempts to update the table config in controller as opposed to detecting later on in the reload path and/or segment refresh / push path



-- 
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 #9333: Add an option to disable the creation of the forward index for a column

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


##########
pinot-segment-local/src/main/java/org/apache/pinot/segment/local/utils/TableConfigUtils.java:
##########
@@ -897,6 +903,40 @@ private static void validateFieldConfigList(@Nullable List<FieldConfig> fieldCon
     }
   }
 
+  /**
+   * Validates the compatibility of the indexes if the column has the forward index disabled. Throws exceptions due to
+   * compatibility mismatch. The checks performed are:
+   *     - Validate dictionary is enabled.
+   *     - Validate inverted index is enabled.
+   *     - Validate that either no range index exists for column or the range index version is at least 2 and isn't a
+   *       multi-value column (since mulit-value defaults to index v1).
+   */
+  private static void validateForwardIndexDisabledIndexCompatibility(String columnName, FieldConfig fieldConfig,
+      IndexingConfig indexingConfigs, List<String> noDictionaryColumns, Schema schema) {
+    Map<String, String> fieldConfigProperties = fieldConfig.getProperties();
+    if (fieldConfigProperties == null) {
+      return;
+    }
+
+    boolean forwardIndexDisabled = Boolean.parseBoolean(fieldConfigProperties.get(FieldConfig.FORWARD_INDEX_DISABLED));
+    if (!forwardIndexDisabled) {
+      return;
+    }
+
+    FieldSpec fieldSpec = schema.getFieldSpecFor(columnName);
+    Preconditions.checkState(noDictionaryColumns == null || !noDictionaryColumns.contains(columnName),

Review Comment:
   done, can you check if I've fixed this in the right way?



-- 
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 #9333: Add an option to disable the creation of the forward index for a column

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


##########
pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/loader/invertedindex/TextIndexHandler.java:
##########
@@ -174,18 +175,46 @@ private void createTextIndexForColumn(SegmentDirectory.Writer segmentWriter, Col
     // segmentDirectory is indicated to us by SegmentDirectoryPaths, we create lucene index there. There is no
     // further need to move around the lucene index directory since it is created with correct directory structure
     // based on segmentVersion.
-    try (ForwardIndexReader forwardIndexReader = LoaderUtils.getForwardIndexReader(segmentWriter, columnMetadata);
-        ForwardIndexReaderContext readerContext = forwardIndexReader.createContext();
-        TextIndexCreator textIndexCreator = indexCreatorProvider.newTextIndexCreator(IndexCreationContext.builder()
-            .withColumnMetadata(columnMetadata).withIndexDir(segmentDirectory).build().forTextIndex(_fstType, true))) {
-      if (columnMetadata.isSingleValue()) {
-        processSVField(segmentWriter, hasDictionary, forwardIndexReader, readerContext, textIndexCreator, numDocs,
-            columnMetadata);
+    try (TextIndexCreator textIndexCreator = indexCreatorProvider.newTextIndexCreator(
+        IndexCreationContext.builder().withColumnMetadata(columnMetadata).withIndexDir(segmentDirectory).build()
+            .forTextIndex(_fstType, true))) {
+      boolean forwardIndexDisabled = !segmentWriter.hasIndexFor(columnName, ColumnIndexType.FORWARD_INDEX);
+      if (forwardIndexDisabled) {
+        try (Dictionary dictionary = LoaderUtils.getDictionary(segmentWriter, columnMetadata)) {
+          // Create the text 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 text index without forward index
+          Preconditions.checkState(dictionary.length() == 1, String.format("Creating text index for forward index "

Review Comment:
   Hi @Jackie-Jiang what do you mean by directly throwing exception?
   
   Without adding this special handling, creation of these indexes for default columns will fail. Today when a new column is added, a default forward index is created. On the reload path I'm skipping the creation of the forward index when this feature is disabled for such default columns. I still create a dictionary though. I do want to create other indices since queries may fail if such segments are accessed and the expected indexes don't exist, right? Or am I missing something here? (i did see the segment preprocessor tests fail when I didn't do special handling for the inverted index for example since the default handlers directly access the forward index to create the auxiliary 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] somandal commented on a diff in pull request #9333: Add an option to disable the creation of the forward index for a column

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


##########
pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/column/PhysicalColumnIndexContainer.java:
##########
@@ -149,7 +154,12 @@ public PhysicalColumnIndexContainer(SegmentDirectory.Reader segmentReader, Colum
           return;
         }
       }
-      _forwardIndex = indexReaderProvider.newForwardIndexReader(fwdIndexBuffer, metadata);
+      if (forwardIndexDisabled && !metadata.isSorted()) {
+        // Forward index disabled flag is a no-op for sorted columns
+        _forwardIndex = null;
+      } else {
+        _forwardIndex = indexReaderProvider.newForwardIndexReader(fwdIndexBuffer, metadata);
+      }

Review Comment:
   done



##########
pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/column/PhysicalColumnIndexContainer.java:
##########
@@ -133,7 +133,12 @@ public PhysicalColumnIndexContainer(SegmentDirectory.Reader segmentReader, Colum
       _rangeIndex = null;
     }
 
-    PinotDataBuffer fwdIndexBuffer = segmentReader.getIndexFor(columnName, ColumnIndexType.FORWARD_INDEX);
+    // Setting the 'fwdIndexBuffer' to null if forward index is enabled. No-op index readers will be setup for the
+    // forward index disabled columns which doesn't require the 'fwdIndexBuffer'.
+    boolean forwardIndexDisabled = !segmentReader.hasIndexFor(columnName, ColumnIndexType.FORWARD_INDEX);

Review Comment:
   good point, 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 #9333: Add an option to disable the creation of the forward index for a column

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


##########
pinot-core/src/main/java/org/apache/pinot/core/common/DataFetcher.java:
##########
@@ -68,9 +68,14 @@ public DataFetcher(Map<String, DataSource> dataSourceMap) {
       String column = entry.getKey();
       DataSource dataSource = entry.getValue();
       DataSourceMetadata dataSourceMetadata = dataSource.getDataSourceMetadata();
+      ForwardIndexReader<?> forwardIndexReader = dataSource.getForwardIndex();
+      if (forwardIndexReader == null) {
+        throw new UnsupportedOperationException(
+            String.format("Forward index disabled for column: %s, cannot create DataFetcher!",
+            dataSourceMetadata.getFieldSpec().getName()));
+      }

Review Comment:
   done - modified this everywhere to use `IllegalStateException`



-- 
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 #9333: Add an option to disable the creation of the forward index for a column

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


##########
pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/loader/invertedindex/TextIndexHandler.java:
##########
@@ -174,18 +175,46 @@ private void createTextIndexForColumn(SegmentDirectory.Writer segmentWriter, Col
     // segmentDirectory is indicated to us by SegmentDirectoryPaths, we create lucene index there. There is no
     // further need to move around the lucene index directory since it is created with correct directory structure
     // based on segmentVersion.
-    try (ForwardIndexReader forwardIndexReader = LoaderUtils.getForwardIndexReader(segmentWriter, columnMetadata);
-        ForwardIndexReaderContext readerContext = forwardIndexReader.createContext();
-        TextIndexCreator textIndexCreator = indexCreatorProvider.newTextIndexCreator(IndexCreationContext.builder()
-            .withColumnMetadata(columnMetadata).withIndexDir(segmentDirectory).build().forTextIndex(_fstType, true))) {
-      if (columnMetadata.isSingleValue()) {
-        processSVField(segmentWriter, hasDictionary, forwardIndexReader, readerContext, textIndexCreator, numDocs,
-            columnMetadata);
+    try (TextIndexCreator textIndexCreator = indexCreatorProvider.newTextIndexCreator(
+        IndexCreationContext.builder().withColumnMetadata(columnMetadata).withIndexDir(segmentDirectory).build()
+            .forTextIndex(_fstType, true))) {
+      boolean forwardIndexDisabled = !segmentWriter.hasIndexFor(columnName, ColumnIndexType.FORWARD_INDEX);
+      if (forwardIndexDisabled) {
+        try (Dictionary dictionary = LoaderUtils.getDictionary(segmentWriter, columnMetadata)) {
+          // Create the text 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 text index without forward index
+          Preconditions.checkState(dictionary.length() == 1, String.format("Creating text index for forward index "

Review Comment:
   So we actually evaluated two approaches for this part:
   
   - Create the forward index, allow it to be marked as sorted. This will skip creation of the inverted index, and all other index handlers will kick in and create the index off of the forward index we create. (some index handlers rely on the forward index to construct the index)
       - Our main concern with this approach was that the segments will be inconsistent, some will have a forward index and others won't. This may give odd behavior at query time depending on the segment touched (e.g. some queries go fine with a given filter but don't go through with a slightly different filter). We wanted to avoid this.
       - We had also explored an option to identify such queries on the Broker or Server planning stage but quickly found that this became very ugly and had lots of exceptions and was error prone.
   - Don't create the forward index but do create the dictionary (just like the normal segment creation path). Modify the index handlers to create the index for such a default column if forward index is disabled without relying on the the forward index. In such cases we expect exactly 1 value anyways. This will give a consistent experience to users who run queries.
       - This has the disadvantage of doing some special handling in the index handlers for this scenario since the forward index won't be available anymore.
   
   Hope the above explains our thinking about the approaches and why we decided to go with the second one. Let me know if you'd like to discuss this in more detail. cc @siddharthteotia 



-- 
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 #9333: Add an option to disable the creation of the forward index for a column

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


##########
pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/column/PhysicalColumnIndexContainer.java:
##########
@@ -142,6 +145,8 @@ public PhysicalColumnIndexContainer(SegmentDirectory.Reader segmentReader, Colum
         // Single-value
         if (metadata.isSorted()) {
           // Sorted
+          // forwardIndexDisabled columns do not need to be handled here as forward index cannot be disabled on a

Review Comment:
   yes it is okay for 157 to be called with `fwdIndexBuffer` as null. We only set this to null if the forward index is disabled. When the forward index reader is created, the No-op forward index reader will be assigned which doesn't need a `fwdIndexBuffer`. I've added a comment where I set `fwdIndexBuffer` to indicate this behavior is expected. Let me know if you would like more clarification on 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] Jackie-Jiang commented on a diff in pull request #9333: Add an option to disable the creation of the forward index for a column

Posted by GitBox <gi...@apache.org>.
Jackie-Jiang commented on code in PR #9333:
URL: https://github.com/apache/pinot/pull/9333#discussion_r976061367


##########
pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/loader/invertedindex/TextIndexHandler.java:
##########
@@ -174,18 +175,46 @@ private void createTextIndexForColumn(SegmentDirectory.Writer segmentWriter, Col
     // segmentDirectory is indicated to us by SegmentDirectoryPaths, we create lucene index there. There is no
     // further need to move around the lucene index directory since it is created with correct directory structure
     // based on segmentVersion.
-    try (ForwardIndexReader forwardIndexReader = LoaderUtils.getForwardIndexReader(segmentWriter, columnMetadata);
-        ForwardIndexReaderContext readerContext = forwardIndexReader.createContext();
-        TextIndexCreator textIndexCreator = indexCreatorProvider.newTextIndexCreator(IndexCreationContext.builder()
-            .withColumnMetadata(columnMetadata).withIndexDir(segmentDirectory).build().forTextIndex(_fstType, true))) {
-      if (columnMetadata.isSingleValue()) {
-        processSVField(segmentWriter, hasDictionary, forwardIndexReader, readerContext, textIndexCreator, numDocs,
-            columnMetadata);
+    try (TextIndexCreator textIndexCreator = indexCreatorProvider.newTextIndexCreator(
+        IndexCreationContext.builder().withColumnMetadata(columnMetadata).withIndexDir(segmentDirectory).build()
+            .forTextIndex(_fstType, true))) {
+      boolean forwardIndexDisabled = !segmentWriter.hasIndexFor(columnName, ColumnIndexType.FORWARD_INDEX);
+      if (forwardIndexDisabled) {
+        try (Dictionary dictionary = LoaderUtils.getDictionary(segmentWriter, columnMetadata)) {
+          // Create the text 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 text index without forward index
+          Preconditions.checkState(dictionary.length() == 1, String.format("Creating text index for forward index "

Review Comment:
   For the MV default column, since we don't generate sorted index for it, we can consider directly creating inverted index in the default column handler.
   Essentially we should try to keep the behavior the same for reloaded segment and segment generated without the data for the default column. That way we don't need any special handling for other index handlers, and the logic will be 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 #9333: Add an option to disable the creation of the forward index for a column

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


##########
pinot-spi/src/main/java/org/apache/pinot/spi/config/table/TableConfig.java:
##########
@@ -143,6 +143,16 @@ public TableConfig(@JsonProperty(value = TABLE_NAME_KEY, required = true) String
     Preconditions.checkArgument(indexingConfig != null, "'tableIndexConfig' must be configured");
     Preconditions.checkArgument(customConfig != null, "'metadata' must be configured");
 
+    if (fieldConfigList != null) {
+      for (FieldConfig fieldConfig : fieldConfigList) {
+        Map<String, String> fieldConfigProperties = fieldConfig.getProperties();
+        if (fieldConfigProperties != null) {
+          boolean forwardIndexDisabled = Boolean.parseBoolean(fieldConfigProperties.getOrDefault(

Review Comment:
   not sure why is this code needed here. we don't seem to be using the return value anywhere locally ?



-- 
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 #9333: Add an option to disable the creation of the forward index for a column

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


##########
pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/creator/impl/SegmentColumnarIndexCreator.java:
##########
@@ -296,6 +309,40 @@ public void init(SegmentGeneratorConfig segmentCreationSpec, SegmentIndexCreatio
     }
   }
 
+  /**
+   * Validates the compatibility of the indexes if the column has the forward index disabled. Throws exceptions due to
+   * compatibility mismatch. The checks performed are:
+   *     - Validate dictionary is enabled.
+   *     - Validate inverted index is enabled.
+   *     - Validate that either no range index exists for column or the range index version is at least 2 and isn't a
+   *       multi-value column (since multi-value defaults to index v1).
+   *
+   * @param columnName Name of the column
+   * @param forwardIndexDisabled Whether the forward index is disabled for column or not
+   * @param dictEnabledColumn Whether the column is dictionary enabled or not
+   * @param columnIndexCreationInfo Column index creation info
+   * @param invertedIndexColumns Set of columns with inverted index enabled
+   * @param rangeIndexColumns Set of columns with range index enabled
+   * @param rangeIndexVersion Range index version
+   * @param fieldSpec FieldSpec of column
+   */
+  private void validateForwardIndexDisabledIndexCompatibility(String columnName, boolean forwardIndexDisabled,
+      boolean dictEnabledColumn, ColumnIndexCreationInfo columnIndexCreationInfo, Set<String> invertedIndexColumns,
+      Set<String> rangeIndexColumns, int rangeIndexVersion, FieldSpec fieldSpec) {
+    if (!forwardIndexDisabled) {
+      return;
+    }
+
+    Preconditions.checkState(dictEnabledColumn,
+        String.format("Cannot disable forward index for column %s without dictionary", columnName));
+    Preconditions.checkState(invertedIndexColumns.contains(columnName),
+        String.format("Cannot disable forward index for column %s without inverted index enabled", columnName));
+    Preconditions.checkState(!rangeIndexColumns.contains(columnName)
+        || (fieldSpec.isSingleValueField() && rangeIndexVersion == BitSlicedRangeIndexCreator.VERSION), String.format(
+            "Cannot disable forward index for column %s which has range index with version < 2 or is multi-value",

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 #9333: Add an option to disable the creation of the forward index for a column

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


##########
pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/column/PhysicalColumnIndexContainer.java:
##########
@@ -133,7 +133,11 @@ public PhysicalColumnIndexContainer(SegmentDirectory.Reader segmentReader, Colum
       _rangeIndex = null;
     }
 
-    PinotDataBuffer fwdIndexBuffer = segmentReader.getIndexFor(columnName, ColumnIndexType.FORWARD_INDEX);
+    // Setting the 'fwdIndexBuffer' to null if forward index is enabled. No-op index readers will be setup for the

Review Comment:
   good catch, 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 #9333: Add an option to disable the creation of the forward index for a column

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


##########
pinot-core/src/test/java/org/apache/pinot/queries/ForwardIndexDisabledSingleValueQueriesTest.java:
##########
@@ -0,0 +1,1088 @@
+/**
+ * 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.queries;
+
+import java.io.File;
+import java.net.URL;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.TimeUnit;
+import org.apache.commons.io.FileUtils;
+import org.apache.pinot.common.response.broker.BrokerResponseNative;
+import org.apache.pinot.common.response.broker.ResultTable;
+import org.apache.pinot.common.utils.DataSchema;
+import org.apache.pinot.segment.local.indexsegment.immutable.ImmutableSegmentLoader;
+import org.apache.pinot.segment.local.segment.creator.impl.SegmentIndexCreationDriverImpl;
+import org.apache.pinot.segment.local.segment.index.loader.IndexLoadingConfig;
+import org.apache.pinot.segment.spi.ColumnMetadata;
+import org.apache.pinot.segment.spi.ImmutableSegment;
+import org.apache.pinot.segment.spi.IndexSegment;
+import org.apache.pinot.segment.spi.creator.SegmentGeneratorConfig;
+import org.apache.pinot.segment.spi.creator.SegmentIndexCreationDriver;
+import org.apache.pinot.spi.config.table.FieldConfig;
+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.data.TimeGranularitySpec;
+import org.apache.pinot.spi.utils.ReadMode;
+import org.apache.pinot.spi.utils.builder.TableConfigBuilder;
+import org.testng.Assert;
+import org.testng.annotations.AfterClass;
+import org.testng.annotations.BeforeClass;
+import org.testng.annotations.Test;
+
+import static org.testng.Assert.assertEquals;
+import static org.testng.Assert.assertNotNull;
+import static org.testng.Assert.assertNull;
+import static org.testng.Assert.assertTrue;
+
+
+/**
+ * The <code>ForwardIndexDisabledSingleValueQueriesTest</code> class sets up the index segment for the no forward
+ * index single-value queries test.
+ * <p>There are totally 18 columns, 30000 records inside the original Avro file where 11 columns are selected to build
+ * the index segment. Selected columns information are as following:
+ * <ul>
+ *   ColumnName, FieldType, DataType, Cardinality, IsSorted, HasInvertedIndex, FwdIndexDisabled: S1, S2, HasRangeIndex
+ *   <li>column1, METRIC, INT, 6582, F, F, F, F, F</li>
+ *   <li>column3, METRIC, INT, 21910, F, F, F, F, F</li>
+ *   <li>column5, DIMENSION, STRING, 1, T, F, F, F, F</li>
+ *   <li>column6, DIMENSION, INT, 608, F, T, T, T, T</li>
+ *   <li>column7, DIMENSION, INT, 146, F, T, T, F, F</li>
+ *   <li>column9, DIMENSION, INT, 1737, F, F, F, F, F</li>
+ *   <li>column11, DIMENSION, STRING, 5, F, T, F, F, F</li>
+ *   <li>column12, DIMENSION, STRING, 5, F, F, F, F, F</li>
+ *   <li>column17, METRIC, INT, 24, F, T, F, F, F</li>
+ *   <li>column18, METRIC, INT, 1440, F, T, F, F, F</li>
+ *   <li>daysSinceEpoch, TIME, INT, 2, T, F, F, F, F</li>
+ * </ul>
+ */
+public class ForwardIndexDisabledSingleValueQueriesTest extends BaseQueriesTest {
+  private static final String AVRO_DATA = "data" + File.separator + "test_data-sv.avro";
+  private static final String SEGMENT_NAME_1 = "testTable_126164076_167572857";
+  private static final String SEGMENT_NAME_2 = "testTable_126164076_167572858";
+  private static final File INDEX_DIR = new File(FileUtils.getTempDirectory(),
+      "ForwardIndexDisabledSingleValueQueriesTest");
+
+  private static final String SELECT_STAR_QUERY = "SELECT * FROM testTable";
+
+  // Hard-coded query filter.
+  private static final String FILTER = " WHERE column1 > 100000000"
+      + " AND column3 BETWEEN 20000000 AND 1000000000"
+      + " AND column5 = 'gFuH'"
+      + " AND (column6 < 500000000 OR column11 NOT IN ('t', 'P'))"
+      + " AND daysSinceEpoch = 126164076";
+
+  private IndexSegment _indexSegment;
+  // Contains 2 index segments, one with 2 columns with forward index disabled, and the other with just 1.
+  private List<IndexSegment> _indexSegments;
+
+  private TableConfig _tableConfig;
+  private List<String> _invertedIndexColumns;
+  private List<String> _forwardIndexDisabledColumns;
+
+  @BeforeClass
+  public void buildAndLoadSegment()
+      throws Exception {
+    FileUtils.deleteQuietly(INDEX_DIR);
+
+    // Get resource file path.
+    URL resource = getClass().getClassLoader().getResource(AVRO_DATA);
+    assertNotNull(resource);
+    String filePath = resource.getFile();
+
+    // Build the segment schema.
+    Schema schema = new Schema.SchemaBuilder().setSchemaName("testTable").addMetric("column1", FieldSpec.DataType.INT)
+        .addMetric("column3", FieldSpec.DataType.INT).addSingleValueDimension("column5", FieldSpec.DataType.STRING)
+        .addSingleValueDimension("column6", FieldSpec.DataType.INT)
+        .addSingleValueDimension("column7", FieldSpec.DataType.INT)
+        .addSingleValueDimension("column9", FieldSpec.DataType.INT)
+        .addSingleValueDimension("column11", FieldSpec.DataType.STRING)
+        .addSingleValueDimension("column12", FieldSpec.DataType.STRING).addMetric("column17", FieldSpec.DataType.INT)
+        .addMetric("column18", FieldSpec.DataType.INT)
+        .addTime(new TimeGranularitySpec(FieldSpec.DataType.INT, TimeUnit.DAYS, "daysSinceEpoch"), null).build();
+
+    createSegment(filePath, SEGMENT_NAME_1, schema);
+    createSegment(filePath, SEGMENT_NAME_2, schema);
+
+    ImmutableSegment immutableSegment1 = loadSegmentWithMetadataChecks(SEGMENT_NAME_1);
+    ImmutableSegment immutableSegment2 = loadSegmentWithMetadataChecks(SEGMENT_NAME_2);
+
+    _indexSegment = immutableSegment1;
+    _indexSegments = Arrays.asList(immutableSegment1, immutableSegment2);
+  }
+
+  private void createSegment(String filePath, String segmentName, Schema schema)
+      throws Exception {
+    // Create field configs for the no forward index columns
+    List<FieldConfig> fieldConfigList = new ArrayList<>();
+    fieldConfigList.add(new FieldConfig("column6", FieldConfig.EncodingType.DICTIONARY, Collections.emptyList(), null,
+        Collections.singletonMap(FieldConfig.FORWARD_INDEX_DISABLED, Boolean.TRUE.toString())));
+    if (segmentName.equals(SEGMENT_NAME_1)) {
+      fieldConfigList.add(new FieldConfig("column7", FieldConfig.EncodingType.DICTIONARY, Collections.emptyList(),
+          null, Collections.singletonMap(FieldConfig.FORWARD_INDEX_DISABLED, Boolean.TRUE.toString())));
+
+      // Build table config based on segment 1 as it contains both columns under no forward index
+      _tableConfig =
+          new TableConfigBuilder(TableType.OFFLINE).setTableName("testTable").setTimeColumnName("daysSinceEpoch")
+              .setFieldConfigList(fieldConfigList).setRangeIndexColumns(Arrays.asList("column6")).build();
+    }
+
+    // Create the segment generator config.
+    SegmentGeneratorConfig segmentGeneratorConfig = new SegmentGeneratorConfig(_tableConfig, schema);
+    segmentGeneratorConfig.setInputFilePath(filePath);
+    segmentGeneratorConfig.setTableName("testTable");
+    segmentGeneratorConfig.setOutDir(INDEX_DIR.getAbsolutePath());
+    segmentGeneratorConfig.setSegmentName(segmentName);
+    // The segment generation code in SegmentColumnarIndexCreator will throw
+    // exception if start and end time in time column are not in acceptable
+    // range. For this test, we first need to fix the input avro data
+    // to have the time column values in allowed range. Until then, the check
+    // is explicitly disabled
+    segmentGeneratorConfig.setSkipTimeValueCheck(true);
+    _invertedIndexColumns = Arrays.asList("column6", "column7", "column11", "column17", "column18");
+    segmentGeneratorConfig.setInvertedIndexCreationColumns(_invertedIndexColumns);
+
+    _forwardIndexDisabledColumns = new ArrayList<>(Arrays.asList("column6", "column7"));
+    segmentGeneratorConfig.setForwardIndexDisabledColumns(_forwardIndexDisabledColumns);
+    segmentGeneratorConfig.setRangeIndexCreationColumns(Arrays.asList("column6"));
+
+    // Build the index segment.
+    SegmentIndexCreationDriver driver = new SegmentIndexCreationDriverImpl();
+    driver.init(segmentGeneratorConfig);
+    driver.build();
+  }
+
+  private ImmutableSegment loadSegmentWithMetadataChecks(String segmentName)
+      throws Exception {
+    IndexLoadingConfig indexLoadingConfig = new IndexLoadingConfig();
+    indexLoadingConfig.setTableConfig(_tableConfig);
+    indexLoadingConfig.setInvertedIndexColumns(new HashSet<>(_invertedIndexColumns));
+    indexLoadingConfig.setForwardIndexDisabledColumns(new HashSet<>(_forwardIndexDisabledColumns));
+    indexLoadingConfig.setRangeIndexColumns(new HashSet<>(Arrays.asList("column6")));
+    indexLoadingConfig.setReadMode(ReadMode.heap);
+
+    ImmutableSegment immutableSegment = ImmutableSegmentLoader.load(new File(INDEX_DIR, segmentName),
+        indexLoadingConfig);
+
+    Map<String, ColumnMetadata> columnMetadataMap1 = immutableSegment.getSegmentMetadata().getColumnMetadataMap();
+    columnMetadataMap1.forEach((column, metadata) -> {
+      if (column.equals("column6") || column.equals("column7")) {
+        assertTrue(metadata.hasDictionary());
+        assertTrue(metadata.isSingleValue());
+        assertNull(immutableSegment.getForwardIndex(column));
+      } else {
+        assertNotNull(immutableSegment.getForwardIndex(column));
+      }
+    });
+
+    return immutableSegment;
+  }
+
+  @AfterClass
+  public void deleteAndDestroySegment() {
+    FileUtils.deleteQuietly(INDEX_DIR);
+    _indexSegments.forEach((IndexSegment::destroy));
+  }
+
+  @Override
+  protected String getFilter() {
+    return FILTER;
+  }
+
+  @Override
+  protected IndexSegment getIndexSegment() {
+    return _indexSegment;
+  }
+
+  @Override
+  protected List<IndexSegment> getIndexSegments() {
+    return _indexSegments;
+  }
+
+  @Test
+  public void testSelectStarQueries() {
+    // Select * without any filters
+    try {
+      getBrokerResponse(SELECT_STAR_QUERY);
+      Assert.fail("Select * query should fail since forwardIndexDisabled on a select column");
+    } catch (IllegalStateException e) {
+      assertTrue(e.getMessage().contains("Forward index disabled for column:")
+          && e.getMessage().contains("cannot create DataFetcher!"));
+    }
+
+    // Select * with filters
+    try {
+      getBrokerResponse(SELECT_STAR_QUERY + FILTER);
+      Assert.fail("Select * query should fail since forwardIndexDisabled on a select column");
+    } catch (IllegalStateException e) {
+      assertTrue(e.getMessage().contains("Forward index disabled for column:")
+          && e.getMessage().contains("cannot create DataFetcher!"));
+    }
+  }
+
+  @Test
+  public void testSelectQueries() {
+    {
+      // Selection query without filters including a column with forwardIndexDisabled enabled on both segments
+      String query = "SELECT column1, column5, column6, column9, column11 FROM testTable";
+      try {
+        getBrokerResponse(query);
+        Assert.fail("Query should fail since forwardIndexDisabled on a select column");
+      } catch (IllegalStateException e) {
+        assertTrue(e.getMessage().contains("Forward index disabled for column:")
+            && e.getMessage().contains("cannot create DataFetcher!"));
+      }
+    }
+    {
+      // Selection query without filters including a column with forwardIndexDisabled enabled on one segment
+      String query = "SELECT column1, column5, column7, column9, column11 FROM testTable";
+      try {
+        getBrokerResponse(query);
+        Assert.fail("Query should fail since forwardIndexDisabled on a select column");
+      } catch (IllegalStateException e) {
+        assertTrue(e.getMessage().contains("Forward index disabled for column:")
+            && e.getMessage().contains("cannot create DataFetcher!"));
+      }
+    }
+    {
+      // Selection query with filters including a column with forwardIndexDisabled enabled on both segments
+      String query = "SELECT column1, column5, column6, column9, column11 FROM testTable WHERE column6 = 2147458029";
+      try {
+        getBrokerResponse(query);
+        Assert.fail("Query should fail since forwardIndexDisabled on a select column");
+      } catch (IllegalStateException e) {
+        assertTrue(e.getMessage().contains("Forward index disabled for column:")
+            && e.getMessage().contains("cannot create DataFetcher!"));
+      }
+    }
+    {
+      // Selection query with filters including a column with forwardIndexDisabled enabled on one segment
+      String query = "SELECT column1, column5, column7, column9, column11 FROM testTable WHERE column7 = 675695";
+      try {
+        getBrokerResponse(query);
+        Assert.fail("Query should fail since forwardIndexDisabled on a select column");
+      } catch (IllegalStateException e) {
+        assertTrue(e.getMessage().contains("Forward index disabled for column:")
+            && e.getMessage().contains("cannot create DataFetcher!"));
+      }
+    }
+    {
+      // Selection query without filters and without columns with forwardIndexDisabled enabled on either segment
+      String query = "SELECT column1, column5, column9, column11 FROM testTable ORDER BY column1";
+      BrokerResponseNative brokerResponseNative = getBrokerResponse(query);
+      assertTrue(brokerResponseNative.getProcessingExceptions() == null
+          || brokerResponseNative.getProcessingExceptions().size() == 0);
+      ResultTable resultTable = brokerResponseNative.getResultTable();
+      assertEquals(brokerResponseNative.getNumRowsResultSet(), 10);
+      assertEquals(brokerResponseNative.getTotalDocs(), 120_000L);
+      assertEquals(brokerResponseNative.getNumDocsScanned(), 120_000L);
+      assertEquals(brokerResponseNative.getNumSegmentsProcessed(), 4L);
+      assertEquals(brokerResponseNative.getNumSegmentsMatched(), 4L);
+      assertEquals(brokerResponseNative.getNumEntriesScannedPostFilter(), 120_120L);
+      assertEquals(brokerResponseNative.getNumEntriesScannedInFilter(), 0L);
+      assertNotNull(brokerResponseNative.getProcessingExceptions());
+      assertEquals(brokerResponseNative.getProcessingExceptions().size(), 0);
+      DataSchema dataSchema = new DataSchema(new String[]{"column1", "column5", "column9", "column11"},
+        new DataSchema.ColumnDataType[]{DataSchema.ColumnDataType.INT, DataSchema.ColumnDataType.STRING,
+            DataSchema.ColumnDataType.INT, DataSchema.ColumnDataType.STRING});
+      assertEquals(resultTable.getDataSchema(), dataSchema);
+      List<Object[]> resultRows = resultTable.getRows();
+      int previousColumn1 = Integer.MIN_VALUE;
+      for (Object[] resultRow : resultRows) {
+        assertEquals(resultRow.length, 4);
+        assertEquals((String) resultRow[1], "gFuH");
+        // Column 1
+        assertTrue((Integer) resultRow[0] >= previousColumn1);
+        previousColumn1 = (Integer) resultRow[0];
+      }
+
+      Object[] firstRow = resultRows.get(0);
+      // Column 11
+      assertEquals((String) firstRow[3], "o");
+    }
+    {
+      // Transform function on a selection clause with a forwardIndexDisabled column in transform
+      String query = "SELECT CONCAT(column6, column9, '-') from testTable";
+      try {
+        getBrokerResponse(query);
+        Assert.fail("Query should fail since forwardIndexDisabled on a column in transform");
+      } catch (IllegalStateException e) {
+        assertTrue(e.getMessage().contains("Forward index disabled for column:")
+            && e.getMessage().contains("cannot create DataFetcher!"));
+      }
+    }
+    {
+      // Transform function on a selection clause without a forwardIndexDisabled column in transform
+      String query = "SELECT CONCAT(column5, column9, '-') from testTable ORDER BY column1";
+      BrokerResponseNative brokerResponseNative = getBrokerResponse(query);
+      assertTrue(brokerResponseNative.getProcessingExceptions() == null
+          || brokerResponseNative.getProcessingExceptions().size() == 0);
+      ResultTable resultTable = brokerResponseNative.getResultTable();
+      assertEquals(brokerResponseNative.getNumRowsResultSet(), 10);
+      assertEquals(brokerResponseNative.getTotalDocs(), 120_000L);
+      assertEquals(brokerResponseNative.getNumDocsScanned(), 120_000L);
+      assertEquals(brokerResponseNative.getNumSegmentsProcessed(), 4L);
+      assertEquals(brokerResponseNative.getNumSegmentsMatched(), 4L);
+      assertEquals(brokerResponseNative.getNumEntriesScannedPostFilter(), 120_080L);
+      assertEquals(brokerResponseNative.getNumEntriesScannedInFilter(), 0L);
+      assertNotNull(brokerResponseNative.getProcessingExceptions());
+      assertEquals(brokerResponseNative.getProcessingExceptions().size(), 0);
+      DataSchema dataSchema = new DataSchema(new String[]{"concat(column5,column9,'-')"},
+          new DataSchema.ColumnDataType[]{DataSchema.ColumnDataType.STRING});
+      assertEquals(resultTable.getDataSchema(), dataSchema);
+      List<Object[]> resultRows = resultTable.getRows();
+      for (Object[] resultRow : resultRows) {
+        assertEquals(resultRow.length, 1);
+        assertTrue(resultRow[0].toString().startsWith("gFuH-"));
+      }
+    }
+    {
+      // Selection query with filters (not including forwardIndexDisabled column) and without columns with
+      // forwardIndexDisabled enabled on either segment
+      String query = "SELECT column1, column5, column9, column11 FROM testTable WHERE column1 > 100000000"
+        + " AND column3 BETWEEN 20000000 AND 1000000000"
+        + " AND column5 = 'gFuH'"
+        + " AND daysSinceEpoch = 126164076 ORDER BY column1";
+      BrokerResponseNative brokerResponseNative = getBrokerResponse(query);
+      assertTrue(brokerResponseNative.getProcessingExceptions() == null
+          || brokerResponseNative.getProcessingExceptions().size() == 0);
+      ResultTable resultTable = brokerResponseNative.getResultTable();
+      assertEquals(brokerResponseNative.getNumRowsResultSet(), 10);
+      assertEquals(brokerResponseNative.getTotalDocs(), 120_000L);
+      assertEquals(brokerResponseNative.getNumDocsScanned(), 42_368L);
+      assertEquals(brokerResponseNative.getNumSegmentsProcessed(), 4L);
+      assertEquals(brokerResponseNative.getNumSegmentsMatched(), 4L);
+      assertEquals(brokerResponseNative.getNumEntriesScannedPostFilter(), 42_488);
+      assertEquals(brokerResponseNative.getNumEntriesScannedInFilter(), 192744L);
+      assertNotNull(brokerResponseNative.getProcessingExceptions());
+      assertEquals(brokerResponseNative.getProcessingExceptions().size(), 0);
+      DataSchema dataSchema = new DataSchema(new String[]{"column1", "column5", "column9", "column11"},
+          new DataSchema.ColumnDataType[]{DataSchema.ColumnDataType.INT, DataSchema.ColumnDataType.STRING,
+              DataSchema.ColumnDataType.INT, DataSchema.ColumnDataType.STRING});
+      assertEquals(resultTable.getDataSchema(), dataSchema);
+      List<Object[]> resultRows = resultTable.getRows();
+      int previousColumn1 = Integer.MIN_VALUE;
+      for (Object[] resultRow : resultRows) {
+        assertEquals(resultRow.length, 4);
+        assertEquals((String) resultRow[1], "gFuH");
+        assertTrue((Integer) resultRow[0] >= previousColumn1);
+        previousColumn1 = (Integer) resultRow[0];
+      }
+
+      Object[] firstRow = resultRows.get(0);
+      // Column 11
+      assertEquals((String) firstRow[3], "P");
+    }
+    {
+      // Selection query with supported filters (including forwardIndexDisabled column) and without columns with
+      // forwardIndexDisabled enabled on either segment
+      String query = "SELECT column1, column5, column9, column11 FROM testTable WHERE column6 = 2147458029 "
+          + "ORDER BY column1";
+      BrokerResponseNative brokerResponseNative = getBrokerResponse(query);
+      assertTrue(brokerResponseNative.getProcessingExceptions() == null
+          || brokerResponseNative.getProcessingExceptions().size() == 0);
+      ResultTable resultTable = brokerResponseNative.getResultTable();
+      assertEquals(brokerResponseNative.getNumRowsResultSet(), 10);
+      assertEquals(brokerResponseNative.getTotalDocs(), 120_000L);
+      assertEquals(brokerResponseNative.getNumDocsScanned(), 16L);
+      assertEquals(brokerResponseNative.getNumSegmentsProcessed(), 4L);
+      assertEquals(brokerResponseNative.getNumSegmentsMatched(), 4L);
+      assertEquals(brokerResponseNative.getNumEntriesScannedPostFilter(), 64L);
+      assertEquals(brokerResponseNative.getNumEntriesScannedInFilter(), 0L);
+      assertNotNull(brokerResponseNative.getProcessingExceptions());
+      assertEquals(brokerResponseNative.getProcessingExceptions().size(), 0);
+      DataSchema dataSchema = new DataSchema(new String[]{"column1", "column5", "column9", "column11"},
+          new DataSchema.ColumnDataType[]{DataSchema.ColumnDataType.INT, DataSchema.ColumnDataType.STRING,
+              DataSchema.ColumnDataType.INT, DataSchema.ColumnDataType.STRING});
+      assertEquals(resultTable.getDataSchema(), dataSchema);
+      List<Object[]> resultRows = resultTable.getRows();
+      for (Object[] resultRow : resultRows) {
+        assertEquals(resultRow.length, 4);
+        assertEquals(resultRow[0], 635553468);
+        assertEquals((String) resultRow[1], "gFuH");
+        assertEquals(resultRow[2], 705242697);
+        assertEquals((String) resultRow[3], "P");
+      }
+    }
+    {
+      // Selection query with supported filters (including forwardIndexDisabled column) and without columns with
+      // forwardIndexDisabled enabled on either segment
+      String query = "SELECT column1, column5, column9, column11 FROM testTable WHERE column7 != 675695 "
+          + "ORDER BY column1";
+      BrokerResponseNative brokerResponseNative = getBrokerResponse(query);
+      assertTrue(brokerResponseNative.getProcessingExceptions() == null
+          || brokerResponseNative.getProcessingExceptions().size() == 0);
+      ResultTable resultTable = brokerResponseNative.getResultTable();
+      assertEquals(brokerResponseNative.getNumRowsResultSet(), 10);
+      assertEquals(brokerResponseNative.getTotalDocs(), 120_000L);
+      assertEquals(brokerResponseNative.getNumDocsScanned(), 119_908L);
+      assertEquals(brokerResponseNative.getNumSegmentsProcessed(), 4L);
+      assertEquals(brokerResponseNative.getNumSegmentsMatched(), 4L);
+      assertEquals(brokerResponseNative.getNumEntriesScannedPostFilter(), 120_028L);
+      assertEquals(brokerResponseNative.getNumEntriesScannedInFilter(), 0L);
+      assertNotNull(brokerResponseNative.getProcessingExceptions());
+      assertEquals(brokerResponseNative.getProcessingExceptions().size(), 0);
+      DataSchema dataSchema = new DataSchema(new String[]{"column1", "column5", "column9", "column11"},
+          new DataSchema.ColumnDataType[]{DataSchema.ColumnDataType.INT, DataSchema.ColumnDataType.STRING,
+              DataSchema.ColumnDataType.INT, DataSchema.ColumnDataType.STRING});
+      assertEquals(resultTable.getDataSchema(), dataSchema);
+      List<Object[]> resultRows = resultTable.getRows();
+      int previousColumn1 = Integer.MIN_VALUE;
+      for (Object[] resultRow : resultRows) {
+        assertEquals(resultRow.length, 4);
+        assertEquals((String) resultRow[1], "gFuH");
+        assertTrue((Integer) resultRow[0] >= previousColumn1);
+        previousColumn1 = (Integer) resultRow[0];
+      }
+
+      Object[] firstRow = resultRows.get(0);
+      // Column 11
+      assertEquals((String) firstRow[3], "o");
+    }
+    {
+      // Selection query with supported filters (including forwardIndexDisabled column) and without columns with
+      // forwardIndexDisabled enabled on either segment
+      String query = "SELECT column1, column5, column9, column11 FROM testTable WHERE column7 IN (675695, 2137685743) "
+          + "ORDER BY column1";
+      BrokerResponseNative brokerResponseNative = getBrokerResponse(query);
+      assertTrue(brokerResponseNative.getProcessingExceptions() == null
+          || brokerResponseNative.getProcessingExceptions().size() == 0);
+      ResultTable resultTable = brokerResponseNative.getResultTable();
+      assertEquals(brokerResponseNative.getNumRowsResultSet(), 10);
+      assertEquals(brokerResponseNative.getTotalDocs(), 120_000L);
+      assertEquals(brokerResponseNative.getNumDocsScanned(), 828L);
+      assertEquals(brokerResponseNative.getNumSegmentsProcessed(), 4L);
+      assertEquals(brokerResponseNative.getNumSegmentsMatched(), 4L);
+      assertEquals(brokerResponseNative.getNumEntriesScannedPostFilter(), 948L);
+      assertEquals(brokerResponseNative.getNumEntriesScannedInFilter(), 0L);
+      assertNotNull(brokerResponseNative.getProcessingExceptions());
+      assertEquals(brokerResponseNative.getProcessingExceptions().size(), 0);
+      DataSchema dataSchema = new DataSchema(new String[]{"column1", "column5", "column9", "column11"},
+          new DataSchema.ColumnDataType[]{DataSchema.ColumnDataType.INT, DataSchema.ColumnDataType.STRING,
+              DataSchema.ColumnDataType.INT, DataSchema.ColumnDataType.STRING});
+      assertEquals(resultTable.getDataSchema(), dataSchema);
+      List<Object[]> resultRows = resultTable.getRows();
+      int previousColumn1 = Integer.MIN_VALUE;
+      for (Object[] resultRow : resultRows) {
+        assertEquals(resultRow.length, 4);
+        assertEquals((String) resultRow[1], "gFuH");
+        assertTrue((Integer) resultRow[0] >= previousColumn1);
+        previousColumn1 = (Integer) resultRow[0];
+      }
+
+      Object[] firstRow = resultRows.get(0);
+      // Column 11
+      assertEquals((String) firstRow[3], "P");
+    }
+    {
+      // Selection query with supported filters (including forwardIndexDisabled column) and without columns with
+      // forwardIndexDisabled enabled on either segment
+      String query = "SELECT column1, column5, column9, column11 FROM testTable WHERE column6 NOT IN "
+          + "(1689277, 2147458029) ORDER BY column1";
+      BrokerResponseNative brokerResponseNative = getBrokerResponse(query);
+      assertTrue(brokerResponseNative.getProcessingExceptions() == null
+          || brokerResponseNative.getProcessingExceptions().size() == 0);
+      ResultTable resultTable = brokerResponseNative.getResultTable();
+      assertEquals(brokerResponseNative.getNumRowsResultSet(), 10);
+      assertEquals(brokerResponseNative.getTotalDocs(), 120_000L);
+      assertEquals(brokerResponseNative.getNumDocsScanned(), 119_980L);
+      assertEquals(brokerResponseNative.getNumSegmentsProcessed(), 4L);
+      assertEquals(brokerResponseNative.getNumSegmentsMatched(), 4L);
+      assertEquals(brokerResponseNative.getNumEntriesScannedPostFilter(), 120_100L);
+      assertEquals(brokerResponseNative.getNumEntriesScannedInFilter(), 0L);
+      assertNotNull(brokerResponseNative.getProcessingExceptions());
+      assertEquals(brokerResponseNative.getProcessingExceptions().size(), 0);
+      DataSchema dataSchema = new DataSchema(new String[]{"column1", "column5", "column9", "column11"},
+          new DataSchema.ColumnDataType[]{DataSchema.ColumnDataType.INT, DataSchema.ColumnDataType.STRING,
+              DataSchema.ColumnDataType.INT, DataSchema.ColumnDataType.STRING});
+      assertEquals(resultTable.getDataSchema(), dataSchema);
+      List<Object[]> resultRows = resultTable.getRows();
+      int previousColumn1 = Integer.MIN_VALUE;
+      for (Object[] resultRow : resultRows) {
+        assertEquals(resultRow.length, 4);
+        assertEquals((String) resultRow[1], "gFuH");
+        assertTrue((Integer) resultRow[0] >= previousColumn1);
+        previousColumn1 = (Integer) resultRow[0];
+      }
+
+      Object[] firstRow = resultRows.get(0);
+      // Column 11
+      assertEquals((String) firstRow[3], "o");
+    }
+    {
+      // Query with literal only in SELECT
+      String query = "SELECT 'marvin' from testTable ORDER BY column1";
+      BrokerResponseNative brokerResponseNative = getBrokerResponse(query);
+      assertTrue(brokerResponseNative.getProcessingExceptions() == null
+          || brokerResponseNative.getProcessingExceptions().size() == 0);
+      ResultTable resultTable = brokerResponseNative.getResultTable();
+      assertEquals(brokerResponseNative.getNumRowsResultSet(), 10);
+      assertEquals(brokerResponseNative.getTotalDocs(), 120_000L);
+      assertEquals(brokerResponseNative.getNumDocsScanned(), 120_000L);
+      assertEquals(brokerResponseNative.getNumSegmentsProcessed(), 4L);
+      assertEquals(brokerResponseNative.getNumSegmentsMatched(), 4L);
+      assertEquals(brokerResponseNative.getNumEntriesScannedPostFilter(), 120_000L);
+      assertEquals(brokerResponseNative.getNumEntriesScannedInFilter(), 0L);
+      assertNotNull(brokerResponseNative.getProcessingExceptions());
+      assertEquals(brokerResponseNative.getProcessingExceptions().size(), 0);
+      assertEquals(resultTable.getDataSchema(), new DataSchema(new String[]{"'marvin'"},
+          new DataSchema.ColumnDataType[]{DataSchema.ColumnDataType.STRING}));
+      List<Object[]> resultRows = resultTable.getRows();
+      for (Object[] resultRow : resultRows) {
+        assertEquals(resultRow.length, 1);
+        assertEquals((String) resultRow[0], "marvin");
+      }
+    }
+    {
+      // Selection query with '<' filter on a forwardIndexDisabled column with range index available
+      String query = "SELECT column1, column5, column9, column11 FROM testTable WHERE column6 < 2147458029 AND "
+          + "column6 > 1699000 ORDER BY column1";
+      BrokerResponseNative brokerResponseNative = getBrokerResponse(query);
+      assertTrue(brokerResponseNative.getProcessingExceptions() == null
+          || brokerResponseNative.getProcessingExceptions().size() == 0);
+      ResultTable resultTable = brokerResponseNative.getResultTable();
+      assertEquals(brokerResponseNative.getNumRowsResultSet(), 10);
+      assertEquals(brokerResponseNative.getTotalDocs(), 120_000L);
+      assertEquals(brokerResponseNative.getNumDocsScanned(), 119_980L);
+      assertEquals(brokerResponseNative.getNumSegmentsProcessed(), 4L);
+      assertEquals(brokerResponseNative.getNumSegmentsMatched(), 4L);
+      assertEquals(brokerResponseNative.getNumEntriesScannedPostFilter(), 120_100L);
+      assertEquals(brokerResponseNative.getNumEntriesScannedInFilter(), 0L);
+      assertNotNull(brokerResponseNative.getProcessingExceptions());
+      assertEquals(brokerResponseNative.getProcessingExceptions().size(), 0);
+      DataSchema dataSchema = new DataSchema(new String[]{"column1", "column5", "column9", "column11"},
+          new DataSchema.ColumnDataType[]{DataSchema.ColumnDataType.INT, DataSchema.ColumnDataType.STRING,
+              DataSchema.ColumnDataType.INT, DataSchema.ColumnDataType.STRING});
+      assertEquals(resultTable.getDataSchema(), dataSchema);
+      List<Object[]> resultRows = resultTable.getRows();
+      int previousColumn1 = Integer.MIN_VALUE;
+      for (Object[] resultRow : resultRows) {
+        assertEquals(resultRow.length, 4);
+        assertEquals((String) resultRow[1], "gFuH");
+        assertTrue((Integer) resultRow[0] >= previousColumn1);
+        previousColumn1 = (Integer) resultRow[0];
+      }
+
+      Object[] firstRow = resultRows.get(0);
+      // Column 11
+      assertEquals((String) firstRow[3], "o");
+    }
+    {
+      // Selection query with '>=' filter on a forwardIndexDisabled column without range index available
+      String query = "SELECT column1, column5, column9, column11 FROM testTable WHERE column7 >= 676000";
+      try {
+        getBrokerResponse(query);
+        Assert.fail("Query should fail since forwardIndexDisabled on a range query column without range index");
+      } catch (IllegalStateException e) {
+        assertTrue(e.getMessage().contains("Forward index disabled for column:")
+            && e.getMessage().contains("creating ScanDocIdSet unsupported!"));
+      }
+    }
+    {
+      // Select query with a filter on a column which doesn't have forwardIndexDisabled enabled
+      String query = "SELECT column1, column5, column9 from testTable WHERE column9 < 50000 ORDER BY column1";
+      BrokerResponseNative brokerResponseNative = getBrokerResponse(query);
+      assertTrue(brokerResponseNative.getProcessingExceptions() == null
+          || brokerResponseNative.getProcessingExceptions().size() == 0);
+      ResultTable resultTable = brokerResponseNative.getResultTable();
+      assertEquals(brokerResponseNative.getNumRowsResultSet(), 4);
+      assertEquals(brokerResponseNative.getTotalDocs(), 120_000L);
+      assertEquals(brokerResponseNative.getNumDocsScanned(), 4L);
+      assertEquals(brokerResponseNative.getNumSegmentsProcessed(), 4L);
+      assertEquals(brokerResponseNative.getNumSegmentsMatched(), 4L);
+      assertEquals(brokerResponseNative.getNumEntriesScannedPostFilter(), 12L);
+      assertEquals(brokerResponseNative.getNumEntriesScannedInFilter(), 120000L);
+      assertNotNull(brokerResponseNative.getProcessingExceptions());
+      assertEquals(brokerResponseNative.getProcessingExceptions().size(), 0);
+      assertEquals(resultTable.getDataSchema(), new DataSchema(new String[]{"column1", "column5", "column9"},
+          new DataSchema.ColumnDataType[]{
+          DataSchema.ColumnDataType.INT, DataSchema.ColumnDataType.STRING, DataSchema.ColumnDataType.INT
+      }));
+      List<Object[]> resultRows = resultTable.getRows();
+      for (Object[] resultRow : resultRows) {
+        assertEquals(resultRow.length, 3);
+      }
+    }
+    {
+      // Transform function on a filter clause for forwardIndexDisabled column in transform
+      String query = "SELECT column1, column11 from testTable WHERE CONCAT(column6, column9, '-') = '1689277-11270'";
+      BrokerResponseNative brokerResponseNative = getBrokerResponse(query);
+      assertTrue(brokerResponseNative.getProcessingExceptions() != null
+          && brokerResponseNative.getProcessingExceptions().size() > 0);
+    }
+    {
+      // Transform function on a filter clause for a non-forwardIndexDisabled column in transform
+      String query = "SELECT column1, column11 from testTable WHERE CONCAT(column5, column9, '-') = 'gFuH-11270' "
+          + "ORDER BY column1";
+      BrokerResponseNative brokerResponseNative = getBrokerResponse(query);
+      assertTrue(brokerResponseNative.getProcessingExceptions() == null
+          || brokerResponseNative.getProcessingExceptions().size() == 0);
+      ResultTable resultTable = brokerResponseNative.getResultTable();
+      assertEquals(brokerResponseNative.getNumRowsResultSet(), 4);
+      assertEquals(brokerResponseNative.getTotalDocs(), 120_000L);
+      assertEquals(brokerResponseNative.getNumDocsScanned(), 4L);
+      assertEquals(brokerResponseNative.getNumSegmentsProcessed(), 4L);
+      assertEquals(brokerResponseNative.getNumSegmentsMatched(), 4L);
+      assertEquals(brokerResponseNative.getNumEntriesScannedPostFilter(), 8L);
+      assertEquals(brokerResponseNative.getNumEntriesScannedInFilter(), 120000L);
+      assertNotNull(brokerResponseNative.getProcessingExceptions());
+      assertEquals(brokerResponseNative.getProcessingExceptions().size(), 0);
+      assertEquals(resultTable.getDataSchema(), new DataSchema(new String[]{"column1", "column11"},
+          new DataSchema.ColumnDataType[]{
+              DataSchema.ColumnDataType.INT, DataSchema.ColumnDataType.STRING
+          }));
+      List<Object[]> resultRows = resultTable.getRows();
+      assertEquals(resultRows.size(), 4);
+      for (Object[] resultRow : resultRows) {
+        assertEquals(resultRow.length, 2);
+        assertEquals(resultRow[0], 815409257);
+        assertEquals(resultRow[1], "P");
+      }
+    }
+  }
+
+  @Test
+  public void testSelectWithDistinctQueries() {
+    {
+      // Select a mix of forwardIndexDisabled and non-forwardIndexDisabled columns with distinct
+      String query = "SELECT DISTINCT column1, column6, column9 FROM testTable LIMIT 10";
+      try {
+        getBrokerResponse(query);
+        Assert.fail("Query should fail since forwardIndexDisabled on a column in select distinct");
+      } catch (IllegalStateException e) {
+        assertTrue(e.getMessage().contains("Forward index disabled for column:")
+            && e.getMessage().contains("cannot create DataFetcher!"));
+      }
+    }
+    {
+      // Select non-forwardIndexDisabled columns with distinct
+      String query = "SELECT DISTINCT column1, column5, column9 FROM testTable ORDER BY column1 LIMIT 10";
+      BrokerResponseNative brokerResponseNative = getBrokerResponse(query);
+      assertTrue(brokerResponseNative.getProcessingExceptions() == null
+          || brokerResponseNative.getProcessingExceptions().size() == 0);
+      ResultTable resultTable = brokerResponseNative.getResultTable();
+      assertEquals(brokerResponseNative.getNumRowsResultSet(), 10);
+      assertEquals(brokerResponseNative.getTotalDocs(), 120_000L);
+      assertEquals(brokerResponseNative.getNumDocsScanned(), 120_000L);
+      assertEquals(brokerResponseNative.getNumSegmentsProcessed(), 4L);
+      assertEquals(brokerResponseNative.getNumSegmentsMatched(), 4L);
+      assertEquals(brokerResponseNative.getNumEntriesScannedPostFilter(), 360_000L);
+      assertEquals(brokerResponseNative.getNumEntriesScannedInFilter(), 0L);
+      assertNotNull(brokerResponseNative.getProcessingExceptions());
+      assertEquals(brokerResponseNative.getProcessingExceptions().size(), 0);
+      assertEquals(resultTable.getDataSchema(), new DataSchema(new String[]{"column1", "column5", "column9"},
+          new DataSchema.ColumnDataType[]{DataSchema.ColumnDataType.INT, DataSchema.ColumnDataType.STRING,
+              DataSchema.ColumnDataType.INT}));
+      List<Object[]> resultRows = resultTable.getRows();
+      int previousColumn1 = Integer.MIN_VALUE;
+      for (Object[] resultRow : resultRows) {
+        assertEquals(resultRow.length, 3);
+        assertEquals(resultRow[1], "gFuH");
+        assertTrue((Integer) resultRow[0] >= previousColumn1);
+        previousColumn1 = (Integer) resultRow[0];
+      }
+    }
+  }
+
+  @Test
+  public void testSelectWithGroupByOrderByQueries() {
+    {
+      // Select a mix of forwardIndexDisabled and non-forwardIndexDisabled columns with group by order by
+      String query = "SELECT column1, column6 FROM testTable GROUP BY column1, column6 ORDER BY column1, column6 "
+          + " LIMIT 10";
+      try {
+        getBrokerResponse(query);
+        Assert.fail("Query should fail since forwardIndexDisabled on a column in select group by order by");
+      } catch (IllegalStateException e) {
+        assertTrue(e.getMessage().contains("Forward index disabled for column:")
+            && e.getMessage().contains("cannot create DataFetcher!"));
+      }
+    }
+    {
+      // Select forwardIndexDisabled columns with group by order by
+      String query = "SELECT column7, column6 FROM testTable GROUP BY column7, column6 ORDER BY column7, column6 "
+          + " LIMIT 10";
+      try {
+        getBrokerResponse(query);
+        Assert.fail("Query should fail since forwardIndexDisabled on a column in select group by order by");
+      } catch (IllegalStateException e) {
+        assertTrue(e.getMessage().contains("Forward index disabled for column:")
+            && e.getMessage().contains("cannot create DataFetcher!"));
+      }
+    }
+    {
+      // Select non-forwardIndexDisabled columns with group by order by
+      String query = "SELECT column1, column5 FROM testTable GROUP BY column1, column5 ORDER BY column1, column5 "
+          + " LIMIT 10";
+      BrokerResponseNative brokerResponseNative = getBrokerResponse(query);
+      assertTrue(brokerResponseNative.getProcessingExceptions() == null
+          || brokerResponseNative.getProcessingExceptions().size() == 0);
+      ResultTable resultTable = brokerResponseNative.getResultTable();
+      assertEquals(brokerResponseNative.getNumRowsResultSet(), 10);
+      assertEquals(brokerResponseNative.getTotalDocs(), 120_000L);
+      assertEquals(brokerResponseNative.getNumDocsScanned(), 120000L);
+      assertEquals(brokerResponseNative.getNumSegmentsProcessed(), 4L);
+      assertEquals(brokerResponseNative.getNumSegmentsMatched(), 4L);
+      assertEquals(brokerResponseNative.getNumEntriesScannedPostFilter(), 240000L);
+      assertEquals(brokerResponseNative.getNumEntriesScannedInFilter(), 0L);
+      assertNotNull(brokerResponseNative.getProcessingExceptions());
+      assertEquals(brokerResponseNative.getProcessingExceptions().size(), 0);
+      assertEquals(resultTable.getDataSchema(), new DataSchema(new String[]{"column1", "column5"},
+          new DataSchema.ColumnDataType[]{DataSchema.ColumnDataType.INT, DataSchema.ColumnDataType.STRING}));
+      List<Object[]> resultRows = resultTable.getRows();
+      int previousVal = -1;
+      for (Object[] resultRow : resultRows) {
+        assertEquals(resultRow.length, 2);
+        assertEquals(resultRow[1], "gFuH");
+        assertTrue((int) resultRow[0] > previousVal);
+        previousVal = (int) resultRow[0];
+      }
+    }
+    {
+      // Select forwardIndexDisabled columns using transform with group by order by
+      String query = "SELECT CONCAT(column1, column6, '-') FROM testTable GROUP BY CONCAT(column1, column6, '-') "
+          + "ORDER BY CONCAT(column1, column6, '-') LIMIT 10";
+      try {
+        getBrokerResponse(query);
+        Assert.fail("Query should fail since forwardIndexDisabled on a transformed column in group by order by");
+      } catch (IllegalStateException e) {
+        assertTrue(e.getMessage().contains("Forward index disabled for column:")
+            && e.getMessage().contains("cannot create DataFetcher!"));
+      }
+    }
+    {
+      // Select non-forwardIndexDisabled columns using transform with group by order by
+      String query = "SELECT CONCAT(column1, column5, '-') FROM testTable GROUP BY CONCAT(column1, column5, '-') "
+          + "ORDER BY CONCAT(column1, column5, '-') LIMIT 10";
+      BrokerResponseNative brokerResponseNative = getBrokerResponse(query);
+      assertTrue(brokerResponseNative.getProcessingExceptions() == null
+          || brokerResponseNative.getProcessingExceptions().size() == 0);
+      ResultTable resultTable = brokerResponseNative.getResultTable();
+      assertEquals(brokerResponseNative.getNumRowsResultSet(), 10);
+      assertEquals(brokerResponseNative.getTotalDocs(), 120_000L);
+      assertEquals(brokerResponseNative.getNumDocsScanned(), 120000L);
+      assertEquals(brokerResponseNative.getNumSegmentsProcessed(), 4L);
+      assertEquals(brokerResponseNative.getNumSegmentsMatched(), 4L);
+      assertEquals(brokerResponseNative.getNumEntriesScannedPostFilter(), 240000L);
+      assertEquals(brokerResponseNative.getNumEntriesScannedInFilter(), 0L);
+      assertNotNull(brokerResponseNative.getProcessingExceptions());
+      assertEquals(brokerResponseNative.getProcessingExceptions().size(), 0);
+      assertEquals(resultTable.getDataSchema(), new DataSchema(new String[]{"concat(column1,column5,'-')"},
+          new DataSchema.ColumnDataType[]{DataSchema.ColumnDataType.STRING}));
+      List<Object[]> resultRows = resultTable.getRows();
+      for (Object[] resultRow : resultRows) {
+        assertEquals(resultRow.length, 1);
+        assertTrue(resultRow[0].toString().endsWith("-gFuH"));
+      }
+    }
+  }
+
+  @Test
+  public void testSelectWithAggregationQueries() {
+    {
+      // Allowed aggregation functions on forwardIndexDisabled columns
+      String query = "SELECT max(column7), min(column7), count(column7), minmaxrange(column7), "
+          + "distinctcount(column7), distinctcounthll(column6), distinctcountrawhll(column7), "
+          + "distinctcountsmarthll(column6) from testTable";
+      BrokerResponseNative brokerResponseNative = getBrokerResponse(query);
+      assertTrue(brokerResponseNative.getProcessingExceptions() == null
+          || brokerResponseNative.getProcessingExceptions().size() == 0);
+      ResultTable resultTable = brokerResponseNative.getResultTable();
+      assertEquals(brokerResponseNative.getNumRowsResultSet(), 1);
+      assertEquals(brokerResponseNative.getTotalDocs(), 120_000L);
+      assertEquals(brokerResponseNative.getNumDocsScanned(), 120_000L);
+      assertEquals(brokerResponseNative.getNumSegmentsProcessed(), 4L);
+      assertEquals(brokerResponseNative.getNumSegmentsMatched(), 4L);
+      assertEquals(brokerResponseNative.getNumEntriesScannedPostFilter(), 0L);
+      assertEquals(brokerResponseNative.getNumEntriesScannedInFilter(), 0L);
+      assertNotNull(brokerResponseNative.getProcessingExceptions());
+      assertEquals(brokerResponseNative.getProcessingExceptions().size(), 0);
+      assertEquals(resultTable.getDataSchema(), new DataSchema(new String[]{"max(column7)", "min(column7)", "count(*)",
+      "minmaxrange(column7)", "distinctcount(column7)", "distinctcounthll(column6)", "distinctcountrawhll(column7)",
+      "distinctcountsmarthll(column6)"},
+          new DataSchema.ColumnDataType[]{DataSchema.ColumnDataType.DOUBLE, DataSchema.ColumnDataType.DOUBLE,
+              DataSchema.ColumnDataType.LONG, DataSchema.ColumnDataType.DOUBLE, DataSchema.ColumnDataType.INT,
+              DataSchema.ColumnDataType.LONG, DataSchema.ColumnDataType.STRING, DataSchema.ColumnDataType.INT}));
+      List<Object[]> resultRows = resultTable.getRows();
+      for (Object[] resultRow : resultRows) {
+        assertEquals(resultRow.length, 8);
+        assertEquals(resultRow[0], 2.137685743E9);
+        assertEquals(resultRow[1], 675695.0);
+        assertEquals(resultRow[2], 120000L);
+        assertEquals(resultRow[3], 2.137010048E9);
+        assertEquals(resultRow[4], 146);
+        assertEquals(resultRow[5], 695L);
+        assertEquals(resultRow[7], 608);
+      }
+    }
+    {
+      // Not allowed aggregation functions on forwardIndexDisabled columns
+      String query = "SELECT sum(column7), avg(column6) from testTable";
+      try {
+        getBrokerResponse(query);
+        Assert.fail("Query should fail since forwardIndexDisabled on a column in unsupported aggregation query");
+      } catch (IllegalStateException e) {
+        assertTrue(e.getMessage().contains("Forward index disabled for column:")
+            && e.getMessage().contains("cannot create DataFetcher!"));
+      }
+    }
+    {
+      // Allowed aggregation functions on forwardIndexDisabled columns with group by on non-forwardIndexDisabled
+      // column
+      String query = "SELECT column1, max(column6) from testTable GROUP BY column1";
+      try {
+        getBrokerResponse(query);
+        Assert.fail("Query should fail since forwardIndexDisabled on a column in unsupported aggregation query");
+      } catch (IllegalStateException e) {
+        assertTrue(e.getMessage().contains("Forward index disabled for column:")
+            && e.getMessage().contains("cannot create DataFetcher!"));
+      }
+    }
+    {
+      // Allowed aggregation functions on forwardIndexDisabled columns with group by order by on
+      // non-forwardIndexDisabled column
+      String query = "SELECT column1, max(column6) from testTable GROUP BY column1 ORDER BY column1";
+      try {
+        getBrokerResponse(query);
+        Assert.fail("Query should fail since forwardIndexDisabled on a column in unsupported aggregation query");
+      } catch (IllegalStateException e) {
+        assertTrue(e.getMessage().contains("Forward index disabled for column:")
+            && e.getMessage().contains("cannot create DataFetcher!"));
+      }
+    }
+    {
+      // Allowed aggregation functions on non-forwardIndexDisabled columns with group by on non-forwardIndexDisabled
+      // column but order by on allowed aggregation function on forwardIndexDisabled column
+      String query = "SELECT column1, max(column9) from testTable GROUP BY column1 ORDER BY min(column6)";

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 #9333: Add an option to disable the creation of the forward index for a column

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


##########
pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/loader/defaultcolumn/BaseDefaultColumnHandler.java:
##########
@@ -224,6 +225,21 @@ Map<String, DefaultColumnAction> computeDefaultColumnActionMap() {
       if (columnMetadata != null) {
         // Column exists in the segment, check if we need to update the value.
 
+        if (_segmentWriter != null && !columnMetadata.isAutoGenerated()) {
+          // Check that forward index disabled isn't enabled / disabled on an existing column (not auto-generated).
+          // TODO: Add support for reloading segments when forward index disabled flag is enabled or disabled
+          boolean forwardIndexDisabled = !_segmentWriter.hasIndexFor(column, ColumnIndexType.FORWARD_INDEX);
+          if (forwardIndexDisabled != _indexLoadingConfig.getForwardIndexDisabledColumns()
+              .contains(column)) {
+            String failureMessage =
+                "Forward index disabled in segment: " + forwardIndexDisabled + " for column: " + column
+                    + " does not match forward index disabled flag: "
+                    + _indexLoadingConfig.getForwardIndexDisabledColumns().contains(column) + " in the TableConfig, "
+                    + "updating this flag is not supported at the moment.";

Review Comment:
   I've modified the comment but I still think we should fail and throw an exception here. let's discuss more about this if you think a warning log is enough. Also this cannot be detected by changes in the table config since there we don't know the status of the actual segment



-- 
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 #9333: Add an option to disable the creation of the forward index for a column

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


##########
pinot-segment-local/src/main/java/org/apache/pinot/segment/local/utils/TableConfigUtils.java:
##########
@@ -897,6 +903,44 @@ private static void validateFieldConfigList(@Nullable List<FieldConfig> fieldCon
     }
   }
 
+  /**
+   * Validates the compatibility of the indexes if the column has the forward index disabled. Throws exceptions due to
+   * compatibility mismatch. The checks performed are:
+   *     - Validate dictionary is enabled.
+   *     - Validate inverted index is enabled.
+   *     - Validate that either no range index exists for column or the range index version is at least 2 and isn't a
+   *       multi-value column (since mulit-value defaults to index v1).
+   */
+  private static void validateForwardIndexDisabledIndexCompatibility(String columnName, FieldConfig fieldConfig,
+      IndexingConfig indexingConfigs, List<String> noDictionaryColumns, Schema schema) {
+    Map<String, String> fieldConfigProperties = fieldConfig.getProperties();
+    if (fieldConfigProperties == null) {
+      return;
+    }
+
+    boolean forwardIndexDisabled = Boolean.parseBoolean(fieldConfigProperties.get(FieldConfig.FORWARD_INDEX_DISABLED));
+    if (!forwardIndexDisabled) {
+      return;
+    }
+
+    FieldSpec fieldSpec = schema.getFieldSpecFor(columnName);
+    Preconditions.checkState(fieldConfig.getEncodingType() == FieldConfig.EncodingType.DICTIONARY
+            || noDictionaryColumns == null || !noDictionaryColumns.contains(columnName),
+        String.format("Forward index disabled column %s must have dictionary enabled", columnName));

Review Comment:
   @somandal  - this is incorrect. 
   
   Error for this scenario should be thrown if column is marked noDict -- either `encodingType is RAW` or `noDictionaryColumns` contains the column.
   
   This code seems to be throwing error for column marked with DICTIONARY encodingType or if it is not present in `noDictionaryColumns` unless I am misreading 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] somandal commented on a diff in pull request #9333: Add an option to disable the creation of the forward index for a column

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


##########
pinot-core/src/main/java/org/apache/pinot/core/operator/filter/ScanBasedFilterOperator.java:
##########
@@ -48,10 +49,16 @@ public class ScanBasedFilterOperator extends BaseFilterOperator {
   @Override
   protected FilterBlock getNextBlock() {
     DataSourceMetadata dataSourceMetadata = _dataSource.getDataSourceMetadata();
+    ForwardIndexReader<?> forwardIndexReader = _dataSource.getForwardIndex();

Review Comment:
   Good catch! moved this to the constructor



-- 
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 #9333: Add an option to disable the creation of the forward index for a column

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


##########
pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/loader/defaultcolumn/BaseDefaultColumnHandler.java:
##########
@@ -388,8 +416,40 @@ protected boolean createColumnV1Indices(String column)
     return true;
   }
 
+  /**
+   * Validates the compatibility of the indexes if the column has the forward index disabled. Throws exceptions due to
+   * compatibility mismatch. The checks performed are:
+   *     - Validate dictionary is enabled.
+   *     - Validate inverted index is enabled.
+   *     - Validate that either no range index exists for column or the range index version is at least 2 and isn't a
+   *       multi-value column (since multi-value defaults to index v1).
+   */
+  protected void validateForwardIndexDisabledConfigsIfPresent(String column, boolean forwardIndexDisabled) {

Review Comment:
   Ok to do in follow-up PR



##########
pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/loader/defaultcolumn/BaseDefaultColumnHandler.java:
##########
@@ -388,8 +416,40 @@ protected boolean createColumnV1Indices(String column)
     return true;
   }
 
+  /**
+   * Validates the compatibility of the indexes if the column has the forward index disabled. Throws exceptions due to
+   * compatibility mismatch. The checks performed are:
+   *     - Validate dictionary is enabled.
+   *     - Validate inverted index is enabled.
+   *     - Validate that either no range index exists for column or the range index version is at least 2 and isn't a
+   *       multi-value column (since multi-value defaults to index v1).
+   */
+  protected void validateForwardIndexDisabledConfigsIfPresent(String column, boolean forwardIndexDisabled) {

Review Comment:
   Ok to do in follow-up PR when reload path is fixed



-- 
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 #9333: Add an option to disable the creation of the forward index for a column

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


##########
pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/creator/impl/SegmentColumnarIndexCreator.java:
##########
@@ -296,6 +309,40 @@ public void init(SegmentGeneratorConfig segmentCreationSpec, SegmentIndexCreatio
     }
   }
 
+  /**
+   * Validates the compatibility of the indexes if the column has the forward index disabled. Throws exceptions due to
+   * compatibility mismatch. The checks performed are:
+   *     - Validate dictionary is enabled.
+   *     - Validate inverted index is enabled.
+   *     - Validate that either no range index exists for column or the range index version is at least 2 and isn't a
+   *       multi-value column (since multi-value defaults to index v1).
+   *
+   * @param columnName Name of the column
+   * @param forwardIndexDisabled Whether the forward index is disabled for column or not
+   * @param dictEnabledColumn Whether the column is dictionary enabled or not
+   * @param columnIndexCreationInfo Column index creation info
+   * @param invertedIndexColumns Set of columns with inverted index enabled
+   * @param rangeIndexColumns Set of columns with range index enabled
+   * @param rangeIndexVersion Range index version
+   * @param fieldSpec FieldSpec of column
+   */
+  private void validateForwardIndexDisabledIndexCompatibility(String columnName, boolean forwardIndexDisabled,
+      boolean dictEnabledColumn, ColumnIndexCreationInfo columnIndexCreationInfo, Set<String> invertedIndexColumns,
+      Set<String> rangeIndexColumns, int rangeIndexVersion, FieldSpec fieldSpec) {
+    if (!forwardIndexDisabled) {
+      return;
+    }
+
+    Preconditions.checkState(dictEnabledColumn,
+        String.format("Cannot disable forward index for column %s without dictionary", columnName));
+    Preconditions.checkState(invertedIndexColumns.contains(columnName),
+        String.format("Cannot disable forward index for column %s without inverted index enabled", columnName));
+    Preconditions.checkState(!rangeIndexColumns.contains(columnName)
+        || (fieldSpec.isSingleValueField() && rangeIndexVersion == BitSlicedRangeIndexCreator.VERSION), String.format(
+            "Cannot disable forward index for column %s which has range index with version < 2 or is multi-value",

Review Comment:
   The range index condition and message is little less readable. 
   
   Suggest breaking it up like done for others
   
   if range index
     - if version < 2 -> feature not supported on column with range index version < 2. either disable range index or use range index version >=2 to use this feature
     - else if MV -> feature not supported on MV column with range index. Disable range index on this column in order to use this feature



-- 
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 #9333: Add an option to disable the creation of the forward index for a column

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


##########
pinot-segment-local/src/main/java/org/apache/pinot/segment/local/utils/TableConfigUtils.java:
##########
@@ -897,6 +903,40 @@ private static void validateFieldConfigList(@Nullable List<FieldConfig> fieldCon
     }
   }
 
+  /**
+   * Validates the compatibility of the indexes if the column has the forward index disabled. Throws exceptions due to
+   * compatibility mismatch. The checks performed are:
+   *     - Validate dictionary is enabled.
+   *     - Validate inverted index is enabled.
+   *     - Validate that either no range index exists for column or the range index version is at least 2 and isn't a
+   *       multi-value column (since mulit-value defaults to index v1).
+   */
+  private static void validateForwardIndexDisabledIndexCompatibility(String columnName, FieldConfig fieldConfig,

Review Comment:
   (nit) we should consider keeping the check either here or during segment generation in `SegmentColumnarIndexCreator`. For now it is fine to safeguard multiple places until feature gets used for a while. 
   
   I am guessing basic validation at the table config level will always be needed since it will come handy even during  enabling / disabling the feature on an existing column. So an unsupported combination will be caught when user attempts to update the table config in controller as opposed to detecting later on the reload path. 



-- 
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 #9333: Add an option to disable the creation of the forward index for a column

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


##########
pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/loader/defaultcolumn/BaseDefaultColumnHandler.java:
##########
@@ -224,6 +225,21 @@ Map<String, DefaultColumnAction> computeDefaultColumnActionMap() {
       if (columnMetadata != null) {
         // Column exists in the segment, check if we need to update the value.
 
+        if (_segmentWriter != null && !columnMetadata.isAutoGenerated()) {
+          // Check that forward index disabled isn't enabled / disabled on an existing column (not auto-generated).
+          // TODO: Add support for reloading segments when forward index disabled flag is enabled or disabled
+          boolean forwardIndexDisabled = !_segmentWriter.hasIndexFor(column, ColumnIndexType.FORWARD_INDEX);
+          if (forwardIndexDisabled != _indexLoadingConfig.getForwardIndexDisabledColumns()
+              .contains(column)) {
+            String failureMessage =
+                "Forward index disabled in segment: " + forwardIndexDisabled + " for column: " + column
+                    + " does not match forward index disabled flag: "
+                    + _indexLoadingConfig.getForwardIndexDisabledColumns().contains(column) + " in the TableConfig, "
+                    + "updating this flag is not supported at the moment.";

Review Comment:
   Not just updating / toggling, even `setting` this flag for the very first time for an existing column is not supported (yet) on the reload path. 
   
   I think we should also log a warning for now and ignore and until the reload support is added, anyone who wants to leverage this feature on an existing column should consider backfilling / refresh



-- 
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 #9333: Add an option to disable the creation of the forward index for a column

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


##########
pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/loader/defaultcolumn/BaseDefaultColumnHandler.java:
##########
@@ -402,6 +462,12 @@ private void createDefaultValueColumnV1Indices(String column)
     boolean isSingleValue = fieldSpec.isSingleValueField();
     int maxNumberOfMultiValueElements = isSingleValue ? 0 : 1;
 
+    // Validate that the forwardIndexDisabled flag, if enabled, is compatible with other indexes and configs
+    // For now the forwardIndexDisabled flag is ignored for default columns but will be handled as part of reload

Review Comment:
   (nit) prefix comment at line 466 with TODO



-- 
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 #9333: Add an option to disable the creation of the forward index for a column

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


##########
pinot-core/src/test/java/org/apache/pinot/queries/ForwardIndexDisabledSingleValueQueriesTest.java:
##########
@@ -0,0 +1,1088 @@
+/**
+ * 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.queries;
+
+import java.io.File;
+import java.net.URL;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.TimeUnit;
+import org.apache.commons.io.FileUtils;
+import org.apache.pinot.common.response.broker.BrokerResponseNative;
+import org.apache.pinot.common.response.broker.ResultTable;
+import org.apache.pinot.common.utils.DataSchema;
+import org.apache.pinot.segment.local.indexsegment.immutable.ImmutableSegmentLoader;
+import org.apache.pinot.segment.local.segment.creator.impl.SegmentIndexCreationDriverImpl;
+import org.apache.pinot.segment.local.segment.index.loader.IndexLoadingConfig;
+import org.apache.pinot.segment.spi.ColumnMetadata;
+import org.apache.pinot.segment.spi.ImmutableSegment;
+import org.apache.pinot.segment.spi.IndexSegment;
+import org.apache.pinot.segment.spi.creator.SegmentGeneratorConfig;
+import org.apache.pinot.segment.spi.creator.SegmentIndexCreationDriver;
+import org.apache.pinot.spi.config.table.FieldConfig;
+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.data.TimeGranularitySpec;
+import org.apache.pinot.spi.utils.ReadMode;
+import org.apache.pinot.spi.utils.builder.TableConfigBuilder;
+import org.testng.Assert;
+import org.testng.annotations.AfterClass;
+import org.testng.annotations.BeforeClass;
+import org.testng.annotations.Test;
+
+import static org.testng.Assert.assertEquals;
+import static org.testng.Assert.assertNotNull;
+import static org.testng.Assert.assertNull;
+import static org.testng.Assert.assertTrue;
+
+
+/**
+ * The <code>ForwardIndexDisabledSingleValueQueriesTest</code> class sets up the index segment for the no forward
+ * index single-value queries test.
+ * <p>There are totally 18 columns, 30000 records inside the original Avro file where 11 columns are selected to build
+ * the index segment. Selected columns information are as following:
+ * <ul>
+ *   ColumnName, FieldType, DataType, Cardinality, IsSorted, HasInvertedIndex, FwdIndexDisabled: S1, S2, HasRangeIndex
+ *   <li>column1, METRIC, INT, 6582, F, F, F, F, F</li>
+ *   <li>column3, METRIC, INT, 21910, F, F, F, F, F</li>
+ *   <li>column5, DIMENSION, STRING, 1, T, F, F, F, F</li>
+ *   <li>column6, DIMENSION, INT, 608, F, T, T, T, T</li>
+ *   <li>column7, DIMENSION, INT, 146, F, T, T, F, F</li>
+ *   <li>column9, DIMENSION, INT, 1737, F, F, F, F, F</li>
+ *   <li>column11, DIMENSION, STRING, 5, F, T, F, F, F</li>
+ *   <li>column12, DIMENSION, STRING, 5, F, F, F, F, F</li>
+ *   <li>column17, METRIC, INT, 24, F, T, F, F, F</li>
+ *   <li>column18, METRIC, INT, 1440, F, T, F, F, F</li>
+ *   <li>daysSinceEpoch, TIME, INT, 2, T, F, F, F, F</li>
+ * </ul>
+ */
+public class ForwardIndexDisabledSingleValueQueriesTest extends BaseQueriesTest {
+  private static final String AVRO_DATA = "data" + File.separator + "test_data-sv.avro";
+  private static final String SEGMENT_NAME_1 = "testTable_126164076_167572857";
+  private static final String SEGMENT_NAME_2 = "testTable_126164076_167572858";
+  private static final File INDEX_DIR = new File(FileUtils.getTempDirectory(),
+      "ForwardIndexDisabledSingleValueQueriesTest");
+
+  private static final String SELECT_STAR_QUERY = "SELECT * FROM testTable";
+
+  // Hard-coded query filter.
+  private static final String FILTER = " WHERE column1 > 100000000"
+      + " AND column3 BETWEEN 20000000 AND 1000000000"
+      + " AND column5 = 'gFuH'"
+      + " AND (column6 < 500000000 OR column11 NOT IN ('t', 'P'))"
+      + " AND daysSinceEpoch = 126164076";
+
+  private IndexSegment _indexSegment;
+  // Contains 2 index segments, one with 2 columns with forward index disabled, and the other with just 1.
+  private List<IndexSegment> _indexSegments;
+
+  private TableConfig _tableConfig;
+  private List<String> _invertedIndexColumns;
+  private List<String> _forwardIndexDisabledColumns;
+
+  @BeforeClass
+  public void buildAndLoadSegment()
+      throws Exception {
+    FileUtils.deleteQuietly(INDEX_DIR);
+
+    // Get resource file path.
+    URL resource = getClass().getClassLoader().getResource(AVRO_DATA);
+    assertNotNull(resource);
+    String filePath = resource.getFile();
+
+    // Build the segment schema.
+    Schema schema = new Schema.SchemaBuilder().setSchemaName("testTable").addMetric("column1", FieldSpec.DataType.INT)
+        .addMetric("column3", FieldSpec.DataType.INT).addSingleValueDimension("column5", FieldSpec.DataType.STRING)
+        .addSingleValueDimension("column6", FieldSpec.DataType.INT)
+        .addSingleValueDimension("column7", FieldSpec.DataType.INT)
+        .addSingleValueDimension("column9", FieldSpec.DataType.INT)
+        .addSingleValueDimension("column11", FieldSpec.DataType.STRING)
+        .addSingleValueDimension("column12", FieldSpec.DataType.STRING).addMetric("column17", FieldSpec.DataType.INT)
+        .addMetric("column18", FieldSpec.DataType.INT)
+        .addTime(new TimeGranularitySpec(FieldSpec.DataType.INT, TimeUnit.DAYS, "daysSinceEpoch"), null).build();
+
+    createSegment(filePath, SEGMENT_NAME_1, schema);
+    createSegment(filePath, SEGMENT_NAME_2, schema);
+
+    ImmutableSegment immutableSegment1 = loadSegmentWithMetadataChecks(SEGMENT_NAME_1);
+    ImmutableSegment immutableSegment2 = loadSegmentWithMetadataChecks(SEGMENT_NAME_2);
+
+    _indexSegment = immutableSegment1;
+    _indexSegments = Arrays.asList(immutableSegment1, immutableSegment2);
+  }
+
+  private void createSegment(String filePath, String segmentName, Schema schema)
+      throws Exception {
+    // Create field configs for the no forward index columns
+    List<FieldConfig> fieldConfigList = new ArrayList<>();
+    fieldConfigList.add(new FieldConfig("column6", FieldConfig.EncodingType.DICTIONARY, Collections.emptyList(), null,
+        Collections.singletonMap(FieldConfig.FORWARD_INDEX_DISABLED, Boolean.TRUE.toString())));
+    if (segmentName.equals(SEGMENT_NAME_1)) {
+      fieldConfigList.add(new FieldConfig("column7", FieldConfig.EncodingType.DICTIONARY, Collections.emptyList(),
+          null, Collections.singletonMap(FieldConfig.FORWARD_INDEX_DISABLED, Boolean.TRUE.toString())));
+
+      // Build table config based on segment 1 as it contains both columns under no forward index
+      _tableConfig =
+          new TableConfigBuilder(TableType.OFFLINE).setTableName("testTable").setTimeColumnName("daysSinceEpoch")
+              .setFieldConfigList(fieldConfigList).setRangeIndexColumns(Arrays.asList("column6")).build();
+    }
+
+    // Create the segment generator config.
+    SegmentGeneratorConfig segmentGeneratorConfig = new SegmentGeneratorConfig(_tableConfig, schema);
+    segmentGeneratorConfig.setInputFilePath(filePath);
+    segmentGeneratorConfig.setTableName("testTable");
+    segmentGeneratorConfig.setOutDir(INDEX_DIR.getAbsolutePath());
+    segmentGeneratorConfig.setSegmentName(segmentName);
+    // The segment generation code in SegmentColumnarIndexCreator will throw
+    // exception if start and end time in time column are not in acceptable
+    // range. For this test, we first need to fix the input avro data
+    // to have the time column values in allowed range. Until then, the check
+    // is explicitly disabled
+    segmentGeneratorConfig.setSkipTimeValueCheck(true);
+    _invertedIndexColumns = Arrays.asList("column6", "column7", "column11", "column17", "column18");
+    segmentGeneratorConfig.setInvertedIndexCreationColumns(_invertedIndexColumns);
+
+    _forwardIndexDisabledColumns = new ArrayList<>(Arrays.asList("column6", "column7"));
+    segmentGeneratorConfig.setForwardIndexDisabledColumns(_forwardIndexDisabledColumns);
+    segmentGeneratorConfig.setRangeIndexCreationColumns(Arrays.asList("column6"));
+
+    // Build the index segment.
+    SegmentIndexCreationDriver driver = new SegmentIndexCreationDriverImpl();
+    driver.init(segmentGeneratorConfig);
+    driver.build();
+  }
+
+  private ImmutableSegment loadSegmentWithMetadataChecks(String segmentName)
+      throws Exception {
+    IndexLoadingConfig indexLoadingConfig = new IndexLoadingConfig();
+    indexLoadingConfig.setTableConfig(_tableConfig);
+    indexLoadingConfig.setInvertedIndexColumns(new HashSet<>(_invertedIndexColumns));
+    indexLoadingConfig.setForwardIndexDisabledColumns(new HashSet<>(_forwardIndexDisabledColumns));
+    indexLoadingConfig.setRangeIndexColumns(new HashSet<>(Arrays.asList("column6")));
+    indexLoadingConfig.setReadMode(ReadMode.heap);
+
+    ImmutableSegment immutableSegment = ImmutableSegmentLoader.load(new File(INDEX_DIR, segmentName),
+        indexLoadingConfig);
+
+    Map<String, ColumnMetadata> columnMetadataMap1 = immutableSegment.getSegmentMetadata().getColumnMetadataMap();
+    columnMetadataMap1.forEach((column, metadata) -> {
+      if (column.equals("column6") || column.equals("column7")) {
+        assertTrue(metadata.hasDictionary());
+        assertTrue(metadata.isSingleValue());
+        assertNull(immutableSegment.getForwardIndex(column));
+      } else {
+        assertNotNull(immutableSegment.getForwardIndex(column));
+      }
+    });
+
+    return immutableSegment;
+  }
+
+  @AfterClass
+  public void deleteAndDestroySegment() {
+    FileUtils.deleteQuietly(INDEX_DIR);
+    _indexSegments.forEach((IndexSegment::destroy));
+  }
+
+  @Override
+  protected String getFilter() {
+    return FILTER;
+  }
+
+  @Override
+  protected IndexSegment getIndexSegment() {
+    return _indexSegment;
+  }
+
+  @Override
+  protected List<IndexSegment> getIndexSegments() {
+    return _indexSegments;
+  }
+
+  @Test
+  public void testSelectStarQueries() {
+    // Select * without any filters
+    try {
+      getBrokerResponse(SELECT_STAR_QUERY);
+      Assert.fail("Select * query should fail since forwardIndexDisabled on a select column");
+    } catch (IllegalStateException e) {
+      assertTrue(e.getMessage().contains("Forward index disabled for column:")
+          && e.getMessage().contains("cannot create DataFetcher!"));
+    }
+
+    // Select * with filters
+    try {
+      getBrokerResponse(SELECT_STAR_QUERY + FILTER);
+      Assert.fail("Select * query should fail since forwardIndexDisabled on a select column");
+    } catch (IllegalStateException e) {
+      assertTrue(e.getMessage().contains("Forward index disabled for column:")
+          && e.getMessage().contains("cannot create DataFetcher!"));
+    }
+  }
+
+  @Test
+  public void testSelectQueries() {
+    {
+      // Selection query without filters including a column with forwardIndexDisabled enabled on both segments
+      String query = "SELECT column1, column5, column6, column9, column11 FROM testTable";
+      try {
+        getBrokerResponse(query);
+        Assert.fail("Query should fail since forwardIndexDisabled on a select column");
+      } catch (IllegalStateException e) {
+        assertTrue(e.getMessage().contains("Forward index disabled for column:")
+            && e.getMessage().contains("cannot create DataFetcher!"));
+      }
+    }
+    {
+      // Selection query without filters including a column with forwardIndexDisabled enabled on one segment
+      String query = "SELECT column1, column5, column7, column9, column11 FROM testTable";
+      try {
+        getBrokerResponse(query);
+        Assert.fail("Query should fail since forwardIndexDisabled on a select column");
+      } catch (IllegalStateException e) {
+        assertTrue(e.getMessage().contains("Forward index disabled for column:")
+            && e.getMessage().contains("cannot create DataFetcher!"));
+      }
+    }
+    {
+      // Selection query with filters including a column with forwardIndexDisabled enabled on both segments
+      String query = "SELECT column1, column5, column6, column9, column11 FROM testTable WHERE column6 = 2147458029";
+      try {
+        getBrokerResponse(query);
+        Assert.fail("Query should fail since forwardIndexDisabled on a select column");
+      } catch (IllegalStateException e) {
+        assertTrue(e.getMessage().contains("Forward index disabled for column:")
+            && e.getMessage().contains("cannot create DataFetcher!"));
+      }
+    }
+    {
+      // Selection query with filters including a column with forwardIndexDisabled enabled on one segment
+      String query = "SELECT column1, column5, column7, column9, column11 FROM testTable WHERE column7 = 675695";
+      try {
+        getBrokerResponse(query);
+        Assert.fail("Query should fail since forwardIndexDisabled on a select column");
+      } catch (IllegalStateException e) {
+        assertTrue(e.getMessage().contains("Forward index disabled for column:")
+            && e.getMessage().contains("cannot create DataFetcher!"));
+      }
+    }
+    {
+      // Selection query without filters and without columns with forwardIndexDisabled enabled on either segment
+      String query = "SELECT column1, column5, column9, column11 FROM testTable ORDER BY column1";
+      BrokerResponseNative brokerResponseNative = getBrokerResponse(query);
+      assertTrue(brokerResponseNative.getProcessingExceptions() == null
+          || brokerResponseNative.getProcessingExceptions().size() == 0);
+      ResultTable resultTable = brokerResponseNative.getResultTable();
+      assertEquals(brokerResponseNative.getNumRowsResultSet(), 10);
+      assertEquals(brokerResponseNative.getTotalDocs(), 120_000L);
+      assertEquals(brokerResponseNative.getNumDocsScanned(), 120_000L);
+      assertEquals(brokerResponseNative.getNumSegmentsProcessed(), 4L);
+      assertEquals(brokerResponseNative.getNumSegmentsMatched(), 4L);
+      assertEquals(brokerResponseNative.getNumEntriesScannedPostFilter(), 120_120L);
+      assertEquals(brokerResponseNative.getNumEntriesScannedInFilter(), 0L);
+      assertNotNull(brokerResponseNative.getProcessingExceptions());
+      assertEquals(brokerResponseNative.getProcessingExceptions().size(), 0);
+      DataSchema dataSchema = new DataSchema(new String[]{"column1", "column5", "column9", "column11"},
+        new DataSchema.ColumnDataType[]{DataSchema.ColumnDataType.INT, DataSchema.ColumnDataType.STRING,
+            DataSchema.ColumnDataType.INT, DataSchema.ColumnDataType.STRING});
+      assertEquals(resultTable.getDataSchema(), dataSchema);
+      List<Object[]> resultRows = resultTable.getRows();
+      int previousColumn1 = Integer.MIN_VALUE;
+      for (Object[] resultRow : resultRows) {
+        assertEquals(resultRow.length, 4);
+        assertEquals((String) resultRow[1], "gFuH");
+        // Column 1
+        assertTrue((Integer) resultRow[0] >= previousColumn1);
+        previousColumn1 = (Integer) resultRow[0];
+      }
+
+      Object[] firstRow = resultRows.get(0);
+      // Column 11
+      assertEquals((String) firstRow[3], "o");
+    }
+    {
+      // Transform function on a selection clause with a forwardIndexDisabled column in transform
+      String query = "SELECT CONCAT(column6, column9, '-') from testTable";

Review Comment:
   I have the following transforms covered in the tests:
   
   ```
   String query = "SELECT column1, column11 from testTable WHERE CONCAT(column6, column9, '-') = '1689277-11270'";
   
   String query = "SELECT column1, column11 from testTable WHERE CONCAT(column5, column9, '-') = 'gFuH-11270' "
             + "ORDER BY column1";
   
   String query = "SELECT CONCAT(column1, column6, '-') FROM testTable GROUP BY CONCAT(column1, column6, '-') "
             + "ORDER BY CONCAT(column1, column6, '-') LIMIT 10";
   
   String query = "SELECT CONCAT(column1, column5, '-') FROM testTable GROUP BY CONCAT(column1, column5, '-') "
             + "ORDER BY CONCAT(column1, column5, '-') LIMIT 10";
   
   String query = "SELECT MAX(ADD(column6, column9)) from testTable LIMIT 10";
   
   String query = "SELECT MAX(ADD(column1, column9)) from testTable LIMIT 10";
   
   String query = "SELECT column1, MAX(ADD(column6, column9)) from testTable GROUP BY column1 LIMIT 10";
   
   String query = "SELECT column1, MAX(ADD(column1, column9)) from testTable GROUP BY column1 LIMIT 10";
   
   String query = "SELECT column1, MAX(ADD(column6, column9)) from testTable GROUP BY column1 ORDER BY column1 "
             + "DESC LIMIT 10";
   
         String query = "SELECT column1, MAX(ADD(column1, column9)) from testTable GROUP BY column1 ORDER BY column1 "
             + "DESC LIMIT 10";
   ```
   
   In addition to the above what other transforms do you think I should add? by nested do you mean TRANSFORM inside a TRANSFORM? 



-- 
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 #9333: Add an option to disable the creation of the forward index for a column

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


##########
pinot-core/src/main/java/org/apache/pinot/core/operator/filter/ScanBasedFilterOperator.java:
##########
@@ -43,6 +44,9 @@ public class ScanBasedFilterOperator extends BaseFilterOperator {
     _dataSource = dataSource;
     _numDocs = numDocs;
     _nullHandlingEnabled = nullHandlingEnabled;
+    Preconditions.checkState(_dataSource.getForwardIndex() != null,
+        "Forward index disabled for column: %s, creating ScanDocIdSet unsupported!",

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 #9333: Add an option to disable the creation of the forward index for a column

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


##########
pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/loader/defaultcolumn/BaseDefaultColumnHandler.java:
##########
@@ -388,8 +416,40 @@ protected boolean createColumnV1Indices(String column)
     return true;
   }
 
+  /**
+   * Validates the compatibility of the indexes if the column has the forward index disabled. Throws exceptions due to
+   * compatibility mismatch. The checks performed are:
+   *     - Validate dictionary is enabled.
+   *     - Validate inverted index is enabled.
+   *     - Validate that either no range index exists for column or the range index version is at least 2 and isn't a
+   *       multi-value column (since multi-value defaults to index v1).
+   */
+  protected void validateForwardIndexDisabledConfigsIfPresent(String column, boolean forwardIndexDisabled) {

Review Comment:
   Got it, thanks for confirming! Do you want me to remove this check from here and segment creation as well in this PR itself?



-- 
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 #9333: Add an option to disable the creation of the forward index for a column

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


##########
pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/readers/forward/NoOpMVForwardIndexReader.java:
##########
@@ -0,0 +1,66 @@
+/**
+ * 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.segment.local.segment.index.readers.forward;
+
+import java.io.IOException;
+import org.apache.pinot.segment.spi.index.reader.ForwardIndexReader;
+import org.apache.pinot.segment.spi.index.reader.ForwardIndexReaderContext;
+import org.apache.pinot.spi.data.FieldSpec;
+
+
+/**
+ * No-op forward index reader meant to be used by multi-value columns with forward index disabled. This is meant to
+ * provide clean exception handling for operations which need to read forward index when forward index is disabled.
+ */
+public class NoOpMVForwardIndexReader implements ForwardIndexReader<ForwardIndexReaderContext> {

Review Comment:
   Unless there is a way to do what you are suggesting without leaking it in several different places in the code, I don't think it is a good idea.



-- 
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 #9333: Add an option to disable the creation of the forward index for a column

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


##########
pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/OfflineClusterIntegrationTest.java:
##########
@@ -143,7 +143,7 @@ public class OfflineClusterIntegrationTest extends BaseClusterIntegrationTestSet
   private static final String MAX_NUM_MULTI_VALUES_MAP_KEY = "maxNumMultiValuesMap";
   // TODO: This might lead to flaky test, as this disk size is not deterministic
   //       as it depends on the iteration order of a HashSet.
-  private static final int DISK_SIZE_IN_BYTES = 20797128;
+  private static final int DISK_SIZE_IN_BYTES = 20843088;

Review Comment:
   so I've added a metadata field in the column metadata which increases the disk size. Also based on the comment here it looks like the size may change depending on the iteration order of the `HashSet`. Let me know if you think I should look into this deeper to compare the difference in size change.
   
   ```
   // TODO: This might lead to flaky test, as this disk size is not deterministic
   //       as it depends on the iteration order of a HashSet.
   ```



-- 
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 #9333: Add an option to disable the creation of the forward index for a column

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


##########
pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/creator/impl/DefaultIndexCreatorProvider.java:
##########
@@ -88,17 +90,25 @@ public ForwardIndexCreator newForwardIndexCreator(IndexCreationContext.Forward c
             context.getMaxRowLengthInBytes());
       }
     } else {
-      if (context.getFieldSpec().isSingleValueField()) {
-        if (context.isSorted()) {
-          return new SingleValueSortedForwardIndexCreator(context.getIndexDir(), context.getFieldSpec().getName(),
-              context.getCardinality());
+      if (context.forwardIndexDisabled()) {

Review Comment:
   done, will do the helper method changes as a follow up to avoid confusion in 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] Jackie-Jiang commented on a diff in pull request #9333: Add an option to disable the creation of the forward index for a column

Posted by GitBox <gi...@apache.org>.
Jackie-Jiang commented on code in PR #9333:
URL: https://github.com/apache/pinot/pull/9333#discussion_r966513551


##########
pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/readers/forward/NoOpMVForwardIndexReader.java:
##########
@@ -0,0 +1,66 @@
+/**
+ * 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.segment.local.segment.index.readers.forward;
+
+import java.io.IOException;
+import org.apache.pinot.segment.spi.index.reader.ForwardIndexReader;
+import org.apache.pinot.segment.spi.index.reader.ForwardIndexReaderContext;
+import org.apache.pinot.spi.data.FieldSpec;
+
+
+/**
+ * No-op forward index reader meant to be used by multi-value columns with forward index disabled. This is meant to
+ * provide clean exception handling for operations which need to read forward index when forward index is disabled.
+ */
+public class NoOpMVForwardIndexReader implements ForwardIndexReader<ForwardIndexReaderContext> {

Review Comment:
   Not really. We have less than 10 usages of `DataSource.getForwardIndex()` in the production code, so it should be easy to track each of them and ensure the no-forward-index case is properly handled. I don't like the idea of handling forward index differently from other index types - all other index types return `null` when the index does not exist.



-- 
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 #9333: Add an option to disable the creation of the forward index for a column

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


##########
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:
   good catch! done



##########
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:
   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 #9333: Add an option to disable the creation of the forward index for a column

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


##########
pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/creator/impl/fwd/MultiValueNoOpForwardIndexCreator.java:
##########
@@ -0,0 +1,53 @@
+/**
+ * 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.segment.local.segment.creator.impl.fwd;
+
+import java.io.IOException;
+import org.apache.pinot.segment.spi.index.creator.ForwardIndexCreator;
+import org.apache.pinot.spi.data.FieldSpec;
+
+
+/**
+ * Forward index creator for dictionary-encoded multi-value column with forward index disabled. This is a no-op.
+ */
+public class MultiValueNoOpForwardIndexCreator implements ForwardIndexCreator {
+  @Override
+  public boolean isDictionaryEncoded() {
+    return true;
+  }
+
+  @Override
+  public boolean isSingleValue() {
+    return false;
+  }
+
+  @Override
+  public FieldSpec.DataType getValueType() {
+    return FieldSpec.DataType.INT;
+  }
+
+  @Override
+  public void putDictIdMV(int[] dictIds) {

Review Comment:
   Nvm.. I now remember our offline discussion and these empty methods are needed to keep the segment generation code clean



##########
pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/creator/impl/fwd/SingleValueNoOpForwardIndexCreator.java:
##########
@@ -0,0 +1,53 @@
+/**
+ * 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.segment.local.segment.creator.impl.fwd;
+
+import java.io.IOException;
+import org.apache.pinot.segment.spi.index.creator.ForwardIndexCreator;
+import org.apache.pinot.spi.data.FieldSpec;
+
+
+/**
+ * Forward index creator for dictionary-encoded single-value column with forward index disabled. This is a no-op.
+ */
+public class SingleValueNoOpForwardIndexCreator implements ForwardIndexCreator {
+  @Override
+  public boolean isDictionaryEncoded() {
+    return true;
+  }
+
+  @Override
+  public boolean isSingleValue() {
+    return true;
+  }
+
+  @Override
+  public FieldSpec.DataType getValueType() {
+    return FieldSpec.DataType.INT;
+  }
+
+  @Override
+  public void putDictId(int dictId) {

Review Comment:
   Ignore 



-- 
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 #9333: Add an option to disable the creation of the forward index for a column

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


##########
pinot-core/src/test/java/org/apache/pinot/queries/ForwardIndexDisabledSingleValueQueriesTest.java:
##########
@@ -0,0 +1,1088 @@
+/**
+ * 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.queries;
+
+import java.io.File;
+import java.net.URL;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.TimeUnit;
+import org.apache.commons.io.FileUtils;
+import org.apache.pinot.common.response.broker.BrokerResponseNative;
+import org.apache.pinot.common.response.broker.ResultTable;
+import org.apache.pinot.common.utils.DataSchema;
+import org.apache.pinot.segment.local.indexsegment.immutable.ImmutableSegmentLoader;
+import org.apache.pinot.segment.local.segment.creator.impl.SegmentIndexCreationDriverImpl;
+import org.apache.pinot.segment.local.segment.index.loader.IndexLoadingConfig;
+import org.apache.pinot.segment.spi.ColumnMetadata;
+import org.apache.pinot.segment.spi.ImmutableSegment;
+import org.apache.pinot.segment.spi.IndexSegment;
+import org.apache.pinot.segment.spi.creator.SegmentGeneratorConfig;
+import org.apache.pinot.segment.spi.creator.SegmentIndexCreationDriver;
+import org.apache.pinot.spi.config.table.FieldConfig;
+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.data.TimeGranularitySpec;
+import org.apache.pinot.spi.utils.ReadMode;
+import org.apache.pinot.spi.utils.builder.TableConfigBuilder;
+import org.testng.Assert;
+import org.testng.annotations.AfterClass;
+import org.testng.annotations.BeforeClass;
+import org.testng.annotations.Test;
+
+import static org.testng.Assert.assertEquals;
+import static org.testng.Assert.assertNotNull;
+import static org.testng.Assert.assertNull;
+import static org.testng.Assert.assertTrue;
+
+
+/**
+ * The <code>ForwardIndexDisabledSingleValueQueriesTest</code> class sets up the index segment for the no forward
+ * index single-value queries test.
+ * <p>There are totally 18 columns, 30000 records inside the original Avro file where 11 columns are selected to build
+ * the index segment. Selected columns information are as following:
+ * <ul>
+ *   ColumnName, FieldType, DataType, Cardinality, IsSorted, HasInvertedIndex, FwdIndexDisabled: S1, S2, HasRangeIndex
+ *   <li>column1, METRIC, INT, 6582, F, F, F, F, F</li>
+ *   <li>column3, METRIC, INT, 21910, F, F, F, F, F</li>
+ *   <li>column5, DIMENSION, STRING, 1, T, F, F, F, F</li>
+ *   <li>column6, DIMENSION, INT, 608, F, T, T, T, T</li>
+ *   <li>column7, DIMENSION, INT, 146, F, T, T, F, F</li>
+ *   <li>column9, DIMENSION, INT, 1737, F, F, F, F, F</li>
+ *   <li>column11, DIMENSION, STRING, 5, F, T, F, F, F</li>
+ *   <li>column12, DIMENSION, STRING, 5, F, F, F, F, F</li>
+ *   <li>column17, METRIC, INT, 24, F, T, F, F, F</li>
+ *   <li>column18, METRIC, INT, 1440, F, T, F, F, F</li>
+ *   <li>daysSinceEpoch, TIME, INT, 2, T, F, F, F, F</li>
+ * </ul>
+ */
+public class ForwardIndexDisabledSingleValueQueriesTest extends BaseQueriesTest {
+  private static final String AVRO_DATA = "data" + File.separator + "test_data-sv.avro";
+  private static final String SEGMENT_NAME_1 = "testTable_126164076_167572857";
+  private static final String SEGMENT_NAME_2 = "testTable_126164076_167572858";
+  private static final File INDEX_DIR = new File(FileUtils.getTempDirectory(),
+      "ForwardIndexDisabledSingleValueQueriesTest");
+
+  private static final String SELECT_STAR_QUERY = "SELECT * FROM testTable";
+
+  // Hard-coded query filter.
+  private static final String FILTER = " WHERE column1 > 100000000"
+      + " AND column3 BETWEEN 20000000 AND 1000000000"
+      + " AND column5 = 'gFuH'"
+      + " AND (column6 < 500000000 OR column11 NOT IN ('t', 'P'))"
+      + " AND daysSinceEpoch = 126164076";
+
+  private IndexSegment _indexSegment;
+  // Contains 2 index segments, one with 2 columns with forward index disabled, and the other with just 1.
+  private List<IndexSegment> _indexSegments;
+
+  private TableConfig _tableConfig;
+  private List<String> _invertedIndexColumns;
+  private List<String> _forwardIndexDisabledColumns;
+
+  @BeforeClass
+  public void buildAndLoadSegment()
+      throws Exception {
+    FileUtils.deleteQuietly(INDEX_DIR);
+
+    // Get resource file path.
+    URL resource = getClass().getClassLoader().getResource(AVRO_DATA);
+    assertNotNull(resource);
+    String filePath = resource.getFile();
+
+    // Build the segment schema.
+    Schema schema = new Schema.SchemaBuilder().setSchemaName("testTable").addMetric("column1", FieldSpec.DataType.INT)
+        .addMetric("column3", FieldSpec.DataType.INT).addSingleValueDimension("column5", FieldSpec.DataType.STRING)
+        .addSingleValueDimension("column6", FieldSpec.DataType.INT)
+        .addSingleValueDimension("column7", FieldSpec.DataType.INT)
+        .addSingleValueDimension("column9", FieldSpec.DataType.INT)
+        .addSingleValueDimension("column11", FieldSpec.DataType.STRING)
+        .addSingleValueDimension("column12", FieldSpec.DataType.STRING).addMetric("column17", FieldSpec.DataType.INT)
+        .addMetric("column18", FieldSpec.DataType.INT)
+        .addTime(new TimeGranularitySpec(FieldSpec.DataType.INT, TimeUnit.DAYS, "daysSinceEpoch"), null).build();
+
+    createSegment(filePath, SEGMENT_NAME_1, schema);
+    createSegment(filePath, SEGMENT_NAME_2, schema);
+
+    ImmutableSegment immutableSegment1 = loadSegmentWithMetadataChecks(SEGMENT_NAME_1);
+    ImmutableSegment immutableSegment2 = loadSegmentWithMetadataChecks(SEGMENT_NAME_2);
+
+    _indexSegment = immutableSegment1;
+    _indexSegments = Arrays.asList(immutableSegment1, immutableSegment2);
+  }
+
+  private void createSegment(String filePath, String segmentName, Schema schema)
+      throws Exception {
+    // Create field configs for the no forward index columns
+    List<FieldConfig> fieldConfigList = new ArrayList<>();
+    fieldConfigList.add(new FieldConfig("column6", FieldConfig.EncodingType.DICTIONARY, Collections.emptyList(), null,
+        Collections.singletonMap(FieldConfig.FORWARD_INDEX_DISABLED, Boolean.TRUE.toString())));
+    if (segmentName.equals(SEGMENT_NAME_1)) {
+      fieldConfigList.add(new FieldConfig("column7", FieldConfig.EncodingType.DICTIONARY, Collections.emptyList(),
+          null, Collections.singletonMap(FieldConfig.FORWARD_INDEX_DISABLED, Boolean.TRUE.toString())));
+
+      // Build table config based on segment 1 as it contains both columns under no forward index
+      _tableConfig =
+          new TableConfigBuilder(TableType.OFFLINE).setTableName("testTable").setTimeColumnName("daysSinceEpoch")
+              .setFieldConfigList(fieldConfigList).setRangeIndexColumns(Arrays.asList("column6")).build();
+    }
+
+    // Create the segment generator config.
+    SegmentGeneratorConfig segmentGeneratorConfig = new SegmentGeneratorConfig(_tableConfig, schema);
+    segmentGeneratorConfig.setInputFilePath(filePath);
+    segmentGeneratorConfig.setTableName("testTable");
+    segmentGeneratorConfig.setOutDir(INDEX_DIR.getAbsolutePath());
+    segmentGeneratorConfig.setSegmentName(segmentName);
+    // The segment generation code in SegmentColumnarIndexCreator will throw
+    // exception if start and end time in time column are not in acceptable
+    // range. For this test, we first need to fix the input avro data
+    // to have the time column values in allowed range. Until then, the check
+    // is explicitly disabled
+    segmentGeneratorConfig.setSkipTimeValueCheck(true);
+    _invertedIndexColumns = Arrays.asList("column6", "column7", "column11", "column17", "column18");
+    segmentGeneratorConfig.setInvertedIndexCreationColumns(_invertedIndexColumns);
+
+    _forwardIndexDisabledColumns = new ArrayList<>(Arrays.asList("column6", "column7"));
+    segmentGeneratorConfig.setForwardIndexDisabledColumns(_forwardIndexDisabledColumns);
+    segmentGeneratorConfig.setRangeIndexCreationColumns(Arrays.asList("column6"));
+
+    // Build the index segment.
+    SegmentIndexCreationDriver driver = new SegmentIndexCreationDriverImpl();
+    driver.init(segmentGeneratorConfig);
+    driver.build();
+  }
+
+  private ImmutableSegment loadSegmentWithMetadataChecks(String segmentName)
+      throws Exception {
+    IndexLoadingConfig indexLoadingConfig = new IndexLoadingConfig();
+    indexLoadingConfig.setTableConfig(_tableConfig);
+    indexLoadingConfig.setInvertedIndexColumns(new HashSet<>(_invertedIndexColumns));
+    indexLoadingConfig.setForwardIndexDisabledColumns(new HashSet<>(_forwardIndexDisabledColumns));
+    indexLoadingConfig.setRangeIndexColumns(new HashSet<>(Arrays.asList("column6")));
+    indexLoadingConfig.setReadMode(ReadMode.heap);
+
+    ImmutableSegment immutableSegment = ImmutableSegmentLoader.load(new File(INDEX_DIR, segmentName),
+        indexLoadingConfig);
+
+    Map<String, ColumnMetadata> columnMetadataMap1 = immutableSegment.getSegmentMetadata().getColumnMetadataMap();
+    columnMetadataMap1.forEach((column, metadata) -> {
+      if (column.equals("column6") || column.equals("column7")) {
+        assertTrue(metadata.hasDictionary());
+        assertTrue(metadata.isSingleValue());
+        assertNull(immutableSegment.getForwardIndex(column));
+      } else {
+        assertNotNull(immutableSegment.getForwardIndex(column));
+      }
+    });
+
+    return immutableSegment;
+  }
+
+  @AfterClass
+  public void deleteAndDestroySegment() {
+    FileUtils.deleteQuietly(INDEX_DIR);
+    _indexSegments.forEach((IndexSegment::destroy));
+  }
+
+  @Override
+  protected String getFilter() {
+    return FILTER;
+  }
+
+  @Override
+  protected IndexSegment getIndexSegment() {
+    return _indexSegment;
+  }
+
+  @Override
+  protected List<IndexSegment> getIndexSegments() {
+    return _indexSegments;
+  }
+
+  @Test
+  public void testSelectStarQueries() {
+    // Select * without any filters
+    try {
+      getBrokerResponse(SELECT_STAR_QUERY);
+      Assert.fail("Select * query should fail since forwardIndexDisabled on a select column");
+    } catch (IllegalStateException e) {
+      assertTrue(e.getMessage().contains("Forward index disabled for column:")
+          && e.getMessage().contains("cannot create DataFetcher!"));
+    }
+
+    // Select * with filters
+    try {
+      getBrokerResponse(SELECT_STAR_QUERY + FILTER);
+      Assert.fail("Select * query should fail since forwardIndexDisabled on a select column");
+    } catch (IllegalStateException e) {
+      assertTrue(e.getMessage().contains("Forward index disabled for column:")
+          && e.getMessage().contains("cannot create DataFetcher!"));
+    }
+  }
+
+  @Test
+  public void testSelectQueries() {
+    {
+      // Selection query without filters including a column with forwardIndexDisabled enabled on both segments
+      String query = "SELECT column1, column5, column6, column9, column11 FROM testTable";
+      try {
+        getBrokerResponse(query);
+        Assert.fail("Query should fail since forwardIndexDisabled on a select column");
+      } catch (IllegalStateException e) {
+        assertTrue(e.getMessage().contains("Forward index disabled for column:")
+            && e.getMessage().contains("cannot create DataFetcher!"));
+      }
+    }
+    {
+      // Selection query without filters including a column with forwardIndexDisabled enabled on one segment
+      String query = "SELECT column1, column5, column7, column9, column11 FROM testTable";
+      try {
+        getBrokerResponse(query);
+        Assert.fail("Query should fail since forwardIndexDisabled on a select column");
+      } catch (IllegalStateException e) {
+        assertTrue(e.getMessage().contains("Forward index disabled for column:")
+            && e.getMessage().contains("cannot create DataFetcher!"));
+      }
+    }
+    {
+      // Selection query with filters including a column with forwardIndexDisabled enabled on both segments
+      String query = "SELECT column1, column5, column6, column9, column11 FROM testTable WHERE column6 = 2147458029";
+      try {
+        getBrokerResponse(query);
+        Assert.fail("Query should fail since forwardIndexDisabled on a select column");
+      } catch (IllegalStateException e) {
+        assertTrue(e.getMessage().contains("Forward index disabled for column:")
+            && e.getMessage().contains("cannot create DataFetcher!"));
+      }
+    }
+    {
+      // Selection query with filters including a column with forwardIndexDisabled enabled on one segment
+      String query = "SELECT column1, column5, column7, column9, column11 FROM testTable WHERE column7 = 675695";

Review Comment:
   Same 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] somandal commented on a diff in pull request #9333: Add an option to disable the creation of the forward index for a column

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


##########
pinot-segment-local/src/main/java/org/apache/pinot/segment/local/utils/TableConfigUtils.java:
##########
@@ -897,6 +903,40 @@ private static void validateFieldConfigList(@Nullable List<FieldConfig> fieldCon
     }
   }
 
+  /**
+   * Validates the compatibility of the indexes if the column has the forward index disabled. Throws exceptions due to
+   * compatibility mismatch. The checks performed are:
+   *     - Validate dictionary is enabled.
+   *     - Validate inverted index is enabled.
+   *     - Validate that either no range index exists for column or the range index version is at least 2 and isn't a
+   *       multi-value column (since mulit-value defaults to index v1).
+   */
+  private static void validateForwardIndexDisabledIndexCompatibility(String columnName, FieldConfig fieldConfig,
+      IndexingConfig indexingConfigs, List<String> noDictionaryColumns, Schema schema) {
+    Map<String, String> fieldConfigProperties = fieldConfig.getProperties();
+    if (fieldConfigProperties == null) {
+      return;
+    }
+
+    boolean forwardIndexDisabled = Boolean.parseBoolean(fieldConfigProperties.get(FieldConfig.FORWARD_INDEX_DISABLED));
+    if (!forwardIndexDisabled) {
+      return;
+    }
+
+    FieldSpec fieldSpec = schema.getFieldSpecFor(columnName);
+    Preconditions.checkState(noDictionaryColumns == null || !noDictionaryColumns.contains(columnName),
+        String.format("Forward index disabled column %s must have dictionary enabled", columnName));
+    Preconditions.checkState(indexingConfigs.getInvertedIndexColumns() != null
+            && indexingConfigs.getInvertedIndexColumns().contains(columnName),
+        String.format("Forward index disabled column %s must have inverted index enabled", columnName));
+    Preconditions.checkState(indexingConfigs.getRangeIndexColumns() == null

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 #9333: Add an option to disable the creation of the forward index for a column

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


##########
pinot-segment-local/src/main/java/org/apache/pinot/segment/local/utils/TableConfigUtils.java:
##########
@@ -897,6 +903,40 @@ private static void validateFieldConfigList(@Nullable List<FieldConfig> fieldCon
     }
   }
 
+  /**
+   * Validates the compatibility of the indexes if the column has the forward index disabled. Throws exceptions due to
+   * compatibility mismatch. The checks performed are:
+   *     - Validate dictionary is enabled.
+   *     - Validate inverted index is enabled.
+   *     - Validate that either no range index exists for column or the range index version is at least 2 and isn't a
+   *       multi-value column (since mulit-value defaults to index v1).
+   */
+  private static void validateForwardIndexDisabledIndexCompatibility(String columnName, FieldConfig fieldConfig,
+      IndexingConfig indexingConfigs, List<String> noDictionaryColumns, Schema schema) {
+    Map<String, String> fieldConfigProperties = fieldConfig.getProperties();
+    if (fieldConfigProperties == null) {
+      return;
+    }
+
+    boolean forwardIndexDisabled = Boolean.parseBoolean(fieldConfigProperties.get(FieldConfig.FORWARD_INDEX_DISABLED));
+    if (!forwardIndexDisabled) {
+      return;
+    }
+
+    FieldSpec fieldSpec = schema.getFieldSpecFor(columnName);
+    Preconditions.checkState(noDictionaryColumns == null || !noDictionaryColumns.contains(columnName),

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 #9333: Add an option to disable the creation of the forward index for a column

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


##########
pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/loader/defaultcolumn/BaseDefaultColumnHandler.java:
##########
@@ -388,8 +416,40 @@ protected boolean createColumnV1Indices(String column)
     return true;
   }
 
+  /**
+   * Validates the compatibility of the indexes if the column has the forward index disabled. Throws exceptions due to
+   * compatibility mismatch. The checks performed are:
+   *     - Validate dictionary is enabled.
+   *     - Validate inverted index is enabled.
+   *     - Validate that either no range index exists for column or the range index version is at least 2 and isn't a
+   *       multi-value column (since multi-value defaults to index v1).
+   */
+  protected void validateForwardIndexDisabledConfigsIfPresent(String column, boolean forwardIndexDisabled) {

Review Comment:
   Why is this function needed here ? 
   
   Table config validator should have already caught the unsupported combinations / mandatory conditions since anyone who wants to toggle the flag and then leverage reload path has to first update the table config successfully on the controller (which should internally use validator)



-- 
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 #9333: Add an option to disable the creation of the forward index for a column

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


##########
pinot-core/src/test/java/org/apache/pinot/queries/ForwardIndexDisabledSingleValueQueriesTest.java:
##########
@@ -0,0 +1,1088 @@
+/**
+ * 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.queries;
+
+import java.io.File;
+import java.net.URL;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.TimeUnit;
+import org.apache.commons.io.FileUtils;
+import org.apache.pinot.common.response.broker.BrokerResponseNative;
+import org.apache.pinot.common.response.broker.ResultTable;
+import org.apache.pinot.common.utils.DataSchema;
+import org.apache.pinot.segment.local.indexsegment.immutable.ImmutableSegmentLoader;
+import org.apache.pinot.segment.local.segment.creator.impl.SegmentIndexCreationDriverImpl;
+import org.apache.pinot.segment.local.segment.index.loader.IndexLoadingConfig;
+import org.apache.pinot.segment.spi.ColumnMetadata;
+import org.apache.pinot.segment.spi.ImmutableSegment;
+import org.apache.pinot.segment.spi.IndexSegment;
+import org.apache.pinot.segment.spi.creator.SegmentGeneratorConfig;
+import org.apache.pinot.segment.spi.creator.SegmentIndexCreationDriver;
+import org.apache.pinot.spi.config.table.FieldConfig;
+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.data.TimeGranularitySpec;
+import org.apache.pinot.spi.utils.ReadMode;
+import org.apache.pinot.spi.utils.builder.TableConfigBuilder;
+import org.testng.Assert;
+import org.testng.annotations.AfterClass;
+import org.testng.annotations.BeforeClass;
+import org.testng.annotations.Test;
+
+import static org.testng.Assert.assertEquals;
+import static org.testng.Assert.assertNotNull;
+import static org.testng.Assert.assertNull;
+import static org.testng.Assert.assertTrue;
+
+
+/**
+ * The <code>ForwardIndexDisabledSingleValueQueriesTest</code> class sets up the index segment for the no forward
+ * index single-value queries test.
+ * <p>There are totally 18 columns, 30000 records inside the original Avro file where 11 columns are selected to build
+ * the index segment. Selected columns information are as following:
+ * <ul>
+ *   ColumnName, FieldType, DataType, Cardinality, IsSorted, HasInvertedIndex, FwdIndexDisabled: S1, S2, HasRangeIndex
+ *   <li>column1, METRIC, INT, 6582, F, F, F, F, F</li>
+ *   <li>column3, METRIC, INT, 21910, F, F, F, F, F</li>
+ *   <li>column5, DIMENSION, STRING, 1, T, F, F, F, F</li>
+ *   <li>column6, DIMENSION, INT, 608, F, T, T, T, T</li>
+ *   <li>column7, DIMENSION, INT, 146, F, T, T, F, F</li>
+ *   <li>column9, DIMENSION, INT, 1737, F, F, F, F, F</li>
+ *   <li>column11, DIMENSION, STRING, 5, F, T, F, F, F</li>
+ *   <li>column12, DIMENSION, STRING, 5, F, F, F, F, F</li>
+ *   <li>column17, METRIC, INT, 24, F, T, F, F, F</li>
+ *   <li>column18, METRIC, INT, 1440, F, T, F, F, F</li>
+ *   <li>daysSinceEpoch, TIME, INT, 2, T, F, F, F, F</li>
+ * </ul>
+ */
+public class ForwardIndexDisabledSingleValueQueriesTest extends BaseQueriesTest {
+  private static final String AVRO_DATA = "data" + File.separator + "test_data-sv.avro";
+  private static final String SEGMENT_NAME_1 = "testTable_126164076_167572857";
+  private static final String SEGMENT_NAME_2 = "testTable_126164076_167572858";
+  private static final File INDEX_DIR = new File(FileUtils.getTempDirectory(),
+      "ForwardIndexDisabledSingleValueQueriesTest");
+
+  private static final String SELECT_STAR_QUERY = "SELECT * FROM testTable";
+
+  // Hard-coded query filter.
+  private static final String FILTER = " WHERE column1 > 100000000"
+      + " AND column3 BETWEEN 20000000 AND 1000000000"
+      + " AND column5 = 'gFuH'"
+      + " AND (column6 < 500000000 OR column11 NOT IN ('t', 'P'))"
+      + " AND daysSinceEpoch = 126164076";
+
+  private IndexSegment _indexSegment;
+  // Contains 2 index segments, one with 2 columns with forward index disabled, and the other with just 1.
+  private List<IndexSegment> _indexSegments;
+
+  private TableConfig _tableConfig;
+  private List<String> _invertedIndexColumns;
+  private List<String> _forwardIndexDisabledColumns;
+
+  @BeforeClass
+  public void buildAndLoadSegment()
+      throws Exception {
+    FileUtils.deleteQuietly(INDEX_DIR);
+
+    // Get resource file path.
+    URL resource = getClass().getClassLoader().getResource(AVRO_DATA);
+    assertNotNull(resource);
+    String filePath = resource.getFile();
+
+    // Build the segment schema.
+    Schema schema = new Schema.SchemaBuilder().setSchemaName("testTable").addMetric("column1", FieldSpec.DataType.INT)
+        .addMetric("column3", FieldSpec.DataType.INT).addSingleValueDimension("column5", FieldSpec.DataType.STRING)
+        .addSingleValueDimension("column6", FieldSpec.DataType.INT)
+        .addSingleValueDimension("column7", FieldSpec.DataType.INT)
+        .addSingleValueDimension("column9", FieldSpec.DataType.INT)
+        .addSingleValueDimension("column11", FieldSpec.DataType.STRING)
+        .addSingleValueDimension("column12", FieldSpec.DataType.STRING).addMetric("column17", FieldSpec.DataType.INT)
+        .addMetric("column18", FieldSpec.DataType.INT)
+        .addTime(new TimeGranularitySpec(FieldSpec.DataType.INT, TimeUnit.DAYS, "daysSinceEpoch"), null).build();
+
+    createSegment(filePath, SEGMENT_NAME_1, schema);
+    createSegment(filePath, SEGMENT_NAME_2, schema);
+
+    ImmutableSegment immutableSegment1 = loadSegmentWithMetadataChecks(SEGMENT_NAME_1);
+    ImmutableSegment immutableSegment2 = loadSegmentWithMetadataChecks(SEGMENT_NAME_2);
+
+    _indexSegment = immutableSegment1;
+    _indexSegments = Arrays.asList(immutableSegment1, immutableSegment2);
+  }
+
+  private void createSegment(String filePath, String segmentName, Schema schema)
+      throws Exception {
+    // Create field configs for the no forward index columns
+    List<FieldConfig> fieldConfigList = new ArrayList<>();
+    fieldConfigList.add(new FieldConfig("column6", FieldConfig.EncodingType.DICTIONARY, Collections.emptyList(), null,
+        Collections.singletonMap(FieldConfig.FORWARD_INDEX_DISABLED, Boolean.TRUE.toString())));
+    if (segmentName.equals(SEGMENT_NAME_1)) {
+      fieldConfigList.add(new FieldConfig("column7", FieldConfig.EncodingType.DICTIONARY, Collections.emptyList(),
+          null, Collections.singletonMap(FieldConfig.FORWARD_INDEX_DISABLED, Boolean.TRUE.toString())));
+
+      // Build table config based on segment 1 as it contains both columns under no forward index
+      _tableConfig =
+          new TableConfigBuilder(TableType.OFFLINE).setTableName("testTable").setTimeColumnName("daysSinceEpoch")
+              .setFieldConfigList(fieldConfigList).setRangeIndexColumns(Arrays.asList("column6")).build();
+    }
+
+    // Create the segment generator config.
+    SegmentGeneratorConfig segmentGeneratorConfig = new SegmentGeneratorConfig(_tableConfig, schema);
+    segmentGeneratorConfig.setInputFilePath(filePath);
+    segmentGeneratorConfig.setTableName("testTable");
+    segmentGeneratorConfig.setOutDir(INDEX_DIR.getAbsolutePath());
+    segmentGeneratorConfig.setSegmentName(segmentName);
+    // The segment generation code in SegmentColumnarIndexCreator will throw
+    // exception if start and end time in time column are not in acceptable
+    // range. For this test, we first need to fix the input avro data
+    // to have the time column values in allowed range. Until then, the check
+    // is explicitly disabled
+    segmentGeneratorConfig.setSkipTimeValueCheck(true);
+    _invertedIndexColumns = Arrays.asList("column6", "column7", "column11", "column17", "column18");
+    segmentGeneratorConfig.setInvertedIndexCreationColumns(_invertedIndexColumns);
+
+    _forwardIndexDisabledColumns = new ArrayList<>(Arrays.asList("column6", "column7"));
+    segmentGeneratorConfig.setForwardIndexDisabledColumns(_forwardIndexDisabledColumns);
+    segmentGeneratorConfig.setRangeIndexCreationColumns(Arrays.asList("column6"));
+
+    // Build the index segment.
+    SegmentIndexCreationDriver driver = new SegmentIndexCreationDriverImpl();
+    driver.init(segmentGeneratorConfig);
+    driver.build();
+  }
+
+  private ImmutableSegment loadSegmentWithMetadataChecks(String segmentName)
+      throws Exception {
+    IndexLoadingConfig indexLoadingConfig = new IndexLoadingConfig();
+    indexLoadingConfig.setTableConfig(_tableConfig);
+    indexLoadingConfig.setInvertedIndexColumns(new HashSet<>(_invertedIndexColumns));
+    indexLoadingConfig.setForwardIndexDisabledColumns(new HashSet<>(_forwardIndexDisabledColumns));
+    indexLoadingConfig.setRangeIndexColumns(new HashSet<>(Arrays.asList("column6")));
+    indexLoadingConfig.setReadMode(ReadMode.heap);
+
+    ImmutableSegment immutableSegment = ImmutableSegmentLoader.load(new File(INDEX_DIR, segmentName),
+        indexLoadingConfig);
+
+    Map<String, ColumnMetadata> columnMetadataMap1 = immutableSegment.getSegmentMetadata().getColumnMetadataMap();
+    columnMetadataMap1.forEach((column, metadata) -> {
+      if (column.equals("column6") || column.equals("column7")) {
+        assertTrue(metadata.hasDictionary());
+        assertTrue(metadata.isSingleValue());
+        assertNull(immutableSegment.getForwardIndex(column));
+      } else {
+        assertNotNull(immutableSegment.getForwardIndex(column));
+      }
+    });
+
+    return immutableSegment;
+  }
+
+  @AfterClass
+  public void deleteAndDestroySegment() {
+    FileUtils.deleteQuietly(INDEX_DIR);
+    _indexSegments.forEach((IndexSegment::destroy));
+  }
+
+  @Override
+  protected String getFilter() {
+    return FILTER;
+  }
+
+  @Override
+  protected IndexSegment getIndexSegment() {
+    return _indexSegment;
+  }
+
+  @Override
+  protected List<IndexSegment> getIndexSegments() {
+    return _indexSegments;
+  }
+
+  @Test
+  public void testSelectStarQueries() {
+    // Select * without any filters
+    try {
+      getBrokerResponse(SELECT_STAR_QUERY);
+      Assert.fail("Select * query should fail since forwardIndexDisabled on a select column");
+    } catch (IllegalStateException e) {
+      assertTrue(e.getMessage().contains("Forward index disabled for column:")
+          && e.getMessage().contains("cannot create DataFetcher!"));
+    }
+
+    // Select * with filters
+    try {
+      getBrokerResponse(SELECT_STAR_QUERY + FILTER);
+      Assert.fail("Select * query should fail since forwardIndexDisabled on a select column");
+    } catch (IllegalStateException e) {
+      assertTrue(e.getMessage().contains("Forward index disabled for column:")
+          && e.getMessage().contains("cannot create DataFetcher!"));
+    }
+  }
+
+  @Test
+  public void testSelectQueries() {
+    {
+      // Selection query without filters including a column with forwardIndexDisabled enabled on both segments
+      String query = "SELECT column1, column5, column6, column9, column11 FROM testTable";
+      try {
+        getBrokerResponse(query);
+        Assert.fail("Query should fail since forwardIndexDisabled on a select column");
+      } catch (IllegalStateException e) {
+        assertTrue(e.getMessage().contains("Forward index disabled for column:")
+            && e.getMessage().contains("cannot create DataFetcher!"));
+      }
+    }
+    {
+      // Selection query without filters including a column with forwardIndexDisabled enabled on one segment
+      String query = "SELECT column1, column5, column7, column9, column11 FROM testTable";
+      try {
+        getBrokerResponse(query);
+        Assert.fail("Query should fail since forwardIndexDisabled on a select column");
+      } catch (IllegalStateException e) {
+        assertTrue(e.getMessage().contains("Forward index disabled for column:")
+            && e.getMessage().contains("cannot create DataFetcher!"));
+      }
+    }
+    {
+      // Selection query with filters including a column with forwardIndexDisabled enabled on both segments
+      String query = "SELECT column1, column5, column6, column9, column11 FROM testTable WHERE column6 = 2147458029";
+      try {
+        getBrokerResponse(query);
+        Assert.fail("Query should fail since forwardIndexDisabled on a select column");
+      } catch (IllegalStateException e) {
+        assertTrue(e.getMessage().contains("Forward index disabled for column:")
+            && e.getMessage().contains("cannot create DataFetcher!"));
+      }
+    }
+    {
+      // Selection query with filters including a column with forwardIndexDisabled enabled on one segment
+      String query = "SELECT column1, column5, column7, column9, column11 FROM testTable WHERE column7 = 675695";
+      try {
+        getBrokerResponse(query);
+        Assert.fail("Query should fail since forwardIndexDisabled on a select column");
+      } catch (IllegalStateException e) {
+        assertTrue(e.getMessage().contains("Forward index disabled for column:")
+            && e.getMessage().contains("cannot create DataFetcher!"));
+      }
+    }
+    {
+      // Selection query without filters and without columns with forwardIndexDisabled enabled on either segment
+      String query = "SELECT column1, column5, column9, column11 FROM testTable ORDER BY column1";
+      BrokerResponseNative brokerResponseNative = getBrokerResponse(query);
+      assertTrue(brokerResponseNative.getProcessingExceptions() == null
+          || brokerResponseNative.getProcessingExceptions().size() == 0);
+      ResultTable resultTable = brokerResponseNative.getResultTable();
+      assertEquals(brokerResponseNative.getNumRowsResultSet(), 10);
+      assertEquals(brokerResponseNative.getTotalDocs(), 120_000L);
+      assertEquals(brokerResponseNative.getNumDocsScanned(), 120_000L);
+      assertEquals(brokerResponseNative.getNumSegmentsProcessed(), 4L);
+      assertEquals(brokerResponseNative.getNumSegmentsMatched(), 4L);
+      assertEquals(brokerResponseNative.getNumEntriesScannedPostFilter(), 120_120L);
+      assertEquals(brokerResponseNative.getNumEntriesScannedInFilter(), 0L);
+      assertNotNull(brokerResponseNative.getProcessingExceptions());
+      assertEquals(brokerResponseNative.getProcessingExceptions().size(), 0);
+      DataSchema dataSchema = new DataSchema(new String[]{"column1", "column5", "column9", "column11"},
+        new DataSchema.ColumnDataType[]{DataSchema.ColumnDataType.INT, DataSchema.ColumnDataType.STRING,
+            DataSchema.ColumnDataType.INT, DataSchema.ColumnDataType.STRING});
+      assertEquals(resultTable.getDataSchema(), dataSchema);
+      List<Object[]> resultRows = resultTable.getRows();
+      int previousColumn1 = Integer.MIN_VALUE;
+      for (Object[] resultRow : resultRows) {
+        assertEquals(resultRow.length, 4);
+        assertEquals((String) resultRow[1], "gFuH");
+        // Column 1
+        assertTrue((Integer) resultRow[0] >= previousColumn1);
+        previousColumn1 = (Integer) resultRow[0];
+      }
+
+      Object[] firstRow = resultRows.get(0);
+      // Column 11
+      assertEquals((String) firstRow[3], "o");
+    }
+    {
+      // Transform function on a selection clause with a forwardIndexDisabled column in transform
+      String query = "SELECT CONCAT(column6, column9, '-') from testTable";
+      try {
+        getBrokerResponse(query);
+        Assert.fail("Query should fail since forwardIndexDisabled on a column in transform");
+      } catch (IllegalStateException e) {
+        assertTrue(e.getMessage().contains("Forward index disabled for column:")
+            && e.getMessage().contains("cannot create DataFetcher!"));
+      }
+    }
+    {
+      // Transform function on a selection clause without a forwardIndexDisabled column in transform
+      String query = "SELECT CONCAT(column5, column9, '-') from testTable ORDER BY column1";
+      BrokerResponseNative brokerResponseNative = getBrokerResponse(query);
+      assertTrue(brokerResponseNative.getProcessingExceptions() == null
+          || brokerResponseNative.getProcessingExceptions().size() == 0);
+      ResultTable resultTable = brokerResponseNative.getResultTable();
+      assertEquals(brokerResponseNative.getNumRowsResultSet(), 10);
+      assertEquals(brokerResponseNative.getTotalDocs(), 120_000L);
+      assertEquals(brokerResponseNative.getNumDocsScanned(), 120_000L);
+      assertEquals(brokerResponseNative.getNumSegmentsProcessed(), 4L);
+      assertEquals(brokerResponseNative.getNumSegmentsMatched(), 4L);
+      assertEquals(brokerResponseNative.getNumEntriesScannedPostFilter(), 120_080L);
+      assertEquals(brokerResponseNative.getNumEntriesScannedInFilter(), 0L);
+      assertNotNull(brokerResponseNative.getProcessingExceptions());
+      assertEquals(brokerResponseNative.getProcessingExceptions().size(), 0);
+      DataSchema dataSchema = new DataSchema(new String[]{"concat(column5,column9,'-')"},
+          new DataSchema.ColumnDataType[]{DataSchema.ColumnDataType.STRING});
+      assertEquals(resultTable.getDataSchema(), dataSchema);
+      List<Object[]> resultRows = resultTable.getRows();
+      for (Object[] resultRow : resultRows) {
+        assertEquals(resultRow.length, 1);
+        assertTrue(resultRow[0].toString().startsWith("gFuH-"));
+      }
+    }
+    {
+      // Selection query with filters (not including forwardIndexDisabled column) and without columns with
+      // forwardIndexDisabled enabled on either segment
+      String query = "SELECT column1, column5, column9, column11 FROM testTable WHERE column1 > 100000000"
+        + " AND column3 BETWEEN 20000000 AND 1000000000"
+        + " AND column5 = 'gFuH'"
+        + " AND daysSinceEpoch = 126164076 ORDER BY column1";
+      BrokerResponseNative brokerResponseNative = getBrokerResponse(query);
+      assertTrue(brokerResponseNative.getProcessingExceptions() == null
+          || brokerResponseNative.getProcessingExceptions().size() == 0);
+      ResultTable resultTable = brokerResponseNative.getResultTable();
+      assertEquals(brokerResponseNative.getNumRowsResultSet(), 10);
+      assertEquals(brokerResponseNative.getTotalDocs(), 120_000L);
+      assertEquals(brokerResponseNative.getNumDocsScanned(), 42_368L);
+      assertEquals(brokerResponseNative.getNumSegmentsProcessed(), 4L);
+      assertEquals(brokerResponseNative.getNumSegmentsMatched(), 4L);
+      assertEquals(brokerResponseNative.getNumEntriesScannedPostFilter(), 42_488);
+      assertEquals(brokerResponseNative.getNumEntriesScannedInFilter(), 192744L);
+      assertNotNull(brokerResponseNative.getProcessingExceptions());
+      assertEquals(brokerResponseNative.getProcessingExceptions().size(), 0);
+      DataSchema dataSchema = new DataSchema(new String[]{"column1", "column5", "column9", "column11"},
+          new DataSchema.ColumnDataType[]{DataSchema.ColumnDataType.INT, DataSchema.ColumnDataType.STRING,
+              DataSchema.ColumnDataType.INT, DataSchema.ColumnDataType.STRING});
+      assertEquals(resultTable.getDataSchema(), dataSchema);
+      List<Object[]> resultRows = resultTable.getRows();
+      int previousColumn1 = Integer.MIN_VALUE;
+      for (Object[] resultRow : resultRows) {
+        assertEquals(resultRow.length, 4);
+        assertEquals((String) resultRow[1], "gFuH");
+        assertTrue((Integer) resultRow[0] >= previousColumn1);
+        previousColumn1 = (Integer) resultRow[0];
+      }
+
+      Object[] firstRow = resultRows.get(0);
+      // Column 11
+      assertEquals((String) firstRow[3], "P");
+    }
+    {
+      // Selection query with supported filters (including forwardIndexDisabled column) and without columns with
+      // forwardIndexDisabled enabled on either segment
+      String query = "SELECT column1, column5, column9, column11 FROM testTable WHERE column6 = 2147458029 "
+          + "ORDER BY column1";
+      BrokerResponseNative brokerResponseNative = getBrokerResponse(query);
+      assertTrue(brokerResponseNative.getProcessingExceptions() == null
+          || brokerResponseNative.getProcessingExceptions().size() == 0);
+      ResultTable resultTable = brokerResponseNative.getResultTable();
+      assertEquals(brokerResponseNative.getNumRowsResultSet(), 10);
+      assertEquals(brokerResponseNative.getTotalDocs(), 120_000L);
+      assertEquals(brokerResponseNative.getNumDocsScanned(), 16L);
+      assertEquals(brokerResponseNative.getNumSegmentsProcessed(), 4L);
+      assertEquals(brokerResponseNative.getNumSegmentsMatched(), 4L);
+      assertEquals(brokerResponseNative.getNumEntriesScannedPostFilter(), 64L);
+      assertEquals(brokerResponseNative.getNumEntriesScannedInFilter(), 0L);
+      assertNotNull(brokerResponseNative.getProcessingExceptions());
+      assertEquals(brokerResponseNative.getProcessingExceptions().size(), 0);
+      DataSchema dataSchema = new DataSchema(new String[]{"column1", "column5", "column9", "column11"},
+          new DataSchema.ColumnDataType[]{DataSchema.ColumnDataType.INT, DataSchema.ColumnDataType.STRING,
+              DataSchema.ColumnDataType.INT, DataSchema.ColumnDataType.STRING});
+      assertEquals(resultTable.getDataSchema(), dataSchema);
+      List<Object[]> resultRows = resultTable.getRows();
+      for (Object[] resultRow : resultRows) {
+        assertEquals(resultRow.length, 4);
+        assertEquals(resultRow[0], 635553468);
+        assertEquals((String) resultRow[1], "gFuH");
+        assertEquals(resultRow[2], 705242697);
+        assertEquals((String) resultRow[3], "P");
+      }
+    }
+    {
+      // Selection query with supported filters (including forwardIndexDisabled column) and without columns with
+      // forwardIndexDisabled enabled on either segment
+      String query = "SELECT column1, column5, column9, column11 FROM testTable WHERE column7 != 675695 "
+          + "ORDER BY column1";
+      BrokerResponseNative brokerResponseNative = getBrokerResponse(query);
+      assertTrue(brokerResponseNative.getProcessingExceptions() == null
+          || brokerResponseNative.getProcessingExceptions().size() == 0);
+      ResultTable resultTable = brokerResponseNative.getResultTable();
+      assertEquals(brokerResponseNative.getNumRowsResultSet(), 10);
+      assertEquals(brokerResponseNative.getTotalDocs(), 120_000L);
+      assertEquals(brokerResponseNative.getNumDocsScanned(), 119_908L);
+      assertEquals(brokerResponseNative.getNumSegmentsProcessed(), 4L);
+      assertEquals(brokerResponseNative.getNumSegmentsMatched(), 4L);
+      assertEquals(brokerResponseNative.getNumEntriesScannedPostFilter(), 120_028L);
+      assertEquals(brokerResponseNative.getNumEntriesScannedInFilter(), 0L);
+      assertNotNull(brokerResponseNative.getProcessingExceptions());
+      assertEquals(brokerResponseNative.getProcessingExceptions().size(), 0);
+      DataSchema dataSchema = new DataSchema(new String[]{"column1", "column5", "column9", "column11"},
+          new DataSchema.ColumnDataType[]{DataSchema.ColumnDataType.INT, DataSchema.ColumnDataType.STRING,
+              DataSchema.ColumnDataType.INT, DataSchema.ColumnDataType.STRING});
+      assertEquals(resultTable.getDataSchema(), dataSchema);
+      List<Object[]> resultRows = resultTable.getRows();
+      int previousColumn1 = Integer.MIN_VALUE;
+      for (Object[] resultRow : resultRows) {
+        assertEquals(resultRow.length, 4);
+        assertEquals((String) resultRow[1], "gFuH");
+        assertTrue((Integer) resultRow[0] >= previousColumn1);
+        previousColumn1 = (Integer) resultRow[0];
+      }
+
+      Object[] firstRow = resultRows.get(0);
+      // Column 11
+      assertEquals((String) firstRow[3], "o");
+    }
+    {
+      // Selection query with supported filters (including forwardIndexDisabled column) and without columns with
+      // forwardIndexDisabled enabled on either segment
+      String query = "SELECT column1, column5, column9, column11 FROM testTable WHERE column7 IN (675695, 2137685743) "
+          + "ORDER BY column1";
+      BrokerResponseNative brokerResponseNative = getBrokerResponse(query);
+      assertTrue(brokerResponseNative.getProcessingExceptions() == null
+          || brokerResponseNative.getProcessingExceptions().size() == 0);
+      ResultTable resultTable = brokerResponseNative.getResultTable();
+      assertEquals(brokerResponseNative.getNumRowsResultSet(), 10);
+      assertEquals(brokerResponseNative.getTotalDocs(), 120_000L);
+      assertEquals(brokerResponseNative.getNumDocsScanned(), 828L);
+      assertEquals(brokerResponseNative.getNumSegmentsProcessed(), 4L);
+      assertEquals(brokerResponseNative.getNumSegmentsMatched(), 4L);
+      assertEquals(brokerResponseNative.getNumEntriesScannedPostFilter(), 948L);
+      assertEquals(brokerResponseNative.getNumEntriesScannedInFilter(), 0L);
+      assertNotNull(brokerResponseNative.getProcessingExceptions());
+      assertEquals(brokerResponseNative.getProcessingExceptions().size(), 0);
+      DataSchema dataSchema = new DataSchema(new String[]{"column1", "column5", "column9", "column11"},
+          new DataSchema.ColumnDataType[]{DataSchema.ColumnDataType.INT, DataSchema.ColumnDataType.STRING,
+              DataSchema.ColumnDataType.INT, DataSchema.ColumnDataType.STRING});
+      assertEquals(resultTable.getDataSchema(), dataSchema);
+      List<Object[]> resultRows = resultTable.getRows();
+      int previousColumn1 = Integer.MIN_VALUE;
+      for (Object[] resultRow : resultRows) {
+        assertEquals(resultRow.length, 4);
+        assertEquals((String) resultRow[1], "gFuH");
+        assertTrue((Integer) resultRow[0] >= previousColumn1);
+        previousColumn1 = (Integer) resultRow[0];
+      }
+
+      Object[] firstRow = resultRows.get(0);
+      // Column 11
+      assertEquals((String) firstRow[3], "P");
+    }
+    {
+      // Selection query with supported filters (including forwardIndexDisabled column) and without columns with
+      // forwardIndexDisabled enabled on either segment
+      String query = "SELECT column1, column5, column9, column11 FROM testTable WHERE column6 NOT IN "
+          + "(1689277, 2147458029) ORDER BY column1";
+      BrokerResponseNative brokerResponseNative = getBrokerResponse(query);
+      assertTrue(brokerResponseNative.getProcessingExceptions() == null
+          || brokerResponseNative.getProcessingExceptions().size() == 0);
+      ResultTable resultTable = brokerResponseNative.getResultTable();
+      assertEquals(brokerResponseNative.getNumRowsResultSet(), 10);
+      assertEquals(brokerResponseNative.getTotalDocs(), 120_000L);
+      assertEquals(brokerResponseNative.getNumDocsScanned(), 119_980L);
+      assertEquals(brokerResponseNative.getNumSegmentsProcessed(), 4L);
+      assertEquals(brokerResponseNative.getNumSegmentsMatched(), 4L);
+      assertEquals(brokerResponseNative.getNumEntriesScannedPostFilter(), 120_100L);
+      assertEquals(brokerResponseNative.getNumEntriesScannedInFilter(), 0L);
+      assertNotNull(brokerResponseNative.getProcessingExceptions());
+      assertEquals(brokerResponseNative.getProcessingExceptions().size(), 0);
+      DataSchema dataSchema = new DataSchema(new String[]{"column1", "column5", "column9", "column11"},
+          new DataSchema.ColumnDataType[]{DataSchema.ColumnDataType.INT, DataSchema.ColumnDataType.STRING,
+              DataSchema.ColumnDataType.INT, DataSchema.ColumnDataType.STRING});
+      assertEquals(resultTable.getDataSchema(), dataSchema);
+      List<Object[]> resultRows = resultTable.getRows();
+      int previousColumn1 = Integer.MIN_VALUE;
+      for (Object[] resultRow : resultRows) {
+        assertEquals(resultRow.length, 4);
+        assertEquals((String) resultRow[1], "gFuH");
+        assertTrue((Integer) resultRow[0] >= previousColumn1);
+        previousColumn1 = (Integer) resultRow[0];
+      }
+
+      Object[] firstRow = resultRows.get(0);
+      // Column 11
+      assertEquals((String) firstRow[3], "o");
+    }
+    {
+      // Query with literal only in SELECT
+      String query = "SELECT 'marvin' from testTable ORDER BY column1";
+      BrokerResponseNative brokerResponseNative = getBrokerResponse(query);
+      assertTrue(brokerResponseNative.getProcessingExceptions() == null
+          || brokerResponseNative.getProcessingExceptions().size() == 0);
+      ResultTable resultTable = brokerResponseNative.getResultTable();
+      assertEquals(brokerResponseNative.getNumRowsResultSet(), 10);
+      assertEquals(brokerResponseNative.getTotalDocs(), 120_000L);
+      assertEquals(brokerResponseNative.getNumDocsScanned(), 120_000L);
+      assertEquals(brokerResponseNative.getNumSegmentsProcessed(), 4L);
+      assertEquals(brokerResponseNative.getNumSegmentsMatched(), 4L);
+      assertEquals(brokerResponseNative.getNumEntriesScannedPostFilter(), 120_000L);
+      assertEquals(brokerResponseNative.getNumEntriesScannedInFilter(), 0L);
+      assertNotNull(brokerResponseNative.getProcessingExceptions());
+      assertEquals(brokerResponseNative.getProcessingExceptions().size(), 0);
+      assertEquals(resultTable.getDataSchema(), new DataSchema(new String[]{"'marvin'"},
+          new DataSchema.ColumnDataType[]{DataSchema.ColumnDataType.STRING}));
+      List<Object[]> resultRows = resultTable.getRows();
+      for (Object[] resultRow : resultRows) {
+        assertEquals(resultRow.length, 1);
+        assertEquals((String) resultRow[0], "marvin");
+      }
+    }
+    {
+      // Selection query with '<' filter on a forwardIndexDisabled column with range index available
+      String query = "SELECT column1, column5, column9, column11 FROM testTable WHERE column6 < 2147458029 AND "
+          + "column6 > 1699000 ORDER BY column1";
+      BrokerResponseNative brokerResponseNative = getBrokerResponse(query);
+      assertTrue(brokerResponseNative.getProcessingExceptions() == null
+          || brokerResponseNative.getProcessingExceptions().size() == 0);
+      ResultTable resultTable = brokerResponseNative.getResultTable();
+      assertEquals(brokerResponseNative.getNumRowsResultSet(), 10);
+      assertEquals(brokerResponseNative.getTotalDocs(), 120_000L);
+      assertEquals(brokerResponseNative.getNumDocsScanned(), 119_980L);
+      assertEquals(brokerResponseNative.getNumSegmentsProcessed(), 4L);
+      assertEquals(brokerResponseNative.getNumSegmentsMatched(), 4L);
+      assertEquals(brokerResponseNative.getNumEntriesScannedPostFilter(), 120_100L);
+      assertEquals(brokerResponseNative.getNumEntriesScannedInFilter(), 0L);
+      assertNotNull(brokerResponseNative.getProcessingExceptions());
+      assertEquals(brokerResponseNative.getProcessingExceptions().size(), 0);
+      DataSchema dataSchema = new DataSchema(new String[]{"column1", "column5", "column9", "column11"},
+          new DataSchema.ColumnDataType[]{DataSchema.ColumnDataType.INT, DataSchema.ColumnDataType.STRING,
+              DataSchema.ColumnDataType.INT, DataSchema.ColumnDataType.STRING});
+      assertEquals(resultTable.getDataSchema(), dataSchema);
+      List<Object[]> resultRows = resultTable.getRows();
+      int previousColumn1 = Integer.MIN_VALUE;
+      for (Object[] resultRow : resultRows) {
+        assertEquals(resultRow.length, 4);
+        assertEquals((String) resultRow[1], "gFuH");
+        assertTrue((Integer) resultRow[0] >= previousColumn1);
+        previousColumn1 = (Integer) resultRow[0];
+      }
+
+      Object[] firstRow = resultRows.get(0);
+      // Column 11
+      assertEquals((String) firstRow[3], "o");
+    }
+    {
+      // Selection query with '>=' filter on a forwardIndexDisabled column without range index available
+      String query = "SELECT column1, column5, column9, column11 FROM testTable WHERE column7 >= 676000";
+      try {
+        getBrokerResponse(query);
+        Assert.fail("Query should fail since forwardIndexDisabled on a range query column without range index");
+      } catch (IllegalStateException e) {
+        assertTrue(e.getMessage().contains("Forward index disabled for column:")
+            && e.getMessage().contains("creating ScanDocIdSet unsupported!"));
+      }
+    }
+    {
+      // Select query with a filter on a column which doesn't have forwardIndexDisabled enabled
+      String query = "SELECT column1, column5, column9 from testTable WHERE column9 < 50000 ORDER BY column1";
+      BrokerResponseNative brokerResponseNative = getBrokerResponse(query);
+      assertTrue(brokerResponseNative.getProcessingExceptions() == null
+          || brokerResponseNative.getProcessingExceptions().size() == 0);
+      ResultTable resultTable = brokerResponseNative.getResultTable();
+      assertEquals(brokerResponseNative.getNumRowsResultSet(), 4);
+      assertEquals(brokerResponseNative.getTotalDocs(), 120_000L);
+      assertEquals(brokerResponseNative.getNumDocsScanned(), 4L);
+      assertEquals(brokerResponseNative.getNumSegmentsProcessed(), 4L);
+      assertEquals(brokerResponseNative.getNumSegmentsMatched(), 4L);
+      assertEquals(brokerResponseNative.getNumEntriesScannedPostFilter(), 12L);
+      assertEquals(brokerResponseNative.getNumEntriesScannedInFilter(), 120000L);
+      assertNotNull(brokerResponseNative.getProcessingExceptions());
+      assertEquals(brokerResponseNative.getProcessingExceptions().size(), 0);
+      assertEquals(resultTable.getDataSchema(), new DataSchema(new String[]{"column1", "column5", "column9"},
+          new DataSchema.ColumnDataType[]{
+          DataSchema.ColumnDataType.INT, DataSchema.ColumnDataType.STRING, DataSchema.ColumnDataType.INT
+      }));
+      List<Object[]> resultRows = resultTable.getRows();
+      for (Object[] resultRow : resultRows) {
+        assertEquals(resultRow.length, 3);
+      }
+    }
+    {
+      // Transform function on a filter clause for forwardIndexDisabled column in transform
+      String query = "SELECT column1, column11 from testTable WHERE CONCAT(column6, column9, '-') = '1689277-11270'";
+      BrokerResponseNative brokerResponseNative = getBrokerResponse(query);
+      assertTrue(brokerResponseNative.getProcessingExceptions() != null
+          && brokerResponseNative.getProcessingExceptions().size() > 0);
+    }
+    {
+      // Transform function on a filter clause for a non-forwardIndexDisabled column in transform
+      String query = "SELECT column1, column11 from testTable WHERE CONCAT(column5, column9, '-') = 'gFuH-11270' "
+          + "ORDER BY column1";
+      BrokerResponseNative brokerResponseNative = getBrokerResponse(query);
+      assertTrue(brokerResponseNative.getProcessingExceptions() == null
+          || brokerResponseNative.getProcessingExceptions().size() == 0);
+      ResultTable resultTable = brokerResponseNative.getResultTable();
+      assertEquals(brokerResponseNative.getNumRowsResultSet(), 4);
+      assertEquals(brokerResponseNative.getTotalDocs(), 120_000L);
+      assertEquals(brokerResponseNative.getNumDocsScanned(), 4L);
+      assertEquals(brokerResponseNative.getNumSegmentsProcessed(), 4L);
+      assertEquals(brokerResponseNative.getNumSegmentsMatched(), 4L);
+      assertEquals(brokerResponseNative.getNumEntriesScannedPostFilter(), 8L);
+      assertEquals(brokerResponseNative.getNumEntriesScannedInFilter(), 120000L);
+      assertNotNull(brokerResponseNative.getProcessingExceptions());
+      assertEquals(brokerResponseNative.getProcessingExceptions().size(), 0);
+      assertEquals(resultTable.getDataSchema(), new DataSchema(new String[]{"column1", "column11"},
+          new DataSchema.ColumnDataType[]{
+              DataSchema.ColumnDataType.INT, DataSchema.ColumnDataType.STRING
+          }));
+      List<Object[]> resultRows = resultTable.getRows();
+      assertEquals(resultRows.size(), 4);
+      for (Object[] resultRow : resultRows) {
+        assertEquals(resultRow.length, 2);
+        assertEquals(resultRow[0], 815409257);
+        assertEquals(resultRow[1], "P");
+      }
+    }
+  }
+
+  @Test
+  public void testSelectWithDistinctQueries() {
+    {
+      // Select a mix of forwardIndexDisabled and non-forwardIndexDisabled columns with distinct
+      String query = "SELECT DISTINCT column1, column6, column9 FROM testTable LIMIT 10";
+      try {
+        getBrokerResponse(query);
+        Assert.fail("Query should fail since forwardIndexDisabled on a column in select distinct");
+      } catch (IllegalStateException e) {
+        assertTrue(e.getMessage().contains("Forward index disabled for column:")
+            && e.getMessage().contains("cannot create DataFetcher!"));
+      }
+    }
+    {
+      // Select non-forwardIndexDisabled columns with distinct
+      String query = "SELECT DISTINCT column1, column5, column9 FROM testTable ORDER BY column1 LIMIT 10";
+      BrokerResponseNative brokerResponseNative = getBrokerResponse(query);
+      assertTrue(brokerResponseNative.getProcessingExceptions() == null
+          || brokerResponseNative.getProcessingExceptions().size() == 0);
+      ResultTable resultTable = brokerResponseNative.getResultTable();
+      assertEquals(brokerResponseNative.getNumRowsResultSet(), 10);
+      assertEquals(brokerResponseNative.getTotalDocs(), 120_000L);
+      assertEquals(brokerResponseNative.getNumDocsScanned(), 120_000L);
+      assertEquals(brokerResponseNative.getNumSegmentsProcessed(), 4L);
+      assertEquals(brokerResponseNative.getNumSegmentsMatched(), 4L);
+      assertEquals(brokerResponseNative.getNumEntriesScannedPostFilter(), 360_000L);
+      assertEquals(brokerResponseNative.getNumEntriesScannedInFilter(), 0L);
+      assertNotNull(brokerResponseNative.getProcessingExceptions());
+      assertEquals(brokerResponseNative.getProcessingExceptions().size(), 0);
+      assertEquals(resultTable.getDataSchema(), new DataSchema(new String[]{"column1", "column5", "column9"},
+          new DataSchema.ColumnDataType[]{DataSchema.ColumnDataType.INT, DataSchema.ColumnDataType.STRING,
+              DataSchema.ColumnDataType.INT}));
+      List<Object[]> resultRows = resultTable.getRows();
+      int previousColumn1 = Integer.MIN_VALUE;
+      for (Object[] resultRow : resultRows) {
+        assertEquals(resultRow.length, 3);
+        assertEquals(resultRow[1], "gFuH");
+        assertTrue((Integer) resultRow[0] >= previousColumn1);
+        previousColumn1 = (Integer) resultRow[0];
+      }
+    }
+  }
+
+  @Test
+  public void testSelectWithGroupByOrderByQueries() {
+    {
+      // Select a mix of forwardIndexDisabled and non-forwardIndexDisabled columns with group by order by
+      String query = "SELECT column1, column6 FROM testTable GROUP BY column1, column6 ORDER BY column1, column6 "
+          + " LIMIT 10";
+      try {
+        getBrokerResponse(query);
+        Assert.fail("Query should fail since forwardIndexDisabled on a column in select group by order by");
+      } catch (IllegalStateException e) {
+        assertTrue(e.getMessage().contains("Forward index disabled for column:")
+            && e.getMessage().contains("cannot create DataFetcher!"));
+      }
+    }
+    {
+      // Select forwardIndexDisabled columns with group by order by
+      String query = "SELECT column7, column6 FROM testTable GROUP BY column7, column6 ORDER BY column7, column6 "
+          + " LIMIT 10";
+      try {
+        getBrokerResponse(query);
+        Assert.fail("Query should fail since forwardIndexDisabled on a column in select group by order by");
+      } catch (IllegalStateException e) {
+        assertTrue(e.getMessage().contains("Forward index disabled for column:")
+            && e.getMessage().contains("cannot create DataFetcher!"));
+      }
+    }
+    {
+      // Select non-forwardIndexDisabled columns with group by order by
+      String query = "SELECT column1, column5 FROM testTable GROUP BY column1, column5 ORDER BY column1, column5 "
+          + " LIMIT 10";
+      BrokerResponseNative brokerResponseNative = getBrokerResponse(query);
+      assertTrue(brokerResponseNative.getProcessingExceptions() == null
+          || brokerResponseNative.getProcessingExceptions().size() == 0);
+      ResultTable resultTable = brokerResponseNative.getResultTable();
+      assertEquals(brokerResponseNative.getNumRowsResultSet(), 10);
+      assertEquals(brokerResponseNative.getTotalDocs(), 120_000L);
+      assertEquals(brokerResponseNative.getNumDocsScanned(), 120000L);
+      assertEquals(brokerResponseNative.getNumSegmentsProcessed(), 4L);
+      assertEquals(brokerResponseNative.getNumSegmentsMatched(), 4L);
+      assertEquals(brokerResponseNative.getNumEntriesScannedPostFilter(), 240000L);
+      assertEquals(brokerResponseNative.getNumEntriesScannedInFilter(), 0L);
+      assertNotNull(brokerResponseNative.getProcessingExceptions());
+      assertEquals(brokerResponseNative.getProcessingExceptions().size(), 0);
+      assertEquals(resultTable.getDataSchema(), new DataSchema(new String[]{"column1", "column5"},
+          new DataSchema.ColumnDataType[]{DataSchema.ColumnDataType.INT, DataSchema.ColumnDataType.STRING}));
+      List<Object[]> resultRows = resultTable.getRows();
+      int previousVal = -1;
+      for (Object[] resultRow : resultRows) {
+        assertEquals(resultRow.length, 2);
+        assertEquals(resultRow[1], "gFuH");
+        assertTrue((int) resultRow[0] > previousVal);
+        previousVal = (int) resultRow[0];
+      }
+    }
+    {
+      // Select forwardIndexDisabled columns using transform with group by order by
+      String query = "SELECT CONCAT(column1, column6, '-') FROM testTable GROUP BY CONCAT(column1, column6, '-') "
+          + "ORDER BY CONCAT(column1, column6, '-') LIMIT 10";
+      try {
+        getBrokerResponse(query);
+        Assert.fail("Query should fail since forwardIndexDisabled on a transformed column in group by order by");
+      } catch (IllegalStateException e) {
+        assertTrue(e.getMessage().contains("Forward index disabled for column:")
+            && e.getMessage().contains("cannot create DataFetcher!"));
+      }
+    }
+    {
+      // Select non-forwardIndexDisabled columns using transform with group by order by
+      String query = "SELECT CONCAT(column1, column5, '-') FROM testTable GROUP BY CONCAT(column1, column5, '-') "
+          + "ORDER BY CONCAT(column1, column5, '-') LIMIT 10";
+      BrokerResponseNative brokerResponseNative = getBrokerResponse(query);
+      assertTrue(brokerResponseNative.getProcessingExceptions() == null
+          || brokerResponseNative.getProcessingExceptions().size() == 0);
+      ResultTable resultTable = brokerResponseNative.getResultTable();
+      assertEquals(brokerResponseNative.getNumRowsResultSet(), 10);
+      assertEquals(brokerResponseNative.getTotalDocs(), 120_000L);
+      assertEquals(brokerResponseNative.getNumDocsScanned(), 120000L);
+      assertEquals(brokerResponseNative.getNumSegmentsProcessed(), 4L);
+      assertEquals(brokerResponseNative.getNumSegmentsMatched(), 4L);
+      assertEquals(brokerResponseNative.getNumEntriesScannedPostFilter(), 240000L);
+      assertEquals(brokerResponseNative.getNumEntriesScannedInFilter(), 0L);
+      assertNotNull(brokerResponseNative.getProcessingExceptions());
+      assertEquals(brokerResponseNative.getProcessingExceptions().size(), 0);
+      assertEquals(resultTable.getDataSchema(), new DataSchema(new String[]{"concat(column1,column5,'-')"},
+          new DataSchema.ColumnDataType[]{DataSchema.ColumnDataType.STRING}));
+      List<Object[]> resultRows = resultTable.getRows();
+      for (Object[] resultRow : resultRows) {
+        assertEquals(resultRow.length, 1);
+        assertTrue(resultRow[0].toString().endsWith("-gFuH"));
+      }
+    }
+  }
+
+  @Test
+  public void testSelectWithAggregationQueries() {
+    {
+      // Allowed aggregation functions on forwardIndexDisabled columns
+      String query = "SELECT max(column7), min(column7), count(column7), minmaxrange(column7), "
+          + "distinctcount(column7), distinctcounthll(column6), distinctcountrawhll(column7), "

Review Comment:
   So `distinctcountbitmap` doesn't work:
   
   ```
   java.lang.IllegalStateException: Forward index disabled for column: column6, cannot create DataFetcher!
   
   	at com.google.common.base.Preconditions.checkState(Preconditions.java:518)
   	at org.apache.pinot.core.common.DataFetcher.<init>(DataFetcher.java:72)
   	at org.apache.pinot.core.operator.ProjectionOperator.<init>(ProjectionOperator.java:46)
   	at org.apache.pinot.core.plan.ProjectionPlanNode.run(ProjectionPlanNode.java:63)
   	at org.apache.pinot.core.plan.TransformPlanNode.run(TransformPlanNode.java:69)
   	at org.apache.pinot.core.plan.AggregationPlanNode.buildNonFilteredAggOperator(AggregationPlanNode.java:225)
   	at org.apache.pinot.core.plan.AggregationPlanNode.run(AggregationPlanNode.java:80)
   	at org.apache.pinot.core.plan.CombinePlanNode.getCombineOperator(CombinePlanNode.java:95)
   	at org.apache.pinot.core.plan.CombinePlanNode.run(CombinePlanNode.java:82)
   	at org.apache.pinot.core.plan.InstanceResponsePlanNode.run(InstanceResponsePlanNode.java:44)
   	at org.apache.pinot.core.plan.GlobalPlanImplV0.execute(GlobalPlanImplV0.java:52)
   	at org.apache.pinot.queries.BaseQueriesTest.getBrokerResponse(BaseQueriesTest.java:203)
   	at org.apache.pinot.queries.BaseQueriesTest.getBrokerResponse(BaseQueriesTest.java:173)
   	at org.apache.pinot.queries.BaseQueriesTest.getBrokerResponse(BaseQueriesTest.java:135)
   	at org.apache.pinot.queries.BaseQueriesTest.getBrokerResponse(BaseQueriesTest.java:107)
   	at org.apache.pinot.queries.ForwardIndexDisabledSingleValueQueriesTest.testSelectWithAggregationQueries(ForwardIndexDisabledSingleValueQueriesTest.java:807)
   	at java.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
   	at java.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)
   	at java.base/jdk.internal.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
   	at java.base/java.lang.reflect.Method.invoke(Method.java:566)
   	at org.testng.internal.MethodInvocationHelper.invokeMethod(MethodInvocationHelper.java:108)
   	at org.testng.internal.Invoker.invokeMethod(Invoker.java:661)
   	at org.testng.internal.Invoker.invokeTestMethod(Invoker.java:869)
   	at org.testng.internal.Invoker.invokeTestMethods(Invoker.java:1193)
   	at org.testng.internal.TestMethodWorker.invokeTestMethods(TestMethodWorker.java:126)
   	at org.testng.internal.TestMethodWorker.run(TestMethodWorker.java:109)
   	at org.testng.TestRunner.privateRun(TestRunner.java:744)
   	at org.testng.TestRunner.run(TestRunner.java:602)
   	at org.testng.SuiteRunner.runTest(SuiteRunner.java:380)
   	at org.testng.SuiteRunner.runSequentially(SuiteRunner.java:375)
   	at org.testng.SuiteRunner.privateRun(SuiteRunner.java:340)
   	at org.testng.SuiteRunner.run(SuiteRunner.java:289)
   	at org.testng.SuiteRunnerWorker.runSuite(SuiteRunnerWorker.java:52)
   	at org.testng.SuiteRunnerWorker.run(SuiteRunnerWorker.java:86)
   	at org.testng.TestNG.runSuitesSequentially(TestNG.java:1301)
   	at org.testng.TestNG.runSuitesLocally(TestNG.java:1226)
   	at org.testng.TestNG.runSuites(TestNG.java:1144)
   	at org.testng.TestNG.run(TestNG.java:1115)
   	at com.intellij.rt.testng.IDEARemoteTestNG.run(IDEARemoteTestNG.java:66)
   	at com.intellij.rt.testng.RemoteTestNGStarter.main(RemoteTestNGStarter.java:109)
   ```
   
   Do you want me to add a negative test or spend time understanding more about why?



-- 
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 #9333: Add an option to disable the creation of the forward index for a column

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


##########
pinot-segment-local/src/main/java/org/apache/pinot/segment/local/utils/TableConfigUtils.java:
##########
@@ -897,6 +903,40 @@ private static void validateFieldConfigList(@Nullable List<FieldConfig> fieldCon
     }
   }
 
+  /**
+   * Validates the compatibility of the indexes if the column has the forward index disabled. Throws exceptions due to
+   * compatibility mismatch. The checks performed are:
+   *     - Validate dictionary is enabled.
+   *     - Validate inverted index is enabled.
+   *     - Validate that either no range index exists for column or the range index version is at least 2 and isn't a
+   *       multi-value column (since mulit-value defaults to index v1).
+   */
+  private static void validateForwardIndexDisabledIndexCompatibility(String columnName, FieldConfig fieldConfig,

Review Comment:
   Yes I agree that we should probably only keep the check in `TableConfigUtils.java` since it'll safeguard even updating the configuration. I've added these checks in other places (segment creation + reload) more as a safeguard and was planning to remove these when I work on the reload changes. I wanted to just test that there are no other ways that we may miss detecting such a change. I can remove these from segment creation and reload if you think they aren't needed for sure.



-- 
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 #9333: Add an option to disable the creation of the forward index for a column

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


##########
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",

Review Comment:
   As long as we have coverage, I am 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 #9333: Add an option to disable the creation of the forward index for a column

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


##########
pinot-segment-local/src/test/java/org/apache/pinot/segment/local/segment/index/loader/defaultcolumn/BaseDefaultColumnHandlerTest.java:
##########
@@ -70,7 +70,7 @@ public void setUp()
     driver.build();
     _segmentDirectory = new File(_indexDir, driver.getSegmentName());
     _committedSegmentMetadata = new SegmentMetadataImpl(_segmentDirectory);
-    _writer = new SegmentLocalFSDirectory(_indexDir, _committedSegmentMetadata, ReadMode.mmap).createWriter();
+    _writer = new SegmentLocalFSDirectory(_segmentDirectory, _committedSegmentMetadata, ReadMode.mmap).createWriter();

Review Comment:
   not sure why this change is 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] somandal commented on pull request #9333: Add an option to disable the creation of the forward index for a column

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

   @Jackie-Jiang @siddharthteotia @jackjlli I've updated the PR with the latest discussions. Let me know what you think! Thanks!


-- 
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 #9333: Add an option to disable the creation of the forward index for a column

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


##########
pinot-core/src/main/java/org/apache/pinot/core/operator/filter/ScanBasedFilterOperator.java:
##########
@@ -48,10 +49,16 @@ public class ScanBasedFilterOperator extends BaseFilterOperator {
   @Override
   protected FilterBlock getNextBlock() {
     DataSourceMetadata dataSourceMetadata = _dataSource.getDataSourceMetadata();
+    ForwardIndexReader<?> forwardIndexReader = _dataSource.getForwardIndex();

Review Comment:
   Instead of `getNextBlock`(), we can probably fail sooner in the constructor of `ScanBasedFilterOperator` which is still during physical planning ?



-- 
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 #9333: Add an option to disable the creation of the forward index for a column

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


##########
pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/loader/IndexLoadingConfig.java:
##########
@@ -315,6 +341,14 @@ public List<String> getSortedColumns() {
     return _sortedColumns;
   }
 
+  /**
+   * For tests only.
+   */
+  @VisibleForTesting
+  public void setSortedColumns(List<String> sortedColumns) {

Review Comment:
   (nit) can't be a list



-- 
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 #9333: Add an option to disable the creation of the forward index for a column

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


##########
pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/column/PhysicalColumnIndexContainer.java:
##########
@@ -133,7 +133,12 @@ public PhysicalColumnIndexContainer(SegmentDirectory.Reader segmentReader, Colum
       _rangeIndex = null;
     }
 
-    PinotDataBuffer fwdIndexBuffer = segmentReader.getIndexFor(columnName, ColumnIndexType.FORWARD_INDEX);
+    // Setting the 'fwdIndexBuffer' to null if forward index is enabled. No-op index readers will be setup for the
+    // forward index disabled columns which doesn't require the 'fwdIndexBuffer'.
+    boolean forwardIndexDisabled = !segmentReader.hasIndexFor(columnName, ColumnIndexType.FORWARD_INDEX);
+    PinotDataBuffer fwdIndexBuffer = forwardIndexDisabled ? null
+        : segmentReader.getIndexFor(columnName, ColumnIndexType.FORWARD_INDEX);

Review Comment:
   I see. No need for this I guess. Leave it as is



-- 
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 #9333: Add an option to disable the creation of the forward index for a column

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


##########
pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/column/PhysicalColumnIndexContainer.java:
##########
@@ -142,6 +145,8 @@ public PhysicalColumnIndexContainer(SegmentDirectory.Reader segmentReader, Colum
         // Single-value
         if (metadata.isSorted()) {
           // Sorted
+          // forwardIndexDisabled columns do not need to be handled here as forward index cannot be disabled on a

Review Comment:
   May be I am mis-reading the if-else block here. For an unsorted dict column with inv index, are we ok with line 157 being executed with null `fwdIndexBuffer` ?



-- 
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 #9333: Add an option to disable the creation of the forward index for a column

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


##########
pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/creator/impl/DefaultIndexCreatorProvider.java:
##########
@@ -88,17 +90,25 @@ public ForwardIndexCreator newForwardIndexCreator(IndexCreationContext.Forward c
             context.getMaxRowLengthInBytes());
       }
     } else {
-      if (context.getFieldSpec().isSingleValueField()) {
-        if (context.isSorted()) {
-          return new SingleValueSortedForwardIndexCreator(context.getIndexDir(), context.getFieldSpec().getName(),
-              context.getCardinality());
+      if (context.forwardIndexDisabled()) {
+        if (context.getFieldSpec().isSingleValueField()) {
+          return new SingleValueNoOpForwardIndexCreator();

Review Comment:
   done, consolidated this into 1 class. will address removing this as a future change



-- 
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 #9333: Add an option to disable the creation of the forward index for a column

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


##########
pinot-segment-local/src/main/java/org/apache/pinot/segment/local/utils/TableConfigUtils.java:
##########
@@ -897,6 +903,41 @@ private static void validateFieldConfigList(@Nullable List<FieldConfig> fieldCon
     }
   }
 
+  /**
+   * Validates the compatibility of the indexes if the column has the forward index disabled. Throws exceptions due to
+   * compatibility mismatch. The checks performed are:
+   *     - Validate dictionary is enabled.
+   *     - Validate inverted index is enabled.
+   *     - Validate that either no range index exists for column or the range index version is at least 2 and isn't a
+   *       multi-value column (since mulit-value defaults to index v1).
+   */
+  private static void validateForwardIndexDisabledIndexCompatibility(String columnName, FieldConfig fieldConfig,
+      IndexingConfig indexingConfigs, List<String> noDictionaryColumns, Schema schema) {
+    Map<String, String> fieldConfigProperties = fieldConfig.getProperties();
+    if (fieldConfigProperties == null) {
+      return;
+    }
+
+    boolean forwardIndexDisabled = Boolean.parseBoolean(fieldConfigProperties
+        .getOrDefault(FieldConfig.FORWARD_INDEX_DISABLED, FieldConfig.DEFAULT_FORWARD_INDEX_DISABLED));

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 #9333: Add an option to disable the creation of the forward index for a column

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


##########
pinot-segment-local/src/main/java/org/apache/pinot/segment/local/utils/TableConfigUtils.java:
##########
@@ -897,6 +903,40 @@ private static void validateFieldConfigList(@Nullable List<FieldConfig> fieldCon
     }
   }
 
+  /**
+   * Validates the compatibility of the indexes if the column has the forward index disabled. Throws exceptions due to
+   * compatibility mismatch. The checks performed are:
+   *     - Validate dictionary is enabled.
+   *     - Validate inverted index is enabled.
+   *     - Validate that either no range index exists for column or the range index version is at least 2 and isn't a
+   *       multi-value column (since mulit-value defaults to index v1).
+   */
+  private static void validateForwardIndexDisabledIndexCompatibility(String columnName, FieldConfig fieldConfig,
+      IndexingConfig indexingConfigs, List<String> noDictionaryColumns, Schema schema) {
+    Map<String, String> fieldConfigProperties = fieldConfig.getProperties();
+    if (fieldConfigProperties == null) {
+      return;
+    }
+
+    boolean forwardIndexDisabled = Boolean.parseBoolean(fieldConfigProperties.get(FieldConfig.FORWARD_INDEX_DISABLED));
+    if (!forwardIndexDisabled) {
+      return;
+    }
+
+    FieldSpec fieldSpec = schema.getFieldSpecFor(columnName);
+    Preconditions.checkState(noDictionaryColumns == null || !noDictionaryColumns.contains(columnName),

Review Comment:
   Looks good. 



-- 
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 #9333: Add an option to disable the creation of the forward index for a column

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


##########
pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/OfflineClusterIntegrationTest.java:
##########
@@ -143,7 +143,7 @@ public class OfflineClusterIntegrationTest extends BaseClusterIntegrationTestSet
   private static final String MAX_NUM_MULTI_VALUES_MAP_KEY = "maxNumMultiValuesMap";
   // TODO: This might lead to flaky test, as this disk size is not deterministic
   //       as it depends on the iteration order of a HashSet.
-  private static final int DISK_SIZE_IN_BYTES = 20797128;
+  private static final int DISK_SIZE_IN_BYTES = 20843088;

Review Comment:
   so I've added a metadata field in the column metadata which increases the disk size. Also based on the comment here it looks like the size may change depending on the iteration order of the `HashSet`. Let me know if you think I should look into this deeper to compare the difference in size change.



##########
pinot-segment-local/src/main/java/org/apache/pinot/segment/local/utils/TableConfigUtils.java:
##########
@@ -869,6 +871,25 @@ private static void validateFieldConfigList(@Nullable List<FieldConfig> fieldCon
           default:
             break;
         }
+
+        // Validate the forward index disabled compatibility if enabled for this column

Review Comment:
   done



##########
pinot-segment-local/src/main/java/org/apache/pinot/segment/local/utils/TableConfigUtils.java:
##########
@@ -869,6 +871,25 @@ private static void validateFieldConfigList(@Nullable List<FieldConfig> fieldCon
           default:
             break;
         }
+
+        // Validate the forward index disabled compatibility if enabled for this column
+        Map<String, String> fieldConfigProperties = fieldConfig.getProperties();
+        if (fieldConfigProperties != null) {
+          boolean forwardIndexDisabled = Boolean.parseBoolean(fieldConfigProperties
+              .getOrDefault(FieldConfig.FORWARD_INDEX_DISABLED, FieldConfig.DEFAULT_FORWARD_INDEX_DISABLED));
+          if (forwardIndexDisabled) {
+            Preconditions.checkState((noDictionaryColumns == null || !noDictionaryColumns.contains(columnName))

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 #9333: Add an option to disable the creation of the forward index for a column

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


##########
pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/readers/forward/NoOpMVForwardIndexReader.java:
##########
@@ -0,0 +1,66 @@
+/**
+ * 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.segment.local.segment.index.readers.forward;
+
+import java.io.IOException;
+import org.apache.pinot.segment.spi.index.reader.ForwardIndexReader;
+import org.apache.pinot.segment.spi.index.reader.ForwardIndexReaderContext;
+import org.apache.pinot.spi.data.FieldSpec;
+
+
+/**
+ * No-op forward index reader meant to be used by multi-value columns with forward index disabled. This is meant to
+ * provide clean exception handling for operations which need to read forward index when forward index is disabled.
+ */
+public class NoOpMVForwardIndexReader implements ForwardIndexReader<ForwardIndexReaderContext> {

Review Comment:
   +1 to what @siddharthteotia says about this. There are too many places which will require null checks and even then we might miss some. New code will also need to deal with such null checks.



-- 
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 #9333: Add an option to disable the creation of the forward index for a column

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


##########
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:
   good catch, updated here and in other related IndexHandler files



-- 
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 pull request #9333: Add an option to disable the creation of the forward index for a column

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

   > We should allow disabling forward index for sorted column, which is a no-op because the inverted index for sorted column can be used as forward index. I believe it is already allowed in the implementation because the default column is always sorted. No need to add this in the validation.
   
   Thanks for the suggestion @Jackie-Jiang. From going through the code it looks like for sorted columns the forward index and inverted index are essentially the same. Code snippet from the `PhysicalColumnIndexCreator`:
   
   ```
           // Single-value
           if (metadata.isSorted()) {
             // Sorted
             // forwardIndexDisabled columns do not need to be handled here as forward index cannot be disabled on a
             // sorted column
             SortedIndexReader<?> sortedIndexReader = indexReaderProvider.newSortedIndexReader(fwdIndexBuffer, metadata);
             _forwardIndex = sortedIndexReader;
             _invertedIndex = sortedIndexReader;
             _fstIndex = null;
             return;
           }
   ```
   
   I also see code on the segment creation path where we skip creating the inverted index if the column is sorted even if the inverted index is enabled for the column. So keeping these in mind, we didn't think it made much sense to disable forward index for such columns.
   
   Let me know your thoughts based on the above. cc @siddharthteotia 
   
   


-- 
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 #9333: Add an option to disable the creation of the forward index for a column

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


##########
pinot-segment-local/src/main/java/org/apache/pinot/segment/local/realtime/converter/stats/MutableColumnStatistics.java:
##########
@@ -145,6 +146,8 @@ public boolean isSorted() {
 
     // Iterate over all data to figure out whether or not it's in sorted order
     MutableForwardIndex mutableForwardIndex = (MutableForwardIndex) _dataSource.getForwardIndex();
+    Preconditions.checkState(mutableForwardIndex != null,

Review Comment:
   Jackie had a similar comment so I've gone ahead and unified it. I think the exception message is enough to clarify when it's not supported due to a valid disabled forward index and where the forward index shouldn't be null.



-- 
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 #9333: Add an option to disable the creation of the forward index for a column

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


##########
pinot-core/src/test/java/org/apache/pinot/queries/ForwardIndexDisabledSingleValueQueriesTest.java:
##########
@@ -0,0 +1,1088 @@
+/**
+ * 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.queries;
+
+import java.io.File;
+import java.net.URL;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.TimeUnit;
+import org.apache.commons.io.FileUtils;
+import org.apache.pinot.common.response.broker.BrokerResponseNative;
+import org.apache.pinot.common.response.broker.ResultTable;
+import org.apache.pinot.common.utils.DataSchema;
+import org.apache.pinot.segment.local.indexsegment.immutable.ImmutableSegmentLoader;
+import org.apache.pinot.segment.local.segment.creator.impl.SegmentIndexCreationDriverImpl;
+import org.apache.pinot.segment.local.segment.index.loader.IndexLoadingConfig;
+import org.apache.pinot.segment.spi.ColumnMetadata;
+import org.apache.pinot.segment.spi.ImmutableSegment;
+import org.apache.pinot.segment.spi.IndexSegment;
+import org.apache.pinot.segment.spi.creator.SegmentGeneratorConfig;
+import org.apache.pinot.segment.spi.creator.SegmentIndexCreationDriver;
+import org.apache.pinot.spi.config.table.FieldConfig;
+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.data.TimeGranularitySpec;
+import org.apache.pinot.spi.utils.ReadMode;
+import org.apache.pinot.spi.utils.builder.TableConfigBuilder;
+import org.testng.Assert;
+import org.testng.annotations.AfterClass;
+import org.testng.annotations.BeforeClass;
+import org.testng.annotations.Test;
+
+import static org.testng.Assert.assertEquals;
+import static org.testng.Assert.assertNotNull;
+import static org.testng.Assert.assertNull;
+import static org.testng.Assert.assertTrue;
+
+
+/**
+ * The <code>ForwardIndexDisabledSingleValueQueriesTest</code> class sets up the index segment for the no forward
+ * index single-value queries test.
+ * <p>There are totally 18 columns, 30000 records inside the original Avro file where 11 columns are selected to build
+ * the index segment. Selected columns information are as following:
+ * <ul>
+ *   ColumnName, FieldType, DataType, Cardinality, IsSorted, HasInvertedIndex, FwdIndexDisabled: S1, S2, HasRangeIndex
+ *   <li>column1, METRIC, INT, 6582, F, F, F, F, F</li>
+ *   <li>column3, METRIC, INT, 21910, F, F, F, F, F</li>
+ *   <li>column5, DIMENSION, STRING, 1, T, F, F, F, F</li>
+ *   <li>column6, DIMENSION, INT, 608, F, T, T, T, T</li>
+ *   <li>column7, DIMENSION, INT, 146, F, T, T, F, F</li>
+ *   <li>column9, DIMENSION, INT, 1737, F, F, F, F, F</li>
+ *   <li>column11, DIMENSION, STRING, 5, F, T, F, F, F</li>
+ *   <li>column12, DIMENSION, STRING, 5, F, F, F, F, F</li>
+ *   <li>column17, METRIC, INT, 24, F, T, F, F, F</li>
+ *   <li>column18, METRIC, INT, 1440, F, T, F, F, F</li>
+ *   <li>daysSinceEpoch, TIME, INT, 2, T, F, F, F, F</li>
+ * </ul>
+ */
+public class ForwardIndexDisabledSingleValueQueriesTest extends BaseQueriesTest {
+  private static final String AVRO_DATA = "data" + File.separator + "test_data-sv.avro";
+  private static final String SEGMENT_NAME_1 = "testTable_126164076_167572857";
+  private static final String SEGMENT_NAME_2 = "testTable_126164076_167572858";
+  private static final File INDEX_DIR = new File(FileUtils.getTempDirectory(),
+      "ForwardIndexDisabledSingleValueQueriesTest");
+
+  private static final String SELECT_STAR_QUERY = "SELECT * FROM testTable";
+
+  // Hard-coded query filter.
+  private static final String FILTER = " WHERE column1 > 100000000"
+      + " AND column3 BETWEEN 20000000 AND 1000000000"
+      + " AND column5 = 'gFuH'"
+      + " AND (column6 < 500000000 OR column11 NOT IN ('t', 'P'))"
+      + " AND daysSinceEpoch = 126164076";
+
+  private IndexSegment _indexSegment;
+  // Contains 2 index segments, one with 2 columns with forward index disabled, and the other with just 1.
+  private List<IndexSegment> _indexSegments;
+
+  private TableConfig _tableConfig;
+  private List<String> _invertedIndexColumns;
+  private List<String> _forwardIndexDisabledColumns;
+
+  @BeforeClass
+  public void buildAndLoadSegment()
+      throws Exception {
+    FileUtils.deleteQuietly(INDEX_DIR);
+
+    // Get resource file path.
+    URL resource = getClass().getClassLoader().getResource(AVRO_DATA);
+    assertNotNull(resource);
+    String filePath = resource.getFile();
+
+    // Build the segment schema.
+    Schema schema = new Schema.SchemaBuilder().setSchemaName("testTable").addMetric("column1", FieldSpec.DataType.INT)
+        .addMetric("column3", FieldSpec.DataType.INT).addSingleValueDimension("column5", FieldSpec.DataType.STRING)
+        .addSingleValueDimension("column6", FieldSpec.DataType.INT)
+        .addSingleValueDimension("column7", FieldSpec.DataType.INT)
+        .addSingleValueDimension("column9", FieldSpec.DataType.INT)
+        .addSingleValueDimension("column11", FieldSpec.DataType.STRING)
+        .addSingleValueDimension("column12", FieldSpec.DataType.STRING).addMetric("column17", FieldSpec.DataType.INT)
+        .addMetric("column18", FieldSpec.DataType.INT)
+        .addTime(new TimeGranularitySpec(FieldSpec.DataType.INT, TimeUnit.DAYS, "daysSinceEpoch"), null).build();
+
+    createSegment(filePath, SEGMENT_NAME_1, schema);
+    createSegment(filePath, SEGMENT_NAME_2, schema);
+
+    ImmutableSegment immutableSegment1 = loadSegmentWithMetadataChecks(SEGMENT_NAME_1);
+    ImmutableSegment immutableSegment2 = loadSegmentWithMetadataChecks(SEGMENT_NAME_2);
+
+    _indexSegment = immutableSegment1;
+    _indexSegments = Arrays.asList(immutableSegment1, immutableSegment2);
+  }
+
+  private void createSegment(String filePath, String segmentName, Schema schema)
+      throws Exception {
+    // Create field configs for the no forward index columns
+    List<FieldConfig> fieldConfigList = new ArrayList<>();
+    fieldConfigList.add(new FieldConfig("column6", FieldConfig.EncodingType.DICTIONARY, Collections.emptyList(), null,
+        Collections.singletonMap(FieldConfig.FORWARD_INDEX_DISABLED, Boolean.TRUE.toString())));
+    if (segmentName.equals(SEGMENT_NAME_1)) {
+      fieldConfigList.add(new FieldConfig("column7", FieldConfig.EncodingType.DICTIONARY, Collections.emptyList(),
+          null, Collections.singletonMap(FieldConfig.FORWARD_INDEX_DISABLED, Boolean.TRUE.toString())));
+
+      // Build table config based on segment 1 as it contains both columns under no forward index
+      _tableConfig =
+          new TableConfigBuilder(TableType.OFFLINE).setTableName("testTable").setTimeColumnName("daysSinceEpoch")
+              .setFieldConfigList(fieldConfigList).setRangeIndexColumns(Arrays.asList("column6")).build();
+    }
+
+    // Create the segment generator config.
+    SegmentGeneratorConfig segmentGeneratorConfig = new SegmentGeneratorConfig(_tableConfig, schema);
+    segmentGeneratorConfig.setInputFilePath(filePath);
+    segmentGeneratorConfig.setTableName("testTable");
+    segmentGeneratorConfig.setOutDir(INDEX_DIR.getAbsolutePath());
+    segmentGeneratorConfig.setSegmentName(segmentName);
+    // The segment generation code in SegmentColumnarIndexCreator will throw
+    // exception if start and end time in time column are not in acceptable
+    // range. For this test, we first need to fix the input avro data
+    // to have the time column values in allowed range. Until then, the check
+    // is explicitly disabled
+    segmentGeneratorConfig.setSkipTimeValueCheck(true);
+    _invertedIndexColumns = Arrays.asList("column6", "column7", "column11", "column17", "column18");
+    segmentGeneratorConfig.setInvertedIndexCreationColumns(_invertedIndexColumns);
+
+    _forwardIndexDisabledColumns = new ArrayList<>(Arrays.asList("column6", "column7"));
+    segmentGeneratorConfig.setForwardIndexDisabledColumns(_forwardIndexDisabledColumns);
+    segmentGeneratorConfig.setRangeIndexCreationColumns(Arrays.asList("column6"));
+
+    // Build the index segment.
+    SegmentIndexCreationDriver driver = new SegmentIndexCreationDriverImpl();
+    driver.init(segmentGeneratorConfig);
+    driver.build();
+  }
+
+  private ImmutableSegment loadSegmentWithMetadataChecks(String segmentName)
+      throws Exception {
+    IndexLoadingConfig indexLoadingConfig = new IndexLoadingConfig();
+    indexLoadingConfig.setTableConfig(_tableConfig);
+    indexLoadingConfig.setInvertedIndexColumns(new HashSet<>(_invertedIndexColumns));
+    indexLoadingConfig.setForwardIndexDisabledColumns(new HashSet<>(_forwardIndexDisabledColumns));
+    indexLoadingConfig.setRangeIndexColumns(new HashSet<>(Arrays.asList("column6")));
+    indexLoadingConfig.setReadMode(ReadMode.heap);
+
+    ImmutableSegment immutableSegment = ImmutableSegmentLoader.load(new File(INDEX_DIR, segmentName),
+        indexLoadingConfig);
+
+    Map<String, ColumnMetadata> columnMetadataMap1 = immutableSegment.getSegmentMetadata().getColumnMetadataMap();
+    columnMetadataMap1.forEach((column, metadata) -> {
+      if (column.equals("column6") || column.equals("column7")) {
+        assertTrue(metadata.hasDictionary());
+        assertTrue(metadata.isSingleValue());
+        assertNull(immutableSegment.getForwardIndex(column));
+      } else {
+        assertNotNull(immutableSegment.getForwardIndex(column));
+      }
+    });
+
+    return immutableSegment;
+  }
+
+  @AfterClass
+  public void deleteAndDestroySegment() {
+    FileUtils.deleteQuietly(INDEX_DIR);
+    _indexSegments.forEach((IndexSegment::destroy));
+  }
+
+  @Override
+  protected String getFilter() {
+    return FILTER;
+  }
+
+  @Override
+  protected IndexSegment getIndexSegment() {
+    return _indexSegment;
+  }
+
+  @Override
+  protected List<IndexSegment> getIndexSegments() {
+    return _indexSegments;
+  }
+
+  @Test
+  public void testSelectStarQueries() {
+    // Select * without any filters
+    try {
+      getBrokerResponse(SELECT_STAR_QUERY);
+      Assert.fail("Select * query should fail since forwardIndexDisabled on a select column");
+    } catch (IllegalStateException e) {
+      assertTrue(e.getMessage().contains("Forward index disabled for column:")
+          && e.getMessage().contains("cannot create DataFetcher!"));
+    }
+
+    // Select * with filters
+    try {
+      getBrokerResponse(SELECT_STAR_QUERY + FILTER);
+      Assert.fail("Select * query should fail since forwardIndexDisabled on a select column");
+    } catch (IllegalStateException e) {
+      assertTrue(e.getMessage().contains("Forward index disabled for column:")
+          && e.getMessage().contains("cannot create DataFetcher!"));
+    }
+  }
+
+  @Test
+  public void testSelectQueries() {
+    {
+      // Selection query without filters including a column with forwardIndexDisabled enabled on both segments
+      String query = "SELECT column1, column5, column6, column9, column11 FROM testTable";
+      try {
+        getBrokerResponse(query);
+        Assert.fail("Query should fail since forwardIndexDisabled on a select column");
+      } catch (IllegalStateException e) {
+        assertTrue(e.getMessage().contains("Forward index disabled for column:")
+            && e.getMessage().contains("cannot create DataFetcher!"));
+      }
+    }
+    {
+      // Selection query without filters including a column with forwardIndexDisabled enabled on one segment
+      String query = "SELECT column1, column5, column7, column9, column11 FROM testTable";
+      try {
+        getBrokerResponse(query);
+        Assert.fail("Query should fail since forwardIndexDisabled on a select column");
+      } catch (IllegalStateException e) {
+        assertTrue(e.getMessage().contains("Forward index disabled for column:")
+            && e.getMessage().contains("cannot create DataFetcher!"));
+      }
+    }
+    {
+      // Selection query with filters including a column with forwardIndexDisabled enabled on both segments
+      String query = "SELECT column1, column5, column6, column9, column11 FROM testTable WHERE column6 = 2147458029";
+      try {
+        getBrokerResponse(query);
+        Assert.fail("Query should fail since forwardIndexDisabled on a select column");
+      } catch (IllegalStateException e) {
+        assertTrue(e.getMessage().contains("Forward index disabled for column:")
+            && e.getMessage().contains("cannot create DataFetcher!"));
+      }
+    }
+    {
+      // Selection query with filters including a column with forwardIndexDisabled enabled on one segment
+      String query = "SELECT column1, column5, column7, column9, column11 FROM testTable WHERE column7 = 675695";
+      try {
+        getBrokerResponse(query);
+        Assert.fail("Query should fail since forwardIndexDisabled on a select column");
+      } catch (IllegalStateException e) {
+        assertTrue(e.getMessage().contains("Forward index disabled for column:")
+            && e.getMessage().contains("cannot create DataFetcher!"));
+      }
+    }
+    {
+      // Selection query without filters and without columns with forwardIndexDisabled enabled on either segment
+      String query = "SELECT column1, column5, column9, column11 FROM testTable ORDER BY column1";
+      BrokerResponseNative brokerResponseNative = getBrokerResponse(query);
+      assertTrue(brokerResponseNative.getProcessingExceptions() == null
+          || brokerResponseNative.getProcessingExceptions().size() == 0);
+      ResultTable resultTable = brokerResponseNative.getResultTable();
+      assertEquals(brokerResponseNative.getNumRowsResultSet(), 10);
+      assertEquals(brokerResponseNative.getTotalDocs(), 120_000L);
+      assertEquals(brokerResponseNative.getNumDocsScanned(), 120_000L);
+      assertEquals(brokerResponseNative.getNumSegmentsProcessed(), 4L);
+      assertEquals(brokerResponseNative.getNumSegmentsMatched(), 4L);
+      assertEquals(brokerResponseNative.getNumEntriesScannedPostFilter(), 120_120L);
+      assertEquals(brokerResponseNative.getNumEntriesScannedInFilter(), 0L);
+      assertNotNull(brokerResponseNative.getProcessingExceptions());
+      assertEquals(brokerResponseNative.getProcessingExceptions().size(), 0);
+      DataSchema dataSchema = new DataSchema(new String[]{"column1", "column5", "column9", "column11"},
+        new DataSchema.ColumnDataType[]{DataSchema.ColumnDataType.INT, DataSchema.ColumnDataType.STRING,
+            DataSchema.ColumnDataType.INT, DataSchema.ColumnDataType.STRING});
+      assertEquals(resultTable.getDataSchema(), dataSchema);
+      List<Object[]> resultRows = resultTable.getRows();
+      int previousColumn1 = Integer.MIN_VALUE;
+      for (Object[] resultRow : resultRows) {
+        assertEquals(resultRow.length, 4);
+        assertEquals((String) resultRow[1], "gFuH");
+        // Column 1
+        assertTrue((Integer) resultRow[0] >= previousColumn1);
+        previousColumn1 = (Integer) resultRow[0];
+      }
+
+      Object[] firstRow = resultRows.get(0);
+      // Column 11
+      assertEquals((String) firstRow[3], "o");
+    }
+    {
+      // Transform function on a selection clause with a forwardIndexDisabled column in transform
+      String query = "SELECT CONCAT(column6, column9, '-') from testTable";
+      try {
+        getBrokerResponse(query);
+        Assert.fail("Query should fail since forwardIndexDisabled on a column in transform");
+      } catch (IllegalStateException e) {
+        assertTrue(e.getMessage().contains("Forward index disabled for column:")
+            && e.getMessage().contains("cannot create DataFetcher!"));
+      }
+    }
+    {
+      // Transform function on a selection clause without a forwardIndexDisabled column in transform
+      String query = "SELECT CONCAT(column5, column9, '-') from testTable ORDER BY column1";
+      BrokerResponseNative brokerResponseNative = getBrokerResponse(query);
+      assertTrue(brokerResponseNative.getProcessingExceptions() == null
+          || brokerResponseNative.getProcessingExceptions().size() == 0);
+      ResultTable resultTable = brokerResponseNative.getResultTable();
+      assertEquals(brokerResponseNative.getNumRowsResultSet(), 10);
+      assertEquals(brokerResponseNative.getTotalDocs(), 120_000L);
+      assertEquals(brokerResponseNative.getNumDocsScanned(), 120_000L);
+      assertEquals(brokerResponseNative.getNumSegmentsProcessed(), 4L);
+      assertEquals(brokerResponseNative.getNumSegmentsMatched(), 4L);
+      assertEquals(brokerResponseNative.getNumEntriesScannedPostFilter(), 120_080L);
+      assertEquals(brokerResponseNative.getNumEntriesScannedInFilter(), 0L);
+      assertNotNull(brokerResponseNative.getProcessingExceptions());
+      assertEquals(brokerResponseNative.getProcessingExceptions().size(), 0);
+      DataSchema dataSchema = new DataSchema(new String[]{"concat(column5,column9,'-')"},
+          new DataSchema.ColumnDataType[]{DataSchema.ColumnDataType.STRING});
+      assertEquals(resultTable.getDataSchema(), dataSchema);
+      List<Object[]> resultRows = resultTable.getRows();
+      for (Object[] resultRow : resultRows) {
+        assertEquals(resultRow.length, 1);
+        assertTrue(resultRow[0].toString().startsWith("gFuH-"));
+      }
+    }
+    {
+      // Selection query with filters (not including forwardIndexDisabled column) and without columns with
+      // forwardIndexDisabled enabled on either segment
+      String query = "SELECT column1, column5, column9, column11 FROM testTable WHERE column1 > 100000000"
+        + " AND column3 BETWEEN 20000000 AND 1000000000"
+        + " AND column5 = 'gFuH'"
+        + " AND daysSinceEpoch = 126164076 ORDER BY column1";
+      BrokerResponseNative brokerResponseNative = getBrokerResponse(query);
+      assertTrue(brokerResponseNative.getProcessingExceptions() == null
+          || brokerResponseNative.getProcessingExceptions().size() == 0);
+      ResultTable resultTable = brokerResponseNative.getResultTable();
+      assertEquals(brokerResponseNative.getNumRowsResultSet(), 10);
+      assertEquals(brokerResponseNative.getTotalDocs(), 120_000L);
+      assertEquals(brokerResponseNative.getNumDocsScanned(), 42_368L);
+      assertEquals(brokerResponseNative.getNumSegmentsProcessed(), 4L);
+      assertEquals(brokerResponseNative.getNumSegmentsMatched(), 4L);
+      assertEquals(brokerResponseNative.getNumEntriesScannedPostFilter(), 42_488);
+      assertEquals(brokerResponseNative.getNumEntriesScannedInFilter(), 192744L);
+      assertNotNull(brokerResponseNative.getProcessingExceptions());
+      assertEquals(brokerResponseNative.getProcessingExceptions().size(), 0);
+      DataSchema dataSchema = new DataSchema(new String[]{"column1", "column5", "column9", "column11"},
+          new DataSchema.ColumnDataType[]{DataSchema.ColumnDataType.INT, DataSchema.ColumnDataType.STRING,
+              DataSchema.ColumnDataType.INT, DataSchema.ColumnDataType.STRING});
+      assertEquals(resultTable.getDataSchema(), dataSchema);
+      List<Object[]> resultRows = resultTable.getRows();
+      int previousColumn1 = Integer.MIN_VALUE;
+      for (Object[] resultRow : resultRows) {
+        assertEquals(resultRow.length, 4);
+        assertEquals((String) resultRow[1], "gFuH");
+        assertTrue((Integer) resultRow[0] >= previousColumn1);
+        previousColumn1 = (Integer) resultRow[0];
+      }
+
+      Object[] firstRow = resultRows.get(0);
+      // Column 11
+      assertEquals((String) firstRow[3], "P");
+    }
+    {
+      // Selection query with supported filters (including forwardIndexDisabled column) and without columns with
+      // forwardIndexDisabled enabled on either segment
+      String query = "SELECT column1, column5, column9, column11 FROM testTable WHERE column6 = 2147458029 "
+          + "ORDER BY column1";
+      BrokerResponseNative brokerResponseNative = getBrokerResponse(query);
+      assertTrue(brokerResponseNative.getProcessingExceptions() == null
+          || brokerResponseNative.getProcessingExceptions().size() == 0);
+      ResultTable resultTable = brokerResponseNative.getResultTable();
+      assertEquals(brokerResponseNative.getNumRowsResultSet(), 10);
+      assertEquals(brokerResponseNative.getTotalDocs(), 120_000L);
+      assertEquals(brokerResponseNative.getNumDocsScanned(), 16L);
+      assertEquals(brokerResponseNative.getNumSegmentsProcessed(), 4L);
+      assertEquals(brokerResponseNative.getNumSegmentsMatched(), 4L);
+      assertEquals(brokerResponseNative.getNumEntriesScannedPostFilter(), 64L);
+      assertEquals(brokerResponseNative.getNumEntriesScannedInFilter(), 0L);
+      assertNotNull(brokerResponseNative.getProcessingExceptions());
+      assertEquals(brokerResponseNative.getProcessingExceptions().size(), 0);
+      DataSchema dataSchema = new DataSchema(new String[]{"column1", "column5", "column9", "column11"},
+          new DataSchema.ColumnDataType[]{DataSchema.ColumnDataType.INT, DataSchema.ColumnDataType.STRING,
+              DataSchema.ColumnDataType.INT, DataSchema.ColumnDataType.STRING});
+      assertEquals(resultTable.getDataSchema(), dataSchema);
+      List<Object[]> resultRows = resultTable.getRows();
+      for (Object[] resultRow : resultRows) {
+        assertEquals(resultRow.length, 4);
+        assertEquals(resultRow[0], 635553468);
+        assertEquals((String) resultRow[1], "gFuH");
+        assertEquals(resultRow[2], 705242697);
+        assertEquals((String) resultRow[3], "P");
+      }
+    }
+    {
+      // Selection query with supported filters (including forwardIndexDisabled column) and without columns with
+      // forwardIndexDisabled enabled on either segment
+      String query = "SELECT column1, column5, column9, column11 FROM testTable WHERE column7 != 675695 "
+          + "ORDER BY column1";
+      BrokerResponseNative brokerResponseNative = getBrokerResponse(query);
+      assertTrue(brokerResponseNative.getProcessingExceptions() == null
+          || brokerResponseNative.getProcessingExceptions().size() == 0);
+      ResultTable resultTable = brokerResponseNative.getResultTable();
+      assertEquals(brokerResponseNative.getNumRowsResultSet(), 10);
+      assertEquals(brokerResponseNative.getTotalDocs(), 120_000L);
+      assertEquals(brokerResponseNative.getNumDocsScanned(), 119_908L);
+      assertEquals(brokerResponseNative.getNumSegmentsProcessed(), 4L);
+      assertEquals(brokerResponseNative.getNumSegmentsMatched(), 4L);
+      assertEquals(brokerResponseNative.getNumEntriesScannedPostFilter(), 120_028L);
+      assertEquals(brokerResponseNative.getNumEntriesScannedInFilter(), 0L);
+      assertNotNull(brokerResponseNative.getProcessingExceptions());
+      assertEquals(brokerResponseNative.getProcessingExceptions().size(), 0);
+      DataSchema dataSchema = new DataSchema(new String[]{"column1", "column5", "column9", "column11"},
+          new DataSchema.ColumnDataType[]{DataSchema.ColumnDataType.INT, DataSchema.ColumnDataType.STRING,
+              DataSchema.ColumnDataType.INT, DataSchema.ColumnDataType.STRING});
+      assertEquals(resultTable.getDataSchema(), dataSchema);
+      List<Object[]> resultRows = resultTable.getRows();
+      int previousColumn1 = Integer.MIN_VALUE;
+      for (Object[] resultRow : resultRows) {
+        assertEquals(resultRow.length, 4);
+        assertEquals((String) resultRow[1], "gFuH");
+        assertTrue((Integer) resultRow[0] >= previousColumn1);
+        previousColumn1 = (Integer) resultRow[0];
+      }
+
+      Object[] firstRow = resultRows.get(0);
+      // Column 11
+      assertEquals((String) firstRow[3], "o");
+    }
+    {
+      // Selection query with supported filters (including forwardIndexDisabled column) and without columns with
+      // forwardIndexDisabled enabled on either segment
+      String query = "SELECT column1, column5, column9, column11 FROM testTable WHERE column7 IN (675695, 2137685743) "
+          + "ORDER BY column1";
+      BrokerResponseNative brokerResponseNative = getBrokerResponse(query);
+      assertTrue(brokerResponseNative.getProcessingExceptions() == null
+          || brokerResponseNative.getProcessingExceptions().size() == 0);
+      ResultTable resultTable = brokerResponseNative.getResultTable();
+      assertEquals(brokerResponseNative.getNumRowsResultSet(), 10);
+      assertEquals(brokerResponseNative.getTotalDocs(), 120_000L);
+      assertEquals(brokerResponseNative.getNumDocsScanned(), 828L);
+      assertEquals(brokerResponseNative.getNumSegmentsProcessed(), 4L);
+      assertEquals(brokerResponseNative.getNumSegmentsMatched(), 4L);
+      assertEquals(brokerResponseNative.getNumEntriesScannedPostFilter(), 948L);
+      assertEquals(brokerResponseNative.getNumEntriesScannedInFilter(), 0L);
+      assertNotNull(brokerResponseNative.getProcessingExceptions());
+      assertEquals(brokerResponseNative.getProcessingExceptions().size(), 0);
+      DataSchema dataSchema = new DataSchema(new String[]{"column1", "column5", "column9", "column11"},
+          new DataSchema.ColumnDataType[]{DataSchema.ColumnDataType.INT, DataSchema.ColumnDataType.STRING,
+              DataSchema.ColumnDataType.INT, DataSchema.ColumnDataType.STRING});
+      assertEquals(resultTable.getDataSchema(), dataSchema);
+      List<Object[]> resultRows = resultTable.getRows();
+      int previousColumn1 = Integer.MIN_VALUE;
+      for (Object[] resultRow : resultRows) {
+        assertEquals(resultRow.length, 4);
+        assertEquals((String) resultRow[1], "gFuH");
+        assertTrue((Integer) resultRow[0] >= previousColumn1);
+        previousColumn1 = (Integer) resultRow[0];
+      }
+
+      Object[] firstRow = resultRows.get(0);
+      // Column 11
+      assertEquals((String) firstRow[3], "P");
+    }
+    {
+      // Selection query with supported filters (including forwardIndexDisabled column) and without columns with
+      // forwardIndexDisabled enabled on either segment
+      String query = "SELECT column1, column5, column9, column11 FROM testTable WHERE column6 NOT IN "
+          + "(1689277, 2147458029) ORDER BY column1";
+      BrokerResponseNative brokerResponseNative = getBrokerResponse(query);
+      assertTrue(brokerResponseNative.getProcessingExceptions() == null
+          || brokerResponseNative.getProcessingExceptions().size() == 0);
+      ResultTable resultTable = brokerResponseNative.getResultTable();
+      assertEquals(brokerResponseNative.getNumRowsResultSet(), 10);
+      assertEquals(brokerResponseNative.getTotalDocs(), 120_000L);
+      assertEquals(brokerResponseNative.getNumDocsScanned(), 119_980L);
+      assertEquals(brokerResponseNative.getNumSegmentsProcessed(), 4L);
+      assertEquals(brokerResponseNative.getNumSegmentsMatched(), 4L);
+      assertEquals(brokerResponseNative.getNumEntriesScannedPostFilter(), 120_100L);
+      assertEquals(brokerResponseNative.getNumEntriesScannedInFilter(), 0L);
+      assertNotNull(brokerResponseNative.getProcessingExceptions());
+      assertEquals(brokerResponseNative.getProcessingExceptions().size(), 0);
+      DataSchema dataSchema = new DataSchema(new String[]{"column1", "column5", "column9", "column11"},
+          new DataSchema.ColumnDataType[]{DataSchema.ColumnDataType.INT, DataSchema.ColumnDataType.STRING,
+              DataSchema.ColumnDataType.INT, DataSchema.ColumnDataType.STRING});
+      assertEquals(resultTable.getDataSchema(), dataSchema);
+      List<Object[]> resultRows = resultTable.getRows();
+      int previousColumn1 = Integer.MIN_VALUE;
+      for (Object[] resultRow : resultRows) {
+        assertEquals(resultRow.length, 4);
+        assertEquals((String) resultRow[1], "gFuH");
+        assertTrue((Integer) resultRow[0] >= previousColumn1);
+        previousColumn1 = (Integer) resultRow[0];
+      }
+
+      Object[] firstRow = resultRows.get(0);
+      // Column 11
+      assertEquals((String) firstRow[3], "o");
+    }
+    {
+      // Query with literal only in SELECT
+      String query = "SELECT 'marvin' from testTable ORDER BY column1";
+      BrokerResponseNative brokerResponseNative = getBrokerResponse(query);
+      assertTrue(brokerResponseNative.getProcessingExceptions() == null
+          || brokerResponseNative.getProcessingExceptions().size() == 0);
+      ResultTable resultTable = brokerResponseNative.getResultTable();
+      assertEquals(brokerResponseNative.getNumRowsResultSet(), 10);
+      assertEquals(brokerResponseNative.getTotalDocs(), 120_000L);
+      assertEquals(brokerResponseNative.getNumDocsScanned(), 120_000L);
+      assertEquals(brokerResponseNative.getNumSegmentsProcessed(), 4L);
+      assertEquals(brokerResponseNative.getNumSegmentsMatched(), 4L);
+      assertEquals(brokerResponseNative.getNumEntriesScannedPostFilter(), 120_000L);
+      assertEquals(brokerResponseNative.getNumEntriesScannedInFilter(), 0L);
+      assertNotNull(brokerResponseNative.getProcessingExceptions());
+      assertEquals(brokerResponseNative.getProcessingExceptions().size(), 0);
+      assertEquals(resultTable.getDataSchema(), new DataSchema(new String[]{"'marvin'"},
+          new DataSchema.ColumnDataType[]{DataSchema.ColumnDataType.STRING}));
+      List<Object[]> resultRows = resultTable.getRows();
+      for (Object[] resultRow : resultRows) {
+        assertEquals(resultRow.length, 1);
+        assertEquals((String) resultRow[0], "marvin");
+      }
+    }
+    {
+      // Selection query with '<' filter on a forwardIndexDisabled column with range index available
+      String query = "SELECT column1, column5, column9, column11 FROM testTable WHERE column6 < 2147458029 AND "
+          + "column6 > 1699000 ORDER BY column1";
+      BrokerResponseNative brokerResponseNative = getBrokerResponse(query);
+      assertTrue(brokerResponseNative.getProcessingExceptions() == null
+          || brokerResponseNative.getProcessingExceptions().size() == 0);
+      ResultTable resultTable = brokerResponseNative.getResultTable();
+      assertEquals(brokerResponseNative.getNumRowsResultSet(), 10);
+      assertEquals(brokerResponseNative.getTotalDocs(), 120_000L);
+      assertEquals(brokerResponseNative.getNumDocsScanned(), 119_980L);
+      assertEquals(brokerResponseNative.getNumSegmentsProcessed(), 4L);
+      assertEquals(brokerResponseNative.getNumSegmentsMatched(), 4L);
+      assertEquals(brokerResponseNative.getNumEntriesScannedPostFilter(), 120_100L);
+      assertEquals(brokerResponseNative.getNumEntriesScannedInFilter(), 0L);
+      assertNotNull(brokerResponseNative.getProcessingExceptions());
+      assertEquals(brokerResponseNative.getProcessingExceptions().size(), 0);
+      DataSchema dataSchema = new DataSchema(new String[]{"column1", "column5", "column9", "column11"},
+          new DataSchema.ColumnDataType[]{DataSchema.ColumnDataType.INT, DataSchema.ColumnDataType.STRING,
+              DataSchema.ColumnDataType.INT, DataSchema.ColumnDataType.STRING});
+      assertEquals(resultTable.getDataSchema(), dataSchema);
+      List<Object[]> resultRows = resultTable.getRows();
+      int previousColumn1 = Integer.MIN_VALUE;
+      for (Object[] resultRow : resultRows) {
+        assertEquals(resultRow.length, 4);
+        assertEquals((String) resultRow[1], "gFuH");
+        assertTrue((Integer) resultRow[0] >= previousColumn1);
+        previousColumn1 = (Integer) resultRow[0];
+      }
+
+      Object[] firstRow = resultRows.get(0);
+      // Column 11
+      assertEquals((String) firstRow[3], "o");
+    }
+    {
+      // Selection query with '>=' filter on a forwardIndexDisabled column without range index available
+      String query = "SELECT column1, column5, column9, column11 FROM testTable WHERE column7 >= 676000";
+      try {
+        getBrokerResponse(query);
+        Assert.fail("Query should fail since forwardIndexDisabled on a range query column without range index");
+      } catch (IllegalStateException e) {
+        assertTrue(e.getMessage().contains("Forward index disabled for column:")
+            && e.getMessage().contains("creating ScanDocIdSet unsupported!"));
+      }
+    }
+    {
+      // Select query with a filter on a column which doesn't have forwardIndexDisabled enabled
+      String query = "SELECT column1, column5, column9 from testTable WHERE column9 < 50000 ORDER BY column1";
+      BrokerResponseNative brokerResponseNative = getBrokerResponse(query);
+      assertTrue(brokerResponseNative.getProcessingExceptions() == null
+          || brokerResponseNative.getProcessingExceptions().size() == 0);
+      ResultTable resultTable = brokerResponseNative.getResultTable();
+      assertEquals(brokerResponseNative.getNumRowsResultSet(), 4);
+      assertEquals(brokerResponseNative.getTotalDocs(), 120_000L);
+      assertEquals(brokerResponseNative.getNumDocsScanned(), 4L);
+      assertEquals(brokerResponseNative.getNumSegmentsProcessed(), 4L);
+      assertEquals(brokerResponseNative.getNumSegmentsMatched(), 4L);
+      assertEquals(brokerResponseNative.getNumEntriesScannedPostFilter(), 12L);
+      assertEquals(brokerResponseNative.getNumEntriesScannedInFilter(), 120000L);
+      assertNotNull(brokerResponseNative.getProcessingExceptions());
+      assertEquals(brokerResponseNative.getProcessingExceptions().size(), 0);
+      assertEquals(resultTable.getDataSchema(), new DataSchema(new String[]{"column1", "column5", "column9"},
+          new DataSchema.ColumnDataType[]{
+          DataSchema.ColumnDataType.INT, DataSchema.ColumnDataType.STRING, DataSchema.ColumnDataType.INT
+      }));
+      List<Object[]> resultRows = resultTable.getRows();
+      for (Object[] resultRow : resultRows) {
+        assertEquals(resultRow.length, 3);
+      }
+    }
+    {
+      // Transform function on a filter clause for forwardIndexDisabled column in transform
+      String query = "SELECT column1, column11 from testTable WHERE CONCAT(column6, column9, '-') = '1689277-11270'";
+      BrokerResponseNative brokerResponseNative = getBrokerResponse(query);
+      assertTrue(brokerResponseNative.getProcessingExceptions() != null
+          && brokerResponseNative.getProcessingExceptions().size() > 0);
+    }
+    {
+      // Transform function on a filter clause for a non-forwardIndexDisabled column in transform
+      String query = "SELECT column1, column11 from testTable WHERE CONCAT(column5, column9, '-') = 'gFuH-11270' "
+          + "ORDER BY column1";
+      BrokerResponseNative brokerResponseNative = getBrokerResponse(query);
+      assertTrue(brokerResponseNative.getProcessingExceptions() == null
+          || brokerResponseNative.getProcessingExceptions().size() == 0);
+      ResultTable resultTable = brokerResponseNative.getResultTable();
+      assertEquals(brokerResponseNative.getNumRowsResultSet(), 4);
+      assertEquals(brokerResponseNative.getTotalDocs(), 120_000L);
+      assertEquals(brokerResponseNative.getNumDocsScanned(), 4L);
+      assertEquals(brokerResponseNative.getNumSegmentsProcessed(), 4L);
+      assertEquals(brokerResponseNative.getNumSegmentsMatched(), 4L);
+      assertEquals(brokerResponseNative.getNumEntriesScannedPostFilter(), 8L);
+      assertEquals(brokerResponseNative.getNumEntriesScannedInFilter(), 120000L);
+      assertNotNull(brokerResponseNative.getProcessingExceptions());
+      assertEquals(brokerResponseNative.getProcessingExceptions().size(), 0);
+      assertEquals(resultTable.getDataSchema(), new DataSchema(new String[]{"column1", "column11"},
+          new DataSchema.ColumnDataType[]{
+              DataSchema.ColumnDataType.INT, DataSchema.ColumnDataType.STRING
+          }));
+      List<Object[]> resultRows = resultTable.getRows();
+      assertEquals(resultRows.size(), 4);
+      for (Object[] resultRow : resultRows) {
+        assertEquals(resultRow.length, 2);
+        assertEquals(resultRow[0], 815409257);
+        assertEquals(resultRow[1], "P");
+      }
+    }
+  }
+
+  @Test
+  public void testSelectWithDistinctQueries() {
+    {
+      // Select a mix of forwardIndexDisabled and non-forwardIndexDisabled columns with distinct
+      String query = "SELECT DISTINCT column1, column6, column9 FROM testTable LIMIT 10";
+      try {
+        getBrokerResponse(query);
+        Assert.fail("Query should fail since forwardIndexDisabled on a column in select distinct");
+      } catch (IllegalStateException e) {
+        assertTrue(e.getMessage().contains("Forward index disabled for column:")
+            && e.getMessage().contains("cannot create DataFetcher!"));
+      }
+    }
+    {
+      // Select non-forwardIndexDisabled columns with distinct
+      String query = "SELECT DISTINCT column1, column5, column9 FROM testTable ORDER BY column1 LIMIT 10";
+      BrokerResponseNative brokerResponseNative = getBrokerResponse(query);
+      assertTrue(brokerResponseNative.getProcessingExceptions() == null
+          || brokerResponseNative.getProcessingExceptions().size() == 0);
+      ResultTable resultTable = brokerResponseNative.getResultTable();
+      assertEquals(brokerResponseNative.getNumRowsResultSet(), 10);
+      assertEquals(brokerResponseNative.getTotalDocs(), 120_000L);
+      assertEquals(brokerResponseNative.getNumDocsScanned(), 120_000L);
+      assertEquals(brokerResponseNative.getNumSegmentsProcessed(), 4L);
+      assertEquals(brokerResponseNative.getNumSegmentsMatched(), 4L);
+      assertEquals(brokerResponseNative.getNumEntriesScannedPostFilter(), 360_000L);
+      assertEquals(brokerResponseNative.getNumEntriesScannedInFilter(), 0L);
+      assertNotNull(brokerResponseNative.getProcessingExceptions());
+      assertEquals(brokerResponseNative.getProcessingExceptions().size(), 0);
+      assertEquals(resultTable.getDataSchema(), new DataSchema(new String[]{"column1", "column5", "column9"},
+          new DataSchema.ColumnDataType[]{DataSchema.ColumnDataType.INT, DataSchema.ColumnDataType.STRING,
+              DataSchema.ColumnDataType.INT}));
+      List<Object[]> resultRows = resultTable.getRows();
+      int previousColumn1 = Integer.MIN_VALUE;
+      for (Object[] resultRow : resultRows) {
+        assertEquals(resultRow.length, 3);
+        assertEquals(resultRow[1], "gFuH");
+        assertTrue((Integer) resultRow[0] >= previousColumn1);
+        previousColumn1 = (Integer) resultRow[0];
+      }
+    }
+  }
+
+  @Test
+  public void testSelectWithGroupByOrderByQueries() {
+    {
+      // Select a mix of forwardIndexDisabled and non-forwardIndexDisabled columns with group by order by
+      String query = "SELECT column1, column6 FROM testTable GROUP BY column1, column6 ORDER BY column1, column6 "
+          + " LIMIT 10";
+      try {
+        getBrokerResponse(query);
+        Assert.fail("Query should fail since forwardIndexDisabled on a column in select group by order by");
+      } catch (IllegalStateException e) {
+        assertTrue(e.getMessage().contains("Forward index disabled for column:")
+            && e.getMessage().contains("cannot create DataFetcher!"));
+      }
+    }
+    {
+      // Select forwardIndexDisabled columns with group by order by
+      String query = "SELECT column7, column6 FROM testTable GROUP BY column7, column6 ORDER BY column7, column6 "
+          + " LIMIT 10";
+      try {
+        getBrokerResponse(query);
+        Assert.fail("Query should fail since forwardIndexDisabled on a column in select group by order by");
+      } catch (IllegalStateException e) {
+        assertTrue(e.getMessage().contains("Forward index disabled for column:")
+            && e.getMessage().contains("cannot create DataFetcher!"));
+      }
+    }
+    {
+      // Select non-forwardIndexDisabled columns with group by order by
+      String query = "SELECT column1, column5 FROM testTable GROUP BY column1, column5 ORDER BY column1, column5 "
+          + " LIMIT 10";
+      BrokerResponseNative brokerResponseNative = getBrokerResponse(query);
+      assertTrue(brokerResponseNative.getProcessingExceptions() == null
+          || brokerResponseNative.getProcessingExceptions().size() == 0);
+      ResultTable resultTable = brokerResponseNative.getResultTable();
+      assertEquals(brokerResponseNative.getNumRowsResultSet(), 10);
+      assertEquals(brokerResponseNative.getTotalDocs(), 120_000L);
+      assertEquals(brokerResponseNative.getNumDocsScanned(), 120000L);
+      assertEquals(brokerResponseNative.getNumSegmentsProcessed(), 4L);
+      assertEquals(brokerResponseNative.getNumSegmentsMatched(), 4L);
+      assertEquals(brokerResponseNative.getNumEntriesScannedPostFilter(), 240000L);
+      assertEquals(brokerResponseNative.getNumEntriesScannedInFilter(), 0L);
+      assertNotNull(brokerResponseNative.getProcessingExceptions());
+      assertEquals(brokerResponseNative.getProcessingExceptions().size(), 0);
+      assertEquals(resultTable.getDataSchema(), new DataSchema(new String[]{"column1", "column5"},
+          new DataSchema.ColumnDataType[]{DataSchema.ColumnDataType.INT, DataSchema.ColumnDataType.STRING}));
+      List<Object[]> resultRows = resultTable.getRows();
+      int previousVal = -1;
+      for (Object[] resultRow : resultRows) {
+        assertEquals(resultRow.length, 2);
+        assertEquals(resultRow[1], "gFuH");
+        assertTrue((int) resultRow[0] > previousVal);
+        previousVal = (int) resultRow[0];
+      }
+    }
+    {
+      // Select forwardIndexDisabled columns using transform with group by order by
+      String query = "SELECT CONCAT(column1, column6, '-') FROM testTable GROUP BY CONCAT(column1, column6, '-') "
+          + "ORDER BY CONCAT(column1, column6, '-') LIMIT 10";
+      try {
+        getBrokerResponse(query);
+        Assert.fail("Query should fail since forwardIndexDisabled on a transformed column in group by order by");
+      } catch (IllegalStateException e) {
+        assertTrue(e.getMessage().contains("Forward index disabled for column:")
+            && e.getMessage().contains("cannot create DataFetcher!"));
+      }
+    }
+    {
+      // Select non-forwardIndexDisabled columns using transform with group by order by
+      String query = "SELECT CONCAT(column1, column5, '-') FROM testTable GROUP BY CONCAT(column1, column5, '-') "
+          + "ORDER BY CONCAT(column1, column5, '-') LIMIT 10";
+      BrokerResponseNative brokerResponseNative = getBrokerResponse(query);
+      assertTrue(brokerResponseNative.getProcessingExceptions() == null
+          || brokerResponseNative.getProcessingExceptions().size() == 0);
+      ResultTable resultTable = brokerResponseNative.getResultTable();
+      assertEquals(brokerResponseNative.getNumRowsResultSet(), 10);
+      assertEquals(brokerResponseNative.getTotalDocs(), 120_000L);
+      assertEquals(brokerResponseNative.getNumDocsScanned(), 120000L);
+      assertEquals(brokerResponseNative.getNumSegmentsProcessed(), 4L);
+      assertEquals(brokerResponseNative.getNumSegmentsMatched(), 4L);
+      assertEquals(brokerResponseNative.getNumEntriesScannedPostFilter(), 240000L);
+      assertEquals(brokerResponseNative.getNumEntriesScannedInFilter(), 0L);
+      assertNotNull(brokerResponseNative.getProcessingExceptions());
+      assertEquals(brokerResponseNative.getProcessingExceptions().size(), 0);
+      assertEquals(resultTable.getDataSchema(), new DataSchema(new String[]{"concat(column1,column5,'-')"},
+          new DataSchema.ColumnDataType[]{DataSchema.ColumnDataType.STRING}));
+      List<Object[]> resultRows = resultTable.getRows();
+      for (Object[] resultRow : resultRows) {
+        assertEquals(resultRow.length, 1);
+        assertTrue(resultRow[0].toString().endsWith("-gFuH"));
+      }
+    }
+  }
+
+  @Test
+  public void testSelectWithAggregationQueries() {
+    {
+      // Allowed aggregation functions on forwardIndexDisabled columns
+      String query = "SELECT max(column7), min(column7), count(column7), minmaxrange(column7), "
+          + "distinctcount(column7), distinctcounthll(column6), distinctcountrawhll(column7), "
+          + "distinctcountsmarthll(column6) from testTable";
+      BrokerResponseNative brokerResponseNative = getBrokerResponse(query);
+      assertTrue(brokerResponseNative.getProcessingExceptions() == null
+          || brokerResponseNative.getProcessingExceptions().size() == 0);
+      ResultTable resultTable = brokerResponseNative.getResultTable();
+      assertEquals(brokerResponseNative.getNumRowsResultSet(), 1);
+      assertEquals(brokerResponseNative.getTotalDocs(), 120_000L);
+      assertEquals(brokerResponseNative.getNumDocsScanned(), 120_000L);
+      assertEquals(brokerResponseNative.getNumSegmentsProcessed(), 4L);
+      assertEquals(brokerResponseNative.getNumSegmentsMatched(), 4L);
+      assertEquals(brokerResponseNative.getNumEntriesScannedPostFilter(), 0L);
+      assertEquals(brokerResponseNative.getNumEntriesScannedInFilter(), 0L);
+      assertNotNull(brokerResponseNative.getProcessingExceptions());
+      assertEquals(brokerResponseNative.getProcessingExceptions().size(), 0);
+      assertEquals(resultTable.getDataSchema(), new DataSchema(new String[]{"max(column7)", "min(column7)", "count(*)",
+      "minmaxrange(column7)", "distinctcount(column7)", "distinctcounthll(column6)", "distinctcountrawhll(column7)",
+      "distinctcountsmarthll(column6)"},
+          new DataSchema.ColumnDataType[]{DataSchema.ColumnDataType.DOUBLE, DataSchema.ColumnDataType.DOUBLE,
+              DataSchema.ColumnDataType.LONG, DataSchema.ColumnDataType.DOUBLE, DataSchema.ColumnDataType.INT,
+              DataSchema.ColumnDataType.LONG, DataSchema.ColumnDataType.STRING, DataSchema.ColumnDataType.INT}));
+      List<Object[]> resultRows = resultTable.getRows();
+      for (Object[] resultRow : resultRows) {
+        assertEquals(resultRow.length, 8);
+        assertEquals(resultRow[0], 2.137685743E9);
+        assertEquals(resultRow[1], 675695.0);
+        assertEquals(resultRow[2], 120000L);
+        assertEquals(resultRow[3], 2.137010048E9);
+        assertEquals(resultRow[4], 146);
+        assertEquals(resultRow[5], 695L);
+        assertEquals(resultRow[7], 608);
+      }
+    }
+    {
+      // Not allowed aggregation functions on forwardIndexDisabled columns
+      String query = "SELECT sum(column7), avg(column6) from testTable";
+      try {
+        getBrokerResponse(query);
+        Assert.fail("Query should fail since forwardIndexDisabled on a column in unsupported aggregation query");
+      } catch (IllegalStateException e) {
+        assertTrue(e.getMessage().contains("Forward index disabled for column:")
+            && e.getMessage().contains("cannot create DataFetcher!"));
+      }
+    }
+    {
+      // Allowed aggregation functions on forwardIndexDisabled columns with group by on non-forwardIndexDisabled
+      // column
+      String query = "SELECT column1, max(column6) from testTable GROUP BY column1";
+      try {
+        getBrokerResponse(query);
+        Assert.fail("Query should fail since forwardIndexDisabled on a column in unsupported aggregation query");
+      } catch (IllegalStateException e) {
+        assertTrue(e.getMessage().contains("Forward index disabled for column:")
+            && e.getMessage().contains("cannot create DataFetcher!"));
+      }
+    }
+    {
+      // Allowed aggregation functions on forwardIndexDisabled columns with group by order by on
+      // non-forwardIndexDisabled column
+      String query = "SELECT column1, max(column6) from testTable GROUP BY column1 ORDER BY column1";
+      try {
+        getBrokerResponse(query);
+        Assert.fail("Query should fail since forwardIndexDisabled on a column in unsupported aggregation query");
+      } catch (IllegalStateException e) {
+        assertTrue(e.getMessage().contains("Forward index disabled for column:")
+            && e.getMessage().contains("cannot create DataFetcher!"));
+      }
+    }
+    {
+      // Allowed aggregation functions on non-forwardIndexDisabled columns with group by on non-forwardIndexDisabled
+      // column but order by on allowed aggregation function on forwardIndexDisabled column
+      String query = "SELECT column1, max(column9) from testTable GROUP BY column1 ORDER BY min(column6)";

Review Comment:
   Cover HAVING 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 #9333: Add an option to disable the creation of the forward index for a column

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


##########
pinot-core/src/test/java/org/apache/pinot/queries/ForwardIndexDisabledSingleValueQueriesTest.java:
##########
@@ -0,0 +1,1088 @@
+/**
+ * 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.queries;
+
+import java.io.File;
+import java.net.URL;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.TimeUnit;
+import org.apache.commons.io.FileUtils;
+import org.apache.pinot.common.response.broker.BrokerResponseNative;
+import org.apache.pinot.common.response.broker.ResultTable;
+import org.apache.pinot.common.utils.DataSchema;
+import org.apache.pinot.segment.local.indexsegment.immutable.ImmutableSegmentLoader;
+import org.apache.pinot.segment.local.segment.creator.impl.SegmentIndexCreationDriverImpl;
+import org.apache.pinot.segment.local.segment.index.loader.IndexLoadingConfig;
+import org.apache.pinot.segment.spi.ColumnMetadata;
+import org.apache.pinot.segment.spi.ImmutableSegment;
+import org.apache.pinot.segment.spi.IndexSegment;
+import org.apache.pinot.segment.spi.creator.SegmentGeneratorConfig;
+import org.apache.pinot.segment.spi.creator.SegmentIndexCreationDriver;
+import org.apache.pinot.spi.config.table.FieldConfig;
+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.data.TimeGranularitySpec;
+import org.apache.pinot.spi.utils.ReadMode;
+import org.apache.pinot.spi.utils.builder.TableConfigBuilder;
+import org.testng.Assert;
+import org.testng.annotations.AfterClass;
+import org.testng.annotations.BeforeClass;
+import org.testng.annotations.Test;
+
+import static org.testng.Assert.assertEquals;
+import static org.testng.Assert.assertNotNull;
+import static org.testng.Assert.assertNull;
+import static org.testng.Assert.assertTrue;
+
+
+/**
+ * The <code>ForwardIndexDisabledSingleValueQueriesTest</code> class sets up the index segment for the no forward
+ * index single-value queries test.
+ * <p>There are totally 18 columns, 30000 records inside the original Avro file where 11 columns are selected to build
+ * the index segment. Selected columns information are as following:
+ * <ul>
+ *   ColumnName, FieldType, DataType, Cardinality, IsSorted, HasInvertedIndex, FwdIndexDisabled: S1, S2, HasRangeIndex
+ *   <li>column1, METRIC, INT, 6582, F, F, F, F, F</li>
+ *   <li>column3, METRIC, INT, 21910, F, F, F, F, F</li>
+ *   <li>column5, DIMENSION, STRING, 1, T, F, F, F, F</li>
+ *   <li>column6, DIMENSION, INT, 608, F, T, T, T, T</li>
+ *   <li>column7, DIMENSION, INT, 146, F, T, T, F, F</li>
+ *   <li>column9, DIMENSION, INT, 1737, F, F, F, F, F</li>
+ *   <li>column11, DIMENSION, STRING, 5, F, T, F, F, F</li>
+ *   <li>column12, DIMENSION, STRING, 5, F, F, F, F, F</li>
+ *   <li>column17, METRIC, INT, 24, F, T, F, F, F</li>
+ *   <li>column18, METRIC, INT, 1440, F, T, F, F, F</li>
+ *   <li>daysSinceEpoch, TIME, INT, 2, T, F, F, F, F</li>
+ * </ul>
+ */
+public class ForwardIndexDisabledSingleValueQueriesTest extends BaseQueriesTest {
+  private static final String AVRO_DATA = "data" + File.separator + "test_data-sv.avro";
+  private static final String SEGMENT_NAME_1 = "testTable_126164076_167572857";
+  private static final String SEGMENT_NAME_2 = "testTable_126164076_167572858";
+  private static final File INDEX_DIR = new File(FileUtils.getTempDirectory(),
+      "ForwardIndexDisabledSingleValueQueriesTest");
+
+  private static final String SELECT_STAR_QUERY = "SELECT * FROM testTable";
+
+  // Hard-coded query filter.
+  private static final String FILTER = " WHERE column1 > 100000000"
+      + " AND column3 BETWEEN 20000000 AND 1000000000"
+      + " AND column5 = 'gFuH'"
+      + " AND (column6 < 500000000 OR column11 NOT IN ('t', 'P'))"
+      + " AND daysSinceEpoch = 126164076";
+
+  private IndexSegment _indexSegment;
+  // Contains 2 index segments, one with 2 columns with forward index disabled, and the other with just 1.
+  private List<IndexSegment> _indexSegments;
+
+  private TableConfig _tableConfig;
+  private List<String> _invertedIndexColumns;
+  private List<String> _forwardIndexDisabledColumns;
+
+  @BeforeClass
+  public void buildAndLoadSegment()
+      throws Exception {
+    FileUtils.deleteQuietly(INDEX_DIR);
+
+    // Get resource file path.
+    URL resource = getClass().getClassLoader().getResource(AVRO_DATA);
+    assertNotNull(resource);
+    String filePath = resource.getFile();
+
+    // Build the segment schema.
+    Schema schema = new Schema.SchemaBuilder().setSchemaName("testTable").addMetric("column1", FieldSpec.DataType.INT)
+        .addMetric("column3", FieldSpec.DataType.INT).addSingleValueDimension("column5", FieldSpec.DataType.STRING)
+        .addSingleValueDimension("column6", FieldSpec.DataType.INT)
+        .addSingleValueDimension("column7", FieldSpec.DataType.INT)
+        .addSingleValueDimension("column9", FieldSpec.DataType.INT)
+        .addSingleValueDimension("column11", FieldSpec.DataType.STRING)
+        .addSingleValueDimension("column12", FieldSpec.DataType.STRING).addMetric("column17", FieldSpec.DataType.INT)
+        .addMetric("column18", FieldSpec.DataType.INT)
+        .addTime(new TimeGranularitySpec(FieldSpec.DataType.INT, TimeUnit.DAYS, "daysSinceEpoch"), null).build();
+
+    createSegment(filePath, SEGMENT_NAME_1, schema);
+    createSegment(filePath, SEGMENT_NAME_2, schema);
+
+    ImmutableSegment immutableSegment1 = loadSegmentWithMetadataChecks(SEGMENT_NAME_1);
+    ImmutableSegment immutableSegment2 = loadSegmentWithMetadataChecks(SEGMENT_NAME_2);
+
+    _indexSegment = immutableSegment1;
+    _indexSegments = Arrays.asList(immutableSegment1, immutableSegment2);
+  }
+
+  private void createSegment(String filePath, String segmentName, Schema schema)
+      throws Exception {
+    // Create field configs for the no forward index columns
+    List<FieldConfig> fieldConfigList = new ArrayList<>();
+    fieldConfigList.add(new FieldConfig("column6", FieldConfig.EncodingType.DICTIONARY, Collections.emptyList(), null,
+        Collections.singletonMap(FieldConfig.FORWARD_INDEX_DISABLED, Boolean.TRUE.toString())));
+    if (segmentName.equals(SEGMENT_NAME_1)) {
+      fieldConfigList.add(new FieldConfig("column7", FieldConfig.EncodingType.DICTIONARY, Collections.emptyList(),
+          null, Collections.singletonMap(FieldConfig.FORWARD_INDEX_DISABLED, Boolean.TRUE.toString())));
+
+      // Build table config based on segment 1 as it contains both columns under no forward index
+      _tableConfig =
+          new TableConfigBuilder(TableType.OFFLINE).setTableName("testTable").setTimeColumnName("daysSinceEpoch")
+              .setFieldConfigList(fieldConfigList).setRangeIndexColumns(Arrays.asList("column6")).build();
+    }
+
+    // Create the segment generator config.
+    SegmentGeneratorConfig segmentGeneratorConfig = new SegmentGeneratorConfig(_tableConfig, schema);
+    segmentGeneratorConfig.setInputFilePath(filePath);
+    segmentGeneratorConfig.setTableName("testTable");
+    segmentGeneratorConfig.setOutDir(INDEX_DIR.getAbsolutePath());
+    segmentGeneratorConfig.setSegmentName(segmentName);
+    // The segment generation code in SegmentColumnarIndexCreator will throw
+    // exception if start and end time in time column are not in acceptable
+    // range. For this test, we first need to fix the input avro data
+    // to have the time column values in allowed range. Until then, the check
+    // is explicitly disabled
+    segmentGeneratorConfig.setSkipTimeValueCheck(true);
+    _invertedIndexColumns = Arrays.asList("column6", "column7", "column11", "column17", "column18");
+    segmentGeneratorConfig.setInvertedIndexCreationColumns(_invertedIndexColumns);
+
+    _forwardIndexDisabledColumns = new ArrayList<>(Arrays.asList("column6", "column7"));
+    segmentGeneratorConfig.setForwardIndexDisabledColumns(_forwardIndexDisabledColumns);
+    segmentGeneratorConfig.setRangeIndexCreationColumns(Arrays.asList("column6"));
+
+    // Build the index segment.
+    SegmentIndexCreationDriver driver = new SegmentIndexCreationDriverImpl();
+    driver.init(segmentGeneratorConfig);
+    driver.build();
+  }
+
+  private ImmutableSegment loadSegmentWithMetadataChecks(String segmentName)
+      throws Exception {
+    IndexLoadingConfig indexLoadingConfig = new IndexLoadingConfig();
+    indexLoadingConfig.setTableConfig(_tableConfig);
+    indexLoadingConfig.setInvertedIndexColumns(new HashSet<>(_invertedIndexColumns));
+    indexLoadingConfig.setForwardIndexDisabledColumns(new HashSet<>(_forwardIndexDisabledColumns));
+    indexLoadingConfig.setRangeIndexColumns(new HashSet<>(Arrays.asList("column6")));
+    indexLoadingConfig.setReadMode(ReadMode.heap);
+
+    ImmutableSegment immutableSegment = ImmutableSegmentLoader.load(new File(INDEX_DIR, segmentName),
+        indexLoadingConfig);
+
+    Map<String, ColumnMetadata> columnMetadataMap1 = immutableSegment.getSegmentMetadata().getColumnMetadataMap();
+    columnMetadataMap1.forEach((column, metadata) -> {
+      if (column.equals("column6") || column.equals("column7")) {
+        assertTrue(metadata.hasDictionary());
+        assertTrue(metadata.isSingleValue());
+        assertNull(immutableSegment.getForwardIndex(column));
+      } else {
+        assertNotNull(immutableSegment.getForwardIndex(column));
+      }
+    });
+
+    return immutableSegment;
+  }
+
+  @AfterClass
+  public void deleteAndDestroySegment() {
+    FileUtils.deleteQuietly(INDEX_DIR);
+    _indexSegments.forEach((IndexSegment::destroy));
+  }
+
+  @Override
+  protected String getFilter() {
+    return FILTER;
+  }
+
+  @Override
+  protected IndexSegment getIndexSegment() {
+    return _indexSegment;
+  }
+
+  @Override
+  protected List<IndexSegment> getIndexSegments() {
+    return _indexSegments;
+  }
+
+  @Test
+  public void testSelectStarQueries() {
+    // Select * without any filters
+    try {
+      getBrokerResponse(SELECT_STAR_QUERY);
+      Assert.fail("Select * query should fail since forwardIndexDisabled on a select column");
+    } catch (IllegalStateException e) {
+      assertTrue(e.getMessage().contains("Forward index disabled for column:")
+          && e.getMessage().contains("cannot create DataFetcher!"));
+    }
+
+    // Select * with filters
+    try {
+      getBrokerResponse(SELECT_STAR_QUERY + FILTER);
+      Assert.fail("Select * query should fail since forwardIndexDisabled on a select column");
+    } catch (IllegalStateException e) {
+      assertTrue(e.getMessage().contains("Forward index disabled for column:")
+          && e.getMessage().contains("cannot create DataFetcher!"));
+    }
+  }
+
+  @Test
+  public void testSelectQueries() {
+    {
+      // Selection query without filters including a column with forwardIndexDisabled enabled on both segments
+      String query = "SELECT column1, column5, column6, column9, column11 FROM testTable";
+      try {
+        getBrokerResponse(query);
+        Assert.fail("Query should fail since forwardIndexDisabled on a select column");
+      } catch (IllegalStateException e) {
+        assertTrue(e.getMessage().contains("Forward index disabled for column:")
+            && e.getMessage().contains("cannot create DataFetcher!"));
+      }
+    }
+    {
+      // Selection query without filters including a column with forwardIndexDisabled enabled on one segment
+      String query = "SELECT column1, column5, column7, column9, column11 FROM testTable";
+      try {
+        getBrokerResponse(query);
+        Assert.fail("Query should fail since forwardIndexDisabled on a select column");
+      } catch (IllegalStateException e) {
+        assertTrue(e.getMessage().contains("Forward index disabled for column:")
+            && e.getMessage().contains("cannot create DataFetcher!"));
+      }
+    }
+    {
+      // Selection query with filters including a column with forwardIndexDisabled enabled on both segments
+      String query = "SELECT column1, column5, column6, column9, column11 FROM testTable WHERE column6 = 2147458029";

Review Comment:
   Can we add a test where `column6` is not in select list but is used in where clause via inverted index `= / IN`



##########
pinot-core/src/test/java/org/apache/pinot/queries/ForwardIndexDisabledSingleValueQueriesTest.java:
##########
@@ -0,0 +1,1088 @@
+/**
+ * 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.queries;
+
+import java.io.File;
+import java.net.URL;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.TimeUnit;
+import org.apache.commons.io.FileUtils;
+import org.apache.pinot.common.response.broker.BrokerResponseNative;
+import org.apache.pinot.common.response.broker.ResultTable;
+import org.apache.pinot.common.utils.DataSchema;
+import org.apache.pinot.segment.local.indexsegment.immutable.ImmutableSegmentLoader;
+import org.apache.pinot.segment.local.segment.creator.impl.SegmentIndexCreationDriverImpl;
+import org.apache.pinot.segment.local.segment.index.loader.IndexLoadingConfig;
+import org.apache.pinot.segment.spi.ColumnMetadata;
+import org.apache.pinot.segment.spi.ImmutableSegment;
+import org.apache.pinot.segment.spi.IndexSegment;
+import org.apache.pinot.segment.spi.creator.SegmentGeneratorConfig;
+import org.apache.pinot.segment.spi.creator.SegmentIndexCreationDriver;
+import org.apache.pinot.spi.config.table.FieldConfig;
+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.data.TimeGranularitySpec;
+import org.apache.pinot.spi.utils.ReadMode;
+import org.apache.pinot.spi.utils.builder.TableConfigBuilder;
+import org.testng.Assert;
+import org.testng.annotations.AfterClass;
+import org.testng.annotations.BeforeClass;
+import org.testng.annotations.Test;
+
+import static org.testng.Assert.assertEquals;
+import static org.testng.Assert.assertNotNull;
+import static org.testng.Assert.assertNull;
+import static org.testng.Assert.assertTrue;
+
+
+/**
+ * The <code>ForwardIndexDisabledSingleValueQueriesTest</code> class sets up the index segment for the no forward
+ * index single-value queries test.
+ * <p>There are totally 18 columns, 30000 records inside the original Avro file where 11 columns are selected to build
+ * the index segment. Selected columns information are as following:
+ * <ul>
+ *   ColumnName, FieldType, DataType, Cardinality, IsSorted, HasInvertedIndex, FwdIndexDisabled: S1, S2, HasRangeIndex
+ *   <li>column1, METRIC, INT, 6582, F, F, F, F, F</li>
+ *   <li>column3, METRIC, INT, 21910, F, F, F, F, F</li>
+ *   <li>column5, DIMENSION, STRING, 1, T, F, F, F, F</li>
+ *   <li>column6, DIMENSION, INT, 608, F, T, T, T, T</li>
+ *   <li>column7, DIMENSION, INT, 146, F, T, T, F, F</li>
+ *   <li>column9, DIMENSION, INT, 1737, F, F, F, F, F</li>
+ *   <li>column11, DIMENSION, STRING, 5, F, T, F, F, F</li>
+ *   <li>column12, DIMENSION, STRING, 5, F, F, F, F, F</li>
+ *   <li>column17, METRIC, INT, 24, F, T, F, F, F</li>
+ *   <li>column18, METRIC, INT, 1440, F, T, F, F, F</li>
+ *   <li>daysSinceEpoch, TIME, INT, 2, T, F, F, F, F</li>
+ * </ul>
+ */
+public class ForwardIndexDisabledSingleValueQueriesTest extends BaseQueriesTest {
+  private static final String AVRO_DATA = "data" + File.separator + "test_data-sv.avro";
+  private static final String SEGMENT_NAME_1 = "testTable_126164076_167572857";
+  private static final String SEGMENT_NAME_2 = "testTable_126164076_167572858";
+  private static final File INDEX_DIR = new File(FileUtils.getTempDirectory(),
+      "ForwardIndexDisabledSingleValueQueriesTest");
+
+  private static final String SELECT_STAR_QUERY = "SELECT * FROM testTable";
+
+  // Hard-coded query filter.
+  private static final String FILTER = " WHERE column1 > 100000000"
+      + " AND column3 BETWEEN 20000000 AND 1000000000"
+      + " AND column5 = 'gFuH'"
+      + " AND (column6 < 500000000 OR column11 NOT IN ('t', 'P'))"
+      + " AND daysSinceEpoch = 126164076";
+
+  private IndexSegment _indexSegment;
+  // Contains 2 index segments, one with 2 columns with forward index disabled, and the other with just 1.
+  private List<IndexSegment> _indexSegments;
+
+  private TableConfig _tableConfig;
+  private List<String> _invertedIndexColumns;
+  private List<String> _forwardIndexDisabledColumns;
+
+  @BeforeClass
+  public void buildAndLoadSegment()
+      throws Exception {
+    FileUtils.deleteQuietly(INDEX_DIR);
+
+    // Get resource file path.
+    URL resource = getClass().getClassLoader().getResource(AVRO_DATA);
+    assertNotNull(resource);
+    String filePath = resource.getFile();
+
+    // Build the segment schema.
+    Schema schema = new Schema.SchemaBuilder().setSchemaName("testTable").addMetric("column1", FieldSpec.DataType.INT)
+        .addMetric("column3", FieldSpec.DataType.INT).addSingleValueDimension("column5", FieldSpec.DataType.STRING)
+        .addSingleValueDimension("column6", FieldSpec.DataType.INT)
+        .addSingleValueDimension("column7", FieldSpec.DataType.INT)
+        .addSingleValueDimension("column9", FieldSpec.DataType.INT)
+        .addSingleValueDimension("column11", FieldSpec.DataType.STRING)
+        .addSingleValueDimension("column12", FieldSpec.DataType.STRING).addMetric("column17", FieldSpec.DataType.INT)
+        .addMetric("column18", FieldSpec.DataType.INT)
+        .addTime(new TimeGranularitySpec(FieldSpec.DataType.INT, TimeUnit.DAYS, "daysSinceEpoch"), null).build();
+
+    createSegment(filePath, SEGMENT_NAME_1, schema);
+    createSegment(filePath, SEGMENT_NAME_2, schema);
+
+    ImmutableSegment immutableSegment1 = loadSegmentWithMetadataChecks(SEGMENT_NAME_1);
+    ImmutableSegment immutableSegment2 = loadSegmentWithMetadataChecks(SEGMENT_NAME_2);
+
+    _indexSegment = immutableSegment1;
+    _indexSegments = Arrays.asList(immutableSegment1, immutableSegment2);
+  }
+
+  private void createSegment(String filePath, String segmentName, Schema schema)
+      throws Exception {
+    // Create field configs for the no forward index columns
+    List<FieldConfig> fieldConfigList = new ArrayList<>();
+    fieldConfigList.add(new FieldConfig("column6", FieldConfig.EncodingType.DICTIONARY, Collections.emptyList(), null,
+        Collections.singletonMap(FieldConfig.FORWARD_INDEX_DISABLED, Boolean.TRUE.toString())));
+    if (segmentName.equals(SEGMENT_NAME_1)) {
+      fieldConfigList.add(new FieldConfig("column7", FieldConfig.EncodingType.DICTIONARY, Collections.emptyList(),
+          null, Collections.singletonMap(FieldConfig.FORWARD_INDEX_DISABLED, Boolean.TRUE.toString())));
+
+      // Build table config based on segment 1 as it contains both columns under no forward index
+      _tableConfig =
+          new TableConfigBuilder(TableType.OFFLINE).setTableName("testTable").setTimeColumnName("daysSinceEpoch")
+              .setFieldConfigList(fieldConfigList).setRangeIndexColumns(Arrays.asList("column6")).build();
+    }
+
+    // Create the segment generator config.
+    SegmentGeneratorConfig segmentGeneratorConfig = new SegmentGeneratorConfig(_tableConfig, schema);
+    segmentGeneratorConfig.setInputFilePath(filePath);
+    segmentGeneratorConfig.setTableName("testTable");
+    segmentGeneratorConfig.setOutDir(INDEX_DIR.getAbsolutePath());
+    segmentGeneratorConfig.setSegmentName(segmentName);
+    // The segment generation code in SegmentColumnarIndexCreator will throw
+    // exception if start and end time in time column are not in acceptable
+    // range. For this test, we first need to fix the input avro data
+    // to have the time column values in allowed range. Until then, the check
+    // is explicitly disabled
+    segmentGeneratorConfig.setSkipTimeValueCheck(true);
+    _invertedIndexColumns = Arrays.asList("column6", "column7", "column11", "column17", "column18");
+    segmentGeneratorConfig.setInvertedIndexCreationColumns(_invertedIndexColumns);
+
+    _forwardIndexDisabledColumns = new ArrayList<>(Arrays.asList("column6", "column7"));
+    segmentGeneratorConfig.setForwardIndexDisabledColumns(_forwardIndexDisabledColumns);
+    segmentGeneratorConfig.setRangeIndexCreationColumns(Arrays.asList("column6"));
+
+    // Build the index segment.
+    SegmentIndexCreationDriver driver = new SegmentIndexCreationDriverImpl();
+    driver.init(segmentGeneratorConfig);
+    driver.build();
+  }
+
+  private ImmutableSegment loadSegmentWithMetadataChecks(String segmentName)
+      throws Exception {
+    IndexLoadingConfig indexLoadingConfig = new IndexLoadingConfig();
+    indexLoadingConfig.setTableConfig(_tableConfig);
+    indexLoadingConfig.setInvertedIndexColumns(new HashSet<>(_invertedIndexColumns));
+    indexLoadingConfig.setForwardIndexDisabledColumns(new HashSet<>(_forwardIndexDisabledColumns));
+    indexLoadingConfig.setRangeIndexColumns(new HashSet<>(Arrays.asList("column6")));
+    indexLoadingConfig.setReadMode(ReadMode.heap);
+
+    ImmutableSegment immutableSegment = ImmutableSegmentLoader.load(new File(INDEX_DIR, segmentName),
+        indexLoadingConfig);
+
+    Map<String, ColumnMetadata> columnMetadataMap1 = immutableSegment.getSegmentMetadata().getColumnMetadataMap();
+    columnMetadataMap1.forEach((column, metadata) -> {
+      if (column.equals("column6") || column.equals("column7")) {
+        assertTrue(metadata.hasDictionary());
+        assertTrue(metadata.isSingleValue());
+        assertNull(immutableSegment.getForwardIndex(column));
+      } else {
+        assertNotNull(immutableSegment.getForwardIndex(column));
+      }
+    });
+
+    return immutableSegment;
+  }
+
+  @AfterClass
+  public void deleteAndDestroySegment() {
+    FileUtils.deleteQuietly(INDEX_DIR);
+    _indexSegments.forEach((IndexSegment::destroy));
+  }
+
+  @Override
+  protected String getFilter() {
+    return FILTER;
+  }
+
+  @Override
+  protected IndexSegment getIndexSegment() {
+    return _indexSegment;
+  }
+
+  @Override
+  protected List<IndexSegment> getIndexSegments() {
+    return _indexSegments;
+  }
+
+  @Test
+  public void testSelectStarQueries() {
+    // Select * without any filters
+    try {
+      getBrokerResponse(SELECT_STAR_QUERY);
+      Assert.fail("Select * query should fail since forwardIndexDisabled on a select column");
+    } catch (IllegalStateException e) {
+      assertTrue(e.getMessage().contains("Forward index disabled for column:")
+          && e.getMessage().contains("cannot create DataFetcher!"));
+    }
+
+    // Select * with filters
+    try {
+      getBrokerResponse(SELECT_STAR_QUERY + FILTER);
+      Assert.fail("Select * query should fail since forwardIndexDisabled on a select column");
+    } catch (IllegalStateException e) {
+      assertTrue(e.getMessage().contains("Forward index disabled for column:")
+          && e.getMessage().contains("cannot create DataFetcher!"));
+    }
+  }
+
+  @Test
+  public void testSelectQueries() {
+    {
+      // Selection query without filters including a column with forwardIndexDisabled enabled on both segments
+      String query = "SELECT column1, column5, column6, column9, column11 FROM testTable";
+      try {
+        getBrokerResponse(query);
+        Assert.fail("Query should fail since forwardIndexDisabled on a select column");
+      } catch (IllegalStateException e) {
+        assertTrue(e.getMessage().contains("Forward index disabled for column:")
+            && e.getMessage().contains("cannot create DataFetcher!"));
+      }
+    }
+    {
+      // Selection query without filters including a column with forwardIndexDisabled enabled on one segment
+      String query = "SELECT column1, column5, column7, column9, column11 FROM testTable";
+      try {
+        getBrokerResponse(query);
+        Assert.fail("Query should fail since forwardIndexDisabled on a select column");
+      } catch (IllegalStateException e) {
+        assertTrue(e.getMessage().contains("Forward index disabled for column:")
+            && e.getMessage().contains("cannot create DataFetcher!"));
+      }
+    }
+    {
+      // Selection query with filters including a column with forwardIndexDisabled enabled on both segments
+      String query = "SELECT column1, column5, column6, column9, column11 FROM testTable WHERE column6 = 2147458029";

Review Comment:
   Can we add a test where `column6` is not in select list but is used in where clause via inverted index `= / IN`. The query should work



-- 
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 #9333: Add an option to disable the creation of the forward index for a column

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


##########
pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/loader/defaultcolumn/BaseDefaultColumnHandler.java:
##########
@@ -363,6 +379,12 @@ protected boolean createColumnV1Indices(String column)
                   argument);
               return false;
             }
+            // TODO: Support creation of derived columns from forward index disabled columns
+            if (!_segmentWriter.hasIndexFor(argument, ColumnIndexType.FORWARD_INDEX)) {
+              LOGGER.warn("Skip creating derived column: {} because argument: {} does not have a forward index", column,

Review Comment:
   Throwing error is better to force the user to not create a `derivedColumn` from `noForwardIndex` column until reload code path supports everything (including derived column handling possibly) ?



-- 
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 #9333: Add an option to disable the creation of the forward index for a column

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


##########
pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/loader/defaultcolumn/BaseDefaultColumnHandler.java:
##########
@@ -388,8 +416,40 @@ protected boolean createColumnV1Indices(String column)
     return true;
   }
 
+  /**
+   * Validates the compatibility of the indexes if the column has the forward index disabled. Throws exceptions due to
+   * compatibility mismatch. The checks performed are:
+   *     - Validate dictionary is enabled.
+   *     - Validate inverted index is enabled.
+   *     - Validate that either no range index exists for column or the range index version is at least 2 and isn't a
+   *       multi-value column (since multi-value defaults to index v1).
+   */
+  protected void validateForwardIndexDisabledConfigsIfPresent(String column, boolean forwardIndexDisabled) {

Review Comment:
   All table config changes must go through controller and it is expected to validate the changed config



-- 
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 #9333: Add an option to disable the creation of the forward index for a column

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


##########
pinot-core/src/test/java/org/apache/pinot/queries/ForwardIndexDisabledSingleValueQueriesTest.java:
##########
@@ -0,0 +1,1088 @@
+/**
+ * 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.queries;
+
+import java.io.File;
+import java.net.URL;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.TimeUnit;
+import org.apache.commons.io.FileUtils;
+import org.apache.pinot.common.response.broker.BrokerResponseNative;
+import org.apache.pinot.common.response.broker.ResultTable;
+import org.apache.pinot.common.utils.DataSchema;
+import org.apache.pinot.segment.local.indexsegment.immutable.ImmutableSegmentLoader;
+import org.apache.pinot.segment.local.segment.creator.impl.SegmentIndexCreationDriverImpl;
+import org.apache.pinot.segment.local.segment.index.loader.IndexLoadingConfig;
+import org.apache.pinot.segment.spi.ColumnMetadata;
+import org.apache.pinot.segment.spi.ImmutableSegment;
+import org.apache.pinot.segment.spi.IndexSegment;
+import org.apache.pinot.segment.spi.creator.SegmentGeneratorConfig;
+import org.apache.pinot.segment.spi.creator.SegmentIndexCreationDriver;
+import org.apache.pinot.spi.config.table.FieldConfig;
+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.data.TimeGranularitySpec;
+import org.apache.pinot.spi.utils.ReadMode;
+import org.apache.pinot.spi.utils.builder.TableConfigBuilder;
+import org.testng.Assert;
+import org.testng.annotations.AfterClass;
+import org.testng.annotations.BeforeClass;
+import org.testng.annotations.Test;
+
+import static org.testng.Assert.assertEquals;
+import static org.testng.Assert.assertNotNull;
+import static org.testng.Assert.assertNull;
+import static org.testng.Assert.assertTrue;
+
+
+/**
+ * The <code>ForwardIndexDisabledSingleValueQueriesTest</code> class sets up the index segment for the no forward
+ * index single-value queries test.
+ * <p>There are totally 18 columns, 30000 records inside the original Avro file where 11 columns are selected to build
+ * the index segment. Selected columns information are as following:
+ * <ul>
+ *   ColumnName, FieldType, DataType, Cardinality, IsSorted, HasInvertedIndex, FwdIndexDisabled: S1, S2, HasRangeIndex
+ *   <li>column1, METRIC, INT, 6582, F, F, F, F, F</li>
+ *   <li>column3, METRIC, INT, 21910, F, F, F, F, F</li>
+ *   <li>column5, DIMENSION, STRING, 1, T, F, F, F, F</li>
+ *   <li>column6, DIMENSION, INT, 608, F, T, T, T, T</li>
+ *   <li>column7, DIMENSION, INT, 146, F, T, T, F, F</li>
+ *   <li>column9, DIMENSION, INT, 1737, F, F, F, F, F</li>
+ *   <li>column11, DIMENSION, STRING, 5, F, T, F, F, F</li>
+ *   <li>column12, DIMENSION, STRING, 5, F, F, F, F, F</li>
+ *   <li>column17, METRIC, INT, 24, F, T, F, F, F</li>
+ *   <li>column18, METRIC, INT, 1440, F, T, F, F, F</li>
+ *   <li>daysSinceEpoch, TIME, INT, 2, T, F, F, F, F</li>
+ * </ul>
+ */
+public class ForwardIndexDisabledSingleValueQueriesTest extends BaseQueriesTest {
+  private static final String AVRO_DATA = "data" + File.separator + "test_data-sv.avro";
+  private static final String SEGMENT_NAME_1 = "testTable_126164076_167572857";
+  private static final String SEGMENT_NAME_2 = "testTable_126164076_167572858";
+  private static final File INDEX_DIR = new File(FileUtils.getTempDirectory(),
+      "ForwardIndexDisabledSingleValueQueriesTest");
+
+  private static final String SELECT_STAR_QUERY = "SELECT * FROM testTable";
+
+  // Hard-coded query filter.
+  private static final String FILTER = " WHERE column1 > 100000000"
+      + " AND column3 BETWEEN 20000000 AND 1000000000"
+      + " AND column5 = 'gFuH'"
+      + " AND (column6 < 500000000 OR column11 NOT IN ('t', 'P'))"
+      + " AND daysSinceEpoch = 126164076";
+
+  private IndexSegment _indexSegment;
+  // Contains 2 index segments, one with 2 columns with forward index disabled, and the other with just 1.
+  private List<IndexSegment> _indexSegments;
+
+  private TableConfig _tableConfig;
+  private List<String> _invertedIndexColumns;
+  private List<String> _forwardIndexDisabledColumns;
+
+  @BeforeClass
+  public void buildAndLoadSegment()
+      throws Exception {
+    FileUtils.deleteQuietly(INDEX_DIR);
+
+    // Get resource file path.
+    URL resource = getClass().getClassLoader().getResource(AVRO_DATA);
+    assertNotNull(resource);
+    String filePath = resource.getFile();
+
+    // Build the segment schema.
+    Schema schema = new Schema.SchemaBuilder().setSchemaName("testTable").addMetric("column1", FieldSpec.DataType.INT)
+        .addMetric("column3", FieldSpec.DataType.INT).addSingleValueDimension("column5", FieldSpec.DataType.STRING)
+        .addSingleValueDimension("column6", FieldSpec.DataType.INT)
+        .addSingleValueDimension("column7", FieldSpec.DataType.INT)
+        .addSingleValueDimension("column9", FieldSpec.DataType.INT)
+        .addSingleValueDimension("column11", FieldSpec.DataType.STRING)
+        .addSingleValueDimension("column12", FieldSpec.DataType.STRING).addMetric("column17", FieldSpec.DataType.INT)
+        .addMetric("column18", FieldSpec.DataType.INT)
+        .addTime(new TimeGranularitySpec(FieldSpec.DataType.INT, TimeUnit.DAYS, "daysSinceEpoch"), null).build();
+
+    createSegment(filePath, SEGMENT_NAME_1, schema);
+    createSegment(filePath, SEGMENT_NAME_2, schema);
+
+    ImmutableSegment immutableSegment1 = loadSegmentWithMetadataChecks(SEGMENT_NAME_1);
+    ImmutableSegment immutableSegment2 = loadSegmentWithMetadataChecks(SEGMENT_NAME_2);
+
+    _indexSegment = immutableSegment1;
+    _indexSegments = Arrays.asList(immutableSegment1, immutableSegment2);
+  }
+
+  private void createSegment(String filePath, String segmentName, Schema schema)
+      throws Exception {
+    // Create field configs for the no forward index columns
+    List<FieldConfig> fieldConfigList = new ArrayList<>();
+    fieldConfigList.add(new FieldConfig("column6", FieldConfig.EncodingType.DICTIONARY, Collections.emptyList(), null,
+        Collections.singletonMap(FieldConfig.FORWARD_INDEX_DISABLED, Boolean.TRUE.toString())));
+    if (segmentName.equals(SEGMENT_NAME_1)) {
+      fieldConfigList.add(new FieldConfig("column7", FieldConfig.EncodingType.DICTIONARY, Collections.emptyList(),
+          null, Collections.singletonMap(FieldConfig.FORWARD_INDEX_DISABLED, Boolean.TRUE.toString())));
+
+      // Build table config based on segment 1 as it contains both columns under no forward index
+      _tableConfig =
+          new TableConfigBuilder(TableType.OFFLINE).setTableName("testTable").setTimeColumnName("daysSinceEpoch")
+              .setFieldConfigList(fieldConfigList).setRangeIndexColumns(Arrays.asList("column6")).build();
+    }
+
+    // Create the segment generator config.
+    SegmentGeneratorConfig segmentGeneratorConfig = new SegmentGeneratorConfig(_tableConfig, schema);
+    segmentGeneratorConfig.setInputFilePath(filePath);
+    segmentGeneratorConfig.setTableName("testTable");
+    segmentGeneratorConfig.setOutDir(INDEX_DIR.getAbsolutePath());
+    segmentGeneratorConfig.setSegmentName(segmentName);
+    // The segment generation code in SegmentColumnarIndexCreator will throw
+    // exception if start and end time in time column are not in acceptable
+    // range. For this test, we first need to fix the input avro data
+    // to have the time column values in allowed range. Until then, the check
+    // is explicitly disabled
+    segmentGeneratorConfig.setSkipTimeValueCheck(true);
+    _invertedIndexColumns = Arrays.asList("column6", "column7", "column11", "column17", "column18");
+    segmentGeneratorConfig.setInvertedIndexCreationColumns(_invertedIndexColumns);
+
+    _forwardIndexDisabledColumns = new ArrayList<>(Arrays.asList("column6", "column7"));
+    segmentGeneratorConfig.setForwardIndexDisabledColumns(_forwardIndexDisabledColumns);
+    segmentGeneratorConfig.setRangeIndexCreationColumns(Arrays.asList("column6"));
+
+    // Build the index segment.
+    SegmentIndexCreationDriver driver = new SegmentIndexCreationDriverImpl();
+    driver.init(segmentGeneratorConfig);
+    driver.build();
+  }
+
+  private ImmutableSegment loadSegmentWithMetadataChecks(String segmentName)
+      throws Exception {
+    IndexLoadingConfig indexLoadingConfig = new IndexLoadingConfig();
+    indexLoadingConfig.setTableConfig(_tableConfig);
+    indexLoadingConfig.setInvertedIndexColumns(new HashSet<>(_invertedIndexColumns));
+    indexLoadingConfig.setForwardIndexDisabledColumns(new HashSet<>(_forwardIndexDisabledColumns));
+    indexLoadingConfig.setRangeIndexColumns(new HashSet<>(Arrays.asList("column6")));
+    indexLoadingConfig.setReadMode(ReadMode.heap);
+
+    ImmutableSegment immutableSegment = ImmutableSegmentLoader.load(new File(INDEX_DIR, segmentName),
+        indexLoadingConfig);
+
+    Map<String, ColumnMetadata> columnMetadataMap1 = immutableSegment.getSegmentMetadata().getColumnMetadataMap();
+    columnMetadataMap1.forEach((column, metadata) -> {
+      if (column.equals("column6") || column.equals("column7")) {
+        assertTrue(metadata.hasDictionary());
+        assertTrue(metadata.isSingleValue());
+        assertNull(immutableSegment.getForwardIndex(column));
+      } else {
+        assertNotNull(immutableSegment.getForwardIndex(column));
+      }
+    });
+
+    return immutableSegment;
+  }
+
+  @AfterClass
+  public void deleteAndDestroySegment() {
+    FileUtils.deleteQuietly(INDEX_DIR);
+    _indexSegments.forEach((IndexSegment::destroy));
+  }
+
+  @Override
+  protected String getFilter() {
+    return FILTER;
+  }
+
+  @Override
+  protected IndexSegment getIndexSegment() {
+    return _indexSegment;
+  }
+
+  @Override
+  protected List<IndexSegment> getIndexSegments() {
+    return _indexSegments;
+  }
+
+  @Test
+  public void testSelectStarQueries() {
+    // Select * without any filters
+    try {
+      getBrokerResponse(SELECT_STAR_QUERY);
+      Assert.fail("Select * query should fail since forwardIndexDisabled on a select column");
+    } catch (IllegalStateException e) {
+      assertTrue(e.getMessage().contains("Forward index disabled for column:")
+          && e.getMessage().contains("cannot create DataFetcher!"));
+    }
+
+    // Select * with filters
+    try {
+      getBrokerResponse(SELECT_STAR_QUERY + FILTER);
+      Assert.fail("Select * query should fail since forwardIndexDisabled on a select column");
+    } catch (IllegalStateException e) {
+      assertTrue(e.getMessage().contains("Forward index disabled for column:")
+          && e.getMessage().contains("cannot create DataFetcher!"));
+    }
+  }
+
+  @Test
+  public void testSelectQueries() {
+    {
+      // Selection query without filters including a column with forwardIndexDisabled enabled on both segments
+      String query = "SELECT column1, column5, column6, column9, column11 FROM testTable";
+      try {
+        getBrokerResponse(query);
+        Assert.fail("Query should fail since forwardIndexDisabled on a select column");
+      } catch (IllegalStateException e) {
+        assertTrue(e.getMessage().contains("Forward index disabled for column:")
+            && e.getMessage().contains("cannot create DataFetcher!"));
+      }
+    }
+    {
+      // Selection query without filters including a column with forwardIndexDisabled enabled on one segment
+      String query = "SELECT column1, column5, column7, column9, column11 FROM testTable";
+      try {
+        getBrokerResponse(query);
+        Assert.fail("Query should fail since forwardIndexDisabled on a select column");
+      } catch (IllegalStateException e) {
+        assertTrue(e.getMessage().contains("Forward index disabled for column:")
+            && e.getMessage().contains("cannot create DataFetcher!"));
+      }
+    }
+    {
+      // Selection query with filters including a column with forwardIndexDisabled enabled on both segments
+      String query = "SELECT column1, column5, column6, column9, column11 FROM testTable WHERE column6 = 2147458029";
+      try {
+        getBrokerResponse(query);
+        Assert.fail("Query should fail since forwardIndexDisabled on a select column");
+      } catch (IllegalStateException e) {
+        assertTrue(e.getMessage().contains("Forward index disabled for column:")
+            && e.getMessage().contains("cannot create DataFetcher!"));
+      }
+    }
+    {
+      // Selection query with filters including a column with forwardIndexDisabled enabled on one segment
+      String query = "SELECT column1, column5, column7, column9, column11 FROM testTable WHERE column7 = 675695";
+      try {
+        getBrokerResponse(query);
+        Assert.fail("Query should fail since forwardIndexDisabled on a select column");
+      } catch (IllegalStateException e) {
+        assertTrue(e.getMessage().contains("Forward index disabled for column:")
+            && e.getMessage().contains("cannot create DataFetcher!"));
+      }
+    }
+    {
+      // Selection query without filters and without columns with forwardIndexDisabled enabled on either segment
+      String query = "SELECT column1, column5, column9, column11 FROM testTable ORDER BY column1";
+      BrokerResponseNative brokerResponseNative = getBrokerResponse(query);
+      assertTrue(brokerResponseNative.getProcessingExceptions() == null
+          || brokerResponseNative.getProcessingExceptions().size() == 0);
+      ResultTable resultTable = brokerResponseNative.getResultTable();
+      assertEquals(brokerResponseNative.getNumRowsResultSet(), 10);
+      assertEquals(brokerResponseNative.getTotalDocs(), 120_000L);
+      assertEquals(brokerResponseNative.getNumDocsScanned(), 120_000L);
+      assertEquals(brokerResponseNative.getNumSegmentsProcessed(), 4L);
+      assertEquals(brokerResponseNative.getNumSegmentsMatched(), 4L);
+      assertEquals(brokerResponseNative.getNumEntriesScannedPostFilter(), 120_120L);
+      assertEquals(brokerResponseNative.getNumEntriesScannedInFilter(), 0L);
+      assertNotNull(brokerResponseNative.getProcessingExceptions());
+      assertEquals(brokerResponseNative.getProcessingExceptions().size(), 0);
+      DataSchema dataSchema = new DataSchema(new String[]{"column1", "column5", "column9", "column11"},
+        new DataSchema.ColumnDataType[]{DataSchema.ColumnDataType.INT, DataSchema.ColumnDataType.STRING,
+            DataSchema.ColumnDataType.INT, DataSchema.ColumnDataType.STRING});
+      assertEquals(resultTable.getDataSchema(), dataSchema);
+      List<Object[]> resultRows = resultTable.getRows();
+      int previousColumn1 = Integer.MIN_VALUE;
+      for (Object[] resultRow : resultRows) {
+        assertEquals(resultRow.length, 4);
+        assertEquals((String) resultRow[1], "gFuH");
+        // Column 1
+        assertTrue((Integer) resultRow[0] >= previousColumn1);
+        previousColumn1 = (Integer) resultRow[0];
+      }
+
+      Object[] firstRow = resultRows.get(0);
+      // Column 11
+      assertEquals((String) firstRow[3], "o");
+    }
+    {
+      // Transform function on a selection clause with a forwardIndexDisabled column in transform
+      String query = "SELECT CONCAT(column6, column9, '-') from testTable";
+      try {
+        getBrokerResponse(query);
+        Assert.fail("Query should fail since forwardIndexDisabled on a column in transform");
+      } catch (IllegalStateException e) {
+        assertTrue(e.getMessage().contains("Forward index disabled for column:")
+            && e.getMessage().contains("cannot create DataFetcher!"));
+      }
+    }
+    {
+      // Transform function on a selection clause without a forwardIndexDisabled column in transform
+      String query = "SELECT CONCAT(column5, column9, '-') from testTable ORDER BY column1";
+      BrokerResponseNative brokerResponseNative = getBrokerResponse(query);
+      assertTrue(brokerResponseNative.getProcessingExceptions() == null
+          || brokerResponseNative.getProcessingExceptions().size() == 0);
+      ResultTable resultTable = brokerResponseNative.getResultTable();
+      assertEquals(brokerResponseNative.getNumRowsResultSet(), 10);
+      assertEquals(brokerResponseNative.getTotalDocs(), 120_000L);
+      assertEquals(brokerResponseNative.getNumDocsScanned(), 120_000L);
+      assertEquals(brokerResponseNative.getNumSegmentsProcessed(), 4L);
+      assertEquals(brokerResponseNative.getNumSegmentsMatched(), 4L);
+      assertEquals(brokerResponseNative.getNumEntriesScannedPostFilter(), 120_080L);
+      assertEquals(brokerResponseNative.getNumEntriesScannedInFilter(), 0L);
+      assertNotNull(brokerResponseNative.getProcessingExceptions());
+      assertEquals(brokerResponseNative.getProcessingExceptions().size(), 0);
+      DataSchema dataSchema = new DataSchema(new String[]{"concat(column5,column9,'-')"},
+          new DataSchema.ColumnDataType[]{DataSchema.ColumnDataType.STRING});
+      assertEquals(resultTable.getDataSchema(), dataSchema);
+      List<Object[]> resultRows = resultTable.getRows();
+      for (Object[] resultRow : resultRows) {
+        assertEquals(resultRow.length, 1);
+        assertTrue(resultRow[0].toString().startsWith("gFuH-"));
+      }
+    }
+    {
+      // Selection query with filters (not including forwardIndexDisabled column) and without columns with
+      // forwardIndexDisabled enabled on either segment
+      String query = "SELECT column1, column5, column9, column11 FROM testTable WHERE column1 > 100000000"
+        + " AND column3 BETWEEN 20000000 AND 1000000000"
+        + " AND column5 = 'gFuH'"
+        + " AND daysSinceEpoch = 126164076 ORDER BY column1";
+      BrokerResponseNative brokerResponseNative = getBrokerResponse(query);
+      assertTrue(brokerResponseNative.getProcessingExceptions() == null
+          || brokerResponseNative.getProcessingExceptions().size() == 0);
+      ResultTable resultTable = brokerResponseNative.getResultTable();
+      assertEquals(brokerResponseNative.getNumRowsResultSet(), 10);
+      assertEquals(brokerResponseNative.getTotalDocs(), 120_000L);
+      assertEquals(brokerResponseNative.getNumDocsScanned(), 42_368L);
+      assertEquals(brokerResponseNative.getNumSegmentsProcessed(), 4L);
+      assertEquals(brokerResponseNative.getNumSegmentsMatched(), 4L);
+      assertEquals(brokerResponseNative.getNumEntriesScannedPostFilter(), 42_488);
+      assertEquals(brokerResponseNative.getNumEntriesScannedInFilter(), 192744L);
+      assertNotNull(brokerResponseNative.getProcessingExceptions());
+      assertEquals(brokerResponseNative.getProcessingExceptions().size(), 0);
+      DataSchema dataSchema = new DataSchema(new String[]{"column1", "column5", "column9", "column11"},
+          new DataSchema.ColumnDataType[]{DataSchema.ColumnDataType.INT, DataSchema.ColumnDataType.STRING,
+              DataSchema.ColumnDataType.INT, DataSchema.ColumnDataType.STRING});
+      assertEquals(resultTable.getDataSchema(), dataSchema);
+      List<Object[]> resultRows = resultTable.getRows();
+      int previousColumn1 = Integer.MIN_VALUE;
+      for (Object[] resultRow : resultRows) {
+        assertEquals(resultRow.length, 4);
+        assertEquals((String) resultRow[1], "gFuH");
+        assertTrue((Integer) resultRow[0] >= previousColumn1);
+        previousColumn1 = (Integer) resultRow[0];
+      }
+
+      Object[] firstRow = resultRows.get(0);
+      // Column 11
+      assertEquals((String) firstRow[3], "P");
+    }
+    {
+      // Selection query with supported filters (including forwardIndexDisabled column) and without columns with
+      // forwardIndexDisabled enabled on either segment
+      String query = "SELECT column1, column5, column9, column11 FROM testTable WHERE column6 = 2147458029 "
+          + "ORDER BY column1";
+      BrokerResponseNative brokerResponseNative = getBrokerResponse(query);
+      assertTrue(brokerResponseNative.getProcessingExceptions() == null
+          || brokerResponseNative.getProcessingExceptions().size() == 0);
+      ResultTable resultTable = brokerResponseNative.getResultTable();
+      assertEquals(brokerResponseNative.getNumRowsResultSet(), 10);
+      assertEquals(brokerResponseNative.getTotalDocs(), 120_000L);
+      assertEquals(brokerResponseNative.getNumDocsScanned(), 16L);
+      assertEquals(brokerResponseNative.getNumSegmentsProcessed(), 4L);
+      assertEquals(brokerResponseNative.getNumSegmentsMatched(), 4L);
+      assertEquals(brokerResponseNative.getNumEntriesScannedPostFilter(), 64L);
+      assertEquals(brokerResponseNative.getNumEntriesScannedInFilter(), 0L);
+      assertNotNull(brokerResponseNative.getProcessingExceptions());
+      assertEquals(brokerResponseNative.getProcessingExceptions().size(), 0);
+      DataSchema dataSchema = new DataSchema(new String[]{"column1", "column5", "column9", "column11"},
+          new DataSchema.ColumnDataType[]{DataSchema.ColumnDataType.INT, DataSchema.ColumnDataType.STRING,
+              DataSchema.ColumnDataType.INT, DataSchema.ColumnDataType.STRING});
+      assertEquals(resultTable.getDataSchema(), dataSchema);
+      List<Object[]> resultRows = resultTable.getRows();
+      for (Object[] resultRow : resultRows) {
+        assertEquals(resultRow.length, 4);
+        assertEquals(resultRow[0], 635553468);
+        assertEquals((String) resultRow[1], "gFuH");
+        assertEquals(resultRow[2], 705242697);
+        assertEquals((String) resultRow[3], "P");
+      }
+    }
+    {
+      // Selection query with supported filters (including forwardIndexDisabled column) and without columns with
+      // forwardIndexDisabled enabled on either segment
+      String query = "SELECT column1, column5, column9, column11 FROM testTable WHERE column7 != 675695 "
+          + "ORDER BY column1";
+      BrokerResponseNative brokerResponseNative = getBrokerResponse(query);
+      assertTrue(brokerResponseNative.getProcessingExceptions() == null
+          || brokerResponseNative.getProcessingExceptions().size() == 0);
+      ResultTable resultTable = brokerResponseNative.getResultTable();
+      assertEquals(brokerResponseNative.getNumRowsResultSet(), 10);
+      assertEquals(brokerResponseNative.getTotalDocs(), 120_000L);
+      assertEquals(brokerResponseNative.getNumDocsScanned(), 119_908L);
+      assertEquals(brokerResponseNative.getNumSegmentsProcessed(), 4L);
+      assertEquals(brokerResponseNative.getNumSegmentsMatched(), 4L);
+      assertEquals(brokerResponseNative.getNumEntriesScannedPostFilter(), 120_028L);
+      assertEquals(brokerResponseNative.getNumEntriesScannedInFilter(), 0L);
+      assertNotNull(brokerResponseNative.getProcessingExceptions());
+      assertEquals(brokerResponseNative.getProcessingExceptions().size(), 0);
+      DataSchema dataSchema = new DataSchema(new String[]{"column1", "column5", "column9", "column11"},
+          new DataSchema.ColumnDataType[]{DataSchema.ColumnDataType.INT, DataSchema.ColumnDataType.STRING,
+              DataSchema.ColumnDataType.INT, DataSchema.ColumnDataType.STRING});
+      assertEquals(resultTable.getDataSchema(), dataSchema);
+      List<Object[]> resultRows = resultTable.getRows();
+      int previousColumn1 = Integer.MIN_VALUE;
+      for (Object[] resultRow : resultRows) {
+        assertEquals(resultRow.length, 4);
+        assertEquals((String) resultRow[1], "gFuH");
+        assertTrue((Integer) resultRow[0] >= previousColumn1);
+        previousColumn1 = (Integer) resultRow[0];
+      }
+
+      Object[] firstRow = resultRows.get(0);
+      // Column 11
+      assertEquals((String) firstRow[3], "o");
+    }
+    {
+      // Selection query with supported filters (including forwardIndexDisabled column) and without columns with
+      // forwardIndexDisabled enabled on either segment
+      String query = "SELECT column1, column5, column9, column11 FROM testTable WHERE column7 IN (675695, 2137685743) "
+          + "ORDER BY column1";
+      BrokerResponseNative brokerResponseNative = getBrokerResponse(query);
+      assertTrue(brokerResponseNative.getProcessingExceptions() == null
+          || brokerResponseNative.getProcessingExceptions().size() == 0);
+      ResultTable resultTable = brokerResponseNative.getResultTable();
+      assertEquals(brokerResponseNative.getNumRowsResultSet(), 10);
+      assertEquals(brokerResponseNative.getTotalDocs(), 120_000L);
+      assertEquals(brokerResponseNative.getNumDocsScanned(), 828L);
+      assertEquals(brokerResponseNative.getNumSegmentsProcessed(), 4L);
+      assertEquals(brokerResponseNative.getNumSegmentsMatched(), 4L);
+      assertEquals(brokerResponseNative.getNumEntriesScannedPostFilter(), 948L);
+      assertEquals(brokerResponseNative.getNumEntriesScannedInFilter(), 0L);
+      assertNotNull(brokerResponseNative.getProcessingExceptions());
+      assertEquals(brokerResponseNative.getProcessingExceptions().size(), 0);
+      DataSchema dataSchema = new DataSchema(new String[]{"column1", "column5", "column9", "column11"},
+          new DataSchema.ColumnDataType[]{DataSchema.ColumnDataType.INT, DataSchema.ColumnDataType.STRING,
+              DataSchema.ColumnDataType.INT, DataSchema.ColumnDataType.STRING});
+      assertEquals(resultTable.getDataSchema(), dataSchema);
+      List<Object[]> resultRows = resultTable.getRows();
+      int previousColumn1 = Integer.MIN_VALUE;
+      for (Object[] resultRow : resultRows) {
+        assertEquals(resultRow.length, 4);
+        assertEquals((String) resultRow[1], "gFuH");
+        assertTrue((Integer) resultRow[0] >= previousColumn1);
+        previousColumn1 = (Integer) resultRow[0];
+      }
+
+      Object[] firstRow = resultRows.get(0);
+      // Column 11
+      assertEquals((String) firstRow[3], "P");
+    }
+    {
+      // Selection query with supported filters (including forwardIndexDisabled column) and without columns with
+      // forwardIndexDisabled enabled on either segment
+      String query = "SELECT column1, column5, column9, column11 FROM testTable WHERE column6 NOT IN "
+          + "(1689277, 2147458029) ORDER BY column1";
+      BrokerResponseNative brokerResponseNative = getBrokerResponse(query);
+      assertTrue(brokerResponseNative.getProcessingExceptions() == null
+          || brokerResponseNative.getProcessingExceptions().size() == 0);
+      ResultTable resultTable = brokerResponseNative.getResultTable();
+      assertEquals(brokerResponseNative.getNumRowsResultSet(), 10);
+      assertEquals(brokerResponseNative.getTotalDocs(), 120_000L);
+      assertEquals(brokerResponseNative.getNumDocsScanned(), 119_980L);
+      assertEquals(brokerResponseNative.getNumSegmentsProcessed(), 4L);
+      assertEquals(brokerResponseNative.getNumSegmentsMatched(), 4L);
+      assertEquals(brokerResponseNative.getNumEntriesScannedPostFilter(), 120_100L);
+      assertEquals(brokerResponseNative.getNumEntriesScannedInFilter(), 0L);
+      assertNotNull(brokerResponseNative.getProcessingExceptions());
+      assertEquals(brokerResponseNative.getProcessingExceptions().size(), 0);
+      DataSchema dataSchema = new DataSchema(new String[]{"column1", "column5", "column9", "column11"},
+          new DataSchema.ColumnDataType[]{DataSchema.ColumnDataType.INT, DataSchema.ColumnDataType.STRING,
+              DataSchema.ColumnDataType.INT, DataSchema.ColumnDataType.STRING});
+      assertEquals(resultTable.getDataSchema(), dataSchema);
+      List<Object[]> resultRows = resultTable.getRows();
+      int previousColumn1 = Integer.MIN_VALUE;
+      for (Object[] resultRow : resultRows) {
+        assertEquals(resultRow.length, 4);
+        assertEquals((String) resultRow[1], "gFuH");
+        assertTrue((Integer) resultRow[0] >= previousColumn1);
+        previousColumn1 = (Integer) resultRow[0];
+      }
+
+      Object[] firstRow = resultRows.get(0);
+      // Column 11
+      assertEquals((String) firstRow[3], "o");
+    }
+    {
+      // Query with literal only in SELECT
+      String query = "SELECT 'marvin' from testTable ORDER BY column1";
+      BrokerResponseNative brokerResponseNative = getBrokerResponse(query);
+      assertTrue(brokerResponseNative.getProcessingExceptions() == null
+          || brokerResponseNative.getProcessingExceptions().size() == 0);
+      ResultTable resultTable = brokerResponseNative.getResultTable();
+      assertEquals(brokerResponseNative.getNumRowsResultSet(), 10);
+      assertEquals(brokerResponseNative.getTotalDocs(), 120_000L);
+      assertEquals(brokerResponseNative.getNumDocsScanned(), 120_000L);
+      assertEquals(brokerResponseNative.getNumSegmentsProcessed(), 4L);
+      assertEquals(brokerResponseNative.getNumSegmentsMatched(), 4L);
+      assertEquals(brokerResponseNative.getNumEntriesScannedPostFilter(), 120_000L);
+      assertEquals(brokerResponseNative.getNumEntriesScannedInFilter(), 0L);
+      assertNotNull(brokerResponseNative.getProcessingExceptions());
+      assertEquals(brokerResponseNative.getProcessingExceptions().size(), 0);
+      assertEquals(resultTable.getDataSchema(), new DataSchema(new String[]{"'marvin'"},
+          new DataSchema.ColumnDataType[]{DataSchema.ColumnDataType.STRING}));
+      List<Object[]> resultRows = resultTable.getRows();
+      for (Object[] resultRow : resultRows) {
+        assertEquals(resultRow.length, 1);
+        assertEquals((String) resultRow[0], "marvin");
+      }
+    }
+    {
+      // Selection query with '<' filter on a forwardIndexDisabled column with range index available
+      String query = "SELECT column1, column5, column9, column11 FROM testTable WHERE column6 < 2147458029 AND "
+          + "column6 > 1699000 ORDER BY column1";
+      BrokerResponseNative brokerResponseNative = getBrokerResponse(query);
+      assertTrue(brokerResponseNative.getProcessingExceptions() == null
+          || brokerResponseNative.getProcessingExceptions().size() == 0);
+      ResultTable resultTable = brokerResponseNative.getResultTable();
+      assertEquals(brokerResponseNative.getNumRowsResultSet(), 10);
+      assertEquals(brokerResponseNative.getTotalDocs(), 120_000L);
+      assertEquals(brokerResponseNative.getNumDocsScanned(), 119_980L);
+      assertEquals(brokerResponseNative.getNumSegmentsProcessed(), 4L);
+      assertEquals(brokerResponseNative.getNumSegmentsMatched(), 4L);
+      assertEquals(brokerResponseNative.getNumEntriesScannedPostFilter(), 120_100L);
+      assertEquals(brokerResponseNative.getNumEntriesScannedInFilter(), 0L);
+      assertNotNull(brokerResponseNative.getProcessingExceptions());
+      assertEquals(brokerResponseNative.getProcessingExceptions().size(), 0);
+      DataSchema dataSchema = new DataSchema(new String[]{"column1", "column5", "column9", "column11"},
+          new DataSchema.ColumnDataType[]{DataSchema.ColumnDataType.INT, DataSchema.ColumnDataType.STRING,
+              DataSchema.ColumnDataType.INT, DataSchema.ColumnDataType.STRING});
+      assertEquals(resultTable.getDataSchema(), dataSchema);
+      List<Object[]> resultRows = resultTable.getRows();
+      int previousColumn1 = Integer.MIN_VALUE;
+      for (Object[] resultRow : resultRows) {
+        assertEquals(resultRow.length, 4);
+        assertEquals((String) resultRow[1], "gFuH");
+        assertTrue((Integer) resultRow[0] >= previousColumn1);
+        previousColumn1 = (Integer) resultRow[0];
+      }
+
+      Object[] firstRow = resultRows.get(0);
+      // Column 11
+      assertEquals((String) firstRow[3], "o");
+    }
+    {
+      // Selection query with '>=' filter on a forwardIndexDisabled column without range index available
+      String query = "SELECT column1, column5, column9, column11 FROM testTable WHERE column7 >= 676000";
+      try {
+        getBrokerResponse(query);
+        Assert.fail("Query should fail since forwardIndexDisabled on a range query column without range index");
+      } catch (IllegalStateException e) {
+        assertTrue(e.getMessage().contains("Forward index disabled for column:")
+            && e.getMessage().contains("creating ScanDocIdSet unsupported!"));
+      }
+    }
+    {
+      // Select query with a filter on a column which doesn't have forwardIndexDisabled enabled
+      String query = "SELECT column1, column5, column9 from testTable WHERE column9 < 50000 ORDER BY column1";
+      BrokerResponseNative brokerResponseNative = getBrokerResponse(query);
+      assertTrue(brokerResponseNative.getProcessingExceptions() == null
+          || brokerResponseNative.getProcessingExceptions().size() == 0);
+      ResultTable resultTable = brokerResponseNative.getResultTable();
+      assertEquals(brokerResponseNative.getNumRowsResultSet(), 4);
+      assertEquals(brokerResponseNative.getTotalDocs(), 120_000L);
+      assertEquals(brokerResponseNative.getNumDocsScanned(), 4L);
+      assertEquals(brokerResponseNative.getNumSegmentsProcessed(), 4L);
+      assertEquals(brokerResponseNative.getNumSegmentsMatched(), 4L);
+      assertEquals(brokerResponseNative.getNumEntriesScannedPostFilter(), 12L);
+      assertEquals(brokerResponseNative.getNumEntriesScannedInFilter(), 120000L);
+      assertNotNull(brokerResponseNative.getProcessingExceptions());
+      assertEquals(brokerResponseNative.getProcessingExceptions().size(), 0);
+      assertEquals(resultTable.getDataSchema(), new DataSchema(new String[]{"column1", "column5", "column9"},
+          new DataSchema.ColumnDataType[]{
+          DataSchema.ColumnDataType.INT, DataSchema.ColumnDataType.STRING, DataSchema.ColumnDataType.INT
+      }));
+      List<Object[]> resultRows = resultTable.getRows();
+      for (Object[] resultRow : resultRows) {
+        assertEquals(resultRow.length, 3);
+      }
+    }
+    {
+      // Transform function on a filter clause for forwardIndexDisabled column in transform
+      String query = "SELECT column1, column11 from testTable WHERE CONCAT(column6, column9, '-') = '1689277-11270'";
+      BrokerResponseNative brokerResponseNative = getBrokerResponse(query);
+      assertTrue(brokerResponseNative.getProcessingExceptions() != null
+          && brokerResponseNative.getProcessingExceptions().size() > 0);
+    }
+    {
+      // Transform function on a filter clause for a non-forwardIndexDisabled column in transform
+      String query = "SELECT column1, column11 from testTable WHERE CONCAT(column5, column9, '-') = 'gFuH-11270' "
+          + "ORDER BY column1";
+      BrokerResponseNative brokerResponseNative = getBrokerResponse(query);
+      assertTrue(brokerResponseNative.getProcessingExceptions() == null
+          || brokerResponseNative.getProcessingExceptions().size() == 0);
+      ResultTable resultTable = brokerResponseNative.getResultTable();
+      assertEquals(brokerResponseNative.getNumRowsResultSet(), 4);
+      assertEquals(brokerResponseNative.getTotalDocs(), 120_000L);
+      assertEquals(brokerResponseNative.getNumDocsScanned(), 4L);
+      assertEquals(brokerResponseNative.getNumSegmentsProcessed(), 4L);
+      assertEquals(brokerResponseNative.getNumSegmentsMatched(), 4L);
+      assertEquals(brokerResponseNative.getNumEntriesScannedPostFilter(), 8L);
+      assertEquals(brokerResponseNative.getNumEntriesScannedInFilter(), 120000L);
+      assertNotNull(brokerResponseNative.getProcessingExceptions());
+      assertEquals(brokerResponseNative.getProcessingExceptions().size(), 0);
+      assertEquals(resultTable.getDataSchema(), new DataSchema(new String[]{"column1", "column11"},
+          new DataSchema.ColumnDataType[]{
+              DataSchema.ColumnDataType.INT, DataSchema.ColumnDataType.STRING
+          }));
+      List<Object[]> resultRows = resultTable.getRows();
+      assertEquals(resultRows.size(), 4);
+      for (Object[] resultRow : resultRows) {
+        assertEquals(resultRow.length, 2);
+        assertEquals(resultRow[0], 815409257);
+        assertEquals(resultRow[1], "P");
+      }
+    }
+  }
+
+  @Test
+  public void testSelectWithDistinctQueries() {
+    {
+      // Select a mix of forwardIndexDisabled and non-forwardIndexDisabled columns with distinct
+      String query = "SELECT DISTINCT column1, column6, column9 FROM testTable LIMIT 10";
+      try {
+        getBrokerResponse(query);
+        Assert.fail("Query should fail since forwardIndexDisabled on a column in select distinct");
+      } catch (IllegalStateException e) {
+        assertTrue(e.getMessage().contains("Forward index disabled for column:")
+            && e.getMessage().contains("cannot create DataFetcher!"));
+      }
+    }
+    {
+      // Select non-forwardIndexDisabled columns with distinct
+      String query = "SELECT DISTINCT column1, column5, column9 FROM testTable ORDER BY column1 LIMIT 10";
+      BrokerResponseNative brokerResponseNative = getBrokerResponse(query);
+      assertTrue(brokerResponseNative.getProcessingExceptions() == null
+          || brokerResponseNative.getProcessingExceptions().size() == 0);
+      ResultTable resultTable = brokerResponseNative.getResultTable();
+      assertEquals(brokerResponseNative.getNumRowsResultSet(), 10);
+      assertEquals(brokerResponseNative.getTotalDocs(), 120_000L);
+      assertEquals(brokerResponseNative.getNumDocsScanned(), 120_000L);
+      assertEquals(brokerResponseNative.getNumSegmentsProcessed(), 4L);
+      assertEquals(brokerResponseNative.getNumSegmentsMatched(), 4L);
+      assertEquals(brokerResponseNative.getNumEntriesScannedPostFilter(), 360_000L);
+      assertEquals(brokerResponseNative.getNumEntriesScannedInFilter(), 0L);
+      assertNotNull(brokerResponseNative.getProcessingExceptions());
+      assertEquals(brokerResponseNative.getProcessingExceptions().size(), 0);
+      assertEquals(resultTable.getDataSchema(), new DataSchema(new String[]{"column1", "column5", "column9"},
+          new DataSchema.ColumnDataType[]{DataSchema.ColumnDataType.INT, DataSchema.ColumnDataType.STRING,
+              DataSchema.ColumnDataType.INT}));
+      List<Object[]> resultRows = resultTable.getRows();
+      int previousColumn1 = Integer.MIN_VALUE;
+      for (Object[] resultRow : resultRows) {
+        assertEquals(resultRow.length, 3);
+        assertEquals(resultRow[1], "gFuH");
+        assertTrue((Integer) resultRow[0] >= previousColumn1);
+        previousColumn1 = (Integer) resultRow[0];
+      }
+    }
+  }
+
+  @Test
+  public void testSelectWithGroupByOrderByQueries() {
+    {
+      // Select a mix of forwardIndexDisabled and non-forwardIndexDisabled columns with group by order by
+      String query = "SELECT column1, column6 FROM testTable GROUP BY column1, column6 ORDER BY column1, column6 "
+          + " LIMIT 10";
+      try {
+        getBrokerResponse(query);
+        Assert.fail("Query should fail since forwardIndexDisabled on a column in select group by order by");
+      } catch (IllegalStateException e) {
+        assertTrue(e.getMessage().contains("Forward index disabled for column:")
+            && e.getMessage().contains("cannot create DataFetcher!"));
+      }
+    }
+    {
+      // Select forwardIndexDisabled columns with group by order by
+      String query = "SELECT column7, column6 FROM testTable GROUP BY column7, column6 ORDER BY column7, column6 "
+          + " LIMIT 10";
+      try {
+        getBrokerResponse(query);
+        Assert.fail("Query should fail since forwardIndexDisabled on a column in select group by order by");
+      } catch (IllegalStateException e) {
+        assertTrue(e.getMessage().contains("Forward index disabled for column:")
+            && e.getMessage().contains("cannot create DataFetcher!"));
+      }
+    }
+    {
+      // Select non-forwardIndexDisabled columns with group by order by
+      String query = "SELECT column1, column5 FROM testTable GROUP BY column1, column5 ORDER BY column1, column5 "
+          + " LIMIT 10";
+      BrokerResponseNative brokerResponseNative = getBrokerResponse(query);
+      assertTrue(brokerResponseNative.getProcessingExceptions() == null
+          || brokerResponseNative.getProcessingExceptions().size() == 0);
+      ResultTable resultTable = brokerResponseNative.getResultTable();
+      assertEquals(brokerResponseNative.getNumRowsResultSet(), 10);
+      assertEquals(brokerResponseNative.getTotalDocs(), 120_000L);
+      assertEquals(brokerResponseNative.getNumDocsScanned(), 120000L);
+      assertEquals(brokerResponseNative.getNumSegmentsProcessed(), 4L);
+      assertEquals(brokerResponseNative.getNumSegmentsMatched(), 4L);
+      assertEquals(brokerResponseNative.getNumEntriesScannedPostFilter(), 240000L);
+      assertEquals(brokerResponseNative.getNumEntriesScannedInFilter(), 0L);
+      assertNotNull(brokerResponseNative.getProcessingExceptions());
+      assertEquals(brokerResponseNative.getProcessingExceptions().size(), 0);
+      assertEquals(resultTable.getDataSchema(), new DataSchema(new String[]{"column1", "column5"},
+          new DataSchema.ColumnDataType[]{DataSchema.ColumnDataType.INT, DataSchema.ColumnDataType.STRING}));
+      List<Object[]> resultRows = resultTable.getRows();
+      int previousVal = -1;
+      for (Object[] resultRow : resultRows) {
+        assertEquals(resultRow.length, 2);
+        assertEquals(resultRow[1], "gFuH");
+        assertTrue((int) resultRow[0] > previousVal);
+        previousVal = (int) resultRow[0];
+      }
+    }
+    {
+      // Select forwardIndexDisabled columns using transform with group by order by
+      String query = "SELECT CONCAT(column1, column6, '-') FROM testTable GROUP BY CONCAT(column1, column6, '-') "
+          + "ORDER BY CONCAT(column1, column6, '-') LIMIT 10";
+      try {
+        getBrokerResponse(query);
+        Assert.fail("Query should fail since forwardIndexDisabled on a transformed column in group by order by");
+      } catch (IllegalStateException e) {
+        assertTrue(e.getMessage().contains("Forward index disabled for column:")
+            && e.getMessage().contains("cannot create DataFetcher!"));
+      }
+    }
+    {
+      // Select non-forwardIndexDisabled columns using transform with group by order by
+      String query = "SELECT CONCAT(column1, column5, '-') FROM testTable GROUP BY CONCAT(column1, column5, '-') "
+          + "ORDER BY CONCAT(column1, column5, '-') LIMIT 10";
+      BrokerResponseNative brokerResponseNative = getBrokerResponse(query);
+      assertTrue(brokerResponseNative.getProcessingExceptions() == null
+          || brokerResponseNative.getProcessingExceptions().size() == 0);
+      ResultTable resultTable = brokerResponseNative.getResultTable();
+      assertEquals(brokerResponseNative.getNumRowsResultSet(), 10);
+      assertEquals(brokerResponseNative.getTotalDocs(), 120_000L);
+      assertEquals(brokerResponseNative.getNumDocsScanned(), 120000L);
+      assertEquals(brokerResponseNative.getNumSegmentsProcessed(), 4L);
+      assertEquals(brokerResponseNative.getNumSegmentsMatched(), 4L);
+      assertEquals(brokerResponseNative.getNumEntriesScannedPostFilter(), 240000L);
+      assertEquals(brokerResponseNative.getNumEntriesScannedInFilter(), 0L);
+      assertNotNull(brokerResponseNative.getProcessingExceptions());
+      assertEquals(brokerResponseNative.getProcessingExceptions().size(), 0);
+      assertEquals(resultTable.getDataSchema(), new DataSchema(new String[]{"concat(column1,column5,'-')"},
+          new DataSchema.ColumnDataType[]{DataSchema.ColumnDataType.STRING}));
+      List<Object[]> resultRows = resultTable.getRows();
+      for (Object[] resultRow : resultRows) {
+        assertEquals(resultRow.length, 1);
+        assertTrue(resultRow[0].toString().endsWith("-gFuH"));
+      }
+    }
+  }
+
+  @Test
+  public void testSelectWithAggregationQueries() {
+    {
+      // Allowed aggregation functions on forwardIndexDisabled columns
+      String query = "SELECT max(column7), min(column7), count(column7), minmaxrange(column7), "
+          + "distinctcount(column7), distinctcounthll(column6), distinctcountrawhll(column7), "
+          + "distinctcountsmarthll(column6) from testTable";
+      BrokerResponseNative brokerResponseNative = getBrokerResponse(query);
+      assertTrue(brokerResponseNative.getProcessingExceptions() == null
+          || brokerResponseNative.getProcessingExceptions().size() == 0);
+      ResultTable resultTable = brokerResponseNative.getResultTable();
+      assertEquals(brokerResponseNative.getNumRowsResultSet(), 1);
+      assertEquals(brokerResponseNative.getTotalDocs(), 120_000L);
+      assertEquals(brokerResponseNative.getNumDocsScanned(), 120_000L);
+      assertEquals(brokerResponseNative.getNumSegmentsProcessed(), 4L);
+      assertEquals(brokerResponseNative.getNumSegmentsMatched(), 4L);
+      assertEquals(brokerResponseNative.getNumEntriesScannedPostFilter(), 0L);
+      assertEquals(brokerResponseNative.getNumEntriesScannedInFilter(), 0L);
+      assertNotNull(brokerResponseNative.getProcessingExceptions());
+      assertEquals(brokerResponseNative.getProcessingExceptions().size(), 0);
+      assertEquals(resultTable.getDataSchema(), new DataSchema(new String[]{"max(column7)", "min(column7)", "count(*)",
+      "minmaxrange(column7)", "distinctcount(column7)", "distinctcounthll(column6)", "distinctcountrawhll(column7)",
+      "distinctcountsmarthll(column6)"},
+          new DataSchema.ColumnDataType[]{DataSchema.ColumnDataType.DOUBLE, DataSchema.ColumnDataType.DOUBLE,
+              DataSchema.ColumnDataType.LONG, DataSchema.ColumnDataType.DOUBLE, DataSchema.ColumnDataType.INT,
+              DataSchema.ColumnDataType.LONG, DataSchema.ColumnDataType.STRING, DataSchema.ColumnDataType.INT}));
+      List<Object[]> resultRows = resultTable.getRows();
+      for (Object[] resultRow : resultRows) {
+        assertEquals(resultRow.length, 8);
+        assertEquals(resultRow[0], 2.137685743E9);
+        assertEquals(resultRow[1], 675695.0);
+        assertEquals(resultRow[2], 120000L);
+        assertEquals(resultRow[3], 2.137010048E9);
+        assertEquals(resultRow[4], 146);
+        assertEquals(resultRow[5], 695L);
+        assertEquals(resultRow[7], 608);
+      }
+    }
+    {
+      // Not allowed aggregation functions on forwardIndexDisabled columns
+      String query = "SELECT sum(column7), avg(column6) from testTable";
+      try {
+        getBrokerResponse(query);
+        Assert.fail("Query should fail since forwardIndexDisabled on a column in unsupported aggregation query");
+      } catch (IllegalStateException e) {
+        assertTrue(e.getMessage().contains("Forward index disabled for column:")
+            && e.getMessage().contains("cannot create DataFetcher!"));
+      }
+    }
+    {
+      // Allowed aggregation functions on forwardIndexDisabled columns with group by on non-forwardIndexDisabled
+      // column
+      String query = "SELECT column1, max(column6) from testTable GROUP BY column1";
+      try {
+        getBrokerResponse(query);
+        Assert.fail("Query should fail since forwardIndexDisabled on a column in unsupported aggregation query");
+      } catch (IllegalStateException e) {
+        assertTrue(e.getMessage().contains("Forward index disabled for column:")
+            && e.getMessage().contains("cannot create DataFetcher!"));
+      }
+    }
+    {
+      // Allowed aggregation functions on forwardIndexDisabled columns with group by order by on
+      // non-forwardIndexDisabled column
+      String query = "SELECT column1, max(column6) from testTable GROUP BY column1 ORDER BY column1";
+      try {
+        getBrokerResponse(query);
+        Assert.fail("Query should fail since forwardIndexDisabled on a column in unsupported aggregation query");
+      } catch (IllegalStateException e) {
+        assertTrue(e.getMessage().contains("Forward index disabled for column:")
+            && e.getMessage().contains("cannot create DataFetcher!"));
+      }
+    }
+    {
+      // Allowed aggregation functions on non-forwardIndexDisabled columns with group by on non-forwardIndexDisabled
+      // column but order by on allowed aggregation function on forwardIndexDisabled column
+      String query = "SELECT column1, max(column9) from testTable GROUP BY column1 ORDER BY min(column6)";

Review Comment:
   Cover `HAVING` 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] somandal commented on a diff in pull request #9333: Add an option to disable the creation of the forward index for a column

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


##########
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",

Review Comment:
   Do you want me to add a test in these files for other types? My other query tests use other types. Is that sufficient?



-- 
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 #9333: Add an option to disable the creation of the forward index for a column

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


##########
pinot-segment-local/src/test/java/org/apache/pinot/segment/local/segment/index/loader/defaultcolumn/BaseDefaultColumnHandlerTest.java:
##########
@@ -70,7 +70,7 @@ public void setUp()
     driver.build();
     _segmentDirectory = new File(_indexDir, driver.getSegmentName());
     _committedSegmentMetadata = new SegmentMetadataImpl(_segmentDirectory);
-    _writer = new SegmentLocalFSDirectory(_indexDir, _committedSegmentMetadata, ReadMode.mmap).createWriter();
+    _writer = new SegmentLocalFSDirectory(_segmentDirectory, _committedSegmentMetadata, ReadMode.mmap).createWriter();

Review Comment:
   So for me this test started failing due to:
   
   ```
   java.lang.RuntimeException: Forward index disabled in segment: true for column: column1 does not match forward index disabled flag: false in the TableConfig, updating this flag is not supported at the moment.
   
   	at org.apache.pinot.segment.local.segment.index.loader.defaultcolumn.BaseDefaultColumnHandler.computeDefaultColumnActionMap(BaseDefaultColumnHandler.java:239)
   	at org.apache.pinot.segment.local.segment.index.loader.defaultcolumn.BaseDefaultColumnHandlerTest.testComputeDefaultColumnActionMapForCommittedSegment(BaseDefaultColumnHandlerTest.java:105)
   	at java.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
   	at java.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)
   	at java.base/jdk.internal.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
   	at java.base/java.lang.reflect.Method.invoke(Method.java:566)
   	at org.testng.internal.MethodInvocationHelper.invokeMethod(MethodInvocationHelper.java:108)
   	at org.testng.internal.Invoker.invokeMethod(Invoker.java:661)
   	at org.testng.internal.Invoker.invokeTestMethod(Invoker.java:869)
   	at org.testng.internal.Invoker.invokeTestMethods(Invoker.java:1193)
   	at org.testng.internal.TestMethodWorker.invokeTestMethods(TestMethodWorker.java:126)
   	at org.testng.internal.TestMethodWorker.run(TestMethodWorker.java:109)
   	at org.testng.TestRunner.privateRun(TestRunner.java:744)
   	at org.testng.TestRunner.run(TestRunner.java:602)
   	at org.testng.SuiteRunner.runTest(SuiteRunner.java:380)
   	at org.testng.SuiteRunner.runSequentially(SuiteRunner.java:375)
   	at org.testng.SuiteRunner.privateRun(SuiteRunner.java:340)
   	at org.testng.SuiteRunner.run(SuiteRunner.java:289)
   	at org.testng.SuiteRunnerWorker.runSuite(SuiteRunnerWorker.java:52)
   	at org.testng.SuiteRunnerWorker.run(SuiteRunnerWorker.java:86)
   	at org.testng.TestNG.runSuitesSequentially(TestNG.java:1301)
   	at org.testng.TestNG.runSuitesLocally(TestNG.java:1226)
   	at org.testng.TestNG.runSuites(TestNG.java:1144)
   	at org.testng.TestNG.run(TestNG.java:1115)
   	at com.intellij.rt.testng.IDEARemoteTestNG.run(IDEARemoteTestNG.java:66)
   	at com.intellij.rt.testng.RemoteTestNGStarter.main(RemoteTestNGStarter.java:109)
   ```
   
   On debugging I found that the forward index did indeed exist correctly but the writer was pointing to the wrong directory. It should point to the segment directory instead of index directory. So just fixed the test as this was a bug. Hope that makes sense



-- 
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 #9333: Add an option to disable the creation of the forward index for a column

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


##########
pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/loader/defaultcolumn/BaseDefaultColumnHandler.java:
##########
@@ -363,6 +379,12 @@ protected boolean createColumnV1Indices(String column)
                   argument);
               return false;
             }
+            // TODO: Support creation of derived columns from forward index disabled columns
+            if (!_segmentWriter.hasIndexFor(argument, ColumnIndexType.FORWARD_INDEX)) {
+              LOGGER.warn("Skip creating derived column: {} because argument: {} does not have a forward index", column,

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 #9333: Add an option to disable the creation of the forward index for a column

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


##########
pinot-core/src/test/java/org/apache/pinot/queries/ForwardIndexDisabledSingleValueQueriesTest.java:
##########
@@ -0,0 +1,1088 @@
+/**
+ * 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.queries;
+
+import java.io.File;
+import java.net.URL;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.TimeUnit;
+import org.apache.commons.io.FileUtils;
+import org.apache.pinot.common.response.broker.BrokerResponseNative;
+import org.apache.pinot.common.response.broker.ResultTable;
+import org.apache.pinot.common.utils.DataSchema;
+import org.apache.pinot.segment.local.indexsegment.immutable.ImmutableSegmentLoader;
+import org.apache.pinot.segment.local.segment.creator.impl.SegmentIndexCreationDriverImpl;
+import org.apache.pinot.segment.local.segment.index.loader.IndexLoadingConfig;
+import org.apache.pinot.segment.spi.ColumnMetadata;
+import org.apache.pinot.segment.spi.ImmutableSegment;
+import org.apache.pinot.segment.spi.IndexSegment;
+import org.apache.pinot.segment.spi.creator.SegmentGeneratorConfig;
+import org.apache.pinot.segment.spi.creator.SegmentIndexCreationDriver;
+import org.apache.pinot.spi.config.table.FieldConfig;
+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.data.TimeGranularitySpec;
+import org.apache.pinot.spi.utils.ReadMode;
+import org.apache.pinot.spi.utils.builder.TableConfigBuilder;
+import org.testng.Assert;
+import org.testng.annotations.AfterClass;
+import org.testng.annotations.BeforeClass;
+import org.testng.annotations.Test;
+
+import static org.testng.Assert.assertEquals;
+import static org.testng.Assert.assertNotNull;
+import static org.testng.Assert.assertNull;
+import static org.testng.Assert.assertTrue;
+
+
+/**
+ * The <code>ForwardIndexDisabledSingleValueQueriesTest</code> class sets up the index segment for the no forward
+ * index single-value queries test.
+ * <p>There are totally 18 columns, 30000 records inside the original Avro file where 11 columns are selected to build
+ * the index segment. Selected columns information are as following:
+ * <ul>
+ *   ColumnName, FieldType, DataType, Cardinality, IsSorted, HasInvertedIndex, FwdIndexDisabled: S1, S2, HasRangeIndex
+ *   <li>column1, METRIC, INT, 6582, F, F, F, F, F</li>
+ *   <li>column3, METRIC, INT, 21910, F, F, F, F, F</li>
+ *   <li>column5, DIMENSION, STRING, 1, T, F, F, F, F</li>
+ *   <li>column6, DIMENSION, INT, 608, F, T, T, T, T</li>
+ *   <li>column7, DIMENSION, INT, 146, F, T, T, F, F</li>
+ *   <li>column9, DIMENSION, INT, 1737, F, F, F, F, F</li>
+ *   <li>column11, DIMENSION, STRING, 5, F, T, F, F, F</li>
+ *   <li>column12, DIMENSION, STRING, 5, F, F, F, F, F</li>
+ *   <li>column17, METRIC, INT, 24, F, T, F, F, F</li>
+ *   <li>column18, METRIC, INT, 1440, F, T, F, F, F</li>
+ *   <li>daysSinceEpoch, TIME, INT, 2, T, F, F, F, F</li>
+ * </ul>
+ */
+public class ForwardIndexDisabledSingleValueQueriesTest extends BaseQueriesTest {
+  private static final String AVRO_DATA = "data" + File.separator + "test_data-sv.avro";
+  private static final String SEGMENT_NAME_1 = "testTable_126164076_167572857";
+  private static final String SEGMENT_NAME_2 = "testTable_126164076_167572858";
+  private static final File INDEX_DIR = new File(FileUtils.getTempDirectory(),
+      "ForwardIndexDisabledSingleValueQueriesTest");
+
+  private static final String SELECT_STAR_QUERY = "SELECT * FROM testTable";
+
+  // Hard-coded query filter.
+  private static final String FILTER = " WHERE column1 > 100000000"
+      + " AND column3 BETWEEN 20000000 AND 1000000000"
+      + " AND column5 = 'gFuH'"
+      + " AND (column6 < 500000000 OR column11 NOT IN ('t', 'P'))"
+      + " AND daysSinceEpoch = 126164076";
+
+  private IndexSegment _indexSegment;
+  // Contains 2 index segments, one with 2 columns with forward index disabled, and the other with just 1.
+  private List<IndexSegment> _indexSegments;
+
+  private TableConfig _tableConfig;
+  private List<String> _invertedIndexColumns;
+  private List<String> _forwardIndexDisabledColumns;
+
+  @BeforeClass
+  public void buildAndLoadSegment()
+      throws Exception {
+    FileUtils.deleteQuietly(INDEX_DIR);
+
+    // Get resource file path.
+    URL resource = getClass().getClassLoader().getResource(AVRO_DATA);
+    assertNotNull(resource);
+    String filePath = resource.getFile();
+
+    // Build the segment schema.
+    Schema schema = new Schema.SchemaBuilder().setSchemaName("testTable").addMetric("column1", FieldSpec.DataType.INT)
+        .addMetric("column3", FieldSpec.DataType.INT).addSingleValueDimension("column5", FieldSpec.DataType.STRING)
+        .addSingleValueDimension("column6", FieldSpec.DataType.INT)
+        .addSingleValueDimension("column7", FieldSpec.DataType.INT)
+        .addSingleValueDimension("column9", FieldSpec.DataType.INT)
+        .addSingleValueDimension("column11", FieldSpec.DataType.STRING)
+        .addSingleValueDimension("column12", FieldSpec.DataType.STRING).addMetric("column17", FieldSpec.DataType.INT)
+        .addMetric("column18", FieldSpec.DataType.INT)
+        .addTime(new TimeGranularitySpec(FieldSpec.DataType.INT, TimeUnit.DAYS, "daysSinceEpoch"), null).build();
+
+    createSegment(filePath, SEGMENT_NAME_1, schema);
+    createSegment(filePath, SEGMENT_NAME_2, schema);
+
+    ImmutableSegment immutableSegment1 = loadSegmentWithMetadataChecks(SEGMENT_NAME_1);
+    ImmutableSegment immutableSegment2 = loadSegmentWithMetadataChecks(SEGMENT_NAME_2);
+
+    _indexSegment = immutableSegment1;
+    _indexSegments = Arrays.asList(immutableSegment1, immutableSegment2);
+  }
+
+  private void createSegment(String filePath, String segmentName, Schema schema)
+      throws Exception {
+    // Create field configs for the no forward index columns
+    List<FieldConfig> fieldConfigList = new ArrayList<>();
+    fieldConfigList.add(new FieldConfig("column6", FieldConfig.EncodingType.DICTIONARY, Collections.emptyList(), null,
+        Collections.singletonMap(FieldConfig.FORWARD_INDEX_DISABLED, Boolean.TRUE.toString())));
+    if (segmentName.equals(SEGMENT_NAME_1)) {
+      fieldConfigList.add(new FieldConfig("column7", FieldConfig.EncodingType.DICTIONARY, Collections.emptyList(),
+          null, Collections.singletonMap(FieldConfig.FORWARD_INDEX_DISABLED, Boolean.TRUE.toString())));
+
+      // Build table config based on segment 1 as it contains both columns under no forward index
+      _tableConfig =
+          new TableConfigBuilder(TableType.OFFLINE).setTableName("testTable").setTimeColumnName("daysSinceEpoch")
+              .setFieldConfigList(fieldConfigList).setRangeIndexColumns(Arrays.asList("column6")).build();
+    }
+
+    // Create the segment generator config.
+    SegmentGeneratorConfig segmentGeneratorConfig = new SegmentGeneratorConfig(_tableConfig, schema);
+    segmentGeneratorConfig.setInputFilePath(filePath);
+    segmentGeneratorConfig.setTableName("testTable");
+    segmentGeneratorConfig.setOutDir(INDEX_DIR.getAbsolutePath());
+    segmentGeneratorConfig.setSegmentName(segmentName);
+    // The segment generation code in SegmentColumnarIndexCreator will throw
+    // exception if start and end time in time column are not in acceptable
+    // range. For this test, we first need to fix the input avro data
+    // to have the time column values in allowed range. Until then, the check
+    // is explicitly disabled
+    segmentGeneratorConfig.setSkipTimeValueCheck(true);
+    _invertedIndexColumns = Arrays.asList("column6", "column7", "column11", "column17", "column18");
+    segmentGeneratorConfig.setInvertedIndexCreationColumns(_invertedIndexColumns);
+
+    _forwardIndexDisabledColumns = new ArrayList<>(Arrays.asList("column6", "column7"));
+    segmentGeneratorConfig.setForwardIndexDisabledColumns(_forwardIndexDisabledColumns);
+    segmentGeneratorConfig.setRangeIndexCreationColumns(Arrays.asList("column6"));
+
+    // Build the index segment.
+    SegmentIndexCreationDriver driver = new SegmentIndexCreationDriverImpl();
+    driver.init(segmentGeneratorConfig);
+    driver.build();
+  }
+
+  private ImmutableSegment loadSegmentWithMetadataChecks(String segmentName)
+      throws Exception {
+    IndexLoadingConfig indexLoadingConfig = new IndexLoadingConfig();
+    indexLoadingConfig.setTableConfig(_tableConfig);
+    indexLoadingConfig.setInvertedIndexColumns(new HashSet<>(_invertedIndexColumns));
+    indexLoadingConfig.setForwardIndexDisabledColumns(new HashSet<>(_forwardIndexDisabledColumns));
+    indexLoadingConfig.setRangeIndexColumns(new HashSet<>(Arrays.asList("column6")));
+    indexLoadingConfig.setReadMode(ReadMode.heap);
+
+    ImmutableSegment immutableSegment = ImmutableSegmentLoader.load(new File(INDEX_DIR, segmentName),
+        indexLoadingConfig);
+
+    Map<String, ColumnMetadata> columnMetadataMap1 = immutableSegment.getSegmentMetadata().getColumnMetadataMap();
+    columnMetadataMap1.forEach((column, metadata) -> {
+      if (column.equals("column6") || column.equals("column7")) {
+        assertTrue(metadata.hasDictionary());
+        assertTrue(metadata.isSingleValue());
+        assertNull(immutableSegment.getForwardIndex(column));
+      } else {
+        assertNotNull(immutableSegment.getForwardIndex(column));
+      }
+    });
+
+    return immutableSegment;
+  }
+
+  @AfterClass
+  public void deleteAndDestroySegment() {
+    FileUtils.deleteQuietly(INDEX_DIR);
+    _indexSegments.forEach((IndexSegment::destroy));
+  }
+
+  @Override
+  protected String getFilter() {
+    return FILTER;
+  }
+
+  @Override
+  protected IndexSegment getIndexSegment() {
+    return _indexSegment;
+  }
+
+  @Override
+  protected List<IndexSegment> getIndexSegments() {
+    return _indexSegments;
+  }
+
+  @Test
+  public void testSelectStarQueries() {
+    // Select * without any filters
+    try {
+      getBrokerResponse(SELECT_STAR_QUERY);
+      Assert.fail("Select * query should fail since forwardIndexDisabled on a select column");
+    } catch (IllegalStateException e) {
+      assertTrue(e.getMessage().contains("Forward index disabled for column:")
+          && e.getMessage().contains("cannot create DataFetcher!"));
+    }
+
+    // Select * with filters
+    try {
+      getBrokerResponse(SELECT_STAR_QUERY + FILTER);
+      Assert.fail("Select * query should fail since forwardIndexDisabled on a select column");
+    } catch (IllegalStateException e) {
+      assertTrue(e.getMessage().contains("Forward index disabled for column:")
+          && e.getMessage().contains("cannot create DataFetcher!"));
+    }
+  }
+
+  @Test
+  public void testSelectQueries() {
+    {
+      // Selection query without filters including a column with forwardIndexDisabled enabled on both segments
+      String query = "SELECT column1, column5, column6, column9, column11 FROM testTable";
+      try {
+        getBrokerResponse(query);
+        Assert.fail("Query should fail since forwardIndexDisabled on a select column");
+      } catch (IllegalStateException e) {
+        assertTrue(e.getMessage().contains("Forward index disabled for column:")
+            && e.getMessage().contains("cannot create DataFetcher!"));
+      }
+    }
+    {
+      // Selection query without filters including a column with forwardIndexDisabled enabled on one segment
+      String query = "SELECT column1, column5, column7, column9, column11 FROM testTable";
+      try {
+        getBrokerResponse(query);
+        Assert.fail("Query should fail since forwardIndexDisabled on a select column");
+      } catch (IllegalStateException e) {
+        assertTrue(e.getMessage().contains("Forward index disabled for column:")
+            && e.getMessage().contains("cannot create DataFetcher!"));
+      }
+    }
+    {
+      // Selection query with filters including a column with forwardIndexDisabled enabled on both segments
+      String query = "SELECT column1, column5, column6, column9, column11 FROM testTable WHERE column6 = 2147458029";
+      try {
+        getBrokerResponse(query);
+        Assert.fail("Query should fail since forwardIndexDisabled on a select column");
+      } catch (IllegalStateException e) {
+        assertTrue(e.getMessage().contains("Forward index disabled for column:")
+            && e.getMessage().contains("cannot create DataFetcher!"));
+      }
+    }
+    {
+      // Selection query with filters including a column with forwardIndexDisabled enabled on one segment
+      String query = "SELECT column1, column5, column7, column9, column11 FROM testTable WHERE column7 = 675695";
+      try {
+        getBrokerResponse(query);
+        Assert.fail("Query should fail since forwardIndexDisabled on a select column");
+      } catch (IllegalStateException e) {
+        assertTrue(e.getMessage().contains("Forward index disabled for column:")
+            && e.getMessage().contains("cannot create DataFetcher!"));
+      }
+    }
+    {
+      // Selection query without filters and without columns with forwardIndexDisabled enabled on either segment
+      String query = "SELECT column1, column5, column9, column11 FROM testTable ORDER BY column1";
+      BrokerResponseNative brokerResponseNative = getBrokerResponse(query);
+      assertTrue(brokerResponseNative.getProcessingExceptions() == null
+          || brokerResponseNative.getProcessingExceptions().size() == 0);
+      ResultTable resultTable = brokerResponseNative.getResultTable();
+      assertEquals(brokerResponseNative.getNumRowsResultSet(), 10);
+      assertEquals(brokerResponseNative.getTotalDocs(), 120_000L);
+      assertEquals(brokerResponseNative.getNumDocsScanned(), 120_000L);
+      assertEquals(brokerResponseNative.getNumSegmentsProcessed(), 4L);
+      assertEquals(brokerResponseNative.getNumSegmentsMatched(), 4L);
+      assertEquals(brokerResponseNative.getNumEntriesScannedPostFilter(), 120_120L);
+      assertEquals(brokerResponseNative.getNumEntriesScannedInFilter(), 0L);
+      assertNotNull(brokerResponseNative.getProcessingExceptions());
+      assertEquals(brokerResponseNative.getProcessingExceptions().size(), 0);
+      DataSchema dataSchema = new DataSchema(new String[]{"column1", "column5", "column9", "column11"},
+        new DataSchema.ColumnDataType[]{DataSchema.ColumnDataType.INT, DataSchema.ColumnDataType.STRING,
+            DataSchema.ColumnDataType.INT, DataSchema.ColumnDataType.STRING});
+      assertEquals(resultTable.getDataSchema(), dataSchema);
+      List<Object[]> resultRows = resultTable.getRows();
+      int previousColumn1 = Integer.MIN_VALUE;
+      for (Object[] resultRow : resultRows) {
+        assertEquals(resultRow.length, 4);
+        assertEquals((String) resultRow[1], "gFuH");
+        // Column 1
+        assertTrue((Integer) resultRow[0] >= previousColumn1);
+        previousColumn1 = (Integer) resultRow[0];
+      }
+
+      Object[] firstRow = resultRows.get(0);
+      // Column 11
+      assertEquals((String) firstRow[3], "o");
+    }
+    {
+      // Transform function on a selection clause with a forwardIndexDisabled column in transform
+      String query = "SELECT CONCAT(column6, column9, '-') from testTable";

Review Comment:
   I've added the following, does this cover them all?
   
   ```
   String query = "SELECT CONCAT(ADD(column1, column6), column5, '-') FROM testTable GROUP BY "
             + "CONCAT(ADD(column1, column6), column5, '-') ORDER BY CONCAT(ADD(column1, column6), column5, '-') LIMIT 10";
   
         String query = "SELECT CONCAT(ADD(column1, column9), column5, '-') FROM testTable GROUP BY "
             + "CONCAT(ADD(column1, column9), column5, '-') ORDER BY CONCAT(ADD(column1, column9), column5, '-') LIMIT 10";
   
         String query = "SELECT column1, column11 from testTable WHERE CONCAT(ADD(column6, column1), column9, '-') = "
             + "'1689277-11270'";
   
         String query = "SELECT column1, column11 from testTable WHERE CONCAT(column5, ADD(column9, column1), '-') = "
             + "'gFuH-2.96708164E8' ORDER BY column1";
   ```



-- 
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 #9333: Add an option to disable the creation of the forward index for a column

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


##########
pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/loader/defaultcolumn/BaseDefaultColumnHandler.java:
##########
@@ -363,6 +379,12 @@ protected boolean createColumnV1Indices(String column)
                   argument);
               return false;
             }
+            // TODO: Support creation of derived columns from forward index disabled columns
+            if (!_segmentWriter.hasIndexFor(argument, ColumnIndexType.FORWARD_INDEX)) {
+              LOGGER.warn("Skip creating derived column: {} because argument: {} does not have a forward index", column,

Review Comment:
   Throwing error is better to force the user to not create a derived column from noForwardIndex column until reload code path supports everything (including derived column handling possibly) ?



-- 
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 #9333: Add an option to disable the creation of the forward index for a column

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


##########
pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/column/PhysicalColumnIndexContainer.java:
##########
@@ -133,7 +133,12 @@ public PhysicalColumnIndexContainer(SegmentDirectory.Reader segmentReader, Colum
       _rangeIndex = null;
     }
 
-    PinotDataBuffer fwdIndexBuffer = segmentReader.getIndexFor(columnName, ColumnIndexType.FORWARD_INDEX);
+    // Setting the 'fwdIndexBuffer' to null if forward index is enabled. No-op index readers will be setup for the
+    // forward index disabled columns which doesn't require the 'fwdIndexBuffer'.
+    boolean forwardIndexDisabled = !segmentReader.hasIndexFor(columnName, ColumnIndexType.FORWARD_INDEX);
+    PinotDataBuffer fwdIndexBuffer = forwardIndexDisabled ? null
+        : segmentReader.getIndexFor(columnName, ColumnIndexType.FORWARD_INDEX);

Review Comment:
   Can we replace 2 calls with just 1 call to `getIndexFor` ?



-- 
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] jackjlli commented on a diff in pull request #9333: Add an option to disable the creation of the forward index for a column

Posted by GitBox <gi...@apache.org>.
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


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

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


##########
pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/creator/impl/SegmentColumnarIndexCreator.java:
##########
@@ -196,6 +204,15 @@ 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), String.format("Cannot disable forward "
+            + "index for column: %s without dictionary and inverted index or which is sorted or which has range "
+            + "index with version < 2", columnName));

Review Comment:
   (nit) may be cleaner to split this into 2-3 conditions. Will also make it clear to user what exact combination is invalid. Consider writing a simple helper method and fold everything into 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] siddharthteotia commented on a diff in pull request #9333: Add an option to disable the creation of the forward index for a column

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


##########
pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/creator/impl/DefaultIndexCreatorProvider.java:
##########
@@ -88,17 +90,25 @@ public ForwardIndexCreator newForwardIndexCreator(IndexCreationContext.Forward c
             context.getMaxRowLengthInBytes());
       }
     } else {
-      if (context.getFieldSpec().isSingleValueField()) {
-        if (context.isSorted()) {
-          return new SingleValueSortedForwardIndexCreator(context.getIndexDir(), context.getFieldSpec().getName(),
-              context.getCardinality());
+      if (context.forwardIndexDisabled()) {

Review Comment:
   (nit) not related to this PR -- do you mind adding a comment under the else branch above line 93 ? something like `Dictionary enabled column`. Will make it bit more readable. Similarly above line 79.
   
   We may also want to make this more readable by having separate helper method but separate PR for that is fine too



-- 
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 #9333: Add an option to disable the creation of the forward index for a column

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


##########
pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/OfflineClusterIntegrationTest.java:
##########
@@ -143,7 +143,7 @@ public class OfflineClusterIntegrationTest extends BaseClusterIntegrationTestSet
   private static final String MAX_NUM_MULTI_VALUES_MAP_KEY = "maxNumMultiValuesMap";
   // TODO: This might lead to flaky test, as this disk size is not deterministic
   //       as it depends on the iteration order of a HashSet.
-  private static final int DISK_SIZE_IN_BYTES = 20797128;
+  private static final int DISK_SIZE_IN_BYTES = 20843088;

Review Comment:
   Why this change ?



-- 
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 #9333: Add an option to disable the creation of the forward index for a column

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


##########
pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/creator/impl/SegmentColumnarIndexCreator.java:
##########
@@ -196,6 +204,15 @@ 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), String.format("Cannot disable forward "
+            + "index for column: %s without dictionary and inverted index or which is sorted or which has range "
+            + "index with version < 2", columnName));

Review Comment:
   good point, 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 #9333: Add an option to disable the creation of the forward index for a column

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


##########
pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/readers/forward/NoOpMVForwardIndexReader.java:
##########
@@ -0,0 +1,66 @@
+/**
+ * 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.segment.local.segment.index.readers.forward;
+
+import java.io.IOException;
+import org.apache.pinot.segment.spi.index.reader.ForwardIndexReader;
+import org.apache.pinot.segment.spi.index.reader.ForwardIndexReaderContext;
+import org.apache.pinot.spi.data.FieldSpec;
+
+
+/**
+ * No-op forward index reader meant to be used by multi-value columns with forward index disabled. This is meant to
+ * provide clean exception handling for operations which need to read forward index when forward index is disabled.
+ */
+public class NoOpMVForwardIndexReader implements ForwardIndexReader<ForwardIndexReaderContext> {

Review Comment:
   I am not sure I follow. Throwing exception in multiple places using if checks is special casing / specializing it as opposed to using NO-OP reader like done in this PR. We can discuss this more 



-- 
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 #9333: Add an option to disable the creation of the forward index for a column

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


##########
pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/creator/impl/SegmentColumnarIndexCreator.java:
##########
@@ -196,6 +204,15 @@ 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)

Review Comment:
   hey @Jackie-Jiang can you check my comment about sorted index on your first comment on this PR? Wanted to know your thoughts based on 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] somandal commented on a diff in pull request #9333: Add an option to disable the creation of the forward index for a column

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


##########
pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/creator/impl/SegmentColumnarIndexCreator.java:
##########
@@ -816,6 +835,7 @@ public static void addColumnMetadataInfo(PropertiesConfiguration properties, Str
         String.valueOf(columnIndexCreationInfo.getTotalNumberOfEntries()));
     properties.setProperty(getKeyFor(column, IS_AUTO_GENERATED),
         String.valueOf(columnIndexCreationInfo.isAutoGenerated()));
+    properties.setProperty(getKeyFor(column, FORWARD_INDEX_DISABLED), String.valueOf(forwardIndexDisabled));

Review Comment:
   done! removed 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] somandal commented on a diff in pull request #9333: Add an option to disable the creation of the forward index for a column

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


##########
pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/OfflineClusterIntegrationTest.java:
##########
@@ -143,7 +143,7 @@ public class OfflineClusterIntegrationTest extends BaseClusterIntegrationTestSet
   private static final String MAX_NUM_MULTI_VALUES_MAP_KEY = "maxNumMultiValuesMap";
   // TODO: This might lead to flaky test, as this disk size is not deterministic
   //       as it depends on the iteration order of a HashSet.
-  private static final int DISK_SIZE_IN_BYTES = 20797128;
+  private static final int DISK_SIZE_IN_BYTES = 20843088;

Review Comment:
   This change is no longer needed since we removed the field from the metadata. I've changed it back to the original value.



-- 
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 #9333: Add an option to disable the creation of the forward index for a column

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


##########
pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/loader/defaultcolumn/BaseDefaultColumnHandler.java:
##########
@@ -388,8 +416,40 @@ protected boolean createColumnV1Indices(String column)
     return true;
   }
 
+  /**
+   * Validates the compatibility of the indexes if the column has the forward index disabled. Throws exceptions due to
+   * compatibility mismatch. The checks performed are:
+   *     - Validate dictionary is enabled.
+   *     - Validate inverted index is enabled.
+   *     - Validate that either no range index exists for column or the range index version is at least 2 and isn't a
+   *       multi-value column (since multi-value defaults to index v1).
+   */
+  protected void validateForwardIndexDisabledConfigsIfPresent(String column, boolean forwardIndexDisabled) {

Review Comment:
   Why is this function needed here ? Table config validator should catch the unsupported combinations / mandatory conditions since anyone who wants to toggle the flag and then leverage reload path has to first update the table config successfully on the controller (which should internally use validator)



-- 
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 #9333: Add an option to disable the creation of the forward index for a column

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


##########
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",

Review Comment:
   Do we have test for this feature with one of INT / FLOAT / LONG ... other than STRING ?



-- 
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 #9333: Add an option to disable the creation of the forward index for a column

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


##########
pinot-core/src/test/java/org/apache/pinot/queries/ForwardIndexDisabledSingleValueQueriesTest.java:
##########
@@ -0,0 +1,1088 @@
+/**
+ * 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.queries;
+
+import java.io.File;
+import java.net.URL;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.TimeUnit;
+import org.apache.commons.io.FileUtils;
+import org.apache.pinot.common.response.broker.BrokerResponseNative;
+import org.apache.pinot.common.response.broker.ResultTable;
+import org.apache.pinot.common.utils.DataSchema;
+import org.apache.pinot.segment.local.indexsegment.immutable.ImmutableSegmentLoader;
+import org.apache.pinot.segment.local.segment.creator.impl.SegmentIndexCreationDriverImpl;
+import org.apache.pinot.segment.local.segment.index.loader.IndexLoadingConfig;
+import org.apache.pinot.segment.spi.ColumnMetadata;
+import org.apache.pinot.segment.spi.ImmutableSegment;
+import org.apache.pinot.segment.spi.IndexSegment;
+import org.apache.pinot.segment.spi.creator.SegmentGeneratorConfig;
+import org.apache.pinot.segment.spi.creator.SegmentIndexCreationDriver;
+import org.apache.pinot.spi.config.table.FieldConfig;
+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.data.TimeGranularitySpec;
+import org.apache.pinot.spi.utils.ReadMode;
+import org.apache.pinot.spi.utils.builder.TableConfigBuilder;
+import org.testng.Assert;
+import org.testng.annotations.AfterClass;
+import org.testng.annotations.BeforeClass;
+import org.testng.annotations.Test;
+
+import static org.testng.Assert.assertEquals;
+import static org.testng.Assert.assertNotNull;
+import static org.testng.Assert.assertNull;
+import static org.testng.Assert.assertTrue;
+
+
+/**
+ * The <code>ForwardIndexDisabledSingleValueQueriesTest</code> class sets up the index segment for the no forward
+ * index single-value queries test.
+ * <p>There are totally 18 columns, 30000 records inside the original Avro file where 11 columns are selected to build
+ * the index segment. Selected columns information are as following:
+ * <ul>
+ *   ColumnName, FieldType, DataType, Cardinality, IsSorted, HasInvertedIndex, FwdIndexDisabled: S1, S2, HasRangeIndex
+ *   <li>column1, METRIC, INT, 6582, F, F, F, F, F</li>
+ *   <li>column3, METRIC, INT, 21910, F, F, F, F, F</li>
+ *   <li>column5, DIMENSION, STRING, 1, T, F, F, F, F</li>
+ *   <li>column6, DIMENSION, INT, 608, F, T, T, T, T</li>
+ *   <li>column7, DIMENSION, INT, 146, F, T, T, F, F</li>
+ *   <li>column9, DIMENSION, INT, 1737, F, F, F, F, F</li>
+ *   <li>column11, DIMENSION, STRING, 5, F, T, F, F, F</li>
+ *   <li>column12, DIMENSION, STRING, 5, F, F, F, F, F</li>
+ *   <li>column17, METRIC, INT, 24, F, T, F, F, F</li>
+ *   <li>column18, METRIC, INT, 1440, F, T, F, F, F</li>
+ *   <li>daysSinceEpoch, TIME, INT, 2, T, F, F, F, F</li>
+ * </ul>
+ */
+public class ForwardIndexDisabledSingleValueQueriesTest extends BaseQueriesTest {
+  private static final String AVRO_DATA = "data" + File.separator + "test_data-sv.avro";
+  private static final String SEGMENT_NAME_1 = "testTable_126164076_167572857";
+  private static final String SEGMENT_NAME_2 = "testTable_126164076_167572858";
+  private static final File INDEX_DIR = new File(FileUtils.getTempDirectory(),
+      "ForwardIndexDisabledSingleValueQueriesTest");
+
+  private static final String SELECT_STAR_QUERY = "SELECT * FROM testTable";
+
+  // Hard-coded query filter.
+  private static final String FILTER = " WHERE column1 > 100000000"
+      + " AND column3 BETWEEN 20000000 AND 1000000000"
+      + " AND column5 = 'gFuH'"
+      + " AND (column6 < 500000000 OR column11 NOT IN ('t', 'P'))"
+      + " AND daysSinceEpoch = 126164076";
+
+  private IndexSegment _indexSegment;
+  // Contains 2 index segments, one with 2 columns with forward index disabled, and the other with just 1.
+  private List<IndexSegment> _indexSegments;
+
+  private TableConfig _tableConfig;
+  private List<String> _invertedIndexColumns;
+  private List<String> _forwardIndexDisabledColumns;
+
+  @BeforeClass
+  public void buildAndLoadSegment()
+      throws Exception {
+    FileUtils.deleteQuietly(INDEX_DIR);
+
+    // Get resource file path.
+    URL resource = getClass().getClassLoader().getResource(AVRO_DATA);
+    assertNotNull(resource);
+    String filePath = resource.getFile();
+
+    // Build the segment schema.
+    Schema schema = new Schema.SchemaBuilder().setSchemaName("testTable").addMetric("column1", FieldSpec.DataType.INT)
+        .addMetric("column3", FieldSpec.DataType.INT).addSingleValueDimension("column5", FieldSpec.DataType.STRING)
+        .addSingleValueDimension("column6", FieldSpec.DataType.INT)
+        .addSingleValueDimension("column7", FieldSpec.DataType.INT)
+        .addSingleValueDimension("column9", FieldSpec.DataType.INT)
+        .addSingleValueDimension("column11", FieldSpec.DataType.STRING)
+        .addSingleValueDimension("column12", FieldSpec.DataType.STRING).addMetric("column17", FieldSpec.DataType.INT)
+        .addMetric("column18", FieldSpec.DataType.INT)
+        .addTime(new TimeGranularitySpec(FieldSpec.DataType.INT, TimeUnit.DAYS, "daysSinceEpoch"), null).build();
+
+    createSegment(filePath, SEGMENT_NAME_1, schema);
+    createSegment(filePath, SEGMENT_NAME_2, schema);
+
+    ImmutableSegment immutableSegment1 = loadSegmentWithMetadataChecks(SEGMENT_NAME_1);
+    ImmutableSegment immutableSegment2 = loadSegmentWithMetadataChecks(SEGMENT_NAME_2);
+
+    _indexSegment = immutableSegment1;
+    _indexSegments = Arrays.asList(immutableSegment1, immutableSegment2);
+  }
+
+  private void createSegment(String filePath, String segmentName, Schema schema)
+      throws Exception {
+    // Create field configs for the no forward index columns
+    List<FieldConfig> fieldConfigList = new ArrayList<>();
+    fieldConfigList.add(new FieldConfig("column6", FieldConfig.EncodingType.DICTIONARY, Collections.emptyList(), null,
+        Collections.singletonMap(FieldConfig.FORWARD_INDEX_DISABLED, Boolean.TRUE.toString())));
+    if (segmentName.equals(SEGMENT_NAME_1)) {
+      fieldConfigList.add(new FieldConfig("column7", FieldConfig.EncodingType.DICTIONARY, Collections.emptyList(),
+          null, Collections.singletonMap(FieldConfig.FORWARD_INDEX_DISABLED, Boolean.TRUE.toString())));
+
+      // Build table config based on segment 1 as it contains both columns under no forward index
+      _tableConfig =
+          new TableConfigBuilder(TableType.OFFLINE).setTableName("testTable").setTimeColumnName("daysSinceEpoch")
+              .setFieldConfigList(fieldConfigList).setRangeIndexColumns(Arrays.asList("column6")).build();
+    }
+
+    // Create the segment generator config.
+    SegmentGeneratorConfig segmentGeneratorConfig = new SegmentGeneratorConfig(_tableConfig, schema);
+    segmentGeneratorConfig.setInputFilePath(filePath);
+    segmentGeneratorConfig.setTableName("testTable");
+    segmentGeneratorConfig.setOutDir(INDEX_DIR.getAbsolutePath());
+    segmentGeneratorConfig.setSegmentName(segmentName);
+    // The segment generation code in SegmentColumnarIndexCreator will throw
+    // exception if start and end time in time column are not in acceptable
+    // range. For this test, we first need to fix the input avro data
+    // to have the time column values in allowed range. Until then, the check
+    // is explicitly disabled
+    segmentGeneratorConfig.setSkipTimeValueCheck(true);
+    _invertedIndexColumns = Arrays.asList("column6", "column7", "column11", "column17", "column18");
+    segmentGeneratorConfig.setInvertedIndexCreationColumns(_invertedIndexColumns);
+
+    _forwardIndexDisabledColumns = new ArrayList<>(Arrays.asList("column6", "column7"));
+    segmentGeneratorConfig.setForwardIndexDisabledColumns(_forwardIndexDisabledColumns);
+    segmentGeneratorConfig.setRangeIndexCreationColumns(Arrays.asList("column6"));
+
+    // Build the index segment.
+    SegmentIndexCreationDriver driver = new SegmentIndexCreationDriverImpl();
+    driver.init(segmentGeneratorConfig);
+    driver.build();
+  }
+
+  private ImmutableSegment loadSegmentWithMetadataChecks(String segmentName)
+      throws Exception {
+    IndexLoadingConfig indexLoadingConfig = new IndexLoadingConfig();
+    indexLoadingConfig.setTableConfig(_tableConfig);
+    indexLoadingConfig.setInvertedIndexColumns(new HashSet<>(_invertedIndexColumns));
+    indexLoadingConfig.setForwardIndexDisabledColumns(new HashSet<>(_forwardIndexDisabledColumns));
+    indexLoadingConfig.setRangeIndexColumns(new HashSet<>(Arrays.asList("column6")));
+    indexLoadingConfig.setReadMode(ReadMode.heap);
+
+    ImmutableSegment immutableSegment = ImmutableSegmentLoader.load(new File(INDEX_DIR, segmentName),
+        indexLoadingConfig);
+
+    Map<String, ColumnMetadata> columnMetadataMap1 = immutableSegment.getSegmentMetadata().getColumnMetadataMap();
+    columnMetadataMap1.forEach((column, metadata) -> {
+      if (column.equals("column6") || column.equals("column7")) {
+        assertTrue(metadata.hasDictionary());
+        assertTrue(metadata.isSingleValue());
+        assertNull(immutableSegment.getForwardIndex(column));
+      } else {
+        assertNotNull(immutableSegment.getForwardIndex(column));
+      }
+    });
+
+    return immutableSegment;
+  }
+
+  @AfterClass
+  public void deleteAndDestroySegment() {
+    FileUtils.deleteQuietly(INDEX_DIR);
+    _indexSegments.forEach((IndexSegment::destroy));
+  }
+
+  @Override
+  protected String getFilter() {
+    return FILTER;
+  }
+
+  @Override
+  protected IndexSegment getIndexSegment() {
+    return _indexSegment;
+  }
+
+  @Override
+  protected List<IndexSegment> getIndexSegments() {
+    return _indexSegments;
+  }
+
+  @Test
+  public void testSelectStarQueries() {
+    // Select * without any filters
+    try {
+      getBrokerResponse(SELECT_STAR_QUERY);
+      Assert.fail("Select * query should fail since forwardIndexDisabled on a select column");
+    } catch (IllegalStateException e) {
+      assertTrue(e.getMessage().contains("Forward index disabled for column:")
+          && e.getMessage().contains("cannot create DataFetcher!"));
+    }
+
+    // Select * with filters
+    try {
+      getBrokerResponse(SELECT_STAR_QUERY + FILTER);
+      Assert.fail("Select * query should fail since forwardIndexDisabled on a select column");
+    } catch (IllegalStateException e) {
+      assertTrue(e.getMessage().contains("Forward index disabled for column:")
+          && e.getMessage().contains("cannot create DataFetcher!"));
+    }
+  }
+
+  @Test
+  public void testSelectQueries() {
+    {
+      // Selection query without filters including a column with forwardIndexDisabled enabled on both segments
+      String query = "SELECT column1, column5, column6, column9, column11 FROM testTable";
+      try {
+        getBrokerResponse(query);
+        Assert.fail("Query should fail since forwardIndexDisabled on a select column");
+      } catch (IllegalStateException e) {
+        assertTrue(e.getMessage().contains("Forward index disabled for column:")
+            && e.getMessage().contains("cannot create DataFetcher!"));
+      }
+    }
+    {
+      // Selection query without filters including a column with forwardIndexDisabled enabled on one segment
+      String query = "SELECT column1, column5, column7, column9, column11 FROM testTable";
+      try {
+        getBrokerResponse(query);
+        Assert.fail("Query should fail since forwardIndexDisabled on a select column");
+      } catch (IllegalStateException e) {
+        assertTrue(e.getMessage().contains("Forward index disabled for column:")
+            && e.getMessage().contains("cannot create DataFetcher!"));
+      }
+    }
+    {
+      // Selection query with filters including a column with forwardIndexDisabled enabled on both segments
+      String query = "SELECT column1, column5, column6, column9, column11 FROM testTable WHERE column6 = 2147458029";
+      try {
+        getBrokerResponse(query);
+        Assert.fail("Query should fail since forwardIndexDisabled on a select column");
+      } catch (IllegalStateException e) {
+        assertTrue(e.getMessage().contains("Forward index disabled for column:")
+            && e.getMessage().contains("cannot create DataFetcher!"));
+      }
+    }
+    {
+      // Selection query with filters including a column with forwardIndexDisabled enabled on one segment
+      String query = "SELECT column1, column5, column7, column9, column11 FROM testTable WHERE column7 = 675695";
+      try {
+        getBrokerResponse(query);
+        Assert.fail("Query should fail since forwardIndexDisabled on a select column");
+      } catch (IllegalStateException e) {
+        assertTrue(e.getMessage().contains("Forward index disabled for column:")
+            && e.getMessage().contains("cannot create DataFetcher!"));
+      }
+    }
+    {
+      // Selection query without filters and without columns with forwardIndexDisabled enabled on either segment
+      String query = "SELECT column1, column5, column9, column11 FROM testTable ORDER BY column1";
+      BrokerResponseNative brokerResponseNative = getBrokerResponse(query);
+      assertTrue(brokerResponseNative.getProcessingExceptions() == null
+          || brokerResponseNative.getProcessingExceptions().size() == 0);
+      ResultTable resultTable = brokerResponseNative.getResultTable();
+      assertEquals(brokerResponseNative.getNumRowsResultSet(), 10);
+      assertEquals(brokerResponseNative.getTotalDocs(), 120_000L);
+      assertEquals(brokerResponseNative.getNumDocsScanned(), 120_000L);
+      assertEquals(brokerResponseNative.getNumSegmentsProcessed(), 4L);
+      assertEquals(brokerResponseNative.getNumSegmentsMatched(), 4L);
+      assertEquals(brokerResponseNative.getNumEntriesScannedPostFilter(), 120_120L);
+      assertEquals(brokerResponseNative.getNumEntriesScannedInFilter(), 0L);
+      assertNotNull(brokerResponseNative.getProcessingExceptions());
+      assertEquals(brokerResponseNative.getProcessingExceptions().size(), 0);
+      DataSchema dataSchema = new DataSchema(new String[]{"column1", "column5", "column9", "column11"},
+        new DataSchema.ColumnDataType[]{DataSchema.ColumnDataType.INT, DataSchema.ColumnDataType.STRING,
+            DataSchema.ColumnDataType.INT, DataSchema.ColumnDataType.STRING});
+      assertEquals(resultTable.getDataSchema(), dataSchema);
+      List<Object[]> resultRows = resultTable.getRows();
+      int previousColumn1 = Integer.MIN_VALUE;
+      for (Object[] resultRow : resultRows) {
+        assertEquals(resultRow.length, 4);
+        assertEquals((String) resultRow[1], "gFuH");
+        // Column 1
+        assertTrue((Integer) resultRow[0] >= previousColumn1);
+        previousColumn1 = (Integer) resultRow[0];
+      }
+
+      Object[] firstRow = resultRows.get(0);
+      // Column 11
+      assertEquals((String) firstRow[3], "o");
+    }
+    {
+      // Transform function on a selection clause with a forwardIndexDisabled column in transform
+      String query = "SELECT CONCAT(column6, column9, '-') from testTable";

Review Comment:
   Also add the transform scenario for WHERE clause, GROUP BY 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] somandal commented on a diff in pull request #9333: Add an option to disable the creation of the forward index for a column

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


##########
pinot-segment-local/src/test/java/org/apache/pinot/segment/local/utils/TableConfigUtilsTest.java:
##########
@@ -895,6 +895,63 @@ public void testValidateFieldConfig() {
     } catch (Exception e) {
       Assert.assertEquals(e.getMessage(), "Set compression codec to null for dictionary encoding type");
     }
+
+    tableConfig = new TableConfigBuilder(TableType.OFFLINE).setTableName(TABLE_NAME)
+        .setNoDictionaryColumns(Arrays.asList("myCol1")).build();
+    try {
+      // Enable forward index disabled flag for a raw column
+      Map<String, String> fieldConfigProperties = new HashMap<>();
+      fieldConfigProperties.put(FieldConfig.FORWARD_INDEX_DISABLED, Boolean.TRUE.toString());
+      FieldConfig fieldConfig = new FieldConfig("myCol1", FieldConfig.EncodingType.RAW, null, null, null, null,
+          fieldConfigProperties);
+      tableConfig.setFieldConfigList(Arrays.asList(fieldConfig));
+      TableConfigUtils.validate(tableConfig, schema);
+      Assert.fail("Should fail for myCol1 without dictionary and with forward index disabled");
+    } catch (Exception e) {
+      Assert.assertEquals(e.getMessage(), "Forward index disabled column myCol1 must have dictionary enabled");
+    }
+
+    try {
+      // Enable forward index disabled flag for a column without inverted index
+      Map<String, String> fieldConfigProperties = new HashMap<>();
+      fieldConfigProperties.put(FieldConfig.FORWARD_INDEX_DISABLED, Boolean.TRUE.toString());
+      FieldConfig fieldConfig = new FieldConfig("myCol2", FieldConfig.EncodingType.DICTIONARY, null, null, null, null,
+          fieldConfigProperties);
+      tableConfig.setFieldConfigList(Arrays.asList(fieldConfig));
+      TableConfigUtils.validate(tableConfig, schema);
+      Assert.fail("Should fail for conflicting myCol2 with forward index disabled but no inverted index");
+    } catch (Exception e) {
+      Assert.assertEquals(e.getMessage(), "Forward index disabled column myCol2 must have inverted index enabled");
+    }
+
+    tableConfig = new TableConfigBuilder(TableType.OFFLINE).setTableName(TABLE_NAME)
+        .setNoDictionaryColumns(Arrays.asList("myCol1")).setInvertedIndexColumns(Arrays.asList("myCol2")).build();
+    try {
+      // Enable forward index disabled flag for a column with inverted index
+      Map<String, String> fieldConfigProperties = new HashMap<>();
+      fieldConfigProperties.put(FieldConfig.FORWARD_INDEX_DISABLED, Boolean.TRUE.toString());
+      FieldConfig fieldConfig = new FieldConfig("myCol2", FieldConfig.EncodingType.DICTIONARY,
+          FieldConfig.IndexType.INVERTED, null, null, null, fieldConfigProperties);
+      tableConfig.setFieldConfigList(Arrays.asList(fieldConfig));
+      TableConfigUtils.validate(tableConfig, schema);
+    } catch (Exception e) {
+      Assert.fail("Should not fail as myCol2 has forward index disabled but inverted index enabled");
+    }
+
+    tableConfig = new TableConfigBuilder(TableType.OFFLINE).setTableName(TABLE_NAME)
+        .setNoDictionaryColumns(Arrays.asList("myCol1")).setInvertedIndexColumns(Arrays.asList("myCol2"))
+        .setSortedColumn("myCol2").build();
+    try {
+      // Enable forward index disabled flag for a column with inverted index and is sorted
+      Map<String, String> fieldConfigProperties = new HashMap<>();
+      fieldConfigProperties.put(FieldConfig.FORWARD_INDEX_DISABLED, Boolean.TRUE.toString());
+      FieldConfig fieldConfig = new FieldConfig("myCol2", FieldConfig.EncodingType.DICTIONARY,
+          FieldConfig.IndexType.INVERTED, null, null, null, fieldConfigProperties);
+      tableConfig.setFieldConfigList(Arrays.asList(fieldConfig));
+      TableConfigUtils.validate(tableConfig, schema);
+    } catch (Exception e) {
+      Assert.fail("Should not fail for myCol2 with forward index disabled but is sorted, this is a no-op");

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 #9333: Add an option to disable the creation of the forward index for a column

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


##########
pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/loader/defaultcolumn/BaseDefaultColumnHandler.java:
##########
@@ -224,6 +225,21 @@ Map<String, DefaultColumnAction> computeDefaultColumnActionMap() {
       if (columnMetadata != null) {
         // Column exists in the segment, check if we need to update the value.
 
+        if (_segmentWriter != null && !columnMetadata.isAutoGenerated()) {
+          // Check that forward index disabled isn't enabled / disabled on an existing column (not auto-generated).
+          // TODO: Add support for reloading segments when forward index disabled flag is enabled or disabled
+          boolean forwardIndexDisabled = !_segmentWriter.hasIndexFor(column, ColumnIndexType.FORWARD_INDEX);
+          if (forwardIndexDisabled != _indexLoadingConfig.getForwardIndexDisabledColumns()
+              .contains(column)) {
+            String failureMessage =
+                "Forward index disabled in segment: " + forwardIndexDisabled + " for column: " + column
+                    + " does not match forward index disabled flag: "
+                    + _indexLoadingConfig.getForwardIndexDisabledColumns().contains(column) + " in the TableConfig, "
+                    + "updating this flag is not supported at the moment.";

Review Comment:
   I've modified the comment but I still think we should fail and throw an exception here. let's discuss more about this if you think a warning log is enough.



-- 
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 #9333: Add an option to disable the creation of the forward index for a column

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


##########
pinot-segment-local/src/main/java/org/apache/pinot/segment/local/utils/TableConfigUtils.java:
##########
@@ -897,6 +903,40 @@ private static void validateFieldConfigList(@Nullable List<FieldConfig> fieldCon
     }
   }
 
+  /**
+   * Validates the compatibility of the indexes if the column has the forward index disabled. Throws exceptions due to
+   * compatibility mismatch. The checks performed are:
+   *     - Validate dictionary is enabled.
+   *     - Validate inverted index is enabled.
+   *     - Validate that either no range index exists for column or the range index version is at least 2 and isn't a
+   *       multi-value column (since mulit-value defaults to index v1).
+   */
+  private static void validateForwardIndexDisabledIndexCompatibility(String columnName, FieldConfig fieldConfig,

Review Comment:
   yup agree. will definitely remove what we don't need for reload follow ups. Will also try this out in a perf cluster maybe to ensure that just the controller code path is sufficient to catch all 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] siddharthteotia commented on a diff in pull request #9333: Add an option to disable the creation of the forward index for a column

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


##########
pinot-segment-local/src/test/java/org/apache/pinot/segment/local/utils/TableConfigUtilsTest.java:
##########
@@ -895,6 +895,63 @@ public void testValidateFieldConfig() {
     } catch (Exception e) {
       Assert.assertEquals(e.getMessage(), "Set compression codec to null for dictionary encoding type");
     }
+
+    tableConfig = new TableConfigBuilder(TableType.OFFLINE).setTableName(TABLE_NAME)
+        .setNoDictionaryColumns(Arrays.asList("myCol1")).build();
+    try {
+      // Enable forward index disabled flag for a raw column
+      Map<String, String> fieldConfigProperties = new HashMap<>();
+      fieldConfigProperties.put(FieldConfig.FORWARD_INDEX_DISABLED, Boolean.TRUE.toString());
+      FieldConfig fieldConfig = new FieldConfig("myCol1", FieldConfig.EncodingType.RAW, null, null, null, null,
+          fieldConfigProperties);
+      tableConfig.setFieldConfigList(Arrays.asList(fieldConfig));
+      TableConfigUtils.validate(tableConfig, schema);
+      Assert.fail("Should fail for myCol1 without dictionary and with forward index disabled");
+    } catch (Exception e) {
+      Assert.assertEquals(e.getMessage(), "Forward index disabled column myCol1 must have dictionary enabled");
+    }
+
+    try {
+      // Enable forward index disabled flag for a column without inverted index
+      Map<String, String> fieldConfigProperties = new HashMap<>();
+      fieldConfigProperties.put(FieldConfig.FORWARD_INDEX_DISABLED, Boolean.TRUE.toString());
+      FieldConfig fieldConfig = new FieldConfig("myCol2", FieldConfig.EncodingType.DICTIONARY, null, null, null, null,
+          fieldConfigProperties);
+      tableConfig.setFieldConfigList(Arrays.asList(fieldConfig));
+      TableConfigUtils.validate(tableConfig, schema);
+      Assert.fail("Should fail for conflicting myCol2 with forward index disabled but no inverted index");
+    } catch (Exception e) {
+      Assert.assertEquals(e.getMessage(), "Forward index disabled column myCol2 must have inverted index enabled");
+    }
+
+    tableConfig = new TableConfigBuilder(TableType.OFFLINE).setTableName(TABLE_NAME)
+        .setNoDictionaryColumns(Arrays.asList("myCol1")).setInvertedIndexColumns(Arrays.asList("myCol2")).build();
+    try {
+      // Enable forward index disabled flag for a column with inverted index
+      Map<String, String> fieldConfigProperties = new HashMap<>();
+      fieldConfigProperties.put(FieldConfig.FORWARD_INDEX_DISABLED, Boolean.TRUE.toString());
+      FieldConfig fieldConfig = new FieldConfig("myCol2", FieldConfig.EncodingType.DICTIONARY,
+          FieldConfig.IndexType.INVERTED, null, null, null, fieldConfigProperties);
+      tableConfig.setFieldConfigList(Arrays.asList(fieldConfig));
+      TableConfigUtils.validate(tableConfig, schema);
+    } catch (Exception e) {
+      Assert.fail("Should not fail as myCol2 has forward index disabled but inverted index enabled");
+    }
+
+    tableConfig = new TableConfigBuilder(TableType.OFFLINE).setTableName(TABLE_NAME)
+        .setNoDictionaryColumns(Arrays.asList("myCol1")).setInvertedIndexColumns(Arrays.asList("myCol2"))
+        .setSortedColumn("myCol2").build();
+    try {
+      // Enable forward index disabled flag for a column with inverted index and is sorted
+      Map<String, String> fieldConfigProperties = new HashMap<>();
+      fieldConfigProperties.put(FieldConfig.FORWARD_INDEX_DISABLED, Boolean.TRUE.toString());
+      FieldConfig fieldConfig = new FieldConfig("myCol2", FieldConfig.EncodingType.DICTIONARY,
+          FieldConfig.IndexType.INVERTED, null, null, null, fieldConfigProperties);
+      tableConfig.setFieldConfigList(Arrays.asList(fieldConfig));
+      TableConfigUtils.validate(tableConfig, schema);
+    } catch (Exception e) {
+      Assert.fail("Should not fail for myCol2 with forward index disabled but is sorted, this is a no-op");

Review Comment:
   Add a test for range index combination scenario 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 #9333: Add an option to disable the creation of the forward index for a column

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


##########
pinot-segment-local/src/main/java/org/apache/pinot/segment/local/utils/TableConfigUtils.java:
##########
@@ -897,6 +903,40 @@ private static void validateFieldConfigList(@Nullable List<FieldConfig> fieldCon
     }
   }
 
+  /**
+   * Validates the compatibility of the indexes if the column has the forward index disabled. Throws exceptions due to
+   * compatibility mismatch. The checks performed are:
+   *     - Validate dictionary is enabled.
+   *     - Validate inverted index is enabled.
+   *     - Validate that either no range index exists for column or the range index version is at least 2 and isn't a
+   *       multi-value column (since mulit-value defaults to index v1).
+   */
+  private static void validateForwardIndexDisabledIndexCompatibility(String columnName, FieldConfig fieldConfig,
+      IndexingConfig indexingConfigs, List<String> noDictionaryColumns, Schema schema) {
+    Map<String, String> fieldConfigProperties = fieldConfig.getProperties();
+    if (fieldConfigProperties == null) {
+      return;
+    }
+
+    boolean forwardIndexDisabled = Boolean.parseBoolean(fieldConfigProperties.get(FieldConfig.FORWARD_INDEX_DISABLED));
+    if (!forwardIndexDisabled) {
+      return;
+    }
+
+    FieldSpec fieldSpec = schema.getFieldSpecFor(columnName);
+    Preconditions.checkState(noDictionaryColumns == null || !noDictionaryColumns.contains(columnName),

Review Comment:
   You should also check that same column has been marked as `RAW` in the `encodingType` in `FieldConfig` since eventually that will be used alone and `noDictionaryColumns` will be deprecated at some point after the migration (hopefully)



-- 
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 #9333: Add an option to disable the creation of the forward index for a column

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


##########
pinot-core/src/main/java/org/apache/pinot/core/operator/filter/ScanBasedFilterOperator.java:
##########
@@ -43,6 +44,9 @@ public class ScanBasedFilterOperator extends BaseFilterOperator {
     _dataSource = dataSource;
     _numDocs = numDocs;
     _nullHandlingEnabled = nullHandlingEnabled;
+    Preconditions.checkState(_dataSource.getForwardIndex() != null,
+        "Forward index disabled for column: %s, creating ScanDocIdSet unsupported!",

Review Comment:
   (nit) a better message may be -- `creating ScanDocIdSet unsupported -> scan based filtering not supported`



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