You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@carbondata.apache.org by jackylk <gi...@git.apache.org> on 2018/02/20 03:39:35 UTC

[GitHub] carbondata pull request #1987: [WIP] Add DataMap developer interface

GitHub user jackylk opened a pull request:

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

    [WIP] Add DataMap developer interface

    Add developer interface for 2 types of DataMap:
    1. IndexDataMap: index type of DataMap to accelerate filter query
    2. OlapDataMap: olap type of DataMap to accelerate olap style query, like SPJG query (select, predicate, join, groupby)
    
     - [X] Any interfaces changed?
     New developer interface added
     - [X] Any backward compatibility impacted?
     No
     - [X] Document update required?
    Yes
     - [X] Testing done
    Yes
     - [X] For large changes, please consider breaking it into sub-tasks under an umbrella JIRA. 
    NA

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

    $ git pull https://github.com/jackylk/incubator-carbondata datamap-interface3

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

    https://github.com/apache/carbondata/pull/1987.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 #1987
    
----
commit 952665a8c1c52f28951463fef989333ae0e6d83e
Author: Jacky Li <ja...@...>
Date:   2018-01-06T12:28:44Z

    [CARBONDATA-1992] Remove partitionId in CarbonTablePath
    
    In CarbonTablePath, there is a deprecated partition id which is always 0, it should be removed to avoid confusion.
    
    This closes #1765

commit 111c3821557820241d1114d87eae2f7cd017e610
Author: Jacky Li <ja...@...>
Date:   2018-01-02T15:46:14Z

    [CARBONDATA-1968] Add external table support
    
    This PR adds support for creating external table with existing carbondata files, using Hive syntax.
    CREATE EXTERNAL TABLE tableName STORED BY 'carbondata' LOCATION 'path'
    
    This closes #1749

commit 80b42ac662ebd2bc243ca91c86b035717223daf4
Author: SangeetaGulia <sa...@...>
Date:   2017-09-21T09:26:26Z

    [CARBONDATA-1827] S3 Carbon Implementation
    
    1.Provide support for s3 in carbondata.
    2.Added S3Example to create carbon table on s3.
    3.Added S3CSVExample to load carbon table using csv from s3.
    
    This closes #1805

commit 71c2d8ca4a3212cff1eedbe78ee03e521f57fbbc
Author: Jacky Li <ja...@...>
Date:   2018-01-31T16:25:31Z

    [REBASE] Solve conflict after rebasing master

commit 15b4e192ee904a2e7c845ac67e0fcf1ba151a683
Author: Jacky Li <ja...@...>
Date:   2018-01-30T13:24:04Z

    [CARBONDATA-2099] Refactor query scan process to improve readability
    
    Unified concepts in scan process flow:
    
    1.QueryModel contains all parameter for scan, it is created by API in CarbonTable. (In future, CarbonTable will be the entry point for various table operations)
    2.Use term ColumnChunk to represent one column in one blocklet, and use ChunkIndex in reader to read specified column chunk
    3.Use term ColumnPage to represent one page in one ColumnChunk
    4.QueryColumn => ProjectionColumn, indicating it is for projection
    
    This closes #1874

commit c3e99681bcd397ed33bc90e8d73b1fd33e0e60f7
Author: Jacky Li <ja...@...>
Date:   2018-01-31T08:14:27Z

    [CARBONDATA-2025] Unify all path construction through CarbonTablePath static method
    
    Refactory CarbonTablePath:
    
    1.Remove CarbonStorePath and use CarbonTablePath only.
    2.Make CarbonTablePath an utility without object creation, it can avoid creating object before using it, thus code is cleaner and GC is less.
    
    This closes #1768

commit e502c59a2d0b95d80db3aff04c749654254eadbe
Author: Jatin <ja...@...>
Date:   2018-01-25T11:23:00Z

    [CARBONDATA-2080] [S3-Implementation] Propagated hadoopConf from driver to executor for s3 implementation in cluster mode.
    
    Problem : hadoopconf was not getting propagated from driver to the executor that's why load was failing to the distributed environment.
    Solution: Setting the Hadoop conf in base class CarbonRDD
    How to verify this PR :
    Execute the load in the cluster mode It should be a success using location s3.
    
    This closes #1860

commit cae74a8cecea74e8899a87dcb7d12e0dec1b8069
Author: sounakr <so...@...>
Date:   2017-09-28T10:51:05Z

    [CARBONDATA-1480]Min Max Index Example for DataMap
    
    Datamap Example. Implementation of Min Max Index through Datamap. And Using the Index while prunning.
    
    This closes #1359

commit e972fd3d5cc8f392d47ca111b2d8f262edb29ac6
Author: ravipesala <ra...@...>
Date:   2017-11-15T14:18:40Z

    [CARBONDATA-1544][Datamap] Datamap FineGrain implementation
    
    Implemented interfaces for FG datamap and integrated to filterscanner to use the pruned bitset from FG datamap.
    FG Query flow as follows.
    1.The user can add FG datamap to any table and implement there interfaces.
    2. Any filter query which hits the table with datamap will call prune method of FGdatamap.
    3. The prune method of FGDatamap return list FineGrainBlocklet , these blocklets contain the information of block, blocklet, page and rowids information as well.
    4. The pruned blocklets are internally wriitten to file and returns only the block , blocklet and filepath information as part of Splits.
    5. Based on the splits scanrdd schedule the tasks.
    6. In filterscanner we check the datamapwriterpath from split and reNoteads the bitset if exists. And pass this bitset as input to it.
    
    This closes #1471

commit cd7eed66bdd7b0044953cb5bf037f6cce38c9e12
Author: xuchuanyin <xu...@...>
Date:   2018-02-08T07:39:45Z

    [HotFix][CheckStyle] Fix import related checkstyle
    
    This closes #1952

commit de92ea9a123b17d903f2d1d4662299315c792954
Author: xuchuanyin <xu...@...>
Date:   2018-02-08T06:35:14Z

    [CARBONDATA-2018][DataLoad] Optimization in reading/writing for sort temp row
    
    Pick up the no-sort fields in the row and pack them as bytes array and skip parsing them during merge sort to reduce CPU consumption
    
    This closes #1792

commit 6dd8b038fc898dbf48ad30adfc870c19eb38e3d0
Author: xuchuanyin <xu...@...>
Date:   2018-02-08T06:42:39Z

    [CARBONDATA-2023][DataLoad] Add size base block allocation in data loading
    
    Carbondata assign blocks to nodes at the beginning of data loading.
    Previous block allocation strategy is block number based and it will
    suffer skewed data problem if the size of input files differs a lot.
    
    We introduced a size based block allocation strategy to optimize data
    loading performance in skewed data scenario.
    
    This closes #1808

commit e5c32ac96f4cf85ef7a42f2a14c31c19418a789b
Author: Jacky Li <ja...@...>
Date:   2018-02-10T02:34:59Z

    Revert "[CARBONDATA-2023][DataLoad] Add size base block allocation in data loading"
    
    This reverts commit 6dd8b038fc898dbf48ad30adfc870c19eb38e3d0.

commit e1c6448cdbfa8d5eab1a861485f953eea3984f1f
Author: Jacky Li <ja...@...>
Date:   2018-02-10T12:11:25Z

    Revert "[CARBONDATA-2018][DataLoad] Optimization in reading/writing for sort temp row"
    
    This reverts commit de92ea9a123b17d903f2d1d4662299315c792954.

commit 7f5751a78c28c8a428fa62a5f82858ac65415c86
Author: Jacky Li <ja...@...>
Date:   2018-02-11T02:12:10Z

    [CARBONDATA-2156] Add interface annotation
    
    InterfaceAudience and InterfaceStability annotation should be added for user and developer
    
    1.InetfaceAudience can be User and Developer
    2.InterfaceStability can be Stable, Evolving, Unstable
    
    This closes #1968

commit a848ccff8aaf3c10970c61b1f85bce56478ca0ac
Author: Jacky Li <ja...@...>
Date:   2018-02-10T11:44:23Z

    [CARBONDATA-1997] Add CarbonWriter SDK API
    
    Added a new module called store-sdk, and added a CarbonWriter API, it can be used to write Carbondata files to a specified folder, without Spark and Hadoop dependency. User can use this API in any environment.
    
    This closes #1967

commit 0d50f65461ae3855db66f44fa06e01174de50ccd
Author: Jacky Li <ja...@...>
Date:   2018-02-11T13:37:04Z

    [CARBONDATA-2159] Remove carbon-spark dependency in store-sdk module
    
    To make assembling JAR of store-sdk module, it should not depend on carbon-spark module
    
    This closes #1970

commit 937bdb867aeac5159f51de3582c4556c949bfb5c
Author: xuchuanyin <xu...@...>
Date:   2018-02-08T06:35:14Z

    [CARBONDATA-2018][DataLoad] Optimization in reading/writing for sort temp row
    
    Pick up the no-sort fields in the row and pack them as bytes array and skip parsing them during merge sort to reduce CPU consumption
    
    This closes #1792

commit fd450b151cb5858504116f560da5cd7a357894e5
Author: xuchuanyin <xu...@...>
Date:   2018-02-08T06:42:39Z

    [CARBONDATA-2023][DataLoad] Add size base block allocation in data loading
    
    Carbondata assign blocks to nodes at the beginning of data loading.
    Previous block allocation strategy is block number based and it will
    suffer skewed data problem if the size of input files differs a lot.
    
    We introduced a size based block allocation strategy to optimize data
    loading performance in skewed data scenario.
    
    This closes #1808

commit 0b15217106a15da8d1607eb8b7b33a29b92595a9
Author: Jacky Li <ja...@...>
Date:   2018-02-13T01:12:09Z

    Support generating assembling JAR for store-sdk module
    
    Support generating assembling JAR for store-sdk module and remove junit dependency
    
    This closes #1976

commit 88c0527f5421d397fb0e907e5fadd608474eadfd
Author: Jacky Li <ja...@...>
Date:   2018-02-20T03:16:53Z

    [CARBONDATA-2186] Add InterfaceAudience.Internal to annotate internal interface
    
    This closes #1986

commit a16ec643cf48cab48f3b479a7839bd870014b463
Author: Ravindra Pesala <ra...@...>
Date:   2017-11-21T10:19:11Z

    Added DataMapChooser and integrated datamapschema syntax to save datamaps to schema
    
    Move the expression conversion of startswith to greaterthan and lessthan tree to carbon layer.

commit 481db5de260e9fc1d6745589997c7b78728e1e36
Author: Jacky Li <ja...@...>
Date:   2018-02-17T15:50:27Z

    resolve conflict

commit bd209d9e0a018d3e88aeeff85d89eba5e287e769
Author: Jacky Li <ja...@...>
Date:   2018-02-18T17:34:15Z

    fix test

commit 303380e97457b39e2e4b626d85b6b2cb58ab5924
Author: Jacky Li <ja...@...>
Date:   2018-02-20T02:53:41Z

    fix test

commit 0d8c17a10455b9015ea2429275e5333d49b088d0
Author: Jacky Li <ja...@...>
Date:   2018-02-20T03:27:12Z

    fix style

commit 7f8a62f9d9b64f78ad010d6bf6cde8e5f8bef0be
Author: Jacky Li <ja...@...>
Date:   2018-02-20T03:30:03Z

    add comment

commit 73dabf8c23b8dd7718eac32861e0959eb13443bb
Author: Jacky Li <ja...@...>
Date:   2018-02-19T19:50:23Z

    datamap-interface

----


---

[GitHub] carbondata issue #1987: [CARBONDATA-2189] Add DataMapProvider developer inte...

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

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



---

[GitHub] carbondata issue #1987: [CARBONDATA-2189] Add DataMapProvider developer inte...

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

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



---

[GitHub] carbondata issue #1987: [CARBONDATA-2189] Add DataMapProvider developer inte...

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

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



---

[GitHub] carbondata issue #1987: [CARBONDATA-2189] Add DataMapProvider developer inte...

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

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



---

[GitHub] carbondata issue #1987: [CARBONDATA-2189] Add DataMapProvider developer inte...

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

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



---

[GitHub] carbondata issue #1987: [CARBONDATA-2189] Add DataMapProvider developer inte...

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

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



---

[GitHub] carbondata issue #1987: [CARBONDATA-2189] Add DataMapProvider developer inte...

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

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



---

[GitHub] carbondata issue #1987: [WIP] Add DataMap developer interface

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

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



---

[GitHub] carbondata issue #1987: [CARBONDATA-2189] Add DataMapProvider developer inte...

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

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



---

[GitHub] carbondata issue #1987: [CARBONDATA-2189] Add DataMapProvider developer inte...

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

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



---

[GitHub] carbondata issue #1987: [CARBONDATA-2189] Add DataMapProvider developer inte...

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

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



---

[GitHub] carbondata issue #1987: [CARBONDATA-2189] Add DataMapProvider developer inte...

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

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



---

[GitHub] carbondata issue #1987: [WIP] Add DataMap developer interface

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

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



---

[GitHub] carbondata issue #1987: [CARBONDATA-2189] Add DataMapProvider developer inte...

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

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



---

[GitHub] carbondata issue #1987: [CARBONDATA-2189] Add DataMapProvider developer inte...

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

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



---

[GitHub] carbondata issue #1987: [CARBONDATA-2189] Add DataMapProvider developer inte...

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

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


---

[GitHub] carbondata issue #1987: [CARBONDATA-2189] Add DataMapProvider developer inte...

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

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



---

[GitHub] carbondata issue #1987: [WIP] Add DataMapProvider developer interface

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

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



---

[GitHub] carbondata issue #1987: [CARBONDATA-2189] Add DataMapProvider developer inte...

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

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



---

[GitHub] carbondata issue #1987: [CARBONDATA-2189] Add DataMapProvider developer inte...

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

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



---

[GitHub] carbondata issue #1987: [CARBONDATA-2189] Add DataMapProvider developer inte...

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

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



---

[GitHub] carbondata issue #1987: [CARBONDATA-2189] Add DataMapProvider developer inte...

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

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



---

[GitHub] carbondata issue #1987: [CARBONDATA-2189] Add DataMapProvider developer inte...

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

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



---

[GitHub] carbondata pull request #1987: [CARBONDATA-2189] Add DataMapProvider develop...

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

    https://github.com/apache/carbondata/pull/1987#discussion_r170245295
  
    --- Diff: core/src/main/java/org/apache/carbondata/core/datamap/DataMapStoreManager.java ---
    @@ -142,46 +142,45 @@ public TableDataMap getDataMap(AbsoluteTableIdentifier identifier, DataMapSchema
        * The datamap is created using datamap name, datamap factory class and table identifier.
        */
       public TableDataMap createAndRegisterDataMap(AbsoluteTableIdentifier identifier,
    -      DataMapSchema dataMapSchema)
    -      throws MalformedDataMapCommandException {
    +      DataMapSchema dataMapSchema) throws MalformedDataMapCommandException {
    +    IndexDataMapFactory indexDataMapFactory;
    +    try {
    +      // try to create datamap by reflection to test whether it is a valid IndexDataMapFactory class
    +      Class<? extends IndexDataMapFactory> factoryClass =
    +          (Class<? extends IndexDataMapFactory>) Class.forName(dataMapSchema.getClassName());
    +      indexDataMapFactory = factoryClass.newInstance();
    +    } catch (ClassNotFoundException e) {
    +      throw new MalformedDataMapCommandException(
    +          "DataMap '" + dataMapSchema.getClassName() + "' not found");
    +    } catch (Throwable e) {
    +      throw new MetadataProcessException(
    +          "failed to create DataMap '" + dataMapSchema.getClassName() + "'", e);
    +    }
    +    return registerDataMap(identifier, dataMapSchema, indexDataMapFactory);
    +  }
    +
    +  public TableDataMap registerDataMap(AbsoluteTableIdentifier identifier,
    --- End diff --
    
    fixed


---

[GitHub] carbondata pull request #1987: [CARBONDATA-2189] Add DataMapProvider develop...

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

    https://github.com/apache/carbondata/pull/1987#discussion_r170226147
  
    --- Diff: integration/spark2/src/main/java/org/apache/carbondata/datamap/TimeseriesDataMapProvider.java ---
    @@ -0,0 +1,47 @@
    +/*
    + * 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.datamap;
    +
    +import java.util.Map;
    +
    +import org.apache.carbondata.core.metadata.schema.table.CarbonTable;
    +import org.apache.carbondata.core.metadata.schema.table.DataMapSchema;
    +
    +import org.apache.spark.sql.SparkSession;
    +import org.apache.spark.sql.execution.command.preaaggregate.PreAggregateTableHelper;
    +import org.apache.spark.sql.execution.command.timeseries.TimeSeriesUtil;
    +import scala.Tuple2;
    +
    +public class TimeseriesDataMapProvider extends PreAggregateDataMapProvider {
    --- End diff --
    
    add interface annotation


---

[GitHub] carbondata issue #1987: [CARBONDATA-2189] Add DataMapProvider developer inte...

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

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



---

[GitHub] carbondata issue #1987: [CARBONDATA-2189] Add DataMapProvider developer inte...

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

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



---

[GitHub] carbondata issue #1987: [CARBONDATA-2189] Add DataMapProvider developer inte...

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

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



---

[GitHub] carbondata issue #1987: [CARBONDATA-2189] Add DataMapProvider developer inte...

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

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



---

[GitHub] carbondata pull request #1987: [CARBONDATA-2189] Add DataMapProvider develop...

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

    https://github.com/apache/carbondata/pull/1987#discussion_r170206534
  
    --- Diff: core/src/main/java/org/apache/carbondata/core/datamap/DataMapStoreManager.java ---
    @@ -142,46 +142,45 @@ public TableDataMap getDataMap(AbsoluteTableIdentifier identifier, DataMapSchema
        * The datamap is created using datamap name, datamap factory class and table identifier.
        */
       public TableDataMap createAndRegisterDataMap(AbsoluteTableIdentifier identifier,
    -      DataMapSchema dataMapSchema)
    -      throws MalformedDataMapCommandException {
    +      DataMapSchema dataMapSchema) throws MalformedDataMapCommandException {
    +    IndexDataMapFactory indexDataMapFactory;
    +    try {
    +      // try to create datamap by reflection to test whether it is a valid IndexDataMapFactory class
    +      Class<? extends IndexDataMapFactory> factoryClass =
    +          (Class<? extends IndexDataMapFactory>) Class.forName(dataMapSchema.getClassName());
    +      indexDataMapFactory = factoryClass.newInstance();
    +    } catch (ClassNotFoundException e) {
    +      throw new MalformedDataMapCommandException(
    +          "DataMap '" + dataMapSchema.getClassName() + "' not found");
    +    } catch (Throwable e) {
    +      throw new MetadataProcessException(
    +          "failed to create DataMap '" + dataMapSchema.getClassName() + "'", e);
    +    }
    +    return registerDataMap(identifier, dataMapSchema, indexDataMapFactory);
    +  }
    +
    +  public TableDataMap registerDataMap(AbsoluteTableIdentifier identifier,
    --- End diff --
    
    use private modifier


---

[GitHub] carbondata pull request #1987: [CARBONDATA-2189] Add DataMapProvider develop...

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

    https://github.com/apache/carbondata/pull/1987#discussion_r170245271
  
    --- Diff: integration/spark2/src/main/java/org/apache/carbondata/datamap/DataMapManager.java ---
    @@ -0,0 +1,53 @@
    +/*
    + * 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.datamap;
    +
    +import org.apache.carbondata.core.metadata.schema.table.DataMapSchema;
    +
    +import static org.apache.carbondata.core.metadata.schema.datamap.DataMapProvider.PREAGGREGATE;
    +import static org.apache.carbondata.core.metadata.schema.datamap.DataMapProvider.TIMESERIES;
    +
    +public class DataMapManager {
    +
    +  private static DataMapManager INSTANCE;
    +
    +  private DataMapManager() { }
    +
    +  public static DataMapManager get() {
    --- End diff --
    
    fixed


---

[GitHub] carbondata issue #1987: [WIP] Add DataMapProvider developer interface

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

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



---

[GitHub] carbondata issue #1987: [CARBONDATA-2189] Add DataMapProvider developer inte...

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

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



---

[GitHub] carbondata issue #1987: [CARBONDATA-2189] Add DataMapProvider developer inte...

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

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



---

[GitHub] carbondata issue #1987: [CARBONDATA-2189] Add DataMapProvider developer inte...

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

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



---

[GitHub] carbondata issue #1987: [CARBONDATA-2189] Add DataMapProvider developer inte...

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

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



---

[GitHub] carbondata issue #1987: [CARBONDATA-2189] Add DataMapProvider developer inte...

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

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



---

[GitHub] carbondata issue #1987: [CARBONDATA-2189] Add DataMapProvider developer inte...

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

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



---

[GitHub] carbondata issue #1987: [WIP] Add DataMapProvider developer interface

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

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



---

[GitHub] carbondata pull request #1987: [CARBONDATA-2189] Add DataMapProvider develop...

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

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


---

[GitHub] carbondata issue #1987: [WIP] Add DataMapProvider developer interface

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

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



---

[GitHub] carbondata issue #1987: [CARBONDATA-2189] Add DataMapProvider developer inte...

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

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



---

[GitHub] carbondata issue #1987: [WIP] Add DataMapProvider developer interface

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

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



---

[GitHub] carbondata issue #1987: [CARBONDATA-2189] Add DataMapProvider developer inte...

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

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



---

[GitHub] carbondata pull request #1987: [CARBONDATA-2189] Add DataMapProvider develop...

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

    https://github.com/apache/carbondata/pull/1987#discussion_r170197506
  
    --- Diff: integration/spark2/src/main/java/org/apache/carbondata/datamap/DataMapManager.java ---
    @@ -0,0 +1,53 @@
    +/*
    + * 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.datamap;
    +
    +import org.apache.carbondata.core.metadata.schema.table.DataMapSchema;
    +
    +import static org.apache.carbondata.core.metadata.schema.datamap.DataMapProvider.PREAGGREGATE;
    +import static org.apache.carbondata.core.metadata.schema.datamap.DataMapProvider.TIMESERIES;
    +
    +public class DataMapManager {
    +
    +  private static DataMapManager INSTANCE;
    +
    +  private DataMapManager() { }
    +
    +  public static DataMapManager get() {
    --- End diff --
    
    add synchronized


---

[GitHub] carbondata issue #1987: [CARBONDATA-2189] Add DataMapProvider developer inte...

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

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



---

[GitHub] carbondata issue #1987: [CARBONDATA-2189] Add DataMapProvider developer inte...

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

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



---

[GitHub] carbondata issue #1987: [CARBONDATA-2189] Add DataMapProvider developer inte...

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

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



---

[GitHub] carbondata issue #1987: [CARBONDATA-2189] Add DataMapProvider developer inte...

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

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



---

[GitHub] carbondata issue #1987: [CARBONDATA-2189] Add DataMapProvider developer inte...

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

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



---

[GitHub] carbondata issue #1987: [WIP] Add DataMapProvider developer interface

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

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



---

[GitHub] carbondata issue #1987: [CARBONDATA-2189] Add DataMapProvider developer inte...

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

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



---

[GitHub] carbondata issue #1987: [CARBONDATA-2189] Add DataMapProvider developer inte...

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

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



---

[GitHub] carbondata issue #1987: [CARBONDATA-2189] Add DataMapProvider developer inte...

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

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



---