Repository: kylin
Updated Branches:
  refs/heads/1.x-staging 2f5d89a8e -> eea691651


KYLIN-1297 correct LoadingCache settings


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

Branch: refs/heads/1.x-staging
Commit: eea6916518a59ce137ac6d26bd539e39bb871a6c
Parents: 2f5d89a
Author: honma <ho...@ebay.com>
Authored: Thu Jan 21 10:28:54 2016 +0800
Committer: honma <ho...@ebay.com>
Committed: Thu Jan 21 10:29:05 2016 +0800

----------------------------------------------------------------------
 .../org/apache/kylin/common/KylinConfig.java    |  5 ++++
 .../apache/kylin/dict/DictionaryManager.java    | 31 +++++++++++---------
 .../storage/hbase/CubeSegmentTupleIterator.java |  1 -
 3 files changed, 22 insertions(+), 15 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kylin/blob/eea69165/common/src/main/java/org/apache/kylin/common/KylinConfig.java
----------------------------------------------------------------------
diff --git a/common/src/main/java/org/apache/kylin/common/KylinConfig.java 
b/common/src/main/java/org/apache/kylin/common/KylinConfig.java
index e0f774a..790e7a5 100644
--- a/common/src/main/java/org/apache/kylin/common/KylinConfig.java
+++ b/common/src/main/java/org/apache/kylin/common/KylinConfig.java
@@ -466,6 +466,11 @@ public class KylinConfig {
         return this.getOptional("kylin.server.mode", "all");
     }
 
+    public int getCachedDictMaxEntrySize() {
+        return Integer.parseInt(getOptional("kylin.dict.cache.max.entry", 
"3000"));
+    }
+
+
     public int getDictionaryMaxCardinality() {
         return 
Integer.parseInt(getOptional("kylin.dictionary.max.cardinality", "5000000"));
     }

http://git-wip-us.apache.org/repos/asf/kylin/blob/eea69165/dictionary/src/main/java/org/apache/kylin/dict/DictionaryManager.java
----------------------------------------------------------------------
diff --git 
a/dictionary/src/main/java/org/apache/kylin/dict/DictionaryManager.java 
b/dictionary/src/main/java/org/apache/kylin/dict/DictionaryManager.java
index aa0bc5d..9294e03 100644
--- a/dictionary/src/main/java/org/apache/kylin/dict/DictionaryManager.java
+++ b/dictionary/src/main/java/org/apache/kylin/dict/DictionaryManager.java
@@ -28,9 +28,6 @@ import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.ExecutionException;
 import java.util.concurrent.TimeUnit;
 
-import com.google.common.cache.CacheBuilder;
-import com.google.common.cache.CacheLoader;
-import com.google.common.cache.LoadingCache;
 import org.apache.commons.compress.utils.IOUtils;
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FileSystem;
@@ -49,6 +46,10 @@ import org.apache.kylin.metadata.model.TblColRef;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import com.google.common.cache.CacheBuilder;
+import com.google.common.cache.CacheLoader;
+import com.google.common.cache.LoadingCache;
+
 public class DictionaryManager {
 
     private static final Logger logger = 
LoggerFactory.getLogger(DictionaryManager.class);
@@ -84,17 +85,19 @@ public class DictionaryManager {
 
     private DictionaryManager(KylinConfig config) {
         this.config = config;
-        this.dictCache = 
CacheBuilder.newBuilder().weakValues().expireAfterWrite(10, 
TimeUnit.MINUTES).build(new CacheLoader<String, DictionaryInfo>() {
-            @Override
-            public DictionaryInfo load(String key) throws Exception {
-                DictionaryInfo dictInfo = DictionaryManager.this.load(key, 
true);
-                if (dictInfo == null) {
-                    return NONE_INDICATOR;
-                } else {
-                    return dictInfo;
-                }
-            }
-        });
+        this.dictCache = 
CacheBuilder.newBuilder().maximumSize(KylinConfig.getInstanceFromEnv().getCachedDictMaxEntrySize())//
+                .expireAfterWrite(1, TimeUnit.DAYS).build(new 
CacheLoader<String, DictionaryInfo>() {
+
+                    @Override
+                    public DictionaryInfo load(String key) throws Exception {
+                        DictionaryInfo dictInfo = 
DictionaryManager.this.load(key, true);
+                        if (dictInfo == null) {
+                            return NONE_INDICATOR;
+                        } else {
+                            return dictInfo;
+                        }
+                    }
+                });
     }
 
     public Dictionary<?> getDictionary(String resourcePath) throws IOException 
{

http://git-wip-us.apache.org/repos/asf/kylin/blob/eea69165/storage/src/main/java/org/apache/kylin/storage/hbase/CubeSegmentTupleIterator.java
----------------------------------------------------------------------
diff --git 
a/storage/src/main/java/org/apache/kylin/storage/hbase/CubeSegmentTupleIterator.java
 
b/storage/src/main/java/org/apache/kylin/storage/hbase/CubeSegmentTupleIterator.java
index 9627efb..26c4f29 100644
--- 
a/storage/src/main/java/org/apache/kylin/storage/hbase/CubeSegmentTupleIterator.java
+++ 
b/storage/src/main/java/org/apache/kylin/storage/hbase/CubeSegmentTupleIterator.java
@@ -462,7 +462,6 @@ public class CubeSegmentTupleIterator implements 
ITupleIterator {
             HBaseColumnDesc hbaseColumn = rowValueDecoder.getHBaseColumn();
             String columnFamily = hbaseColumn.getColumnFamilyName();
             String qualifier = hbaseColumn.getQualifier();
-            // FIXME: avoidable bytes array creation, why not use 
res.getValueAsByteBuffer directly?
             byte[] valueBytes = res.getValue(Bytes.toBytes(columnFamily), 
Bytes.toBytes(qualifier));
             rowValueDecoder.decode(valueBytes);
             List<String> measureNames = rowValueDecoder.getNames();

Reply via email to