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

[GitHub] carbondata pull request #2290: [CARBONDATA-2389] Search mode support lucene ...

GitHub user xubo245 opened a pull request:

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

    [CARBONDATA-2389] Search mode support lucene datamap

    Search mode support lucene datamap
    Be sure to do all of the following checklist to help us incorporate 
    your contribution quickly and easily:
    
     - [ ] Any interfaces changed?
     Yes,org.apache.carbondata.core.datamap.dev.expr.DataMapExprWrapper#prune(org.apache.carbondata.core.datamap.DataMapDistributable, java.util.List<org.apache.carbondata.core.indexstore.PartitionSpec>)
     - [ ] Any backward compatibility impacted?
     No
     - [ ] Document update required?
    Yes
     - [ ] 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.
           add some test case
     - [ ] For large changes, please consider breaking it into sub-tasks under an umbrella JIRA. 
    No

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

    $ git pull https://github.com/xubo245/carbondata CARBONDATA-2389-searchModeLuceneDM

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

    https://github.com/apache/carbondata/pull/2290.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 #2290
    
----
commit 550e331114709c233da764ee4cf32c351ad01e2b
Author: xubo245 <60...@...>
Date:   2018-05-09T13:20:59Z

    [CARBONDATA-2389] Search mode support lucene datamap

----


---

[GitHub] carbondata pull request #2290: [CARBONDATA-2389] Search mode support lucene ...

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

    https://github.com/apache/carbondata/pull/2290#discussion_r191298984
  
    --- Diff: hadoop/src/main/java/org/apache/carbondata/hadoop/CarbonRecordReader.java ---
    @@ -77,8 +77,10 @@ public void initialize(InputSplit inputSplit, TaskAttemptContext context)
         } else {
           throw new RuntimeException("unsupported input split type: " + inputSplit);
         }
    -    List<TableBlockInfo> tableBlockInfoList = CarbonInputSplit.createBlocks(splitList);
    -    queryModel.setTableBlockInfos(tableBlockInfoList);
    +    if (queryModel.getTableBlockInfos().isEmpty()) {
    --- End diff --
    
    please add comment


---

[GitHub] carbondata pull request #2290: [WIP][CARBONDATA-2389] Search mode support lu...

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

    https://github.com/apache/carbondata/pull/2290#discussion_r187768465
  
    --- Diff: core/src/main/java/org/apache/carbondata/core/datamap/DataMapStoreManager.java ---
    @@ -90,13 +90,18 @@ private DataMapStoreManager() {
           String dbName = carbonTable.getDatabaseName();
           String tableName = carbonTable.getTableName();
           String dmName = dataMap.getDataMapSchema().getDataMapName();
    -      boolean isDmVisible = sessionInfo.getSessionParams().getProperty(
    -          String.format("%s%s.%s.%s", CarbonCommonConstants.CARBON_DATAMAP_VISIBLE,
    -              dbName, tableName, dmName), "true").trim().equalsIgnoreCase("true");
    -      if (!isDmVisible) {
    -        LOGGER.warn(String.format("Ignore invisible datamap %s on table %s.%s",
    -            dmName, dbName, tableName));
    -        dataMapIterator.remove();
    +      if (sessionInfo != null) {
    --- End diff --
    
    When will sessionInfo be null?


---

[GitHub] carbondata issue #2290: [CARBONDATA-2389] Search mode support lucene datamap

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

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



---

[GitHub] carbondata pull request #2290: [WIP][CARBONDATA-2389] Search mode support lu...

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

    https://github.com/apache/carbondata/pull/2290#discussion_r187872177
  
    --- Diff: core/src/main/java/org/apache/carbondata/core/datamap/dev/expr/AndDataMapExprWrapper.java ---
    @@ -59,6 +60,21 @@ public AndDataMapExprWrapper(DataMapExprWrapper left, DataMapExprWrapper right,
         return andBlocklets;
       }
     
    +  @Override
    +  public List<ExtendedBlocklet> prune(DataMapDistributable distributable,
    --- End diff --
    
    changed


---

[GitHub] carbondata issue #2290: [CARBONDATA-2389] Search mode support lucene datamap

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

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



---

[GitHub] carbondata issue #2290: [WIP][CARBONDATA-2389] Search mode support lucene da...

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

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



---

[GitHub] carbondata issue #2290: [CARBONDATA-2389] Search mode support lucene datamap

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

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


---

[GitHub] carbondata pull request #2290: [CARBONDATA-2389] Search mode support lucene ...

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

    https://github.com/apache/carbondata/pull/2290#discussion_r191299007
  
    --- Diff: datamap/lucene/src/main/java/org/apache/carbondata/datamap/lucene/LuceneDataMapFactoryBase.java ---
    @@ -235,7 +236,8 @@ public DataMapBuilder createBuilder(Segment segment, String shardName) {
         }
         for (CarbonFile indexDir : indexDirs) {
           // Filter out the tasks which are filtered through CG datamap.
    -      if (!segment.getFilteredIndexShardNames().contains(indexDir.getName())) {
    +      if (getDataMapLevel() != DataMapLevel.FG &&
    +              !segment.getFilteredIndexShardNames().contains(indexDir.getName())) {
    --- End diff --
    
    incorrect identation


---

[GitHub] carbondata pull request #2290: [CARBONDATA-2389] Search mode support lucene ...

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

    https://github.com/apache/carbondata/pull/2290#discussion_r191299224
  
    --- Diff: core/src/main/java/org/apache/carbondata/core/scan/scanner/impl/BlockletFilterScanner.java ---
    @@ -169,6 +169,10 @@ private BlockletScannedResult executeFilter(RawBlockletColumnChunks rawBlockletC
         // apply filter on actual data, for each page
         BitSetGroup bitSetGroup = this.filterExecuter.applyFilter(rawBlockletColumnChunks,
             useBitSetPipeLine);
    +    // if bitSetGroup is nul, then new BitSetGroup object, which can avoid NPE
    --- End diff --
    
    Please describe why it is null, and when it is null


---

[GitHub] carbondata issue #2290: [CARBONDATA-2389] Search mode support lucene datamap

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

    https://github.com/apache/carbondata/pull/2290
  
    @xuchuanyin I am not sure, no plan yet


---

[GitHub] carbondata issue #2290: [WIP][CARBONDATA-2389] Search mode support lucene da...

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

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



---

[GitHub] carbondata issue #2290: [CARBONDATA-2389] Search mode support lucene datamap

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

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



---

[GitHub] carbondata issue #2290: [CARBONDATA-2389] Search mode support lucene datamap

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

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



---

[GitHub] carbondata pull request #2290: [CARBONDATA-2389] Search mode support lucene ...

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

    https://github.com/apache/carbondata/pull/2290#discussion_r191447507
  
    --- Diff: datamap/lucene/src/main/java/org/apache/carbondata/datamap/lucene/LuceneDataMapFactoryBase.java ---
    @@ -235,7 +236,8 @@ public DataMapBuilder createBuilder(Segment segment, String shardName) {
         }
         for (CarbonFile indexDir : indexDirs) {
           // Filter out the tasks which are filtered through CG datamap.
    -      if (!segment.getFilteredIndexShardNames().contains(indexDir.getName())) {
    +      if (getDataMapLevel() != DataMapLevel.FG &&
    +              !segment.getFilteredIndexShardNames().contains(indexDir.getName())) {
    --- End diff --
    
    ok,done


---

[GitHub] carbondata pull request #2290: [CARBONDATA-2389] Search mode support lucene ...

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

    https://github.com/apache/carbondata/pull/2290#discussion_r191417277
  
    --- Diff: store/search/src/main/scala/org/apache/spark/rpc/Master.scala ---
    @@ -142,7 +142,7 @@ class Master(sparkConf: SparkConf) {
           (address, schedulable.ref.ask[ShutdownResponse](ShutdownRequest("user")))
         }
         futures.foreach { case (address, future) =>
    -      ThreadUtils.awaitResult(future, Duration.apply("10s"))
    +      ThreadUtils.awaitResult(future, Duration.apply("100s"))
    --- End diff --
    
    ok, down, default value is 10s


---

[GitHub] carbondata issue #2290: [WIP][CARBONDATA-2389] Search mode support lucene da...

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

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



---

[GitHub] carbondata issue #2290: [CARBONDATA-2389] Search mode support lucene datamap

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

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



---

[GitHub] carbondata issue #2290: [CARBONDATA-2389] Search mode support lucene datamap

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

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



---

[GitHub] carbondata issue #2290: [CARBONDATA-2389] Search mode support lucene datamap

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

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



---

[GitHub] carbondata issue #2290: [WIP][CARBONDATA-2389] Search mode support lucene da...

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

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



---

[GitHub] carbondata issue #2290: [WIP][CARBONDATA-2389] Search mode support lucene da...

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

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



---

[GitHub] carbondata issue #2290: [CARBONDATA-2389] Search mode support lucene datamap

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

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



---

[GitHub] carbondata issue #2290: [CARBONDATA-2389] Search mode support lucene datamap

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

    https://github.com/apache/carbondata/pull/2290
  
    please remove the CSV in this PR


---

[GitHub] carbondata pull request #2290: [WIP][CARBONDATA-2389] Search mode support lu...

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

    https://github.com/apache/carbondata/pull/2290#discussion_r187768472
  
    --- Diff: core/src/main/java/org/apache/carbondata/core/datamap/dev/expr/AndDataMapExprWrapper.java ---
    @@ -59,6 +60,21 @@ public AndDataMapExprWrapper(DataMapExprWrapper left, DataMapExprWrapper right,
         return andBlocklets;
       }
     
    +  @Override
    +  public List<ExtendedBlocklet> prune(DataMapDistributable distributable,
    --- End diff --
    
    Is the indent correct?


---

[GitHub] carbondata issue #2290: [CARBONDATA-2389] Search mode support lucene datamap

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

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



---

[GitHub] carbondata issue #2290: [CARBONDATA-2389] Search mode support lucene datamap

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

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



---

[GitHub] carbondata issue #2290: [CARBONDATA-2389] Search mode support lucene datamap

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

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


---

[GitHub] carbondata issue #2290: [CARBONDATA-2389] Search mode support lucene datamap

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

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



---

[GitHub] carbondata issue #2290: [CARBONDATA-2389] Search mode support lucene datamap

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

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



---

[GitHub] carbondata issue #2290: [CARBONDATA-2389] Search mode support lucene datamap

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

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



---

[GitHub] carbondata issue #2290: [WIP][CARBONDATA-2389] Search mode support lucene da...

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

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



---

[GitHub] carbondata pull request #2290: [CARBONDATA-2389] Search mode support lucene ...

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

    https://github.com/apache/carbondata/pull/2290#discussion_r189758718
  
    --- Diff: core/src/main/java/org/apache/carbondata/core/readcommitter/LatestFilesReadCommittedScope.java ---
    @@ -40,9 +40,26 @@
     public class LatestFilesReadCommittedScope implements ReadCommittedScope {
     
       private String carbonFilePath;
    +  private String segmentId;
       private ReadCommittedIndexFileSnapShot readCommittedIndexFileSnapShot;
       private LoadMetadataDetails[] loadMetadataDetails;
     
    +  /**
    +   * a new constructor of this class, which supports obtain lucene index in search mode
    +   *
    +   * @param path      carbon file path
    +   * @param segmentId segment id
    +   */
    +  public LatestFilesReadCommittedScope(String path, String segmentId) {
    --- End diff --
    
    Can not operate like this, because this(path) has takeCarbonIndexFileSnapShot method, which dependency on segmentId.


---

[GitHub] carbondata issue #2290: [CARBONDATA-2389] Search mode support lucene datamap

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

    https://github.com/apache/carbondata/pull/2290
  
    Please review it. @jackylk @ravipesala @QiangCai  


---

[GitHub] carbondata pull request #2290: [WIP][CARBONDATA-2389] Search mode support lu...

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

    https://github.com/apache/carbondata/pull/2290#discussion_r187874636
  
    --- Diff: core/src/main/java/org/apache/carbondata/core/datamap/dev/expr/DataMapExprWrapper.java ---
    @@ -40,6 +41,18 @@
       List<ExtendedBlocklet> prune(List<Segment> segments, List<PartitionSpec> partitionsToPrune)
           throws IOException;
     
    +  /**
    +   * prune blocklet according distributable
    +   *
    +   * @param distributable     distributable
    +   * @param partitionsToPrune partitions to prune
    +   * @return the pruned ExtendedBlocklet list
    +   * @throws IOException
    +   */
    +  List<ExtendedBlocklet> prune(DataMapDistributable distributable,
    --- End diff --
    
    ok


---

[GitHub] carbondata issue #2290: [WIP][CARBONDATA-2389] Search mode support lucene da...

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

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



---

[GitHub] carbondata issue #2290: [CARBONDATA-2389] Search mode support lucene datamap

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

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


---

[GitHub] carbondata pull request #2290: [WIP][CARBONDATA-2389] Search mode support lu...

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

    https://github.com/apache/carbondata/pull/2290#discussion_r187768616
  
    --- Diff: core/src/main/java/org/apache/carbondata/core/readcommitter/LatestFilesReadCommittedScope.java ---
    @@ -40,9 +40,26 @@
     public class LatestFilesReadCommittedScope implements ReadCommittedScope {
     
       private String carbonFilePath;
    +  private String segmentId;
       private ReadCommittedIndexFileSnapShot readCommittedIndexFileSnapShot;
       private LoadMetadataDetails[] loadMetadataDetails;
     
    +  /**
    +   * a new constructor of this class, which supports obtain lucene index in search mode
    +   *
    +   * @param path      carbon file path
    +   * @param segmentId segment id
    +   */
    +  public LatestFilesReadCommittedScope(String path, String segmentId) {
    --- End diff --
    
    too much duplicate codes there. It can internally call `this(path)` to reduce the code


---

[GitHub] carbondata issue #2290: [CARBONDATA-2389] Search mode support lucene datamap

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

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



---

[GitHub] carbondata pull request #2290: [CARBONDATA-2389] Search mode support lucene ...

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

    https://github.com/apache/carbondata/pull/2290#discussion_r188127574
  
    --- Diff: core/src/main/java/org/apache/carbondata/core/datamap/dev/expr/AndDataMapExprWrapper.java ---
    @@ -59,6 +60,21 @@ public AndDataMapExprWrapper(DataMapExprWrapper left, DataMapExprWrapper right,
         return andBlocklets;
       }
     
    +  @Override
    +  public List<ExtendedBlocklet> prune(DataMapDistributable distributable,
    +                                      List<PartitionSpec> partitionsToPrune)
    +          throws IOException {
    +    List<ExtendedBlocklet> leftPrune = left.prune(distributable, partitionsToPrune);
    +    List<ExtendedBlocklet> rightPrune = right.prune(distributable, partitionsToPrune);
    +    List<ExtendedBlocklet> andBlocklets = new ArrayList<>();
    +    for (ExtendedBlocklet blocklet : leftPrune) {
    +      if (rightPrune.contains(blocklet)) {
    --- End diff --
    
    I mean that the `rightPrune.contains(blocklet)`will internally call the method `equals` in `ExtendBlocklet`...


---

[GitHub] carbondata pull request #2290: [WIP][CARBONDATA-2389] Search mode support lu...

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

    https://github.com/apache/carbondata/pull/2290#discussion_r187874345
  
    --- Diff: core/src/main/java/org/apache/carbondata/core/datamap/dev/expr/AndDataMapExprWrapper.java ---
    @@ -59,6 +60,21 @@ public AndDataMapExprWrapper(DataMapExprWrapper left, DataMapExprWrapper right,
         return andBlocklets;
       }
     
    +  @Override
    +  public List<ExtendedBlocklet> prune(DataMapDistributable distributable,
    +                                      List<PartitionSpec> partitionsToPrune)
    +          throws IOException {
    +    List<ExtendedBlocklet> leftPrune = left.prune(distributable, partitionsToPrune);
    +    List<ExtendedBlocklet> rightPrune = right.prune(distributable, partitionsToPrune);
    +    List<ExtendedBlocklet> andBlocklets = new ArrayList<>();
    +    for (ExtendedBlocklet blocklet : leftPrune) {
    +      if (rightPrune.contains(blocklet)) {
    --- End diff --
    
    no use this method of the class.
    only use the method:
    
    org.apache.carbondata.core.datamap.dev.expr.DataMapExprWrapperImpl#prune(org.apache.carbondata.core.datamap.DataMapDistributable, java.util.List<org.apache.carbondata.core.indexstore.PartitionSpec>)


---

[GitHub] carbondata pull request #2290: [CARBONDATA-2389] Search mode support lucene ...

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

    https://github.com/apache/carbondata/pull/2290#discussion_r189598687
  
    --- Diff: core/src/main/java/org/apache/carbondata/core/readcommitter/LatestFilesReadCommittedScope.java ---
    @@ -40,9 +40,26 @@
     public class LatestFilesReadCommittedScope implements ReadCommittedScope {
     
       private String carbonFilePath;
    +  private String segmentId;
       private ReadCommittedIndexFileSnapShot readCommittedIndexFileSnapShot;
       private LoadMetadataDetails[] loadMetadataDetails;
     
    +  /**
    +   * a new constructor of this class, which supports obtain lucene index in search mode
    +   *
    +   * @param path      carbon file path
    +   * @param segmentId segment id
    +   */
    +  public LatestFilesReadCommittedScope(String path, String segmentId) {
    --- End diff --
    
    ok


---

[GitHub] carbondata pull request #2290: [CARBONDATA-2389] Search mode support lucene ...

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

    https://github.com/apache/carbondata/pull/2290#discussion_r188128386
  
    --- Diff: core/src/main/java/org/apache/carbondata/core/readcommitter/LatestFilesReadCommittedScope.java ---
    @@ -40,9 +40,26 @@
     public class LatestFilesReadCommittedScope implements ReadCommittedScope {
     
       private String carbonFilePath;
    +  private String segmentId;
       private ReadCommittedIndexFileSnapShot readCommittedIndexFileSnapShot;
       private LoadMetadataDetails[] loadMetadataDetails;
     
    +  /**
    +   * a new constructor of this class, which supports obtain lucene index in search mode
    +   *
    +   * @param path      carbon file path
    +   * @param segmentId segment id
    +   */
    +  public LatestFilesReadCommittedScope(String path, String segmentId) {
    --- End diff --
    
    I mean that these code can be reduced to
    ```
    this(path);
    this.segmentId=segmentId;
    ```
    and you should move the next constructor(Line65) above this constructor.


---

[GitHub] carbondata issue #2290: [WIP][CARBONDATA-2389] Search mode support lucene da...

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

    https://github.com/apache/carbondata/pull/2290
  
    Hi, @xubo245 Will you continue working on make search mode using other types of datamap?


---

[GitHub] carbondata pull request #2290: [CARBONDATA-2389] Search mode support lucene ...

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

    https://github.com/apache/carbondata/pull/2290#discussion_r191298660
  
    --- Diff: store/search/src/main/java/org/apache/carbondata/store/worker/SearchRequestHandler.java ---
    @@ -158,22 +163,36 @@ private QueryModel prune(int queryId, CarbonTable table, QueryModel queryModel,
           CarbonMultiBlockSplit mbSplit, DataMapExprWrapper datamap) throws IOException {
         Objects.requireNonNull(datamap);
         List<Segment> segments = new LinkedList<>();
    +    HashMap<String, Integer> uniqueSegments = new HashMap<>();
         for (CarbonInputSplit split : mbSplit.getAllSplits()) {
    -      segments.add(
    -          Segment.toSegment(split.getSegmentId(),
    -              new LatestFilesReadCommittedScope(table.getTablePath())));
    +      String segmentId = split.getSegmentId();
    +      if (uniqueSegments.get(segmentId) == null) {
    +        segments.add(Segment.toSegment(
    +                segmentId,
    +                new LatestFilesReadCommittedScope(table.getTablePath(), segmentId)));
    +        uniqueSegments.put(segmentId, 1);
    +      } else {
    +        uniqueSegments.put(segmentId, uniqueSegments.get(segmentId) + 1);
    +      }
    +    }
    +
    +    List<DataMapDistributableWrapper> distributables = datamap.toDistributable(segments);
    +    List<ExtendedBlocklet> prunnedBlocklets = new LinkedList<ExtendedBlocklet>();
    +    for (int i = 0; i < distributables.size(); i++) {
    +      DataMapDistributable dataMapDistributable = distributables.get(i).getDistributable();
    +      prunnedBlocklets.addAll(datamap.prune(dataMapDistributable, null));
         }
    -    List<ExtendedBlocklet> prunnedBlocklets = datamap.prune(segments, null);
     
    -    List<String> pathToRead = new LinkedList<>();
    -    for (ExtendedBlocklet prunnedBlocklet : prunnedBlocklets) {
    -      pathToRead.add(prunnedBlocklet.getPath());
    +    HashMap<String, ExtendedBlocklet> pathToRead = new HashMap<>();
    +    for (ExtendedBlocklet prunedBlocklet : prunnedBlocklets) {
    +      pathToRead.put(prunedBlocklet.getFilePath(), prunedBlocklet);
         }
     
         List<TableBlockInfo> blocks = queryModel.getTableBlockInfos();
         List<TableBlockInfo> blockToRead = new LinkedList<>();
         for (TableBlockInfo block : blocks) {
    -      if (pathToRead.contains(block.getFilePath())) {
    +      if (pathToRead.keySet().contains(block.getFilePath())) {
    +        block.setDataMapWriterPath(pathToRead.get(block.getFilePath()).getDataMapWriterPath());
    --- End diff --
    
    Why need to set this?


---

[GitHub] carbondata issue #2290: [CARBONDATA-2389] Search mode support lucene datamap

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

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



---

[GitHub] carbondata issue #2290: [CARBONDATA-2389] Search mode support lucene datamap

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

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


---

[GitHub] carbondata pull request #2290: [CARBONDATA-2389] Search mode support lucene ...

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

    https://github.com/apache/carbondata/pull/2290#discussion_r191442174
  
    --- Diff: core/src/main/java/org/apache/carbondata/core/scan/scanner/impl/BlockletFilterScanner.java ---
    @@ -169,6 +169,10 @@ private BlockletScannedResult executeFilter(RawBlockletColumnChunks rawBlockletC
         // apply filter on actual data, for each page
         BitSetGroup bitSetGroup = this.filterExecuter.applyFilter(rawBlockletColumnChunks,
             useBitSetPipeLine);
    +    // if bitSetGroup is nul, then new BitSetGroup object, which can avoid NPE
    --- End diff --
    
    before PR2242, it will be null if the bitSetGroup of rawBlockletColumnChunks is null, so we can remove this now


---

[GitHub] carbondata issue #2290: [WIP][CARBONDATA-2389] Search mode support lucene da...

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

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



---

[GitHub] carbondata issue #2290: [CARBONDATA-2389] Search mode support lucene datamap

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

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



---

[GitHub] carbondata issue #2290: [CARBONDATA-2389] Search mode support lucene datamap

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

    https://github.com/apache/carbondata/pull/2290
  
    @jackylk @QiangCai Please review it.


---

[GitHub] carbondata pull request #2290: [CARBONDATA-2389] Search mode support lucene ...

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

    https://github.com/apache/carbondata/pull/2290#discussion_r188127829
  
    --- Diff: core/src/main/java/org/apache/carbondata/core/datamap/DataMapStoreManager.java ---
    @@ -90,13 +90,18 @@ private DataMapStoreManager() {
           String dbName = carbonTable.getDatabaseName();
           String tableName = carbonTable.getTableName();
           String dmName = dataMap.getDataMapSchema().getDataMapName();
    -      boolean isDmVisible = sessionInfo.getSessionParams().getProperty(
    -          String.format("%s%s.%s.%s", CarbonCommonConstants.CARBON_DATAMAP_VISIBLE,
    -              dbName, tableName, dmName), "true").trim().equalsIgnoreCase("true");
    -      if (!isDmVisible) {
    -        LOGGER.warn(String.format("Ignore invisible datamap %s on table %s.%s",
    -            dmName, dbName, tableName));
    -        dataMapIterator.remove();
    +      if (sessionInfo != null) {
    --- End diff --
    
    So the parameters we set in the env will not be used? Such as query on specified segments, invisible datamaps and so on?


---

[GitHub] carbondata pull request #2290: [CARBONDATA-2389] Search mode support lucene ...

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

    https://github.com/apache/carbondata/pull/2290#discussion_r189598051
  
    --- Diff: core/src/main/java/org/apache/carbondata/core/datamap/dev/expr/AndDataMapExprWrapper.java ---
    @@ -59,6 +60,21 @@ public AndDataMapExprWrapper(DataMapExprWrapper left, DataMapExprWrapper right,
         return andBlocklets;
       }
     
    +  @Override
    +  public List<ExtendedBlocklet> prune(DataMapDistributable distributable,
    +                                      List<PartitionSpec> partitionsToPrune)
    +          throws IOException {
    +    List<ExtendedBlocklet> leftPrune = left.prune(distributable, partitionsToPrune);
    +    List<ExtendedBlocklet> rightPrune = right.prune(distributable, partitionsToPrune);
    +    List<ExtendedBlocklet> andBlocklets = new ArrayList<>();
    +    for (ExtendedBlocklet blocklet : leftPrune) {
    +      if (rightPrune.contains(blocklet)) {
    --- End diff --
    
    When will you use org.apache.carbondata.core.datamap.dev.expr.AndDataMapExprWrapper#prune(java.util.List<org.apache.carbondata.core.datamap.Segment>, java.util.List<org.apache.carbondata.core.indexstore.PartitionSpec>) or org.apache.carbondata.core.datamap.dev.expr.AndDataMapExprWrapper#prune(org.apache.carbondata.core.datamap.DataMapDistributable, java.util.List<org.apache.carbondata.core.indexstore.PartitionSpec>)?


---

[GitHub] carbondata issue #2290: [CARBONDATA-2389] Search mode support lucene datamap

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

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



---

[GitHub] carbondata issue #2290: [CARBONDATA-2389] Search mode support lucene datamap

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

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



---

[GitHub] carbondata pull request #2290: [CARBONDATA-2389] Search mode support lucene ...

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

    https://github.com/apache/carbondata/pull/2290#discussion_r191448581
  
    --- Diff: core/src/main/java/org/apache/carbondata/core/datamap/DataMapStoreManager.java ---
    @@ -96,13 +96,19 @@ private DataMapStoreManager() {
           String dbName = carbonTable.getDatabaseName();
           String tableName = carbonTable.getTableName();
           String dmName = dataMap.getDataMapSchema().getDataMapName();
    -      boolean isDmVisible = sessionInfo.getSessionParams().getProperty(
    -          String.format("%s%s.%s.%s", CarbonCommonConstants.CARBON_DATAMAP_VISIBLE,
    -              dbName, tableName, dmName), "true").trim().equalsIgnoreCase("true");
    -      if (!isDmVisible) {
    -        LOGGER.warn(String.format("Ignore invisible datamap %s on table %s.%s",
    -            dmName, dbName, tableName));
    -        dataMapIterator.remove();
    +      if (sessionInfo != null) {
    --- End diff --
    
    ok,done


---

[GitHub] carbondata issue #2290: [CARBONDATA-2389] Search mode support lucene datamap

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

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



---

[GitHub] carbondata issue #2290: [WIP][CARBONDATA-2389] Search mode support lucene da...

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

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



---

[GitHub] carbondata issue #2290: [CARBONDATA-2389] Search mode support lucene datamap

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

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



---

[GitHub] carbondata pull request #2290: [CARBONDATA-2389] Search mode support lucene ...

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

    https://github.com/apache/carbondata/pull/2290#discussion_r189522994
  
    --- Diff: core/src/main/java/org/apache/carbondata/core/datamap/DataMapStoreManager.java ---
    @@ -90,13 +90,18 @@ private DataMapStoreManager() {
           String dbName = carbonTable.getDatabaseName();
           String tableName = carbonTable.getTableName();
           String dmName = dataMap.getDataMapSchema().getDataMapName();
    -      boolean isDmVisible = sessionInfo.getSessionParams().getProperty(
    -          String.format("%s%s.%s.%s", CarbonCommonConstants.CARBON_DATAMAP_VISIBLE,
    -              dbName, tableName, dmName), "true").trim().equalsIgnoreCase("true");
    -      if (!isDmVisible) {
    -        LOGGER.warn(String.format("Ignore invisible datamap %s on table %s.%s",
    -            dmName, dbName, tableName));
    -        dataMapIterator.remove();
    +      if (sessionInfo != null) {
    --- End diff --
    
    Yes, there are not carbon session info in search mode. Can we add datamap visible status  into CabonProperties?


---

[GitHub] carbondata pull request #2290: [WIP][CARBONDATA-2389] Search mode support lu...

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

    https://github.com/apache/carbondata/pull/2290#discussion_r187878321
  
    --- Diff: store/search/src/main/scala/org/apache/spark/rpc/Master.scala ---
    @@ -234,7 +232,7 @@ class Master(sparkConf: SparkConf) {
           // if we have enough data already, we do not need to collect more result
           if (rowCount < globalLimit) {
             // wait for worker for 10s
    --- End diff --
    
    ok, thanks.


---

[GitHub] carbondata issue #2290: [WIP][CARBONDATA-2389] Search mode support lucene da...

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

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



---

[GitHub] carbondata issue #2290: [WIP][CARBONDATA-2389] Search mode support lucene da...

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

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



---

[GitHub] carbondata issue #2290: [WIP][CARBONDATA-2389] Search mode support lucene da...

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

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



---

[GitHub] carbondata issue #2290: [CARBONDATA-2389] Search mode support lucene datamap

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

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



---

[GitHub] carbondata issue #2290: [WIP][CARBONDATA-2389] Search mode support lucene da...

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

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



---

[GitHub] carbondata pull request #2290: [CARBONDATA-2389] Search mode support lucene ...

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

    https://github.com/apache/carbondata/pull/2290#discussion_r191447282
  
    --- Diff: hadoop/src/main/java/org/apache/carbondata/hadoop/CarbonRecordReader.java ---
    @@ -77,8 +77,10 @@ public void initialize(InputSplit inputSplit, TaskAttemptContext context)
         } else {
           throw new RuntimeException("unsupported input split type: " + inputSplit);
         }
    -    List<TableBlockInfo> tableBlockInfoList = CarbonInputSplit.createBlocks(splitList);
    -    queryModel.setTableBlockInfos(tableBlockInfoList);
    +    if (queryModel.getTableBlockInfos().isEmpty()) {
    --- End diff --
    
    ok, done


---

[GitHub] carbondata issue #2290: [WIP][CARBONDATA-2389] Search mode support lucene da...

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

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



---

[GitHub] carbondata pull request #2290: [CARBONDATA-2389] Search mode support lucene ...

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

    https://github.com/apache/carbondata/pull/2290#discussion_r191300521
  
    --- Diff: core/src/main/java/org/apache/carbondata/core/readcommitter/LatestFilesReadCommittedScope.java ---
    @@ -42,10 +39,27 @@
     public class LatestFilesReadCommittedScope implements ReadCommittedScope {
     
       private String carbonFilePath;
    +  private String segmentId;
       private ReadCommittedIndexFileSnapShot readCommittedIndexFileSnapShot;
       private LoadMetadataDetails[] loadMetadataDetails;
     
    -  public LatestFilesReadCommittedScope(String path)  {
    +  /**
    +   * a new constructor of this class, which supports obtain lucene index in search mode
    --- End diff --
    
    Better not to mention search mode. Describe what this constructor will do. 


---

[GitHub] carbondata pull request #2290: [CARBONDATA-2389] Search mode support lucene ...

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

    https://github.com/apache/carbondata/pull/2290#discussion_r191300474
  
    --- Diff: core/src/main/java/org/apache/carbondata/core/readcommitter/LatestFilesReadCommittedScope.java ---
    @@ -42,10 +39,27 @@
     public class LatestFilesReadCommittedScope implements ReadCommittedScope {
     
       private String carbonFilePath;
    +  private String segmentId;
       private ReadCommittedIndexFileSnapShot readCommittedIndexFileSnapShot;
       private LoadMetadataDetails[] loadMetadataDetails;
     
    -  public LatestFilesReadCommittedScope(String path)  {
    +  /**
    +   * a new constructor of this class, which supports obtain lucene index in search mode
    +   *
    +   * @param path      carbon file path
    +   * @param segmentId segment id
    +   */
    +  public LatestFilesReadCommittedScope(String path, String segmentId) {
    +    this.carbonFilePath = path;
    +    this.segmentId = segmentId;
    +    try {
    +      takeCarbonIndexFileSnapShot();
    +    } catch (IOException ex) {
    +      throw new RuntimeException("Error while taking index snapshot", ex);
    +    }
    +  }
    +
    +  public LatestFilesReadCommittedScope(String path) {
    --- End diff --
    
    This can call above method


---

[GitHub] carbondata pull request #2290: [CARBONDATA-2389] Search mode support lucene ...

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

    https://github.com/apache/carbondata/pull/2290#discussion_r191450160
  
    --- Diff: core/src/main/java/org/apache/carbondata/core/readcommitter/LatestFilesReadCommittedScope.java ---
    @@ -42,10 +39,27 @@
     public class LatestFilesReadCommittedScope implements ReadCommittedScope {
     
       private String carbonFilePath;
    +  private String segmentId;
       private ReadCommittedIndexFileSnapShot readCommittedIndexFileSnapShot;
       private LoadMetadataDetails[] loadMetadataDetails;
     
    -  public LatestFilesReadCommittedScope(String path)  {
    +  /**
    +   * a new constructor of this class, which supports obtain lucene index in search mode
    +   *
    +   * @param path      carbon file path
    +   * @param segmentId segment id
    +   */
    +  public LatestFilesReadCommittedScope(String path, String segmentId) {
    +    this.carbonFilePath = path;
    +    this.segmentId = segmentId;
    +    try {
    +      takeCarbonIndexFileSnapShot();
    +    } catch (IOException ex) {
    +      throw new RuntimeException("Error while taking index snapshot", ex);
    +    }
    +  }
    +
    +  public LatestFilesReadCommittedScope(String path) {
    --- End diff --
    
    ok, done


---

[GitHub] carbondata issue #2290: [CARBONDATA-2389] Search mode support lucene datamap

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

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



---

[GitHub] carbondata issue #2290: [CARBONDATA-2389] Search mode support lucene datamap

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

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



---

[GitHub] carbondata issue #2290: [CARBONDATA-2389] Search mode support lucene datamap

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

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



---

[GitHub] carbondata issue #2290: [WIP][CARBONDATA-2389] Search mode support lucene da...

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

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



---

[GitHub] carbondata issue #2290: [CARBONDATA-2389] Search mode support lucene datamap

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

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



---

[GitHub] carbondata issue #2290: [WIP][CARBONDATA-2389] Search mode support lucene da...

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

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



---

[GitHub] carbondata issue #2290: [CARBONDATA-2389] Search mode support lucene datamap

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

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



---

[GitHub] carbondata pull request #2290: [CARBONDATA-2389] Search mode support lucene ...

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

    https://github.com/apache/carbondata/pull/2290#discussion_r191450259
  
    --- Diff: core/src/main/java/org/apache/carbondata/core/readcommitter/LatestFilesReadCommittedScope.java ---
    @@ -42,10 +39,27 @@
     public class LatestFilesReadCommittedScope implements ReadCommittedScope {
     
       private String carbonFilePath;
    +  private String segmentId;
       private ReadCommittedIndexFileSnapShot readCommittedIndexFileSnapShot;
       private LoadMetadataDetails[] loadMetadataDetails;
     
    -  public LatestFilesReadCommittedScope(String path)  {
    +  /**
    +   * a new constructor of this class, which supports obtain lucene index in search mode
    --- End diff --
    
    ok, remove it


---

[GitHub] carbondata issue #2290: [CARBONDATA-2389] Search mode support lucene datamap

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

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



---

[GitHub] carbondata pull request #2290: [WIP][CARBONDATA-2389] Search mode support lu...

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

    https://github.com/apache/carbondata/pull/2290#discussion_r187875618
  
    --- Diff: core/src/main/java/org/apache/carbondata/core/readcommitter/LatestFilesReadCommittedScope.java ---
    @@ -40,9 +40,26 @@
     public class LatestFilesReadCommittedScope implements ReadCommittedScope {
     
       private String carbonFilePath;
    +  private String segmentId;
       private ReadCommittedIndexFileSnapShot readCommittedIndexFileSnapShot;
       private LoadMetadataDetails[] loadMetadataDetails;
     
    +  /**
    +   * a new constructor of this class, which supports obtain lucene index in search mode
    +   *
    +   * @param path      carbon file path
    +   * @param segmentId segment id
    +   */
    +  public LatestFilesReadCommittedScope(String path, String segmentId) {
    --- End diff --
    
    I don't understand you means.
    This method adds segmentId for building indexPath.


---

[GitHub] carbondata issue #2290: [CARBONDATA-2389] Search mode support lucene datamap

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

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



---

[GitHub] carbondata pull request #2290: [WIP][CARBONDATA-2389] Search mode support lu...

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

    https://github.com/apache/carbondata/pull/2290#discussion_r187768530
  
    --- Diff: core/src/main/java/org/apache/carbondata/core/datamap/dev/expr/AndDataMapExprWrapper.java ---
    @@ -59,6 +60,21 @@ public AndDataMapExprWrapper(DataMapExprWrapper left, DataMapExprWrapper right,
         return andBlocklets;
       }
     
    +  @Override
    +  public List<ExtendedBlocklet> prune(DataMapDistributable distributable,
    +                                      List<PartitionSpec> partitionsToPrune)
    +          throws IOException {
    +    List<ExtendedBlocklet> leftPrune = left.prune(distributable, partitionsToPrune);
    +    List<ExtendedBlocklet> rightPrune = right.prune(distributable, partitionsToPrune);
    +    List<ExtendedBlocklet> andBlocklets = new ArrayList<>();
    +    for (ExtendedBlocklet blocklet : leftPrune) {
    +      if (rightPrune.contains(blocklet)) {
    --- End diff --
    
    Have you ever validate the correctness of this? The `equals` and `hashCode` method in `ExtendedBlocklet` only make use of its member `segmentId`. I'm not sure whether it is correct.


---

[GitHub] carbondata pull request #2290: [CARBONDATA-2389] Search mode support lucene ...

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

    https://github.com/apache/carbondata/pull/2290#discussion_r191362538
  
    --- Diff: core/src/main/java/org/apache/carbondata/core/readcommitter/LatestFilesReadCommittedScope.java ---
    @@ -40,9 +40,26 @@
     public class LatestFilesReadCommittedScope implements ReadCommittedScope {
     
       private String carbonFilePath;
    +  private String segmentId;
       private ReadCommittedIndexFileSnapShot readCommittedIndexFileSnapShot;
       private LoadMetadataDetails[] loadMetadataDetails;
     
    +  /**
    +   * a new constructor of this class, which supports obtain lucene index in search mode
    +   *
    +   * @param path      carbon file path
    +   * @param segmentId segment id
    +   */
    +  public LatestFilesReadCommittedScope(String path, String segmentId) {
    --- End diff --
    
    I change the constructor to   
    public LatestFilesReadCommittedScope(String path) {
        this(path, null);
      }


---

[GitHub] carbondata issue #2290: [CARBONDATA-2389] Search mode support lucene datamap

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

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



---

[GitHub] carbondata issue #2290: [CARBONDATA-2389] Search mode support lucene datamap

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

    https://github.com/apache/carbondata/pull/2290
  
    @jackylk  CI pass


---

[GitHub] carbondata pull request #2290: [CARBONDATA-2389] Search mode support lucene ...

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

    https://github.com/apache/carbondata/pull/2290#discussion_r191300111
  
    --- Diff: core/src/main/java/org/apache/carbondata/core/datamap/DataMapStoreManager.java ---
    @@ -96,13 +96,19 @@ private DataMapStoreManager() {
           String dbName = carbonTable.getDatabaseName();
           String tableName = carbonTable.getTableName();
           String dmName = dataMap.getDataMapSchema().getDataMapName();
    -      boolean isDmVisible = sessionInfo.getSessionParams().getProperty(
    -          String.format("%s%s.%s.%s", CarbonCommonConstants.CARBON_DATAMAP_VISIBLE,
    -              dbName, tableName, dmName), "true").trim().equalsIgnoreCase("true");
    -      if (!isDmVisible) {
    -        LOGGER.warn(String.format("Ignore invisible datamap %s on table %s.%s",
    -            dmName, dbName, tableName));
    -        dataMapIterator.remove();
    +      if (sessionInfo != null) {
    --- End diff --
    
    please add comment to describe why this check is needed


---

[GitHub] carbondata pull request #2290: [WIP][CARBONDATA-2389] Search mode support lu...

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

    https://github.com/apache/carbondata/pull/2290#discussion_r187768741
  
    --- Diff: store/search/src/main/scala/org/apache/spark/rpc/Master.scala ---
    @@ -234,7 +232,7 @@ class Master(sparkConf: SparkConf) {
           // if we have enough data already, we do not need to collect more result
           if (rowCount < globalLimit) {
             // wait for worker for 10s
    --- End diff --
    
    please modify the comments as well


---

[GitHub] carbondata pull request #2290: [WIP][CARBONDATA-2389] Search mode support lu...

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

    https://github.com/apache/carbondata/pull/2290#discussion_r187768650
  
    --- Diff: core/src/main/java/org/apache/carbondata/core/scan/scanner/impl/BlockletFilterScanner.java ---
    @@ -169,6 +169,10 @@ private BlockletScannedResult executeFilter(RawBlockletColumnChunks rawBlockletC
         // apply filter on actual data, for each page
         BitSetGroup bitSetGroup = this.filterExecuter.applyFilter(rawBlockletColumnChunks,
             useBitSetPipeLine);
    +    // if bitSetGroup is nul, then new BitSetGroup object, which can avoid NPE
    --- End diff --
    
    Is `Lucene` will introduce this problem as you described or is it a bug caused by other scenario?


---

[GitHub] carbondata issue #2290: [CARBONDATA-2389] Search mode support lucene datamap

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

    https://github.com/apache/carbondata/pull/2290
  
    Please help to review it. @jackylk @ravipesala @QiangCai


---

[GitHub] carbondata issue #2290: [CARBONDATA-2389] Search mode support lucene datamap

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

    https://github.com/apache/carbondata/pull/2290
  
    @jackylk Ok, removed.


---

[GitHub] carbondata pull request #2290: [CARBONDATA-2389] Search mode support lucene ...

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

    https://github.com/apache/carbondata/pull/2290#discussion_r191298867
  
    --- Diff: store/search/src/main/scala/org/apache/spark/rpc/Master.scala ---
    @@ -142,7 +142,7 @@ class Master(sparkConf: SparkConf) {
           (address, schedulable.ref.ask[ShutdownResponse](ShutdownRequest("user")))
         }
         futures.foreach { case (address, future) =>
    -      ThreadUtils.awaitResult(future, Duration.apply("10s"))
    +      ThreadUtils.awaitResult(future, Duration.apply("100s"))
    --- End diff --
    
    Please add a CarbonProperty for search mode, and make the default as 10


---

[GitHub] carbondata pull request #2290: [WIP][CARBONDATA-2389] Search mode support lu...

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

    https://github.com/apache/carbondata/pull/2290#discussion_r187877018
  
    --- Diff: core/src/main/java/org/apache/carbondata/core/scan/scanner/impl/BlockletFilterScanner.java ---
    @@ -169,6 +169,10 @@ private BlockletScannedResult executeFilter(RawBlockletColumnChunks rawBlockletC
         // apply filter on actual data, for each page
         BitSetGroup bitSetGroup = this.filterExecuter.applyFilter(rawBlockletColumnChunks,
             useBitSetPipeLine);
    +    // if bitSetGroup is nul, then new BitSetGroup object, which can avoid NPE
    --- End diff --
    
    It will throw exception when use lucent with search mode. I didn't check other scenario.
    



---

[GitHub] carbondata pull request #2290: [WIP][CARBONDATA-2389] Search mode support lu...

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

    https://github.com/apache/carbondata/pull/2290#discussion_r187768562
  
    --- Diff: core/src/main/java/org/apache/carbondata/core/datamap/dev/expr/DataMapExprWrapper.java ---
    @@ -40,6 +41,18 @@
       List<ExtendedBlocklet> prune(List<Segment> segments, List<PartitionSpec> partitionsToPrune)
           throws IOException;
     
    +  /**
    +   * prune blocklet according distributable
    +   *
    +   * @param distributable     distributable
    +   * @param partitionsToPrune partitions to prune
    +   * @return the pruned ExtendedBlocklet list
    +   * @throws IOException
    +   */
    +  List<ExtendedBlocklet> prune(DataMapDistributable distributable,
    --- End diff --
    
    there are other indent problems like this in this PR, better to optimize them all


---

[GitHub] carbondata issue #2290: [CARBONDATA-2389] Search mode support lucene datamap

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

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



---

[GitHub] carbondata issue #2290: [CARBONDATA-2389] Search mode support lucene datamap

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

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


---

[GitHub] carbondata issue #2290: [WIP][CARBONDATA-2389] Search mode support lucene da...

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

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



---

[GitHub] carbondata pull request #2290: [CARBONDATA-2389] Search mode support lucene ...

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

    https://github.com/apache/carbondata/pull/2290#discussion_r191447617
  
    --- Diff: core/src/main/java/org/apache/carbondata/core/datamap/DataMapStoreManager.java ---
    @@ -96,13 +96,19 @@ private DataMapStoreManager() {
           String dbName = carbonTable.getDatabaseName();
           String tableName = carbonTable.getTableName();
           String dmName = dataMap.getDataMapSchema().getDataMapName();
    -      boolean isDmVisible = sessionInfo.getSessionParams().getProperty(
    -          String.format("%s%s.%s.%s", CarbonCommonConstants.CARBON_DATAMAP_VISIBLE,
    -              dbName, tableName, dmName), "true").trim().equalsIgnoreCase("true");
    -      if (!isDmVisible) {
    -        LOGGER.warn(String.format("Ignore invisible datamap %s on table %s.%s",
    -            dmName, dbName, tableName));
    -        dataMapIterator.remove();
    +      if (sessionInfo != null) {
    +        boolean isDmVisible = sessionInfo.getSessionParams().getProperty(
    +                String.format("%s%s.%s.%s", CarbonCommonConstants.CARBON_DATAMAP_VISIBLE,
    +                        dbName, tableName, dmName), "true").trim().equalsIgnoreCase("true");
    +        if (!isDmVisible) {
    +          LOGGER.warn(String.format("Ignore invisible datamap %s on table %s.%s",
    +                  dmName, dbName, tableName));
    +          dataMapIterator.remove();
    +        }
    +      } else {
    +        // TODO: need support get the visible status of datamp in the future
    +        String message = "Carbon session info is null";
    +        LOGGER.audit(message);
    --- End diff --
    
    ok,done


---

[GitHub] carbondata pull request #2290: [CARBONDATA-2389] Search mode support lucene ...

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

    https://github.com/apache/carbondata/pull/2290#discussion_r191432727
  
    --- Diff: store/search/src/main/java/org/apache/carbondata/store/worker/SearchRequestHandler.java ---
    @@ -158,22 +163,36 @@ private QueryModel prune(int queryId, CarbonTable table, QueryModel queryModel,
           CarbonMultiBlockSplit mbSplit, DataMapExprWrapper datamap) throws IOException {
         Objects.requireNonNull(datamap);
         List<Segment> segments = new LinkedList<>();
    +    HashMap<String, Integer> uniqueSegments = new HashMap<>();
         for (CarbonInputSplit split : mbSplit.getAllSplits()) {
    -      segments.add(
    -          Segment.toSegment(split.getSegmentId(),
    -              new LatestFilesReadCommittedScope(table.getTablePath())));
    +      String segmentId = split.getSegmentId();
    +      if (uniqueSegments.get(segmentId) == null) {
    +        segments.add(Segment.toSegment(
    +                segmentId,
    +                new LatestFilesReadCommittedScope(table.getTablePath(), segmentId)));
    +        uniqueSegments.put(segmentId, 1);
    +      } else {
    +        uniqueSegments.put(segmentId, uniqueSegments.get(segmentId) + 1);
    +      }
    +    }
    +
    +    List<DataMapDistributableWrapper> distributables = datamap.toDistributable(segments);
    +    List<ExtendedBlocklet> prunnedBlocklets = new LinkedList<ExtendedBlocklet>();
    +    for (int i = 0; i < distributables.size(); i++) {
    +      DataMapDistributable dataMapDistributable = distributables.get(i).getDistributable();
    +      prunnedBlocklets.addAll(datamap.prune(dataMapDistributable, null));
         }
    -    List<ExtendedBlocklet> prunnedBlocklets = datamap.prune(segments, null);
     
    -    List<String> pathToRead = new LinkedList<>();
    -    for (ExtendedBlocklet prunnedBlocklet : prunnedBlocklets) {
    -      pathToRead.add(prunnedBlocklet.getPath());
    +    HashMap<String, ExtendedBlocklet> pathToRead = new HashMap<>();
    +    for (ExtendedBlocklet prunedBlocklet : prunnedBlocklets) {
    +      pathToRead.put(prunedBlocklet.getFilePath(), prunedBlocklet);
         }
     
         List<TableBlockInfo> blocks = queryModel.getTableBlockInfos();
         List<TableBlockInfo> blockToRead = new LinkedList<>();
         for (TableBlockInfo block : blocks) {
    -      if (pathToRead.contains(block.getFilePath())) {
    +      if (pathToRead.keySet().contains(block.getFilePath())) {
    +        block.setDataMapWriterPath(pathToRead.get(block.getFilePath()).getDataMapWriterPath());
    --- End diff --
    
    if not set this, the bitSetGroup will be null (reason refer org.apache.carbondata.core.indexstore.blockletindex.BlockletDataRefNode#getIndexedData).
    
    And  it will throw exception in org.apache.carbondata.core.scan.filter.executer.RowLevelFilterExecuterImpl#applyFilter(org.apache.carbondata.core.scan.processor.RawBlockletColumnChunks, boolean)
    



---

[GitHub] carbondata pull request #2290: [CARBONDATA-2389] Search mode support lucene ...

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

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


---

[GitHub] carbondata pull request #2290: [CARBONDATA-2389] Search mode support lucene ...

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

    https://github.com/apache/carbondata/pull/2290#discussion_r191300560
  
    --- Diff: core/src/main/java/org/apache/carbondata/core/readcommitter/LatestFilesReadCommittedScope.java ---
    @@ -42,10 +39,27 @@
     public class LatestFilesReadCommittedScope implements ReadCommittedScope {
     
       private String carbonFilePath;
    +  private String segmentId;
       private ReadCommittedIndexFileSnapShot readCommittedIndexFileSnapShot;
       private LoadMetadataDetails[] loadMetadataDetails;
     
    -  public LatestFilesReadCommittedScope(String path)  {
    +  /**
    +   * a new constructor of this class, which supports obtain lucene index in search mode
    +   *
    +   * @param path      carbon file path
    +   * @param segmentId segment id
    +   */
    +  public LatestFilesReadCommittedScope(String path, String segmentId) {
    +    this.carbonFilePath = path;
    --- End diff --
    
    Add `Objects.requireNotNull` for path
    and segmentId can be null?


---

[GitHub] carbondata issue #2290: [WIP][CARBONDATA-2389] Search mode support lucene da...

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

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



---

[GitHub] carbondata pull request #2290: [WIP][CARBONDATA-2389] Search mode support lu...

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

    https://github.com/apache/carbondata/pull/2290#discussion_r187871019
  
    --- Diff: core/src/main/java/org/apache/carbondata/core/datamap/DataMapStoreManager.java ---
    @@ -90,13 +90,18 @@ private DataMapStoreManager() {
           String dbName = carbonTable.getDatabaseName();
           String tableName = carbonTable.getTableName();
           String dmName = dataMap.getDataMapSchema().getDataMapName();
    -      boolean isDmVisible = sessionInfo.getSessionParams().getProperty(
    -          String.format("%s%s.%s.%s", CarbonCommonConstants.CARBON_DATAMAP_VISIBLE,
    -              dbName, tableName, dmName), "true").trim().equalsIgnoreCase("true");
    -      if (!isDmVisible) {
    -        LOGGER.warn(String.format("Ignore invisible datamap %s on table %s.%s",
    -            dmName, dbName, tableName));
    -        dataMapIterator.remove();
    +      if (sessionInfo != null) {
    --- End diff --
    
    Session info will be null when use the search mode, Searcher=>handleRequest=>DataMapChooser=>getAllVisibleDataMap


---

[GitHub] carbondata pull request #2290: [WIP][CARBONDATA-2389] Search mode support lu...

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

    https://github.com/apache/carbondata/pull/2290#discussion_r187878047
  
    --- Diff: datamap/lucene/src/main/java/org/apache/carbondata/datamap/lucene/LuceneDataMapFactoryBase.java ---
    @@ -168,7 +169,8 @@ public DataMapBuilder createBuilder(Segment segment, String shardName) {
             getAllIndexDirs(tableIdentifier.getTablePath(), segment.getSegmentNo());
         for (CarbonFile indexDir : indexDirs) {
           // Filter out the tasks which are filtered through CG datamap.
    -      if (!segment.getFilteredIndexShardNames().contains(indexDir.getName())) {
    +      if (getDataMapLevel() != DataMapLevel.FG &&
    --- End diff --
    
    The design of continue is for CG. FG can't skip it by the continue, otherwise it will led to error later.


---

[GitHub] carbondata pull request #2290: [CARBONDATA-2389] Search mode support lucene ...

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

    https://github.com/apache/carbondata/pull/2290#discussion_r191451235
  
    --- Diff: core/src/main/java/org/apache/carbondata/core/readcommitter/LatestFilesReadCommittedScope.java ---
    @@ -42,10 +39,27 @@
     public class LatestFilesReadCommittedScope implements ReadCommittedScope {
     
       private String carbonFilePath;
    +  private String segmentId;
       private ReadCommittedIndexFileSnapShot readCommittedIndexFileSnapShot;
       private LoadMetadataDetails[] loadMetadataDetails;
     
    -  public LatestFilesReadCommittedScope(String path)  {
    +  /**
    +   * a new constructor of this class, which supports obtain lucene index in search mode
    +   *
    +   * @param path      carbon file path
    +   * @param segmentId segment id
    +   */
    +  public LatestFilesReadCommittedScope(String path, String segmentId) {
    +    this.carbonFilePath = path;
    --- End diff --
    
    ok, done.
    segmentId can be null.


---

[GitHub] carbondata pull request #2290: [CARBONDATA-2389] Search mode support lucene ...

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

    https://github.com/apache/carbondata/pull/2290#discussion_r191299753
  
    --- Diff: core/src/main/java/org/apache/carbondata/core/datamap/DataMapStoreManager.java ---
    @@ -96,13 +96,19 @@ private DataMapStoreManager() {
           String dbName = carbonTable.getDatabaseName();
           String tableName = carbonTable.getTableName();
           String dmName = dataMap.getDataMapSchema().getDataMapName();
    -      boolean isDmVisible = sessionInfo.getSessionParams().getProperty(
    -          String.format("%s%s.%s.%s", CarbonCommonConstants.CARBON_DATAMAP_VISIBLE,
    -              dbName, tableName, dmName), "true").trim().equalsIgnoreCase("true");
    -      if (!isDmVisible) {
    -        LOGGER.warn(String.format("Ignore invisible datamap %s on table %s.%s",
    -            dmName, dbName, tableName));
    -        dataMapIterator.remove();
    +      if (sessionInfo != null) {
    +        boolean isDmVisible = sessionInfo.getSessionParams().getProperty(
    +                String.format("%s%s.%s.%s", CarbonCommonConstants.CARBON_DATAMAP_VISIBLE,
    +                        dbName, tableName, dmName), "true").trim().equalsIgnoreCase("true");
    +        if (!isDmVisible) {
    +          LOGGER.warn(String.format("Ignore invisible datamap %s on table %s.%s",
    +                  dmName, dbName, tableName));
    +          dataMapIterator.remove();
    +        }
    +      } else {
    +        // TODO: need support get the visible status of datamp in the future
    +        String message = "Carbon session info is null";
    +        LOGGER.audit(message);
    --- End diff --
    
    use info instead of audit, audit is intended for table operation


---

[GitHub] carbondata issue #2290: [WIP][CARBONDATA-2389] Search mode support lucene da...

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

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



---

[GitHub] carbondata issue #2290: [WIP][CARBONDATA-2389] Search mode support lucene da...

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

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



---

[GitHub] carbondata issue #2290: [WIP][CARBONDATA-2389] Search mode support lucene da...

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

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



---

[GitHub] carbondata pull request #2290: [WIP][CARBONDATA-2389] Search mode support lu...

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

    https://github.com/apache/carbondata/pull/2290#discussion_r187768680
  
    --- Diff: datamap/lucene/src/main/java/org/apache/carbondata/datamap/lucene/LuceneDataMapFactoryBase.java ---
    @@ -168,7 +169,8 @@ public DataMapBuilder createBuilder(Segment segment, String shardName) {
             getAllIndexDirs(tableIdentifier.getTablePath(), segment.getSegmentNo());
         for (CarbonFile indexDir : indexDirs) {
           // Filter out the tasks which are filtered through CG datamap.
    -      if (!segment.getFilteredIndexShardNames().contains(indexDir.getName())) {
    +      if (getDataMapLevel() != DataMapLevel.FG &&
    --- End diff --
    
    What does this for?
    If it is only for CG datamap, then you can judge outside this loop.


---

[GitHub] carbondata issue #2290: [WIP][CARBONDATA-2389] Search mode support lucene da...

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

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



---

[GitHub] carbondata pull request #2290: [WIP][CARBONDATA-2389] Search mode support lu...

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

    https://github.com/apache/carbondata/pull/2290#discussion_r187768550
  
    --- Diff: core/src/main/java/org/apache/carbondata/core/datamap/dev/expr/DataMapExprWrapper.java ---
    @@ -40,6 +41,18 @@
       List<ExtendedBlocklet> prune(List<Segment> segments, List<PartitionSpec> partitionsToPrune)
           throws IOException;
     
    +  /**
    +   * prune blocklet according distributable
    +   *
    +   * @param distributable     distributable
    +   * @param partitionsToPrune partitions to prune
    +   * @return the pruned ExtendedBlocklet list
    +   * @throws IOException
    +   */
    +  List<ExtendedBlocklet> prune(DataMapDistributable distributable,
    --- End diff --
    
    the indent problem?


---

[GitHub] carbondata issue #2290: [CARBONDATA-2389] Search mode support lucene datamap

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

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



---