You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@carbondata.apache.org by GitBox <gi...@apache.org> on 2020/02/25 16:02:46 UTC

[GitHub] [carbondata] maheshrajus opened a new pull request #3639: [WIP] Secondary Index enable on partition Table

maheshrajus opened a new pull request #3639: [WIP] Secondary Index enable on partition Table
URL: https://github.com/apache/carbondata/pull/3639
 
 
    ### Why is this PR needed?
    
    [WIP] Secondary Index enable on partition Table
    ### What changes were proposed in this PR?
   
       
    ### Does this PR introduce any user interface change?
    - No
    - Yes. (please explain the change and update document)
   
    ### Is any new testcase added?
    - No
    - Yes
   
       
   

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] Indhumathi27 commented on a change in pull request #3639: [CARBONDATA-3724] Secondary Index enable on partition Table

Posted by GitBox <gi...@apache.org>.
Indhumathi27 commented on a change in pull request #3639: [CARBONDATA-3724] Secondary Index enable on partition Table
URL: https://github.com/apache/carbondata/pull/3639#discussion_r387473487
 
 

 ##########
 File path: core/src/main/java/org/apache/carbondata/core/scan/scanner/impl/BlockletFilterScanner.java
 ##########
 @@ -203,8 +203,8 @@ private BlockletScannedResult executeFilter(RawBlockletColumnChunks rawBlockletC
     BlockletScannedResult scannedResult =
         new FilterQueryScannedResult(blockExecutionInfo, queryStatisticsModel);
     scannedResult.setBlockletId(
-        blockExecutionInfo.getBlockIdString() + CarbonCommonConstants.FILE_SEPARATOR +
-            rawBlockletColumnChunks.getDataBlock().blockletIndex());
+        blockExecutionInfo.getBlockIdString(),
+            "" + rawBlockletColumnChunks.getDataBlock().blockletIndex());
 
 Review comment:
   remove "" if not required

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] asfgit closed pull request #3639: [CARBONDATA-3724] Secondary Index enable on partition Table

Posted by GitBox <gi...@apache.org>.
asfgit closed pull request #3639: [CARBONDATA-3724] Secondary Index enable on partition Table
URL: https://github.com/apache/carbondata/pull/3639
 
 
   

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] kunal642 commented on issue #3639: [CARBONDATA-3724] Secondary Index enable on partition Table

Posted by GitBox <gi...@apache.org>.
kunal642 commented on issue #3639: [CARBONDATA-3724] Secondary Index enable on partition Table
URL: https://github.com/apache/carbondata/pull/3639#issuecomment-605919858
 
 
   @Indhumathi27 @jackylk @akashrn5 Please review 

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] maheshrajus commented on a change in pull request #3639: [CARBONDATA-3724] Secondary Index enable on partition Table

Posted by GitBox <gi...@apache.org>.
maheshrajus commented on a change in pull request #3639: [CARBONDATA-3724] Secondary Index enable on partition Table
URL: https://github.com/apache/carbondata/pull/3639#discussion_r387565314
 
 

 ##########
 File path: core/src/main/java/org/apache/carbondata/core/indexstore/blockletindex/BlockDataMap.java
 ##########
 @@ -788,7 +788,13 @@ private boolean addBlockBasedOnMinMaxValue(FilterExecuter filterExecuter, byte[]
       byte[][] minValue, boolean[] minMaxFlag, String filePath, int blockletId) {
     BitSet bitSet = null;
     if (filterExecuter instanceof ImplicitColumnFilterExecutor) {
-      String uniqueBlockPath = filePath.substring(filePath.lastIndexOf("/Part") + 1);
+      String uniqueBlockPath;
+      if (segmentPropertiesWrapper.getCarbonTable().isHivePartitionTable()) {
 
 Review comment:
   blockletDataMapInfo is not stored in BlockDataMap and we are getting it from DataMapModel.  In segmentPropertiesWrapper i just added getter for carbontable. i feel this is ok. Any suggestions ?

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] CarbonDataQA1 commented on issue #3639: [WIP] Secondary Index enable on partition Table

Posted by GitBox <gi...@apache.org>.
CarbonDataQA1 commented on issue #3639: [WIP] Secondary Index enable on partition Table
URL: https://github.com/apache/carbondata/pull/3639#issuecomment-591273784
 
 
   Build Success with Spark 2.4.4, Please check CI http://121.244.95.60:12545/job/ApacheCarbon_PR_Builder_2.4.4/484/
   

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] Indhumathi27 commented on a change in pull request #3639: [CARBONDATA-3724] Secondary Index enable on partition Table

Posted by GitBox <gi...@apache.org>.
Indhumathi27 commented on a change in pull request #3639: [CARBONDATA-3724] Secondary Index enable on partition Table
URL: https://github.com/apache/carbondata/pull/3639#discussion_r387454934
 
 

 ##########
 File path: core/src/main/java/org/apache/carbondata/core/mutate/CarbonUpdateUtil.java
 ##########
 @@ -58,6 +58,14 @@
   private static final Logger LOGGER =
           LogServiceFactory.getLogService(CarbonUpdateUtil.class.getName());
 
+  /**
+   * returns required filed from tuple id
+   *
+   */
+  public static String getRequiredFieldFromTID(String Tid, int index) {
+    return Tid.split("/")[index];
 
 Review comment:
   ```suggestion
       return Tid.split(CarbonCommonConstants.FILE_SEPARATOR)[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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] akashrn5 commented on a change in pull request #3639: [CARBONDATA-3724] Secondary Index enable on partition Table

Posted by GitBox <gi...@apache.org>.
akashrn5 commented on a change in pull request #3639: [CARBONDATA-3724] Secondary Index enable on partition Table
URL: https://github.com/apache/carbondata/pull/3639#discussion_r386084353
 
 

 ##########
 File path: integration/spark/src/test/scala/org/apache/carbondata/spark/testsuite/iud/DeleteCarbonTableTestCase.scala
 ##########
 @@ -40,17 +40,20 @@ class DeleteCarbonTableTestCase extends QueryTest with BeforeAndAfterAll {
     sql("drop database  if exists iud_db cascade")
     sql("create database  iud_db")
 
-    sql("""create table iud_db.source2 (c11 string,c22 int,c33 string,c55 string, c66 int) STORED AS carbondata""")
+    sql(
+      """create table iud_db.source2 (c11 string,c22 int,c33 string,c55 string, c66 int) STORED
+        |AS carbondata""".stripMargin)
     sql(s"""LOAD DATA LOCAL INPATH '$resourcesPath/IUD/source2.csv' INTO table iud_db.source2""")
     sql("use iud_db")
   }
+
   test("delete data from carbon table with alias [where clause ]") {
     sql("""create table iud_db.dest (c1 string,c2 int,c3 string,c5 string) STORED AS carbondata""")
     sql(s"""LOAD DATA LOCAL INPATH '$resourcesPath/IUD/dest.csv' INTO table iud_db.dest""")
     sql("""delete from iud_db.dest d where d.c1 = 'a'""").show
     checkAnswer(
       sql("""select c2 from iud_db.dest"""),
-      Seq(Row(2), Row(3),Row(4), Row(5))
+      Seq(Row(2), Row(3), Row(4), Row(5))
 
 Review comment:
   revert

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] CarbonDataQA1 commented on issue #3639: [CARBONDATA-3724] Secondary Index enable on partition Table

Posted by GitBox <gi...@apache.org>.
CarbonDataQA1 commented on issue #3639: [CARBONDATA-3724] Secondary Index enable on partition Table
URL: https://github.com/apache/carbondata/pull/3639#issuecomment-593985123
 
 
   Build Success with Spark 2.3.4, Please check CI http://121.244.95.60:12545/job/ApacheCarbonPRBuilder2.3/2290/
   

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] CarbonDataQA1 commented on issue #3639: [CARBONDATA-3724] Secondary Index enable on partition Table

Posted by GitBox <gi...@apache.org>.
CarbonDataQA1 commented on issue #3639: [CARBONDATA-3724] Secondary Index enable on partition Table
URL: https://github.com/apache/carbondata/pull/3639#issuecomment-594511978
 
 
   Build Failed  with Spark 2.3.4, Please check CI http://121.244.95.60:12545/job/ApacheCarbonPRBuilder2.3/2318/
   

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] CarbonDataQA1 commented on issue #3639: [CARBONDATA-3724] Secondary Index enable on partition Table

Posted by GitBox <gi...@apache.org>.
CarbonDataQA1 commented on issue #3639: [CARBONDATA-3724] Secondary Index enable on partition Table
URL: https://github.com/apache/carbondata/pull/3639#issuecomment-611925344
 
 
   Build Success with Spark 2.4.5, Please check CI http://121.244.95.60:12545/job/ApacheCarbon_PR_Builder_2.4.5/991/
   

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] maheshrajus commented on a change in pull request #3639: [CARBONDATA-3724] Secondary Index enable on partition Table

Posted by GitBox <gi...@apache.org>.
maheshrajus commented on a change in pull request #3639: [CARBONDATA-3724] Secondary Index enable on partition Table
URL: https://github.com/apache/carbondata/pull/3639#discussion_r387552657
 
 

 ##########
 File path: index/secondary-index/src/test/scala/org/apache/carbondata/spark/testsuite/secondaryindex/TestSIWithPartition.scala
 ##########
 @@ -0,0 +1,379 @@
+/*
+ * 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.execution.SparkPlan
+import org.apache.spark.sql.secondaryindex.joins.BroadCastSIFilterPushJoin
+import org.apache.spark.sql.test.util.QueryTest
+import org.scalatest.{BeforeAndAfterAll, Ignore}
+
+class TestSIWithPartition extends QueryTest with BeforeAndAfterAll {
+
+  override protected def beforeAll(): Unit = {
+    sql("drop table if exists uniqdata1")
+    sql(
+      "CREATE TABLE uniqdata1 (CUST_ID INT,CUST_NAME STRING,DOB timestamp,DOJ timestamp," +
+      "BIGINT_COLUMN1 bigint,BIGINT_COLUMN2 bigint,DECIMAL_COLUMN1 DECIMAL(30, 10)," +
+      "DECIMAL_COLUMN2 DECIMAL(36, 10),Double_COLUMN1 double, Double_COLUMN2 double," +
+      "INTEGER_COLUMN1 int) PARTITIONED BY(ACTIVE_EMUI_VERSION string) STORED AS carbondata " +
+      "TBLPROPERTIES('TABLE_BLOCKSIZE'='256 MB')")
+    sql(s"LOAD DATA LOCAL INPATH '$resourcesPath/restructure/data_2000.csv' INTO " +
+        "TABLE uniqdata1 partition(ACTIVE_EMUI_VERSION='abc') OPTIONS('DELIMITER'=',', " +
+        "'BAD_RECORDS_LOGGER_ENABLE'='FALSE', 'BAD_RECORDS_ACTION'='FORCE','FILEHEADER'='CUST_ID," +
+        "CUST_NAME,ACTIVE_EMUI_VERSION,DOB,DOJ,BIGINT_COLUMN1,BIGINT_COLUMN2,DECIMAL_COLUMN1," +
+        "DECIMAL_COLUMN2,Double_COLUMN1,Double_COLUMN2,INTEGER_COLUMN1')")
+    sql(s"LOAD DATA LOCAL INPATH '$resourcesPath/restructure/data_2000.csv' INTO " +
+        "TABLE uniqdata1 partition(ACTIVE_EMUI_VERSION='abc') OPTIONS('DELIMITER'=',', " +
+        "'BAD_RECORDS_LOGGER_ENABLE'='FALSE', 'BAD_RECORDS_ACTION'='FORCE','FILEHEADER'='CUST_ID," +
+        "CUST_NAME,ACTIVE_EMUI_VERSION,DOB,DOJ,BIGINT_COLUMN1,BIGINT_COLUMN2,DECIMAL_COLUMN1," +
+        "DECIMAL_COLUMN2,Double_COLUMN1,Double_COLUMN2,INTEGER_COLUMN1')")
+    sql(s"LOAD DATA LOCAL INPATH '$resourcesPath/restructure/data_2000.csv' INTO " +
+        "TABLE uniqdata1 partition(ACTIVE_EMUI_VERSION='abc') OPTIONS('DELIMITER'=',', " +
+        "'BAD_RECORDS_LOGGER_ENABLE'='FALSE', 'BAD_RECORDS_ACTION'='FORCE','FILEHEADER'='CUST_ID," +
+        "CUST_NAME,ACTIVE_EMUI_VERSION,DOB,DOJ,BIGINT_COLUMN1,BIGINT_COLUMN2,DECIMAL_COLUMN1," +
+        "DECIMAL_COLUMN2,Double_COLUMN1,Double_COLUMN2,INTEGER_COLUMN1')")
+    sql(s"LOAD DATA LOCAL INPATH '$resourcesPath/restructure/data_2000.csv' INTO " +
+        "TABLE uniqdata1 partition(ACTIVE_EMUI_VERSION='abc') OPTIONS('DELIMITER'=',', " +
+        "'BAD_RECORDS_LOGGER_ENABLE'='FALSE', 'BAD_RECORDS_ACTION'='FORCE','FILEHEADER'='CUST_ID," +
+        "CUST_NAME,ACTIVE_EMUI_VERSION,DOB,DOJ,BIGINT_COLUMN1,BIGINT_COLUMN2,DECIMAL_COLUMN1," +
+        "DECIMAL_COLUMN2,Double_COLUMN1,Double_COLUMN2,INTEGER_COLUMN1')")
+  }
+
+  test("Testing SI on partition column") {
+    sql("drop index if exists indextable1 on uniqdata1")
+    intercept[UnsupportedOperationException] {
+      sql("create index indextable1 on table uniqdata1 (ACTIVE_EMUI_VERSION) AS 'carbondata'")
+    }
+  }
+
+  test("Testing SI without partition column") {
+    sql("drop index if exists indextable1 on uniqdata1")
+    sql("create index indextable1 on table uniqdata1 (DOB, CUST_NAME) AS 'carbondata'")
+    val withoutIndex =
+      sql("select * from uniqdata1 where ni(CUST_NAME='CUST_NAME_00108')")
+        .collect().toSeq
+
+    checkAnswer(sql("select * from uniqdata1 where CUST_NAME='CUST_NAME_00108'"),
+      withoutIndex)
+
+    val df = sql("select * from uniqdata1 where CUST_NAME='CUST_NAME_00108'")
+      .queryExecution
+      .sparkPlan
+    if (!isFilterPushedDownToSI(df)) {
+      assert(false)
+    } else {
+      assert(true)
+    }
+  }
+
+  test("Testing SI with partition column[where clause]") {
+    sql("drop index if exists indextable1 on uniqdata1")
+    sql("create index indextable1 on table uniqdata1 (DOB, CUST_NAME) AS 'carbondata'")
+    val withoutIndex =
+      sql(
+        "select * from uniqdata1 where ni(CUST_NAME='CUST_NAME_00108' and ACTIVE_EMUI_VERSION = " +
+        "'abc')")
+        .collect().toSeq
+
+    checkAnswer(sql(
+      "select * from uniqdata1 where CUST_NAME='CUST_NAME_00108' and ACTIVE_EMUI_VERSION = 'abc'"),
+      withoutIndex)
+
+    val df = sql(
+      "select * from uniqdata1 where CUST_NAME='CUST_NAME_00108' and ACTIVE_EMUI_VERSION = 'abc'")
+      .queryExecution
+      .sparkPlan
+    if (!isFilterPushedDownToSI(df)) {
+      assert(false)
+    } else {
+      assert(true)
+    }
+  }
+
+  test("Testing SI on partition table with OR condition") {
+    sql("drop index if exists indextable1 on uniqdata1")
+    sql("create index indextable1 on table uniqdata1 (DOB, CUST_NAME) AS 'carbondata'")
+    val withoutIndex =
+      sql(
+        "select * from uniqdata1 where ni(CUST_NAME='CUST_NAME_00108' OR ACTIVE_EMUI_VERSION = " +
+        "'abc')")
+        .collect().toSeq
+
+    checkAnswer(sql(
+      "select * from uniqdata1 where CUST_NAME='CUST_NAME_00108' OR ACTIVE_EMUI_VERSION = 'abc'"),
+      withoutIndex)
+
+    val df = sql(
+      "select * from uniqdata1 where CUST_NAME='CUST_NAME_00108' OR ACTIVE_EMUI_VERSION = 'abc'")
+      .queryExecution
+      .sparkPlan
+    if (!isFilterPushedDownToSI(df)) {
+      assert(true)
+    } else {
+      assert(false)
+    }
+  }
+
+  test("Testing SI on partition table with combination of OR OR") {
+    sql("drop index if exists indextable1 on uniqdata1")
+    sql("create index indextable1 on table uniqdata1 (DOB, CUST_NAME) AS 'carbondata'")
+    val withoutIndex =
+      sql(
+        "select * from uniqdata1 where ni(CUST_NAME='CUST_NAME_00108' OR CUST_ID='9000' OR " +
+        "ACTIVE_EMUI_VERSION = " +
+        "'abc')")
+        .collect().toSeq
+
+    checkAnswer(sql(
+      "select * from uniqdata1 where CUST_NAME='CUST_NAME_00108' OR CUST_ID='9000' OR " +
+      "ACTIVE_EMUI_VERSION = 'abc'"),
+      withoutIndex)
+
+    val df = sql(
+      "select * from uniqdata1 where CUST_NAME='CUST_NAME_00108' OR CUST_ID='9000' OR " +
+      "ACTIVE_EMUI_VERSION = 'abc'")
+      .queryExecution
+      .sparkPlan
+    if (!isFilterPushedDownToSI(df)) {
+      assert(true)
+    } else {
+      assert(false)
+    }
+  }
+
+  test("Testing SI on partition table with combination of OR AND") {
+    sql("drop index if exists indextable1 on uniqdata1")
+    sql("create index indextable1 on table uniqdata1 (DOB, CUST_NAME) AS 'carbondata'")
+    val withoutIndex =
+      sql(
+        "select * from uniqdata1 where ni(CUST_NAME='CUST_NAME_00108' OR CUST_ID='9000' AND " +
+        "ACTIVE_EMUI_VERSION = " +
+        "'abc')")
+        .collect().toSeq
+
+    checkAnswer(sql(
+      "select * from uniqdata1 where CUST_NAME='CUST_NAME_00108' OR CUST_ID='9000' AND " +
+      "ACTIVE_EMUI_VERSION = 'abc'"),
+      withoutIndex)
+
+    val df = sql(
+      "select * from uniqdata1 where CUST_NAME='CUST_NAME_00108' OR CUST_ID='9000' AND " +
+      "ACTIVE_EMUI_VERSION = 'abc'")
+      .queryExecution
+      .sparkPlan
+    if (!isFilterPushedDownToSI(df)) {
+      assert(true)
+    } else {
+      assert(false)
+    }
+  }
+
+  test("Testing SI on partition table with combination of AND OR") {
+    sql("drop index if exists indextable1 on uniqdata1")
+    sql("create index indextable1 on table uniqdata1 (DOB, CUST_NAME) AS 'carbondata'")
+    val withoutIndex =
+      sql(
+        "select * from uniqdata1 where ni(CUST_NAME='CUST_NAME_00108' AND CUST_ID='9000' OR " +
+        "ACTIVE_EMUI_VERSION = " +
+        "'abc')")
+        .collect().toSeq
+
+    checkAnswer(sql(
+      "select * from uniqdata1 where CUST_NAME='CUST_NAME_00108' AND CUST_ID='9000' OR " +
+      "ACTIVE_EMUI_VERSION = 'abc'"),
+      withoutIndex)
+
+    val df = sql(
+      "select * from uniqdata1 where CUST_NAME='CUST_NAME_00108' AND CUST_ID='9000' OR " +
+      "ACTIVE_EMUI_VERSION = 'abc'")
+      .queryExecution
+      .sparkPlan
+    if (!isFilterPushedDownToSI(df)) {
+      assert(true)
+    } else {
+      assert(false)
+    }
+  }
+
+  test("Testing SI on partition table with combination of AND AND") {
+    sql("drop index if exists indextable1 on uniqdata1")
+    sql("create index indextable1 on table uniqdata1 (DOB, CUST_NAME) AS 'carbondata'")
+    val withoutIndex =
+      sql(
+        "select * from uniqdata1 where ni(CUST_NAME='CUST_NAME_00108' AND CUST_ID='9000' AND " +
+        "ACTIVE_EMUI_VERSION = " +
+        "'abc')")
+        .collect().toSeq
+
+    checkAnswer(sql(
+      "select * from uniqdata1 where CUST_NAME='CUST_NAME_00108' AND CUST_ID='9000' AND " +
+      "ACTIVE_EMUI_VERSION = 'abc'"),
+      withoutIndex)
+
+    val df = sql(
+      "select * from uniqdata1 where CUST_NAME='CUST_NAME_00108' AND CUST_ID='9000' AND " +
+      "ACTIVE_EMUI_VERSION = 'abc'")
+      .queryExecution
+      .sparkPlan
+    if (!isFilterPushedDownToSI(df)) {
+      assert(false)
+    } else {
+      assert(true)
+    }
+  }
+
+  test("Testing SI on partition table with major compaction") {
+    sql("drop index if exists indextable1 on uniqdata1")
+    sql("create index indextable1 on table uniqdata1 (DOB, CUST_NAME) AS 'carbondata'")
+    val withoutIndex =
+      sql(
+        "select * from uniqdata1 where CUST_NAME='CUST_NAME_00108' and ACTIVE_EMUI_VERSION = " +
+        "'abc'")
+        .collect().toSeq
+
+    sql("alter table uniqdata1 compact 'major'")
+
+    checkAnswer(sql(
+      "select * from uniqdata1 where CUST_NAME='CUST_NAME_00108' and ACTIVE_EMUI_VERSION = 'abc'"),
+      withoutIndex)
+
+    val df = sql(
+      "select * from uniqdata1 where CUST_NAME='CUST_NAME_00108' and ACTIVE_EMUI_VERSION = 'abc'")
+      .queryExecution
+      .sparkPlan
+    if (!isFilterPushedDownToSI(df)) {
+      assert(false)
+    } else {
+      assert(true)
+    }
+  }
+
+  test("Testing SI on partition table with minor compaction") {
+    sql("drop index if exists indextable1 on uniqdata1")
+    sql("create index indextable1 on table uniqdata1 (DOB, CUST_NAME) AS 'carbondata'")
+
+    val withoutIndex =
+      sql(
+        "select * from uniqdata1 where CUST_NAME='CUST_NAME_00108' and ACTIVE_EMUI_VERSION = " +
+        "'abc'")
+        .collect().toSeq
+
+    sql("alter table uniqdata1 compact 'minor'")
+
+    checkAnswer(sql(
+      "select * from uniqdata1 where CUST_NAME='CUST_NAME_00108' and ACTIVE_EMUI_VERSION = 'abc'"),
+      withoutIndex)
+
+    val df = sql(
+      "select * from uniqdata1 where CUST_NAME='CUST_NAME_00108' and ACTIVE_EMUI_VERSION = 'abc'")
+      .queryExecution
+      .sparkPlan
+    if (!isFilterPushedDownToSI(df)) {
+      assert(false)
+    } else {
+      assert(true)
+    }
+  }
+
+  test("Testing SI on partition table with delete") {
+    sql("drop index if exists indextable1 on uniqdata1")
+    sql("create index indextable1 on table uniqdata1 (DOB, CUST_NAME) AS 'carbondata'")
+
+    checkAnswer(sql(
+      "select count(*) from uniqdata1 where CUST_NAME='CUST_NAME_00108' and ACTIVE_EMUI_VERSION =" +
+      " 'abc'"),
+      Seq(Row(4)))
+
+    sql("delete from uniqdata1 where CUST_NAME='CUST_NAME_00108'").show()
+
+    checkAnswer(sql(
+      "select count(*) from uniqdata1 where CUST_NAME='CUST_NAME_00108' and ACTIVE_EMUI_VERSION =" +
+      " 'abc'"),
+      Seq(Row(0)))
+
+    val df = sql(
+      "select * from uniqdata1 where CUST_NAME='CUST_NAME_00108' and ACTIVE_EMUI_VERSION = 'abc'")
+      .queryExecution
+      .sparkPlan
+    if (!isFilterPushedDownToSI(df)) {
+      assert(false)
+    } else {
+      assert(true)
+    }
+  }
+
+  test("Testing SI on partition table with update") {
+    sql("drop index if exists indextable1 on uniqdata1")
+    sql("create index indextable1 on table uniqdata1 (DOB, CUST_NAME) AS 'carbondata'")
+
+    checkAnswer(sql(
+      "select count(*) from uniqdata1 where CUST_ID='9000' and ACTIVE_EMUI_VERSION = 'abc'"),
+      Seq(Row(4)))
+    intercept[RuntimeException] {
+      sql("update uniqdata1 d set (d.CUST_ID) = ('8000')  where d.CUST_ID = '9000'").show()
+    }
+  }
+
+  test("Testing SI on partition table with rename") {
+    sql("drop index if exists indextable1 on uniqdata1")
+    sql("create index indextable1 on table uniqdata1 (DOB, CUST_NAME) AS 'carbondata'")
+
+    val withoutIndex =
+      sql(
+        "select * from uniqdata1 where CUST_NAME='CUST_NAME_00108' and ACTIVE_EMUI_VERSION = " +
+        "'abc'")
+        .collect().toSeq
+
+    sql("alter table uniqdata1 change CUST_NAME test string")
+
+    checkAnswer(sql(
+      "select * from uniqdata1 where test='CUST_NAME_00108' and ACTIVE_EMUI_VERSION = 'abc'"),
+      withoutIndex)
+
+    val df = sql(
+      "select * from uniqdata1 where test='CUST_NAME_00108' and ACTIVE_EMUI_VERSION = 'abc'")
+      .queryExecution
+      .sparkPlan
+    if (!isFilterPushedDownToSI(df)) {
+      assert(false)
+    } else {
+      assert(true)
+    }
+  }
+
+  override protected def afterAll(): Unit = {
+    sql("drop index if exists indextable1 on uniqdata1")
+    sql("drop table if exists uniqdata1")
+  }
+
+  /**
+   * Method to check whether the filter is push down to SI table or not
+   *
+   * @param sparkPlan
+   * @return
+   */
+  private def isFilterPushedDownToSI(sparkPlan: SparkPlan): Boolean = {
 
 Review comment:
   OK

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] CarbonDataQA1 commented on issue #3639: [WIP] Secondary Index enable on partition Table

Posted by GitBox <gi...@apache.org>.
CarbonDataQA1 commented on issue #3639: [WIP] Secondary Index enable on partition Table
URL: https://github.com/apache/carbondata/pull/3639#issuecomment-592065881
 
 
   Build Success with Spark 2.4.4, Please check CI http://121.244.95.60:12545/job/ApacheCarbon_PR_Builder_2.4.4/514/
   

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] Indhumathi27 commented on a change in pull request #3639: [CARBONDATA-3724] Secondary Index enable on partition Table

Posted by GitBox <gi...@apache.org>.
Indhumathi27 commented on a change in pull request #3639: [CARBONDATA-3724] Secondary Index enable on partition Table
URL: https://github.com/apache/carbondata/pull/3639#discussion_r387548712
 
 

 ##########
 File path: index/secondary-index/src/test/scala/org/apache/carbondata/spark/testsuite/secondaryindex/TestSIWithPartition.scala
 ##########
 @@ -0,0 +1,379 @@
+/*
+ * 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.execution.SparkPlan
+import org.apache.spark.sql.secondaryindex.joins.BroadCastSIFilterPushJoin
+import org.apache.spark.sql.test.util.QueryTest
+import org.scalatest.{BeforeAndAfterAll, Ignore}
+
+class TestSIWithPartition extends QueryTest with BeforeAndAfterAll {
+
+  override protected def beforeAll(): Unit = {
+    sql("drop table if exists uniqdata1")
+    sql(
+      "CREATE TABLE uniqdata1 (CUST_ID INT,CUST_NAME STRING,DOB timestamp,DOJ timestamp," +
+      "BIGINT_COLUMN1 bigint,BIGINT_COLUMN2 bigint,DECIMAL_COLUMN1 DECIMAL(30, 10)," +
+      "DECIMAL_COLUMN2 DECIMAL(36, 10),Double_COLUMN1 double, Double_COLUMN2 double," +
+      "INTEGER_COLUMN1 int) PARTITIONED BY(ACTIVE_EMUI_VERSION string) STORED AS carbondata " +
+      "TBLPROPERTIES('TABLE_BLOCKSIZE'='256 MB')")
+    sql(s"LOAD DATA LOCAL INPATH '$resourcesPath/restructure/data_2000.csv' INTO " +
+        "TABLE uniqdata1 partition(ACTIVE_EMUI_VERSION='abc') OPTIONS('DELIMITER'=',', " +
+        "'BAD_RECORDS_LOGGER_ENABLE'='FALSE', 'BAD_RECORDS_ACTION'='FORCE','FILEHEADER'='CUST_ID," +
+        "CUST_NAME,ACTIVE_EMUI_VERSION,DOB,DOJ,BIGINT_COLUMN1,BIGINT_COLUMN2,DECIMAL_COLUMN1," +
+        "DECIMAL_COLUMN2,Double_COLUMN1,Double_COLUMN2,INTEGER_COLUMN1')")
+    sql(s"LOAD DATA LOCAL INPATH '$resourcesPath/restructure/data_2000.csv' INTO " +
+        "TABLE uniqdata1 partition(ACTIVE_EMUI_VERSION='abc') OPTIONS('DELIMITER'=',', " +
+        "'BAD_RECORDS_LOGGER_ENABLE'='FALSE', 'BAD_RECORDS_ACTION'='FORCE','FILEHEADER'='CUST_ID," +
+        "CUST_NAME,ACTIVE_EMUI_VERSION,DOB,DOJ,BIGINT_COLUMN1,BIGINT_COLUMN2,DECIMAL_COLUMN1," +
+        "DECIMAL_COLUMN2,Double_COLUMN1,Double_COLUMN2,INTEGER_COLUMN1')")
+    sql(s"LOAD DATA LOCAL INPATH '$resourcesPath/restructure/data_2000.csv' INTO " +
+        "TABLE uniqdata1 partition(ACTIVE_EMUI_VERSION='abc') OPTIONS('DELIMITER'=',', " +
+        "'BAD_RECORDS_LOGGER_ENABLE'='FALSE', 'BAD_RECORDS_ACTION'='FORCE','FILEHEADER'='CUST_ID," +
+        "CUST_NAME,ACTIVE_EMUI_VERSION,DOB,DOJ,BIGINT_COLUMN1,BIGINT_COLUMN2,DECIMAL_COLUMN1," +
+        "DECIMAL_COLUMN2,Double_COLUMN1,Double_COLUMN2,INTEGER_COLUMN1')")
+    sql(s"LOAD DATA LOCAL INPATH '$resourcesPath/restructure/data_2000.csv' INTO " +
+        "TABLE uniqdata1 partition(ACTIVE_EMUI_VERSION='abc') OPTIONS('DELIMITER'=',', " +
+        "'BAD_RECORDS_LOGGER_ENABLE'='FALSE', 'BAD_RECORDS_ACTION'='FORCE','FILEHEADER'='CUST_ID," +
+        "CUST_NAME,ACTIVE_EMUI_VERSION,DOB,DOJ,BIGINT_COLUMN1,BIGINT_COLUMN2,DECIMAL_COLUMN1," +
+        "DECIMAL_COLUMN2,Double_COLUMN1,Double_COLUMN2,INTEGER_COLUMN1')")
+  }
+
+  test("Testing SI on partition column") {
+    sql("drop index if exists indextable1 on uniqdata1")
+    intercept[UnsupportedOperationException] {
+      sql("create index indextable1 on table uniqdata1 (ACTIVE_EMUI_VERSION) AS 'carbondata'")
+    }
+  }
+
+  test("Testing SI without partition column") {
+    sql("drop index if exists indextable1 on uniqdata1")
+    sql("create index indextable1 on table uniqdata1 (DOB, CUST_NAME) AS 'carbondata'")
+    val withoutIndex =
+      sql("select * from uniqdata1 where ni(CUST_NAME='CUST_NAME_00108')")
+        .collect().toSeq
+
+    checkAnswer(sql("select * from uniqdata1 where CUST_NAME='CUST_NAME_00108'"),
+      withoutIndex)
+
+    val df = sql("select * from uniqdata1 where CUST_NAME='CUST_NAME_00108'")
+      .queryExecution
+      .sparkPlan
+    if (!isFilterPushedDownToSI(df)) {
+      assert(false)
+    } else {
+      assert(true)
+    }
+  }
+
+  test("Testing SI with partition column[where clause]") {
+    sql("drop index if exists indextable1 on uniqdata1")
+    sql("create index indextable1 on table uniqdata1 (DOB, CUST_NAME) AS 'carbondata'")
+    val withoutIndex =
+      sql(
+        "select * from uniqdata1 where ni(CUST_NAME='CUST_NAME_00108' and ACTIVE_EMUI_VERSION = " +
+        "'abc')")
+        .collect().toSeq
+
+    checkAnswer(sql(
+      "select * from uniqdata1 where CUST_NAME='CUST_NAME_00108' and ACTIVE_EMUI_VERSION = 'abc'"),
+      withoutIndex)
+
+    val df = sql(
+      "select * from uniqdata1 where CUST_NAME='CUST_NAME_00108' and ACTIVE_EMUI_VERSION = 'abc'")
+      .queryExecution
+      .sparkPlan
+    if (!isFilterPushedDownToSI(df)) {
+      assert(false)
+    } else {
+      assert(true)
+    }
+  }
+
+  test("Testing SI on partition table with OR condition") {
+    sql("drop index if exists indextable1 on uniqdata1")
+    sql("create index indextable1 on table uniqdata1 (DOB, CUST_NAME) AS 'carbondata'")
+    val withoutIndex =
+      sql(
+        "select * from uniqdata1 where ni(CUST_NAME='CUST_NAME_00108' OR ACTIVE_EMUI_VERSION = " +
+        "'abc')")
+        .collect().toSeq
+
+    checkAnswer(sql(
+      "select * from uniqdata1 where CUST_NAME='CUST_NAME_00108' OR ACTIVE_EMUI_VERSION = 'abc'"),
+      withoutIndex)
+
+    val df = sql(
+      "select * from uniqdata1 where CUST_NAME='CUST_NAME_00108' OR ACTIVE_EMUI_VERSION = 'abc'")
+      .queryExecution
+      .sparkPlan
+    if (!isFilterPushedDownToSI(df)) {
+      assert(true)
+    } else {
+      assert(false)
+    }
+  }
+
+  test("Testing SI on partition table with combination of OR OR") {
+    sql("drop index if exists indextable1 on uniqdata1")
+    sql("create index indextable1 on table uniqdata1 (DOB, CUST_NAME) AS 'carbondata'")
+    val withoutIndex =
+      sql(
+        "select * from uniqdata1 where ni(CUST_NAME='CUST_NAME_00108' OR CUST_ID='9000' OR " +
+        "ACTIVE_EMUI_VERSION = " +
+        "'abc')")
+        .collect().toSeq
+
+    checkAnswer(sql(
+      "select * from uniqdata1 where CUST_NAME='CUST_NAME_00108' OR CUST_ID='9000' OR " +
+      "ACTIVE_EMUI_VERSION = 'abc'"),
+      withoutIndex)
+
+    val df = sql(
+      "select * from uniqdata1 where CUST_NAME='CUST_NAME_00108' OR CUST_ID='9000' OR " +
+      "ACTIVE_EMUI_VERSION = 'abc'")
+      .queryExecution
+      .sparkPlan
+    if (!isFilterPushedDownToSI(df)) {
+      assert(true)
+    } else {
+      assert(false)
+    }
+  }
+
+  test("Testing SI on partition table with combination of OR AND") {
+    sql("drop index if exists indextable1 on uniqdata1")
+    sql("create index indextable1 on table uniqdata1 (DOB, CUST_NAME) AS 'carbondata'")
+    val withoutIndex =
+      sql(
+        "select * from uniqdata1 where ni(CUST_NAME='CUST_NAME_00108' OR CUST_ID='9000' AND " +
+        "ACTIVE_EMUI_VERSION = " +
+        "'abc')")
+        .collect().toSeq
+
+    checkAnswer(sql(
+      "select * from uniqdata1 where CUST_NAME='CUST_NAME_00108' OR CUST_ID='9000' AND " +
+      "ACTIVE_EMUI_VERSION = 'abc'"),
+      withoutIndex)
+
+    val df = sql(
+      "select * from uniqdata1 where CUST_NAME='CUST_NAME_00108' OR CUST_ID='9000' AND " +
+      "ACTIVE_EMUI_VERSION = 'abc'")
+      .queryExecution
+      .sparkPlan
+    if (!isFilterPushedDownToSI(df)) {
+      assert(true)
+    } else {
+      assert(false)
+    }
+  }
+
+  test("Testing SI on partition table with combination of AND OR") {
+    sql("drop index if exists indextable1 on uniqdata1")
+    sql("create index indextable1 on table uniqdata1 (DOB, CUST_NAME) AS 'carbondata'")
+    val withoutIndex =
+      sql(
+        "select * from uniqdata1 where ni(CUST_NAME='CUST_NAME_00108' AND CUST_ID='9000' OR " +
+        "ACTIVE_EMUI_VERSION = " +
+        "'abc')")
+        .collect().toSeq
+
+    checkAnswer(sql(
+      "select * from uniqdata1 where CUST_NAME='CUST_NAME_00108' AND CUST_ID='9000' OR " +
+      "ACTIVE_EMUI_VERSION = 'abc'"),
+      withoutIndex)
+
+    val df = sql(
+      "select * from uniqdata1 where CUST_NAME='CUST_NAME_00108' AND CUST_ID='9000' OR " +
+      "ACTIVE_EMUI_VERSION = 'abc'")
+      .queryExecution
+      .sparkPlan
+    if (!isFilterPushedDownToSI(df)) {
+      assert(true)
+    } else {
+      assert(false)
+    }
+  }
+
+  test("Testing SI on partition table with combination of AND AND") {
+    sql("drop index if exists indextable1 on uniqdata1")
+    sql("create index indextable1 on table uniqdata1 (DOB, CUST_NAME) AS 'carbondata'")
+    val withoutIndex =
+      sql(
+        "select * from uniqdata1 where ni(CUST_NAME='CUST_NAME_00108' AND CUST_ID='9000' AND " +
+        "ACTIVE_EMUI_VERSION = " +
+        "'abc')")
+        .collect().toSeq
+
+    checkAnswer(sql(
+      "select * from uniqdata1 where CUST_NAME='CUST_NAME_00108' AND CUST_ID='9000' AND " +
+      "ACTIVE_EMUI_VERSION = 'abc'"),
+      withoutIndex)
+
+    val df = sql(
+      "select * from uniqdata1 where CUST_NAME='CUST_NAME_00108' AND CUST_ID='9000' AND " +
+      "ACTIVE_EMUI_VERSION = 'abc'")
+      .queryExecution
+      .sparkPlan
+    if (!isFilterPushedDownToSI(df)) {
+      assert(false)
+    } else {
+      assert(true)
+    }
+  }
+
+  test("Testing SI on partition table with major compaction") {
+    sql("drop index if exists indextable1 on uniqdata1")
+    sql("create index indextable1 on table uniqdata1 (DOB, CUST_NAME) AS 'carbondata'")
+    val withoutIndex =
+      sql(
+        "select * from uniqdata1 where CUST_NAME='CUST_NAME_00108' and ACTIVE_EMUI_VERSION = " +
+        "'abc'")
+        .collect().toSeq
+
+    sql("alter table uniqdata1 compact 'major'")
+
+    checkAnswer(sql(
+      "select * from uniqdata1 where CUST_NAME='CUST_NAME_00108' and ACTIVE_EMUI_VERSION = 'abc'"),
+      withoutIndex)
+
+    val df = sql(
+      "select * from uniqdata1 where CUST_NAME='CUST_NAME_00108' and ACTIVE_EMUI_VERSION = 'abc'")
+      .queryExecution
+      .sparkPlan
+    if (!isFilterPushedDownToSI(df)) {
+      assert(false)
+    } else {
+      assert(true)
+    }
+  }
+
+  test("Testing SI on partition table with minor compaction") {
+    sql("drop index if exists indextable1 on uniqdata1")
+    sql("create index indextable1 on table uniqdata1 (DOB, CUST_NAME) AS 'carbondata'")
+
+    val withoutIndex =
+      sql(
+        "select * from uniqdata1 where CUST_NAME='CUST_NAME_00108' and ACTIVE_EMUI_VERSION = " +
+        "'abc'")
+        .collect().toSeq
+
+    sql("alter table uniqdata1 compact 'minor'")
+
+    checkAnswer(sql(
+      "select * from uniqdata1 where CUST_NAME='CUST_NAME_00108' and ACTIVE_EMUI_VERSION = 'abc'"),
+      withoutIndex)
+
+    val df = sql(
+      "select * from uniqdata1 where CUST_NAME='CUST_NAME_00108' and ACTIVE_EMUI_VERSION = 'abc'")
+      .queryExecution
+      .sparkPlan
+    if (!isFilterPushedDownToSI(df)) {
+      assert(false)
+    } else {
+      assert(true)
+    }
+  }
+
+  test("Testing SI on partition table with delete") {
+    sql("drop index if exists indextable1 on uniqdata1")
+    sql("create index indextable1 on table uniqdata1 (DOB, CUST_NAME) AS 'carbondata'")
+
+    checkAnswer(sql(
+      "select count(*) from uniqdata1 where CUST_NAME='CUST_NAME_00108' and ACTIVE_EMUI_VERSION =" +
+      " 'abc'"),
+      Seq(Row(4)))
+
+    sql("delete from uniqdata1 where CUST_NAME='CUST_NAME_00108'").show()
+
+    checkAnswer(sql(
+      "select count(*) from uniqdata1 where CUST_NAME='CUST_NAME_00108' and ACTIVE_EMUI_VERSION =" +
+      " 'abc'"),
+      Seq(Row(0)))
+
+    val df = sql(
+      "select * from uniqdata1 where CUST_NAME='CUST_NAME_00108' and ACTIVE_EMUI_VERSION = 'abc'")
+      .queryExecution
+      .sparkPlan
+    if (!isFilterPushedDownToSI(df)) {
+      assert(false)
+    } else {
+      assert(true)
+    }
+  }
+
+  test("Testing SI on partition table with update") {
+    sql("drop index if exists indextable1 on uniqdata1")
+    sql("create index indextable1 on table uniqdata1 (DOB, CUST_NAME) AS 'carbondata'")
+
+    checkAnswer(sql(
+      "select count(*) from uniqdata1 where CUST_ID='9000' and ACTIVE_EMUI_VERSION = 'abc'"),
+      Seq(Row(4)))
+    intercept[RuntimeException] {
+      sql("update uniqdata1 d set (d.CUST_ID) = ('8000')  where d.CUST_ID = '9000'").show()
+    }
+  }
+
+  test("Testing SI on partition table with rename") {
+    sql("drop index if exists indextable1 on uniqdata1")
+    sql("create index indextable1 on table uniqdata1 (DOB, CUST_NAME) AS 'carbondata'")
+
+    val withoutIndex =
+      sql(
+        "select * from uniqdata1 where CUST_NAME='CUST_NAME_00108' and ACTIVE_EMUI_VERSION = " +
+        "'abc'")
+        .collect().toSeq
+
+    sql("alter table uniqdata1 change CUST_NAME test string")
+
+    checkAnswer(sql(
+      "select * from uniqdata1 where test='CUST_NAME_00108' and ACTIVE_EMUI_VERSION = 'abc'"),
+      withoutIndex)
+
+    val df = sql(
+      "select * from uniqdata1 where test='CUST_NAME_00108' and ACTIVE_EMUI_VERSION = 'abc'")
+      .queryExecution
+      .sparkPlan
+    if (!isFilterPushedDownToSI(df)) {
+      assert(false)
+    } else {
+      assert(true)
+    }
+  }
+
+  override protected def afterAll(): Unit = {
+    sql("drop index if exists indextable1 on uniqdata1")
+    sql("drop table if exists uniqdata1")
+  }
+
+  /**
+   * Method to check whether the filter is push down to SI table or not
+   *
+   * @param sparkPlan
+   * @return
+   */
+  private def isFilterPushedDownToSI(sparkPlan: SparkPlan): Boolean = {
 
 Review comment:
   But isFilterPushedDownToSI is common for all testClass. Can move this method to a Util and call from all Testclasses

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] maheshrajus commented on a change in pull request #3639: [CARBONDATA-3724] Secondary Index enable on partition Table

Posted by GitBox <gi...@apache.org>.
maheshrajus commented on a change in pull request #3639: [CARBONDATA-3724] Secondary Index enable on partition Table
URL: https://github.com/apache/carbondata/pull/3639#discussion_r387613118
 
 

 ##########
 File path: hadoop/src/main/java/org/apache/carbondata/hadoop/api/CarbonTableInputFormat.java
 ##########
 @@ -188,7 +189,10 @@
 
     List<Segment> segmentToAccess =
         getFilteredSegment(job, validAndInProgressSegments, false, readCommittedScope);
-
+    String segmentFileName = job.getConfiguration().get(CarbonCommonConstants.CURRENT_SEGMENTFILE);
+    if (segmentFileName != null) {
+      segmentToAccess.get(0).setSegmentFileName(segmentFileName + CarbonTablePath.SEGMENT_EXT);
 
 Review comment:
   OK

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] ajantha-bhat commented on issue #3639: [WIP] Secondary Index enable on partition Table

Posted by GitBox <gi...@apache.org>.
ajantha-bhat commented on issue #3639: [WIP] Secondary Index enable on partition Table
URL: https://github.com/apache/carbondata/pull/3639#issuecomment-590949644
 
 
   retest this please

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] maheshrajus commented on issue #3639: [WIP] Secondary Index enable on partition Table

Posted by GitBox <gi...@apache.org>.
maheshrajus commented on issue #3639: [WIP] Secondary Index enable on partition Table
URL: https://github.com/apache/carbondata/pull/3639#issuecomment-591318276
 
 
   retest this please

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] maheshrajus commented on a change in pull request #3639: [CARBONDATA-3724] Secondary Index enable on partition Table

Posted by GitBox <gi...@apache.org>.
maheshrajus commented on a change in pull request #3639: [CARBONDATA-3724] Secondary Index enable on partition Table
URL: https://github.com/apache/carbondata/pull/3639#discussion_r387529136
 
 

 ##########
 File path: index/secondary-index/src/test/scala/org/apache/carbondata/spark/testsuite/secondaryindex/TestSIWithPartition.scala
 ##########
 @@ -0,0 +1,379 @@
+/*
+ * 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.execution.SparkPlan
+import org.apache.spark.sql.secondaryindex.joins.BroadCastSIFilterPushJoin
+import org.apache.spark.sql.test.util.QueryTest
+import org.scalatest.{BeforeAndAfterAll, Ignore}
+
+class TestSIWithPartition extends QueryTest with BeforeAndAfterAll {
+
+  override protected def beforeAll(): Unit = {
+    sql("drop table if exists uniqdata1")
+    sql(
+      "CREATE TABLE uniqdata1 (CUST_ID INT,CUST_NAME STRING,DOB timestamp,DOJ timestamp," +
+      "BIGINT_COLUMN1 bigint,BIGINT_COLUMN2 bigint,DECIMAL_COLUMN1 DECIMAL(30, 10)," +
+      "DECIMAL_COLUMN2 DECIMAL(36, 10),Double_COLUMN1 double, Double_COLUMN2 double," +
+      "INTEGER_COLUMN1 int) PARTITIONED BY(ACTIVE_EMUI_VERSION string) STORED AS carbondata " +
+      "TBLPROPERTIES('TABLE_BLOCKSIZE'='256 MB')")
+    sql(s"LOAD DATA LOCAL INPATH '$resourcesPath/restructure/data_2000.csv' INTO " +
+        "TABLE uniqdata1 partition(ACTIVE_EMUI_VERSION='abc') OPTIONS('DELIMITER'=',', " +
+        "'BAD_RECORDS_LOGGER_ENABLE'='FALSE', 'BAD_RECORDS_ACTION'='FORCE','FILEHEADER'='CUST_ID," +
+        "CUST_NAME,ACTIVE_EMUI_VERSION,DOB,DOJ,BIGINT_COLUMN1,BIGINT_COLUMN2,DECIMAL_COLUMN1," +
+        "DECIMAL_COLUMN2,Double_COLUMN1,Double_COLUMN2,INTEGER_COLUMN1')")
+    sql(s"LOAD DATA LOCAL INPATH '$resourcesPath/restructure/data_2000.csv' INTO " +
+        "TABLE uniqdata1 partition(ACTIVE_EMUI_VERSION='abc') OPTIONS('DELIMITER'=',', " +
+        "'BAD_RECORDS_LOGGER_ENABLE'='FALSE', 'BAD_RECORDS_ACTION'='FORCE','FILEHEADER'='CUST_ID," +
+        "CUST_NAME,ACTIVE_EMUI_VERSION,DOB,DOJ,BIGINT_COLUMN1,BIGINT_COLUMN2,DECIMAL_COLUMN1," +
+        "DECIMAL_COLUMN2,Double_COLUMN1,Double_COLUMN2,INTEGER_COLUMN1')")
+    sql(s"LOAD DATA LOCAL INPATH '$resourcesPath/restructure/data_2000.csv' INTO " +
+        "TABLE uniqdata1 partition(ACTIVE_EMUI_VERSION='abc') OPTIONS('DELIMITER'=',', " +
+        "'BAD_RECORDS_LOGGER_ENABLE'='FALSE', 'BAD_RECORDS_ACTION'='FORCE','FILEHEADER'='CUST_ID," +
+        "CUST_NAME,ACTIVE_EMUI_VERSION,DOB,DOJ,BIGINT_COLUMN1,BIGINT_COLUMN2,DECIMAL_COLUMN1," +
+        "DECIMAL_COLUMN2,Double_COLUMN1,Double_COLUMN2,INTEGER_COLUMN1')")
+    sql(s"LOAD DATA LOCAL INPATH '$resourcesPath/restructure/data_2000.csv' INTO " +
+        "TABLE uniqdata1 partition(ACTIVE_EMUI_VERSION='abc') OPTIONS('DELIMITER'=',', " +
+        "'BAD_RECORDS_LOGGER_ENABLE'='FALSE', 'BAD_RECORDS_ACTION'='FORCE','FILEHEADER'='CUST_ID," +
+        "CUST_NAME,ACTIVE_EMUI_VERSION,DOB,DOJ,BIGINT_COLUMN1,BIGINT_COLUMN2,DECIMAL_COLUMN1," +
+        "DECIMAL_COLUMN2,Double_COLUMN1,Double_COLUMN2,INTEGER_COLUMN1')")
+  }
+
+  test("Testing SI on partition column") {
+    sql("drop index if exists indextable1 on uniqdata1")
+    intercept[UnsupportedOperationException] {
+      sql("create index indextable1 on table uniqdata1 (ACTIVE_EMUI_VERSION) AS 'carbondata'")
+    }
+  }
+
+  test("Testing SI without partition column") {
+    sql("drop index if exists indextable1 on uniqdata1")
+    sql("create index indextable1 on table uniqdata1 (DOB, CUST_NAME) AS 'carbondata'")
+    val withoutIndex =
+      sql("select * from uniqdata1 where ni(CUST_NAME='CUST_NAME_00108')")
+        .collect().toSeq
+
+    checkAnswer(sql("select * from uniqdata1 where CUST_NAME='CUST_NAME_00108'"),
+      withoutIndex)
+
+    val df = sql("select * from uniqdata1 where CUST_NAME='CUST_NAME_00108'")
+      .queryExecution
+      .sparkPlan
+    if (!isFilterPushedDownToSI(df)) {
+      assert(false)
+    } else {
+      assert(true)
+    }
+  }
+
+  test("Testing SI with partition column[where clause]") {
+    sql("drop index if exists indextable1 on uniqdata1")
+    sql("create index indextable1 on table uniqdata1 (DOB, CUST_NAME) AS 'carbondata'")
+    val withoutIndex =
+      sql(
+        "select * from uniqdata1 where ni(CUST_NAME='CUST_NAME_00108' and ACTIVE_EMUI_VERSION = " +
+        "'abc')")
+        .collect().toSeq
+
+    checkAnswer(sql(
+      "select * from uniqdata1 where CUST_NAME='CUST_NAME_00108' and ACTIVE_EMUI_VERSION = 'abc'"),
+      withoutIndex)
+
+    val df = sql(
+      "select * from uniqdata1 where CUST_NAME='CUST_NAME_00108' and ACTIVE_EMUI_VERSION = 'abc'")
+      .queryExecution
+      .sparkPlan
+    if (!isFilterPushedDownToSI(df)) {
+      assert(false)
+    } else {
+      assert(true)
+    }
+  }
+
+  test("Testing SI on partition table with OR condition") {
+    sql("drop index if exists indextable1 on uniqdata1")
+    sql("create index indextable1 on table uniqdata1 (DOB, CUST_NAME) AS 'carbondata'")
+    val withoutIndex =
+      sql(
+        "select * from uniqdata1 where ni(CUST_NAME='CUST_NAME_00108' OR ACTIVE_EMUI_VERSION = " +
+        "'abc')")
+        .collect().toSeq
+
+    checkAnswer(sql(
+      "select * from uniqdata1 where CUST_NAME='CUST_NAME_00108' OR ACTIVE_EMUI_VERSION = 'abc'"),
+      withoutIndex)
+
+    val df = sql(
+      "select * from uniqdata1 where CUST_NAME='CUST_NAME_00108' OR ACTIVE_EMUI_VERSION = 'abc'")
+      .queryExecution
+      .sparkPlan
+    if (!isFilterPushedDownToSI(df)) {
+      assert(true)
+    } else {
+      assert(false)
+    }
+  }
+
+  test("Testing SI on partition table with combination of OR OR") {
+    sql("drop index if exists indextable1 on uniqdata1")
+    sql("create index indextable1 on table uniqdata1 (DOB, CUST_NAME) AS 'carbondata'")
+    val withoutIndex =
+      sql(
+        "select * from uniqdata1 where ni(CUST_NAME='CUST_NAME_00108' OR CUST_ID='9000' OR " +
+        "ACTIVE_EMUI_VERSION = " +
+        "'abc')")
+        .collect().toSeq
+
+    checkAnswer(sql(
+      "select * from uniqdata1 where CUST_NAME='CUST_NAME_00108' OR CUST_ID='9000' OR " +
+      "ACTIVE_EMUI_VERSION = 'abc'"),
+      withoutIndex)
+
+    val df = sql(
+      "select * from uniqdata1 where CUST_NAME='CUST_NAME_00108' OR CUST_ID='9000' OR " +
+      "ACTIVE_EMUI_VERSION = 'abc'")
+      .queryExecution
+      .sparkPlan
+    if (!isFilterPushedDownToSI(df)) {
+      assert(true)
+    } else {
+      assert(false)
+    }
+  }
+
+  test("Testing SI on partition table with combination of OR AND") {
+    sql("drop index if exists indextable1 on uniqdata1")
+    sql("create index indextable1 on table uniqdata1 (DOB, CUST_NAME) AS 'carbondata'")
+    val withoutIndex =
+      sql(
+        "select * from uniqdata1 where ni(CUST_NAME='CUST_NAME_00108' OR CUST_ID='9000' AND " +
+        "ACTIVE_EMUI_VERSION = " +
+        "'abc')")
+        .collect().toSeq
+
+    checkAnswer(sql(
+      "select * from uniqdata1 where CUST_NAME='CUST_NAME_00108' OR CUST_ID='9000' AND " +
+      "ACTIVE_EMUI_VERSION = 'abc'"),
+      withoutIndex)
+
+    val df = sql(
+      "select * from uniqdata1 where CUST_NAME='CUST_NAME_00108' OR CUST_ID='9000' AND " +
+      "ACTIVE_EMUI_VERSION = 'abc'")
+      .queryExecution
+      .sparkPlan
+    if (!isFilterPushedDownToSI(df)) {
+      assert(true)
+    } else {
+      assert(false)
+    }
+  }
+
+  test("Testing SI on partition table with combination of AND OR") {
+    sql("drop index if exists indextable1 on uniqdata1")
+    sql("create index indextable1 on table uniqdata1 (DOB, CUST_NAME) AS 'carbondata'")
+    val withoutIndex =
+      sql(
+        "select * from uniqdata1 where ni(CUST_NAME='CUST_NAME_00108' AND CUST_ID='9000' OR " +
+        "ACTIVE_EMUI_VERSION = " +
+        "'abc')")
+        .collect().toSeq
+
+    checkAnswer(sql(
+      "select * from uniqdata1 where CUST_NAME='CUST_NAME_00108' AND CUST_ID='9000' OR " +
+      "ACTIVE_EMUI_VERSION = 'abc'"),
+      withoutIndex)
+
+    val df = sql(
+      "select * from uniqdata1 where CUST_NAME='CUST_NAME_00108' AND CUST_ID='9000' OR " +
+      "ACTIVE_EMUI_VERSION = 'abc'")
+      .queryExecution
+      .sparkPlan
+    if (!isFilterPushedDownToSI(df)) {
+      assert(true)
+    } else {
+      assert(false)
+    }
+  }
+
+  test("Testing SI on partition table with combination of AND AND") {
+    sql("drop index if exists indextable1 on uniqdata1")
+    sql("create index indextable1 on table uniqdata1 (DOB, CUST_NAME) AS 'carbondata'")
+    val withoutIndex =
+      sql(
+        "select * from uniqdata1 where ni(CUST_NAME='CUST_NAME_00108' AND CUST_ID='9000' AND " +
+        "ACTIVE_EMUI_VERSION = " +
+        "'abc')")
+        .collect().toSeq
+
+    checkAnswer(sql(
+      "select * from uniqdata1 where CUST_NAME='CUST_NAME_00108' AND CUST_ID='9000' AND " +
+      "ACTIVE_EMUI_VERSION = 'abc'"),
+      withoutIndex)
+
+    val df = sql(
+      "select * from uniqdata1 where CUST_NAME='CUST_NAME_00108' AND CUST_ID='9000' AND " +
+      "ACTIVE_EMUI_VERSION = 'abc'")
+      .queryExecution
+      .sparkPlan
+    if (!isFilterPushedDownToSI(df)) {
+      assert(false)
+    } else {
+      assert(true)
+    }
+  }
+
+  test("Testing SI on partition table with major compaction") {
+    sql("drop index if exists indextable1 on uniqdata1")
+    sql("create index indextable1 on table uniqdata1 (DOB, CUST_NAME) AS 'carbondata'")
+    val withoutIndex =
+      sql(
+        "select * from uniqdata1 where CUST_NAME='CUST_NAME_00108' and ACTIVE_EMUI_VERSION = " +
+        "'abc'")
+        .collect().toSeq
+
+    sql("alter table uniqdata1 compact 'major'")
+
+    checkAnswer(sql(
+      "select * from uniqdata1 where CUST_NAME='CUST_NAME_00108' and ACTIVE_EMUI_VERSION = 'abc'"),
+      withoutIndex)
+
+    val df = sql(
+      "select * from uniqdata1 where CUST_NAME='CUST_NAME_00108' and ACTIVE_EMUI_VERSION = 'abc'")
+      .queryExecution
+      .sparkPlan
+    if (!isFilterPushedDownToSI(df)) {
+      assert(false)
+    } else {
+      assert(true)
+    }
+  }
+
+  test("Testing SI on partition table with minor compaction") {
+    sql("drop index if exists indextable1 on uniqdata1")
+    sql("create index indextable1 on table uniqdata1 (DOB, CUST_NAME) AS 'carbondata'")
+
+    val withoutIndex =
+      sql(
+        "select * from uniqdata1 where CUST_NAME='CUST_NAME_00108' and ACTIVE_EMUI_VERSION = " +
+        "'abc'")
+        .collect().toSeq
+
+    sql("alter table uniqdata1 compact 'minor'")
+
+    checkAnswer(sql(
+      "select * from uniqdata1 where CUST_NAME='CUST_NAME_00108' and ACTIVE_EMUI_VERSION = 'abc'"),
+      withoutIndex)
+
+    val df = sql(
+      "select * from uniqdata1 where CUST_NAME='CUST_NAME_00108' and ACTIVE_EMUI_VERSION = 'abc'")
+      .queryExecution
+      .sparkPlan
+    if (!isFilterPushedDownToSI(df)) {
+      assert(false)
+    } else {
+      assert(true)
+    }
+  }
+
+  test("Testing SI on partition table with delete") {
+    sql("drop index if exists indextable1 on uniqdata1")
+    sql("create index indextable1 on table uniqdata1 (DOB, CUST_NAME) AS 'carbondata'")
+
+    checkAnswer(sql(
+      "select count(*) from uniqdata1 where CUST_NAME='CUST_NAME_00108' and ACTIVE_EMUI_VERSION =" +
+      " 'abc'"),
+      Seq(Row(4)))
+
+    sql("delete from uniqdata1 where CUST_NAME='CUST_NAME_00108'").show()
+
+    checkAnswer(sql(
+      "select count(*) from uniqdata1 where CUST_NAME='CUST_NAME_00108' and ACTIVE_EMUI_VERSION =" +
+      " 'abc'"),
+      Seq(Row(0)))
+
+    val df = sql(
+      "select * from uniqdata1 where CUST_NAME='CUST_NAME_00108' and ACTIVE_EMUI_VERSION = 'abc'")
+      .queryExecution
+      .sparkPlan
+    if (!isFilterPushedDownToSI(df)) {
+      assert(false)
+    } else {
+      assert(true)
+    }
+  }
+
+  test("Testing SI on partition table with update") {
+    sql("drop index if exists indextable1 on uniqdata1")
+    sql("create index indextable1 on table uniqdata1 (DOB, CUST_NAME) AS 'carbondata'")
+
+    checkAnswer(sql(
+      "select count(*) from uniqdata1 where CUST_ID='9000' and ACTIVE_EMUI_VERSION = 'abc'"),
+      Seq(Row(4)))
+    intercept[RuntimeException] {
+      sql("update uniqdata1 d set (d.CUST_ID) = ('8000')  where d.CUST_ID = '9000'").show()
+    }
+  }
+
+  test("Testing SI on partition table with rename") {
+    sql("drop index if exists indextable1 on uniqdata1")
+    sql("create index indextable1 on table uniqdata1 (DOB, CUST_NAME) AS 'carbondata'")
+
+    val withoutIndex =
+      sql(
+        "select * from uniqdata1 where CUST_NAME='CUST_NAME_00108' and ACTIVE_EMUI_VERSION = " +
+        "'abc'")
+        .collect().toSeq
+
+    sql("alter table uniqdata1 change CUST_NAME test string")
+
+    checkAnswer(sql(
+      "select * from uniqdata1 where test='CUST_NAME_00108' and ACTIVE_EMUI_VERSION = 'abc'"),
+      withoutIndex)
+
+    val df = sql(
+      "select * from uniqdata1 where test='CUST_NAME_00108' and ACTIVE_EMUI_VERSION = 'abc'")
+      .queryExecution
+      .sparkPlan
+    if (!isFilterPushedDownToSI(df)) {
+      assert(false)
+    } else {
+      assert(true)
+    }
+  }
+
+  override protected def afterAll(): Unit = {
+    sql("drop index if exists indextable1 on uniqdata1")
+    sql("drop table if exists uniqdata1")
+  }
+
+  /**
+   * Method to check whether the filter is push down to SI table or not
+   *
+   * @param sparkPlan
+   * @return
+   */
+  private def isFilterPushedDownToSI(sparkPlan: SparkPlan): Boolean = {
 
 Review comment:
   Better all test cases should be independent. TestSIWithPartition.scala specific to SI with partition.

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] akashrn5 commented on a change in pull request #3639: [CARBONDATA-3724] Secondary Index enable on partition Table

Posted by GitBox <gi...@apache.org>.
akashrn5 commented on a change in pull request #3639: [CARBONDATA-3724] Secondary Index enable on partition Table
URL: https://github.com/apache/carbondata/pull/3639#discussion_r386084106
 
 

 ##########
 File path: hadoop/src/main/java/org/apache/carbondata/hadoop/api/CarbonTableInputFormat.java
 ##########
 @@ -188,7 +189,10 @@
 
     List<Segment> segmentToAccess =
         getFilteredSegment(job, validAndInProgressSegments, false, readCommittedScope);
-
+    String segmentFileName = job.getConfiguration().get("current.segmentfile");
 
 Review comment:
   what is current.segmentfile? can we add current.segmentfile as constant?

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] jackylk commented on a change in pull request #3639: [CARBONDATA-3724] Secondary Index enable on partition Table

Posted by GitBox <gi...@apache.org>.
jackylk commented on a change in pull request #3639: [CARBONDATA-3724] Secondary Index enable on partition Table
URL: https://github.com/apache/carbondata/pull/3639#discussion_r386006297
 
 

 ##########
 File path: core/src/main/java/org/apache/carbondata/core/scan/scanner/impl/BlockletFullScanner.java
 ##########
 @@ -83,9 +82,8 @@ public BlockletScannedResult scanBlocklet(
         .get(QueryStatisticsConstants.TOTAL_PAGE_SCANNED);
     totalPagesScanned.addCountStatistic(QueryStatisticsConstants.TOTAL_PAGE_SCANNED,
         totalPagesScanned.getCount() + rawBlockletColumnChunks.getDataBlock().numberOfPages());
-    String blockletId = blockExecutionInfo.getBlockIdString() + CarbonCommonConstants.FILE_SEPARATOR
-        + rawBlockletColumnChunks.getDataBlock().blockletIndex();
-    scannedResult.setBlockletId(blockletId);
+    scannedResult.setBlockletId(blockExecutionInfo.getBlockIdString(),
+        "" + rawBlockletColumnChunks.getDataBlock().blockletIndex());
 
 Review comment:
   use String.valueOf 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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] jackylk commented on a change in pull request #3639: [CARBONDATA-3724] Secondary Index enable on partition Table

Posted by GitBox <gi...@apache.org>.
jackylk commented on a change in pull request #3639: [CARBONDATA-3724] Secondary Index enable on partition Table
URL: https://github.com/apache/carbondata/pull/3639#discussion_r386006406
 
 

 ##########
 File path: integration/spark/src/main/scala/org/apache/spark/sql/secondaryindex/rdd/CarbonSecondaryIndexRDD.scala
 ##########
 @@ -190,24 +191,23 @@ class CarbonSecondaryIndexRDD[K, V](
     val startTime = System.currentTimeMillis()
     val absoluteTableIdentifier: AbsoluteTableIdentifier = AbsoluteTableIdentifier.from(
       carbonStoreLocation, databaseName, factTableName, tableId)
-    val updateStatusManager: SegmentUpdateStatusManager = new SegmentUpdateStatusManager(
-      carbonLoadModel.getCarbonDataLoadSchema.getCarbonTable)
     val jobConf: JobConf = new JobConf(hadoopConf)
     SparkHadoopUtil.get.addCredentials(jobConf)
     val job: Job = new Job(jobConf)
-    val format = CarbonInputFormatUtil.createCarbonInputFormat(absoluteTableIdentifier, job)
+
+    if (carbonLoadModel.getCarbonDataLoadSchema.getCarbonTable.isHivePartitionTable) {
+      job.getConfiguration
 
 Review comment:
   Why do you set this config, please write 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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] Indhumathi27 commented on a change in pull request #3639: [CARBONDATA-3724] Secondary Index enable on partition Table

Posted by GitBox <gi...@apache.org>.
Indhumathi27 commented on a change in pull request #3639: [CARBONDATA-3724] Secondary Index enable on partition Table
URL: https://github.com/apache/carbondata/pull/3639#discussion_r387472308
 
 

 ##########
 File path: hadoop/src/main/java/org/apache/carbondata/hadoop/api/CarbonTableInputFormat.java
 ##########
 @@ -188,7 +189,10 @@
 
     List<Segment> segmentToAccess =
         getFilteredSegment(job, validAndInProgressSegments, false, readCommittedScope);
-
+    String segmentFileName = job.getConfiguration().get(CarbonCommonConstants.CURRENT_SEGMENTFILE);
+    if (segmentFileName != null) {
+      segmentToAccess.get(0).setSegmentFileName(segmentFileName + CarbonTablePath.SEGMENT_EXT);
 
 Review comment:
   why you are setting segmentFileName to only first segment of segmentToAccess list? Can you please add comments

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] ajantha-bhat commented on issue #3639: [WIP] Secondary Index enable on partition Table

Posted by GitBox <gi...@apache.org>.
ajantha-bhat commented on issue #3639: [WIP] Secondary Index enable on partition Table
URL: https://github.com/apache/carbondata/pull/3639#issuecomment-590949550
 
 
   add to whitelist

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] akashrn5 commented on a change in pull request #3639: [CARBONDATA-3724] Secondary Index enable on partition Table

Posted by GitBox <gi...@apache.org>.
akashrn5 commented on a change in pull request #3639: [CARBONDATA-3724] Secondary Index enable on partition Table
URL: https://github.com/apache/carbondata/pull/3639#discussion_r386084370
 
 

 ##########
 File path: integration/spark/src/test/scala/org/apache/carbondata/spark/testsuite/iud/DeleteCarbonTableTestCase.scala
 ##########
 @@ -119,18 +122,22 @@ class DeleteCarbonTableTestCase extends QueryTest with BeforeAndAfterAll {
 
   test("partition delete data from carbon table with alias [where clause ]") {
     sql("drop table if exists iud_db.dest")
-    sql("""create table iud_db.dest (c1 string,c2 int,c5 string) PARTITIONED BY(c3 string) STORED AS carbondata""")
+    sql(
+      """create table iud_db.dest (c1 string,c2 int,c5 string) PARTITIONED BY(c3 string) STORED AS
+        |carbondata""".stripMargin)
     sql(s"""LOAD DATA LOCAL INPATH '$resourcesPath/IUD/dest.csv' INTO table iud_db.dest""")
     sql("""delete from iud_db.dest d where d.c1 = 'a'""").show
     checkAnswer(
       sql("""select c2 from iud_db.dest"""),
-      Seq(Row(2), Row(3),Row(4), Row(5))
+      Seq(Row(2), Row(3), Row(4), Row(5))
 
 Review comment:
   revert

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] Indhumathi27 commented on a change in pull request #3639: [CARBONDATA-3724] Secondary Index enable on partition Table

Posted by GitBox <gi...@apache.org>.
Indhumathi27 commented on a change in pull request #3639: [CARBONDATA-3724] Secondary Index enable on partition Table
URL: https://github.com/apache/carbondata/pull/3639#discussion_r387614485
 
 

 ##########
 File path: core/src/main/java/org/apache/carbondata/core/indexstore/blockletindex/BlockDataMap.java
 ##########
 @@ -788,7 +788,13 @@ private boolean addBlockBasedOnMinMaxValue(FilterExecuter filterExecuter, byte[]
       byte[][] minValue, boolean[] minMaxFlag, String filePath, int blockletId) {
     BitSet bitSet = null;
     if (filterExecuter instanceof ImplicitColumnFilterExecutor) {
-      String uniqueBlockPath = filePath.substring(filePath.lastIndexOf("/Part") + 1);
+      String uniqueBlockPath;
+      if (segmentPropertiesWrapper.getCarbonTable().isHivePartitionTable()) {
 
 Review comment:
   ok

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] Indhumathi27 commented on a change in pull request #3639: [CARBONDATA-3724] Secondary Index enable on partition Table

Posted by GitBox <gi...@apache.org>.
Indhumathi27 commented on a change in pull request #3639: [CARBONDATA-3724] Secondary Index enable on partition Table
URL: https://github.com/apache/carbondata/pull/3639#discussion_r387477530
 
 

 ##########
 File path: core/src/main/java/org/apache/carbondata/core/indexstore/blockletindex/BlockDataMap.java
 ##########
 @@ -788,7 +788,13 @@ private boolean addBlockBasedOnMinMaxValue(FilterExecuter filterExecuter, byte[]
       byte[][] minValue, boolean[] minMaxFlag, String filePath, int blockletId) {
     BitSet bitSet = null;
     if (filterExecuter instanceof ImplicitColumnFilterExecutor) {
-      String uniqueBlockPath = filePath.substring(filePath.lastIndexOf("/Part") + 1);
+      String uniqueBlockPath;
+      if (segmentPropertiesWrapper.getCarbonTable().isHivePartitionTable()) {
 
 Review comment:
   In init method, blockletDataMapInfo already has carbonTable. Can use it, instead of getting from segmentPropertiesWrapper and remove getCarbonTable method from SegmentPropertiesWrapper

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] maheshrajus commented on a change in pull request #3639: [CARBONDATA-3724] Secondary Index enable on partition Table

Posted by GitBox <gi...@apache.org>.
maheshrajus commented on a change in pull request #3639: [CARBONDATA-3724] Secondary Index enable on partition Table
URL: https://github.com/apache/carbondata/pull/3639#discussion_r386896310
 
 

 ##########
 File path: core/src/main/java/org/apache/carbondata/core/util/CarbonUtil.java
 ##########
 @@ -2779,10 +2797,18 @@ public static String getBlockId(AbsoluteTableIdentifier identifier, String fileP
         } else {
           partitionDir = "";
         }
-        // Replace / with # on partition director to support multi level partitioning. And access
-        // them all as a single entity.
-        blockId = partitionDir.replace("/", "#") + CarbonCommonConstants.FILE_SEPARATOR
-            + segmentId + CarbonCommonConstants.FILE_SEPARATOR + blockName;
+        if (isPartitionTable) {
+          blockId =
+              partitionDir.replace("/", "#")
 
 Review comment:
   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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] maheshrajus commented on a change in pull request #3639: [CARBONDATA-3724] Secondary Index enable on partition Table

Posted by GitBox <gi...@apache.org>.
maheshrajus commented on a change in pull request #3639: [CARBONDATA-3724] Secondary Index enable on partition Table
URL: https://github.com/apache/carbondata/pull/3639#discussion_r387522724
 
 

 ##########
 File path: core/src/main/java/org/apache/carbondata/core/mutate/CarbonUpdateUtil.java
 ##########
 @@ -58,6 +58,14 @@
   private static final Logger LOGGER =
           LogServiceFactory.getLogService(CarbonUpdateUtil.class.getName());
 
+  /**
+   * returns required filed from tuple id
+   *
+   */
+  public static String getRequiredFieldFromTID(String Tid, int index) {
+    return Tid.split("/")[index];
 
 Review comment:
   OK

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] CarbonDataQA1 commented on issue #3639: [CARBONDATA-3724] Secondary Index enable on partition Table

Posted by GitBox <gi...@apache.org>.
CarbonDataQA1 commented on issue #3639: [CARBONDATA-3724] Secondary Index enable on partition Table
URL: https://github.com/apache/carbondata/pull/3639#issuecomment-611922578
 
 
   Build Success with Spark 2.3.4, Please check CI http://121.244.95.60:12545/job/ApacheCarbonPRBuilder2.3/2704/
   

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] maheshrajus commented on a change in pull request #3639: [CARBONDATA-3724] Secondary Index enable on partition Table

Posted by GitBox <gi...@apache.org>.
maheshrajus commented on a change in pull request #3639: [CARBONDATA-3724] Secondary Index enable on partition Table
URL: https://github.com/apache/carbondata/pull/3639#discussion_r387450095
 
 

 ##########
 File path: integration/spark/src/main/scala/org/apache/spark/sql/secondaryindex/rdd/CarbonSecondaryIndexRDD.scala
 ##########
 @@ -190,24 +191,23 @@ class CarbonSecondaryIndexRDD[K, V](
     val startTime = System.currentTimeMillis()
     val absoluteTableIdentifier: AbsoluteTableIdentifier = AbsoluteTableIdentifier.from(
       carbonStoreLocation, databaseName, factTableName, tableId)
-    val updateStatusManager: SegmentUpdateStatusManager = new SegmentUpdateStatusManager(
-      carbonLoadModel.getCarbonDataLoadSchema.getCarbonTable)
     val jobConf: JobConf = new JobConf(hadoopConf)
     SparkHadoopUtil.get.addCredentials(jobConf)
     val job: Job = new Job(jobConf)
-    val format = CarbonInputFormatUtil.createCarbonInputFormat(absoluteTableIdentifier, job)
+
+    if (carbonLoadModel.getCarbonDataLoadSchema.getCarbonTable.isHivePartitionTable) {
+      job.getConfiguration
 
 Review comment:
   OK

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] maheshrajus commented on a change in pull request #3639: [CARBONDATA-3724] Secondary Index enable on partition Table

Posted by GitBox <gi...@apache.org>.
maheshrajus commented on a change in pull request #3639: [CARBONDATA-3724] Secondary Index enable on partition Table
URL: https://github.com/apache/carbondata/pull/3639#discussion_r386975490
 
 

 ##########
 File path: core/src/main/java/org/apache/carbondata/core/mutate/data/BlockMappingVO.java
 ##########
 @@ -30,6 +30,8 @@
 
   private Map<String, RowCountDetailsVO> completeBlockRowDetailVO;
 
+  private Map<String, String> blockToSegmentMapping;
 
 Review comment:
   This map will help us to finding the segment id from the block path. I will add proper comments to this map in code. Can you suggest me any alternative to get the segment Id, 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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] maheshrajus commented on a change in pull request #3639: [CARBONDATA-3724] Secondary Index enable on partition Table

Posted by GitBox <gi...@apache.org>.
maheshrajus commented on a change in pull request #3639: [CARBONDATA-3724] Secondary Index enable on partition Table
URL: https://github.com/apache/carbondata/pull/3639#discussion_r386894958
 
 

 ##########
 File path: core/src/main/java/org/apache/carbondata/core/util/CarbonUtil.java
 ##########
 @@ -2756,11 +2756,29 @@ private static long getMaxOfBlockAndFileSize(long blockSize, long fileSize) {
    * @param identifier
    * @param filePath
    * @param segmentId
+   * @param isTransactionalTable
    * @param isStandardTable
    * @return
    */
   public static String getBlockId(AbsoluteTableIdentifier identifier, String filePath,
       String segmentId, boolean isTransactionalTable, boolean isStandardTable) {
+    return getBlockId(identifier, filePath, segmentId, isTransactionalTable, isStandardTable,
+        false);
+  }
+
+  /**
+   * Generate the blockid as per the block path
+   *
+   * @param identifier
 
 Review comment:
   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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] maheshrajus commented on a change in pull request #3639: [CARBONDATA-3724] Secondary Index enable on partition Table

Posted by GitBox <gi...@apache.org>.
maheshrajus commented on a change in pull request #3639: [CARBONDATA-3724] Secondary Index enable on partition Table
URL: https://github.com/apache/carbondata/pull/3639#discussion_r387530524
 
 

 ##########
 File path: core/src/main/java/org/apache/carbondata/core/scan/scanner/impl/BlockletFilterScanner.java
 ##########
 @@ -203,8 +203,8 @@ private BlockletScannedResult executeFilter(RawBlockletColumnChunks rawBlockletC
     BlockletScannedResult scannedResult =
         new FilterQueryScannedResult(blockExecutionInfo, queryStatisticsModel);
     scannedResult.setBlockletId(
-        blockExecutionInfo.getBlockIdString() + CarbonCommonConstants.FILE_SEPARATOR +
-            rawBlockletColumnChunks.getDataBlock().blockletIndex());
+        blockExecutionInfo.getBlockIdString(),
+            "" + rawBlockletColumnChunks.getDataBlock().blockletIndex());
 
 Review comment:
   OK

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] akashrn5 commented on a change in pull request #3639: [CARBONDATA-3724] Secondary Index enable on partition Table

Posted by GitBox <gi...@apache.org>.
akashrn5 commented on a change in pull request #3639: [CARBONDATA-3724] Secondary Index enable on partition Table
URL: https://github.com/apache/carbondata/pull/3639#discussion_r386084021
 
 

 ##########
 File path: core/src/main/java/org/apache/carbondata/core/util/CarbonUtil.java
 ##########
 @@ -2756,11 +2756,29 @@ private static long getMaxOfBlockAndFileSize(long blockSize, long fileSize) {
    * @param identifier
    * @param filePath
    * @param segmentId
+   * @param isTransactionalTable
    * @param isStandardTable
    * @return
    */
   public static String getBlockId(AbsoluteTableIdentifier identifier, String filePath,
       String segmentId, boolean isTransactionalTable, boolean isStandardTable) {
+    return getBlockId(identifier, filePath, segmentId, isTransactionalTable, isStandardTable,
+        false);
+  }
+
+  /**
+   * Generate the blockid as per the block path
+   *
+   * @param identifier
 
 Review comment:
   remove @param from comment, i not explained and straight forward

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] CarbonDataQA1 commented on issue #3639: [WIP] Secondary Index enable on partition Table

Posted by GitBox <gi...@apache.org>.
CarbonDataQA1 commented on issue #3639: [WIP] Secondary Index enable on partition Table
URL: https://github.com/apache/carbondata/pull/3639#issuecomment-591560386
 
 
   Build Failed  with Spark 2.3.4, Please check CI http://121.244.95.60:12545/job/ApacheCarbonPRBuilder2.3/2202/
   

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] CarbonDataQA1 commented on issue #3639: [CARBONDATA-3724] Secondary Index enable on partition Table

Posted by GitBox <gi...@apache.org>.
CarbonDataQA1 commented on issue #3639: [CARBONDATA-3724] Secondary Index enable on partition Table
URL: https://github.com/apache/carbondata/pull/3639#issuecomment-594513695
 
 
   Build Failed  with Spark 2.4.4, Please check CI http://121.244.95.60:12545/job/ApacheCarbon_PR_Builder_2.4.4/611/
   

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] CarbonDataQA1 commented on issue #3639: [CARBONDATA-3724] Secondary Index enable on partition Table

Posted by GitBox <gi...@apache.org>.
CarbonDataQA1 commented on issue #3639: [CARBONDATA-3724] Secondary Index enable on partition Table
URL: https://github.com/apache/carbondata/pull/3639#issuecomment-593947932
 
 
   Build Success with Spark 2.4.4, Please check CI http://121.244.95.60:12545/job/ApacheCarbon_PR_Builder_2.4.4/584/
   

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] Indhumathi27 commented on a change in pull request #3639: [CARBONDATA-3724] Secondary Index enable on partition Table

Posted by GitBox <gi...@apache.org>.
Indhumathi27 commented on a change in pull request #3639: [CARBONDATA-3724] Secondary Index enable on partition Table
URL: https://github.com/apache/carbondata/pull/3639#discussion_r387548712
 
 

 ##########
 File path: index/secondary-index/src/test/scala/org/apache/carbondata/spark/testsuite/secondaryindex/TestSIWithPartition.scala
 ##########
 @@ -0,0 +1,379 @@
+/*
+ * 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.execution.SparkPlan
+import org.apache.spark.sql.secondaryindex.joins.BroadCastSIFilterPushJoin
+import org.apache.spark.sql.test.util.QueryTest
+import org.scalatest.{BeforeAndAfterAll, Ignore}
+
+class TestSIWithPartition extends QueryTest with BeforeAndAfterAll {
+
+  override protected def beforeAll(): Unit = {
+    sql("drop table if exists uniqdata1")
+    sql(
+      "CREATE TABLE uniqdata1 (CUST_ID INT,CUST_NAME STRING,DOB timestamp,DOJ timestamp," +
+      "BIGINT_COLUMN1 bigint,BIGINT_COLUMN2 bigint,DECIMAL_COLUMN1 DECIMAL(30, 10)," +
+      "DECIMAL_COLUMN2 DECIMAL(36, 10),Double_COLUMN1 double, Double_COLUMN2 double," +
+      "INTEGER_COLUMN1 int) PARTITIONED BY(ACTIVE_EMUI_VERSION string) STORED AS carbondata " +
+      "TBLPROPERTIES('TABLE_BLOCKSIZE'='256 MB')")
+    sql(s"LOAD DATA LOCAL INPATH '$resourcesPath/restructure/data_2000.csv' INTO " +
+        "TABLE uniqdata1 partition(ACTIVE_EMUI_VERSION='abc') OPTIONS('DELIMITER'=',', " +
+        "'BAD_RECORDS_LOGGER_ENABLE'='FALSE', 'BAD_RECORDS_ACTION'='FORCE','FILEHEADER'='CUST_ID," +
+        "CUST_NAME,ACTIVE_EMUI_VERSION,DOB,DOJ,BIGINT_COLUMN1,BIGINT_COLUMN2,DECIMAL_COLUMN1," +
+        "DECIMAL_COLUMN2,Double_COLUMN1,Double_COLUMN2,INTEGER_COLUMN1')")
+    sql(s"LOAD DATA LOCAL INPATH '$resourcesPath/restructure/data_2000.csv' INTO " +
+        "TABLE uniqdata1 partition(ACTIVE_EMUI_VERSION='abc') OPTIONS('DELIMITER'=',', " +
+        "'BAD_RECORDS_LOGGER_ENABLE'='FALSE', 'BAD_RECORDS_ACTION'='FORCE','FILEHEADER'='CUST_ID," +
+        "CUST_NAME,ACTIVE_EMUI_VERSION,DOB,DOJ,BIGINT_COLUMN1,BIGINT_COLUMN2,DECIMAL_COLUMN1," +
+        "DECIMAL_COLUMN2,Double_COLUMN1,Double_COLUMN2,INTEGER_COLUMN1')")
+    sql(s"LOAD DATA LOCAL INPATH '$resourcesPath/restructure/data_2000.csv' INTO " +
+        "TABLE uniqdata1 partition(ACTIVE_EMUI_VERSION='abc') OPTIONS('DELIMITER'=',', " +
+        "'BAD_RECORDS_LOGGER_ENABLE'='FALSE', 'BAD_RECORDS_ACTION'='FORCE','FILEHEADER'='CUST_ID," +
+        "CUST_NAME,ACTIVE_EMUI_VERSION,DOB,DOJ,BIGINT_COLUMN1,BIGINT_COLUMN2,DECIMAL_COLUMN1," +
+        "DECIMAL_COLUMN2,Double_COLUMN1,Double_COLUMN2,INTEGER_COLUMN1')")
+    sql(s"LOAD DATA LOCAL INPATH '$resourcesPath/restructure/data_2000.csv' INTO " +
+        "TABLE uniqdata1 partition(ACTIVE_EMUI_VERSION='abc') OPTIONS('DELIMITER'=',', " +
+        "'BAD_RECORDS_LOGGER_ENABLE'='FALSE', 'BAD_RECORDS_ACTION'='FORCE','FILEHEADER'='CUST_ID," +
+        "CUST_NAME,ACTIVE_EMUI_VERSION,DOB,DOJ,BIGINT_COLUMN1,BIGINT_COLUMN2,DECIMAL_COLUMN1," +
+        "DECIMAL_COLUMN2,Double_COLUMN1,Double_COLUMN2,INTEGER_COLUMN1')")
+  }
+
+  test("Testing SI on partition column") {
+    sql("drop index if exists indextable1 on uniqdata1")
+    intercept[UnsupportedOperationException] {
+      sql("create index indextable1 on table uniqdata1 (ACTIVE_EMUI_VERSION) AS 'carbondata'")
+    }
+  }
+
+  test("Testing SI without partition column") {
+    sql("drop index if exists indextable1 on uniqdata1")
+    sql("create index indextable1 on table uniqdata1 (DOB, CUST_NAME) AS 'carbondata'")
+    val withoutIndex =
+      sql("select * from uniqdata1 where ni(CUST_NAME='CUST_NAME_00108')")
+        .collect().toSeq
+
+    checkAnswer(sql("select * from uniqdata1 where CUST_NAME='CUST_NAME_00108'"),
+      withoutIndex)
+
+    val df = sql("select * from uniqdata1 where CUST_NAME='CUST_NAME_00108'")
+      .queryExecution
+      .sparkPlan
+    if (!isFilterPushedDownToSI(df)) {
+      assert(false)
+    } else {
+      assert(true)
+    }
+  }
+
+  test("Testing SI with partition column[where clause]") {
+    sql("drop index if exists indextable1 on uniqdata1")
+    sql("create index indextable1 on table uniqdata1 (DOB, CUST_NAME) AS 'carbondata'")
+    val withoutIndex =
+      sql(
+        "select * from uniqdata1 where ni(CUST_NAME='CUST_NAME_00108' and ACTIVE_EMUI_VERSION = " +
+        "'abc')")
+        .collect().toSeq
+
+    checkAnswer(sql(
+      "select * from uniqdata1 where CUST_NAME='CUST_NAME_00108' and ACTIVE_EMUI_VERSION = 'abc'"),
+      withoutIndex)
+
+    val df = sql(
+      "select * from uniqdata1 where CUST_NAME='CUST_NAME_00108' and ACTIVE_EMUI_VERSION = 'abc'")
+      .queryExecution
+      .sparkPlan
+    if (!isFilterPushedDownToSI(df)) {
+      assert(false)
+    } else {
+      assert(true)
+    }
+  }
+
+  test("Testing SI on partition table with OR condition") {
+    sql("drop index if exists indextable1 on uniqdata1")
+    sql("create index indextable1 on table uniqdata1 (DOB, CUST_NAME) AS 'carbondata'")
+    val withoutIndex =
+      sql(
+        "select * from uniqdata1 where ni(CUST_NAME='CUST_NAME_00108' OR ACTIVE_EMUI_VERSION = " +
+        "'abc')")
+        .collect().toSeq
+
+    checkAnswer(sql(
+      "select * from uniqdata1 where CUST_NAME='CUST_NAME_00108' OR ACTIVE_EMUI_VERSION = 'abc'"),
+      withoutIndex)
+
+    val df = sql(
+      "select * from uniqdata1 where CUST_NAME='CUST_NAME_00108' OR ACTIVE_EMUI_VERSION = 'abc'")
+      .queryExecution
+      .sparkPlan
+    if (!isFilterPushedDownToSI(df)) {
+      assert(true)
+    } else {
+      assert(false)
+    }
+  }
+
+  test("Testing SI on partition table with combination of OR OR") {
+    sql("drop index if exists indextable1 on uniqdata1")
+    sql("create index indextable1 on table uniqdata1 (DOB, CUST_NAME) AS 'carbondata'")
+    val withoutIndex =
+      sql(
+        "select * from uniqdata1 where ni(CUST_NAME='CUST_NAME_00108' OR CUST_ID='9000' OR " +
+        "ACTIVE_EMUI_VERSION = " +
+        "'abc')")
+        .collect().toSeq
+
+    checkAnswer(sql(
+      "select * from uniqdata1 where CUST_NAME='CUST_NAME_00108' OR CUST_ID='9000' OR " +
+      "ACTIVE_EMUI_VERSION = 'abc'"),
+      withoutIndex)
+
+    val df = sql(
+      "select * from uniqdata1 where CUST_NAME='CUST_NAME_00108' OR CUST_ID='9000' OR " +
+      "ACTIVE_EMUI_VERSION = 'abc'")
+      .queryExecution
+      .sparkPlan
+    if (!isFilterPushedDownToSI(df)) {
+      assert(true)
+    } else {
+      assert(false)
+    }
+  }
+
+  test("Testing SI on partition table with combination of OR AND") {
+    sql("drop index if exists indextable1 on uniqdata1")
+    sql("create index indextable1 on table uniqdata1 (DOB, CUST_NAME) AS 'carbondata'")
+    val withoutIndex =
+      sql(
+        "select * from uniqdata1 where ni(CUST_NAME='CUST_NAME_00108' OR CUST_ID='9000' AND " +
+        "ACTIVE_EMUI_VERSION = " +
+        "'abc')")
+        .collect().toSeq
+
+    checkAnswer(sql(
+      "select * from uniqdata1 where CUST_NAME='CUST_NAME_00108' OR CUST_ID='9000' AND " +
+      "ACTIVE_EMUI_VERSION = 'abc'"),
+      withoutIndex)
+
+    val df = sql(
+      "select * from uniqdata1 where CUST_NAME='CUST_NAME_00108' OR CUST_ID='9000' AND " +
+      "ACTIVE_EMUI_VERSION = 'abc'")
+      .queryExecution
+      .sparkPlan
+    if (!isFilterPushedDownToSI(df)) {
+      assert(true)
+    } else {
+      assert(false)
+    }
+  }
+
+  test("Testing SI on partition table with combination of AND OR") {
+    sql("drop index if exists indextable1 on uniqdata1")
+    sql("create index indextable1 on table uniqdata1 (DOB, CUST_NAME) AS 'carbondata'")
+    val withoutIndex =
+      sql(
+        "select * from uniqdata1 where ni(CUST_NAME='CUST_NAME_00108' AND CUST_ID='9000' OR " +
+        "ACTIVE_EMUI_VERSION = " +
+        "'abc')")
+        .collect().toSeq
+
+    checkAnswer(sql(
+      "select * from uniqdata1 where CUST_NAME='CUST_NAME_00108' AND CUST_ID='9000' OR " +
+      "ACTIVE_EMUI_VERSION = 'abc'"),
+      withoutIndex)
+
+    val df = sql(
+      "select * from uniqdata1 where CUST_NAME='CUST_NAME_00108' AND CUST_ID='9000' OR " +
+      "ACTIVE_EMUI_VERSION = 'abc'")
+      .queryExecution
+      .sparkPlan
+    if (!isFilterPushedDownToSI(df)) {
+      assert(true)
+    } else {
+      assert(false)
+    }
+  }
+
+  test("Testing SI on partition table with combination of AND AND") {
+    sql("drop index if exists indextable1 on uniqdata1")
+    sql("create index indextable1 on table uniqdata1 (DOB, CUST_NAME) AS 'carbondata'")
+    val withoutIndex =
+      sql(
+        "select * from uniqdata1 where ni(CUST_NAME='CUST_NAME_00108' AND CUST_ID='9000' AND " +
+        "ACTIVE_EMUI_VERSION = " +
+        "'abc')")
+        .collect().toSeq
+
+    checkAnswer(sql(
+      "select * from uniqdata1 where CUST_NAME='CUST_NAME_00108' AND CUST_ID='9000' AND " +
+      "ACTIVE_EMUI_VERSION = 'abc'"),
+      withoutIndex)
+
+    val df = sql(
+      "select * from uniqdata1 where CUST_NAME='CUST_NAME_00108' AND CUST_ID='9000' AND " +
+      "ACTIVE_EMUI_VERSION = 'abc'")
+      .queryExecution
+      .sparkPlan
+    if (!isFilterPushedDownToSI(df)) {
+      assert(false)
+    } else {
+      assert(true)
+    }
+  }
+
+  test("Testing SI on partition table with major compaction") {
+    sql("drop index if exists indextable1 on uniqdata1")
+    sql("create index indextable1 on table uniqdata1 (DOB, CUST_NAME) AS 'carbondata'")
+    val withoutIndex =
+      sql(
+        "select * from uniqdata1 where CUST_NAME='CUST_NAME_00108' and ACTIVE_EMUI_VERSION = " +
+        "'abc'")
+        .collect().toSeq
+
+    sql("alter table uniqdata1 compact 'major'")
+
+    checkAnswer(sql(
+      "select * from uniqdata1 where CUST_NAME='CUST_NAME_00108' and ACTIVE_EMUI_VERSION = 'abc'"),
+      withoutIndex)
+
+    val df = sql(
+      "select * from uniqdata1 where CUST_NAME='CUST_NAME_00108' and ACTIVE_EMUI_VERSION = 'abc'")
+      .queryExecution
+      .sparkPlan
+    if (!isFilterPushedDownToSI(df)) {
+      assert(false)
+    } else {
+      assert(true)
+    }
+  }
+
+  test("Testing SI on partition table with minor compaction") {
+    sql("drop index if exists indextable1 on uniqdata1")
+    sql("create index indextable1 on table uniqdata1 (DOB, CUST_NAME) AS 'carbondata'")
+
+    val withoutIndex =
+      sql(
+        "select * from uniqdata1 where CUST_NAME='CUST_NAME_00108' and ACTIVE_EMUI_VERSION = " +
+        "'abc'")
+        .collect().toSeq
+
+    sql("alter table uniqdata1 compact 'minor'")
+
+    checkAnswer(sql(
+      "select * from uniqdata1 where CUST_NAME='CUST_NAME_00108' and ACTIVE_EMUI_VERSION = 'abc'"),
+      withoutIndex)
+
+    val df = sql(
+      "select * from uniqdata1 where CUST_NAME='CUST_NAME_00108' and ACTIVE_EMUI_VERSION = 'abc'")
+      .queryExecution
+      .sparkPlan
+    if (!isFilterPushedDownToSI(df)) {
+      assert(false)
+    } else {
+      assert(true)
+    }
+  }
+
+  test("Testing SI on partition table with delete") {
+    sql("drop index if exists indextable1 on uniqdata1")
+    sql("create index indextable1 on table uniqdata1 (DOB, CUST_NAME) AS 'carbondata'")
+
+    checkAnswer(sql(
+      "select count(*) from uniqdata1 where CUST_NAME='CUST_NAME_00108' and ACTIVE_EMUI_VERSION =" +
+      " 'abc'"),
+      Seq(Row(4)))
+
+    sql("delete from uniqdata1 where CUST_NAME='CUST_NAME_00108'").show()
+
+    checkAnswer(sql(
+      "select count(*) from uniqdata1 where CUST_NAME='CUST_NAME_00108' and ACTIVE_EMUI_VERSION =" +
+      " 'abc'"),
+      Seq(Row(0)))
+
+    val df = sql(
+      "select * from uniqdata1 where CUST_NAME='CUST_NAME_00108' and ACTIVE_EMUI_VERSION = 'abc'")
+      .queryExecution
+      .sparkPlan
+    if (!isFilterPushedDownToSI(df)) {
+      assert(false)
+    } else {
+      assert(true)
+    }
+  }
+
+  test("Testing SI on partition table with update") {
+    sql("drop index if exists indextable1 on uniqdata1")
+    sql("create index indextable1 on table uniqdata1 (DOB, CUST_NAME) AS 'carbondata'")
+
+    checkAnswer(sql(
+      "select count(*) from uniqdata1 where CUST_ID='9000' and ACTIVE_EMUI_VERSION = 'abc'"),
+      Seq(Row(4)))
+    intercept[RuntimeException] {
+      sql("update uniqdata1 d set (d.CUST_ID) = ('8000')  where d.CUST_ID = '9000'").show()
+    }
+  }
+
+  test("Testing SI on partition table with rename") {
+    sql("drop index if exists indextable1 on uniqdata1")
+    sql("create index indextable1 on table uniqdata1 (DOB, CUST_NAME) AS 'carbondata'")
+
+    val withoutIndex =
+      sql(
+        "select * from uniqdata1 where CUST_NAME='CUST_NAME_00108' and ACTIVE_EMUI_VERSION = " +
+        "'abc'")
+        .collect().toSeq
+
+    sql("alter table uniqdata1 change CUST_NAME test string")
+
+    checkAnswer(sql(
+      "select * from uniqdata1 where test='CUST_NAME_00108' and ACTIVE_EMUI_VERSION = 'abc'"),
+      withoutIndex)
+
+    val df = sql(
+      "select * from uniqdata1 where test='CUST_NAME_00108' and ACTIVE_EMUI_VERSION = 'abc'")
+      .queryExecution
+      .sparkPlan
+    if (!isFilterPushedDownToSI(df)) {
+      assert(false)
+    } else {
+      assert(true)
+    }
+  }
+
+  override protected def afterAll(): Unit = {
+    sql("drop index if exists indextable1 on uniqdata1")
+    sql("drop table if exists uniqdata1")
+  }
+
+  /**
+   * Method to check whether the filter is push down to SI table or not
+   *
+   * @param sparkPlan
+   * @return
+   */
+  private def isFilterPushedDownToSI(sparkPlan: SparkPlan): Boolean = {
 
 Review comment:
   But isFilterPushedDownToSI is common for all testClasses. Can move this method to a Util and call from all Testclasses

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] maheshrajus commented on issue #3639: [WIP] Secondary Index enable on partition Table

Posted by GitBox <gi...@apache.org>.
maheshrajus commented on issue #3639: [WIP] Secondary Index enable on partition Table
URL: https://github.com/apache/carbondata/pull/3639#issuecomment-591868800
 
 
   retest this please

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] akashrn5 commented on a change in pull request #3639: [CARBONDATA-3724] Secondary Index enable on partition Table

Posted by GitBox <gi...@apache.org>.
akashrn5 commented on a change in pull request #3639: [CARBONDATA-3724] Secondary Index enable on partition Table
URL: https://github.com/apache/carbondata/pull/3639#discussion_r386083977
 
 

 ##########
 File path: core/src/main/java/org/apache/carbondata/core/statusmanager/SegmentUpdateStatusManager.java
 ##########
 @@ -176,6 +176,10 @@ public SegmentUpdateDetails getDetailsForABlock(String key) {
 
   }
 
+  public Map<String, SegmentUpdateDetails> getBlockAndDetailsMap() {
 
 Review comment:
   please add comment what it returns and what it contains

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] akashrn5 commented on a change in pull request #3639: [CARBONDATA-3724] Secondary Index enable on partition Table

Posted by GitBox <gi...@apache.org>.
akashrn5 commented on a change in pull request #3639: [CARBONDATA-3724] Secondary Index enable on partition Table
URL: https://github.com/apache/carbondata/pull/3639#discussion_r386084358
 
 

 ##########
 File path: integration/spark/src/test/scala/org/apache/carbondata/spark/testsuite/iud/DeleteCarbonTableTestCase.scala
 ##########
 @@ -70,7 +73,7 @@ class DeleteCarbonTableTestCase extends QueryTest with BeforeAndAfterAll {
     sql("""delete from dest where c1 IN ('d', 'e')""").show
     checkAnswer(
       sql("""select c1 from dest"""),
-      Seq(Row("a"), Row("b"),Row("c"))
+      Seq(Row("a"), Row("b"), Row("c"))
 
 Review comment:
   revert

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] CarbonDataQA1 commented on issue #3639: [CARBONDATA-3724] Secondary Index enable on partition Table

Posted by GitBox <gi...@apache.org>.
CarbonDataQA1 commented on issue #3639: [CARBONDATA-3724] Secondary Index enable on partition Table
URL: https://github.com/apache/carbondata/pull/3639#issuecomment-608445468
 
 
   Build Success with Spark 2.3.4, Please check CI http://121.244.95.60:12545/job/ApacheCarbonPRBuilder2.3/2629/
   

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] CarbonDataQA1 commented on issue #3639: [WIP] Secondary Index enable on partition Table

Posted by GitBox <gi...@apache.org>.
CarbonDataQA1 commented on issue #3639: [WIP] Secondary Index enable on partition Table
URL: https://github.com/apache/carbondata/pull/3639#issuecomment-591950341
 
 
   Build Success with Spark 2.3.4, Please check CI http://121.244.95.60:12545/job/ApacheCarbonPRBuilder2.3/2210/
   

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] Indhumathi27 commented on a change in pull request #3639: [CARBONDATA-3724] Secondary Index enable on partition Table

Posted by GitBox <gi...@apache.org>.
Indhumathi27 commented on a change in pull request #3639: [CARBONDATA-3724] Secondary Index enable on partition Table
URL: https://github.com/apache/carbondata/pull/3639#discussion_r387584627
 
 

 ##########
 File path: core/src/main/java/org/apache/carbondata/core/indexstore/blockletindex/BlockDataMap.java
 ##########
 @@ -788,7 +788,13 @@ private boolean addBlockBasedOnMinMaxValue(FilterExecuter filterExecuter, byte[]
       byte[][] minValue, boolean[] minMaxFlag, String filePath, int blockletId) {
     BitSet bitSet = null;
     if (filterExecuter instanceof ImplicitColumnFilterExecutor) {
-      String uniqueBlockPath = filePath.substring(filePath.lastIndexOf("/Part") + 1);
+      String uniqueBlockPath;
+      if (segmentPropertiesWrapper.getCarbonTable().isHivePartitionTable()) {
 
 Review comment:
   Since you only need isHivePartitionTable and tablePath, can get from DataMapModel and assign to class-level variables

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] CarbonDataQA1 commented on issue #3639: [CARBONDATA-3724] Secondary Index enable on partition Table

Posted by GitBox <gi...@apache.org>.
CarbonDataQA1 commented on issue #3639: [CARBONDATA-3724] Secondary Index enable on partition Table
URL: https://github.com/apache/carbondata/pull/3639#issuecomment-594607987
 
 
   Build Success with Spark 2.3.4, Please check CI http://121.244.95.60:12545/job/ApacheCarbonPRBuilder2.3/2323/
   

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] kunal642 commented on issue #3639: [CARBONDATA-3724] Secondary Index enable on partition Table

Posted by GitBox <gi...@apache.org>.
kunal642 commented on issue #3639: [CARBONDATA-3724] Secondary Index enable on partition Table
URL: https://github.com/apache/carbondata/pull/3639#issuecomment-608451581
 
 
   @jackylk Please review

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] CarbonDataQA1 commented on issue #3639: [CARBONDATA-3724] Secondary Index enable on partition Table

Posted by GitBox <gi...@apache.org>.
CarbonDataQA1 commented on issue #3639: [CARBONDATA-3724] Secondary Index enable on partition Table
URL: https://github.com/apache/carbondata/pull/3639#issuecomment-605857670
 
 
   Build Success with Spark 2.3.4, Please check CI http://121.244.95.60:12545/job/ApacheCarbonPRBuilder2.3/2592/
   

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] kunal642 commented on issue #3639: [CARBONDATA-3724] Secondary Index enable on partition Table

Posted by GitBox <gi...@apache.org>.
kunal642 commented on issue #3639: [CARBONDATA-3724] Secondary Index enable on partition Table
URL: https://github.com/apache/carbondata/pull/3639#issuecomment-611881831
 
 
   retest this please

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] ajantha-bhat commented on issue #3639: [WIP] Secondary Index enable on partition Table

Posted by GitBox <gi...@apache.org>.
ajantha-bhat commented on issue #3639: [WIP] Secondary Index enable on partition Table
URL: https://github.com/apache/carbondata/pull/3639#issuecomment-591242095
 
 
   retest this please

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] CarbonDataQA1 commented on issue #3639: [WIP] Secondary Index enable on partition Table

Posted by GitBox <gi...@apache.org>.
CarbonDataQA1 commented on issue #3639: [WIP] Secondary Index enable on partition Table
URL: https://github.com/apache/carbondata/pull/3639#issuecomment-591243583
 
 
   Build Failed  with Spark 2.3.4, Please check CI http://121.244.95.60:12545/job/ApacheCarbonPRBuilder2.3/2178/
   

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] jackylk commented on a change in pull request #3639: [CARBONDATA-3724] Secondary Index enable on partition Table

Posted by GitBox <gi...@apache.org>.
jackylk commented on a change in pull request #3639: [CARBONDATA-3724] Secondary Index enable on partition Table
URL: https://github.com/apache/carbondata/pull/3639#discussion_r386006290
 
 

 ##########
 File path: core/src/main/java/org/apache/carbondata/core/scan/scanner/impl/BlockletFilterScanner.java
 ##########
 @@ -453,8 +453,8 @@ private BlockletScannedResult executeFilterForPages(
     scannedResult.setPageIdFiltered(pageFilteredPages);
     scannedResult.setLazyBlockletLoader(lazyBlocklet);
     scannedResult.setBlockletId(
-        blockExecutionInfo.getBlockIdString() + CarbonCommonConstants.FILE_SEPARATOR
-            + rawBlockletColumnChunks.getDataBlock().blockletIndex());
+        blockExecutionInfo.getBlockIdString(),
+            "" + rawBlockletColumnChunks.getDataBlock().blockletIndex());
 
 Review comment:
   use String.valueOf 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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] CarbonDataQA1 commented on issue #3639: [CARBONDATA-3724] Secondary Index enable on partition Table

Posted by GitBox <gi...@apache.org>.
CarbonDataQA1 commented on issue #3639: [CARBONDATA-3724] Secondary Index enable on partition Table
URL: https://github.com/apache/carbondata/pull/3639#issuecomment-594603884
 
 
   Build Success with Spark 2.4.4, Please check CI http://121.244.95.60:12545/job/ApacheCarbon_PR_Builder_2.4.4/616/
   

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] CarbonDataQA1 commented on issue #3639: [WIP] Secondary Index enable on partition Table

Posted by GitBox <gi...@apache.org>.
CarbonDataQA1 commented on issue #3639: [WIP] Secondary Index enable on partition Table
URL: https://github.com/apache/carbondata/pull/3639#issuecomment-591920433
 
 
   Build Success with Spark 2.4.4, Please check CI http://121.244.95.60:12545/job/ApacheCarbon_PR_Builder_2.4.4/511/
   

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] CarbonDataQA1 commented on issue #3639: [WIP] Secondary Index enable on partition Table

Posted by GitBox <gi...@apache.org>.
CarbonDataQA1 commented on issue #3639: [WIP] Secondary Index enable on partition Table
URL: https://github.com/apache/carbondata/pull/3639#issuecomment-591242992
 
 
   Build Failed  with Spark 2.4.4, Please check CI http://121.244.95.60:12545/job/ApacheCarbon_PR_Builder_2.4.4/479/
   

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] maheshrajus commented on a change in pull request #3639: [CARBONDATA-3724] Secondary Index enable on partition Table

Posted by GitBox <gi...@apache.org>.
maheshrajus commented on a change in pull request #3639: [CARBONDATA-3724] Secondary Index enable on partition Table
URL: https://github.com/apache/carbondata/pull/3639#discussion_r386905844
 
 

 ##########
 File path: integration/spark/src/test/scala/org/apache/carbondata/spark/testsuite/iud/DeleteCarbonTableTestCase.scala
 ##########
 @@ -40,17 +40,20 @@ class DeleteCarbonTableTestCase extends QueryTest with BeforeAndAfterAll {
     sql("drop database  if exists iud_db cascade")
     sql("create database  iud_db")
 
-    sql("""create table iud_db.source2 (c11 string,c22 int,c33 string,c55 string, c66 int) STORED AS carbondata""")
+    sql(
+      """create table iud_db.source2 (c11 string,c22 int,c33 string,c55 string, c66 int) STORED
+        |AS carbondata""".stripMargin)
     sql(s"""LOAD DATA LOCAL INPATH '$resourcesPath/IUD/source2.csv' INTO table iud_db.source2""")
     sql("use iud_db")
   }
+
   test("delete data from carbon table with alias [where clause ]") {
     sql("""create table iud_db.dest (c1 string,c2 int,c3 string,c5 string) STORED AS carbondata""")
     sql(s"""LOAD DATA LOCAL INPATH '$resourcesPath/IUD/dest.csv' INTO table iud_db.dest""")
     sql("""delete from iud_db.dest d where d.c1 = 'a'""").show
     checkAnswer(
       sql("""select c2 from iud_db.dest"""),
-      Seq(Row(2), Row(3),Row(4), Row(5))
+      Seq(Row(2), Row(3), Row(4), Row(5))
 
 Review comment:
   Space is not followed after each parameter.  So corrected it as part of code formatting

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] maheshrajus commented on a change in pull request #3639: [CARBONDATA-3724] Secondary Index enable on partition Table

Posted by GitBox <gi...@apache.org>.
maheshrajus commented on a change in pull request #3639: [CARBONDATA-3724] Secondary Index enable on partition Table
URL: https://github.com/apache/carbondata/pull/3639#discussion_r386906020
 
 

 ##########
 File path: integration/spark/src/test/scala/org/apache/carbondata/spark/testsuite/iud/DeleteCarbonTableTestCase.scala
 ##########
 @@ -70,7 +73,7 @@ class DeleteCarbonTableTestCase extends QueryTest with BeforeAndAfterAll {
     sql("""delete from dest where c1 IN ('d', 'e')""").show
     checkAnswer(
       sql("""select c1 from dest"""),
-      Seq(Row("a"), Row("b"),Row("c"))
+      Seq(Row("a"), Row("b"), Row("c"))
 
 Review comment:
   Space is not followed after each parameter.  So corrected it as part of code formatting

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] maheshrajus commented on a change in pull request #3639: [CARBONDATA-3724] Secondary Index enable on partition Table

Posted by GitBox <gi...@apache.org>.
maheshrajus commented on a change in pull request #3639: [CARBONDATA-3724] Secondary Index enable on partition Table
URL: https://github.com/apache/carbondata/pull/3639#discussion_r386906068
 
 

 ##########
 File path: integration/spark/src/test/scala/org/apache/carbondata/spark/testsuite/iud/DeleteCarbonTableTestCase.scala
 ##########
 @@ -119,18 +122,22 @@ class DeleteCarbonTableTestCase extends QueryTest with BeforeAndAfterAll {
 
   test("partition delete data from carbon table with alias [where clause ]") {
     sql("drop table if exists iud_db.dest")
-    sql("""create table iud_db.dest (c1 string,c2 int,c5 string) PARTITIONED BY(c3 string) STORED AS carbondata""")
+    sql(
+      """create table iud_db.dest (c1 string,c2 int,c5 string) PARTITIONED BY(c3 string) STORED AS
+        |carbondata""".stripMargin)
     sql(s"""LOAD DATA LOCAL INPATH '$resourcesPath/IUD/dest.csv' INTO table iud_db.dest""")
     sql("""delete from iud_db.dest d where d.c1 = 'a'""").show
     checkAnswer(
       sql("""select c2 from iud_db.dest"""),
-      Seq(Row(2), Row(3),Row(4), Row(5))
+      Seq(Row(2), Row(3), Row(4), Row(5))
 
 Review comment:
   Space is not followed after each parameter.  So corrected it as part of code formatting

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] maheshrajus commented on a change in pull request #3639: [CARBONDATA-3724] Secondary Index enable on partition Table

Posted by GitBox <gi...@apache.org>.
maheshrajus commented on a change in pull request #3639: [CARBONDATA-3724] Secondary Index enable on partition Table
URL: https://github.com/apache/carbondata/pull/3639#discussion_r387612741
 
 

 ##########
 File path: core/src/main/java/org/apache/carbondata/core/indexstore/blockletindex/BlockDataMap.java
 ##########
 @@ -788,7 +788,13 @@ private boolean addBlockBasedOnMinMaxValue(FilterExecuter filterExecuter, byte[]
       byte[][] minValue, boolean[] minMaxFlag, String filePath, int blockletId) {
     BitSet bitSet = null;
     if (filterExecuter instanceof ImplicitColumnFilterExecutor) {
-      String uniqueBlockPath = filePath.substring(filePath.lastIndexOf("/Part") + 1);
+      String uniqueBlockPath;
+      if (segmentPropertiesWrapper.getCarbonTable().isHivePartitionTable()) {
 
 Review comment:
   For avoiding extra memory for two instance variables, we can use existing segmentPropertiesWrapper getter for carbontable.

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] CarbonDataQA1 commented on issue #3639: [WIP] Secondary Index enable on partition Table

Posted by GitBox <gi...@apache.org>.
CarbonDataQA1 commented on issue #3639: [WIP] Secondary Index enable on partition Table
URL: https://github.com/apache/carbondata/pull/3639#issuecomment-591327346
 
 
   Build Success with Spark 2.4.4, Please check CI http://121.244.95.60:12545/job/ApacheCarbon_PR_Builder_2.4.4/494/
   

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] kunal642 commented on issue #3639: [CARBONDATA-3724] Secondary Index enable on partition Table

Posted by GitBox <gi...@apache.org>.
kunal642 commented on issue #3639: [CARBONDATA-3724] Secondary Index enable on partition Table
URL: https://github.com/apache/carbondata/pull/3639#issuecomment-605785626
 
 
   @maheshrajus 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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] ajantha-bhat commented on issue #3639: [WIP] Secondary Index enable on partition Table

Posted by GitBox <gi...@apache.org>.
ajantha-bhat commented on issue #3639: [WIP] Secondary Index enable on partition Table
URL: https://github.com/apache/carbondata/pull/3639#issuecomment-591242060
 
 
   add to whitelist

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] Indhumathi27 commented on a change in pull request #3639: [CARBONDATA-3724] Secondary Index enable on partition Table

Posted by GitBox <gi...@apache.org>.
Indhumathi27 commented on a change in pull request #3639: [CARBONDATA-3724] Secondary Index enable on partition Table
URL: https://github.com/apache/carbondata/pull/3639#discussion_r387467200
 
 

 ##########
 File path: index/secondary-index/src/test/scala/org/apache/carbondata/spark/testsuite/secondaryindex/TestSIWithPartition.scala
 ##########
 @@ -0,0 +1,379 @@
+/*
+ * 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.execution.SparkPlan
+import org.apache.spark.sql.secondaryindex.joins.BroadCastSIFilterPushJoin
+import org.apache.spark.sql.test.util.QueryTest
+import org.scalatest.{BeforeAndAfterAll, Ignore}
+
+class TestSIWithPartition extends QueryTest with BeforeAndAfterAll {
+
+  override protected def beforeAll(): Unit = {
+    sql("drop table if exists uniqdata1")
+    sql(
+      "CREATE TABLE uniqdata1 (CUST_ID INT,CUST_NAME STRING,DOB timestamp,DOJ timestamp," +
+      "BIGINT_COLUMN1 bigint,BIGINT_COLUMN2 bigint,DECIMAL_COLUMN1 DECIMAL(30, 10)," +
+      "DECIMAL_COLUMN2 DECIMAL(36, 10),Double_COLUMN1 double, Double_COLUMN2 double," +
+      "INTEGER_COLUMN1 int) PARTITIONED BY(ACTIVE_EMUI_VERSION string) STORED AS carbondata " +
+      "TBLPROPERTIES('TABLE_BLOCKSIZE'='256 MB')")
+    sql(s"LOAD DATA LOCAL INPATH '$resourcesPath/restructure/data_2000.csv' INTO " +
+        "TABLE uniqdata1 partition(ACTIVE_EMUI_VERSION='abc') OPTIONS('DELIMITER'=',', " +
+        "'BAD_RECORDS_LOGGER_ENABLE'='FALSE', 'BAD_RECORDS_ACTION'='FORCE','FILEHEADER'='CUST_ID," +
+        "CUST_NAME,ACTIVE_EMUI_VERSION,DOB,DOJ,BIGINT_COLUMN1,BIGINT_COLUMN2,DECIMAL_COLUMN1," +
+        "DECIMAL_COLUMN2,Double_COLUMN1,Double_COLUMN2,INTEGER_COLUMN1')")
+    sql(s"LOAD DATA LOCAL INPATH '$resourcesPath/restructure/data_2000.csv' INTO " +
+        "TABLE uniqdata1 partition(ACTIVE_EMUI_VERSION='abc') OPTIONS('DELIMITER'=',', " +
+        "'BAD_RECORDS_LOGGER_ENABLE'='FALSE', 'BAD_RECORDS_ACTION'='FORCE','FILEHEADER'='CUST_ID," +
+        "CUST_NAME,ACTIVE_EMUI_VERSION,DOB,DOJ,BIGINT_COLUMN1,BIGINT_COLUMN2,DECIMAL_COLUMN1," +
+        "DECIMAL_COLUMN2,Double_COLUMN1,Double_COLUMN2,INTEGER_COLUMN1')")
+    sql(s"LOAD DATA LOCAL INPATH '$resourcesPath/restructure/data_2000.csv' INTO " +
+        "TABLE uniqdata1 partition(ACTIVE_EMUI_VERSION='abc') OPTIONS('DELIMITER'=',', " +
+        "'BAD_RECORDS_LOGGER_ENABLE'='FALSE', 'BAD_RECORDS_ACTION'='FORCE','FILEHEADER'='CUST_ID," +
+        "CUST_NAME,ACTIVE_EMUI_VERSION,DOB,DOJ,BIGINT_COLUMN1,BIGINT_COLUMN2,DECIMAL_COLUMN1," +
+        "DECIMAL_COLUMN2,Double_COLUMN1,Double_COLUMN2,INTEGER_COLUMN1')")
+    sql(s"LOAD DATA LOCAL INPATH '$resourcesPath/restructure/data_2000.csv' INTO " +
+        "TABLE uniqdata1 partition(ACTIVE_EMUI_VERSION='abc') OPTIONS('DELIMITER'=',', " +
+        "'BAD_RECORDS_LOGGER_ENABLE'='FALSE', 'BAD_RECORDS_ACTION'='FORCE','FILEHEADER'='CUST_ID," +
+        "CUST_NAME,ACTIVE_EMUI_VERSION,DOB,DOJ,BIGINT_COLUMN1,BIGINT_COLUMN2,DECIMAL_COLUMN1," +
+        "DECIMAL_COLUMN2,Double_COLUMN1,Double_COLUMN2,INTEGER_COLUMN1')")
+  }
+
+  test("Testing SI on partition column") {
+    sql("drop index if exists indextable1 on uniqdata1")
+    intercept[UnsupportedOperationException] {
+      sql("create index indextable1 on table uniqdata1 (ACTIVE_EMUI_VERSION) AS 'carbondata'")
+    }
+  }
+
+  test("Testing SI without partition column") {
+    sql("drop index if exists indextable1 on uniqdata1")
+    sql("create index indextable1 on table uniqdata1 (DOB, CUST_NAME) AS 'carbondata'")
+    val withoutIndex =
+      sql("select * from uniqdata1 where ni(CUST_NAME='CUST_NAME_00108')")
+        .collect().toSeq
+
+    checkAnswer(sql("select * from uniqdata1 where CUST_NAME='CUST_NAME_00108'"),
+      withoutIndex)
+
+    val df = sql("select * from uniqdata1 where CUST_NAME='CUST_NAME_00108'")
+      .queryExecution
+      .sparkPlan
+    if (!isFilterPushedDownToSI(df)) {
+      assert(false)
+    } else {
+      assert(true)
+    }
+  }
+
+  test("Testing SI with partition column[where clause]") {
+    sql("drop index if exists indextable1 on uniqdata1")
+    sql("create index indextable1 on table uniqdata1 (DOB, CUST_NAME) AS 'carbondata'")
+    val withoutIndex =
+      sql(
+        "select * from uniqdata1 where ni(CUST_NAME='CUST_NAME_00108' and ACTIVE_EMUI_VERSION = " +
+        "'abc')")
+        .collect().toSeq
+
+    checkAnswer(sql(
+      "select * from uniqdata1 where CUST_NAME='CUST_NAME_00108' and ACTIVE_EMUI_VERSION = 'abc'"),
+      withoutIndex)
+
+    val df = sql(
+      "select * from uniqdata1 where CUST_NAME='CUST_NAME_00108' and ACTIVE_EMUI_VERSION = 'abc'")
+      .queryExecution
+      .sparkPlan
+    if (!isFilterPushedDownToSI(df)) {
+      assert(false)
+    } else {
+      assert(true)
+    }
+  }
+
+  test("Testing SI on partition table with OR condition") {
+    sql("drop index if exists indextable1 on uniqdata1")
+    sql("create index indextable1 on table uniqdata1 (DOB, CUST_NAME) AS 'carbondata'")
+    val withoutIndex =
+      sql(
+        "select * from uniqdata1 where ni(CUST_NAME='CUST_NAME_00108' OR ACTIVE_EMUI_VERSION = " +
+        "'abc')")
+        .collect().toSeq
+
+    checkAnswer(sql(
+      "select * from uniqdata1 where CUST_NAME='CUST_NAME_00108' OR ACTIVE_EMUI_VERSION = 'abc'"),
+      withoutIndex)
+
+    val df = sql(
+      "select * from uniqdata1 where CUST_NAME='CUST_NAME_00108' OR ACTIVE_EMUI_VERSION = 'abc'")
+      .queryExecution
+      .sparkPlan
+    if (!isFilterPushedDownToSI(df)) {
+      assert(true)
+    } else {
+      assert(false)
+    }
+  }
+
+  test("Testing SI on partition table with combination of OR OR") {
+    sql("drop index if exists indextable1 on uniqdata1")
+    sql("create index indextable1 on table uniqdata1 (DOB, CUST_NAME) AS 'carbondata'")
+    val withoutIndex =
+      sql(
+        "select * from uniqdata1 where ni(CUST_NAME='CUST_NAME_00108' OR CUST_ID='9000' OR " +
+        "ACTIVE_EMUI_VERSION = " +
+        "'abc')")
+        .collect().toSeq
+
+    checkAnswer(sql(
+      "select * from uniqdata1 where CUST_NAME='CUST_NAME_00108' OR CUST_ID='9000' OR " +
+      "ACTIVE_EMUI_VERSION = 'abc'"),
+      withoutIndex)
+
+    val df = sql(
+      "select * from uniqdata1 where CUST_NAME='CUST_NAME_00108' OR CUST_ID='9000' OR " +
+      "ACTIVE_EMUI_VERSION = 'abc'")
+      .queryExecution
+      .sparkPlan
+    if (!isFilterPushedDownToSI(df)) {
+      assert(true)
+    } else {
+      assert(false)
+    }
+  }
+
+  test("Testing SI on partition table with combination of OR AND") {
+    sql("drop index if exists indextable1 on uniqdata1")
+    sql("create index indextable1 on table uniqdata1 (DOB, CUST_NAME) AS 'carbondata'")
+    val withoutIndex =
+      sql(
+        "select * from uniqdata1 where ni(CUST_NAME='CUST_NAME_00108' OR CUST_ID='9000' AND " +
+        "ACTIVE_EMUI_VERSION = " +
+        "'abc')")
+        .collect().toSeq
+
+    checkAnswer(sql(
+      "select * from uniqdata1 where CUST_NAME='CUST_NAME_00108' OR CUST_ID='9000' AND " +
+      "ACTIVE_EMUI_VERSION = 'abc'"),
+      withoutIndex)
+
+    val df = sql(
+      "select * from uniqdata1 where CUST_NAME='CUST_NAME_00108' OR CUST_ID='9000' AND " +
+      "ACTIVE_EMUI_VERSION = 'abc'")
+      .queryExecution
+      .sparkPlan
+    if (!isFilterPushedDownToSI(df)) {
+      assert(true)
+    } else {
+      assert(false)
+    }
+  }
+
+  test("Testing SI on partition table with combination of AND OR") {
+    sql("drop index if exists indextable1 on uniqdata1")
+    sql("create index indextable1 on table uniqdata1 (DOB, CUST_NAME) AS 'carbondata'")
+    val withoutIndex =
+      sql(
+        "select * from uniqdata1 where ni(CUST_NAME='CUST_NAME_00108' AND CUST_ID='9000' OR " +
+        "ACTIVE_EMUI_VERSION = " +
+        "'abc')")
+        .collect().toSeq
+
+    checkAnswer(sql(
+      "select * from uniqdata1 where CUST_NAME='CUST_NAME_00108' AND CUST_ID='9000' OR " +
+      "ACTIVE_EMUI_VERSION = 'abc'"),
+      withoutIndex)
+
+    val df = sql(
+      "select * from uniqdata1 where CUST_NAME='CUST_NAME_00108' AND CUST_ID='9000' OR " +
+      "ACTIVE_EMUI_VERSION = 'abc'")
+      .queryExecution
+      .sparkPlan
+    if (!isFilterPushedDownToSI(df)) {
+      assert(true)
+    } else {
+      assert(false)
+    }
+  }
+
+  test("Testing SI on partition table with combination of AND AND") {
+    sql("drop index if exists indextable1 on uniqdata1")
+    sql("create index indextable1 on table uniqdata1 (DOB, CUST_NAME) AS 'carbondata'")
+    val withoutIndex =
+      sql(
+        "select * from uniqdata1 where ni(CUST_NAME='CUST_NAME_00108' AND CUST_ID='9000' AND " +
+        "ACTIVE_EMUI_VERSION = " +
+        "'abc')")
+        .collect().toSeq
+
+    checkAnswer(sql(
+      "select * from uniqdata1 where CUST_NAME='CUST_NAME_00108' AND CUST_ID='9000' AND " +
+      "ACTIVE_EMUI_VERSION = 'abc'"),
+      withoutIndex)
+
+    val df = sql(
+      "select * from uniqdata1 where CUST_NAME='CUST_NAME_00108' AND CUST_ID='9000' AND " +
+      "ACTIVE_EMUI_VERSION = 'abc'")
+      .queryExecution
+      .sparkPlan
+    if (!isFilterPushedDownToSI(df)) {
+      assert(false)
+    } else {
+      assert(true)
+    }
+  }
+
+  test("Testing SI on partition table with major compaction") {
+    sql("drop index if exists indextable1 on uniqdata1")
+    sql("create index indextable1 on table uniqdata1 (DOB, CUST_NAME) AS 'carbondata'")
+    val withoutIndex =
+      sql(
+        "select * from uniqdata1 where CUST_NAME='CUST_NAME_00108' and ACTIVE_EMUI_VERSION = " +
+        "'abc'")
+        .collect().toSeq
+
+    sql("alter table uniqdata1 compact 'major'")
+
+    checkAnswer(sql(
+      "select * from uniqdata1 where CUST_NAME='CUST_NAME_00108' and ACTIVE_EMUI_VERSION = 'abc'"),
+      withoutIndex)
+
+    val df = sql(
+      "select * from uniqdata1 where CUST_NAME='CUST_NAME_00108' and ACTIVE_EMUI_VERSION = 'abc'")
+      .queryExecution
+      .sparkPlan
+    if (!isFilterPushedDownToSI(df)) {
+      assert(false)
+    } else {
+      assert(true)
+    }
+  }
+
+  test("Testing SI on partition table with minor compaction") {
+    sql("drop index if exists indextable1 on uniqdata1")
+    sql("create index indextable1 on table uniqdata1 (DOB, CUST_NAME) AS 'carbondata'")
+
+    val withoutIndex =
+      sql(
+        "select * from uniqdata1 where CUST_NAME='CUST_NAME_00108' and ACTIVE_EMUI_VERSION = " +
+        "'abc'")
+        .collect().toSeq
+
+    sql("alter table uniqdata1 compact 'minor'")
+
+    checkAnswer(sql(
+      "select * from uniqdata1 where CUST_NAME='CUST_NAME_00108' and ACTIVE_EMUI_VERSION = 'abc'"),
+      withoutIndex)
+
+    val df = sql(
+      "select * from uniqdata1 where CUST_NAME='CUST_NAME_00108' and ACTIVE_EMUI_VERSION = 'abc'")
+      .queryExecution
+      .sparkPlan
+    if (!isFilterPushedDownToSI(df)) {
+      assert(false)
+    } else {
+      assert(true)
+    }
+  }
+
+  test("Testing SI on partition table with delete") {
+    sql("drop index if exists indextable1 on uniqdata1")
+    sql("create index indextable1 on table uniqdata1 (DOB, CUST_NAME) AS 'carbondata'")
+
+    checkAnswer(sql(
+      "select count(*) from uniqdata1 where CUST_NAME='CUST_NAME_00108' and ACTIVE_EMUI_VERSION =" +
+      " 'abc'"),
+      Seq(Row(4)))
+
+    sql("delete from uniqdata1 where CUST_NAME='CUST_NAME_00108'").show()
+
+    checkAnswer(sql(
+      "select count(*) from uniqdata1 where CUST_NAME='CUST_NAME_00108' and ACTIVE_EMUI_VERSION =" +
+      " 'abc'"),
+      Seq(Row(0)))
+
+    val df = sql(
+      "select * from uniqdata1 where CUST_NAME='CUST_NAME_00108' and ACTIVE_EMUI_VERSION = 'abc'")
+      .queryExecution
+      .sparkPlan
+    if (!isFilterPushedDownToSI(df)) {
+      assert(false)
+    } else {
+      assert(true)
+    }
+  }
+
+  test("Testing SI on partition table with update") {
+    sql("drop index if exists indextable1 on uniqdata1")
+    sql("create index indextable1 on table uniqdata1 (DOB, CUST_NAME) AS 'carbondata'")
+
+    checkAnswer(sql(
+      "select count(*) from uniqdata1 where CUST_ID='9000' and ACTIVE_EMUI_VERSION = 'abc'"),
+      Seq(Row(4)))
+    intercept[RuntimeException] {
+      sql("update uniqdata1 d set (d.CUST_ID) = ('8000')  where d.CUST_ID = '9000'").show()
+    }
+  }
+
+  test("Testing SI on partition table with rename") {
+    sql("drop index if exists indextable1 on uniqdata1")
+    sql("create index indextable1 on table uniqdata1 (DOB, CUST_NAME) AS 'carbondata'")
+
+    val withoutIndex =
+      sql(
+        "select * from uniqdata1 where CUST_NAME='CUST_NAME_00108' and ACTIVE_EMUI_VERSION = " +
+        "'abc'")
+        .collect().toSeq
+
+    sql("alter table uniqdata1 change CUST_NAME test string")
+
+    checkAnswer(sql(
+      "select * from uniqdata1 where test='CUST_NAME_00108' and ACTIVE_EMUI_VERSION = 'abc'"),
+      withoutIndex)
+
+    val df = sql(
+      "select * from uniqdata1 where test='CUST_NAME_00108' and ACTIVE_EMUI_VERSION = 'abc'")
+      .queryExecution
+      .sparkPlan
+    if (!isFilterPushedDownToSI(df)) {
+      assert(false)
+    } else {
+      assert(true)
+    }
+  }
+
+  override protected def afterAll(): Unit = {
+    sql("drop index if exists indextable1 on uniqdata1")
+    sql("drop table if exists uniqdata1")
+  }
+
+  /**
+   * Method to check whether the filter is push down to SI table or not
+   *
+   * @param sparkPlan
+   * @return
+   */
+  private def isFilterPushedDownToSI(sparkPlan: SparkPlan): Boolean = {
 
 Review comment:
   Please reuse the method from TestSecondaryIndexForORFilterPushDown class

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] CarbonDataQA1 commented on issue #3639: [WIP] Secondary Index enable on partition Table

Posted by GitBox <gi...@apache.org>.
CarbonDataQA1 commented on issue #3639: [WIP] Secondary Index enable on partition Table
URL: https://github.com/apache/carbondata/pull/3639#issuecomment-591290350
 
 
   Build Failed  with Spark 2.3.4, Please check CI http://121.244.95.60:12545/job/ApacheCarbonPRBuilder2.3/2184/
   

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] CarbonDataQA1 commented on issue #3639: [WIP] Secondary Index enable on partition Table

Posted by GitBox <gi...@apache.org>.
CarbonDataQA1 commented on issue #3639: [WIP] Secondary Index enable on partition Table
URL: https://github.com/apache/carbondata/pull/3639#issuecomment-592104285
 
 
   Build Success with Spark 2.3.4, Please check CI http://121.244.95.60:12545/job/ApacheCarbonPRBuilder2.3/2214/
   

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] CarbonDataQA1 commented on issue #3639: [CARBONDATA-3724] Secondary Index enable on partition Table

Posted by GitBox <gi...@apache.org>.
CarbonDataQA1 commented on issue #3639: [CARBONDATA-3724] Secondary Index enable on partition Table
URL: https://github.com/apache/carbondata/pull/3639#issuecomment-605854854
 
 
   Build Success with Spark 2.4.5, Please check CI http://121.244.95.60:12545/job/ApacheCarbon_PR_Builder_2.4.5/884/
   

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] CarbonDataQA1 commented on issue #3639: [WIP] Secondary Index enable on partition Table

Posted by GitBox <gi...@apache.org>.
CarbonDataQA1 commented on issue #3639: [WIP] Secondary Index enable on partition Table
URL: https://github.com/apache/carbondata/pull/3639#issuecomment-591877095
 
 
   Build Success with Spark 2.4.4, Please check CI http://121.244.95.60:12545/job/ApacheCarbon_PR_Builder_2.4.4/510/
   

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] Indhumathi27 commented on a change in pull request #3639: [CARBONDATA-3724] Secondary Index enable on partition Table

Posted by GitBox <gi...@apache.org>.
Indhumathi27 commented on a change in pull request #3639: [CARBONDATA-3724] Secondary Index enable on partition Table
URL: https://github.com/apache/carbondata/pull/3639#discussion_r387473487
 
 

 ##########
 File path: core/src/main/java/org/apache/carbondata/core/scan/scanner/impl/BlockletFilterScanner.java
 ##########
 @@ -203,8 +203,8 @@ private BlockletScannedResult executeFilter(RawBlockletColumnChunks rawBlockletC
     BlockletScannedResult scannedResult =
         new FilterQueryScannedResult(blockExecutionInfo, queryStatisticsModel);
     scannedResult.setBlockletId(
-        blockExecutionInfo.getBlockIdString() + CarbonCommonConstants.FILE_SEPARATOR +
-            rawBlockletColumnChunks.getDataBlock().blockletIndex());
+        blockExecutionInfo.getBlockIdString(),
+            "" + rawBlockletColumnChunks.getDataBlock().blockletIndex());
 
 Review comment:
   remove "" if not required. Instead can use String.valueOf

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] maheshrajus commented on a change in pull request #3639: [CARBONDATA-3724] Secondary Index enable on partition Table

Posted by GitBox <gi...@apache.org>.
maheshrajus commented on a change in pull request #3639: [CARBONDATA-3724] Secondary Index enable on partition Table
URL: https://github.com/apache/carbondata/pull/3639#discussion_r386983083
 
 

 ##########
 File path: hadoop/src/main/java/org/apache/carbondata/hadoop/api/CarbonTableInputFormat.java
 ##########
 @@ -188,7 +189,10 @@
 
     List<Segment> segmentToAccess =
         getFilteredSegment(job, validAndInProgressSegments, false, readCommittedScope);
-
+    String segmentFileName = job.getConfiguration().get("current.segmentfile");
 
 Review comment:
   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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] akashrn5 commented on a change in pull request #3639: [CARBONDATA-3724] Secondary Index enable on partition Table

Posted by GitBox <gi...@apache.org>.
akashrn5 commented on a change in pull request #3639: [CARBONDATA-3724] Secondary Index enable on partition Table
URL: https://github.com/apache/carbondata/pull/3639#discussion_r386084217
 
 

 ##########
 File path: integration/spark/src/main/scala/org/apache/spark/sql/secondaryindex/rdd/CarbonSecondaryIndexRDD.scala
 ##########
 @@ -190,24 +191,23 @@ class CarbonSecondaryIndexRDD[K, V](
     val startTime = System.currentTimeMillis()
     val absoluteTableIdentifier: AbsoluteTableIdentifier = AbsoluteTableIdentifier.from(
       carbonStoreLocation, databaseName, factTableName, tableId)
-    val updateStatusManager: SegmentUpdateStatusManager = new SegmentUpdateStatusManager(
-      carbonLoadModel.getCarbonDataLoadSchema.getCarbonTable)
     val jobConf: JobConf = new JobConf(hadoopConf)
     SparkHadoopUtil.get.addCredentials(jobConf)
     val job: Job = new Job(jobConf)
-    val format = CarbonInputFormatUtil.createCarbonInputFormat(absoluteTableIdentifier, job)
+
+    if (carbonLoadModel.getCarbonDataLoadSchema.getCarbonTable.isHivePartitionTable) {
+      job.getConfiguration
+        .set("current.segmentfile", segmentId + "_" + carbonLoadModel.getFactTimeStamp)
 
 Review comment:
   use CarbonCommonConstants.underscore

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] jackylk commented on a change in pull request #3639: [CARBONDATA-3724] Secondary Index enable on partition Table

Posted by GitBox <gi...@apache.org>.
jackylk commented on a change in pull request #3639: [CARBONDATA-3724] Secondary Index enable on partition Table
URL: https://github.com/apache/carbondata/pull/3639#discussion_r386006342
 
 

 ##########
 File path: core/src/main/java/org/apache/carbondata/core/mutate/data/BlockMappingVO.java
 ##########
 @@ -30,6 +30,8 @@
 
   private Map<String, RowCountDetailsVO> completeBlockRowDetailVO;
 
+  private Map<String, String> blockToSegmentMapping;
 
 Review comment:
   There are too many mappings in this class, can you avoid it. Its meaning is not easy to understand

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] CarbonDataQA1 commented on issue #3639: [WIP] Secondary Index enable on partition Table

Posted by GitBox <gi...@apache.org>.
CarbonDataQA1 commented on issue #3639: [WIP] Secondary Index enable on partition Table
URL: https://github.com/apache/carbondata/pull/3639#issuecomment-590942397
 
 
   Can one of the admins verify this patch?

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] maheshrajus commented on a change in pull request #3639: [CARBONDATA-3724] Secondary Index enable on partition Table

Posted by GitBox <gi...@apache.org>.
maheshrajus commented on a change in pull request #3639: [CARBONDATA-3724] Secondary Index enable on partition Table
URL: https://github.com/apache/carbondata/pull/3639#discussion_r386892338
 
 

 ##########
 File path: core/src/main/java/org/apache/carbondata/core/scan/scanner/impl/BlockletFullScanner.java
 ##########
 @@ -83,9 +82,8 @@ public BlockletScannedResult scanBlocklet(
         .get(QueryStatisticsConstants.TOTAL_PAGE_SCANNED);
     totalPagesScanned.addCountStatistic(QueryStatisticsConstants.TOTAL_PAGE_SCANNED,
         totalPagesScanned.getCount() + rawBlockletColumnChunks.getDataBlock().numberOfPages());
-    String blockletId = blockExecutionInfo.getBlockIdString() + CarbonCommonConstants.FILE_SEPARATOR
-        + rawBlockletColumnChunks.getDataBlock().blockletIndex();
-    scannedResult.setBlockletId(blockletId);
+    scannedResult.setBlockletId(blockExecutionInfo.getBlockIdString(),
+        "" + rawBlockletColumnChunks.getDataBlock().blockletIndex());
 
 Review comment:
   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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] maheshrajus commented on a change in pull request #3639: [CARBONDATA-3724] Secondary Index enable on partition Table

Posted by GitBox <gi...@apache.org>.
maheshrajus commented on a change in pull request #3639: [CARBONDATA-3724] Secondary Index enable on partition Table
URL: https://github.com/apache/carbondata/pull/3639#discussion_r386897897
 
 

 ##########
 File path: integration/spark/src/main/scala/org/apache/spark/sql/secondaryindex/rdd/CarbonSecondaryIndexRDD.scala
 ##########
 @@ -190,24 +191,23 @@ class CarbonSecondaryIndexRDD[K, V](
     val startTime = System.currentTimeMillis()
     val absoluteTableIdentifier: AbsoluteTableIdentifier = AbsoluteTableIdentifier.from(
       carbonStoreLocation, databaseName, factTableName, tableId)
-    val updateStatusManager: SegmentUpdateStatusManager = new SegmentUpdateStatusManager(
-      carbonLoadModel.getCarbonDataLoadSchema.getCarbonTable)
     val jobConf: JobConf = new JobConf(hadoopConf)
     SparkHadoopUtil.get.addCredentials(jobConf)
     val job: Job = new Job(jobConf)
-    val format = CarbonInputFormatUtil.createCarbonInputFormat(absoluteTableIdentifier, job)
+
+    if (carbonLoadModel.getCarbonDataLoadSchema.getCarbonTable.isHivePartitionTable) {
+      job.getConfiguration
+        .set("current.segmentfile", segmentId + "_" + carbonLoadModel.getFactTimeStamp)
 
 Review comment:
   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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] CarbonDataQA1 commented on issue #3639: [CARBONDATA-3724] Secondary Index enable on partition Table

Posted by GitBox <gi...@apache.org>.
CarbonDataQA1 commented on issue #3639: [CARBONDATA-3724] Secondary Index enable on partition Table
URL: https://github.com/apache/carbondata/pull/3639#issuecomment-608450723
 
 
   Build Success with Spark 2.4.5, Please check CI http://121.244.95.60:12545/job/ApacheCarbon_PR_Builder_2.4.5/920/
   

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] CarbonDataQA1 commented on issue #3639: [WIP] Secondary Index enable on partition Table

Posted by GitBox <gi...@apache.org>.
CarbonDataQA1 commented on issue #3639: [WIP] Secondary Index enable on partition Table
URL: https://github.com/apache/carbondata/pull/3639#issuecomment-591349123
 
 
   Build Failed  with Spark 2.3.4, Please check CI http://121.244.95.60:12545/job/ApacheCarbonPRBuilder2.3/2193/
   

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] maheshrajus commented on a change in pull request #3639: [CARBONDATA-3724] Secondary Index enable on partition Table

Posted by GitBox <gi...@apache.org>.
maheshrajus commented on a change in pull request #3639: [CARBONDATA-3724] Secondary Index enable on partition Table
URL: https://github.com/apache/carbondata/pull/3639#discussion_r386891998
 
 

 ##########
 File path: core/src/main/java/org/apache/carbondata/core/scan/scanner/impl/BlockletFilterScanner.java
 ##########
 @@ -453,8 +453,8 @@ private BlockletScannedResult executeFilterForPages(
     scannedResult.setPageIdFiltered(pageFilteredPages);
     scannedResult.setLazyBlockletLoader(lazyBlocklet);
     scannedResult.setBlockletId(
-        blockExecutionInfo.getBlockIdString() + CarbonCommonConstants.FILE_SEPARATOR
-            + rawBlockletColumnChunks.getDataBlock().blockletIndex());
+        blockExecutionInfo.getBlockIdString(),
+            "" + rawBlockletColumnChunks.getDataBlock().blockletIndex());
 
 Review comment:
   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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] kunal642 commented on issue #3639: [CARBONDATA-3724] Secondary Index enable on partition Table

Posted by GitBox <gi...@apache.org>.
kunal642 commented on issue #3639: [CARBONDATA-3724] Secondary Index enable on partition Table
URL: https://github.com/apache/carbondata/pull/3639#issuecomment-612058072
 
 
   LGTM

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] Indhumathi27 commented on a change in pull request #3639: [CARBONDATA-3724] Secondary Index enable on partition Table

Posted by GitBox <gi...@apache.org>.
Indhumathi27 commented on a change in pull request #3639: [CARBONDATA-3724] Secondary Index enable on partition Table
URL: https://github.com/apache/carbondata/pull/3639#discussion_r387477530
 
 

 ##########
 File path: core/src/main/java/org/apache/carbondata/core/indexstore/blockletindex/BlockDataMap.java
 ##########
 @@ -788,7 +788,13 @@ private boolean addBlockBasedOnMinMaxValue(FilterExecuter filterExecuter, byte[]
       byte[][] minValue, boolean[] minMaxFlag, String filePath, int blockletId) {
     BitSet bitSet = null;
     if (filterExecuter instanceof ImplicitColumnFilterExecutor) {
-      String uniqueBlockPath = filePath.substring(filePath.lastIndexOf("/Part") + 1);
+      String uniqueBlockPath;
+      if (segmentPropertiesWrapper.getCarbonTable().isHivePartitionTable()) {
 
 Review comment:
   In init method, blockletDataMapInfo already has carbonTable. Can use it, instead of getting from segmentPropertiesWrapper

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] akashrn5 commented on a change in pull request #3639: [CARBONDATA-3724] Secondary Index enable on partition Table

Posted by GitBox <gi...@apache.org>.
akashrn5 commented on a change in pull request #3639: [CARBONDATA-3724] Secondary Index enable on partition Table
URL: https://github.com/apache/carbondata/pull/3639#discussion_r386084044
 
 

 ##########
 File path: core/src/main/java/org/apache/carbondata/core/util/CarbonUtil.java
 ##########
 @@ -2779,10 +2797,18 @@ public static String getBlockId(AbsoluteTableIdentifier identifier, String fileP
         } else {
           partitionDir = "";
         }
-        // Replace / with # on partition director to support multi level partitioning. And access
-        // them all as a single entity.
-        blockId = partitionDir.replace("/", "#") + CarbonCommonConstants.FILE_SEPARATOR
-            + segmentId + CarbonCommonConstants.FILE_SEPARATOR + blockName;
+        if (isPartitionTable) {
+          blockId =
+              partitionDir.replace("/", "#")
 
 Review comment:
   Use CarbonCommonConstants.Fileseparator

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] ajantha-bhat removed a comment on issue #3639: [WIP] Secondary Index enable on partition Table

Posted by GitBox <gi...@apache.org>.
ajantha-bhat removed a comment on issue #3639: [WIP] Secondary Index enable on partition Table
URL: https://github.com/apache/carbondata/pull/3639#issuecomment-590949644
 
 
   retest this please

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] ajantha-bhat removed a comment on issue #3639: [WIP] Secondary Index enable on partition Table

Posted by GitBox <gi...@apache.org>.
ajantha-bhat removed a comment on issue #3639: [WIP] Secondary Index enable on partition Table
URL: https://github.com/apache/carbondata/pull/3639#issuecomment-590949550
 
 
   add to whitelist

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] CarbonDataQA1 commented on issue #3639: [WIP] Secondary Index enable on partition Table

Posted by GitBox <gi...@apache.org>.
CarbonDataQA1 commented on issue #3639: [WIP] Secondary Index enable on partition Table
URL: https://github.com/apache/carbondata/pull/3639#issuecomment-591524291
 
 
   Build Success with Spark 2.4.4, Please check CI http://121.244.95.60:12545/job/ApacheCarbon_PR_Builder_2.4.4/503/
   

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services