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 2015/06/11 08:37:36 UTC

[12/14] incubator-kylin git commit: KYLIN-812 Upgrade to Calcite 0.9.2

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/6480a9a7/atopcalcite/src/main/java/org/eigenbase/sql2rel/SqlToRelConverter.java
----------------------------------------------------------------------
diff --git a/atopcalcite/src/main/java/org/eigenbase/sql2rel/SqlToRelConverter.java b/atopcalcite/src/main/java/org/eigenbase/sql2rel/SqlToRelConverter.java
index 796640c..dd7ee01 100644
--- a/atopcalcite/src/main/java/org/eigenbase/sql2rel/SqlToRelConverter.java
+++ b/atopcalcite/src/main/java/org/eigenbase/sql2rel/SqlToRelConverter.java
@@ -45,6 +45,7 @@ import org.eigenbase.util.mapping.Mappings;
 import org.eigenbase.util14.*;
 
 import net.hydromatic.linq4j.Ord;
+
 import net.hydromatic.optiq.ModifiableTable;
 import net.hydromatic.optiq.TranslatableTable;
 import net.hydromatic.optiq.prepare.Prepare;
@@ -65,7 +66,6 @@ import static org.eigenbase.util.Static.RESOURCE;
  * <p>The public entry points are: {@link #convertQuery},
  * {@link #convertExpression(SqlNode)}.
  */
-@SuppressWarnings({"unused", "rawtypes", "unchecked", "incomplete-switch", "deprecation"})
 public class SqlToRelConverter {
   //~ Static fields/initializers ---------------------------------------------
 
@@ -1487,7 +1487,7 @@ public class SqlToRelConverter {
    *                                     node, only register it if it's a scalar
    *                                     subquery
    */
-private void findSubqueries(
+  private void findSubqueries(
       Blackboard bb,
       SqlNode node,
       RelOptUtil.Logic logic,

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/6480a9a7/cube/src/main/java/org/apache/kylin/cube/model/CubeDesc.java
----------------------------------------------------------------------
diff --git a/cube/src/main/java/org/apache/kylin/cube/model/CubeDesc.java b/cube/src/main/java/org/apache/kylin/cube/model/CubeDesc.java
index a810376..b03268f 100644
--- a/cube/src/main/java/org/apache/kylin/cube/model/CubeDesc.java
+++ b/cube/src/main/java/org/apache/kylin/cube/model/CubeDesc.java
@@ -530,39 +530,6 @@ public class CubeDesc extends RootPersistentEntity {
                  */
             }
         }
-
-
-        {
-            // drop those columns (like lookup table's PK but used as a dimension) that can be derived
-            List<TblColRef> derivedCols = Lists.newArrayList();
-            for (TblColRef col : dimensionColumns) {
-                if (isDerived(col) == true)
-                    derivedCols.add(col);
-            }
-
-            if (derivedCols.size() > 0) {
-                dimensionColumns.removeAll(derivedCols);
-            }
-        }
-
-        {
-            // for hierarchy columns, if a column can be derived, use the host col to replace
-            for (DimensionDesc dim : dimensions) {
-                if (dim.isHierarchy()) {
-                    for (int i = 0; i < dim.getColumnRefs().length; i++) {
-                        TblColRef colRef = dim.getColumnRefs()[i];
-                        if (isDerived(colRef)) {
-                            TblColRef[] hostCols = derivedToHostMap.get(colRef).columns;
-                            if (hostCols.length == 1) {
-                               dim.getHierarchy()[i].setColumnRef(hostCols[0]);
-                            } else {
-                                throw new IllegalStateException();
-                            }
-                        }
-                    }
-                }
-            }
-        }
     }
 
     private String[][] splitDerivedColumnAndExtra(String[] derived) {

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/6480a9a7/jdbc/src/main/java/org/apache/kylin/jdbc/KylinPrepare.java
----------------------------------------------------------------------
diff --git a/jdbc/src/main/java/org/apache/kylin/jdbc/KylinPrepare.java b/jdbc/src/main/java/org/apache/kylin/jdbc/KylinPrepare.java
index 7f20d5b..0e4fb35 100644
--- a/jdbc/src/main/java/org/apache/kylin/jdbc/KylinPrepare.java
+++ b/jdbc/src/main/java/org/apache/kylin/jdbc/KylinPrepare.java
@@ -1,98 +1,103 @@
-/*
- * 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.jdbc;
-
-import java.util.List;
-
-import net.hydromatic.avatica.AvaticaParameter;
-import net.hydromatic.avatica.AvaticaPrepareResult;
-import net.hydromatic.avatica.ColumnMetaData;
-import net.hydromatic.avatica.Cursor;
-import net.hydromatic.linq4j.Enumerator;
-import net.hydromatic.optiq.runtime.EnumeratorCursor;
-
-/**
- * Interface of kylin prepare statement implementation
- * 
- * @author xduo
- * 
- */
-public interface KylinPrepare {
-
-    PrepareResult prepare(String sql);
-
-    /**
-     * The result of preparing a query. It gives the Avatica driver framework
-     * the information it needs to create a prepared statement, or to execute a
-     * statement directly, without an explicit prepare step.
-     */
-    public static class PrepareResult implements AvaticaPrepareResult {
-        public final String sql; // for debug
-        public final ColumnMetaData.StructType structType;
-        public final Enumerator<Object[]> enumerator;
-        public final List<AvaticaParameter> parameterList;
-
-        public PrepareResult(String sql, List<AvaticaParameter> parameterList, Enumerator<Object[]> enumerator, ColumnMetaData.StructType structType) {
-            super();
-            this.sql = sql;
-            this.parameterList = parameterList;
-            this.enumerator = enumerator;
-            this.structType = structType;
-        }
-
-        public Cursor createCursor() {
-            return new EnumeratorCursor<Object[]>(enumerator) {
-                @Override
-                protected Getter createGetter(int ordinal) {
-                    return new ArrayEnumeratorGetter(ordinal);
-                }
-
-                /**
-                 * Row field accessor via index
-                 */
-                class ArrayEnumeratorGetter extends AbstractGetter {
-                    protected final int field;
-
-                    public ArrayEnumeratorGetter(int field) {
-                        this.field = field;
-                    }
-
-                    public Object getObject() {
-                        Object o = current()[field];
-                        wasNull[0] = o == null;
-                        return o;
-                    }
-                }
-            };
-        }
-
-        public List<ColumnMetaData> getColumnList() {
-            return structType.columns;
-        }
-
-        public List<AvaticaParameter> getParameterList() {
-            return parameterList;
-        }
-
-        public String getSql() {
-            return sql;
-        }
-    }
-
-}
+/*
+ * 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.jdbc;
+
+import java.util.List;
+import java.util.Map;
+
+import net.hydromatic.avatica.AvaticaParameter;
+import net.hydromatic.avatica.AvaticaPrepareResult;
+import net.hydromatic.avatica.ColumnMetaData;
+import net.hydromatic.avatica.Cursor;
+import net.hydromatic.linq4j.Enumerator;
+import net.hydromatic.optiq.runtime.EnumeratorCursor;
+
+/**
+ * Interface of kylin prepare statement implementation
+ * 
+ * @author xduo
+ * 
+ */
+public interface KylinPrepare {
+
+    PrepareResult prepare(String sql);
+
+    /**
+     * The result of preparing a query. It gives the Avatica driver framework
+     * the information it needs to create a prepared statement, or to execute a
+     * statement directly, without an explicit prepare step.
+     */
+    public static class PrepareResult implements AvaticaPrepareResult {
+        public final String sql; // for debug
+        public final ColumnMetaData.StructType structType;
+        public final Enumerator<Object[]> enumerator;
+        public final List<AvaticaParameter> parameterList;
+
+        public PrepareResult(String sql, List<AvaticaParameter> parameterList, Enumerator<Object[]> enumerator, ColumnMetaData.StructType structType) {
+            super();
+            this.sql = sql;
+            this.parameterList = parameterList;
+            this.enumerator = enumerator;
+            this.structType = structType;
+        }
+
+        public Cursor createCursor() {
+            return new EnumeratorCursor<Object[]>(enumerator) {
+                @Override
+                protected Getter createGetter(int ordinal) {
+                    return new ArrayEnumeratorGetter(ordinal);
+                }
+
+                /**
+                 * Row field accessor via index
+                 */
+                class ArrayEnumeratorGetter extends AbstractGetter {
+                    protected final int field;
+
+                    public ArrayEnumeratorGetter(int field) {
+                        this.field = field;
+                    }
+
+                    public Object getObject() {
+                        Object o = current()[field];
+                        wasNull[0] = o == null;
+                        return o;
+                    }
+                }
+            };
+        }
+
+        public List<ColumnMetaData> getColumnList() {
+            return structType.columns;
+        }
+
+        public List<AvaticaParameter> getParameterList() {
+            return parameterList;
+        }
+
+        public Map<String, Object> getInternalParameters() {
+            return null;
+        }
+
+        public String getSql() {
+            return sql;
+        }
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/6480a9a7/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index cf776db..5748328 100644
--- a/pom.xml
+++ b/pom.xml
@@ -84,7 +84,7 @@
         <spring.framework.version>3.1.2.RELEASE</spring.framework.version>
 
         <!-- Optiq Version -->
-        <optiq.version>0.9.1-incubating</optiq.version>
+        <optiq.version>0.9.2-incubating</optiq.version>
         <linq4j.version>0.4</linq4j.version>
 
         <!-- Metrics Codahale Version -->

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/6480a9a7/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 83723c2..e228d82 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
@@ -49,6 +49,7 @@ import org.eigenbase.reltype.RelDataTypeFactory;
 import org.eigenbase.reltype.RelDataTypeField;
 import org.eigenbase.sql.SqlAggFunction;
 import org.eigenbase.sql.SqlIdentifier;
+import org.eigenbase.sql.fun.SqlStdOperatorTable;
 import org.eigenbase.sql.fun.SqlSumEmptyIsZeroAggFunction;
 import org.eigenbase.sql.parser.SqlParserPos;
 import org.eigenbase.sql.type.InferTypes;
@@ -59,6 +60,7 @@ import org.eigenbase.sql.validate.SqlUserDefinedAggFunction;
 import org.eigenbase.util.Util;
 
 import com.google.common.base.Preconditions;
+
 import org.apache.kylin.metadata.model.ColumnDesc;
 import org.apache.kylin.metadata.model.TableDesc;
 import org.apache.kylin.metadata.model.FunctionDesc;
@@ -304,7 +306,8 @@ public class OLAPAggregateRel extends AggregateRelBase implements OLAPRel, Enume
         if (func.isCountDistinct()) {
             newAgg = createHyperLogLogAggFunction(fieldType);
         } else if (func.isCount()) {
-            newAgg = new SqlSumEmptyIsZeroAggFunction(fieldType);
+            //newAgg = new SqlSumEmptyIsZeroAggFunction(fieldType);
+            newAgg = SqlStdOperatorTable.SUM0;
         }
 
         // rebuild aggregate call

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/6480a9a7/query/src/main/java/org/apache/kylin/query/relnode/OLAPRel.java
----------------------------------------------------------------------
diff --git a/query/src/main/java/org/apache/kylin/query/relnode/OLAPRel.java b/query/src/main/java/org/apache/kylin/query/relnode/OLAPRel.java
index 3c34ab4..06cad92 100644
--- a/query/src/main/java/org/apache/kylin/query/relnode/OLAPRel.java
+++ b/query/src/main/java/org/apache/kylin/query/relnode/OLAPRel.java
@@ -18,6 +18,7 @@
 
 package org.apache.kylin.query.relnode;
 
+import java.util.LinkedHashMap;
 import java.util.Stack;
 
 import net.hydromatic.optiq.rules.java.EnumerableRel;
@@ -139,7 +140,7 @@ public interface OLAPRel extends RelNode {
         private OLAPContext parentContext;
 
         public JavaImplementor(EnumerableRelImplementor enumImplementor) {
-            super(enumImplementor.getRexBuilder());
+            super(enumImplementor.getRexBuilder(), new LinkedHashMap<String, Object>());
         }
 
         public OLAPContext getParentContext() {