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/02/21 12:51:59 UTC

[04/52] [abbrv] kylin git commit: KYLIN-1343 Query IT pass

KYLIN-1343 Query IT pass


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

Branch: refs/heads/1.x-HBase1.1.3
Commit: d0f610fc9ec9962fd206a9d47b6f712688d2db11
Parents: df7ae17
Author: Yang Li <li...@apache.org>
Authored: Sun Jan 24 08:20:42 2016 +0800
Committer: Yang Li <li...@apache.org>
Committed: Sun Jan 24 08:20:42 2016 +0800

----------------------------------------------------------------------
 .../apache/calcite/sql2rel/RelFieldTrimmer.java | 44 ++++++++++++++++++++
 .../calcite/sql2rel/SqlToRelConverter.java      | 16 +++----
 .../kylin/query/relnode/OLAPAggregateRel.java   |  6 ++-
 .../apache/kylin/query/relnode/OLAPJoinRel.java |  2 +-
 .../kylin/query/relnode/OLAPTableScan.java      |  4 ++
 .../apache/kylin/query/schema/OLAPTable.java    | 17 ++++----
 query/src/test/resources/logging.properties     |  2 +-
 query/src/test/resources/query/sql/query86.sql  | 30 +++++++++++++
 .../resources/query/sql/query86.sql.disabled    | 30 -------------
 query/src/test/resources/query/sql/query87.sql  | 30 +++++++++++++
 .../resources/query/sql/query87.sql.disabled    | 30 -------------
 query/src/test/resources/query/sql/query88.sql  | 30 +++++++++++++
 .../resources/query/sql/query88.sql.disabled    | 30 -------------
 query/src/test/resources/query/sql/query89.sql  | 30 +++++++++++++
 .../resources/query/sql/query89.sql.disabled    | 30 -------------
 15 files changed, 193 insertions(+), 138 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kylin/blob/d0f610fc/atopcalcite/src/main/java/org/apache/calcite/sql2rel/RelFieldTrimmer.java
----------------------------------------------------------------------
diff --git a/atopcalcite/src/main/java/org/apache/calcite/sql2rel/RelFieldTrimmer.java b/atopcalcite/src/main/java/org/apache/calcite/sql2rel/RelFieldTrimmer.java
new file mode 100644
index 0000000..f88157c
--- /dev/null
+++ b/atopcalcite/src/main/java/org/apache/calcite/sql2rel/RelFieldTrimmer.java
@@ -0,0 +1,44 @@
+/*
+ * 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.calcite.sql2rel;
+
+import org.apache.calcite.plan.RelOptCluster;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.core.RelFactories;
+import org.apache.calcite.sql.validate.SqlValidator;
+import org.apache.calcite.tools.RelBuilder;
+
+/*
+ * OVERRIDE POINT:
+ * - disable the whole RelFieldTrimmer
+ */
+
+public class RelFieldTrimmer {
+
+    public RelFieldTrimmer(SqlValidator validator, RelBuilder relBuilder) {
+    }
+
+    public RelFieldTrimmer(SqlValidator validator, RelOptCluster cluster, RelFactories.ProjectFactory projectFactory, RelFactories.FilterFactory filterFactory, RelFactories.JoinFactory joinFactory, RelFactories.SemiJoinFactory semiJoinFactory, RelFactories.SortFactory sortFactory, RelFactories.AggregateFactory aggregateFactory, RelFactories.SetOpFactory setOpFactory) {
+    }
+
+    public RelNode trim(RelNode rootRel) {
+        return rootRel;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/kylin/blob/d0f610fc/atopcalcite/src/main/java/org/apache/calcite/sql2rel/SqlToRelConverter.java
----------------------------------------------------------------------
diff --git a/atopcalcite/src/main/java/org/apache/calcite/sql2rel/SqlToRelConverter.java b/atopcalcite/src/main/java/org/apache/calcite/sql2rel/SqlToRelConverter.java
index 76203e8..aed7c27 100644
--- a/atopcalcite/src/main/java/org/apache/calcite/sql2rel/SqlToRelConverter.java
+++ b/atopcalcite/src/main/java/org/apache/calcite/sql2rel/SqlToRelConverter.java
@@ -1,12 +1,13 @@
 /*
- * 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
+ * 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
+ *     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,
@@ -14,6 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
+
 package org.apache.calcite.sql2rel;
 
 import org.apache.calcite.avatica.util.Spaces;

http://git-wip-us.apache.org/repos/asf/kylin/blob/d0f610fc/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 3d1ddb3..26b987a 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
@@ -43,6 +43,7 @@ import org.apache.calcite.rel.metadata.RelMetadataQuery;
 import org.apache.calcite.rel.type.RelDataType;
 import org.apache.calcite.rel.type.RelDataTypeFactory;
 import org.apache.calcite.rel.type.RelDataTypeField;
+import org.apache.calcite.rel.type.RelDataTypeFactory.FieldInfoBuilder;
 import org.apache.calcite.schema.AggregateFunction;
 import org.apache.calcite.schema.FunctionParameter;
 import org.apache.calcite.schema.impl.AggregateFunctionImpl;
@@ -63,6 +64,7 @@ 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.query.schema.OLAPTable;
 
 import com.google.common.base.Preconditions;
 
@@ -281,6 +283,7 @@ public class OLAPAggregateRel extends Aggregate implements OLAPRel {
         fillbackOptimizedColumn();
 
         ColumnRowType inputColumnRowType = ((OLAPRel) getInput()).getColumnRowType();
+        RelDataTypeFactory typeFactory = getCluster().getTypeFactory();
         for (int i = 0; i < this.aggregations.size(); i++) {
             FunctionDesc aggFunc = this.aggregations.get(i);
 
@@ -291,7 +294,8 @@ public class OLAPAggregateRel extends Aggregate implements OLAPRel {
 
             if (aggFunc.needRewrite()) {
                 String rewriteFieldName = aggFunc.getRewriteFieldName();
-                this.context.rewriteFields.put(rewriteFieldName, null);
+                RelDataType rewriteFieldType = OLAPTable.createSqlType(typeFactory, aggFunc.getRewriteFieldType(), true);
+                this.context.rewriteFields.put(rewriteFieldName, rewriteFieldType);
 
                 TblColRef column = buildRewriteColumn(aggFunc);
                 this.context.metricsColumns.add(column);

http://git-wip-us.apache.org/repos/asf/kylin/blob/d0f610fc/query/src/main/java/org/apache/kylin/query/relnode/OLAPJoinRel.java
----------------------------------------------------------------------
diff --git a/query/src/main/java/org/apache/kylin/query/relnode/OLAPJoinRel.java b/query/src/main/java/org/apache/kylin/query/relnode/OLAPJoinRel.java
index c079357..623a43e 100644
--- a/query/src/main/java/org/apache/kylin/query/relnode/OLAPJoinRel.java
+++ b/query/src/main/java/org/apache/kylin/query/relnode/OLAPJoinRel.java
@@ -281,7 +281,7 @@ public class OLAPJoinRel extends EnumerableJoin implements OLAPRel {
         implementor.visitChild(this, this.right);
 
         this.rowType = this.deriveRowType();
-
+        
         if (this.isTopJoin && RewriteImplementor.needRewrite(this.context)) {
             // find missed rewrite fields
             int paramIndex = this.rowType.getFieldList().size();

http://git-wip-us.apache.org/repos/asf/kylin/blob/d0f610fc/query/src/main/java/org/apache/kylin/query/relnode/OLAPTableScan.java
----------------------------------------------------------------------
diff --git a/query/src/main/java/org/apache/kylin/query/relnode/OLAPTableScan.java b/query/src/main/java/org/apache/kylin/query/relnode/OLAPTableScan.java
index 48cf662..45d00b3 100644
--- a/query/src/main/java/org/apache/kylin/query/relnode/OLAPTableScan.java
+++ b/query/src/main/java/org/apache/kylin/query/relnode/OLAPTableScan.java
@@ -47,8 +47,10 @@ import org.apache.calcite.rel.rules.AggregateProjectMergeRule;
 import org.apache.calcite.rel.rules.FilterJoinRule;
 import org.apache.calcite.rel.rules.FilterProjectTransposeRule;
 import org.apache.calcite.rel.rules.JoinCommuteRule;
+import org.apache.calcite.rel.rules.JoinPushExpressionsRule;
 import org.apache.calcite.rel.rules.JoinPushThroughJoinRule;
 import org.apache.calcite.rel.rules.ReduceExpressionsRule;
+import org.apache.calcite.rel.rules.SortJoinTransposeRule;
 import org.apache.calcite.rel.type.RelDataType;
 import org.apache.calcite.rel.type.RelDataTypeFactory;
 import org.apache.calcite.rel.type.RelDataTypeField;
@@ -148,6 +150,8 @@ public class OLAPTableScan extends TableScan implements OLAPRel, EnumerableRel {
         planner.removeRule(AggregateJoinTransposeRule.INSTANCE);
         planner.removeRule(AggregateProjectMergeRule.INSTANCE);
         planner.removeRule(FilterProjectTransposeRule.INSTANCE);
+        planner.removeRule(SortJoinTransposeRule.INSTANCE);
+        planner.removeRule(JoinPushExpressionsRule.INSTANCE);
         // distinct count will be split into a separated query that is joined with the left query
         planner.removeRule(AggregateExpandDistinctAggregatesRule.INSTANCE);
 

http://git-wip-us.apache.org/repos/asf/kylin/blob/d0f610fc/query/src/main/java/org/apache/kylin/query/schema/OLAPTable.java
----------------------------------------------------------------------
diff --git a/query/src/main/java/org/apache/kylin/query/schema/OLAPTable.java b/query/src/main/java/org/apache/kylin/query/schema/OLAPTable.java
index 8b1ad29..fad33ed 100644
--- a/query/src/main/java/org/apache/kylin/query/schema/OLAPTable.java
+++ b/query/src/main/java/org/apache/kylin/query/schema/OLAPTable.java
@@ -43,6 +43,7 @@ import org.apache.calcite.schema.impl.AbstractTableQueryable;
 import org.apache.calcite.sql.type.SqlTypeName;
 import org.apache.calcite.sql.type.SqlTypeUtil;
 import org.apache.calcite.util.ImmutableBitSet;
+import org.apache.kylin.metadata.datatype.DataType;
 import org.apache.kylin.metadata.model.ColumnDesc;
 import org.apache.kylin.metadata.model.FunctionDesc;
 import org.apache.kylin.metadata.model.MeasureDesc;
@@ -128,20 +129,20 @@ public class OLAPTable extends AbstractQueryableTable implements TranslatableTab
     private RelDataType deriveRowType(RelDataTypeFactory typeFactory) {
         RelDataTypeFactory.FieldInfoBuilder fieldInfo = typeFactory.builder();
         for (ColumnDesc column : exposedColumns) {
-            RelDataType sqlType = createSqlType(typeFactory, column);
+            RelDataType sqlType = createSqlType(typeFactory, column.getType(), column.isNullable());
             sqlType = SqlTypeUtil.addCharsetAndCollation(sqlType, typeFactory);
             fieldInfo.add(column.getName(), sqlType);
         }
         return typeFactory.createStructType(fieldInfo);
     }
 
-    private RelDataType createSqlType(RelDataTypeFactory typeFactory, ColumnDesc column) {
-        SqlTypeName sqlTypeName = SQLTYPE_MAPPING.get(column.getTypeName());
+    public static RelDataType createSqlType(RelDataTypeFactory typeFactory, DataType dataType, boolean isNullable) {
+        SqlTypeName sqlTypeName = SQLTYPE_MAPPING.get(dataType.getName());
         if (sqlTypeName == null)
-            throw new IllegalArgumentException("Unrecognized column type " + column.getTypeName() + " from " + column);
+            throw new IllegalArgumentException("Unrecognized data type " + dataType);
 
-        int precision = column.getTypePrecision();
-        int scale = column.getTypeScale();
+        int precision = dataType.getPrecision();
+        int scale = dataType.getScale();
 
         RelDataType result;
         if (precision >= 0 && scale >= 0)
@@ -152,7 +153,7 @@ public class OLAPTable extends AbstractQueryableTable implements TranslatableTab
             result = typeFactory.createSqlType(sqlTypeName);
 
         // due to left join and uncertain data quality, dimension value can be null
-        if (column.isNullable()) {
+        if (isNullable) {
             result = typeFactory.createTypeWithNullability(result, true);
         } else {
             result = typeFactory.createTypeWithNullability(result, false);
@@ -160,7 +161,7 @@ public class OLAPTable extends AbstractQueryableTable implements TranslatableTab
 
         return result;
     }
-
+    
     private List<ColumnDesc> listSourceColumns() {
         ProjectManager mgr = ProjectManager.getInstance(olapSchema.getConfig());
         List<ColumnDesc> tableColumns = Lists.newArrayList(mgr.listExposedColumns(olapSchema.getProjectName(), sourceTable.getIdentity()));

http://git-wip-us.apache.org/repos/asf/kylin/blob/d0f610fc/query/src/test/resources/logging.properties
----------------------------------------------------------------------
diff --git a/query/src/test/resources/logging.properties b/query/src/test/resources/logging.properties
index 5767d05..4edb856 100644
--- a/query/src/test/resources/logging.properties
+++ b/query/src/test/resources/logging.properties
@@ -17,6 +17,6 @@
 
 handlers=java.util.logging.ConsoleHandler
 .level=INFO
-#org.eigenbase.relopt.RelOptPlanner.level=FINEST
+#org.apache.calcite.plan.RelOptPlanner.level=FINEST
 java.util.logging.ConsoleHandler.level=ALL
 java.util.logging.ConsoleHandler.formatter=org.apache.kylin.common.util.MyLogFormatter
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/kylin/blob/d0f610fc/query/src/test/resources/query/sql/query86.sql
----------------------------------------------------------------------
diff --git a/query/src/test/resources/query/sql/query86.sql b/query/src/test/resources/query/sql/query86.sql
new file mode 100644
index 0000000..e551a45
--- /dev/null
+++ b/query/src/test/resources/query/sql/query86.sql
@@ -0,0 +1,30 @@
+--
+-- 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 meta_categ_name, count(1) as cnt, sum(price) as GMV 
+
+ from test_kylin_fact 
+ left JOIN edw.test_cal_dt as test_cal_dt
+ ON test_kylin_fact.cal_dt = test_cal_dt.cal_dt
+ left 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
+ left JOIN edw.test_sites as test_sites
+ ON test_kylin_fact.lstg_site_id = test_sites.site_id
+
+ where meta_categ_name not in ('', 'a')
+ group by meta_categ_name 

http://git-wip-us.apache.org/repos/asf/kylin/blob/d0f610fc/query/src/test/resources/query/sql/query86.sql.disabled
----------------------------------------------------------------------
diff --git a/query/src/test/resources/query/sql/query86.sql.disabled b/query/src/test/resources/query/sql/query86.sql.disabled
deleted file mode 100644
index e551a45..0000000
--- a/query/src/test/resources/query/sql/query86.sql.disabled
+++ /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 meta_categ_name, count(1) as cnt, sum(price) as GMV 
-
- from test_kylin_fact 
- left JOIN edw.test_cal_dt as test_cal_dt
- ON test_kylin_fact.cal_dt = test_cal_dt.cal_dt
- left 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
- left JOIN edw.test_sites as test_sites
- ON test_kylin_fact.lstg_site_id = test_sites.site_id
-
- where meta_categ_name not in ('', 'a')
- group by meta_categ_name 

http://git-wip-us.apache.org/repos/asf/kylin/blob/d0f610fc/query/src/test/resources/query/sql/query87.sql
----------------------------------------------------------------------
diff --git a/query/src/test/resources/query/sql/query87.sql b/query/src/test/resources/query/sql/query87.sql
new file mode 100644
index 0000000..cc6dca5
--- /dev/null
+++ b/query/src/test/resources/query/sql/query87.sql
@@ -0,0 +1,30 @@
+--
+-- 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 meta_categ_name, count(1) as cnt, sum(price) as GMV 
+
+ from test_kylin_fact 
+ left JOIN edw.test_cal_dt as test_cal_dt
+ ON test_kylin_fact.cal_dt = test_cal_dt.cal_dt
+ left 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
+ left JOIN edw.test_sites as test_sites
+ ON test_kylin_fact.lstg_site_id = test_sites.site_id
+
+ where meta_categ_name is not null
+ group by meta_categ_name 

http://git-wip-us.apache.org/repos/asf/kylin/blob/d0f610fc/query/src/test/resources/query/sql/query87.sql.disabled
----------------------------------------------------------------------
diff --git a/query/src/test/resources/query/sql/query87.sql.disabled b/query/src/test/resources/query/sql/query87.sql.disabled
deleted file mode 100644
index cc6dca5..0000000
--- a/query/src/test/resources/query/sql/query87.sql.disabled
+++ /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 meta_categ_name, count(1) as cnt, sum(price) as GMV 
-
- from test_kylin_fact 
- left JOIN edw.test_cal_dt as test_cal_dt
- ON test_kylin_fact.cal_dt = test_cal_dt.cal_dt
- left 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
- left JOIN edw.test_sites as test_sites
- ON test_kylin_fact.lstg_site_id = test_sites.site_id
-
- where meta_categ_name is not null
- group by meta_categ_name 

http://git-wip-us.apache.org/repos/asf/kylin/blob/d0f610fc/query/src/test/resources/query/sql/query88.sql
----------------------------------------------------------------------
diff --git a/query/src/test/resources/query/sql/query88.sql b/query/src/test/resources/query/sql/query88.sql
new file mode 100644
index 0000000..c7899fd
--- /dev/null
+++ b/query/src/test/resources/query/sql/query88.sql
@@ -0,0 +1,30 @@
+--
+-- 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 meta_categ_name, count(1) as cnt, sum(price) as GMV 
+
+ from test_kylin_fact 
+ left JOIN edw.test_cal_dt as test_cal_dt
+ ON test_kylin_fact.cal_dt = test_cal_dt.cal_dt
+ left 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
+ left JOIN edw.test_sites as test_sites
+ ON test_kylin_fact.lstg_site_id = test_sites.site_id
+
+ where meta_categ_name not in ('Unknown')
+ group by meta_categ_name 

http://git-wip-us.apache.org/repos/asf/kylin/blob/d0f610fc/query/src/test/resources/query/sql/query88.sql.disabled
----------------------------------------------------------------------
diff --git a/query/src/test/resources/query/sql/query88.sql.disabled b/query/src/test/resources/query/sql/query88.sql.disabled
deleted file mode 100644
index c7899fd..0000000
--- a/query/src/test/resources/query/sql/query88.sql.disabled
+++ /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 meta_categ_name, count(1) as cnt, sum(price) as GMV 
-
- from test_kylin_fact 
- left JOIN edw.test_cal_dt as test_cal_dt
- ON test_kylin_fact.cal_dt = test_cal_dt.cal_dt
- left 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
- left JOIN edw.test_sites as test_sites
- ON test_kylin_fact.lstg_site_id = test_sites.site_id
-
- where meta_categ_name not in ('Unknown')
- group by meta_categ_name 

http://git-wip-us.apache.org/repos/asf/kylin/blob/d0f610fc/query/src/test/resources/query/sql/query89.sql
----------------------------------------------------------------------
diff --git a/query/src/test/resources/query/sql/query89.sql b/query/src/test/resources/query/sql/query89.sql
new file mode 100644
index 0000000..578b93f
--- /dev/null
+++ b/query/src/test/resources/query/sql/query89.sql
@@ -0,0 +1,30 @@
+--
+-- 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 meta_categ_name, count(1) as cnt, sum(price) as GMV 
+
+ from test_kylin_fact 
+ left JOIN edw.test_cal_dt as test_cal_dt
+ ON test_kylin_fact.cal_dt = test_cal_dt.cal_dt
+ left 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
+ left JOIN edw.test_sites as test_sites
+ ON test_kylin_fact.lstg_site_id = test_sites.site_id
+
+ where meta_categ_name not in ('Unknown', 'ToyHobbies', '', 'a', 'BookMagazines')
+ group by meta_categ_name 

http://git-wip-us.apache.org/repos/asf/kylin/blob/d0f610fc/query/src/test/resources/query/sql/query89.sql.disabled
----------------------------------------------------------------------
diff --git a/query/src/test/resources/query/sql/query89.sql.disabled b/query/src/test/resources/query/sql/query89.sql.disabled
deleted file mode 100644
index 578b93f..0000000
--- a/query/src/test/resources/query/sql/query89.sql.disabled
+++ /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 meta_categ_name, count(1) as cnt, sum(price) as GMV 
-
- from test_kylin_fact 
- left JOIN edw.test_cal_dt as test_cal_dt
- ON test_kylin_fact.cal_dt = test_cal_dt.cal_dt
- left 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
- left JOIN edw.test_sites as test_sites
- ON test_kylin_fact.lstg_site_id = test_sites.site_id
-
- where meta_categ_name not in ('Unknown', 'ToyHobbies', '', 'a', 'BookMagazines')
- group by meta_categ_name