You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@carbondata.apache.org by anubhav100 <gi...@git.apache.org> on 2018/04/04 09:44:18 UTC

[GitHub] carbondata pull request #2139: [CARBONDATA-2267] [Presto] Support Reading Ca...

GitHub user anubhav100 opened a pull request:

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

    [CARBONDATA-2267] [Presto] Support Reading CarbonData Partition From Presto Integration

    Be sure to do all of the following checklist to help us incorporate 
    your contribution quickly and easily:
    
     - [ ] Any interfaces changed?
     
     - [ ] Any backward compatibility impacted?
     
     - [ ] Document update required?
    
     - [ ] Testing done
            Please provide details on 
            - Whether new unit test cases have been added or why no new tests are required?
            - How it is tested? Please attach test report.
            - Is it a performance related change? Please attach the performance test report.
            - Any additional information to help reviewers in testing this change.
           
     - [ ] For large changes, please consider breaking it into sub-tasks under an umbrella JIRA. 
    


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

    $ git pull https://github.com/anubhav100/incubator-carbondata prestopartition

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

    https://github.com/apache/carbondata/pull/2139.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 #2139
    
----
commit 9a8cf766e97882a3671cba8d566489f5918cc948
Author: anubhav100 <an...@...>
Date:   2018-04-04T08:18:15Z

    added logic for fetching the partition columns in presto

commit 3faf8e0e32e175e89b43edaceef23da0a03927b6
Author: Geetika Gupta <ge...@...>
Date:   2018-04-04T08:37:36Z

    Refactored CarbonTableReader to add partition spec to configuration object

----


---

[GitHub] carbondata issue #2139: [CARBONDATA-2267] [Presto] Support Reading CarbonDat...

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

    https://github.com/apache/carbondata/pull/2139
  
    SDV Build Success , Please check CI http://144.76.159.231:8080/job/ApacheSDVTests/4343/



---

[GitHub] carbondata issue #2139: [CARBONDATA-2267] [Presto] Support Reading CarbonDat...

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

    https://github.com/apache/carbondata/pull/2139
  
    Build Success with Spark 2.2.1, Please check CI http://88.99.58.216:8080/job/ApacheCarbonPRBuilder/3596/



---

[GitHub] carbondata issue #2139: [CARBONDATA-2267] [Presto] Support Reading CarbonDat...

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

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


---

[GitHub] carbondata issue #2139: [CARBONDATA-2267] [Presto] Support Reading CarbonDat...

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

    https://github.com/apache/carbondata/pull/2139
  
    LGTM


---

[GitHub] carbondata pull request #2139: [CARBONDATA-2267] [Presto] Support Reading Ca...

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

    https://github.com/apache/carbondata/pull/2139#discussion_r180649360
  
    --- Diff: integration/presto/src/main/java/org/apache/carbondata/presto/PrestoFilterUtil.java ---
    @@ -69,6 +72,13 @@
     
       private static Map<Integer, Expression> filterMap = new HashMap<>();
     
    +  private final static String HIVE_DEFAULT_DYNAMIC_PARTITION = "__HIVE_DEFAULT_PARTITION__";
    --- End diff --
    
    HIVE_DEFAULT_DYNAMIC_PARTITION is created when the partition value for that column is null so the data goes to __HIVE_DEFAULT_PARTITION__ partition. This check is added so that we can query the data from __HIVE_DEFAULT_PARTITION__ as well.


---

[GitHub] carbondata pull request #2139: [CARBONDATA-2267] [Presto] Support Reading Ca...

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

    https://github.com/apache/carbondata/pull/2139#discussion_r180553823
  
    --- Diff: integration/presto/src/main/java/org/apache/carbondata/presto/PrestoFilterUtil.java ---
    @@ -69,6 +72,13 @@
     
       private static Map<Integer, Expression> filterMap = new HashMap<>();
     
    +  private final static String HIVE_DEFAULT_DYNAMIC_PARTITION = "__HIVE_DEFAULT_PARTITION__";
    --- End diff --
    
    can you explain, why need to add "HIVE_DEFAULT_DYNAMIC_PARTITION" ? what purpose?
    if (value == null) {
            filter.add(carbonDataColumnHandle.getColumnName() + "=" + HIVE_DEFAULT_DYNAMIC_PARTITION);


---

[GitHub] carbondata issue #2139: [CARBONDATA-2267] [Presto] Support Reading CarbonDat...

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

    https://github.com/apache/carbondata/pull/2139
  
    Build Success with Spark 2.2.1, Please check CI http://88.99.58.216:8080/job/ApacheCarbonPRBuilder/3603/



---

[GitHub] carbondata issue #2139: [CARBONDATA-2267] [Presto] Support Reading CarbonDat...

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

    https://github.com/apache/carbondata/pull/2139
  
    @chenliang613 we have updated all info with hive partition in pr title and also put a detail description comment of what PR does please review


---

[GitHub] carbondata issue #2139: [CARBONDATA-2267] [Presto] Support Reading CarbonDat...

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

    https://github.com/apache/carbondata/pull/2139
  
    SDV Build Success , Please check CI http://144.76.159.231:8080/job/ApacheSDVTests/4338/



---

[GitHub] carbondata issue #2139: [CARBONDATA-2267] [Presto] Support Reading CarbonDat...

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

    https://github.com/apache/carbondata/pull/2139
  
    SDV Build Success , Please check CI http://144.76.159.231:8080/job/ApacheSDVTests/4334/



---

[GitHub] carbondata issue #2139: [CARBONDATA-2267] [Presto] Support Reading CarbonDat...

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

    https://github.com/apache/carbondata/pull/2139
  
    Build Success with Spark 2.2.1, Please check CI http://88.99.58.216:8080/job/ApacheCarbonPRBuilder/3647/



---

[GitHub] carbondata pull request #2139: [CARBONDATA-2267] [Presto] Support Reading Ca...

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

    https://github.com/apache/carbondata/pull/2139#discussion_r180672102
  
    --- Diff: integration/presto/src/main/java/org/apache/carbondata/presto/PrestoFilterUtil.java ---
    @@ -69,6 +72,13 @@
     
       private static Map<Integer, Expression> filterMap = new HashMap<>();
     
    +  private final static String HIVE_DEFAULT_DYNAMIC_PARTITION = "__HIVE_DEFAULT_PARTITION__";
    +  private final static String PARTITION_VALUE_WILDCARD = "";
    --- End diff --
    
    In hive PARTITION_VALUE_WILDCARD is used to select all the partitions from the table when we don't have any filter conditions on the partition column in our query, but in our code we are already quering all the partitions for the above case. So we can remove this part.


---

[GitHub] carbondata issue #2139: [CARBONDATA-2267] [Presto] Support Reading CarbonDat...

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

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


---

[GitHub] carbondata issue #2139: [CARBONDATA-2267] [Presto] Support Reading CarbonDat...

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

    https://github.com/apache/carbondata/pull/2139
  
    Build Success with Spark 2.1.0, Please check CI http://136.243.101.176:8080/job/ApacheCarbonPRBuilder1/4869/



---

[GitHub] carbondata pull request #2139: [CARBONDATA-2267] [Presto] Support Reading Ca...

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

    https://github.com/apache/carbondata/pull/2139#discussion_r179913431
  
    --- Diff: integration/presto/src/main/java/org/apache/carbondata/presto/impl/CarbonTableReader.java ---
    @@ -415,14 +440,56 @@ public TBase create() {
         return result;
       }
     
    +  /** Returns list of partition specs to query based on the domain constraints
    +   * @param constraints
    +   * @param carbonTable
    +   * @throws IOException
    +   */
    +  private List<PartitionSpec> findRequiredPartitions(TupleDomain<ColumnHandle> constraints, CarbonTable carbonTable,
    +      LoadMetadataDetails[]loadMetadataDetails)  {
    +    Set<PartitionSpec> partitionSpecs = new HashSet<>();
    +    List<PartitionSpec> prunePartitions = new ArrayList();
    +
    +    for (LoadMetadataDetails loadMetadataDetail : loadMetadataDetails) {
    +      SegmentFileStore segmentFileStore = null;
    +      try {
    +        segmentFileStore =
    +            new SegmentFileStore(carbonTable.getTablePath(), loadMetadataDetail.getSegmentFile());
    +        partitionSpecs.addAll(segmentFileStore.getPartitionSpecs());
    +
    +      } catch (IOException e) {
    +        e.printStackTrace();
    +      }
    +    }
    +
    +    List<String> partitionValuesFromExpression =
    +        PrestoFilterUtil.getPartitionFilters(carbonTable, constraints);
    +
    +    List<List<String>> partitionSpecNamesList = partitionSpecs.stream().map(
    +        PartitionSpec::getPartitions).collect(Collectors.toList());
    +
    +    List<PartitionSpec> partitionSpecsList = new ArrayList(partitionSpecs);
    +
    +    for (int i = 0; i < partitionSpecNamesList.size(); i++) {
    +      List<String> partitionSpecNames = partitionSpecNamesList.get(i);
    +      if (partitionSpecNames.containsAll(partitionValuesFromExpression)) {
    +        prunePartitions
    +            .add(partitionSpecsList.get(i));
    +      }
    +    }
    +    return prunePartitions;
    +  }
    +
       private CarbonTableInputFormat<Object>  createInputFormat( Configuration conf,
    -       AbsoluteTableIdentifier identifier, Expression filterExpression)
    -          throws IOException {
    +      AbsoluteTableIdentifier identifier, Expression filterExpression, List<PartitionSpec> filteredPartitions)
    +      throws IOException {
         CarbonTableInputFormat format = new CarbonTableInputFormat<Object>();
         CarbonTableInputFormat.setTablePath(conf,
    -            identifier.appendWithLocalPrefix(identifier.getTablePath()));
    +        identifier.appendWithLocalPrefix(identifier.getTablePath()));
         CarbonTableInputFormat.setFilterPredicates(conf, filterExpression);
    -
    +    if(filteredPartitions.size() != 0) {
    +      CarbonTableInputFormat.setPartitionsToPrune(conf, new ArrayList<>(filteredPartitions));
    --- End diff --
    
    Why Create a new ArrayList here when the calling method is already returning a List


---

[GitHub] carbondata issue #2139: [CARBONDATA-2267] [Presto] Support Reading CarbonDat...

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

    https://github.com/apache/carbondata/pull/2139
  
    In this PR we are handling the scenario of querying data to standard hive partitioned table in carbondata. 
    
    Earlier we were iterating over all the partitions to get the data but now as we have identified the required partitions, so the query will be performed only on the required partition.
    Firstly we are identifying the partition columns through domain constraints in presto and after that we are adding the Partition Spec to the configuration object to query partition.



---

[GitHub] carbondata issue #2139: [CARBONDATA-2267] [Presto] Support Reading CarbonDat...

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

    https://github.com/apache/carbondata/pull/2139
  
    Build Failed with Spark 2.2.1, Please check CI http://88.99.58.216:8080/job/ApacheCarbonPRBuilder/3602/



---

[GitHub] carbondata issue #2139: [CARBONDATA-2267] [Presto] Support Reading CarbonDat...

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

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


---

[GitHub] carbondata pull request #2139: [CARBONDATA-2267] [Presto] Support Reading Ca...

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

    https://github.com/apache/carbondata/pull/2139#discussion_r180789005
  
    --- Diff: integration/presto/src/main/java/org/apache/carbondata/presto/PrestoFilterUtil.java ---
    @@ -69,6 +72,13 @@
     
       private static Map<Integer, Expression> filterMap = new HashMap<>();
     
    +  private final static String HIVE_DEFAULT_DYNAMIC_PARTITION = "__HIVE_DEFAULT_PARTITION__";
    --- End diff --
    
    ok


---

[GitHub] carbondata pull request #2139: [CARBONDATA-2267] [Presto] Support Reading Ca...

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

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


---

[GitHub] carbondata issue #2139: [CARBONDATA-2267] [Presto] Support Reading CarbonDat...

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

    https://github.com/apache/carbondata/pull/2139
  
    Build Success with Spark 2.1.0, Please check CI http://136.243.101.176:8080/job/ApacheCarbonPRBuilder1/4825/



---

[GitHub] carbondata issue #2139: [CARBONDATA-2267] [Presto] Support Reading CarbonDat...

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

    https://github.com/apache/carbondata/pull/2139
  
    Build Failed  with Spark 2.1.0, Please check CI http://136.243.101.176:8080/job/ApacheCarbonPRBuilder1/4818/



---

[GitHub] carbondata pull request #2139: [CARBONDATA-2267] [Presto] Support Reading Ca...

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

    https://github.com/apache/carbondata/pull/2139#discussion_r179913191
  
    --- Diff: integration/presto/src/main/java/org/apache/carbondata/presto/impl/CarbonTableReader.java ---
    @@ -387,11 +398,25 @@ public TBase create() {
         config.set(CarbonTableInputFormat.DATABASE_NAME, carbonTable.getDatabaseName());
         config.set(CarbonTableInputFormat.TABLE_NAME, carbonTable.getTableName());
     
    +    JobConf jobConf = new JobConf(config);
    +    List<PartitionSpec> filteredPartitions = new ArrayList();
    +
    +    try {
    +      loadMetadataDetails= SegmentStatusManager
    +          .readTableStatusFile(CarbonTablePath.getTableStatusFilePath(carbonTable.getTablePath()));
    +    } catch (IOException e) {
    +      e.printStackTrace();
    --- End diff --
    
    The Error should be logged and if there is an IOException than it needs to be handled or thrown to make sure that error is propogated.


---

[GitHub] carbondata issue #2139: [CARBONDATA-2267] [Presto] Support Reading CarbonDat...

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

    https://github.com/apache/carbondata/pull/2139
  
    @chenliang613


---

[GitHub] carbondata pull request #2139: [CARBONDATA-2267] [Presto] Support Reading Ca...

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

    https://github.com/apache/carbondata/pull/2139#discussion_r179913482
  
    --- Diff: integration/presto/src/main/java/org/apache/carbondata/presto/impl/CarbonTableReader.java ---
    @@ -415,14 +440,56 @@ public TBase create() {
         return result;
       }
     
    +  /** Returns list of partition specs to query based on the domain constraints
    +   * @param constraints
    +   * @param carbonTable
    +   * @throws IOException
    +   */
    +  private List<PartitionSpec> findRequiredPartitions(TupleDomain<ColumnHandle> constraints, CarbonTable carbonTable,
    +      LoadMetadataDetails[]loadMetadataDetails)  {
    +    Set<PartitionSpec> partitionSpecs = new HashSet<>();
    +    List<PartitionSpec> prunePartitions = new ArrayList();
    +
    +    for (LoadMetadataDetails loadMetadataDetail : loadMetadataDetails) {
    +      SegmentFileStore segmentFileStore = null;
    +      try {
    +        segmentFileStore =
    +            new SegmentFileStore(carbonTable.getTablePath(), loadMetadataDetail.getSegmentFile());
    +        partitionSpecs.addAll(segmentFileStore.getPartitionSpecs());
    +
    +      } catch (IOException e) {
    +        e.printStackTrace();
    +      }
    +    }
    +
    +    List<String> partitionValuesFromExpression =
    +        PrestoFilterUtil.getPartitionFilters(carbonTable, constraints);
    +
    +    List<List<String>> partitionSpecNamesList = partitionSpecs.stream().map(
    +        PartitionSpec::getPartitions).collect(Collectors.toList());
    +
    +    List<PartitionSpec> partitionSpecsList = new ArrayList(partitionSpecs);
    +
    +    for (int i = 0; i < partitionSpecNamesList.size(); i++) {
    --- End diff --
    
    I am not sure about this logic , you have a set and you create a specsNameList and then you create a list of partitionSpecs and then you check that if the list has all fiteredPartition you add it to prunedPartitiion, why not do it in the 1st map itself instead of complicating code


---

[GitHub] carbondata issue #2139: [CARBONDATA-2267] [Presto] Support Reading CarbonDat...

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

    https://github.com/apache/carbondata/pull/2139
  
    Build Success with Spark 2.1.0, Please check CI http://136.243.101.176:8080/job/ApacheCarbonPRBuilder1/4862/



---

[GitHub] carbondata issue #2139: [CARBONDATA-2267] [Presto] Support Reading CarbonDat...

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

    https://github.com/apache/carbondata/pull/2139
  
    retest sdv please


---

[GitHub] carbondata issue #2139: [CARBONDATA-2267] [Presto] Support Reading CarbonDat...

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

    https://github.com/apache/carbondata/pull/2139
  
    Build Success with Spark 2.2.1, Please check CI http://88.99.58.216:8080/job/ApacheCarbonPRBuilder/3640/



---

[GitHub] carbondata issue #2139: [CARBONDATA-2267] [Presto] Support Reading CarbonDat...

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

    https://github.com/apache/carbondata/pull/2139
  
    Build Success with Spark 2.2.1, Please check CI http://88.99.58.216:8080/job/ApacheCarbonPRBuilder/3721/



---

[GitHub] carbondata issue #2139: [CARBONDATA-2267] [Presto] Support Reading CarbonDat...

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

    https://github.com/apache/carbondata/pull/2139
  
    rebased,commits squashed,build passing


---

[GitHub] carbondata issue #2139: [CARBONDATA-2267] [Presto] Support Reading CarbonDat...

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

    https://github.com/apache/carbondata/pull/2139
  
    Build Failed  with Spark 2.1.0, Please check CI http://136.243.101.176:8080/job/ApacheCarbonPRBuilder1/4864/



---

[GitHub] carbondata pull request #2139: [CARBONDATA-2267] [Presto] Support Reading Ca...

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

    https://github.com/apache/carbondata/pull/2139#discussion_r179913270
  
    --- Diff: integration/presto/src/main/java/org/apache/carbondata/presto/impl/CarbonTableReader.java ---
    @@ -415,14 +440,56 @@ public TBase create() {
         return result;
       }
     
    +  /** Returns list of partition specs to query based on the domain constraints
    +   * @param constraints
    +   * @param carbonTable
    +   * @throws IOException
    +   */
    +  private List<PartitionSpec> findRequiredPartitions(TupleDomain<ColumnHandle> constraints, CarbonTable carbonTable,
    +      LoadMetadataDetails[]loadMetadataDetails)  {
    +    Set<PartitionSpec> partitionSpecs = new HashSet<>();
    +    List<PartitionSpec> prunePartitions = new ArrayList();
    +
    +    for (LoadMetadataDetails loadMetadataDetail : loadMetadataDetails) {
    +      SegmentFileStore segmentFileStore = null;
    +      try {
    +        segmentFileStore =
    +            new SegmentFileStore(carbonTable.getTablePath(), loadMetadataDetail.getSegmentFile());
    +        partitionSpecs.addAll(segmentFileStore.getPartitionSpecs());
    +
    +      } catch (IOException e) {
    +        e.printStackTrace();
    --- End diff --
    
    Please do not use e.printstacktrace, log the exception and then handle it appropriately


---

[GitHub] carbondata issue #2139: [CARBONDATA-2267] [Presto] Support Reading CarbonDat...

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

    https://github.com/apache/carbondata/pull/2139
  
    SDV Build Success , Please check CI http://144.76.159.231:8080/job/ApacheSDVTests/4333/



---

[GitHub] carbondata issue #2139: [CARBONDATA-2267] [Presto] Support Reading CarbonDat...

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

    https://github.com/apache/carbondata/pull/2139
  
    Build Success with Spark 2.2.1, Please check CI http://88.99.58.216:8080/job/ApacheCarbonPRBuilder/3643/



---

[GitHub] carbondata issue #2139: [CARBONDATA-2267] [Presto] Support Reading CarbonDat...

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

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


---

[GitHub] carbondata pull request #2139: [CARBONDATA-2267] [Presto] Support Reading Ca...

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

    https://github.com/apache/carbondata/pull/2139#discussion_r180554217
  
    --- Diff: integration/presto/src/main/java/org/apache/carbondata/presto/PrestoFilterUtil.java ---
    @@ -69,6 +72,13 @@
     
       private static Map<Integer, Expression> filterMap = new HashMap<>();
     
    +  private final static String HIVE_DEFAULT_DYNAMIC_PARTITION = "__HIVE_DEFAULT_PARTITION__";
    +  private final static String PARTITION_VALUE_WILDCARD = "";
    --- End diff --
    
    Why need to add "wildcard" for partition filter?


---

[GitHub] carbondata issue #2139: [CARBONDATA-2267] [Presto] Support Reading CarbonDat...

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

    https://github.com/apache/carbondata/pull/2139
  
    SDV Build Success , Please check CI http://144.76.159.231:8080/job/ApacheSDVTests/4331/



---

[GitHub] carbondata issue #2139: [CARBONDATA-2267] [Presto] Support Reading CarbonDat...

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

    https://github.com/apache/carbondata/pull/2139
  
    Build Success with Spark 2.1.0, Please check CI http://136.243.101.176:8080/job/ApacheCarbonPRBuilder1/4826/



---

[GitHub] carbondata issue #2139: [CARBONDATA-2267] [Presto] Support Reading CarbonDat...

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

    https://github.com/apache/carbondata/pull/2139
  
    Build Failed  with Spark 2.1.0, Please check CI http://136.243.101.176:8080/job/ApacheCarbonPRBuilder1/4816/



---

[GitHub] carbondata issue #2139: [CARBONDATA-2267] [Presto] Support Reading CarbonDat...

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

    https://github.com/apache/carbondata/pull/2139
  
    Build Success with Spark 2.1.0, Please check CI http://136.243.101.176:8080/job/ApacheCarbonPRBuilder1/4874/



---

[GitHub] carbondata issue #2139: [CARBONDATA-2267] [Presto] Support Reading CarbonDat...

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

    https://github.com/apache/carbondata/pull/2139
  
    Build Success with Spark 2.1.0, Please check CI http://136.243.101.176:8080/job/ApacheCarbonPRBuilder1/4940/



---

[GitHub] carbondata issue #2139: [CARBONDATA-2267] [Presto] Support Reading CarbonDat...

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

    https://github.com/apache/carbondata/pull/2139
  
    SDV Build Fail , Please check CI http://144.76.159.231:8080/job/ApacheSDVTests/4332/



---

[GitHub] carbondata issue #2139: [CARBONDATA-2267] [Presto] Support Reading CarbonDat...

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

    https://github.com/apache/carbondata/pull/2139
  
    Build Success with Spark 2.2.1, Please check CI http://88.99.58.216:8080/job/ApacheCarbonPRBuilder/3652/



---

[GitHub] carbondata issue #2139: [CARBONDATA-2267] [Presto] Support Reading CarbonDat...

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

    https://github.com/apache/carbondata/pull/2139
  
    Build Success with Spark 2.1.0, Please check CI http://136.243.101.176:8080/job/ApacheCarbonPRBuilder1/4819/



---

[GitHub] carbondata issue #2139: [CARBONDATA-2267] [Presto] Support Reading CarbonDat...

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

    https://github.com/apache/carbondata/pull/2139
  
    @chenliang613 please review this pr


---

[GitHub] carbondata issue #2139: [CARBONDATA-2267] [Presto] Support Reading CarbonDat...

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

    https://github.com/apache/carbondata/pull/2139
  
    Build Failed with Spark 2.2.1, Please check CI http://88.99.58.216:8080/job/ApacheCarbonPRBuilder/3595/



---

[GitHub] carbondata issue #2139: [CARBONDATA-2267] [Presto] Support Reading CarbonDat...

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

    https://github.com/apache/carbondata/pull/2139
  
    Currently, there are two kinds of partition:
    1. Standard hive partition: same as Apache Hive and Spark
    2.CarbonData partition : Hash,List,Range partition
    
    So , In this PR, please update all info with "hive partition". 


---

[GitHub] carbondata issue #2139: [CARBONDATA-2267] [Presto] Support Reading CarbonDat...

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

    https://github.com/apache/carbondata/pull/2139
  
    Build Success with Spark 2.1.0, Please check CI http://136.243.101.176:8080/job/ApacheCarbonPRBuilder1/4865/



---