You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by dp...@apache.org on 2019/03/04 12:46:39 UTC

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

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

dpavlov pushed a commit to branch ignite-11461-java11
in repository https://gitbox.apache.org/repos/asf/ignite.git

commit 71a6a124b06ca0ff595c1c74015e3c17da5ca062
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() {