You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@kylin.apache.org by li...@apache.org on 2016/07/08 10:22:41 UTC

[1/3] kylin git commit: KYLIN 1792 Test case

Repository: kylin
Updated Branches:
  refs/heads/master 43b4e4ac0 -> d48b9aac1


KYLIN 1792 Test case

Signed-off-by: Li Yang <li...@apache.org>


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

Branch: refs/heads/master
Commit: 7dc8b06c0c90e2df64a10d27487f1ba8422a2908
Parents: 43b4e4a
Author: Cheng Wang <ch...@kyligence.io>
Authored: Wed Jul 6 09:49:49 2016 +0800
Committer: Li Yang <li...@apache.org>
Committed: Fri Jul 8 18:09:10 2016 +0800

----------------------------------------------------------------------
 .../resources/query/sql_subquery/query11.sql    | 25 ++++++++++++++++++++
 1 file changed, 25 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kylin/blob/7dc8b06c/kylin-it/src/test/resources/query/sql_subquery/query11.sql
----------------------------------------------------------------------
diff --git a/kylin-it/src/test/resources/query/sql_subquery/query11.sql b/kylin-it/src/test/resources/query/sql_subquery/query11.sql
new file mode 100644
index 0000000..ce1f978
--- /dev/null
+++ b/kylin-it/src/test/resources/query/sql_subquery/query11.sql
@@ -0,0 +1,25 @@
+--
+-- Licensed to the Apache Software Foundation (ASF) under one
+-- or more contributor license agreements.  See the NOTICE file
+-- distributed with this work for additional information
+-- regarding copyright ownership.  The ASF licenses this file
+-- to you under the Apache License, Version 2.0 (the
+-- "License"); you may not use this file except in compliance
+-- with the License.  You may obtain a copy of the License at
+--
+--     http://www.apache.org/licenses/LICENSE-2.0
+--
+-- Unless required by applicable law or agreed to in writing, software
+-- distributed under the License is distributed on an "AS IS" BASIS,
+-- WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+-- See the License for the specific language governing permissions and
+-- limitations under the License.
+--
+
+select test_category_groupings.meta_categ_name,
+sum(price) as sum_price
+from test_kylin_fact as test_kylin_fact
+inner join  (select leaf_categ_id,site_id,meta_categ_name from test_category_groupings ) as test_category_groupings
+on test_kylin_fact.leaf_categ_id = test_category_groupings.leaf_categ_id and
+test_kylin_fact.lstg_site_id = test_category_groupings.site_id
+group by test_category_groupings.meta_categ_name


[3/3] kylin git commit: minor, add toString() on route Candidate

Posted by li...@apache.org.
minor, add toString() on route Candidate


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

Branch: refs/heads/master
Commit: d48b9aac1ecc03ab8945cc75384c0d295cc2c47d
Parents: 6b45c94
Author: Li Yang <li...@apache.org>
Authored: Fri Jul 8 18:22:32 2016 +0800
Committer: Li Yang <li...@apache.org>
Committed: Fri Jul 8 18:22:32 2016 +0800

----------------------------------------------------------------------
 .../src/main/java/org/apache/kylin/query/routing/Candidate.java  | 4 ++++
 1 file changed, 4 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kylin/blob/d48b9aac/query/src/main/java/org/apache/kylin/query/routing/Candidate.java
----------------------------------------------------------------------
diff --git a/query/src/main/java/org/apache/kylin/query/routing/Candidate.java b/query/src/main/java/org/apache/kylin/query/routing/Candidate.java
index c7c99ab..ab7884a 100644
--- a/query/src/main/java/org/apache/kylin/query/routing/Candidate.java
+++ b/query/src/main/java/org/apache/kylin/query/routing/Candidate.java
@@ -95,4 +95,8 @@ public class Candidate implements Comparable<Candidate> {
         return 0;
     }
 
+    @Override
+    public String toString() {
+        return realization.toString();
+    }
 }


[2/3] kylin git commit: KYLIN-1843: support LRU cache on SnapshotManager

Posted by li...@apache.org.
KYLIN-1843: support LRU cache on SnapshotManager

Signed-off-by: Li Yang <li...@apache.org>


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

Branch: refs/heads/master
Commit: 6b45c945af9948c17a971d530c0153b3564f1985
Parents: 7dc8b06
Author: Yiming Liu <li...@gmail.com>
Authored: Thu Jul 7 12:30:25 2016 +0800
Committer: Li Yang <li...@apache.org>
Committed: Fri Jul 8 18:13:55 2016 +0800

----------------------------------------------------------------------
 .../apache/kylin/common/KylinConfigBase.java    |  4 ++
 .../kylin/dict/lookup/SnapshotManager.java      | 42 +++++++++++++++-----
 2 files changed, 37 insertions(+), 9 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kylin/blob/6b45c945/core-common/src/main/java/org/apache/kylin/common/KylinConfigBase.java
----------------------------------------------------------------------
diff --git a/core-common/src/main/java/org/apache/kylin/common/KylinConfigBase.java b/core-common/src/main/java/org/apache/kylin/common/KylinConfigBase.java
index bfbaaa4..6b528f7 100644
--- a/core-common/src/main/java/org/apache/kylin/common/KylinConfigBase.java
+++ b/core-common/src/main/java/org/apache/kylin/common/KylinConfigBase.java
@@ -502,6 +502,10 @@ abstract public class KylinConfigBase implements Serializable {
         return Integer.parseInt(getOptional("kylin.dict.cache.max.entry", "3000"));
     }
 
+    public int getCachedSnapshotMaxEntrySize(){
+        return Integer.parseInt(getOptional("kylin.snapshot.cache.max.entry", "500"));
+    }
+
     public boolean getQueryRunLocalCoprocessor() {
         return Boolean.parseBoolean(getOptional("kylin.query.run.local.coprocessor", "false"));
     }

http://git-wip-us.apache.org/repos/asf/kylin/blob/6b45c945/core-dictionary/src/main/java/org/apache/kylin/dict/lookup/SnapshotManager.java
----------------------------------------------------------------------
diff --git a/core-dictionary/src/main/java/org/apache/kylin/dict/lookup/SnapshotManager.java b/core-dictionary/src/main/java/org/apache/kylin/dict/lookup/SnapshotManager.java
index 6e93ae4..c7b0d26 100644
--- a/core-dictionary/src/main/java/org/apache/kylin/dict/lookup/SnapshotManager.java
+++ b/core-dictionary/src/main/java/org/apache/kylin/dict/lookup/SnapshotManager.java
@@ -21,6 +21,8 @@ package org.apache.kylin.dict.lookup;
 import java.io.IOException;
 import java.util.NavigableSet;
 import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.TimeUnit;
 
 import org.apache.kylin.common.KylinConfig;
 import org.apache.kylin.common.persistence.ResourceStore;
@@ -31,6 +33,12 @@ import org.apache.kylin.source.ReadableTable.TableSignature;
 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;
+import com.google.common.cache.RemovalListener;
+import com.google.common.cache.RemovalNotification;
+
 /**
  * @author yangli9
  */
@@ -61,33 +69,49 @@ public class SnapshotManager {
     // ============================================================================
 
     private KylinConfig config;
-    private ConcurrentHashMap<String, SnapshotTable> snapshotCache; // resource
+    private LoadingCache<String, SnapshotTable> snapshotCache; // resource
 
     // path ==>
     // SnapshotTable
 
     private SnapshotManager(KylinConfig config) {
         this.config = config;
-        snapshotCache = new ConcurrentHashMap<String, SnapshotTable>();
+        this.snapshotCache = CacheBuilder.newBuilder().removalListener(new RemovalListener<String, SnapshotTable>() {
+            @Override
+            public void onRemoval(RemovalNotification<String, SnapshotTable> notification) {
+                SnapshotManager.logger.info("Snapshot with resource path " + notification.getKey() + " is removed due to " + notification.getCause());
+            }
+        }).maximumSize(config.getCachedSnapshotMaxEntrySize())//
+                .expireAfterWrite(1, TimeUnit.DAYS).build(new CacheLoader<String, SnapshotTable>() {
+                    @Override
+                    public SnapshotTable load(String key) throws Exception {
+                        SnapshotTable snapshotTable = SnapshotManager.this.load(key, true);
+                        return snapshotTable;
+                    }
+                });
     }
 
     public void wipeoutCache() {
-        snapshotCache.clear();
+        snapshotCache.invalidateAll();
     }
 
     public SnapshotTable getSnapshotTable(String resourcePath) throws IOException {
-        SnapshotTable r = snapshotCache.get(resourcePath);
-        if (r == null) {
-            r = load(resourcePath, true);
-            snapshotCache.put(resourcePath, r);
+        try {
+            SnapshotTable r = snapshotCache.get(resourcePath);
+            if (r == null) {
+                r = load(resourcePath, true);
+                snapshotCache.put(resourcePath, r);
+            }
+            return r;
+        } catch (ExecutionException e) {
+            throw new RuntimeException(e.getCause());
         }
-        return r;
     }
 
     public void removeSnapshot(String resourcePath) throws IOException {
         ResourceStore store = MetadataManager.getInstance(this.config).getStore();
         store.deleteResource(resourcePath);
-        snapshotCache.remove(resourcePath);
+        snapshotCache.invalidate(resourcePath);
     }
 
     public SnapshotTable buildSnapshot(ReadableTable table, TableDesc tableDesc) throws IOException {