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/03/04 11:41:07 UTC

[ignite] branch master updated (094bddf -> 30a4ac63)

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

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


    from 094bddf  IGNITE-11437 Start grid in remote JVM fails in test framework if TDE is enabled (#6202)
     add b26bbb2  IGNITE-11322 [USABILITY] Extend Node FAILED message by add consistentId if it exist - Fixes #6180.
     add 6c562a9  IGNITE-11199 Add extra logging for client-server connections in TCP discovery - Fixes #6048.
     new 45c4dc9  IGNITE-10937: JDBC Thin Driver: add data page scan support. This closes #6114.
     new 70a4d69  Merge remote-tracking branch 'apache/master'
     new 30a4ac63 IGNITE-11462: Fix mvcc test GridCachePartitionedNearDisabledMvccTxMultiThreadedSelfTest.

The 3 revisions listed above as "new" are entirely new to this
repository and will be described in separate emails.  The revisions
listed as "add" were already present in the repository and have only
been added to this reference.


Summary of changes:
 .../jdbc/suite/IgniteJdbcDriverTestSuite.java      |   2 +
 .../thin/JdbcThinDataPageScanPropertySelfTest.java | 209 +++++++++++++++++++++
 .../apache/ignite/IgniteJdbcThinDataSource.java    |  17 ++
 .../internal/jdbc/thin/ConnectionProperties.java   |  13 ++
 .../jdbc/thin/ConnectionPropertiesImpl.java        |  32 +++-
 .../ignite/internal/jdbc/thin/JdbcThinTcpIo.java   |   5 +
 .../ignite/internal/jdbc/thin/JdbcThinUtils.java   |  45 ++++-
 .../odbc/jdbc/JdbcConnectionContext.java           |  11 +-
 .../processors/odbc/jdbc/JdbcRequestHandler.java   |  32 +++-
 .../processors/odbc/odbc/OdbcRequestHandler.java   |   3 +-
 .../processors/query/SqlClientContext.java         |  15 +-
 .../IgniteMvccTxMultiThreadedAbstractTest.java     |   5 -
 .../processors/cache/IgniteTxAbstractTest.java     |   6 +
 ...achePartitionedMvccTxMultiThreadedSelfTest.java |   7 +-
 14 files changed, 376 insertions(+), 26 deletions(-)
 create mode 100644 modules/clients/src/test/java/org/apache/ignite/jdbc/thin/JdbcThinDataPageScanPropertySelfTest.java


[ignite] 03/03: IGNITE-11462: Fix mvcc test GridCachePartitionedNearDisabledMvccTxMultiThreadedSelfTest.

Posted by am...@apache.org.
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

commit 30a4ac63c2955250bff0c8d12437706e75c530ac
Author: amashenkov <an...@gmail.com>
AuthorDate: Mon Mar 4 14:40:59 2019 +0300

    IGNITE-11462: Fix mvcc test GridCachePartitionedNearDisabledMvccTxMultiThreadedSelfTest.
---
 .../processors/cache/IgniteMvccTxMultiThreadedAbstractTest.java    | 5 -----
 .../ignite/internal/processors/cache/IgniteTxAbstractTest.java     | 6 ++++++
 .../near/GridCachePartitionedMvccTxMultiThreadedSelfTest.java      | 7 +++----
 3 files changed, 9 insertions(+), 9 deletions(-)

diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteMvccTxMultiThreadedAbstractTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteMvccTxMultiThreadedAbstractTest.java
index a6c0b57..bfd80dc 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteMvccTxMultiThreadedAbstractTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteMvccTxMultiThreadedAbstractTest.java
@@ -36,11 +36,6 @@ import static org.apache.ignite.transactions.TransactionIsolation.REPEATABLE_REA
  */
 @Ignore("https://issues.apache.org/jira/browse/IGNITE-9470")
 public abstract class IgniteMvccTxMultiThreadedAbstractTest extends IgniteTxAbstractTest {
-    /** {@inheritDoc} */
-    @Override protected void beforeTest() throws Exception {
-        // No-op.
-    }
-
     /**
      * @return Thread count.
      */
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteTxAbstractTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteTxAbstractTest.java
index a6ca1b7..9c33fb9 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteTxAbstractTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteTxAbstractTest.java
@@ -24,6 +24,7 @@ import java.util.Random;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.ConcurrentMap;
 import java.util.concurrent.atomic.AtomicInteger;
+import javax.cache.CacheException;
 import org.apache.ignite.Ignite;
 import org.apache.ignite.IgniteCache;
 import org.apache.ignite.IgniteCheckedException;
@@ -32,11 +33,13 @@ import org.apache.ignite.configuration.CacheConfiguration;
 import org.apache.ignite.internal.IgniteKernal;
 import org.apache.ignite.internal.util.typedef.F;
 import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.testframework.MvccFeatureChecker;
 import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
 import org.apache.ignite.transactions.Transaction;
 import org.apache.ignite.transactions.TransactionConcurrency;
 import org.apache.ignite.transactions.TransactionIsolation;
 import org.apache.ignite.transactions.TransactionOptimisticException;
+import org.apache.ignite.transactions.TransactionSerializationException;
 
 import static org.apache.ignite.transactions.TransactionConcurrency.OPTIMISTIC;
 import static org.apache.ignite.transactions.TransactionIsolation.SERIALIZABLE;
@@ -209,6 +212,9 @@ abstract class IgniteTxAbstractTest extends GridCommonAbstractTest {
                     throw e;
                 }
             }
+            catch (CacheException ex) {
+                MvccFeatureChecker.assertMvccWriteConflict(ex);
+            }
             catch (Throwable e) {
                 log.error("Unexpected error: " + e, e);
 
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedMvccTxMultiThreadedSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedMvccTxMultiThreadedSelfTest.java
index afe3e1c..022a451 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedMvccTxMultiThreadedSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedMvccTxMultiThreadedSelfTest.java
@@ -31,16 +31,15 @@ import static org.apache.ignite.cache.CacheWriteSynchronizationMode.FULL_SYNC;
  */
 public class GridCachePartitionedMvccTxMultiThreadedSelfTest extends IgniteMvccTxMultiThreadedAbstractTest {
     /** {@inheritDoc} */
-    @Override protected void beforeTest() throws Exception {
+    @Override protected void beforeTestsStarted() throws Exception {
         if (nearEnabled())
             MvccFeatureChecker.skipIfNotSupported(MvccFeatureChecker.Feature.NEAR_CACHE);
+
+        super.beforeTestsStarted();
     }
 
     /** {@inheritDoc} */
     @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception {
-        if (nearEnabled())
-            MvccFeatureChecker.skipIfNotSupported(MvccFeatureChecker.Feature.NEAR_CACHE);
-
         IgniteConfiguration cfg = super.getConfiguration(igniteInstanceName);
 
         CacheConfiguration<?, ?> ccfg = defaultCacheConfiguration();


[ignite] 02/03: Merge remote-tracking branch 'apache/master'

Posted by am...@apache.org.
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

commit 70a4d69f793c6263324426cdf05647f35bab379a
Merge: 45c4dc9 094bddf
Author: Andrey V. Mashenkov <an...@gmail.com>
AuthorDate: Mon Mar 4 14:39:29 2019 +0300

    Merge remote-tracking branch 'apache/master'

 CONTRIBUTING.md                                                       | 2 +-
 .../apache/ignite/spi/encryption/keystore/KeystoreEncryptionSpi.java  | 4 ++--
 2 files changed, 3 insertions(+), 3 deletions(-)


[ignite] 01/03: IGNITE-10937: JDBC Thin Driver: add data page scan support. This closes #6114.

Posted by am...@apache.org.
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

commit 45c4dc98e0eac33cccd2e24acb3e9882f098cad1
Author: Pavel Kuznetsov <pa...@gmail.ru>
AuthorDate: Mon Mar 4 12:29:52 2019 +0300

    IGNITE-10937: JDBC Thin Driver: add data page scan support. This closes #6114.
---
 .../jdbc/suite/IgniteJdbcDriverTestSuite.java      |   2 +
 .../thin/JdbcThinDataPageScanPropertySelfTest.java | 209 +++++++++++++++++++++
 .../apache/ignite/IgniteJdbcThinDataSource.java    |  17 ++
 .../internal/jdbc/thin/ConnectionProperties.java   |  13 ++
 .../jdbc/thin/ConnectionPropertiesImpl.java        |  32 +++-
 .../ignite/internal/jdbc/thin/JdbcThinTcpIo.java   |   5 +
 .../ignite/internal/jdbc/thin/JdbcThinUtils.java   |  45 ++++-
 .../odbc/jdbc/JdbcConnectionContext.java           |  11 +-
 .../processors/odbc/jdbc/JdbcRequestHandler.java   |  32 +++-
 .../processors/odbc/odbc/OdbcRequestHandler.java   |   3 +-
 .../processors/query/SqlClientContext.java         |  15 +-
 11 files changed, 367 insertions(+), 17 deletions(-)

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 7934ca2..bdd3700 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
@@ -52,6 +52,7 @@ import org.apache.ignite.jdbc.thin.JdbcThinConnectionMvccEnabledSelfTest;
 import org.apache.ignite.jdbc.thin.JdbcThinConnectionSSLTest;
 import org.apache.ignite.jdbc.thin.JdbcThinConnectionSelfTest;
 import org.apache.ignite.jdbc.thin.JdbcThinConnectionTimeoutSelfTest;
+import org.apache.ignite.jdbc.thin.JdbcThinDataPageScanPropertySelfTest;
 import org.apache.ignite.jdbc.thin.JdbcThinDataSourceSelfTest;
 import org.apache.ignite.jdbc.thin.JdbcThinDeleteStatementSelfTest;
 import org.apache.ignite.jdbc.thin.JdbcThinDynamicIndexAtomicPartitionedNearSelfTest;
@@ -141,6 +142,7 @@ import org.junit.runners.Suite;
     org.apache.ignite.internal.jdbc2.JdbcStreamingSelfTest.class,
     JdbcThinStreamingNotOrderedSelfTest.class,
     JdbcThinStreamingOrderedSelfTest.class,
+    JdbcThinDataPageScanPropertySelfTest.class,
 
     // DDL tests.
     org.apache.ignite.internal.jdbc2.JdbcDynamicIndexAtomicPartitionedNearSelfTest.class,
diff --git a/modules/clients/src/test/java/org/apache/ignite/jdbc/thin/JdbcThinDataPageScanPropertySelfTest.java b/modules/clients/src/test/java/org/apache/ignite/jdbc/thin/JdbcThinDataPageScanPropertySelfTest.java
new file mode 100644
index 0000000..40e1d78
--- /dev/null
+++ b/modules/clients/src/test/java/org/apache/ignite/jdbc/thin/JdbcThinDataPageScanPropertySelfTest.java
@@ -0,0 +1,209 @@
+/*
+ * 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.jdbc.thin;
+
+import java.sql.Connection;
+import java.sql.PreparedStatement;
+import java.util.List;
+import java.util.Queue;
+import java.util.concurrent.LinkedBlockingQueue;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.cache.query.FieldsQueryCursor;
+import org.apache.ignite.cache.query.SqlFieldsQuery;
+import org.apache.ignite.internal.processors.cache.mvcc.MvccQueryTracker;
+import org.apache.ignite.internal.processors.query.GridQueryCancel;
+import org.apache.ignite.internal.processors.query.GridQueryProcessor;
+import org.apache.ignite.internal.processors.query.SqlClientContext;
+import org.apache.ignite.internal.processors.query.h2.IgniteH2Indexing;
+import org.apache.ignite.testframework.GridTestUtils;
+import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+import org.jetbrains.annotations.Nullable;
+import org.junit.Before;
+import org.junit.Test;
+
+/**
+ * Check that data page scan property defined in the thin driver correctly passed to Indexing.
+ */
+public class JdbcThinDataPageScanPropertySelfTest extends GridCommonAbstractTest {
+    /** Batch size for streaming/batch mode. */
+    private static final int BATCH_SIZE = 10;
+
+    /** How many queries to execute in total in case bach/streaming mode. Should be greater than batch size. */
+    private static final int TOTAL_QUERIES_TO_EXECUTE = 25;
+
+    /** Initial size of the test table. */
+    private static final int INITIAL_ROWS_CNT = 100;
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTestsStarted() throws Exception {
+        GridQueryProcessor.idxCls = IndexingWithQueries.class;
+
+        startGrids(3);
+    }
+
+    /**
+     * Execute provided sql update query.
+     */
+    private void executeUpdate(String sql) throws Exception {
+        try (Connection conn = GridTestUtils.connect(grid(0), null)) {
+            try (PreparedStatement upd = conn.prepareStatement(sql)) {
+                upd.executeUpdate();
+            }
+        }
+    }
+
+    /**
+     * Create some table to query, fill data.
+     */
+    @Before
+    public void inint() throws Exception {
+        executeUpdate("DROP TABLE IF EXISTS TEST");
+
+        executeUpdate("CREATE TABLE TEST (id INT PRIMARY KEY, val INT)");
+
+        IgniteCache<Integer, Integer> cache = grid(0).cache("SQL_PUBLIC_TEST");
+
+        for (int i = 0; i < INITIAL_ROWS_CNT; i++)
+            executeUpdate("INSERT INTO TEST VALUES (" + i + ", " + (i + 1) + ")");
+
+        IndexingWithQueries.queries.clear();
+    }
+
+    /**
+     * Verify single queries.
+     */
+    @Test
+    public void testDataPageScanSingle() throws Exception {
+        checkDataPageScan("SELECT * FROM TEST WHERE val > 42", null);
+        checkDataPageScan("UPDATE TEST SET val = val + 1 WHERE val > 10", null);
+
+        checkDataPageScan("SELECT id FROM TEST WHERE val < 3", true);
+        checkDataPageScan("UPDATE TEST SET val = val + 3 WHERE val < 3", true);
+
+        checkDataPageScan("SELECT val FROM TEST WHERE id = 5", false);
+        checkDataPageScan("UPDATE TEST SET val = val - 5 WHERE val < 100", false);
+    }
+
+    /**
+     * Verify the case property is set on connection and batched operations are performed.
+     */
+    @Test
+    public void testDataPageScanBatching() throws Exception {
+        checkDataPageScanInBatch("UPDATE TEST SET val = ? WHERE val > 10", null);
+
+        checkDataPageScanInBatch("UPDATE TEST SET val = val + 3 WHERE val < ?", true);
+
+        checkDataPageScanInBatch("UPDATE TEST SET val = val - 5 WHERE val < ?", false);
+    }
+
+    /**
+     * Checks that executed query executed has the same data page scan flag value as we specified in the connection
+     * properties. Queries are executed with batching.
+     *
+     * @param qryWithParam query that has one int positional parameter to run.
+     * @param dps data page scan value to test.
+     */
+    private void checkDataPageScanInBatch(String qryWithParam, @Nullable Boolean dps) throws Exception {
+        String params = (dps == null) ? null : "dataPageScanEnabled=" + dps;
+
+        try (Connection conn = GridTestUtils.connect(grid(0), params)) {
+            try (PreparedStatement upd = conn.prepareStatement(qryWithParam)) {
+                for (int i = 0; i < TOTAL_QUERIES_TO_EXECUTE; i++) {
+                    upd.setInt(1, i);
+                    upd.addBatch();
+
+                    if ((i + 1) % BATCH_SIZE == 0 || (i + 1) == TOTAL_QUERIES_TO_EXECUTE)
+                        upd.executeBatch();
+                }
+            }
+        }
+
+        // Some update operations may produce additional SELECTS, but all the queries should have specified flag value.
+        boolean containsOrig = IndexingWithQueries.queries.stream()
+            .anyMatch(executedQry -> qryWithParam.equals(executedQry.getSql()));
+
+        assertTrue("Original query have not been executed.", containsOrig);
+
+        IndexingWithQueries.queries.forEach(query ->
+            assertEquals("Data page scan flag value is unexpected for query " + query, dps,
+                query.isDataPageScanEnabled())
+        );
+
+        int executed = IndexingWithQueries.queries.size();
+
+        assertTrue(
+            "Expected that there are executed at least " + TOTAL_QUERIES_TO_EXECUTE + " queries. " +
+                "But executed only " + executed,
+            executed >= TOTAL_QUERIES_TO_EXECUTE);
+
+        IndexingWithQueries.queries.clear();
+    }
+
+    /**
+     * Checks that executed query has the same data page scan flag value as we specified in the connection properties.
+     *
+     * @param qry query with no positional parameters.
+     * @param dps data page scan value to test.
+     */
+    private void checkDataPageScan(String qry, @Nullable Boolean dps) throws Exception {
+        String params = (dps == null) ? null : "dataPageScanEnabled=" + dps;
+
+        try (Connection conn = GridTestUtils.connect(grid(0), params)) {
+            try (PreparedStatement stmt = conn.prepareStatement(qry)) {
+                stmt.execute();
+            }
+        }
+
+        // Some update operations may produce additional SELECTS, but all the queries should have specified flag value.
+        boolean containsOrig = IndexingWithQueries.queries.stream()
+            .anyMatch(executedQry -> qry.equals(executedQry.getSql()));
+
+        assertTrue("Original query have not been executed.", containsOrig);
+
+        IndexingWithQueries.queries.forEach(executedQry ->
+            assertEquals("Data page scan flag value is unexpected for query " + executedQry, dps,
+                executedQry.isDataPageScanEnabled())
+        );
+
+        int executed = IndexingWithQueries.queries.size();
+
+        assertTrue(
+            "Expected that there are executed at least one query. " +
+                "But executed only " + executed,
+            executed >= 1);
+
+        IndexingWithQueries.queries.clear();
+    }
+
+    /**
+     * Indexing that remembers all the sql fields queries have been executed.
+     */
+    private static class IndexingWithQueries extends IgniteH2Indexing {
+        /** All the queries that have been executed using this indexing. */
+        static final Queue<SqlFieldsQuery> queries = new LinkedBlockingQueue<>();
+
+        /** {@inheritDoc} */
+        @Override public List<FieldsQueryCursor<List<?>>> querySqlFields(String schemaName, SqlFieldsQuery qry,
+            @Nullable SqlClientContext cliCtx, boolean keepBinary, boolean failOnMultipleStmts,
+            MvccQueryTracker tracker, GridQueryCancel cancel, boolean registerAsNewQry) {
+            queries.add(qry);
+
+            return super.querySqlFields(schemaName, qry, cliCtx, keepBinary, failOnMultipleStmts, tracker, cancel, registerAsNewQry);
+        }
+    }
+}
diff --git a/modules/core/src/main/java/org/apache/ignite/IgniteJdbcThinDataSource.java b/modules/core/src/main/java/org/apache/ignite/IgniteJdbcThinDataSource.java
index 05c37a5..616579a 100644
--- a/modules/core/src/main/java/org/apache/ignite/IgniteJdbcThinDataSource.java
+++ b/modules/core/src/main/java/org/apache/ignite/IgniteJdbcThinDataSource.java
@@ -30,6 +30,7 @@ import org.apache.ignite.internal.jdbc.thin.ConnectionPropertiesImpl;
 import org.apache.ignite.internal.processors.odbc.SqlStateCode;
 import org.apache.ignite.internal.util.HostAndPortRange;
 import org.apache.ignite.internal.util.typedef.F;
+import org.jetbrains.annotations.Nullable;
 
 /**
  * JDBC thin DataSource implementation.
@@ -610,4 +611,20 @@ public class IgniteJdbcThinDataSource implements DataSource, Serializable {
     public String getPassword() {
         return props.getPassword();
     }
+
+    /**
+     * @return {@code True} if data page scan support is enabled for this connection, {@code false} if it's disabled and
+     * {@code null} for server default.
+     */
+    @Nullable public Boolean isDataPageScanEnabled() {
+        return props.isDataPageScanEnabled();
+    }
+
+    /**
+     * @param dataPageScan if {@code True} then data page scan support is enabled for this connection, if {@code false}
+     * then it's disabled, if {@code null} then server should use its default settings.
+     */
+    public void setDataPageScanEnabled(@Nullable Boolean dataPageScan) {
+        props.setDataPageScanEnabled(dataPageScan);
+    }
 }
\ No newline at end of file
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/jdbc/thin/ConnectionProperties.java b/modules/core/src/main/java/org/apache/ignite/internal/jdbc/thin/ConnectionProperties.java
index 5e40bbf..d59e979 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/jdbc/thin/ConnectionProperties.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/jdbc/thin/ConnectionProperties.java
@@ -19,6 +19,7 @@ package org.apache.ignite.internal.jdbc.thin;
 
 import java.sql.SQLException;
 import org.apache.ignite.internal.util.HostAndPortRange;
+import org.jetbrains.annotations.Nullable;
 
 /**
  * Provide access and manipulations with connection JDBC properties.
@@ -392,4 +393,16 @@ public interface ConnectionProperties {
      * @return User's password.
      */
     public String getPassword();
+
+    /**
+     * @return {@code True} if data page scan support is enabled for this connection, {@code false} if it's disabled
+     *     and {@code null} for server default.
+     */
+    @Nullable public Boolean isDataPageScanEnabled();
+
+    /**
+     * @param dataPageScanEnabled {@code True} if data page scan support is enabled for this connection,
+     *     if {@code false} then it's disabled, if {@code null} then server should use its default settings.
+     */
+    public void setDataPageScanEnabled(@Nullable Boolean dataPageScanEnabled);
 }
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/jdbc/thin/ConnectionPropertiesImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/jdbc/thin/ConnectionPropertiesImpl.java
index 51a3837..0a0ff8a 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/jdbc/thin/ConnectionPropertiesImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/jdbc/thin/ConnectionPropertiesImpl.java
@@ -26,9 +26,10 @@ import java.util.StringTokenizer;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.configuration.ClientConnectorConfiguration;
 import org.apache.ignite.internal.processors.odbc.SqlStateCode;
-import org.apache.ignite.internal.util.HostAndPortRange;
 import org.apache.ignite.internal.processors.query.NestedTxMode;
+import org.apache.ignite.internal.util.HostAndPortRange;
 import org.apache.ignite.internal.util.typedef.F;
+import org.jetbrains.annotations.Nullable;
 
 /**
  * Holds JDBC connection properties.
@@ -183,6 +184,11 @@ public class ConnectionPropertiesImpl implements ConnectionProperties, Serializa
     private StringProperty passwd = new StringProperty(
         "password", "User's password", null, null, false, null);
 
+    /** Data page scan flag. */
+    private BooleanProperty dataPageScanEnabled = new BooleanProperty("dataPageScanEnabled",
+        "Whether data page scan for queries is allowed. If not specified, server defines the default behaviour.",
+        null, false);
+
     /** Properties array. */
     private final ConnectionProperty [] propsArray = {
         distributedJoins, enforceJoinOrder, collocated, replicatedOnly, autoCloseServerCursor,
@@ -191,7 +197,8 @@ public class ConnectionPropertiesImpl implements ConnectionProperties, Serializa
         sslClientCertificateKeyStoreUrl, sslClientCertificateKeyStorePassword, sslClientCertificateKeyStoreType,
         sslTrustCertificateKeyStoreUrl, sslTrustCertificateKeyStorePassword, sslTrustCertificateKeyStoreType,
         sslTrustAll, sslFactory,
-        user, passwd
+        user, passwd,
+        dataPageScanEnabled
     };
 
     /** {@inheritDoc} */
@@ -487,6 +494,16 @@ public class ConnectionPropertiesImpl implements ConnectionProperties, Serializa
         return passwd.value();
     }
 
+    /** {@inheritDoc} */
+    @Override public @Nullable Boolean isDataPageScanEnabled() {
+        return dataPageScanEnabled.value();
+    }
+
+    /** {@inheritDoc} */
+    @Override public void setDataPageScanEnabled(@Nullable Boolean dataPageScanEnabled) {
+        this.dataPageScanEnabled.setValue(dataPageScanEnabled);
+    }
+
     /**
      * @param url URL connection.
      * @param props Environment properties.
@@ -911,7 +928,7 @@ public class ConnectionPropertiesImpl implements ConnectionProperties, Serializa
         private static final String [] boolChoices = new String[] {Boolean.TRUE.toString(), Boolean.FALSE.toString()};
 
         /** Value. */
-        private boolean val;
+        private Boolean val;
 
         /**
          * @param name Name.
@@ -919,7 +936,7 @@ public class ConnectionPropertiesImpl implements ConnectionProperties, Serializa
          * @param dfltVal Default value.
          * @param required {@code true} if the property is required.
          */
-        BooleanProperty(String name, String desc, boolean dfltVal, boolean required) {
+        BooleanProperty(String name, String desc, @Nullable Boolean dfltVal, boolean required) {
             super(name, desc, dfltVal, boolChoices, required);
 
             val = dfltVal;
@@ -928,7 +945,7 @@ public class ConnectionPropertiesImpl implements ConnectionProperties, Serializa
         /**
          * @return Property value.
          */
-        boolean value() {
+        @Nullable Boolean value() {
             return val;
         }
 
@@ -949,13 +966,16 @@ public class ConnectionPropertiesImpl implements ConnectionProperties, Serializa
 
         /** {@inheritDoc} */
         @Override String valueObject() {
+            if (val == null)
+                return null;
+
             return Boolean.toString(val);
         }
 
         /**
          * @param val Property value to set.
          */
-        void setValue(boolean val) {
+        void setValue(Boolean val) {
             this.val = val;
         }
     }
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/jdbc/thin/JdbcThinTcpIo.java b/modules/core/src/main/java/org/apache/ignite/internal/jdbc/thin/JdbcThinTcpIo.java
index 206e62c..2b8a0af 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/jdbc/thin/JdbcThinTcpIo.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/jdbc/thin/JdbcThinTcpIo.java
@@ -56,6 +56,8 @@ import org.apache.ignite.internal.util.typedef.F;
 import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.lang.IgniteProductVersion;
 
+import static org.apache.ignite.internal.jdbc.thin.JdbcThinUtils.nullableBooleanToByte;
+
 /**
  * JDBC IO layer implementation based on blocking IPC streams.
  */
@@ -345,6 +347,9 @@ public class JdbcThinTcpIo {
         if (ver.compareTo(VER_2_7_0) >= 0)
             writer.writeString(connProps.nestedTxMode());
 
+        if (ver.compareTo(VER_2_8_0) >= 0)
+            writer.writeByte(nullableBooleanToByte(connProps.isDataPageScanEnabled()));
+
         if (!F.isEmpty(connProps.getUsername())) {
             assert ver.compareTo(VER_2_5_0) >= 0 : "Authentication is supported since 2.5";
 
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/jdbc/thin/JdbcThinUtils.java b/modules/core/src/main/java/org/apache/ignite/internal/jdbc/thin/JdbcThinUtils.java
index c129370..55be62c 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/jdbc/thin/JdbcThinUtils.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/jdbc/thin/JdbcThinUtils.java
@@ -22,11 +22,13 @@ import java.sql.Time;
 import java.sql.Timestamp;
 import java.sql.Types;
 import java.util.Date;
+import org.jetbrains.annotations.Nullable;
 
 import static java.sql.Types.BIGINT;
 import static java.sql.Types.BINARY;
 import static java.sql.Types.BOOLEAN;
 import static java.sql.Types.DATE;
+import static java.sql.Types.DECIMAL;
 import static java.sql.Types.DOUBLE;
 import static java.sql.Types.FLOAT;
 import static java.sql.Types.INTEGER;
@@ -36,8 +38,6 @@ import static java.sql.Types.TIME;
 import static java.sql.Types.TIMESTAMP;
 import static java.sql.Types.TINYINT;
 import static java.sql.Types.VARCHAR;
-import static java.sql.Types.DECIMAL;
-
 import static org.apache.ignite.internal.jdbc.thin.ConnectionPropertiesImpl.PROP_PREFIX;
 
 /**
@@ -53,6 +53,15 @@ public class JdbcThinUtils {
     /** Hostname property name. */
     public static final String PROP_HOST = PROP_PREFIX + "host";
 
+    /** Byte representation of value "default" */
+    private static final byte BYTE_DEFAULT = 2;
+
+    /** Byte representation of value "disabled". */
+    private static final byte BYTE_ENABLED = 1;
+
+    /** Byte representation of value "disabled". */
+    private static final byte BYTE_DISABLED = 0;
+
     /**
      * Converts Java class name to type from {@link Types}.
      *
@@ -163,4 +172,36 @@ public class JdbcThinUtils {
             float.class.getName().equals(cls) ||
             double.class.getName().equals(cls));
     }
+
+    /**
+     * Converts raw byte value to the nullable Boolean. Useful for the deserialization in the handshake.
+     *
+     * @param raw byte value to convert to Boolean.
+     * @return converted value.
+     */
+    @Nullable public static Boolean nullableBooleanFromByte(byte raw) {
+        switch (raw) {
+            case BYTE_DEFAULT:
+                return null;
+            case BYTE_ENABLED:
+                return Boolean.TRUE;
+            case BYTE_DISABLED:
+                return Boolean.FALSE;
+            default:
+                throw new NumberFormatException("Incorrect byte: " + raw + ". Impossible to read nullable Boolean from it.");
+        }
+    }
+
+    /**
+     * Converts nullable Boolean to the raw byte. Useful for the serialization in the handshake.
+     *
+     * @param val value to convert.
+     * @return byte representation.
+     */
+    public static byte nullableBooleanToByte(@Nullable Boolean val) {
+        if (val == null)
+            return BYTE_DEFAULT;
+
+        return val ? BYTE_ENABLED : BYTE_DISABLED;
+    }
 }
\ No newline at end of file
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcConnectionContext.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcConnectionContext.java
index bd6b328..125cc02 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcConnectionContext.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcConnectionContext.java
@@ -35,6 +35,8 @@ import org.apache.ignite.internal.util.GridSpinBusyLock;
 import org.apache.ignite.internal.util.nio.GridNioSession;
 import org.apache.ignite.internal.util.typedef.F;
 
+import static org.apache.ignite.internal.jdbc.thin.JdbcThinUtils.nullableBooleanFromByte;
+
 /**
  * JDBC Connection Context.
  */
@@ -160,6 +162,12 @@ public class JdbcConnectionContext extends ClientListenerAbstractConnectionConte
             }
         }
 
+
+        Boolean dataPageScanEnabled = null;
+
+        if (ver.compareTo(VER_2_8_0) >= 0)
+            dataPageScanEnabled = nullableBooleanFromByte(reader.readByte());
+
         if (ver.compareTo(VER_2_5_0) >= 0) {
             String user = null;
             String passwd = null;
@@ -193,7 +201,8 @@ public class JdbcConnectionContext extends ClientListenerAbstractConnectionConte
         };
 
         handler = new JdbcRequestHandler(ctx, busyLock, sender, maxCursors, distributedJoins, enforceJoinOrder,
-            collocated, replicatedOnly, autoCloseCursors, lazyExec, skipReducerOnUpdate, nestedTxMode, actx, ver);
+            collocated, replicatedOnly, autoCloseCursors, lazyExec, skipReducerOnUpdate, nestedTxMode,
+            dataPageScanEnabled, actx, ver);
 
         handler.start();
     }
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcRequestHandler.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcRequestHandler.java
index c2db8e0..e968574 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcRequestHandler.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcRequestHandler.java
@@ -67,6 +67,7 @@ import org.apache.ignite.lang.IgniteBiTuple;
 import org.apache.ignite.transactions.TransactionAlreadyCompletedException;
 import org.apache.ignite.transactions.TransactionDuplicateKeyException;
 import org.apache.ignite.transactions.TransactionSerializationException;
+import org.jetbrains.annotations.Nullable;
 
 import static org.apache.ignite.internal.processors.odbc.jdbc.JdbcBulkLoadBatchRequest.CMD_CONTINUE;
 import static org.apache.ignite.internal.processors.odbc.jdbc.JdbcBulkLoadBatchRequest.CMD_FINISHED_EOF;
@@ -165,11 +166,23 @@ public class JdbcRequestHandler implements ClientListenerRequestHandler {
      * @param actx Authentication context.
      * @param protocolVer Protocol version.
      */
-    public JdbcRequestHandler(GridKernalContext ctx, GridSpinBusyLock busyLock,
-        ClientListenerResponseSender sender, int maxCursors,
-        boolean distributedJoins, boolean enforceJoinOrder, boolean collocated, boolean replicatedOnly,
-        boolean autoCloseCursors, boolean lazy, boolean skipReducerOnUpdate, NestedTxMode nestedTxMode,
-        AuthorizationContext actx, ClientListenerProtocolVersion protocolVer) {
+    public JdbcRequestHandler(
+        GridKernalContext ctx,
+        GridSpinBusyLock busyLock,
+        ClientListenerResponseSender sender,
+        int maxCursors,
+        boolean distributedJoins,
+        boolean enforceJoinOrder,
+        boolean collocated,
+        boolean replicatedOnly,
+        boolean autoCloseCursors,
+        boolean lazy,
+        boolean skipReducerOnUpdate,
+        NestedTxMode nestedTxMode,
+        @Nullable Boolean dataPageScanEnabled,
+        AuthorizationContext actx,
+        ClientListenerProtocolVersion protocolVer
+    ) {
         this.ctx = ctx;
         this.sender = sender;
 
@@ -189,7 +202,8 @@ public class JdbcRequestHandler implements ClientListenerRequestHandler {
             collocated,
             replicatedOnly,
             lazy,
-            skipReducerOnUpdate
+            skipReducerOnUpdate,
+            dataPageScanEnabled
         );
 
         this.busyLock = busyLock;
@@ -549,6 +563,9 @@ public class JdbcRequestHandler implements ClientListenerRequestHandler {
             qry.setNestedTxMode(nestedTxMode);
             qry.setAutoCommit(req.autoCommit());
 
+            if (cliCtx.dataPageScanEnabled() != null)
+                qry.setDataPageScanEnabled(cliCtx.dataPageScanEnabled());
+
             if (req.pageSize() <= 0)
                 return new JdbcResponse(IgniteQueryErrorCode.UNKNOWN, "Invalid fetch size: " + req.pageSize());
 
@@ -862,6 +879,9 @@ public class JdbcRequestHandler implements ClientListenerRequestHandler {
                     qry.setNestedTxMode(nestedTxMode);
                     qry.setAutoCommit(req.autoCommit());
 
+                    if (cliCtx.dataPageScanEnabled() != null)
+                        qry.setDataPageScanEnabled(cliCtx.dataPageScanEnabled());
+
                     qry.setSchema(schemaName);
                 }
 
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/odbc/OdbcRequestHandler.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/odbc/OdbcRequestHandler.java
index abc8085..4723742 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/odbc/OdbcRequestHandler.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/odbc/OdbcRequestHandler.java
@@ -166,7 +166,8 @@ public class OdbcRequestHandler implements ClientListenerRequestHandler {
             collocated,
             replicatedOnly,
             lazy,
-            skipReducerOnUpdate
+            skipReducerOnUpdate,
+            null
         );
 
         this.busyLock = busyLock;
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/SqlClientContext.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/SqlClientContext.java
index 1f2fe43..f51fdb4 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/SqlClientContext.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/SqlClientContext.java
@@ -28,6 +28,7 @@ import org.apache.ignite.internal.GridKernalContext;
 import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.internal.util.worker.GridWorker;
 import org.apache.ignite.thread.IgniteThread;
+import org.jetbrains.annotations.Nullable;
 
 /**
  * Container for connection properties passed by various drivers (JDBC, ODBC drivers) having notion of an
@@ -57,6 +58,9 @@ public class SqlClientContext implements AutoCloseable {
     /** Skip reducer on update flag. */
     private final boolean skipReducerOnUpdate;
 
+    /** Data page scan support for query execution. */
+    private final @Nullable Boolean dataPageScanEnabled;
+
     /** Monitor. */
     private final Object mux = new Object();
 
@@ -103,7 +107,8 @@ public class SqlClientContext implements AutoCloseable {
      */
     public SqlClientContext(GridKernalContext ctx, Factory<GridWorker> orderedBatchWorkerFactory,
         boolean distributedJoins, boolean enforceJoinOrder,
-        boolean collocated, boolean replicatedOnly, boolean lazy, boolean skipReducerOnUpdate) {
+        boolean collocated, boolean replicatedOnly, boolean lazy, boolean skipReducerOnUpdate,
+        @Nullable Boolean dataPageScanEnabled) {
         this.ctx = ctx;
         this.orderedBatchWorkerFactory = orderedBatchWorkerFactory;
         this.distributedJoins = distributedJoins;
@@ -112,6 +117,7 @@ public class SqlClientContext implements AutoCloseable {
         this.replicatedOnly = replicatedOnly;
         this.lazy = lazy;
         this.skipReducerOnUpdate = skipReducerOnUpdate;
+        this.dataPageScanEnabled = dataPageScanEnabled;
 
         log = ctx.log(SqlClientContext.class.getName());
     }
@@ -214,6 +220,13 @@ public class SqlClientContext implements AutoCloseable {
     }
 
     /**
+     * @return Data page scan flag or {@code null} if not set.
+     */
+    public @Nullable Boolean dataPageScanEnabled() {
+        return dataPageScanEnabled;
+    }
+
+    /**
      * @return Streaming state flag (on or off).
      */
     public boolean isStream() {