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

[63/69] [abbrv] ignite git commit: Merge branch 'ignite-1.6.8' into ignite-1.7.2

Merge branch 'ignite-1.6.8' into ignite-1.7.2


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

Branch: refs/heads/ignite-3199-1
Commit: 3b391849629d63c0925862fbf2b3ec2a2e429c24
Parents: 79745ee 65c92fa
Author: isapego <is...@gridgain.com>
Authored: Fri Sep 9 14:25:52 2016 +0300
Committer: isapego <is...@gridgain.com>
Committed: Fri Sep 9 14:25:52 2016 +0300

----------------------------------------------------------------------
 .../ignite/configuration/OdbcConfiguration.java |  98 +++-
 .../local/LocalIgfsSecondaryFileSystem.java     |  15 +-
 .../apache/ignite/internal/GridLoggerProxy.java |   3 +-
 .../org/apache/ignite/internal/IgnitionEx.java  |  17 +-
 .../ignite/internal/binary/BinaryContext.java   |   3 +
 .../internal/binary/BinaryMarshaller.java       |  28 +-
 .../internal/binary/BinaryReaderExImpl.java     |  24 +
 .../internal/binary/BinaryReaderHandles.java    |   2 +-
 .../internal/binary/BinaryWriterExImpl.java     |  18 +
 .../binary/builder/BinaryObjectBuilderImpl.java |   2 +-
 .../client/GridClientConfiguration.java         |   1 -
 .../internal/cluster/ClusterGroupAdapter.java   |   2 +-
 .../cluster/ClusterNodeLocalMapImpl.java        |   3 +-
 .../processors/cache/GridCacheAdapter.java      |   2 +-
 .../processors/cache/GridCacheContext.java      |   2 +-
 .../processors/cache/GridCacheProcessor.java    |   5 +-
 .../processors/cache/GridCacheTtlManager.java   |  24 +-
 .../processors/cache/IgniteCacheProxy.java      |  41 ++
 .../binary/CacheObjectBinaryProcessorImpl.java  |  40 +-
 .../near/GridNearOptimisticTxPrepareFuture.java |   2 +-
 .../cache/query/GridCacheSqlQuery.java          |  11 +-
 .../datastreamer/DataStreamerImpl.java          |  23 +-
 .../processors/igfs/IgfsDataManager.java        |   2 +-
 .../internal/processors/igfs/IgfsImpl.java      | 216 +++++---
 .../local/LocalFileSystemSizeVisitor.java       |  60 +++
 .../processors/odbc/OdbcMessageParser.java      |  10 +-
 .../internal/processors/odbc/OdbcProcessor.java |  54 +-
 .../processors/odbc/OdbcRequestHandler.java     |  31 +-
 .../processors/odbc/escape/OdbcEscapeType.java  |  13 +-
 .../processors/odbc/escape/OdbcEscapeUtils.java |  58 ++-
 .../query/PlatformAbstractQueryCursor.java      |  11 +-
 .../cache/query/PlatformFieldsQueryCursor.java  |   6 +
 .../processors/task/GridTaskWorker.java         |   2 +-
 .../ignite/internal/util/IgniteUtils.java       |  60 ++-
 .../ignite/marshaller/AbstractMarshaller.java   |  41 +-
 .../AbstractNodeNameAwareMarshaller.java        | 142 ++++++
 .../ignite/marshaller/MarshallerUtils.java      |  58 +++
 .../ignite/marshaller/jdk/JdkMarshaller.java    |  40 +-
 .../optimized/OptimizedMarshaller.java          |  12 +-
 .../sharedfs/SharedFsCheckpointSpi.java         |   7 +-
 .../ignite/spi/discovery/tcp/ClientImpl.java    |  11 +-
 .../ignite/spi/discovery/tcp/ServerImpl.java    |  29 +-
 .../spi/discovery/tcp/TcpDiscoverySpi.java      |  20 +-
 .../ignite/stream/socket/SocketStreamer.java    |  18 +-
 .../ignite/internal/ClusterGroupSelfTest.java   |   2 +-
 .../internal/GridEventStorageSelfTest.java      |  97 +++-
 .../binary/BinaryMarshallerSelfTest.java        |  38 ++
 .../BinaryObjectBuilderAdditionalSelfTest.java  |  14 +
 .../cache/GridCacheAbstractFullApiSelfTest.java |   2 +-
 .../cache/GridLocalIgniteSerializationTest.java | 378 ++++++++++++++
 .../IgniteCacheExpiryPolicyAbstractTest.java    |  14 +-
 .../GridCacheQueryTransformerSelfTest.java      |   9 +-
 .../igfs/IgfsAbstractBaseSelfTest.java          |   7 +-
 .../processors/igfs/IgfsAbstractSelfTest.java   |  31 +-
 ...SecondaryFileSystemDualAbstractSelfTest.java |  63 ++-
 .../processors/igfs/IgfsMaxSizeSelfTest.java    | 122 +++++
 .../processors/igfs/IgfsProcessorSelfTest.java  |  11 +-
 .../processors/igfs/IgfsProxySelfTest.java      |  32 ++
 .../processors/igfs/IgfsStreamsSelfTest.java    |   2 +-
 .../odbc/OdbcEscapeSequenceSelfTest.java        | 210 +++++++-
 .../odbc/OdbcProcessorValidationSelfTest.java   |  37 +-
 .../spi/discovery/tcp/TcpDiscoverySelfTest.java |   2 +-
 .../ignite/testsuites/IgniteBasicTestSuite.java |   2 +
 .../ignite/testsuites/IgniteIgfsTestSuite.java  |   3 +
 .../query/h2/opt/GridH2ValueCacheObject.java    |   9 -
 ...niteBinaryObjectLocalQueryArgumentsTest.java |  28 ++
 ...aryObjectQueryArgumentsOffheapLocalTest.java |  28 ++
 ...teBinaryObjectQueryArgumentsOffheapTest.java |  30 ++
 .../IgniteBinaryObjectQueryArgumentsTest.java   | 472 +++++++++++++++++-
 .../query/h2/sql/GridQueryParsingTest.java      |   2 +-
 .../IgniteCacheQuerySelfTestSuite.java          |   9 +
 .../ignite/impl/binary/binary_reader_impl.h     |   2 +-
 .../common/include/ignite/common/concurrent.h   |   5 +-
 .../cpp/common/include/ignite/ignite_error.h    |  11 +-
 .../platforms/cpp/common/src/ignite_error.cpp   |   2 +-
 .../cpp/core-test/src/cache_query_test.cpp      | 300 ++++++++---
 modules/platforms/cpp/core/Makefile.am          |   1 +
 modules/platforms/cpp/core/include/Makefile.am  |   1 +
 .../include/ignite/cache/query/query_cursor.h   |   6 +-
 .../ignite/cache/query/query_fields_cursor.h    |   4 +-
 .../ignite/impl/cache/query/query_batch.h       | 148 ++++++
 .../impl/cache/query/query_fields_row_impl.h    |  30 +-
 .../ignite/impl/cache/query/query_impl.h        |  30 +-
 .../platforms/cpp/core/project/vs/core.vcxproj  |   2 +
 .../cpp/core/project/vs/core.vcxproj.filters    |   6 +
 .../core/src/impl/cache/query/query_batch.cpp   |  52 ++
 .../core/src/impl/cache/query/query_impl.cpp    | 180 ++++---
 modules/platforms/cpp/odbc-test/Makefile.am     |   2 +
 .../odbc-test/include/sql_test_suite_fixture.h  |   6 +
 .../cpp/odbc-test/project/vs/odbc-test.vcxproj  |   2 +
 .../project/vs/odbc-test.vcxproj.filters        |   6 +
 .../cpp/odbc-test/src/configuration_test.cpp    |  14 +-
 .../cpp/odbc-test/src/queries_test.cpp          |   1 +
 .../src/sql_date_time_functions_test.cpp        | 213 ++++++++
 .../cpp/odbc-test/src/sql_outer_join_test.cpp   | 498 +++++++++++++++++++
 .../odbc-test/src/sql_string_functions_test.cpp |  63 +++
 .../odbc-test/src/sql_test_suite_fixture.cpp    |  25 +-
 .../include/ignite/odbc/config/configuration.h  |  52 +-
 .../cpp/odbc/include/ignite/odbc/result_page.h  |   3 -
 .../odbc/system/ui/dsn_configuration_window.h   |   8 +
 .../src/system/ui/dsn_configuration_window.cpp  |  23 +-
 .../cpp/odbc/os/win/src/system/ui/window.cpp    |   2 +-
 modules/platforms/cpp/odbc/src/column.cpp       |  14 +-
 .../cpp/odbc/src/config/configuration.cpp       |  30 +-
 .../cpp/odbc/src/config/connection_info.cpp     |  24 +-
 modules/platforms/cpp/odbc/src/dsn_config.cpp   |  14 +-
 .../platforms/cpp/odbc/src/query/data_query.cpp |   2 +-
 .../Impl/Cache/Query/FieldsQueryCursor.cs       |   3 +
 108 files changed, 4106 insertions(+), 600 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/3b391849/modules/core/src/main/java/org/apache/ignite/internal/cluster/ClusterGroupAdapter.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/ignite/blob/3b391849/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheContext.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/ignite/blob/3b391849/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheProxy.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/ignite/blob/3b391849/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/binary/CacheObjectBinaryProcessorImpl.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/ignite/blob/3b391849/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheSqlQuery.java
----------------------------------------------------------------------
diff --cc modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheSqlQuery.java
index 6b81ed1,bcb37c5..7feb091
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheSqlQuery.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheSqlQuery.java
@@@ -20,9 -20,9 +20,12 @@@ package org.apache.ignite.internal.proc
  import java.nio.ByteBuffer;
  import java.util.LinkedHashMap;
  import org.apache.ignite.IgniteCheckedException;
 +import org.apache.ignite.IgniteException;
++import org.apache.ignite.IgniteException;
  import org.apache.ignite.internal.GridDirectTransient;
  import org.apache.ignite.internal.GridKernalContext;
+ import org.apache.ignite.internal.binary.BinaryMarshaller;
++import org.apache.ignite.internal.binary.BinaryMarshaller;
  import org.apache.ignite.internal.util.tostring.GridToStringInclude;
  import org.apache.ignite.internal.util.typedef.F;
  import org.apache.ignite.internal.util.typedef.internal.A;
@@@ -153,12 -154,13 +156,18 @@@ public class GridCacheSqlQuery implemen
  
          assert paramsBytes != null;
  
 -        final ClassLoader ldr = U.resolveClassLoader(ctx.config());
 +        try {
-             params = m.unmarshal(paramsBytes, U.resolveClassLoader(ctx.config()));
++            final ClassLoader ldr = U.resolveClassLoader(ctx.config());
+ 
 -        if (m instanceof BinaryMarshaller)
 -            // To avoid deserializing of enum types.
 -            params = ((BinaryMarshaller)m).binaryMarshaller().unmarshal(paramsBytes, ldr);
 -        else
 -            params = m.unmarshal(paramsBytes, ldr);
++            if (m instanceof BinaryMarshaller)
++                // To avoid deserializing of enum types.
++                params = ((BinaryMarshaller)m).binaryMarshaller().unmarshal(paramsBytes, ldr);
++            else
++                params = m.unmarshal(paramsBytes, ldr);
 +        }
 +        catch (IgniteCheckedException e) {
 +            throw new IgniteException(e);
 +        }
      }
  
      /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/3b391849/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcMessageParser.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/ignite/blob/3b391849/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcRequestHandler.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/ignite/blob/3b391849/modules/core/src/main/java/org/apache/ignite/internal/util/IgniteUtils.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/ignite/blob/3b391849/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ClientImpl.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/ignite/blob/3b391849/modules/core/src/test/java/org/apache/ignite/internal/ClusterGroupSelfTest.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/ignite/blob/3b391849/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractFullApiSelfTest.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/ignite/blob/3b391849/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2ValueCacheObject.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/ignite/blob/3b391849/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/sql/GridQueryParsingTest.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/ignite/blob/3b391849/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite.java
----------------------------------------------------------------------
diff --cc modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite.java
index 4198535,e7f55a1..b19fcee
--- a/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite.java
@@@ -22,19 -22,15 +22,23 @@@ import org.apache.ignite.internal.proce
  import org.apache.ignite.internal.processors.cache.GridCacheQueryIndexDisabledSelfTest;
  import org.apache.ignite.internal.processors.cache.GridCacheQueryInternalKeysSelfTest;
  import org.apache.ignite.internal.processors.cache.GridCacheQuerySerializationSelfTest;
 -import org.apache.ignite.internal.processors.cache.GridCacheReduceQueryMultithreadedSelfTest;
  import org.apache.ignite.internal.processors.cache.IgniteBinaryObjectFieldsQuerySelfTest;
+ import org.apache.ignite.internal.processors.cache.IgniteBinaryObjectLocalQueryArgumentsTest;
+ import org.apache.ignite.internal.processors.cache.IgniteBinaryObjectQueryArgumentsOffheapLocalTest;
+ import org.apache.ignite.internal.processors.cache.IgniteBinaryObjectQueryArgumentsOffheapTest;
+ import org.apache.ignite.internal.processors.cache.IgniteBinaryObjectQueryArgumentsTest;
  import org.apache.ignite.internal.processors.cache.IgniteBinaryWrappedObjectFieldsQuerySelfTest;
  import org.apache.ignite.internal.processors.cache.IgniteCacheCollocatedQuerySelfTest;
 +import org.apache.ignite.internal.processors.cache.IgniteCacheCrossCacheJoinRandomTest;
 +import org.apache.ignite.internal.processors.cache.IgniteCacheDistributedJoinCollocatedAndNotTest;
 +import org.apache.ignite.internal.processors.cache.IgniteCacheDistributedJoinCustomAffinityMapper;
 +import org.apache.ignite.internal.processors.cache.IgniteCacheDistributedJoinPartitionedAndReplicatedTest;
 +import org.apache.ignite.internal.processors.cache.IgniteCacheDistributedJoinQueryConditionsTest;
 +import org.apache.ignite.internal.processors.cache.IgniteCacheDistributedJoinTest;
  import org.apache.ignite.internal.processors.cache.IgniteCacheDuplicateEntityConfigurationSelfTest;
 +import org.apache.ignite.internal.processors.cache.IgniteCacheDistributedJoinNoIndexTest;
 +import org.apache.ignite.internal.processors.cache.IgniteCacheJoinPartitionedAndReplicatedTest;
 +import org.apache.ignite.internal.processors.cache.IgniteCacheJoinQueryWithAffinityKeyTest;
  import org.apache.ignite.internal.processors.cache.IgniteCacheLargeResultSelfTest;
  import org.apache.ignite.internal.processors.cache.IgniteCacheOffheapEvictQueryTest;
  import org.apache.ignite.internal.processors.cache.IgniteCacheOffheapIndexScanTest;
@@@ -128,19 -121,12 +132,24 @@@ public class IgniteCacheQuerySelfTestSu
          suite.addTestSuite(IgniteCacheQueryH2IndexingLeakTest.class);
          suite.addTestSuite(IgniteCacheQueryNoRebalanceSelfTest.class);
          suite.addTestSuite(GridCacheQueryTransformerSelfTest.class);
 +        suite.addTestSuite(IgniteCachePrimitiveFieldsQuerySelfTest.class);
 +
 +        suite.addTestSuite(IgniteCacheJoinQueryWithAffinityKeyTest.class);
 +        suite.addTestSuite(IgniteCacheDistributedJoinCollocatedAndNotTest.class);
 +        suite.addTestSuite(IgniteCacheDistributedJoinPartitionedAndReplicatedTest.class);
 +        suite.addTestSuite(IgniteCacheDistributedJoinQueryConditionsTest.class);
 +        suite.addTestSuite(IgniteCacheDistributedJoinTest.class);
 +        suite.addTestSuite(IgniteCacheJoinPartitionedAndReplicatedTest.class);
 +        suite.addTestSuite(IgniteCacheDistributedJoinNoIndexTest.class);
 +        suite.addTestSuite(IgniteCrossCachesJoinsQueryTest.class);
 +        suite.addTestSuite(IgniteCacheCrossCacheJoinRandomTest.class);
 +        suite.addTestSuite(IgniteCacheDistributedJoinCustomAffinityMapper.class);
  
+         suite.addTestSuite(IgniteBinaryObjectQueryArgumentsTest.class);
+         suite.addTestSuite(IgniteBinaryObjectQueryArgumentsOffheapTest.class);
+         suite.addTestSuite(IgniteBinaryObjectQueryArgumentsOffheapLocalTest.class);
+         suite.addTestSuite(IgniteBinaryObjectLocalQueryArgumentsTest.class);
+ 
          return suite;
      }
  }

http://git-wip-us.apache.org/repos/asf/ignite/blob/3b391849/modules/platforms/cpp/core-test/src/cache_query_test.cpp
----------------------------------------------------------------------
diff --cc modules/platforms/cpp/core-test/src/cache_query_test.cpp
index e3fba02,b8cd612..9a5371b
--- a/modules/platforms/cpp/core-test/src/cache_query_test.cpp
+++ b/modules/platforms/cpp/core-test/src/cache_query_test.cpp
@@@ -296,93 -212,6 +296,93 @@@ namespace ignit
      }
  }
  
- /**
-  * Test setup fixture.
-  */
- struct CacheQueryTestSuiteFixture
- {
-     Ignite StartNode(const char* name)
-     {
-         IgniteConfiguration cfg;
- 
-         cfg.jvmOpts.push_back("-Xdebug");
-         cfg.jvmOpts.push_back("-Xnoagent");
-         cfg.jvmOpts.push_back("-Djava.compiler=NONE");
-         cfg.jvmOpts.push_back("-agentlib:jdwp=transport=dt_socket,server=y,suspend=n,address=5005");
-         cfg.jvmOpts.push_back("-XX:+HeapDumpOnOutOfMemoryError");
- 
- #ifdef IGNITE_TESTS_32
-         cfg.jvmInitMem = 256;
-         cfg.jvmMaxMem = 768;
- #else
-         cfg.jvmInitMem = 1024;
-         cfg.jvmMaxMem = 4096;
- #endif
- 
-         cfg.springCfgPath.assign(getenv("IGNITE_NATIVE_TEST_CPP_CONFIG_PATH")).append("/cache-query.xml");
- 
-         IgniteError err;
- 
-         Ignite grid0 = Ignition::Start(cfg, name, &err);
- 
-         if (err.GetCode() != IgniteError::IGNITE_SUCCESS)
-             BOOST_ERROR(err.GetText());
- 
-         return grid0;
-     }
- 
- 
-     /**
-      * Constructor.
-      */
-     CacheQueryTestSuiteFixture() : 
-         grid(StartNode("Node1"))
-     {
-         // No-op.
-     }
- 
-     /**
-      * Destructor.
-      */
-     ~CacheQueryTestSuiteFixture()
-     {
-         Ignition::StopAll(true);
-     }
- 
-     /** Person cache accessor. */
-     Cache<int, QueryPerson> GetPersonCache()
-     {
-         return grid.GetCache<int, QueryPerson>("QueryPerson");
-     }
- 
-     /** Relation cache accessor. */
-     Cache<int, QueryRelation> GetRelationCache()
-     {
-         return grid.GetCache<int, QueryRelation>("QueryRelation");
-     }
- 
-     /** Node started during the test. */
-     Ignite grid;
- };
++///**
++// * Test setup fixture.
++// */
++//struct CacheQueryTestSuiteFixture
++//{
++//    Ignite StartNode(const char* name)
++//    {
++//        IgniteConfiguration cfg;
++//
++//        cfg.jvmOpts.push_back("-Xdebug");
++//        cfg.jvmOpts.push_back("-Xnoagent");
++//        cfg.jvmOpts.push_back("-Djava.compiler=NONE");
++//        cfg.jvmOpts.push_back("-agentlib:jdwp=transport=dt_socket,server=y,suspend=n,address=5005");
++//        cfg.jvmOpts.push_back("-XX:+HeapDumpOnOutOfMemoryError");
++//
++//#ifdef IGNITE_TESTS_32
++//        cfg.jvmInitMem = 256;
++//        cfg.jvmMaxMem = 768;
++//#else
++//        cfg.jvmInitMem = 1024;
++//        cfg.jvmMaxMem = 4096;
++//#endif
++//
++//        cfg.springCfgPath.assign(getenv("IGNITE_NATIVE_TEST_CPP_CONFIG_PATH")).append("/cache-query.xml");
++//
++//        IgniteError err;
++//
++//        Ignite grid0 = Ignition::Start(cfg, name, &err);
++//
++//        if (err.GetCode() != IgniteError::IGNITE_SUCCESS)
++//            BOOST_ERROR(err.GetText());
++//
++//        return grid0;
++//    }
++//
++//
++//    /**
++//     * Constructor.
++//     */
++//    CacheQueryTestSuiteFixture() : 
++//        grid(StartNode("Node1"))
++//    {
++//        // No-op.
++//    }
++//
++//    /**
++//     * Destructor.
++//     */
++//    ~CacheQueryTestSuiteFixture()
++//    {
++//        Ignition::StopAll(true);
++//    }
++//
++//    /** Person cache accessor. */
++//    Cache<int, QueryPerson> GetPersonCache()
++//    {
++//        return grid.GetCache<int, QueryPerson>("QueryPerson");
++//    }
++//
++    ///** Relation cache accessor. */
++    //Cache<int, QueryRelation> GetRelationCache()
++    //{
++    //    return grid.GetCache<int, QueryRelation>("QueryRelation");
++    //}
++//
++//    /** Node started during the test. */
++//    Ignite grid;
++//};
 +
 +/**
 + * Count number of records returned by cursor.
 + *
 + * @param cur Cursor.
 + */
 +template<typename Cursor>
 +int CountRecords(Cursor& cur)
 +{
 +    int number = 0;
 +    while (cur.HasNext())
 +    {
 +        ++number;
 +        cur.GetNext();
 +    }
 +
 +    return number;
 +}
 +
  /**
   * Ensure that HasNext() fails.
   *
@@@ -654,6 -465,131 +654,137 @@@ void CheckMultipleGetAll(QueryCursor<in
      }
  }
  
+ /**
+  * Test setup fixture.
+  */
+ struct CacheQueryTestSuiteFixture
+ {
+     Ignite StartNode(const char* name)
+     {
+         IgniteConfiguration cfg;
+ 
+         cfg.jvmOpts.push_back("-Xdebug");
+         cfg.jvmOpts.push_back("-Xnoagent");
+         cfg.jvmOpts.push_back("-Djava.compiler=NONE");
+         cfg.jvmOpts.push_back("-agentlib:jdwp=transport=dt_socket,server=y,suspend=n,address=5005");
+         cfg.jvmOpts.push_back("-XX:+HeapDumpOnOutOfMemoryError");
+ 
+ #ifdef IGNITE_TESTS_32
+         cfg.jvmInitMem = 256;
+         cfg.jvmMaxMem = 768;
+ #else
+         cfg.jvmInitMem = 1024;
+         cfg.jvmMaxMem = 4096;
+ #endif
+ 
+         cfg.springCfgPath.assign(getenv("IGNITE_NATIVE_TEST_CPP_CONFIG_PATH")).append("/cache-query.xml");
+ 
+         IgniteError err;
+ 
+         Ignite grid0 = Ignition::Start(cfg, name, &err);
+ 
+         if (err.GetCode() != IgniteError::IGNITE_SUCCESS)
+             BOOST_ERROR(err.GetText());
+ 
+         return grid0;
+     }
+ 
+     void CheckFieldsQueryPages(int32_t pageSize, int32_t pagesNum, int32_t additionalNum)
+     {
+         // Test simple query.
+         Cache<int, QueryPerson> cache = GetPersonCache();
+ 
+         // Test query with two fields of different type.
+         SqlFieldsQuery qry("select name, age from QueryPerson");
+ 
+         QueryFieldsCursor cursor = cache.Query(qry);
+         CheckEmpty(cursor);
+ 
+         const int32_t entryCnt = pageSize * pagesNum + additionalNum; // Number of entries.
+ 
+         qry.SetPageSize(pageSize);
+ 
+         for (int i = 0; i < entryCnt; i++)
+         {
+             std::stringstream stream;
+ 
+             stream << "A" << i;
+ 
+             cache.Put(i, QueryPerson(stream.str(), i * 10, BinaryUtils::MakeDateLocal(1970 + i),
+                 BinaryUtils::MakeTimestampLocal(2016, 1, 1, i / 60, i % 60)));
+         }
+ 
+         cursor = cache.Query(qry);
+ 
+         IgniteError error;
+ 
+         for (int i = 0; i < entryCnt; i++)
+         {
+             std::stringstream stream;
+ 
+             stream << "A" << i;
+ 
+             std::string expected_name = stream.str();
+             int expected_age = i * 10;
+ 
+             BOOST_REQUIRE(cursor.HasNext(error));
+             BOOST_REQUIRE(error.GetCode() == IgniteError::IGNITE_SUCCESS);
+ 
+             QueryFieldsRow row = cursor.GetNext(error);
+             BOOST_REQUIRE(error.GetCode() == IgniteError::IGNITE_SUCCESS);
+ 
+             BOOST_REQUIRE(row.HasNext(error));
+             BOOST_REQUIRE(error.GetCode() == IgniteError::IGNITE_SUCCESS);
+ 
+             std::string name = row.GetNext<std::string>(error);
+             BOOST_REQUIRE(error.GetCode() == IgniteError::IGNITE_SUCCESS);
+ 
+             BOOST_REQUIRE(name == expected_name);
+ 
+             int age = row.GetNext<int>(error);
+             BOOST_REQUIRE(error.GetCode() == IgniteError::IGNITE_SUCCESS);
+ 
+             BOOST_REQUIRE(age == expected_age);
+ 
+             BOOST_REQUIRE(!row.HasNext());
+         }
+ 
+         CheckEmpty(cursor);
+     }
+ 
+     /**
+      * Constructor.
+      */
+     CacheQueryTestSuiteFixture() : 
+         grid(StartNode("Node1"))
+     {
+         // No-op.
+     }
+ 
+     /**
+      * Destructor.
+      */
+     ~CacheQueryTestSuiteFixture()
+     {
+         Ignition::StopAll(true);
+     }
+ 
+     /** Person cache accessor. */
+     Cache<int, QueryPerson> GetPersonCache()
+     {
+         return grid.GetCache<int, QueryPerson>("cache");
+     }
+ 
++    /** Relation cache accessor. */
++    Cache<int, QueryRelation> GetRelationCache()
++    {
++        return grid.GetCache<int, QueryRelation>("QueryRelation");
++    }
++
+     /** Node started during the test. */
+     Ignite grid;
+ };
+ 
  BOOST_FIXTURE_TEST_SUITE(CacheQueryTestSuite, CacheQueryTestSuiteFixture)
  
  /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/3b391849/modules/platforms/cpp/odbc-test/src/configuration_test.cpp
----------------------------------------------------------------------
diff --cc modules/platforms/cpp/odbc-test/src/configuration_test.cpp
index 1851eae,bfdb220..3893ad9
--- a/modules/platforms/cpp/odbc-test/src/configuration_test.cpp
+++ b/modules/platforms/cpp/odbc-test/src/configuration_test.cpp
@@@ -37,8 -37,7 +37,9 @@@ namespac
      const uint16_t testServerPort = 4242;
      const std::string testCacheName = "TestCache";
      const std::string testDsn = "Ignite DSN";
+     const int32_t testPageSize = 4321;
 +    const bool testDistributedJoins = true;
 +    const bool testEnforceJoinOrder = true;
  
      const std::string testAddress = testServerHost + ':' + ignite::common::LexicalCast<std::string>(testServerPort);
  }
@@@ -96,16 -77,14 +97,18 @@@ void CheckConnectionConfig(const Config
      BOOST_CHECK_EQUAL(cfg.GetAddress(), testAddress);
      BOOST_CHECK_EQUAL(cfg.GetCache(), testCacheName);
      BOOST_CHECK_EQUAL(cfg.GetDsn(), std::string());
+     BOOST_CHECK_EQUAL(cfg.GetPageSize(), testPageSize);
 +    BOOST_CHECK_EQUAL(cfg.IsDistributedJoins(), testDistributedJoins);
 +    BOOST_CHECK_EQUAL(cfg.IsEnforceJoinOrder(), testEnforceJoinOrder);
  
      std::stringstream constructor;
  
      constructor << "address=" << testAddress << ';'
                  << "cache=" << testCacheName << ';'
 +                << "distributed_joins=" << (testDistributedJoins ? "true" : "false") << ';'
                  << "driver={" << testDriverName << "};"
-                 << "enforce_join_order=" << (testEnforceJoinOrder ? "true" : "false") << ';';
++                << "enforce_join_order=" << (testEnforceJoinOrder ? "true" : "false") << ';'
+                 << "page_size=" << testPageSize << ';';
  
      const std::string& expectedStr = constructor.str();
  
@@@ -120,8 -99,7 +123,9 @@@ void CheckDsnConfig(const Configuration
      BOOST_CHECK_EQUAL(cfg.GetAddress(), Configuration::DefaultValue::address);
      BOOST_CHECK_EQUAL(cfg.GetHost(), std::string());
      BOOST_CHECK_EQUAL(cfg.GetTcpPort(), Configuration::DefaultValue::port);
+     BOOST_CHECK_EQUAL(cfg.GetPageSize(), Configuration::DefaultValue::pageSize);
 +    BOOST_CHECK_EQUAL(cfg.IsDistributedJoins(), false);
 +    BOOST_CHECK_EQUAL(cfg.IsEnforceJoinOrder(), false);
  }
  
  BOOST_AUTO_TEST_SUITE(ConfigurationTestSuite)
@@@ -133,8 -111,7 +137,9 @@@ BOOST_AUTO_TEST_CASE(CheckTestValuesNot
      BOOST_CHECK_NE(testServerPort, Configuration::DefaultValue::port);
      BOOST_CHECK_NE(testCacheName, Configuration::DefaultValue::cache);
      BOOST_CHECK_NE(testDsn, Configuration::DefaultValue::dsn);
+     BOOST_CHECK_NE(testPageSize, Configuration::DefaultValue::pageSize);
 +    BOOST_CHECK_NE(testDistributedJoins, Configuration::DefaultValue::distributedJoins);
 +    BOOST_CHECK_NE(testEnforceJoinOrder, Configuration::DefaultValue::enforceJoinOrder);
  }
  
  BOOST_AUTO_TEST_CASE(TestConnectStringUppercase)
@@@ -146,8 -123,7 +151,9 @@@
      constructor << "DRIVER={" << testDriverName << "};"
                  << "ADDRESS=" << testAddress << ';'
                  << "CACHE=" << testCacheName << ';'
 +                << "DISTRIBUTED_JOINS=" << (testDistributedJoins ? "TRUE" : "FALSE") << ';'
-                 << "ENFORCE_JOIN_ORDER=" << (testEnforceJoinOrder ? "TRUE" : "FALSE");
++                << "ENFORCE_JOIN_ORDER=" << (testEnforceJoinOrder ? "TRUE" : "FALSE") << ';'
+                 << "PAGE_SIZE=" << testPageSize;
  
      const std::string& connectStr = constructor.str();
  
@@@ -164,9 -140,8 +170,10 @@@ BOOST_AUTO_TEST_CASE(TestConnectStringL
  
      constructor << "driver={" << testDriverName << "};"
                  << "address=" << testAddress << ';'
+                 << "page_size=" << testPageSize << ';'
 -                << "cache=" << testCacheName << ';';
 +                << "cache=" << testCacheName << ';'
 +                << "distributed_joins=" << (testDistributedJoins ? "true" : "false") << ';'
 +                << "enforce_join_order=" << (testEnforceJoinOrder ? "true" : "false");
  
      const std::string& connectStr = constructor.str();
  
@@@ -183,9 -158,8 +190,10 @@@ BOOST_AUTO_TEST_CASE(TestConnectStringZ
  
      constructor << "driver={" << testDriverName << "};"
                  << "address=" << testAddress << ';'
+                 << "page_size=" << testPageSize << ';'
 -                << "cache=" << testCacheName << ';';
 +                << "cache=" << testCacheName << ';'
 +                << "distributed_joins=" << (testDistributedJoins ? "true" : "false") << ';'
 +                << "enforce_join_order=" << (testEnforceJoinOrder ? "true" : "false");
  
      const std::string& connectStr = constructor.str();
  
@@@ -202,9 -176,8 +210,10 @@@ BOOST_AUTO_TEST_CASE(TestConnectStringM
  
      constructor << "Driver={" << testDriverName << "};"
                  << "Address=" << testAddress << ';'
+                 << "Page_Size=" << testPageSize << ';'
 -                << "Cache=" << testCacheName << ';';
 +                << "Cache=" << testCacheName << ';'
 +                << "Distributed_Joins=" << (testDistributedJoins ? "True" : "False") << ';'
 +                << "Enforce_Join_Order=" << (testEnforceJoinOrder ? "True" : "False");
  
      const std::string& connectStr = constructor.str();
  
@@@ -221,9 -194,8 +230,10 @@@ BOOST_AUTO_TEST_CASE(TestConnectStringW
  
      constructor << "DRIVER = {" << testDriverName << "} ;\n"
                  << " ADDRESS =" << testAddress << "; "
+                 << "   PAGE_SIZE= " << testPageSize << ';'
 -                << "CACHE = \n\r" << testCacheName << ';';
 +                << "CACHE = \n\r" << testCacheName << ';'
 +                << "   DISTRIBUTED_JOINS=" << (testDistributedJoins ? "TRUE" : "FALSE") << ';'
 +                << "ENFORCE_JOIN_ORDER=   " << (testEnforceJoinOrder ? "TRUE  " : "FALSE  ");
  
      const std::string& connectStr = constructor.str();
  

http://git-wip-us.apache.org/repos/asf/ignite/blob/3b391849/modules/platforms/cpp/odbc-test/src/queries_test.cpp
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/ignite/blob/3b391849/modules/platforms/cpp/odbc/include/ignite/odbc/config/configuration.h
----------------------------------------------------------------------
diff --cc modules/platforms/cpp/odbc/include/ignite/odbc/config/configuration.h
index 8e19a6e,b5f385e..05a9ec3
--- a/modules/platforms/cpp/odbc/include/ignite/odbc/config/configuration.h
+++ b/modules/platforms/cpp/odbc/include/ignite/odbc/config/configuration.h
@@@ -62,14 -62,11 +62,17 @@@ namespace ignit
                      /** Connection attribute keyword for port attribute. */
                      static const std::string port;
  
 +                    /** Connection attribute keyword for distributed joins attribute. */
 +                    static const std::string distributedJoins;
 +
 +                    /** Connection attribute keyword for enforce join order attribute. */
 +                    static const std::string enforceJoinOrder;
 +
                      /** Connection attribute keyword for protocol version attribute. */
                      static const std::string protocolVersion;
+ 
+                     /** Connection attribute keyword for fetch results page size attribute. */
+                     static const std::string pageSize;
                  };
  
                  /** Default values for configuration. */
@@@ -96,11 -93,8 +99,14 @@@
                      /** Default value for port attribute. */
                      static const uint16_t port;
  
+                     /** Default value for fetch results page size attribute. */
+                     static const int32_t pageSize;
++
 +                    /** Default value for distributed joins attribute. */
 +                    static const bool distributedJoins;
 +
 +                    /** Default value for enforce join order attribute. */
 +                    static const bool enforceJoinOrder;
                  };
  
                  /**
@@@ -263,70 -263,53 +275,92 @@@
                  }
  
                  /**
 +                 * Check distributed joins flag.
 +                 *
 +                 * @return True if distributed joins are enabled.
 +                 */
 +                bool IsDistributedJoins() const
 +                {
 +                    return GetBoolValue(Key::distributedJoins, DefaultValue::distributedJoins);
 +                }
 +
 +                /**
 +                 * Set distributed joins.
 +                 *
 +                 * @param val Value to set.
 +                 */
 +                void SetDistributedJoins(bool val)
 +                {
 +                    SetBoolValue(Key::distributedJoins, val);
 +                }
 +
 +                /**
 +                 * Check enforce join order flag.
 +                 *
 +                 * @return True if enforcing of join order is enabled.
 +                 */
 +                bool IsEnforceJoinOrder() const
 +                {
 +                    return GetBoolValue(Key::enforceJoinOrder, DefaultValue::enforceJoinOrder);
 +                }
 +
 +                /**
 +                 * Set enforce joins.
 +                 *
 +                 * @param val Value to set.
 +                 */
 +                void SetEnforceJoinOrder(bool val)
 +                {
 +                    SetBoolValue(Key::enforceJoinOrder, val);
 +                }
 +
 +                /**
+                  * Get protocol version.
+                  *
+                  * @return Protocol version.
+                  */
+                 ProtocolVersion GetProtocolVersion() const;
+ 
+                 /**
+                  * Set protocol version.
+                  *
+                  * @param version Version to set.
+                  */
+                 void SetProtocolVersion(const std::string& version)
+                 {
+                     arguments[Key::protocolVersion] = version;
+                 }
+ 
+                 /**
 +                 * Get argument map.
 +                 *
 +                 * @return Argument map.
 +                 */
 +                const ArgumentMap& GetMap() const
 +                {
 +                    return arguments;
 +                }
 +
 +                /**
-                  * Get protocol version.
+                  * Get fetch results page size.
                   *
-                  * @return Protocol version.
+                  * @return Fetch results page size.
                   */
-                 ProtocolVersion GetProtocolVersion() const;
- 
+                 int32_t GetPageSize() const
+                 {
+                     return static_cast<int32_t>(GetIntValue(Key::pageSize, DefaultValue::pageSize));
+                 }
 -
                  /**
-                  * Set protocol version.
+                  * Set fetch results page size.
                   *
-                  * @param version Version to set.
+                  * @param size Fetch results page size.
                   */
-                 void SetProtocolVersion(const std::string& version);
+                 void SetPageSize(int32_t size)
+                 {
+                     arguments[Key::pageSize] = common::LexicalCast<std::string>(size);
+                 }
  
                  /**
 -                 * Get argument map.
 -                 *
 -                 * @return Argument map.
 -                 */
 -                const ArgumentMap& GetMap() const
 -                {
 -                    return arguments;
 -                }
 -
 -                /**
                   * Get string value from the config.
                   *
                   * @param key Configuration key.

http://git-wip-us.apache.org/repos/asf/ignite/blob/3b391849/modules/platforms/cpp/odbc/include/ignite/odbc/system/ui/dsn_configuration_window.h
----------------------------------------------------------------------
diff --cc modules/platforms/cpp/odbc/include/ignite/odbc/system/ui/dsn_configuration_window.h
index d2cb569,f034a8b..32f3520
--- a/modules/platforms/cpp/odbc/include/ignite/odbc/system/ui/dsn_configuration_window.h
+++ b/modules/platforms/cpp/odbc/include/ignite/odbc/system/ui/dsn_configuration_window.h
@@@ -46,10 -46,8 +46,12 @@@ namespace ignit
                          ID_ADDRESS_LABEL,
                          ID_CACHE_EDIT,
                          ID_CACHE_LABEL,
+                         ID_PAGE_SIZE_EDIT,
+                         ID_PAGE_SIZE_LABEL,
 +                        ID_DISTRIBUTED_JOINS_CHECK_BOX,
 +                        ID_ENFORCE_JOIN_ORDER_CHECK_BOX,
 +                        ID_PROTOCOL_VERSION_LABEL,
 +                        ID_PROTOCOL_VERSION_COMBO_BOX,
                          ID_OK_BUTTON,
                          ID_CANCEL_BUTTON
                      };
@@@ -120,18 -118,12 +122,24 @@@
                      /** DSN cache edit field. */
                      std::auto_ptr<Window> cacheEdit;
  
+                     /** DSN fetch page size edit field label. */
+                     std::auto_ptr<Window> pageSizeLabel;
+ 
+                     /** DSN fetch page size edit field. */
+                     std::auto_ptr<Window> pageSizeEdit;
+ 
 +                    /** Distributed joins CheckBox. */
 +                    std::auto_ptr<Window> distributedJoinsCheckBox;
 +
 +                    /** Enforce join order CheckBox. */
 +                    std::auto_ptr<Window> enforceJoinOrderCheckBox;
 +
 +                    /** Protocol version edit field. */
 +                    std::auto_ptr<Window> protocolVersionLabel;
 +
 +                    /** Protocol verion ComboBox. */
 +                    std::auto_ptr<Window> protocolVersionComboBox;
 +
                      /** Ok button. */
                      std::auto_ptr<Window> okButton;
  

http://git-wip-us.apache.org/repos/asf/ignite/blob/3b391849/modules/platforms/cpp/odbc/os/win/src/system/ui/dsn_configuration_window.cpp
----------------------------------------------------------------------
diff --cc modules/platforms/cpp/odbc/os/win/src/system/ui/dsn_configuration_window.cpp
index 663333a,49f87d8..a758bd9
--- a/modules/platforms/cpp/odbc/os/win/src/system/ui/dsn_configuration_window.cpp
+++ b/modules/platforms/cpp/odbc/os/win/src/system/ui/dsn_configuration_window.cpp
@@@ -30,7 -30,7 +30,7 @@@ namespace ignit
                  DsnConfigurationWindow::DsnConfigurationWindow(Window* parent, config::Configuration& config):
                      CustomWindow(parent, "IgniteConfigureDsn", "Configure Apache Ignite DSN"),
                      width(360),
-                     height(230),
 -                    height(200),
++                    height(270),
                      connectionSettingsGroupBox(),
                      nameLabel(),
                      nameEdit(),
@@@ -38,10 -38,8 +38,12 @@@
                      addressEdit(),
                      cacheLabel(),
                      cacheEdit(),
+                     pageSizeLabel(),
+                     pageSizeEdit(),
 +                    distributedJoinsCheckBox(),
 +                    enforceJoinOrderCheckBox(),
 +                    protocolVersionLabel(),
 +                    protocolVersionComboBox(),
                      okButton(),
                      cancelButton(),
                      config(config),
@@@ -117,40 -115,14 +119,50 @@@
  
                      rowPos += interval + rowSize;
  
+                     std::string tmp = common::LexicalCast<std::string>(config.GetPageSize());
+                     val = tmp.c_str();
+                     pageSizeLabel = CreateLabel(labelPosX, rowPos, labelSizeX,
+                         rowSize, "Page size:", ID_PAGE_SIZE_LABEL);
+ 
+                     pageSizeEdit = CreateEdit(editPosX, rowPos, editSizeX, 
+                         rowSize, val, ID_PAGE_SIZE_EDIT, ES_NUMBER);
+ 
++                    rowPos += interval + rowSize;
++
 +                    protocolVersionLabel = CreateLabel(labelPosX, rowPos, labelSizeX, rowSize,
 +                        "Protocol version:", ID_PROTOCOL_VERSION_LABEL);
 +                    protocolVersionComboBox = CreateComboBox(editPosX, rowPos, editSizeX, rowSize,
 +                        "Protocol version", ID_PROTOCOL_VERSION_COMBO_BOX);
 +
 +                    int id = 0;
 +
 +                    const ProtocolVersion::StringToVersionMap& versionMap = ProtocolVersion::GetMap();
 +
 +                    ProtocolVersion::StringToVersionMap::const_iterator it;
 +                    for (it = versionMap.begin(); it != versionMap.end(); ++it)
 +                    {
 +                        protocolVersionComboBox->AddString(it->first);
 +
 +                        if (it->second == config.GetProtocolVersion())
 +                            protocolVersionComboBox->SetSelection(id);
 +
 +                        ++id;
 +                    }
 +
 +                    rowPos += interval + rowSize;
 +
 +                    distributedJoinsCheckBox = CreateCheckBox(editPosX, rowPos, checkBoxSize, rowSize,
 +                        "Distributed Joins", ID_DISTRIBUTED_JOINS_CHECK_BOX, config.IsDistributedJoins());
 +
 +                    enforceJoinOrderCheckBox = CreateCheckBox(editPosX + checkBoxSize + interval, rowPos, checkBoxSize,
 +                        rowSize, "Enforce Join Order", ID_ENFORCE_JOIN_ORDER_CHECK_BOX, config.IsEnforceJoinOrder());
 +
 +                    if (!config.GetProtocolVersion().IsDistributedJoinsSupported())
 +                    {
 +                        distributedJoinsCheckBox->SetEnabled(false);
 +                        enforceJoinOrderCheckBox->SetEnabled(false);
 +                    }
 +
                      rowPos += interval * 2 + rowSize;
  
                      connectionSettingsGroupBox = CreateGroupBox(margin, sectionBegin, width - 2 * margin,
@@@ -265,29 -198,26 +277,37 @@@
                      std::string dsn;
                      std::string address;
                      std::string cache;
+                     std::string pageSizeStr;
 +                    std::string version;
 +
 +                    bool distributedJoins;
 +                    bool enforceJoinOrder;
  
                      nameEdit->GetText(dsn);
                      addressEdit->GetText(address);
                      cacheEdit->GetText(cache);
 +                    protocolVersionComboBox->GetText(version);
+                     pageSizeEdit->GetText(pageSizeStr);
+ 
+                     int32_t pageSize = common::LexicalCast<int32_t>(pageSizeStr);
+ 
+                     if (pageSize <= 0)
+                         pageSize = config.GetPageSize();
  
                      common::StripSurroundingWhitespaces(address);
                      common::StripSurroundingWhitespaces(dsn);
  
 +                    distributedJoins = distributedJoinsCheckBox->IsEnabled() && distributedJoinsCheckBox->IsChecked();
 +                    enforceJoinOrder = enforceJoinOrderCheckBox->IsEnabled() && enforceJoinOrderCheckBox->IsChecked();
 +
                      LOG_MSG("Retriving arguments:\n");
 -                    LOG_MSG("DSN:        %s\n", dsn.c_str());
 -                    LOG_MSG("Address:    %s\n", address.c_str());
 -                    LOG_MSG("Cache:      %s\n", cache.c_str());
 -                    LOG_MSG("Page size:  %d\n", pageSize);
 +                    LOG_MSG("DSN:                %s\n", dsn.c_str());
 +                    LOG_MSG("Address:            %s\n", address.c_str());
 +                    LOG_MSG("Cache:              %s\n", cache.c_str());
++                    LOG_MSG("Page size:          %d\n", pageSize);
 +                    LOG_MSG("Protocol version:   %s\n", version.c_str());
 +                    LOG_MSG("Distributed Joins:  %s\n", distributedJoins ? "true" : "false");
 +                    LOG_MSG("Enforce Join Order: %s\n", enforceJoinOrder ? "true" : "false");
  
                      if (dsn.empty())
                          throw IgniteError(IgniteError::IGNITE_ERR_GENERIC, "DSN name can not be empty.");
@@@ -295,9 -225,7 +315,10 @@@
                      cfg.SetDsn(dsn);
                      cfg.SetAddress(address);
                      cfg.SetCache(cache);
+                     cfg.SetPageSize(pageSize);
 +                    cfg.SetProtocolVersion(version);
 +                    cfg.SetDistributedJoins(distributedJoins);
 +                    cfg.SetEnforceJoinOrder(enforceJoinOrder);
                  }
              }
          }

http://git-wip-us.apache.org/repos/asf/ignite/blob/3b391849/modules/platforms/cpp/odbc/src/config/configuration.cpp
----------------------------------------------------------------------
diff --cc modules/platforms/cpp/odbc/src/config/configuration.cpp
index dbe40bd,74ccaaf..f39e562
--- a/modules/platforms/cpp/odbc/src/config/configuration.cpp
+++ b/modules/platforms/cpp/odbc/src/config/configuration.cpp
@@@ -38,24 -38,20 +38,25 @@@ namespace ignit
              const std::string Configuration::Key::address           = "address";
              const std::string Configuration::Key::server            = "server";
              const std::string Configuration::Key::port              = "port";
 +            const std::string Configuration::Key::distributedJoins  = "distributed_joins";
 +            const std::string Configuration::Key::enforceJoinOrder  = "enforce_join_order";
              const std::string Configuration::Key::protocolVersion   = "protocol_version";
+             const std::string Configuration::Key::pageSize          = "page_size";
  
-             const std::string Configuration::DefaultValue::dsn             = "Apache Ignite DSN";
-             const std::string Configuration::DefaultValue::driver          = "Apache Ignite";
-             const std::string Configuration::DefaultValue::cache           = "";
-             const std::string Configuration::DefaultValue::address         = "";
-             const std::string Configuration::DefaultValue::server          = "";
+             const std::string Configuration::DefaultValue::dsn      = "Apache Ignite DSN";
+             const std::string Configuration::DefaultValue::driver   = "Apache Ignite";
+             const std::string Configuration::DefaultValue::cache    = "";
+             const std::string Configuration::DefaultValue::address  = "";
+             const std::string Configuration::DefaultValue::server   = "";
  
-             const uint16_t Configuration::DefaultValue::port = 10800;
+             const uint16_t Configuration::DefaultValue::port    = 10800;
+             const int32_t Configuration::DefaultValue::pageSize = 1024;
  
 +            const bool Configuration::DefaultValue::distributedJoins = false;
 +            const bool Configuration::DefaultValue::enforceJoinOrder = false;
 +
              const ProtocolVersion& Configuration::DefaultValue::protocolVersion = ProtocolVersion::GetCurrent();
  
- 
              Configuration::Configuration() :
                  arguments()
              {

http://git-wip-us.apache.org/repos/asf/ignite/blob/3b391849/modules/platforms/cpp/odbc/src/dsn_config.cpp
----------------------------------------------------------------------
diff --cc modules/platforms/cpp/odbc/src/dsn_config.cpp
index a304567,356bcaa..2edeab0
--- a/modules/platforms/cpp/odbc/src/dsn_config.cpp
+++ b/modules/platforms/cpp/odbc/src/dsn_config.cpp
@@@ -91,11 -91,13 +91,17 @@@ namespace ignit
          void ReadDsnConfiguration(const char* dsn, Configuration& config)
          {
              std::string address = ReadDsnString(dsn, Configuration::Key::address, config.GetAddress().c_str());
+ 
              std::string server = ReadDsnString(dsn, Configuration::Key::server, config.GetHost().c_str());
+ 
              uint16_t port = ReadDsnInt(dsn, Configuration::Key::port, config.GetTcpPort());
+ 
              std::string cache = ReadDsnString(dsn, Configuration::Key::cache, config.GetCache().c_str());
+ 
 +            bool distributedJoins = ReadDsnBool(dsn, Configuration::Key::distributedJoins, config.IsDistributedJoins());
++
 +            bool enforceJoinOrder = ReadDsnBool(dsn, Configuration::Key::enforceJoinOrder, config.IsEnforceJoinOrder());
++
              std::string version = ReadDsnString(dsn, Configuration::Key::protocolVersion,
                  config.GetProtocolVersion().ToString().c_str());
  
@@@ -105,11 -110,8 +114,10 @@@
              config.SetHost(server);
              config.SetTcpPort(port);
              config.SetCache(cache);
 +            config.SetDistributedJoins(distributedJoins);
 +            config.SetEnforceJoinOrder(enforceJoinOrder);
              config.SetProtocolVersion(version);
- 
-             LOG_MSG("%d\n", __LINE__);
+             config.SetPageSize(pageSize);
          }
      }
  }