You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@carbondata.apache.org by kumarvishal09 <gi...@git.apache.org> on 2016/12/20 06:41:49 UTC

[GitHub] incubator-carbondata pull request #450: [CARBONDATA-545]Added support for of...

GitHub user kumarvishal09 opened a pull request:

    https://github.com/apache/incubator-carbondata/pull/450

    [CARBONDATA-545]Added support for offheap storage in query

    **Problem**
    There are lots of gc when carbon is processing more number of records during query, which is impacting carbon query performance.To solve this gc problem happening when query output is too huge or when more number of records are processed.
    
    **Solution**
    Currently we are storing all the data which is read during query from carbon data file in heap, when number of query output is huge it is causing more gc. Instead of storing in heap we can store this data in offheap and will clear when scanning is finished for that query.

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

    $ git pull https://github.com/kumarvishal09/incubator-carbondata QueryOffHeapStorage

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

    https://github.com/apache/incubator-carbondata/pull/450.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 #450
    
----
commit 941d1a32e846b9b4a34667b11cc1468885f0f79a
Author: kumarvishal <ku...@gmail.com>
Date:   2016-12-14T03:33:41Z

    Added support for offheap storage in query

----


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-carbondata issue #450: [CARBONDATA-545]Added support for offheap s...

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

    https://github.com/apache/incubator-carbondata/pull/450
  
    Build Success with Spark 1.5.2, Please check CI http://136.243.101.176:8080/job/ApacheCarbonPRBuilder/501/



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-carbondata issue #450: [CARBONDATA-545]Added support for offheap s...

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

    https://github.com/apache/incubator-carbondata/pull/450
  
    Build Success with Spark 1.5.2, Please check CI http://136.243.101.176:8080/job/ApacheCarbonPRBuilder/256/



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-carbondata issue #450: [CARBONDATA-545]Added support for offheap s...

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

    https://github.com/apache/incubator-carbondata/pull/450
  
    @ravipesala Ok i will add the old flow for now.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-carbondata issue #450: [CARBONDATA-545]Added support for offheap s...

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

    https://github.com/apache/incubator-carbondata/pull/450
  
    Build Success with Spark 1.5.2, Please check CI http://136.243.101.176:8080/job/ApacheCarbonPRBuilder/502/



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-carbondata pull request #450: [CARBONDATA-545]Added support for of...

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

    https://github.com/apache/incubator-carbondata/pull/450#discussion_r93943284
  
    --- Diff: core/src/main/java/org/apache/carbondata/core/carbon/datastore/chunk/store/impl/AbstractDimensionDataChunkStore.java ---
    @@ -0,0 +1,170 @@
    +/*
    + * 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.carbon.datastore.chunk.store.impl;
    +
    +import org.apache.carbondata.core.carbon.datastore.chunk.store.DimensionDataChunkStore;
    +import org.apache.carbondata.core.constants.CarbonCommonConstants;
    +import org.apache.carbondata.core.memory.MemoryAllocatorFactory;
    +import org.apache.carbondata.core.memory.MemoryBlock;
    +import org.apache.carbondata.core.unsafe.CarbonUnsafe;
    +
    +/**
    + * Responsibility is to store dimension data in memory.
    + * storage can be on heap or offheap.
    + */
    +public abstract class AbstractDimensionDataChunkStore implements DimensionDataChunkStore {
    +
    +  /**
    +   * memory block for data page
    +   */
    +  protected MemoryBlock dataPageMemoryBlock;
    +
    +  /**
    +   * memory block for inverted index
    +   */
    +  protected MemoryBlock invertedIndexMemoryBlock;
    +
    +  /**
    +   * memory block for inverted index reverse
    +   */
    +  protected MemoryBlock invertedIndexReverseMemoryBlock;
    +
    +  /**
    +   * to check whether dimension column was explicitly sorted or not
    +   */
    +  protected boolean isExplictSorted;
    +
    +  /**
    +   * is memory released
    +   */
    +  protected boolean isMemoryReleased;
    +
    +  /**
    +   * Constructor
    +   *
    +   * @param totalSize      total size of the data to be kept
    +   * @param isInvertedIdex is inverted index present
    +   * @param numberOfRows   total number of rows
    +   */
    +  public AbstractDimensionDataChunkStore(int totalSize, boolean isInvertedIdex, int numberOfRows) {
    +    // allocating the data page
    +    this.dataPageMemoryBlock =
    +        MemoryAllocatorFactory.INSATANCE.getMemoryAllocator().allocate(totalSize);
    +    this.isExplictSorted = isInvertedIdex;
    +    if (isInvertedIdex) {
    +      // allocating the inverted index page memory
    +      invertedIndexMemoryBlock = MemoryAllocatorFactory.INSATANCE.getMemoryAllocator()
    +          .allocate(CarbonCommonConstants.INT_SIZE_IN_BYTE * numberOfRows);
    +      // allocating the inverted index reverese page memory
    +      invertedIndexReverseMemoryBlock = MemoryAllocatorFactory.INSATANCE.getMemoryAllocator()
    +          .allocate(CarbonCommonConstants.INT_SIZE_IN_BYTE * numberOfRows);
    +    }
    +  }
    +
    +  /**
    +   * Below method will be used to put the rows and its metadata in offheap
    +   *
    +   * @param invertedIndex        inverted index to be stored
    +   * @param invertedIndexReverse inverted index reverse to be stored
    +   * @param data                 data to be stored
    +   */
    +  @Override public void putArray(final int[] invertedIndex, final int[] invertedIndexReverse,
    +      final byte[] data) {
    +    // copy the data to memory
    +    CarbonUnsafe.unsafe
    +        .copyMemory(data, CarbonUnsafe.BYTE_ARRAY_OFFSET, dataPageMemoryBlock.getBaseObject(),
    +            dataPageMemoryBlock.getBaseOffset(), dataPageMemoryBlock.size());
    --- End diff --
    
    Here, the copyMemory will not check whether the data is out of boundary. It is better to wrap it with a safe boundary check. Same to other xxxChunkStore class below and the getxxx function which read data from off-heap memory internally.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-carbondata pull request #450: [CARBONDATA-545]Added support for of...

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

    https://github.com/apache/incubator-carbondata/pull/450#discussion_r94003981
  
    --- Diff: core/src/main/java/org/apache/carbondata/core/carbon/datastore/chunk/reader/dimension/v2/CompressedDimensionChunkFileBasedReaderV2.java ---
    @@ -145,63 +144,52 @@ public CompressedDimensionChunkFileBasedReaderV2(final BlockletInfo blockletInfo
           long currentDimensionOffset = dimensionChunksOffset.get(blockIndex);
           data = fileReader.readByteArray(filePath, currentDimensionOffset,
               (int) (dimensionChunksOffset.get(blockIndex + 1) - currentDimensionOffset));
    -      dimensionChunk = new byte[dimensionChunksLength.get(blockIndex)];
    -      System.arraycopy(data, copySourcePoint, dimensionChunk, 0,
    -          dimensionChunksLength.get(blockIndex));
    -      dimensionColumnChunk = CarbonUtil.readDataChunk(dimensionChunk);
    +      dimensionColumnChunk =
    +          CarbonUtil.readDataChunk(data, 0, dimensionChunksLength.get(blockIndex));
    --- End diff --
    
    it will be 0 but better to use copySourcePoint ,  I will update


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-carbondata issue #450: [CARBONDATA-545]Added support for offheap s...

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

    https://github.com/apache/incubator-carbondata/pull/450
  
    Build Success with Spark 1.5.2, Please check CI http://136.243.101.176:8080/job/ApacheCarbonPRBuilder/500/



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-carbondata pull request #450: [CARBONDATA-545]Added support for of...

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

    https://github.com/apache/incubator-carbondata/pull/450#discussion_r94775168
  
    --- Diff: core/src/main/java/org/apache/carbondata/core/carbon/datastore/chunk/store/impl/ByteMeasureChunkStore.java ---
    @@ -0,0 +1,56 @@
    +/*
    + * 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.carbon.datastore.chunk.store.impl;
    +
    +import org.apache.carbondata.core.memory.MemoryAllocatorFactory;
    +import org.apache.carbondata.core.unsafe.CarbonUnsafe;
    +
    +/**
    + * Responsible for storing Byte array data to memory.
    + * memory can be on heap or offheap based on the user configuration
    + */
    +public class ByteMeasureChunkStore extends AbstractMeasureDataChunkStore<byte[]> {
    +
    +  public ByteMeasureChunkStore(int numberOfRows) {
    +    super(numberOfRows);
    +  }
    +
    +  /**
    +   * Below method will be used to put byte array data to memory
    +   *
    +   * @param data
    +   */
    +  @Override public void putData(byte[] data) {
    +    this.dataPageMemoryBlock =
    +        MemoryAllocatorFactory.INSATANCE.getMemoryAllocator().allocate(data.length);
    +    // copy the data to memory
    +    CarbonUnsafe.unsafe
    +        .copyMemory(data, CarbonUnsafe.SHORT_ARRAY_OFFSET, dataPageMemoryBlock.getBaseObject(),
    --- End diff --
    
    Ya i missed I will update


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-carbondata pull request #450: [CARBONDATA-545]Added support for of...

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

    https://github.com/apache/incubator-carbondata/pull/450#discussion_r93832594
  
    --- Diff: core/src/main/java/org/apache/carbondata/core/carbon/datastore/chunk/reader/dimension/v2/CompressedDimensionChunkFileBasedReaderV2.java ---
    @@ -145,63 +144,52 @@ public CompressedDimensionChunkFileBasedReaderV2(final BlockletInfo blockletInfo
           long currentDimensionOffset = dimensionChunksOffset.get(blockIndex);
           data = fileReader.readByteArray(filePath, currentDimensionOffset,
               (int) (dimensionChunksOffset.get(blockIndex + 1) - currentDimensionOffset));
    -      dimensionChunk = new byte[dimensionChunksLength.get(blockIndex)];
    -      System.arraycopy(data, copySourcePoint, dimensionChunk, 0,
    -          dimensionChunksLength.get(blockIndex));
    -      dimensionColumnChunk = CarbonUtil.readDataChunk(dimensionChunk);
    +      dimensionColumnChunk =
    +          CarbonUtil.readDataChunk(data, 0, dimensionChunksLength.get(blockIndex));
    --- End diff --
    
    How about to use copySourcePoint instead of "0" to keep aligned with the code below?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-carbondata pull request #450: [CARBONDATA-545]Added support for of...

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

    https://github.com/apache/incubator-carbondata/pull/450#discussion_r93832635
  
    --- Diff: core/src/main/java/org/apache/carbondata/core/carbon/datastore/chunk/reader/measure/AbstractMeasureChunkReader.java ---
    @@ -31,14 +31,19 @@
       protected String filePath;
     
       /**
    +   * total number of records
    +   */
    +  protected int numberOfRecords;
    --- End diff --
    
    What does the "numberOfRecords" used for?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-carbondata issue #450: [CARBONDATA-545]Added support for offheap s...

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

    https://github.com/apache/incubator-carbondata/pull/450
  
    LGTM, Thank you for working on this.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-carbondata issue #450: [CARBONDATA-545]Added support for offheap s...

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

    https://github.com/apache/incubator-carbondata/pull/450
  
    Build Success with Spark 1.5.2, Please check CI http://136.243.101.176:8080/job/ApacheCarbonPRBuilder/489/



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-carbondata pull request #450: [CARBONDATA-545]Added support for of...

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

    https://github.com/apache/incubator-carbondata/pull/450#discussion_r93832426
  
    --- Diff: core/src/main/java/org/apache/carbondata/core/carbon/datastore/chunk/DimensionColumnDataChunk.java ---
    @@ -68,17 +78,37 @@ int fillConvertedChunkData(int[] rowMapping, ColumnVectorInfo[] vectorInfo, int
       byte[] getChunkData(int columnIndex);
     
       /**
    -   * Below method will be used get the chunk attributes
    -   *
    -   * @return chunk attributes
    +   * @return inverted index
    +   */
    +  int getInvertedIndex(int index);
    +
    +  /**
    +   * @return whether column is dictionary column or not
    +   */
    +  boolean isNoDicitionaryColumn();
    +
    +  /**
    +   * @return length of each column
        */
    -  DimensionChunkAttributes getAttributes();
    +  int getColumnValueSize();
     
       /**
    -   * Below method will be used to return the complete data chunk
    -   * This will be required during filter query
    +   * @return whether columns where explictly sorted or not
    +   */
    +  boolean isExplictSorted();
    --- End diff --
    
    Does it means "explicit"?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-carbondata issue #450: [CARBONDATA-545]Added support for offheap s...

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

    https://github.com/apache/incubator-carbondata/pull/450
  
    Build Failed  with Spark 1.5.2, Please check CI http://136.243.101.176:8080/job/ApacheCarbonPRBuilder/481/



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-carbondata issue #450: [CARBONDATA-545]Added support for offheap s...

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

    https://github.com/apache/incubator-carbondata/pull/450
  
    Build Failed  with Spark 1.5.2, Please check CI http://136.243.101.176:8080/job/ApacheCarbonPRBuilder/504/



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-carbondata issue #450: [CARBONDATA-545]Added support for offheap s...

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

    https://github.com/apache/incubator-carbondata/pull/450
  
    @kumarvishal09 Please don't remove the old flow(with out using unsafe). The current flow using unsafe operations may not be reliable so we keep the old flow as default and once this stable we may remove it later


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-carbondata pull request #450: [CARBONDATA-545]Added support for of...

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

    https://github.com/apache/incubator-carbondata/pull/450#discussion_r93948200
  
    --- Diff: core/src/main/java/org/apache/carbondata/core/carbon/datastore/chunk/store/impl/ByteMeasureChunkStore.java ---
    @@ -0,0 +1,56 @@
    +/*
    + * 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.carbon.datastore.chunk.store.impl;
    +
    +import org.apache.carbondata.core.memory.MemoryAllocatorFactory;
    +import org.apache.carbondata.core.unsafe.CarbonUnsafe;
    +
    +/**
    + * Responsible for storing Byte array data to memory.
    + * memory can be on heap or offheap based on the user configuration
    + */
    +public class ByteMeasureChunkStore extends AbstractMeasureDataChunkStore<byte[]> {
    +
    +  public ByteMeasureChunkStore(int numberOfRows) {
    +    super(numberOfRows);
    +  }
    +
    +  /**
    +   * Below method will be used to put byte array data to memory
    +   *
    +   * @param data
    +   */
    +  @Override public void putData(byte[] data) {
    +    this.dataPageMemoryBlock =
    +        MemoryAllocatorFactory.INSATANCE.getMemoryAllocator().allocate(data.length);
    --- End diff --
    
    As the isMemoryReleased is false after construct the class, if then call freeMemory before calling putData, the object will free a null object. Or if putData is called twice continuously, the older dataPageMemoryBlock will not be referenced and the memory it holds will leak. It seems this class must be careful use to avoid memory problem on off-heap. It not has a strong status check to make program robust.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-carbondata issue #450: [CARBONDATA-545]Added support for offheap s...

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

    https://github.com/apache/incubator-carbondata/pull/450
  
    Build Failed  with Spark 1.5.2, Please check CI http://136.243.101.176:8080/job/ApacheCarbonPRBuilder/486/



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-carbondata issue #450: [CARBONDATA-545]Added support for offheap s...

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

    https://github.com/apache/incubator-carbondata/pull/450
  
    Build Failed  with Spark 1.5.2, Please check CI http://136.243.101.176:8080/job/ApacheCarbonPRBuilder/482/



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-carbondata issue #450: [CARBONDATA-545]Added support for offheap s...

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

    https://github.com/apache/incubator-carbondata/pull/450
  
    Build Success with Spark 1.5.2, Please check CI http://136.243.101.176:8080/job/ApacheCarbonPRBuilder/507/



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-carbondata issue #450: [CARBONDATA-545]Added support for offheap s...

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

    https://github.com/apache/incubator-carbondata/pull/450
  
    retest this please


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-carbondata pull request #450: [CARBONDATA-545]Added support for of...

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

    https://github.com/apache/incubator-carbondata/pull/450


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-carbondata pull request #450: [CARBONDATA-545]Added support for of...

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

    https://github.com/apache/incubator-carbondata/pull/450#discussion_r94003998
  
    --- Diff: core/src/main/java/org/apache/carbondata/core/datastorage/store/compression/ValueCompressonHolder.java ---
    @@ -78,13 +86,22 @@ public static void unCompress(DataType dataType, UnCompressValue value, byte[] d
     
         UnCompressValue compress();
     
    -    UnCompressValue uncompress(DataType dataType);
    +    UnCompressValue uncompress(DataType dataType, byte[] compressData, int offset, int length,
    +        int decimal, Object maxValueObject);
    +
    +    void setUncomressValue(T data, int decimal, Object maxValueObject);
    --- End diff --
    
    I will update the method name


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-carbondata pull request #450: [CARBONDATA-545]Added support for of...

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

    https://github.com/apache/incubator-carbondata/pull/450#discussion_r94775294
  
    --- Diff: core/src/main/java/org/apache/carbondata/core/carbon/datastore/chunk/store/impl/ByteMeasureChunkStore.java ---
    @@ -0,0 +1,56 @@
    +/*
    + * 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.carbon.datastore.chunk.store.impl;
    +
    +import org.apache.carbondata.core.memory.MemoryAllocatorFactory;
    +import org.apache.carbondata.core.unsafe.CarbonUnsafe;
    +
    +/**
    + * Responsible for storing Byte array data to memory.
    + * memory can be on heap or offheap based on the user configuration
    + */
    +public class ByteMeasureChunkStore extends AbstractMeasureDataChunkStore<byte[]> {
    +
    +  public ByteMeasureChunkStore(int numberOfRows) {
    +    super(numberOfRows);
    +  }
    +
    +  /**
    +   * Below method will be used to put byte array data to memory
    +   *
    +   * @param data
    +   */
    +  @Override public void putData(byte[] data) {
    +    this.dataPageMemoryBlock =
    +        MemoryAllocatorFactory.INSATANCE.getMemoryAllocator().allocate(data.length);
    --- End diff --
    
    I will add assert to check the same, thanks for pointing this out


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-carbondata issue #450: [CARBONDATA-545]Added support for offheap s...

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

    https://github.com/apache/incubator-carbondata/pull/450
  
    Build Failed  with Spark 1.5.2, Please check CI http://136.243.101.176:8080/job/ApacheCarbonPRBuilder/495/



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-carbondata pull request #450: [CARBONDATA-545]Added support for of...

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

    https://github.com/apache/incubator-carbondata/pull/450#discussion_r94004597
  
    --- Diff: core/src/main/java/org/apache/carbondata/core/carbon/datastore/chunk/reader/measure/AbstractMeasureChunkReader.java ---
    @@ -31,14 +31,19 @@
       protected String filePath;
     
       /**
    +   * total number of records
    +   */
    +  protected int numberOfRecords;
    --- End diff --
    
    I will remove


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-carbondata issue #450: [CARBONDATA-545]Added support for offheap s...

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

    https://github.com/apache/incubator-carbondata/pull/450
  
    Build Failed  with Spark 1.5.2, Please check CI http://136.243.101.176:8080/job/ApacheCarbonPRBuilder/484/



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-carbondata issue #450: [CARBONDATA-545]Added support for offheap s...

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

    https://github.com/apache/incubator-carbondata/pull/450
  
    Build Failed  with Spark 1.5.2, Please check CI http://136.243.101.176:8080/job/ApacheCarbonPRBuilder/485/



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-carbondata issue #450: [CARBONDATA-545]Added support for offheap s...

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

    https://github.com/apache/incubator-carbondata/pull/450
  
    Build Failed  with Spark 1.5.2, Please check CI http://136.243.101.176:8080/job/ApacheCarbonPRBuilder/487/



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-carbondata issue #450: [CARBONDATA-545]Added support for offheap s...

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

    https://github.com/apache/incubator-carbondata/pull/450
  
    @kumarvishal09 
    please rebase and fix some known issues.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-carbondata issue #450: [CARBONDATA-545]Added support for offheap s...

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

    https://github.com/apache/incubator-carbondata/pull/450
  
    Build Failed  with Spark 1.5.2, Please check CI http://136.243.101.176:8080/job/ApacheCarbonPRBuilder/488/



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-carbondata pull request #450: [CARBONDATA-545]Added support for of...

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

    https://github.com/apache/incubator-carbondata/pull/450#discussion_r93943712
  
    --- Diff: core/src/main/java/org/apache/carbondata/core/carbon/datastore/chunk/store/impl/ByteMeasureChunkStore.java ---
    @@ -0,0 +1,56 @@
    +/*
    + * 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.carbon.datastore.chunk.store.impl;
    +
    +import org.apache.carbondata.core.memory.MemoryAllocatorFactory;
    +import org.apache.carbondata.core.unsafe.CarbonUnsafe;
    +
    +/**
    + * Responsible for storing Byte array data to memory.
    + * memory can be on heap or offheap based on the user configuration
    + */
    +public class ByteMeasureChunkStore extends AbstractMeasureDataChunkStore<byte[]> {
    +
    +  public ByteMeasureChunkStore(int numberOfRows) {
    +    super(numberOfRows);
    +  }
    +
    +  /**
    +   * Below method will be used to put byte array data to memory
    +   *
    +   * @param data
    +   */
    +  @Override public void putData(byte[] data) {
    +    this.dataPageMemoryBlock =
    +        MemoryAllocatorFactory.INSATANCE.getMemoryAllocator().allocate(data.length);
    +    // copy the data to memory
    +    CarbonUnsafe.unsafe
    +        .copyMemory(data, CarbonUnsafe.SHORT_ARRAY_OFFSET, dataPageMemoryBlock.getBaseObject(),
    --- End diff --
    
    It seems should be BYTE_ARRAY_OFFSET


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-carbondata issue #450: [CARBONDATA-545]Added support for offheap s...

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

    https://github.com/apache/incubator-carbondata/pull/450
  
    Build Success with Spark 1.5.2, Please check CI http://136.243.101.176:8080/job/ApacheCarbonPRBuilder/506/



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-carbondata issue #450: [CARBONDATA-545]Added support for offheap s...

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

    https://github.com/apache/incubator-carbondata/pull/450
  
    Build Failed  with Spark 1.5.2, Please check CI http://136.243.101.176:8080/job/ApacheCarbonPRBuilder/505/



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-carbondata pull request #450: [CARBONDATA-545]Added support for of...

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

    https://github.com/apache/incubator-carbondata/pull/450#discussion_r93832815
  
    --- Diff: core/src/main/java/org/apache/carbondata/core/datastorage/store/compression/ValueCompressonHolder.java ---
    @@ -78,13 +86,22 @@ public static void unCompress(DataType dataType, UnCompressValue value, byte[] d
     
         UnCompressValue compress();
     
    -    UnCompressValue uncompress(DataType dataType);
    +    UnCompressValue uncompress(DataType dataType, byte[] compressData, int offset, int length,
    +        int decimal, Object maxValueObject);
    +
    +    void setUncomressValue(T data, int decimal, Object maxValueObject);
    --- End diff --
    
    Should the function name be "setUncompressValue"?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-carbondata issue #450: [CARBONDATA-545]Added support for offheap s...

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

    https://github.com/apache/incubator-carbondata/pull/450
  
    Build Failed  with Spark 1.5.2, Please check CI http://136.243.101.176:8080/job/ApacheCarbonPRBuilder/483/



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-carbondata pull request #450: [CARBONDATA-545]Added support for of...

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

    https://github.com/apache/incubator-carbondata/pull/450#discussion_r94003961
  
    --- Diff: core/src/main/java/org/apache/carbondata/core/carbon/datastore/chunk/DimensionColumnDataChunk.java ---
    @@ -68,17 +78,37 @@ int fillConvertedChunkData(int[] rowMapping, ColumnVectorInfo[] vectorInfo, int
       byte[] getChunkData(int columnIndex);
     
       /**
    -   * Below method will be used get the chunk attributes
    -   *
    -   * @return chunk attributes
    +   * @return inverted index
    +   */
    +  int getInvertedIndex(int index);
    +
    +  /**
    +   * @return whether column is dictionary column or not
    +   */
    +  boolean isNoDicitionaryColumn();
    +
    +  /**
    +   * @return length of each column
        */
    -  DimensionChunkAttributes getAttributes();
    +  int getColumnValueSize();
     
       /**
    -   * Below method will be used to return the complete data chunk
    -   * This will be required during filter query
    +   * @return whether columns where explictly sorted or not
    +   */
    +  boolean isExplictSorted();
    --- End diff --
    
    I will update.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-carbondata issue #450: [CARBONDATA-545]Added support for offheap s...

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

    https://github.com/apache/incubator-carbondata/pull/450
  
    Build Failed  with Spark 1.5.2, Please check CI http://136.243.101.176:8080/job/ApacheCarbonPRBuilder/491/



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-carbondata pull request #450: [CARBONDATA-545]Added support for of...

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

    https://github.com/apache/incubator-carbondata/pull/450#discussion_r94775425
  
    --- Diff: core/src/main/java/org/apache/carbondata/core/carbon/datastore/chunk/store/impl/AbstractDimensionDataChunkStore.java ---
    @@ -0,0 +1,170 @@
    +/*
    + * 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.carbon.datastore.chunk.store.impl;
    +
    +import org.apache.carbondata.core.carbon.datastore.chunk.store.DimensionDataChunkStore;
    +import org.apache.carbondata.core.constants.CarbonCommonConstants;
    +import org.apache.carbondata.core.memory.MemoryAllocatorFactory;
    +import org.apache.carbondata.core.memory.MemoryBlock;
    +import org.apache.carbondata.core.unsafe.CarbonUnsafe;
    +
    +/**
    + * Responsibility is to store dimension data in memory.
    + * storage can be on heap or offheap.
    + */
    +public abstract class AbstractDimensionDataChunkStore implements DimensionDataChunkStore {
    +
    +  /**
    +   * memory block for data page
    +   */
    +  protected MemoryBlock dataPageMemoryBlock;
    +
    +  /**
    +   * memory block for inverted index
    +   */
    +  protected MemoryBlock invertedIndexMemoryBlock;
    +
    +  /**
    +   * memory block for inverted index reverse
    +   */
    +  protected MemoryBlock invertedIndexReverseMemoryBlock;
    +
    +  /**
    +   * to check whether dimension column was explicitly sorted or not
    +   */
    +  protected boolean isExplictSorted;
    +
    +  /**
    +   * is memory released
    +   */
    +  protected boolean isMemoryReleased;
    +
    +  /**
    +   * Constructor
    +   *
    +   * @param totalSize      total size of the data to be kept
    +   * @param isInvertedIdex is inverted index present
    +   * @param numberOfRows   total number of rows
    +   */
    +  public AbstractDimensionDataChunkStore(int totalSize, boolean isInvertedIdex, int numberOfRows) {
    +    // allocating the data page
    +    this.dataPageMemoryBlock =
    +        MemoryAllocatorFactory.INSATANCE.getMemoryAllocator().allocate(totalSize);
    +    this.isExplictSorted = isInvertedIdex;
    +    if (isInvertedIdex) {
    +      // allocating the inverted index page memory
    +      invertedIndexMemoryBlock = MemoryAllocatorFactory.INSATANCE.getMemoryAllocator()
    +          .allocate(CarbonCommonConstants.INT_SIZE_IN_BYTE * numberOfRows);
    +      // allocating the inverted index reverese page memory
    +      invertedIndexReverseMemoryBlock = MemoryAllocatorFactory.INSATANCE.getMemoryAllocator()
    +          .allocate(CarbonCommonConstants.INT_SIZE_IN_BYTE * numberOfRows);
    +    }
    +  }
    +
    +  /**
    +   * Below method will be used to put the rows and its metadata in offheap
    +   *
    +   * @param invertedIndex        inverted index to be stored
    +   * @param invertedIndexReverse inverted index reverse to be stored
    +   * @param data                 data to be stored
    +   */
    +  @Override public void putArray(final int[] invertedIndex, final int[] invertedIndexReverse,
    +      final byte[] data) {
    +    // copy the data to memory
    +    CarbonUnsafe.unsafe
    +        .copyMemory(data, CarbonUnsafe.BYTE_ARRAY_OFFSET, dataPageMemoryBlock.getBaseObject(),
    +            dataPageMemoryBlock.getBaseOffset(), dataPageMemoryBlock.size());
    --- End diff --
    
    Ok I will fix it, Thanks for pointing this out


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---