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 2017/04/08 11:56:08 UTC

[26/35] kylin git commit: KYLIN-2341 report error on unsupported aggr func

KYLIN-2341 report error on unsupported aggr func


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

Branch: refs/heads/master-hbase0.98
Commit: 7af5529ef566434b34131a6b77c0f32e738bb318
Parents: 2057831
Author: Yang Li <li...@apache.org>
Authored: Mon Apr 3 22:35:21 2017 +0800
Committer: Yang Li <li...@apache.org>
Committed: Mon Apr 3 22:35:21 2017 +0800

----------------------------------------------------------------------
 .../kylin/metadata/model/FunctionDesc.java      |  8 +--
 .../kylin/metadata/model/ParameterDesc.java     | 57 ++++++++++----------
 .../apache/kylin/metadata/model/TblColRef.java  | 10 ++++
 .../resources/query/sql_casewhen/query04.sql    | 30 -----------
 .../kylin/query/relnode/OLAPAggregateRel.java   |  4 +-
 .../kylin/query/relnode/OLAPProjectRel.java     |  2 +-
 6 files changed, 44 insertions(+), 67 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kylin/blob/7af5529e/core-metadata/src/main/java/org/apache/kylin/metadata/model/FunctionDesc.java
----------------------------------------------------------------------
diff --git a/core-metadata/src/main/java/org/apache/kylin/metadata/model/FunctionDesc.java b/core-metadata/src/main/java/org/apache/kylin/metadata/model/FunctionDesc.java
index 61c5fac..e969be2 100644
--- a/core-metadata/src/main/java/org/apache/kylin/metadata/model/FunctionDesc.java
+++ b/core-metadata/src/main/java/org/apache/kylin/metadata/model/FunctionDesc.java
@@ -19,7 +19,6 @@
 package org.apache.kylin.metadata.model;
 
 import java.io.Serializable;
-import java.util.ArrayList;
 import java.util.Collections;
 import java.util.LinkedHashMap;
 import java.util.List;
@@ -41,6 +40,7 @@ import com.google.common.collect.Sets;
 
 /**
  */
+@SuppressWarnings("serial")
 @JsonAutoDetect(fieldVisibility = Visibility.NONE, getterVisibility = Visibility.NONE, isGetterVisibility = Visibility.NONE, setterVisibility = Visibility.NONE)
 public class FunctionDesc implements Serializable {
 
@@ -90,17 +90,13 @@ public class FunctionDesc implements Serializable {
         expression = expression.toUpperCase();
         returnDataType = DataType.getType(returnType);
 
-        ArrayList<TblColRef> colRefs = Lists.newArrayList();
         for (ParameterDesc p = parameter; p != null; p = p.getNextParameter()) {
             if (p.isColumnType()) {
                 TblColRef colRef = model.findColumn(p.getValue());
                 p.setValue(colRef.getIdentity());
-                colRefs.add(colRef);
+                p.setColRef(colRef);
             }
         }
-
-        if (parameter != null)
-            parameter.setColRefs(colRefs);
     }
 
     private void reInitMeasureType() {

http://git-wip-us.apache.org/repos/asf/kylin/blob/7af5529e/core-metadata/src/main/java/org/apache/kylin/metadata/model/ParameterDesc.java
----------------------------------------------------------------------
diff --git a/core-metadata/src/main/java/org/apache/kylin/metadata/model/ParameterDesc.java b/core-metadata/src/main/java/org/apache/kylin/metadata/model/ParameterDesc.java
index 272c4ee..0cf31b1 100644
--- a/core-metadata/src/main/java/org/apache/kylin/metadata/model/ParameterDesc.java
+++ b/core-metadata/src/main/java/org/apache/kylin/metadata/model/ParameterDesc.java
@@ -20,6 +20,7 @@ package org.apache.kylin.metadata.model;
 
 import java.io.Serializable;
 import java.io.UnsupportedEncodingException;
+import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.List;
 import java.util.Set;
@@ -28,8 +29,6 @@ import com.fasterxml.jackson.annotation.JsonAutoDetect;
 import com.fasterxml.jackson.annotation.JsonAutoDetect.Visibility;
 import com.fasterxml.jackson.annotation.JsonInclude;
 import com.fasterxml.jackson.annotation.JsonProperty;
-import com.google.common.collect.ImmutableList;
-import com.google.common.collect.Iterables;
 import com.google.common.collect.Sets;
 
 /**
@@ -49,7 +48,7 @@ public class ParameterDesc implements Serializable {
             TblColRef col = (TblColRef) obj;
             r.type = FunctionDesc.PARAMETER_TYPE_COLUMN;
             r.value = col.getIdentity();
-            r.colRefs = ImmutableList.of(col);
+            r.colRef = col;
         } else {
             r.type = FunctionDesc.PARAMETER_TYPE_CONSTANT;
             r.value = (String) obj;
@@ -57,12 +56,6 @@ public class ParameterDesc implements Serializable {
 
         if (objs.length >= 2) {
             r.nextParameter = newInstance(Arrays.copyOfRange(objs, 1, objs.length));
-            if (r.nextParameter.colRefs.size() > 0) {
-                if (r.colRefs.isEmpty())
-                    r.colRefs = r.nextParameter.colRefs;
-                else
-                    r.colRefs = ImmutableList.copyOf(Iterables.concat(r.colRefs, r.nextParameter.colRefs));
-            }
         }
         return r;
     }
@@ -76,7 +69,8 @@ public class ParameterDesc implements Serializable {
     @JsonInclude(JsonInclude.Include.NON_NULL)
     private ParameterDesc nextParameter;
 
-    private List<TblColRef> colRefs = ImmutableList.of();
+    private TblColRef colRef = null;
+    private List<TblColRef> allColRefsIncludingNexts = null;
     private Set<PlainParameter> plainParameters = null;
 
     // Lazy evaluation
@@ -102,13 +96,28 @@ public class ParameterDesc implements Serializable {
     void setValue(String value) {
         this.value = value;
     }
+    
+    public TblColRef getColRef() {
+        return colRef;
+    }
 
-    public List<TblColRef> getColRefs() {
-        return colRefs;
+    void setColRef(TblColRef colRef) {
+        this.colRef = colRef;
     }
 
-    void setColRefs(List<TblColRef> colRefs) {
-        this.colRefs = colRefs;
+    public List<TblColRef> getColRefs() {
+        if (allColRefsIncludingNexts == null) {
+            List<TblColRef> all = new ArrayList<>(2);
+            ParameterDesc p = this;
+            while (p != null) {
+                if (p.isColumnType())
+                    all.add(p.getColRef());
+                
+                p = p.nextParameter;
+            }
+            allColRefsIncludingNexts = all;
+        }
+        return allColRefsIncludingNexts;
     }
 
     public ParameterDesc getNextParameter() {
@@ -132,17 +141,12 @@ public class ParameterDesc implements Serializable {
             return false;
 
         ParameterDesc p = this, q = that;
-        int refi = 0, refj = 0;
         for (; p != null && q != null; p = p.nextParameter, q = q.nextParameter) {
             if (p.isColumnType()) {
                 if (q.isColumnType() == false)
                     return false;
-                if (refi >= this.colRefs.size() || refj >= that.colRefs.size())
-                    return false;
-                if (this.colRefs.get(refi).equals(that.colRefs.get(refj)) == false)
+                if (this.getColRef().equals(that.getColRef()) == false)
                     return false;
-                refi++;
-                refj++;
             } else {
                 if (q.isColumnType() == true)
                     return false;
@@ -171,13 +175,14 @@ public class ParameterDesc implements Serializable {
     @Override
     public int hashCode() {
         int result = type != null ? type.hashCode() : 0;
-        result = 31 * result + (colRefs != null ? colRefs.hashCode() : 0);
+        result = 31 * result + (colRef != null ? colRef.hashCode() : 0);
         return result;
     }
 
     @Override
     public String toString() {
-        return "ParameterDesc [type=" + type + ", value=" + value + ", nextParam=" + nextParameter + "]";
+        String thisStr = isColumnType() ? colRef.toString() : value;
+        return nextParameter == null ? thisStr : thisStr + "," + nextParameter.toString();
     }
 
     /**
@@ -201,11 +206,9 @@ public class ParameterDesc implements Serializable {
         static Set<PlainParameter> createFromParameterDesc(ParameterDesc parameterDesc) {
             Set<PlainParameter> result = Sets.newHashSet();
             ParameterDesc local = parameterDesc;
-            List<TblColRef> totalColRef = parameterDesc.colRefs;
-            Integer colIndex = 0;
             while (local != null) {
                 if (local.isColumnType()) {
-                    result.add(createSingleColumnParameter(local, totalColRef.get(colIndex++)));
+                    result.add(createSingleColumnParameter(local));
                 } else {
                     result.add(createSingleValueParameter(local));
                 }
@@ -221,11 +224,11 @@ public class ParameterDesc implements Serializable {
             return single;
         }
 
-        static PlainParameter createSingleColumnParameter(ParameterDesc parameterDesc, TblColRef colRef) {
+        static PlainParameter createSingleColumnParameter(ParameterDesc parameterDesc) {
             PlainParameter single = new PlainParameter();
             single.type = parameterDesc.type;
             single.value = parameterDesc.value;
-            single.colRef = colRef;
+            single.colRef = parameterDesc.colRef;
             return single;
         }
 

http://git-wip-us.apache.org/repos/asf/kylin/blob/7af5529e/core-metadata/src/main/java/org/apache/kylin/metadata/model/TblColRef.java
----------------------------------------------------------------------
diff --git a/core-metadata/src/main/java/org/apache/kylin/metadata/model/TblColRef.java b/core-metadata/src/main/java/org/apache/kylin/metadata/model/TblColRef.java
index 88abb9d..4661bc9 100644
--- a/core-metadata/src/main/java/org/apache/kylin/metadata/model/TblColRef.java
+++ b/core-metadata/src/main/java/org/apache/kylin/metadata/model/TblColRef.java
@@ -54,12 +54,18 @@ public class TblColRef implements Serializable {
 
     // used by projection rewrite, see OLAPProjectRel
     public static TblColRef newInnerColumn(String columnName, InnerDataTypeEnum dataType) {
+        return newInnerColumn(columnName, dataType, null);
+    }
+    
+    // used by projection rewrite, see OLAPProjectRel
+    public static TblColRef newInnerColumn(String columnName, InnerDataTypeEnum dataType, String parserDescription) {
         ColumnDesc column = new ColumnDesc();
         column.setName(columnName);
         TableDesc table = new TableDesc();
         column.setTable(table);
         TblColRef colRef = new TblColRef(column);
         colRef.markInnerColumn(dataType);
+        colRef.parserDescription = parserDescription;
         return colRef;
     }
     
@@ -101,6 +107,7 @@ public class TblColRef implements Serializable {
     private TableRef table;
     private ColumnDesc column;
     private String identity;
+    private String parserDescription;
 
     TblColRef(ColumnDesc column) {
         this.column = column;
@@ -198,6 +205,9 @@ public class TblColRef implements Serializable {
 
     @Override
     public String toString() {
+        if (isInnerColumn() && parserDescription != null)
+            return parserDescription;
+        
         String alias = table == null ? "UNKNOWN_MODEL" : table.getAlias();
         String tableName = column.getTable() == null ? "NULL" : column.getTable().getName();
         String tableIdentity = column.getTable() == null ? "NULL" : column.getTable().getIdentity();

http://git-wip-us.apache.org/repos/asf/kylin/blob/7af5529e/kylin-it/src/test/resources/query/sql_casewhen/query04.sql
----------------------------------------------------------------------
diff --git a/kylin-it/src/test/resources/query/sql_casewhen/query04.sql b/kylin-it/src/test/resources/query/sql_casewhen/query04.sql
deleted file mode 100644
index 646da0a..0000000
--- a/kylin-it/src/test/resources/query/sql_casewhen/query04.sql
+++ /dev/null
@@ -1,30 +0,0 @@
---
--- 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
-    SUM(CASE
-        WHEN lstg_format_name LIKE 'Other%' THEN price
-        ELSE 0
-    END) AS gmv
-FROM
-    test_kylin_fact
-        INNER JOIN
-    edw.test_cal_dt AS test_cal_dt ON test_kylin_fact.cal_dt = test_cal_dt.cal_dt
-        INNER JOIN
-    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
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/kylin/blob/7af5529e/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 c7f66a0..b669e3b 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
@@ -262,9 +262,7 @@ public class OLAPAggregateRel extends Aggregate implements OLAPRel {
                 }
                 for (Integer index : aggCall.getArgList().subList(0, columnsCount)) {
                     TblColRef column = inputColumnRowType.getColumnByIndex(index);
-                    if (!column.isInnerColumn()) {
-                        columns.add(column);
-                    }
+                    columns.add(column);
                 }
                 if (!columns.isEmpty()) {
                     parameter = ParameterDesc.newInstance(columns.toArray(new TblColRef[columns.size()]));

http://git-wip-us.apache.org/repos/asf/kylin/blob/7af5529e/query/src/main/java/org/apache/kylin/query/relnode/OLAPProjectRel.java
----------------------------------------------------------------------
diff --git a/query/src/main/java/org/apache/kylin/query/relnode/OLAPProjectRel.java b/query/src/main/java/org/apache/kylin/query/relnode/OLAPProjectRel.java
index aa03e3e..1d0de40 100644
--- a/query/src/main/java/org/apache/kylin/query/relnode/OLAPProjectRel.java
+++ b/query/src/main/java/org/apache/kylin/query/relnode/OLAPProjectRel.java
@@ -204,7 +204,7 @@ public class OLAPProjectRel extends Project implements OLAPRel {
         for (RexNode operand : call.getOperands()) {
             translateRexNode(operand, inputColumnRowType, fieldName, sourceCollector);
         }
-        return TblColRef.newInnerColumn(fieldName, InnerDataTypeEnum.LITERAL);
+        return TblColRef.newInnerColumn(fieldName, InnerDataTypeEnum.LITERAL, call.toString());
     }
 
     @Override