You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by is...@apache.org on 2020/12/31 05:07:49 UTC

[ignite] branch master updated: IGNITE-9109: SQL API for C++ thin

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

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


The following commit(s) were added to refs/heads/master by this push:
     new ecb345f  IGNITE-9109: SQL API for C++ thin
ecb345f is described below

commit ecb345f9655919c1589c41bc32372658532f6457
Author: Igor Sapego <is...@apache.org>
AuthorDate: Thu Dec 31 08:04:31 2020 +0300

    IGNITE-9109: SQL API for C++ thin
    
    This closes #8628
---
 modules/platforms/cpp/CMakeLists.txt               |   4 +-
 .../include/ignite/binary/binary_raw_reader.h      |   2 +-
 .../include/ignite/binary/binary_raw_writer.h      |   2 +-
 .../binary/include/ignite/binary/binary_reader.h   |   2 +-
 .../binary/include/ignite/binary/binary_writer.h   |   2 +-
 .../ignite/impl/binary/binary_reader_impl.h        | 161 +++++--
 .../ignite/impl/interop/interop_input_stream.h     |  27 +-
 .../include/ignite/impl/interop/interop_memory.h   |  18 +-
 .../binary/src/impl/binary/binary_reader_impl.cpp  | 284 ++++++++-----
 .../cpp/binary/src/impl/interop/interop_memory.cpp |  28 +-
 modules/platforms/cpp/core-test/CMakeLists.txt     |   3 +-
 .../cpp/core-test/project/vs/core-test.vcxproj     |   1 +
 .../core-test/project/vs/core-test.vcxproj.filters |   3 +
 modules/platforms/cpp/core/CMakeLists.txt          |   3 +-
 .../cpp/core/include/ignite/cache/cache.h          |   4 +-
 .../core/include/ignite/cache/query/query_cursor.h |   4 +-
 .../ignite/cache/query/query_fields_cursor.h       |   4 +-
 .../include/ignite/cache/query/query_fields_row.h  |   4 +-
 modules/platforms/cpp/core/include/ignite/ignite.h |   4 +-
 .../core/include/ignite/transactions/transaction.h |   4 +-
 .../include/ignite/transactions/transactions.h     |   4 +-
 .../cluster-compute-example/CMakeLists.txt         |   3 +-
 .../cpp/examples/compute-example/CMakeLists.txt    |   3 +-
 .../continuous-query-example/CMakeLists.txt        |   3 +-
 .../cpp/examples/odbc-example/CMakeLists.txt       |   3 +-
 .../cpp/examples/put-get-example/CMakeLists.txt    |   3 +-
 .../cpp/examples/query-example/CMakeLists.txt      |   3 +-
 .../thin-client-put-get-example/CMakeLists.txt     |   3 +-
 modules/platforms/cpp/ignite/CMakeLists.txt        |   2 +-
 modules/platforms/cpp/jni/CMakeLists.txt           |   3 +-
 modules/platforms/cpp/odbc-test/CMakeLists.txt     |   3 +-
 modules/platforms/cpp/odbc/CMakeLists.txt          |   3 +-
 .../platforms/cpp/thin-client-test/CMakeLists.txt  |   4 +-
 .../config/sql-query-fields-32.xml                 |  52 +++
 .../config/sql-query-fields-default.xml            | 125 ++++++
 .../thin-client-test/config/sql-query-fields.xml   |  35 ++
 .../thin-client-test/include/ignite/test_type.h    | 183 ++++++++
 .../project/vs/thin-client-test.vcxproj            |   5 +
 .../project/vs/thin-client-test.vcxproj.filters    |  15 +
 .../thin-client-test/src/sql_fields_query_test.cpp | 467 +++++++++++++++++++++
 modules/platforms/cpp/thin-client/CMakeLists.txt   |   4 +-
 .../ignite/impl/thin/cache/cache_client_proxy.h    |  12 +
 .../thin/cache/query/query_fields_cursor_impl.h    | 147 +++++++
 .../impl/thin/cache/query/query_fields_row_impl.h  | 197 +++++++++
 .../include/ignite/impl/thin/copyable.h            | 106 +++++
 .../include/ignite/impl/thin/copyable_writable.h   | 175 ++++++++
 .../include/ignite/thin/cache/cache_client.h       |  30 +-
 .../ignite/thin/cache/query/query_fields_cursor.h  |  94 +++++
 .../ignite/thin/cache/query/query_fields_row.h     | 104 +++++
 .../ignite/thin/cache/query/query_sql_fields.h     | 465 ++++++++++++++++++++
 .../include/ignite/thin/ignite_client.h            |   6 +-
 .../include/ignite/thin/transactions/transaction.h |   6 +-
 .../ignite/thin/transactions/transactions.h        |   6 +-
 .../cpp/thin-client/project/vs/thin-client.vcxproj |  12 +
 .../project/vs/thin-client.vcxproj.filters         |  36 ++
 .../src/cache/query/query_fields_cursor.cpp        |  69 +++
 .../src/cache/query/query_fields_row.cpp           |  59 +++
 .../src/impl/cache/cache_client_impl.cpp           |  25 +-
 .../thin-client/src/impl/cache/cache_client_impl.h |  14 +-
 .../src/impl/cache/cache_client_proxy.cpp          |   8 +
 .../thin-client/src/impl/cache/query/cursor_page.h | 127 ++++++
 .../impl/cache/query/query_fields_cursor_impl.h    | 229 ++++++++++
 .../src/impl/cache/query/query_fields_row_impl.h   | 129 ++++++
 .../cpp/thin-client/src/impl/data_router.h         |  15 +-
 .../platforms/cpp/thin-client/src/impl/message.cpp |  69 +++
 .../platforms/cpp/thin-client/src/impl/message.h   | 203 ++++++++-
 66 files changed, 3605 insertions(+), 228 deletions(-)

diff --git a/modules/platforms/cpp/CMakeLists.txt b/modules/platforms/cpp/CMakeLists.txt
index 4e95887..7b8e83b 100644
--- a/modules/platforms/cpp/CMakeLists.txt
+++ b/modules/platforms/cpp/CMakeLists.txt
@@ -41,14 +41,14 @@ endif("${isSystemDir}" STREQUAL "-1")
 if (WIN32)
     add_definitions(-DUNICODE=1)
 
-    add_compile_options(/source-charset:utf-8)
-
     set (CMAKE_LIBRARY_OUTPUT_DIRECTORY ${CMAKE_BINARY_DIR})
 
     set (CMAKE_RUNTIME_OUTPUT_DIRECTORY ${CMAKE_BINARY_DIR})
 endif()
 
 if (MSVC)
+    add_compile_options(/source-charset:utf-8 /execution-charset:utf-8)
+
     add_definitions(-D_CRT_SECURE_NO_WARNINGS -D_SCL_SECURE_NO_WARNINGS)
 endif()
 
diff --git a/modules/platforms/cpp/binary/include/ignite/binary/binary_raw_reader.h b/modules/platforms/cpp/binary/include/ignite/binary/binary_raw_reader.h
index eb0391c..d0845ab 100644
--- a/modules/platforms/cpp/binary/include/ignite/binary/binary_raw_reader.h
+++ b/modules/platforms/cpp/binary/include/ignite/binary/binary_raw_reader.h
@@ -43,7 +43,7 @@ namespace ignite
         /**
          * Binary raw reader.
          *
-         * This class implemented as a reference to an implementation so copying
+         * This class is implemented as a reference to an implementation so copying
          * of this class instance will only create another reference to the same
          * underlying object.
          *
diff --git a/modules/platforms/cpp/binary/include/ignite/binary/binary_raw_writer.h b/modules/platforms/cpp/binary/include/ignite/binary/binary_raw_writer.h
index 8150db0..18319fa 100644
--- a/modules/platforms/cpp/binary/include/ignite/binary/binary_raw_writer.h
+++ b/modules/platforms/cpp/binary/include/ignite/binary/binary_raw_writer.h
@@ -41,7 +41,7 @@ namespace ignite
         /**
          * Binary raw writer.
          *
-         * This class implemented as a reference to an implementation so copying
+         * This class is implemented as a reference to an implementation so copying
          * of this class instance will only create another reference to the same
          * underlying object.
          *
diff --git a/modules/platforms/cpp/binary/include/ignite/binary/binary_reader.h b/modules/platforms/cpp/binary/include/ignite/binary/binary_reader.h
index 7a9eb68..a939805 100644
--- a/modules/platforms/cpp/binary/include/ignite/binary/binary_reader.h
+++ b/modules/platforms/cpp/binary/include/ignite/binary/binary_reader.h
@@ -40,7 +40,7 @@ namespace ignite
         /**
          * Binary reader.
          *
-         * This class implemented as a reference to an implementation so copying
+         * This class is implemented as a reference to an implementation so copying
          * of this class instance will only create another reference to the same
          * underlying object.
          *
diff --git a/modules/platforms/cpp/binary/include/ignite/binary/binary_writer.h b/modules/platforms/cpp/binary/include/ignite/binary/binary_writer.h
index c293935..575139c 100644
--- a/modules/platforms/cpp/binary/include/ignite/binary/binary_writer.h
+++ b/modules/platforms/cpp/binary/include/ignite/binary/binary_writer.h
@@ -37,7 +37,7 @@ namespace ignite
         /**
          * Binary writer.
          *
-         * This class implemented as a reference to an implementation so copying
+         * This class is implemented as a reference to an implementation so copying
          * of this class instance will only create another reference to the same
          * underlying object.
          *
diff --git a/modules/platforms/cpp/binary/include/ignite/impl/binary/binary_reader_impl.h b/modules/platforms/cpp/binary/include/ignite/impl/binary/binary_reader_impl.h
index b618418..b165d8e 100644
--- a/modules/platforms/cpp/binary/include/ignite/impl/binary/binary_reader_impl.h
+++ b/modules/platforms/cpp/binary/include/ignite/impl/binary/binary_reader_impl.h
@@ -20,6 +20,9 @@
 
 #include <stdint.h>
 
+#include <vector>
+#include <string>
+
 #include <ignite/common/common.h>
 #include <ignite/common/fixed_size_array.h>
 
@@ -1080,8 +1083,41 @@ namespace ignite
 
                         default:
                         {
-                            IGNITE_ERROR_2(ignite::IgniteError::IGNITE_ERR_BINARY, 
-                                           "Unexpected header during deserialization: ", (hdr & 0xFF));
+                            IGNITE_ERROR_2(ignite::IgniteError::IGNITE_ERR_BINARY,
+                                "Unexpected header during deserialization: ", static_cast<int>(hdr & 0xFF));
+                        }
+                    }
+                }
+
+                /**
+                 * Read object.
+                 *
+                 * @return Read object.
+                 */
+                template<typename R, typename T>
+                void ReadTopObject0(std::vector<T>& res)
+                {
+                    int8_t hdr = stream->ReadInt8();
+
+                    switch (hdr)
+                    {
+                        case IGNITE_TYPE_ARRAY:
+                        {
+                            int32_t elementNum = stream->ReadInt32();
+
+                            res.clear();
+                            res.reserve(static_cast<size_t>(elementNum));
+
+                            for (int32_t i = 0; i < elementNum; ++i)
+                                res.push_back(ReadObject<T>());
+
+                            return;
+                        }
+
+                        default:
+                        {
+                            IGNITE_ERROR_2(ignite::IgniteError::IGNITE_ERR_BINARY,
+                                "Unexpected header during deserialization: ", static_cast<int>(hdr & 0xFF));
                         }
                     }
                 }
@@ -1269,36 +1305,33 @@ namespace ignite
                     const char* fieldName, 
                     T(*func) (interop::InteropInputStream*), 
                     const int8_t expHdr, 
-                    T dflt
-                )
+                    T dflt)
                 {
-                    {
-                        CheckRawMode(false);
-                        CheckSingleMode(true);
+                    CheckRawMode(false);
+                    CheckSingleMode(true);
 
-                        int32_t fieldId = idRslvr->GetFieldId(typeId, fieldName);
-                        int32_t fieldPos = FindField(fieldId);
+                    int32_t fieldId = idRslvr->GetFieldId(typeId, fieldName);
+                    int32_t fieldPos = FindField(fieldId);
 
-                        if (fieldPos <= 0)
-                            return dflt;
+                    if (fieldPos <= 0)
+                        return dflt;
 
-                        stream->Position(fieldPos);
+                    stream->Position(fieldPos);
 
-                        int8_t typeId = stream->ReadInt8();
-                        
-                        if (typeId == IGNITE_HDR_NULL)
-                            return dflt;
+                    int8_t typeId = stream->ReadInt8();
 
-                        if (typeId != expHdr)
-                        {
-                            int32_t pos = stream->Position();
+                    if (typeId == IGNITE_HDR_NULL)
+                        return dflt;
 
-                            IGNITE_ERROR_FORMATTED_3(IgniteError::IGNITE_ERR_BINARY, "Invalid type ID", 
-                                "position", pos, "expected", static_cast<int>(expHdr), "actual", static_cast<int>(typeId))
-                        }
+                    if (typeId != expHdr)
+                    {
+                        int32_t pos = stream->Position();
 
-                        return func(stream);
+                        IGNITE_ERROR_FORMATTED_3(IgniteError::IGNITE_ERR_BINARY, "Invalid type ID",
+                            "position", pos, "expected", static_cast<int>(expHdr), "actual", static_cast<int>(typeId))
                     }
+
+                    return func(stream);
                 }
 
                 /**
@@ -1378,29 +1411,63 @@ namespace ignite
                     T* res,
                     const int32_t len,
                     interop::InteropInputStream* stream,
-                    void(*func)(interop::InteropInputStream*, T* const, const int32_t),
+                    void(*func)(interop::InteropInputStream*, T*, int32_t),
                     const int8_t expHdr
                 )
                 {
+                    int8_t hdr = stream->ReadInt8();
+
+                    if (hdr == expHdr)
                     {
-                        int8_t hdr = stream->ReadInt8();
+                        int32_t realLen = stream->ReadInt32();
 
-                        if (hdr == expHdr)
-                        {
-                            int32_t realLen = stream->ReadInt32();
+                        if (realLen == 0 || (res && len >= realLen))
+                            func(stream, res, realLen);
+                        else
+                            stream->Position(stream->Position() - 5);
 
-                            if (realLen == 0 || (res && len >= realLen))
-                                func(stream, res, realLen);
-                            else
-                                stream->Position(stream->Position() - 5);
+                        return realLen;
+                    }
+                    else if (hdr != IGNITE_HDR_NULL)
+                        ThrowOnInvalidHeader(stream->Position() - 1, expHdr, hdr);
 
-                            return realLen;
-                        }
-                        else if (hdr != IGNITE_HDR_NULL)
-                            ThrowOnInvalidHeader(stream->Position() - 1, expHdr, hdr);
+                    return -1;
+                }
 
-                        return -1;
+                /**
+                 * Internal read array to vector routine.
+                 *
+                 * @param res Resulting array.
+                 * @param stream Stream.
+                 * @param func Function to be invoked on stream.
+                 * @param expHdr Expected header.
+                 * @return Length.
+                 */
+                template<typename T>
+                static void ReadArrayToVectorInternal(
+                        std::vector<T>& res,
+                        interop::InteropInputStream* stream,
+                        void(*func)(interop::InteropInputStream*, T*, int32_t),
+                        const int8_t expHdr
+                )
+                {
+                    int8_t hdr = stream->ReadInt8();
+
+                    if (hdr == expHdr)
+                    {
+                        int32_t realLen = stream->ReadInt32();
+
+                        if (realLen > 0)
+                        {
+                            res.resize(realLen, 0);
+
+                            func(stream, &res[0], realLen);
+                        }
                     }
+                    else if (hdr == IGNITE_HDR_NULL)
+                        res.clear();
+                    else
+                        ThrowOnInvalidHeader(stream->Position() - 1, expHdr, hdr);
                 }
 
                 /**
@@ -1581,6 +1648,26 @@ namespace ignite
             BinaryReaderImpl::ReadTopObject0<ignite::binary::BinaryReader, std::string>(std::string& res);
 
             template<>
+            void IGNITE_IMPORT_EXPORT
+            BinaryReaderImpl::ReadTopObject0<
+                ignite::binary::BinaryReader, std::vector<int8_t> >(std::vector<int8_t>& res);
+
+            template<>
+            void IGNITE_IMPORT_EXPORT
+            BinaryReaderImpl::ReadTopObject0<
+                ignite::binary::BinaryReader, std::vector<int16_t> >(std::vector<int16_t>& res);
+
+            template<>
+            void IGNITE_IMPORT_EXPORT
+            BinaryReaderImpl::ReadTopObject0<
+                ignite::binary::BinaryReader, std::vector<int32_t> >(std::vector<int32_t>& res);
+
+            template<>
+            void IGNITE_IMPORT_EXPORT
+            BinaryReaderImpl::ReadTopObject0<
+                ignite::binary::BinaryReader, std::vector<int64_t> >(std::vector<int64_t>& res);
+
+            template<>
             inline int8_t BinaryReaderImpl::GetNull() const
             {
                 return 0;
diff --git a/modules/platforms/cpp/binary/include/ignite/impl/interop/interop_input_stream.h b/modules/platforms/cpp/binary/include/ignite/impl/interop/interop_input_stream.h
index 4dc8f84..3e14171 100644
--- a/modules/platforms/cpp/binary/include/ignite/impl/interop/interop_input_stream.h
+++ b/modules/platforms/cpp/binary/include/ignite/impl/interop/interop_input_stream.h
@@ -36,7 +36,7 @@ namespace ignite
                  *
                  * @param mem Memory.
                  */
-                InteropInputStream(InteropMemory* mem);
+                explicit InteropInputStream(InteropMemory* mem);
 
                 /**
                  * Read signed 8-byte int.
@@ -59,7 +59,7 @@ namespace ignite
                  * @param res Allocated array.
                  * @param len Length.
                  */
-                void ReadInt8Array(int8_t* const res, const int32_t len);
+                void ReadInt8Array(int8_t* res, int32_t len);
 
                 /**
                  * Read bool.
@@ -74,7 +74,7 @@ namespace ignite
                  * @param res Allocated array.
                  * @param len Length.
                  */
-                void ReadBoolArray(bool* const res, const int32_t len);
+                void ReadBoolArray(bool* res, int32_t len);
 
                 /**
                  * Read signed 16-byte int.
@@ -97,7 +97,7 @@ namespace ignite
                  * @param res Allocated array.
                  * @param len Length.
                  */
-                void ReadInt16Array(int16_t* const res, const int32_t len);
+                void ReadInt16Array(int16_t* res, int32_t len);
 
                 /**
                  * Read unsigned 16-byte int.
@@ -112,7 +112,7 @@ namespace ignite
                  * @param res Allocated array.
                  * @param len Length.
                  */
-                void ReadUInt16Array(uint16_t* const res, const int32_t len);
+                void ReadUInt16Array(uint16_t* res, int32_t len);
 
                 /**
                  * Read signed 32-byte int.
@@ -135,7 +135,7 @@ namespace ignite
                  * @param res Allocated array.
                  * @param len Length.
                  */
-                void ReadInt32Array(int32_t* const res, const int32_t len);
+                void ReadInt32Array(int32_t* res, int32_t len);
 
                 /**
                  * Read signed 64-byte int.
@@ -150,7 +150,7 @@ namespace ignite
                  * @param res Allocated array.
                  * @param len Length.
                  */
-                void ReadInt64Array(int64_t* const res, const int32_t len);
+                void ReadInt64Array(int64_t* res, int32_t len);
 
                 /**
                  * Read float.
@@ -165,7 +165,7 @@ namespace ignite
                  * @param res Allocated array.
                  * @param len Length.
                  */
-                void ReadFloatArray(float* const res, const int32_t len);
+                void ReadFloatArray(float* res, int32_t len);
 
                 /**
                  * Read double.
@@ -180,7 +180,7 @@ namespace ignite
                  * @param res Allocated array.
                  * @param len Length.
                  */
-                void ReadDoubleArray(double* const res, const int32_t len);
+                void ReadDoubleArray(double* res, int32_t len);
 
                 /**
                  * Get remaining bytes.
@@ -215,6 +215,15 @@ namespace ignite
                  */
                 void Synchronize();
 
+                /**
+                 * Get memory.
+                 * @return Underlying memory.
+                 */
+                InteropMemory* GetMemory()
+                {
+                    return mem;
+                }
+
             private:
                 /** Memory. */
                 InteropMemory* mem; 
diff --git a/modules/platforms/cpp/binary/include/ignite/impl/interop/interop_memory.h b/modules/platforms/cpp/binary/include/ignite/impl/interop/interop_memory.h
index d991fe4..22955ed 100644
--- a/modules/platforms/cpp/binary/include/ignite/impl/interop/interop_memory.h
+++ b/modules/platforms/cpp/binary/include/ignite/impl/interop/interop_memory.h
@@ -234,7 +234,7 @@ namespace ignite
                 /**
                  * Reallocate memory.
                  *
-                 * @param cap Desired capactiy.
+                 * @param cap Desired capacity.
                  */
                 virtual void Reallocate(int32_t cap) = 0;
             protected:
@@ -260,7 +260,7 @@ namespace ignite
                  *
                  * @param memPtr Memory pointer.
                  */
-                explicit InteropUnpooledMemory(int8_t* memPtr);
+                explicit InteropUnpooledMemory(int8_t* memPtr = 0);
 
                 /**
                  * Destructor.
@@ -268,7 +268,21 @@ namespace ignite
                 ~InteropUnpooledMemory();
 
                 virtual void Reallocate(int32_t cap);
+
+                /**
+                 * Try get owning copy.
+                 *
+                 * @param mem Memory instance to transfer ownership to.
+                 * @return True on success
+                 */
+                bool TryGetOwnership(InteropUnpooledMemory& mem);
+
             private:
+                /**
+                 * Release all resources.
+                 */
+                void CleanUp();
+
                 /** Whether this instance is owner of memory chunk. */
                 bool owning; 
 
diff --git a/modules/platforms/cpp/binary/src/impl/binary/binary_reader_impl.cpp b/modules/platforms/cpp/binary/src/impl/binary/binary_reader_impl.cpp
index 08119cd..af768b1 100644
--- a/modules/platforms/cpp/binary/src/impl/binary/binary_reader_impl.cpp
+++ b/modules/platforms/cpp/binary/src/impl/binary/binary_reader_impl.cpp
@@ -60,7 +60,7 @@ namespace ignite
                 return ReadRaw<int8_t>(BinaryUtils::ReadInt8);
             }
 
-            int32_t BinaryReaderImpl::ReadInt8Array(int8_t* res, const int32_t len)
+            int32_t BinaryReaderImpl::ReadInt8Array(int8_t* res, int32_t len)
             {
                 return ReadRawArray<int8_t>(res, len, BinaryUtils::ReadInt8Array, IGNITE_TYPE_ARRAY_BYTE);
             }
@@ -799,146 +799,178 @@ namespace ignite
 
             void BinaryReaderImpl::Skip()
             {
-                int8_t typeId = stream->ReadInt8();
-                switch (typeId) {
-                case IGNITE_TYPE_OPTM_MARSH:
+                int8_t hdr = stream->ReadInt8();
+                switch (hdr)
                 {
-                    int32_t realLen = stream->ReadInt32();
-                    stream->Ignore(realLen);
-                    return;
-                }
+                    case IGNITE_TYPE_BINARY:
+                    {
+                        int32_t portLen = stream->ReadInt32(); // Total length of binary object.
+                        stream->Ignore(portLen + 4);
+                        return;
+                    }
 
-                case IGNITE_TYPE_BYTE:
-                case IGNITE_TYPE_BOOL:
-                    stream->Ignore(1);
-                    return;
+                    case IGNITE_TYPE_DECIMAL:
+                    {
+                        stream->Ignore(4); // scale
+                        int32_t magLen = stream->ReadInt32();
 
-                case IGNITE_TYPE_SHORT:
-                case IGNITE_TYPE_CHAR:
-                    stream->Ignore(2);
-                    return;
+                        stream->Ignore(magLen);
+                        return;
+                    }
 
-                case IGNITE_TYPE_INT:
-                case IGNITE_TYPE_FLOAT:
-                    stream->Ignore(4);
-                    return;
+                    case IGNITE_TYPE_OPTM_MARSH:
+                    {
+                        int32_t realLen = stream->ReadInt32();
+                        stream->Ignore(realLen);
+                        return;
+                    }
 
-                case IGNITE_TYPE_LONG:
-                case IGNITE_TYPE_DOUBLE:
-                case IGNITE_TYPE_DATE:
-                case IGNITE_TYPE_TIME:
-                    stream->Ignore(8);
-                    return;
+                    case IGNITE_TYPE_BYTE:
+                    case IGNITE_TYPE_BOOL:
+                    {
+                        stream->Ignore(1);
+                        return;
+                    }
 
-                case IGNITE_TYPE_UUID:
-                    stream->Ignore(16);
-                    return;
+                    case IGNITE_TYPE_SHORT:
+                    case IGNITE_TYPE_CHAR:
+                    {
+                        stream->Ignore(2);
+                        return;
+                    }
 
-                case IGNITE_TYPE_STRING:
-                case IGNITE_TYPE_ARRAY_BYTE:
-                case IGNITE_TYPE_ARRAY_BOOL:
-                {
-                    int32_t realLen = stream->ReadInt32();
-                    if (realLen > 0)
-                        stream->Ignore(realLen);
+                    case IGNITE_TYPE_INT:
+                    case IGNITE_TYPE_FLOAT:
+                    {
+                        stream->Ignore(4);
+                        return;
+                    }
 
-                    return;
-                }
+                    case IGNITE_TYPE_LONG:
+                    case IGNITE_TYPE_DOUBLE:
+                    case IGNITE_TYPE_DATE:
+                    case IGNITE_TYPE_TIME:
+                    case IGNITE_TYPE_ENUM:
+                    case IGNITE_TYPE_BINARY_ENUM:
+                    {
+                        stream->Ignore(8);
+                        return;
+                    }
 
-                case IGNITE_TYPE_ARRAY_SHORT:
-                case IGNITE_TYPE_ARRAY_CHAR:
-                {
-                    int32_t realLen = stream->ReadInt32();
-                    if (realLen > 0)
-                        stream->Ignore(realLen * 2);
+                    case IGNITE_TYPE_UUID:
+                    {
+                        stream->Ignore(16);
+                        return;
+                    }
 
-                    return;
-                }
+                    case IGNITE_TYPE_STRING:
+                    case IGNITE_TYPE_ARRAY_BYTE:
+                    case IGNITE_TYPE_ARRAY_BOOL:
+                    {
+                        int32_t realLen = stream->ReadInt32();
+                        if (realLen > 0)
+                            stream->Ignore(realLen);
 
-                case IGNITE_TYPE_ARRAY_INT:
-                case IGNITE_TYPE_ARRAY_FLOAT:
-                {
-                    int32_t realLen = stream->ReadInt32();
-                    if (realLen > 0)
-                        stream->Ignore(realLen * 4);
+                        return;
+                    }
 
-                    return;
-                }
+                    case IGNITE_TYPE_ARRAY_SHORT:
+                    case IGNITE_TYPE_ARRAY_CHAR:
+                    {
+                        int32_t realLen = stream->ReadInt32();
+                        if (realLen > 0)
+                            stream->Ignore(realLen * 2);
 
-                case IGNITE_TYPE_ARRAY_LONG:
-                case IGNITE_TYPE_ARRAY_DOUBLE:
-                {
-                    int32_t realLen = stream->ReadInt32();
-                    if (realLen > 0)
-                        stream->Ignore(realLen * 8);
+                        return;
+                    }
 
-                    return;
-                }
+                    case IGNITE_TYPE_ARRAY_INT:
+                    case IGNITE_TYPE_ARRAY_FLOAT:
+                    {
+                        int32_t realLen = stream->ReadInt32();
+                        if (realLen > 0)
+                            stream->Ignore(realLen * 4);
 
-                case IGNITE_TYPE_ARRAY_STRING:
-                case IGNITE_TYPE_ARRAY_UUID:
-                case IGNITE_TYPE_ARRAY_DATE:
-                case IGNITE_TYPE_ARRAY_TIMESTAMP:
-                case IGNITE_TYPE_ARRAY_TIME:
-                case IGNITE_TYPE_ARRAY:
-                {
-                    int32_t cnt = stream->ReadInt32();
-                    for (int32_t i = 0; i < cnt; i++)
-                        Skip();
+                        return;
+                    }
 
-                    return;
-                }
+                    case IGNITE_TYPE_ARRAY_LONG:
+                    case IGNITE_TYPE_ARRAY_DOUBLE:
+                    {
+                        int32_t realLen = stream->ReadInt32();
+                        if (realLen > 0)
+                            stream->Ignore(realLen * 8);
 
-                case IGNITE_TYPE_COLLECTION:
-                {
-                    int32_t cnt = stream->ReadInt32();
+                        return;
+                    }
 
-                    // Collection type ID.
-                    stream->Ignore(1);
+                    case IGNITE_TYPE_ARRAY_STRING:
+                    case IGNITE_TYPE_ARRAY_UUID:
+                    case IGNITE_TYPE_ARRAY_DATE:
+                    case IGNITE_TYPE_ARRAY_TIMESTAMP:
+                    case IGNITE_TYPE_ARRAY_TIME:
+                    case IGNITE_TYPE_ARRAY_DECIMAL:
+                    case IGNITE_TYPE_ARRAY:
+                    {
+                        int32_t cnt = stream->ReadInt32();
+                        for (int32_t i = 0; i < cnt; i++)
+                            Skip();
 
-                    for (int32_t i = 0; i < cnt; i++)
-                        Skip();
+                        return;
+                    }
 
-                    return;
-                }
+                    case IGNITE_TYPE_COLLECTION:
+                    {
+                        int32_t cnt = stream->ReadInt32();
 
-                case IGNITE_TYPE_MAP:
-                {
-                    int32_t cnt = stream->ReadInt32();
+                        // Collection type ID.
+                        stream->Ignore(1);
 
-                    // Map type ID.
-                    stream->Ignore(1);
+                        for (int32_t i = 0; i < cnt; i++)
+                            Skip();
 
-                    for (int32_t i = 0; i < cnt; i++)
-                    {
-                        Skip();
-                        Skip();
+                        return;
                     }
 
-                    return;
-                }
+                    case IGNITE_TYPE_MAP:
+                    {
+                        int32_t cnt = stream->ReadInt32();
 
-                case IGNITE_TYPE_TIMESTAMP:
-                    stream->Ignore(12);
-                    return;
+                        // Map type ID.
+                        stream->Ignore(1);
 
-                case IGNITE_HDR_FULL:
-                {
-                    int32_t objectBegin = stream->Position() - 1;
-                    int32_t len = stream->ReadInt32(objectBegin + IGNITE_OFFSET_LEN);
-                    stream->Position(objectBegin + len);
-                    return;
-                }
+                        for (int32_t i = 0; i < cnt; i++)
+                        {
+                            Skip();
+                            Skip();
+                        }
 
-                case IGNITE_HDR_NULL:
-                    return;
+                        return;
+                    }
 
-                default:
-                {
-                    int32_t pos = stream->Position() - 1;
-                    IGNITE_ERROR_FORMATTED_2(IgniteError::IGNITE_ERR_BINARY, "Invalid header", "position", pos,
-                        "unsupported type", static_cast<int>(typeId));
-                }
+                    case IGNITE_TYPE_TIMESTAMP:
+                    {
+                        stream->Ignore(12);
+                        return;
+                    }
+
+                    case IGNITE_HDR_FULL:
+                    {
+                        int32_t objectBegin = stream->Position() - 1;
+                        int32_t objectLen = stream->ReadInt32(objectBegin + IGNITE_OFFSET_LEN);
+                        stream->Position(objectBegin + objectLen);
+                        return;
+                    }
+
+                    case IGNITE_HDR_NULL:
+                        return;
+
+                    default:
+                    {
+                        int32_t pos = stream->Position() - 1;
+                        IGNITE_ERROR_FORMATTED_2(IgniteError::IGNITE_ERR_BINARY, "Invalid header", "position", pos,
+                            "unsupported type", static_cast<int>(hdr));
+                    }
                 }
             }
 
@@ -1063,6 +1095,34 @@ namespace ignite
                 }
             }
 
+            template<>
+            void BinaryReaderImpl::ReadTopObject0<
+                    ignite::binary::BinaryReader, std::vector<int8_t> >(std::vector<int8_t>& res)
+            {
+                ReadArrayToVectorInternal<int8_t>(res, stream, BinaryUtils::ReadInt8Array, IGNITE_TYPE_ARRAY_BYTE);
+            }
+
+            template<>
+            void BinaryReaderImpl::ReadTopObject0<
+                    ignite::binary::BinaryReader, std::vector<int16_t> >(std::vector<int16_t>& res)
+            {
+                ReadArrayToVectorInternal<int16_t>(res, stream, BinaryUtils::ReadInt16Array, IGNITE_TYPE_ARRAY_SHORT);
+            }
+
+            template<>
+            void BinaryReaderImpl::ReadTopObject0<
+                    ignite::binary::BinaryReader, std::vector<int32_t> >(std::vector<int32_t>& res)
+            {
+                ReadArrayToVectorInternal<int32_t>(res, stream, BinaryUtils::ReadInt32Array, IGNITE_TYPE_ARRAY_INT);
+            }
+
+            template<>
+            void BinaryReaderImpl::ReadTopObject0<
+                    ignite::binary::BinaryReader, std::vector<int64_t> >(std::vector<int64_t>& res)
+            {
+                ReadArrayToVectorInternal<int64_t>(res, stream, BinaryUtils::ReadInt64Array, IGNITE_TYPE_ARRAY_LONG);
+            }
+
             template <typename T>
             T BinaryReaderImpl::ReadTopObject0(const int8_t expHdr, T(*func)(InteropInputStream*))
             {
diff --git a/modules/platforms/cpp/binary/src/impl/interop/interop_memory.cpp b/modules/platforms/cpp/binary/src/impl/interop/interop_memory.cpp
index c0d28aa..443bd25 100644
--- a/modules/platforms/cpp/binary/src/impl/interop/interop_memory.cpp
+++ b/modules/platforms/cpp/binary/src/impl/interop/interop_memory.cpp
@@ -155,10 +155,7 @@ namespace ignite
 
             InteropUnpooledMemory::~InteropUnpooledMemory()
             {
-                if (owning) {
-                    free(Data());
-                    free(memPtr);
-                }
+                CleanUp();
             }
 
             void InteropUnpooledMemory::Reallocate(int32_t cap)
@@ -171,6 +168,29 @@ namespace ignite
                 Data(memPtr, realloc(Data(memPtr), cap));
                 Capacity(memPtr, cap);
             }
+
+            bool InteropUnpooledMemory::TryGetOwnership(InteropUnpooledMemory &mem)
+            {
+                if (!owning)
+                    return false;
+
+                mem.CleanUp();
+                mem.owning = true;
+                mem.memPtr = memPtr;
+
+                owning = false;
+
+                return true;
+            }
+
+            void InteropUnpooledMemory::CleanUp()
+            {
+                if (owning)
+                {
+                    free(Data());
+                    free(memPtr);
+                }
+            }
         }
     }
 }
diff --git a/modules/platforms/cpp/core-test/CMakeLists.txt b/modules/platforms/cpp/core-test/CMakeLists.txt
index 9372263..d308aa1 100644
--- a/modules/platforms/cpp/core-test/CMakeLists.txt
+++ b/modules/platforms/cpp/core-test/CMakeLists.txt
@@ -25,7 +25,8 @@ endif()
 
 find_package(Boost 1.53 REQUIRED COMPONENTS unit_test_framework chrono thread system)
 
-include_directories(include ${Boost_INCLUDE_DIRS} ${JNI_INCLUDE_DIRS})
+include_directories(SYSTEM ${Boost_INCLUDE_DIRS} ${JNI_INCLUDE_DIRS})
+include_directories(include)
 
 set(SOURCES src/reference_test.cpp
         src/bits_test.cpp
diff --git a/modules/platforms/cpp/core-test/project/vs/core-test.vcxproj b/modules/platforms/cpp/core-test/project/vs/core-test.vcxproj
index 0923a80..fc81a15 100644
--- a/modules/platforms/cpp/core-test/project/vs/core-test.vcxproj
+++ b/modules/platforms/cpp/core-test/project/vs/core-test.vcxproj
@@ -65,6 +65,7 @@
     <ClCompile Include="..\..\src\cache_invoke_test.cpp" />
     <ClCompile Include="..\..\src\cluster_group_test.cpp" />
     <ClCompile Include="..\..\src\cluster_node_test.cpp" />
+    <ClCompile Include="..\..\src\cluster_test.cpp" />
     <ClCompile Include="..\..\src\compute_test.cpp" />
     <ClCompile Include="..\..\src\concurrent_test.cpp" />
     <ClCompile Include="..\..\src\date_time_test.cpp" />
diff --git a/modules/platforms/cpp/core-test/project/vs/core-test.vcxproj.filters b/modules/platforms/cpp/core-test/project/vs/core-test.vcxproj.filters
index 8bc31ca..79abb4e 100644
--- a/modules/platforms/cpp/core-test/project/vs/core-test.vcxproj.filters
+++ b/modules/platforms/cpp/core-test/project/vs/core-test.vcxproj.filters
@@ -94,6 +94,9 @@
     <ClCompile Include="..\..\src\cluster_node_test.cpp">
       <Filter>Code</Filter>
     </ClCompile>
+    <ClCompile Include="..\..\src\cluster_test.cpp">
+      <Filter>Code</Filter>
+    </ClCompile>
     <ClCompile Include="..\..\src\affinity_test.cpp">
       <Filter>Code</Filter>
     </ClCompile>
diff --git a/modules/platforms/cpp/core/CMakeLists.txt b/modules/platforms/cpp/core/CMakeLists.txt
index 6c631bc..5cd1a99 100644
--- a/modules/platforms/cpp/core/CMakeLists.txt
+++ b/modules/platforms/cpp/core/CMakeLists.txt
@@ -17,7 +17,8 @@
 
 project(ignite)
 
-include_directories(${JNI_INCLUDE_DIRS} include)
+include_directories(SYSTEM ${JNI_INCLUDE_DIRS})
+include_directories(include)
 
 set(TARGET ${PROJECT_NAME})
 
diff --git a/modules/platforms/cpp/core/include/ignite/cache/cache.h b/modules/platforms/cpp/core/include/ignite/cache/cache.h
index 01c1c81..1f1a4bf 100644
--- a/modules/platforms/cpp/core/include/ignite/cache/cache.h
+++ b/modules/platforms/cpp/core/include/ignite/cache/cache.h
@@ -56,9 +56,9 @@ namespace ignite
          * copy-constructable and assignable. Also BinaryType class
          * template should be specialized for both types.
          *
-         * This class implemented as a reference to an implementation so copying
+         * This class is implemented as a reference to an implementation so copying
          * of this class instance will only create another reference to the same
-         * underlying object. Underlying object released automatically once all
+         * underlying object. Underlying object will be released automatically once all
          * the instances are destructed.
          *
          * @tparam K Cache key type.
diff --git a/modules/platforms/cpp/core/include/ignite/cache/query/query_cursor.h b/modules/platforms/cpp/core/include/ignite/cache/query/query_cursor.h
index e77706b..fc41e67 100644
--- a/modules/platforms/cpp/core/include/ignite/cache/query/query_cursor.h
+++ b/modules/platforms/cpp/core/include/ignite/cache/query/query_cursor.h
@@ -45,9 +45,9 @@ namespace ignite
              * copy-constructable and assignable. Also BinaryType class
              * template should be specialized for both types.
              *
-             * This class implemented as a reference to an implementation so copying
+             * This class is implemented as a reference to an implementation so copying
              * of this class instance will only create another reference to the same
-             * underlying object. Underlying object released automatically once all
+             * underlying object. Underlying object will be released automatically once all
              * the instances are destructed.
              */
             template<typename K, typename V>
diff --git a/modules/platforms/cpp/core/include/ignite/cache/query/query_fields_cursor.h b/modules/platforms/cpp/core/include/ignite/cache/query/query_fields_cursor.h
index 36e5f5c..e74f2ce 100644
--- a/modules/platforms/cpp/core/include/ignite/cache/query/query_fields_cursor.h
+++ b/modules/platforms/cpp/core/include/ignite/cache/query/query_fields_cursor.h
@@ -42,9 +42,9 @@ namespace ignite
             /**
              * Query fields cursor.
              *
-             * This class implemented as a reference to an implementation so copying
+             * This class is implemented as a reference to an implementation so copying
              * of this class instance will only create another reference to the same
-             * underlying object. Underlying object released automatically once all
+             * underlying object. Underlying object will be released automatically once all
              * the instances are destructed.
              */
             class QueryFieldsCursor
diff --git a/modules/platforms/cpp/core/include/ignite/cache/query/query_fields_row.h b/modules/platforms/cpp/core/include/ignite/cache/query/query_fields_row.h
index 8ed587c..a78d2d6 100644
--- a/modules/platforms/cpp/core/include/ignite/cache/query/query_fields_row.h
+++ b/modules/platforms/cpp/core/include/ignite/cache/query/query_fields_row.h
@@ -41,9 +41,9 @@ namespace ignite
             /**
              * Query fields cursor.
              *
-             * This class implemented as a reference to an implementation so copying
+             * This class is implemented as a reference to an implementation so copying
              * of this class instance will only create another reference to the same
-             * underlying object. Underlying object released automatically once all
+             * underlying object. Underlying object will be released automatically once all
              * the instances are destructed.
              */
             class QueryFieldsRow
diff --git a/modules/platforms/cpp/core/include/ignite/ignite.h b/modules/platforms/cpp/core/include/ignite/ignite.h
index 07245e4..f422a56 100644
--- a/modules/platforms/cpp/core/include/ignite/ignite.h
+++ b/modules/platforms/cpp/core/include/ignite/ignite.h
@@ -37,9 +37,9 @@ namespace ignite
     /**
      * Main interface to operate with %Ignite.
      *
-     * This class implemented as a reference to an implementation so copying
+     * This class is implemented as a reference to an implementation so copying
      * of this class instance will only create another reference to the same
-     * underlying object. Underlying object released automatically once all
+     * underlying object. Underlying object will be released automatically once all
      * the instances are destructed.
      */
     class IGNITE_IMPORT_EXPORT Ignite
diff --git a/modules/platforms/cpp/core/include/ignite/transactions/transaction.h b/modules/platforms/cpp/core/include/ignite/transactions/transaction.h
index b42a253..58b1119 100644
--- a/modules/platforms/cpp/core/include/ignite/transactions/transaction.h
+++ b/modules/platforms/cpp/core/include/ignite/transactions/transaction.h
@@ -39,9 +39,9 @@ namespace ignite
          * @see TransactionConcurrency and TransactionIsolation for details on
          * the supported isolation levels and concurrency models.
          *
-         * This class implemented as a reference to an implementation so copying
+         * This class is implemented as a reference to an implementation so copying
          * of this class instance will only create another reference to the same
-         * underlying object. Underlying object released automatically once all
+         * underlying object. Underlying object will be released automatically once all
          * the instances are destructed.
          */
         class IGNITE_FRIEND_EXPORT Transaction
diff --git a/modules/platforms/cpp/core/include/ignite/transactions/transactions.h b/modules/platforms/cpp/core/include/ignite/transactions/transactions.h
index 579dcd2..8817c96 100644
--- a/modules/platforms/cpp/core/include/ignite/transactions/transactions.h
+++ b/modules/platforms/cpp/core/include/ignite/transactions/transactions.h
@@ -37,9 +37,9 @@ namespace ignite
         /**
          * %Transactions facade.
          *
-         * This class implemented as a reference to an implementation so copying
+         * This class is implemented as a reference to an implementation so copying
          * of this class instance will only create another reference to the same
-         * underlying object. Underlying object released automatically once all
+         * underlying object. Underlying object will be released automatically once all
          * the instances are destructed.
          */
         class IGNITE_FRIEND_EXPORT Transactions
diff --git a/modules/platforms/cpp/examples/cluster-compute-example/CMakeLists.txt b/modules/platforms/cpp/examples/cluster-compute-example/CMakeLists.txt
index 2a8b977..22554f1 100644
--- a/modules/platforms/cpp/examples/cluster-compute-example/CMakeLists.txt
+++ b/modules/platforms/cpp/examples/cluster-compute-example/CMakeLists.txt
@@ -23,7 +23,8 @@ find_package(Threads)
 find_package(Java 1.8 REQUIRED)
 find_package(JNI REQUIRED)
 
-include_directories(../include ${IGNITE_INCLUDE_DIR} ${JNI_INCLUDE_DIRS})
+include_directories(SYSTEM ${IGNITE_INCLUDE_DIR} ${JNI_INCLUDE_DIRS})
+include_directories(../include)
 
 add_executable(${TARGET} src/cluster_compute_example.cpp)
 
diff --git a/modules/platforms/cpp/examples/compute-example/CMakeLists.txt b/modules/platforms/cpp/examples/compute-example/CMakeLists.txt
index 6fc1cf7..e336d96 100644
--- a/modules/platforms/cpp/examples/compute-example/CMakeLists.txt
+++ b/modules/platforms/cpp/examples/compute-example/CMakeLists.txt
@@ -24,7 +24,8 @@ find_package(Threads)
 find_package(Java 1.8 REQUIRED)
 find_package(JNI REQUIRED)
 
-include_directories(../include ${IGNITE_INCLUDE_DIR} ${JNI_INCLUDE_DIRS})
+include_directories(SYSTEM ${IGNITE_INCLUDE_DIR} ${JNI_INCLUDE_DIRS})
+include_directories(../include)
 
 add_executable(${TARGET} src/compute_example.cpp)
 
diff --git a/modules/platforms/cpp/examples/continuous-query-example/CMakeLists.txt b/modules/platforms/cpp/examples/continuous-query-example/CMakeLists.txt
index 31cb10f..a5c8f97 100644
--- a/modules/platforms/cpp/examples/continuous-query-example/CMakeLists.txt
+++ b/modules/platforms/cpp/examples/continuous-query-example/CMakeLists.txt
@@ -24,7 +24,8 @@ find_package(Threads)
 find_package(Java 1.8 REQUIRED)
 find_package(JNI REQUIRED)
 
-include_directories(../include ${IGNITE_INCLUDE_DIR} ${JNI_INCLUDE_DIRS})
+include_directories(SYSTEM ${IGNITE_INCLUDE_DIR} ${JNI_INCLUDE_DIRS})
+include_directories(../include)
 
 add_executable(${TARGET} src/continuous_query_example.cpp)
 
diff --git a/modules/platforms/cpp/examples/odbc-example/CMakeLists.txt b/modules/platforms/cpp/examples/odbc-example/CMakeLists.txt
index 386558e..3a11e8e 100644
--- a/modules/platforms/cpp/examples/odbc-example/CMakeLists.txt
+++ b/modules/platforms/cpp/examples/odbc-example/CMakeLists.txt
@@ -25,7 +25,8 @@ find_package(Java 1.8 REQUIRED)
 find_package(JNI REQUIRED)
 find_package(ODBC REQUIRED)
 
-include_directories(../include ${IGNITE_INCLUDE_DIR} ${JNI_INCLUDE_DIRS})
+include_directories(SYSTEM ${IGNITE_INCLUDE_DIR} ${JNI_INCLUDE_DIRS})
+include_directories(../include)
 
 add_executable(${TARGET} src/odbc_example.cpp)
 
diff --git a/modules/platforms/cpp/examples/put-get-example/CMakeLists.txt b/modules/platforms/cpp/examples/put-get-example/CMakeLists.txt
index fefb760..e5463a3 100644
--- a/modules/platforms/cpp/examples/put-get-example/CMakeLists.txt
+++ b/modules/platforms/cpp/examples/put-get-example/CMakeLists.txt
@@ -24,7 +24,8 @@ find_package(Threads)
 find_package(Java 1.8 REQUIRED)
 find_package(JNI REQUIRED)
 
-include_directories(../include ${IGNITE_INCLUDE_DIR} ${JNI_INCLUDE_DIRS})
+include_directories(SYSTEM ${IGNITE_INCLUDE_DIR} ${JNI_INCLUDE_DIRS})
+include_directories(../include)
 
 add_executable(${TARGET} src/put_get_example.cpp)
 
diff --git a/modules/platforms/cpp/examples/query-example/CMakeLists.txt b/modules/platforms/cpp/examples/query-example/CMakeLists.txt
index 4f543bc..e4144a5 100644
--- a/modules/platforms/cpp/examples/query-example/CMakeLists.txt
+++ b/modules/platforms/cpp/examples/query-example/CMakeLists.txt
@@ -24,7 +24,8 @@ find_package(Threads)
 find_package(Java 1.8 REQUIRED)
 find_package(JNI REQUIRED)
 
-include_directories(../include ${IGNITE_INCLUDE_DIR} ${JNI_INCLUDE_DIRS})
+include_directories(SYSTEM ${IGNITE_INCLUDE_DIR} ${JNI_INCLUDE_DIRS})
+include_directories(../include)
 
 add_executable(${TARGET} src/query_example.cpp)
 
diff --git a/modules/platforms/cpp/examples/thin-client-put-get-example/CMakeLists.txt b/modules/platforms/cpp/examples/thin-client-put-get-example/CMakeLists.txt
index fab5a1f..d187eca 100644
--- a/modules/platforms/cpp/examples/thin-client-put-get-example/CMakeLists.txt
+++ b/modules/platforms/cpp/examples/thin-client-put-get-example/CMakeLists.txt
@@ -22,7 +22,8 @@ set(TARGET ignite-${PROJECT_NAME})
 find_package(Ignite)
 find_package(Threads)
 
-include_directories(../include ${IGNITE_INCLUDE_DIR})
+include_directories(SYSTEM ${IGNITE_INCLUDE_DIR})
+include_directories(../include)
 
 add_executable(${TARGET} src/thin_client_put_get_example.cpp)
 
diff --git a/modules/platforms/cpp/ignite/CMakeLists.txt b/modules/platforms/cpp/ignite/CMakeLists.txt
index 1ffefaf..27cf7df 100644
--- a/modules/platforms/cpp/ignite/CMakeLists.txt
+++ b/modules/platforms/cpp/ignite/CMakeLists.txt
@@ -19,7 +19,7 @@ project(ignite-bin)
 
 set(TARGET ignite-bin)
 
-include_directories(${JNI_INCLUDE_DIRS})
+include_directories(SYSTEM ${JNI_INCLUDE_DIRS})
 
 set(SOURCES src/ignite.cpp)
 
diff --git a/modules/platforms/cpp/jni/CMakeLists.txt b/modules/platforms/cpp/jni/CMakeLists.txt
index 0783c45..5073195 100644
--- a/modules/platforms/cpp/jni/CMakeLists.txt
+++ b/modules/platforms/cpp/jni/CMakeLists.txt
@@ -21,7 +21,8 @@ set(TARGET ${PROJECT_NAME})
 
 find_library(JVM_LIBRARY jvm ${JAVA_JVM_LIBRARY_DIRECTORIES})
 
-include_directories(${JNI_INCLUDE_DIRS} include)
+include_directories(SYSTEM ${JNI_INCLUDE_DIRS})
+include_directories(include)
 
 set(SOURCES src/java.cpp src/exports.cpp)
 
diff --git a/modules/platforms/cpp/odbc-test/CMakeLists.txt b/modules/platforms/cpp/odbc-test/CMakeLists.txt
index 3a08f42..51d1501 100644
--- a/modules/platforms/cpp/odbc-test/CMakeLists.txt
+++ b/modules/platforms/cpp/odbc-test/CMakeLists.txt
@@ -27,7 +27,8 @@ find_package(Boost 1.53 REQUIRED COMPONENTS unit_test_framework chrono thread sy
 
 find_package(ODBC REQUIRED)
 
-include_directories(include ../odbc/include ../network/include ${ODBC_INCLUDE_DIRS} ${Boost_INCLUDE_DIRS} ${JNI_INCLUDE_DIRS})
+include_directories(SYSTEM ${ODBC_INCLUDE_DIRS} ${Boost_INCLUDE_DIRS} ${JNI_INCLUDE_DIRS})
+include_directories(include ../odbc/include ../network/include)
 
 set(SOURCES src/teamcity/teamcity_boost.cpp
         src/teamcity/teamcity_messages.cpp
diff --git a/modules/platforms/cpp/odbc/CMakeLists.txt b/modules/platforms/cpp/odbc/CMakeLists.txt
index 524d9bc..6d6d50b 100644
--- a/modules/platforms/cpp/odbc/CMakeLists.txt
+++ b/modules/platforms/cpp/odbc/CMakeLists.txt
@@ -21,7 +21,8 @@ set(TARGET ${PROJECT_NAME})
 
 find_package(ODBC REQUIRED)
 
-include_directories(include ${ODBC_INCLUDE_DIRS})
+include_directories(SYSTEM ${ODBC_INCLUDE_DIRS})
+include_directories(include)
 
 set(SOURCES src/app/application_data_buffer.cpp
         src/app/parameter.cpp
diff --git a/modules/platforms/cpp/thin-client-test/CMakeLists.txt b/modules/platforms/cpp/thin-client-test/CMakeLists.txt
index 535cb1a..1df71b7 100644
--- a/modules/platforms/cpp/thin-client-test/CMakeLists.txt
+++ b/modules/platforms/cpp/thin-client-test/CMakeLists.txt
@@ -25,13 +25,15 @@ endif()
 
 find_package(Boost 1.53 REQUIRED COMPONENTS unit_test_framework chrono thread system)
 
-include_directories(include ${Boost_INCLUDE_DIRS} ${JNI_INCLUDE_DIRS})
+include_directories(SYSTEM ${Boost_INCLUDE_DIRS} ${JNI_INCLUDE_DIRS})
+include_directories(include)
 
 set(SOURCES src/teamcity/teamcity_boost.cpp
         src/teamcity/teamcity_messages.cpp
         src/cache_client_test.cpp
         src/test_utils.cpp
         src/ignite_client_test.cpp
+        src/sql_fields_query_test.cpp
         src/auth_test.cpp
         src/tx_test.cpp
         src/ssl_test.cpp
diff --git a/modules/platforms/cpp/thin-client-test/config/sql-query-fields-32.xml b/modules/platforms/cpp/thin-client-test/config/sql-query-fields-32.xml
new file mode 100644
index 0000000..33f1fa0
--- /dev/null
+++ b/modules/platforms/cpp/thin-client-test/config/sql-query-fields-32.xml
@@ -0,0 +1,52 @@
+<?xml version="1.0" encoding="UTF-8"?>
+
+<!--
+    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.
+-->
+
+<!--
+    Ignite Spring configuration file to startup grid cache.
+-->
+<beans xmlns="http://www.springframework.org/schema/beans"
+       xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+       xmlns:util="http://www.springframework.org/schema/util"
+       xsi:schemaLocation="
+        http://www.springframework.org/schema/beans
+        http://www.springframework.org/schema/beans/spring-beans.xsd
+        http://www.springframework.org/schema/util
+        http://www.springframework.org/schema/util/spring-util.xsd">
+    <import resource="sql-query-fields-default.xml"/>
+
+    <bean parent="grid.cfg">
+        <property name="memoryConfiguration">
+            <bean class="org.apache.ignite.configuration.MemoryConfiguration">
+                <property name="systemCacheInitialSize" value="#{10 * 1024 * 1024}"/>
+                <property name="systemCacheMaxSize" value="#{40 * 1024 * 1024}"/>
+                <property name="defaultMemoryPolicyName" value="dfltPlc"/>
+
+                <property name="memoryPolicies">
+                    <list>
+                        <bean class="org.apache.ignite.configuration.MemoryPolicyConfiguration">
+                            <property name="name" value="dfltPlc"/>
+                            <property name="maxSize" value="#{100 * 1024 * 1024}"/>
+                            <property name="initialSize" value="#{10 * 1024 * 1024}"/>
+                        </bean>
+                    </list>
+                </property>
+            </bean>
+        </property>
+    </bean>
+</beans>
diff --git a/modules/platforms/cpp/thin-client-test/config/sql-query-fields-default.xml b/modules/platforms/cpp/thin-client-test/config/sql-query-fields-default.xml
new file mode 100644
index 0000000..9a716a3
--- /dev/null
+++ b/modules/platforms/cpp/thin-client-test/config/sql-query-fields-default.xml
@@ -0,0 +1,125 @@
+<?xml version="1.0" encoding="UTF-8"?>
+
+<!--
+    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.
+-->
+
+<!--
+    Ignite Spring configuration file to startup grid cache.
+-->
+<beans xmlns="http://www.springframework.org/schema/beans"
+       xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+       xmlns:util="http://www.springframework.org/schema/util"
+       xsi:schemaLocation="
+        http://www.springframework.org/schema/beans
+        http://www.springframework.org/schema/beans/spring-beans.xsd
+        http://www.springframework.org/schema/util
+        http://www.springframework.org/schema/util/spring-util.xsd">
+    <bean abstract="true" id="grid.cfg" class="org.apache.ignite.configuration.IgniteConfiguration">
+        <property name="localHost" value="127.0.0.1"/>
+        <property name="connectorConfiguration"><null/></property>
+
+        <property name="clientConnectorConfiguration">
+            <bean class="org.apache.ignite.configuration.ClientConnectorConfiguration">
+                <property name="host" value="127.0.0.1"/>
+                <property name="port" value="11110"/>
+                <property name="portRange" value="10"/>
+            </bean>
+        </property>
+
+        <property name="includeEventTypes">
+            <util:constant static-field="org.apache.ignite.events.EventType.EVTS_CACHE"/>
+        </property>
+
+        <property name="cacheConfiguration">
+            <list>
+                <bean parent="test-type-template"/>
+            </list>
+        </property>
+
+        <property name="discoverySpi">
+            <bean class="org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi">
+                <property name="ipFinder">
+                    <bean class="org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder">
+                        <property name="addresses">
+                            <list>
+                                <!-- In distributed environment, replace with actual host IP address. -->
+                                <value>127.0.0.1:47500..47503</value>
+                            </list>
+                        </property>
+                    </bean>
+                </property>
+                <property name="socketTimeout" value="300" />
+            </bean>
+        </property>
+
+        <property name="transactionConfiguration">
+            <bean class="org.apache.ignite.configuration.TransactionConfiguration">
+                <property name="txSerializableEnabled" value="true"/>
+            </bean>
+        </property>
+    </bean>
+
+    <bean id="test-type-template" abstract="true" class="org.apache.ignite.configuration.CacheConfiguration">
+        <property name="name" value="cacheAllFields"/>
+        <property name="writeSynchronizationMode" value="FULL_SYNC"/>
+        <property name="cacheMode" value="PARTITIONED"/>
+        <property name="atomicityMode" value="TRANSACTIONAL"/>
+
+        <!-- Configure type metadata to enable queries. -->
+        <property name="queryEntities">
+            <list>
+                <bean class="org.apache.ignite.cache.QueryEntity">
+                    <property name="keyType" value="java.lang.Long"/>
+                    <property name="valueType" value="TestType"/>
+
+                    <property name="fields">
+                        <map>
+                            <entry key="i8Field" value="java.lang.Byte"/>
+                            <entry key="i16Field" value="java.lang.Short"/>
+                            <entry key="i32Field" value="java.lang.Integer"/>
+                            <entry key="i64Field" value="java.lang.Long"/>
+                            <entry key="strField" value="java.lang.String"/>
+                            <entry key="floatField" value="java.lang.Float"/>
+                            <entry key="doubleField" value="java.lang.Double"/>
+                            <entry key="boolField" value="java.lang.Boolean"/>
+                            <entry key="guidField" value="java.util.UUID"/>
+                            <entry key="dateField" value="java.util.Date"/>
+                            <entry key="timeField" value="java.sql.Time"/>
+                            <entry key="timestampField" value="java.sql.Timestamp"/>
+                            <entry key="i8ArrayField" value="[B"/>
+                        </map>
+                    </property>
+
+                    <property name="keyFields">
+                        <list></list>
+                    </property>
+
+                    <property name="indexes">
+                        <list>
+                            <bean class="org.apache.ignite.cache.QueryIndex">
+                                <constructor-arg value="i32Field"/>
+                            </bean>
+                            <bean class="org.apache.ignite.cache.QueryIndex">
+                                <constructor-arg value="i64Field"/>
+                            </bean>
+                        </list>
+                    </property>
+                </bean>
+            </list>
+        </property>
+    </bean>
+</beans>
diff --git a/modules/platforms/cpp/thin-client-test/config/sql-query-fields.xml b/modules/platforms/cpp/thin-client-test/config/sql-query-fields.xml
new file mode 100644
index 0000000..2b18f0d
--- /dev/null
+++ b/modules/platforms/cpp/thin-client-test/config/sql-query-fields.xml
@@ -0,0 +1,35 @@
+<?xml version="1.0" encoding="UTF-8"?>
+
+<!--
+    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.
+-->
+
+<!--
+    Ignite Spring configuration file to startup grid cache.
+-->
+<beans xmlns="http://www.springframework.org/schema/beans"
+       xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+       xmlns:util="http://www.springframework.org/schema/util"
+       xsi:schemaLocation="
+        http://www.springframework.org/schema/beans
+        http://www.springframework.org/schema/beans/spring-beans.xsd
+        http://www.springframework.org/schema/util
+        http://www.springframework.org/schema/util/spring-util.xsd">
+    <import resource="sql-query-fields-default.xml"/>
+
+    <bean parent="grid.cfg"/>
+
+</beans>
diff --git a/modules/platforms/cpp/thin-client-test/include/ignite/test_type.h b/modules/platforms/cpp/thin-client-test/include/ignite/test_type.h
new file mode 100644
index 0000000..6cefb98
--- /dev/null
+++ b/modules/platforms/cpp/thin-client-test/include/ignite/test_type.h
@@ -0,0 +1,183 @@
+/*
+ * 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.
+ */
+
+#ifndef _IGNITE_THIN_CLIENT_TEST_TEST_TYPE
+#define _IGNITE_THIN_CLIENT_TEST_TEST_TYPE
+
+#include <string>
+
+#include <ignite/ignite.h>
+
+namespace ignite
+{
+    struct TestType
+    {
+        TestType() :
+            allNulls(false),
+            i8Field(0),
+            i16Field(0),
+            i32Field(0),
+            i64Field(0),
+            floatField(0.0f),
+            doubleField(0.0),
+            boolField(false),
+            dateField(),
+            timeField(),
+            timestampField()
+        {
+            // No-op.
+        }
+
+        TestType(int8_t i8Field, int16_t i16Field, int32_t i32Field,
+            int64_t i64Field, const std::string& strField, float floatField,
+            double doubleField, bool boolField, const Guid& guidField,
+            const Date& dateField, const Time& timeField, const Timestamp& timestampField) :
+            allNulls(false),
+            i8Field(i8Field),
+            i16Field(i16Field),
+            i32Field(i32Field),
+            i64Field(i64Field),
+            strField(strField),
+            floatField(floatField),
+            doubleField(doubleField),
+            boolField(boolField),
+            guidField(guidField),
+            dateField(dateField),
+            timeField(timeField),
+            timestampField(timestampField)
+        {
+            // No-op.
+        }
+
+        friend bool operator==(const TestType& one, const TestType& two)
+        {
+            return
+                one.allNulls == two.allNulls &&
+                one.i8Field == two.i8Field &&
+                one.i16Field == two.i16Field &&
+                one.i32Field == two.i32Field &&
+                one.i64Field == two.i64Field &&
+                one.strField == two.strField &&
+                one.floatField == two.floatField &&
+                one.doubleField == two.doubleField &&
+                one.boolField == two.boolField &&
+                one.guidField == two.guidField &&
+                one.dateField == two.dateField &&
+                one.timeField == two.timeField &&
+                one.timestampField == two.timestampField &&
+                one.i8ArrayField == two.i8ArrayField;
+        }
+
+        bool allNulls;
+        int8_t i8Field;
+        int16_t i16Field;
+        int32_t i32Field;
+        int64_t i64Field;
+        std::string strField;
+        float floatField;
+        double doubleField;
+        bool boolField;
+        Guid guidField;
+        Date dateField;
+        Time timeField;
+        Timestamp timestampField;
+        std::vector<int8_t> i8ArrayField;
+    };
+
+    namespace binary
+    {
+        IGNITE_BINARY_TYPE_START(ignite::TestType)
+
+            typedef ignite::TestType TestType;
+
+            IGNITE_BINARY_GET_TYPE_ID_AS_HASH(TestType)
+            IGNITE_BINARY_GET_TYPE_NAME_AS_IS(TestType)
+            IGNITE_BINARY_GET_FIELD_ID_AS_HASH
+            IGNITE_BINARY_IS_NULL_FALSE(TestType)
+            IGNITE_BINARY_GET_NULL_DEFAULT_CTOR(TestType)
+
+            static void Write(BinaryWriter& writer, const TestType& obj)
+            {
+                if (!obj.allNulls)
+                {
+                    writer.WriteInt8("i8Field", obj.i8Field);
+                    writer.WriteInt16("i16Field", obj.i16Field);
+                    writer.WriteInt32("i32Field", obj.i32Field);
+                    writer.WriteInt64("i64Field", obj.i64Field);
+                    writer.WriteString("strField", obj.strField);
+                    writer.WriteFloat("floatField", obj.floatField);
+                    writer.WriteDouble("doubleField", obj.doubleField);
+                    writer.WriteBool("boolField", obj.boolField);
+                    writer.WriteGuid("guidField", obj.guidField);
+                    writer.WriteDate("dateField", obj.dateField);
+                    writer.WriteTime("timeField", obj.timeField);
+                    writer.WriteTimestamp("timestampField", obj.timestampField);
+                    if (obj.i8ArrayField.empty())
+                    {
+                        writer.WriteNull("i8ArrayField");
+                    }
+                    else
+                    {
+                        writer.WriteInt8Array("i8ArrayField", &obj.i8ArrayField[0], static_cast<int32_t>(obj.i8ArrayField.size()));
+                    }
+                }
+                else
+                {
+                    writer.WriteNull("i8Field");
+                    writer.WriteNull("i16Field");
+                    writer.WriteNull("i32Field");
+                    writer.WriteNull("i64Field");
+                    writer.WriteNull("strField");
+                    writer.WriteNull("floatField");
+                    writer.WriteNull("doubleField");
+                    writer.WriteNull("boolField");
+                    writer.WriteNull("guidField");
+                    writer.WriteNull("dateField");
+                    writer.WriteNull("timeField");
+                    writer.WriteNull("timestampField");
+                    writer.WriteNull("i8ArrayField");
+                }
+            }
+
+            static void Read(BinaryReader& reader, TestType& dst)
+            {
+                dst.i8Field = reader.ReadInt8("i8Field");
+                dst.i16Field = reader.ReadInt16("i16Field");
+                dst.i32Field = reader.ReadInt32("i32Field");
+                dst.i64Field = reader.ReadInt64("i64Field");
+                dst.strField = reader.ReadString("strField");
+                dst.floatField = reader.ReadFloat("floatField");
+                dst.doubleField = reader.ReadDouble("doubleField");
+                dst.boolField = reader.ReadBool("boolField");
+                dst.guidField = reader.ReadGuid("guidField");
+                dst.dateField = reader.ReadDate("dateField");
+                dst.timeField = reader.ReadTime("timeField");
+                dst.timestampField = reader.ReadTimestamp("timestampField");
+
+                int32_t len = reader.ReadInt8Array("i8ArrayField", 0, 0);
+                if (len > 0)
+                {
+                    dst.i8ArrayField.resize(len);
+                    reader.ReadInt8Array("i8ArrayField", &dst.i8ArrayField[0], len);
+                }
+            }
+
+        IGNITE_BINARY_TYPE_END
+    }
+}
+
+#endif // _IGNITE_THIN_CLIENT_TEST_TEST_TYPE
diff --git a/modules/platforms/cpp/thin-client-test/project/vs/thin-client-test.vcxproj b/modules/platforms/cpp/thin-client-test/project/vs/thin-client-test.vcxproj
index 118c490..d700806 100644
--- a/modules/platforms/cpp/thin-client-test/project/vs/thin-client-test.vcxproj
+++ b/modules/platforms/cpp/thin-client-test/project/vs/thin-client-test.vcxproj
@@ -23,6 +23,7 @@
     <ClCompile Include="..\..\src\cache_client_test.cpp" />
     <ClCompile Include="..\..\src\ignite_client_test.cpp" />
     <ClCompile Include="..\..\src\ssl_test.cpp" />
+    <ClCompile Include="..\..\src\sql_fields_query_test.cpp" />
     <ClCompile Include="..\..\src\teamcity\teamcity_boost.cpp" />
     <ClCompile Include="..\..\src\teamcity\teamcity_messages.cpp" />
     <ClCompile Include="..\..\src\test_utils.cpp" />
@@ -30,6 +31,7 @@
   </ItemGroup>
   <ItemGroup>
     <ClInclude Include="..\..\include\ignite\complex_type.h" />
+    <ClInclude Include="..\..\include\ignite\test_type.h" />
     <ClInclude Include="..\..\include\teamcity\teamcity_messages.h" />
     <ClInclude Include="..\..\include\test_utils.h" />
   </ItemGroup>
@@ -63,6 +65,9 @@
     <None Include="..\..\config\ssl-32.xml" />
     <None Include="..\..\config\ssl-default.xml" />
     <None Include="..\..\config\ssl.xml" />
+    <None Include="..\..\config\sql-query-fields-32.xml" />
+    <None Include="..\..\config\sql-query-fields-default.xml" />
+    <None Include="..\..\config\sql-query-fields.xml" />
   </ItemGroup>
   <PropertyGroup Label="Globals">
     <ProjectGuid>{5662F10A-9C40-45D6-AFF8-E93573FEAABA}</ProjectGuid>
diff --git a/modules/platforms/cpp/thin-client-test/project/vs/thin-client-test.vcxproj.filters b/modules/platforms/cpp/thin-client-test/project/vs/thin-client-test.vcxproj.filters
index 17c63e0..5a9c6e8 100644
--- a/modules/platforms/cpp/thin-client-test/project/vs/thin-client-test.vcxproj.filters
+++ b/modules/platforms/cpp/thin-client-test/project/vs/thin-client-test.vcxproj.filters
@@ -32,6 +32,9 @@
     <ClCompile Include="..\..\src\ssl_test.cpp">
       <Filter>Code</Filter>
     </ClCompile>
+    <ClCompile Include="..\..\src\sql_fields_query_test.cpp">
+      <Filter>Code</Filter>
+    </ClCompile>
     <ClCompile Include="..\..\src\auth_test.cpp">
       <Filter>Code</Filter>
     </ClCompile>
@@ -49,6 +52,9 @@
     <ClInclude Include="..\..\include\ignite\complex_type.h">
       <Filter>Code\Types</Filter>
     </ClInclude>
+    <ClInclude Include="..\..\include\ignite\test_type.h">
+      <Filter>Code\Types</Filter>
+    </ClInclude>
   </ItemGroup>
   <ItemGroup>
     <None Include="..\..\config\cache.xml">
@@ -87,5 +93,14 @@
     <None Include="..\..\config\non-ssl-default.xml">
       <Filter>Configs</Filter>
     </None>
+    <None Include="..\..\config\sql-query-fields.xml">
+      <Filter>Configs</Filter>
+    </None>
+    <None Include="..\..\config\sql-query-fields-32.xml">
+      <Filter>Configs</Filter>
+    </None>
+    <None Include="..\..\config\sql-query-fields-default.xml">
+      <Filter>Configs</Filter>
+    </None>
   </ItemGroup>
 </Project>
\ No newline at end of file
diff --git a/modules/platforms/cpp/thin-client-test/src/sql_fields_query_test.cpp b/modules/platforms/cpp/thin-client-test/src/sql_fields_query_test.cpp
new file mode 100644
index 0000000..c027727
--- /dev/null
+++ b/modules/platforms/cpp/thin-client-test/src/sql_fields_query_test.cpp
@@ -0,0 +1,467 @@
+/*
+ * 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.
+ */
+
+#include <boost/test/unit_test.hpp>
+
+#include <ignite/ignition.h>
+
+#include <ignite/thin/ignite_client_configuration.h>
+#include <ignite/thin/ignite_client.h>
+
+#include <ignite/test_type.h>
+#include <test_utils.h>
+
+using namespace ignite::thin;
+using namespace ignite::thin::cache::query;
+using namespace boost::unit_test;
+
+class SqlFieldsQueryTestSuiteFixture
+{
+public:
+    static ignite::Ignite StartNode(const char* name)
+    {
+        return ignite_test::StartCrossPlatformServerNode("sql-query-fields.xml", name);
+    }
+
+    SqlFieldsQueryTestSuiteFixture()
+    {
+        serverNode = StartNode("ServerNode");
+
+        IgniteClientConfiguration cfg;
+
+        cfg.SetEndPoints("127.0.0.1:11110");
+
+        client = IgniteClient::Start(cfg);
+
+        cacheAllFields = client.GetCache<int64_t, ignite::TestType>("cacheAllFields");
+    }
+
+    ~SqlFieldsQueryTestSuiteFixture()
+    {
+        ignite::Ignition::StopAll(false);
+    }
+
+protected:
+    /** Server node. */
+    ignite::Ignite serverNode;
+
+    /** Client. */
+    IgniteClient client;
+
+    /** Cache with TestType. */
+    cache::CacheClient<int64_t, ignite::TestType> cacheAllFields;
+};
+
+BOOST_AUTO_TEST_SUITE(SqlFieldsQueryBasicTestSuite)
+
+BOOST_AUTO_TEST_CASE(SqlFieldsQueryDefaults)
+{
+    std::string sql("select * from TestType");
+
+    SqlFieldsQuery qry(sql);
+
+    BOOST_CHECK_EQUAL(qry.GetSql(), sql);
+    BOOST_CHECK_EQUAL(qry.GetTimeout(), 0);
+    BOOST_CHECK_EQUAL(qry.GetMaxRows(), 0);
+    BOOST_CHECK_EQUAL(qry.GetPageSize(), 1024);
+    BOOST_CHECK_EQUAL(qry.GetSchema(), std::string());
+
+    BOOST_CHECK(!qry.IsLocal());
+    BOOST_CHECK(!qry.IsCollocated());
+    BOOST_CHECK(!qry.IsDistributedJoins());
+    BOOST_CHECK(!qry.IsEnforceJoinOrder());
+    BOOST_CHECK(!qry.IsLazy());
+}
+
+BOOST_AUTO_TEST_CASE(SqlFieldsQuerySetGet)
+{
+    std::string sql("select * from TestType");
+
+    SqlFieldsQuery qry(sql);
+
+    qry.SetTimeout(1000);
+    qry.SetMaxRows(100);
+    qry.SetPageSize(4096);
+    qry.SetSchema("PUBLIC");
+
+    qry.SetLocal(true);
+    qry.SetCollocated(true);
+    qry.SetDistributedJoins(true);
+    qry.SetEnforceJoinOrder(true);
+    qry.SetLazy(true);
+
+    BOOST_CHECK_EQUAL(qry.GetSql(), sql);
+    BOOST_CHECK_EQUAL(qry.GetTimeout(), 1000);
+    BOOST_CHECK_EQUAL(qry.GetMaxRows(), 100);
+    BOOST_CHECK_EQUAL(qry.GetPageSize(), 4096);
+    BOOST_CHECK_EQUAL(qry.GetSchema(), std::string("PUBLIC"));
+
+    BOOST_CHECK(qry.IsLocal());
+    BOOST_CHECK(qry.IsCollocated());
+    BOOST_CHECK(qry.IsDistributedJoins());
+    BOOST_CHECK(qry.IsEnforceJoinOrder());
+    BOOST_CHECK(qry.IsLazy());
+}
+
+BOOST_AUTO_TEST_SUITE_END()
+
+/**
+ * Check that error empty cursor error.
+ *
+ * @param err Error.
+ */
+bool IsCursorEmptyError(const ignite::IgniteError& err)
+{
+    return err.GetCode() == ignite::IgniteError::IGNITE_ERR_GENERIC &&
+        std::string(err.GetText()) == "The cursor is empty";
+}
+
+/**
+ * Check that cursor is empty.
+ *
+ * @param cursor Cursor.
+ */
+void CheckCursorEmpty(QueryFieldsCursor& cursor)
+{
+    BOOST_CHECK(!cursor.HasNext());
+    BOOST_CHECK_EXCEPTION(cursor.GetNext(), ignite::IgniteError, IsCursorEmptyError);
+}
+
+/**
+ * Check that row is empty.
+ *
+ * @param row Row.
+ */
+void CheckRowCursorEmpty(QueryFieldsRow& row)
+{
+    BOOST_CHECK(!row.HasNext());
+    BOOST_CHECK_EXCEPTION(row.GetNext<int8_t>(), ignite::IgniteError, IsCursorEmptyError);
+}
+
+/**
+ * Check that row columns equal value fields.
+ *
+ * @param row Row.
+ * @param val Value.
+ */
+void CheckRowEqualsValue(QueryFieldsRow& row, const ignite::TestType& val)
+{
+    BOOST_CHECK_EQUAL(row.GetNext<int8_t>(), val.i8Field);
+    BOOST_CHECK_EQUAL(row.GetNext<int16_t>(), val.i16Field);
+    BOOST_CHECK_EQUAL(row.GetNext<int32_t>(), val.i32Field);
+    BOOST_CHECK_EQUAL(row.GetNext<int64_t>(), val.i64Field);
+    BOOST_CHECK_EQUAL(row.GetNext<std::string>(), val.strField);
+    BOOST_CHECK_CLOSE(row.GetNext<float>(), val.floatField, 0.0001f);
+    BOOST_CHECK_CLOSE(row.GetNext<double>(), val.doubleField, 0.0001);
+    BOOST_CHECK_EQUAL(row.GetNext<bool>(), val.boolField);
+    BOOST_CHECK_EQUAL(row.GetNext<ignite::Guid>(), val.guidField);
+    BOOST_CHECK(row.GetNext<ignite::Date>() == val.dateField);
+    BOOST_CHECK_EQUAL(row.GetNext<ignite::Time>().GetMilliseconds(), val.timeField.GetMilliseconds());
+    BOOST_CHECK(row.GetNext<ignite::Timestamp>() == val.timestampField);
+
+    std::vector<int8_t> resArray = row.GetNext< std::vector<int8_t> >();
+
+    BOOST_CHECK_EQUAL_COLLECTIONS(
+            resArray.begin(), resArray.end(),
+            val.i8ArrayField.begin(), val.i8ArrayField.end());
+}
+
+/**
+ * Make custom test value.
+ *
+ * @param seed Seed to generate value.
+ */
+ignite::TestType MakeCustomTestValue(int32_t seed)
+{
+    ignite::TestType val;
+
+    val.i8Field = seed;
+    val.i16Field = 2 * seed;
+    val.i32Field = 4 * seed;
+    val.i64Field = 8 * seed;
+    val.strField = "Lorem ipsum";
+    val.floatField = 16.0f * seed;
+    val.doubleField = 32.0 * seed;
+    val.boolField = ((seed % 2) == 0);
+    val.guidField = ignite::Guid(0x1020304050607080 * seed, 0x9000A0B0C0D0E0F0 * seed);
+    val.dateField = ignite::Date(235682736 * seed);
+    val.timeField = ignite::Time((124523 * seed) % (24 * 60 * 60 * 1000));
+    val.timestampField = ignite::Timestamp(128341594123 * seed);
+
+    val.i8ArrayField.push_back(9 * seed);
+    val.i8ArrayField.push_back(6 * seed);
+    val.i8ArrayField.push_back(3 * seed);
+    val.i8ArrayField.push_back(42 * seed);
+
+    return val;
+}
+
+
+BOOST_FIXTURE_TEST_SUITE(SqlFieldsQueryTestSuite, SqlFieldsQueryTestSuiteFixture)
+
+BOOST_AUTO_TEST_CASE(SelectEmpty)
+{
+    SqlFieldsQuery qry("select * from TestType");
+
+    QueryFieldsCursor cursor = cacheAllFields.Query(qry);
+
+    CheckCursorEmpty(cursor);
+
+    const std::vector<std::string>& columns = cursor.GetColumnNames();
+
+    BOOST_CHECK_EQUAL(columns.size(), 13);
+
+    BOOST_CHECK_EQUAL(columns.at(0), "I8FIELD");
+    BOOST_CHECK_EQUAL(columns.at(1), "I16FIELD");
+    BOOST_CHECK_EQUAL(columns.at(2), "I32FIELD");
+    BOOST_CHECK_EQUAL(columns.at(3), "I64FIELD");
+    BOOST_CHECK_EQUAL(columns.at(4), "STRFIELD");
+    BOOST_CHECK_EQUAL(columns.at(5), "FLOATFIELD");
+    BOOST_CHECK_EQUAL(columns.at(6), "DOUBLEFIELD");
+    BOOST_CHECK_EQUAL(columns.at(7), "BOOLFIELD");
+    BOOST_CHECK_EQUAL(columns.at(8), "GUIDFIELD");
+    BOOST_CHECK_EQUAL(columns.at(9), "DATEFIELD");
+    BOOST_CHECK_EQUAL(columns.at(10), "TIMEFIELD");
+    BOOST_CHECK_EQUAL(columns.at(11), "TIMESTAMPFIELD");
+    BOOST_CHECK_EQUAL(columns.at(12), "I8ARRAYFIELD");
+}
+
+BOOST_AUTO_TEST_CASE(SelectSingleValue)
+{
+    ignite::TestType val = MakeCustomTestValue(1);
+
+    cacheAllFields.Put(42, val);
+
+    SqlFieldsQuery qry("select i8Field, i16Field, i32Field, i64Field, strField, floatField, "
+        "doubleField, boolField, guidField, dateField, timeField, timestampField, i8ArrayField FROM TestType");
+
+    QueryFieldsCursor cursor = cacheAllFields.Query(qry);
+
+    BOOST_CHECK(cursor.HasNext());
+
+    QueryFieldsRow row = cursor.GetNext();
+
+    BOOST_CHECK(row.HasNext());
+
+    CheckRowEqualsValue(row, val);
+
+    CheckRowCursorEmpty(row);
+    CheckCursorEmpty(cursor);
+}
+
+BOOST_AUTO_TEST_CASE(SelectTwoValues)
+{
+    ignite::TestType val1 = MakeCustomTestValue(1);
+    ignite::TestType val2 = MakeCustomTestValue(2);
+
+    cacheAllFields.Put(1, val1);
+    cacheAllFields.Put(2, val2);
+
+    SqlFieldsQuery qry("select i8Field, i16Field, i32Field, i64Field, strField, floatField, "
+        "doubleField, boolField, guidField, dateField, timeField, timestampField, i8ArrayField FROM TestType "
+        "ORDER BY _key");
+
+    QueryFieldsCursor cursor = cacheAllFields.Query(qry);
+
+    BOOST_CHECK(cursor.HasNext());
+    QueryFieldsRow row = cursor.GetNext();
+    BOOST_CHECK(row.HasNext());
+
+    CheckRowEqualsValue(row, val1);
+    CheckRowCursorEmpty(row);
+
+    BOOST_CHECK(cursor.HasNext());
+    row = cursor.GetNext();
+    BOOST_CHECK(row.HasNext());
+
+    CheckRowEqualsValue(row, val2);
+
+    CheckRowCursorEmpty(row);
+    CheckCursorEmpty(cursor);
+}
+
+BOOST_AUTO_TEST_CASE(Select10000Values)
+{
+    const int32_t num = 10000;
+
+    std::map<int64_t, ignite::TestType> values;
+
+    for (int32_t i = 0; i < num; ++i)
+        values[i] = MakeCustomTestValue(i);
+
+    BOOST_CHECK_EQUAL(values.size(), static_cast<size_t>(num));
+
+    cacheAllFields.PutAll(values);
+
+    SqlFieldsQuery qry("select i8Field, i16Field, i32Field, i64Field, strField, floatField, "
+        "doubleField, boolField, guidField, dateField, timeField, timestampField, i8ArrayField FROM TestType "
+        "ORDER BY _key");
+
+    QueryFieldsCursor cursor = cacheAllFields.Query(qry);
+
+    for (int64_t i = 0; i < num; ++i)
+    {
+        BOOST_CHECK(cursor.HasNext());
+        QueryFieldsRow row = cursor.GetNext();
+        BOOST_CHECK(row.HasNext());
+
+        CheckRowEqualsValue(row, values[i]);
+        CheckRowCursorEmpty(row);
+    }
+
+    CheckCursorEmpty(cursor);
+}
+
+BOOST_AUTO_TEST_CASE(Select10ValuesPageSize1)
+{
+    const int32_t num = 10;
+
+    std::map<int64_t, ignite::TestType> values;
+
+    for (int32_t i = 0; i < num; ++i)
+        values[i] = MakeCustomTestValue(i);
+
+    BOOST_CHECK_EQUAL(values.size(), static_cast<size_t>(num));
+
+    cacheAllFields.PutAll(values);
+
+    SqlFieldsQuery qry("select i8Field, i16Field, i32Field, i64Field, strField, floatField, "
+                       "doubleField, boolField, guidField, dateField, timeField, timestampField, i8ArrayField FROM TestType "
+                       "ORDER BY _key");
+
+    qry.SetPageSize(1);
+
+    QueryFieldsCursor cursor = cacheAllFields.Query(qry);
+
+    for (int64_t i = 0; i < num; ++i)
+    {
+        BOOST_CHECK(cursor.HasNext());
+        QueryFieldsRow row = cursor.GetNext();
+        BOOST_CHECK(row.HasNext());
+
+        CheckRowEqualsValue(row, values[i]);
+        CheckRowCursorEmpty(row);
+    }
+
+    CheckCursorEmpty(cursor);
+}
+
+
+BOOST_AUTO_TEST_CASE(SelectKeyValue)
+{
+    const int64_t key = 123;
+    const ignite::TestType val = MakeCustomTestValue(1);
+
+    cacheAllFields.Put(key, val);
+
+    SqlFieldsQuery qry("select _key, _val FROM TestType");
+
+    QueryFieldsCursor cursor = cacheAllFields.Query(qry);
+
+    BOOST_CHECK(cursor.HasNext());
+
+    QueryFieldsRow row = cursor.GetNext();
+
+    BOOST_CHECK(row.HasNext());
+
+    const int64_t keyActual = row.GetNext<int64_t>();
+    const ignite::TestType valActual = row.GetNext<ignite::TestType>();
+
+    BOOST_CHECK_EQUAL(key, keyActual);
+    BOOST_CHECK(val == valActual);
+
+    CheckRowCursorEmpty(row);
+    CheckCursorEmpty(cursor);
+}
+
+BOOST_AUTO_TEST_CASE(SelectTwoValuesInDifferentOrder)
+{
+    typedef ignite::common::concurrent::SharedPointer<void> SP_Void;
+
+    ignite::TestType val1 = MakeCustomTestValue(1);
+    ignite::TestType val2 = MakeCustomTestValue(2);
+
+    cacheAllFields.Put(1, val1);
+    cacheAllFields.Put(2, val2);
+
+    SqlFieldsQuery qry("select i8Field, i16Field, i32Field, i64Field, strField, floatField, "
+        "doubleField, boolField, guidField, dateField, timeField, timestampField, i8ArrayField FROM TestType "
+        "ORDER BY _key");
+
+    // Checking if everything going to be OK if we destroy cursor before fetching rows.
+    QueryFieldsRow row1 = QueryFieldsRow(SP_Void());
+    QueryFieldsRow row2 = QueryFieldsRow(SP_Void());
+
+    {
+        QueryFieldsCursor cursor = cacheAllFields.Query(qry);
+
+        BOOST_CHECK(cursor.HasNext());
+        row1 = cursor.GetNext();
+
+        BOOST_CHECK(cursor.HasNext());
+        row2 = cursor.GetNext();
+
+        CheckCursorEmpty(cursor);
+    }
+
+    BOOST_CHECK(row2.HasNext());
+    CheckRowEqualsValue(row2, val2);
+    CheckRowCursorEmpty(row2);
+
+    BOOST_CHECK(row1.HasNext());
+    CheckRowEqualsValue(row1, val1);
+    CheckRowCursorEmpty(row1);
+}
+
+BOOST_AUTO_TEST_CASE(CreateTableInsertSelect)
+{
+    SqlFieldsQuery qry("create table TestTable(id int primary key, val int)");
+    qry.SetSchema("PUBLIC");
+
+    QueryFieldsCursor cursor = cacheAllFields.Query(qry);
+
+    QueryFieldsRow row = cursor.GetNext();
+    BOOST_CHECK(row.HasNext());
+    BOOST_CHECK_EQUAL(row.GetNext<int64_t>(), 0);
+    CheckRowCursorEmpty(row);
+
+    qry.SetSql("insert into TestTable(id, val) values(1, 2)");
+
+    cursor = cacheAllFields.Query(qry);
+
+    row = cursor.GetNext();
+    BOOST_CHECK(row.HasNext());
+    BOOST_CHECK_EQUAL(row.GetNext<int64_t>(), 1);
+    CheckRowCursorEmpty(row);
+
+    qry.SetSql("select id, val from TestTable");
+
+    cursor = cacheAllFields.Query(qry);
+
+    BOOST_CHECK(cursor.HasNext());
+
+    row = cursor.GetNext();
+
+    BOOST_CHECK(row.HasNext());
+    BOOST_CHECK_EQUAL(row.GetNext<int32_t>(), 1);
+    BOOST_CHECK_EQUAL(row.GetNext<int32_t>(), 2);
+    CheckRowCursorEmpty(row);
+
+    CheckCursorEmpty(cursor);
+}
+
+BOOST_AUTO_TEST_SUITE_END()
diff --git a/modules/platforms/cpp/thin-client/CMakeLists.txt b/modules/platforms/cpp/thin-client/CMakeLists.txt
index feb896a..059b012 100644
--- a/modules/platforms/cpp/thin-client/CMakeLists.txt
+++ b/modules/platforms/cpp/thin-client/CMakeLists.txt
@@ -36,7 +36,9 @@ set(SOURCES src/impl/data_channel.cpp
         src/impl/cache/cache_client_impl.cpp
         src/impl/transactions/transactions_impl.cpp
         src/impl/transactions/transactions_proxy.cpp
-        src/ignite_client.cpp)
+        src/ignite_client.cpp
+		src/cache/query/query_fields_cursor.cpp
+		src/cache/query/query_fields_row.cpp)
 
 add_library(${TARGET} SHARED ${SOURCES})
 
diff --git a/modules/platforms/cpp/thin-client/include/ignite/impl/thin/cache/cache_client_proxy.h b/modules/platforms/cpp/thin-client/include/ignite/impl/thin/cache/cache_client_proxy.h
index 54b7e80..da5f42b 100644
--- a/modules/platforms/cpp/thin-client/include/ignite/impl/thin/cache/cache_client_proxy.h
+++ b/modules/platforms/cpp/thin-client/include/ignite/impl/thin/cache/cache_client_proxy.h
@@ -20,6 +20,9 @@
 
 #include <ignite/common/concurrent.h>
 
+#include <ignite/thin/cache/query/query_fields_cursor.h>
+#include <ignite/thin/cache/query/query_sql_fields.h>
+
 namespace ignite
 {
     namespace impl
@@ -286,6 +289,15 @@ namespace ignite
                     void GetAndPutIfAbsent(const WritableKey& key, const Writable& valIn, Readable& valOut);
 
                     /**
+                     * Perform SQL fields query.
+                     *
+                     * @param qry Query.
+                     * @return Query cursor.
+                     */
+                    ignite::thin::cache::query::QueryFieldsCursor Query(
+                            const ignite::thin::cache::query::SqlFieldsQuery& qry);
+
+                    /**
                      * Get from CacheClient.
                      * Use for testing purposes only.
                      */
diff --git a/modules/platforms/cpp/thin-client/include/ignite/impl/thin/cache/query/query_fields_cursor_impl.h b/modules/platforms/cpp/thin-client/include/ignite/impl/thin/cache/query/query_fields_cursor_impl.h
new file mode 100644
index 0000000..13af4cb
--- /dev/null
+++ b/modules/platforms/cpp/thin-client/include/ignite/impl/thin/cache/query/query_fields_cursor_impl.h
@@ -0,0 +1,147 @@
+/*
+ * 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.
+ */
+#ifndef _IGNITE_IMPL_THIN_CACHE_QUERY_QUERY_FIELDS_CURSOR_IMPL
+#define _IGNITE_IMPL_THIN_CACHE_QUERY_QUERY_FIELDS_CURSOR_IMPL
+
+#include <ignite/ignite_error.h>
+
+#include "ignite/impl/ignite_environment.h"
+#include "ignite/impl/operations.h"
+#include "ignite/impl/cache/query/query_batch.h"
+
+namespace ignite
+{
+    namespace impl
+    {
+        namespace thin
+        {
+            namespace cache
+            {
+                namespace query
+                {
+                    class QueryFieldsRowImpl;
+
+                    /**
+                     * Query cursor implementation.
+                     */
+                    class IGNITE_IMPORT_EXPORT QueryCursorImpl
+                    {
+                    public:
+                        /**
+                         * Constructor.
+                         *
+                         * @param env Environment.
+                         * @param javaRef Java reference.
+                         */
+                        QueryCursorImpl(ignite::common::concurrent::SharedPointer<IgniteEnvironment> env, jobject javaRef);
+
+                        /**
+                         * Destructor.
+                         */
+                        ~QueryCursorImpl();
+
+                        /**
+                         * Check whether next result exists.
+                         *
+                         * @param err Error.
+                         * @return True if exists.
+                         */
+                        bool HasNext(IgniteError& err);
+
+                        /**
+                         * Get next object.
+                         *
+                         * @param op Operation.
+                         * @param err Error.
+                         */
+                        void GetNext(OutputOperation& op, IgniteError& err);
+
+                        /**
+                         * Get next row.
+                         *
+                         * @param err Error.
+                         * @return Output row.
+                         */
+                        QueryFieldsRowImpl* GetNextRow(IgniteError& err);
+
+                        /**
+                         * Get all cursor entries.
+                         *
+                         * @param op Operation.
+                         * @param err Error.
+                         */
+                        void GetAll(OutputOperation& op, IgniteError& err);
+
+                        /**
+                         * Get all cursor entries.
+                         *
+                         * @param op Operation.
+                         */
+                        void GetAll(OutputOperation& op);
+
+                    private:
+                        /** Environment. */
+                        ignite::common::concurrent::SharedPointer<impl::IgniteEnvironment> env;
+
+                        /** Handle to Java object. */
+                        jobject javaRef;
+
+                        /** Current result batch. */
+                        QueryBatch* batch;
+
+                        /** Whether cursor has no more elements available. */
+                        bool endReached;
+
+                        /** Whether iteration methods were called. */
+                        bool iterCalled;
+
+                        /** Whether GetAll() method was called. */
+                        bool getAllCalled;
+
+                        IGNITE_NO_COPY_ASSIGNMENT(QueryCursorImpl);
+
+                        /**
+                         * Create Java-side iterator if needed.
+                         *
+                         * @param err Error.
+                         * @return True in case of success, false if an error is thrown.
+                         */
+                        bool CreateIteratorIfNeeded(IgniteError& err);
+
+                       /**
+                         * Get next result batch if update is needed.
+                         *
+                         * @param err Error.
+                         * @return True if operation has been successful.
+                         */
+                        bool GetNextBatchIfNeeded(IgniteError& err);
+
+                        /**
+                         * Check whether Java-side iterator has next element.
+                         *
+                         * @param err Error.
+                         * @return True if the next element is available.
+                         */
+                        bool IteratorHasNext(IgniteError& err);
+                    };
+                }
+            }
+        }
+    }
+}
+
+#endif //_IGNITE_IMPL_THIN_CACHE_QUERY_QUERY_FIELDS_CURSOR_IMPL
\ No newline at end of file
diff --git a/modules/platforms/cpp/thin-client/include/ignite/impl/thin/cache/query/query_fields_row_impl.h b/modules/platforms/cpp/thin-client/include/ignite/impl/thin/cache/query/query_fields_row_impl.h
new file mode 100644
index 0000000..3c08901
--- /dev/null
+++ b/modules/platforms/cpp/thin-client/include/ignite/impl/thin/cache/query/query_fields_row_impl.h
@@ -0,0 +1,197 @@
+/*
+ * 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.
+ */
+
+#ifndef _IGNITE_IMPL_CACHE_QUERY_CACHE_QUERY_FIELDS_ROW_IMPL
+#define _IGNITE_IMPL_CACHE_QUERY_CACHE_QUERY_FIELDS_ROW_IMPL
+
+#include <ignite/common/concurrent.h>
+#include <ignite/ignite_error.h>
+
+namespace ignite
+{
+    namespace impl
+    {
+        namespace cache
+        {
+            namespace query
+            {
+                /**
+                 * Query fields cursor implementation.
+                 */
+                class IGNITE_IMPORT_EXPORT QueryFieldsRowImpl
+                {
+                public:
+                    typedef common::concurrent::SharedPointer<interop::InteropMemory> SP_InteropMemory;
+
+                    /**
+                     * Constructor.
+                     *
+                     * @param mem Memory containig row data.
+                     */
+                    QueryFieldsRowImpl(SP_InteropMemory mem, int32_t rowBegin, int32_t columnNum) :
+                        mem(mem),
+                        stream(mem.Get()),
+                        reader(&stream),
+                        columnNum(columnNum),
+                        processed(0)
+                    {
+                        stream.Position(rowBegin);
+                    }
+
+                    /**
+                     * Check whether next entry exists.
+                     *
+                     * @return True if next entry exists.
+                     */
+                    bool HasNext()
+                    {
+                        IgniteError err;
+
+                        bool res = HasNext(err);
+
+                        IgniteError::ThrowIfNeeded(err);
+
+                        return res;
+                    }
+
+                    /**
+                     * Check whether next entry exists.
+                     *
+                     * @param err Error.
+                     * @return True if next entry exists.
+                     */
+                    bool HasNext(IgniteError& err)
+                    {
+                        if (IsValid())
+                            return processed < columnNum;
+                        else
+                        {
+                            err = IgniteError(IgniteError::IGNITE_ERR_GENERIC,
+                                "Instance is not usable (did you check for error?).");
+
+                            return false;
+                        }
+                    }
+
+                    /**
+                     * Get next entry.
+                     *
+                     * @return Next entry.
+                     */
+                    template<typename T>
+                    T GetNext()
+                    {
+                        IgniteError err;
+
+                        QueryFieldsRowImpl res = GetNext<T>(err);
+
+                        IgniteError::ThrowIfNeeded(err);
+
+                        return res;
+                    }
+
+                    /**
+                     * Get next entry.
+                     *
+                     * @param err Error.
+                     * @return Next entry.
+                     */
+                    template<typename T>
+                    T GetNext(IgniteError& err)
+                    {
+                        if (IsValid()) {
+                            ++processed;
+                            return reader.ReadTopObject<T>();
+                        }
+                        else
+                        {
+                            err = IgniteError(IgniteError::IGNITE_ERR_GENERIC,
+                                "Instance is not usable (did you check for error?).");
+
+                            return T();
+                        }
+                    }
+
+                    /**
+                     * Get next entry assuming it's an array of 8-byte signed
+                     * integers. Maps to "byte[]" type in Java.
+                     *
+                     * @param dst Array to store data to.
+                     * @param len Expected length of array.
+                     * @return Actual amount of elements read. If "len" argument is less than actual
+                     *     array size or resulting array is set to null, nothing will be written
+                     *     to resulting array and returned value will contain required array length.
+                     *     -1 will be returned in case array in stream was null.
+                     */
+                    int32_t GetNextInt8Array(int8_t* dst, int32_t len)
+                    {
+                        if (IsValid()) {
+
+                            int32_t actualLen = reader.ReadInt8Array(dst, len);
+
+                            if (actualLen == 0 || (dst && len >= actualLen))
+                                ++processed;
+
+                            return actualLen;
+                        }
+                        else
+                        {
+                            throw IgniteError(IgniteError::IGNITE_ERR_GENERIC,
+                                "Instance is not usable (did you check for error?).");
+                        }
+                    }
+
+                    /**
+                     * Check if the instance is valid.
+                     *
+                     * Invalid instance can be returned if some of the previous
+                     * operations have resulted in a failure. For example invalid
+                     * instance can be returned by not-throwing version of method
+                     * in case of error. Invalid instances also often can be
+                     * created using default constructor.
+                     *
+                     * @return True if the instance is valid and can be used.
+                     */
+                    bool IsValid()
+                    {
+                        return mem.Get() != 0;
+                    }
+
+                private:
+                    /** Row memory. */
+                    SP_InteropMemory mem;
+
+                    /** Row data stream. */
+                    interop::InteropInputStream stream;
+
+                    /** Row data reader. */
+                    binary::BinaryReaderImpl reader;
+
+                    /** Number of elements in a row. */
+                    int32_t columnNum;
+
+                    /** Number of elements that have been read by now. */
+                    int32_t processed;
+
+                    IGNITE_NO_COPY_ASSIGNMENT(QueryFieldsRowImpl);
+                };
+            }
+        }
+    }
+}
+
+#endif //_IGNITE_IMPL_CACHE_QUERY_CACHE_QUERY_FIELDS_ROW_IMPL
diff --git a/modules/platforms/cpp/thin-client/include/ignite/impl/thin/copyable.h b/modules/platforms/cpp/thin-client/include/ignite/impl/thin/copyable.h
new file mode 100644
index 0000000..025a7b6
--- /dev/null
+++ b/modules/platforms/cpp/thin-client/include/ignite/impl/thin/copyable.h
@@ -0,0 +1,106 @@
+/*
+ * 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.
+ */
+
+#ifndef _IGNITE_IMPL_THIN_COPYABLE
+#define _IGNITE_IMPL_THIN_COPYABLE
+
+#include <ignite/binary/binary_raw_writer.h>
+
+namespace ignite
+{
+    namespace impl
+    {
+        namespace thin
+        {
+            /**
+             * Abstraction to any type that can be copied.
+             *
+             * @tparam T Type of returned copy.
+             */
+            template<typename T>
+            class Copyable
+            {
+            public:
+                /** Type of returned copy. */
+                typedef T CopyType;
+
+                /**
+                 * Destructor.
+                 */
+                virtual ~Copyable()
+                {
+                    // No-op.
+                }
+
+                /**
+                 * Copy value.
+                 *
+                 * @return A copy of the object.
+                 */
+                virtual CopyType* Copy() const = 0;
+            };
+
+            /**
+             * Implementation of the Copyable class template for a concrete type.
+             *
+             * @tparam T Type of returned copy.
+             */
+            template<typename T>
+            class CopyableImpl : public Copyable<T>
+            {
+            public:
+                /** Type of returned copy. */
+                typedef T CopyType;
+
+                /**
+                 * Constructor.
+                 *
+                 * @param value Value.
+                 */
+                CopyableImpl(const CopyType& value) :
+                    value(value)
+                {
+                    // No-op.
+                }
+
+                /**
+                 * Destructor.
+                 */
+                virtual ~CopyableImpl()
+                {
+                    // No-op.
+                }
+
+                /**
+                 * Copy value.
+                 *
+                 * @return A copy of the object.
+                 */
+                virtual CopyType* Copy() const
+                {
+                    return new CopyType(value);
+                }
+
+            private:
+                /** Value. */
+                const CopyType& value;
+            };
+        }
+    }
+}
+
+#endif // _IGNITE_IMPL_THIN_COPYABLE
diff --git a/modules/platforms/cpp/thin-client/include/ignite/impl/thin/copyable_writable.h b/modules/platforms/cpp/thin-client/include/ignite/impl/thin/copyable_writable.h
new file mode 100644
index 0000000..e748540
--- /dev/null
+++ b/modules/platforms/cpp/thin-client/include/ignite/impl/thin/copyable_writable.h
@@ -0,0 +1,175 @@
+/*
+ * 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.
+ */
+
+#ifndef _IGNITE_IMPL_THIN_COPYABLE_WRITABLE
+#define _IGNITE_IMPL_THIN_COPYABLE_WRITABLE
+
+#include <ignite/impl/thin/copyable.h>
+#include <ignite/impl/thin/writable.h>
+
+namespace ignite
+{
+    namespace impl
+    {
+        namespace thin
+        {
+            /**
+             * Copyable Writable value.
+             */
+            class CopyableWritable : public Writable, Copyable<CopyableWritable>
+            {
+            public:
+                /**
+                 * Destructor.
+                 */
+                virtual ~CopyableWritable()
+                {
+                    // No-op.
+                }
+
+                /**
+                 * Copy value.
+                 *
+                 * @return A copy of the object.
+                 */
+                virtual CopyableWritable* Copy() const = 0;
+
+                /**
+                 * Write value using writer.
+                 *
+                 * @param writer Writer to use.
+                 */
+                virtual void Write(binary::BinaryWriterImpl& writer) const = 0;
+            };
+
+            /**
+             * Copyable Writable value implementation for a concrete type.
+             *
+             * @tparam T Value type.
+             */
+            template<typename T>
+            class CopyableWritableImpl : public CopyableWritable
+            {
+            public:
+                /** Value type. */
+                typedef T ValueType;
+
+                /**
+                 * Constructor.
+                 *
+                 * @param value Value.
+                 */
+                CopyableWritableImpl(const ValueType& value) :
+                    value(value)
+                {
+                    // No-op.
+                }
+
+                /**
+                 * Destructor.
+                 */
+                virtual ~CopyableWritableImpl()
+                {
+                    // No-op.
+                }
+
+                /**
+                 * Copy value.
+                 *
+                 * @return A copy of the object.
+                 */
+                virtual CopyableWritable* Copy() const
+                {
+                    return new CopyableWritableImpl(value);
+                }
+
+                /**
+                 * Write value using writer.
+                 *
+                 * @param writer Writer to use.
+                 */
+                virtual void Write(binary::BinaryWriterImpl& writer) const
+                {
+                    writer.WriteObject(value);
+                }
+
+            private:
+                /** Value. */
+                ValueType value;
+            };
+
+            /**
+             * Copyable Writable value implementation for int8_t array type.
+             *
+             * @tparam Iter Iterator type.
+             */
+            template<typename Iter>
+            class CopyableWritableInt8ArrayImpl : public CopyableWritable
+            {
+            public:
+                /** Iterator type. */
+                typedef Iter IteratorType;
+
+                /**
+                 * Constructor.
+                 *
+                 * @param begin Begin iterator.
+                 * @param end End iterator.
+                 */
+                CopyableWritableInt8ArrayImpl(IteratorType begin, IteratorType end) :
+                    values(begin, end)
+                {
+                    // No-op.
+                }
+
+                /**
+                 * Destructor.
+                 */
+                virtual ~CopyableWritableInt8ArrayImpl()
+                {
+                    // No-op.
+                }
+
+                /**
+                 * Copy value.
+                 *
+                 * @return A copy of the object.
+                 */
+                virtual CopyableWritable* Copy() const
+                {
+                    return new CopyableWritableInt8ArrayImpl(values.begin(), values.end());
+                }
+
+                /**
+                 * Write value using writer.
+                 *
+                 * @param writer Writer to use.
+                 */
+                virtual void Write(binary::BinaryWriterImpl& writer) const
+                {
+                    writer.WriteInt8Array(&values[0], static_cast<int32_t>(values.size()));
+                }
+
+            private:
+                /** Value. */
+                std::vector<int8_t> values;
+            };
+        }
+    }
+}
+
+#endif // _IGNITE_IMPL_THIN_COPYABLE_WRITABLE
diff --git a/modules/platforms/cpp/thin-client/include/ignite/thin/cache/cache_client.h b/modules/platforms/cpp/thin-client/include/ignite/thin/cache/cache_client.h
index b9cf259..e53ad98 100644
--- a/modules/platforms/cpp/thin-client/include/ignite/thin/cache/cache_client.h
+++ b/modules/platforms/cpp/thin-client/include/ignite/thin/cache/cache_client.h
@@ -25,10 +25,13 @@
 
 #include <ignite/common/concurrent.h>
 
+#include <ignite/thin/cache/query/query_fields_cursor.h>
+#include <ignite/thin/cache/query/query_sql_fields.h>
+
 #include <ignite/impl/thin/writable.h>
 #include <ignite/impl/thin/writable_key.h>
-#include <ignite/impl/thin/readable.h>
 
+#include <ignite/impl/thin/readable.h>
 #include <ignite/impl/thin/cache/cache_client_proxy.h>
 
 namespace ignite
@@ -45,9 +48,9 @@ namespace ignite
              * Both key and value types should be default-constructable, copy-constructable and assignable. Also
              * BinaryType class  template should be specialized for both types, if they are not one of the basic types.
              *
-             * This class implemented as a reference to an implementation so copying of this class instance will only
-             * create another reference to the same underlying object. Underlying object released automatically once all
-             * the instances are destructed.
+             * This class is implemented as a reference to an implementation so copying of this class instance will only
+             * create another reference to the same underlying object. Underlying object will be released automatically
+             * once all the instances are destructed.
              *
              * @tparam K Cache key type.
              * @tparam V Cache value type.
@@ -535,7 +538,7 @@ namespace ignite
                  * its turn may load the value from the swap storage, and consecutively, if it's not in swap, from
                  * the underlying persistent storage.
                  *
-                 *  If the returned value is not needed, method putxIfAbsent() should be used instead of this one to
+                 * If the returned value is not needed, method putxIfAbsent() should be used instead of this one to
                  * avoid the overhead associated with returning of the previous value.
                  *
                  * If write-through is enabled, the stored value will be persisted to store.
@@ -560,10 +563,10 @@ namespace ignite
                  * If cache previously contained value for the given key, then this value is returned.
                  *
                  * In case of PARTITIONED or REPLICATED caches, the value will be loaded from the primary node, which in
-                 * its turn may load the value from the swap storage, and consecutively, if it's not in swap, from
-                 * the underlying persistent storage.
+                 * its turn may load the value from the swap storage, and consecutively, if it's not in swap, from the
+                 * underlying persistent storage.
                  *
-                 *  If the returned value is not needed, method putxIfAbsent() should be used instead of this one to
+                 * If the returned value is not needed, method putxIfAbsent() should be used instead of this one to
                  * avoid the overhead associated with returning of the previous value.
                  *
                  * If write-through is enabled, the stored value will be persisted to store.
@@ -583,6 +586,17 @@ namespace ignite
                 }
 
                 /**
+                 * Perform SQL fields query.
+                 *
+                 * @param qry Query.
+                 * @return Query fields cursor.
+                 */
+                query::QueryFieldsCursor Query(const query::SqlFieldsQuery& qry)
+                {
+                    return proxy.Query(qry);
+                }
+
+                /**
                  * Refresh affinity mapping.
                  *
                  * @deprecated Does nothing since Apache Ignite 2.8. Affinity mapping is refreshed automatically now.
diff --git a/modules/platforms/cpp/thin-client/include/ignite/thin/cache/query/query_fields_cursor.h b/modules/platforms/cpp/thin-client/include/ignite/thin/cache/query/query_fields_cursor.h
new file mode 100644
index 0000000..2ba87f7
--- /dev/null
+++ b/modules/platforms/cpp/thin-client/include/ignite/thin/cache/query/query_fields_cursor.h
@@ -0,0 +1,94 @@
+/*
+ * 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.
+ */
+
+/**
+ * @file
+ * Declares ignite::thin::cache::query::QueryFieldsCursor class.
+ */
+
+#ifndef _IGNITE_THIN_CACHE_QUERY_QUERY_FIELDS_CURSOR
+#define _IGNITE_THIN_CACHE_QUERY_QUERY_FIELDS_CURSOR
+
+#include <vector>
+
+#include <ignite/common/concurrent.h>
+
+#include <ignite/ignite_error.h>
+#include <ignite/thin/cache/query/query_fields_row.h>
+
+namespace ignite
+{
+    namespace thin
+    {
+        namespace cache
+        {
+            namespace query
+            {
+                /**
+                 * Query fields cursor.
+                 *
+                 * This class is implemented as a reference to an implementation so copying of this class instance will
+                 * only create another reference to the same underlying object. Underlying object will be released
+                 * automatically once all the instances are destructed.
+                 */
+                class IGNITE_IMPORT_EXPORT QueryFieldsCursor
+                {
+                public:
+                    /**
+                     * Constructor.
+                     *
+                     * Internal method. Should not be used by user.
+                     *
+                     * @param impl Implementation.
+                     */
+                    explicit QueryFieldsCursor(const common::concurrent::SharedPointer<void>& impl);
+
+                    /**
+                     * Check whether next entry exists.
+                     *
+                     * @return True if next entry exists.
+                     *
+                     * @throw IgniteError class instance in case of failure.
+                     */
+                    bool HasNext();
+
+                    /**
+                     * Get next entry.
+                     *
+                     * @return Next entry.
+                     *
+                     * @throw IgniteError class instance in case of failure.
+                     */
+                    QueryFieldsRow GetNext();
+
+                    /**
+                     * Get column names.
+                     *
+                     * @return Column names.
+                     */
+                    const std::vector<std::string>& GetColumnNames() const;
+
+                private:
+                    /** Implementation delegate. */
+                    common::concurrent::SharedPointer<void> impl;
+                };
+            }
+        }
+    }
+}
+
+#endif //_IGNITE_THIN_CACHE_QUERY_QUERY_FIELDS_CURSOR
diff --git a/modules/platforms/cpp/thin-client/include/ignite/thin/cache/query/query_fields_row.h b/modules/platforms/cpp/thin-client/include/ignite/thin/cache/query/query_fields_row.h
new file mode 100644
index 0000000..1fd1d20
--- /dev/null
+++ b/modules/platforms/cpp/thin-client/include/ignite/thin/cache/query/query_fields_row.h
@@ -0,0 +1,104 @@
+/*
+ * 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.
+ */
+
+/**
+ * @file
+ * Declares ignite::thin::cache::query::QueryFieldsRow class.
+ */
+
+#ifndef _IGNITE_THIN_CACHE_QUERY_QUERY_FIELDS_ROW
+#define _IGNITE_THIN_CACHE_QUERY_QUERY_FIELDS_ROW
+
+#include <ignite/common/concurrent.h>
+#include <ignite/ignite_error.h>
+
+#include <ignite/impl/thin/readable.h>
+
+namespace ignite
+{
+    namespace thin
+    {
+        namespace cache
+        {
+            namespace query
+            {
+                /**
+                 * Query fields row.
+                 *
+                 * This class is implemented as a reference to an implementation so copying of this class instance will
+                 * only create another reference to the same underlying object. Underlying object will be released
+                 * automatically once all the instances are destructed.
+                 */
+                class IGNITE_IMPORT_EXPORT QueryFieldsRow
+                {
+                public:
+                    /**
+                     * Constructor.
+                     *
+                     * Internal method. Should not be used by user.
+                     *
+                     * @param impl Implementation.
+                     */
+                    explicit QueryFieldsRow(const common::concurrent::SharedPointer<void>& impl);
+
+                    /**
+                     * Check whether next entry exists.
+                     *
+                     * @return True if next entry exists.
+                     */
+                    bool HasNext();
+
+                    /**
+                     * Get next entry.
+                     *
+                     * @tparam T Value type to get. Should be default-constructable, copy-constructable and assignable.
+                     *     Also BinaryType class template should be specialized for this type.
+                     *
+                     * @return Next entry.
+                     *
+                     * @throw IgniteError class instance in case of failure.
+                     */
+                    template<typename T>
+                    T GetNext()
+                    {
+                        T res;
+                        impl::thin::ReadableImpl<T> readable(res);
+
+                        InternalGetNext(readable);
+
+                        return res;
+                    }
+
+                private:
+                    /**
+                     * Get next entry.
+                     *
+                     * @param readable Value to read.
+                     *
+                     * @throw IgniteError class instance in case of failure.
+                     */
+                    void InternalGetNext(impl::thin::Readable& readable);
+
+                    /** Implementation delegate. */
+                    common::concurrent::SharedPointer<void> impl;
+                };
+            }
+        }
+    }    
+}
+
+#endif //_IGNITE_THIN_CACHE_QUERY_QUERY_FIELDS_ROW
diff --git a/modules/platforms/cpp/thin-client/include/ignite/thin/cache/query/query_sql_fields.h b/modules/platforms/cpp/thin-client/include/ignite/thin/cache/query/query_sql_fields.h
new file mode 100644
index 0000000..d350138
--- /dev/null
+++ b/modules/platforms/cpp/thin-client/include/ignite/thin/cache/query/query_sql_fields.h
@@ -0,0 +1,465 @@
+/*
+ * 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.
+ */
+
+/**
+ * @file
+ * Declares ignite::thin::cache::query::SqlFieldsQuery class.
+ */
+
+#ifndef _IGNITE_THIN_CACHE_QUERY_QUERY_SQL_FIELDS
+#define _IGNITE_THIN_CACHE_QUERY_QUERY_SQL_FIELDS
+
+#include <stdint.h>
+#include <string>
+#include <vector>
+
+#include <ignite/impl/thin/copyable_writable.h>
+
+namespace ignite
+{
+    namespace impl
+    {
+        namespace thin
+        {
+            // Forward declaration
+            class SqlFieldsQueryRequest;
+        }
+    }
+
+    namespace thin
+    {
+        namespace cache
+        {
+            namespace query
+            {
+                /**
+                 * SQL fields query for thin client.
+                 */
+                class SqlFieldsQuery
+                {
+                public:
+                    friend class ignite::impl::thin::SqlFieldsQueryRequest;
+
+                    /**
+                     * Constructor.
+                     *
+                     * @param sql SQL string.
+                     */
+                    explicit SqlFieldsQuery(const std::string& sql) :
+                        sql(sql),
+                        schema(),
+                        pageSize(1024),
+                        maxRows(0),
+                        timeout(0),
+                        loc(false),
+                        distributedJoins(false),
+                        enforceJoinOrder(false),
+                        lazy(false),
+                        collocated(false),
+                        args()
+                    {
+                        // No-op.
+                    }
+
+                    /**
+                     * Copy constructor.
+                     *
+                     * @param other Other instance.
+                     */
+                    SqlFieldsQuery(const SqlFieldsQuery& other) :
+                        sql(other.sql),
+                        schema(other.schema),
+                        pageSize(other.pageSize),
+                        maxRows(other.maxRows),
+                        timeout(other.timeout),
+                        loc(other.loc),
+                        distributedJoins(other.distributedJoins),
+                        enforceJoinOrder(other.enforceJoinOrder),
+                        lazy(other.lazy),
+                        collocated(other.collocated),
+                        args()
+                    {
+                        args.reserve(other.args.size());
+
+                        typedef std::vector<impl::thin::CopyableWritable*>::const_iterator Iter;
+
+                        for (Iter i = other.args.begin(); i != other.args.end(); ++i)
+                            args.push_back((*i)->Copy());
+                    }
+
+                    /**
+                     * Assignment operator.
+                     *
+                     * @param other Other instance.
+                     */
+                    SqlFieldsQuery& operator=(const SqlFieldsQuery& other)
+                    {
+                        if (this != &other)
+                        {
+                            SqlFieldsQuery tmp(other);
+
+                            Swap(tmp);
+                        }
+
+                        return *this;
+                    }
+
+                    /**
+                     * Destructor.
+                     */
+                    ~SqlFieldsQuery()
+                    {
+                        ClearArguments();
+                    }
+
+                    /**
+                     * Efficiently swaps contents with another SqlQuery instance.
+                     *
+                     * @param other Other instance.
+                     */
+                    void Swap(SqlFieldsQuery& other)
+                    {
+                        if (this != &other)
+                        {
+                            using std::swap;
+
+                            swap(sql, other.sql);
+                            swap(schema, other.schema);
+                            swap(pageSize, other.pageSize);
+                            swap(maxRows, other.maxRows);
+                            swap(timeout, other.timeout);
+                            swap(loc, other.loc);
+                            swap(distributedJoins, other.distributedJoins);
+                            swap(enforceJoinOrder, other.enforceJoinOrder);
+                            swap(lazy, other.lazy);
+                            swap(collocated, other.collocated);
+                            swap(args, other.args);
+                        }
+                    }
+
+                    /**
+                     * Get SQL string.
+                     *
+                     * @return SQL string.
+                     */
+                    const std::string& GetSql() const
+                    {
+                        return sql;
+                    }
+
+                    /**
+                     * Set SQL string.
+                     *
+                     * @param sql SQL string.
+                     */
+                    void SetSql(const std::string& sql)
+                    {
+                        this->sql = sql;
+                    }
+
+                    /**
+                     * Set schema name for the query.
+                     * If not set, current cache name is used, which means you can omit schema name for tables within
+                     * the current cache.
+                     *
+                     * @param schema Schema. Empty string to unset.
+                     */
+                    void SetSchema(const std::string& schema)
+                    {
+                        this->schema = schema;
+                    }
+
+                    /**
+                     * Get schema name for the query.
+                     *
+                     * If not set, current cache name is used, which means you can omit schema name for tables within
+                     * the current cache.
+                     *
+                     * @return Schema. Empty string if not set.
+                     */
+                    const std::string& GetSchema() const
+                    {
+                        return schema;
+                    }
+
+                    /**
+                     * Get page size.
+                     *
+                     * @return Page size.
+                     */
+                    int32_t GetPageSize() const
+                    {
+                        return pageSize;
+                    }
+
+                    /**
+                     * Set page size.
+                     *
+                     * @param pageSize Page size.
+                     */
+                    void SetPageSize(int32_t pageSize)
+                    {
+                        this->pageSize = pageSize;
+                    }
+
+                    /**
+                     * Set maximum number of rows.
+                     *
+                     * @param maxRows Max rows.
+                     */
+                    void SetMaxRows(int32_t maxRows)
+                    {
+                        this->maxRows = maxRows;
+                    }
+
+                    /**
+                     * Get maximum number of rows.
+                     *
+                     * @return Max rows.
+                     */
+                    int32_t GetMaxRows() const
+                    {
+                        return maxRows;
+                    }
+
+                    /**
+                     * Set query execution timeout in milliseconds.
+                     *
+                     * @param timeout Timeout in milliseconds.
+                     */
+                    void SetTimeout(int64_t timeout)
+                    {
+                        this->timeout = timeout;
+                    }
+
+                    /**
+                     * Get query execution timeout in milliseconds.
+                     *
+                     * @return Timeout in milliseconds.
+                     */
+                    int64_t GetTimeout() const
+                    {
+                        return timeout;
+                    }
+
+                    /**
+                     * Get local flag.
+                     *
+                     * @return Local flag.
+                     */
+                    bool IsLocal() const
+                    {
+                        return loc;
+                    }
+
+                    /**
+                     * Set local flag.
+                     *
+                     * @param loc Local flag.
+                     */
+                    void SetLocal(bool loc)
+                    {
+                        this->loc = loc;
+                    }
+
+                    /**
+                     * Check if distributed joins are enabled for this query.
+                     *
+                     * @return True If distributed join enabled.
+                     */
+                    bool IsDistributedJoins() const
+                    {
+                        return distributedJoins;
+                    }
+
+                    /**
+                     * Specify if distributed joins are enabled for this query.
+                     *
+                     * When disabled, join results will only contain collocated data (joins work locally).
+                     * When enabled, joins work as expected, no matter how the data is distributed.
+                     *
+                     * @param enabled Distributed joins enabled.
+                     */
+                    void SetDistributedJoins(bool enabled)
+                    {
+                        distributedJoins = enabled;
+                    }
+
+                    /**
+                     * Checks if join order of tables if enforced.
+                     *
+                     * @return Flag value.
+                     */
+                    bool IsEnforceJoinOrder() const
+                    {
+                        return enforceJoinOrder;
+                    }
+
+                    /**
+                     * Sets flag to enforce join order of tables in the query.
+                     *
+                     * If set to true query optimizer will not reorder tables in join. By default is false.
+                     *
+                     * It is not recommended to enable this property unless you are sure that your indexes and the query
+                     * itself are correct and tuned as much as possible but query optimizer still produces wrong join
+                     * order.
+                     *
+                     * @param enforce Flag value.
+                     */
+                    void SetEnforceJoinOrder(bool enforce)
+                    {
+                        enforceJoinOrder = enforce;
+                    }
+
+                    /**
+                     * Gets lazy query execution flag.
+                     *
+                     * See SetLazy(bool) for more information.
+                     *
+                     * @return Lazy flag.
+                     */
+                    bool IsLazy() const
+                    {
+                        return lazy;
+                    }
+
+                    /**
+                     * Sets lazy query execution flag.
+                     *
+                     * By default Ignite attempts to fetch the whole query result set to memory and send it to the
+                     * client. For small and medium result sets this provides optimal performance and minimize duration
+                     * of internal database locks, thus increasing concurrency.
+                     *
+                     * If result set is too big to fit in available memory this could lead to excessive GC pauses and
+                     * even OutOfMemoryError. Use this flag as a hint for Ignite to fetch result set lazily, thus
+                     * minimizing memory consumption at the cost of moderate performance hit.
+                     *
+                     * Defaults to @c false, meaning that the whole result set is fetched to memory eagerly.
+                     *
+                     * @param lazy Lazy query execution flag.
+                     */
+                    void SetLazy(bool lazy)
+                    {
+                        this->lazy = lazy;
+                    }
+
+                    /**
+                     * Checks if this query is collocated.
+                     *
+                     * @return @c true If the query is collocated.
+                     */
+                    bool IsCollocated()
+                    {
+                        return collocated;
+                    }
+
+                    /**
+                     * Sets flag defining if this query is collocated.
+                     *
+                     * Collocation flag is used for optimization purposes of queries with GROUP BY statements.
+                     * Whenever Ignite executes a distributed query, it sends sub-queries to individual cluster members.
+                     * If you know in advance that the elements of your query selection are collocated together on the
+                     * same node and you group by collocated key (primary or affinity key), then Ignite can make
+                     * significant performance and network optimizations by grouping data on remote nodes.
+                     *
+                     * @param collocated Flag value.
+                     */
+                    void SetCollocated(bool collocated)
+                    {
+                        this->collocated = collocated;
+                    }
+
+                    /**
+                     * Add argument for the query.
+                     *
+                     * @tparam T Type of argument. Should be should be copy-constructable and assignable. BinaryType
+                     * class template should be specialized for this type.
+                     *
+                     * @param arg Argument.
+                     */
+                    template<typename T>
+                    void AddArgument(const T& arg)
+                    {
+                        args.push_back(new impl::thin::CopyableWritableImpl<T>(arg));
+                    }
+
+                    /**
+                     * Add int8_t array as an argument.
+                     *
+                     * @tparam Iter Iterator type. Should provide standard iterator functionality.
+                     *
+                     * @param begin Begin iterator of sequence to write.
+                     * @param end End iterator of sequence to write.
+                     */
+                    template<typename Iter>
+                    void AddInt8ArrayArgument(Iter begin, Iter end)
+                    {
+                        args.push_back(new impl::thin::CopyableWritableInt8ArrayImpl<Iter>(begin, end));
+                    }
+
+                    /**
+                     * Remove all added arguments.
+                     */
+                    void ClearArguments()
+                    {
+                        std::vector<impl::thin::CopyableWritable*>::iterator iter;
+                        for (iter = args.begin(); iter != args.end(); ++iter)
+                            delete *iter;
+
+                        args.clear();
+                    }
+
+                private:
+                    /** SQL string. */
+                    std::string sql;
+
+                    /** SQL Schema. */
+                    std::string schema;
+
+                    /** Page size. */
+                    int32_t pageSize;
+
+                    /** Max rows to fetch. */
+                    int32_t maxRows;
+
+                    /** Timeout. */
+                    int64_t timeout;
+
+                    /** Local flag. */
+                    bool loc;
+
+                    /** Distributed joins flag. */
+                    bool distributedJoins;
+
+                    /** Enforce join order flag. */
+                    bool enforceJoinOrder;
+
+                    /** Lazy flag. */
+                    bool lazy;
+
+                    /** Collocated flag. */
+                    bool collocated;
+
+                    /** Arguments. */
+                    std::vector<impl::thin::CopyableWritable*> args;
+                };
+            }
+        }
+    }
+}
+
+#endif //_IGNITE_THIN_CACHE_QUERY_QUERY_SQL_FIELDS
diff --git a/modules/platforms/cpp/thin-client/include/ignite/thin/ignite_client.h b/modules/platforms/cpp/thin-client/include/ignite/thin/ignite_client.h
index b540edb..d8d9a88 100644
--- a/modules/platforms/cpp/thin-client/include/ignite/thin/ignite_client.h
+++ b/modules/platforms/cpp/thin-client/include/ignite/thin/ignite_client.h
@@ -41,9 +41,9 @@ namespace ignite
          * This is an entry point for Thin C++ Ignite client. Its main purpose is to establish connection to the remote
          * server nodes.
          *
-         * This class implemented as a reference to an implementation so copying of this class instance will only
-         * create another reference to the same underlying object. Underlying object released automatically once all
-         * the instances are destructed.
+         * This class is implemented as a reference to an implementation so copying of this class instance will only
+         * create another reference to the same underlying object. Underlying object will be released automatically once
+         * all the instances are destructed.
          */
         class IGNITE_IMPORT_EXPORT IgniteClient
         {
diff --git a/modules/platforms/cpp/thin-client/include/ignite/thin/transactions/transaction.h b/modules/platforms/cpp/thin-client/include/ignite/thin/transactions/transaction.h
index 97d75e6..43f5223 100644
--- a/modules/platforms/cpp/thin-client/include/ignite/thin/transactions/transaction.h
+++ b/modules/platforms/cpp/thin-client/include/ignite/thin/transactions/transaction.h
@@ -31,9 +31,9 @@ namespace ignite
              *
              * Implements main transactionsl API.
              *
-             * This class implemented as a reference to an implementation so copying of this class instance will only
-             * create another reference to the same underlying object. Underlying object released automatically once all
-             * the instances are destructed.
+             * This class is implemented as a reference to an implementation so copying of this class instance will only
+             * create another reference to the same underlying object. Underlying object will be released automatically
+             * once all the instances are destructed.
              */
             class ClientTransaction {
 
diff --git a/modules/platforms/cpp/thin-client/include/ignite/thin/transactions/transactions.h b/modules/platforms/cpp/thin-client/include/ignite/thin/transactions/transactions.h
index 3a76586..f6822e3 100644
--- a/modules/platforms/cpp/thin-client/include/ignite/thin/transactions/transactions.h
+++ b/modules/platforms/cpp/thin-client/include/ignite/thin/transactions/transactions.h
@@ -36,9 +36,9 @@ namespace ignite
              *
              * This is an entry point for Thin C++ Ignite transactions.
              *
-             * This class implemented as a reference to an implementation so copying of this class instance will only
-             * create another reference to the same underlying object. Underlying object released automatically once all
-             * the instances are destructed.
+             * This class is implemented as a reference to an implementation so copying of this class instance will only
+             * create another reference to the same underlying object. Underlying object will be released automatically
+             * once all the instances are destructed.
              */
             class ClientTransactions {
             public:
diff --git a/modules/platforms/cpp/thin-client/project/vs/thin-client.vcxproj b/modules/platforms/cpp/thin-client/project/vs/thin-client.vcxproj
index a4b8490..4d60182 100644
--- a/modules/platforms/cpp/thin-client/project/vs/thin-client.vcxproj
+++ b/modules/platforms/cpp/thin-client/project/vs/thin-client.vcxproj
@@ -154,6 +154,8 @@
   </ItemDefinitionGroup>
   <ItemGroup>
     <ClCompile Include="..\..\src\ignite_client.cpp" />
+    <ClCompile Include="..\..\src\cache\query\query_fields_cursor.cpp" />
+    <ClCompile Include="..\..\src\cache\query\query_fields_row.cpp" />
     <ClCompile Include="..\..\src\impl\affinity\affinity_assignment.cpp" />
     <ClCompile Include="..\..\src\impl\affinity\affinity_manager.cpp" />
     <ClCompile Include="..\..\src\impl\affinity\affinity_topology_version.cpp" />
@@ -172,11 +174,18 @@
   </ItemGroup>
   <ItemGroup>
     <ClInclude Include="..\..\include\ignite\impl\thin\cache\cache_client_proxy.h" />
+    <ClInclude Include="..\..\include\ignite\impl\thin\cache\query\query_fields_cursor_impl.h" />
+    <ClInclude Include="..\..\include\ignite\impl\thin\cache\query\query_fields_row_impl.h" />
+    <ClInclude Include="..\..\include\ignite\impl\thin\copyable.h" />
+    <ClInclude Include="..\..\include\ignite\impl\thin\copyable_writable.h" />
     <ClInclude Include="..\..\include\ignite\impl\thin\readable.h" />
     <ClInclude Include="..\..\include\ignite\impl\thin\writable.h" />
     <ClInclude Include="..\..\include\ignite\impl\thin\writable_key.h" />
     <ClInclude Include="..\..\include\ignite\thin\cache\cache_client.h" />
     <ClInclude Include="..\..\include\ignite\thin\cache\cache_peek_mode.h" />
+    <ClInclude Include="..\..\include\ignite\thin\cache\query\query_fields_cursor.h" />
+    <ClInclude Include="..\..\include\ignite\thin\cache\query\query_fields_row.h" />
+    <ClInclude Include="..\..\include\ignite\thin\cache\query\query_sql_fields.h" />
     <ClInclude Include="..\..\include\ignite\thin\ignite_client.h" />
     <ClInclude Include="..\..\include\ignite\thin\ignite_client_configuration.h" />
     <ClInclude Include="..\..\include\ignite\thin\ssl_mode.h" />
@@ -187,6 +196,9 @@
     <ClInclude Include="..\..\src\impl\affinity\cache_affinity_configs.h" />
     <ClInclude Include="..\..\src\impl\affinity\node_partitions.h" />
     <ClInclude Include="..\..\src\impl\cache\cache_client_impl.h" />
+    <ClInclude Include="..\..\src\impl\cache\query\cursor_page.h" />
+    <ClInclude Include="..\..\src\impl\cache\query\query_fields_cursor_impl.h" />
+    <ClInclude Include="..\..\src\impl\cache\query\query_fields_row_impl.h" />
     <ClInclude Include="..\..\src\impl\data_channel.h" />
     <ClInclude Include="..\..\src\impl\data_router.h" />
     <ClInclude Include="..\..\src\impl\ignite_client_impl.h" />
diff --git a/modules/platforms/cpp/thin-client/project/vs/thin-client.vcxproj.filters b/modules/platforms/cpp/thin-client/project/vs/thin-client.vcxproj.filters
index a8cfbca..263325c 100644
--- a/modules/platforms/cpp/thin-client/project/vs/thin-client.vcxproj.filters
+++ b/modules/platforms/cpp/thin-client/project/vs/thin-client.vcxproj.filters
@@ -25,6 +25,12 @@
     <ClCompile Include="..\..\src\ignite_client.cpp">
       <Filter>Code</Filter>
     </ClCompile>
+    <ClCompile Include="..\..\src\cache\query\query_fields_cursor.cpp">
+      <Filter>Code\cache\query</Filter>
+    </ClCompile>
+    <ClCompile Include="..\..\src\cache\query\query_fields_row.cpp">
+      <Filter>Code\cache\query</Filter>
+    </ClCompile>
     <ClCompile Include="..\..\src\impl\data_router.cpp">
       <Filter>Code\impl</Filter>
     </ClCompile>
@@ -81,9 +87,21 @@
     <ClInclude Include="..\..\include\ignite\impl\thin\readable.h">
       <Filter>Code\impl</Filter>
     </ClInclude>
+    <ClInclude Include="..\..\include\ignite\impl\thin\copyable.h">
+      <Filter>Code\impl</Filter>
+    </ClInclude>
+    <ClInclude Include="..\..\include\ignite\impl\thin\copyable_writable.h">
+      <Filter>Code\impl</Filter>
+    </ClInclude>
     <ClInclude Include="..\..\include\ignite\impl\thin\cache\cache_client_proxy.h">
       <Filter>Code\impl\cache</Filter>
     </ClInclude>
+    <ClInclude Include="..\..\include\ignite\impl\thin\cache\query\query_fields_cursor_impl.h">
+      <Filter>Code\impl\cache\query</Filter>
+    </ClInclude>
+    <ClInclude Include="..\..\include\ignite\impl\thin\cache\query\query_fields_row_impl.h">
+      <Filter>Code\impl\cache\query</Filter>
+    </ClInclude>
     <ClInclude Include="..\..\include\ignite\impl\thin\writable_key.h">
       <Filter>Code\impl</Filter>
     </ClInclude>
@@ -111,9 +129,27 @@
     <ClInclude Include="..\..\src\impl\cache\cache_client_impl.h">
       <Filter>Code\impl\cache</Filter>
     </ClInclude>
+    <ClInclude Include="..\..\src\impl\cache\query\cursor_page.h">
+      <Filter>Code\impl\cache\query</Filter>
+    </ClInclude>
+    <ClInclude Include="..\..\src\impl\cache\query\query_fields_cursor_impl.h">
+      <Filter>Code\impl\cache\query</Filter>
+    </ClInclude>
+    <ClInclude Include="..\..\src\impl\cache\query\query_fields_row_impl.h">
+      <Filter>Code\impl\cache\query</Filter>
+    </ClInclude>
     <ClInclude Include="..\..\include\ignite\thin\cache\cache_peek_mode.h">
       <Filter>Code\cache</Filter>
     </ClInclude>
+    <ClInclude Include="..\..\include\ignite\thin\cache\query\query_fields_cursor.h">
+      <Filter>Code\cache\query</Filter>
+    </ClInclude>
+    <ClInclude Include="..\..\include\ignite\thin\cache\query\query_fields_row.h">
+      <Filter>Code\cache\query</Filter>
+    </ClInclude>
+    <ClInclude Include="..\..\include\ignite\thin\cache\query\query_sql_fields.h">
+      <Filter>Code\cache\query</Filter>
+    </ClInclude>
     <ClInclude Include="..\..\src\impl\remote_type_updater.h">
       <Filter>Code\impl</Filter>
     </ClInclude>
diff --git a/modules/platforms/cpp/thin-client/src/cache/query/query_fields_cursor.cpp b/modules/platforms/cpp/thin-client/src/cache/query/query_fields_cursor.cpp
new file mode 100644
index 0000000..af3fdb3
--- /dev/null
+++ b/modules/platforms/cpp/thin-client/src/cache/query/query_fields_cursor.cpp
@@ -0,0 +1,69 @@
+/*
+ * 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.
+ */
+
+#include <ignite/thin/cache/query/query_fields_cursor.h>
+
+#include "impl/cache/query/query_fields_cursor_impl.h"
+
+namespace
+{
+    using namespace ignite::common::concurrent;
+    using namespace ignite::impl::thin::cache::query;
+
+    QueryFieldsCursorImpl& GetQueryFieldsCursorImpl(SharedPointer<void>& ptr)
+    {
+        return *reinterpret_cast<QueryFieldsCursorImpl*>(ptr.Get());
+    }
+
+    const QueryFieldsCursorImpl& GetQueryFieldsCursorImpl(const SharedPointer<void>& ptr)
+    {
+        return *reinterpret_cast<const QueryFieldsCursorImpl*>(ptr.Get());
+    }
+}
+
+namespace ignite
+{
+    namespace thin
+    {
+        namespace cache
+        {
+            namespace query
+            {
+                QueryFieldsCursor::QueryFieldsCursor(const common::concurrent::SharedPointer<void> &impl) :
+                    impl(impl)
+                {
+                    // No-op.
+                }
+
+                bool query::QueryFieldsCursor::HasNext()
+                {
+                    return GetQueryFieldsCursorImpl(impl).HasNext();
+                }
+
+                QueryFieldsRow QueryFieldsCursor::GetNext()
+                {
+                    return GetQueryFieldsCursorImpl(impl).GetNext();
+                }
+
+                const std::vector<std::string>& QueryFieldsCursor::GetColumnNames() const
+                {
+                    return GetQueryFieldsCursorImpl(impl).GetColumns();
+                }
+            }
+        }
+    }
+}
diff --git a/modules/platforms/cpp/thin-client/src/cache/query/query_fields_row.cpp b/modules/platforms/cpp/thin-client/src/cache/query/query_fields_row.cpp
new file mode 100644
index 0000000..8cdb23c
--- /dev/null
+++ b/modules/platforms/cpp/thin-client/src/cache/query/query_fields_row.cpp
@@ -0,0 +1,59 @@
+/*
+ * 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.
+ */
+
+#include <ignite/thin/cache/query/query_fields_row.h>
+
+#include "impl/cache/query/query_fields_row_impl.h"
+
+namespace
+{
+    using namespace ignite::common::concurrent;
+    using namespace ignite::impl::thin::cache::query;
+
+    QueryFieldsRowImpl& GetQueryFieldsRowImpl(SharedPointer<void>& ptr)
+    {
+        return *reinterpret_cast<QueryFieldsRowImpl*>(ptr.Get());
+    }
+}
+
+namespace ignite
+{
+    namespace thin
+    {
+        namespace cache
+        {
+            namespace query
+            {
+                QueryFieldsRow::QueryFieldsRow(const common::concurrent::SharedPointer<void>& impl) :
+                    impl(impl)
+                {
+                    // No-op.
+                }
+
+                bool QueryFieldsRow::HasNext()
+                {
+                    return GetQueryFieldsRowImpl(impl).HasNext();
+                }
+
+                void QueryFieldsRow::InternalGetNext(impl::thin::Readable &readable)
+                {
+                    GetQueryFieldsRowImpl(impl).GetNext(readable);
+                }
+            }
+        }
+    }    
+}
diff --git a/modules/platforms/cpp/thin-client/src/impl/cache/cache_client_impl.cpp b/modules/platforms/cpp/thin-client/src/impl/cache/cache_client_impl.cpp
index b128cc0..51b0c28 100644
--- a/modules/platforms/cpp/thin-client/src/impl/cache/cache_client_impl.cpp
+++ b/modules/platforms/cpp/thin-client/src/impl/cache/cache_client_impl.cpp
@@ -91,12 +91,14 @@ namespace ignite
                 }
 
                 template<typename ReqT, typename RspT>
-                void CacheClientImpl::SyncMessage(const ReqT& req, RspT& rsp)
+                SP_DataChannel CacheClientImpl::SyncMessage(const ReqT& req, RspT& rsp)
                 {
-                    router.Get()->SyncMessage(req, rsp);
+                    SP_DataChannel channel = router.Get()->SyncMessage(req, rsp);
 
                     if (rsp.GetStatus() != ResponseStatus::SUCCESS)
                         throw IgniteError(IgniteError::IGNITE_ERR_CACHE, rsp.GetError().c_str());
+
+                    return channel;
                 }
 
                 template<typename ReqT>
@@ -368,6 +370,25 @@ namespace ignite
 
                     SyncCacheKeyMessage(key, req, rsp);
                 }
+
+                query::SP_QueryFieldsCursorImpl CacheClientImpl::Query(
+                    const ignite::thin::cache::query::SqlFieldsQuery &qry)
+                {
+                    SqlFieldsQueryRequest req(id, qry);
+                    SqlFieldsQueryResponse rsp;
+
+                    SP_DataChannel channel = SyncMessage(req, rsp);
+
+                    query::SP_QueryFieldsCursorImpl cursorImpl(
+                        new query::QueryFieldsCursorImpl(
+                            rsp.GetCursorId(),
+                            rsp.GetColumns(),
+                            rsp.GetCursorPage(),
+                            channel,
+                            static_cast<int32_t>(qry.GetTimeout())));
+
+                    return cursorImpl;
+                }
             }
         }
     }
diff --git a/modules/platforms/cpp/thin-client/src/impl/cache/cache_client_impl.h b/modules/platforms/cpp/thin-client/src/impl/cache/cache_client_impl.h
index e97fe17..d74ad29 100644
--- a/modules/platforms/cpp/thin-client/src/impl/cache/cache_client_impl.h
+++ b/modules/platforms/cpp/thin-client/src/impl/cache/cache_client_impl.h
@@ -21,8 +21,11 @@
 #include <stdint.h>
 #include <string>
 
+#include <ignite/thin/cache/query/query_sql_fields.h>
+
 #include "impl/data_router.h"
 #include "impl/transactions/transactions_impl.h"
+#include "impl/cache/query/query_fields_cursor_impl.h"
 
 namespace ignite
 {
@@ -287,6 +290,14 @@ namespace ignite
                      */
                     void GetAndPutIfAbsent(const WritableKey& key, const Writable& valIn, Readable& valOut);
 
+                    /**
+                     * Perform SQL fields query.
+                     *
+                     * @param qry Query.
+                     * @return Query cursor.
+                     */
+                    query::SP_QueryFieldsCursorImpl Query(const ignite::thin::cache::query::SqlFieldsQuery &qry);
+
                 private:
                     /**
                      * Synchronously send request message and receive response.
@@ -307,10 +318,11 @@ namespace ignite
                      *
                      * @param req Request message.
                      * @param rsp Response message.
+                     * @return Channel that was used for request.
                      * @throw IgniteError on error.
                      */
                     template<typename ReqT, typename RspT>
-                    void SyncMessage(const ReqT& req, RspT& rsp);
+                    SP_DataChannel SyncMessage(const ReqT& req, RspT& rsp);
 
                     /** Data router. */
                     SP_DataRouter router;
diff --git a/modules/platforms/cpp/thin-client/src/impl/cache/cache_client_proxy.cpp b/modules/platforms/cpp/thin-client/src/impl/cache/cache_client_proxy.cpp
index 840665c..a5795ee 100644
--- a/modules/platforms/cpp/thin-client/src/impl/cache/cache_client_proxy.cpp
+++ b/modules/platforms/cpp/thin-client/src/impl/cache/cache_client_proxy.cpp
@@ -149,6 +149,14 @@ namespace ignite
                 {
                     GetCacheImpl(impl).GetAndPutIfAbsent(key, valIn, valOut);
                 }
+
+                ignite::thin::cache::query::QueryFieldsCursor CacheClientProxy::Query(
+                        const ignite::thin::cache::query::SqlFieldsQuery &qry)
+                {
+                    query::SP_QueryFieldsCursorImpl cursorImpl = GetCacheImpl(impl).Query(qry);
+
+                    return ignite::thin::cache::query::QueryFieldsCursor(cursorImpl);
+                }
             }
         }
     }
diff --git a/modules/platforms/cpp/thin-client/src/impl/cache/query/cursor_page.h b/modules/platforms/cpp/thin-client/src/impl/cache/query/cursor_page.h
new file mode 100644
index 0000000..34be0a7
--- /dev/null
+++ b/modules/platforms/cpp/thin-client/src/impl/cache/query/cursor_page.h
@@ -0,0 +1,127 @@
+/*
+ * 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.
+ */
+
+#ifndef _IGNITE_IMPL_THIN_CACHE_QUERY_CURSOR_PAGE
+#define _IGNITE_IMPL_THIN_CACHE_QUERY_CURSOR_PAGE
+
+#include <ignite/binary/binary_raw_writer.h>
+
+namespace ignite
+{
+    namespace impl
+    {
+        namespace thin
+        {
+            namespace cache
+            {
+                namespace query
+                {
+                    /**
+                     * Cursor page.
+                     */
+                    class CursorPage
+                    {
+                    public:
+                        /**
+                         * Constructor.
+                         */
+                        CursorPage() :
+                            mem()
+                        {
+                            // No-op.
+                        }
+
+                        /**
+                         * Destructor.
+                         */
+                        virtual ~CursorPage()
+                        {
+                            // No-op.
+                        }
+
+                        /**
+                         * Read page using reader.
+                         *
+                         * @param reader Reader to use.
+                         */
+                        void Read(binary::BinaryReaderImpl &reader)
+                        {
+                            interop::InteropInputStream* stream = reader.GetStream();
+
+                            rowNum = reader.ReadInt32();
+
+                            startPos = stream->Position();
+
+                            interop::InteropUnpooledMemory* streamMem =
+                                static_cast<interop::InteropUnpooledMemory*>(stream->GetMemory());
+
+                            bool gotOwnership = streamMem->TryGetOwnership(mem);
+
+                            (void) gotOwnership;
+                            assert(gotOwnership);
+                        }
+
+                        /**
+                         * Get row number.
+                         *
+                         * @return Row number.
+                         */
+                        int32_t GetRowNum() const
+                        {
+                            return rowNum;
+                        }
+
+                        /**
+                         * Get starting position in memory.
+                         *
+                         * @return Start position.
+                         */
+                        int32_t GetStartPos() const
+                        {
+                            return startPos;
+                        }
+
+                        /**
+                         * Get memory.
+                         *
+                         * @return Page memory.
+                         */
+                        interop::InteropUnpooledMemory* GetMemory()
+                        {
+                            return &mem;
+                        }
+
+                    private:
+                        /** Row Number. */
+                        int32_t rowNum;
+
+                        /** Start position. */
+                        int32_t startPos;
+
+                        /** Page memory. */
+                        interop::InteropUnpooledMemory mem;
+                    };
+
+                    /** Cursor page shared pointer. */
+                    typedef common::concurrent::SharedPointer<CursorPage> SP_CursorPage;
+                }
+            }
+        }
+    }
+}
+
+#endif // _IGNITE_IMPL_THIN_CACHE_QUERY_CURSOR_PAGE
diff --git a/modules/platforms/cpp/thin-client/src/impl/cache/query/query_fields_cursor_impl.h b/modules/platforms/cpp/thin-client/src/impl/cache/query/query_fields_cursor_impl.h
new file mode 100644
index 0000000..6e03c71
--- /dev/null
+++ b/modules/platforms/cpp/thin-client/src/impl/cache/query/query_fields_cursor_impl.h
@@ -0,0 +1,229 @@
+/*
+ * 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.
+ */
+
+#ifndef _IGNITE_IMPL_THIN_CACHE_QUERY_QUERY_FIELDS_CURSOR_IMPL
+#define _IGNITE_IMPL_THIN_CACHE_QUERY_QUERY_FIELDS_CURSOR_IMPL
+
+#include <ignite/common/concurrent.h>
+
+#include <ignite/thin/cache/query/query_fields_row.h>
+
+#include "impl/cache/query/cursor_page.h"
+#include "impl/cache/query/query_fields_row_impl.h"
+#include "impl/data_router.h"
+#include "impl/message.h"
+
+namespace ignite
+{
+    namespace impl
+    {
+        namespace thin
+        {
+            namespace cache
+            {
+                namespace query
+                {
+                    /**
+                     * Query Fields Cursor Implementation.
+                     */
+                    class QueryFieldsCursorImpl
+                    {
+                    public:
+                        /**
+                         * Constructor.
+                         *
+                         * @param id Cursor ID.
+                         * @param columns Column names.
+                         * @param cursorPage Cursor page.
+                         * @param channel Data channel. Used to request new page.
+                         * @param timeout Timeout.
+                         */
+                        QueryFieldsCursorImpl(
+                                int64_t id,
+                                const std::vector<std::string>& columns,
+                                const SP_CursorPage &cursorPage,
+                                const SP_DataChannel& channel,
+                                int32_t timeout) :
+                            id(id),
+                            columns(columns),
+                            page(cursorPage),
+                            channel(channel),
+                            timeout(timeout),
+                            currentRow(0),
+                            stream(page.Get()->GetMemory()),
+                            reader(&stream),
+                            endReached(false)
+                        {
+                            stream.Position(page.Get()->GetStartPos());
+
+                            CheckEnd();
+                        }
+
+                        /**
+                         * Destructor.
+                         */
+                        virtual ~QueryFieldsCursorImpl()
+                        {
+                            // No-op.
+                        }
+
+                        /**
+                         * Check whether next entry exists.
+                         *
+                         * @return @c true if next entry exists.
+                         *
+                         * @throw IgniteError class instance in case of failure.
+                         */
+                        bool HasNext()
+                        {
+                            return !endReached;
+                        }
+
+                        /**
+                         * Get next entry.
+                         *
+                         * This method should only be used on the valid instance.
+                         *
+                         * @return Next entry.
+                         *
+                         * @throw IgniteError class instance in case of failure.
+                         */
+                        ignite::thin::cache::query::QueryFieldsRow GetNext()
+                        {
+                            if (!HasNext())
+                                throw IgniteError(IgniteError::IGNITE_ERR_GENERIC, "The cursor is empty");
+
+                            if (IsUpdateNeeded())
+                                Update();
+
+                            SP_QueryFieldsRowImpl rowImpl(
+                                new QueryFieldsRowImpl(
+                                        static_cast<int32_t>(columns.size()),
+                                        page,
+                                        stream.Position()));
+
+                            SkipRow();
+
+                            return ignite::thin::cache::query::QueryFieldsRow(rowImpl);
+                        }
+
+                        /**
+                         * Get column names.
+                         *
+                         * @return Column names.
+                         */
+                        const std::vector<std::string>& GetColumns() const
+                        {
+                            return columns;
+                        }
+
+                    private:
+                        /**
+                         * Check whether next page should be retrieved from the server.
+                         *
+                         * @return @c true if next page should be fetched.
+                         */
+                        bool IsUpdateNeeded()
+                        {
+                            return !page.IsValid() && !endReached;
+                        }
+
+                        /**
+                         * Fetch next cursor page.
+                         */
+                        void Update()
+                        {
+                            SqlFieldsCursorGetPageRequest req(id);
+                            SqlFieldsCursorGetPageResponse rsp;
+
+                            DataChannel* channel0 = channel.Get();
+
+                            if (!channel0)
+                                throw IgniteError(IgniteError::IGNITE_ERR_GENERIC, "Connection is not established");
+
+                            channel0->SyncMessage(req, rsp, timeout);
+
+                            page = rsp.GetCursorPage();
+                            currentRow = 0;
+
+                            stream = interop::InteropInputStream(page.Get()->GetMemory());
+                            stream.Position(page.Get()->GetStartPos());
+                        }
+
+                        /**
+                         * Skip position to the next row.
+                         */
+                        void SkipRow()
+                        {
+                            for (size_t i = 0; i < columns.size(); ++i)
+                                reader.Skip();
+
+                            ++currentRow;
+
+                            CheckEnd();
+                        }
+
+                        /**
+                         * Check whether end is reached.
+                         */
+                        void CheckEnd()
+                        {
+                            if (currentRow == page.Get()->GetRowNum())
+                            {
+                                bool hasNextPage = reader.ReadBool();
+                                endReached = !hasNextPage;
+
+                                page = SP_CursorPage();
+                            }
+                        }
+
+                        /** Cursor ID. */
+                        int64_t id;
+
+                        /** Column names. */
+                        std::vector<std::string> columns;
+
+                        /** Cursor page. */
+                        SP_CursorPage page;
+
+                        /** Data channel. */
+                        SP_DataChannel channel;
+
+                        /** Timeout in milliseconds. */
+                        int32_t timeout;
+
+                        /** Current row in page. */
+                        int32_t currentRow;
+
+                        /** Stream. */
+                        interop::InteropInputStream stream;
+
+                        /** Reader. */
+                        binary::BinaryReaderImpl reader;
+
+                        /** End reached. */
+                        bool endReached;
+                    };
+
+                    typedef common::concurrent::SharedPointer<QueryFieldsCursorImpl> SP_QueryFieldsCursorImpl;
+                }
+            }
+        }
+    }
+}
+
+#endif // _IGNITE_IMPL_THIN_CACHE_QUERY_QUERY_FIELDS_CURSOR_IMPL
diff --git a/modules/platforms/cpp/thin-client/src/impl/cache/query/query_fields_row_impl.h b/modules/platforms/cpp/thin-client/src/impl/cache/query/query_fields_row_impl.h
new file mode 100644
index 0000000..3ff2d65
--- /dev/null
+++ b/modules/platforms/cpp/thin-client/src/impl/cache/query/query_fields_row_impl.h
@@ -0,0 +1,129 @@
+/*
+ * 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.
+ */
+
+/**
+ * @file
+ * Declares ignite::impl::thin::cache::query::QueryFieldsRow class.
+ */
+
+#ifndef _IGNITE_IMPL_THIN_CACHE_QUERY_QUERY_FIELDS_ROW_IMPL
+#define _IGNITE_IMPL_THIN_CACHE_QUERY_QUERY_FIELDS_ROW_IMPL
+
+#include <ignite/common/concurrent.h>
+#include <ignite/ignite_error.h>
+
+#include <ignite/impl/binary/binary_reader_impl.h>
+
+#include <ignite/impl/thin/readable.h>
+
+#include "impl/cache/query/cursor_page.h"
+
+namespace ignite
+{
+    namespace impl
+    {
+        namespace thin
+        {
+            namespace cache
+            {
+                namespace query
+                {
+                    /**
+                     * Query fields row implementation.
+                     */
+                    class QueryFieldsRowImpl
+                    {
+                    public:
+                        /**
+                         * Constructor.
+                         *
+                         * @param size Row size in elements.
+                         * @param cursorPage Cursor page.
+                         * @param posInMem Row starting position in memory.
+                         */
+                        QueryFieldsRowImpl(int32_t size, const SP_CursorPage& cursorPage, int32_t posInMem) :
+                            size(size),
+                            pos(0),
+                            page(cursorPage),
+                            stream(page.Get()->GetMemory()),
+                            reader(&stream)
+                        {
+                            stream.Position(posInMem);
+                        }
+
+                        /**
+                         * Check whether next entry exists.
+                         *
+                         * @return True if next entry exists.
+                         */
+                        bool HasNext() const
+                        {
+                            return pos < size;
+                        }
+
+                        /**
+                         * Get next entry.
+                         *
+                         * @param readable Value to read.
+                         *
+                         * @throw IgniteError class instance in case of failure.
+                         */
+                        void GetNext(Readable& readable)
+                        {
+                            if (!HasNext())
+                                throw IgniteError(IgniteError::IGNITE_ERR_GENERIC, "The cursor is empty");
+                            
+                            readable.Read(reader);
+                            ++pos;
+                        }
+
+                        /**
+                         * Get size of the row in elements.
+                         *
+                         * @return
+                         */
+                        int32_t GetSize() const
+                        {
+                            return size;
+                        }
+
+                    private:
+                        /** Number of elements in row. */
+                        int32_t size;
+
+                        /** Current position in row. */
+                        int32_t pos;
+
+                        /** Cursor page. */
+                        SP_CursorPage page;
+
+                        /** Stream. */
+                        interop::InteropInputStream stream;
+
+                        /** Reader. */
+                        binary::BinaryReaderImpl reader;
+                    };
+
+                    /** Query field row implementation shared pointer. */
+                    typedef common::concurrent::SharedPointer<QueryFieldsRowImpl> SP_QueryFieldsRowImpl;
+                }
+            }
+        }
+    }    
+}
+
+#endif //_IGNITE_IMPL_THIN_CACHE_QUERY_QUERY_FIELDS_ROW_IMPL
diff --git a/modules/platforms/cpp/thin-client/src/impl/data_router.h b/modules/platforms/cpp/thin-client/src/impl/data_router.h
index 2b27ae8..baa74ac 100644
--- a/modules/platforms/cpp/thin-client/src/impl/data_router.h
+++ b/modules/platforms/cpp/thin-client/src/impl/data_router.h
@@ -116,10 +116,11 @@ namespace ignite
                  *
                  * @param req Request message.
                  * @param rsp Response message.
+                 * @return Channel that was used for request.
                  * @throw IgniteError on error.
                  */
                 template<typename ReqT, typename RspT>
-                void SyncMessage(const ReqT& req, RspT& rsp)
+                SP_DataChannel SyncMessage(const ReqT& req, RspT& rsp)
                 {
                     SP_DataChannel channel = GetRandomChannel();
 
@@ -128,6 +129,8 @@ namespace ignite
                     SyncMessagePreferredChannelNoMetaUpdate(req, rsp, channel);
 
                     ProcessMeta(metaVer);
+
+                    return channel;
                 }
 
                 /**
@@ -136,10 +139,11 @@ namespace ignite
                  * @param req Request message.
                  * @param rsp Response message.
                  * @param hint Preferred server node to use.
+                 * @return Channel that was used for request.
                  * @throw IgniteError on error.
                  */
                 template<typename ReqT, typename RspT>
-                void SyncMessage(const ReqT& req, RspT& rsp, const Guid& hint)
+                SP_DataChannel SyncMessage(const ReqT& req, RspT& rsp, const Guid& hint)
                 {
                     SP_DataChannel channel = GetBestChannel(hint);
 
@@ -148,6 +152,8 @@ namespace ignite
                     SyncMessagePreferredChannelNoMetaUpdate(req, rsp, channel);
 
                     ProcessMeta(metaVer);
+
+                    return channel;
                 }
 
                 /**
@@ -157,14 +163,17 @@ namespace ignite
                  *
                  * @param req Request message.
                  * @param rsp Response message.
+                 * @return Channel that was used for request.
                  * @throw IgniteError on error.
                  */
                 template<typename ReqT, typename RspT>
-                void SyncMessageNoMetaUpdate(const ReqT& req, RspT& rsp)
+                SP_DataChannel SyncMessageNoMetaUpdate(const ReqT& req, RspT& rsp)
                 {
                     SP_DataChannel channel = GetRandomChannel();
 
                     SyncMessagePreferredChannelNoMetaUpdate(req, rsp, channel);
+
+                    return channel;
                 }
 
                 /**
diff --git a/modules/platforms/cpp/thin-client/src/impl/message.cpp b/modules/platforms/cpp/thin-client/src/impl/message.cpp
index d6080e7..43397e0 100644
--- a/modules/platforms/cpp/thin-client/src/impl/message.cpp
+++ b/modules/platforms/cpp/thin-client/src/impl/message.cpp
@@ -343,6 +343,75 @@ namespace ignite
             {
                 value = reader.ReadInt32();
             }
+
+            SqlFieldsQueryRequest::SqlFieldsQueryRequest(
+                int32_t cacheId,
+                const ignite::thin::cache::query::SqlFieldsQuery &qry
+                ) :
+                CacheRequest<RequestType::QUERY_SQL_FIELDS>(cacheId, false),
+                qry(qry)
+            {
+                // No-op.
+            }
+
+            void SqlFieldsQueryRequest::Write(binary::BinaryWriterImpl& writer, const ProtocolVersion& ver) const
+            {
+                CacheRequest<RequestType::QUERY_SQL_FIELDS>::Write(writer, ver);
+
+                if (qry.schema.empty())
+                    writer.WriteNull();
+                else
+                    writer.WriteString(qry.schema);
+
+                writer.WriteInt32(qry.pageSize);
+                writer.WriteInt32(qry.maxRows);
+                writer.WriteString(qry.sql);
+                writer.WriteInt32(static_cast<int32_t>(qry.args.size()));
+
+                std::vector<impl::thin::CopyableWritable*>::const_iterator it;
+
+                for (it = qry.args.begin(); it != qry.args.end(); ++it)
+                    (*it)->Write(writer);
+
+                writer.WriteInt8(0); // Statement type - Any
+
+                writer.WriteBool(qry.distributedJoins);
+                writer.WriteBool(qry.loc);
+                writer.WriteBool(false); // Replicated only
+                writer.WriteBool(qry.enforceJoinOrder);
+                writer.WriteBool(qry.collocated);
+                writer.WriteBool(qry.lazy);
+                writer.WriteInt64(qry.timeout);
+                writer.WriteBool(true); // Include field names
+            }
+
+            void SqlFieldsQueryResponse::ReadOnSuccess(binary::BinaryReaderImpl& reader, const ProtocolVersion&)
+            {
+                ignite::binary::BinaryRawReader rawReader(&reader);
+
+                cursorId = rawReader.ReadInt64();
+
+                int32_t columnsCnt = rawReader.ReadInt32();
+
+                columns.reserve(static_cast<size_t>(columnsCnt));
+
+                for (int32_t i = 0; i < columnsCnt; ++i)
+                {
+                    columns.push_back(rawReader.ReadString());
+                }
+
+                cursorPage.Get()->Read(reader);
+            }
+
+            void SqlFieldsCursorGetPageRequest::Write(binary::BinaryWriterImpl& writer, const ProtocolVersion&) const
+            {
+                writer.WriteInt64(cursorId);
+            }
+
+            void SqlFieldsCursorGetPageResponse::ReadOnSuccess(binary::BinaryReaderImpl&reader, const ProtocolVersion&)
+            {
+                cursorPage.Get()->Read(reader);
+            }
         }
     }
 }
diff --git a/modules/platforms/cpp/thin-client/src/impl/message.h b/modules/platforms/cpp/thin-client/src/impl/message.h
index 1b5ec68..80e5437 100644
--- a/modules/platforms/cpp/thin-client/src/impl/message.h
+++ b/modules/platforms/cpp/thin-client/src/impl/message.h
@@ -22,17 +22,19 @@
 #include <string>
 #include <vector>
 
+#include <ignite/thin/cache/query/query_sql_fields.h>
+#include <ignite/thin/transactions/transaction_consts.h>
+
 #include <ignite/impl/binary/binary_writer_impl.h>
 #include <ignite/impl/binary/binary_reader_impl.h>
 
 #include <ignite/impl/thin/writable.h>
 #include <ignite/impl/thin/readable.h>
 
-#include <ignite/thin/transactions/transaction_consts.h>
-
-#include "impl/protocol_version.h"
 #include "impl/affinity/affinity_topology_version.h"
 #include "impl/affinity/partition_awareness_group.h"
+#include "impl/cache/query/cursor_page.h"
+#include "impl/protocol_version.h"
 
 namespace ignite
 {
@@ -147,12 +149,15 @@ namespace ignite
                     /** Cache destroy. */
                     CACHE_DESTROY = 1056,
 
-                    /** Cache nodes and partitions request. */
-                    CACHE_NODE_PARTITIONS = 1100,
-
                     /** Cache partitions request. */
                     CACHE_PARTITIONS = 1101,
 
+                    /** SQL fields query request. */
+                    QUERY_SQL_FIELDS = 2004,
+
+                    /** SQL fields query get next cursor page request. */
+                    QUERY_SQL_FIELDS_CURSOR_GET_PAGE = 2005,
+
                     /** Get binary type info. */
                     GET_BINARY_TYPE = 3002,
 
@@ -796,6 +801,77 @@ namespace ignite
             };
 
             /**
+             * Cache SQL fields query request.
+             */
+            class SqlFieldsQueryRequest : public CacheRequest<RequestType::QUERY_SQL_FIELDS>
+            {
+            public:
+                /**
+                 * Constructor.
+                 *
+                 * @param cacheId Cache ID.
+                 * @param qry SQL query.
+                 */
+                explicit SqlFieldsQueryRequest(int32_t cacheId, const ignite::thin::cache::query::SqlFieldsQuery &qry);
+
+                /**
+                 * Destructor.
+                 */
+                virtual ~SqlFieldsQueryRequest()
+                {
+                    // No-op.
+                }
+
+                /**
+                 * Write request using provided writer.
+                 * @param writer Writer.
+                 * @param ver Version.
+                 */
+                virtual void Write(binary::BinaryWriterImpl& writer, const ProtocolVersion& ver) const;
+
+            private:
+                /** Query. */
+                const ignite::thin::cache::query::SqlFieldsQuery &qry;
+            };
+
+            /**
+             * Cache SQL fields cursor get page request.
+             */
+            class SqlFieldsCursorGetPageRequest : public Request<RequestType::QUERY_SQL_FIELDS_CURSOR_GET_PAGE>
+            {
+            public:
+                /**
+                 * Constructor.
+                 *
+                 * @param cursorId Cursor ID.
+                 */
+                explicit SqlFieldsCursorGetPageRequest(int64_t cursorId) :
+                    cursorId(cursorId)
+                {
+                    // No-op.
+                }
+
+                /**
+                 * Destructor.
+                 */
+                virtual ~SqlFieldsCursorGetPageRequest()
+                {
+                    // No-op.
+                }
+
+                /**
+                 * Write request using provided writer.
+                 * @param writer Writer.
+                 * @param ver Version.
+                 */
+                virtual void Write(binary::BinaryWriterImpl& writer, const ProtocolVersion& ver) const;
+
+            private:
+                /** Cursor ID. */
+                const int64_t cursorId;
+            };
+
+            /**
              * General response.
              */
             class Response
@@ -1207,6 +1283,121 @@ namespace ignite
                 /** Value. */
                 int32_t value;
             };
+
+            /**
+             * Cache SQL fields query response.
+             */
+            class SqlFieldsQueryResponse : public Response
+            {
+            public:
+                /**
+                 * Constructor.
+                 */
+                SqlFieldsQueryResponse() :
+                    cursorId(0),
+                    cursorPage(new cache::query::CursorPage())
+                {
+                    // No-op.
+                }
+
+                /**
+                 * Destructor.
+                 */
+                virtual ~SqlFieldsQueryResponse()
+                {
+                    // No-op.
+                }
+
+                /**
+                 * Get cursor ID.
+                 *
+                 * @return Cursor ID.
+                 */
+                int64_t GetCursorId() const
+                {
+                    return cursorId;
+                }
+
+                /**
+                 * Get columns.
+                 *
+                 * @return Column names.
+                 */
+                const std::vector<std::string>& GetColumns() const
+                {
+                    return columns;
+                }
+
+                /**
+                 * Get cursor page.
+                 * @return Cursor page.
+                 */
+                cache::query::SP_CursorPage GetCursorPage() const
+                {
+                    return cursorPage;
+                }
+
+                /**
+                 * Read data if response status is ResponseStatus::SUCCESS.
+                 *
+                 * @param reader Reader.
+                 */
+                virtual void ReadOnSuccess(binary::BinaryReaderImpl& reader, const ProtocolVersion&);
+
+            private:
+                /** Cursor ID. */
+                int64_t cursorId;
+
+                /** Column names. */
+                std::vector<std::string> columns;
+
+                /** Cursor Page. */
+                cache::query::SP_CursorPage cursorPage;
+            };
+
+            /**
+             * Cache SQL fields cursor get page response.
+             */
+            class SqlFieldsCursorGetPageResponse : public Response
+            {
+            public:
+                /**
+                 * Constructor.
+                 */
+                SqlFieldsCursorGetPageResponse() :
+                    cursorPage(new cache::query::CursorPage())
+                {
+                    // No-op.
+                }
+
+                /**
+                 * Destructor.
+                 */
+                virtual ~SqlFieldsCursorGetPageResponse()
+                {
+                    // No-op.
+                }
+
+                /**
+                 * Get cursor page.
+                 * @return Cursor page.
+                 */
+                cache::query::SP_CursorPage GetCursorPage() const
+                {
+                    return cursorPage;
+                }
+
+                /**
+                 * Read data if response status is ResponseStatus::SUCCESS.
+                 *
+                 * @param reader Reader.
+                 */
+                virtual void ReadOnSuccess(binary::BinaryReaderImpl& reader, const ProtocolVersion&);
+
+            private:
+                /** Cursor Page. */
+                cache::query::SP_CursorPage cursorPage;
+            };
         }
     }
 }