[GitHub] [carbondata] Indhumathi27 opened a new pull request #3778: [WIP] Support array<string> with SI

classic Classic list List threaded Threaded
77 messages Options
1234
Reply | Threaded
Open this post in threaded view
|

[GitHub] [carbondata] ajantha-bhat edited a comment on pull request #3778: [CARBONDATA-3916] Support array<string> with SI

GitBox

ajantha-bhat edited a comment on pull request #3778:
URL: https://github.com/apache/carbondata/pull/3778#issuecomment-662251770


   a) I feel we should handle the SI support for all array of primitive not just string.
   b) Better to discuss the row level SI solution in community once ?
   
   @QiangCai , @kunal642 @Indhumathi27


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

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


Reply | Threaded
Open this post in threaded view
|

[GitHub] [carbondata] ajantha-bhat commented on a change in pull request #3778: [CARBONDATA-3916] Support array<string> with SI

GitBox
In reply to this post by GitBox

ajantha-bhat commented on a change in pull request #3778:
URL: https://github.com/apache/carbondata/pull/3778#discussion_r458547213



##########
File path: index/secondary-index/src/test/scala/org/apache/carbondata/spark/testsuite/secondaryindex/TestSIWithComplexArrayType.scala
##########
@@ -0,0 +1,136 @@
+/*
+ * 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.carbondata.spark.testsuite.secondaryindex
+
+import org.apache.spark.sql.Row
+import org.apache.spark.sql.test.util.QueryTest
+import org.scalatest.BeforeAndAfterEach
+
+import org.apache.carbondata.spark.testsuite.secondaryindex.TestSecondaryIndexUtils.isFilterPushedDownToSI
+
+class TestSIWithComplexArrayType extends QueryTest with BeforeAndAfterEach {
+
+  override def beforeEach(): Unit = {
+    sql("drop table if exists complextable")
+  }
+
+  override def afterEach(): Unit = {
+    sql("drop index if exists index_1 on complextable")
+    sql("drop table if exists complextable")
+  }
+
+  test("test array<string> on secondary index") {

Review comment:
       a) I feel we should handle the SI support for all array of primitive not just string.
   b) Better to discuss the row level SI solution in community once ?
   c) Also if multiple SI created for one primitive and one complex column. query gives 0 rows. Need to handle 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.

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


Reply | Threaded
Open this post in threaded view
|

[GitHub] [carbondata] QiangCai commented on a change in pull request #3778: [CARBONDATA-3916] Support array<string> with SI

GitBox
In reply to this post by GitBox

QiangCai commented on a change in pull request #3778:
URL: https://github.com/apache/carbondata/pull/3778#discussion_r458509321



##########
File path: core/src/main/java/org/apache/carbondata/core/scan/complextypes/ArrayQueryType.java
##########
@@ -97,21 +97,35 @@ public void fillRequiredBlockData(RawBlockletColumnChunks blockChunkHolder)
 
   @Override
   public Object getDataBasedOnDataType(ByteBuffer dataBuffer) {
-    Object[] data = fillData(dataBuffer);
+    return getDataBasedOnDataType(dataBuffer, false);
+  }
+
+  @Override
+  public Object getDataBasedOnDataType(ByteBuffer dataBuffer, boolean getBytesData) {
+    Object[] data = fillData(dataBuffer, false);
     if (data == null) {
       return null;
     }
     return DataTypeUtil.getDataTypeConverter().wrapWithGenericArrayData(data);
   }
 
-  protected Object[] fillData(ByteBuffer dataBuffer) {
+  @Override
+  public Object[] getObjectArrayDataBasedOnDataType(ByteBuffer dataBuffer) {
+    Object[] data = fillData(dataBuffer, true);
+    if (data == null) {
+      return null;
+    }
+    return data;

Review comment:
       return fillData(dataBuffer, true);

##########
File path: integration/spark/src/main/scala/org/apache/spark/sql/secondaryindex/query/SecondaryIndexQueryResultProcessor.java
##########
@@ -281,12 +310,45 @@ private void processResult(List<CarbonIterator<RowBatch>> detailQueryResultItera
       }
     }
 
+    if (!complexDimensionInfoMap.isEmpty() && complexColumnParentBlockIndexes.length > 0) {
+      // In case of complex array type, flatten the data and add for sorting
+      // TODO: Handle for nested array and other complex types
+      for (int k = 0; k < wrapper.getComplexTypesKeys().length; k++) {
+        byte[] complexKeyByIndex = wrapper.getComplexKeyByIndex(k);
+        ByteBuffer byteArrayInput = ByteBuffer.wrap(complexKeyByIndex);
+        GenericQueryType genericQueryType =
+            complexDimensionInfoMap.get(complexColumnParentBlockIndexes[k]);
+        short length = byteArrayInput.getShort(2);
+        // get flattened array data
+        Object[] data = genericQueryType.getObjectArrayDataBasedOnDataType(byteArrayInput);
+        if (length != 1) {
+          for (int j = 1; j < length; j++) {
+            preparedRow[i] = getData(data, j);
+            preparedRow[i + 1] = implicitColumnByteArray;
+            addRowForSorting(preparedRow.clone());
+          }
+          // add first row
+          preparedRow[i] = getData(data, 0);
+        } else {
+          preparedRow[i] = getData(data, 0);
+        }

Review comment:
       }
   preparedRow[i] = getData(data, 0);

##########
File path: integration/spark/src/main/scala/org/apache/spark/sql/secondaryindex/query/SecondaryIndexQueryResultProcessor.java
##########
@@ -226,8 +239,16 @@ private void processResult(List<CarbonIterator<RowBatch>> detailQueryResultItera
     for (CarbonIterator<RowBatch> detailQueryIterator : detailQueryResultIteratorList) {
       while (detailQueryIterator.hasNext()) {
         RowBatch batchResult = detailQueryIterator.next();
+        DetailQueryResultIterator queryIterator = (DetailQueryResultIterator) detailQueryIterator;
+        BlockExecutionInfo blockExecutionInfo = queryIterator.getBlockExecutionInfo();
+        // get complex dimension info map from block execution info
+        Map<Integer, GenericQueryType> complexDimensionInfoMap =
+            blockExecutionInfo.getComplexDimensionInfoMap();
+        int[] complexColumnParentBlockIndexes =
+            blockExecutionInfo.getComplexColumnParentBlockIndexes();

Review comment:
       move to outside of while loop




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

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


Reply | Threaded
Open this post in threaded view
|

[GitHub] [carbondata] ajantha-bhat commented on a change in pull request #3778: [CARBONDATA-3916] Support array<string> with SI

GitBox
In reply to this post by GitBox

ajantha-bhat commented on a change in pull request #3778:
URL: https://github.com/apache/carbondata/pull/3778#discussion_r458561948



##########
File path: index/secondary-index/src/test/scala/org/apache/carbondata/spark/testsuite/secondaryindex/TestSIWithComplexArrayType.scala
##########
@@ -0,0 +1,136 @@
+/*
+ * 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.carbondata.spark.testsuite.secondaryindex
+
+import org.apache.spark.sql.Row
+import org.apache.spark.sql.test.util.QueryTest
+import org.scalatest.BeforeAndAfterEach
+
+import org.apache.carbondata.spark.testsuite.secondaryindex.TestSecondaryIndexUtils.isFilterPushedDownToSI
+
+class TestSIWithComplexArrayType extends QueryTest with BeforeAndAfterEach {
+
+  override def beforeEach(): Unit = {
+    sql("drop table if exists complextable")
+  }
+
+  override def afterEach(): Unit = {
+    sql("drop index if exists index_1 on complextable")
+    sql("drop table if exists complextable")
+  }
+
+  test("test array<string> on secondary index") {

Review comment:
       also add a test case for c)




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

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


Reply | Threaded
Open this post in threaded view
|

[GitHub] [carbondata] QiangCai commented on pull request #3778: [CARBONDATA-3916] Support array<string> with SI

GitBox
In reply to this post by GitBox

QiangCai commented on pull request #3778:
URL: https://github.com/apache/carbondata/pull/3778#issuecomment-662267686


   Agree with @ajantha-bhat


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

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


Reply | Threaded
Open this post in threaded view
|

[GitHub] [carbondata] ajantha-bhat commented on a change in pull request #3778: [CARBONDATA-3916] Support array<string> with SI

GitBox
In reply to this post by GitBox

ajantha-bhat commented on a change in pull request #3778:
URL: https://github.com/apache/carbondata/pull/3778#discussion_r460795333



##########
File path: index/secondary-index/src/test/scala/org/apache/carbondata/spark/testsuite/secondaryindex/TestSIWithComplexArrayType.scala
##########
@@ -0,0 +1,136 @@
+/*
+ * 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.carbondata.spark.testsuite.secondaryindex
+
+import org.apache.spark.sql.Row
+import org.apache.spark.sql.test.util.QueryTest
+import org.scalatest.BeforeAndAfterEach
+
+import org.apache.carbondata.spark.testsuite.secondaryindex.TestSecondaryIndexUtils.isFilterPushedDownToSI
+
+class TestSIWithComplexArrayType extends QueryTest with BeforeAndAfterEach {
+
+  override def beforeEach(): Unit = {
+    sql("drop table if exists complextable")
+  }
+
+  override def afterEach(): Unit = {
+    sql("drop index if exists index_1 on complextable")
+    sql("drop table if exists complextable")
+  }
+
+  test("test array<string> on secondary index") {

Review comment:
       d) If two array_contains() present with AND in query. When it is pushed down as equal to filter in SI. It will give 0 rows as SI is flattened and it cannot find two values in one row. Need to handle that also




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

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


Reply | Threaded
Open this post in threaded view
|

[GitHub] [carbondata] CarbonDataQA1 commented on pull request #3778: [WIP][CARBONDATA-3916] Support array complex type with SI

GitBox
In reply to this post by GitBox

CarbonDataQA1 commented on pull request #3778:
URL: https://github.com/apache/carbondata/pull/3778#issuecomment-665812800






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

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


Reply | Threaded
Open this post in threaded view
|

[GitHub] [carbondata] QiangCai commented on pull request #3778: [WIP][CARBONDATA-3916] Support array complex type with SI

GitBox
In reply to this post by GitBox

QiangCai commented on pull request #3778:
URL: https://github.com/apache/carbondata/pull/3778#issuecomment-671194649


   please rebase 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.

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


Reply | Threaded
Open this post in threaded view
|

[GitHub] [carbondata] CarbonDataQA1 commented on pull request #3778: [WIP][CARBONDATA-3916] Support array complex type with SI

GitBox
In reply to this post by GitBox

CarbonDataQA1 commented on pull request #3778:
URL: https://github.com/apache/carbondata/pull/3778#issuecomment-671340388


   Build Failed  with Spark 2.3.4, Please check CI http://121.244.95.60:12545/job/ApacheCarbonPRBuilder2.3/3682/
   


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

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


Reply | Threaded
Open this post in threaded view
|

[GitHub] [carbondata] CarbonDataQA1 commented on pull request #3778: [WIP][CARBONDATA-3916] Support array complex type with SI

GitBox
In reply to this post by GitBox

CarbonDataQA1 commented on pull request #3778:
URL: https://github.com/apache/carbondata/pull/3778#issuecomment-671341729


   Build Failed  with Spark 2.4.5, Please check CI http://121.244.95.60:12545/job/ApacheCarbon_PR_Builder_2.4.5/1943/
   


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

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


Reply | Threaded
Open this post in threaded view
|

[GitHub] [carbondata] Indhumathi27 commented on a change in pull request #3778: [WIP][CARBONDATA-3916] Support array complex type with SI

GitBox
In reply to this post by GitBox

Indhumathi27 commented on a change in pull request #3778:
URL: https://github.com/apache/carbondata/pull/3778#discussion_r467978296



##########
File path: index/secondary-index/src/test/scala/org/apache/carbondata/spark/testsuite/secondaryindex/TestSIWithComplexArrayType.scala
##########
@@ -0,0 +1,136 @@
+/*
+ * 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.carbondata.spark.testsuite.secondaryindex
+
+import org.apache.spark.sql.Row
+import org.apache.spark.sql.test.util.QueryTest
+import org.scalatest.BeforeAndAfterEach
+
+import org.apache.carbondata.spark.testsuite.secondaryindex.TestSecondaryIndexUtils.isFilterPushedDownToSI
+
+class TestSIWithComplexArrayType extends QueryTest with BeforeAndAfterEach {
+
+  override def beforeEach(): Unit = {
+    sql("drop table if exists complextable")
+  }
+
+  override def afterEach(): Unit = {
+    sql("drop index if exists index_1 on complextable")
+    sql("drop table if exists complextable")
+  }
+
+  test("test array<string> on secondary index") {

Review comment:
       currenlty, for d), for query having more than one array contains filter will not SI. query will hit main table only. added testcase for c)




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

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


Reply | Threaded
Open this post in threaded view
|

[GitHub] [carbondata] CarbonDataQA1 commented on pull request #3778: [WIP][CARBONDATA-3916] Support array complex type with SI

GitBox
In reply to this post by GitBox

CarbonDataQA1 commented on pull request #3778:
URL: https://github.com/apache/carbondata/pull/3778#issuecomment-671496551


   Build Success with Spark 2.3.4, Please check CI http://121.244.95.60:12545/job/ApacheCarbonPRBuilder2.3/3687/
   


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

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


Reply | Threaded
Open this post in threaded view
|

[GitHub] [carbondata] CarbonDataQA1 commented on pull request #3778: [WIP][CARBONDATA-3916] Support array complex type with SI

GitBox
In reply to this post by GitBox

CarbonDataQA1 commented on pull request #3778:
URL: https://github.com/apache/carbondata/pull/3778#issuecomment-671499205


   Build Success with Spark 2.4.5, Please check CI http://121.244.95.60:12545/job/ApacheCarbon_PR_Builder_2.4.5/1948/
   


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

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


Reply | Threaded
Open this post in threaded view
|

[GitHub] [carbondata] QiangCai commented on pull request #3778: [WIP][CARBONDATA-3916] Support array complex type with SI

GitBox
In reply to this post by GitBox

QiangCai commented on pull request #3778:
URL: https://github.com/apache/carbondata/pull/3778#issuecomment-673324648


   please rebase


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

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


Reply | Threaded
Open this post in threaded view
|

[GitHub] [carbondata] CarbonDataQA1 commented on pull request #3778: [CARBONDATA-3916] Support array complex type with SI

GitBox
In reply to this post by GitBox

CarbonDataQA1 commented on pull request #3778:
URL: https://github.com/apache/carbondata/pull/3778#issuecomment-673637872


   Build Failed  with Spark 2.3.4, Please check CI http://121.244.95.60:12545/job/ApacheCarbonPRBuilder2.3/3713/
   


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

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


Reply | Threaded
Open this post in threaded view
|

[GitHub] [carbondata] CarbonDataQA1 commented on pull request #3778: [CARBONDATA-3916] Support array complex type with SI

GitBox
In reply to this post by GitBox

CarbonDataQA1 commented on pull request #3778:
URL: https://github.com/apache/carbondata/pull/3778#issuecomment-673649958


   Build Failed  with Spark 2.4.5, Please check CI http://121.244.95.60:12545/job/ApacheCarbon_PR_Builder_2.4.5/1974/
   


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

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


Reply | Threaded
Open this post in threaded view
|

[GitHub] [carbondata] CarbonDataQA1 commented on pull request #3778: [CARBONDATA-3916] Support array complex type with SI

GitBox
In reply to this post by GitBox

CarbonDataQA1 commented on pull request #3778:
URL: https://github.com/apache/carbondata/pull/3778#issuecomment-673882872


   Build Failed  with Spark 2.4.5, Please check CI http://121.244.95.60:12545/job/ApacheCarbon_PR_Builder_2.4.5/1977/
   


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

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


Reply | Threaded
Open this post in threaded view
|

[GitHub] [carbondata] CarbonDataQA1 commented on pull request #3778: [CARBONDATA-3916] Support array complex type with SI

GitBox
In reply to this post by GitBox

CarbonDataQA1 commented on pull request #3778:
URL: https://github.com/apache/carbondata/pull/3778#issuecomment-673883264


   Build Failed  with Spark 2.3.4, Please check CI http://121.244.95.60:12545/job/ApacheCarbonPRBuilder2.3/3716/
   


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

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


Reply | Threaded
Open this post in threaded view
|

[GitHub] [carbondata] CarbonDataQA1 commented on pull request #3778: [CARBONDATA-3916] Support array complex type with SI

GitBox
In reply to this post by GitBox

CarbonDataQA1 commented on pull request #3778:
URL: https://github.com/apache/carbondata/pull/3778#issuecomment-673940891






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

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


Reply | Threaded
Open this post in threaded view
|

[GitHub] [carbondata] QiangCai commented on a change in pull request #3778: [CARBONDATA-3916] Support array complex type with SI

GitBox
In reply to this post by GitBox

QiangCai commented on a change in pull request #3778:
URL: https://github.com/apache/carbondata/pull/3778#discussion_r469772276



##########
File path: core/src/main/java/org/apache/carbondata/core/scan/complextypes/ArrayQueryType.java
##########
@@ -39,7 +39,7 @@ public ArrayQueryType(String name, String parentName, int columnIndex) {
 
   @Override
   public void addChildren(GenericQueryType children) {
-    if (this.getName().equals(children.getParentName())) {
+    if (null == this.getName() || this.getName().equals(children.getParentName())) {

Review comment:
       When the name can be null?

##########
File path: core/src/main/java/org/apache/carbondata/core/constants/CarbonCommonConstants.java
##########
@@ -2456,4 +2456,15 @@ private CarbonCommonConstants() {
    * property which defines the insert stage flow
    */
   public static final String IS_INSERT_STAGE = "is_insert_stage";
+
+  /**
+   * Until the threshold for complex filter is reached, row id will be set to the bitset in
+   * implicit filter during secondary index pruning
+   */
+  public static final String SI_COMPLEX_FILTER_THRESHOLD = "carbon.si.complex.filter.threshold";

Review comment:
       better to move all constants of SI together to one place of this class.

##########
File path: core/src/main/java/org/apache/carbondata/core/scan/expression/conditional/ImplicitExpression.java
##########
@@ -41,39 +44,62 @@
    * map that contains the mapping of block id to the valid blocklets in that block which contain
    * the data as per the applied filter
    */
-  private Map<String, Set<Integer>> blockIdToBlockletIdMapping;
+  private final Map<String, Set<String>> blockIdToBlockletIdMapping;
+
+  /**
+   * checks if implicit filter exceeds complex filter threshold
+   */
+  private boolean isThresholdReached;
 
   public ImplicitExpression(List<Expression> implicitFilterList) {
+    final Logger LOGGER = LogServiceFactory.getLogService(getClass().getName());

Review comment:
       move LOGGER to be a static field of this class

##########
File path: core/src/main/java/org/apache/carbondata/core/indexstore/blockletindex/BlockletDataRefNode.java
##########
@@ -221,4 +221,9 @@ public int numberOfNodes() {
   public List<TableBlockInfo> getBlockInfos() {

Review comment:
       after we added getTableBlockInfo(), can we remove this method?

##########
File path: core/src/main/java/org/apache/carbondata/core/scan/expression/conditional/ImplicitExpression.java
##########
@@ -41,39 +44,62 @@
    * map that contains the mapping of block id to the valid blocklets in that block which contain
    * the data as per the applied filter
    */
-  private Map<String, Set<Integer>> blockIdToBlockletIdMapping;
+  private final Map<String, Set<String>> blockIdToBlockletIdMapping;
+
+  /**
+   * checks if implicit filter exceeds complex filter threshold
+   */
+  private boolean isThresholdReached;
 
   public ImplicitExpression(List<Expression> implicitFilterList) {
+    final Logger LOGGER = LogServiceFactory.getLogService(getClass().getName());
     // initialize map with half the size of filter list as one block id can contain
     // multiple blocklets
     blockIdToBlockletIdMapping = new HashMap<>(implicitFilterList.size() / 2);
     for (Expression value : implicitFilterList) {
       String blockletPath = ((LiteralExpression) value).getLiteralExpValue().toString();
       addBlockEntry(blockletPath);
     }
+    int complexFilterThreshold = CarbonProperties.getInstance().getComplexFilterThresholdForSI();
+    isThresholdReached = implicitFilterList.size() > complexFilterThreshold;
+    if (isThresholdReached) {
+      LOGGER.info("Implicit Filter Size: " + implicitFilterList.size() + ", Threshold is: "
+          + complexFilterThreshold);
+    }
   }
 
-  public ImplicitExpression(Map<String, Set<Integer>> blockIdToBlockletIdMapping) {
+  public ImplicitExpression(Map<String, Set<String>> blockIdToBlockletIdMapping) {
     this.blockIdToBlockletIdMapping = blockIdToBlockletIdMapping;
   }
 
   private void addBlockEntry(String blockletPath) {

Review comment:
       The logic of this method is hard to understand.
   Can we add a flag into ImplicitExpression when it is created?
   if it is blocklet level, we addBlockletEntry.
   if it is row level, we addRowEntry.
   
   

##########
File path: core/src/main/java/org/apache/carbondata/core/scan/complextypes/ArrayQueryType.java
##########
@@ -97,21 +97,31 @@ public void fillRequiredBlockData(RawBlockletColumnChunks blockChunkHolder)
 
   @Override
   public Object getDataBasedOnDataType(ByteBuffer dataBuffer) {
-    Object[] data = fillData(dataBuffer);
+    return getDataBasedOnDataType(dataBuffer, false);
+  }
+
+  @Override
+  public Object getDataBasedOnDataType(ByteBuffer dataBuffer, boolean getBytesData) {

Review comment:
       how about to keep the old method and add a new method getObjectDataBasedOnDataType?
   It will not need this boolean parameter.




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

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


1234