[GitHub] carbondata pull request #2706: [WIP] multiple issue fixes for varchar column...

classic Classic list List threaded Threaded
57 messages Options
123
Reply | Threaded
Open this post in threaded view
|

[GitHub] carbondata pull request #2706: [WIP] multiple issue fixes for varchar column...

qiuchenjian-2
GitHub user ajantha-bhat opened a pull request:

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

    [WIP] multiple issue fixes for varchar column and complex columns, row that grows more than 2MB

    **[WIP] multiple issue fixes for varchar column and complex columns that grows more than 2MB
   
    Fixed:
    1. varchar data length is more than 2MB, buffer overflow exception (thread local row buffer)
    2. read data from carbon file having one row of varchar data with 150 MB length is very slow.
   
   
    TODO:
    3. Jvm crash when data size is more than 128 MB in unsafe sort step.
    4. analyze and remove min, max for varchar columns
    5. check and restricting size of varchar columns.**
   
   
    Be sure to do all of the following checklist to help us incorporate
    your contribution quickly and easily:
   
     - [ ] Any interfaces changed?
     
     - [ ] Any backward compatibility impacted?
     
     - [ ] Document update required?
   
     - [ ] Testing done
            Please provide details on
            - Whether new unit test cases have been added or why no new tests are required?
            - How it is tested? Please attach test report.
            - Is it a performance related change? Please attach the performance test report.
            - Any additional information to help reviewers in testing this change.
           
     - [ ] For large changes, please consider breaking it into sub-tasks under an umbrella JIRA.
   


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

    $ git pull https://github.com/ajantha-bhat/carbondata ensureArray

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

    https://github.com/apache/carbondata/pull/2706.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 #2706
   
----
commit 761514ae12d58a5fa3bc3ca1377eb5bff7b9625f
Author: ajantha-bhat <ajanthabhat@...>
Date:   2018-09-10T18:04:56Z

    [WIP] multiple issue fixes for varchar column and complex columns that grows more than 2MB

----


---
Reply | Threaded
Open this post in threaded view
|

[GitHub] carbondata issue #2706: [WIP] multiple issue fixes for varchar column and co...

qiuchenjian-2
Github user CarbonDataQA commented on the issue:

    https://github.com/apache/carbondata/pull/2706
 
    Build Failed  with Spark 2.1.0, Please check CI http://136.243.101.176:8080/job/ApacheCarbonPRBuilder2.1/213/



---
Reply | Threaded
Open this post in threaded view
|

[GitHub] carbondata issue #2706: [WIP] multiple issue fixes for varchar column and co...

qiuchenjian-2
In reply to this post by qiuchenjian-2
Github user CarbonDataQA commented on the issue:

    https://github.com/apache/carbondata/pull/2706
 
    Build Failed  with Spark 2.3.1, Please check CI http://136.243.101.176:8080/job/ApacheCarbonPRBuilder2.3/8452/



---
Reply | Threaded
Open this post in threaded view
|

[GitHub] carbondata issue #2706: [WIP] multiple issue fixes for varchar column and co...

qiuchenjian-2
In reply to this post by qiuchenjian-2
Github user CarbonDataQA commented on the issue:

    https://github.com/apache/carbondata/pull/2706
 
    Build Failed with Spark 2.2.1, Please check CI http://95.216.28.178:8080/job/ApacheCarbonPRBuilder1/382/



---
Reply | Threaded
Open this post in threaded view
|

[GitHub] carbondata issue #2706: [CARBONDATA-2927] multiple issue fixes for varchar c...

qiuchenjian-2
In reply to this post by qiuchenjian-2
Github user ajantha-bhat commented on the issue:

    https://github.com/apache/carbondata/pull/2706
 
    @kumarvishal09 @ravipesala : please do in-depth review for this PR. impact is more.


---
Reply | Threaded
Open this post in threaded view
|

[GitHub] carbondata issue #2706: [CARBONDATA-2927] multiple issue fixes for varchar c...

qiuchenjian-2
In reply to this post by qiuchenjian-2
Github user CarbonDataQA commented on the issue:

    https://github.com/apache/carbondata/pull/2706
 
    Build Success with Spark 2.1.0, Please check CI http://136.243.101.176:8080/job/ApacheCarbonPRBuilder2.1/237/



---
Reply | Threaded
Open this post in threaded view
|

[GitHub] carbondata issue #2706: [CARBONDATA-2927] multiple issue fixes for varchar c...

qiuchenjian-2
In reply to this post by qiuchenjian-2
Github user CarbonDataQA commented on the issue:

    https://github.com/apache/carbondata/pull/2706
 
    Build Failed with Spark 2.2.1, Please check CI http://95.216.28.178:8080/job/ApacheCarbonPRBuilder1/406/



---
Reply | Threaded
Open this post in threaded view
|

[GitHub] carbondata issue #2706: [CARBONDATA-2927] multiple issue fixes for varchar c...

qiuchenjian-2
In reply to this post by qiuchenjian-2
Github user CarbonDataQA commented on the issue:

    https://github.com/apache/carbondata/pull/2706
 
    Build Failed  with Spark 2.3.1, Please check CI http://136.243.101.176:8080/job/ApacheCarbonPRBuilder2.3/8476/



---
Reply | Threaded
Open this post in threaded view
|

[GitHub] carbondata pull request #2706: [CARBONDATA-2927] multiple issue fixes for va...

qiuchenjian-2
In reply to this post by qiuchenjian-2
Github user xuchuanyin commented on a diff in the pull request:

    https://github.com/apache/carbondata/pull/2706#discussion_r216884374
 
    --- Diff: core/src/main/java/org/apache/carbondata/core/memory/UnsafeMemoryManager.java ---
    @@ -200,7 +200,7 @@ public static MemoryBlock allocateMemoryWithRetry(long taskId, long size)
         }
         if (baseBlock == null) {
           INSTANCE.printCurrentMemoryUsage();
    -      throw new MemoryException("Not enough memory");
    +      throw new MemoryException("Not enough memory, increase carbon.unsafe.working.memory.in.mb");
    --- End diff --
   
    I think you can optimize the error message to
    `Not enough unsafe working memory (total: , available: , request: )`


---
Reply | Threaded
Open this post in threaded view
|

[GitHub] carbondata pull request #2706: [CARBONDATA-2927] multiple issue fixes for va...

qiuchenjian-2
In reply to this post by qiuchenjian-2
Github user xuchuanyin commented on a diff in the pull request:

    https://github.com/apache/carbondata/pull/2706#discussion_r216885202
 
    --- Diff: processing/src/main/java/org/apache/carbondata/processing/loading/sort/SortStepRowHandler.java ---
    @@ -570,23 +589,31 @@ public int writeRawRowAsIntermediateSortTempRowToUnsafeMemory(Object[] row,
       private void packNoSortFieldsToBytes(Object[] row, ByteBuffer rowBuffer) {
         // convert dict & no-sort
         for (int idx = 0; idx < this.dictNoSortDimCnt; idx++) {
    +      // cannot exceed default 2MB, hence no need to call ensureArraySize
           rowBuffer.putInt((int) row[this.dictNoSortDimIdx[idx]]);
         }
         // convert no-dict & no-sort
         for (int idx = 0; idx < this.noDictNoSortDimCnt; idx++) {
           byte[] bytes = (byte[]) row[this.noDictNoSortDimIdx[idx]];
    +      // cannot exceed default 2MB, hence no need to call ensureArraySize
           rowBuffer.putShort((short) bytes.length);
           rowBuffer.put(bytes);
         }
         // convert varchar dims
         for (int idx = 0; idx < this.varcharDimCnt; idx++) {
           byte[] bytes = (byte[]) row[this.varcharDimIdx[idx]];
    +      // can exceed default 2MB, hence need to call ensureArraySize
    +      rowBuffer = UnsafeSortDataRows
    --- End diff --
   
    Should we call this method per row per column?
    Since in most scenarios, 2MB per row is enough, so will the method calling here cause performance decrease?


---
Reply | Threaded
Open this post in threaded view
|

[GitHub] carbondata pull request #2706: [CARBONDATA-2927] multiple issue fixes for va...

qiuchenjian-2
In reply to this post by qiuchenjian-2
Github user xuchuanyin commented on a diff in the pull request:

    https://github.com/apache/carbondata/pull/2706#discussion_r216885323
 
    --- Diff: processing/src/main/java/org/apache/carbondata/processing/loading/sort/SortStepRowHandler.java ---
    @@ -598,26 +625,53 @@ private void packNoSortFieldsToBytes(Object[] row, ByteBuffer rowBuffer) {
           tmpValue = row[this.measureIdx[idx]];
           tmpDataType = this.dataTypes[idx];
           if (null == tmpValue) {
    +        // can exceed default 2MB, hence need to call ensureArraySize
    +        rowBuffer = UnsafeSortDataRows
    +            .ensureArraySize(1);
             rowBuffer.put((byte) 0);
             continue;
           }
    +      // can exceed default 2MB, hence need to call ensureArraySize
    +      rowBuffer = UnsafeSortDataRows
    +          .ensureArraySize(1);
    --- End diff --
   
    bad indent, can be moved to previous line
    The same with line#642, line#647


---
Reply | Threaded
Open this post in threaded view
|

[GitHub] carbondata pull request #2706: [CARBONDATA-2927] multiple issue fixes for va...

qiuchenjian-2
In reply to this post by qiuchenjian-2
Github user xuchuanyin commented on a diff in the pull request:

    https://github.com/apache/carbondata/pull/2706#discussion_r216885637
 
    --- Diff: processing/src/main/java/org/apache/carbondata/processing/loading/sort/unsafe/UnsafeSortDataRows.java ---
    @@ -72,7 +72,7 @@
     
       private SortParameters parameters;
       private TableFieldStat tableFieldStat;
    -  private ThreadLocal<ByteBuffer> rowBuffer;
    +  private static ThreadLocal<ByteBuffer> rowBuffer;
    --- End diff --
   
    I think the 'static' here may cause problem for concurrent loading. Each loading should their own rowBuffer.


---
Reply | Threaded
Open this post in threaded view
|

[GitHub] carbondata pull request #2706: [CARBONDATA-2927] multiple issue fixes for va...

qiuchenjian-2
In reply to this post by qiuchenjian-2
Github user xuchuanyin commented on a diff in the pull request:

    https://github.com/apache/carbondata/pull/2706#discussion_r216885885
 
    --- Diff: processing/src/main/java/org/apache/carbondata/processing/loading/sort/unsafe/UnsafeSortDataRows.java ---
    @@ -326,6 +335,19 @@ private void startFileBasedMerge() throws InterruptedException {
         dataSorterAndWriterExecutorService.awaitTermination(2, TimeUnit.DAYS);
       }
     
    +  public static ByteBuffer ensureArraySize(int requestSize) {
    --- End diff --
   
    please give a comment that this method is used to increase the rowbuffer during loading.


---
Reply | Threaded
Open this post in threaded view
|

[GitHub] carbondata pull request #2706: [CARBONDATA-2927] multiple issue fixes for va...

qiuchenjian-2
In reply to this post by qiuchenjian-2
Github user xuchuanyin commented on a diff in the pull request:

    https://github.com/apache/carbondata/pull/2706#discussion_r216884722
 
    --- Diff: processing/src/main/java/org/apache/carbondata/processing/loading/sort/SortStepRowHandler.java ---
    @@ -559,7 +572,13 @@ public int writeRawRowAsIntermediateSortTempRowToUnsafeMemory(Object[] row,
         return size;
       }
     
    -
    +  private void validateUnsafeMemoryBlockSizeLimit(long unsafeRemainingLength, int size)
    --- End diff --
   
    please optimize the parameter name of 'size' for better reading, it seems that it represents the requestedSize


---
Reply | Threaded
Open this post in threaded view
|

[GitHub] carbondata pull request #2706: [CARBONDATA-2927] multiple issue fixes for va...

qiuchenjian-2
In reply to this post by qiuchenjian-2
Github user xuchuanyin commented on a diff in the pull request:

    https://github.com/apache/carbondata/pull/2706#discussion_r216885444
 
    --- Diff: processing/src/main/java/org/apache/carbondata/processing/loading/sort/unsafe/UnsafeCarbonRowPage.java ---
    @@ -59,12 +60,11 @@ public UnsafeCarbonRowPage(TableFieldStat tableFieldStat, MemoryBlock memoryBloc
         this.taskId = taskId;
         buffer = new IntPointerBuffer(this.taskId);
         this.dataBlock = memoryBlock;
    -    // TODO Only using 98% of space for safe side.May be we can have different logic.
    -    sizeToBeUsed = dataBlock.size() - (dataBlock.size() * 5) / 100;
    +    sizeToBeUsed = dataBlock.size();
    --- End diff --
   
    Is the old comment outdated? Have you ensured the 'safe side' it mentioned?


---
Reply | Threaded
Open this post in threaded view
|

[GitHub] carbondata pull request #2706: [CARBONDATA-2927] multiple issue fixes for va...

qiuchenjian-2
In reply to this post by qiuchenjian-2
Github user xuchuanyin commented on a diff in the pull request:

    https://github.com/apache/carbondata/pull/2706#discussion_r216885250
 
    --- Diff: processing/src/main/java/org/apache/carbondata/processing/loading/sort/SortStepRowHandler.java ---
    @@ -598,26 +625,53 @@ private void packNoSortFieldsToBytes(Object[] row, ByteBuffer rowBuffer) {
           tmpValue = row[this.measureIdx[idx]];
           tmpDataType = this.dataTypes[idx];
           if (null == tmpValue) {
    +        // can exceed default 2MB, hence need to call ensureArraySize
    +        rowBuffer = UnsafeSortDataRows
    +            .ensureArraySize(1);
    --- End diff --
   
    bad indent, can be moved to previous line


---
Reply | Threaded
Open this post in threaded view
|

[GitHub] carbondata pull request #2706: [CARBONDATA-2927] multiple issue fixes for va...

qiuchenjian-2
In reply to this post by qiuchenjian-2
Github user xuchuanyin commented on a diff in the pull request:

    https://github.com/apache/carbondata/pull/2706#discussion_r216885804
 
    --- Diff: processing/src/main/java/org/apache/carbondata/processing/loading/sort/unsafe/UnsafeSortDataRows.java ---
    @@ -240,11 +249,11 @@ public void addRow(Object[] row) throws CarbonSortKeyAndGroupByException {
               throw new CarbonSortKeyAndGroupByException(ex);
             }
             rowPage.addRow(row, rowBuffer.get());
    -      } catch (Exception e) {
    -        LOGGER.error(
    -            "exception occurred while trying to acquire a semaphore lock: " + e.getMessage());
    -        throw new CarbonSortKeyAndGroupByException(e);
           }
    +    } catch (Exception e) {
    +      LOGGER
    --- End diff --
   
    bad indent. we can move the msg to next line and keep method call in this line


---
Reply | Threaded
Open this post in threaded view
|

[GitHub] carbondata pull request #2706: [CARBONDATA-2927] multiple issue fixes for va...

qiuchenjian-2
In reply to this post by qiuchenjian-2
Github user xuchuanyin commented on a diff in the pull request:

    https://github.com/apache/carbondata/pull/2706#discussion_r216884982
 
    --- Diff: processing/src/main/java/org/apache/carbondata/processing/loading/sort/SortStepRowHandler.java ---
    @@ -570,23 +589,31 @@ public int writeRawRowAsIntermediateSortTempRowToUnsafeMemory(Object[] row,
       private void packNoSortFieldsToBytes(Object[] row, ByteBuffer rowBuffer) {
         // convert dict & no-sort
         for (int idx = 0; idx < this.dictNoSortDimCnt; idx++) {
    +      // cannot exceed default 2MB, hence no need to call ensureArraySize
           rowBuffer.putInt((int) row[this.dictNoSortDimIdx[idx]]);
         }
         // convert no-dict & no-sort
         for (int idx = 0; idx < this.noDictNoSortDimCnt; idx++) {
           byte[] bytes = (byte[]) row[this.noDictNoSortDimIdx[idx]];
    +      // cannot exceed default 2MB, hence no need to call ensureArraySize
    --- End diff --
   
    for one column, it may not exceed 2MB, what if we lots of no-sort-no-dict columns?


---
Reply | Threaded
Open this post in threaded view
|

[GitHub] carbondata pull request #2706: [CARBONDATA-2927] multiple issue fixes for va...

qiuchenjian-2
In reply to this post by qiuchenjian-2
Github user xuchuanyin commented on a diff in the pull request:

    https://github.com/apache/carbondata/pull/2706#discussion_r216886119
 
    --- Diff: processing/src/main/java/org/apache/carbondata/processing/loading/sort/unsafe/UnsafeSortDataRows.java ---
    @@ -326,6 +335,19 @@ private void startFileBasedMerge() throws InterruptedException {
         dataSorterAndWriterExecutorService.awaitTermination(2, TimeUnit.DAYS);
       }
     
    +  public static ByteBuffer ensureArraySize(int requestSize) {
    --- End diff --
   
    If we increase the rowbuffer runtime, is there a way to decrease it? Or if there is no need to do so, how long will this rowbuffer last?


---
Reply | Threaded
Open this post in threaded view
|

[GitHub] carbondata issue #2706: [CARBONDATA-2927] multiple issue fixes for varchar c...

qiuchenjian-2
In reply to this post by qiuchenjian-2
Github user xuchuanyin commented on the issue:

    https://github.com/apache/carbondata/pull/2706
 
    @ajantha-bhat
    Hi, I think the main problem may be that you set the 'rowbuffer' as static which should not be shared among different data loadings.
   
    Besides, the judgement for increasing rowBuffer size per row per column may decrease data loading performance.
   
    As a result, I'd like to implement this in an easier way.
   
    We can add a table propery or load option for the size of row buffer. Just keep the previous row-buffer related code as it is. All you need is to change the initial size of the rowbuffer based on the table property or load option.
   
    @kumarvishal09 @ravipesala How do you think?


---
123