You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by am...@apache.org on 2019/06/14 09:31:39 UTC

[ignite] branch master updated: IGNITE-11910: JDBC v2: adds 'schema' to URL parameters, makes 'cache' parameter is optional for the most cases. This closes #6609.

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

amashenkov pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/ignite.git


The following commit(s) were added to refs/heads/master by this push:
     new 7097512  IGNITE-11910: JDBC v2: adds 'schema' to URL parameters, makes 'cache' parameter is optional for the most cases. This closes #6609.
7097512 is described below

commit 70975124397fb85ad5c572ee9d4c2aafd25e2b2a
Author: tledkov <tl...@gridgain.com>
AuthorDate: Fri Jun 14 12:30:57 2019 +0300

    IGNITE-11910: JDBC v2: adds 'schema' to URL parameters, makes 'cache' parameter is optional for the most cases. This closes #6609.
---
 .../jdbc2/JdbcConnectionWithoutCacheNameTest.java  | 125 ++++++++++++++++++++
 .../internal/jdbc2/JdbcNoDefaultCacheTest.java     | 130 ---------------------
 .../jdbc/suite/IgniteJdbcDriverTestSuite.java      |   4 +-
 .../java/org/apache/ignite/IgniteJdbcDriver.java   |   9 +-
 .../ignite/internal/jdbc2/JdbcConnection.java      |  34 +++++-
 .../JdbcQueryMultipleStatementsNotAllowTask.java   |  56 +++++++++
 .../jdbc2/JdbcQueryMultipleStatementsTask.java     |  18 ++-
 .../ignite/internal/jdbc2/JdbcStatement.java       |  16 ++-
 8 files changed, 251 insertions(+), 141 deletions(-)

diff --git a/modules/clients/src/test/java/org/apache/ignite/internal/jdbc2/JdbcConnectionWithoutCacheNameTest.java b/modules/clients/src/test/java/org/apache/ignite/internal/jdbc2/JdbcConnectionWithoutCacheNameTest.java
new file mode 100644
index 0000000..ec72d9c
--- /dev/null
+++ b/modules/clients/src/test/java/org/apache/ignite/internal/jdbc2/JdbcConnectionWithoutCacheNameTest.java
@@ -0,0 +1,125 @@
+/*
+ * 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.jdbc2;
+
+import java.sql.Connection;
+import java.sql.DriverManager;
+import java.sql.SQLException;
+import java.sql.Statement;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.configuration.ConnectorConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.testframework.GridTestUtils;
+import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+import org.junit.Test;
+
+import static org.apache.ignite.IgniteJdbcDriver.CFG_URL_PREFIX;
+import static org.apache.ignite.cache.CacheMode.PARTITIONED;
+import static org.apache.ignite.cache.CacheWriteSynchronizationMode.FULL_SYNC;
+
+/**
+ * Test connection without cache name.
+ */
+public class JdbcConnectionWithoutCacheNameTest extends GridCommonAbstractTest {
+    /** Ignite config path for JDBC v2 client. */
+    private static final String CFG_PATH = "modules/clients/src/test/config/jdbc-config.xml";
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(igniteInstanceName);
+
+        CacheConfiguration<?, ?> cache = defaultCacheConfiguration();
+
+        cache.setCacheMode(PARTITIONED);
+        cache.setBackups(1);
+        cache.setWriteSynchronizationMode(FULL_SYNC);
+        cache.setSqlSchema("\"default\"");
+
+        cfg.setCacheConfiguration(cache);
+
+        cfg.setConnectorConfiguration(new ConnectorConfiguration());
+
+        return cfg;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTest() throws Exception {
+        stopAllGrids();
+
+        super.afterTest();
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTest() throws Exception {
+        super.beforeTest();
+
+        startGrid("srv");
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    @Test
+    public void testWithoutCache() throws Exception {
+        try (Connection c = DriverManager.getConnection(CFG_URL_PREFIX + CFG_PATH)) {
+            try (Statement stmt = c.createStatement()) {
+                stmt.execute("CREATE TABLE TEST(ID INT PRIMARY KEY, VAL VARCHAR)");
+                stmt.execute("INSERT INTO TEST VALUES(1, '1')");
+                stmt.execute("DROP TABLE TEST ");
+            }
+        }
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    @Test
+    public void testSchemaParameter() throws Exception {
+        try (Connection c = DriverManager.getConnection(CFG_URL_PREFIX + "schema=\"default\"@" + CFG_PATH)) {
+            try (Statement stmt = c.createStatement()) {
+                stmt.execute("CREATE TABLE TEST(ID INT PRIMARY KEY, VAL VARCHAR)");
+                stmt.execute("INSERT INTO TEST VALUES(1, '1')");
+            }
+        }
+
+        try (Connection c = DriverManager.getConnection(CFG_URL_PREFIX + CFG_PATH)) {
+            try (Statement stmt = c.createStatement()) {
+                stmt.execute("DROP TABLE \"default\".TEST ");
+            }
+        }
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    @Test
+    public void testFailOnMultipleStatements() throws Exception {
+        try (Connection c = DriverManager.getConnection(CFG_URL_PREFIX + CFG_PATH)) {
+            try (Statement stmt = c.createStatement()) {
+                stmt.execute("CREATE TABLE TEST(ID INT PRIMARY KEY, VAL VARCHAR)");
+
+                GridTestUtils.assertThrows(log, () -> {
+                    stmt.execute("INSERT INTO TEST VALUES(0, '0'); " +
+                        "INSERT INTO TEST VALUES(1, '1');");
+
+                    return null;
+                }, SQLException.class, "Multiple statements queries are not supported.");
+            }
+        }
+    }
+}
\ No newline at end of file
diff --git a/modules/clients/src/test/java/org/apache/ignite/internal/jdbc2/JdbcNoDefaultCacheTest.java b/modules/clients/src/test/java/org/apache/ignite/internal/jdbc2/JdbcNoDefaultCacheTest.java
deleted file mode 100644
index 36590a0..0000000
--- a/modules/clients/src/test/java/org/apache/ignite/internal/jdbc2/JdbcNoDefaultCacheTest.java
+++ /dev/null
@@ -1,130 +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.
- */
-
-package org.apache.ignite.internal.jdbc2;
-
-import java.sql.Connection;
-import java.sql.DriverManager;
-import java.sql.SQLException;
-import java.sql.Statement;
-import java.util.concurrent.Callable;
-import org.apache.ignite.Ignite;
-import org.apache.ignite.IgniteCache;
-import org.apache.ignite.configuration.CacheConfiguration;
-import org.apache.ignite.configuration.IgniteConfiguration;
-import org.apache.ignite.testframework.GridTestUtils;
-import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
-import org.jetbrains.annotations.NotNull;
-import org.junit.Test;
-
-import static org.apache.ignite.IgniteJdbcDriver.CFG_URL_PREFIX;
-
-/**
- *
- */
-public class JdbcNoDefaultCacheTest extends GridCommonAbstractTest {
-    /** First cache name. */
-    private static final String CACHE1_NAME = "cache1";
-
-    /** Second cache name. */
-    private static final String CACHE2_NAME = "cache2";
-
-    /** Ignite configuration URL. */
-    private static final String CFG_URL = "modules/clients/src/test/config/jdbc-config.xml";
-
-    /** Grid count. */
-    private static final int GRID_CNT = 2;
-
-    /** {@inheritDoc} */
-    @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception {
-        IgniteConfiguration cfg = super.getConfiguration(igniteInstanceName);
-
-        cfg.setCacheConfiguration(cacheConfiguration(CACHE1_NAME), cacheConfiguration(CACHE2_NAME));
-
-        return cfg;
-    }
-
-    /**
-     * @param name Cache name.
-     * @return Cache configuration.
-     * @throws Exception In case of error.
-     */
-    private CacheConfiguration cacheConfiguration(@NotNull String name) throws Exception {
-        CacheConfiguration cfg = defaultCacheConfiguration();
-
-        cfg.setIndexedTypes(Integer.class, Integer.class);
-
-        cfg.setName(name);
-
-        return cfg;
-    }
-
-    /** {@inheritDoc} */
-    @Override protected void beforeTestsStarted() throws Exception {
-        startGridsMultiThreaded(GRID_CNT);
-
-        Ignite ignite = ignite(0);
-
-        IgniteCache<Integer, Integer> cache1 = ignite.cache(CACHE1_NAME);
-        IgniteCache<Integer, Integer> cache2 = ignite.cache(CACHE2_NAME);
-
-        for (int i = 0; i < 10; i++) {
-            cache1.put(i, i * 2);
-            cache2.put(i, i * 3);
-        }
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    @Test
-    public void testDefaults() throws Exception {
-        String url = CFG_URL_PREFIX + CFG_URL;
-
-        try (Connection conn = DriverManager.getConnection(url)) {
-            assertNotNull(conn);
-            assertTrue(((JdbcConnection)conn).ignite().configuration().isClientMode());
-        }
-
-        try (Connection conn = DriverManager.getConnection(url + '/')) {
-            assertNotNull(conn);
-            assertTrue(((JdbcConnection)conn).ignite().configuration().isClientMode());
-        }
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    @Test
-    public void testNoCacheNameQuery() throws Exception {
-        try (
-            Connection conn = DriverManager.getConnection(CFG_URL_PREFIX + CFG_URL);
-            final Statement stmt = conn.createStatement()) {
-            assertNotNull(stmt);
-            assertFalse(stmt.isClosed());
-
-            Throwable throwable = GridTestUtils.assertThrows(null, new Callable<Void>() {
-                @Override public Void call() throws Exception {
-                    stmt.execute("select t._key, t._val from \"cache1\".Integer t");
-                    return null;
-                }
-            }, SQLException.class, "Failed to query Ignite.");
-
-            assertEquals(throwable.getCause().getMessage(), "Ouch! Argument is invalid: Cache name must not be null or empty.");
-        }
-    }
-}
diff --git a/modules/clients/src/test/java/org/apache/ignite/jdbc/suite/IgniteJdbcDriverTestSuite.java b/modules/clients/src/test/java/org/apache/ignite/jdbc/suite/IgniteJdbcDriverTestSuite.java
index 32ae2d5..62ec8af 100644
--- a/modules/clients/src/test/java/org/apache/ignite/jdbc/suite/IgniteJdbcDriverTestSuite.java
+++ b/modules/clients/src/test/java/org/apache/ignite/jdbc/suite/IgniteJdbcDriverTestSuite.java
@@ -82,9 +82,9 @@ import org.apache.ignite.jdbc.thin.JdbcThinSelectAfterAlterTable;
 import org.apache.ignite.jdbc.thin.JdbcThinStatementCancelSelfTest;
 import org.apache.ignite.jdbc.thin.JdbcThinStatementSelfTest;
 import org.apache.ignite.jdbc.thin.JdbcThinStatementTimeoutSelfTest;
-import org.apache.ignite.jdbc.thin.JdbcThinStreamingResetStreamTest;
 import org.apache.ignite.jdbc.thin.JdbcThinStreamingNotOrderedSelfTest;
 import org.apache.ignite.jdbc.thin.JdbcThinStreamingOrderedSelfTest;
+import org.apache.ignite.jdbc.thin.JdbcThinStreamingResetStreamTest;
 import org.apache.ignite.jdbc.thin.JdbcThinTcpIoTest;
 import org.apache.ignite.jdbc.thin.JdbcThinTransactionsClientAutoCommitComplexSelfTest;
 import org.apache.ignite.jdbc.thin.JdbcThinTransactionsClientNoAutoCommitComplexSelfTest;
@@ -128,7 +128,7 @@ import org.junit.runners.Suite;
     org.apache.ignite.internal.jdbc2.JdbcMetadataSelfTest.class,
     org.apache.ignite.internal.jdbc2.JdbcEmptyCacheSelfTest.class,
     org.apache.ignite.internal.jdbc2.JdbcLocalCachesSelfTest.class,
-    org.apache.ignite.internal.jdbc2.JdbcNoDefaultCacheTest.class,
+    org.apache.ignite.internal.jdbc2.JdbcConnectionWithoutCacheNameTest.class,
     org.apache.ignite.internal.jdbc2.JdbcMergeStatementSelfTest.class,
     org.apache.ignite.internal.jdbc2.JdbcBinaryMarshallerMergeStatementSelfTest.class,
     org.apache.ignite.internal.jdbc2.JdbcUpdateStatementSelfTest.class,
diff --git a/modules/core/src/main/java/org/apache/ignite/IgniteJdbcDriver.java b/modules/core/src/main/java/org/apache/ignite/IgniteJdbcDriver.java
index 2a146ca..61a944f 100644
--- a/modules/core/src/main/java/org/apache/ignite/IgniteJdbcDriver.java
+++ b/modules/core/src/main/java/org/apache/ignite/IgniteJdbcDriver.java
@@ -334,6 +334,9 @@ public class IgniteJdbcDriver implements Driver {
     /** Parameter: replicated only flag (SQL hint). */
     public static final String PARAM_LAZY = "lazy";
 
+    /** Parameter: schema name. */
+    public static final String PARAM_SCHEMA = "schema";
+
     /** Hostname property name. */
     public static final String PROP_HOST = PROP_PREFIX + "host";
 
@@ -385,6 +388,9 @@ public class IgniteJdbcDriver implements Driver {
     /** Lazy property name. */
     public static final String PROP_LAZY = PROP_PREFIX + PARAM_LAZY;
 
+    /** Schema property name. */
+    public static final String PROP_SCHEMA = PROP_PREFIX + PARAM_SCHEMA;
+
     /** Cache name property name. */
     public static final String PROP_CFG = PROP_PREFIX + "cfg";
 
@@ -458,7 +464,8 @@ public class IgniteJdbcDriver implements Driver {
             new JdbcDriverPropertyInfo("Lazy query execution", info.getProperty(PROP_LAZY), ""),
             new JdbcDriverPropertyInfo("Transactions Allowed", info.getProperty(PROP_TX_ALLOWED), ""),
             new JdbcDriverPropertyInfo("Queries with multiple statements allowed", info.getProperty(PROP_MULTIPLE_STMTS), ""),
-            new JdbcDriverPropertyInfo("Skip reducer on update", info.getProperty(PROP_SKIP_REDUCER_ON_UPDATE), "")
+            new JdbcDriverPropertyInfo("Skip reducer on update", info.getProperty(PROP_SKIP_REDUCER_ON_UPDATE), ""),
+            new JdbcDriverPropertyInfo("Schema name", info.getProperty(PROP_SCHEMA), "")
         );
 
         if (info.getProperty(PROP_CFG) != null)
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/jdbc2/JdbcConnection.java b/modules/core/src/main/java/org/apache/ignite/internal/jdbc2/JdbcConnection.java
index f33315c..48befd2 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/jdbc2/JdbcConnection.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/jdbc2/JdbcConnection.java
@@ -69,6 +69,7 @@ import org.apache.ignite.internal.util.typedef.F;
 import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.lang.IgniteBiTuple;
 import org.apache.ignite.lang.IgniteCallable;
+import org.apache.ignite.lang.IgniteProductVersion;
 import org.apache.ignite.resources.IgniteInstanceResource;
 
 import static java.sql.ResultSet.CONCUR_READ_ONLY;
@@ -84,6 +85,7 @@ import static org.apache.ignite.IgniteJdbcDriver.PROP_LAZY;
 import static org.apache.ignite.IgniteJdbcDriver.PROP_LOCAL;
 import static org.apache.ignite.IgniteJdbcDriver.PROP_MULTIPLE_STMTS;
 import static org.apache.ignite.IgniteJdbcDriver.PROP_NODE_ID;
+import static org.apache.ignite.IgniteJdbcDriver.PROP_SCHEMA;
 import static org.apache.ignite.IgniteJdbcDriver.PROP_SKIP_REDUCER_ON_UPDATE;
 import static org.apache.ignite.IgniteJdbcDriver.PROP_STREAMING;
 import static org.apache.ignite.IgniteJdbcDriver.PROP_STREAMING_ALLOW_OVERWRITE;
@@ -101,6 +103,14 @@ public class JdbcConnection implements Connection {
     /** Null stub. */
     private static final String NULL = "null";
 
+    /** Multiple statements supported since version. */
+    private static final IgniteProductVersion MULTIPLE_STATEMENTS_SUPPORTED_SINCE =
+        IgniteProductVersion.fromString("2.4.0");
+
+    /** Multiple statements V2 task supported since version. */
+    private static final IgniteProductVersion MULTIPLE_STATEMENTS_TASK_V2_SUPPORTED_SINCE =
+        IgniteProductVersion.fromString("2.8.0");
+
     /**
      * Ignite nodes cache.
      *
@@ -215,6 +225,7 @@ public class JdbcConnection implements Connection {
 
         multipleStmts = Boolean.parseBoolean(props.getProperty(PROP_MULTIPLE_STMTS));
         skipReducerOnUpdate = Boolean.parseBoolean(props.getProperty(PROP_SKIP_REDUCER_ON_UPDATE));
+        schemaName = QueryUtils.normalizeSchemaName(null, props.getProperty(PROP_SCHEMA));
 
         String nodeIdProp = props.getProperty(PROP_NODE_ID);
 
@@ -241,10 +252,13 @@ public class JdbcConnection implements Connection {
                         IgniteQueryErrorCode.CACHE_NOT_FOUND);
                 }
 
-                schemaName = QueryUtils.normalizeSchemaName(cacheName, cacheDesc.cacheConfiguration().getSqlSchema());
+                if (schemaName == null)
+                    schemaName = QueryUtils.normalizeSchemaName(cacheName, cacheDesc.cacheConfiguration().getSqlSchema());
+            }
+            else {
+                if (schemaName == null)
+                    schemaName = QueryUtils.DFLT_SCHEMA;
             }
-            else
-                schemaName = QueryUtils.DFLT_SCHEMA;
         }
         catch (Exception e) {
             close();
@@ -851,6 +865,20 @@ public class JdbcConnection implements Connection {
     }
 
     /**
+     * @return {@code true} if multiple statements allowed, {@code false} otherwise.
+     */
+    boolean isMultipleStatementsSupported() {
+        return U.isOldestNodeVersionAtLeast(MULTIPLE_STATEMENTS_SUPPORTED_SINCE, ignite.cluster().nodes());
+    }
+
+    /**
+     * @return {@code true} if multiple statements allowed, {@code false} otherwise.
+     */
+    boolean isMultipleStatementsTaskV2Supported() {
+        return U.isOldestNodeVersionAtLeast(MULTIPLE_STATEMENTS_TASK_V2_SUPPORTED_SINCE, ignite.cluster().nodes());
+    }
+
+    /**
      * @return {@code true} if update on server is enabled, {@code false} otherwise.
      */
     boolean skipReducerOnUpdate() {
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/jdbc2/JdbcQueryMultipleStatementsNotAllowTask.java b/modules/core/src/main/java/org/apache/ignite/internal/jdbc2/JdbcQueryMultipleStatementsNotAllowTask.java
new file mode 100644
index 0000000..6d295f6
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/jdbc2/JdbcQueryMultipleStatementsNotAllowTask.java
@@ -0,0 +1,56 @@
+/*
+ * 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.jdbc2;
+
+import org.apache.ignite.Ignite;
+import org.apache.ignite.IgniteJdbcDriver;
+
+/**
+ * Task for SQL queries execution through {@link IgniteJdbcDriver}.
+ * The query can contains several SQL statements.
+ */
+class JdbcQueryMultipleStatementsNotAllowTask extends JdbcQueryMultipleStatementsTask {
+    /** Serial version uid. */
+    private static final long serialVersionUID = 0L;
+
+    /**
+     * @param ignite Ignite.
+     * @param schemaName Schema name.
+     * @param sql Sql query.
+     * @param isQry Operation type flag - query or not - to enforce query type check.
+     * @param loc Local execution flag.
+     * @param args Args.
+     * @param fetchSize Fetch size.
+     * @param locQry Local query flag.
+     * @param collocatedQry Collocated query flag.
+     * @param distributedJoins Distributed joins flag.
+     * @param enforceJoinOrder Enforce joins order falg.
+     * @param lazy Lazy query execution flag.
+     */
+    public JdbcQueryMultipleStatementsNotAllowTask(Ignite ignite, String schemaName, String sql, Boolean isQry, boolean loc,
+        Object[] args, int fetchSize, boolean locQry, boolean collocatedQry, boolean distributedJoins,
+        boolean enforceJoinOrder, boolean lazy) {
+        super(ignite, schemaName, sql, isQry, loc, args, fetchSize, locQry, collocatedQry, distributedJoins,
+            enforceJoinOrder, lazy);
+    }
+
+    /** {@inheritDoc} */
+    @Override protected boolean allowMultipleStatements() {
+        return false;
+    }
+}
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/jdbc2/JdbcQueryMultipleStatementsTask.java b/modules/core/src/main/java/org/apache/ignite/internal/jdbc2/JdbcQueryMultipleStatementsTask.java
index 3e5d575..91af6bc 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/jdbc2/JdbcQueryMultipleStatementsTask.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/jdbc2/JdbcQueryMultipleStatementsTask.java
@@ -17,12 +17,15 @@
 
 package org.apache.ignite.internal.jdbc2;
 
+import java.sql.SQLException;
 import java.util.ArrayList;
 import java.util.List;
 import java.util.UUID;
 import org.apache.ignite.Ignite;
 import org.apache.ignite.IgniteJdbcDriver;
+import org.apache.ignite.cache.query.BulkLoadContextCursor;
 import org.apache.ignite.cache.query.FieldsQueryCursor;
+import org.apache.ignite.cache.query.QueryCursor;
 import org.apache.ignite.cache.query.SqlFieldsQuery;
 import org.apache.ignite.internal.GridKernalContext;
 import org.apache.ignite.internal.IgniteKernal;
@@ -123,11 +126,18 @@ class JdbcQueryMultipleStatementsTask implements IgniteCallable<List<JdbcStateme
 
         GridKernalContext ctx = ((IgniteKernal)ignite).context();
 
-        List<FieldsQueryCursor<List<?>>> curs = ctx.query().querySqlFields(qry, true, false);
+        List<FieldsQueryCursor<List<?>>> curs = ctx.query().querySqlFields(
+            qry, true, !allowMultipleStatements());
 
         List<JdbcStatementResultInfo> resultsInfo = new ArrayList<>(curs.size());
 
         for (FieldsQueryCursor<List<?>> cur0 : curs) {
+            if (cur0 instanceof BulkLoadContextCursor) {
+                curs.forEach(QueryCursor::close);
+
+                throw new SQLException("COPY command is currently supported only in thin JDBC driver.");
+            }
+
             QueryCursorImpl<List<?>> cur = (QueryCursorImpl<List<?>>)cur0;
 
             long updCnt = -1;
@@ -165,4 +175,10 @@ class JdbcQueryMultipleStatementsTask implements IgniteCallable<List<JdbcStateme
         return resultsInfo;
     }
 
+    /**
+     * @return {@code true} if query with multiple statements is allowed.
+     */
+    protected boolean allowMultipleStatements() {
+        return true;
+    }
 }
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/jdbc2/JdbcStatement.java b/modules/core/src/main/java/org/apache/ignite/internal/jdbc2/JdbcStatement.java
index 7b4bdb8..0c305b3 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/jdbc2/JdbcStatement.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/jdbc2/JdbcStatement.java
@@ -116,10 +116,18 @@ public class JdbcStatement implements Statement {
         UUID nodeId = conn.nodeId();
 
         boolean loc = nodeId == null;
+        JdbcQueryMultipleStatementsTask qryTask;
 
-        JdbcQueryMultipleStatementsTask qryTask = new JdbcQueryMultipleStatementsTask(loc ? ignite : null, conn.schemaName(),
-            sql, isQuery, loc, getArgs(), fetchSize, conn.isLocalQuery(), conn.isCollocatedQuery(),
-            conn.isDistributedJoins(), conn.isEnforceJoinOrder(), conn.isLazy());
+        if (!conn.isMultipleStatementsAllowed() && conn.isMultipleStatementsTaskV2Supported()) {
+            qryTask = new JdbcQueryMultipleStatementsNotAllowTask(loc ? ignite : null, conn.schemaName(),
+                sql, isQuery, loc, getArgs(), fetchSize, conn.isLocalQuery(), conn.isCollocatedQuery(),
+                conn.isDistributedJoins(), conn.isEnforceJoinOrder(), conn.isLazy());
+        }
+        else {
+            qryTask = new JdbcQueryMultipleStatementsTask(loc ? ignite : null, conn.schemaName(),
+                sql, isQuery, loc, getArgs(), fetchSize, conn.isLocalQuery(), conn.isCollocatedQuery(),
+                conn.isDistributedJoins(), conn.isEnforceJoinOrder(), conn.isLazy());
+        }
 
         try {
             List<JdbcStatementResultInfo> rsInfos =
@@ -206,7 +214,7 @@ public class JdbcStatement implements Statement {
      * @throws SQLException On error.
      */
     protected void execute0(String sql, Boolean isQuery) throws SQLException {
-        if (conn.isMultipleStatementsAllowed())
+        if (conn.isMultipleStatementsSupported())
             executeMultipleStatement(sql, isQuery);
         else
             executeSingle(sql, isQuery);