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

[GitHub] carbondata pull request #2134: [CARBONDATA-2310] Refactored code to improve ...

GitHub user dhatchayani opened a pull request:

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

    [CARBONDATA-2310] Refactored code to improve Distributable interface

    Refactored code to improve Distributable interface
    
     - [ ] Any interfaces changed?
     
     - [ ] Any backward compatibility impacted?
     
     - [ ] Document update required?
    
     - [x] Testing done
            Manual Testing
           
     - [ ] For large changes, please consider breaking it into sub-tasks under an umbrella JIRA. 
    


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

    $ git pull https://github.com/dhatchayani/carbondata distributable

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

    https://github.com/apache/carbondata/pull/2134.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 #2134
    
----
commit dfad93daa4c9cf549f5a9d0d4362024354964454
Author: dhatchayani <dh...@...>
Date:   2018-04-03T05:49:43Z

    [CARBONDATA-2310] Refactored code to improve Distributable interface

----


---

[GitHub] carbondata issue #2134: [CARBONDATA-2310] Refactored code to improve Distrib...

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

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



---

[GitHub] carbondata issue #2134: [CARBONDATA-2310] Refactored code to improve Distrib...

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

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



---

[GitHub] carbondata pull request #2134: [CARBONDATA-2310] Refactored code to improve ...

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

    https://github.com/apache/carbondata/pull/2134#discussion_r180988586
  
    --- Diff: core/src/main/java/org/apache/carbondata/core/datamap/dev/CacheableDataMap.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.core.datamap.dev;
    +
    +import java.io.IOException;
    +import java.util.List;
    +
    +import org.apache.carbondata.core.datamap.DataMapDistributable;
    +import org.apache.carbondata.core.memory.MemoryException;
    +
    +/**
    + * Interface for data map caching
    + */
    +public interface CacheableDataMap {
    --- End diff --
    
    I think BlockletDataMap is already cached, why this is needed


---

[GitHub] carbondata issue #2134: [CARBONDATA-2310] Refactored code to improve Distrib...

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

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



---

[GitHub] carbondata pull request #2134: [WIP][CARBONDATA-2310] Refactored code to imp...

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

    https://github.com/apache/carbondata/pull/2134#discussion_r178723724
  
    --- Diff: core/src/main/java/org/apache/carbondata/core/indexstore/row/UnsafeDataMapRow.java ---
    @@ -30,7 +30,12 @@
      */
     public class UnsafeDataMapRow extends DataMapRow {
     
    -  private MemoryBlock block;
    +  /**
    +   * default serial version ID.
    +   */
    +  private static final long serialVersionUID = 1L;
    +
    +  private transient MemoryBlock block;
    --- End diff --
    
    Add reason for making it transient
    Reason: As it is an unsafe memory block it is not recommended to serialize


---

[GitHub] carbondata issue #2134: [CARBONDATA-2310] Refactored code to improve Distrib...

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

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


---

[GitHub] carbondata issue #2134: [WIP][CARBONDATA-2310] Refactored code to improve Di...

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

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



---

[GitHub] carbondata issue #2134: [CARBONDATA-2310] Refactored code to improve Distrib...

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

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



---

[GitHub] carbondata issue #2134: [WIP][CARBONDATA-2310] Refactored code to improve Di...

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

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


---

[GitHub] carbondata issue #2134: [WIP][CARBONDATA-2310] Refactored code to improve Di...

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

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



---

[GitHub] carbondata pull request #2134: [WIP][CARBONDATA-2310] Refactored code to imp...

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

    https://github.com/apache/carbondata/pull/2134#discussion_r178720946
  
    --- Diff: core/src/main/java/org/apache/carbondata/core/cache/dictionary/ReverseDictionaryCache.java ---
    @@ -168,6 +168,10 @@ public ReverseDictionaryCache(CarbonLRUCache carbonLRUCache) {
                 CacheType.REVERSE_DICTIONARY));
       }
     
    +  @Override public void put(DictionaryColumnUniqueIdentifier key, Dictionary value) {
    +
    +  }
    +
    --- End diff --
    
    override the method in abstract class and in the implementation throw UnsupportedException for both reverse and forward cache


---

[GitHub] carbondata issue #2134: [WIP][CARBONDATA-2310] Refactored code to improve Di...

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

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



---

[GitHub] carbondata pull request #2134: [WIP][CARBONDATA-2310] Refactored code to imp...

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

    https://github.com/apache/carbondata/pull/2134#discussion_r178721874
  
    --- Diff: core/src/main/java/org/apache/carbondata/core/datastore/BlockIndexStore.java ---
    @@ -229,6 +229,10 @@ private String getLruCacheKey(AbsoluteTableIdentifier absoluteTableIdentifier,
             .remove(getLruCacheKey(tableBlockUniqueIdentifier.getAbsoluteTableIdentifier(), blockInfo));
       }
     
    +  @Override public void put(TableBlockUniqueIdentifier key, AbstractIndex value) {
    +
    +  }
    +
    --- End diff --
    
    throw unsupportedException


---

[GitHub] carbondata pull request #2134: [WIP][CARBONDATA-2310] Refactored code to imp...

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

    https://github.com/apache/carbondata/pull/2134#discussion_r178723434
  
    --- Diff: core/src/main/java/org/apache/carbondata/core/indexstore/blockletindex/BlockletDataMap.java ---
    @@ -77,6 +78,11 @@
      */
     public class BlockletDataMap implements DataMap, Cacheable {
     
    +  /**
    +   * default serial version ID.
    +   */
    +  private static final long serialVersionUID = 1L;
    --- End diff --
    
    generate proper serial version UID


---

[GitHub] carbondata issue #2134: [CARBONDATA-2310] Refactored code to improve Distrib...

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

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



---

[GitHub] carbondata pull request #2134: [WIP][CARBONDATA-2310] Refactored code to imp...

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

    https://github.com/apache/carbondata/pull/2134#discussion_r178724959
  
    --- Diff: core/src/main/java/org/apache/carbondata/core/indexstore/blockletindex/BlockletDataMapFactory.java ---
    @@ -156,15 +167,18 @@ private ExtendedBlocklet getExtendedBlocklet(List<TableBlockIndexUniqueIdentifie
       @Override
       public List<DataMapDistributable> toDistributable(Segment segment) {
         List<DataMapDistributable> distributables = new ArrayList<>();
    +    Map<String, String> indexFiles = null;
         try {
           CarbonFile[] carbonIndexFiles;
           if (segment.getSegmentFileName() == null) {
             carbonIndexFiles = SegmentIndexFileStore.getCarbonIndexFiles(
                 CarbonTablePath.getSegmentPath(identifier.getTablePath(), segment.getSegmentNo()));
           } else {
    +        // TODO: Buggy code, this code will not work as we need to list only the
    +        // physically existing files
    --- End diff --
    
    Remove this TODO and handle partition case


---

[GitHub] carbondata pull request #2134: [WIP][CARBONDATA-2310] Refactored code to imp...

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

    https://github.com/apache/carbondata/pull/2134#discussion_r178724390
  
    --- Diff: core/src/main/java/org/apache/carbondata/core/indexstore/blockletindex/BlockletDataMapDistributable.java ---
    @@ -38,4 +38,5 @@ public BlockletDataMapDistributable(String indexFilePath) {
       public String getFilePath() {
         return filePath;
       }
    +
    --- End diff --
    
    remove empty line


---

[GitHub] carbondata issue #2134: [WIP][CARBONDATA-2310] Refactored code to improve Di...

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

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



---

[GitHub] carbondata issue #2134: [CARBONDATA-2310] Refactored code to improve Distrib...

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

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



---

[GitHub] carbondata pull request #2134: [WIP][CARBONDATA-2310] Refactored code to imp...

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

    https://github.com/apache/carbondata/pull/2134#discussion_r178723320
  
    --- Diff: core/src/main/java/org/apache/carbondata/core/indexstore/UnsafeMemoryDMStore.java ---
    @@ -177,10 +204,45 @@ public UnsafeDataMapRow getUnsafeRow(int index) {
         return new UnsafeDataMapRow(schema, memoryBlock, pointers[index]);
       }
     
    -  public void finishWriting() throws MemoryException {
    +  /**
    +   * Add the index row to dataMapRows, basically to in memory.
    +   *
    +   * @param indexRow
    +   * @return
    +   */
    +  private void addDataMapRow(DataMapRow indexRow) throws MemoryException {
    +    dataMapRows.add(indexRow);
    +  }
    +
    +  /**
    +   * This method will write all the dataMapRows to unsafe
    +   *
    +   * @throws MemoryException
    +   * @throws IOException
    +   */
    +  private void adddataMapRowToUnsafe() throws MemoryException, IOException {
    --- End diff --
    
    Correct the typo in method name and rename to addDataMapRowsToUnsafe


---

[GitHub] carbondata issue #2134: [WIP][CARBONDATA-2310] Refactored code to improve Di...

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

    https://github.com/apache/carbondata/pull/2134
  
    Add few UT's to test this code


---

[GitHub] carbondata issue #2134: [WIP][CARBONDATA-2310] Refactored code to improve Di...

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

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



---

[GitHub] carbondata issue #2134: [WIP][CARBONDATA-2310] Refactored code to improve Di...

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

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



---

[GitHub] carbondata pull request #2134: [WIP][CARBONDATA-2310] Refactored code to imp...

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

    https://github.com/apache/carbondata/pull/2134#discussion_r179341377
  
    --- Diff: core/src/main/java/org/apache/carbondata/core/indexstore/UnsafeMemoryDMStore.java ---
    @@ -52,11 +62,23 @@
     
       private final long taskId = ThreadLocalTaskInfo.getCarbonTaskInfo().getTaskId();
     
    -  public UnsafeMemoryDMStore(CarbonRowSchema[] schema) throws MemoryException {
    +  private boolean addRowToUnsafe = true;
    --- End diff --
    
    Class name itself `UnsafeMemoryDMStore` so it does not make sense to have this variable.
    Please create AbstractClass and give two implementations with safe and Unsafe.


---

[GitHub] carbondata issue #2134: [CARBONDATA-2310] Refactored code to improve Distrib...

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

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



---

[GitHub] carbondata issue #2134: [WIP][CARBONDATA-2310] Refactored code to improve Di...

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

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



---

[GitHub] carbondata issue #2134: [WIP][CARBONDATA-2310] Refactored code to improve Di...

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

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



---

[GitHub] carbondata issue #2134: [WIP][CARBONDATA-2310] Refactored code to improve Di...

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

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



---

[GitHub] carbondata pull request #2134: [CARBONDATA-2310] Refactored code to improve ...

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

    https://github.com/apache/carbondata/pull/2134#discussion_r181629972
  
    --- Diff: core/src/main/java/org/apache/carbondata/core/datamap/dev/CacheableDataMap.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.core.datamap.dev;
    +
    +import java.io.IOException;
    +import java.util.List;
    +
    +import org.apache.carbondata.core.datamap.DataMapDistributable;
    +import org.apache.carbondata.core.memory.MemoryException;
    +
    +/**
    + * Interface for data map caching
    + */
    +public interface CacheableDataMap {
    --- End diff --
    
    BlockletDataMap is already cached. As our cache is of closed structure, this is exposed to put the cacheable entry to cache.


---

[GitHub] carbondata issue #2134: [WIP][CARBONDATA-2310] Refactored code to improve Di...

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

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



---

[GitHub] carbondata issue #2134: [WIP][CARBONDATA-2310] Refactored code to improve Di...

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

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



---

[GitHub] carbondata issue #2134: [WIP][CARBONDATA-2310] Refactored code to improve Di...

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

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



---

[GitHub] carbondata issue #2134: [WIP][CARBONDATA-2310] Refactored code to improve Di...

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

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



---

[GitHub] carbondata issue #2134: [WIP][CARBONDATA-2310] Refactored code to improve Di...

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

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



---

[GitHub] carbondata issue #2134: [WIP][CARBONDATA-2310] Refactored code to improve Di...

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

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



---

[GitHub] carbondata pull request #2134: [CARBONDATA-2310] Refactored code to improve ...

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

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


---

[GitHub] carbondata pull request #2134: [WIP][CARBONDATA-2310] Refactored code to imp...

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

    https://github.com/apache/carbondata/pull/2134#discussion_r179416485
  
    --- Diff: core/src/main/java/org/apache/carbondata/core/indexstore/UnsafeMemoryDMStore.java ---
    @@ -52,11 +62,23 @@
     
       private final long taskId = ThreadLocalTaskInfo.getCarbonTaskInfo().getTaskId();
     
    -  public UnsafeMemoryDMStore(CarbonRowSchema[] schema) throws MemoryException {
    +  private boolean addRowToUnsafe = true;
    --- End diff --
    
    This variable is to decide whether to write row by row to unsafe or to collect all rows and then write all to unsafe.


---

[GitHub] carbondata pull request #2134: [WIP][CARBONDATA-2310] Refactored code to imp...

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

    https://github.com/apache/carbondata/pull/2134#discussion_r178721496
  
    --- Diff: hadoop/src/main/java/org/apache/carbondata/hadoop/api/DataMapJob.java ---
    @@ -19,15 +19,21 @@
     import java.io.Serializable;
     import java.util.List;
     
    +import org.apache.carbondata.core.datamap.dev.DataMap;
     import org.apache.carbondata.core.indexstore.ExtendedBlocklet;
    +import org.apache.carbondata.core.metadata.schema.table.CarbonTable;
     import org.apache.carbondata.core.scan.filter.resolver.FilterResolverIntf;
     
    +import org.apache.hadoop.mapreduce.lib.input.FileInputFormat;
    +
     /**
      * Distributable datamap job to execute the #DistributableDataMapFormat in cluster. it prunes the
      * datamaps distributably and returns the final blocklet list
      */
     public interface DataMapJob extends Serializable {
     
    +  List<DataMap> execute(FileInputFormat<Void, DataMap> dataMapFormat, CarbonTable carbonTable);
    +
    --- End diff --
    
    Change the argument order, first should be carbonTable


---

[GitHub] carbondata pull request #2134: [WIP][CARBONDATA-2310] Refactored code to imp...

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

    https://github.com/apache/carbondata/pull/2134#discussion_r179341666
  
    --- Diff: core/src/main/java/org/apache/carbondata/core/indexstore/blockletindex/BlockletDataMap.java ---
    @@ -186,23 +193,28 @@ public void init(DataMapModel dataMapModel) throws IOException, MemoryException
             }
           }
         }
    -    if (unsafeMemoryDMStore != null) {
    -      unsafeMemoryDMStore.finishWriting();
    -    }
         if (null != unsafeMemorySummaryDMStore) {
           addTaskSummaryRowToUnsafeMemoryStore(
               summaryRow,
               schemaBinary,
               filePath,
               fileName,
               segmentId);
    -      unsafeMemorySummaryDMStore.finishWriting();
         }
         LOGGER.info(
             "Time taken to load blocklet datamap from file : " + dataMapModel.getFilePath() + "is " + (
                 System.currentTimeMillis() - startTime));
       }
     
    +  @Override public void commit() throws MemoryException, IOException {
    --- End diff --
    
    This is specific to BlockletDataMap so lets only keep this class not at interface level.
    And also I am not sure why we should separate a commit method. Is there any way to avoid this method?


---

[GitHub] carbondata pull request #2134: [WIP][CARBONDATA-2310] Refactored code to imp...

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

    https://github.com/apache/carbondata/pull/2134#discussion_r179339971
  
    --- Diff: core/src/main/java/org/apache/carbondata/core/datamap/dev/DataMap.java ---
    @@ -67,4 +68,14 @@
        */
       void clear();
     
    +  /**
    +   * This method will be required for dataMaps that require 2 stage of construction.
    +   * Ideal scenario will be first stage contains all the processing logic and second
    +   * stage includes updating to database.
    +   * Method usage can differ based on scenario and implementation
    +   *
    +   * @throws MemoryException
    +   */
    +  void commit() throws MemoryException, IOException;
    --- End diff --
    
    I think it does not make sense to have this method in interface level. Caching should only be restricted to DataMapFactory not to the level of DataMap. 


---

[GitHub] carbondata issue #2134: [WIP][CARBONDATA-2310] Refactored code to improve Di...

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

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



---

[GitHub] carbondata issue #2134: [CARBONDATA-2310] Refactored code to improve Distrib...

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

    https://github.com/apache/carbondata/pull/2134
  
    Can you explain what is refactored in the PR description


---

[GitHub] carbondata pull request #2134: [WIP][CARBONDATA-2310] Refactored code to imp...

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

    https://github.com/apache/carbondata/pull/2134#discussion_r178723563
  
    --- Diff: core/src/main/java/org/apache/carbondata/core/indexstore/blockletindex/BlockletDataMap.java ---
    @@ -124,10 +130,12 @@
     
       private UnsafeMemoryDMStore unsafeMemorySummaryDMStore;
     
    -  private SegmentProperties segmentProperties;
    +  private transient SegmentProperties segmentProperties;
    --- End diff --
    
    Add reason why it is made transient
    Reason: As it is a heavy object it is not recommended to serialize this object


---

[GitHub] carbondata issue #2134: [WIP][CARBONDATA-2310] Refactored code to improve Di...

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

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



---

[GitHub] carbondata issue #2134: [WIP][CARBONDATA-2310] Refactored code to improve Di...

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

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



---

[GitHub] carbondata pull request #2134: [WIP][CARBONDATA-2310] Refactored code to imp...

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

    https://github.com/apache/carbondata/pull/2134#discussion_r179340305
  
    --- Diff: core/src/main/java/org/apache/carbondata/core/datamap/dev/DataMapFactory.java ---
    @@ -62,6 +63,12 @@
        */
       void fireEvent(Event event);
     
    +  /**
    +   * Add the dataMap to cache
    +   * @param dataMap
    +   */
    +  void addDataMapToCache(DataMap dataMap) throws IOException, MemoryException;
    --- End diff --
    
    Better add these methods in another interface CacheableDataMap and implement it to BlockletDataMapFactory directly.
    ```
    CacheableDataMap {
      void cache(List<DataMap> dataMaps)
      List<DataMapDistributable> getAllUncachedDataMaps()
    }
    ```


---

[GitHub] carbondata issue #2134: [CARBONDATA-2310] Refactored code to improve Distrib...

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

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



---

[GitHub] carbondata issue #2134: [CARBONDATA-2310] Refactored code to improve Distrib...

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

    https://github.com/apache/carbondata/pull/2134
  
    @dhatchayani ...Merged to 1.3 branch, kindly raise the PR for master


---

[GitHub] carbondata issue #2134: [CARBONDATA-2310] Refactored code to improve Distrib...

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

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



---

[GitHub] carbondata issue #2134: [WIP][CARBONDATA-2310] Refactored code to improve Di...

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

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



---

[GitHub] carbondata issue #2134: [CARBONDATA-2310] Refactored code to improve Distrib...

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

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



---

[GitHub] carbondata issue #2134: [WIP][CARBONDATA-2310] Refactored code to improve Di...

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

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



---

[GitHub] carbondata pull request #2134: [WIP][CARBONDATA-2310] Refactored code to imp...

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

    https://github.com/apache/carbondata/pull/2134#discussion_r178724054
  
    --- Diff: core/src/main/java/org/apache/carbondata/core/indexstore/UnsafeMemoryDMStore.java ---
    @@ -32,9 +37,14 @@
     /**
      * Store the data map row @{@link DataMapRow} data to unsafe.
      */
    -public class UnsafeMemoryDMStore {
    +public class UnsafeMemoryDMStore implements Serializable {
    +
    +  /**
    +   * default serial version ID.
    +   */
    +  private static final long serialVersionUID = 1L;
     
    -  private MemoryBlock memoryBlock;
    +  private transient MemoryBlock memoryBlock;
    --- End diff --
    
    Add reason for making it transient


---

[GitHub] carbondata issue #2134: [WIP][CARBONDATA-2310] Refactored code to improve Di...

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

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



---

[GitHub] carbondata issue #2134: [CARBONDATA-2310] Refactored code to improve Distrib...

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

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



---

[GitHub] carbondata issue #2134: [WIP][CARBONDATA-2310] Refactored code to improve Di...

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

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



---

[GitHub] carbondata pull request #2134: [WIP][CARBONDATA-2310] Refactored code to imp...

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

    https://github.com/apache/carbondata/pull/2134#discussion_r178723089
  
    --- Diff: core/src/main/java/org/apache/carbondata/core/indexstore/UnsafeMemoryDMStore.java ---
    @@ -97,12 +119,17 @@ public void addIndexRowToUnsafe(DataMapRow indexRow) throws MemoryException {
         int rowSize = indexRow.getTotalSizeInBytes();
         // Check whether allocated memory is sufficient or not.
         ensureSize(rowSize);
    -    int pointer = runningLength;
    +    if (addRowToUnsafe) {
    +      int pointer = runningLength;
     
    -    for (int i = 0; i < schema.length; i++) {
    -      addToUnsafe(schema[i], indexRow, i);
    +      for (int i = 0; i < schema.length; i++) {
    +        addToUnsafe(schema[i], indexRow, i);
    +      }
    +      pointers[rowCount++] = pointer;
    +    } else {
    +      // add dataMap rows to in memory
    +      addDataMapRow(indexRow);
    --- End diff --
    
    Rename the method name to addDataMapRowToMemory


---

[GitHub] carbondata pull request #2134: [WIP][CARBONDATA-2310] Refactored code to imp...

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

    https://github.com/apache/carbondata/pull/2134#discussion_r178721900
  
    --- Diff: core/src/main/java/org/apache/carbondata/core/datastore/SegmentTaskIndexStore.java ---
    @@ -140,6 +140,10 @@ public SegmentTaskIndexWrapper get(TableSegmentUniqueIdentifier tableSegmentUniq
         lruCache.remove(tableSegmentUniqueIdentifier.getUniqueTableSegmentIdentifier());
       }
     
    +  @Override public void put(TableSegmentUniqueIdentifier key, SegmentTaskIndexWrapper value) {
    +
    +  }
    +
    --- End diff --
    
    throw unsupportedException


---

[GitHub] carbondata issue #2134: [WIP][CARBONDATA-2310] Refactored code to improve Di...

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

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



---

[GitHub] carbondata pull request #2134: [CARBONDATA-2310] Refactored code to improve ...

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

    https://github.com/apache/carbondata/pull/2134#discussion_r180988394
  
    --- Diff: core/src/main/java/org/apache/carbondata/core/cache/dictionary/AbstractDictionaryCache.java ---
    @@ -59,6 +59,10 @@ public AbstractDictionaryCache(CarbonLRUCache carbonLRUCache) {
         initThreadPoolSize();
       }
     
    +  @Override public void put(DictionaryColumnUniqueIdentifier key, Dictionary value) {
    --- End diff --
    
    move @Override to previous line
    please follow this in future


---

[GitHub] carbondata issue #2134: [WIP][CARBONDATA-2310] Refactored code to improve Di...

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

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



---

[GitHub] carbondata issue #2134: [WIP][CARBONDATA-2310] Refactored code to improve Di...

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

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



---

[GitHub] carbondata issue #2134: [CARBONDATA-2310] Refactored code to improve Distrib...

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

    https://github.com/apache/carbondata/pull/2134
  
    LGTM, can be merged after build fix


---

[GitHub] carbondata issue #2134: [WIP][CARBONDATA-2310] Refactored code to improve Di...

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

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



---

[GitHub] carbondata issue #2134: [WIP][CARBONDATA-2310] Refactored code to improve Di...

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

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



---

[GitHub] carbondata issue #2134: [WIP][CARBONDATA-2310] Refactored code to improve Di...

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

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



---

[GitHub] carbondata issue #2134: [WIP][CARBONDATA-2310] Refactored code to improve Di...

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

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



---

[GitHub] carbondata pull request #2134: [WIP][CARBONDATA-2310] Refactored code to imp...

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

    https://github.com/apache/carbondata/pull/2134#discussion_r178722797
  
    --- Diff: core/src/main/java/org/apache/carbondata/core/indexstore/BlockletDataMapIndexStore.java ---
    @@ -206,6 +206,31 @@ public void invalidate(TableBlockIndexUniqueIdentifier tableSegmentUniqueIdentif
         lruCache.remove(tableSegmentUniqueIdentifier.getUniqueTableSegmentIdentifier());
       }
     
    +  @Override public void put(TableBlockIndexUniqueIdentifier tableBlockIndexUniqueIdentifier,
    +      BlockletDataMap blockletDataMap) throws IOException, MemoryException {
    +    String uniqueTableSegmentIdentifier =
    +        tableBlockIndexUniqueIdentifier.getUniqueTableSegmentIdentifier();
    +    Object lock = segmentLockMap.get(uniqueTableSegmentIdentifier);
    +    if (lock == null) {
    +      lock = addAndGetSegmentLock(uniqueTableSegmentIdentifier);
    +    }
    +    if (null == getIfPresent(tableBlockIndexUniqueIdentifier)) {
    --- End diff --
    
    add a comment why getIfPresent Check is required -
    Reason: As dataMap will use unsafe memory, it is not recommended to overwrite an existing entry as in that case clearing unsafe memory need to be taken card. If at all datamap entry in the cache need to be overwritten then use the invalidate interface and then use the put interface


---

[GitHub] carbondata issue #2134: [CARBONDATA-2310] Refactored code to improve Distrib...

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

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



---

[GitHub] carbondata issue #2134: [WIP][CARBONDATA-2310] Refactored code to improve Di...

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

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



---

[GitHub] carbondata issue #2134: [CARBONDATA-2310] Refactored code to improve Distrib...

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

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



---

[GitHub] carbondata issue #2134: [WIP][CARBONDATA-2310] Refactored code to improve Di...

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

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



---

[GitHub] carbondata issue #2134: [WIP][CARBONDATA-2310] Refactored code to improve Di...

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

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



---

[GitHub] carbondata issue #2134: [CARBONDATA-2310] Refactored code to improve Distrib...

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

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



---

[GitHub] carbondata issue #2134: [WIP][CARBONDATA-2310] Refactored code to improve Di...

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

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



---

[GitHub] carbondata issue #2134: [CARBONDATA-2310] Refactored code to improve Distrib...

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

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


---

[GitHub] carbondata pull request #2134: [CARBONDATA-2310] Refactored code to improve ...

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

    https://github.com/apache/carbondata/pull/2134#discussion_r181629990
  
    --- Diff: core/src/main/java/org/apache/carbondata/core/cache/dictionary/AbstractDictionaryCache.java ---
    @@ -59,6 +59,10 @@ public AbstractDictionaryCache(CarbonLRUCache carbonLRUCache) {
         initThreadPoolSize();
       }
     
    +  @Override public void put(DictionaryColumnUniqueIdentifier key, Dictionary value) {
    --- End diff --
    
    ok


---

[GitHub] carbondata issue #2134: [CARBONDATA-2310] Refactored code to improve Distrib...

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

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



---

[GitHub] carbondata issue #2134: [WIP][CARBONDATA-2310] Refactored code to improve Di...

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

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



---