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 2023/03/21 12:43:51 UTC

[ignite-3] branch main updated: IGNITE-18291 Enhance C++ Binary Tuple code (#1594)

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

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


The following commit(s) were added to refs/heads/main by this push:
     new 5eebe133b3 IGNITE-18291 Enhance C++ Binary Tuple code (#1594)
5eebe133b3 is described below

commit 5eebe133b37046c75dd72383356e521930b0b3b9
Author: Aleksey Demakov <ad...@gmail.com>
AuthorDate: Tue Mar 21 15:43:44 2023 +0300

    IGNITE-18291 Enhance C++ Binary Tuple code (#1594)
---
 modules/platforms/cpp/CMakeLists.txt               |   2 +-
 .../cpp/ignite/client/detail/client_data_type.h    | 190 +++++++++++++++++++
 .../ignite/client/detail/compute/compute_impl.cpp  |   9 +-
 .../cpp/ignite/client/detail/node_connection.cpp   |   2 +-
 .../cpp/ignite/client/detail/sql/result_set_impl.h |   4 +-
 .../cpp/ignite/client/detail/sql/sql_impl.cpp      |   5 +-
 .../cpp/ignite/client/detail/sql/sql_impl.h        |   1 -
 .../cpp/ignite/client/detail/table/schema.h        |  19 +-
 .../cpp/ignite/client/detail/table/table_impl.cpp  |   5 +-
 .../cpp/ignite/client/detail/table/tables_impl.cpp |   2 +-
 .../client/detail/transaction/transactions_impl.h  |   1 -
 .../platforms/cpp/ignite/client/detail/utils.cpp   | 187 +++++++++----------
 modules/platforms/cpp/ignite/client/detail/utils.h |  15 +-
 .../platforms/cpp/ignite/client/ignite_client.h    |   1 -
 modules/platforms/cpp/ignite/client/primitive.h    |  53 ++++--
 .../platforms/cpp/ignite/client/primitive_test.cpp |  44 ++---
 .../cpp/ignite/client/sql/column_metadata.h        |   8 +-
 .../cpp/ignite/client/table/key_value_view.h       |   2 -
 .../cpp/ignite/client/table/record_view.h          |   2 -
 modules/platforms/cpp/ignite/client/table/table.h  |   1 -
 modules/platforms/cpp/ignite/common/CMakeLists.txt |   3 +
 modules/platforms/cpp/ignite/common/bytes_view.h   |  26 ++-
 modules/platforms/cpp/ignite/common/ignite_date.h  |  14 +-
 .../common/{ignite_date.h => ignite_duration.h}    |  64 +++----
 modules/platforms/cpp/ignite/common/ignite_error.h |  14 +-
 .../common/{ignite_date.h => ignite_period.h}      |  64 +++----
 modules/platforms/cpp/ignite/common/ignite_time.h  |  14 +-
 .../sql/sql_column_type.h => common/ignite_type.h} |  18 +-
 modules/platforms/cpp/ignite/common/uuid.h         |  13 +-
 .../platforms/cpp/ignite/schema/ignite_type.cpp    |  80 --------
 modules/platforms/cpp/ignite/schema/ignite_type.h  |  79 --------
 .../cpp/ignite/{schema => tuple}/CMakeLists.txt    |   2 -
 .../{schema => tuple}/binary_tuple_builder.cpp     | 121 +++++++++---
 .../{schema => tuple}/binary_tuple_builder.h       | 206 ++++++++++++++++-----
 .../ignite/{schema => tuple}/binary_tuple_header.h |   6 +-
 .../{schema => tuple}/binary_tuple_parser.cpp      |  76 +++++++-
 .../ignite/{schema => tuple}/binary_tuple_parser.h |  62 +++++--
 .../ignite/{schema => tuple}/binary_tuple_schema.h |  14 +-
 .../cpp/ignite/{schema => tuple}/column_info.h     |  17 +-
 .../cpp/ignite/{schema => tuple}/tuple_assembler.h |   0
 .../cpp/ignite/{schema => tuple}/tuple_test.cpp    |  36 ++--
 .../platforms/cpp/ignite/{schema => tuple}/types.h |  15 +-
 .../platforms/cpp/tests/client-test/sql_test.cpp   |   8 +-
 .../cpp/tests/client-test/transactions_test.cpp    |   2 +-
 44 files changed, 912 insertions(+), 595 deletions(-)

diff --git a/modules/platforms/cpp/CMakeLists.txt b/modules/platforms/cpp/CMakeLists.txt
index a12077763f..755bf41cb8 100644
--- a/modules/platforms/cpp/CMakeLists.txt
+++ b/modules/platforms/cpp/CMakeLists.txt
@@ -149,7 +149,7 @@ include(ignite_test)
 
 # Add common libraries along with their unit tests if any.
 add_subdirectory(ignite/common)
-add_subdirectory(ignite/schema)
+add_subdirectory(ignite/tuple)
 
 # Add client libraries.
 if (${ENABLE_CLIENT})
diff --git a/modules/platforms/cpp/ignite/client/detail/client_data_type.h b/modules/platforms/cpp/ignite/client/detail/client_data_type.h
new file mode 100644
index 0000000000..fb0a139e34
--- /dev/null
+++ b/modules/platforms/cpp/ignite/client/detail/client_data_type.h
@@ -0,0 +1,190 @@
+/*
+ * 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.
+ */
+
+#pragma once
+
+#include <ignite/common/ignite_error.h>
+#include <ignite/common/ignite_type.h>
+
+namespace ignite::detail {
+
+/**
+ * Column data type encoding for the client protocol.
+ */
+struct client_data_type {
+    /** Byte. */
+    static constexpr std::int32_t INT8 = 1;
+
+    /** Short. */
+    static constexpr std::int32_t INT16 = 2;
+
+    /** Int. */
+    static constexpr std::int32_t INT32 = 3;
+
+    /** Long. */
+    static constexpr std::int32_t INT64 = 4;
+
+    /** Float. */
+    static constexpr std::int32_t FLOAT = 5;
+
+    /** Double. */
+    static constexpr std::int32_t DOUBLE = 6;
+
+    /** Decimal. */
+    static constexpr std::int32_t DECIMAL = 7;
+
+    /** UUID. */
+    static constexpr std::int32_t UUID = 8;
+
+    /** String. */
+    static constexpr std::int32_t STRING = 9;
+
+    /** Byte array. */
+    static constexpr std::int32_t BYTES = 10;
+
+    /** BitMask. */
+    static constexpr std::int32_t BITMASK = 11;
+
+    /** Date. */
+    static constexpr std::int32_t DATE = 12;
+
+    /** Time. */
+    static constexpr std::int32_t TIME = 13;
+
+    /** DateTime. */
+    static constexpr std::int32_t DATETIME = 14;
+
+    /** Timestamp. */
+    static constexpr std::int32_t TIMESTAMP = 15;
+
+    /** Number. */
+    static constexpr std::int32_t NUMBER = 16;
+
+    /** Boolean. */
+    static constexpr std::int32_t BOOLEAN = 17;
+
+    /** Duration. */
+    static constexpr std::int32_t DURATION = 18;
+
+    /** Period. */
+    static constexpr std::int32_t PERIOD = 19;
+
+    /**
+     * Get the ignite type matching a client data type value.
+     *
+     * @param t Client data type.
+     * @return Matching ignite type.
+     */
+    static ignite_type to_ignite_type(std::int32_t t) {
+        switch (t) {
+            case INT8:
+                return ignite_type::INT8;
+            case INT16:
+                return ignite_type::INT16;
+            case INT32:
+                return ignite_type::INT32;
+            case INT64:
+                return ignite_type::INT64;
+            case FLOAT:
+                return ignite_type::FLOAT;
+            case DOUBLE:
+                return ignite_type::DOUBLE;
+            case DECIMAL:
+                return ignite_type::DECIMAL;
+            case UUID:
+                return ignite_type::UUID;
+            case STRING:
+                return ignite_type::STRING;
+            case BYTES:
+                return ignite_type::BYTE_ARRAY;
+            case BITMASK:
+                return ignite_type::BITMASK;
+            case DATE:
+                return ignite_type::DATE;
+            case TIME:
+                return ignite_type::TIME;
+            case DATETIME:
+                return ignite_type::DATETIME;
+            case TIMESTAMP:
+                return ignite_type::TIMESTAMP;
+            case NUMBER:
+                return ignite_type::NUMBER;
+            case BOOLEAN:
+                return ignite_type::BOOLEAN;
+            case DURATION:
+                return ignite_type::DURATION;
+            case PERIOD:
+                return ignite_type::PERIOD;
+            default:
+                throw ignite_error("Value is out of range for Ignite type: " + std::to_string(t));
+        }
+    }
+
+    /**
+     * Get the client data type value matching an ignite type.
+     *
+     * @param t Ignite type.
+     * @return Matching client data type.
+     */
+    static std::int32_t from_ignite_type(ignite_type t) {
+        switch (t) {
+            case ignite_type::BOOLEAN:
+                return BOOLEAN;
+            case ignite_type::INT8:
+                return INT8;
+            case ignite_type::INT16:
+                return INT16;
+            case ignite_type::INT32:
+                return INT32;
+            case ignite_type::INT64:
+                return INT64;
+            case ignite_type::FLOAT:
+                return FLOAT;
+            case ignite_type::DOUBLE:
+                return DOUBLE;
+            case ignite_type::DECIMAL:
+                return DECIMAL;
+            case ignite_type::DATE:
+                return DATE;
+            case ignite_type::TIME:
+                return TIME;
+            case ignite_type::DATETIME:
+                return DATETIME;
+            case ignite_type::TIMESTAMP:
+                return TIMESTAMP;
+            case ignite_type::UUID:
+                return UUID;
+            case ignite_type::BITMASK:
+                return BITMASK;
+            case ignite_type::STRING:
+                return STRING;
+            case ignite_type::BYTE_ARRAY:
+                return BYTES;
+            case ignite_type::PERIOD:
+                return PERIOD;
+            case ignite_type::DURATION:
+                return DURATION;
+            case ignite_type::NUMBER:
+                return NUMBER;
+            default:
+                throw ignite_error("Value is out of range for Ignite type: "
+                    + std::to_string(static_cast<std::underlying_type_t<ignite_type>>(t)));
+        }
+    }
+};
+
+} // namespace ignite::detail
diff --git a/modules/platforms/cpp/ignite/client/detail/compute/compute_impl.cpp b/modules/platforms/cpp/ignite/client/detail/compute/compute_impl.cpp
index d3edf24873..e2c1b64913 100644
--- a/modules/platforms/cpp/ignite/client/detail/compute/compute_impl.cpp
+++ b/modules/platforms/cpp/ignite/client/detail/compute/compute_impl.cpp
@@ -15,9 +15,12 @@
  * limitations under the License.
  */
 
-#include "ignite/client/detail/compute/compute_impl.h"
+#include "compute_impl.h"
+
+#include "ignite/client/detail/client_data_type.h"
 #include "ignite/client/detail/utils.h"
-#include "ignite/schema/binary_tuple_builder.h"
+
+#include "ignite/tuple/binary_tuple_builder.h"
 
 namespace ignite::detail {
 
@@ -58,7 +61,7 @@ std::optional<primitive> read_primitive_from_binary_tuple(protocol::reader &read
     auto tuple_data = reader.read_binary();
     binary_tuple_parser parser(3, tuple_data);
 
-    auto typ = ignite_type(binary_tuple_parser::get_int32(parser.get_next().value()));
+    auto typ = client_data_type::to_ignite_type(binary_tuple_parser::get_int32(parser.get_next().value()));
     auto scale = binary_tuple_parser::get_int32(parser.get_next().value());
     return read_next_column(parser, typ, scale);
 }
diff --git a/modules/platforms/cpp/ignite/client/detail/node_connection.cpp b/modules/platforms/cpp/ignite/client/detail/node_connection.cpp
index a17c2b944f..0b4d8ea745 100644
--- a/modules/platforms/cpp/ignite/client/detail/node_connection.cpp
+++ b/modules/platforms/cpp/ignite/client/detail/node_connection.cpp
@@ -47,7 +47,7 @@ bool node_connection::handshake() {
     std::vector<std::byte> message;
     {
         protocol::buffer_adapter buffer(message);
-        buffer.write_raw(bytes_view(protocol::MAGIC_BYTES.data(), protocol::MAGIC_BYTES.size()));
+        buffer.write_raw(bytes_view(protocol::MAGIC_BYTES));
 
         protocol::write_message_to_buffer(buffer, [&context = m_protocol_context](protocol::writer &writer) {
             auto ver = context.get_version();
diff --git a/modules/platforms/cpp/ignite/client/detail/sql/result_set_impl.h b/modules/platforms/cpp/ignite/client/detail/sql/result_set_impl.h
index a0dfe0a630..dcbadde46a 100644
--- a/modules/platforms/cpp/ignite/client/detail/sql/result_set_impl.h
+++ b/modules/platforms/cpp/ignite/client/detail/sql/result_set_impl.h
@@ -21,7 +21,7 @@
 #include "ignite/client/detail/utils.h"
 #include "ignite/client/sql/result_set_metadata.h"
 #include "ignite/client/table/ignite_tuple.h"
-#include "ignite/schema/binary_tuple_parser.h"
+#include "ignite/tuple/binary_tuple_parser.h"
 
 #include <cstdint>
 
@@ -218,7 +218,7 @@ private:
 
             auto name = protocol::unpack_object<std::string>(arr.ptr[0]);
             auto nullable = protocol::unpack_object<bool>(arr.ptr[1]);
-            auto typ = column_type(protocol::unpack_object<std::int32_t>(arr.ptr[2]));
+            auto typ = ignite_type(protocol::unpack_object<std::int32_t>(arr.ptr[2]));
             auto scale = protocol::unpack_object<std::int32_t>(arr.ptr[3]);
             auto precision = protocol::unpack_object<std::int32_t>(arr.ptr[4]);
 
diff --git a/modules/platforms/cpp/ignite/client/detail/sql/sql_impl.cpp b/modules/platforms/cpp/ignite/client/detail/sql/sql_impl.cpp
index 7918689f0d..481e0946ba 100644
--- a/modules/platforms/cpp/ignite/client/detail/sql/sql_impl.cpp
+++ b/modules/platforms/cpp/ignite/client/detail/sql/sql_impl.cpp
@@ -15,11 +15,12 @@
  * limitations under the License.
  */
 
-#include "ignite/client/detail/sql/sql_impl.h"
+#include "sql_impl.h"
+
 #include "ignite/client/detail/sql/result_set_impl.h"
 #include "ignite/client/detail/utils.h"
 
-#include "ignite/schema/binary_tuple_builder.h"
+#include <ignite/tuple/binary_tuple_builder.h>
 
 namespace ignite::detail {
 
diff --git a/modules/platforms/cpp/ignite/client/detail/sql/sql_impl.h b/modules/platforms/cpp/ignite/client/detail/sql/sql_impl.h
index 719f5c2175..935cef0dbc 100644
--- a/modules/platforms/cpp/ignite/client/detail/sql/sql_impl.h
+++ b/modules/platforms/cpp/ignite/client/detail/sql/sql_impl.h
@@ -34,7 +34,6 @@ namespace ignite::detail {
 class sql_impl {
 public:
     // Default
-    ~sql_impl() = default;
     sql_impl(sql_impl &&) noexcept = default;
     sql_impl &operator=(sql_impl &&) noexcept = default;
 
diff --git a/modules/platforms/cpp/ignite/client/detail/table/schema.h b/modules/platforms/cpp/ignite/client/detail/table/schema.h
index 09b3c41366..81f5cdf656 100644
--- a/modules/platforms/cpp/ignite/client/detail/table/schema.h
+++ b/modules/platforms/cpp/ignite/client/detail/table/schema.h
@@ -17,29 +17,20 @@
 
 #pragma once
 
+#include "ignite/client/detail/client_data_type.h"
+
 #include "ignite/common/ignite_error.h"
+#include "ignite/common/ignite_type.h"
 #include "ignite/protocol/utils.h"
-#include "ignite/schema/ignite_type.h"
 
 #include <msgpack.h>
 
+#include <array>
 #include <memory>
 #include <string>
 
 namespace ignite::detail {
 
-/**
- * Get Ignite type from int value.
- *
- * @param val Value.
- * @return Matching client data type.
- */
-inline ignite_type ignite_type_from_int(std::int32_t val) {
-    if (val < 1 || val >= std::int32_t(ignite_type::LAST))
-        throw ignite_error("Value is out of range for Ignite type: " + std::to_string(val));
-    return ignite_type(val);
-}
-
 /**
  * Column.
  */
@@ -68,7 +59,7 @@ struct column {
 
         column res{};
         res.name = protocol::unpack_object<std::string>(arr.ptr[0]);
-        res.type = ignite_type_from_int(protocol::unpack_object<std::int32_t>(arr.ptr[1]));
+        res.type = client_data_type::to_ignite_type(protocol::unpack_object<std::int32_t>(arr.ptr[1]));
         res.is_key = protocol::unpack_object<bool>(arr.ptr[2]);
         res.nullable = protocol::unpack_object<bool>(arr.ptr[3]);
         res.scale = protocol::unpack_object<std::int32_t>(arr.ptr[5]);
diff --git a/modules/platforms/cpp/ignite/client/detail/table/table_impl.cpp b/modules/platforms/cpp/ignite/client/detail/table/table_impl.cpp
index eb6b3a4db9..e0e700d9b0 100644
--- a/modules/platforms/cpp/ignite/client/detail/table/table_impl.cpp
+++ b/modules/platforms/cpp/ignite/client/detail/table/table_impl.cpp
@@ -15,7 +15,8 @@
  * limitations under the License.
  */
 
-#include "ignite/client/detail/table/table_impl.h"
+#include "table_impl.h"
+
 #include "ignite/client/detail/transaction/transaction_impl.h"
 #include "ignite/client/detail/utils.h"
 #include "ignite/client/table/table.h"
@@ -25,7 +26,7 @@
 #include "ignite/protocol/bitset_span.h"
 #include "ignite/protocol/reader.h"
 #include "ignite/protocol/writer.h"
-#include "ignite/schema/binary_tuple_parser.h"
+#include "ignite/tuple/binary_tuple_parser.h"
 
 namespace ignite::detail {
 
diff --git a/modules/platforms/cpp/ignite/client/detail/table/tables_impl.cpp b/modules/platforms/cpp/ignite/client/detail/table/tables_impl.cpp
index 067c1eb328..177982d3c9 100644
--- a/modules/platforms/cpp/ignite/client/detail/table/tables_impl.cpp
+++ b/modules/platforms/cpp/ignite/client/detail/table/tables_impl.cpp
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-#include "ignite/client/detail/table/tables_impl.h"
+#include "tables_impl.h"
 
 #include <ignite/protocol/reader.h>
 #include <ignite/protocol/writer.h>
diff --git a/modules/platforms/cpp/ignite/client/detail/transaction/transactions_impl.h b/modules/platforms/cpp/ignite/client/detail/transaction/transactions_impl.h
index 530ec43096..1b1aa3c943 100644
--- a/modules/platforms/cpp/ignite/client/detail/transaction/transactions_impl.h
+++ b/modules/platforms/cpp/ignite/client/detail/transaction/transactions_impl.h
@@ -33,7 +33,6 @@ namespace ignite::detail {
 class transactions_impl {
 public:
     // Default
-    ~transactions_impl() = default;
     transactions_impl(transactions_impl &&) noexcept = default;
     transactions_impl &operator=(transactions_impl &&) noexcept = default;
 
diff --git a/modules/platforms/cpp/ignite/client/detail/utils.cpp b/modules/platforms/cpp/ignite/client/detail/utils.cpp
index 6f1fab8f02..e6ed387510 100644
--- a/modules/platforms/cpp/ignite/client/detail/utils.cpp
+++ b/modules/platforms/cpp/ignite/client/detail/utils.cpp
@@ -15,9 +15,12 @@
  * limitations under the License.
  */
 
-#include "ignite/client/detail/utils.h"
-#include "ignite/common/bits.h"
-#include "ignite/common/uuid.h"
+#include "utils.h"
+
+#include "client_data_type.h"
+
+#include <ignite/common/bits.h>
+#include <ignite/common/uuid.h>
 
 #include <string>
 
@@ -57,7 +60,7 @@ void claim_column(binary_tuple_builder &builder, ignite_type typ, const primitiv
         case ignite_type::STRING:
             builder.claim_string(value.get<std::string>());
             break;
-        case ignite_type::BINARY:
+        case ignite_type::BYTE_ARRAY:
             builder.claim_bytes(value.get<std::vector<std::byte>>());
             break;
         case ignite_type::DECIMAL: {
@@ -81,6 +84,12 @@ void claim_column(binary_tuple_builder &builder, ignite_type typ, const primitiv
         case ignite_type::TIMESTAMP:
             builder.claim_timestamp(value.get<ignite_timestamp>());
             break;
+        case ignite_type::PERIOD:
+            builder.claim_period(value.get<ignite_period>());
+            break;
+        case ignite_type::DURATION:
+            builder.claim_duration(value.get<ignite_duration>());
+            break;
         case ignite_type::BITMASK:
             builder.claim_bytes(value.get<bit_array>().get_raw());
             break;
@@ -123,7 +132,7 @@ void append_column(binary_tuple_builder &builder, ignite_type typ, const primiti
         case ignite_type::STRING:
             builder.append_string(value.get<std::string>());
             break;
-        case ignite_type::BINARY:
+        case ignite_type::BYTE_ARRAY:
             builder.append_bytes(value.get<std::vector<std::byte>>());
             break;
         case ignite_type::DECIMAL: {
@@ -147,6 +156,12 @@ void append_column(binary_tuple_builder &builder, ignite_type typ, const primiti
         case ignite_type::TIMESTAMP:
             builder.append_timestamp(value.get<ignite_timestamp>());
             break;
+        case ignite_type::PERIOD:
+            builder.append_period(value.get<ignite_period>());
+            break;
+        case ignite_type::DURATION:
+            builder.append_duration(value.get<ignite_duration>());
+            break;
         case ignite_type::BITMASK:
             builder.append_bytes(value.get<bit_array>().get_raw());
             break;
@@ -205,7 +220,7 @@ std::vector<std::byte> pack_tuple(
  * @param scale Scale.
  */
 void claim_type_and_scale(binary_tuple_builder &builder, ignite_type typ, std::int32_t scale = 0) {
-    builder.claim_int32(std::int32_t(typ));
+    builder.claim_int32(client_data_type::from_ignite_type(typ));
     builder.claim_int32(scale);
 }
 
@@ -217,7 +232,7 @@ void claim_type_and_scale(binary_tuple_builder &builder, ignite_type typ, std::i
  * @param scale Scale.
  */
 void append_type_and_scale(binary_tuple_builder &builder, ignite_type typ, std::int32_t scale = 0) {
-    builder.append_int32(std::int32_t(typ));
+    builder.append_int32(client_data_type::from_ignite_type(typ));
     builder.append_int32(scale);
 }
 
@@ -230,95 +245,103 @@ void claim_primitive_with_type(binary_tuple_builder &builder, const primitive &v
     }
 
     switch (value.get_type()) {
-        case column_type::BOOLEAN: {
+        case ignite_type::BOOLEAN: {
             claim_type_and_scale(builder, ignite_type::INT8);
             builder.claim_int8(1);
             break;
         }
-        case column_type::INT8: {
+        case ignite_type::INT8: {
             claim_type_and_scale(builder, ignite_type::INT8);
             builder.claim_int8(value.get<std::int8_t>());
             break;
         }
-        case column_type::INT16: {
+        case ignite_type::INT16: {
             claim_type_and_scale(builder, ignite_type::INT16);
             builder.claim_int16(value.get<std::int16_t>());
             break;
         }
-        case column_type::INT32: {
+        case ignite_type::INT32: {
             claim_type_and_scale(builder, ignite_type::INT32);
             builder.claim_int32(value.get<std::int32_t>());
             break;
         }
-        case column_type::INT64: {
+        case ignite_type::INT64: {
             claim_type_and_scale(builder, ignite_type::INT64);
             builder.claim_int64(value.get<std::int64_t>());
             break;
         }
-        case column_type::FLOAT: {
+        case ignite_type::FLOAT: {
             claim_type_and_scale(builder, ignite_type::FLOAT);
             builder.claim_float(value.get<float>());
             break;
         }
-        case column_type::DOUBLE: {
+        case ignite_type::DOUBLE: {
             claim_type_and_scale(builder, ignite_type::DOUBLE);
             builder.claim_double(value.get<double>());
             break;
         }
-        case column_type::UUID: {
+        case ignite_type::UUID: {
             claim_type_and_scale(builder, ignite_type::UUID);
             builder.claim_uuid(value.get<uuid>());
             break;
         }
-        case column_type::STRING: {
+        case ignite_type::STRING: {
             claim_type_and_scale(builder, ignite_type::STRING);
             builder.claim_string(value.get<std::string>());
             break;
         }
-        case column_type::BYTE_ARRAY: {
-            claim_type_and_scale(builder, ignite_type::BINARY);
+        case ignite_type::BYTE_ARRAY: {
+            claim_type_and_scale(builder, ignite_type::BYTE_ARRAY);
             auto &data = value.get<std::vector<std::byte>>();
-            builder.claim(ignite_type::BINARY, data);
+            builder.claim(ignite_type::BYTE_ARRAY, data);
             break;
         }
-        case column_type::DECIMAL: {
+        case ignite_type::DECIMAL: {
             const auto &dec_value = value.get<big_decimal>();
             claim_type_and_scale(builder, ignite_type::DECIMAL, dec_value.get_scale());
             builder.claim_number(dec_value);
             break;
         }
-        case column_type::NUMBER: {
+        case ignite_type::NUMBER: {
             claim_type_and_scale(builder, ignite_type::NUMBER);
             builder.claim_number(value.get<big_integer>());
             break;
         }
-        case column_type::DATE: {
+        case ignite_type::DATE: {
             claim_type_and_scale(builder, ignite_type::DATE);
             builder.claim_date(value.get<ignite_date>());
             break;
         }
-        case column_type::TIME: {
+        case ignite_type::TIME: {
             claim_type_and_scale(builder, ignite_type::TIME);
             builder.claim_time(value.get<ignite_time>());
             break;
         }
-        case column_type::DATETIME: {
+        case ignite_type::DATETIME: {
             claim_type_and_scale(builder, ignite_type::DATETIME);
             builder.claim_date_time(value.get<ignite_date_time>());
             break;
         }
-        case column_type::TIMESTAMP: {
+        case ignite_type::TIMESTAMP: {
             claim_type_and_scale(builder, ignite_type::TIMESTAMP);
             builder.claim_timestamp(value.get<ignite_timestamp>());
             break;
         }
-        case column_type::BITMASK: {
+        case ignite_type::PERIOD: {
+            claim_type_and_scale(builder, ignite_type::PERIOD);
+            builder.claim_period(value.get<ignite_period>());
+            break;
+        }
+        case ignite_type::DURATION: {
+            claim_type_and_scale(builder, ignite_type::DURATION);
+            builder.claim_duration(value.get<ignite_duration>());
+            break;
+        }
+        case ignite_type::BITMASK: {
             claim_type_and_scale(builder, ignite_type::BITMASK);
             builder.claim_bytes(value.get<bit_array>().get_raw());
             break;
         }
-        case column_type::PERIOD:
-        case column_type::DURATION:
         default:
             throw ignite_error("Unsupported type: " + std::to_string(int(value.get_type())));
     }
@@ -333,95 +356,103 @@ void append_primitive_with_type(binary_tuple_builder &builder, const primitive &
     }
 
     switch (value.get_type()) {
-        case column_type::BOOLEAN: {
+        case ignite_type::BOOLEAN: {
             append_type_and_scale(builder, ignite_type::INT8);
             builder.append_int8(1);
             break;
         }
-        case column_type::INT8: {
+        case ignite_type::INT8: {
             append_type_and_scale(builder, ignite_type::INT8);
             builder.append_int8(value.get<std::int8_t>());
             break;
         }
-        case column_type::INT16: {
+        case ignite_type::INT16: {
             append_type_and_scale(builder, ignite_type::INT16);
             builder.append_int16(value.get<std::int16_t>());
             break;
         }
-        case column_type::INT32: {
+        case ignite_type::INT32: {
             append_type_and_scale(builder, ignite_type::INT32);
             builder.append_int32(value.get<std::int32_t>());
             break;
         }
-        case column_type::INT64: {
+        case ignite_type::INT64: {
             append_type_and_scale(builder, ignite_type::INT64);
             builder.append_int64(value.get<std::int64_t>());
             break;
         }
-        case column_type::FLOAT: {
+        case ignite_type::FLOAT: {
             append_type_and_scale(builder, ignite_type::FLOAT);
             builder.append_float(value.get<float>());
             break;
         }
-        case column_type::DOUBLE: {
+        case ignite_type::DOUBLE: {
             append_type_and_scale(builder, ignite_type::DOUBLE);
             builder.append_double(value.get<double>());
             break;
         }
-        case column_type::UUID: {
+        case ignite_type::UUID: {
             append_type_and_scale(builder, ignite_type::UUID);
             builder.append_uuid(value.get<uuid>());
             break;
         }
-        case column_type::STRING: {
+        case ignite_type::STRING: {
             append_type_and_scale(builder, ignite_type::STRING);
             builder.append_string(value.get<std::string>());
             break;
         }
-        case column_type::BYTE_ARRAY: {
-            append_type_and_scale(builder, ignite_type::BINARY);
+        case ignite_type::BYTE_ARRAY: {
+            append_type_and_scale(builder, ignite_type::BYTE_ARRAY);
             auto &data = value.get<std::vector<std::byte>>();
-            builder.append(ignite_type::BINARY, data);
+            builder.append(ignite_type::BYTE_ARRAY, data);
             break;
         }
-        case column_type::DECIMAL: {
+        case ignite_type::DECIMAL: {
             const auto &dec_value = value.get<big_decimal>();
             append_type_and_scale(builder, ignite_type::DECIMAL, dec_value.get_scale());
             builder.append_number(dec_value);
             break;
         }
-        case column_type::NUMBER: {
+        case ignite_type::NUMBER: {
             append_type_and_scale(builder, ignite_type::NUMBER);
             builder.append_number(value.get<big_integer>());
             break;
         }
-        case column_type::DATE: {
+        case ignite_type::DATE: {
             append_type_and_scale(builder, ignite_type::DATE);
             builder.append_date(value.get<ignite_date>());
             break;
         }
-        case column_type::TIME: {
+        case ignite_type::TIME: {
             append_type_and_scale(builder, ignite_type::TIME);
             builder.append_time(value.get<ignite_time>());
             break;
         }
-        case column_type::DATETIME: {
+        case ignite_type::DATETIME: {
             append_type_and_scale(builder, ignite_type::DATETIME);
             builder.append_date_time(value.get<ignite_date_time>());
             break;
         }
-        case column_type::TIMESTAMP: {
+        case ignite_type::TIMESTAMP: {
             append_type_and_scale(builder, ignite_type::TIMESTAMP);
             builder.append_timestamp(value.get<ignite_timestamp>());
             break;
         }
-        case column_type::BITMASK: {
+        case ignite_type::PERIOD: {
+            append_type_and_scale(builder, ignite_type::PERIOD);
+            builder.append_period(value.get<ignite_period>());
+            break;
+        }
+        case ignite_type::DURATION: {
+            append_type_and_scale(builder, ignite_type::DURATION);
+            builder.append_duration(value.get<ignite_duration>());
+            break;
+        }
+        case ignite_type::BITMASK: {
             append_type_and_scale(builder, ignite_type::BITMASK);
             builder.append_bytes(value.get<bit_array>().get_raw());
             break;
         }
-        case column_type::PERIOD:
-        case column_type::DURATION:
         default:
             throw ignite_error("Unsupported type: " + std::to_string(int(value.get_type())));
     }
@@ -435,6 +466,8 @@ primitive read_next_column(binary_tuple_parser &parser, ignite_type typ, std::in
     auto val = val_opt.value();
 
     switch (typ) {
+        case ignite_type::BOOLEAN:
+            return binary_tuple_parser::get_int8(val) != 0;
         case ignite_type::INT8:
             return binary_tuple_parser::get_int8(val);
         case ignite_type::INT16:
@@ -451,7 +484,7 @@ primitive read_next_column(binary_tuple_parser &parser, ignite_type typ, std::in
             return binary_tuple_parser::get_uuid(val);
         case ignite_type::STRING:
             return std::string(reinterpret_cast<const char *>(val.data()), val.size());
-        case ignite_type::BINARY:
+        case ignite_type::BYTE_ARRAY:
             return std::vector<std::byte>(val);
         case ignite_type::DECIMAL:
             return binary_tuple_parser::get_decimal(val, scale);
@@ -465,6 +498,10 @@ primitive read_next_column(binary_tuple_parser &parser, ignite_type typ, std::in
             return binary_tuple_parser::get_date_time(val);
         case ignite_type::TIMESTAMP:
             return binary_tuple_parser::get_timestamp(val);
+        case ignite_type::PERIOD:
+            return binary_tuple_parser::get_period(val);
+        case ignite_type::DURATION:
+            return binary_tuple_parser::get_duration(val);
         case ignite_type::BITMASK:
             return bit_array(val);
         default:
@@ -472,56 +509,6 @@ primitive read_next_column(binary_tuple_parser &parser, ignite_type typ, std::in
     }
 }
 
-primitive read_next_column(binary_tuple_parser &parser, column_type typ, std::int32_t scale) {
-    auto val_opt = parser.get_next();
-    if (!val_opt)
-        return {};
-
-    auto val = val_opt.value();
-
-    switch (typ) {
-        case column_type::BOOLEAN:
-            return binary_tuple_parser::get_int8(val) != 0;
-        case column_type::INT8:
-            return binary_tuple_parser::get_int8(val);
-        case column_type::INT16:
-            return binary_tuple_parser::get_int16(val);
-        case column_type::INT32:
-            return binary_tuple_parser::get_int32(val);
-        case column_type::INT64:
-            return binary_tuple_parser::get_int64(val);
-        case column_type::FLOAT:
-            return binary_tuple_parser::get_float(val);
-        case column_type::DOUBLE:
-            return binary_tuple_parser::get_double(val);
-        case column_type::UUID:
-            return binary_tuple_parser::get_uuid(val);
-        case column_type::STRING:
-            return std::string(reinterpret_cast<const char *>(val.data()), val.size());
-        case column_type::BYTE_ARRAY:
-            return std::vector<std::byte>(val);
-        case column_type::DECIMAL:
-            return binary_tuple_parser::get_decimal(val, scale);
-        case column_type::NUMBER:
-            return binary_tuple_parser::get_number(val);
-        case column_type::DATE:
-            return binary_tuple_parser::get_date(val);
-        case column_type::TIME:
-            return binary_tuple_parser::get_time(val);
-        case column_type::DATETIME:
-            return binary_tuple_parser::get_date_time(val);
-        case column_type::TIMESTAMP:
-            return binary_tuple_parser::get_timestamp(val);
-        case column_type::BITMASK:
-            return bit_array(val);
-        case column_type::PERIOD:
-        case column_type::DURATION:
-            // TODO: IGNITE-18745 Support period and duration types
-        default:
-            throw ignite_error("Type with id " + std::to_string(int(typ)) + " is not yet supported");
-    }
-}
-
 ignite_tuple concat(const ignite_tuple &left, const ignite_tuple &right) {
     // TODO: IGNITE-18855 eliminate unnecessary tuple transformation;
 
diff --git a/modules/platforms/cpp/ignite/client/detail/utils.h b/modules/platforms/cpp/ignite/client/detail/utils.h
index 6715cb409f..866861e419 100644
--- a/modules/platforms/cpp/ignite/client/detail/utils.h
+++ b/modules/platforms/cpp/ignite/client/detail/utils.h
@@ -23,9 +23,8 @@
 #include "ignite/client/transaction/transaction.h"
 
 #include "ignite/protocol/writer.h"
-#include "ignite/schema/binary_tuple_builder.h"
-#include "ignite/schema/binary_tuple_parser.h"
-#include "ignite/schema/ignite_type.h"
+#include "ignite/tuple/binary_tuple_builder.h"
+#include "ignite/tuple/binary_tuple_parser.h"
 
 namespace ignite::detail {
 
@@ -55,16 +54,6 @@ void append_primitive_with_type(binary_tuple_builder &builder, const primitive &
  */
 [[nodiscard]] primitive read_next_column(binary_tuple_parser &parser, ignite_type typ, std::int32_t scale);
 
-/**
- * Read column value from binary tuple.
- *
- * @param parser Binary tuple parser.
- * @param typ Column type.
- * @param scale Column scale.
- * @return Column value.
- */
-[[nodiscard]] primitive read_next_column(binary_tuple_parser &parser, column_type typ, std::int32_t scale);
-
 /**
  * Tuple concatenation function.
  *
diff --git a/modules/platforms/cpp/ignite/client/ignite_client.h b/modules/platforms/cpp/ignite/client/ignite_client.h
index b20e10e8dd..90b128fb92 100644
--- a/modules/platforms/cpp/ignite/client/ignite_client.h
+++ b/modules/platforms/cpp/ignite/client/ignite_client.h
@@ -46,7 +46,6 @@ class ignite_client {
 public:
     // Default
     ignite_client() = default;
-    ~ignite_client() = default;
     ignite_client(ignite_client &&) = default;
     ignite_client &operator=(ignite_client &&) = default;
 
diff --git a/modules/platforms/cpp/ignite/client/primitive.h b/modules/platforms/cpp/ignite/client/primitive.h
index f9ecd689f4..fec32c8aa9 100644
--- a/modules/platforms/cpp/ignite/client/primitive.h
+++ b/modules/platforms/cpp/ignite/client/primitive.h
@@ -17,16 +17,18 @@
 
 #pragma once
 
-#include "ignite/client/sql/sql_column_type.h"
-#include "ignite/common/big_decimal.h"
-#include "ignite/common/big_integer.h"
-#include "ignite/common/bit_array.h"
-#include "ignite/common/ignite_date.h"
-#include "ignite/common/ignite_date_time.h"
-#include "ignite/common/ignite_error.h"
-#include "ignite/common/ignite_time.h"
-#include "ignite/common/ignite_timestamp.h"
-#include "ignite/common/uuid.h"
+#include <ignite/common/big_decimal.h>
+#include <ignite/common/big_integer.h>
+#include <ignite/common/bit_array.h>
+#include <ignite/common/ignite_date.h>
+#include <ignite/common/ignite_date_time.h>
+#include <ignite/common/ignite_duration.h>
+#include <ignite/common/ignite_error.h>
+#include <ignite/common/ignite_period.h>
+#include <ignite/common/ignite_time.h>
+#include <ignite/common/ignite_timestamp.h>
+#include <ignite/common/ignite_type.h>
+#include <ignite/common/uuid.h>
 
 #include <cstdint>
 #include <optional>
@@ -207,6 +209,22 @@ public:
     primitive(ignite_timestamp value) // NOLINT(google-explicit-constructor)
         : m_value(value) {}
 
+    /**
+     * Constructor for period value.
+     *
+     * @param value Value.
+     */
+    primitive(ignite_period value) // NOLINT(google-explicit-constructor)
+        : m_value(value) {}
+
+    /**
+     * Constructor for duration value.
+     *
+     * @param value Value.
+     */
+    primitive(ignite_duration value) // NOLINT(google-explicit-constructor)
+        : m_value(value) {}
+
     /**
      * Constructor for bitmask value.
      *
@@ -240,6 +258,8 @@ public:
             || std::is_same_v<T, ignite_date_time> // DateTime
             || std::is_same_v<T, ignite_time> // Time
             || std::is_same_v<T, ignite_timestamp> // Timestamp
+            || std::is_same_v<T, ignite_period> // Period
+            || std::is_same_v<T, ignite_duration> // Duration
             || std::is_same_v<T, bit_array> // Bit Array
         ) {
             return std::get<T>(m_value);
@@ -260,10 +280,10 @@ public:
      *
      * @return Primitive type.
      */
-    [[nodiscard]] column_type get_type() const noexcept {
+    [[nodiscard]] ignite_type get_type() const noexcept {
         if (is_null())
-            return column_type::UNDEFINED;
-        return static_cast<column_type>(m_value.index() - 1);
+            return ignite_type::UNDEFINED;
+        return static_cast<ignite_type>(m_value.index() - 1);
     }
 
     /**
@@ -289,9 +309,6 @@ public:
     }
 
 private:
-    /** Unsupported type. */
-    typedef void *unsupported_type;
-
     /** Value type. */
     typedef std::variant<std::nullptr_t,
         bool, // Bool = 0
@@ -310,8 +327,8 @@ private:
         bit_array, // Bitmask = 13
         std::string, // String = 14
         std::vector<std::byte>, // Bytes = 15
-        unsupported_type, // Period = 16
-        unsupported_type, // Duration = 17
+        ignite_period, // Period = 16
+        ignite_duration, // Duration = 17
         big_integer // Big Integer = 18
         >
         value_type;
diff --git a/modules/platforms/cpp/ignite/client/primitive_test.cpp b/modules/platforms/cpp/ignite/client/primitive_test.cpp
index b71b3033e3..4bbab0d52c 100644
--- a/modules/platforms/cpp/ignite/client/primitive_test.cpp
+++ b/modules/platforms/cpp/ignite/client/primitive_test.cpp
@@ -22,40 +22,42 @@
 using namespace ignite;
 
 template<typename T>
-void check_primitive_type(column_type expected) {
+void check_primitive_type(ignite_type expected) {
     primitive val(T{});
     EXPECT_EQ(val.get_type(), expected);
 }
 
 TEST(primitive, get_column_type) {
-    check_primitive_type<nullptr_t>(column_type::UNDEFINED);
-    check_primitive_type<bool>(column_type::BOOLEAN);
-    check_primitive_type<int8_t>(column_type::INT8);
-    check_primitive_type<int16_t>(column_type::INT16);
-    check_primitive_type<int32_t>(column_type::INT32);
-    check_primitive_type<int64_t>(column_type::INT64);
-    check_primitive_type<float>(column_type::FLOAT);
-    check_primitive_type<double>(column_type::DOUBLE);
-    check_primitive_type<big_decimal>(column_type::DECIMAL);
-    check_primitive_type<ignite_date>(column_type::DATE);
-    check_primitive_type<ignite_time>(column_type::TIME);
-    check_primitive_type<ignite_date_time>(column_type::DATETIME);
-    check_primitive_type<ignite_timestamp>(column_type::TIMESTAMP);
-    check_primitive_type<uuid>(column_type::UUID);
-    check_primitive_type<bit_array>(column_type::BITMASK);
-    check_primitive_type<std::string>(column_type::STRING);
-    check_primitive_type<std::vector<std::byte>>(column_type::BYTE_ARRAY);
-    check_primitive_type<big_integer>(column_type::NUMBER);
+    check_primitive_type<nullptr_t>(ignite_type::UNDEFINED);
+    check_primitive_type<bool>(ignite_type::BOOLEAN);
+    check_primitive_type<int8_t>(ignite_type::INT8);
+    check_primitive_type<int16_t>(ignite_type::INT16);
+    check_primitive_type<int32_t>(ignite_type::INT32);
+    check_primitive_type<int64_t>(ignite_type::INT64);
+    check_primitive_type<float>(ignite_type::FLOAT);
+    check_primitive_type<double>(ignite_type::DOUBLE);
+    check_primitive_type<big_decimal>(ignite_type::DECIMAL);
+    check_primitive_type<ignite_date>(ignite_type::DATE);
+    check_primitive_type<ignite_time>(ignite_type::TIME);
+    check_primitive_type<ignite_date_time>(ignite_type::DATETIME);
+    check_primitive_type<ignite_timestamp>(ignite_type::TIMESTAMP);
+    check_primitive_type<ignite_period>(ignite_type::PERIOD);
+    check_primitive_type<ignite_duration>(ignite_type::DURATION);
+    check_primitive_type<uuid>(ignite_type::UUID);
+    check_primitive_type<bit_array>(ignite_type::BITMASK);
+    check_primitive_type<std::string>(ignite_type::STRING);
+    check_primitive_type<std::vector<std::byte>>(ignite_type::BYTE_ARRAY);
+    check_primitive_type<big_integer>(ignite_type::NUMBER);
 }
 
 TEST(primitive, null_value_by_nullptr) {
     primitive val(nullptr);
-    EXPECT_EQ(val.get_type(), column_type::UNDEFINED);
+    EXPECT_EQ(val.get_type(), ignite_type::UNDEFINED);
     EXPECT_TRUE(val.is_null());
 }
 
 TEST(primitive, null_value_by_nullopt) {
     primitive val(std::nullopt);
-    EXPECT_EQ(val.get_type(), column_type::UNDEFINED);
+    EXPECT_EQ(val.get_type(), ignite_type::UNDEFINED);
     EXPECT_TRUE(val.is_null());
 }
diff --git a/modules/platforms/cpp/ignite/client/sql/column_metadata.h b/modules/platforms/cpp/ignite/client/sql/column_metadata.h
index 93044add4b..1782362666 100644
--- a/modules/platforms/cpp/ignite/client/sql/column_metadata.h
+++ b/modules/platforms/cpp/ignite/client/sql/column_metadata.h
@@ -18,7 +18,7 @@
 #pragma once
 
 #include "ignite/client/sql/column_origin.h"
-#include "ignite/client/sql/sql_column_type.h"
+#include "ignite/common/ignite_type.h"
 
 #include <cstdint>
 #include <string>
@@ -43,7 +43,7 @@ public:
      * @param nullable Column nullability.
      * @param origin Column origin.
      */
-    column_metadata(std::string name, column_type type, std::int32_t precision, std::int32_t scale, bool nullable,
+    column_metadata(std::string name, ignite_type type, std::int32_t precision, std::int32_t scale, bool nullable,
         column_origin origin)
         : m_name(std::move(name))
         , m_type(type)
@@ -64,7 +64,7 @@ public:
      *
      * @return Column type.
      */
-    [[nodiscard]] column_type type() const { return m_type; }
+    [[nodiscard]] ignite_type type() const { return m_type; }
 
     /**
      * Gets the column precision, or -1 when not applicable to the current
@@ -107,7 +107,7 @@ private:
     std::string m_name;
 
     /** Column type. */
-    column_type m_type{column_type::UNDEFINED};
+    ignite_type m_type{ignite_type::UNDEFINED};
 
     /** Precision. */
     std::int32_t m_precision{0};
diff --git a/modules/platforms/cpp/ignite/client/table/key_value_view.h b/modules/platforms/cpp/ignite/client/table/key_value_view.h
index 10a517ac58..26fbafc0a7 100644
--- a/modules/platforms/cpp/ignite/client/table/key_value_view.h
+++ b/modules/platforms/cpp/ignite/client/table/key_value_view.h
@@ -51,7 +51,6 @@ public:
 
     // Default
     key_value_view() = default;
-    ~key_value_view() = default;
     key_value_view(key_value_view &&) noexcept = default;
     key_value_view &operator=(key_value_view &&) noexcept = default;
 };
@@ -73,7 +72,6 @@ public:
 
     // Default
     key_value_view() = default;
-    ~key_value_view() = default;
     key_value_view(key_value_view &&) noexcept = default;
     key_value_view &operator=(key_value_view &&) noexcept = default;
 
diff --git a/modules/platforms/cpp/ignite/client/table/record_view.h b/modules/platforms/cpp/ignite/client/table/record_view.h
index c0f8fcabaf..5a9d10b298 100644
--- a/modules/platforms/cpp/ignite/client/table/record_view.h
+++ b/modules/platforms/cpp/ignite/client/table/record_view.h
@@ -50,7 +50,6 @@ public:
 
     // Default
     record_view() = default;
-    ~record_view() = default;
     record_view(record_view &&) noexcept = default;
     record_view &operator=(record_view &&) noexcept = default;
 };
@@ -71,7 +70,6 @@ public:
 
     // Default
     record_view() = default;
-    ~record_view() = default;
     record_view(record_view &&) noexcept = default;
     record_view &operator=(record_view &&) noexcept = default;
 
diff --git a/modules/platforms/cpp/ignite/client/table/table.h b/modules/platforms/cpp/ignite/client/table/table.h
index d946d93fb0..24721dacde 100644
--- a/modules/platforms/cpp/ignite/client/table/table.h
+++ b/modules/platforms/cpp/ignite/client/table/table.h
@@ -44,7 +44,6 @@ class table {
 public:
     // Default
     table() = default;
-    ~table() = default;
     table(table &&) noexcept = default;
     table &operator=(table &&) noexcept = default;
 
diff --git a/modules/platforms/cpp/ignite/common/CMakeLists.txt b/modules/platforms/cpp/ignite/common/CMakeLists.txt
index 1c5a786c28..3fa3eae660 100644
--- a/modules/platforms/cpp/ignite/common/CMakeLists.txt
+++ b/modules/platforms/cpp/ignite/common/CMakeLists.txt
@@ -30,10 +30,13 @@ set(PUBLIC_HEADERS
     end_point.h
     ignite_date.h
     ignite_date_time.h
+    ignite_duration.h
     ignite_error.h
+    ignite_period.h
     ignite_result.h
     ignite_time.h
     ignite_timestamp.h
+    ignite_type.h
     uuid.h
 )
 
diff --git a/modules/platforms/cpp/ignite/common/bytes_view.h b/modules/platforms/cpp/ignite/common/bytes_view.h
index 3ac84f93fb..af8e4d79c2 100644
--- a/modules/platforms/cpp/ignite/common/bytes_view.h
+++ b/modules/platforms/cpp/ignite/common/bytes_view.h
@@ -17,7 +17,9 @@
 
 #pragma once
 
+#include <array>
 #include <cstddef>
+#include <string>
 #include <string_view>
 #include <vector>
 
@@ -25,18 +27,32 @@ namespace ignite {
 
 /** A slice of raw bytes. */
 struct bytes_view : std::basic_string_view<std::byte> {
-    using Base = std::basic_string_view<std::byte>;
+    using base_type = std::basic_string_view<std::byte>;
 
     constexpr bytes_view() noexcept = default;
 
     constexpr bytes_view(const std::byte *data, std::size_t size) noexcept
-        : Base(data, size) {}
+        : base_type(data, size) {}
 
-    constexpr bytes_view(const Base &v) noexcept // NOLINT(google-explicit-constructor)
-        : Base(v.data(), v.size()) {}
+    constexpr bytes_view(const base_type &v) noexcept // NOLINT(google-explicit-constructor)
+        : base_type(v.data(), v.size()) {}
+
+    template<std::size_t SIZE>
+    constexpr bytes_view(const char (&v)[SIZE]) noexcept // NOLINT(google-explicit-constructor)
+        : base_type(reinterpret_cast<const std::byte *>(v), SIZE) {}
+
+    template<std::size_t SIZE>
+    constexpr bytes_view(const std::array<std::byte, SIZE> &v) noexcept // NOLINT(google-explicit-constructor)
+        : base_type(v.data(), v.size()) {}
+
+    bytes_view(const std::string &v) noexcept // NOLINT(google-explicit-constructor)
+        : base_type(reinterpret_cast<const std::byte *>(v.data()), v.size()) {}
+
+    bytes_view(const std::string_view &v) noexcept // NOLINT(google-explicit-constructor)
+        : base_type(reinterpret_cast<const std::byte *>(v.data()), v.size()) {}
 
     bytes_view(const std::vector<std::byte> &v) noexcept // NOLINT(google-explicit-constructor)
-        : Base(v.data(), v.size()) {}
+        : base_type(v.data(), v.size()) {}
 
     explicit operator std::vector<std::byte>() const { return {begin(), end()}; }
 };
diff --git a/modules/platforms/cpp/ignite/common/ignite_date.h b/modules/platforms/cpp/ignite/common/ignite_date.h
index 5b9ad32df7..8f1411f90c 100644
--- a/modules/platforms/cpp/ignite/common/ignite_date.h
+++ b/modules/platforms/cpp/ignite/common/ignite_date.h
@@ -40,7 +40,7 @@ public:
      * @param month Month-of-year, from 1 (January) to 12 (December).
      * @param day_of_month Day-of-month, from 1 to 31.
      */
-    constexpr ignite_date(int year, int month, int day_of_month)
+    constexpr ignite_date(std::int32_t year, std::int32_t month, std::int32_t day_of_month)
         : m_year(year)
         , m_month(std::int8_t(month))
         , m_day(std::int8_t(day_of_month)) {
@@ -50,17 +50,17 @@ public:
     /**
      * Gets the year field.
      */
-    [[nodiscard]] constexpr int get_year() const noexcept { return m_year; }
+    [[nodiscard]] constexpr std::int32_t get_year() const noexcept { return m_year; }
 
     /**
      * Gets the month-of-year field.
      */
-    [[nodiscard]] constexpr int get_month() const noexcept { return m_month; }
+    [[nodiscard]] constexpr std::int_fast8_t get_month() const noexcept { return m_month; }
 
     /**
      * Gets the day-of-month field.
      */
-    [[nodiscard]] constexpr int get_day_of_month() const noexcept { return m_day; }
+    [[nodiscard]] constexpr std::int_fast8_t get_day_of_month() const noexcept { return m_day; }
 
     /**
      * Compare to another value.
@@ -79,9 +79,9 @@ public:
     }
 
 private:
-    std::int32_t m_year = 0;
-    std::int8_t m_month = 1;
-    std::int8_t m_day = 1;
+    std::int_least32_t m_year = 0;
+    std::int_least8_t m_month = 1;
+    std::int_least8_t m_day = 1;
 };
 
 /**
diff --git a/modules/platforms/cpp/ignite/common/ignite_date.h b/modules/platforms/cpp/ignite/common/ignite_duration.h
similarity index 58%
copy from modules/platforms/cpp/ignite/common/ignite_date.h
copy to modules/platforms/cpp/ignite/common/ignite_duration.h
index 5b9ad32df7..232298723f 100644
--- a/modules/platforms/cpp/ignite/common/ignite_date.h
+++ b/modules/platforms/cpp/ignite/common/ignite_duration.h
@@ -22,45 +22,38 @@
 namespace ignite {
 
 /**
- * @brief A date.
+ * @brief A time-based amount of time.
  *
- * This is modeled after java.time.LocalDate.
+ * This is modeled after java.time.Duration.
  */
-class ignite_date {
+class ignite_duration {
 public:
     /**
      * Default constructor.
      */
-    constexpr ignite_date() noexcept = default;
+    constexpr ignite_duration() noexcept = default;
 
     /**
      * Constructor.
      *
-     * @param year Year to represent, from -999,999,999 to 999,999,999.
-     * @param month Month-of-year, from 1 (January) to 12 (December).
-     * @param day_of_month Day-of-month, from 1 to 31.
+     * @param seconds Number of seconds.
+     * @param nanos Fractional second component in nanoseconds.
      */
-    constexpr ignite_date(int year, int month, int day_of_month)
-        : m_year(year)
-        , m_month(std::int8_t(month))
-        , m_day(std::int8_t(day_of_month)) {
+    constexpr ignite_duration(std::int64_t seconds, std::int32_t nanos)
+        : seconds(seconds)
+        , nanos(nanos) {
         // TODO: check that arguments are in valid ranges.
     }
 
     /**
-     * Gets the year field.
+     * Gets the number of seconds.
      */
-    [[nodiscard]] constexpr int get_year() const noexcept { return m_year; }
+    constexpr std::int64_t get_seconds() const noexcept { return seconds; }
 
     /**
-     * Gets the month-of-year field.
+     * Gets the number of nanoseconds.
      */
-    [[nodiscard]] constexpr int get_month() const noexcept { return m_month; }
-
-    /**
-     * Gets the day-of-month field.
-     */
-    [[nodiscard]] constexpr int get_day_of_month() const noexcept { return m_day; }
+    constexpr std::int32_t get_nano() const noexcept { return nanos; }
 
     /**
      * Compare to another value.
@@ -68,20 +61,19 @@ public:
      * @param other Instance to compare to.
      * @return Zero if equals, negative number if less, and positive if greater.
      */
-    [[nodiscard]] constexpr int compare(const ignite_date &other) const noexcept {
-        if (m_year != other.m_year) {
-            return m_year - other.m_year;
+    constexpr int compare(const ignite_duration &other) const noexcept {
+        if (seconds != other.seconds) {
+            return seconds < other.seconds ? -1 : 1;
         }
-        if (m_month != other.m_month) {
-            return m_month - other.m_month;
-        }
-        return m_day - other.m_day;
+        return nanos - other.nanos;
     }
 
 private:
-    std::int32_t m_year = 0;
-    std::int8_t m_month = 1;
-    std::int8_t m_day = 1;
+    /** Number of seconds. */
+    std::int64_t seconds = 0;
+
+    /** Fractional second component in nanoseconds. */
+    std::int32_t nanos = 0;
 };
 
 /**
@@ -91,7 +83,7 @@ private:
  * @param rhs Second value.
  * @return true If the first value is equal to the second.
  */
-constexpr bool operator==(const ignite_date &lhs, const ignite_date &rhs) noexcept {
+constexpr bool operator==(const ignite_duration &lhs, const ignite_duration &rhs) noexcept {
     return lhs.compare(rhs) == 0;
 }
 
@@ -102,7 +94,7 @@ constexpr bool operator==(const ignite_date &lhs, const ignite_date &rhs) noexce
  * @param rhs Second value.
  * @return true If the first value is not equal to the second.
  */
-constexpr bool operator!=(const ignite_date &lhs, const ignite_date &rhs) noexcept {
+constexpr bool operator!=(const ignite_duration &lhs, const ignite_duration &rhs) noexcept {
     return lhs.compare(rhs) != 0;
 }
 
@@ -113,7 +105,7 @@ constexpr bool operator!=(const ignite_date &lhs, const ignite_date &rhs) noexce
  * @param rhs Second value.
  * @return true If the first value is less than the second.
  */
-constexpr bool operator<(const ignite_date &lhs, const ignite_date &rhs) noexcept {
+constexpr bool operator<(const ignite_duration &lhs, const ignite_duration &rhs) noexcept {
     return lhs.compare(rhs) < 0;
 }
 
@@ -124,7 +116,7 @@ constexpr bool operator<(const ignite_date &lhs, const ignite_date &rhs) noexcep
  * @param rhs Second value.
  * @return true If the first value is less than or equal to the second.
  */
-constexpr bool operator<=(const ignite_date &lhs, const ignite_date &rhs) noexcept {
+constexpr bool operator<=(const ignite_duration &lhs, const ignite_duration &rhs) noexcept {
     return lhs.compare(rhs) <= 0;
 }
 
@@ -135,7 +127,7 @@ constexpr bool operator<=(const ignite_date &lhs, const ignite_date &rhs) noexce
  * @param rhs Second value.
  * @return true If the first value is greater than the second.
  */
-constexpr bool operator>(const ignite_date &lhs, const ignite_date &rhs) noexcept {
+constexpr bool operator>(const ignite_duration &lhs, const ignite_duration &rhs) noexcept {
     return lhs.compare(rhs) > 0;
 }
 
@@ -146,7 +138,7 @@ constexpr bool operator>(const ignite_date &lhs, const ignite_date &rhs) noexcep
  * @param rhs Second value.
  * @return true If the first value is greater than or equal to the second.
  */
-constexpr bool operator>=(const ignite_date &lhs, const ignite_date &rhs) noexcept {
+constexpr bool operator>=(const ignite_duration &lhs, const ignite_duration &rhs) noexcept {
     return lhs.compare(rhs) >= 0;
 }
 
diff --git a/modules/platforms/cpp/ignite/common/ignite_error.h b/modules/platforms/cpp/ignite/common/ignite_error.h
index 4ee2853e84..cbcd1d18d6 100644
--- a/modules/platforms/cpp/ignite/common/ignite_error.h
+++ b/modules/platforms/cpp/ignite/common/ignite_error.h
@@ -51,7 +51,7 @@ public:
      *
      * @param message Message.
      */
-    explicit ignite_error(std::string message)
+    explicit ignite_error(std::string message) noexcept
         : m_status_code(status_code::GENERIC)
         , m_message(std::move(message))
         , m_cause() {} // NOLINT(bugprone-throw-keyword-missing)
@@ -62,7 +62,7 @@ public:
      * @param statusCode Status code.
      * @param message Message.
      */
-    explicit ignite_error(status_code statusCode, std::string message)
+    explicit ignite_error(status_code statusCode, std::string message) noexcept
         : m_status_code(statusCode)
         , m_message(std::move(message))
         , m_cause() {} // NOLINT(bugprone-throw-keyword-missing)
@@ -74,10 +74,10 @@ public:
      * @param message Message.
      * @param cause Error cause.
      */
-    explicit ignite_error(status_code statusCode, std::string message, const std::exception_ptr &cause)
+    explicit ignite_error(status_code statusCode, std::string message, std::exception_ptr cause) noexcept
         : m_status_code(statusCode)
         , m_message(std::move(message))
-        , m_cause(cause) {} // NOLINT(bugprone-throw-keyword-missing)
+        , m_cause(std::move(cause)) {} // NOLINT(bugprone-throw-keyword-missing)
 
     /**
      * Get error message.
@@ -87,21 +87,21 @@ public:
     /**
      * Get error message as std::string.
      */
-    [[nodiscard]] const std::string &what_str() const { return m_message; }
+    [[nodiscard]] const std::string &what_str() const noexcept { return m_message; }
 
     /**
      * Get status code.
      *
      * @return Status code.
      */
-    [[nodiscard]] status_code get_status_code() const { return m_status_code; }
+    [[nodiscard]] status_code get_status_code() const noexcept { return m_status_code; }
 
     /**
      * Get error cause.
      *
      * @return Error cause. Can be empty.
      */
-    [[nodiscard]] std::exception_ptr get_cause() { return m_cause; }
+    [[nodiscard]] std::exception_ptr get_cause() const noexcept { return m_cause; }
 
 private:
     /** Status code. */
diff --git a/modules/platforms/cpp/ignite/common/ignite_date.h b/modules/platforms/cpp/ignite/common/ignite_period.h
similarity index 59%
copy from modules/platforms/cpp/ignite/common/ignite_date.h
copy to modules/platforms/cpp/ignite/common/ignite_period.h
index 5b9ad32df7..ba22991c9d 100644
--- a/modules/platforms/cpp/ignite/common/ignite_date.h
+++ b/modules/platforms/cpp/ignite/common/ignite_period.h
@@ -22,45 +22,45 @@
 namespace ignite {
 
 /**
- * @brief A date.
+ * @brief A date-based amount of time.
  *
- * This is modeled after java.time.LocalDate.
+ * This is modeled after java.time.Period.
  */
-class ignite_date {
+class ignite_period {
 public:
     /**
      * Default constructor.
      */
-    constexpr ignite_date() noexcept = default;
+    constexpr ignite_period() noexcept = default;
 
     /**
      * Constructor.
      *
-     * @param year Year to represent, from -999,999,999 to 999,999,999.
-     * @param month Month-of-year, from 1 (January) to 12 (December).
-     * @param day_of_month Day-of-month, from 1 to 31.
+     * @param years The number of years.
+     * @param months The number of months.
+     * @param days The number of days.
      */
-    constexpr ignite_date(int year, int month, int day_of_month)
-        : m_year(year)
-        , m_month(std::int8_t(month))
-        , m_day(std::int8_t(day_of_month)) {
+    constexpr ignite_period(std::int32_t years, std::int32_t months, std::int32_t days)
+        : years(years)
+        , months(months)
+        , days(days) {
         // TODO: check that arguments are in valid ranges.
     }
 
     /**
-     * Gets the year field.
+     * Gets the years field.
      */
-    [[nodiscard]] constexpr int get_year() const noexcept { return m_year; }
+    constexpr std::int32_t get_years() const noexcept { return years; }
 
     /**
-     * Gets the month-of-year field.
+     * Gets the months field.
      */
-    [[nodiscard]] constexpr int get_month() const noexcept { return m_month; }
+    constexpr std::int32_t get_months() const noexcept { return months; }
 
     /**
-     * Gets the day-of-month field.
+     * Gets the days field.
      */
-    [[nodiscard]] constexpr int get_day_of_month() const noexcept { return m_day; }
+    constexpr std::int32_t get_days() const noexcept { return days; }
 
     /**
      * Compare to another value.
@@ -68,20 +68,20 @@ public:
      * @param other Instance to compare to.
      * @return Zero if equals, negative number if less, and positive if greater.
      */
-    [[nodiscard]] constexpr int compare(const ignite_date &other) const noexcept {
-        if (m_year != other.m_year) {
-            return m_year - other.m_year;
+    constexpr int compare(const ignite_period &other) const noexcept {
+        if (years != other.years) {
+            return years - other.years;
         }
-        if (m_month != other.m_month) {
-            return m_month - other.m_month;
+        if (months != other.months) {
+            return months - other.months;
         }
-        return m_day - other.m_day;
+        return days - other.days;
     }
 
 private:
-    std::int32_t m_year = 0;
-    std::int8_t m_month = 1;
-    std::int8_t m_day = 1;
+    std::int_least32_t years = 0;
+    std::int_least32_t months = 0;
+    std::int_least32_t days = 0;
 };
 
 /**
@@ -91,7 +91,7 @@ private:
  * @param rhs Second value.
  * @return true If the first value is equal to the second.
  */
-constexpr bool operator==(const ignite_date &lhs, const ignite_date &rhs) noexcept {
+constexpr bool operator==(const ignite_period &lhs, const ignite_period &rhs) noexcept {
     return lhs.compare(rhs) == 0;
 }
 
@@ -102,7 +102,7 @@ constexpr bool operator==(const ignite_date &lhs, const ignite_date &rhs) noexce
  * @param rhs Second value.
  * @return true If the first value is not equal to the second.
  */
-constexpr bool operator!=(const ignite_date &lhs, const ignite_date &rhs) noexcept {
+constexpr bool operator!=(const ignite_period &lhs, const ignite_period &rhs) noexcept {
     return lhs.compare(rhs) != 0;
 }
 
@@ -113,7 +113,7 @@ constexpr bool operator!=(const ignite_date &lhs, const ignite_date &rhs) noexce
  * @param rhs Second value.
  * @return true If the first value is less than the second.
  */
-constexpr bool operator<(const ignite_date &lhs, const ignite_date &rhs) noexcept {
+constexpr bool operator<(const ignite_period &lhs, const ignite_period &rhs) noexcept {
     return lhs.compare(rhs) < 0;
 }
 
@@ -124,7 +124,7 @@ constexpr bool operator<(const ignite_date &lhs, const ignite_date &rhs) noexcep
  * @param rhs Second value.
  * @return true If the first value is less than or equal to the second.
  */
-constexpr bool operator<=(const ignite_date &lhs, const ignite_date &rhs) noexcept {
+constexpr bool operator<=(const ignite_period &lhs, const ignite_period &rhs) noexcept {
     return lhs.compare(rhs) <= 0;
 }
 
@@ -135,7 +135,7 @@ constexpr bool operator<=(const ignite_date &lhs, const ignite_date &rhs) noexce
  * @param rhs Second value.
  * @return true If the first value is greater than the second.
  */
-constexpr bool operator>(const ignite_date &lhs, const ignite_date &rhs) noexcept {
+constexpr bool operator>(const ignite_period &lhs, const ignite_period &rhs) noexcept {
     return lhs.compare(rhs) > 0;
 }
 
@@ -146,7 +146,7 @@ constexpr bool operator>(const ignite_date &lhs, const ignite_date &rhs) noexcep
  * @param rhs Second value.
  * @return true If the first value is greater than or equal to the second.
  */
-constexpr bool operator>=(const ignite_date &lhs, const ignite_date &rhs) noexcept {
+constexpr bool operator>=(const ignite_period &lhs, const ignite_period &rhs) noexcept {
     return lhs.compare(rhs) >= 0;
 }
 
diff --git a/modules/platforms/cpp/ignite/common/ignite_time.h b/modules/platforms/cpp/ignite/common/ignite_time.h
index 6cfbdaf24d..b5b6bf8787 100644
--- a/modules/platforms/cpp/ignite/common/ignite_time.h
+++ b/modules/platforms/cpp/ignite/common/ignite_time.h
@@ -53,17 +53,17 @@ public:
     /**
      * Gets the hour-of-day field.
      */
-    [[nodiscard]] constexpr auto get_hour() const noexcept { return m_hour; }
+    [[nodiscard]] constexpr std::int_fast8_t get_hour() const noexcept { return m_hour; }
 
     /**
      * Gets the m_minute-of-m_hour field.
      */
-    [[nodiscard]] constexpr auto get_minute() const noexcept { return m_minute; }
+    [[nodiscard]] constexpr std::int_fast8_t get_minute() const noexcept { return m_minute; }
 
     /**
      * Gets the second-of-m_minute field.
      */
-    [[nodiscard]] constexpr auto get_second() const noexcept { return m_second; }
+    [[nodiscard]] constexpr std::int_fast8_t get_second() const noexcept { return m_second; }
 
     /**
      * Gets the nano-of-second field.
@@ -90,10 +90,10 @@ public:
     }
 
 private:
-    std::int8_t m_hour = 0;
-    std::int8_t m_minute = 0;
-    std::int8_t m_second = 0;
-    std::int32_t m_nano = 0;
+    std::int_least8_t m_hour = 0;
+    std::int_least8_t m_minute = 0;
+    std::int_least8_t m_second = 0;
+    std::int_least32_t m_nano = 0;
 };
 
 /**
diff --git a/modules/platforms/cpp/ignite/client/sql/sql_column_type.h b/modules/platforms/cpp/ignite/common/ignite_type.h
similarity index 81%
rename from modules/platforms/cpp/ignite/client/sql/sql_column_type.h
rename to modules/platforms/cpp/ignite/common/ignite_type.h
index 74e4ebb3f2..a65e0b98ba 100644
--- a/modules/platforms/cpp/ignite/client/sql/sql_column_type.h
+++ b/modules/platforms/cpp/ignite/common/ignite_type.h
@@ -17,12 +17,16 @@
 
 #pragma once
 
+#include <ostream>
+
 namespace ignite {
 
 /**
- * SQL column type.
+ * @brief Supported types for column data.
+ *
+ * @note type of enum value is int, size is same as sizeof(int).
  */
-enum class column_type {
+enum class ignite_type {
     /** Boolean. */
     BOOLEAN = 0,
 
@@ -44,7 +48,7 @@ enum class column_type {
     /** 64-bit double-precision floating-point number. */
     DOUBLE = 6,
 
-    /** A decimal floating-point number. */
+    /** A decimal fixed-point number. */
     DECIMAL = 7,
 
     /** Timezone-free date. */
@@ -84,4 +88,12 @@ enum class column_type {
     UNDEFINED
 };
 
+/**
+ * @brief Writes a ignite_type value to an output stream.
+ */
+inline std::ostream &operator<<(std::ostream &os, const ignite_type t) {
+    os << static_cast<std::underlying_type_t<ignite_type>>(t);
+    return os;
+}
+
 } // namespace ignite
diff --git a/modules/platforms/cpp/ignite/common/uuid.h b/modules/platforms/cpp/ignite/common/uuid.h
index 4268d8d9aa..e87ba2dcd4 100644
--- a/modules/platforms/cpp/ignite/common/uuid.h
+++ b/modules/platforms/cpp/ignite/common/uuid.h
@@ -201,7 +201,7 @@ template<typename C, typename T>
     auto part4 = static_cast<std::uint16_t>(lsb >> 48);
     uint64_t part5 = lsb & 0x0000FFFFFFFFFFFFU;
 
-    std::ios_base::fmtflags savedFlags = os.flags();
+    std::ios_base::fmtflags saved_flags = os.flags();
 
     // clang-format off
     os  << std::hex 
@@ -212,7 +212,7 @@ template<typename C, typename T>
         << std::setfill<C>('0') << std::setw(12) << part5;
     // clang-format on
 
-    os.flags(savedFlags);
+    os.flags(saved_flags);
 
     return os;
 }
@@ -226,9 +226,9 @@ template<typename C, typename T>
  */
 template<typename C, typename T>
 ::std::basic_istream<C, T> &operator>>(std::basic_istream<C, T> &is, uuid &result) {
-    uint64_t parts[5];
+    std::uint64_t parts[5];
 
-    std::ios_base::fmtflags savedFlags = is.flags();
+    std::ios_base::fmtflags saved_flags = is.flags();
 
     is >> std::hex;
 
@@ -237,13 +237,14 @@ template<typename C, typename T>
 
         is >> parts[i] >> delim;
 
-        if (delim != static_cast<C>('-'))
+        if (delim != static_cast<C>('-')) {
             return is;
+        }
     }
 
     is >> parts[4];
 
-    is.flags(savedFlags);
+    is.flags(saved_flags);
 
     result =
         uuid(std::int64_t((parts[0] << 32) | (parts[1] << 16) | parts[2]), std::int64_t((parts[3] << 48) | parts[4]));
diff --git a/modules/platforms/cpp/ignite/schema/ignite_type.cpp b/modules/platforms/cpp/ignite/schema/ignite_type.cpp
deleted file mode 100644
index db4ea6d84f..0000000000
--- a/modules/platforms/cpp/ignite/schema/ignite_type.cpp
+++ /dev/null
@@ -1,80 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-#include "ignite_type.h"
-
-#include <stdexcept>
-#include <string>
-
-namespace ignite {
-
-bool is_fixed_size_type(ignite_type t) {
-    switch (t) {
-        case ignite_type::INT8:
-        case ignite_type::INT16:
-        case ignite_type::INT32:
-        case ignite_type::INT64:
-        case ignite_type::FLOAT:
-        case ignite_type::DOUBLE:
-        case ignite_type::UUID:
-        case ignite_type::DATE:
-        case ignite_type::TIME:
-        case ignite_type::DATETIME:
-        case ignite_type::TIMESTAMP:
-            return true;
-        default:
-            return false;
-    }
-}
-
-SizeT get_type_size(ignite_type t) {
-    switch (t) {
-        case ignite_type::INT8:
-            return 1;
-        case ignite_type::INT16:
-            return 2;
-        case ignite_type::INT32:
-            return 4;
-        case ignite_type::INT64:
-            return 8;
-        case ignite_type::FLOAT:
-            return 4;
-        case ignite_type::DOUBLE:
-            return 8;
-        case ignite_type::UUID:
-            return 16;
-        case ignite_type::DATE:
-            return 3;
-        case ignite_type::TIME:
-            return 5;
-        case ignite_type::DATETIME:
-            return 8;
-        case ignite_type::TIMESTAMP:
-            return 10;
-        case ignite_type::BITMASK:
-        case ignite_type::NUMBER:
-        case ignite_type::DECIMAL:
-        case ignite_type::STRING:
-        case ignite_type::BINARY:
-            /* Only fixed size types are supported for now. */
-            throw std::logic_error("Can't get size of variable-size type id " + std::to_string(static_cast<int>(t)));
-        default:
-            throw std::logic_error("Unsupported type id " + std::to_string(static_cast<int>(t)) + " in schema");
-    }
-}
-
-} // namespace ignite
diff --git a/modules/platforms/cpp/ignite/schema/ignite_type.h b/modules/platforms/cpp/ignite/schema/ignite_type.h
deleted file mode 100644
index 7f27807306..0000000000
--- a/modules/platforms/cpp/ignite/schema/ignite_type.h
+++ /dev/null
@@ -1,79 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-#pragma once
-
-#include "types.h"
-
-#include <ostream>
-
-namespace ignite {
-
-/**
- * @brief Supported types for stored objects.
- *
- * @note type of enum value is int, size is same as sizeof(int).
- */
-enum class ignite_type {
-    INT8 = 1, /**< 1-byte signed integer. */
-    INT16 = 2, /**< 2-byte signed integer. */
-    INT32 = 3, /**< 4-byte signed integer. */
-    INT64 = 4, /**< 8-byte signed integer. */
-    FLOAT = 5, /**< 4-byte floating-point number. */
-    DOUBLE = 6, /**< 8-byte floating-point number. */
-    DECIMAL = 7, /**< Variable-length fixed-point number. */
-    UUID = 8, /**< UUID (16 bytes). */
-    STRING = 9, /**< A UTF-8 encoded string. */
-    BINARY = 10, /**< Variable-size byte array. */
-    BITMASK = 11, /**< A fixed-length bitmask of n bits. */
-    DATE = 12, /**< A timezone-free date encoded into 3 bytes as a year (1 sign bit + 14 bits), month
-               (4 bits), day (5 bits). */
-    TIME = 13, /**< A timezone-free time encoded into 5 bytes as padding (3 bits), hour
-               (5 bits), minute (6 bits), second (6 bits), microsecond (20 bits). */
-    DATETIME = 14, /**< A timezone-free datetime encoded into 8 bytes as (date, time). */
-    TIMESTAMP = 15, /**< Number of microseconds since Jan 1, 1970 00:00:00.000000 (with no
-                    timezone) encoded into 10 bytes. */
-    NUMBER = 16, /**< Variable-length integer number (optionally bound by n bytes in size). */
-    LAST /**< Can be used internally for input data range checks. */
-};
-
-/**
- * @brief Returns true if type has fixed size in bytes.
- *
- * @param t Type to investigate.
- * @return true if type has fixed size.
- * @return false if type has variable size.
- */
-bool is_fixed_size_type(ignite_type t);
-
-/**
- * @brief Returns size of type if it has fixed size.
- *
- * @param t Type to investigate.
- * @return Size in bytes.
- */
-SizeT get_type_size(ignite_type t);
-
-/**
- * @brief Writes a ignite_type value to an output stream.
- */
-inline std::ostream &operator<<(std::ostream &os, const ignite_type t) {
-    os << static_cast<std::underlying_type<ignite_type>::type>(t);
-    return os;
-}
-
-} // namespace ignite
diff --git a/modules/platforms/cpp/ignite/schema/CMakeLists.txt b/modules/platforms/cpp/ignite/tuple/CMakeLists.txt
similarity index 97%
rename from modules/platforms/cpp/ignite/schema/CMakeLists.txt
rename to modules/platforms/cpp/ignite/tuple/CMakeLists.txt
index bc8fade208..ed12f581ed 100644
--- a/modules/platforms/cpp/ignite/schema/CMakeLists.txt
+++ b/modules/platforms/cpp/ignite/tuple/CMakeLists.txt
@@ -22,7 +22,6 @@ set(TARGET ${PROJECT_NAME})
 set(SOURCES
     binary_tuple_builder.cpp
     binary_tuple_parser.cpp
-    ignite_type.cpp
 )
 
 set(PUBLIC_HEADERS
@@ -31,7 +30,6 @@ set(PUBLIC_HEADERS
     binary_tuple_parser.h
     binary_tuple_schema.h
     column_info.h
-    ignite_type.h
     types.h
 )
 
diff --git a/modules/platforms/cpp/ignite/schema/binary_tuple_builder.cpp b/modules/platforms/cpp/ignite/tuple/binary_tuple_builder.cpp
similarity index 76%
rename from modules/platforms/cpp/ignite/schema/binary_tuple_builder.cpp
rename to modules/platforms/cpp/ignite/tuple/binary_tuple_builder.cpp
index ad78dda79b..9fc00bd4d3 100644
--- a/modules/platforms/cpp/ignite/schema/binary_tuple_builder.cpp
+++ b/modules/platforms/cpp/ignite/tuple/binary_tuple_builder.cpp
@@ -60,7 +60,7 @@ void store_time(std::byte *dest, const ignite_time &value, std::size_t size) {
 
 } // namespace
 
-binary_tuple_builder::binary_tuple_builder(IntT element_count) noexcept
+binary_tuple_builder::binary_tuple_builder(number_t element_count) noexcept
     : element_count(element_count) {
 }
 
@@ -98,8 +98,10 @@ void binary_tuple_builder::layout() {
     element_index = 0;
 }
 
-SizeT binary_tuple_builder::gauge(ignite_type type, bytes_view bytes) {
+data_size_t binary_tuple_builder::gauge(ignite_type type, bytes_view bytes) {
     switch (type) {
+        case ignite_type::BOOLEAN:
+            return gauge_bool(binary_tuple_parser::get_bool(bytes));
         case ignite_type::INT8:
             return gauge_int8(binary_tuple_parser::get_int8(bytes));
         case ignite_type::INT16:
@@ -117,9 +119,9 @@ SizeT binary_tuple_builder::gauge(ignite_type type, bytes_view bytes) {
             // For a decimal there is no point to know scale here, treating it as big_integer will do.
             return gauge_number(binary_tuple_parser::get_number(bytes));
         case ignite_type::STRING:
-        case ignite_type::BINARY:
+        case ignite_type::BYTE_ARRAY:
         case ignite_type::BITMASK:
-            return static_cast<SizeT>(bytes.size());
+            return static_cast<data_size_t>(bytes.size());
         case ignite_type::UUID:
             return gauge_uuid(binary_tuple_parser::get_uuid(bytes));
         case ignite_type::DATE:
@@ -130,6 +132,10 @@ SizeT binary_tuple_builder::gauge(ignite_type type, bytes_view bytes) {
             return gauge_date_time(binary_tuple_parser::get_date_time(bytes));
         case ignite_type::TIMESTAMP:
             return gauge_timestamp(binary_tuple_parser::get_timestamp(bytes));
+        case ignite_type::PERIOD:
+            return gauge_period(binary_tuple_parser::get_period(bytes));
+        case ignite_type::DURATION:
+            return gauge_duration(binary_tuple_parser::get_duration(bytes));
         default:
             throw std::logic_error("Unsupported type " + std::to_string(static_cast<int>(type)));
     }
@@ -143,21 +149,33 @@ void binary_tuple_builder::append_bytes(bytes_view bytes) {
     append_entry();
 }
 
+void binary_tuple_builder::append_bool(bytes_view bytes) {
+    data_size_t size = gauge_int8(binary_tuple_parser::get_int8(bytes));
+    assert(size <= bytes.size());
+    append_bytes({bytes.data(), size});
+}
+
+void binary_tuple_builder::append_bool(bool value) {
+    static std::byte true_byte{1};
+    data_size_t size = gauge_bool(value);
+    append_bytes({&true_byte, size});
+}
+
 void binary_tuple_builder::append_int8(bytes_view bytes) {
-    SizeT size = gauge_int8(binary_tuple_parser::get_int8(bytes));
+    data_size_t size = gauge_int8(binary_tuple_parser::get_int8(bytes));
     assert(size <= bytes.size());
     append_bytes({bytes.data(), size});
 }
 
 void binary_tuple_builder::append_int8(std::int8_t value) {
-    SizeT size = gauge_int8(value);
+    data_size_t size = gauge_int8(value);
     append_bytes({reinterpret_cast<const std::byte *>(&value), size});
 }
 
 void binary_tuple_builder::append_int16(bytes_view bytes) {
     auto value = binary_tuple_parser::get_int16(bytes);
 
-    SizeT size = gauge_int16(value);
+    data_size_t size = gauge_int16(value);
     assert(size <= bytes.size());
 
     if constexpr (is_little_endian_platform()) {
@@ -169,7 +187,7 @@ void binary_tuple_builder::append_int16(bytes_view bytes) {
 }
 
 void binary_tuple_builder::append_int16(std::int16_t value) {
-    SizeT size = gauge_int16(value);
+    data_size_t size = gauge_int16(value);
 
     if constexpr (!is_little_endian_platform()) {
         value = bytes::reverse(value);
@@ -180,7 +198,7 @@ void binary_tuple_builder::append_int16(std::int16_t value) {
 void binary_tuple_builder::append_int32(bytes_view bytes) {
     auto value = binary_tuple_parser::get_int32(bytes);
 
-    SizeT size = gauge_int32(value);
+    data_size_t size = gauge_int32(value);
     assert(size <= bytes.size());
 
     if constexpr (is_little_endian_platform()) {
@@ -192,7 +210,7 @@ void binary_tuple_builder::append_int32(bytes_view bytes) {
 }
 
 void binary_tuple_builder::append_int32(std::int32_t value) {
-    SizeT size = gauge_int32(value);
+    data_size_t size = gauge_int32(value);
 
     if constexpr (!is_little_endian_platform()) {
         value = bytes::reverse(value);
@@ -203,7 +221,7 @@ void binary_tuple_builder::append_int32(std::int32_t value) {
 void binary_tuple_builder::append_int64(bytes_view bytes) {
     auto value = binary_tuple_parser::get_int64(bytes);
 
-    SizeT size = gauge_int64(value);
+    data_size_t size = gauge_int64(value);
     assert(size <= bytes.size());
 
     if constexpr (is_little_endian_platform()) {
@@ -215,7 +233,7 @@ void binary_tuple_builder::append_int64(bytes_view bytes) {
 }
 
 void binary_tuple_builder::append_int64(std::int64_t value) {
-    SizeT size = gauge_int64(value);
+    data_size_t size = gauge_int64(value);
 
     if constexpr (!is_little_endian_platform()) {
         value = bytes::reverse(value);
@@ -229,7 +247,7 @@ void binary_tuple_builder::append_float(bytes_view bytes) {
 }
 
 void binary_tuple_builder::append_float(float value) {
-    SizeT size = gauge_float(value);
+    data_size_t size = gauge_float(value);
 
     assert(element_index < element_count);
     assert(next_value + size <= value_base + value_area_size);
@@ -249,7 +267,7 @@ void binary_tuple_builder::append_double(bytes_view bytes) {
 }
 
 void binary_tuple_builder::append_double(double value) {
-    SizeT size = gauge_double(value);
+    data_size_t size = gauge_double(value);
     assert(element_index < element_count);
     assert(next_value + size <= value_base + value_area_size);
 
@@ -273,7 +291,7 @@ void binary_tuple_builder::append_number(bytes_view bytes) {
 }
 
 void binary_tuple_builder::append_number(const big_integer &value) {
-    SizeT size = gauge_number(value);
+    data_size_t size = gauge_number(value);
     assert(element_index < element_count);
     assert(next_value + size <= value_base + value_area_size);
 
@@ -286,7 +304,7 @@ void binary_tuple_builder::append_number(const big_integer &value) {
 }
 
 void binary_tuple_builder::append_number(const big_decimal &value) {
-    SizeT size = gauge_number(value);
+    data_size_t size = gauge_number(value);
     assert(element_index < element_count);
     assert(next_value + size <= value_base + value_area_size);
 
@@ -304,7 +322,7 @@ void binary_tuple_builder::append_uuid(bytes_view bytes) {
 }
 
 void binary_tuple_builder::append_uuid(uuid value) {
-    SizeT size = gauge_uuid(value);
+    data_size_t size = gauge_uuid(value);
     assert(element_index < element_count);
     assert(next_value + size <= value_base + value_area_size);
 
@@ -324,7 +342,7 @@ void binary_tuple_builder::append_date(bytes_view bytes) {
 }
 
 void binary_tuple_builder::append_date(const ignite_date &value) {
-    SizeT size = gauge_date(value);
+    data_size_t size = gauge_date(value);
     assert(element_index < element_count);
     assert(next_value + size <= value_base + value_area_size);
 
@@ -343,7 +361,7 @@ void binary_tuple_builder::append_time(bytes_view bytes) {
 }
 
 void binary_tuple_builder::append_time(const ignite_time &value) {
-    SizeT size = gauge_time(value);
+    data_size_t size = gauge_time(value);
     assert(element_index < element_count);
     assert(next_value + size <= value_base + value_area_size);
 
@@ -362,7 +380,7 @@ void binary_tuple_builder::append_date_time(bytes_view bytes) {
 }
 
 void binary_tuple_builder::append_date_time(const ignite_date_time &value) {
-    SizeT size = gauge_date_time(value);
+    data_size_t size = gauge_date_time(value);
     assert(element_index < element_count);
     assert(next_value + size <= value_base + value_area_size);
 
@@ -382,7 +400,7 @@ void binary_tuple_builder::append_timestamp(bytes_view bytes) {
 }
 
 void binary_tuple_builder::append_timestamp(const ignite_timestamp &value) {
-    SizeT size = gauge_timestamp(value);
+    data_size_t size = gauge_timestamp(value);
     assert(element_index < element_count);
     assert(next_value + size <= value_base + value_area_size);
 
@@ -398,8 +416,63 @@ void binary_tuple_builder::append_timestamp(const ignite_timestamp &value) {
     append_entry();
 }
 
+void binary_tuple_builder::append_period(bytes_view bytes) {
+    auto value = binary_tuple_parser::get_period(bytes);
+    append_period(value);
+}
+
+void binary_tuple_builder::append_period(const ignite_period &value) {
+    data_size_t size = gauge_period(value);
+    assert(element_index < element_count);
+    assert(next_value + size <= value_base + value_area_size);
+
+    if (size != 0) {
+        assert(size == 3 || size == 6 || size == 12);
+        if (size == 3) {
+            bytes::store<endian::LITTLE>(next_value, std::uint8_t(value.get_years()));
+            bytes::store<endian::LITTLE>(next_value + 1, std::uint8_t(value.get_months()));
+            bytes::store<endian::LITTLE>(next_value + 2, std::uint8_t(value.get_days()));
+        } else if (size == 6) {
+            bytes::store<endian::LITTLE>(next_value, std::uint16_t(value.get_years()));
+            bytes::store<endian::LITTLE>(next_value + 2, std::uint16_t(value.get_months()));
+            bytes::store<endian::LITTLE>(next_value + 4, std::uint16_t(value.get_days()));
+        } else {
+            bytes::store<endian::LITTLE>(next_value, value.get_years());
+            bytes::store<endian::LITTLE>(next_value + 4, value.get_months());
+            bytes::store<endian::LITTLE>(next_value + 8, value.get_days());
+        }
+        next_value += size;
+    }
+
+    append_entry();
+}
+
+void binary_tuple_builder::append_duration(bytes_view bytes) {
+    auto value = binary_tuple_parser::get_duration(bytes);
+    append_duration(value);
+}
+
+void binary_tuple_builder::append_duration(const ignite_duration &value) {
+    data_size_t size = gauge_duration(value);
+    assert(element_index < element_count);
+    assert(next_value + size <= value_base + value_area_size);
+
+    if (size != 0) {
+        assert(size == 8 || size == 12);
+        bytes::store<endian::LITTLE>(next_value, value.get_seconds());
+        if (size == 12) {
+            bytes::store<endian::LITTLE>(next_value + 8, value.get_nano());
+        }
+        next_value += size;
+    }
+
+    append_entry();
+}
+
 void binary_tuple_builder::append(ignite_type type, const bytes_view &bytes) {
     switch (type) {
+        case ignite_type::BOOLEAN:
+            return append_bool(bytes);
         case ignite_type::INT8:
             return append_int8(bytes);
         case ignite_type::INT16:
@@ -417,7 +490,7 @@ void binary_tuple_builder::append(ignite_type type, const bytes_view &bytes) {
             // For a decimal there is no point to know scale here, treating it as big_integer will do.
             return append_number(bytes);
         case ignite_type::STRING:
-        case ignite_type::BINARY:
+        case ignite_type::BYTE_ARRAY:
         case ignite_type::BITMASK:
             return append_bytes(bytes);
         case ignite_type::UUID:
@@ -430,6 +503,10 @@ void binary_tuple_builder::append(ignite_type type, const bytes_view &bytes) {
             return append_date_time(bytes);
         case ignite_type::TIMESTAMP:
             return append_timestamp(bytes);
+        case ignite_type::PERIOD:
+            return append_period(bytes);
+        case ignite_type::DURATION:
+            return append_duration(bytes);
         default:
             throw std::logic_error("Unsupported type " + std::to_string(static_cast<int>(type)));
     }
diff --git a/modules/platforms/cpp/ignite/schema/binary_tuple_builder.h b/modules/platforms/cpp/ignite/tuple/binary_tuple_builder.h
similarity index 76%
rename from modules/platforms/cpp/ignite/schema/binary_tuple_builder.h
rename to modules/platforms/cpp/ignite/tuple/binary_tuple_builder.h
index e0ab436f47..326a65dfb9 100644
--- a/modules/platforms/cpp/ignite/schema/binary_tuple_builder.h
+++ b/modules/platforms/cpp/ignite/tuple/binary_tuple_builder.h
@@ -18,15 +18,17 @@
 #pragma once
 
 #include "binary_tuple_schema.h"
-#include "ignite/common/big_decimal.h"
-#include "ignite/common/big_integer.h"
-#include "ignite/common/ignite_date.h"
-#include "ignite/common/ignite_date_time.h"
-#include "ignite/common/ignite_time.h"
-#include "ignite/common/ignite_timestamp.h"
 
+#include <ignite/common/big_decimal.h>
+#include <ignite/common/big_integer.h>
 #include <ignite/common/bytes.h>
 #include <ignite/common/bytes_view.h>
+#include <ignite/common/ignite_date.h>
+#include <ignite/common/ignite_date_time.h>
+#include <ignite/common/ignite_duration.h>
+#include <ignite/common/ignite_period.h>
+#include <ignite/common/ignite_time.h>
+#include <ignite/common/ignite_timestamp.h>
 #include <ignite/common/uuid.h>
 
 #include <cassert>
@@ -55,15 +57,15 @@ namespace ignite {
  * 5. Finally, the resulting binary tuple is obtained with the @ref build call.
  */
 class binary_tuple_builder {
-    const IntT element_count; /**< Total number of elements. */
+    const number_t element_count; /**< Total number of elements. */
 
-    IntT element_index; /**< Index of the next element to add. */
+    number_t element_index; /**< Index of the next element to add. */
 
-    IntT null_elements; /**< The number of null elements. */
+    number_t null_elements; /**< The number of null elements. */
 
-    SizeT value_area_size; /**< Total size of all values. */
+    data_size_t value_area_size; /**< Total size of all values. */
 
-    SizeT entry_size; /**< Size of an offset table entry. */
+    data_size_t entry_size; /**< Size of an offset table entry. */
 
     std::byte *next_entry; /**< Position for the next offset table entry. */
 
@@ -79,7 +81,7 @@ public:
      *
      * @param schema Binary tuple schema.
      */
-    explicit binary_tuple_builder(IntT element_count) noexcept;
+    explicit binary_tuple_builder(number_t element_count) noexcept;
 
     /**
      * @brief Starts a new tuple.
@@ -98,14 +100,21 @@ public:
     /**
      * @brief Assigns a binary value for the next element.
      *
-     * @param valueSize required size for the value
+     * @param size required size for the value
      */
-    void claim(SizeT valueSize) noexcept {
+    void claim(data_size_t size) noexcept {
         assert(element_index < element_count);
-        value_area_size += valueSize;
+        value_area_size += size;
         element_index++;
     }
 
+    /**
+     * @brief Assigns a binary value for the next element.
+     *
+     * @param value Element value.
+     */
+    void claim_bool(bool value) noexcept { claim(gauge_bool(value)); }
+
     /**
      * @brief Assigns a binary value for the next element.
      *
@@ -202,14 +211,28 @@ public:
      *
      * @param value Element value.
      */
-    void claim_string(const std::string &value) noexcept { claim(SizeT(value.size())); }
+    void claim_period(const ignite_period &value) noexcept { claim(gauge_period(value)); }
+
+    /**
+     * @brief Assigns a binary value for the next element.
+     *
+     * @param value Element value.
+     */
+    void claim_duration(const ignite_duration &value) noexcept { claim(gauge_duration(value)); }
 
     /**
      * @brief Assigns a binary value for the next element.
      *
      * @param value Element value.
      */
-    void claim_bytes(const bytes_view &value) noexcept { claim(SizeT(value.size())); }
+    void claim_string(const std::string &value) noexcept { claim(data_size_t(value.size())); }
+
+    /**
+     * @brief Assigns a binary value for the next element.
+     *
+     * @param value Element value.
+     */
+    void claim_bytes(const bytes_view &value) noexcept { claim(data_size_t(value.size())); }
 
     /**
      * @brief Assigns a binary value for the next element.
@@ -244,8 +267,8 @@ public:
      */
     template<typename BytesT>
     void claim(const binary_tuple_schema &schema, const std::vector<std::optional<BytesT>> &tuple) noexcept {
-        for (IntT i = 0; i < schema.num_elements(); i++) {
-            claim(schema.get_element(i).dataType, tuple[i]);
+        for (number_t i = 0; i < schema.num_elements(); i++) {
+            claim(schema.get_element(i).type, tuple[i]);
         }
     }
 
@@ -298,8 +321,8 @@ public:
      */
     template<typename BytesT>
     void append(const binary_tuple_schema &schema, const std::vector<std::optional<BytesT>> &tuple) {
-        for (IntT i = 0; i < schema.num_elements(); i++) {
-            append(schema.get_element(i).dataType, tuple[i]);
+        for (number_t i = 0; i < schema.num_elements(); i++) {
+            append(schema.get_element(i).type, tuple[i]);
         }
     }
 
@@ -310,6 +333,24 @@ public:
      */
     void append_bytes(bytes_view bytes);
 
+    /**
+     * @brief Writes binary value of specified element.
+     *
+     * The written value may differ from the original because of value compression.
+     *
+     * @param bytes Binary element value.
+     */
+    void append_bool(bytes_view bytes);
+
+    /**
+     * @brief Writes binary value of specified element.
+     *
+     * The written value may differ from the original because of value compression.
+     *
+     * @param bytes Binary element value.
+     */
+    void append_bool(bool value);
+
     /**
      * @brief Writes binary value of specified element.
      *
@@ -535,6 +576,42 @@ public:
      */
     void append_timestamp(const ignite_timestamp &value);
 
+    /**
+     * @brief Writes binary value of specified element.
+     *
+     * The written value may differ from the original because of value compression.
+     *
+     * @param bytes Binary element value.
+     */
+    void append_period(bytes_view bytes);
+
+    /**
+     * @brief Writes binary value of specified element.
+     *
+     * The written value may differ from the original because of value compression.
+     *
+     * @param value Time value.
+     */
+    void append_period(const ignite_period &value);
+
+    /**
+     * @brief Writes binary value of specified element.
+     *
+     * The written value may differ from the original because of value compression.
+     *
+     * @param bytes Binary element value.
+     */
+    void append_duration(bytes_view bytes);
+
+    /**
+     * @brief Writes binary value of specified element.
+     *
+     * The written value may differ from the original because of value compression.
+     *
+     * @param value Time value.
+     */
+    void append_duration(const ignite_duration &value);
+
     /**
      * @brief Appends a string as the next element.
      *
@@ -576,16 +653,17 @@ private:
     /**
      * @brief Checks if a value of a given integer type can be compressed to a smaller integer type.
      *
-     * @tparam SRC Source integer type.
      * @tparam TGT Target integer type.
+     * @tparam SRC Source integer type.
      * @param value Source value.
      * @return true If the source value can be compressed.
      * @return false If the source value cannot be compressed.
      */
-    template<typename SRC, typename TGT>
+    template<typename TGT, typename SRC>
     static bool fits(SRC value) noexcept {
         static_assert(std::is_signed_v<SRC>);
         static_assert(std::is_signed_v<TGT>);
+        static_assert(sizeof(TGT) < sizeof(SRC));
         // Check if TGT::min <= value <= TGT::max.
         return std::make_unsigned_t<SRC>(value + std::numeric_limits<TGT>::max() + 1)
             <= std::numeric_limits<std::make_unsigned_t<TGT>>::max();
@@ -597,7 +675,7 @@ private:
      * @param value Actual element value.
      * @return Required size.
      */
-    static SizeT gauge_int8(std::int8_t value) noexcept { return value == 0 ? 0 : sizeof(std::int8_t); }
+    static data_size_t gauge_bool(bool value) noexcept { return !value ? 0 : sizeof(std::int8_t); }
 
     /**
      * @brief Computes required binary size for a given value.
@@ -605,8 +683,16 @@ private:
      * @param value Actual element value.
      * @return Required size.
      */
-    static SizeT gauge_int16(std::int16_t value) noexcept {
-        if (fits<std::int16_t, std::int8_t>(value)) {
+    static data_size_t gauge_int8(std::int8_t value) noexcept { return value == 0 ? 0 : sizeof(std::int8_t); }
+
+    /**
+     * @brief Computes required binary size for a given value.
+     *
+     * @param value Actual element value.
+     * @return Required size.
+     */
+    static data_size_t gauge_int16(std::int16_t value) noexcept {
+        if (fits<std::int8_t>(value)) {
             return gauge_int8(std::int8_t(value));
         }
         return sizeof(std::int16_t);
@@ -618,11 +704,11 @@ private:
      * @param value Actual element value.
      * @return Required size.
      */
-    static SizeT gauge_int32(std::int32_t value) noexcept {
-        if (fits<std::int32_t, std::int8_t>(value)) {
+    static data_size_t gauge_int32(std::int32_t value) noexcept {
+        if (fits<std::int8_t>(value)) {
             return gauge_int8(std::int8_t(value));
         }
-        if (fits<std::int32_t, std::int16_t>(value)) {
+        if (fits<std::int16_t>(value)) {
             return sizeof(std::int16_t);
         }
         return sizeof(std::int32_t);
@@ -634,11 +720,11 @@ private:
      * @param value Actual element value.
      * @return Required size.
      */
-    static SizeT gauge_int64(std::int64_t value) noexcept {
-        if (fits<std::int64_t, std::int16_t>(value)) {
+    static data_size_t gauge_int64(std::int64_t value) noexcept {
+        if (fits<std::int16_t>(value)) {
             return gauge_int16(std::int16_t(value));
         }
-        if (fits<std::int64_t, std::int32_t>(value)) {
+        if (fits<std::int32_t>(value)) {
             return sizeof(std::int32_t);
         }
         return sizeof(std::int64_t);
@@ -650,7 +736,7 @@ private:
      * @param value Actual element value.
      * @return Required size.
      */
-    static SizeT gauge_float(float value) noexcept { return value == 0.0f ? 0 : sizeof(float); }
+    static data_size_t gauge_float(float value) noexcept { return value == 0.0f ? 0 : sizeof(float); }
 
     /**
      * @brief Computes required binary size for a given value.
@@ -658,7 +744,7 @@ private:
      * @param value Actual element value.
      * @return Required size.
      */
-    static SizeT gauge_double(double value) noexcept {
+    static data_size_t gauge_double(double value) noexcept {
         auto floatValue = static_cast<float>(value);
         return floatValue == value ? gauge_float(floatValue) : sizeof(double);
     }
@@ -669,8 +755,8 @@ private:
      * @param value Actual element value.
      * @return Required size.
      */
-    static SizeT gauge_number(const big_integer &value) noexcept {
-        return SizeT(value.is_zero() ? 0 : value.byte_size());
+    static data_size_t gauge_number(const big_integer &value) noexcept {
+        return data_size_t(value.is_zero() ? 0 : value.byte_size());
     }
 
     /**
@@ -679,8 +765,8 @@ private:
      * @param value Actual element value.
      * @return Required size.
      */
-    static SizeT gauge_number(const big_decimal &value) noexcept {
-        return SizeT(value.is_zero() ? 0 : value.get_unscaled_value().byte_size());
+    static data_size_t gauge_number(const big_decimal &value) noexcept {
+        return data_size_t(value.is_zero() ? 0 : value.get_unscaled_value().byte_size());
     }
 
     /**
@@ -689,7 +775,7 @@ private:
      * @param value Actual element value.
      * @return Required size.
      */
-    static SizeT gauge_uuid(const uuid &value) noexcept { return value == uuid() ? 0 : 16; }
+    static data_size_t gauge_uuid(const uuid &value) noexcept { return value == uuid() ? 0 : 16; }
 
     /**
      * @brief Computes required binary size for a given value.
@@ -697,7 +783,7 @@ private:
      * @param value Actual element value.
      * @return Required size.
      */
-    static SizeT gauge_date(const ignite_date &value) noexcept { return value == ignite_date() ? 0 : 3; }
+    static data_size_t gauge_date(const ignite_date &value) noexcept { return value == ignite_date() ? 0 : 3; }
 
     /**
      * @brief Computes required binary size for a given value.
@@ -705,7 +791,7 @@ private:
      * @param value Actual element value.
      * @return Required size.
      */
-    static SizeT gauge_time(const ignite_time &value) noexcept {
+    static data_size_t gauge_time(const ignite_time &value) noexcept {
         if (value == ignite_time()) {
             return 0;
         }
@@ -725,7 +811,7 @@ private:
      * @param value Actual element value.
      * @return Required size.
      */
-    static SizeT gauge_date_time(const ignite_date_time &value) noexcept {
+    static data_size_t gauge_date_time(const ignite_date_time &value) noexcept {
         if (value == ignite_date_time()) {
             return 0;
         }
@@ -745,10 +831,42 @@ private:
      * @param value Actual element value.
      * @return Required size.
      */
-    static SizeT gauge_timestamp(const ignite_timestamp &value) noexcept {
+    static data_size_t gauge_timestamp(const ignite_timestamp &value) noexcept {
         return value == ignite_timestamp() ? 0 : value.get_nano() == 0 ? 8 : 12;
     }
 
+    /**
+     * @brief Computes required binary size for a given value.
+     *
+     * @param value Actual element value.
+     * @return Required size.
+     */
+    static data_size_t gauge_period(const ignite_period &value) noexcept {
+        const auto y = value.get_years();
+        const auto m = value.get_months();
+        const auto d = value.get_days();
+        if (fits<std::int8_t>(y) && fits<std::int8_t>(m) && fits<std::int8_t>(d)) {
+            if (value == ignite_period()) {
+                return 0;
+            }
+            return 3;
+        }
+        if (fits<std::int16_t>(y) && fits<std::int16_t>(m) && fits<std::int16_t>(d)) {
+            return 6;
+        }
+        return 12;
+    }
+
+    /**
+     * @brief Computes required binary size for a given value.
+     *
+     * @param value Actual element value.
+     * @return Required size.
+     */
+    static data_size_t gauge_duration(const ignite_duration &value) noexcept {
+        return value == ignite_duration() ? 0 : value.get_nano() == 0 ? 8 : 12;
+    }
+
     /**
      * @brief Computes required binary size for a given value.
      *
@@ -756,7 +874,7 @@ private:
      * @param bytes Binary element value.
      * @return Required size.
      */
-    static SizeT gauge(ignite_type type, bytes_view bytes);
+    static data_size_t gauge(ignite_type type, bytes_view bytes);
 
     /**
      * @brief Adds an entry to the offset table.
diff --git a/modules/platforms/cpp/ignite/schema/binary_tuple_header.h b/modules/platforms/cpp/ignite/tuple/binary_tuple_header.h
similarity index 88%
rename from modules/platforms/cpp/ignite/schema/binary_tuple_header.h
rename to modules/platforms/cpp/ignite/tuple/binary_tuple_header.h
index 319873db9f..45e0aca146 100644
--- a/modules/platforms/cpp/ignite/schema/binary_tuple_header.h
+++ b/modules/platforms/cpp/ignite/tuple/binary_tuple_header.h
@@ -43,7 +43,7 @@ struct binary_tuple_header {
     std::byte flags{0};
 
     /** Encodes size as bit mask. */
-    static constexpr unsigned int size_to_flags(SizeT size) noexcept {
+    static constexpr unsigned int size_to_flags(data_size_t size) noexcept {
         if (size <= UINT8_MAX) {
             return 0b00;
         } else if (size <= UINT16_MAX) {
@@ -54,7 +54,7 @@ struct binary_tuple_header {
     }
 
     /** Sets the size of offset-table entries based on the value area size. */
-    unsigned int set_entry_size(SizeT value_area_size) noexcept {
+    unsigned int set_entry_size(data_size_t value_area_size) noexcept {
         const unsigned size_log2 = size_to_flags(value_area_size);
         flags &= ~VARLEN_ENTRY_SIZE_MASK;
         flags |= std::byte(size_log2);
@@ -62,7 +62,7 @@ struct binary_tuple_header {
     }
 
     /** Gets the size of a single offset-table entry, in bytes. */
-    SizeT get_entry_size() const noexcept { return 1u << static_cast<unsigned>(flags & VARLEN_ENTRY_SIZE_MASK); }
+    data_size_t get_entry_size() const noexcept { return 1u << static_cast<unsigned>(flags & VARLEN_ENTRY_SIZE_MASK); }
 
     /** Sets the nullmap flag on. */
     void set_nullmap_flag() noexcept { flags |= NULLMAP_FLAG; }
diff --git a/modules/platforms/cpp/ignite/schema/binary_tuple_parser.cpp b/modules/platforms/cpp/ignite/tuple/binary_tuple_parser.cpp
similarity index 78%
rename from modules/platforms/cpp/ignite/schema/binary_tuple_parser.cpp
rename to modules/platforms/cpp/ignite/tuple/binary_tuple_parser.cpp
index 28ad95c048..31f256b360 100644
--- a/modules/platforms/cpp/ignite/schema/binary_tuple_parser.cpp
+++ b/modules/platforms/cpp/ignite/tuple/binary_tuple_parser.cpp
@@ -73,7 +73,7 @@ ignite_time load_time(bytes_view bytes) {
 
 } // namespace
 
-binary_tuple_parser::binary_tuple_parser(IntT num_elements, bytes_view data)
+binary_tuple_parser::binary_tuple_parser(number_t num_elements, bytes_view data)
     : binary_tuple(data)
     , element_count(num_elements)
     , element_index(0)
@@ -86,12 +86,12 @@ binary_tuple_parser::binary_tuple_parser(IntT num_elements, bytes_view data)
     entry_size = header.get_entry_size();
     has_nullmap = header.get_nullmap_flag();
 
-    SizeT nullmap_size = 0;
+    data_size_t nullmap_size = 0;
     if (has_nullmap) {
         nullmap_size = binary_tuple_schema::get_nullmap_size(element_count);
     }
 
-    SizeT table_size = entry_size * element_count;
+    data_size_t table_size = entry_size * element_count;
     next_entry = binary_tuple.data() + binary_tuple_header::SIZE + nullmap_size;
     value_base = next_entry + table_size;
 
@@ -115,7 +115,7 @@ binary_tuple_parser::binary_tuple_parser(IntT num_elements, bytes_view data)
     }
 }
 
-element_view binary_tuple_parser::get_next() {
+value_view binary_tuple_parser::get_next() {
     assert(num_parsed_elements() < num_elements());
 
     ++element_index;
@@ -136,10 +136,10 @@ element_view binary_tuple_parser::get_next() {
     return bytes_view(value, length);
 }
 
-tuple_view binary_tuple_parser::parse(IntT num) {
+tuple_view binary_tuple_parser::parse(number_t num) {
     assert(element_index == 0);
 
-    if (num == NO_NUM) {
+    if (num == NOT_NUM) {
         num = num_elements();
     }
 
@@ -152,10 +152,10 @@ tuple_view binary_tuple_parser::parse(IntT num) {
     return tuple;
 }
 
-key_tuple_view binary_tuple_parser::parse_key(IntT num) {
+key_tuple_view binary_tuple_parser::parse_key(number_t num) {
     assert(element_index == 0);
 
-    if (num == NO_NUM) {
+    if (num == NOT_NUM) {
         num = num_elements();
     }
 
@@ -168,6 +168,17 @@ key_tuple_view binary_tuple_parser::parse_key(IntT num) {
     return key;
 }
 
+bool binary_tuple_parser::get_bool(bytes_view bytes) {
+    switch (bytes.size()) {
+        case 0:
+            return false;
+        case 1:
+            return load_as<std::int8_t>(bytes) != 0;
+        default:
+            throw std::out_of_range("Bad element size");
+    }
+}
+
 std::int8_t binary_tuple_parser::get_int8(bytes_view bytes) {
     switch (bytes.size()) {
         case 0:
@@ -314,8 +325,8 @@ ignite_timestamp binary_tuple_parser::get_timestamp(bytes_view bytes) {
             return ignite_timestamp(seconds, 0);
         }
         case 12: {
-            std::int64_t seconds = load_as<std::int64_t>(bytes);
-            std::int32_t nanos = load_as<std::int32_t>(bytes, 8);
+            auto seconds = load_as<std::int64_t>(bytes);
+            auto nanos = load_as<std::int32_t>(bytes, 8);
             return ignite_timestamp(seconds, nanos);
         }
         default:
@@ -323,4 +334,49 @@ ignite_timestamp binary_tuple_parser::get_timestamp(bytes_view bytes) {
     }
 }
 
+ignite_period binary_tuple_parser::get_period(bytes_view bytes) {
+    switch (bytes.size()) {
+        case 0:
+            return ignite_period();
+        case 3: {
+            auto years = load_as<std::int8_t>(bytes, 8);
+            auto months = load_as<std::int8_t>(bytes, 8);
+            auto days = load_as<std::int8_t>(bytes, 8);
+            return ignite_period(years, months, days);
+        }
+        case 6: {
+            auto years = load_as<std::int16_t>(bytes, 8);
+            auto months = load_as<std::int16_t>(bytes, 8);
+            auto days = load_as<std::int16_t>(bytes, 8);
+            return ignite_period(years, months, days);
+        }
+        case 12: {
+            auto years = load_as<std::int32_t>(bytes, 8);
+            auto months = load_as<std::int32_t>(bytes, 8);
+            auto days = load_as<std::int32_t>(bytes, 8);
+            return ignite_period(years, months, days);
+        }
+        default:
+            throw std::out_of_range("Bad element size");
+    }
+}
+
+ignite_duration binary_tuple_parser::get_duration(bytes_view bytes) {
+    switch (bytes.size()) {
+        case 0:
+            return ignite_duration();
+        case 8: {
+            auto seconds = load_as<std::int64_t>(bytes);
+            return ignite_duration(seconds, 0);
+        }
+        case 12: {
+            auto seconds = load_as<std::int64_t>(bytes);
+            auto nanos = load_as<std::int32_t>(bytes, 8);
+            return ignite_duration(seconds, nanos);
+        }
+        default:
+            throw std::out_of_range("Bad element size");
+    }
+}
+
 } // namespace ignite
diff --git a/modules/platforms/cpp/ignite/schema/binary_tuple_parser.h b/modules/platforms/cpp/ignite/tuple/binary_tuple_parser.h
similarity index 75%
rename from modules/platforms/cpp/ignite/schema/binary_tuple_parser.h
rename to modules/platforms/cpp/ignite/tuple/binary_tuple_parser.h
index 1eb0e88826..938febebc4 100644
--- a/modules/platforms/cpp/ignite/schema/binary_tuple_parser.h
+++ b/modules/platforms/cpp/ignite/tuple/binary_tuple_parser.h
@@ -18,14 +18,16 @@
 #pragma once
 
 #include "binary_tuple_schema.h"
-#include "ignite/common/big_decimal.h"
-#include "ignite/common/big_integer.h"
-#include "ignite/common/ignite_date.h"
-#include "ignite/common/ignite_date_time.h"
-#include "ignite/common/ignite_time.h"
-#include "ignite/common/ignite_timestamp.h"
 
+#include <ignite/common/big_decimal.h>
+#include <ignite/common/big_integer.h>
 #include <ignite/common/bytes_view.h>
+#include <ignite/common/ignite_date.h>
+#include <ignite/common/ignite_date_time.h>
+#include <ignite/common/ignite_duration.h>
+#include <ignite/common/ignite_period.h>
+#include <ignite/common/ignite_time.h>
+#include <ignite/common/ignite_timestamp.h>
 #include <ignite/common/uuid.h>
 
 namespace ignite {
@@ -38,13 +40,13 @@ namespace ignite {
 class binary_tuple_parser {
     bytes_view binary_tuple; /**< The binary tuple to parse. */
 
-    const IntT element_count; /**< Total number of elements. */
+    const number_t element_count; /**< Total number of elements. */
 
-    IntT element_index; /**< Index of the next element to parse. */
+    number_t element_index; /**< Index of the next element to parse. */
 
     bool has_nullmap; /**< Flag that indicates if the tuple contains a nullmap. */
 
-    SizeT entry_size; /**< Size of an offset table entry. */
+    data_size_t entry_size; /**< Size of an offset table entry. */
 
     const std::byte *next_entry; /**< Position of the next offset table entry. */
 
@@ -64,7 +66,7 @@ public:
      * @param num_elements Number of tuple elements.
      * @param data Binary tuple buffer.
      */
-    explicit binary_tuple_parser(IntT num_elements, bytes_view data);
+    explicit binary_tuple_parser(number_t num_elements, bytes_view data);
 
     /**
      * @brief Gets the original binary tuple.
@@ -78,44 +80,52 @@ public:
      *
      * @return Tuple size.
      */
-    SizeT get_size() const noexcept { return static_cast<SizeT>(binary_tuple.size()); }
+    data_size_t get_size() const noexcept { return static_cast<data_size_t>(binary_tuple.size()); }
 
     /**
      * @brief Gets the expected total number of tuple elements.
      *
      * @return Number of elements.
      */
-    IntT num_elements() const noexcept { return element_count; }
+    number_t num_elements() const noexcept { return element_count; }
 
     /**
      * @brief Gets the number of parsed tuple elements.
      *
      * @return Number of parsed elements.
      */
-    IntT num_parsed_elements() const noexcept { return element_index; }
+    number_t num_parsed_elements() const noexcept { return element_index; }
 
     /**
      * @brief Gets the next value of the tuple.
      *
      * @return The next value.
      */
-    element_view get_next();
+    value_view get_next();
 
     /**
      * @brief Gets a series of values.
      *
-     * @param num Required number of values. The value of NO_NUM means all the available values.
+     * @param num Required number of values. The value of NOT_NUM means all the available values.
      * @return A set of values.
      */
-    tuple_view parse(IntT num = NO_NUM);
+    tuple_view parse(number_t num = NOT_NUM);
 
     /**
      * @brief Gets a series of values presuming they belong to a key. So no NULL values are allowed.
      *
-     * @param num Required number of values. The value of NO_NUM means all the available values.
+     * @param num Required number of values. The value of NOT_NUM means all the available values.
      * @return A set of values.
      */
-    key_tuple_view parse_key(IntT num = NO_NUM);
+    key_tuple_view parse_key(number_t num = NOT_NUM);
+
+    /**
+     * @brief Reads value of specified element.
+     *
+     * @param bytes Binary view of the element.
+     * @return Element value.
+     */
+    static bool get_bool(bytes_view bytes);
 
     /**
      * @brief Reads value of specified element.
@@ -221,6 +231,22 @@ public:
      * @return Element value.
      */
     static ignite_timestamp get_timestamp(bytes_view bytes);
+
+    /**
+     * @brief Reads value of specified element.
+     *
+     * @param bytes Binary view of the element.
+     * @return Element value.
+     */
+    static ignite_period get_period(bytes_view bytes);
+
+    /**
+     * @brief Reads value of specified element.
+     *
+     * @param bytes Binary view of the element.
+     * @return Element value.
+     */
+    static ignite_duration get_duration(bytes_view bytes);
 };
 
 } // namespace ignite
diff --git a/modules/platforms/cpp/ignite/schema/binary_tuple_schema.h b/modules/platforms/cpp/ignite/tuple/binary_tuple_schema.h
similarity index 85%
rename from modules/platforms/cpp/ignite/schema/binary_tuple_schema.h
rename to modules/platforms/cpp/ignite/tuple/binary_tuple_schema.h
index 93e0f0ff84..bc8a712430 100644
--- a/modules/platforms/cpp/ignite/schema/binary_tuple_schema.h
+++ b/modules/platforms/cpp/ignite/tuple/binary_tuple_schema.h
@@ -77,7 +77,7 @@ public:
      *
      * @return Number of elements.
      */
-    IntT num_elements() const noexcept { return static_cast<IntT>(elements.size()); }
+    number_t num_elements() const noexcept { return static_cast<number_t>(elements.size()); }
 
     /**
      * @brief Gets element info.
@@ -85,14 +85,14 @@ public:
      * @param index Element number.
      * @return Element info.
      */
-    const column_info &get_element(IntT index) const { return elements[index]; }
+    const column_info &get_element(number_t index) const { return elements[index]; }
 
     /**
      * @brief Gets the nullmap size.
      *
      * @return Nullmap size in bytes.
      */
-    static constexpr SizeT get_nullmap_size(IntT num_elements) noexcept { return (num_elements + 7) / 8; }
+    static constexpr data_size_t get_nullmap_size(number_t num_elements) noexcept { return (num_elements + 7) / 8; }
 
     /**
      * @brief Gets offset of the byte that contains null-bit of a given tuple element.
@@ -100,7 +100,9 @@ public:
      * @param index Tuple element index.
      * @return Offset of the required byte relative to the tuple start.
      */
-    static constexpr SizeT get_null_offset(IntT index) noexcept { return binary_tuple_header::SIZE + index / 8; }
+    static constexpr data_size_t get_null_offset(number_t index) noexcept {
+        return binary_tuple_header::SIZE + index / 8;
+    }
 
     /**
      * @brief Gets a null-bit mask corresponding to a given tuple element.
@@ -108,7 +110,7 @@ public:
      * @param index Tuple element index.
      * @return Mask to extract the required null-bit.
      */
-    static constexpr std::byte get_null_mask(IntT index) noexcept { return std::byte{1} << (index % 8); }
+    static constexpr std::byte get_null_mask(number_t index) noexcept { return std::byte{1} << (index % 8); }
 
     /**
      * @brief Checks if a null-bit is set for a given tuple element.
@@ -121,7 +123,7 @@ public:
      * @return true If the required null-bit is set.
      * @return false If the required null-bit is clear.
      */
-    static bool has_null(const bytes_view &tuple, IntT index) noexcept {
+    static bool has_null(const bytes_view &tuple, number_t index) noexcept {
         return (tuple[get_null_offset(index)] & get_null_mask(index)) != std::byte{0};
     }
 };
diff --git a/modules/platforms/cpp/ignite/schema/column_info.h b/modules/platforms/cpp/ignite/tuple/column_info.h
similarity index 69%
rename from modules/platforms/cpp/ignite/schema/column_info.h
rename to modules/platforms/cpp/ignite/tuple/column_info.h
index af4e7f52aa..756a8d3601 100644
--- a/modules/platforms/cpp/ignite/schema/column_info.h
+++ b/modules/platforms/cpp/ignite/tuple/column_info.h
@@ -17,7 +17,7 @@
 
 #pragma once
 
-#include "ignite_type.h"
+#include <ignite/common/ignite_type.h>
 
 namespace ignite {
 
@@ -25,16 +25,17 @@ namespace ignite {
  * @brief Basic column info.
  */
 struct column_info {
-    ignite_type dataType;
-    bool nullable;
-
-    bool hasFixedSize() const { return is_fixed_size_type(dataType); }
+    /** Data type of the values in this column. */
+    ignite_type type;
 
-    size_t getFixedSize() const { return get_type_size(dataType); }
+    /** True if the column values may be NULL, false otherwise. */
+    bool nullable;
 
-    bool operator==(const column_info &other) const { return dataType == other.dataType && nullable == other.nullable; }
+    bool operator==(const column_info &other) const noexcept {
+        return type == other.type && nullable == other.nullable;
+    }
 
-    bool operator!=(const column_info &other) const { return !(operator==(other)); }
+    bool operator!=(const column_info &other) const noexcept { return !(operator==(other)); }
 };
 
 } // namespace ignite
diff --git a/modules/platforms/cpp/ignite/schema/tuple_assembler.h b/modules/platforms/cpp/ignite/tuple/tuple_assembler.h
similarity index 100%
rename from modules/platforms/cpp/ignite/schema/tuple_assembler.h
rename to modules/platforms/cpp/ignite/tuple/tuple_assembler.h
diff --git a/modules/platforms/cpp/ignite/schema/tuple_test.cpp b/modules/platforms/cpp/ignite/tuple/tuple_test.cpp
similarity index 97%
rename from modules/platforms/cpp/ignite/schema/tuple_test.cpp
rename to modules/platforms/cpp/ignite/tuple/tuple_test.cpp
index d76801b76e..aaf0ba0773 100644
--- a/modules/platforms/cpp/ignite/schema/tuple_test.cpp
+++ b/modules/platforms/cpp/ignite/tuple/tuple_test.cpp
@@ -103,7 +103,7 @@ std::string read_tuple(std::optional<bytes_view> data) {
 struct SchemaDescriptor {
     std::vector<column_info> columns;
 
-    [[nodiscard]] IntT length() const { return IntT(columns.size()); }
+    [[nodiscard]] number_t length() const { return number_t(columns.size()); }
 
     [[nodiscard]] binary_tuple_schema to_tuple_schema() const {
         return binary_tuple_schema({columns.begin(), columns.end()});
@@ -181,7 +181,7 @@ TEST(tuple, AllTypes) {
         {ignite_type::TIMESTAMP, false},
         {ignite_type::STRING, false},
         {ignite_type::UUID, false},
-        {ignite_type::BINARY, false},
+        {ignite_type::BYTE_ARRAY, false},
     };
 
     binary_tuple_schema sch = schema.to_tuple_schema();
@@ -584,7 +584,7 @@ TEST(tuple, ZeroLengthVarlen) { // NOLINT(cert-err58-cpp)
 
     schema.columns = {
         {ignite_type::INT32, false},
-        {ignite_type::BINARY, false},
+        {ignite_type::BYTE_ARRAY, false},
     };
 
     // Single zero-length vector of bytes.
@@ -598,7 +598,7 @@ TEST(tuple, ZeroLengthVarlen) { // NOLINT(cert-err58-cpp)
 
     // Two zero-length vectors of bytes.
     {
-        schema.columns.emplace_back(column_info{ignite_type::BINARY, false});
+        schema.columns.emplace_back(column_info{ignite_type::BYTE_ARRAY, false});
 
         auto values = std::make_tuple(int32_t{0}, ""s, ""s);
 
@@ -612,8 +612,8 @@ TEST(tuple, ZeroLengthVarlen) { // NOLINT(cert-err58-cpp)
         schema.columns.erase(schema.columns.begin() + 1, schema.columns.end());
 
         schema.columns.emplace_back(column_info{ignite_type::INT32, false});
-        schema.columns.emplace_back(column_info{ignite_type::BINARY, false});
-        schema.columns.emplace_back(column_info{ignite_type::BINARY, false});
+        schema.columns.emplace_back(column_info{ignite_type::BYTE_ARRAY, false});
+        schema.columns.emplace_back(column_info{ignite_type::BYTE_ARRAY, false});
 
         auto values = std::make_tuple(int32_t{0}, int32_t{123}, ""s, ""s);
 
@@ -628,7 +628,7 @@ TEST(tuple, SingleVarlen) { // NOLINT(cert-err58-cpp)
 
     schema.columns = {
         {ignite_type::INT32, false},
-        {ignite_type::BINARY, false},
+        {ignite_type::BYTE_ARRAY, false},
     };
 
     auto values = std::make_tuple(int32_t{0}, "\1\2\3"s);
@@ -643,7 +643,7 @@ TEST(tuple, TinyVarlenFormatOverflowLarge) { // NOLINT(cert-err58-cpp)
 
     schema.columns.emplace_back(column_info{ignite_type::INT32, false});
     for (int i = 0; i < 300; i++) {
-        schema.columns.emplace_back(column_info{ignite_type::BINARY, false});
+        schema.columns.emplace_back(column_info{ignite_type::BYTE_ARRAY, false});
     }
 
     // Flags - 1 zero byte
@@ -670,7 +670,7 @@ TEST(tuple, TinyVarlenFormatOverflowMedium) { // NOLINT(cert-err58-cpp)
 
     schema.columns.emplace_back(column_info{ignite_type::INT32, false});
     for (int i = 0; i < 300; i++) {
-        schema.columns.emplace_back(column_info{ignite_type::BINARY, false});
+        schema.columns.emplace_back(column_info{ignite_type::BYTE_ARRAY, false});
     }
 
     // Flags - 1 zero byte
@@ -706,7 +706,7 @@ TEST(tuple, TinyVarlenFormatOverflowMedium) { // NOLINT(cert-err58-cpp)
         EXPECT_EQ(1, next->size());
         EXPECT_EQ(i, read_tuple<int8_t>(next.value()));
     }
-    for (IntT i = 4; i < schema.length(); i++) {
+    for (number_t i = 4; i < schema.length(); i++) {
         auto next = tp.get_next();
         EXPECT_TRUE(next.has_value());
         EXPECT_EQ(0, next.value().size());
@@ -724,8 +724,8 @@ TEST(tuple, ExpectedVarlenTupleBinaries) { // NOLINT(cert-err58-cpp)
         {ignite_type::INT32, false},
         {ignite_type::INT32, false},
         {ignite_type::INT32, false},
-        {ignite_type::BINARY, false},
-        {ignite_type::BINARY, false},
+        {ignite_type::BYTE_ARRAY, false},
+        {ignite_type::BYTE_ARRAY, false},
         {ignite_type::STRING, false},
     };
 
@@ -927,7 +927,7 @@ TEST(tuple, StringAfterNull) {
     SchemaDescriptor schema;
 
     schema.columns.emplace_back(column_info{ignite_type::INT32, false});
-    schema.columns.emplace_back(column_info{ignite_type::BINARY, true});
+    schema.columns.emplace_back(column_info{ignite_type::BYTE_ARRAY, true});
     schema.columns.emplace_back(column_info{ignite_type::STRING, false});
 
     // 101, null, "Bob"
@@ -970,7 +970,7 @@ TEST(tuple, TupleWriteRead) { // NOLINT(cert-err58-cpp)
     schema.columns.emplace_back(column_info{ignite_type::INT8, false});
 
     for (bool nullable : {false, true}) {
-        for (IntT i = 0; i < schema.length(); i++) {
+        for (number_t i = 0; i < schema.length(); i++) {
             schema.columns[i].nullable = nullable;
         }
 
@@ -1043,7 +1043,7 @@ TEST(tuple, Int8TupleWriteRead) { // NOLINT(cert-err58-cpp)
     schema.columns.emplace_back(column_info{ignite_type::INT8, false});
 
     for (bool nullable : {false, true}) {
-        for (IntT i = 0; i < schema.length(); i++) {
+        for (number_t i = 0; i < schema.length(); i++) {
             schema.columns[i].nullable = nullable;
         }
 
@@ -1187,7 +1187,7 @@ TEST(tuple, VarlenMediumTest) { // NOLINT(cert-err58-cpp)
 
     schema.columns.emplace_back(column_info{ignite_type::INT32, false});
     for (int i = 0; i < 300; i++) {
-        schema.columns.emplace_back(column_info{ignite_type::BINARY, false});
+        schema.columns.emplace_back(column_info{ignite_type::BYTE_ARRAY, false});
     }
 
     {
@@ -1220,7 +1220,7 @@ TEST(tuple, VarlenMediumTest) { // NOLINT(cert-err58-cpp)
         binary_tuple_parser tp(schema.length(), tuple);
 
         EXPECT_EQ(100500, read_tuple<int32_t>(tp.get_next()));
-        for (IntT i = 1; i < schema.length(); i++) {
+        for (number_t i = 1; i < schema.length(); i++) {
             EXPECT_EQ(value, read_tuple<std::string>(tp.get_next()));
         }
     }
@@ -1259,7 +1259,7 @@ TEST(tuple, VarlenMediumTest) { // NOLINT(cert-err58-cpp)
         binary_tuple_parser tp(schema.length(), tuple);
 
         EXPECT_EQ(100500, read_tuple<int32_t>(tp.get_next()));
-        for (IntT i = 1; i < schema.length(); i++) {
+        for (number_t i = 1; i < schema.length(); i++) {
             EXPECT_EQ(value, read_tuple<std::string>(tp.get_next()));
         }
     }
diff --git a/modules/platforms/cpp/ignite/schema/types.h b/modules/platforms/cpp/ignite/tuple/types.h
similarity index 75%
rename from modules/platforms/cpp/ignite/schema/types.h
rename to modules/platforms/cpp/ignite/tuple/types.h
index 2b93d49572..7eab5c4044 100644
--- a/modules/platforms/cpp/ignite/schema/types.h
+++ b/modules/platforms/cpp/ignite/tuple/types.h
@@ -25,22 +25,23 @@
 
 namespace ignite {
 
-/** C++ version of java int. Used as column number, etc. */
-using IntT = int32_t;
+/** C++ version of Java int. Used as a column number, etc. */
+using number_t = int32_t;
 
 /** Data size for columns and entire rows too. */
-using SizeT = uint32_t;
+using data_size_t = uint32_t;
 
 /** Non-existent column/element number. */
-static constexpr IntT NO_NUM = -1;
+static constexpr number_t NOT_NUM = -1;
 
 /** Binary value for a potentially nullable column. */
-using element_view = std::optional<bytes_view>;
+using value_view = std::optional<bytes_view>;
 
 /** A set of binary values for a whole or partial row. */
-using tuple_view = std::vector<element_view>;
+using tuple_view = std::vector<value_view>;
 
-/** A set of binary values for the key part of a row. */
+/** A set of binary values for the key part of a row. Key columns must be all non-null so
+    there is no need to wrap the bytes view into ats::optional. */
 using key_tuple_view = std::vector<bytes_view>;
 
 } // namespace ignite
diff --git a/modules/platforms/cpp/tests/client-test/sql_test.cpp b/modules/platforms/cpp/tests/client-test/sql_test.cpp
index df572adb85..cc92d30787 100644
--- a/modules/platforms/cpp/tests/client-test/sql_test.cpp
+++ b/modules/platforms/cpp/tests/client-test/sql_test.cpp
@@ -75,7 +75,7 @@ protected:
 };
 
 void check_columns(
-    const result_set_metadata &meta, std::initializer_list<std::tuple<std::string, column_type>> columns) {
+    const result_set_metadata &meta, std::initializer_list<std::tuple<std::string, ignite_type>> columns) {
 
     ASSERT_EQ(columns.size(), meta.columns().size());
     size_t i = 0;
@@ -94,7 +94,7 @@ TEST_F(sql_test, sql_simple_select) {
     EXPECT_TRUE(result_set.has_rowset());
     EXPECT_EQ(-1, result_set.affected_rows());
 
-    check_columns(result_set.metadata(), {{"42", column_type::INT32}, {"'Lorem'", column_type::STRING}});
+    check_columns(result_set.metadata(), {{"42", ignite_type::INT32}, {"'Lorem'", ignite_type::STRING}});
 
     auto page = result_set.current_page();
 
@@ -113,7 +113,7 @@ TEST_F(sql_test, sql_table_select) {
     EXPECT_TRUE(result_set.has_rowset());
     EXPECT_EQ(-1, result_set.affected_rows());
 
-    check_columns(result_set.metadata(), {{"ID", column_type::INT32}, {"VAL", column_type::STRING}});
+    check_columns(result_set.metadata(), {{"ID", ignite_type::INT32}, {"VAL", ignite_type::STRING}});
 
     auto page = result_set.current_page();
 
@@ -138,7 +138,7 @@ TEST_F(sql_test, sql_select_multiple_pages) {
     EXPECT_TRUE(result_set.has_rowset());
     EXPECT_EQ(-1, result_set.affected_rows());
 
-    check_columns(result_set.metadata(), {{"ID", column_type::INT32}, {"VAL", column_type::STRING}});
+    check_columns(result_set.metadata(), {{"ID", ignite_type::INT32}, {"VAL", ignite_type::STRING}});
 
     for (std::int32_t i = 0; i < 10; ++i) {
         auto page = result_set.current_page();
diff --git a/modules/platforms/cpp/tests/client-test/transactions_test.cpp b/modules/platforms/cpp/tests/client-test/transactions_test.cpp
index 744387a95a..135d52d1bc 100644
--- a/modules/platforms/cpp/tests/client-test/transactions_test.cpp
+++ b/modules/platforms/cpp/tests/client-test/transactions_test.cpp
@@ -64,7 +64,7 @@ TEST_F(transactions_test, empty_transaction_commit) {
     auto api = m_client.get_transactions();
 
     auto tx = api.begin();
-    tx.rollback();
+    tx.commit();
 }
 
 TEST_F(transactions_test, commit_updates_data) {