carbondata-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From ravipes...@apache.org
Subject [1/2] incubator-carbondata git commit: Dictionary performance issue with multiple task in same executor
Date Thu, 02 Mar 2017 18:31:07 GMT
Repository: incubator-carbondata
Updated Branches:
  refs/heads/master 53397873b -> b93173fcd


Dictionary performance issue with multiple task in same executor


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

Branch: refs/heads/master
Commit: c79bd5255a3dac9ea7db698f23046d56e95d399f
Parents: 5339787
Author: kumarvishal <kumarvishal.1802@gmail.com>
Authored: Tue Feb 28 18:28:02 2017 +0800
Committer: kumarvishal <kumarvishal.1802@gmail.com>
Committed: Wed Mar 1 20:33:11 2017 +0800

----------------------------------------------------------------------
 .../dictionary/ForwardDictionaryCache.java      | 38 +++++++++++++++-----
 1 file changed, 30 insertions(+), 8 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/c79bd525/core/src/main/java/org/apache/carbondata/core/cache/dictionary/ForwardDictionaryCache.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/cache/dictionary/ForwardDictionaryCache.java
b/core/src/main/java/org/apache/carbondata/core/cache/dictionary/ForwardDictionaryCache.java
index c2d653c..e46f3f5 100644
--- a/core/src/main/java/org/apache/carbondata/core/cache/dictionary/ForwardDictionaryCache.java
+++ b/core/src/main/java/org/apache/carbondata/core/cache/dictionary/ForwardDictionaryCache.java
@@ -19,7 +19,9 @@ package org.apache.carbondata.core.cache.dictionary;
 
 import java.io.IOException;
 import java.util.ArrayList;
+import java.util.HashMap;
 import java.util.List;
+import java.util.Map;
 import java.util.concurrent.Callable;
 import java.util.concurrent.ExecutorService;
 import java.util.concurrent.Executors;
@@ -35,9 +37,8 @@ import org.apache.carbondata.core.cache.CarbonLRUCache;
  * This class implements methods to create dictionary cache which will hold
  * dictionary chunks for look up of surrogate keys and values
  */
-public class ForwardDictionaryCache<K extends DictionaryColumnUniqueIdentifier,
-                                    V extends Dictionary>
-    extends AbstractDictionaryCache<K, V> {
+public class ForwardDictionaryCache<K extends
+    DictionaryColumnUniqueIdentifier, V extends Dictionary> extends AbstractDictionaryCache<K,
V> {
 
   /**
    * Attribute for Carbon LOGGER
@@ -45,6 +46,9 @@ public class ForwardDictionaryCache<K extends DictionaryColumnUniqueIdentifier,
   private static final LogService LOGGER =
       LogServiceFactory.getLogService(ForwardDictionaryCache.class.getName());
 
+  private static final Map<DictionaryColumnUniqueIdentifier, Object> DICTIONARY_LOCK_OBJECT
=
+      new HashMap<>();
+
   /**
    * @param carbonStorePath
    * @param carbonLRUCache
@@ -77,8 +81,7 @@ public class ForwardDictionaryCache<K extends DictionaryColumnUniqueIdentifier,
    * @throws IOException in case memory is not sufficient to load dictionary into memory
    */
   @Override public List<Dictionary> getAll(
-      List<DictionaryColumnUniqueIdentifier> dictionaryColumnUniqueIdentifiers)
-      throws IOException {
+      List<DictionaryColumnUniqueIdentifier> dictionaryColumnUniqueIdentifiers) throws
IOException {
     boolean exceptionOccurredInDictionaryLoading = false;
     String exceptionMessage = "";
     List<Dictionary> forwardDictionaryObjectList =
@@ -89,7 +92,27 @@ public class ForwardDictionaryCache<K extends DictionaryColumnUniqueIdentifier,
     for (final DictionaryColumnUniqueIdentifier uniqueIdent : dictionaryColumnUniqueIdentifiers)
{
       taskSubmitList.add(executorService.submit(new Callable<Dictionary>() {
         @Override public Dictionary call() throws IOException {
-          Dictionary dictionary = getDictionary(uniqueIdent);
+          // in case of multiple task for same query same executor
+          // only one task should load the dictionary
+          // others will wait on monitor and get the loaded dictionary values
+          Object lockObject = DICTIONARY_LOCK_OBJECT.get(uniqueIdent);
+          // if lock object is null
+          if (null == lockObject) {
+            // Acquire the lock on map
+            synchronized (DICTIONARY_LOCK_OBJECT) {
+              // double checking the dictionary lock object
+              lockObject = DICTIONARY_LOCK_OBJECT.get(uniqueIdent);
+              // if still it is null add new lock object
+              if (null == lockObject) {
+                lockObject = new Object();
+                DICTIONARY_LOCK_OBJECT.put(uniqueIdent, lockObject);
+              }
+            }
+          }
+          Dictionary dictionary = null;
+          synchronized (lockObject) {
+            dictionary = getDictionary(uniqueIdent);
+          }
           return dictionary;
         }
       }));
@@ -161,8 +184,7 @@ public class ForwardDictionaryCache<K extends DictionaryColumnUniqueIdentifier,
    * @throws IOException in case memory is not sufficient to load dictionary into memory
    */
   private Dictionary getDictionary(
-      DictionaryColumnUniqueIdentifier dictionaryColumnUniqueIdentifier)
-      throws IOException {
+      DictionaryColumnUniqueIdentifier dictionaryColumnUniqueIdentifier) throws IOException
{
     Dictionary forwardDictionary = null;
     // dictionary is only for primitive data type
     assert (!dictionaryColumnUniqueIdentifier.getDataType().isComplexType());


Mime
View raw message