You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@carbondata.apache.org by manishgupta88 <gi...@git.apache.org> on 2017/05/25 13:29:43 UTC

[GitHub] carbondata pull request #952: [CARBONDATA-1094] Wrong results returned by th...

GitHub user manishgupta88 opened a pull request:

    https://github.com/apache/carbondata/pull/952

    [CARBONDATA-1094] Wrong results returned by the query in case inverted index is not created on a column

    Problem: Wrong results returned by the query in case inverted index is not created on a column
    
    Fix: When inverted index does not exist for a column or column is not a sort column then
    1. Block or blocklet cannot be pruned as data for that column is not sorted
    2. While applying the filter linear search should be applied instead of binary search as binary search can be applied only on sorted data
    
    Verified result
    ------------------
    SELECT * FROM index1 WHERE city >= 'Shanghai'
    +---+-----+----------+
    | id| name|      city|
    +---+-----+----------+
    | 11|James|Washington|
    | 20|Kevin| Singapore|
    |  9| Mary|     Tokyo|
    | 16| Paul|  Shanghai|
    |  4| Sara|     Tokyo|
    +---+-----+----------+

You can merge this pull request into a Git repository by running:

    $ git pull https://github.com/manishgupta88/incubator-carbondata inverted_index_filter_issue_fix

Alternatively you can review and apply these changes as the patch at:

    https://github.com/apache/carbondata/pull/952.patch

To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:

    This closes #952
    
----
commit 87d7bb2bd74569905f5c24e9cb91735df8ac27da
Author: manishgupta88 <to...@gmail.com>
Date:   2017-05-25T13:14:30Z

    Problem: Wrong results returned by the query in case inverted index is not created on a column
    
    Fix: When inverted index does not exist for a column or column is not a sort column then
    1. Block or blocklet cannot be pruned as data for that column is not sorted
    2. While applying the filter linear search should be applied instead of binary search as binary search can be applied only on sorted data

----


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] carbondata pull request #952: [CARBONDATA-1094] Wrong results returned by th...

Posted by kumarvishal09 <gi...@git.apache.org>.
Github user kumarvishal09 commented on a diff in the pull request:

    https://github.com/apache/carbondata/pull/952#discussion_r118667577
  
    --- Diff: core/src/main/java/org/apache/carbondata/core/scan/filter/executer/RowLevelRangeGrtThanFiterExecuterImpl.java ---
    @@ -88,6 +88,13 @@ private void ifDefaultValueMatchesFilter() {
       private boolean isScanRequired(byte[] blockMaxValue, byte[][] filterValues) {
         boolean isScanRequired = false;
         if (isDimensionPresentInCurrentBlock[0]) {
    +      // if a column is unsorted, min and max for that column will not be correct and hence
    --- End diff --
    
    During data loading even if data is not sorted we are calculating the min max so not need to add this code, in all the filter classes only if inverted index is false we need to go iterative approach. Please check in all the rowlevel range filter 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] carbondata pull request #952: [CARBONDATA-1094] Wrong results returned by th...

Posted by asfgit <gi...@git.apache.org>.
Github user asfgit closed the pull request at:

    https://github.com/apache/carbondata/pull/952


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] carbondata issue #952: [CARBONDATA-1094] Wrong results returned by the query...

Posted by kumarvishal09 <gi...@git.apache.org>.
Github user kumarvishal09 commented on the issue:

    https://github.com/apache/carbondata/pull/952
  
    restest this please


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] carbondata issue #952: [CARBONDATA-1094] Wrong results returned by the query...

Posted by kumarvishal09 <gi...@git.apache.org>.
Github user kumarvishal09 commented on the issue:

    https://github.com/apache/carbondata/pull/952
  
    LGTM
    @sounakr Please have a look on range filter changes. 



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] carbondata pull request #952: [CARBONDATA-1094] Wrong results returned by th...

Posted by kumarvishal09 <gi...@git.apache.org>.
Github user kumarvishal09 commented on a diff in the pull request:

    https://github.com/apache/carbondata/pull/952#discussion_r118632591
  
    --- Diff: core/src/main/java/org/apache/carbondata/core/scan/filter/executer/IncludeFilterExecuterImpl.java ---
    @@ -137,19 +150,31 @@ private BitSet setFilterdIndexToBitSet(DimensionColumnDataChunk dimensionColumnD
           int numerOfRows) {
         BitSet bitSet = new BitSet(numerOfRows);
         if (dimensionColumnDataChunk instanceof FixedLengthDimensionDataChunk) {
    +      int startIndex = 0;
           byte[][] filterValues = dimColumnExecuterInfo.getFilterKeys();
    -      if (filterValues.length > 1) {
    --- End diff --
    
    Any reason for removing this code. Its an optimization added for unsorted chunk data. As filter values are sorted we can do reverse comparison. 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] carbondata pull request #952: [CARBONDATA-1094] Wrong results returned by th...

Posted by manishgupta88 <gi...@git.apache.org>.
Github user manishgupta88 commented on a diff in the pull request:

    https://github.com/apache/carbondata/pull/952#discussion_r118672486
  
    --- Diff: core/src/main/java/org/apache/carbondata/core/scan/filter/executer/ExcludeFilterExecuterImpl.java ---
    @@ -43,6 +52,11 @@ public ExcludeFilterExecuterImpl(DimColumnResolvedFilterInfo dimColEvaluatorInfo
         this.segmentProperties = segmentProperties;
         FilterUtil.prepareKeysFromSurrogates(dimColEvaluatorInfo.getFilterValues(), segmentProperties,
             dimColEvaluatorInfo.getDimension(), dimColumnExecuterInfo);
    +    // check if inverted index exists for the filter column
    +    isInvertedIndexColumn = CarbonUtil
    +        .hasEncoding(dimColEvaluatorInfo.getDimension().getEncoder(),
    +            Encoding.INVERTED_INDEX);
    +    isSortColumn = dimColEvaluatorInfo.getDimension().getColumnSchema().isSortColumn();
    --- End diff --
    
    During data load, according to the implementation, And operator is applied between isUseInvertedIndex and isSortColumn property. If anyone is false it will not sort the column vertically and will not create inverted index for that column. But in master schema the boolean flag will still be according to the values specified by the user. Therefor we need to check false as if anyone is false inverted index will not exist for that column


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] carbondata pull request #952: [CARBONDATA-1094] Wrong results returned by th...

Posted by kumarvishal09 <gi...@git.apache.org>.
Github user kumarvishal09 commented on a diff in the pull request:

    https://github.com/apache/carbondata/pull/952#discussion_r118666846
  
    --- Diff: core/src/main/java/org/apache/carbondata/core/scan/filter/executer/RangeValueFilterExecuterImpl.java ---
    @@ -242,7 +256,14 @@ public boolean isScanRequired(byte[] blockMinValue, byte[] blockMaxValue, byte[]
         //                       Block Min <-----------------------> Block Max
         //         Filter Min <-----------------------------------------------> Filter Max
     
    -    if (isDimensionPresentInCurrentBlock == true) {
    +    if (isDimensionPresentInCurrentBlock) {
    +      // if a column is unsorted, min and max for that column will not be correct and hence
    --- End diff --
    
    During data loading even if data is not sorted we are calculating the min max so not need to add this code, in all the filter classes only if inverted index is false we need to go iterative approach


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] carbondata pull request #952: [CARBONDATA-1094] Wrong results returned by th...

Posted by kumarvishal09 <gi...@git.apache.org>.
Github user kumarvishal09 commented on a diff in the pull request:

    https://github.com/apache/carbondata/pull/952#discussion_r118667317
  
    --- Diff: core/src/main/java/org/apache/carbondata/core/scan/filter/executer/RangeValueFilterExecuterImpl.java ---
    @@ -474,80 +495,142 @@ private BitSet setFilterdIndexToBitSet(DimensionColumnDataChunk dimensionColumnD
           int numerOfRows) {
         BitSet bitSet = new BitSet(numerOfRows);
         // if (dimensionColumnDataChunk instanceof FixedLengthDimensionDataChunk) {
    -    int start = 0;
    -    int startMin = 0;
    -    int endMax = 0;
    -    int startIndex = 0;
         byte[][] filterValues = this.filterRangesValues;
    -    // For Range expression we expect two values. The First is the Min Value and Second is the
    -    // Max value.
    -    if (startBlockMinIsDefaultStart == false) {
    -
    -      start = CarbonUtil
    -          .getFirstIndexUsingBinarySearch(dimensionColumnDataChunk,
    -              startIndex, numerOfRows - 1, filterValues[0], greaterThanExp);
    +    if (dimensionColumnDataChunk.isExplicitSorted()) {
    +      int start = 0;
    +      int startMin = 0;
    +      int endMax = 0;
    +      int startIndex = 0;
    +      // For Range expression we expect two values. The First is the Min Value and Second is the
    +      // Max value.
    +      if (startBlockMinIsDefaultStart == false) {
     
    -      if (greaterThanExp == true && start >= 0) {
             start = CarbonUtil
    -            .nextGreaterValueToTarget(start, dimensionColumnDataChunk, filterValues[0],
    -                numerOfRows);
    -      }
    +            .getFirstIndexUsingBinarySearch(dimensionColumnDataChunk, startIndex, numerOfRows - 1,
    +                filterValues[0], greaterThanExp);
     
    -      if (start < 0) {
    -        start = -(start + 1);
    -        if (start == numerOfRows) {
    -          start = start - 1;
    +        if (greaterThanExp == true && start >= 0) {
    +          start = CarbonUtil
    +              .nextGreaterValueToTarget(start, dimensionColumnDataChunk, filterValues[0],
    +                  numerOfRows);
             }
    -        // Method will compare the tentative index value after binary search, this tentative
    -        // index needs to be compared by the filter member if its >= filter then from that
    -        // index the bitset will be considered for filtering process.
    -        if ((ByteUtil.compare(filterValues[0], dimensionColumnDataChunk.getChunkData(start)))
    -            > 0) {
    -          start = start + 1;
    +
    +        if (start < 0) {
    +          start = -(start + 1);
    +          if (start == numerOfRows) {
    +            start = start - 1;
    +          }
    +          // Method will compare the tentative index value after binary search, this tentative
    +          // index needs to be compared by the filter member if its >= filter then from that
    +          // index the bitset will be considered for filtering process.
    +          if ((ByteUtil.compare(filterValues[0], dimensionColumnDataChunk.getChunkData(start)))
    +              > 0) {
    +            start = start + 1;
    +          }
             }
    +        startMin = start;
    +      } else {
    +        startMin = startIndex;
           }
    -      startMin = start;
    -    } else {
    -      startMin = startIndex;
    -    }
    -
    -    if (endBlockMaxisDefaultEnd == false) {
    -      start = CarbonUtil
    -          .getFirstIndexUsingBinarySearch(dimensionColumnDataChunk, startIndex, numerOfRows - 1,
    -              filterValues[1], lessThanEqualExp);
     
    -      if (lessThanExp == true && start >= 0) {
    -        start =
    -            CarbonUtil.nextLesserValueToTarget(start, dimensionColumnDataChunk, filterValues[1]);
    -      }
    +      if (endBlockMaxisDefaultEnd == false) {
    +        start = CarbonUtil
    +            .getFirstIndexUsingBinarySearch(dimensionColumnDataChunk, startIndex, numerOfRows - 1,
    +                filterValues[1], lessThanEqualExp);
     
    -      if (start < 0) {
    -        start = -(start + 1);
    -        if (start == numerOfRows) {
    -          start = start - 1;
    +        if (lessThanExp == true && start >= 0) {
    +          start =
    +              CarbonUtil.nextLesserValueToTarget(start, dimensionColumnDataChunk, filterValues[1]);
             }
    -        // In case the start is less than 0, then positive value of start is pointing to the next
    -        // value of the searched key. So move to the previous one.
    -        if ((ByteUtil.compare(filterValues[1], dimensionColumnDataChunk.getChunkData(start))
    -            < 0)) {
    -          start = start - 1;
    +
    +        if (start < 0) {
    +          start = -(start + 1);
    +          if (start == numerOfRows) {
    +            start = start - 1;
    +          }
    +          // In case the start is less than 0, then positive value of start is pointing to the next
    +          // value of the searched key. So move to the previous one.
    +          if ((ByteUtil.compare(filterValues[1], dimensionColumnDataChunk.getChunkData(start))
    +              < 0)) {
    +            start = start - 1;
    +          }
             }
    +        endMax = start;
    +      } else {
    +        endMax = numerOfRows - 1;
    +      }
    +
    +      for (int j = startMin; j <= endMax; j++) {
    +        bitSet.set(j);
    +      }
    +
    +      // Binary Search cannot be done on '@NU#LL$!", so need to check and compare for null on
    +      // matching row.
    +      if (dimensionColumnDataChunk.isNoDicitionaryColumn()) {
    +        updateForNoDictionaryColumn(startMin, endMax, dimensionColumnDataChunk, bitSet);
           }
    -      endMax = start;
         } else {
    -      endMax = numerOfRows - 1;
    +      // evaluate result for lower range value first and then perform and operation in the
    +      // upper range value in order to compute the final result
    +      bitSet = evaluateGreaterThanFilterForUnsortedColumn(dimensionColumnDataChunk, filterValues[0],
    +          numerOfRows);
    +      bitSet.and(evaluateLessThanFilterForUnsortedColumn(dimensionColumnDataChunk, filterValues[1],
    +          numerOfRows));
         }
    +    return bitSet;
    +  }
     
    -    for (int j = startMin; j <= endMax; j++) {
    -      bitSet.set(j);
    +  /**
    +   * This method will evaluate the result for filter column based on the lower range value
    +   *
    +   * @param dimensionColumnDataChunk
    +   * @param filterValue
    +   * @param numberOfRows
    +   * @return
    +   */
    +  private BitSet evaluateGreaterThanFilterForUnsortedColumn(
    +      DimensionColumnDataChunk dimensionColumnDataChunk, byte[] filterValue, int numberOfRows) {
    +    BitSet bitSet = new BitSet(numberOfRows);
    --- End diff --
    
    As we are adding @NU#LL$! for No dictionary null values in this method after getting great values or greater than equal to values we need to check whether null values are present or not if it is present then we need to remove it from bitset


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] carbondata pull request #952: [CARBONDATA-1094] Wrong results returned by th...

Posted by kumarvishal09 <gi...@git.apache.org>.
Github user kumarvishal09 commented on a diff in the pull request:

    https://github.com/apache/carbondata/pull/952#discussion_r118631844
  
    --- Diff: core/src/main/java/org/apache/carbondata/core/scan/filter/executer/ExcludeFilterExecuterImpl.java ---
    @@ -43,6 +52,11 @@ public ExcludeFilterExecuterImpl(DimColumnResolvedFilterInfo dimColEvaluatorInfo
         this.segmentProperties = segmentProperties;
         FilterUtil.prepareKeysFromSurrogates(dimColEvaluatorInfo.getFilterValues(), segmentProperties,
             dimColEvaluatorInfo.getDimension(), dimColumnExecuterInfo);
    +    // check if inverted index exists for the filter column
    +    isInvertedIndexColumn = CarbonUtil
    +        .hasEncoding(dimColEvaluatorInfo.getDimension().getEncoder(),
    +            Encoding.INVERTED_INDEX);
    +    isSortColumn = dimColEvaluatorInfo.getDimension().getColumnSchema().isSortColumn();
    --- End diff --
    
    Please correct me...Inverted Index and sort column are two different property. Sort column property is for dimension which is participating in global sort. If any column which is not present in sort column can be sorted vertically(column wise), so i think isSortedColumn is not required


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] carbondata pull request #952: [CARBONDATA-1094] Wrong results returned by th...

Posted by kumarvishal09 <gi...@git.apache.org>.
Github user kumarvishal09 commented on a diff in the pull request:

    https://github.com/apache/carbondata/pull/952#discussion_r118668176
  
    --- Diff: core/src/main/java/org/apache/carbondata/core/scan/filter/executer/RowLevelFilterExecuterImpl.java ---
    @@ -94,6 +94,14 @@
        * Applicable for restructure scenarios
        */
       protected boolean[] isMeasurePresentInCurrentBlock;
    +  /**
    +   * flag to check whether inverted index exists for the filter column
    +   */
    +  protected boolean isInvertedIndexColumn;
    +  /**
    +   * flag to check whether the column is included for sorting during data load
    +   */
    +  protected boolean isSortColumn;
    --- End diff --
    
    Please correct me isSortColumn property is for global sort, so filter does not care about whether column was part of global sort or not. Only we need to check whether column data is sorted or not so isSortColumn check and this variable is not required. Please check and handled in all the places if not required


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] carbondata pull request #952: [CARBONDATA-1094] Wrong results returned by th...

Posted by kumarvishal09 <gi...@git.apache.org>.
Github user kumarvishal09 commented on a diff in the pull request:

    https://github.com/apache/carbondata/pull/952#discussion_r118667394
  
    --- Diff: core/src/main/java/org/apache/carbondata/core/scan/filter/executer/RangeValueFilterExecuterImpl.java ---
    @@ -474,80 +495,142 @@ private BitSet setFilterdIndexToBitSet(DimensionColumnDataChunk dimensionColumnD
           int numerOfRows) {
         BitSet bitSet = new BitSet(numerOfRows);
         // if (dimensionColumnDataChunk instanceof FixedLengthDimensionDataChunk) {
    -    int start = 0;
    -    int startMin = 0;
    -    int endMax = 0;
    -    int startIndex = 0;
         byte[][] filterValues = this.filterRangesValues;
    -    // For Range expression we expect two values. The First is the Min Value and Second is the
    -    // Max value.
    -    if (startBlockMinIsDefaultStart == false) {
    -
    -      start = CarbonUtil
    -          .getFirstIndexUsingBinarySearch(dimensionColumnDataChunk,
    -              startIndex, numerOfRows - 1, filterValues[0], greaterThanExp);
    +    if (dimensionColumnDataChunk.isExplicitSorted()) {
    +      int start = 0;
    +      int startMin = 0;
    +      int endMax = 0;
    +      int startIndex = 0;
    +      // For Range expression we expect two values. The First is the Min Value and Second is the
    +      // Max value.
    +      if (startBlockMinIsDefaultStart == false) {
     
    -      if (greaterThanExp == true && start >= 0) {
             start = CarbonUtil
    -            .nextGreaterValueToTarget(start, dimensionColumnDataChunk, filterValues[0],
    -                numerOfRows);
    -      }
    +            .getFirstIndexUsingBinarySearch(dimensionColumnDataChunk, startIndex, numerOfRows - 1,
    +                filterValues[0], greaterThanExp);
     
    -      if (start < 0) {
    -        start = -(start + 1);
    -        if (start == numerOfRows) {
    -          start = start - 1;
    +        if (greaterThanExp == true && start >= 0) {
    +          start = CarbonUtil
    +              .nextGreaterValueToTarget(start, dimensionColumnDataChunk, filterValues[0],
    +                  numerOfRows);
             }
    -        // Method will compare the tentative index value after binary search, this tentative
    -        // index needs to be compared by the filter member if its >= filter then from that
    -        // index the bitset will be considered for filtering process.
    -        if ((ByteUtil.compare(filterValues[0], dimensionColumnDataChunk.getChunkData(start)))
    -            > 0) {
    -          start = start + 1;
    +
    +        if (start < 0) {
    +          start = -(start + 1);
    +          if (start == numerOfRows) {
    +            start = start - 1;
    +          }
    +          // Method will compare the tentative index value after binary search, this tentative
    +          // index needs to be compared by the filter member if its >= filter then from that
    +          // index the bitset will be considered for filtering process.
    +          if ((ByteUtil.compare(filterValues[0], dimensionColumnDataChunk.getChunkData(start)))
    +              > 0) {
    +            start = start + 1;
    +          }
             }
    +        startMin = start;
    +      } else {
    +        startMin = startIndex;
           }
    -      startMin = start;
    -    } else {
    -      startMin = startIndex;
    -    }
    -
    -    if (endBlockMaxisDefaultEnd == false) {
    -      start = CarbonUtil
    -          .getFirstIndexUsingBinarySearch(dimensionColumnDataChunk, startIndex, numerOfRows - 1,
    -              filterValues[1], lessThanEqualExp);
     
    -      if (lessThanExp == true && start >= 0) {
    -        start =
    -            CarbonUtil.nextLesserValueToTarget(start, dimensionColumnDataChunk, filterValues[1]);
    -      }
    +      if (endBlockMaxisDefaultEnd == false) {
    +        start = CarbonUtil
    +            .getFirstIndexUsingBinarySearch(dimensionColumnDataChunk, startIndex, numerOfRows - 1,
    +                filterValues[1], lessThanEqualExp);
     
    -      if (start < 0) {
    -        start = -(start + 1);
    -        if (start == numerOfRows) {
    -          start = start - 1;
    +        if (lessThanExp == true && start >= 0) {
    +          start =
    +              CarbonUtil.nextLesserValueToTarget(start, dimensionColumnDataChunk, filterValues[1]);
             }
    -        // In case the start is less than 0, then positive value of start is pointing to the next
    -        // value of the searched key. So move to the previous one.
    -        if ((ByteUtil.compare(filterValues[1], dimensionColumnDataChunk.getChunkData(start))
    -            < 0)) {
    -          start = start - 1;
    +
    +        if (start < 0) {
    +          start = -(start + 1);
    +          if (start == numerOfRows) {
    +            start = start - 1;
    +          }
    +          // In case the start is less than 0, then positive value of start is pointing to the next
    +          // value of the searched key. So move to the previous one.
    +          if ((ByteUtil.compare(filterValues[1], dimensionColumnDataChunk.getChunkData(start))
    +              < 0)) {
    +            start = start - 1;
    +          }
             }
    +        endMax = start;
    +      } else {
    +        endMax = numerOfRows - 1;
    +      }
    +
    +      for (int j = startMin; j <= endMax; j++) {
    +        bitSet.set(j);
    +      }
    +
    +      // Binary Search cannot be done on '@NU#LL$!", so need to check and compare for null on
    +      // matching row.
    +      if (dimensionColumnDataChunk.isNoDicitionaryColumn()) {
    +        updateForNoDictionaryColumn(startMin, endMax, dimensionColumnDataChunk, bitSet);
           }
    -      endMax = start;
         } else {
    -      endMax = numerOfRows - 1;
    +      // evaluate result for lower range value first and then perform and operation in the
    +      // upper range value in order to compute the final result
    +      bitSet = evaluateGreaterThanFilterForUnsortedColumn(dimensionColumnDataChunk, filterValues[0],
    +          numerOfRows);
    +      bitSet.and(evaluateLessThanFilterForUnsortedColumn(dimensionColumnDataChunk, filterValues[1],
    +          numerOfRows));
         }
    +    return bitSet;
    +  }
     
    -    for (int j = startMin; j <= endMax; j++) {
    -      bitSet.set(j);
    +  /**
    +   * This method will evaluate the result for filter column based on the lower range value
    +   *
    +   * @param dimensionColumnDataChunk
    +   * @param filterValue
    +   * @param numberOfRows
    +   * @return
    +   */
    +  private BitSet evaluateGreaterThanFilterForUnsortedColumn(
    +      DimensionColumnDataChunk dimensionColumnDataChunk, byte[] filterValue, int numberOfRows) {
    +    BitSet bitSet = new BitSet(numberOfRows);
    +    if (greaterThanExp) {
    +      for (int i = 0; i < numberOfRows; i++) {
    +        if ((ByteUtil.compare(dimensionColumnDataChunk.getChunkData(i), filterValue) > 0)) {
    +          bitSet.set(i);
    +        }
    +      }
    +    } else if (greaterThanEqualExp) {
    +      for (int i = 0; i < numberOfRows; i++) {
    +        if ((ByteUtil.compare(dimensionColumnDataChunk.getChunkData(i), filterValue) >= 0)) {
    +          bitSet.set(i);
    +        }
    +      }
         }
    +    return bitSet;
    +  }
     
    -    // Binary Search cannot be done on '@NU#LL$!", so need to check and compare for null on
    -    // matching row.
    -    if (dimensionColumnDataChunk.isNoDicitionaryColumn()) {
    -      updateForNoDictionaryColumn(startMin, endMax, dimensionColumnDataChunk, bitSet);
    +  /**
    +   * This method will evaluate the result for filter column based on the upper range value
    +   *
    +   * @param dimensionColumnDataChunk
    +   * @param filterValue
    +   * @param numberOfRows
    +   * @return
    +   */
    +  private BitSet evaluateLessThanFilterForUnsortedColumn(
    +      DimensionColumnDataChunk dimensionColumnDataChunk, byte[] filterValue, int numberOfRows) {
    --- End diff --
    
    As we are adding @nu#LL$! for No dictionary null values in this method after getting less values or less than equal to values we need to check whether null values are present or not if it is present then we need to remove it from bitset


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---