You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by vk...@apache.org on 2015/09/30 02:05:19 UTC

ignite git commit: IGNITE-1552 - Fixed parsing for SQL table function

Repository: ignite
Updated Branches:
  refs/heads/master c2cedb0ed -> 740291871


IGNITE-1552 - Fixed parsing for SQL table function


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

Branch: refs/heads/master
Commit: 740291871013bef818edf3faf19c8d4c6eab1ba5
Parents: c2cedb0
Author: S.Vladykin <sv...@gridgain.com>
Authored: Tue Sep 29 17:04:52 2015 -0700
Committer: Valentin Kulichenko <va...@gmail.com>
Committed: Tue Sep 29 17:04:52 2015 -0700

----------------------------------------------------------------------
 .../processors/query/h2/sql/GridSqlArray.java   | 52 ++++++++++++
 .../processors/query/h2/sql/GridSqlElement.java |  2 +-
 .../query/h2/sql/GridSqlFunction.java           | 60 ++++++++------
 .../query/h2/sql/GridSqlFunctionType.java       |  3 +
 .../query/h2/sql/GridSqlPlaceholder.java        |  7 +-
 .../query/h2/sql/GridSqlQueryParser.java        | 84 ++++++++++++--------
 .../processors/query/h2/sql/GridSqlType.java    | 29 ++++++-
 .../query/h2/sql/GridQueryParsingTest.java      | 27 +++++++
 8 files changed, 199 insertions(+), 65 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/74029187/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlArray.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlArray.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlArray.java
new file mode 100644
index 0000000..69e98bf
--- /dev/null
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlArray.java
@@ -0,0 +1,52 @@
+/*
+ * 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.ignite.internal.processors.query.h2.sql;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import org.h2.util.StatementBuilder;
+
+/**
+ * SQL Array: (1, 2, ?, 'abc')
+ */
+public class GridSqlArray extends GridSqlElement {
+    /**
+     * @param size Array size.
+     */
+    public GridSqlArray(int size) {
+        super(size == 0 ? Collections.<GridSqlElement>emptyList() : new ArrayList<GridSqlElement>(size));
+    }
+
+    /** {@inheritDoc} */
+    @Override public String getSQL() {
+        if (size() == 0)
+            return "()";
+
+        StatementBuilder buff = new StatementBuilder("(");
+
+        for (GridSqlElement e : children) {
+            buff.appendExceptFirst(", ");
+            buff.append(e.getSQL());
+        }
+
+        if (size() == 1)
+            buff.append(',');
+
+        return buff.append(')').toString();
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/74029187/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlElement.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlElement.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlElement.java
index 41b7ba4..57d3c57 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlElement.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlElement.java
@@ -25,7 +25,7 @@ import java.util.List;
  */
 public abstract class GridSqlElement implements Iterable<GridSqlElement> {
     /** */
-    protected List<GridSqlElement> children;
+    protected final List<GridSqlElement> children;
 
     /** */
     private GridSqlType resultType;

http://git-wip-us.apache.org/repos/asf/ignite/blob/74029187/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlFunction.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlFunction.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlFunction.java
index 51433c0..7cd7a6b 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlFunction.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlFunction.java
@@ -27,7 +27,6 @@ import org.h2.value.ValueString;
 
 import static org.apache.ignite.internal.processors.query.h2.sql.GridSqlFunctionType.CASE;
 import static org.apache.ignite.internal.processors.query.h2.sql.GridSqlFunctionType.CAST;
-import static org.apache.ignite.internal.processors.query.h2.sql.GridSqlFunctionType.CONVERT;
 import static org.apache.ignite.internal.processors.query.h2.sql.GridSqlFunctionType.UNKNOWN_FUNCTION;
 
 /**
@@ -70,7 +69,7 @@ public class GridSqlFunction extends GridSqlElement {
         super(new ArrayList<GridSqlElement>());
 
         if (name == null)
-            throw new NullPointerException();
+            throw new NullPointerException("name");
 
         if (type == null)
             type = UNKNOWN_FUNCTION;
@@ -113,31 +112,46 @@ public class GridSqlFunction extends GridSqlElement {
 
         buff.append('(');
 
-        if (type == CAST) {
-            String castType = resultType().sql();
+        switch (type) {
+            case CAST:
+            case CONVERT:
+                assert size() == 1;
 
-            assert !F.isEmpty(castType) : castType;
-            assert size() == 1;
+                String castType = resultType().sql();
 
-            buff.append(child().getSQL()).append(" AS ").append(castType);
-        }
-        else if (type == CONVERT) {
-            String castType = resultType().sql();
+                assert !F.isEmpty(castType) : castType;
 
-            assert !F.isEmpty(castType) : castType;
-            assert size() == 1;
+                buff.append(child().getSQL());
+                buff.append(type == CAST ? " AS " : ",");
+                buff.append(castType);
 
-            buff.append(child().getSQL()).append(',').append(castType);
-        }
-        else if (type == GridSqlFunctionType.EXTRACT) {
-            ValueString v = (ValueString)((GridSqlConst)child(0)).value();
-            buff.append(v.getString()).append(" FROM ").append(child(1).getSQL());
-        }
-        else {
-            for (GridSqlElement e : children) {
-                buff.appendExceptFirst(", ");
-                buff.append(e.getSQL());
-            }
+                break;
+
+            case EXTRACT:
+                ValueString v = (ValueString)((GridSqlConst)child(0)).value();
+                buff.append(v.getString()).append(" FROM ").append(child(1).getSQL());
+
+                break;
+
+            case TABLE:
+                for (GridSqlElement e : children) {
+                    buff.appendExceptFirst(", ");
+
+                    // id int = ?, name varchar = ('aaa', 'bbb')
+                    buff.append(((GridSqlAlias)e).alias())
+                        .append(' ')
+                        .append(e.resultType().sql())
+                        .append('=')
+                        .append(e.child().getSQL());
+                }
+
+                break;
+
+            default:
+                for (GridSqlElement e : children) {
+                    buff.appendExceptFirst(", ");
+                    buff.append(e.getSQL());
+                }
         }
 
         return buff.append(')').toString();

http://git-wip-us.apache.org/repos/asf/ignite/blob/74029187/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlFunctionType.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlFunctionType.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlFunctionType.java
index 7b78d7a..8e56d33 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlFunctionType.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlFunctionType.java
@@ -61,6 +61,9 @@ public enum GridSqlFunctionType {
     /** */
     SYSTEM_RANGE,
 
+    /** TABLE and TABLE_DISTINCT */
+    TABLE,
+
     /** Constant for all other functions. */
     UNKNOWN_FUNCTION;
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/74029187/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlPlaceholder.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlPlaceholder.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlPlaceholder.java
index aa00c69..0bb69a8 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlPlaceholder.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlPlaceholder.java
@@ -44,12 +44,7 @@ public class GridSqlPlaceholder extends GridSqlElement {
     }
 
     /** {@inheritDoc} */
-    @Override public GridSqlElement addChild(GridSqlElement expr) {
-        throw new IllegalStateException();
-    }
-
-    /** {@inheritDoc} */
-    @Override public GridSqlElement child(int idx) {
+    @Override public GridSqlElement resultType(GridSqlType type) {
         throw new IllegalStateException();
     }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/74029187/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlQueryParser.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlQueryParser.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlQueryParser.java
index bffa36e..2789796 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlQueryParser.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlQueryParser.java
@@ -19,10 +19,8 @@ package org.apache.ignite.internal.processors.query.h2.sql;
 
 import java.lang.reflect.Field;
 import java.util.ArrayList;
-import java.util.HashSet;
 import java.util.IdentityHashMap;
 import java.util.List;
-import java.util.Set;
 import org.apache.ignite.IgniteException;
 import org.h2.command.Command;
 import org.h2.command.Prepared;
@@ -42,11 +40,13 @@ import org.h2.expression.ConditionInSelect;
 import org.h2.expression.ConditionNot;
 import org.h2.expression.Expression;
 import org.h2.expression.ExpressionColumn;
+import org.h2.expression.ExpressionList;
 import org.h2.expression.Function;
 import org.h2.expression.JavaFunction;
 import org.h2.expression.Operation;
 import org.h2.expression.Parameter;
 import org.h2.expression.Subquery;
+import org.h2.expression.TableFunction;
 import org.h2.expression.ValueExpression;
 import org.h2.jdbc.JdbcPreparedStatement;
 import org.h2.result.SortOrder;
@@ -57,7 +57,6 @@ import org.h2.table.Table;
 import org.h2.table.TableBase;
 import org.h2.table.TableFilter;
 import org.h2.table.TableView;
-import org.h2.value.Value;
 import org.jetbrains.annotations.Nullable;
 
 import static org.apache.ignite.internal.processors.query.h2.sql.GridSqlOperationType.AND;
@@ -83,6 +82,8 @@ import static org.apache.ignite.internal.processors.query.h2.sql.GridSqlOperatio
 import static org.apache.ignite.internal.processors.query.h2.sql.GridSqlOperationType.SMALLER;
 import static org.apache.ignite.internal.processors.query.h2.sql.GridSqlOperationType.SMALLER_EQUAL;
 import static org.apache.ignite.internal.processors.query.h2.sql.GridSqlOperationType.SPATIAL_INTERSECTS;
+import static org.apache.ignite.internal.processors.query.h2.sql.GridSqlType.fromColumn;
+import static org.apache.ignite.internal.processors.query.h2.sql.GridSqlType.fromExpression;
 
 /**
  * H2 Query parser.
@@ -155,6 +156,9 @@ public class GridSqlQueryParser {
         getter(ConditionInConstantSet.class, "valueList");
 
     /** */
+    private static final Getter<ExpressionList, Expression[]> EXPR_LIST = getter(ExpressionList.class, "list");
+
+    /** */
     private static final Getter<ConditionInSelect, Expression> LEFT_CIS = getter(ConditionInSelect.class, "left");
 
     /** */
@@ -198,6 +202,9 @@ public class GridSqlQueryParser {
     private static final Getter<FunctionTable, Expression> FUNC_EXPR = getter(FunctionTable.class, "functionExpr");
 
     /** */
+    private static final Getter<TableFunction, Column[]> FUNC_TBL_COLS = getter(TableFunction.class, "columnList");
+
+    /** */
     private static final Getter<JavaFunction, FunctionAlias> FUNC_ALIAS = getter(JavaFunction.class, "functionAlias");
 
     /** */
@@ -293,8 +300,6 @@ public class GridSqlQueryParser {
         Expression where = CONDITION.get(select);
         res.where(parseExpression(where, false));
 
-        Set<TableFilter> allFilters = new HashSet<>(select.getTopFilters());
-
         GridSqlElement from = null;
 
         TableFilter filter = select.getTopTableFilter();
@@ -308,16 +313,12 @@ public class GridSqlQueryParser {
             from = from == null ? gridFilter : new GridSqlJoin(from, gridFilter, filter.isJoinOuter(),
                 parseExpression(filter.getJoinCondition(), false));
 
-            allFilters.remove(filter);
-
             filter = filter.getJoin();
         }
         while (filter != null);
 
         res.from(from);
 
-        assert allFilters.isEmpty();
-
         ArrayList<Expression> expressions = select.getExpressions();
 
         for (int i = 0; i < expressions.size(); i++)
@@ -419,18 +420,8 @@ public class GridSqlQueryParser {
         if (res == null) {
             res = parseExpression0(expression, calcTypes);
 
-            if (calcTypes) {
-                GridSqlType type = GridSqlType.UNKNOWN;
-
-                if (expression.getType() != Value.UNKNOWN) {
-                    Column c = new Column(null, expression.getType(), expression.getPrecision(), expression.getScale(),
-                        expression.getDisplaySize());
-
-                    type = new GridSqlType(c.getType(), c.getScale(), c.getPrecision(), c.getDisplaySize(), c.getCreateSQL());
-                }
-
-                res.resultType(type);
-            }
+            if (calcTypes)
+                res.resultType(fromExpression(expression));
 
             h2ObjToGridObj.put(expression, res);
         }
@@ -577,24 +568,38 @@ public class GridSqlQueryParser {
             GridSqlFunction res = new GridSqlFunction(null, f.getName());
 
             if (f.getArgs() != null) {
-                for (Expression arg : f.getArgs()) {
-                    if (arg == null) {
-                        if (f.getFunctionType() != Function.CASE)
-                            throw new IllegalStateException("Function type with null arg: " + f.getFunctionType());
+                if (f.getFunctionType() == Function.TABLE || f.getFunctionType() == Function.TABLE_DISTINCT) {
+                    Column[] cols = FUNC_TBL_COLS.get((TableFunction)f);
+                    Expression[] args = f.getArgs();
+
+                    assert cols.length == args.length;
 
-                        res.addChild(GridSqlPlaceholder.EMPTY);
+                    for (int i = 0; i < cols.length; i++) {
+                        GridSqlElement arg = parseExpression(args[i], calcTypes);
+
+                        GridSqlAlias alias = new GridSqlAlias(cols[i].getName(), arg, false);
+
+                        alias.resultType(fromColumn(cols[i]));
+
+                        res.addChild(alias);
+                    }
+                }
+                else {
+                    for (Expression arg : f.getArgs()) {
+                        if (arg == null) {
+                            if (f.getFunctionType() != Function.CASE)
+                                throw new IllegalStateException("Function type with null arg: " + f.getFunctionType());
+
+                            res.addChild(GridSqlPlaceholder.EMPTY);
+                        }
+                        else
+                            res.addChild(parseExpression(arg, calcTypes));
                     }
-                    else
-                        res.addChild(parseExpression(arg, calcTypes));
                 }
             }
 
-            if (f.getFunctionType() == Function.CAST || f.getFunctionType() == Function.CONVERT) {
-                Column c = new Column(null, f.getType(), f.getPrecision(), f.getScale(), f.getDisplaySize());
-
-                res.resultType(new GridSqlType(c.getType(), c.getScale(), c.getPrecision(),
-                    c.getDisplaySize(), c.getCreateSQL()));
-            }
+            if (f.getFunctionType() == Function.CAST || f.getFunctionType() == Function.CONVERT)
+                res.resultType(fromExpression(f));
 
             return res;
         }
@@ -629,6 +634,17 @@ public class GridSqlQueryParser {
             return res;
         }
 
+        if (expression instanceof ExpressionList) {
+            Expression[] exprs = EXPR_LIST.get((ExpressionList)expression);
+
+            GridSqlArray res = new GridSqlArray(exprs.length);
+
+            for (Expression expr : exprs)
+                res.addChild(parseExpression(expr, calcTypes));
+
+            return res;
+        }
+
         throw new IgniteException("Unsupported expression: " + expression + " [type=" +
             expression.getClass().getSimpleName() + ']');
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/74029187/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlType.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlType.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlType.java
index 6ff3231..febf174 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlType.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlType.java
@@ -17,7 +17,10 @@
 
 package org.apache.ignite.internal.processors.query.h2.sql;
 
+import org.apache.ignite.internal.util.typedef.F;
 import org.apache.ignite.internal.util.typedef.internal.S;
+import org.h2.expression.Expression;
+import org.h2.table.Column;
 import org.h2.value.Value;
 import org.h2.value.ValueDouble;
 import org.h2.value.ValueLong;
@@ -62,7 +65,9 @@ public final class GridSqlType {
      * @param displaySize Display size.
      * @param sql SQL definition of the type.
      */
-    public GridSqlType(int type, int scale, long precision, int displaySize, String sql) {
+    private GridSqlType(int type, int scale, long precision, int displaySize, String sql) {
+        assert !F.isEmpty(sql) || type == Value.UNKNOWN;
+
         this.type = type;
         this.scale = scale;
         this.precision = precision;
@@ -71,6 +76,28 @@ public final class GridSqlType {
     }
 
     /**
+     * @param c Column to take type definition from.
+     * @return Type.
+     */
+    public static GridSqlType fromColumn(Column c) {
+        if (c.getName() != null)
+            c = new Column(null, c.getType(), c.getPrecision(), c.getScale(), c.getDisplaySize());
+
+        return new GridSqlType(c.getType(), c.getScale(), c.getPrecision(), c.getDisplaySize(), c.getCreateSQL());
+    }
+
+    /**
+     * @param e Expression to take type from.
+     * @return Type.
+     */
+    public static GridSqlType fromExpression(Expression e) {
+        if (e.getType() == Value.UNKNOWN)
+            return UNKNOWN;
+
+        return fromColumn(new Column(null, e.getType(), e.getPrecision(), e.getScale(), e.getDisplaySize()));
+    }
+
+    /**
      * @return Get H2 type.
      */
     public int type() {

http://git-wip-us.apache.org/repos/asf/ignite/blob/74029187/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/sql/GridQueryParsingTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/sql/GridQueryParsingTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/sql/GridQueryParsingTest.java
index f969a48..2be5d1a 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/sql/GridQueryParsingTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/sql/GridQueryParsingTest.java
@@ -98,6 +98,33 @@ public class GridQueryParsingTest extends GridCommonAbstractTest {
      * @throws Exception If failed.
      */
     public void testAllExamples() throws Exception {
+        checkQuery("select 42");
+        checkQuery("select ()");
+        checkQuery("select (1)");
+        checkQuery("select (1 + 1)");
+        checkQuery("select (1,)");
+        checkQuery("select (?)");
+        checkQuery("select (?,)");
+        checkQuery("select (1, 2)");
+        checkQuery("select (?, ? + 1, 2 + 2) as z");
+        checkQuery("select (1,(1,(1,(1,(1,?)))))");
+        checkQuery("select (select 1)");
+        checkQuery("select (select 1, select ?)");
+        checkQuery("select ((select 1), select ? + ?)");
+
+        checkQuery("select extract(year from ?)");
+        checkQuery("select convert(?, timestamp)");
+
+        checkQuery("select * from table(id bigint = 1)");
+        checkQuery("select * from table(id bigint = (1))");
+        checkQuery("select * from table(id bigint = (1,))");
+        checkQuery("select * from table(id bigint = (1,), name varchar = 'asd')");
+        checkQuery("select * from table(id bigint = (1,2), name varchar = 'asd')");
+        checkQuery("select * from table(id bigint = (1,2), name varchar = ('asd',))");
+        checkQuery("select * from table(id bigint = (1,2), name varchar = ?)");
+        checkQuery("select * from table(id bigint = (1,2), name varchar = (?,))");
+        checkQuery("select * from table(id bigint = ?, name varchar = ('abc', 'def', 100, ?)) t");
+
         checkQuery("select ? limit ? offset ?");
 
         checkQuery("select cool1()");