You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@carbondata.apache.org by lionelcao <gi...@git.apache.org> on 2017/07/03 10:02:01 UTC

[GitHub] carbondata pull request #1125: [CarbonData-1250] change default partition id...

GitHub user lionelcao opened a pull request:

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

    [CarbonData-1250] change default partition id & Add TaskId group in partitionInfo

    This PR change the default partition id from Max to 0 and add taskId management related code to support later add(split)/drop(merge) partition operations.
    
    1. change default partition id from Max to 0
    2. add taskIdInPartitionOrder in partitionInfo to manage the taskId to partition map
    3. remove some useless file
    4. extract some common used function to PartitionUtils
    5. change CarbonInputFormat to use correct taskId
    6. modify test case according to the taskId change.
    7. support range info in ascending order validation
    8. fix some typo

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

    $ git pull https://github.com/lionelcao/carbondata carbon_910_13

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

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

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

    This closes #1125
    
----
commit 83d134919b8962ac3d6269f7c910666527c3a4c5
Author: lionelcao <wh...@gmail.com>
Date:   2017-06-29T16:00:32Z

    [CARBONDATA-1250] change default partition id from Max to 0

commit 5fb7447cc03b8204e98e90129037eee828912bf8
Author: lionelcao <wh...@gmail.com>
Date:   2017-06-29T16:06:30Z

    extract common function

commit fd928d8476b0907dd1cb0347803775b495fe5f74
Author: lionelcao <wh...@gmail.com>
Date:   2017-06-30T15:05:06Z

    add taskid group in partition info and change query filter code
    
    remove partition statistics for now
    
    fix style error
    
    empty commit
    
    add taskid group in partitionInfo
    
    remove unrelated code
    
    fix style error
    
    fix style error
    
    fix bug: getSplits error for non-partition table
    
    change import order
    
    remove unrelated code and test CI
    
    change import order
    
    change query filter code according to default partition id changing
    
    thrift converter
    
    remove useless comment
    
    remove unrelated code

----


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

[GitHub] carbondata issue #1125: [CarbonData-1250] Change default partition id & Add ...

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

    https://github.com/apache/carbondata/pull/1125
  
    Build Failed  with Spark 2.1.0, Please check CI http://136.243.101.176:8080/job/ApacheCarbonPRBuilder/2938/



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

[GitHub] carbondata pull request #1125: [CarbonData-1250] Change default partition id...

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

    https://github.com/apache/carbondata/pull/1125#discussion_r125954525
  
    --- Diff: core/src/main/java/org/apache/carbondata/core/metadata/schema/PartitionInfo.java ---
    @@ -43,13 +44,30 @@
       private List<List<String>> listInfo;
     
       /**
    -   * number of partitions
    +   * number of hash partitions
        */
    -  private int numPartitions;
    +  private int hashNumber;
    +
    +  /**
    +   * total count of partitions
    +   */
    +  private int numberOfPartitions;
    +
    +  /**
    +   * task id of max partition
    +   */
    +  private int MAX_PARTITION;
    +
    +  /**
    +   * record the task id in partition order
    --- End diff --
    
    please correct comment


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

[GitHub] carbondata issue #1125: [CarbonData-1250] Change default partition id & Add ...

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

    https://github.com/apache/carbondata/pull/1125
  
    Build Failed with Spark 1.6, Please check CI http://144.76.159.231:8080/job/ApacheCarbonPRBuilder/359/



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

[GitHub] carbondata pull request #1125: [CarbonData-1250] Change default partition id...

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

    https://github.com/apache/carbondata/pull/1125#discussion_r125954219
  
    --- Diff: core/src/main/java/org/apache/carbondata/core/metadata/schema/PartitionInfo.java ---
    @@ -43,13 +44,30 @@
       private List<List<String>> listInfo;
     
       /**
    -   * number of partitions
    +   * number of hash partitions
        */
    -  private int numPartitions;
    +  private int hashNumber;
    +
    +  /**
    +   * total count of partitions
    +   */
    +  private int numberOfPartitions;
    --- End diff --
    
    for hash partition,  hashNumber is same with numberOfPartitions.
    why we need two variables?


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

[GitHub] carbondata issue #1125: [CARBONDATA-1250] Change default partition id & Add ...

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

    https://github.com/apache/carbondata/pull/1125
  
    Local Verification pass
    LGTM


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

[GitHub] carbondata pull request #1125: [CARBONDATA-1250] Change default partition id...

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

    https://github.com/apache/carbondata/pull/1125#discussion_r126878337
  
    --- Diff: integration/spark-common/src/main/scala/org/apache/carbondata/spark/util/CommonUtil.scala ---
    @@ -54,6 +53,12 @@ object CommonUtil {
     
       val FIXED_DECIMAL = """decimal\(\s*(\d+)\s*,\s*(\-?\d+)\s*\)""".r
       val FIXED_DECIMALTYPE = """decimaltype\(\s*(\d+)\s*,\s*(\-?\d+)\s*\)""".r
    +  val timestampFormatter = new SimpleDateFormat(CarbonProperties.getInstance
    --- End diff --
    
    SimpleDateFormat is not thread safe. 
    In mutiple threads, it will return wrong value.


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

[GitHub] carbondata issue #1125: [CarbonData-1250] change default partition id & Add ...

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

    https://github.com/apache/carbondata/pull/1125
  
    Build Failed with Spark 1.6, Please check CI http://144.76.159.231:8080/job/ApacheCarbonPRBuilder/295/



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

[GitHub] carbondata pull request #1125: [CarbonData-1250] change default partition id...

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

    https://github.com/apache/carbondata/pull/1125#discussion_r125304887
  
    --- Diff: core/src/main/java/org/apache/carbondata/core/scan/filter/partition/PartitionFilterUtil.java ---
    @@ -229,35 +229,37 @@ public static BitSet getPartitionMapForRangeFilter(PartitionInfo partitionInfo,
           // filter value is in default partition
           if (isGreaterThan) {
             // GreaterThan(>), GreaterThanEqualTo(>=)
    -        partitionMap.set(numPartitions);
    +        partitionMap.set(0);
           } else {
             // LessThan(<), LessThanEqualTo(<=)
    -        partitionMap.set(0, partitioner.numPartitions());
    +        partitionMap.set(1, partitioner.numPartitions());
           }
         } else {
           // filter value is not in default partition
           if (result == 0) {
             // if result is 0, the filter value is a bound value of range partition.
             if (isGreaterThan) {
               // GreaterThan(>), GreaterThanEqualTo(>=)
    -          partitionMap.set(partitionIndex + 1, partitioner.numPartitions());
    +          partitionMap.set(partitionIndex + 1, partitioner.numPartitions() - 1);
    --- End diff --
    
    partitionMap.set(partitionIndex + 2, partitioner.numPartitions() - 1);


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

[GitHub] carbondata pull request #1125: [CARBONDATA-1250] Change default partition id...

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

    https://github.com/apache/carbondata/pull/1125#discussion_r126876084
  
    --- Diff: format/src/main/thrift/schema.thrift ---
    @@ -132,9 +132,12 @@ struct SchemaEvolution{
     struct PartitionInfo{
         1: required list<ColumnSchema> partition_columns;
         2: required PartitionType partition_type;
    -    3: optional i32 num_partitions;  // number of partitions defined in hash partition table
    +    3: optional i32 hash_number;  // number of partitions defined in hash partition table
         4: optional list<list<string>> list_info; // value list of list partition table
         5: optional list<string> range_info;  // range value list of range partition table
    +    6: optional list<i32> partitionIdList; // partition Id list
    --- End diff --
    
    change the name to partition_ids


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

[GitHub] carbondata issue #1125: [CARBONDATA-1250] Change default partition id & Add ...

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

    https://github.com/apache/carbondata/pull/1125
  
    local verify passed for spark1.6 and spark2.1


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

[GitHub] carbondata issue #1125: [CarbonData-1250] Change default partition id & Add ...

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

    https://github.com/apache/carbondata/pull/1125
  
    Build Failed  with Spark 2.1.0, Please check CI http://136.243.101.176:8080/job/ApacheCarbonPRBuilder/2901/



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

[GitHub] carbondata issue #1125: [CarbonData-1250] Change default partition id & Add ...

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

    https://github.com/apache/carbondata/pull/1125
  
    Build Failed  with Spark 2.1.0, Please check CI http://136.243.101.176:8080/job/ApacheCarbonPRBuilder/2949/



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

[GitHub] carbondata pull request #1125: [CarbonData-1250] Change default partition id...

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

    https://github.com/apache/carbondata/pull/1125#discussion_r125428800
  
    --- Diff: format/src/main/thrift/schema.thrift ---
    @@ -135,6 +135,9 @@ struct PartitionInfo{
         3: optional i32 num_partitions;  // number of partitions defined in hash partition table
    --- End diff --
    
    Same as Hash_num_parititions


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

[GitHub] carbondata issue #1125: [CarbonData-1250] Change default partition id & Add ...

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

    https://github.com/apache/carbondata/pull/1125
  
    Build Failed  with Spark 2.1.0, Please check CI http://136.243.101.176:8080/job/ApacheCarbonPRBuilder/2935/



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

[GitHub] carbondata issue #1125: [CarbonData-1250] change default partition id & Add ...

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

    https://github.com/apache/carbondata/pull/1125
  
    Can one of the admins verify this patch?


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

[GitHub] carbondata pull request #1125: [CarbonData-1250] Change default partition id...

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

    https://github.com/apache/carbondata/pull/1125#discussion_r126062119
  
    --- Diff: core/src/main/java/org/apache/carbondata/core/metadata/schema/PartitionInfo.java ---
    @@ -43,13 +44,30 @@
       private List<List<String>> listInfo;
     
       /**
    -   * number of partitions
    +   * number of hash partitions
        */
    -  private int numPartitions;
    +  private int hashNumber;
    +
    +  /**
    +   * total count of partitions
    +   */
    +  private int numberOfPartitions;
    +
    +  /**
    +   * task id of max partition
    +   */
    +  private int MAX_PARTITION;
    --- End diff --
    
    MAX_PARTITION will be used in alter table partition. it's increased only. 


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

[GitHub] carbondata issue #1125: [CarbonData-1250] change default partition id & Add ...

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

    https://github.com/apache/carbondata/pull/1125
  
    Build Failed  with Spark 2.1.0, Please check CI http://136.243.101.176:8080/job/ApacheCarbonPRBuilder/2887/



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

[GitHub] carbondata issue #1125: [CARBONDATA-1250] Change default partition id & Add ...

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

    https://github.com/apache/carbondata/pull/1125
  
    Build Failed with Spark 1.6, Please check CI http://144.76.159.231:8080/job/ApacheCarbonPRBuilder/454/



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

[GitHub] carbondata pull request #1125: [CarbonData-1250] change default partition id...

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

    https://github.com/apache/carbondata/pull/1125#discussion_r125319466
  
    --- Diff: core/src/main/java/org/apache/carbondata/core/scan/partition/RangePartitioner.java ---
    @@ -17,16 +17,17 @@
     
     package org.apache.carbondata.core.scan.partition;
     
    -import java.io.Serializable;
    -import java.math.BigDecimal;
     import java.text.SimpleDateFormat;
     import java.util.List;
     
     import org.apache.carbondata.core.constants.CarbonCommonConstants;
     import org.apache.carbondata.core.metadata.datatype.DataType;
     import org.apache.carbondata.core.metadata.schema.PartitionInfo;
     import org.apache.carbondata.core.util.ByteUtil;
    +
    --- End diff --
    
    DELETE


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

[GitHub] carbondata issue #1125: [CarbonData-1250] Change default partition id & Add ...

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

    https://github.com/apache/carbondata/pull/1125
  
    Build Failed with Spark 1.6, Please check CI http://144.76.159.231:8080/job/ApacheCarbonPRBuilder/386/



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

[GitHub] carbondata issue #1125: [CarbonData-1250] Change default partition id & Add ...

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

    https://github.com/apache/carbondata/pull/1125
  
    Build Failed  with Spark 2.1.0, Please check CI http://136.243.101.176:8080/job/ApacheCarbonPRBuilder/3010/



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

[GitHub] carbondata issue #1125: [CarbonData-1250] change default partition id & Add ...

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

    https://github.com/apache/carbondata/pull/1125
  
    Build Failed with Spark 1.6, Please check CI http://144.76.159.231:8080/job/ApacheCarbonPRBuilder/287/



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

[GitHub] carbondata pull request #1125: [CARBONDATA-1250] Change default partition id...

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

    https://github.com/apache/carbondata/pull/1125#discussion_r126873790
  
    --- Diff: core/src/main/java/org/apache/carbondata/core/metadata/converter/ThriftWrapperSchemaConverterImpl.java ---
    @@ -218,7 +218,11 @@
                 fromWrapperToExternalPartitionType(wrapperPartitionInfo.getPartitionType()));
         externalPartitionInfo.setList_info(wrapperPartitionInfo.getListInfo());
         externalPartitionInfo.setRange_info(wrapperPartitionInfo.getRangeInfo());
    -    externalPartitionInfo.setNum_partitions(wrapperPartitionInfo.getNumPartitions());
    +    externalPartitionInfo.setHash_number(wrapperPartitionInfo.getHashNumber());
    +    externalPartitionInfo.setNumOfPartitions(wrapperPartitionInfo.getNumberOfPartitions());
    +    externalPartitionInfo.setMax_partition(wrapperPartitionInfo.getMAX_PARTITION());
    --- End diff --
    
    how the user set Max_partition?


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

[GitHub] carbondata pull request #1125: [CarbonData-1250] Change default partition id...

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

    https://github.com/apache/carbondata/pull/1125#discussion_r125427240
  
    --- Diff: core/src/main/java/org/apache/carbondata/core/metadata/converter/ThriftWrapperSchemaConverterImpl.java ---
    @@ -219,6 +219,10 @@
         externalPartitionInfo.setList_info(wrapperPartitionInfo.getListInfo());
         externalPartitionInfo.setRange_info(wrapperPartitionInfo.getRangeInfo());
         externalPartitionInfo.setNum_partitions(wrapperPartitionInfo.getNumPartitions());
    +    externalPartitionInfo.setNumOfPartitions(wrapperPartitionInfo.getNumberOfPartitions());
    --- End diff --
    
    I think it may be better that use Hash_numPartition. otherwise users may confused about this two num of partitions.


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

[GitHub] carbondata issue #1125: [CarbonData-1250] change default partition id & Add ...

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

    https://github.com/apache/carbondata/pull/1125
  
    Build Failed  with Spark 2.1.0, Please check CI http://136.243.101.176:8080/job/ApacheCarbonPRBuilder/2870/



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

[GitHub] carbondata pull request #1125: [CarbonData-1250] Change default partition id...

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

    https://github.com/apache/carbondata/pull/1125#discussion_r125955474
  
    --- Diff: core/src/main/java/org/apache/carbondata/core/scan/filter/partition/PartitionFilterUtil.java ---
    @@ -229,35 +229,37 @@ public static BitSet getPartitionMapForRangeFilter(PartitionInfo partitionInfo,
           // filter value is in default partition
           if (isGreaterThan) {
             // GreaterThan(>), GreaterThanEqualTo(>=)
    -        partitionMap.set(numPartitions);
    +        partitionMap.set(0);
           } else {
             // LessThan(<), LessThanEqualTo(<=)
    -        partitionMap.set(0, partitioner.numPartitions());
    +        partitionMap.set(0, partitioner.numPartitions() - 1);
           }
         } else {
           // filter value is not in default partition
           if (result == 0) {
             // if result is 0, the filter value is a bound value of range partition.
             if (isGreaterThan) {
               // GreaterThan(>), GreaterThanEqualTo(>=)
    -          partitionMap.set(partitionIndex + 1, partitioner.numPartitions());
    +          partitionMap.set(partitionIndex + 2, partitioner.numPartitions() - 1);
    +          partitionMap.set(0);
             } else {
               if (isEqualTo) {
                 // LessThanEqualTo(<=)
    -            partitionMap.set(0, partitionIndex + 2);
    +            partitionMap.set(1, partitionIndex + 3);
               } else {
                 // LessThan(<)
    -            partitionMap.set(0, partitionIndex + 1);
    +            partitionMap.set(1, partitionIndex + 2);
               }
             }
           } else {
             // the filter value is not a bound value of range partition
             if (isGreaterThan) {
               // GreaterThan(>), GreaterThanEqualTo(>=)
    -          partitionMap.set(partitionIndex, partitioner.numPartitions());
    +          partitionMap.set(partitionIndex + 1, partitioner.numPartitions() - 1);
    --- End diff --
    
    why we lost 2 partitions


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

[GitHub] carbondata pull request #1125: [CarbonData-1250] Change default partition id...

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

    https://github.com/apache/carbondata/pull/1125#discussion_r125954460
  
    --- Diff: core/src/main/java/org/apache/carbondata/core/metadata/schema/PartitionInfo.java ---
    @@ -43,13 +44,30 @@
       private List<List<String>> listInfo;
     
       /**
    -   * number of partitions
    +   * number of hash partitions
        */
    -  private int numPartitions;
    +  private int hashNumber;
    +
    +  /**
    +   * total count of partitions
    +   */
    +  private int numberOfPartitions;
    +
    +  /**
    +   * task id of max partition
    +   */
    +  private int MAX_PARTITION;
    --- End diff --
    
    why need to save MAX_PARTITION to schema?
    please correct comment.


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

[GitHub] carbondata issue #1125: [CarbonData-1250] Change default partition id & Add ...

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

    https://github.com/apache/carbondata/pull/1125
  
    Build Failed with Spark 1.6, Please check CI http://144.76.159.231:8080/job/ApacheCarbonPRBuilder/303/



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

[GitHub] carbondata pull request #1125: [CARBONDATA-1250] Change default partition id...

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

    https://github.com/apache/carbondata/pull/1125#discussion_r126876119
  
    --- Diff: format/src/main/thrift/schema.thrift ---
    @@ -132,9 +132,12 @@ struct SchemaEvolution{
     struct PartitionInfo{
         1: required list<ColumnSchema> partition_columns;
         2: required PartitionType partition_type;
    -    3: optional i32 num_partitions;  // number of partitions defined in hash partition table
    +    3: optional i32 hash_number;  // number of partitions defined in hash partition table
         4: optional list<list<string>> list_info; // value list of list partition table
         5: optional list<string> range_info;  // range value list of range partition table
    +    6: optional list<i32> partitionIdList; // partition Id list
    +    7: optional i32 numOfPartitions;  // total partition count
    --- End diff --
    
    remove it


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

[GitHub] carbondata pull request #1125: [CARBONDATA-1250] Change default partition id...

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

    https://github.com/apache/carbondata/pull/1125#discussion_r126875952
  
    --- Diff: format/src/main/thrift/schema.thrift ---
    @@ -132,9 +132,12 @@ struct SchemaEvolution{
     struct PartitionInfo{
         1: required list<ColumnSchema> partition_columns;
         2: required PartitionType partition_type;
    -    3: optional i32 num_partitions;  // number of partitions defined in hash partition table
    +    3: optional i32 hash_number;  // number of partitions defined in hash partition table
    --- End diff --
    
    please keep num_partitions


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

[GitHub] carbondata issue #1125: [CarbonData-1250] Change default partition id & Add ...

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

    https://github.com/apache/carbondata/pull/1125
  
    Build Failed with Spark 1.6, Please check CI http://144.76.159.231:8080/job/ApacheCarbonPRBuilder/352/



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

[GitHub] carbondata issue #1125: [CarbonData-1250] Change default partition id & Add ...

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

    https://github.com/apache/carbondata/pull/1125
  
    Build Failed with Spark 1.6, Please check CI http://144.76.159.231:8080/job/ApacheCarbonPRBuilder/349/



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

[GitHub] carbondata pull request #1125: [CarbonData-1250] Change default partition id...

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

    https://github.com/apache/carbondata/pull/1125#discussion_r125958721
  
    --- Diff: hadoop/src/main/java/org/apache/carbondata/hadoop/CarbonInputFormat.java ---
    @@ -417,18 +441,26 @@ protected Expression getFilterPredicates(Configuration configuration) {
               getSegmentAbstractIndexs(job, absoluteTableIdentifier, segmentId, cacheClient,
                   updateStatusManager);
           List<DataRefNode> resultFilterredBlocks = new LinkedList<DataRefNode>();
    +      int partitionIndex = -1;
    +      List<Integer> taskIdGroup = new ArrayList<>();
    +      if (partitionInfo != null) {
    +        taskIdGroup = partitionInfo.getTaskIdInPartitionOrder();
    --- End diff --
    
    what is task id?


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

[GitHub] carbondata pull request #1125: [CarbonData-1250] Change default partition id...

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

    https://github.com/apache/carbondata/pull/1125#discussion_r125957306
  
    --- Diff: hadoop/src/main/java/org/apache/carbondata/hadoop/CarbonInputFormat.java ---
    @@ -274,28 +275,27 @@ private static AbsoluteTableIdentifier getAbsoluteTableIdentifier(Configuration
           CarbonInputFormatUtil.processFilterExpression(filter, carbonTable);
     
           // prune partitions for filter query on partition table
    -      BitSet matchedPartitions = null;
    -      if (null != filter) {
    -        PartitionInfo partitionInfo = carbonTable.getPartitionInfo(carbonTable.getFactTableName());
    -        if (null != partitionInfo) {
    -          Partitioner partitioner = PartitionUtil.getPartitioner(partitionInfo);
    -          matchedPartitions = new FilterExpressionProcessor()
    -              .getFilteredPartitions(filter, partitionInfo, partitioner);
    +      BitSet matchedPartitions = setMatchedPartitions(null, carbonTable, filter);
    +      if (partitionInfo != null) {
    +        Partitioner partitioner = PartitionUtil.getPartitioner(partitionInfo);
    +        if (matchedPartitions != null) {
               if (matchedPartitions.cardinality() == 0) {
                 // no partition is required
                 return new ArrayList<InputSplit>();
    -          }
    -          if (matchedPartitions.cardinality() == partitioner.numPartitions()) {
    +          } else if (matchedPartitions.cardinality() == partitioner.numPartitions()) {
                 // all partitions are required, no need to prune partitions
                 matchedPartitions = null;
               }
             }
           }
     
    +
    +
    --- End diff --
    
    remove this line


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

[GitHub] carbondata issue #1125: [CarbonData-1250] Change default partition id & Add ...

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

    https://github.com/apache/carbondata/pull/1125
  
    Build Failed with Spark 1.6, Please check CI http://144.76.159.231:8080/job/ApacheCarbonPRBuilder/397/



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

[GitHub] carbondata pull request #1125: [CarbonData-1250] Change default partition id...

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

    https://github.com/apache/carbondata/pull/1125#discussion_r126062042
  
    --- Diff: core/src/main/java/org/apache/carbondata/core/metadata/schema/PartitionInfo.java ---
    @@ -43,13 +44,30 @@
       private List<List<String>> listInfo;
     
       /**
    -   * number of partitions
    +   * number of hash partitions
        */
    -  private int numPartitions;
    +  private int hashNumber;
    +
    +  /**
    +   * total count of partitions
    +   */
    +  private int numberOfPartitions;
    --- End diff --
    
    I think it represent different concept. Though they are equal in hash table. That would be more clear for developers.


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

[GitHub] carbondata pull request #1125: [CarbonData-1250] Change default partition id...

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

    https://github.com/apache/carbondata/pull/1125#discussion_r125960197
  
    --- Diff: hadoop/src/main/java/org/apache/carbondata/hadoop/CarbonInputFormat.java ---
    @@ -327,6 +327,29 @@ private static AbsoluteTableIdentifier getAbsoluteTableIdentifier(Configuration
         return carbonSplits;
       }
     
    +  private BitSet setMatchedPartitions(String partitionIds, CarbonTable carbonTable,
    +      Expression filter) {
    +    BitSet matchedPartitions = null;
    +    if (null != partitionIds) {
    +      String[] partList = partitionIds.replace("[","").replace("]","").split(",");
    +      matchedPartitions = new BitSet(Integer.parseInt(partList[0]));
    +      for (String partitionId : partList) {
    +        matchedPartitions.set(Integer.parseInt(partitionId));
    +      }
    +    } else {
    +      if (null != filter) {
    +        PartitionInfo partitionInfo =
    +            carbonTable.getPartitionInfo(carbonTable.getFactTableName());
    +        if (null != partitionInfo) {
    +          Partitioner partitioner = PartitionUtil.getPartitioner(partitionInfo);
    +          matchedPartitions = new FilterExpressionProcessor()
    +              .getFilteredPartitions(filter, partitionInfo, partitioner);
    --- End diff --
    
    I think we need modify getFilteredPartitions and FilterExpression to use partition id instead of array index.


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

[GitHub] carbondata issue #1125: [CarbonData-1250] Change default partition id & Add ...

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

    https://github.com/apache/carbondata/pull/1125
  
    Build Failed  with Spark 2.1.0, Please check CI http://136.243.101.176:8080/job/ApacheCarbonPRBuilder/2946/



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

[GitHub] carbondata issue #1125: [CarbonData-1250] Change default partition id & Add ...

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

    https://github.com/apache/carbondata/pull/1125
  
    Build Failed with Spark 1.6, Please check CI http://144.76.159.231:8080/job/ApacheCarbonPRBuilder/422/



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

[GitHub] carbondata issue #1125: [CarbonData-1250] Change default partition id & Add ...

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

    https://github.com/apache/carbondata/pull/1125
  
    Build Failed  with Spark 2.1.0, Please check CI http://136.243.101.176:8080/job/ApacheCarbonPRBuilder/2945/



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

[GitHub] carbondata issue #1125: [CarbonData-1250] Change default partition id & Add ...

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

    https://github.com/apache/carbondata/pull/1125
  
    Build Failed with Spark 1.6, Please check CI http://144.76.159.231:8080/job/ApacheCarbonPRBuilder/351/



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

[GitHub] carbondata pull request #1125: [CarbonData-1250] Change default partition id...

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

    https://github.com/apache/carbondata/pull/1125#discussion_r125427623
  
    --- Diff: core/src/main/java/org/apache/carbondata/core/metadata/schema/table/CarbonTable.java ---
    @@ -99,10 +98,6 @@
       private Map<String, PartitionInfo> tablePartitionMap;
     
       /**
    -   * statistic information of partition table
    -   */
    -  private PartitionStatistic partitionStatistic;
    -  /**
    --- End diff --
    
    Should keep this line.


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

[GitHub] carbondata pull request #1125: [CARBONDATA-1250] Change default partition id...

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

    https://github.com/apache/carbondata/pull/1125#discussion_r126876163
  
    --- Diff: format/src/main/thrift/schema.thrift ---
    @@ -132,9 +132,12 @@ struct SchemaEvolution{
     struct PartitionInfo{
         1: required list<ColumnSchema> partition_columns;
         2: required PartitionType partition_type;
    -    3: optional i32 num_partitions;  // number of partitions defined in hash partition table
    +    3: optional i32 hash_number;  // number of partitions defined in hash partition table
         4: optional list<list<string>> list_info; // value list of list partition table
         5: optional list<string> range_info;  // range value list of range partition table
    +    6: optional list<i32> partitionIdList; // partition Id list
    +    7: optional i32 numOfPartitions;  // total partition count
    +    8: optional i32 max_partition;  // max task id for now
    --- End diff --
    
    what is it?


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

[GitHub] carbondata issue #1125: [CarbonData-1250] Change default partition id & Add ...

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

    https://github.com/apache/carbondata/pull/1125
  
    Build Failed with Spark 1.6, Please check CI http://144.76.159.231:8080/job/ApacheCarbonPRBuilder/362/



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

[GitHub] carbondata pull request #1125: [CarbonData-1250] Change default partition id...

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

    https://github.com/apache/carbondata/pull/1125#discussion_r125955225
  
    --- Diff: core/src/main/java/org/apache/carbondata/core/scan/filter/partition/PartitionFilterUtil.java ---
    @@ -229,35 +229,37 @@ public static BitSet getPartitionMapForRangeFilter(PartitionInfo partitionInfo,
           // filter value is in default partition
           if (isGreaterThan) {
             // GreaterThan(>), GreaterThanEqualTo(>=)
    -        partitionMap.set(numPartitions);
    +        partitionMap.set(0);
           } else {
             // LessThan(<), LessThanEqualTo(<=)
    -        partitionMap.set(0, partitioner.numPartitions());
    +        partitionMap.set(0, partitioner.numPartitions() - 1);
           }
         } else {
           // filter value is not in default partition
           if (result == 0) {
             // if result is 0, the filter value is a bound value of range partition.
             if (isGreaterThan) {
               // GreaterThan(>), GreaterThanEqualTo(>=)
    -          partitionMap.set(partitionIndex + 1, partitioner.numPartitions());
    +          partitionMap.set(partitionIndex + 2, partitioner.numPartitions() - 1);
    --- End diff --
    
    why we lost two partition?


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

[GitHub] carbondata issue #1125: [CarbonData-1250] Change default partition id & Add ...

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

    https://github.com/apache/carbondata/pull/1125
  
    Build Failed with Spark 1.6, Please check CI http://144.76.159.231:8080/job/ApacheCarbonPRBuilder/315/



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

[GitHub] carbondata pull request #1125: [CarbonData-1250] Change default partition id...

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

    https://github.com/apache/carbondata/pull/1125#discussion_r126061892
  
    --- Diff: hadoop/src/main/java/org/apache/carbondata/hadoop/CarbonInputFormat.java ---
    @@ -417,18 +441,26 @@ protected Expression getFilterPredicates(Configuration configuration) {
               getSegmentAbstractIndexs(job, absoluteTableIdentifier, segmentId, cacheClient,
                   updateStatusManager);
           List<DataRefNode> resultFilterredBlocks = new LinkedList<DataRefNode>();
    +      int partitionIndex = -1;
    +      List<Integer> taskIdGroup = new ArrayList<>();
    +      if (partitionInfo != null) {
    +        taskIdGroup = partitionInfo.getTaskIdInPartitionOrder();
    --- End diff --
    
    Fixed.  it was missed when I change taskIdGroup to partitionIdList


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

[GitHub] carbondata issue #1125: [CarbonData-1250] Change default partition id & Add ...

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

    https://github.com/apache/carbondata/pull/1125
  
    Build Failed with Spark 1.6, Please check CI http://144.76.159.231:8080/job/ApacheCarbonPRBuilder/399/



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

[GitHub] carbondata issue #1125: [CarbonData-1250] Change default partition id & Add ...

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

    https://github.com/apache/carbondata/pull/1125
  
    retest this please


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

[GitHub] carbondata issue #1125: [CARBONDATA-1250] Change default partition id & Add ...

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

    https://github.com/apache/carbondata/pull/1125
  
    Build Failed  with Spark 2.1.0, Please check CI http://136.243.101.176:8080/job/ApacheCarbonPRBuilder/3044/



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

[GitHub] carbondata pull request #1125: [CARBONDATA-1250] Change default partition id...

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

    https://github.com/apache/carbondata/pull/1125#discussion_r126874942
  
    --- Diff: core/src/main/java/org/apache/carbondata/core/metadata/schema/PartitionInfo.java ---
    @@ -43,13 +44,30 @@
       private List<List<String>> listInfo;
     
       /**
    -   * number of partitions
    +   * number of hash partitions
        */
    -  private int numPartitions;
    +  private int hashNumber;
    +
    +  /**
    +   * total count of partitions
    +   */
    +  private int numberOfPartitions;
    --- End diff --
    
    not require hashNumber, numberOfPartitions is enough.


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

[GitHub] carbondata pull request #1125: [CarbonData-1250] change default partition id...

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

    https://github.com/apache/carbondata/pull/1125#discussion_r125301153
  
    --- Diff: core/src/main/java/org/apache/carbondata/core/metadata/converter/ThriftWrapperSchemaConverterImpl.java ---
    @@ -219,6 +219,10 @@
         externalPartitionInfo.setList_info(wrapperPartitionInfo.getListInfo());
         externalPartitionInfo.setRange_info(wrapperPartitionInfo.getRangeInfo());
         externalPartitionInfo.setNum_partitions(wrapperPartitionInfo.getNumPartitions());
    +    externalPartitionInfo.setNumOfPartitions(wrapperPartitionInfo.getNumberOfPartitions());
    --- End diff --
    
    NumPartitions is definition for Hash partition table.
    NumOfPartitions is total partition count of partition table(according to schema, not fact data), it's defined by the length of range info/list info and hash partition number.


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

[GitHub] carbondata issue #1125: [CARBONDATA-1250] Change default partition id & Add ...

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

    https://github.com/apache/carbondata/pull/1125
  
    Build Failed  with Spark 2.1.0, Please check CI http://136.243.101.176:8080/job/ApacheCarbonPRBuilder/3042/



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

[GitHub] carbondata pull request #1125: [CarbonData-1250] Change default partition id...

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

    https://github.com/apache/carbondata/pull/1125#discussion_r125954839
  
    --- Diff: core/src/main/java/org/apache/carbondata/core/scan/filter/partition/PartitionFilterUtil.java ---
    @@ -229,35 +229,37 @@ public static BitSet getPartitionMapForRangeFilter(PartitionInfo partitionInfo,
           // filter value is in default partition
           if (isGreaterThan) {
             // GreaterThan(>), GreaterThanEqualTo(>=)
    -        partitionMap.set(numPartitions);
    +        partitionMap.set(0);
           } else {
             // LessThan(<), LessThanEqualTo(<=)
    -        partitionMap.set(0, partitioner.numPartitions());
    +        partitionMap.set(0, partitioner.numPartitions() - 1);
    --- End diff --
    
    why we lost a partition?


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

[GitHub] carbondata pull request #1125: [CARBONDATA-1250] Change default partition id...

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

    https://github.com/apache/carbondata/pull/1125#discussion_r126876373
  
    --- Diff: hadoop/src/main/java/org/apache/carbondata/hadoop/CarbonInputFormat.java ---
    @@ -266,26 +264,22 @@ private static AbsoluteTableIdentifier getAbsoluteTableIdentifier(Configuration
           // process and resolve the expression
           Expression filter = getFilterPredicates(job.getConfiguration());
           CarbonTable carbonTable = getCarbonTable(job.getConfiguration());
    +      PartitionInfo partitionInfo = carbonTable.getPartitionInfo(carbonTable.getFactTableName());
    --- End diff --
    
    please move code  to 274 line.
    Variable carbonTable should be checked before it was used.


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

[GitHub] carbondata issue #1125: [CarbonData-1250] Change default partition id & Add ...

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

    https://github.com/apache/carbondata/pull/1125
  
    Build Failed  with Spark 2.1.0, Please check CI http://136.243.101.176:8080/job/ApacheCarbonPRBuilder/2988/



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

[GitHub] carbondata issue #1125: [CarbonData-1250] Change default partition id & Add ...

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

    https://github.com/apache/carbondata/pull/1125
  
    Build Failed  with Spark 2.1.0, Please check CI http://136.243.101.176:8080/job/ApacheCarbonPRBuilder/2975/



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

[GitHub] carbondata pull request #1125: [CarbonData-1250] Change default partition id...

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

    https://github.com/apache/carbondata/pull/1125#discussion_r125957701
  
    --- Diff: hadoop/src/main/java/org/apache/carbondata/hadoop/CarbonInputFormat.java ---
    @@ -274,28 +275,27 @@ private static AbsoluteTableIdentifier getAbsoluteTableIdentifier(Configuration
           CarbonInputFormatUtil.processFilterExpression(filter, carbonTable);
     
           // prune partitions for filter query on partition table
    -      BitSet matchedPartitions = null;
    -      if (null != filter) {
    -        PartitionInfo partitionInfo = carbonTable.getPartitionInfo(carbonTable.getFactTableName());
    -        if (null != partitionInfo) {
    -          Partitioner partitioner = PartitionUtil.getPartitioner(partitionInfo);
    -          matchedPartitions = new FilterExpressionProcessor()
    -              .getFilteredPartitions(filter, partitionInfo, partitioner);
    +      BitSet matchedPartitions = setMatchedPartitions(null, carbonTable, filter);
    +      if (partitionInfo != null) {
    +        Partitioner partitioner = PartitionUtil.getPartitioner(partitionInfo);
    --- End diff --
    
    this method was invoked twice


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

[GitHub] carbondata issue #1125: [CarbonData-1250] Change default partition id & Add ...

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

    https://github.com/apache/carbondata/pull/1125
  
    Build Failed  with Spark 2.1.0, Please check CI http://136.243.101.176:8080/job/ApacheCarbonPRBuilder/2986/



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

[GitHub] carbondata issue #1125: [CarbonData-1250] change default partition id & Add ...

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

    https://github.com/apache/carbondata/pull/1125
  
    Build Failed  with Spark 2.1.0, Please check CI http://136.243.101.176:8080/job/ApacheCarbonPRBuilder/2879/



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

[GitHub] carbondata issue #1125: [CarbonData-1250] Change default partition id & Add ...

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

    https://github.com/apache/carbondata/pull/1125
  
    Build Failed  with Spark 2.1.0, Please check CI http://136.243.101.176:8080/job/ApacheCarbonPRBuilder/2939/



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

[GitHub] carbondata issue #1125: [CarbonData-1250] Change default partition id & Add ...

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

    https://github.com/apache/carbondata/pull/1125
  
    Build Failed with Spark 1.6, Please check CI http://144.76.159.231:8080/job/ApacheCarbonPRBuilder/358/



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

[GitHub] carbondata pull request #1125: [CarbonData-1250] change default partition id...

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

    https://github.com/apache/carbondata/pull/1125#discussion_r125301950
  
    --- Diff: core/src/main/java/org/apache/carbondata/core/metadata/converter/ThriftWrapperSchemaConverterImpl.java ---
    @@ -219,6 +219,10 @@
         externalPartitionInfo.setList_info(wrapperPartitionInfo.getListInfo());
         externalPartitionInfo.setRange_info(wrapperPartitionInfo.getRangeInfo());
         externalPartitionInfo.setNum_partitions(wrapperPartitionInfo.getNumPartitions());
    +    externalPartitionInfo.setNumOfPartitions(wrapperPartitionInfo.getNumberOfPartitions());
    +    externalPartitionInfo.setMax_partition(wrapperPartitionInfo.getMAX_PARTITION());
    +    externalPartitionInfo.setTaskId_In_PartitionOrder(wrapperPartitionInfo
    --- End diff --
    
    TaskIdInPartitionOrder is an array used to ensure taskId could mapping to correct partition after alter table partition.
    It will be used when alter partition table. (The operation method will be uploaded with alter table PR)


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

[GitHub] carbondata issue #1125: [CarbonData-1250] change default partition id & Add ...

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

    https://github.com/apache/carbondata/pull/1125
  
    Build Failed  with Spark 2.1.0, Please check CI http://136.243.101.176:8080/job/ApacheCarbonPRBuilder/2886/



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

[GitHub] carbondata pull request #1125: [CARBONDATA-1250] Change default partition id...

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

    https://github.com/apache/carbondata/pull/1125#discussion_r126873687
  
    --- Diff: core/src/main/java/org/apache/carbondata/core/metadata/converter/ThriftWrapperSchemaConverterImpl.java ---
    @@ -218,7 +218,11 @@
                 fromWrapperToExternalPartitionType(wrapperPartitionInfo.getPartitionType()));
         externalPartitionInfo.setList_info(wrapperPartitionInfo.getListInfo());
         externalPartitionInfo.setRange_info(wrapperPartitionInfo.getRangeInfo());
    -    externalPartitionInfo.setNum_partitions(wrapperPartitionInfo.getNumPartitions());
    +    externalPartitionInfo.setHash_number(wrapperPartitionInfo.getHashNumber());
    +    externalPartitionInfo.setNumOfPartitions(wrapperPartitionInfo.getNumberOfPartitions());
    --- End diff --
    
    just need NumOfPartitions, not require hash_number


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

[GitHub] carbondata pull request #1125: [CARBONDATA-1250] Change default partition id...

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

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


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

[GitHub] carbondata issue #1125: [CARBONDATA-1250] Change default partition id & Add ...

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

    https://github.com/apache/carbondata/pull/1125
  
    Build Failed with Spark 1.6, Please check CI http://144.76.159.231:8080/job/ApacheCarbonPRBuilder/457/



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

[GitHub] carbondata pull request #1125: [CarbonData-1250] Change default partition id...

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

    https://github.com/apache/carbondata/pull/1125#discussion_r125956098
  
    --- Diff: core/src/main/java/org/apache/carbondata/core/util/comparator/Comparator.java ---
    @@ -0,0 +1,89 @@
    +/*
    + * 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.core.util.comparator;
    +
    +import java.math.BigDecimal;
    +
    +import org.apache.carbondata.core.metadata.datatype.DataType;
    +import org.apache.carbondata.core.util.ByteUtil;
    +
    +public final class Comparator {
    +
    +  private static SerializableComparator comparator;
    --- End diff --
    
    no need


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

[GitHub] carbondata issue #1125: [CarbonData-1250] change default partition id & Add ...

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

    https://github.com/apache/carbondata/pull/1125
  
    Build Failed with Spark 1.6, Please check CI http://144.76.159.231:8080/job/ApacheCarbonPRBuilder/302/



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