You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by iv...@apache.org on 2021/08/09 13:22:08 UTC

[ignite] branch sql-calcite updated: IGNITE-14749 Fix multiline DDL and DML statements don't work - Fixes #9291.

This is an automated email from the ASF dual-hosted git repository.

ivandasch pushed a commit to branch sql-calcite
in repository https://gitbox.apache.org/repos/asf/ignite.git


The following commit(s) were added to refs/heads/sql-calcite by this push:
     new 2a0c9f2  IGNITE-14749 Fix multiline DDL and DML statements don't work - Fixes #9291.
2a0c9f2 is described below

commit 2a0c9f26660eace51252d97b12ada2be305110e3
Author: Ivan Daschinsky <iv...@apache.org>
AuthorDate: Mon Aug 9 16:21:14 2021 +0300

    IGNITE-14749 Fix multiline DDL and DML statements don't work - Fixes #9291.
    
    Signed-off-by: Ivan Daschinsky <iv...@apache.org>
---
 .../query/calcite/exec/ExecutionServiceImpl.java   | 175 +++++++++------------
 .../query/calcite/prepare/IgnitePlanner.java       |   3 +-
 .../query/calcite/prepare/QueryPlanCache.java      |   9 +-
 .../query/calcite/prepare/QueryPlanCacheImpl.java  |  28 ++--
 .../query/calcite/prepare/QueryPlanFactory.java    |   4 +-
 .../processors/query/calcite/util/Commons.java     |  37 +++++
 .../query/calcite/CalciteQueryProcessorTest.java   |   6 +-
 .../integration/TableDdlIntegrationTest.java       |  25 +++
 .../query/calcite/jdbc/JdbcQueryTest.java          |  25 +++
 9 files changed, 188 insertions(+), 124 deletions(-)

diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/ExecutionServiceImpl.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/ExecutionServiceImpl.java
index 323e849..9683a06 100644
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/ExecutionServiceImpl.java
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/ExecutionServiceImpl.java
@@ -18,7 +18,7 @@
 package org.apache.ignite.internal.processors.query.calcite.exec;
 
 import java.util.ArrayList;
-import java.util.Collection;
+import java.util.Collections;
 import java.util.HashSet;
 import java.util.Iterator;
 import java.util.List;
@@ -28,7 +28,6 @@ import java.util.Set;
 import java.util.UUID;
 import java.util.concurrent.ConcurrentHashMap;
 
-import com.google.common.collect.ImmutableList;
 import org.apache.calcite.plan.Context;
 import org.apache.calcite.plan.Contexts;
 import org.apache.calcite.plan.ConventionTraitDef;
@@ -36,13 +35,13 @@ import org.apache.calcite.plan.RelOptUtil;
 import org.apache.calcite.plan.RelTraitDef;
 import org.apache.calcite.rel.RelCollationTraitDef;
 import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.schema.SchemaPlus;
 import org.apache.calcite.sql.SqlDdl;
 import org.apache.calcite.sql.SqlExplain;
 import org.apache.calcite.sql.SqlExplainLevel;
 import org.apache.calcite.sql.SqlKind;
 import org.apache.calcite.sql.SqlNode;
 import org.apache.calcite.sql.SqlNodeList;
-import org.apache.calcite.sql.parser.SqlParseException;
 import org.apache.calcite.sql.type.SqlTypeName;
 import org.apache.calcite.tools.Frameworks;
 import org.apache.calcite.tools.ValidationException;
@@ -113,7 +112,6 @@ import org.apache.ignite.internal.util.typedef.F;
 import org.apache.ignite.internal.util.typedef.T2;
 import org.apache.ignite.internal.util.typedef.X;
 import org.apache.ignite.internal.util.typedef.internal.U;
-import org.jetbrains.annotations.NotNull;
 import org.jetbrains.annotations.Nullable;
 
 import static java.util.Collections.singletonList;
@@ -386,31 +384,28 @@ public class ExecutionServiceImpl<Row> extends AbstractService implements Execut
         String qry,
         Object[] params
     ) {
-        PlanningContext pctx = createContext(Commons.convert(ctx), topologyVersion(), localNodeId(), schema, qry, params);
+        QueryPlan plan = queryPlanCache().queryPlan(new CacheKey(getDefaultSchema(schema).getName(), qry));
+        if (plan != null) {
+            PlanningContext pctx = createContext(ctx, schema, qry, params);
 
-        List<QueryPlan> qryPlans = queryPlanCache().queryPlan(pctx, new CacheKey(pctx.schemaName(), pctx.query()), this::prepareQuery);
-
-        return executePlans(qryPlans, pctx);
-    }
+            return Collections.singletonList(executePlan(UUID.randomUUID(), pctx, plan));
+        }
 
-    /**
-     * Executes prepared plans.
-     * @param qryPlans Query plans.
-     * @param pctx Query context.
-     * @return List of query result cursors.
-     */
-    @NotNull public List<FieldsQueryCursor<List<?>>> executePlans(
-        Collection<QueryPlan> qryPlans,
-        PlanningContext pctx
-    ) {
-        List<FieldsQueryCursor<List<?>>> cursors = new ArrayList<>(qryPlans.size());
+        SqlNodeList qryList = Commons.parse(qry, FRAMEWORK_CONFIG.getParserConfig());
+        List<FieldsQueryCursor<List<?>>> cursors = new ArrayList<>(qryList.size());
 
-        for (QueryPlan plan : qryPlans) {
-            UUID qryId = UUID.randomUUID();
+        for (final SqlNode qry0: qryList) {
+            PlanningContext pctx = createContext(ctx, schema, qry0.toString(), params);
 
-            FieldsQueryCursor<List<?>> cur = executePlan(qryId, pctx, plan);
+            if (qryList.size() == 1) {
+                plan = queryPlanCache().queryPlan(
+                    pctx, new CacheKey(pctx.schemaName(), pctx.query()),
+                    pctx0 -> prepareSingle(qry0, pctx0));
+            }
+            else
+                plan = prepareSingle(qry0, pctx);
 
-            cursors.add(cur);
+            cursors.add(executePlan(UUID.randomUUID(), pctx, plan));
         }
 
         return cursors;
@@ -473,6 +468,11 @@ public class ExecutionServiceImpl<Row> extends AbstractService implements Execut
     }
 
     /** */
+    private PlanningContext createContext(QueryContext ctx, @Nullable String schema, String qry, Object[] params) {
+        return createContext(Commons.convert(ctx), topologyVersion(), localNodeId(), schema, qry, params);
+    }
+
+    /** */
     private PlanningContext createContext(Context parent, AffinityTopologyVersion topVer, UUID originator,
         @Nullable String schema, String qry, Object[] params) {
         RelTraitDef<?>[] traitDefs = {
@@ -488,9 +488,7 @@ public class ExecutionServiceImpl<Row> extends AbstractService implements Execut
             .originatingNodeId(originator)
             .parentContext(parent)
             .frameworkConfig(Frameworks.newConfigBuilder(FRAMEWORK_CONFIG)
-                .defaultSchema(schema != null
-                    ? schemaHolder().schema().getSubSchema(schema)
-                    : schemaHolder().schema())
+                .defaultSchema(getDefaultSchema(schema))
                 .traitDefs(traitDefs)
                 .build())
             .query(qry)
@@ -501,76 +499,51 @@ public class ExecutionServiceImpl<Row> extends AbstractService implements Execut
     }
 
     /** */
-    private List<QueryPlan> prepareQuery(PlanningContext ctx) {
-        try {
-            String qry = ctx.query();
-
-            assert qry != null;
-
-            // Parse query.
-            SqlNode sqlNode = ctx.planner().parse(qry);
-
-            if (single(sqlNode))
-                return singletonList(prepareSingle(sqlNode, ctx));
-
-            List<SqlNode> nodes = ((SqlNodeList) sqlNode).getList();
-            List<QueryPlan> res = new ArrayList<>(nodes.size());
-
-            for (SqlNode node : nodes)
-                res.add(prepareSingle(node, ctx));
-
-            return res;
-        }
-        catch (IgniteSQLException e) {
-            throw e;
-        }
-        catch (SqlParseException e) {
-            throw new IgniteSQLException("Failed to parse query.", IgniteQueryErrorCode.PARSING, e);
-        }
-        catch (ValidationException e) {
-            throw new IgniteSQLException("Failed to validate query.", IgniteQueryErrorCode.PARSING, e);
-        }
-        catch (Exception e) {
-            throw new IgniteSQLException("Failed to plan query.", IgniteQueryErrorCode.UNKNOWN, e);
-        }
+    private SchemaPlus getDefaultSchema(String schema) {
+        return schema != null ? schemaHolder().schema().getSubSchema(schema) : schemaHolder().schema();
     }
 
     /** */
-    private List<QueryPlan> prepareFragment(PlanningContext ctx) {
-        return ImmutableList.of(new FragmentPlan(fromJson(ctx, ctx.query())));
+    private QueryPlan prepareFragment(PlanningContext ctx) {
+        return new FragmentPlan(fromJson(ctx, ctx.query()));
     }
 
     /** */
-    private QueryPlan prepareSingle(SqlNode sqlNode, PlanningContext ctx) throws ValidationException {
-        assert single(sqlNode);
-
-        ctx.planner().reset();
-
-        if (SqlKind.DDL.contains(sqlNode.getKind()))
-            return prepareDdl(sqlNode, ctx);
-
-        switch (sqlNode.getKind()) {
-            case SELECT:
-            case ORDER_BY:
-            case WITH:
-            case VALUES:
-            case UNION:
-            case EXCEPT:
-            case INTERSECT:
-                return prepareQuery(sqlNode, ctx);
-
-            case INSERT:
-            case DELETE:
-            case UPDATE:
-                return prepareDml(sqlNode, ctx);
-
-            case EXPLAIN:
-                return prepareExplain(sqlNode, ctx);
-
-            default:
-                throw new IgniteSQLException("Unsupported operation [" +
-                    "sqlNodeKind=" + sqlNode.getKind() + "; " +
-                    "querySql=\"" + ctx.query() + "\"]", IgniteQueryErrorCode.UNSUPPORTED_OPERATION);
+    private QueryPlan prepareSingle(SqlNode sqlNode, PlanningContext ctx) {
+        try {
+            assert single(sqlNode);
+
+            ctx.planner().reset();
+
+            if (SqlKind.DDL.contains(sqlNode.getKind()))
+                return prepareDdl(sqlNode, ctx);
+
+            switch (sqlNode.getKind()) {
+                case SELECT:
+                case ORDER_BY:
+                case WITH:
+                case VALUES:
+                case UNION:
+                case EXCEPT:
+                case INTERSECT:
+                    return prepareQuery(sqlNode, ctx);
+
+                case INSERT:
+                case DELETE:
+                case UPDATE:
+                    return prepareDml(sqlNode, ctx);
+
+                case EXPLAIN:
+                    return prepareExplain(sqlNode, ctx);
+
+                default:
+                    throw new IgniteSQLException("Unsupported operation [" +
+                        "sqlNodeKind=" + sqlNode.getKind() + "; " +
+                        "querySql=\"" + ctx.query() + "\"]", IgniteQueryErrorCode.UNSUPPORTED_OPERATION);
+            }
+        }
+        catch (ValidationException e) {
+            throw new IgniteSQLException("Failed to validate query.", IgniteQueryErrorCode.PARSING, e);
         }
     }
 
@@ -647,10 +620,16 @@ public class ExecutionServiceImpl<Row> extends AbstractService implements Execut
     }
 
     /** */
-    private FieldsQueryCursor<List<?>> executePlan(UUID qryId, PlanningContext pctx, QueryPlan plan) {
+    private FieldsQueryCursor<List<?>> executePlan(
+        UUID qryId,
+        PlanningContext pctx,
+        QueryPlan plan
+    ) {
         switch (plan.type()) {
             case DML:
-                // TODO a barrier between previous operation and this one
+                ListFieldsQueryCursor<?> cur = executeQuery(qryId, (MultiStepPlan)plan, pctx);
+                cur.iterator().hasNext();
+                return cur;
             case QUERY:
                 return executeQuery(qryId, (MultiStepPlan) plan, pctx);
             case EXPLAIN:
@@ -677,7 +656,7 @@ public class ExecutionServiceImpl<Row> extends AbstractService implements Execut
     }
 
     /** */
-    private FieldsQueryCursor<List<?>> executeQuery(UUID qryId, MultiStepPlan plan, PlanningContext pctx) {
+    private ListFieldsQueryCursor<?> executeQuery(UUID qryId, MultiStepPlan plan, PlanningContext pctx) {
         plan.init(pctx);
 
         List<Fragment> fragments = plan.fragments();
@@ -833,17 +812,15 @@ public class ExecutionServiceImpl<Row> extends AbstractService implements Execut
             PlanningContext pctx = createContext(Contexts.empty(), msg.topologyVersion(), nodeId, msg.schema(),
                 msg.root(), msg.parameters());
 
-            List<QueryPlan> qryPlans = queryPlanCache().queryPlan(
+            QueryPlan qryPlan = queryPlanCache().queryPlan(
                 pctx,
                 new CacheKey(pctx.schemaName(), pctx.query()),
                 this::prepareFragment
             );
 
-            assert qryPlans.size() == 1 && qryPlans.get(0).type() == QueryPlan.Type.FRAGMENT;
-
-            FragmentPlan plan = (FragmentPlan)qryPlans.get(0);
+            assert qryPlan.type() == QueryPlan.Type.FRAGMENT;
 
-            executeFragment(msg.queryId(), plan, pctx, msg.fragmentDescription());
+            executeFragment(msg.queryId(), (FragmentPlan)qryPlan, pctx, msg.fragmentDescription());
         }
         catch (Throwable ex) {
             U.error(log, "Failed to start query fragment ", ex);
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/IgnitePlanner.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/IgnitePlanner.java
index ed79642..1179634 100644
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/IgnitePlanner.java
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/IgnitePlanner.java
@@ -66,6 +66,7 @@ import org.apache.ignite.internal.processors.query.IgniteSQLException;
 import org.apache.ignite.internal.processors.query.calcite.metadata.IgniteMetadata;
 import org.apache.ignite.internal.processors.query.calcite.metadata.RelMetadataQueryEx;
 import org.apache.ignite.internal.processors.query.calcite.type.IgniteTypeFactory;
+import org.apache.ignite.internal.processors.query.calcite.util.Commons;
 
 /**
  * Query planer.
@@ -161,7 +162,7 @@ public class IgnitePlanner implements Planner, RelOptTable.ViewExpander {
 
     /** {@inheritDoc} */
     @Override public SqlNode parse(Reader reader) throws SqlParseException {
-        SqlNodeList sqlNodes = SqlParser.create(reader, parserCfg).parseStmtList();
+        SqlNodeList sqlNodes = Commons.parse(reader, parserCfg);
 
         return sqlNodes.size() == 1 ? sqlNodes.get(0) : sqlNodes;
     }
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/QueryPlanCache.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/QueryPlanCache.java
index 60c2fdd..e249c3b 100644
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/QueryPlanCache.java
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/QueryPlanCache.java
@@ -17,7 +17,6 @@
 
 package org.apache.ignite.internal.processors.query.calcite.prepare;
 
-import java.util.List;
 import org.apache.ignite.internal.processors.query.calcite.util.Service;
 
 /**
@@ -30,7 +29,13 @@ public interface QueryPlanCache extends Service {
      * @param factory Factory method to generate a plan on cache miss.
      * @return Query plan.
      */
-    List<QueryPlan> queryPlan(PlanningContext ctx, CacheKey key, QueryPlanFactory factory);
+    QueryPlan queryPlan(PlanningContext ctx, CacheKey key, QueryPlanFactory factory);
+
+    /**
+     * @param key Cache key.
+     * @return Query plan.
+     */
+    QueryPlan queryPlan(CacheKey key);
 
     /**
      * Clear cache.
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/QueryPlanCacheImpl.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/QueryPlanCacheImpl.java
index b709d12..3d9a90e 100644
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/QueryPlanCacheImpl.java
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/QueryPlanCacheImpl.java
@@ -17,7 +17,6 @@
 
 package org.apache.ignite.internal.processors.query.calcite.prepare;
 
-import java.util.List;
 import java.util.Map;
 import org.apache.ignite.internal.GridKernalContext;
 import org.apache.ignite.internal.cache.query.index.Index;
@@ -25,7 +24,6 @@ import org.apache.ignite.internal.processors.cache.GridCacheContextInfo;
 import org.apache.ignite.internal.processors.query.GridQueryIndexDescriptor;
 import org.apache.ignite.internal.processors.query.GridQueryTypeDescriptor;
 import org.apache.ignite.internal.processors.query.calcite.util.AbstractService;
-import org.apache.ignite.internal.processors.query.calcite.util.Commons;
 import org.apache.ignite.internal.processors.query.schema.SchemaChangeListener;
 import org.apache.ignite.internal.processors.subscription.GridInternalSubscriptionProcessor;
 import org.apache.ignite.internal.util.GridBoundedConcurrentLinkedHashMap;
@@ -41,7 +39,7 @@ public class QueryPlanCacheImpl extends AbstractService implements QueryPlanCach
     private GridInternalSubscriptionProcessor subscriptionProcessor;
 
     /** */
-    private volatile Map<CacheKey, List<QueryPlan>> cache;
+    private volatile Map<CacheKey, QueryPlan> cache;
 
     /**
      * @param ctx Kernal context.
@@ -73,21 +71,17 @@ public class QueryPlanCacheImpl extends AbstractService implements QueryPlanCach
     }
 
     /** {@inheritDoc} */
-    @Override public List<QueryPlan> queryPlan(PlanningContext ctx, CacheKey key, QueryPlanFactory factory) {
-        Map<CacheKey, List<QueryPlan>> cache = this.cache;
-
-        List<QueryPlan> template = cache.get(key);
-
-        if (template != null)
-            return Commons.transform(template, QueryPlan::copy);
-        else {
-            List<QueryPlan> prepared = factory.create(ctx);
-
-            if (prepared.size() == 1) // do not cache multiline queries.
-                cache.putIfAbsent(key, prepared);
+    @Override public QueryPlan queryPlan(PlanningContext ctx, CacheKey key, QueryPlanFactory factory) {
+        Map<CacheKey, QueryPlan> cache = this.cache;
+        QueryPlan plan = cache.computeIfAbsent(key, k -> factory.create(ctx));
+        return plan.copy();
+    }
 
-            return Commons.transform(prepared, QueryPlan::copy);
-        }
+    /** {@inheritDoc} */
+    @Override public QueryPlan queryPlan(CacheKey key) {
+        Map<CacheKey, QueryPlan> cache = this.cache;
+        QueryPlan plan = cache.get(key);
+        return plan != null ? plan.copy() : null;
     }
 
     /** {@inheritDoc} */
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/QueryPlanFactory.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/QueryPlanFactory.java
index f3391d3..91dc8c5 100644
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/QueryPlanFactory.java
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/QueryPlanFactory.java
@@ -17,8 +17,6 @@
 
 package org.apache.ignite.internal.processors.query.calcite.prepare;
 
-import java.util.List;
-
 /**
  *
  */
@@ -27,5 +25,5 @@ public interface QueryPlanFactory {
      * @param ctx Planning context.
      * @return Query plan.
      */
-    List<QueryPlan> create(PlanningContext ctx);
+    QueryPlan create(PlanningContext ctx);
 }
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/util/Commons.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/util/Commons.java
index 1ae1f2d..0b90f2d 100644
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/util/Commons.java
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/util/Commons.java
@@ -17,6 +17,7 @@
 
 package org.apache.ignite.internal.processors.query.calcite.util;
 
+import java.io.Reader;
 import java.io.StringReader;
 import java.util.ArrayList;
 import java.util.Arrays;
@@ -38,8 +39,12 @@ import org.apache.calcite.plan.Context;
 import org.apache.calcite.plan.Contexts;
 import org.apache.calcite.plan.RelOptCluster;
 import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.sql.SqlNodeList;
+import org.apache.calcite.sql.parser.SqlParseException;
+import org.apache.calcite.sql.parser.SqlParser;
 import org.apache.calcite.util.ImmutableBitSet;
 import org.apache.calcite.util.ImmutableIntList;
+import org.apache.calcite.util.SourceStringReader;
 import org.apache.calcite.util.Util;
 import org.apache.calcite.util.mapping.Mapping;
 import org.apache.calcite.util.mapping.MappingType;
@@ -48,6 +53,8 @@ import org.apache.ignite.IgniteException;
 import org.apache.ignite.IgniteLogger;
 import org.apache.ignite.internal.GridComponent;
 import org.apache.ignite.internal.GridKernalContext;
+import org.apache.ignite.internal.processors.cache.query.IgniteQueryErrorCode;
+import org.apache.ignite.internal.processors.query.IgniteSQLException;
 import org.apache.ignite.internal.processors.query.QueryContext;
 import org.apache.ignite.internal.processors.query.calcite.exec.RowHandler;
 import org.apache.ignite.internal.processors.query.calcite.exec.exp.ExpressionFactoryImpl;
@@ -188,6 +195,36 @@ public final class Commons {
     }
 
     /**
+     * Parses a SQL statement.
+     *
+     * @param qry Query string.
+     * @param parserCfg Parser config.
+     * @return Parsed query.
+     */
+    public static SqlNodeList parse(String qry, SqlParser.Config parserCfg) {
+        try {
+            return parse(new SourceStringReader(qry), parserCfg);
+        }
+        catch (SqlParseException e) {
+            throw new IgniteSQLException("Failed to parse query.", IgniteQueryErrorCode.PARSING, e);
+        }
+    }
+
+    /**
+     * Parses a SQL statement.
+     *
+     * @param reader Source string reader.
+     * @param parserCfg Parser config.
+     * @return Parsed query.
+     * @throws org.apache.calcite.sql.parser.SqlParseException on parse error.
+     */
+    public static SqlNodeList parse(Reader reader, SqlParser.Config parserCfg) throws SqlParseException {
+        SqlParser parser = SqlParser.create(reader, parserCfg);
+
+        return parser.parseStmtList();
+    }
+
+    /**
      * @param params Parameters.
      * @return Parameters map.
      */
diff --git a/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/CalciteQueryProcessorTest.java b/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/CalciteQueryProcessorTest.java
index 5b95d37..97664de 100644
--- a/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/CalciteQueryProcessorTest.java
+++ b/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/CalciteQueryProcessorTest.java
@@ -855,12 +855,14 @@ public class CalciteQueryProcessorTest extends GridCommonAbstractTest {
         List<FieldsQueryCursor<List<?>>> query = engine.query(null, "PUBLIC",
             "" +
                 "select * from DEVELOPER d, PROJECT p where d.projectId = p._key and d._key = ?;" +
+                "select * from DEVELOPER d, PROJECT p where d.projectId = p._key and d._key = 10;" +
                 "select * from DEVELOPER d, PROJECT p where d.projectId = p._key and d._key = ?", 0, 1);
 
-        assertEquals(2, query.size());
+        assertEquals(3, query.size());
 
         assertEqualsCollections(Arrays.asList("Igor", 1, "Calcite"), F.first(query.get(0).getAll()));
-        assertEqualsCollections(Arrays.asList("Roman", 0, "Ignite"), F.first(query.get(1).getAll()));
+        assertEquals(0, query.get(1).getAll().size());
+        assertEqualsCollections(Arrays.asList("Roman", 0, "Ignite"), F.first(query.get(2).getAll()));
     }
 
     /**
diff --git a/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/integration/TableDdlIntegrationTest.java b/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/integration/TableDdlIntegrationTest.java
index 9fb0c2c..ac81cae 100644
--- a/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/integration/TableDdlIntegrationTest.java
+++ b/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/integration/TableDdlIntegrationTest.java
@@ -657,6 +657,31 @@ public class TableDdlIntegrationTest extends AbstractDdlIntegrationTest {
     }
 
     /**
+     * Tests that multiline statements with DDL and DML works as expected.
+     */
+    @Test
+    public void testMulitlineWithCreateTable() {
+        String multiLineQuery = "CREATE TABLE test (val0 int primary key, val1 varchar);" +
+            "INSERT INTO test(val0, val1) VALUES (0, 'test0');" +
+            "ALTER TABLE test ADD COLUMN val2 int;" +
+            "INSERT INTO test(val0, val1, val2) VALUES(1, 'test1', 10);" +
+            "ALTER TABLE test DROP COLUMN val2;";
+
+        executeSql(multiLineQuery);
+
+        List<List<?>> res = executeSql("SELECT * FROM test order by val0");
+        assertEquals(2, res.size());
+
+        for (int i = 0; i < res.size(); i++) {
+            List<?> row = res.get(i);
+
+            assertEquals(2, row.size());
+            assertEquals(i, row.get(0));
+            assertEquals("test" + i, row.get(1));
+        }
+    }
+
+    /**
      * Asserts that executeSql throws an exception.
      *
      * @param sql Query.
diff --git a/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/jdbc/JdbcQueryTest.java b/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/jdbc/JdbcQueryTest.java
index c9efa83..270c6a8 100644
--- a/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/jdbc/JdbcQueryTest.java
+++ b/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/jdbc/JdbcQueryTest.java
@@ -103,6 +103,31 @@ public class JdbcQueryTest extends GridCommonAbstractTest {
      * @throws SQLException If failed.
      */
     @Test
+    public void testMultilineQuery() throws Exception {
+        String multiLineQuery = "CREATE TABLE test (val0 int primary key, val1 varchar);" +
+            "INSERT INTO test(val0, val1) VALUES (0, 'test0');" +
+            "ALTER TABLE test ADD COLUMN val2 int;" +
+            "INSERT INTO test(val0, val1, val2) VALUES(1, 'test1', 10);" +
+            "ALTER TABLE test DROP COLUMN val2;";
+        stmt.execute(multiLineQuery);
+
+        try (ResultSet rs = stmt.executeQuery("select * from test order by val0")) {
+            int i;
+            for (i = 0; rs.next(); i++) {
+                assertEquals(i, rs.getInt(1));
+                assertEquals("test" + i, rs.getString(2));
+            }
+            assertEquals(2, i);
+        }
+
+        stmt.execute("drop table test");
+        stmt.close();
+    }
+
+    /**
+     * @throws SQLException If failed.
+     */
+    @Test
     public void testQueryColumnTypes() throws Exception {
         stmt.execute("CREATE TABLE t1 (id INT NOT NULL, " +
             "bool_col BOOLEAN, " +