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

[GitHub] carbondata pull request #2685: [WIP] Support backward compatability in filef...

GitHub user ravipesala opened a pull request:

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

    [WIP] Support backward compatability in fileformat and added tests for load with different sort orders

    … 
    
    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/ravipesala/incubator-carbondata fileformat-back-compat

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

    https://github.com/apache/carbondata/pull/2685.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 #2685
    
----
commit c265f37caf6d12eb2158e7d5d1cbbe0d777710ed
Author: ravipesala <ra...@...>
Date:   2018-08-30T15:11:06Z

    Support backward compatability in fileformat and added tests for load with different sort orders

----


---

[GitHub] carbondata issue #2685: [CARBONDATA-2910] Support backward compatability in ...

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

    https://github.com/apache/carbondata/pull/2685
  
    Build Failed with Spark 2.2.1, Please check CI http://95.216.28.178:8080/job/ApacheCarbonPRBuilder1/305/



---

[GitHub] carbondata pull request #2685: [WIP] Support backward compatability in filef...

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

    https://github.com/apache/carbondata/pull/2685#discussion_r214627971
  
    --- Diff: core/src/main/java/org/apache/carbondata/core/util/AbstractDataFileFooterConverter.java ---
    @@ -153,6 +165,14 @@ private static BitSet getPresenceMeta(
           for (int i = 0; i < table_columns.size(); i++) {
             columnSchemaList.add(thriftColumnSchemaToWrapperColumnSchema(table_columns.get(i)));
           }
    +      // In case of non transactional table just set columnuniqueid as columnName to support
    +      // backward compatabiity. non transactional tables column uniqueid is always equal to
    +      // columnname
    +      if (!isTransactionalTable) {
    --- End diff --
    
    Is the columnSchema only for runtime usage?
    If not, Will it be problem if use change nonTransactionalTable to transactionalTable?


---

[GitHub] carbondata issue #2685: [WIP] Support backward compatability in fileformat a...

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

    https://github.com/apache/carbondata/pull/2685
  
    Build Failed with Spark 2.2.1, Please check CI http://95.216.28.178:8080/job/ApacheCarbonPRBuilder1/167/



---

[GitHub] carbondata issue #2685: [CARBONDATA-2910] Support backward compatability in ...

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

    https://github.com/apache/carbondata/pull/2685
  
    Build Success with Spark 2.2.1, Please check CI http://95.216.28.178:8080/job/ApacheCarbonPRBuilder1/212/



---

[GitHub] carbondata issue #2685: [WIP] Support backward compatability in fileformat a...

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

    https://github.com/apache/carbondata/pull/2685
  
    Build Failed with Spark 2.2.1, Please check CI http://95.216.28.178:8080/job/ApacheCarbonPRBuilder1/182/



---

[GitHub] carbondata issue #2685: [CARBONDATA-2910] Support backward compatability in ...

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

    https://github.com/apache/carbondata/pull/2685
  
    Build Failed with Spark 2.2.1, Please check CI http://95.216.28.178:8080/job/ApacheCarbonPRBuilder1/302/



---

[GitHub] carbondata issue #2685: [WIP] Support backward compatability in fileformat a...

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

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



---

[GitHub] carbondata issue #2685: [CARBONDATA-2910] Support backward compatability in ...

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

    https://github.com/apache/carbondata/pull/2685
  
    Build Failed  with Spark 2.3.1, Please check CI http://136.243.101.176:8080/job/ApacheCarbonPRBuilder2.3/8372/



---

[GitHub] carbondata issue #2685: [CARBONDATA-2910] Support backward compatability in ...

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

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


---

[GitHub] carbondata pull request #2685: [WIP] Support backward compatability in filef...

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

    https://github.com/apache/carbondata/pull/2685#discussion_r214626402
  
    --- Diff: hadoop/src/main/java/org/apache/carbondata/hadoop/testutil/StoreCreator.java ---
    @@ -101,18 +101,32 @@
     
       private static LogService LOG =
           LogServiceFactory.getLogService(StoreCreator.class.getCanonicalName());
    -  private static AbsoluteTableIdentifier absoluteTableIdentifier;
    -  private static String storePath = null;
    +  private AbsoluteTableIdentifier absoluteTableIdentifier;
    +  private String storePath = null;
    +  private String csvPath;
    +  private boolean dictionary;
    +  private List<String> sortCOls = new ArrayList<>();
    --- End diff --
    
    typing error


---

[GitHub] carbondata pull request #2685: [CARBONDATA-2910] Support backward compatabil...

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

    https://github.com/apache/carbondata/pull/2685#discussion_r215855721
  
    --- Diff: core/src/main/java/org/apache/carbondata/core/indexstore/blockletindex/BlockDataMap.java ---
    @@ -631,6 +638,24 @@ private boolean useMinMaxForExecutorPruning(FilterResolverIntf filterResolverInt
         return useMinMaxForPruning;
       }
     
    +  @Override
    +  public List<Blocklet> prune(Expression expression, SegmentProperties segmentProperties,
    +      List<PartitionSpec> partitions, AbsoluteTableIdentifier identifier) throws IOException {
    +    FilterResolverIntf filterResolverIntf = null;
    +    if (expression != null) {
    +      SegmentProperties properties = getSegmentProperties();
    --- End diff --
    
    ok


---

[GitHub] carbondata issue #2685: [CARBONDATA-2910] Support backward compatability in ...

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

    https://github.com/apache/carbondata/pull/2685
  
    Build Failed  with Spark 2.1.0, Please check CI http://136.243.101.176:8080/job/ApacheCarbonPRBuilder2.1/139/



---

[GitHub] carbondata issue #2685: [CARBONDATA-2910] Support backward compatability in ...

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

    https://github.com/apache/carbondata/pull/2685
  
    1. I think there may be some bugs in index datamap currently to support 'different sort columns per load'
    2. What about different dictionary_include per load?


---

[GitHub] carbondata issue #2685: [CARBONDATA-2910] Support backward compatability in ...

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

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


---

[GitHub] carbondata issue #2685: [CARBONDATA-2910] Support backward compatability in ...

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

    https://github.com/apache/carbondata/pull/2685
  
    Build Success with Spark 2.2.1, Please check CI http://95.216.28.178:8080/job/ApacheCarbonPRBuilder1/317/



---

[GitHub] carbondata issue #2685: [WIP] Support backward compatability in fileformat a...

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

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



---

[GitHub] carbondata issue #2685: [CARBONDATA-2910] Support backward compatability in ...

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

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



---

[GitHub] carbondata issue #2685: [CARBONDATA-2910] Support backward compatability in ...

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

    https://github.com/apache/carbondata/pull/2685
  
    Build Success with Spark 2.1.0, Please check CI http://136.243.101.176:8080/job/ApacheCarbonPRBuilder2.1/149/



---

[GitHub] carbondata issue #2685: [CARBONDATA-2910] Support backward compatability in ...

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

    https://github.com/apache/carbondata/pull/2685
  
    Build Failed  with Spark 2.1.0, Please check CI http://136.243.101.176:8080/job/ApacheCarbonPRBuilder2.1/134/



---

[GitHub] carbondata issue #2685: [CARBONDATA-2910] Support backward compatability in ...

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

    https://github.com/apache/carbondata/pull/2685
  
    Build Failed  with Spark 2.3.1, Please check CI http://136.243.101.176:8080/job/ApacheCarbonPRBuilder2.3/8388/



---

[GitHub] carbondata pull request #2685: [WIP] Support backward compatability in filef...

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

    https://github.com/apache/carbondata/pull/2685#discussion_r214627216
  
    --- Diff: core/src/main/java/org/apache/carbondata/core/util/AbstractDataFileFooterConverter.java ---
    @@ -135,6 +135,18 @@ private static BitSet getPresenceMeta(
        * @throws IOException problem while reading the index file
        */
       public List<DataFileFooter> getIndexInfo(String filePath, byte[] fileData) throws IOException {
    +    return getIndexInfo(filePath, fileData, true);
    +  }
    +
    +  /**
    +   * Below method will be used to get the index info from index file
    +   *
    +   * @param filePath           file path of the index file
    --- End diff --
    
    These comments do no provide any useful information, better to remove them


---

[GitHub] carbondata issue #2685: [CARBONDATA-2910] Support backward compatability in ...

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

    https://github.com/apache/carbondata/pull/2685
  
    Build Failed with Spark 2.2.1, Please check CI http://95.216.28.178:8080/job/ApacheCarbonPRBuilder1/201/



---

[GitHub] carbondata pull request #2685: [CARBONDATA-2910] Support backward compatabil...

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

    https://github.com/apache/carbondata/pull/2685#discussion_r215857396
  
    --- Diff: core/src/main/java/org/apache/carbondata/core/indexstore/blockletindex/BlockDataMap.java ---
    @@ -630,6 +638,24 @@ private boolean useMinMaxForExecutorPruning(FilterResolverIntf filterResolverInt
         return useMinMaxForPruning;
       }
     
    +  @Override
    +  public List<Blocklet> prune(Expression expression, SegmentProperties segmentProperties,
    +      List<PartitionSpec> partitions, AbsoluteTableIdentifier identifier) throws IOException {
    +    FilterResolverIntf filterResolverIntf = null;
    +    if (expression != null) {
    +      SegmentProperties properties = getSegmentProperties();
    +      QueryModel.FilterProcessVO processVO =
    +          new QueryModel.FilterProcessVO(properties.getDimensions(), properties.getMeasures(),
    +              new ArrayList<CarbonDimension>());
    +      QueryModel.processFilterExpression(processVO, expression, null, null);
    +      // Optimize Filter Expression and fit RANGE filters is conditions apply.
    +      FilterOptimizer rangeFilterOptimizer = new RangeFilterOptmizer(expression);
    +      rangeFilterOptimizer.optimizeFilter();
    +      filterResolverIntf = CarbonTable.resolveFilter(expression, identifier);
    --- End diff --
    
    I don't get your point. Here we need to convert to filterResolverIntf from expression for every datamap as columns meta could be different for each carbonindex.


---

[GitHub] carbondata pull request #2685: [CARBONDATA-2910] Support backward compatabil...

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

    https://github.com/apache/carbondata/pull/2685#discussion_r215566911
  
    --- Diff: core/src/main/java/org/apache/carbondata/core/indexstore/blockletindex/BlockDataMap.java ---
    @@ -631,6 +638,24 @@ private boolean useMinMaxForExecutorPruning(FilterResolverIntf filterResolverInt
         return useMinMaxForPruning;
       }
     
    +  @Override
    +  public List<Blocklet> prune(Expression expression, SegmentProperties segmentProperties,
    +      List<PartitionSpec> partitions, AbsoluteTableIdentifier identifier) throws IOException {
    +    FilterResolverIntf filterResolverIntf = null;
    +    if (expression != null) {
    +      SegmentProperties properties = getSegmentProperties();
    --- End diff --
    
    use the `segmentProperties` object getting passed in method arguments


---

[GitHub] carbondata pull request #2685: [CARBONDATA-2910] Support backward compatabil...

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

    https://github.com/apache/carbondata/pull/2685#discussion_r214886972
  
    --- Diff: core/src/main/java/org/apache/carbondata/core/indexstore/blockletindex/BlockDataMap.java ---
    @@ -630,6 +638,24 @@ private boolean useMinMaxForExecutorPruning(FilterResolverIntf filterResolverInt
         return useMinMaxForPruning;
       }
     
    +  @Override
    +  public List<Blocklet> prune(Expression expression, SegmentProperties segmentProperties,
    +      List<PartitionSpec> partitions, AbsoluteTableIdentifier identifier) throws IOException {
    +    FilterResolverIntf filterResolverIntf = null;
    +    if (expression != null) {
    +      SegmentProperties properties = getSegmentProperties();
    +      QueryModel.FilterProcessVO processVO =
    +          new QueryModel.FilterProcessVO(properties.getDimensions(), properties.getMeasures(),
    +              new ArrayList<CarbonDimension>());
    +      QueryModel.processFilterExpression(processVO, expression, null, null);
    +      // Optimize Filter Expression and fit RANGE filters is conditions apply.
    +      FilterOptimizer rangeFilterOptimizer = new RangeFilterOptmizer(expression);
    +      rangeFilterOptimizer.optimizeFilter();
    +      filterResolverIntf = CarbonTable.resolveFilter(expression, identifier);
    --- End diff --
    
    can we pull up the transformation from expression to filterResolverIntf  so that we can reuse most code instead of adding another `prune` method everywhere


---

[GitHub] carbondata issue #2685: [CARBONDATA-2910] Support backward compatability in ...

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

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



---

[GitHub] carbondata issue #2685: [CARBONDATA-2910] Support backward compatability in ...

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

    https://github.com/apache/carbondata/pull/2685
  
    Build Failed with Spark 2.2.1, Please check CI http://95.216.28.178:8080/job/ApacheCarbonPRBuilder1/199/



---

[GitHub] carbondata issue #2685: [CARBONDATA-2910] Support backward compatability in ...

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

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



---

[GitHub] carbondata issue #2685: [CARBONDATA-2910] Support backward compatability in ...

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

    https://github.com/apache/carbondata/pull/2685
  
    Build Failed  with Spark 2.3.1, Please check CI http://136.243.101.176:8080/job/ApacheCarbonPRBuilder2.3/8377/



---

[GitHub] carbondata issue #2685: [CARBONDATA-2910] Support backward compatability in ...

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

    https://github.com/apache/carbondata/pull/2685
  
    @xuchuanyin This fix is not considered for otherdatamaps as we target only for non transactional table.
    And also we dctionary_include cannot be done on non transactional table


---

[GitHub] carbondata issue #2685: [CARBONDATA-2910] Support backward compatability in ...

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

    https://github.com/apache/carbondata/pull/2685
  
    Build Failed with Spark 2.2.1, Please check CI http://95.216.28.178:8080/job/ApacheCarbonPRBuilder1/208/



---

[GitHub] carbondata pull request #2685: [CARBONDATA-2910] Support backward compatabil...

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

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


---

[GitHub] carbondata issue #2685: [CARBONDATA-2910] Support backward compatability in ...

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

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



---

[GitHub] carbondata pull request #2685: [WIP] Support backward compatability in filef...

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

    https://github.com/apache/carbondata/pull/2685#discussion_r214625972
  
    --- Diff: integration/spark-datasource/src/test/scala/org/apache/spark/sql/carbondata/datasource/SparkCarbonDataSourceTest.scala ---
    @@ -590,12 +596,60 @@ class SparkCarbonDataSourceTest extends FunSuite with BeforeAndAfterAll {
         }
       }
     
    +  test("test read using old data") {
    +    val store = new StoreCreator(new File(warehouse1).getAbsolutePath,
    +      new File(warehouse1 + "../../../../../hadoop/src/test/resources/data.csv").getCanonicalPath,
    +      false)
    +    store.createCarbonStore()
    +    FileFactory.deleteAllFilesOfDir(new File(warehouse1+"/testdb/testtable/Fact/Part0/Segment_0/0"))
    +    val dfread = spark.read.format("carbon").load(warehouse1+"/testdb/testtable/Fact/Part0/Segment_0")
    +    dfread.show(false)
    +    spark.sql("drop table if exists parquet_table")
    +  }
    +
    +  test("test read using different sort order data") {
    +    spark.sql("drop table if exists old_comp")
    +    FileFactory.deleteAllFilesOfDir(new File(warehouse1+"/testdb"))
    +    val store = new StoreCreator(new File(warehouse1).getAbsolutePath,
    +      new File(warehouse1 + "../../../../../hadoop/src/test/resources/data.csv").getCanonicalPath,
    +      false)
    +    store.setSortCOls(new util.ArrayList[String](Seq ("name").asJava))
    +    var model = store.createTableAndLoadModel(false)
    +    model.setSegmentId("0")
    +    store.createCarbonStore(model)
    +    FileFactory.deleteAllFilesOfDir(new File(warehouse1+"/testdb/testtable/Fact/Part0/Segment_0/0"))
    +    store.setSortCOls(new util.ArrayList[String](Seq ("country,phonetype").asJava))
    +    model = store.createTableAndLoadModel(false)
    +    model.setSegmentId("1")
    +    store.createCarbonStore(model)
    +    FileFactory.deleteAllFilesOfDir(new File(warehouse1+"/testdb/testtable/Fact/Part0/Segment_1/0"))
    +    store.setSortCOls(new util.ArrayList[String](Seq ("date").asJava))
    +    model = store.createTableAndLoadModel(false)
    +    model.setSegmentId("2")
    +    store.createCarbonStore(model)
    +    FileFactory.deleteAllFilesOfDir(new File(warehouse1+"/testdb/testtable/Fact/Part0/Segment_2/0"))
    +    store.setSortCOls(new util.ArrayList[String](Seq ("serialname").asJava))
    +    model = store.createTableAndLoadModel(false)
    +    model.setSegmentId("3")
    +    store.createCarbonStore(model)
    +    FileFactory.deleteAllFilesOfDir(new File(warehouse1+"/testdb/testtable/Fact/Part0/Segment_3/0"))
    +    spark.sql(s"create table old_comp(id int, date string, country string, name string, phonetype string, serialname string, salary int) using carbon options(path='$warehouse1/testdb/testtable/Fact/Part0/', 'sort_columns'='name')")
    +    assert(spark.sql("select * from old_comp where country='china'").count() == 3396)
    +    assert(spark.sql("select * from old_comp ").count() == 4000)
    +    spark.sql("drop table if exists old_comp")
    +
    +    spark.sql(s"create table old_comp1 using carbon options(path='$warehouse1/testdb/testtable/Fact/Part0/')")
    +    assert(spark.sql("select * from old_comp1 where country='china'").count() == 3396)
    +    assert(spark.sql("select * from old_comp1 ").count() == 4000)
    +    spark.sql("drop table if exists old_comp1")
    +    FileFactory.deleteAllFilesOfDir(new File(warehouse1+"/testdb"))
    +  }
     
       override protected def beforeAll(): Unit = {
         drop
       }
     
    -  override def afterAll(): Unit = {
    --- End diff --
    
    unnecessary modification...


---

[GitHub] carbondata issue #2685: [CARBONDATA-2910] Support backward compatability in ...

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

    https://github.com/apache/carbondata/pull/2685
  
    Build Success with Spark 2.1.0, Please check CI http://136.243.101.176:8080/job/ApacheCarbonPRBuilder2.1/7/



---