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 2017/11/16 13:27:26 UTC

[GitHub] carbondata pull request #1510: [WIP] Supported DataMap chooser and expressio...

GitHub user ravipesala opened a pull request:

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

    [WIP] Supported DataMap chooser and expression for supporting multiple datamaps in single query

      This PR supports 3 features.
     1. Load datamaps from the DataMapSchema which are created through DDL.
     2. DataMap Chooser: It chooses the datamap out of available datamaps based on simple logic. Like if there is filter condition on column1 then for supposing 2 datamaps(1. column1 2. column1+column2) are supporting this column then we choose the datamap which has fewer columns that is the first datamap.
     3. Expression support: Based on the filter expressions we convert them to the possible DataMap expressions and do apply expression on it. 
      For example, there are 2 datamaps available on table1 
      Datamap1 : column1
      Datamap2 : column2 
      Query: `select * from table1 where column1 ='a' and column2 = `b` `
      For the above query, we create datamap expression as `AndDataMapExpression(Datamap1, DataMap2)`. So for the above query both the datamaps are included and the output of them will be applied AND condition to improve the performance
    
     - [X] Any interfaces changed?
       Yes, DataMap interfaces are changed in order to support datamap expression.
     - [X] Any backward compatibility impacted?
       NO
     
     - [X] Document update required?
       Yes, dev guide need to be updated
     - [X] Testing done
           Tests are added
           
     - [ ] 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 datamap-fg-chooser1

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

    https://github.com/apache/carbondata/pull/1510.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 #1510
    
----
commit c2b898468acdfb3d1738c34ce0557100700f6f80
Author: ravipesala <ra...@gmail.com>
Date:   2017-11-15T14:18:40Z

    Added FG implementation and testcases

commit 3a405355874c051fd640cc3ad00049a1a73fd8c6
Author: ravipesala <ra...@gmail.com>
Date:   2017-11-08T16:00:32Z

    Added DataMapChooser

----


---

[GitHub] carbondata issue #1510: [CARBONDATA-1543] Supported DataMap chooser and expr...

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

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



---

[GitHub] carbondata issue #1510: [CARBONDATA-1543] Supported DataMap chooser and expr...

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

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



---

[GitHub] carbondata issue #1510: [WIP] Supported DataMap chooser and expression for s...

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

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



---

[GitHub] carbondata issue #1510: [CARBONDATA-1543] Supported DataMap chooser and expr...

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

    https://github.com/apache/carbondata/pull/1510
  
    Thanks. 
    LTGM


---

[GitHub] carbondata issue #1510: [CARBONDATA-1543] Supported DataMap chooser and expr...

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

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



---

[GitHub] carbondata issue #1510: [WIP] Supported DataMap chooser and expression for s...

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

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



---

[GitHub] carbondata issue #1510: [CARBONDATA-1543] Supported DataMap chooser and expr...

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

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



---

[GitHub] carbondata issue #1510: [CARBONDATA-1543] Supported DataMap chooser and expr...

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

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



---

[GitHub] carbondata pull request #1510: [CARBONDATA-1543] Supported DataMap chooser a...

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

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


---

[GitHub] carbondata issue #1510: [WIP] Supported DataMap chooser and expression for s...

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

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



---

[GitHub] carbondata issue #1510: [CARBONDATA-1543] Supported DataMap chooser and expr...

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

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



---

[GitHub] carbondata issue #1510: [CARBONDATA-1543] Supported DataMap chooser and expr...

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

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



---

[GitHub] carbondata issue #1510: [WIP] Supported DataMap chooser and expression for s...

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

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



---

[GitHub] carbondata pull request #1510: [CARBONDATA-1543] Supported DataMap chooser a...

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

    https://github.com/apache/carbondata/pull/1510#discussion_r156602474
  
    --- Diff: core/src/main/java/org/apache/carbondata/core/datamap/DataMapChooser.java ---
    @@ -0,0 +1,269 @@
    +/*
    + * 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.datamap;
    +
    +import java.util.ArrayList;
    +import java.util.Collections;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Set;
    +
    +import org.apache.carbondata.core.datamap.dev.expr.AndDataMapExprWrapper;
    +import org.apache.carbondata.core.datamap.dev.expr.DataMapExprWrapper;
    +import org.apache.carbondata.core.datamap.dev.expr.DataMapExprWrapperImpl;
    +import org.apache.carbondata.core.datamap.dev.expr.OrDataMapExprWrapper;
    +import org.apache.carbondata.core.metadata.schema.table.CarbonTable;
    +import org.apache.carbondata.core.scan.expression.ColumnExpression;
    +import org.apache.carbondata.core.scan.expression.Expression;
    +import org.apache.carbondata.core.scan.expression.logical.AndExpression;
    +import org.apache.carbondata.core.scan.expression.logical.OrExpression;
    +import org.apache.carbondata.core.scan.filter.intf.ExpressionType;
    +import org.apache.carbondata.core.scan.filter.resolver.FilterResolverIntf;
    +import org.apache.carbondata.core.scan.filter.resolver.resolverinfo.TrueConditionalResolverImpl;
    +
    +/**
    + * This chooser does 2 jobs.
    + * 1. Based on filter expression it converts the available datamaps to datamap expression.
    + *   For example, there are 2 datamaps available on table1
    + *   Datamap1 : column1
    + *   Datamap2 : column2
    + *   Query: select * from table1 where column1 ='a' and column2 =b
    + *   For the above query, we create datamap expression as AndDataMapExpression(Datamap1, DataMap2).
    + *   So for the above query both the datamaps are included and the output of them will be
    + *   applied AND condition to improve the performance
    + *
    + * 2. It chooses the datamap out of available datamaps based on simple logic.
    + *   Like if there is filter condition on column1 then for
    + *   supposing 2 datamaps(1. column1 2. column1+column2) are supporting this column then we choose
    + *   the datamap which has fewer columns that is the first datamap.
    + */
    +public class DataMapChooser {
    +
    +  public DataMapExprWrapper choose(CarbonTable carbonTable,
    +      FilterResolverIntf resolverIntf) {
    +    if (resolverIntf != null) {
    +      Expression expression = resolverIntf.getFilterExpression();
    +      // First check for FG datamaps if any exist
    +      List<TableDataMap> allDataMapFG =
    +          DataMapStoreManager.getInstance().getAllDataMap(carbonTable, DataMapType.FG);
    +      ExpressionTuple tuple = selectDataMap(expression, allDataMapFG);
    +      if (tuple.dataMapExprWrapper == null) {
    +        // Check for CG datamap
    +        List<TableDataMap> allDataMapCG =
    +            DataMapStoreManager.getInstance().getAllDataMap(carbonTable, DataMapType.CG);
    +        tuple = selectDataMap(expression, allDataMapCG);
    +      }
    +      if (tuple.dataMapExprWrapper != null) {
    +        return tuple.dataMapExprWrapper;
    +      }
    +    }
    +    // Return the default datamap if no other datamap exists.
    +    return new DataMapExprWrapperImpl(DataMapStoreManager.getInstance()
    --- End diff --
    
    make the line wrap easier to read, like:
    ```
        return new DataMapExprWrapperImpl(
            DataMapStoreManager.getInstance().getDefaultDataMap(
                carbonTable.getAbsoluteTableIdentifier()), resolverIntf);
    ```


---

[GitHub] carbondata issue #1510: [CARBONDATA-1543] Supported DataMap chooser and expr...

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

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



---

[GitHub] carbondata issue #1510: [CARBONDATA-1543] Supported DataMap chooser and expr...

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

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



---

[GitHub] carbondata issue #1510: [WIP] Supported DataMap chooser and expression for s...

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

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



---

[GitHub] carbondata issue #1510: [CARBONDATA-1543] Supported DataMap chooser and expr...

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

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



---

[GitHub] carbondata issue #1510: [CARBONDATA-1543] Supported DataMap chooser and expr...

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

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



---

[GitHub] carbondata issue #1510: [CARBONDATA-1543] Supported DataMap chooser and expr...

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

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



---

[GitHub] carbondata issue #1510: [CARBONDATA-1543] Supported DataMap chooser and expr...

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

    https://github.com/apache/carbondata/pull/1510
  
    @jackylk Please review, I have moved the `like` expression conversion to carbon layer in this PR itself, so like expression can be accessed as `STARTSWITH` or `ENDSWITH` or `CONTAINSWITH` to the datamap


---

[GitHub] carbondata issue #1510: [CARBONDATA-1543] Supported DataMap chooser and expr...

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

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



---

[GitHub] carbondata issue #1510: [CARBONDATA-1543] Supported DataMap chooser and expr...

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

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



---

[GitHub] carbondata issue #1510: [CARBONDATA-1543] Supported DataMap chooser and expr...

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

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



---

[GitHub] carbondata issue #1510: [CARBONDATA-1543] Supported DataMap chooser and expr...

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

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



---

[GitHub] carbondata issue #1510: [CARBONDATA-1543] Supported DataMap chooser and expr...

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

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



---

[GitHub] carbondata issue #1510: [CARBONDATA-1543] Supported DataMap chooser and expr...

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

    https://github.com/apache/carbondata/pull/1510
  
    @xubochin LuceneDataMap can only include the expression type which it supports inside metadata so that remaining expressions carbon can handle it. 


---

[GitHub] carbondata issue #1510: [WIP] Supported DataMap chooser and expression for s...

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

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



---

[GitHub] carbondata issue #1510: [CARBONDATA-1543] Supported DataMap chooser and expr...

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

    https://github.com/apache/carbondata/pull/1510
  
    except ENDSWITH , other like string can pass to Lucene ?


---

[GitHub] carbondata issue #1510: [CARBONDATA-1543] Supported DataMap chooser and expr...

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

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



---

[GitHub] carbondata issue #1510: [CARBONDATA-1543] Supported DataMap chooser and expr...

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

    https://github.com/apache/carbondata/pull/1510
  
    Build Failed with Spark 2.2.0, Please check CI http://88.99.58.216:8080/job/ApacheCarbonPRBuilder/1092/



---

[GitHub] carbondata issue #1510: [WIP] Supported DataMap chooser and expression for s...

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

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



---

[GitHub] carbondata issue #1510: [WIP] Supported DataMap chooser and expression for s...

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

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



---

[GitHub] carbondata issue #1510: [WIP] Supported DataMap chooser and expression for s...

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

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



---

[GitHub] carbondata issue #1510: [CARBONDATA-1543] Supported DataMap chooser and expr...

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

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



---

[GitHub] carbondata issue #1510: [CARBONDATA-1543] Supported DataMap chooser and expr...

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

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



---

[GitHub] carbondata issue #1510: [CARBONDATA-1543] Supported DataMap chooser and expr...

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

    https://github.com/apache/carbondata/pull/1510
  
    Thought this code , I understand that : STARTSWITH is "%xxx",ENDSWITH is "%xxx",CONTAINSWITH is "xxx", is it right ?  Luceen only use STARTSWITH. 


---

[GitHub] carbondata issue #1510: [WIP] Supported DataMap chooser and expression for s...

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

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



---

[GitHub] carbondata issue #1510: [CARBONDATA-1543] Supported DataMap chooser and expr...

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

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



---

[GitHub] carbondata issue #1510: [CARBONDATA-1543] Supported DataMap chooser and expr...

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

    https://github.com/apache/carbondata/pull/1510
  
    Merged into datamap branch


---

[GitHub] carbondata issue #1510: [CARBONDATA-1543] Supported DataMap chooser and expr...

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

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



---

[GitHub] carbondata issue #1510: [CARBONDATA-1543] Supported DataMap chooser and expr...

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

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



---

[GitHub] carbondata issue #1510: [CARBONDATA-1543] Supported DataMap chooser and expr...

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

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



---

[GitHub] carbondata issue #1510: [WIP] Supported DataMap chooser and expression for s...

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

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



---

[GitHub] carbondata issue #1510: [CARBONDATA-1543] Supported DataMap chooser and expr...

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

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



---

[GitHub] carbondata issue #1510: [CARBONDATA-1543] Supported DataMap chooser and expr...

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

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



---

[GitHub] carbondata issue #1510: [WIP] Supported DataMap chooser and expression for s...

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

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



---

[GitHub] carbondata issue #1510: [CARBONDATA-1543] Supported DataMap chooser and expr...

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

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



---

[GitHub] carbondata issue #1510: [CARBONDATA-1543] Supported DataMap chooser and expr...

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

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



---

[GitHub] carbondata issue #1510: [CARBONDATA-1543] Supported DataMap chooser and expr...

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

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



---

[GitHub] carbondata issue #1510: [CARBONDATA-1543] Supported DataMap chooser and expr...

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

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



---

[GitHub] carbondata issue #1510: [CARBONDATA-1543] Supported DataMap chooser and expr...

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

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



---

[GitHub] carbondata issue #1510: [CARBONDATA-1543] Supported DataMap chooser and expr...

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

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



---

[GitHub] carbondata issue #1510: [CARBONDATA-1543] Supported DataMap chooser and expr...

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

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



---

[GitHub] carbondata issue #1510: [CARBONDATA-1543] Supported DataMap chooser and expr...

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

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



---

[GitHub] carbondata issue #1510: [CARBONDATA-1543] Supported DataMap chooser and expr...

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

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



---

[GitHub] carbondata issue #1510: [WIP] Supported DataMap chooser and expression for s...

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

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



---

[GitHub] carbondata issue #1510: [WIP] Supported DataMap chooser and expression for s...

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

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



---

[GitHub] carbondata issue #1510: [CARBONDATA-1543] Supported DataMap chooser and expr...

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

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


---

[GitHub] carbondata issue #1510: [WIP] Supported DataMap chooser and expression for s...

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

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



---