You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@carbondata.apache.org by gv...@apache.org on 2016/11/04 12:20:46 UTC

[1/2] incubator-carbondata git commit: Problem: Dictionary cache not getting cleared after task completion in dictionary decoder

Repository: incubator-carbondata
Updated Branches:
  refs/heads/master f53d323fa -> ac62b58d7


Problem: Dictionary cache not getting cleared after task completion in dictionary decoder

Analysis: Currently LRU cache eviction policy is based on dictionary access count. For cache to remove a entry its access count must be 0. In dictionary decoder after conversion of surrogate key to actual value the access count for dictionary columns in query is not getting decremented due to which it will never be cleared from memory when LRU cache size is configured.

Fix: Add a task completion listener which will take care of clearing the dictionary in case of both success and failure

Impact area: LRU cache eviction policy which can lead to query and data load failure


Project: http://git-wip-us.apache.org/repos/asf/incubator-carbondata/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-carbondata/commit/7402645d
Tree: http://git-wip-us.apache.org/repos/asf/incubator-carbondata/tree/7402645d
Diff: http://git-wip-us.apache.org/repos/asf/incubator-carbondata/diff/7402645d

Branch: refs/heads/master
Commit: 7402645dacea9fd215f7f27b2c21615cf218a867
Parents: f53d323
Author: manishgupta88 <to...@gmail.com>
Authored: Thu Nov 3 21:18:03 2016 +0530
Committer: Venkata Ramana G <ra...@huawei.com>
Committed: Fri Nov 4 15:10:32 2016 +0530

----------------------------------------------------------------------
 .../org/apache/spark/sql/CarbonDictionaryDecoder.scala   | 11 +++++++++++
 1 file changed, 11 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/7402645d/integration/spark/src/main/scala/org/apache/spark/sql/CarbonDictionaryDecoder.scala
----------------------------------------------------------------------
diff --git a/integration/spark/src/main/scala/org/apache/spark/sql/CarbonDictionaryDecoder.scala b/integration/spark/src/main/scala/org/apache/spark/sql/CarbonDictionaryDecoder.scala
index 12cdf9d..e0a8757 100644
--- a/integration/spark/src/main/scala/org/apache/spark/sql/CarbonDictionaryDecoder.scala
+++ b/integration/spark/src/main/scala/org/apache/spark/sql/CarbonDictionaryDecoder.scala
@@ -17,6 +17,7 @@
 
 package org.apache.spark.sql
 
+import org.apache.spark.TaskContext
 import org.apache.spark.rdd.RDD
 import org.apache.spark.sql.catalyst.InternalRow
 import org.apache.spark.sql.catalyst.errors.attachTree
@@ -169,6 +170,16 @@ case class CarbonDictionaryDecoder(
           val dicts: Seq[Dictionary] = getDictionary(absoluteTableIdentifiers,
             forwardDictionaryCache)
           val dictIndex = dicts.zipWithIndex.filter(x => x._1 != null).map(x => x._2)
+          // add a task completion listener to clear dictionary that is a decisive factor for
+          // LRU eviction policy
+          val dictionaryTaskCleaner = TaskContext.get
+          dictionaryTaskCleaner.addTaskCompletionListener(context =>
+            dicts.foreach { dictionary =>
+              if (null != dictionary) {
+                dictionary.clear
+              }
+            }
+          )
           new Iterator[InternalRow] {
             val unsafeProjection = UnsafeProjection.create(output.map(_.dataType).toArray)
             var flag = true


[2/2] incubator-carbondata git commit: [CACRBONDATA-375] Dictionary cache not getting cleared in dictionary decoder.This closes #292

Posted by gv...@apache.org.
[CACRBONDATA-375] Dictionary cache not getting cleared in dictionary decoder.This closes #292


Project: http://git-wip-us.apache.org/repos/asf/incubator-carbondata/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-carbondata/commit/ac62b58d
Tree: http://git-wip-us.apache.org/repos/asf/incubator-carbondata/tree/ac62b58d
Diff: http://git-wip-us.apache.org/repos/asf/incubator-carbondata/diff/ac62b58d

Branch: refs/heads/master
Commit: ac62b58d7920b235abe47718f541d2e1fc3b07bb
Parents: f53d323 7402645
Author: Venkata Ramana G <ra...@huawei.com>
Authored: Fri Nov 4 17:50:29 2016 +0530
Committer: Venkata Ramana G <ra...@huawei.com>
Committed: Fri Nov 4 17:50:29 2016 +0530

----------------------------------------------------------------------
 .../org/apache/spark/sql/CarbonDictionaryDecoder.scala   | 11 +++++++++++
 1 file changed, 11 insertions(+)
----------------------------------------------------------------------