You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@kylin.apache.org by ma...@apache.org on 2016/09/20 10:46:14 UTC

[1/2] kylin git commit: KYLIN-2030 query on lookup table with no derived

Repository: kylin
Updated Branches:
  refs/heads/KYLIN-2030 [created] 9b99a0cc8


KYLIN-2030 query on lookup table with no derived


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

Branch: refs/heads/KYLIN-2030
Commit: 9b99a0cc8b0631c023f820bc062ecd00b1d9869f
Parents: d6b9650
Author: Hongbin Ma <ma...@apache.org>
Authored: Tue Sep 20 18:45:36 2016 +0800
Committer: Hongbin Ma <ma...@apache.org>
Committed: Tue Sep 20 18:45:41 2016 +0800

----------------------------------------------------------------------
 .../kylin/cube/CubeCapabilityChecker.java       | 24 +++++++++++++++++---
 1 file changed, 21 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kylin/blob/9b99a0cc/core-cube/src/main/java/org/apache/kylin/cube/CubeCapabilityChecker.java
----------------------------------------------------------------------
diff --git a/core-cube/src/main/java/org/apache/kylin/cube/CubeCapabilityChecker.java b/core-cube/src/main/java/org/apache/kylin/cube/CubeCapabilityChecker.java
index caef529..1eada16 100644
--- a/core-cube/src/main/java/org/apache/kylin/cube/CubeCapabilityChecker.java
+++ b/core-cube/src/main/java/org/apache/kylin/cube/CubeCapabilityChecker.java
@@ -25,14 +25,18 @@ import java.util.List;
 import java.util.Set;
 
 import org.apache.commons.lang.StringUtils;
+import org.apache.kylin.common.KylinConfig;
 import org.apache.kylin.cube.model.CubeDesc;
 import org.apache.kylin.measure.MeasureType;
 import org.apache.kylin.measure.basic.BasicMeasureType;
+import org.apache.kylin.metadata.MetadataManager;
 import org.apache.kylin.metadata.filter.UDF.MassInTupleFilter;
+import org.apache.kylin.metadata.model.ColumnDesc;
 import org.apache.kylin.metadata.model.FunctionDesc;
 import org.apache.kylin.metadata.model.IStorageAware;
 import org.apache.kylin.metadata.model.MeasureDesc;
 import org.apache.kylin.metadata.model.ParameterDesc;
+import org.apache.kylin.metadata.model.TableDesc;
 import org.apache.kylin.metadata.model.TblColRef;
 import org.apache.kylin.metadata.realization.CapabilityResult;
 import org.apache.kylin.metadata.realization.SQLDigest;
@@ -73,7 +77,21 @@ public class CubeCapabilityChecker {
 
         // try dimension-as-measure
         if (!unmatchedAggregations.isEmpty()) {
-            tryDimensionAsMeasures(unmatchedAggregations, digest, cube, result);
+            if (cube.getDescriptor().getFactTable().equals(digest.factTable)) {
+                tryDimensionAsMeasures(unmatchedAggregations, digest, cube, result, cube.getDescriptor().listDimensionColumnsIncludingDerived());
+            } else {
+                //deal with query on lookup table, like https://issues.apache.org/jira/browse/KYLIN-2030
+                if (cube.getSegments().get(0).getSnapshots().containsKey(digest.factTable)) {
+                    TableDesc tableDesc = MetadataManager.getInstance(KylinConfig.getInstanceFromEnv()).getTableDesc(digest.factTable);
+                    Set<TblColRef> dimCols = Sets.newHashSet();
+                    for (ColumnDesc columnDesc : tableDesc.getColumns()) {
+                        dimCols.add(columnDesc.getRef());
+                    }
+                    tryDimensionAsMeasures(unmatchedAggregations, digest, cube, result, dimCols);
+                } else {
+                    logger.info("Skip tryDimensionAsMeasures because current cube {} does not touch lookup table {} at all", cube.getName(), digest.factTable);
+                }
+            }
         }
 
         if (!unmatchedDimensions.isEmpty()) {
@@ -129,7 +147,7 @@ public class CubeCapabilityChecker {
         return result;
     }
 
-    private static void tryDimensionAsMeasures(Collection<FunctionDesc> unmatchedAggregations, SQLDigest digest, CubeInstance cube, CapabilityResult result) {
+    private static void tryDimensionAsMeasures(Collection<FunctionDesc> unmatchedAggregations, SQLDigest digest, CubeInstance cube, CapabilityResult result, Set<TblColRef> dimCols) {
         CubeDesc cubeDesc = cube.getDescriptor();
         Collection<FunctionDesc> cubeFuncs = cubeDesc.listAllFunctions();
 
@@ -154,7 +172,7 @@ public class CubeCapabilityChecker {
                 continue;
             }
             List<TblColRef> neededCols = parameterDesc.getColRefs();
-            if (neededCols.size() > 0 && cubeDesc.listDimensionColumnsIncludingDerived().containsAll(neededCols) && FunctionDesc.BUILT_IN_AGGREGATIONS.contains(functionDesc.getExpression())) {
+            if (neededCols.size() > 0 && dimCols.containsAll(neededCols) && FunctionDesc.BUILT_IN_AGGREGATIONS.contains(functionDesc.getExpression())) {
                 result.influences.add(new CapabilityResult.DimensionAsMeasure(functionDesc));
                 it.remove();
                 continue;


[2/2] kylin git commit: KYLIN-2029 distinct count query on lookup table

Posted by ma...@apache.org.
KYLIN-2029 distinct count query on lookup table


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

Branch: refs/heads/KYLIN-2030
Commit: d6b9650246af1b2331b694a7becda5f804fe2449
Parents: 5a049a5
Author: Hongbin Ma <ma...@apache.org>
Authored: Tue Sep 20 18:19:19 2016 +0800
Committer: Hongbin Ma <ma...@apache.org>
Committed: Tue Sep 20 18:45:41 2016 +0800

----------------------------------------------------------------------
 .../test/resources/query/sql_lookup/query05.sql  | 19 +++++++++++++++++++
 .../kylin/query/relnode/OLAPAggregateRel.java    |  8 ++++++--
 2 files changed, 25 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kylin/blob/d6b96502/kylin-it/src/test/resources/query/sql_lookup/query05.sql
----------------------------------------------------------------------
diff --git a/kylin-it/src/test/resources/query/sql_lookup/query05.sql b/kylin-it/src/test/resources/query/sql_lookup/query05.sql
new file mode 100644
index 0000000..31fdf61
--- /dev/null
+++ b/kylin-it/src/test/resources/query/sql_lookup/query05.sql
@@ -0,0 +1,19 @@
+--
+-- 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 count(distinct META_CATEG_NAME) as CNT ,max(META_CATEG_NAME) as y from test_category_groupings

http://git-wip-us.apache.org/repos/asf/kylin/blob/d6b96502/query/src/main/java/org/apache/kylin/query/relnode/OLAPAggregateRel.java
----------------------------------------------------------------------
diff --git a/query/src/main/java/org/apache/kylin/query/relnode/OLAPAggregateRel.java b/query/src/main/java/org/apache/kylin/query/relnode/OLAPAggregateRel.java
index f55c86f..97efb27 100644
--- a/query/src/main/java/org/apache/kylin/query/relnode/OLAPAggregateRel.java
+++ b/query/src/main/java/org/apache/kylin/query/relnode/OLAPAggregateRel.java
@@ -265,7 +265,7 @@ public class OLAPAggregateRel extends Aggregate implements OLAPRel {
         implementor.visitChild(this, getInput());
 
         // only rewrite the innermost aggregation
-        if (!this.afterAggregate && RewriteImplementor.needRewrite(this.context)) {
+        if (!this.afterAggregate) {
             // rewrite the aggCalls
             this.rewriteAggCalls = new ArrayList<AggregateCall>(aggCalls.size());
             for (int i = 0; i < this.aggCalls.size(); i++) {
@@ -354,11 +354,15 @@ public class OLAPAggregateRel extends Aggregate implements OLAPRel {
         }
     }
 
+    private boolean noPrecaculatedFieldsAvailable() {
+        return !this.context.hasPrecalculatedFields() || !RewriteImplementor.needRewrite(this.context);
+    }
+
     private AggregateCall rewriteAggregateCall(AggregateCall aggCall, FunctionDesc func) {
 
         //if it's not a cube, then the "needRewriteField func" should not resort to any rewrite fields, 
         // which do not exist at all
-        if (!this.context.hasPrecalculatedFields() && func.needRewriteField()) {
+        if (noPrecaculatedFieldsAvailable() && func.needRewriteField()) {
             logger.info(func + "skip rewriteAggregateCall because no pre-aggregated field available");
             return aggCall;
         }