You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@carbondata.apache.org by sounakr <gi...@git.apache.org> on 2017/09/14 22:37:51 UTC

[GitHub] carbondata pull request #1359: [CARBONDATA-1480]Min Max DataMap

GitHub user sounakr opened a pull request:

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

    [CARBONDATA-1480]Min Max DataMap

    Datamap Example. Implementation of Min Max Index through Datamap. And Using the Index while prunning. 
    ---


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

    $ git pull https://github.com/sounakr/incubator-carbondata minmax

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

    https://github.com/apache/carbondata/pull/1359.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 #1359
    
----
commit a46e3b7c609e070f052017edabef9355668cf00a
Author: sounakr <so...@gmail.com>
Date:   2017-09-13T11:57:23Z

    Min Max DataMap

----


---

[GitHub] carbondata issue #1359: [CARBONDATA-1480]Min Max Index Example for DataMap

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

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



---

[GitHub] carbondata issue #1359: [CARBONDATA-1480]Min Max Index Example for DataMap

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

    https://github.com/apache/carbondata/pull/1359
  
    Retest this please.


---

[GitHub] carbondata issue #1359: [CARBONDATA-1480]Min Max Index Example for DataMap

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

    https://github.com/apache/carbondata/pull/1359
  
    Build Success with Spark 1.6, Please check CI http://88.99.58.216:8080/job/ApacheCarbonPRBuilder/203/



---

[GitHub] carbondata issue #1359: [CARBONDATA-1480]Min Max Index Example for DataMap

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

    https://github.com/apache/carbondata/pull/1359
  
    @sounakr Please add example based on the PR https://github.com/apache/carbondata/pull/1376 .


---

[GitHub] carbondata issue #1359: [CARBONDATA-1480]Min Max Index Example for DataMap

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

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



---

[GitHub] carbondata issue #1359: [CARBONDATA-1480]Min Max Index Example for DataMap

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

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



---

[GitHub] carbondata pull request #1359: [CARBONDATA-1480]Min Max Index Example for Da...

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

    https://github.com/apache/carbondata/pull/1359#discussion_r139889234
  
    --- Diff: core/src/main/java/org/apache/carbondata/core/datamap/dev/DataMap.java ---
    @@ -42,6 +44,15 @@
        */
       List<Blocklet> prune(FilterResolverIntf filterExp);
     
    +  /**
    +   * Prune the datamap with blockletId. It returns the list of
    +   * blocklets where these filters can exist.
    +   *
    +   * @param filterExp
    +   * @param blockletId
    +   * @return
    +   */
    +  List<Blocklet> pruneBlockletFromBlockId(FilterResolverIntf filterExp, int blockletId);
    --- End diff --
    
    what is blockletId? I don't think this method is required in datamap


---

[GitHub] carbondata issue #1359: [CARBONDATA-1480]Min Max Index Example for DataMap

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

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



---

[GitHub] carbondata issue #1359: [CARBONDATA-1480]Min Max Index Example for DataMap

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

    https://github.com/apache/carbondata/pull/1359
  
    Build Success with Spark 1.6, Please check CI http://88.99.58.216:8080/job/ApacheCarbonPRBuilder/311/



---

[GitHub] carbondata issue #1359: [CARBONDATA-1480]Min Max Index Example for DataMap

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

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



---

[GitHub] carbondata issue #1359: [CARBONDATA-1480]Min Max Index Example for DataMap

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

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



---

[GitHub] carbondata issue #1359: [CARBONDATA-1480]Min Max Index Example for DataMap

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

    https://github.com/apache/carbondata/pull/1359
  
    Retest this please.


---

[GitHub] carbondata issue #1359: [CARBONDATA-1480]Min Max Index Example for DataMap

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

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



---

[GitHub] carbondata issue #1359: [CARBONDATA-1480]Min Max Index Example for DataMap

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

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



---

[GitHub] carbondata pull request #1359: [CARBONDATA-1480]Min Max Index Example for Da...

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

    https://github.com/apache/carbondata/pull/1359#discussion_r143711222
  
    --- Diff: examples/spark2/src/main/scala/org/apache/carbondata/examples/MinMaxDataMapFactory.java ---
    @@ -0,0 +1,157 @@
    +/*
    + * 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.examples;
    +
    +import java.io.IOException;
    +import java.util.ArrayList;
    +import java.util.Arrays;
    +import java.util.HashMap;
    +import java.util.List;
    +import java.util.Map;
    +
    +import org.apache.carbondata.core.cache.Cache;
    +import org.apache.carbondata.core.cache.CacheProvider;
    +import org.apache.carbondata.core.cache.CacheType;
    +import org.apache.carbondata.core.datamap.DataMapDistributable;
    +import org.apache.carbondata.core.datamap.DataMapMeta;
    +import org.apache.carbondata.core.datamap.TableDataMap;
    +import org.apache.carbondata.core.datamap.dev.DataMap;
    +import org.apache.carbondata.core.datamap.dev.DataMapFactory;
    +import org.apache.carbondata.core.datamap.dev.DataMapWriter;
    +import org.apache.carbondata.core.datastore.filesystem.CarbonFile;
    +import org.apache.carbondata.core.datastore.filesystem.CarbonFileFilter;
    +import org.apache.carbondata.core.datastore.impl.FileFactory;
    +import org.apache.carbondata.core.events.ChangeEvent;
    +import org.apache.carbondata.core.indexstore.TableBlockIndexUniqueIdentifier;
    +import org.apache.carbondata.core.indexstore.blockletindex.BlockletDataMap;
    +import org.apache.carbondata.core.indexstore.schema.FilterType;
    +import org.apache.carbondata.core.memory.MemoryException;
    +import org.apache.carbondata.core.metadata.AbsoluteTableIdentifier;
    +
    +
    +/**
    + * Min Max DataMap Factory
    + */
    +public class MinMaxDataMapFactory implements DataMapFactory {
    +
    +  private AbsoluteTableIdentifier identifier;
    +
    +  // segmentId -> list of index file
    +  private Map<String, List<TableBlockIndexUniqueIdentifier>> segmentMap = new HashMap<>();
    +
    +  @Override
    +  public void init(AbsoluteTableIdentifier identifier, String dataMapName) {
    +    this.identifier = identifier;
    +  }
    +
    +  /**
    +   * createWriter will return the MinMaxDataWriter.
    +   * @param segmentId
    +   * @return
    +   */
    +  @Override
    +  public DataMapWriter createWriter(String segmentId) {
    +    return new MinMaxDataWriter();
    +  }
    +
    +  /**
    +   * getDataMaps Factory method Initializes the Min Max Data Map and returns.
    +   * @param segmentId
    +   * @return
    +   * @throws IOException
    +   */
    +  @Override
    +  public List<DataMap> getDataMaps(String segmentId) throws IOException {
    +    List<TableBlockIndexUniqueIdentifier> tableBlockIndexUniqueIdentifiers =
    +        segmentMap.get(segmentId);
    +    List<DataMap> dataMapList = new ArrayList<>();
    +    if (tableBlockIndexUniqueIdentifiers == null) {
    +      tableBlockIndexUniqueIdentifiers = new ArrayList<>();
    +      CarbonFile[] listFiles = getCarbonIndexFiles(segmentId);
    +      for (int i = 0; i < listFiles.length; i++) {
    +        tableBlockIndexUniqueIdentifiers.add(
    +            new TableBlockIndexUniqueIdentifier(identifier, segmentId, listFiles[i].getName()));
    +      }
    +    }
    +    // Form a dataMap of Type MinMaxDataMap.
    +    MinMaxDataMap dataMap = new MinMaxDataMap();
    +    try {
    +      dataMap.init(tableBlockIndexUniqueIdentifiers.get(0).getFilePath());
    +    } catch (MemoryException ex) {
    +
    +    }
    +    dataMapList.add(dataMap);
    +    return dataMapList;
    +  }
    +
    +  /**
    +   * Routine to retrieve the carbonIndex.
    +   * @param segmentId
    +   * @return
    +   */
    +  private CarbonFile[] getCarbonIndexFiles(String segmentId) {
    --- End diff --
    
    why this method required


---

[GitHub] carbondata issue #1359: [CARBONDATA-1480]Min Max Index Example for DataMap

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

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


---

[GitHub] carbondata pull request #1359: [CARBONDATA-1480]Min Max Index Example for Da...

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

    https://github.com/apache/carbondata/pull/1359#discussion_r139059342
  
    --- Diff: core/src/main/java/org/apache/carbondata/core/datamap/dev/DataMap.java ---
    @@ -31,7 +31,8 @@
       /**
        * It is called to load the data map to memory or to initialize it.
        */
    -  void init(String filePath) throws MemoryException, IOException;
    +  void init(String blockletIndexPath, String customIndexPath, String segmentId)
    --- End diff --
    
    The `filepath` supposed to be either index folder name or index file name, so I don't think this extra information is required here.
    And also `blockletIndexPath` is not supposed passed as we have carbonIndex exists in other datamap and we supposed to use it.


---

[GitHub] carbondata issue #1359: [CARBONDATA-1480]Min Max Index Example for DataMap

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

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



---

[GitHub] carbondata issue #1359: [CARBONDATA-1480]Min Max Index Example for DataMap

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

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



---

[GitHub] carbondata pull request #1359: [CARBONDATA-1480]Min Max Index Example for Da...

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

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


---

[GitHub] carbondata issue #1359: [CARBONDATA-1480]Min Max Index Example for DataMap

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

    https://github.com/apache/carbondata/pull/1359
  
    Retest this please.


---

[GitHub] carbondata issue #1359: [CARBONDATA-1480]Min Max Index Example for DataMap

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

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



---

[GitHub] carbondata issue #1359: [CARBONDATA-1480]Min Max Index Example for DataMap

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

    https://github.com/apache/carbondata/pull/1359
  
    Build Failed with Spark 1.6, Please check CI http://88.99.58.216:8080/job/ApacheCarbonPRBuilder/278/



---

[GitHub] carbondata issue #1359: [CARBONDATA-1480]Min Max Index Example for DataMap

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

    https://github.com/apache/carbondata/pull/1359
  
    Build Success with Spark 1.6, Please check CI http://88.99.58.216:8080/job/ApacheCarbonPRBuilder/114/



---

[GitHub] carbondata pull request #1359: [CARBONDATA-1480]Min Max Index Example for Da...

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

    https://github.com/apache/carbondata/pull/1359#discussion_r143709953
  
    --- Diff: core/src/main/java/org/apache/carbondata/core/indexstore/SegmentPropertiesFetcher.java ---
    @@ -0,0 +1,30 @@
    +/*
    + * 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.indexstore;
    +
    +import java.io.IOException;
    +
    +import org.apache.carbondata.core.datastore.block.SegmentProperties;
    +import org.apache.carbondata.core.metadata.AbsoluteTableIdentifier;
    +
    +public interface SegmentPropertiesFetcher {
    +
    +  SegmentProperties getSegmentProperties(String filePath) throws IOException;
    +
    +  SegmentProperties getSegmentProperties(AbsoluteTableIdentifier absoluteTableIdentifier);
    --- End diff --
    
    Done. Removed it.


---

[GitHub] carbondata issue #1359: [CARBONDATA-1480]Min Max Index Example for DataMap

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

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



---

[GitHub] carbondata issue #1359: [CARBONDATA-1480]Min Max Index Example for DataMap

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

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



---

[GitHub] carbondata pull request #1359: [CARBONDATA-1480]Min Max Index Example for Da...

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

    https://github.com/apache/carbondata/pull/1359#discussion_r139123880
  
    --- Diff: core/src/main/java/org/apache/carbondata/core/datamap/dev/DataMapWriter.java ---
    @@ -32,7 +32,12 @@
       /**
        * End of block notification
        */
    -  void onBlockEnd(String blockId);
    +  void onBlockEnd(String blockId, String directoryPath);
    +
    +  /**
    +   * End of block notification when index got created.
    +   */
    +  void onBlockEndWithIndex(String blockId, String directoryPath);
    --- End diff --
    
    But during onBlockEnd as the carbonIndex is not yet written, we wont be able to access the carbonIndex files. In the example i am gathering informations from CarbonIndex Files too. 
    Better to keep hook after writing Index Files also. In future we may need some more hooks at different points.


---

[GitHub] carbondata pull request #1359: [CARBONDATA-1480]Min Max Index Example for Da...

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

    https://github.com/apache/carbondata/pull/1359#discussion_r143708431
  
    --- Diff: core/src/main/java/org/apache/carbondata/core/indexstore/SegmentPropertiesFetcher.java ---
    @@ -0,0 +1,30 @@
    +/*
    + * 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.indexstore;
    +
    +import java.io.IOException;
    +
    +import org.apache.carbondata.core.datastore.block.SegmentProperties;
    +import org.apache.carbondata.core.metadata.AbsoluteTableIdentifier;
    +
    +public interface SegmentPropertiesFetcher {
    +
    +  SegmentProperties getSegmentProperties(String filePath) throws IOException;
    +
    +  SegmentProperties getSegmentProperties(AbsoluteTableIdentifier absoluteTableIdentifier);
    --- End diff --
    
    Don't add if it is not used


---

[GitHub] carbondata issue #1359: [CARBONDATA-1480]Min Max Index Example for DataMap

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

    https://github.com/apache/carbondata/pull/1359
  
    @ravipesala and @jackylk , sure will make it simple. Will check if some more interfaces needs to be opened.


---

[GitHub] carbondata issue #1359: [CARBONDATA-1480]Min Max Index Example for DataMap

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

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



---

[GitHub] carbondata pull request #1359: [CARBONDATA-1480]Min Max Index Example for Da...

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

    https://github.com/apache/carbondata/pull/1359#discussion_r145031889
  
    --- Diff: core/src/main/java/org/apache/carbondata/core/datamap/dev/DataMapWriter.java ---
    @@ -45,14 +45,13 @@
        * @param blockletId sequence number of blocklet in the block
        */
       void onBlockletEnd(int blockletId);
    -
       /**
        * Add the column pages row to the datamap, order of pages is same as `indexColumns` in
        * DataMapMeta returned in DataMapFactory.
        *
        * Implementation should copy the content of `pages` as needed, because `pages` memory
        * may be freed after this method returns, if using unsafe column page.
        */
    -  void onPageAdded(int blockletId, int pageId, ColumnPage[] pages);
    +  void onPageAdded(int blockletId, int pageId, ColumnPage[] pages, String directoryPath);
    --- End diff --
    
    why do we require block path? 


---

[GitHub] carbondata issue #1359: [CARBONDATA-1480]Min Max Index Example for DataMap

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

    https://github.com/apache/carbondata/pull/1359
  
    Build Success with Spark 1.6, Please check CI http://88.99.58.216:8080/job/ApacheCarbonPRBuilder/291/



---

[GitHub] carbondata issue #1359: [CARBONDATA-1480]Min Max Index Example for DataMap

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

    https://github.com/apache/carbondata/pull/1359
  
    Build Success with Spark 1.6, Please check CI http://88.99.58.216:8080/job/ApacheCarbonPRBuilder/105/



---

[GitHub] carbondata issue #1359: [CARBONDATA-1480]Min Max Index Example for DataMap

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

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



---

[GitHub] carbondata issue #1359: [CARBONDATA-1480]Min Max Index Example for DataMap

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

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



---

[GitHub] carbondata issue #1359: [CARBONDATA-1480]Min Max Index Example for DataMap

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

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



---

[GitHub] carbondata issue #1359: [CARBONDATA-1480]Min Max Index Example for DataMap

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

    https://github.com/apache/carbondata/pull/1359
  
    Build Success with Spark 1.6, Please check CI http://88.99.58.216:8080/job/ApacheCarbonPRBuilder/347/



---

[GitHub] carbondata issue #1359: [CARBONDATA-1480]Min Max Index Example for DataMap

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

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



---

[GitHub] carbondata issue #1359: [CARBONDATA-1480]Min Max Index Example for DataMap

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

    https://github.com/apache/carbondata/pull/1359
  
    Build Success with Spark 1.6, Please check CI http://88.99.58.216:8080/job/ApacheCarbonPRBuilder/113/



---

[GitHub] carbondata issue #1359: [CARBONDATA-1480]Min Max Index Example for DataMap

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

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



---

[GitHub] carbondata issue #1359: [CARBONDATA-1480]Min Max Index Example for DataMap

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

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



---

[GitHub] carbondata pull request #1359: [CARBONDATA-1480]Min Max Index Example for Da...

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

    https://github.com/apache/carbondata/pull/1359#discussion_r143710556
  
    --- Diff: examples/spark2/src/main/scala/org/apache/carbondata/examples/MinMaxBlockletComparator.java ---
    @@ -0,0 +1,134 @@
    +/*
    + * 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.examples;
    +
    +import java.nio.ByteBuffer;
    +import java.util.Comparator;
    +
    +import org.apache.carbondata.core.util.ByteUtil;
    +
    +
    +/**
    + * Data map comparator
    + */
    +public class MinMaxBlockletComparator implements Comparator<byte[]> {
    --- End diff --
    
    I think this class is not required


---

[GitHub] carbondata issue #1359: [CARBONDATA-1480]Min Max Index Example for DataMap

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

    https://github.com/apache/carbondata/pull/1359
  
    Build Success with Spark 1.6, Please check CI http://88.99.58.216:8080/job/ApacheCarbonPRBuilder/47/



---

[GitHub] carbondata issue #1359: [CARBONDATA-1480]Min Max Index Example for DataMap

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

    https://github.com/apache/carbondata/pull/1359
  
    Build Success with Spark 1.6, Please check CI http://88.99.58.216:8080/job/ApacheCarbonPRBuilder/292/



---

[GitHub] carbondata issue #1359: [CARBONDATA-1480]Min Max Index Example for DataMap

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

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



---

[GitHub] carbondata issue #1359: [CARBONDATA-1480]Min Max Index Example for DataMap

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

    https://github.com/apache/carbondata/pull/1359
  
    @sounakr I feel this same as Ravindra, let's make the example in a simplest way, so that developers can understand the concept of datamap and the usage of it in short time.  


---

[GitHub] carbondata pull request #1359: [CARBONDATA-1480]Min Max Index Example for Da...

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

    https://github.com/apache/carbondata/pull/1359#discussion_r143714171
  
    --- Diff: examples/spark2/src/main/scala/org/apache/carbondata/examples/MinMaxBlockletComparator.java ---
    @@ -0,0 +1,134 @@
    +/*
    + * 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.examples;
    +
    +import java.nio.ByteBuffer;
    +import java.util.Comparator;
    +
    +import org.apache.carbondata.core.util.ByteUtil;
    +
    +
    +/**
    + * Data map comparator
    + */
    +public class MinMaxBlockletComparator implements Comparator<byte[]> {
    --- End diff --
    
    Removed


---

[GitHub] carbondata issue #1359: [CARBONDATA-1480]Min Max Index Example for DataMap

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

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



---

[GitHub] carbondata issue #1359: [CARBONDATA-1480]Min Max Index Example for DataMap

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

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



---

[GitHub] carbondata issue #1359: [CARBONDATA-1480]Min Max Index Example for DataMap

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

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



---

[GitHub] carbondata issue #1359: [CARBONDATA-1480]Min Max Index Example for DataMap

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

    https://github.com/apache/carbondata/pull/1359
  
    Retest this please.


---

[GitHub] carbondata pull request #1359: [CARBONDATA-1480]Min Max Index Example for Da...

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

    https://github.com/apache/carbondata/pull/1359#discussion_r143714764
  
    --- Diff: examples/spark2/src/main/scala/org/apache/carbondata/examples/MinMaxDataMap.java ---
    @@ -0,0 +1,160 @@
    +/*
    + * 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.examples;
    +
    +import java.io.BufferedReader;
    +import java.io.DataInputStream;
    +import java.io.IOException;
    +import java.io.InputStreamReader;
    +import java.util.ArrayList;
    +import java.util.BitSet;
    +import java.util.List;
    +
    +import org.apache.carbondata.common.logging.LogService;
    +import org.apache.carbondata.common.logging.LogServiceFactory;
    +import org.apache.carbondata.core.cache.Cacheable;
    +import org.apache.carbondata.core.constants.CarbonCommonConstants;
    +import org.apache.carbondata.core.datamap.dev.DataMap;
    +import org.apache.carbondata.core.datastore.IndexKey;
    +import org.apache.carbondata.core.datastore.block.SegmentProperties;
    +import org.apache.carbondata.core.datastore.filesystem.CarbonFile;
    +import org.apache.carbondata.core.datastore.filesystem.CarbonFileFilter;
    +import org.apache.carbondata.core.datastore.impl.FileFactory;
    +import org.apache.carbondata.core.fileoperations.AtomicFileOperations;
    +import org.apache.carbondata.core.fileoperations.AtomicFileOperationsImpl;
    +import org.apache.carbondata.core.indexstore.Blocklet;
    +import org.apache.carbondata.core.memory.MemoryException;
    +import org.apache.carbondata.core.scan.filter.FilterUtil;
    +import org.apache.carbondata.core.scan.filter.executer.FilterExecuter;
    +import org.apache.carbondata.core.scan.filter.resolver.FilterResolverIntf;
    +import org.apache.carbondata.core.util.CarbonUtil;
    +
    +import com.google.gson.Gson;
    +
    +/**
    + * Datamap implementation for min max blocklet.
    + */
    +public class MinMaxDataMap implements DataMap, Cacheable {
    +
    +  private static final LogService LOGGER =
    +      LogServiceFactory.getLogService(MinMaxDataMap.class.getName());
    +
    +  public static final String NAME = "clustered.minmax.btree.blocklet";
    +
    +  private String filePath;
    +
    +  private MinMaxIndexBlockDetails[] readMinMaxDataMap;
    +
    +  @Override public void init(String filePath) throws MemoryException, IOException {
    +    this.filePath = filePath;
    +    CarbonFile[] listFiles = getCarbonIndexFiles(filePath, "0");
    +    for (int i = 0; i < listFiles.length; i++) {
    +      readMinMaxDataMap = readJson(listFiles[i].getPath());
    +    }
    +  }
    +
    +  private CarbonFile[] getCarbonIndexFiles(String filePath, String segmentId) {
    --- End diff --
    
    Done.


---

[GitHub] carbondata issue #1359: [CARBONDATA-1480]Min Max Index Example for DataMap

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

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



---

[GitHub] carbondata issue #1359: [CARBONDATA-1480]Min Max Index Example for DataMap

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

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



---

[GitHub] carbondata issue #1359: [CARBONDATA-1480]Min Max Index Example for DataMap

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

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



---

[GitHub] carbondata issue #1359: [CARBONDATA-1480]Min Max Index Example for DataMap

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

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



---

[GitHub] carbondata issue #1359: [CARBONDATA-1480]Min Max Index Example for DataMap

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

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



---

[GitHub] carbondata issue #1359: [CARBONDATA-1480]Min Max Index Example for DataMap

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

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



---

[GitHub] carbondata pull request #1359: [CARBONDATA-1480]Min Max Index Example for Da...

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

    https://github.com/apache/carbondata/pull/1359#discussion_r143708356
  
    --- Diff: core/src/main/java/org/apache/carbondata/core/indexstore/SegmentPropertiesFetcher.java ---
    @@ -0,0 +1,30 @@
    +/*
    + * 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.indexstore;
    +
    +import java.io.IOException;
    +
    +import org.apache.carbondata.core.datastore.block.SegmentProperties;
    +import org.apache.carbondata.core.metadata.AbsoluteTableIdentifier;
    +
    +public interface SegmentPropertiesFetcher {
    +
    +  SegmentProperties getSegmentProperties(String filePath) throws IOException;
    --- End diff --
    
    I thnk it is segmentID


---

[GitHub] carbondata issue #1359: [CARBONDATA-1480]Min Max Index Example for DataMap

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

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



---

[GitHub] carbondata pull request #1359: [CARBONDATA-1480]Min Max Index Example for Da...

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

    https://github.com/apache/carbondata/pull/1359#discussion_r143741467
  
    --- Diff: core/src/main/java/org/apache/carbondata/core/indexstore/blockletindex/BlockletDataMapFactory.java ---
    @@ -219,4 +225,27 @@ public DataMapMeta getMeta() {
         // TODO: pass SORT_COLUMNS into this class
         return null;
       }
    +
    +  @Override public SegmentProperties getSegmentProperties(String segmentId) throws IOException {
    +    SegmentProperties segmentProperties = null;
    --- End diff --
    
    Done.


---

[GitHub] carbondata issue #1359: [CARBONDATA-1480]Min Max Index Example for DataMap

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

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



---

[GitHub] carbondata issue #1359: [CARBONDATA-1480]Min Max Index Example for DataMap

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

    https://github.com/apache/carbondata/pull/1359
  
    Retest this please


---

[GitHub] carbondata issue #1359: [CARBONDATA-1480]Min Max Index Example for DataMap

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

    https://github.com/apache/carbondata/pull/1359
  
    Retest this please


---

[GitHub] carbondata issue #1359: [CARBONDATA-1480]Min Max Index Example for DataMap

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

    https://github.com/apache/carbondata/pull/1359
  
    Build Success with Spark 1.6, Please check CI http://88.99.58.216:8080/job/ApacheCarbonPRBuilder/313/



---

[GitHub] carbondata issue #1359: [CARBONDATA-1480]Min Max Index Example for DataMap

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

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



---

[GitHub] carbondata pull request #1359: [CARBONDATA-1480]Min Max Index Example for Da...

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

    https://github.com/apache/carbondata/pull/1359#discussion_r144249578
  
    --- Diff: core/src/main/java/org/apache/carbondata/core/datamap/dev/DataMapWriter.java ---
    @@ -45,14 +45,13 @@
        * @param blockletId sequence number of blocklet in the block
        */
       void onBlockletEnd(int blockletId);
    -
       /**
        * Add the column pages row to the datamap, order of pages is same as `indexColumns` in
        * DataMapMeta returned in DataMapFactory.
        *
        * Implementation should copy the content of `pages` as needed, because `pages` memory
        * may be freed after this method returns, if using unsafe column page.
        */
    -  void onPageAdded(int blockletId, int pageId, ColumnPage[] pages);
    +  void onPageAdded(int blockletId, int pageId, ColumnPage[] pages, String directoryPath);
    --- End diff --
    
    This is not a directory path, instead the full path to the block. SO this is constant for a block. So removed it from OnPageAdded and passed as a extra parameter to OnBlockStart as this blockPath has to be initialized on every block start.


---

[GitHub] carbondata issue #1359: [CARBONDATA-1480]Min Max Index Example for DataMap

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

    https://github.com/apache/carbondata/pull/1359
  
    Retest this please.


---

[GitHub] carbondata pull request #1359: [CARBONDATA-1480]Min Max Index Example for Da...

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

    https://github.com/apache/carbondata/pull/1359#discussion_r139890369
  
    --- Diff: core/src/main/java/org/apache/carbondata/core/datamap/dev/DataMap.java ---
    @@ -42,6 +44,15 @@
        */
       List<Blocklet> prune(FilterResolverIntf filterExp);
     
    +  /**
    +   * Prune the datamap with blockletId. It returns the list of
    +   * blocklets where these filters can exist.
    +   *
    +   * @param filterExp
    +   * @param blockletId
    +   * @return
    +   */
    +  List<Blocklet> pruneBlockletFromBlockId(FilterResolverIntf filterExp, int blockletId);
    --- End diff --
    
    BlockletId is the output of Min Max DataMap and the same is passed to BlockletDataMap in order to form the complete blocklet. 
    Instead of declaring the method pruneBlockletFromBlockId in DataMap, the same can be made a local function to blockletId.



---

[GitHub] carbondata issue #1359: [CARBONDATA-1480]Min Max Index Example for DataMap

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

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



---

[GitHub] carbondata pull request #1359: [CARBONDATA-1480]Min Max Index Example for Da...

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

    https://github.com/apache/carbondata/pull/1359#discussion_r144206409
  
    --- Diff: core/src/main/java/org/apache/carbondata/core/datamap/dev/DataMapWriter.java ---
    @@ -45,14 +45,13 @@
        * @param blockletId sequence number of blocklet in the block
        */
       void onBlockletEnd(int blockletId);
    -
       /**
        * Add the column pages row to the datamap, order of pages is same as `indexColumns` in
        * DataMapMeta returned in DataMapFactory.
        *
        * Implementation should copy the content of `pages` as needed, because `pages` memory
        * may be freed after this method returns, if using unsafe column page.
        */
    -  void onPageAdded(int blockletId, int pageId, ColumnPage[] pages);
    +  void onPageAdded(int blockletId, int pageId, ColumnPage[] pages, String directoryPath);
    --- End diff --
    
    instead of passing the directoryPath for every call, better add init method and pass the segment directory path to it.


---

[GitHub] carbondata pull request #1359: [CARBONDATA-1480]Min Max Index Example for Da...

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

    https://github.com/apache/carbondata/pull/1359#discussion_r143711091
  
    --- Diff: examples/spark2/src/main/scala/org/apache/carbondata/examples/MinMaxDataMap.java ---
    @@ -0,0 +1,160 @@
    +/*
    + * 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.examples;
    +
    +import java.io.BufferedReader;
    +import java.io.DataInputStream;
    +import java.io.IOException;
    +import java.io.InputStreamReader;
    +import java.util.ArrayList;
    +import java.util.BitSet;
    +import java.util.List;
    +
    +import org.apache.carbondata.common.logging.LogService;
    +import org.apache.carbondata.common.logging.LogServiceFactory;
    +import org.apache.carbondata.core.cache.Cacheable;
    +import org.apache.carbondata.core.constants.CarbonCommonConstants;
    +import org.apache.carbondata.core.datamap.dev.DataMap;
    +import org.apache.carbondata.core.datastore.IndexKey;
    +import org.apache.carbondata.core.datastore.block.SegmentProperties;
    +import org.apache.carbondata.core.datastore.filesystem.CarbonFile;
    +import org.apache.carbondata.core.datastore.filesystem.CarbonFileFilter;
    +import org.apache.carbondata.core.datastore.impl.FileFactory;
    +import org.apache.carbondata.core.fileoperations.AtomicFileOperations;
    +import org.apache.carbondata.core.fileoperations.AtomicFileOperationsImpl;
    +import org.apache.carbondata.core.indexstore.Blocklet;
    +import org.apache.carbondata.core.memory.MemoryException;
    +import org.apache.carbondata.core.scan.filter.FilterUtil;
    +import org.apache.carbondata.core.scan.filter.executer.FilterExecuter;
    +import org.apache.carbondata.core.scan.filter.resolver.FilterResolverIntf;
    +import org.apache.carbondata.core.util.CarbonUtil;
    +
    +import com.google.gson.Gson;
    +
    +/**
    + * Datamap implementation for min max blocklet.
    + */
    +public class MinMaxDataMap implements DataMap, Cacheable {
    --- End diff --
    
    don't implement Cacheable


---

[GitHub] carbondata issue #1359: [CARBONDATA-1480]Min Max Index Example for DataMap

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

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



---

[GitHub] carbondata issue #1359: [CARBONDATA-1480]Min Max Index Example for DataMap

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

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



---

[GitHub] carbondata issue #1359: [CARBONDATA-1480]Min Max Index Example for DataMap

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

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



---

[GitHub] carbondata issue #1359: [CARBONDATA-1480]Min Max Index Example for DataMap

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

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



---

[GitHub] carbondata issue #1359: [CARBONDATA-1480]Min Max Index Example for DataMap

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

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



---

[GitHub] carbondata pull request #1359: [CARBONDATA-1480]Min Max Index Example for Da...

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

    https://github.com/apache/carbondata/pull/1359#discussion_r143710670
  
    --- Diff: examples/spark2/src/main/scala/org/apache/carbondata/examples/MinMaxDataMap.java ---
    @@ -0,0 +1,160 @@
    +/*
    + * 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.examples;
    +
    +import java.io.BufferedReader;
    +import java.io.DataInputStream;
    +import java.io.IOException;
    +import java.io.InputStreamReader;
    +import java.util.ArrayList;
    +import java.util.BitSet;
    +import java.util.List;
    +
    +import org.apache.carbondata.common.logging.LogService;
    +import org.apache.carbondata.common.logging.LogServiceFactory;
    +import org.apache.carbondata.core.cache.Cacheable;
    +import org.apache.carbondata.core.constants.CarbonCommonConstants;
    +import org.apache.carbondata.core.datamap.dev.DataMap;
    +import org.apache.carbondata.core.datastore.IndexKey;
    +import org.apache.carbondata.core.datastore.block.SegmentProperties;
    +import org.apache.carbondata.core.datastore.filesystem.CarbonFile;
    +import org.apache.carbondata.core.datastore.filesystem.CarbonFileFilter;
    +import org.apache.carbondata.core.datastore.impl.FileFactory;
    +import org.apache.carbondata.core.fileoperations.AtomicFileOperations;
    +import org.apache.carbondata.core.fileoperations.AtomicFileOperationsImpl;
    +import org.apache.carbondata.core.indexstore.Blocklet;
    +import org.apache.carbondata.core.memory.MemoryException;
    +import org.apache.carbondata.core.scan.filter.FilterUtil;
    +import org.apache.carbondata.core.scan.filter.executer.FilterExecuter;
    +import org.apache.carbondata.core.scan.filter.resolver.FilterResolverIntf;
    +import org.apache.carbondata.core.util.CarbonUtil;
    +
    +import com.google.gson.Gson;
    +
    +/**
    + * Datamap implementation for min max blocklet.
    + */
    +public class MinMaxDataMap implements DataMap, Cacheable {
    +
    +  private static final LogService LOGGER =
    +      LogServiceFactory.getLogService(MinMaxDataMap.class.getName());
    +
    +  public static final String NAME = "clustered.minmax.btree.blocklet";
    +
    +  private String filePath;
    +
    +  private MinMaxIndexBlockDetails[] readMinMaxDataMap;
    +
    +  @Override public void init(String filePath) throws MemoryException, IOException {
    +    this.filePath = filePath;
    +    CarbonFile[] listFiles = getCarbonIndexFiles(filePath, "0");
    +    for (int i = 0; i < listFiles.length; i++) {
    +      readMinMaxDataMap = readJson(listFiles[i].getPath());
    +    }
    +  }
    +
    +  private CarbonFile[] getCarbonIndexFiles(String filePath, String segmentId) {
    --- End diff --
    
    change the method nme 


---

[GitHub] carbondata issue #1359: [CARBONDATA-1480]Min Max Index Example for DataMap

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

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



---

[GitHub] carbondata issue #1359: [CARBONDATA-1480]Min Max Index Example for DataMap

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

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



---

[GitHub] carbondata issue #1359: [CARBONDATA-1480]Min Max Index Example for DataMap

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

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



---

[GitHub] carbondata issue #1359: [CARBONDATA-1480]Min Max Index Example for DataMap

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

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



---

[GitHub] carbondata issue #1359: [CARBONDATA-1480]Min Max Index Example for DataMap

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

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



---

[GitHub] carbondata issue #1359: [CARBONDATA-1480]Min Max Index Example for DataMap

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

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



---

[GitHub] carbondata issue #1359: [CARBONDATA-1480]Min Max Index Example for DataMap

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

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



---

[GitHub] carbondata issue #1359: [CARBONDATA-1480]Min Max Index Example for DataMap

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

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



---

[GitHub] carbondata issue #1359: [CARBONDATA-1480]Min Max Index Example for DataMap

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

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



---

[GitHub] carbondata issue #1359: [CARBONDATA-1480]Min Max Index Example for DataMap

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

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



---

[GitHub] carbondata pull request #1359: [CARBONDATA-1480]Min Max Index Example for Da...

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

    https://github.com/apache/carbondata/pull/1359#discussion_r139058897
  
    --- Diff: core/src/main/java/org/apache/carbondata/core/datamap/dev/DataMapWriter.java ---
    @@ -32,7 +32,12 @@
       /**
        * End of block notification
        */
    -  void onBlockEnd(String blockId);
    +  void onBlockEnd(String blockId, String directoryPath);
    +
    +  /**
    +   * End of block notification when index got created.
    +   */
    +  void onBlockEndWithIndex(String blockId, String directoryPath);
    --- End diff --
    
    Why is this method required, why not `onBlockEnd` is enough?


---

[GitHub] carbondata pull request #1359: [CARBONDATA-1480]Min Max Index Example for Da...

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

    https://github.com/apache/carbondata/pull/1359#discussion_r139059518
  
    --- Diff: examples/spark2/src/main/scala/org/apache/carbondata/examples/MinMaxDataMapFactory.java ---
    @@ -0,0 +1,141 @@
    +/*
    + * 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.examples;
    +
    +import java.io.IOException;
    +import java.util.ArrayList;
    +import java.util.Arrays;
    +import java.util.HashMap;
    +import java.util.List;
    +import java.util.Map;
    +
    +import org.apache.carbondata.core.cache.Cache;
    +import org.apache.carbondata.core.cache.CacheProvider;
    +import org.apache.carbondata.core.cache.CacheType;
    +import org.apache.carbondata.core.datamap.DataMapDistributable;
    +import org.apache.carbondata.core.datamap.DataMapMeta;
    +import org.apache.carbondata.core.datamap.TableDataMap;
    +import org.apache.carbondata.core.datamap.dev.DataMap;
    +import org.apache.carbondata.core.datamap.dev.DataMapFactory;
    +import org.apache.carbondata.core.datamap.dev.DataMapWriter;
    +import org.apache.carbondata.core.datastore.filesystem.CarbonFile;
    +import org.apache.carbondata.core.datastore.filesystem.CarbonFileFilter;
    +import org.apache.carbondata.core.datastore.impl.FileFactory;
    +import org.apache.carbondata.core.events.ChangeEvent;
    +import org.apache.carbondata.core.indexstore.TableBlockIndexUniqueIdentifier;
    +import org.apache.carbondata.core.indexstore.blockletindex.BlockletDataMap;
    +import org.apache.carbondata.core.indexstore.schema.FilterType;
    +import org.apache.carbondata.core.memory.MemoryException;
    +import org.apache.carbondata.core.metadata.AbsoluteTableIdentifier;
    +
    +
    +/**
    + * Table map for blocklet
    + */
    +public class MinMaxDataMapFactory implements DataMapFactory {
    +
    +  private AbsoluteTableIdentifier identifier;
    +
    +  // segmentId -> list of index file
    +  private Map<String, List<TableBlockIndexUniqueIdentifier>> segmentMap = new HashMap<>();
    +
    +  private Cache<TableBlockIndexUniqueIdentifier, DataMap> cache;
    +
    +  @Override
    +  public void init(AbsoluteTableIdentifier identifier, String dataMapName) {
    +    this.identifier = identifier;
    +    cache = CacheProvider.getInstance()
    --- End diff --
    
    what is the use of this cache when don't use anywhere


---

[GitHub] carbondata issue #1359: [CARBONDATA-1480]Min Max Index Example for DataMap

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

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



---

[GitHub] carbondata issue #1359: [CARBONDATA-1480]Min Max Index Example for DataMap

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

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



---

[GitHub] carbondata pull request #1359: [CARBONDATA-1480]Min Max Index Example for Da...

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

    https://github.com/apache/carbondata/pull/1359#discussion_r139122092
  
    --- Diff: core/src/main/java/org/apache/carbondata/core/datamap/dev/DataMap.java ---
    @@ -31,7 +31,8 @@
       /**
        * It is called to load the data map to memory or to initialize it.
        */
    -  void init(String filePath) throws MemoryException, IOException;
    +  void init(String blockletIndexPath, String customIndexPath, String segmentId)
    --- End diff --
    
    it should be independent of other indexes


---

[GitHub] carbondata issue #1359: [CARBONDATA-1480]Min Max Index Example for DataMap

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

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



---

[GitHub] carbondata pull request #1359: [CARBONDATA-1480]Min Max Index Example for Da...

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

    https://github.com/apache/carbondata/pull/1359#discussion_r143721417
  
    --- Diff: examples/spark2/src/main/scala/org/apache/carbondata/examples/MinMaxDataMapFactory.java ---
    @@ -0,0 +1,157 @@
    +/*
    + * 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.examples;
    +
    +import java.io.IOException;
    +import java.util.ArrayList;
    +import java.util.Arrays;
    +import java.util.HashMap;
    +import java.util.List;
    +import java.util.Map;
    +
    +import org.apache.carbondata.core.cache.Cache;
    +import org.apache.carbondata.core.cache.CacheProvider;
    +import org.apache.carbondata.core.cache.CacheType;
    +import org.apache.carbondata.core.datamap.DataMapDistributable;
    +import org.apache.carbondata.core.datamap.DataMapMeta;
    +import org.apache.carbondata.core.datamap.TableDataMap;
    +import org.apache.carbondata.core.datamap.dev.DataMap;
    +import org.apache.carbondata.core.datamap.dev.DataMapFactory;
    +import org.apache.carbondata.core.datamap.dev.DataMapWriter;
    +import org.apache.carbondata.core.datastore.filesystem.CarbonFile;
    +import org.apache.carbondata.core.datastore.filesystem.CarbonFileFilter;
    +import org.apache.carbondata.core.datastore.impl.FileFactory;
    +import org.apache.carbondata.core.events.ChangeEvent;
    +import org.apache.carbondata.core.indexstore.TableBlockIndexUniqueIdentifier;
    +import org.apache.carbondata.core.indexstore.blockletindex.BlockletDataMap;
    +import org.apache.carbondata.core.indexstore.schema.FilterType;
    +import org.apache.carbondata.core.memory.MemoryException;
    +import org.apache.carbondata.core.metadata.AbsoluteTableIdentifier;
    +
    +
    +/**
    + * Min Max DataMap Factory
    + */
    +public class MinMaxDataMapFactory implements DataMapFactory {
    +
    +  private AbsoluteTableIdentifier identifier;
    +
    +  // segmentId -> list of index file
    +  private Map<String, List<TableBlockIndexUniqueIdentifier>> segmentMap = new HashMap<>();
    +
    +  @Override
    +  public void init(AbsoluteTableIdentifier identifier, String dataMapName) {
    +    this.identifier = identifier;
    +  }
    +
    +  /**
    +   * createWriter will return the MinMaxDataWriter.
    +   * @param segmentId
    +   * @return
    +   */
    +  @Override
    +  public DataMapWriter createWriter(String segmentId) {
    +    return new MinMaxDataWriter();
    +  }
    +
    +  /**
    +   * getDataMaps Factory method Initializes the Min Max Data Map and returns.
    +   * @param segmentId
    +   * @return
    +   * @throws IOException
    +   */
    +  @Override
    +  public List<DataMap> getDataMaps(String segmentId) throws IOException {
    +    List<TableBlockIndexUniqueIdentifier> tableBlockIndexUniqueIdentifiers =
    +        segmentMap.get(segmentId);
    +    List<DataMap> dataMapList = new ArrayList<>();
    +    if (tableBlockIndexUniqueIdentifiers == null) {
    +      tableBlockIndexUniqueIdentifiers = new ArrayList<>();
    +      CarbonFile[] listFiles = getCarbonIndexFiles(segmentId);
    +      for (int i = 0; i < listFiles.length; i++) {
    +        tableBlockIndexUniqueIdentifiers.add(
    +            new TableBlockIndexUniqueIdentifier(identifier, segmentId, listFiles[i].getName()));
    +      }
    +    }
    +    // Form a dataMap of Type MinMaxDataMap.
    +    MinMaxDataMap dataMap = new MinMaxDataMap();
    +    try {
    +      dataMap.init(tableBlockIndexUniqueIdentifiers.get(0).getFilePath());
    +    } catch (MemoryException ex) {
    +
    +    }
    +    dataMapList.add(dataMap);
    +    return dataMapList;
    +  }
    +
    +  /**
    +   * Routine to retrieve the carbonIndex.
    +   * @param segmentId
    +   * @return
    +   */
    +  private CarbonFile[] getCarbonIndexFiles(String segmentId) {
    --- End diff --
    
    removed


---

[GitHub] carbondata issue #1359: [CARBONDATA-1480]Min Max Index Example for DataMap

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

    https://github.com/apache/carbondata/pull/1359
  
    Build Success with Spark 1.6, Please check CI http://88.99.58.216:8080/job/ApacheCarbonPRBuilder/247/



---

[GitHub] carbondata pull request #1359: [CARBONDATA-1480]Min Max Index Example for Da...

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

    https://github.com/apache/carbondata/pull/1359#discussion_r143709914
  
    --- Diff: core/src/main/java/org/apache/carbondata/core/indexstore/SegmentPropertiesFetcher.java ---
    @@ -0,0 +1,30 @@
    +/*
    + * 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.indexstore;
    +
    +import java.io.IOException;
    +
    +import org.apache.carbondata.core.datastore.block.SegmentProperties;
    +import org.apache.carbondata.core.metadata.AbsoluteTableIdentifier;
    +
    +public interface SegmentPropertiesFetcher {
    +
    +  SegmentProperties getSegmentProperties(String filePath) throws IOException;
    --- End diff --
    
    Yes, it is SegmentId. Rectified.


---

[GitHub] carbondata pull request #1359: [CARBONDATA-1480]Min Max Index Example for Da...

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

    https://github.com/apache/carbondata/pull/1359#discussion_r139092331
  
    --- Diff: core/src/main/java/org/apache/carbondata/core/datamap/dev/DataMap.java ---
    @@ -31,7 +31,8 @@
       /**
        * It is called to load the data map to memory or to initialize it.
        */
    -  void init(String filePath) throws MemoryException, IOException;
    +  void init(String blockletIndexPath, String customIndexPath, String segmentId)
    --- End diff --
    
    For Min Max Index creation like segment properties and other things i am taking input from regular carbonindex file too. So by design we can have one parameter as primitive index path other can be of the new custom index file path. 


---

[GitHub] carbondata issue #1359: [CARBONDATA-1480]Min Max Index Example for DataMap

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

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



---

[GitHub] carbondata issue #1359: [CARBONDATA-1480]Min Max Index Example for DataMap

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

    https://github.com/apache/carbondata/pull/1359
  
    Build Success with Spark 1.6, Please check CI http://88.99.58.216:8080/job/ApacheCarbonPRBuilder/287/



---

[GitHub] carbondata pull request #1359: [CARBONDATA-1480]Min Max Index Example for Da...

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

    https://github.com/apache/carbondata/pull/1359#discussion_r139124564
  
    --- Diff: core/src/main/java/org/apache/carbondata/core/datamap/dev/DataMap.java ---
    @@ -31,7 +31,8 @@
       /**
        * It is called to load the data map to memory or to initialize it.
        */
    -  void init(String filePath) throws MemoryException, IOException;
    +  void init(String blockletIndexPath, String customIndexPath, String segmentId)
    --- End diff --
    
    In this example Along with Min and Max Information i am keeping few more information for building the BlockLet. Both indexes are independent but with the current example implementation i read the Min and Max index and and then read the carbonindex index also in order to get the column cardanality and segmentproperties. These values are used to form the blocklet used for pruning. 


---

[GitHub] carbondata issue #1359: [CARBONDATA-1480]Min Max Index Example for DataMap

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

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



---

[GitHub] carbondata issue #1359: [CARBONDATA-1480]Min Max Index Example for DataMap

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

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



---

[GitHub] carbondata issue #1359: [CARBONDATA-1480]Min Max Index Example for DataMap

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

    https://github.com/apache/carbondata/pull/1359
  
    Build Success with Spark 1.6, Please check CI http://88.99.58.216:8080/job/ApacheCarbonPRBuilder/246/



---

[GitHub] carbondata pull request #1359: [CARBONDATA-1480]Min Max Index Example for Da...

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

    https://github.com/apache/carbondata/pull/1359#discussion_r139122030
  
    --- Diff: core/src/main/java/org/apache/carbondata/core/datamap/dev/DataMapWriter.java ---
    @@ -32,7 +32,12 @@
       /**
        * End of block notification
        */
    -  void onBlockEnd(String blockId);
    +  void onBlockEnd(String blockId, String directoryPath);
    +
    +  /**
    +   * End of block notification when index got created.
    +   */
    +  void onBlockEndWithIndex(String blockId, String directoryPath);
    --- End diff --
    
    I did not get the meaning of index. it is supposed to be independent of other indexes. I think onBlockEnd event is enough for writing the index file.


---

[GitHub] carbondata issue #1359: [CARBONDATA-1480]Min Max Index Example for DataMap

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

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



---

[GitHub] carbondata issue #1359: [CARBONDATA-1480]Min Max Index Example for DataMap

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

    https://github.com/apache/carbondata/pull/1359
  
    Retest this please.


---

[GitHub] carbondata pull request #1359: [CARBONDATA-1480]Min Max Index Example for Da...

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

    https://github.com/apache/carbondata/pull/1359#discussion_r139068734
  
    --- Diff: core/src/main/java/org/apache/carbondata/core/datamap/dev/DataMapWriter.java ---
    @@ -32,7 +32,12 @@
       /**
        * End of block notification
        */
    -  void onBlockEnd(String blockId);
    +  void onBlockEnd(String blockId, String directoryPath);
    +
    +  /**
    +   * End of block notification when index got created.
    +   */
    +  void onBlockEndWithIndex(String blockId, String directoryPath);
    --- End diff --
    
    onBlockEnd Method is called once the block is written. onBlockEndWithIndex is called once the index is also written after the carbondata is written out. 


---

[GitHub] carbondata pull request #1359: [CARBONDATA-1480]Min Max Index Example for Da...

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

    https://github.com/apache/carbondata/pull/1359#discussion_r143710464
  
    --- Diff: core/src/main/java/org/apache/carbondata/core/indexstore/blockletindex/BlockletDataMapFactory.java ---
    @@ -219,4 +225,27 @@ public DataMapMeta getMeta() {
         // TODO: pass SORT_COLUMNS into this class
         return null;
       }
    +
    +  @Override public SegmentProperties getSegmentProperties(String segmentId) throws IOException {
    +    SegmentProperties segmentProperties = null;
    --- End diff --
    
    SegmentProperties need to be cached, it shouldn't read from disk always


---

[GitHub] carbondata pull request #1359: [CARBONDATA-1480]Min Max Index Example for Da...

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

    https://github.com/apache/carbondata/pull/1359#discussion_r139123481
  
    --- Diff: examples/spark2/src/main/scala/org/apache/carbondata/examples/MinMaxDataMapFactory.java ---
    @@ -0,0 +1,141 @@
    +/*
    + * 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.examples;
    +
    +import java.io.IOException;
    +import java.util.ArrayList;
    +import java.util.Arrays;
    +import java.util.HashMap;
    +import java.util.List;
    +import java.util.Map;
    +
    +import org.apache.carbondata.core.cache.Cache;
    +import org.apache.carbondata.core.cache.CacheProvider;
    +import org.apache.carbondata.core.cache.CacheType;
    +import org.apache.carbondata.core.datamap.DataMapDistributable;
    +import org.apache.carbondata.core.datamap.DataMapMeta;
    +import org.apache.carbondata.core.datamap.TableDataMap;
    +import org.apache.carbondata.core.datamap.dev.DataMap;
    +import org.apache.carbondata.core.datamap.dev.DataMapFactory;
    +import org.apache.carbondata.core.datamap.dev.DataMapWriter;
    +import org.apache.carbondata.core.datastore.filesystem.CarbonFile;
    +import org.apache.carbondata.core.datastore.filesystem.CarbonFileFilter;
    +import org.apache.carbondata.core.datastore.impl.FileFactory;
    +import org.apache.carbondata.core.events.ChangeEvent;
    +import org.apache.carbondata.core.indexstore.TableBlockIndexUniqueIdentifier;
    +import org.apache.carbondata.core.indexstore.blockletindex.BlockletDataMap;
    +import org.apache.carbondata.core.indexstore.schema.FilterType;
    +import org.apache.carbondata.core.memory.MemoryException;
    +import org.apache.carbondata.core.metadata.AbsoluteTableIdentifier;
    +
    +
    +/**
    + * Table map for blocklet
    + */
    +public class MinMaxDataMapFactory implements DataMapFactory {
    +
    +  private AbsoluteTableIdentifier identifier;
    +
    +  // segmentId -> list of index file
    +  private Map<String, List<TableBlockIndexUniqueIdentifier>> segmentMap = new HashMap<>();
    +
    +  private Cache<TableBlockIndexUniqueIdentifier, DataMap> cache;
    +
    +  @Override
    +  public void init(AbsoluteTableIdentifier identifier, String dataMapName) {
    +    this.identifier = identifier;
    +    cache = CacheProvider.getInstance()
    --- End diff --
    
    Removed.


---

[GitHub] carbondata issue #1359: [CARBONDATA-1480]Min Max Index Example for DataMap

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

    https://github.com/apache/carbondata/pull/1359
  
    @sounakr can you make it simpler. Please add datamap that can just return blocklet details with block+blockletid. Lets worn on integration on other PR.


---

[GitHub] carbondata issue #1359: [CARBONDATA-1480]Min Max Index Example for DataMap

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

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



---

[GitHub] carbondata issue #1359: [CARBONDATA-1480]Min Max Index Example for DataMap

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

    https://github.com/apache/carbondata/pull/1359
  
    Build Success with Spark 1.6, Please check CI http://88.99.58.216:8080/job/ApacheCarbonPRBuilder/32/



---

[GitHub] carbondata issue #1359: [CARBONDATA-1480]Min Max Index Example for DataMap

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

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



---

[GitHub] carbondata pull request #1359: [CARBONDATA-1480]Min Max Index Example for Da...

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

    https://github.com/apache/carbondata/pull/1359#discussion_r143714831
  
    --- Diff: examples/spark2/src/main/scala/org/apache/carbondata/examples/MinMaxDataMap.java ---
    @@ -0,0 +1,160 @@
    +/*
    + * 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.examples;
    +
    +import java.io.BufferedReader;
    +import java.io.DataInputStream;
    +import java.io.IOException;
    +import java.io.InputStreamReader;
    +import java.util.ArrayList;
    +import java.util.BitSet;
    +import java.util.List;
    +
    +import org.apache.carbondata.common.logging.LogService;
    +import org.apache.carbondata.common.logging.LogServiceFactory;
    +import org.apache.carbondata.core.cache.Cacheable;
    +import org.apache.carbondata.core.constants.CarbonCommonConstants;
    +import org.apache.carbondata.core.datamap.dev.DataMap;
    +import org.apache.carbondata.core.datastore.IndexKey;
    +import org.apache.carbondata.core.datastore.block.SegmentProperties;
    +import org.apache.carbondata.core.datastore.filesystem.CarbonFile;
    +import org.apache.carbondata.core.datastore.filesystem.CarbonFileFilter;
    +import org.apache.carbondata.core.datastore.impl.FileFactory;
    +import org.apache.carbondata.core.fileoperations.AtomicFileOperations;
    +import org.apache.carbondata.core.fileoperations.AtomicFileOperationsImpl;
    +import org.apache.carbondata.core.indexstore.Blocklet;
    +import org.apache.carbondata.core.memory.MemoryException;
    +import org.apache.carbondata.core.scan.filter.FilterUtil;
    +import org.apache.carbondata.core.scan.filter.executer.FilterExecuter;
    +import org.apache.carbondata.core.scan.filter.resolver.FilterResolverIntf;
    +import org.apache.carbondata.core.util.CarbonUtil;
    +
    +import com.google.gson.Gson;
    +
    +/**
    + * Datamap implementation for min max blocklet.
    + */
    +public class MinMaxDataMap implements DataMap, Cacheable {
    --- End diff --
    
    Done


---

[GitHub] carbondata issue #1359: [CARBONDATA-1480]Min Max Index Example for DataMap

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

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



---