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/18 03:24:11 UTC

[2/2] kylin git commit: KYLIN-2021 Cognos issues

KYLIN-2021 Cognos issues


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

Branch: refs/heads/master
Commit: 7955b093de5bc04547ae2ca9ed12a78670c38e81
Parents: aa256fd
Author: Hongbin Ma <ma...@apache.org>
Authored: Fri Sep 16 17:52:53 2016 +0800
Committer: Hongbin Ma <ma...@apache.org>
Committed: Sun Sep 18 11:23:26 2016 +0800

----------------------------------------------------------------------
 .../kylin/metadata/filter/TupleFilter.java      |  2 +-
 .../metadata/filter/TupleFilterSerializer.java  |  3 +
 .../metadata/filter/UnsupportedTupleFilter.java | 59 ++++++++++++++++++++
 .../storage/gtrecord/CubeSegmentScanner.java    |  1 +
 .../apache/kylin/query/ITKylinQueryTest.java    |  6 ++
 .../query/sql_h2_uncapable/query01.sql          | 22 ++++++++
 .../query/sql_h2_uncapable/query02.sql          | 22 ++++++++
 .../query/sql_h2_uncapable/query03.sql          |  1 +
 .../query/sql_h2_uncapable/query04.sql          |  1 +
 .../kylin/query/relnode/OLAPFilterRel.java      |  4 +-
 .../org/apache/kylin/rest/util/QueryUtil.java   | 27 +++++++++
 .../apache/kylin/rest/util/QueryUtilTest.java   | 51 +++++++++++++++++
 12 files changed, 197 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kylin/blob/7955b093/core-metadata/src/main/java/org/apache/kylin/metadata/filter/TupleFilter.java
----------------------------------------------------------------------
diff --git a/core-metadata/src/main/java/org/apache/kylin/metadata/filter/TupleFilter.java b/core-metadata/src/main/java/org/apache/kylin/metadata/filter/TupleFilter.java
index 900cc35..285172c 100644
--- a/core-metadata/src/main/java/org/apache/kylin/metadata/filter/TupleFilter.java
+++ b/core-metadata/src/main/java/org/apache/kylin/metadata/filter/TupleFilter.java
@@ -38,7 +38,7 @@ import com.google.common.collect.Maps;
 public abstract class TupleFilter {
 
     public enum FilterOperatorEnum {
-        EQ(1), NEQ(2), GT(3), LT(4), GTE(5), LTE(6), ISNULL(7), ISNOTNULL(8), IN(9), NOTIN(10), AND(20), OR(21), NOT(22), COLUMN(30), CONSTANT(31), DYNAMIC(32), EXTRACT(33), CASE(34), FUNCTION(35), MASSIN(36), EVAL_FUNC(37);
+        EQ(1), NEQ(2), GT(3), LT(4), GTE(5), LTE(6), ISNULL(7), ISNOTNULL(8), IN(9), NOTIN(10), AND(20), OR(21), NOT(22), COLUMN(30), CONSTANT(31), DYNAMIC(32), EXTRACT(33), CASE(34), FUNCTION(35), MASSIN(36), EVAL_FUNC(37), UNSUPPORTED(38);
 
         private final int value;
 

http://git-wip-us.apache.org/repos/asf/kylin/blob/7955b093/core-metadata/src/main/java/org/apache/kylin/metadata/filter/TupleFilterSerializer.java
----------------------------------------------------------------------
diff --git a/core-metadata/src/main/java/org/apache/kylin/metadata/filter/TupleFilterSerializer.java b/core-metadata/src/main/java/org/apache/kylin/metadata/filter/TupleFilterSerializer.java
index 04984f2..a051ea9 100644
--- a/core-metadata/src/main/java/org/apache/kylin/metadata/filter/TupleFilterSerializer.java
+++ b/core-metadata/src/main/java/org/apache/kylin/metadata/filter/TupleFilterSerializer.java
@@ -188,6 +188,9 @@ public class TupleFilterSerializer {
         case FUNCTION:
             filter = new BuiltInFunctionTupleFilter(null);
             break;
+        case UNSUPPORTED:
+            filter = new UnsupportedTupleFilter(op);
+            break;
         case EVAL_FUNC:
             filter = new EvaluatableFunctionTupleFilter(null);
             break;

http://git-wip-us.apache.org/repos/asf/kylin/blob/7955b093/core-metadata/src/main/java/org/apache/kylin/metadata/filter/UnsupportedTupleFilter.java
----------------------------------------------------------------------
diff --git a/core-metadata/src/main/java/org/apache/kylin/metadata/filter/UnsupportedTupleFilter.java b/core-metadata/src/main/java/org/apache/kylin/metadata/filter/UnsupportedTupleFilter.java
new file mode 100644
index 0000000..85605d4
--- /dev/null
+++ b/core-metadata/src/main/java/org/apache/kylin/metadata/filter/UnsupportedTupleFilter.java
@@ -0,0 +1,59 @@
+/*
+ * 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.
+ */
+
+package org.apache.kylin.metadata.filter;
+
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.Collection;
+
+import org.apache.kylin.metadata.tuple.IEvaluatableTuple;
+
+public class UnsupportedTupleFilter extends TupleFilter {
+
+    //    protected UnsupportedTupleFilter(List<TupleFilter> filters, FilterOperatorEnum op) {
+    //        super(filters, op);
+    //    }
+
+    public UnsupportedTupleFilter(FilterOperatorEnum op) {
+        super(new ArrayList<TupleFilter>(), op);
+    }
+
+    @Override
+    public boolean isEvaluable() {
+        return false;
+    }
+
+    @Override
+    public boolean evaluate(IEvaluatableTuple tuple, IFilterCodeSystem<?> cs) {
+        throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public Collection<?> getValues() {
+        throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public void serialize(IFilterCodeSystem<?> cs, ByteBuffer buffer) {
+    }
+
+    @Override
+    public void deserialize(IFilterCodeSystem<?> cs, ByteBuffer buffer) {
+    }
+}

http://git-wip-us.apache.org/repos/asf/kylin/blob/7955b093/core-storage/src/main/java/org/apache/kylin/storage/gtrecord/CubeSegmentScanner.java
----------------------------------------------------------------------
diff --git a/core-storage/src/main/java/org/apache/kylin/storage/gtrecord/CubeSegmentScanner.java b/core-storage/src/main/java/org/apache/kylin/storage/gtrecord/CubeSegmentScanner.java
index f32831a..926ba48 100644
--- a/core-storage/src/main/java/org/apache/kylin/storage/gtrecord/CubeSegmentScanner.java
+++ b/core-storage/src/main/java/org/apache/kylin/storage/gtrecord/CubeSegmentScanner.java
@@ -61,6 +61,7 @@ public class CubeSegmentScanner implements IGTScanner {
         //is working on its own copy
         byte[] serialize = TupleFilterSerializer.serialize(originalfilter, StringCodeSystem.INSTANCE);
         TupleFilter filter = TupleFilterSerializer.deserialize(serialize, StringCodeSystem.INSTANCE);
+        
         // translate FunctionTupleFilter to IN clause
         ITupleFilterTransformer translator = new BuiltInFunctionTransformer(cubeSeg.getDimensionEncodingMap());
         filter = translator.transform(filter);

http://git-wip-us.apache.org/repos/asf/kylin/blob/7955b093/kylin-it/src/test/java/org/apache/kylin/query/ITKylinQueryTest.java
----------------------------------------------------------------------
diff --git a/kylin-it/src/test/java/org/apache/kylin/query/ITKylinQueryTest.java b/kylin-it/src/test/java/org/apache/kylin/query/ITKylinQueryTest.java
index 2417d68..59a3a04 100644
--- a/kylin-it/src/test/java/org/apache/kylin/query/ITKylinQueryTest.java
+++ b/kylin-it/src/test/java/org/apache/kylin/query/ITKylinQueryTest.java
@@ -181,6 +181,12 @@ public class ITKylinQueryTest extends KylinTestBase {
         batchExecuteQuery(getQueryFolderPrefix() + "src/test/resources/query/tableau_probing");
     }
 
+    //h2 cannot run these queries
+    @Test
+    public void testH2Uncapable() throws Exception {
+        batchExecuteQuery(getQueryFolderPrefix() + "src/test/resources/query/sql_h2_uncapable");
+    }
+
     @Test
     public void testCommonQuery() throws Exception {
         execAndCompQuery(getQueryFolderPrefix() + "src/test/resources/query/sql", null, true);

http://git-wip-us.apache.org/repos/asf/kylin/blob/7955b093/kylin-it/src/test/resources/query/sql_h2_uncapable/query01.sql
----------------------------------------------------------------------
diff --git a/kylin-it/src/test/resources/query/sql_h2_uncapable/query01.sql b/kylin-it/src/test/resources/query/sql_h2_uncapable/query01.sql
new file mode 100644
index 0000000..795c6f3
--- /dev/null
+++ b/kylin-it/src/test/resources/query/sql_h2_uncapable/query01.sql
@@ -0,0 +1,22 @@
+--
+-- 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 timestampadd(DAY,1,WEEK_BEG_DT) as x ,WEEK_BEG_DT
+ 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) 
+ GROUP BY TEST_CAL_DT.WEEK_BEG_DT

http://git-wip-us.apache.org/repos/asf/kylin/blob/7955b093/kylin-it/src/test/resources/query/sql_h2_uncapable/query02.sql
----------------------------------------------------------------------
diff --git a/kylin-it/src/test/resources/query/sql_h2_uncapable/query02.sql b/kylin-it/src/test/resources/query/sql_h2_uncapable/query02.sql
new file mode 100644
index 0000000..f3ad957
--- /dev/null
+++ b/kylin-it/src/test/resources/query/sql_h2_uncapable/query02.sql
@@ -0,0 +1,22 @@
+--
+-- 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 timestampadd(DAY,1,WEEK_BEG_DT) as x,sum(price)
+ 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) 
+ GROUP BY timestampadd(DAY,1,WEEK_BEG_DT)

http://git-wip-us.apache.org/repos/asf/kylin/blob/7955b093/kylin-it/src/test/resources/query/sql_h2_uncapable/query03.sql
----------------------------------------------------------------------
diff --git a/kylin-it/src/test/resources/query/sql_h2_uncapable/query03.sql b/kylin-it/src/test/resources/query/sql_h2_uncapable/query03.sql
new file mode 100644
index 0000000..ee89723
--- /dev/null
+++ b/kylin-it/src/test/resources/query/sql_h2_uncapable/query03.sql
@@ -0,0 +1 @@
+select "TEST_KYLIN_FACT"."CAL_DT" as "CAL_DT"  from "DEFAULT"."TEST_KYLIN_FACT" "TEST_KYLIN_FACT" where ("TEST_KYLIN_FACT"."CAL_DT" ) >= DATE'2013-01-07' + interval '3' day  group by "TEST_KYLIN_FACT"."CAL_DT" order by "TEST_KYLIN_FACT"."CAL_DT"
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/kylin/blob/7955b093/kylin-it/src/test/resources/query/sql_h2_uncapable/query04.sql
----------------------------------------------------------------------
diff --git a/kylin-it/src/test/resources/query/sql_h2_uncapable/query04.sql b/kylin-it/src/test/resources/query/sql_h2_uncapable/query04.sql
new file mode 100644
index 0000000..25fd2bf
--- /dev/null
+++ b/kylin-it/src/test/resources/query/sql_h2_uncapable/query04.sql
@@ -0,0 +1 @@
+select "TEST_KYLIN_FACT"."CAL_DT" as "CAL_DT"  from "DEFAULT"."TEST_KYLIN_FACT" "TEST_KYLIN_FACT" where ("TEST_KYLIN_FACT"."CAL_DT" + interval '3' day ) >= DATE'2013-01-07'  group by "TEST_KYLIN_FACT"."CAL_DT" order by "TEST_KYLIN_FACT"."CAL_DT"
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/kylin/blob/7955b093/query/src/main/java/org/apache/kylin/query/relnode/OLAPFilterRel.java
----------------------------------------------------------------------
diff --git a/query/src/main/java/org/apache/kylin/query/relnode/OLAPFilterRel.java b/query/src/main/java/org/apache/kylin/query/relnode/OLAPFilterRel.java
index 4cb2859..b34cc6f 100644
--- a/query/src/main/java/org/apache/kylin/query/relnode/OLAPFilterRel.java
+++ b/query/src/main/java/org/apache/kylin/query/relnode/OLAPFilterRel.java
@@ -61,6 +61,7 @@ import org.apache.kylin.metadata.filter.DynamicTupleFilter;
 import org.apache.kylin.metadata.filter.ExtractTupleFilter;
 import org.apache.kylin.metadata.filter.LogicalTupleFilter;
 import org.apache.kylin.metadata.filter.TupleFilter;
+import org.apache.kylin.metadata.filter.UnsupportedTupleFilter;
 import org.apache.kylin.metadata.filter.TupleFilter.FilterOperatorEnum;
 import org.apache.kylin.metadata.filter.function.Functions;
 import org.apache.kylin.metadata.model.TblColRef;
@@ -141,7 +142,8 @@ public class OLAPFilterRel extends Filter implements OLAPRel {
                 filter = Functions.getFunctionTupleFilter(op.getName());
                 break;
             default:
-                throw new UnsupportedOperationException(op.getName());
+                filter = new UnsupportedTupleFilter(FilterOperatorEnum.UNSUPPORTED);
+                break;
             }
 
             for (RexNode operand : call.operands) {

http://git-wip-us.apache.org/repos/asf/kylin/blob/7955b093/server-base/src/main/java/org/apache/kylin/rest/util/QueryUtil.java
----------------------------------------------------------------------
diff --git a/server-base/src/main/java/org/apache/kylin/rest/util/QueryUtil.java b/server-base/src/main/java/org/apache/kylin/rest/util/QueryUtil.java
index a2d0402..210ebc8 100644
--- a/server-base/src/main/java/org/apache/kylin/rest/util/QueryUtil.java
+++ b/server-base/src/main/java/org/apache/kylin/rest/util/QueryUtil.java
@@ -43,6 +43,8 @@ public class QueryUtil {
     private static final Pattern PTN_GROUP_BY = Pattern.compile(S1 + "GROUP" + SM + "BY" + S1, Pattern.CASE_INSENSITIVE);
     private static final Pattern PTN_HAVING_COUNT_GREATER_THAN_ZERO = Pattern.compile(S1 + "HAVING" + SM + "[(]?" + S0 + "COUNT" + S0 + "[(]" + S0 + "1" + S0 + "[)]" + S0 + ">" + S0 + "0" + S0 + "[)]?", Pattern.CASE_INSENSITIVE);
     private static final Pattern PTN_SUM_1 = Pattern.compile(S1 + "SUM" + S0 + "[(]" + S0 + "[1]" + S0 + "[)]" + S1, Pattern.CASE_INSENSITIVE);
+    private static final Pattern PTN_INTERVAL = Pattern.compile("interval" + SM + "(floor\\()([\\d\\.]+)(\\))" + SM + "(second|minute|hour|day|month|year)", Pattern.CASE_INSENSITIVE);
+    private static final Pattern PTN_CONCAT = Pattern.compile("concat\\(.+?\\)");//non-greedy
 
     // private static final Pattern PTN_HAVING_ESCAPE_FUNCTION =
     // Pattern.compile("\\{fn" + "(" + S0 + ")" + "\\}",
@@ -150,6 +152,31 @@ public class QueryUtil {
             sql = sql.substring(0, m.start()) + " COUNT(1) " + sql.substring(m.end());
         }
 
+        // ( date '2001-09-28' + interval floor(1) day ) generated by cognos
+        // calcite only recognizes date '2001-09-28' + interval '1' day
+        while (true) {
+            m = PTN_INTERVAL.matcher(sql);
+            if (!m.find())
+                break;
+
+            int value = (int) Math.floor(Double.valueOf(m.group(2)));
+            sql = sql.substring(0, m.start(1)) + "'" + value + "'" + sql.substring(m.end(3));
+        }
+
+        //according to https://issues.apache.org/jira/browse/CALCITE-1375,
+        //{fn concat('a','b')} will succeed but concat('a','b') will fail 
+        StringBuilder sb = new StringBuilder();
+        while (true) {
+            m = PTN_CONCAT.matcher(sql);
+            if (!m.find())
+                break;
+
+            sb.append(sql.substring(0, m.start()) + "{fn " + m.group(0) + " }");
+            sql = sql.substring(m.end());
+        }
+        String temp = sb.toString() + sql;
+        sql = "".equals(temp) ? sql : temp;
+
         return sql;
     }
 

http://git-wip-us.apache.org/repos/asf/kylin/blob/7955b093/server-base/src/test/java/org/apache/kylin/rest/util/QueryUtilTest.java
----------------------------------------------------------------------
diff --git a/server-base/src/test/java/org/apache/kylin/rest/util/QueryUtilTest.java b/server-base/src/test/java/org/apache/kylin/rest/util/QueryUtilTest.java
new file mode 100644
index 0000000..9305410
--- /dev/null
+++ b/server-base/src/test/java/org/apache/kylin/rest/util/QueryUtilTest.java
@@ -0,0 +1,51 @@
+/*
+ * 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.
+ */
+
+package org.apache.kylin.rest.util;
+
+import org.apache.kylin.rest.request.SQLRequest;
+import org.junit.Assert;
+import org.junit.Test;
+
+public class QueryUtilTest {
+    @Test
+    public void testHealInterval() {
+        {
+            SQLRequest sqlRequest = new SQLRequest();
+            sqlRequest.setSql("select ( date '2001-09-28' + interval floor(1.2) day) from test_kylin_fact");
+            String s = QueryUtil.massageSql(sqlRequest);
+            Assert.assertEquals("select ( date '2001-09-28' + interval '1' day) from test_kylin_fact", s);
+        }
+        {
+            SQLRequest sqlRequest = new SQLRequest();
+            sqlRequest.setSql("select ( date '2001-09-28' + interval floor(2) month) from test_kylin_fact group by ( date '2001-09-28' + interval floor(2) month)");
+            String s = QueryUtil.massageSql(sqlRequest);
+            Assert.assertEquals("select ( date '2001-09-28' + interval '2' month) from test_kylin_fact group by ( date '2001-09-28' + interval '2' month)", s);
+        }
+    }
+
+    @Test
+    public void testHealConcat() {
+        {
+            SQLRequest sqlRequest = new SQLRequest();
+            sqlRequest.setSql("select concat(\"TEST_KYLIN_FACT\".\"LSTG_FORMAT_NAME\",\"TEST_KYLIN_FACT\".\"LSTG_FORMAT_NAME\") concat(\"TEST_KYLIN_FACT\".\"LSTG_FORMAT_NAME\",\"TEST_KYLIN_FACT\".\"LSTG_FORMAT_NAME\") ()");
+            String s = QueryUtil.massageSql(sqlRequest);
+            Assert.assertEquals("select {fn concat(\"TEST_KYLIN_FACT\".\"LSTG_FORMAT_NAME\",\"TEST_KYLIN_FACT\".\"LSTG_FORMAT_NAME\") } {fn concat(\"TEST_KYLIN_FACT\".\"LSTG_FORMAT_NAME\",\"TEST_KYLIN_FACT\".\"LSTG_FORMAT_NAME\") } ()", s);
+        }
+    }
+}