You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@carbondata.apache.org by zzcclp <gi...@git.apache.org> on 2017/08/18 09:13:10 UTC

[GitHub] carbondata pull request #1266: [CARBONDATA-1393] Avoid to throw NPE when exe...

GitHub user zzcclp opened a pull request:

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

    [CARBONDATA-1393] Avoid to throw NPE when execute 'freeMemory' of UnsafeMemoryManager/UnsafeSortMemoryManager.

    UnsafeMemoryManager.freeMemoryAll(long taskId) may run before freeMemory(long taskId, MemoryBlock memoryBlock), so taskIdToMemoryBlockMap.get(taskId) will return null and then throw NPE.

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

    $ git pull https://github.com/zzcclp/carbondata CARBONDATA-1393

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

    https://github.com/apache/carbondata/pull/1266.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 #1266
    
----
commit f64ed8a385ccb8c704b7e9e532742116eb0ab454
Author: Zhang Zhichao <44...@qq.com>
Date:   2017-08-18T09:04:50Z

    [CARBONDATA-1393] Avoid to throw NPE when execute 'freeMemory' of UnsafeMemoryManager/UnsafeSortMemoryManager.
    
    Avoid to throw NPE when execute 'freeMemory' of UnsafeMemoryManager/UnsafeSortMemoryManager.

----


---
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] carbondata issue #1266: [CARBONDATA-1393] Avoid to throw NPE when execute 'f...

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

    https://github.com/apache/carbondata/pull/1266
  
    @ravipesala do you mean that even if taskIdToMemoryBlockMap does not contains taskId, it should execute allocator.free(memoryBlock), and we just need to check already cleaned or not before freeing?


---
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] carbondata issue #1266: [CARBONDATA-1393] Avoid to throw NPE when execute 'f...

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

    https://github.com/apache/carbondata/pull/1266
  
    SDV Build Failed with Spark 2.1, Please check CI http://144.76.159.231:8080/job/ApacheSDVTests/321/



---
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] carbondata issue #1266: [CARBONDATA-1393] Avoid to throw NPE when execute 'f...

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

    https://github.com/apache/carbondata/pull/1266
  
    SDV Build Success with Spark 2.1, Please check CI http://144.76.159.231:8080/job/ApacheSDVTests/326/



---
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] carbondata pull request #1266: [CARBONDATA-1393] Avoid to throw NPE when exe...

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

    https://github.com/apache/carbondata/pull/1266#discussion_r134168452
  
    --- Diff: core/src/main/java/org/apache/carbondata/core/memory/UnsafeMemoryManager.java ---
    @@ -105,13 +105,15 @@ private synchronized MemoryBlock allocateMemory(long taskId, long memoryRequeste
       }
     
       public synchronized void freeMemory(long taskId, MemoryBlock memoryBlock) {
    -    taskIdToMemoryBlockMap.get(taskId).remove(memoryBlock);
    -    allocator.free(memoryBlock);
    -    memoryUsed -= memoryBlock.size();
    -    memoryUsed = memoryUsed < 0 ? 0 : memoryUsed;
    -    LOGGER.info(
    -        "Freeing memory of size: " + memoryBlock.size() + "available memory:  " + (totalMemory
    -            - memoryUsed));
    +    if (taskIdToMemoryBlockMap.containsKey(taskId)) {
    --- End diff --
    
    @jackylk , @ravipesala  I ran the following code in my IDE:
    
    ```
    spark.sql("""
            |  CREATE TABLE IF NOT EXISTS study_carbondata (
            |    stringField1          string,
            |    stringField2          string, 
            |    stringField3          string, 
            |    intField              int, 
            |    longField             bigint,
            |    int2Field             int 
            |  )
            |  STORED BY 'carbondata'
            |  TBLPROPERTIES('DICTIONARY_INCLUDE'='stringField1, stringField2, stringField3, longField',
            |    'SORT_COLUMNS'='stringField1, stringField2, stringField3, intField',
            |    'NO_INVERTED_INDEX'='longField',
            |    'TABLE_BLOCKSIZE'='8'
            |  )
           """.stripMargin)
    df3.write
          .format("carbondata")
          .option("tableName", "study_carbondata")
          .option("compress", "true")  // just valid when tempCSV is true
          .option("tempCSV", "false")
          .option("single_pass", "true") 
          .option("sort_scope", "LOCAL_SORT") //GLOBAL_SORT  LOCAL_SORT
          .mode(SaveMode.Append)
          .save()
    df3.write
          .format("carbondata")
          .option("tableName", "study_carbondata")
          .option("compress", "true")  // just valid when tempCSV is true
          .option("tempCSV", "false")
          .option("single_pass", "true") 
          .option("sort_scope", "LOCAL_SORT") //GLOBAL_SORT  LOCAL_SORT
          .mode(SaveMode.Append)
          .save()
    
    ```
    **the configs in carbon.properties:**
    
    ```
    carbon.enable.auto.load.merge=true
    carbon.compaction.level.threshold=2,4
    carbon.major.compaction.size=1024
    carbon.number.of.cores.while.loading=2
    carbon.number.of.cores.while.compacting=2
    enable.unsafe.columnpage=true
    enable.unsafe.in.query.processing=true
    enable.unsafe.sort=true
    carbon.global.sort.rdd.storage.level=MEMORY_AND_DISK_SER
    ```
    
    I added some log in the method 'UnsafeMemoryManager.freeMemory' and method 'UnsafeMemoryManager.freeMemoryAll', and found that:
    
    2017-08-21 15:42:25,240 - [Executor task launch worker for task 12][partitionID:carbondata;queryID:21481913656922] **contains Key: true 21481895071366** 
    2017-08-21 15:42:25,241 - [Executor task launch worker for task 12][partitionID:carbondata;queryID:21481913656922] **remove Key: 21481895071366** 
    2017-08-21 15:42:25,245 - [Executor task launch worker for task 12][partitionID:carbondata;queryID:21481913656922] **contains Key: false 21481895071366** 
    
    **taskId '21481895071366' had been removed before getting from 'taskIdToMemoryBlockMap', so the NPE was occurred:**
    
    ```
    Job aborted due to stage failure: Task 0 in stage 9.0 failed 1 times, most recent failure: Lost task 0.0 in stage 9.0 (TID 12, localhost, executor driver): java.lang.NullPointerException
            at org.apache.carbondata.core.memory.UnsafeMemoryManager.freeMemory(UnsafeMemoryManager.java:109)
            at org.apache.carbondata.core.datastore.page.UnsafeFixLengthColumnPage.freeMemory(UnsafeFixLengthColumnPage.java:317)
            at org.apache.carbondata.core.datastore.page.LazyColumnPage.freeMemory(LazyColumnPage.java:202)
            at org.apache.carbondata.core.datastore.chunk.MeasureColumnDataChunk.freeMemory(MeasureColumnDataChunk.java:70)
            at org.apache.carbondata.core.scan.result.AbstractScannedResult.freeMemory(AbstractScannedResult.java:517)
            at org.apache.carbondata.core.scan.processor.AbstractDataBlockIterator.close(AbstractDataBlockIterator.java:228)
            at org.apache.carbondata.core.scan.result.iterator.AbstractDetailQueryResultIterator.close(AbstractDetailQueryResultIterator.java:306)
            at org.apache.carbondata.core.scan.executor.impl.AbstractQueryExecutor.finish(AbstractQueryExecutor.java:544)
            at org.apache.carbondata.processing.merger.CarbonCompactionExecutor.finish(CarbonCompactionExecutor.java:173)
            at org.apache.carbondata.spark.rdd.CarbonMergerRDD$$anon$1.<init>(CarbonMergerRDD.scala:242)
            at org.apache.carbondata.spark.rdd.CarbonMergerRDD.internalCompute(CarbonMergerRDD.scala:79)
            at org.apache.carbondata.spark.rdd.CarbonRDD.compute(CarbonRDD.scala:61)
            at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:323)
            at org.apache.spark.rdd.RDD.iterator(RDD.scala:287)
            at org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:87)
            at org.apache.spark.scheduler.Task.run(Task.scala:99)
            at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:322)
            at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)
            at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)
            at java.lang.Thread.run(Thread.java:745)
    
    Driver stacktrace:
    java.util.concurrent.ExecutionException: org.apache.spark.SparkException: Job aborted due to stage failure: Task 0 in stage 9.0 failed 1 times, most recent failure: Lost task 0.0 in stage 9.0 (TID 12, localhost, executor driver): java.lang.NullPointerException
            at org.apache.carbondata.core.memory.UnsafeMemoryManager.freeMemory(UnsafeMemoryManager.java:109)
            at org.apache.carbondata.core.datastore.page.UnsafeFixLengthColumnPage.freeMemory(UnsafeFixLengthColumnPage.java:317)
            at org.apache.carbondata.core.datastore.page.LazyColumnPage.freeMemory(LazyColumnPage.java:202)
            at org.apache.carbondata.core.datastore.chunk.MeasureColumnDataChunk.freeMemory(MeasureColumnDataChunk.java:70)
            at org.apache.carbondata.core.scan.result.AbstractScannedResult.freeMemory(AbstractScannedResult.java:517)
            at org.apache.carbondata.core.scan.processor.AbstractDataBlockIterator.close(AbstractDataBlockIterator.java:228)
            at org.apache.carbondata.core.scan.result.iterator.AbstractDetailQueryResultIterator.close(AbstractDetailQueryResultIterator.java:306)
            at org.apache.carbondata.core.scan.executor.impl.AbstractQueryExecutor.finish(AbstractQueryExecutor.java:544)
            at org.apache.carbondata.processing.merger.CarbonCompactionExecutor.finish(CarbonCompactionExecutor.java:173)
            at org.apache.carbondata.spark.rdd.CarbonMergerRDD$$anon$1.<init>(CarbonMergerRDD.scala:242)
            at org.apache.carbondata.spark.rdd.CarbonMergerRDD.internalCompute(CarbonMergerRDD.scala:79)
            at org.apache.carbondata.spark.rdd.CarbonRDD.compute(CarbonRDD.scala:61)
            at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:323)
            at org.apache.spark.rdd.RDD.iterator(RDD.scala:287)
            at org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:87)
            at org.apache.spark.scheduler.Task.run(Task.scala:99)
            at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:322)
            at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)
            at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)
            at java.lang.Thread.run(Thread.java:745)
    ```
    
    Note: **NPE occurs in the compaction phase, sometimes in the data loading phase I tested before, I think it's a potential problem.**



---
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] carbondata issue #1266: [CARBONDATA-1393] Avoid to throw NPE when execute 'f...

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

    https://github.com/apache/carbondata/pull/1266
  
    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] carbondata issue #1266: [CARBONDATA-1393] Avoid to throw NPE when execute 'f...

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

    https://github.com/apache/carbondata/pull/1266
  
    SDV Build Failed with Spark 2.1, Please check CI http://144.76.159.231:8080/job/ApacheSDVTests/315/



---
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] carbondata issue #1266: [CARBONDATA-1393] Avoid to throw NPE when execute 'f...

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

    https://github.com/apache/carbondata/pull/1266
  
    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] carbondata issue #1266: [CARBONDATA-1393] Avoid to throw NPE when execute 'f...

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

    https://github.com/apache/carbondata/pull/1266
  
    SDV Build Failed with Spark 2.1, Please check CI http://144.76.159.231:8080/job/ApacheSDVTests/317/



---
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] carbondata issue #1266: [CARBONDATA-1393] Avoid to throw NPE when execute 'f...

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

    https://github.com/apache/carbondata/pull/1266
  
    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] carbondata pull request #1266: [CARBONDATA-1393] Avoid to throw NPE when exe...

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

    https://github.com/apache/carbondata/pull/1266#discussion_r134081572
  
    --- Diff: core/src/main/java/org/apache/carbondata/core/memory/UnsafeMemoryManager.java ---
    @@ -105,13 +105,15 @@ private synchronized MemoryBlock allocateMemory(long taskId, long memoryRequeste
       }
     
       public synchronized void freeMemory(long taskId, MemoryBlock memoryBlock) {
    -    taskIdToMemoryBlockMap.get(taskId).remove(memoryBlock);
    -    allocator.free(memoryBlock);
    -    memoryUsed -= memoryBlock.size();
    -    memoryUsed = memoryUsed < 0 ? 0 : memoryUsed;
    -    LOGGER.info(
    -        "Freeing memory of size: " + memoryBlock.size() + "available memory:  " + (totalMemory
    -            - memoryUsed));
    +    if (taskIdToMemoryBlockMap.containsKey(taskId)) {
    --- End diff --
    
    we cannot just rely on the `taskIdToMemoryBlockMap` , if code requests to clean memoryblock then we must clean it. so `allocator.free(memoryBlock)` has to be called in all scenarios. So introduce a boolean flag inside `MemoryBlock` to check already cleaned or not . if it is not cleaned then clean the block the set the boolean to `true`. Same needs to be added in `freeMemoryAll` also.


---
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] carbondata issue #1266: [CARBONDATA-1393] Avoid to throw NPE when execute 'f...

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

    https://github.com/apache/carbondata/pull/1266
  
    @CarbonDataQA @jackylk @ravipesala can anyone review this pr?


---
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] carbondata issue #1266: [CARBONDATA-1393] Avoid to throw NPE when execute 'f...

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

    https://github.com/apache/carbondata/pull/1266
  
    SDV Build Failed with Spark 2.1, Please check CI http://144.76.159.231:8080/job/ApacheSDVTests/312/



---
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] carbondata pull request #1266: [CARBONDATA-1393] Avoid to throw NPE when exe...

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

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


---
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] carbondata issue #1266: [CARBONDATA-1393] Avoid to throw NPE when execute 'f...

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

    https://github.com/apache/carbondata/pull/1266
  
    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] carbondata issue #1266: [CARBONDATA-1393] Avoid to throw NPE when execute 'f...

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

    https://github.com/apache/carbondata/pull/1266
  
    SDV Build Success with Spark 2.1, Please check CI http://144.76.159.231:8080/job/ApacheSDVTests/322/



---
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] carbondata issue #1266: [CARBONDATA-1393] Avoid to throw NPE when execute 'f...

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

    https://github.com/apache/carbondata/pull/1266
  
    @ravipesala @jackylk please review, thanks.


---
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] carbondata issue #1266: [CARBONDATA-1393] Avoid to throw NPE when execute 'f...

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

    https://github.com/apache/carbondata/pull/1266
  
    SDV Build Success with Spark 2.1, Please check CI http://144.76.159.231:8080/job/ApacheSDVTests/285/



---
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] carbondata issue #1266: [CARBONDATA-1393] Avoid to throw NPE when execute 'f...

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

    https://github.com/apache/carbondata/pull/1266
  
    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] carbondata pull request #1266: [CARBONDATA-1393] Avoid to throw NPE when exe...

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

    https://github.com/apache/carbondata/pull/1266#discussion_r134118989
  
    --- Diff: core/src/main/java/org/apache/carbondata/core/memory/UnsafeMemoryManager.java ---
    @@ -105,13 +105,15 @@ private synchronized MemoryBlock allocateMemory(long taskId, long memoryRequeste
       }
     
       public synchronized void freeMemory(long taskId, MemoryBlock memoryBlock) {
    -    taskIdToMemoryBlockMap.get(taskId).remove(memoryBlock);
    -    allocator.free(memoryBlock);
    -    memoryUsed -= memoryBlock.size();
    -    memoryUsed = memoryUsed < 0 ? 0 : memoryUsed;
    -    LOGGER.info(
    -        "Freeing memory of size: " + memoryBlock.size() + "available memory:  " + (totalMemory
    -            - memoryUsed));
    +    if (taskIdToMemoryBlockMap.containsKey(taskId)) {
    --- End diff --
    
    I think it is better to get the root cause of this problem. Can you describe the steps to reproduce the error, or can you add a test case to show error?


---
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] carbondata issue #1266: [CARBONDATA-1393] Avoid to throw NPE when execute 'f...

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

    https://github.com/apache/carbondata/pull/1266
  
    Thanks @ravipesala @jackylk 


---
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] carbondata issue #1266: [CARBONDATA-1393] Avoid to throw NPE when execute 'f...

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

    https://github.com/apache/carbondata/pull/1266
  
    SDV Build Success with Spark 2.1, Please check CI http://144.76.159.231:8080/job/ApacheSDVTests/283/



---
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] carbondata issue #1266: [CARBONDATA-1393] Avoid to throw NPE when execute 'f...

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

    https://github.com/apache/carbondata/pull/1266
  
    we cannot just rely on the taskIdToMemoryBlockMap , if code requests to clean memoryblock then we must clean it. so allocator.free(memoryBlock) has to be called in all scenarios.taskid is introduced in loading flow to clean all registered memory blocks. it is not all safe to relay on taskID.  So it is bettere to introduce a boolean flag inside MemoryBlock to check already cleaned or not. if it is not cleaned then clean the block the set the boolean to true. Same needs to be added in freeMemoryAll also.


---
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] carbondata issue #1266: [CARBONDATA-1393] Avoid to throw NPE when execute 'f...

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

    https://github.com/apache/carbondata/pull/1266
  
    @zzcclp yes 


---
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] carbondata issue #1266: [CARBONDATA-1393] Avoid to throw NPE when execute 'f...

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

    https://github.com/apache/carbondata/pull/1266
  
    @ravipesala, @jackylk  I have introduced a boolean flag inside MemoryBlock to check already cleaned or not, please review , thanks.


---
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] carbondata issue #1266: [CARBONDATA-1393] Avoid to throw NPE when execute 'f...

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

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


---
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] carbondata issue #1266: [CARBONDATA-1393] Avoid to throw NPE when execute 'f...

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

    https://github.com/apache/carbondata/pull/1266
  
    retest it 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] carbondata issue #1266: [CARBONDATA-1393] Avoid to throw NPE when execute 'f...

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

    https://github.com/apache/carbondata/pull/1266
  
    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] carbondata issue #1266: [CARBONDATA-1393] Avoid to throw NPE when execute 'f...

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

    https://github.com/apache/carbondata/pull/1266
  
    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] carbondata issue #1266: [CARBONDATA-1393] Avoid to throw NPE when execute 'f...

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

    https://github.com/apache/carbondata/pull/1266
  
    rebase, 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] carbondata pull request #1266: [CARBONDATA-1393] Avoid to throw NPE when exe...

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

    https://github.com/apache/carbondata/pull/1266#discussion_r134106795
  
    --- Diff: core/src/main/java/org/apache/carbondata/core/memory/UnsafeMemoryManager.java ---
    @@ -105,13 +105,15 @@ private synchronized MemoryBlock allocateMemory(long taskId, long memoryRequeste
       }
     
       public synchronized void freeMemory(long taskId, MemoryBlock memoryBlock) {
    -    taskIdToMemoryBlockMap.get(taskId).remove(memoryBlock);
    -    allocator.free(memoryBlock);
    -    memoryUsed -= memoryBlock.size();
    -    memoryUsed = memoryUsed < 0 ? 0 : memoryUsed;
    -    LOGGER.info(
    -        "Freeing memory of size: " + memoryBlock.size() + "available memory:  " + (totalMemory
    -            - memoryUsed));
    +    if (taskIdToMemoryBlockMap.containsKey(taskId)) {
    --- End diff --
    
    In method 'freeMemoryAll', all memoryBlocks belongs to taskId will be freed by allocator, if taskIdToMemoryBlockMap.containsKey(taskId) == false, it means that  all memoryBlocks belongs to this taskId were freed, so we don't need to free memoryBlock again.


---
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.
---