You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by am...@apache.org on 2021/03/26 14:16:07 UTC

[ignite-3] branch main updated: IGNITE-14035: Table access API. (#33)

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

amashenkov 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 96b8e3a  IGNITE-14035: Table access API. (#33)
96b8e3a is described below

commit 96b8e3a14c77ea9b85205d86149c34c5f1bed730
Author: Andrew V. Mashenkov <AM...@users.noreply.github.com>
AuthorDate: Fri Mar 26 17:16:01 2021 +0300

    IGNITE-14035: Table access API. (#33)
    
    Also, rename classes Tuple->Row and TupleAssembler->RowAssembler.
---
 check-rules/checkstyle-rules.xml                   |  11 +-
 modules/README.md                                  |   7 +-
 modules/api/pom.xml                                |  38 +-
 .../org/apache/ignite/binary/BinaryObject.java}    |  13 +-
 .../java/org/apache/ignite/lang/IgniteFuture.java} |  11 +-
 .../org/apache/ignite/table/InvocationContext.java |  68 +++
 .../org/apache/ignite/table/InvokeProcessor.java   |  47 +++
 .../ignite/table/InvokeProcessorException.java}    |   9 +-
 .../apache/ignite/table/KeyValueBinaryView.java}   |  15 +-
 .../java/org/apache/ignite/table/KeyValueView.java | 349 ++++++++++++++++
 .../java/org/apache/ignite/table/RecordView.java   |  42 ++
 .../main/java/org/apache/ignite/table/Table.java   |  92 +++++
 .../java/org/apache/ignite/table/TableView.java    | 309 ++++++++++++++
 .../main/java/org/apache/ignite/table/Tuple.java   | 100 +++++
 .../org/apache/ignite/table/TupleBuilder.java}     |  21 +-
 .../org/apache/ignite/table/mapper/KeyMapper.java} |   8 +-
 .../org/apache/ignite/table/mapper/Mappers.java}   |  35 +-
 .../apache/ignite/table/mapper/RecordMapper.java}  |  20 +-
 .../apache/ignite/table/mapper/ValueMapper.java}   |  22 +-
 .../org/apache/ignite/table}/package-info.java     |   5 +-
 .../presto/bytecode/DynamicClassLoader.java        |   9 -
 .../expression/ConstantBytecodeExpression.java     |   2 +-
 modules/schema/README.md                           |  12 +-
 .../ignite/internal/schema/AssemblyException.java  |   2 +-
 .../org/apache/ignite/internal/schema/Bitmask.java |   2 +-
 .../{ByteBufferTuple.java => ByteBufferRow.java}   |  28 +-
 .../org/apache/ignite/internal/schema/Columns.java |   4 +-
 .../ignite/internal/schema/ExpandableByteBuf.java  |   4 +-
 .../internal/schema/InvalidTypeException.java      |   2 +-
 .../ignite/internal/schema/NativeTypeSpec.java     |  30 +-
 .../org/apache/ignite/internal/schema/README.md    |  28 +-
 .../internal/schema/{Tuple.java => Row.java}       |  72 +++-
 .../{TupleAssembler.java => RowAssembler.java}     |  44 +-
 .../ignite/internal/schema/SchemaDescriptor.java   |  22 +
 .../schema/marshaller/AbstractSerializer.java      |  40 +-
 .../internal/schema/marshaller/MarshallerUtil.java |   4 +-
 .../internal/schema/marshaller/Serializer.java     |   2 +-
 .../marshaller/asm/AsmSerializerGenerator.java     |  22 +-
 ...nerator.java => ColumnAccessCodeGenerator.java} |  44 +-
 .../asm/IdentityMarshallerCodeGenerator.java       |  10 +-
 .../marshaller/asm/MarshallerCodeGenerator.java    |   6 +-
 .../asm/ObjectMarshallerCodeGenerator.java         |  16 +-
 .../marshaller/reflection/FieldAccessor.java       |  74 ++--
 .../marshaller/reflection/JavaSerializer.java      |  24 +-
 .../schema/marshaller/reflection/Marshaller.java   |  16 +-
 .../ignite/internal/schema/package-info.java       |   2 +-
 .../schema/{TupleTest.java => RowTest.java}        |  14 +-
 .../marshaller/reflection/FieldAccessorTest.java   |  76 ++--
 modules/table/README.md                            |   2 +
 pom.xml => modules/table/pom.xml                   |  50 ++-
 .../org/apache/ignite/binary/BinaryObjects.java}   |  32 +-
 .../internal/schema/marshaller/Marshaller.java     |  74 ++++
 .../ignite/internal/storage/TableStorage.java}     |  25 +-
 .../apache/ignite/internal/table/KVViewImpl.java   | 228 +++++++++++
 .../internal/table/KeyValueBinaryViewImpl.java     | 241 +++++++++++
 .../ignite/internal/table/RecordViewImpl.java      | 249 +++++++++++
 .../apache/ignite/internal/table/RowChunk.java}    |  14 +-
 .../ignite/internal/table/RowChunkAdapter.java     | 106 +++++
 .../apache/ignite/internal/table/TableImpl.java    | 265 ++++++++++++
 .../apache/ignite/internal/table/TableRow.java}    |  22 +-
 .../ignite/internal/table/TableRowAdapter.java     | 142 +++++++
 .../ignite/internal/table}/package-info.java       |   5 +-
 .../test/java/org/apache/ignite/table/Example.java | 456 +++++++++++++++++++++
 .../apache/ignite/table/impl/TestTableRowImpl.java | 132 ++++++
 .../ignite/table/impl/TestTableStorageImpl.java    |  82 ++++
 pom.xml                                            |   1 +
 66 files changed, 3570 insertions(+), 389 deletions(-)

diff --git a/check-rules/checkstyle-rules.xml b/check-rules/checkstyle-rules.xml
index 36b4b86..bc010f9 100644
--- a/check-rules/checkstyle-rules.xml
+++ b/check-rules/checkstyle-rules.xml
@@ -17,8 +17,9 @@
   limitations under the License.
 -->
 
-<!DOCTYPE module PUBLIC "-//Puppy Crawl//DTD Check Configuration 1.3//EN"
-    "http://www.puppycrawl.com/dtds/configuration_1_3.dtd">
+<!DOCTYPE module PUBLIC
+    "-//Checkstyle//DTD Checkstyle Configuration 1.3//EN"
+    "http://checkstyle.org/dtds/configuration_1_3.dtd">
 <module name="Checker">
     <property name="charset" value="UTF-8"/>
 
@@ -29,7 +30,13 @@
         <property name="eachLine" value="true"/>
     </module>
 
+    <!-- SuppressWarning Filter. https://checkstyle.sourceforge.io/config_filters.html#SuppressWarningsFilter -->
+    <module name="SuppressWarningsFilter" />
+
     <module name="TreeWalker">
+        <!-- Handler for SuppressWarning Filter. https://checkstyle.sourceforge.io/config_filters.html#SuppressWarningsFilter -->
+        <module name="SuppressWarningsHolder" />
+
         <!-- Coding Checks. See: https://checkstyle.sourceforge.io/config_coding.html -->
         <module name="SimplifyBooleanExpression"/>
 
diff --git a/modules/README.md b/modules/README.md
index 05bf434..beaddcf 100644
--- a/modules/README.md
+++ b/modules/README.md
@@ -10,7 +10,7 @@ We prohibit cyclic dependencies between modules in order to simplify JIGSAW migr
 
 Module Name | Description
 ----------- | -----------
-[api](api/README.md)|Ignite public API
+[api](api/README.md)|Ignite public API.
 [bytecode](bytecode/README.md)|Ignite Bytecode module.
 [cli](cli/README.md)|Ignite CLI implementation
 [cli-common](cli-common/README.md)|Shared interfaces definitions for pluggable CLIng
@@ -18,5 +18,6 @@ Module Name | Description
 [configuration-annotation-processor](configuration-annotation-processor/README.md)|Tooling for generating Ignite configuration model classes from configuration schema definition
 [network](network/README.md)|Networking module: group membership and message passi
 [rest](rest/README.md)|REST management endpoint bindings and command handlers
-[runner](runner/README.md)|Ignite server node runner. The module that wires up the Ignite components and handles node lifecycle
-[schema](schema/README.md)|Ignite schema API implementation and schema management classes.
\ No newline at end of file
+[runner](runner/README.md)|Ignite server node runner. The module that wires up the Ignite components and handles node lifecycle.
+[schema](schema/README.md)|Ignite schema API implementation and schema management classes.
+[table](table/README.md)|Ignite table API implementation.
\ No newline at end of file
diff --git a/modules/api/pom.xml b/modules/api/pom.xml
index 95a9778..2ca3c60 100644
--- a/modules/api/pom.xml
+++ b/modules/api/pom.xml
@@ -1,25 +1,22 @@
 <?xml version="1.0" encoding="UTF-8"?>
 
 <!--
-  ~ Licensed to the Apache Software Foundation (ASF) under one or more
-  ~ contributor license agreements.  See the NOTICE file distributed with
-  ~ this work for additional information regarding copyright ownership.
-  ~ The ASF licenses this file to You under the Apache License, Version 2.0
-  ~ (the "License"); you may not use this file except in compliance with
-  ~ the License.  You may obtain a copy of the License at
-  ~
-  ~      http://www.apache.org/licenses/LICENSE-2.0
-  ~
-  ~ Unless required by applicable law or agreed to in writing, software
-  ~ distributed under the License is distributed on an "AS IS" BASIS,
-  ~ WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-  ~ See the License for the specific language governing permissions and
-  ~ limitations under the License.
-  -->
+  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
 
-<!--
-    POM file.
+       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.
 -->
+
 <project xmlns="http://maven.apache.org/POM/4.0.0"
          xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
          xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
@@ -34,4 +31,11 @@
 
     <artifactId>ignite-api</artifactId>
     <version>3.0.0-SNAPSHOT</version>
+
+    <dependencies>
+        <dependency>
+            <groupId>org.jetbrains</groupId>
+            <artifactId>annotations</artifactId>
+        </dependency>
+    </dependencies>
 </project>
diff --git a/modules/schema/src/main/java/org/apache/ignite/internal/schema/package-info.java b/modules/api/src/main/java/org/apache/ignite/binary/BinaryObject.java
similarity index 77%
copy from modules/schema/src/main/java/org/apache/ignite/internal/schema/package-info.java
copy to modules/api/src/main/java/org/apache/ignite/binary/BinaryObject.java
index fe5f0e3..b6648c4 100644
--- a/modules/schema/src/main/java/org/apache/ignite/internal/schema/package-info.java
+++ b/modules/api/src/main/java/org/apache/ignite/binary/BinaryObject.java
@@ -15,8 +15,15 @@
  * limitations under the License.
  */
 
+package org.apache.ignite.binary;
+
 /**
- * <!-- Package description. -->
- * Contains schema description, tuple assembly and field accessor classes.
+ * Binary object wraps byte[].
  */
-package org.apache.ignite.internal.schema;
\ No newline at end of file
+//TODO: IGNITE-14316: Replace this stub with a proper interface.
+public interface BinaryObject {
+    /**
+     * @return Serialized data.
+     */
+    byte[] toBytes();
+}
diff --git a/modules/schema/src/main/java/org/apache/ignite/internal/schema/package-info.java b/modules/api/src/main/java/org/apache/ignite/lang/IgniteFuture.java
similarity index 74%
copy from modules/schema/src/main/java/org/apache/ignite/internal/schema/package-info.java
copy to modules/api/src/main/java/org/apache/ignite/lang/IgniteFuture.java
index fe5f0e3..aa27703 100644
--- a/modules/schema/src/main/java/org/apache/ignite/internal/schema/package-info.java
+++ b/modules/api/src/main/java/org/apache/ignite/lang/IgniteFuture.java
@@ -15,8 +15,13 @@
  * limitations under the License.
  */
 
+package org.apache.ignite.lang;
+
+import java.util.concurrent.CompletionStage;
+import java.util.concurrent.Future;
+
 /**
- * <!-- Package description. -->
- * Contains schema description, tuple assembly and field accessor classes.
+ * Future providing chaining capabilities for the construction of computation pipelines.
  */
-package org.apache.ignite.internal.schema;
\ No newline at end of file
+public interface IgniteFuture<T> extends CompletionStage<T>, Future<T> {
+}
diff --git a/modules/api/src/main/java/org/apache/ignite/table/InvocationContext.java b/modules/api/src/main/java/org/apache/ignite/table/InvocationContext.java
new file mode 100644
index 0000000..8a3bda0
--- /dev/null
+++ b/modules/api/src/main/java/org/apache/ignite/table/InvocationContext.java
@@ -0,0 +1,68 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.table;
+
+/**
+ * Invocation context provides access to invoke operation call parameters,
+ * a method to set a new value for the key.
+ * <p>
+ * InvokeProcessor executes atomically under lock which makes impossible
+ * to trigger 'live-schema' upgrade within the invoke operation.
+ * Any try to update the row leading to schema change will end up with {@link InvokeProcessorException}.
+ *
+ * @param <K> Target object type.
+ * @param <V> Value object type.
+ * @apiNote New value MUST BE compliant with the current schema version.
+ */
+public interface InvocationContext<K, V> {
+    /**
+     * @return Processor invocation arguments provided by user to invoke operation.
+     */
+    Object[] args();
+
+    /**
+     * Returns an object the user provide to invoke call for running invoke processor against the associated row.
+     * <p>
+     * Depending on Table view the invoke operation is called on, the returning value is either value object or
+     * record object or tuple with value fields set.
+     *
+     * @return Object which target row is associated with.
+     */
+    K key();
+
+    /**
+     * Returns current value object for the target row.
+     * <p>
+     * Depending on Table view the invoke operation is called on, the returning value is either value object or
+     * record object or tuple with value fields set or {@code null} for non-existed row.
+     *
+     * @return Current value of target row or {@code null} if value associated with the key is not exists.
+     */
+    V value();
+
+    /**
+     * Sets a new value object for the target row.
+     * <p>
+     * Depending on Table view the invoke operation is called on, a new value can be either value object or
+     * record object or tuple with value fields set or {@code null} for removal.
+     *
+     * @param val Value object to set.
+     * @throws InvokeProcessorException if new value is not compliant with the current schema.
+     */
+    void value(V val);
+}
diff --git a/modules/api/src/main/java/org/apache/ignite/table/InvokeProcessor.java b/modules/api/src/main/java/org/apache/ignite/table/InvokeProcessor.java
new file mode 100644
index 0000000..9959ade
--- /dev/null
+++ b/modules/api/src/main/java/org/apache/ignite/table/InvokeProcessor.java
@@ -0,0 +1,47 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.table;
+
+import java.io.Serializable;
+
+/**
+ * Invoke processor interface provides API to run code on server side against a table record
+ * associated with provided key.
+ * <p>
+ * For non-binary projections row will be deserialized to user object(s) before the invocation
+ * and serialized back if a new value was set via {@link InvocationContext#value(V)}.
+ * <p>
+ * Invoke operation arguments along with invoke operation result classes MUST be serializable
+ * as they can be transferred over network.
+ *
+ * @param <K> Key object type.
+ * @param <V> Value type.
+ * @param <R> Processor result type.
+ * @apiNote Distributed deployment MUST be used for processor code load instead of load form the classpath
+ * to guarantee same code revision in the grid.
+ */
+public interface InvokeProcessor<K, V, R extends Serializable> extends Serializable {
+    /**
+     * Processes table record and return the result.
+     *
+     * @param ctx Invocation context.
+     * @return Invoke processor result.
+     * @throws InvokeProcessorException If failed during data processing.
+     */
+    R process(InvocationContext<K, V> ctx) throws InvokeProcessorException;
+}
diff --git a/modules/schema/src/main/java/org/apache/ignite/internal/schema/package-info.java b/modules/api/src/main/java/org/apache/ignite/table/InvokeProcessorException.java
similarity index 84%
copy from modules/schema/src/main/java/org/apache/ignite/internal/schema/package-info.java
copy to modules/api/src/main/java/org/apache/ignite/table/InvokeProcessorException.java
index fe5f0e3..4c8b013 100644
--- a/modules/schema/src/main/java/org/apache/ignite/internal/schema/package-info.java
+++ b/modules/api/src/main/java/org/apache/ignite/table/InvokeProcessorException.java
@@ -15,8 +15,11 @@
  * limitations under the License.
  */
 
+package org.apache.ignite.table;
+
 /**
- * <!-- Package description. -->
- * Contains schema description, tuple assembly and field accessor classes.
+ * InvokeProcessor invocation exception.
  */
-package org.apache.ignite.internal.schema;
\ No newline at end of file
+public class InvokeProcessorException extends RuntimeException {
+
+}
diff --git a/modules/schema/src/main/java/org/apache/ignite/internal/schema/InvalidTypeException.java b/modules/api/src/main/java/org/apache/ignite/table/KeyValueBinaryView.java
similarity index 70%
copy from modules/schema/src/main/java/org/apache/ignite/internal/schema/InvalidTypeException.java
copy to modules/api/src/main/java/org/apache/ignite/table/KeyValueBinaryView.java
index d03ca73..75b95db 100644
--- a/modules/schema/src/main/java/org/apache/ignite/internal/schema/InvalidTypeException.java
+++ b/modules/api/src/main/java/org/apache/ignite/table/KeyValueBinaryView.java
@@ -15,16 +15,17 @@
  * limitations under the License.
  */
 
-package org.apache.ignite.internal.schema;
+package org.apache.ignite.table;
 
 /**
- * An exception thrown when an attempt to read an invalid type from a tuple is performed.
+ * Key-value view of table provides methods to access the data using key-value approach and
+ * regarding the binary object concept.
  */
-public class InvalidTypeException extends IllegalArgumentException {
+public interface KeyValueBinaryView extends KeyValueView<Tuple, Tuple> {
     /**
-     * @param msg Error message.
+     * Factory method shortcut.
+     *
+     * @return Tuple builder.
      */
-    public InvalidTypeException(String msg) {
-        super(msg);
-    }
+    TupleBuilder tupleBuilder();
 }
diff --git a/modules/api/src/main/java/org/apache/ignite/table/KeyValueView.java b/modules/api/src/main/java/org/apache/ignite/table/KeyValueView.java
new file mode 100644
index 0000000..cb86b25
--- /dev/null
+++ b/modules/api/src/main/java/org/apache/ignite/table/KeyValueView.java
@@ -0,0 +1,349 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.table;
+
+import java.io.Serializable;
+import java.util.Collection;
+import java.util.Map;
+import org.apache.ignite.lang.IgniteFuture;
+import org.apache.ignite.table.mapper.Mappers;
+import org.jetbrains.annotations.NotNull;
+
+/**
+ * Key-Value view of table provides methods to access table records.
+ *
+ * @param <K> Mapped key type.
+ * @param <V> Mapped value type.
+ * @apiNote 'Key/value class field' &gt;-&lt 'table column' mapping laid down in implementation.
+ * @see Mappers
+ */
+public interface KeyValueView<K, V> {
+    /**
+     * Gets a value associated with the given key.
+     *
+     * @param key The key whose associated value is to be returned.
+     * @return Value or {@code null}, if it does not exist.
+     */
+    V get(K key);
+
+    /**
+     * Asynchronously gets a value associated with the given key.
+     *
+     * @param key The key whose associated value is to be returned.
+     * @return Future representing pending completion of the operation.
+     */
+    @NotNull IgniteFuture<V> getAsync(K key);
+
+    /**
+     * Get values associated with given keys.
+     *
+     * @param keys Keys whose associated values are to be returned.
+     * @return Values associated with given keys.
+     */
+    Map<K, V> getAll(Collection<K> keys);
+
+    /**
+     * Get values associated with given keys.
+     *
+     * @param keys Keys whose associated values are to be returned.
+     * @return Future representing pending completion of the operation.
+     */
+    @NotNull IgniteFuture<Map<K, V>> getAllAsync(Collection<K> keys);
+
+    /**
+     * Determines if the table contains an entry for the specified key.
+     *
+     * @param key The key whose presence is to be tested.
+     * @return {@code True} if a value exists for the specified key, {@code false} otherwise.
+     */
+    boolean contains(K key);
+
+    /**
+     * Puts value associated with given key into the table.
+     *
+     * @param key Key with which the specified value is to be associated.
+     * @param val Value to be associated with the specified key.
+     */
+    void put(K key, V val);
+
+    /**
+     * Asynchronously puts value associated with given key into the table.
+     *
+     * @param key Key with which the specified value is to be associated.
+     * @param val Value to be associated with the specified key.
+     * @return Future representing pending completion of the operation.
+     */
+    @NotNull IgniteFuture<Void> putAsync(K key, V val);
+
+    /**
+     * Put associated key-value pairs.
+     *
+     * @param pairs Key-value pairs.
+     */
+    void putAll(Map<K, V> pairs);
+
+    /**
+     * Asynchronously put associated key-value pairs.
+     *
+     * @param pairs Key-value pairs.
+     * @return Future representing pending completion of the operation.
+     */
+    @NotNull IgniteFuture<Void> putAllAsync(Map<K, V> pairs);
+
+    /**
+     * Puts new or replaces existed value associated with given key into the table.
+     *
+     * @param key Key with which the specified value is to be associated.
+     * @param val Value to be associated with the specified key.
+     * @return Replaced value or {@code null}, if not existed.
+     */
+    V getAndPut(K key, V val);
+
+    /**
+     * Asynchronously puts new or replaces existed value associated with given key into the table.
+     *
+     * @param key Key with which the specified value is to be associated.
+     * @param val Value to be associated with the specified key.
+     * @return Future representing pending completion of the operation.
+     */
+    @NotNull IgniteFuture<V> getAndPutAsync(K key, V val);
+
+    /**
+     * Puts value associated with given key into the table if not exists.
+     *
+     * @param key Key with which the specified value is to be associated.
+     * @param val Value to be associated with the specified key.
+     * @return {@code True} if successful, {@code false} otherwise.
+     */
+    boolean putIfAbsent(K key, V val);
+
+    /**
+     * Asynchronously puts value associated with given key into the table if not exists.
+     *
+     * @param key Key with which the specified value is to be associated.
+     * @param val Value to be associated with the specified key.
+     * @return Future representing pending completion of the operation.
+     */
+    @NotNull IgniteFuture<Boolean> putIfAbsentAsync(K key, V val);
+
+    /**
+     * Removes value associated with given key from the table.
+     *
+     * @param key Key whose mapping is to be removed from the table.
+     * @return {@code True} if a value associated with the specified key was successfully removed, {@code false} otherwise.
+     */
+    boolean remove(K key);
+
+    /**
+     * Asynchronously removes value associated with given key from the table.
+     *
+     * @param key Key whose mapping is to be removed from the table.
+     * @return Future representing pending completion of the operation.
+     */
+    @NotNull IgniteFuture<Boolean> removeAsync(K key);
+
+    /**
+     * Removes expected value associated with given key from the table.
+     *
+     * @param key Key whose associated value is to be removed from the table.
+     * @param val Expected value.
+     * @return {@code True} if the expected value for the specified key was successfully removed, {@code false} otherwise.
+     */
+    boolean remove(K key, V val);
+
+    /**
+     * Asynchronously removes expected value associated with given key from the table.
+     *
+     * @param key Key whose associated value is to be removed from the table.
+     * @param val Expected value.
+     * @return Future representing pending completion of the operation.
+     */
+    @NotNull IgniteFuture<Boolean> removeAsync(K key, V val);
+
+    /**
+     * Remove values associated with given keys from the table.
+     *
+     * @param keys Keys whose mapping is to be removed from the table.
+     * @return Keys whose values were not existed.
+     */
+    Collection<K> removeAll(Collection<K> keys);
+
+    /**
+     * Asynchronously remove values associated with given keys from the table.
+     *
+     * @param keys Keys whose mapping is to be removed from the table.
+     * @return Future representing pending completion of the operation.
+     */
+    @NotNull IgniteFuture<K> removeAllAsync(Collection<K> keys);
+
+    /**
+     * Gets then removes value associated with given key from the table.
+     *
+     * @param key Key whose associated value is to be removed from the table.
+     * @return Removed value or {@code null}, if not existed.
+     */
+    V getAndRemove(K key);
+
+    /**
+     * Asynchronously gets then removes value associated with given key from the table.
+     *
+     * @param key Key whose mapping is to be removed from the table.
+     * @return Future representing pending completion of the operation.
+     */
+    @NotNull IgniteFuture<V> getAndRemoveAsync(K key);
+
+    /**
+     * Replaces the value for a key only if exists. This is equivalent to
+     * <pre><code>
+     * if (cache.containsKey(key)) {
+     *   cache.put(key, value);
+     *   return true;
+     * } else {
+     *   return false;
+     * }</code></pre>
+     * except that the action is performed atomically.
+     *
+     * @param key Key with which the specified value is associated.
+     * @param val Value to be associated with the specified key.
+     * @return {@code True} if an old value was replaced, {@code false} otherwise.
+     */
+    boolean replace(K key, V val);
+
+    /**
+     * Asynchronously replaces the value for a key only if exists.
+     * See {@link #replace(Object, Object)}.
+     *
+     * @param key Key with which the specified value is associated.
+     * @param val Value to be associated with the specified key.
+     * @return Future representing pending completion of the operation.
+     */
+    @NotNull IgniteFuture<Boolean> replaceAsync(K key, V val);
+
+    /**
+     * Replaces the expected value for a key. This is equivalent to
+     * <pre><code>
+     * if (cache.get(key) == oldVal) {
+     *   cache.put(key, newVal);
+     *   return true;
+     * } else {
+     *   return false;
+     * }</code></pre>
+     * except that the action is performed atomically.
+     *
+     * @param key Key with which the specified value is associated.
+     * @param oldVal Expected value associated with the specified key.
+     * @param newVal Value to be associated with the specified key.
+     * @return {@code True} if an old value was replaced, {@code false} otherwise.
+     */
+    boolean replace(K key, V oldVal, V newVal);
+
+    /**
+     * Asynchronously replaces the expected value for a key.
+     * See {@link #replace(Object, Object, Object)}
+     *
+     * @param key Key with which the specified value is associated.
+     * @param oldVal Expected value associated with the specified key.
+     * @param newVal Value to be associated with the specified key.
+     * @return Future representing pending completion of the operation.
+     */
+    @NotNull IgniteFuture<Boolean> replaceAsync(K key, V oldVal, V newVal);
+
+    /**
+     * Replaces the value for a given key only if exists. This is equivalent to
+     * <pre><code>
+     * if (cache.containsKey(key)) {
+     *   V oldValue = cache.get(key);
+     *   cache.put(key, value);
+     *   return oldValue;
+     * } else {
+     *   return null;
+     * }
+     * </code></pre>
+     * except that the action is performed atomically.
+     *
+     * @param key Key with which the specified value is associated.
+     * @param val Value to be associated with the specified key.
+     * @return Replaced value, or {@code null} if not existed.
+     */
+    V getAndReplace(K key, V val);
+
+    /**
+     * Asynchronously replaces the value for a given key only if exists.
+     * See {@link #getAndReplace(Object, Object)}
+     *
+     * @param key Key with which the specified value is associated.
+     * @param val Value to be associated with the specified key.
+     * @return Future representing pending completion of the operation.
+     */
+    @NotNull IgniteFuture<V> getAndReplaceAsync(K key, V val);
+
+    /**
+     * Executes invoke processor code against the value associated with the provided key.
+     *
+     * @param key Key associated with the value that invoke processor will be applied to.
+     * @param proc Invoke processor.
+     * @param args Optional invoke processor arguments.
+     * @param <R> Invoke processor result type.
+     * @return Result of the processing.
+     * @see InvokeProcessor
+     */
+    <R extends Serializable> R invoke(K key, InvokeProcessor<K, V, R> proc, Serializable... args);
+
+    /**
+     * Asynchronously executes invoke processor code against the value associated with the provided key.
+     *
+     * @param key Key associated with the value that invoke processor will be applied to.
+     * @param proc Invoke processor.
+     * @param args Optional invoke processor arguments.
+     * @param <R> Invoke processor result type.
+     * @return Future representing pending completion of the operation.
+     * @see InvokeProcessor
+     */
+    @NotNull <R extends Serializable> IgniteFuture<R> invokeAsync(K key, InvokeProcessor<K, V, R> proc,
+        Serializable... args);
+
+    /**
+     * Executes invoke processor code against values associated with the provided keys.
+     *
+     * @param <R> Invoke processor result type.
+     * @param keys Ordered collection of keys which values associated with should be processed.
+     * @param proc Invoke processor.
+     * @param args Optional invoke processor arguments.
+     * @return Results of the processing.
+     * @see InvokeProcessor
+     */
+    <R extends Serializable> Map<K, R> invokeAll(
+        Collection<K> keys,
+        InvokeProcessor<K, V, R> proc,
+        Serializable... args);
+
+    /**
+     * Asynchronously executes invoke processor code against values associated with the provided keys.
+     *
+     * @param <R> Invoke processor result type.
+     * @param keys Ordered collection of keys which values associated with should be processed.
+     * @param proc Invoke processor.
+     * @param args Optional invoke processor arguments.
+     * @return Future representing pending completion of the operation.
+     * @see InvokeProcessor
+     */
+    @NotNull <R extends Serializable> IgniteFuture<Map<K, R>> invokeAllAsync(
+        Collection<K> keys,
+        InvokeProcessor<K, V, R> proc,
+        Serializable... args);
+}
diff --git a/modules/api/src/main/java/org/apache/ignite/table/RecordView.java b/modules/api/src/main/java/org/apache/ignite/table/RecordView.java
new file mode 100644
index 0000000..2ad84d1
--- /dev/null
+++ b/modules/api/src/main/java/org/apache/ignite/table/RecordView.java
@@ -0,0 +1,42 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.table;
+
+import org.apache.ignite.table.mapper.Mappers;
+
+/**
+ * Record view of table provides methods to access table records.
+ * <p>
+ * @param <R> Record type.
+ * @apiNote 'Record class field' &gt;-&lt 'table column' mapping laid down in implementation.
+ * @apiNote Some methods require a record with the only key fields set. This is not mandatory requirement
+ * and value fields will be just ignored.
+ * @see Mappers
+ */
+public interface RecordView<R> extends TableView<R> {
+    /**
+     * Fills given record with the values from the table.
+     * Similar to {@link #get(Object)}, but return original object with filled value fields.
+     * <p>
+     * All value fields of given object will be rewritten.
+     *
+     * @param recObjToFill Record object with key fields to be filled.
+     * @return Record with all fields filled from the table.
+     */
+    R fill(R recObjToFill);
+}
diff --git a/modules/api/src/main/java/org/apache/ignite/table/Table.java b/modules/api/src/main/java/org/apache/ignite/table/Table.java
new file mode 100644
index 0000000..d388297
--- /dev/null
+++ b/modules/api/src/main/java/org/apache/ignite/table/Table.java
@@ -0,0 +1,92 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.table;
+
+import org.apache.ignite.table.mapper.KeyMapper;
+import org.apache.ignite.table.mapper.Mappers;
+import org.apache.ignite.table.mapper.RecordMapper;
+import org.apache.ignite.table.mapper.ValueMapper;
+
+/**
+ * Table view of table provides methods to access table records regarding binary object concept.
+ * <p>
+ * Provided different views (key-value vs record) and approaches (mapped-object vs binary) to reach the data.
+ * <p>
+ * Binary table views might be useful in cases (but not limited) if user key-value classes are not in classpath
+ * or serialization/deserialization is unwanted due to performance reasons.
+ *
+ * @apiNote Some methods require a record with the only key columns set. This is not mandatory requirement
+ * and value columns will be just ignored.
+ * @see RecordView
+ * @see Table
+ * @see KeyValueView
+ * @see KeyValueBinaryView
+ */
+public interface Table extends TableView<Tuple> {
+    /**
+     * Creates record view of table for record class mapper provided.
+     *
+     * @param recMapper Record class mapper.
+     * @return Table record view.
+     */
+    <R> RecordView<R> recordView(RecordMapper<R> recMapper);
+
+    /**
+     * Creates key-value view of table for key-value class mappers provided.
+     *
+     * @param keyMapper Key class mapper.
+     * @param valMapper Value class mapper.
+     * @return Table key-value view.
+     */
+    <K, V> KeyValueView<K, V> kvView(KeyMapper<K> keyMapper, ValueMapper<V> valMapper);
+
+    /**
+     * Creates key-value view of table regarding the binary object concept.
+     *
+     * @return Table key-value view.
+     */
+    KeyValueBinaryView kvView();
+
+    /**
+     * Creates record view of table for record class provided.
+     *
+     * @param recCls Record class.
+     * @return Table record view.
+     */
+    default <R> RecordView<R> recordView(Class<R> recCls) {
+        return recordView(Mappers.ofRowClass(recCls));
+    }
+
+    /**
+     * Creates key-value view of table for key and value classes provided.
+     *
+     * @param keyCls Key class.
+     * @param valCls Value class.
+     * @return Table key-value view.
+     */
+    default <K, V> KeyValueView<K, V> kvView(Class<K> keyCls, Class<V> valCls) {
+        return kvView(Mappers.ofKeyClass(keyCls), Mappers.ofValueClass(valCls));
+    }
+
+    /**
+     * Factory method shortcut.
+     *
+     * @return Tuple builder.
+     */
+    TupleBuilder tupleBuilder();
+}
diff --git a/modules/api/src/main/java/org/apache/ignite/table/TableView.java b/modules/api/src/main/java/org/apache/ignite/table/TableView.java
new file mode 100644
index 0000000..ebe6bb3
--- /dev/null
+++ b/modules/api/src/main/java/org/apache/ignite/table/TableView.java
@@ -0,0 +1,309 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.table;
+
+import java.io.Serializable;
+import java.util.Collection;
+import java.util.Map;
+import org.apache.ignite.lang.IgniteFuture;
+import org.jetbrains.annotations.NotNull;
+
+/**
+ * Table view interface provides methods to access table records.
+ *
+ * @param <R> Mapped record type.
+ * @apiNote Some methods require a record with the only key columns set. This is not mandatory requirement
+ * and value columns will be just ignored.
+ */
+public interface TableView<R> {
+    /**
+     * Gets a record with same key columns values as given one from the table.
+     *
+     * @param keyRec Record with key columns set.
+     * @return Record with all columns filled from the table.
+     */
+    R get(R keyRec);
+
+    /**
+     * Asynchronously gets a record with same key columns values as given one from the table.
+     *
+     * @param keyRec Record with key columns set.
+     * @return Future representing pending completion of the operation.
+     */
+    @NotNull IgniteFuture<R> getAsync(R keyRec);
+
+    /**
+     * Get records from the table.
+     *
+     * @param keyRecs Records with key columns set.
+     * @return Records with all columns filled from the table.
+     */
+    Collection<R> getAll(Collection<R> keyRecs);
+
+    /**
+     * Asynchronously get records from the table.
+     *
+     * @param keyRecs Records with key columns set.
+     * @return Future representing pending completion of the operation.
+     */
+    @NotNull IgniteFuture<Collection<R>> getAllAsync(Collection<R> keyRecs);
+
+    /**
+     * Inserts a record into the table if does not exist or replaces the existed one.
+     *
+     * @param rec Record to insert into the table.
+     */
+    void upsert(R rec);
+
+    /**
+     * Asynchronously inserts a record into the table if does not exist or replaces the existed one.
+     *
+     * @param rec Record to insert into the table.
+     * @return Future representing pending completion of the operation.
+     */
+    @NotNull IgniteFuture<Void> upsertAsync(R rec);
+
+    /**
+     * Insert records into the table if does not exist or replaces the existed one.
+     *
+     * @param recs Records to insert into the table.
+     */
+    void upsertAll(Collection<R> recs);
+
+    /**
+     * Asynchronously inserts a record into the table if does not exist or replaces the existed one.
+     *
+     * @param recs Records to insert into the table.
+     * @return Future representing pending completion of the operation.
+     */
+    @NotNull IgniteFuture<Void> upsertAllAsync(Collection<R> recs);
+
+    /**
+     * Inserts a record into the table or replaces if exists and return replaced previous record.
+     *
+     * @param rec Record to insert into the table.
+     * @return Replaced record or {@code null} if not existed.
+     */
+    R getAndUpsert(R rec);
+
+    /**
+     * Asynchronously inserts a record into the table or replaces if exists and return replaced previous record.
+     *
+     * @param rec Record to insert into the table.
+     * @return Future representing pending completion of the operation.
+     */
+    @NotNull IgniteFuture<R> getAndUpsertAsync(R rec);
+
+    /**
+     * Inserts a record into the table if not exists.
+     *
+     * @param rec Record to insert into the table.
+     * @return {@code True} if successful, {@code false} otherwise.
+     */
+    boolean insert(R rec);
+
+    /**
+     * Asynchronously inserts a record into the table if not exists.
+     *
+     * @param rec Record to insert into the table.
+     * @return Future representing pending completion of the operation.
+     */
+    @NotNull IgniteFuture<Boolean> insertAsync(R rec);
+
+    /**
+     * Insert records into the table which do not exist, skipping existed ones.
+     *
+     * @param recs Records to insert into the table.
+     * @return Skipped records.
+     */
+    Collection<R> insertAll(Collection<R> recs);
+
+    /**
+     * Asynchronously insert records into the table which do not exist, skipping existed ones.
+     *
+     * @param recs Records to insert into the table.
+     * @return Future representing pending completion of the operation.
+     */
+    @NotNull IgniteFuture<Collection<R>> insertAllAsync(Collection<R> recs);
+
+    /**
+     * Replaces an existed record associated with the same key columns values as the given one has.
+     *
+     * @param rec Record to replace with.
+     * @return {@code True} if old record was found and replaced successfully, {@code false} otherwise.
+     */
+    boolean replace(R rec);
+
+    /**
+     * Asynchronously replaces an existed record associated with the same key columns values as the given one has.
+     *
+     * @param rec Record to replace with.
+     * @return Future representing pending completion of the operation.
+     */
+    @NotNull IgniteFuture<Boolean> replaceAsync(R rec);
+
+    /**
+     * Replaces an expected record in the table with the given new one.
+     *
+     * @param oldRec Record to replace.
+     * @param newRec Record to replace with.
+     * @return {@code True} if the old record replaced successfully, {@code false} otherwise.
+     */
+    boolean replace(R oldRec, R newRec);
+
+    /**
+     * Asynchronously replaces an expected record in the table with the given new one.
+     *
+     * @param oldRec Record to replace.
+     * @param newRec Record to replace with.
+     * @return Future representing pending completion of the operation.
+     */
+    @NotNull IgniteFuture<Boolean> replaceAsync(R oldRec, R newRec);
+
+    /**
+     * Gets an existed record associated with the same key columns values as the given one has,
+     * then replaces with the given one.
+     *
+     * @param rec Record to replace with.
+     * @return Replaced record or {@code null} if not existed.
+     */
+    R getAndReplace(R rec);
+
+    /**
+     * Asynchronously gets an existed record associated with the same key columns values as the given one has,
+     * then replaces with the given one.
+     *
+     * @param rec Record to replace with.
+     * @return Future representing pending completion of the operation.
+     */
+    @NotNull IgniteFuture<R> getAndReplaceAsync(R rec);
+
+    /**
+     * Deletes a record with the same key columns values as the given one from the table.
+     *
+     * @param keyRec Record with key columns set.
+     * @return {@code True} if removed successfully, {@code false} otherwise.
+     */
+    boolean delete(R keyRec);
+
+    /**
+     * Asynchronously deletes a record with the same key columns values as the given one from the table.
+     *
+     * @param keyRec Record with key columns set.
+     * @return Future representing pending completion of the operation.
+     */
+    @NotNull IgniteFuture<Boolean> deleteAsync(R keyRec);
+
+    /**
+     * Deletes the given record from the table.
+     *
+     * @param oldRec Record to delete.
+     * @return {@code True} if removed successfully, {@code false} otherwise.
+     */
+    boolean deleteExact(R oldRec);
+
+    /**
+     * Asynchronously deletes given record from the table.
+     *
+     * @param oldRec Record to delete.
+     * @return Future representing pending completion of the operation.
+     */
+    @NotNull IgniteFuture<Boolean> deleteExactAsync(R oldRec);
+
+    /**
+     * Gets then deletes a record with the same key columns values from the table.
+     *
+     * @param rec Record with key columns set.
+     * @return Removed record or {@code null} if not existed.
+     */
+    R getAndDelete(R rec);
+
+    /**
+     * Asynchronously gets then deletes a record with the same key columns values from the table.
+     *
+     * @param rec Record with key columns set.
+     * @return Future representing pending completion of the operation.
+     */
+    @NotNull IgniteFuture<R> getAndDeleteAsync(R rec);
+
+    /**
+     * Remove records with the same key columns values as the given one has from the table.
+     *
+     * @param recs Records with key columns set.
+     * @return Records with key columns set that were not exists.
+     */
+    Collection<R> deleteAll(Collection<R> recs);
+
+    /**
+     * Asynchronously remove records with the same key columns values as the given one has from the table.
+     *
+     * @param recs Records with key columns set.
+     * @return Future representing pending completion of the operation.
+     */
+    @NotNull IgniteFuture<Collection<R>> deleteAllAsync(Collection<R> recs);
+
+    /**
+     * Remove given records from the table.
+     *
+     * @param recs Records to delete.
+     * @return Records that were not deleted.
+     */
+    Collection<R> deleteAllExact(Collection<R> recs);
+
+    /**
+     * Asynchronously remove given records from the table.
+     *
+     * @param recs Records to delete.
+     * @return Future representing pending completion of the operation.
+     */
+    @NotNull IgniteFuture<Collection<R>> deleteAllExactAsync(Collection<R> recs);
+
+    /**
+     * Executes an InvokeProcessor code against a record with the same key columns values as the given one has.
+     *
+     * @param keyRec Record with key columns set.
+     * @return Results of the processing.
+     */
+    <T extends Serializable> T invoke(R keyRec, InvokeProcessor<R, R, T> proc);
+
+    /**
+     * Asynchronously executes an InvokeProcessor code against a record
+     * with the same key columns values as the given one has.
+     *
+     * @param keyRec Record with key columns set.
+     * @return Future representing pending completion of the operation.
+     */
+    @NotNull <T extends Serializable> IgniteFuture<T> invokeAsync(R keyRec, InvokeProcessor<R, R, T> proc);
+
+    /**
+     * Executes an InvokeProcessor code against records with the same key columns values as the given ones has.
+     *
+     * @param keyRecs Records with key columns set.
+     * @return Results of the processing.
+     */
+    <T extends Serializable> Map<R, T> invokeAll(Collection<R> keyRecs, InvokeProcessor<R, R, T> proc);
+
+    /**
+     * Asynchronously executes an InvokeProcessor against records with the same key columns values as the given ones has.
+     *
+     * @param keyRecs Records with key columns set.
+     * @return Results of the processing.
+     */
+    @NotNull <T extends Serializable> IgniteFuture<Map<R, T>> invokeAllAsync(Collection<R> keyRecs,
+        InvokeProcessor<R, R, T> proc);
+}
diff --git a/modules/api/src/main/java/org/apache/ignite/table/Tuple.java b/modules/api/src/main/java/org/apache/ignite/table/Tuple.java
new file mode 100644
index 0000000..40fda9d
--- /dev/null
+++ b/modules/api/src/main/java/org/apache/ignite/table/Tuple.java
@@ -0,0 +1,100 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.table;
+
+import org.apache.ignite.binary.BinaryObject;
+
+/**
+ * Tuple represents arbitrary set of columns whose values is accessible by column name.
+ *
+ * Provides specialized method for some value-types to avoid boxing/unboxing.
+ */
+public interface Tuple {
+    /**
+     * Gets column value for given column name.
+     *
+     * @param colName Column name.
+     * @param <T> Value type.
+     * @return Column value.
+     */
+    <T> T value(String colName);
+
+    /**
+     * Gets binary object column.
+     *
+     * @param colName Column name.
+     * @return Column value.
+     */
+    BinaryObject binaryObjectField(String colName);
+
+    /**
+     * Gets {@code byte} column value.
+     *
+     * @param colName Column name.
+     * @return Column value.
+     */
+    byte byteValue(String colName);
+
+    /**
+     * Gets {@code short} column value.
+     *
+     * @param colName Column name.
+     * @return Column value.
+     */
+    short shortValue(String colName);
+
+    /**
+     * Gets {@code int} column value.
+     *
+     * @param colName Column name.
+     * @return Column value.
+     */
+    int intValue(String colName);
+
+    /**
+     * Gets {@code long} column value.
+     *
+     * @param colName Column name.
+     * @return Column value.
+     */
+    long longValue(String colName);
+
+    /**
+     * Gets {@code float} column value.
+     *
+     * @param colName Column name.
+     * @return Column value.
+     */
+    float floatValue(String colName);
+
+    /**
+     * Gets {@code double} column value.
+     *
+     * @param colName Column name.
+     * @return Column value.
+     */
+    double doubleValue(String colName);
+
+    /**
+     * Gets {@code String} column value.
+     *
+     * @param colName Column name.
+     * @return Column value.
+     */
+    String stringValue(String colName);
+}
diff --git a/modules/schema/src/main/java/org/apache/ignite/internal/schema/InvalidTypeException.java b/modules/api/src/main/java/org/apache/ignite/table/TupleBuilder.java
similarity index 70%
copy from modules/schema/src/main/java/org/apache/ignite/internal/schema/InvalidTypeException.java
copy to modules/api/src/main/java/org/apache/ignite/table/TupleBuilder.java
index d03ca73..266f0d5 100644
--- a/modules/schema/src/main/java/org/apache/ignite/internal/schema/InvalidTypeException.java
+++ b/modules/api/src/main/java/org/apache/ignite/table/TupleBuilder.java
@@ -15,16 +15,23 @@
  * limitations under the License.
  */
 
-package org.apache.ignite.internal.schema;
+package org.apache.ignite.table;
 
 /**
- * An exception thrown when an attempt to read an invalid type from a tuple is performed.
+ * Tuple builder interface.
  */
-public class InvalidTypeException extends IllegalArgumentException {
+public interface TupleBuilder {
     /**
-     * @param msg Error message.
+     * Sets column value.
+     *
+     * @param colName Column name.
+     * @param value Value to set.
+     * @return {@code this} for chaining.
      */
-    public InvalidTypeException(String msg) {
-        super(msg);
-    }
+    TupleBuilder set(String colName, Object value);
+
+    /**
+     * @return Tuple.
+     */
+    Tuple build();
 }
diff --git a/modules/schema/src/main/java/org/apache/ignite/internal/schema/package-info.java b/modules/api/src/main/java/org/apache/ignite/table/mapper/KeyMapper.java
similarity index 84%
copy from modules/schema/src/main/java/org/apache/ignite/internal/schema/package-info.java
copy to modules/api/src/main/java/org/apache/ignite/table/mapper/KeyMapper.java
index fe5f0e3..6ab45e9 100644
--- a/modules/schema/src/main/java/org/apache/ignite/internal/schema/package-info.java
+++ b/modules/api/src/main/java/org/apache/ignite/table/mapper/KeyMapper.java
@@ -15,8 +15,10 @@
  * limitations under the License.
  */
 
+package org.apache.ignite.table.mapper;
+
 /**
- * <!-- Package description. -->
- * Contains schema description, tuple assembly and field accessor classes.
+ *
  */
-package org.apache.ignite.internal.schema;
\ No newline at end of file
+public interface KeyMapper<T> {
+}
diff --git a/modules/schema/src/main/java/org/apache/ignite/internal/schema/InvalidTypeException.java b/modules/api/src/main/java/org/apache/ignite/table/mapper/Mappers.java
similarity index 53%
copy from modules/schema/src/main/java/org/apache/ignite/internal/schema/InvalidTypeException.java
copy to modules/api/src/main/java/org/apache/ignite/table/mapper/Mappers.java
index d03ca73..c2e461c 100644
--- a/modules/schema/src/main/java/org/apache/ignite/internal/schema/InvalidTypeException.java
+++ b/modules/api/src/main/java/org/apache/ignite/table/mapper/Mappers.java
@@ -15,16 +15,39 @@
  * limitations under the License.
  */
 
-package org.apache.ignite.internal.schema;
+package org.apache.ignite.table.mapper;
 
 /**
- * An exception thrown when an attempt to read an invalid type from a tuple is performed.
+ * Mappers shortcut methods.
  */
-public class InvalidTypeException extends IllegalArgumentException {
+public final class Mappers {
+    public static <K> KeyMapper<K> ofKeyClass(Class<K> keyCls) {
+        return null;
+    }
+
+    public static <V> ValueMapper<V> ofValueClass(Class<V> keyCls) {
+        return null;
+    }
+
+    public static <V> ValueMapper.Builder<V> ofValueClassBuilder(Class<V> valCls) {
+        return null;
+    }
+
+    public static <R> KeyMapper<R> identity() {
+        return null;
+    }
+
+    public static <R> RecordMapper<R> ofRowClass(Class<R> rowCls) {
+        return null;
+    }
+
+    public static <R> RecordMapper.Builder<R> ofRowClassBuilder(Class<R> targetClass) {
+        return null;
+    }
+
     /**
-     * @param msg Error message.
+     * Stub.
      */
-    public InvalidTypeException(String msg) {
-        super(msg);
+    private Mappers() {
     }
 }
diff --git a/modules/schema/src/main/java/org/apache/ignite/internal/schema/InvalidTypeException.java b/modules/api/src/main/java/org/apache/ignite/table/mapper/RecordMapper.java
similarity index 68%
copy from modules/schema/src/main/java/org/apache/ignite/internal/schema/InvalidTypeException.java
copy to modules/api/src/main/java/org/apache/ignite/table/mapper/RecordMapper.java
index d03ca73..96655df 100644
--- a/modules/schema/src/main/java/org/apache/ignite/internal/schema/InvalidTypeException.java
+++ b/modules/api/src/main/java/org/apache/ignite/table/mapper/RecordMapper.java
@@ -15,16 +15,20 @@
  * limitations under the License.
  */
 
-package org.apache.ignite.internal.schema;
+package org.apache.ignite.table.mapper;
+
+import java.util.function.Function;
+import org.apache.ignite.table.Tuple;
 
 /**
- * An exception thrown when an attempt to read an invalid type from a tuple is performed.
+ *
  */
-public class InvalidTypeException extends IllegalArgumentException {
-    /**
-     * @param msg Error message.
-     */
-    public InvalidTypeException(String msg) {
-        super(msg);
+public interface RecordMapper<R> {
+    public interface Builder<R> {
+        public Builder<R> map(String fieldName, Class<?> targetClass);
+
+        public Builder<R> map(String fieldName, Function<Tuple, Object> mapping);
+
+        public RecordMapper<R> build();
     }
 }
diff --git a/modules/schema/src/main/java/org/apache/ignite/internal/schema/InvalidTypeException.java b/modules/api/src/main/java/org/apache/ignite/table/mapper/ValueMapper.java
similarity index 65%
copy from modules/schema/src/main/java/org/apache/ignite/internal/schema/InvalidTypeException.java
copy to modules/api/src/main/java/org/apache/ignite/table/mapper/ValueMapper.java
index d03ca73..da4b402 100644
--- a/modules/schema/src/main/java/org/apache/ignite/internal/schema/InvalidTypeException.java
+++ b/modules/api/src/main/java/org/apache/ignite/table/mapper/ValueMapper.java
@@ -15,16 +15,22 @@
  * limitations under the License.
  */
 
-package org.apache.ignite.internal.schema;
+package org.apache.ignite.table.mapper;
+
+import java.util.function.Function;
+import org.apache.ignite.table.Tuple;
 
 /**
- * An exception thrown when an attempt to read an invalid type from a tuple is performed.
+ *
  */
-public class InvalidTypeException extends IllegalArgumentException {
-    /**
-     * @param msg Error message.
-     */
-    public InvalidTypeException(String msg) {
-        super(msg);
+public interface ValueMapper<V> {
+    public interface Builder<V> {
+        public Builder<V> deserializeTo(Class<?> cls);
+
+        public Builder<V> map(String fieldName, Class<?> cls);
+
+        public Builder<V> map(String fieldName, Function<Tuple, Object> mapper);
+
+        public ValueMapper<V> build();
     }
 }
diff --git a/modules/schema/src/main/java/org/apache/ignite/internal/schema/package-info.java b/modules/api/src/main/java/org/apache/ignite/table/package-info.java
similarity index 84%
copy from modules/schema/src/main/java/org/apache/ignite/internal/schema/package-info.java
copy to modules/api/src/main/java/org/apache/ignite/table/package-info.java
index fe5f0e3..fe8d7f7 100644
--- a/modules/schema/src/main/java/org/apache/ignite/internal/schema/package-info.java
+++ b/modules/api/src/main/java/org/apache/ignite/table/package-info.java
@@ -16,7 +16,6 @@
  */
 
 /**
- * <!-- Package description. -->
- * Contains schema description, tuple assembly and field accessor classes.
+ * Table access API.
  */
-package org.apache.ignite.internal.schema;
\ No newline at end of file
+package org.apache.ignite.table;
\ No newline at end of file
diff --git a/modules/bytecode/src/main/java/com/facebook/presto/bytecode/DynamicClassLoader.java b/modules/bytecode/src/main/java/com/facebook/presto/bytecode/DynamicClassLoader.java
index c33787c..7ae3f59 100644
--- a/modules/bytecode/src/main/java/com/facebook/presto/bytecode/DynamicClassLoader.java
+++ b/modules/bytecode/src/main/java/com/facebook/presto/bytecode/DynamicClassLoader.java
@@ -36,15 +36,6 @@ public class DynamicClassLoader
         this(parentClassLoader, Map.of());
     }
 
-    // TODO: this is a hack that should be removed
-    @Deprecated
-    public DynamicClassLoader(ClassLoader overrideClassLoader, ClassLoader parentClassLoader)
-    {
-        super(parentClassLoader);
-        this.callSiteBindings = Map.of();
-        this.overrideClassLoader = Optional.of(overrideClassLoader);
-    }
-
     public DynamicClassLoader(ClassLoader parentClassLoader, Map<Long, MethodHandle> callSiteBindings)
     {
         super(parentClassLoader);
diff --git a/modules/bytecode/src/main/java/com/facebook/presto/bytecode/expression/ConstantBytecodeExpression.java b/modules/bytecode/src/main/java/com/facebook/presto/bytecode/expression/ConstantBytecodeExpression.java
index e1fc0f2..ad5739b 100644
--- a/modules/bytecode/src/main/java/com/facebook/presto/bytecode/expression/ConstantBytecodeExpression.java
+++ b/modules/bytecode/src/main/java/com/facebook/presto/bytecode/expression/ConstantBytecodeExpression.java
@@ -54,7 +54,7 @@ class ConstantBytecodeExpression
         if (value instanceof ParameterizedType) {
             return ((ParameterizedType)value).getSimpleName() + ".class";
         }
-        // todo escape string
+        // escape string
         if (value instanceof String) {
             return "\"" + value + "\"";
         }
diff --git a/modules/schema/README.md b/modules/schema/README.md
index e5f396f..7ec07c2 100644
--- a/modules/schema/README.md
+++ b/modules/schema/README.md
@@ -5,7 +5,7 @@ This module provides implementation for schema management components:
 * Public API for schema definition and evolution
 * Schema manager component that implements necessary machinary to translate schema management commands to corresponding
   metastorage modifications, as well as schema modification event processing logic 
-* Necessary logic to build and upgrade tuples - rows of specific schema that encode user data in schema-defined format.
+* Necessary logic to build and upgrade rows of specific schema that encode user data in schema-defined format.
 
 ## Schema-aware tables
 We require that at any moment in time an Ignite table has only one most recent relevant schema. Upon schema 
@@ -13,13 +13,13 @@ modification, we assign a monotonically growing identifier to each version of th
 are provided by the underlying distributed metastorage. The history of schema versions must be kept in the metastorage 
 for a long enough period of time to allow upgrade of all existing data stored in a given table.
               
-Given a schema evolution history, a tuple migration from version `N-k` to version `N` is a straightforward operation. 
+Given a schema evolution history, a row migration from version `N-k` to version `N` is a straightforward operation. 
 We identify fields that were dropped during the last k schema operations and fields that were added (taking into account
-default field values) and update the tuple based on the field modifications. Afterward, the updated tuple is written in 
-the schema version `N` layout format. The tuple upgrade may happen on read with an optional writeback or on next update. 
-Additionally, tuple upgrade in background is possible.
+default field values) and update the row based on the field modifications. Afterward, the updated row is written in 
+the schema version `N` layout format. The row upgrade may happen on read with an optional writeback or on next update. 
+Additionally, row upgrade in background is possible.
               
-Since the tuple key hashcode is inlined to the tuple data for quick key lookups, we require that the set of key columns 
+Since the row key hashcode is inlined to the row data for quick key lookups, we require that the set of key columns 
 do not change during the schema evolution. In the future, we may remove this restriction, but this will require careful 
 hashcode calculation adjustments. Removing a column from the key columns does not seem to be possible since it may 
 produce duplicates, and we assume PK has no duplicates.
diff --git a/modules/schema/src/main/java/org/apache/ignite/internal/schema/AssemblyException.java b/modules/schema/src/main/java/org/apache/ignite/internal/schema/AssemblyException.java
index 8d01c91..b62fc94 100644
--- a/modules/schema/src/main/java/org/apache/ignite/internal/schema/AssemblyException.java
+++ b/modules/schema/src/main/java/org/apache/ignite/internal/schema/AssemblyException.java
@@ -18,7 +18,7 @@
 package org.apache.ignite.internal.schema;
 
 /**
- * The exception is thrown when the tuple assembler encountered an unrecoverable error during the field encoding.
+ * The exception is thrown when the row assembler encountered an unrecoverable error during the field encoding.
  * After the exception is thrown, the assembler remains in an invalid state and should be discarded.
  */
 public class AssemblyException extends RuntimeException {
diff --git a/modules/schema/src/main/java/org/apache/ignite/internal/schema/Bitmask.java b/modules/schema/src/main/java/org/apache/ignite/internal/schema/Bitmask.java
index 1314d32..c3b1eed 100644
--- a/modules/schema/src/main/java/org/apache/ignite/internal/schema/Bitmask.java
+++ b/modules/schema/src/main/java/org/apache/ignite/internal/schema/Bitmask.java
@@ -36,7 +36,7 @@ public class Bitmask extends NativeType {
     }
 
     /**
-     * Creates a bitmask type of size <code>bits</code>. In tuple will round up to the closest full byte.
+     * Creates a bitmask type of size <code>bits</code>. In row will round up to the closest full byte.
      *
      * @param bits The number of bits in the bitmask.
      */
diff --git a/modules/schema/src/main/java/org/apache/ignite/internal/schema/ByteBufferTuple.java b/modules/schema/src/main/java/org/apache/ignite/internal/schema/ByteBufferRow.java
similarity index 76%
rename from modules/schema/src/main/java/org/apache/ignite/internal/schema/ByteBufferTuple.java
rename to modules/schema/src/main/java/org/apache/ignite/internal/schema/ByteBufferRow.java
index 86a4174..001346a 100644
--- a/modules/schema/src/main/java/org/apache/ignite/internal/schema/ByteBufferTuple.java
+++ b/modules/schema/src/main/java/org/apache/ignite/internal/schema/ByteBufferRow.java
@@ -22,16 +22,16 @@ import java.nio.ByteOrder;
 import java.nio.charset.StandardCharsets;
 
 /**
- * Heap byte buffer-based tuple.
+ * Heap byte buffer-based row.
  */
-public class ByteBufferTuple extends Tuple {
+public class ByteBufferRow extends Row {
     /** */
     private final ByteBuffer buf;
 
     /**
-     * @param arr Array representation of the tuple.
+     * @param arr Array representation of the row.
      */
-    public ByteBufferTuple(SchemaDescriptor sch, byte[] arr) {
+    public ByteBufferRow(SchemaDescriptor sch, byte[] arr) {
         super(sch);
 
         buf = ByteBuffer.wrap(arr);
@@ -88,4 +88,24 @@ public class ByteBufferTuple extends Tuple {
     @Override protected String readString(int off, int len) {
         return new String(buf.array(), off, len, StandardCharsets.UTF_8);
     }
+
+    /** {@inheritDoc} */
+    @Override public byte[] rowBytes() {
+        return buf.array();
+    }
+
+    /** {@inheritDoc} */
+    @Override public byte[] keyChunkBytes() {
+        final int len = readInteger(KEY_CHUNK_OFFSET);
+
+        return readBytes(KEY_HASH_FIELD_OFFSET, len); // Includes key-hash.
+    }
+
+    /** {@inheritDoc} */
+    @Override public byte[] valueChunkBytes() {
+        int off = KEY_CHUNK_OFFSET + readInteger(KEY_CHUNK_OFFSET);
+        int len = readInteger(off);
+
+        return readBytes(off, len);
+    }
 }
diff --git a/modules/schema/src/main/java/org/apache/ignite/internal/schema/Columns.java b/modules/schema/src/main/java/org/apache/ignite/internal/schema/Columns.java
index 0773b58..6a3f1df 100644
--- a/modules/schema/src/main/java/org/apache/ignite/internal/schema/Columns.java
+++ b/modules/schema/src/main/java/org/apache/ignite/internal/schema/Columns.java
@@ -21,8 +21,8 @@ import java.util.Arrays;
 import java.util.NoSuchElementException;
 
 /**
- * A set of columns representing a key or a value chunk in tuple. Instance of Columns provides necessary machinery
- * to locate a column value in a concrete tuple.
+ * A set of columns representing a key or a value chunk in a row.
+ * Provides necessary machinery to locate a column value in a concrete row.
  */
 public class Columns {
     /** */
diff --git a/modules/schema/src/main/java/org/apache/ignite/internal/schema/ExpandableByteBuf.java b/modules/schema/src/main/java/org/apache/ignite/internal/schema/ExpandableByteBuf.java
index 542b539..b60655c 100644
--- a/modules/schema/src/main/java/org/apache/ignite/internal/schema/ExpandableByteBuf.java
+++ b/modules/schema/src/main/java/org/apache/ignite/internal/schema/ExpandableByteBuf.java
@@ -26,8 +26,8 @@ import java.nio.charset.CoderResult;
 import java.util.Arrays;
 
 /**
- * A simple byte array wrapper to allow dynamic byte array expansion during the tuple construction. Grows exponentially
- * up to 1MB, then expands by 1 MB each time an expansion is required. Values are always written in LITTLE_ENDIAN
+ * A simple byte array wrapper to allow dynamic byte array expansion during the row construction. Grows exponentially
+ * up to 1 MB, then expands by 1 MB each time an expansion is required. Values are always written in LITTLE_ENDIAN
  * format.
  * <p>
  * Additionally, it tracks the high watermark of the values ever written to the buffer so that only written bytes are
diff --git a/modules/schema/src/main/java/org/apache/ignite/internal/schema/InvalidTypeException.java b/modules/schema/src/main/java/org/apache/ignite/internal/schema/InvalidTypeException.java
index d03ca73..b4345b3 100644
--- a/modules/schema/src/main/java/org/apache/ignite/internal/schema/InvalidTypeException.java
+++ b/modules/schema/src/main/java/org/apache/ignite/internal/schema/InvalidTypeException.java
@@ -18,7 +18,7 @@
 package org.apache.ignite.internal.schema;
 
 /**
- * An exception thrown when an attempt to read an invalid type from a tuple is performed.
+ * An exception thrown when an attempt to read an invalid type from a row is performed.
  */
 public class InvalidTypeException extends IllegalArgumentException {
     /**
diff --git a/modules/schema/src/main/java/org/apache/ignite/internal/schema/NativeTypeSpec.java b/modules/schema/src/main/java/org/apache/ignite/internal/schema/NativeTypeSpec.java
index 72a3e6b..a5ae386 100644
--- a/modules/schema/src/main/java/org/apache/ignite/internal/schema/NativeTypeSpec.java
+++ b/modules/schema/src/main/java/org/apache/ignite/internal/schema/NativeTypeSpec.java
@@ -23,7 +23,7 @@ package org.apache.ignite.internal.schema;
  * bitmask of size <code>n</code> bits or number of max n bytes are created using static methods.
  *
  * An instance of native type provides necessary indirection to read any field as an instance of
- * {@code java.lang.Object} to avoid switching inside the tuple methods.
+ * {@code java.lang.Object} to avoid switching inside the row methods.
  */
 public enum NativeTypeSpec {
     /**
@@ -31,7 +31,7 @@ public enum NativeTypeSpec {
      */
     BYTE("byte", true) {
         /** {@inheritDoc} */
-        @Override public Object objectValue(Tuple tup, int colIdx) {
+        @Override public Object objectValue(Row tup, int colIdx) {
             return tup.byteValueBoxed(colIdx);
         }
     },
@@ -41,7 +41,7 @@ public enum NativeTypeSpec {
      */
     SHORT("short", true) {
         /** {@inheritDoc} */
-        @Override public Object objectValue(Tuple tup, int colIdx) {
+        @Override public Object objectValue(Row tup, int colIdx) {
             return tup.shortValueBoxed(colIdx);
         }
     },
@@ -51,7 +51,7 @@ public enum NativeTypeSpec {
      */
     INTEGER("integer", true) {
         /** {@inheritDoc} */
-        @Override public Object objectValue(Tuple tup, int colIdx) {
+        @Override public Object objectValue(Row tup, int colIdx) {
             return tup.intValueBoxed(colIdx);
         }
     },
@@ -61,7 +61,7 @@ public enum NativeTypeSpec {
      */
     LONG("long", true) {
         /** {@inheritDoc} */
-        @Override public Object objectValue(Tuple tup, int colIdx) {
+        @Override public Object objectValue(Row tup, int colIdx) {
             return tup.longValueBoxed(colIdx);
         }
     },
@@ -71,7 +71,7 @@ public enum NativeTypeSpec {
      */
     FLOAT("float", true) {
         /** {@inheritDoc} */
-        @Override public Object objectValue(Tuple tup, int colIdx) {
+        @Override public Object objectValue(Row tup, int colIdx) {
             return tup.floatValueBoxed(colIdx);
         }
     },
@@ -81,7 +81,7 @@ public enum NativeTypeSpec {
      */
     DOUBLE("double", true) {
         /** {@inheritDoc} */
-        @Override public Object objectValue(Tuple tup, int colIdx) {
+        @Override public Object objectValue(Row tup, int colIdx) {
             return tup.doubleValueBoxed(colIdx);
         }
     },
@@ -91,7 +91,7 @@ public enum NativeTypeSpec {
      */
     UUID("uuid", true) {
         /** {@inheritDoc} */
-        @Override public Object objectValue(Tuple tup, int colIdx) {
+        @Override public Object objectValue(Row tup, int colIdx) {
             return tup.uuidValue(colIdx);
         }
     },
@@ -101,7 +101,7 @@ public enum NativeTypeSpec {
      */
     STRING("string") {
         /** {@inheritDoc} */
-        @Override public Object objectValue(Tuple tup, int colIdx) {
+        @Override public Object objectValue(Row tup, int colIdx) {
             return tup.stringValue(colIdx);
         }
     },
@@ -111,7 +111,7 @@ public enum NativeTypeSpec {
      */
     BYTES("blob") {
         /** {@inheritDoc} */
-        @Override public Object objectValue(Tuple tup, int colIdx) {
+        @Override public Object objectValue(Row tup, int colIdx) {
             return tup.bytesValue(colIdx);
         }
     },
@@ -121,7 +121,7 @@ public enum NativeTypeSpec {
      */
     BITMASK("bitmask", true) {
         /** {@inheritDoc} */
-        @Override public Object objectValue(Tuple tup, int colIdx) {
+        @Override public Object objectValue(Row tup, int colIdx) {
             return tup.bitmaskValue(colIdx);
         }
     };
@@ -160,16 +160,16 @@ public enum NativeTypeSpec {
     }
 
     /**
-     * Indirection method for getting an Object representation of the given type from the tuple. This method
-     * does any type conversions and will throw an exception if tuple schema column type differs from this
+     * Indirection method for getting an Object representation of the given type from the rows. This method
+     * does no type conversions and will throw an exception if row column type differs from this
      * type.
      *
-     * @param tup Tuple to read the value from.
+     * @param row Row to read the value from.
      * @param colIdx Column index to read.
      * @return An Object representation of the value.
      * @throws InvalidTypeException If this native type differs from the actual type of {@code colIdx}.
      */
-    public abstract Object objectValue(Tuple tup, int colIdx) throws InvalidTypeException;
+    public abstract Object objectValue(Row row, int colIdx) throws InvalidTypeException;
 
     /** {@inheritDoc} */
     @Override public String toString() {
diff --git a/modules/schema/src/main/java/org/apache/ignite/internal/schema/README.md b/modules/schema/src/main/java/org/apache/ignite/internal/schema/README.md
index 435c7be..97fea08 100644
--- a/modules/schema/src/main/java/org/apache/ignite/internal/schema/README.md
+++ b/modules/schema/src/main/java/org/apache/ignite/internal/schema/README.md
@@ -1,5 +1,5 @@
 This package provides necessary infrastructure to create, read, convert to and from POJO classes
-schema-defined tuples.
+schema-defined rows.
 
 ### Schema definition
 
@@ -38,12 +38,12 @@ Binary|Variable|Variable-size byte array
 Arbitrary nested object serialization at this point is not supported, but can be provided in the future by either 
 explicit inlining, or by providing an upper-level serialization primitive that will be mapped to a `Binary` column.
 
-### Tuple layout
-A tuple itself does not contain any type metadata and only contains necessary information required for fast column 
-lookup. In a tuple, key columns and value columns are separated and written to chunks with identical structure 
+### Row layout
+A row itself does not contain any type metadata and only contains necessary information required for fast column 
+lookup. In a row, key columns and value columns are separated and written to chunks with identical structure 
 (so that chunk is self-sufficient, and, provided with the column types can be read independently).
 
-Tuple structure has the following format:
+Row structure has the following format:
 
     ┌─────────────────────────────┬─────────────────────┐
     │           Header            │        Data         │
@@ -67,21 +67,21 @@ Each chunk section has the following structure:
     └─────────┴─────────────────────────┴─────────────────┴─────────────────────────┴──────────┴──────────┘
 All columns within a group are split into groups of fixed-size columns and variable-size columns. Withing the group of 
 fixsize columns, the columns are sorted by size, then by column name. Within the group of varsize columns, the columns 
-are sorted by column name. Inside a tuple default values and nulls are omitted and encoded in the null-defaults map 
+are sorted by column name. Inside a row default values and nulls are omitted and encoded in the null-defaults map 
 (essentially, a bitset). The size of the varsize columns offsets table is equal to the number of non-null non-default 
 varsize columns multiplied by 2 (a single entry in the offsets table is 2 bytes). The offset stored in the offsets table 
 is calculated from the beginning of the chunk.
 
-### Tuple construction and access
-To assemble a tuple with some schema, an instance of `org.apache.ignite.internal.schema.TupleAssembler`
-must be used which provides the low-level API for building tuples. When using the tuple assembler, the
+### Row construction and access
+To assemble a row with some schema, an instance of `org.apache.ignite.internal.schema.RowAssembler`
+must be used which provides the low-level API for building rows. When using the row assembler, the
 columns must be passed to the assembler in the internal schema sort order. Additionally, when constructing
-the instance of the assembler, the user should pre-calculate the size of the tuple to avoid extra array copies,
+the instance of the assembler, the user should pre-calculate the size of the row to avoid extra array copies,
 and the number of non-null varlen columns for key and value chunks. Less restrictive building techniques
-are provided by class (de)serializers and tuple builder, which take care of sizing and column order.
+are provided by class (de)serializers and row builder, which take care of sizing and column order.
 
-To read column values of a tuple, one needs to construct a subclass of
-`org.apache.ignite.internal.schema.Tuple` which provides necessary logic to read arbitrary columns with
-type checking. For primitive types, `org.apache.ignite.internal.schema.Tuple` provides boxed and non-boxed
+To read column values of a row, one needs to construct a subclass of
+`org.apache.ignite.internal.schema.Row` which provides necessary logic to read arbitrary columns with
+type checking. For primitive types, `org.apache.ignite.internal.schema.Row` provides boxed and non-boxed
 value methods to avoid boxing in scenarios where boxing can be avoided (deserialization of non-null columns to
 POJO primitives, for example).
diff --git a/modules/schema/src/main/java/org/apache/ignite/internal/schema/Tuple.java b/modules/schema/src/main/java/org/apache/ignite/internal/schema/Row.java
similarity index 85%
rename from modules/schema/src/main/java/org/apache/ignite/internal/schema/Tuple.java
rename to modules/schema/src/main/java/org/apache/ignite/internal/schema/Row.java
index 1d2a8ec..71dce1d 100644
--- a/modules/schema/src/main/java/org/apache/ignite/internal/schema/Tuple.java
+++ b/modules/schema/src/main/java/org/apache/ignite/internal/schema/Row.java
@@ -22,21 +22,21 @@ import java.util.UUID;
 
 /**
  * The class contains non-generic methods to read boxed and unboxed primitives based on the schema column types.
- * Any type conversions and coercions should be implemented outside of the tuple by the key-value or query runtime.
+ * Any type conversions and coercions should be implemented outside the row by the key-value or query runtime.
  * When a non-boxed primitive is read from a null column value, it is converted to the primitive type default value.
  */
-public abstract class Tuple {
+public abstract class Row {
     /** */
-    public static final int SCHEMA_VERSION_FIELD_SIZE = 2;
+    public static final int SCHEMA_VERSION_OFFSET = 0;
 
     /** */
-    public static final int FLAGS_FIELD_SIZE = 2;
+    public static final int FLAGS_FIELD_OFFSET = SCHEMA_VERSION_OFFSET + 2;
 
     /** */
-    public static final int KEY_HASH_FIELD_SIZE = 4;
+    public static final int KEY_HASH_FIELD_OFFSET = FLAGS_FIELD_OFFSET + 2;
 
     /** */
-    public static final int TUPLE_HEADER_SIZE = SCHEMA_VERSION_FIELD_SIZE + FLAGS_FIELD_SIZE + KEY_HASH_FIELD_SIZE;
+    public static final int KEY_CHUNK_OFFSET = KEY_HASH_FIELD_OFFSET + 4;
 
     /** */
     public static final int TOTAL_LEN_FIELD_SIZE = 4;
@@ -47,17 +47,39 @@ public abstract class Tuple {
     /** */
     public static final int VARLEN_COLUMN_OFFSET_FIELD_SIZE = 2;
 
-    /** Schema descriptor for which this tuple was created. */
+    /** */
+    public static final class RowFlags {
+        /** Tombstone flag. */
+        public static final int TOMBSTONE = 1;
+
+        /** Null-value flag. */
+        public static final int NULL_VALUE = 1 << 1;
+
+        /** Stub. */
+        private RowFlags() {
+        }
+    }
+
+    /** Schema descriptor for which this row was created. */
     private final SchemaDescriptor schema;
 
     /**
      * @param schema Schema instance.
      */
-    protected Tuple(SchemaDescriptor schema) {
+    protected Row(SchemaDescriptor schema) {
         this.schema = schema;
     }
 
     /**
+     * @return {@code True} if row has non-null value, {@code false} otherwise.
+     */
+    public boolean hasValue() {
+        short flags = readShort(FLAGS_FIELD_OFFSET);
+
+        return (flags & (RowFlags.NULL_VALUE | RowFlags.TOMBSTONE)) == 0;
+    }
+
+    /**
      */
     public byte byteValue(int col) {
         long off = findColumn(col, NativeTypeSpec.BYTE);
@@ -214,7 +236,7 @@ public abstract class Tuple {
 
     /**
      * Gets the column offset and length encoded into a single 8-byte value (4 least significant bytes encoding the
-     * offset from the beginning of the tuple and 4 most significant bytes encoding the field length for varlength
+     * offset from the beginning of the row and 4 most significant bytes encoding the field length for varlength
      * columns). The offset and length should be extracted using {@link #offset(long)} and {@link #length(long)}
      * methods.
      * Will also validate that the actual column type matches the requested column type, throwing
@@ -232,7 +254,7 @@ public abstract class Tuple {
         boolean keyCol = schema.keyColumn(colIdx);
         Columns cols = keyCol ? schema.keyColumns() : schema.valueColumns();
 
-        int off = TUPLE_HEADER_SIZE;
+        int off = KEY_CHUNK_OFFSET;
 
         if (!keyCol) {
             // Jump to the next chunk, the size of the first chunk is written at the chunk start.
@@ -257,9 +279,9 @@ public abstract class Tuple {
     }
 
     /**
-     * Checks the tuple's null map for the given column index in the chunk.
+     * Checks the row's null map for the given column index in the chunk.
      *
-     * @param baseOff Offset of the chunk start in the tuple.
+     * @param baseOff Offset of the chunk start in the row.
      * @param idx Offset of the column in the chunk.
      * @return {@code true} if the column value is {@code null}.
      */
@@ -276,10 +298,10 @@ public abstract class Tuple {
 
     /**
      * Utility method to extract the column offset from the {@link #findColumn(int, NativeTypeSpec)} result. The
-     * offset is calculated from the beginning of the tuple.
+     * offset is calculated from the beginning of the row.
      *
      * @param offLen {@code findColumn} invocation result.
-     * @return Column offset from the beginning of the tuple.
+     * @return Column offset from the beginning of the row.
      */
     private static int offset(long offLen) {
         return (int)offLen;
@@ -306,7 +328,7 @@ public abstract class Tuple {
      * @param cols Columns chunk.
      * @param baseOff Chunk base offset.
      * @param idx Column index in the chunk.
-     * @return Encoded offset (from the tuple start) and length of the column with the given index.
+     * @return Encoded offset (from the row start) and length of the column with the given index.
      */
     private long varlenColumnOffsetAndLength(Columns cols, int baseOff, int idx) {
         int nullMapOff = nullMapOffset(baseOff);
@@ -349,13 +371,13 @@ public abstract class Tuple {
     }
 
     /**
-     * Calculates the offset of the fixlen column with the given index in the tuple. It essentially folds the null map
+     * Calculates the offset of the fixlen column with the given index in the row. It essentially folds the null map
      * with the column lengths to calculate the size of non-null columns preceding the requested column.
      *
      * @param cols Columns chunk.
      * @param baseOff Chunk base offset.
      * @param idx Column index in the chunk.
-     * @return Encoded offset (from the tuple start) of the requested fixlen column.
+     * @return Encoded offset (from the row start) of the requested fixlen column.
      */
     int fixlenColumnOffset(Columns cols, int baseOff, int idx) {
         int nullMapOff = nullMapOffset(baseOff);
@@ -379,7 +401,7 @@ public abstract class Tuple {
 
     /**
      * @param baseOff Chunk base offset.
-     * @return Null map offset from the tuple start for the chunk with the given base.
+     * @return Null map offset from the row start for the chunk with the given base.
      */
     private int nullMapOffset(int baseOff) {
         int varlenTblSize = readShort(baseOff + TOTAL_LEN_FIELD_SIZE);
@@ -389,7 +411,7 @@ public abstract class Tuple {
 
     /**
      * @param baseOff Chunk base offset.
-     * @return Offset of the varlen table from the tuple start for the chunk with the given base.
+     * @return Offset of the varlen table from the row start for the chunk with the given base.
      */
     private int vartableOffset(int baseOff) {
         return baseOff + TOTAL_LEN_FIELD_SIZE + VARLEN_TABLE_SIZE_FIELD_SIZE;
@@ -426,4 +448,16 @@ public abstract class Tuple {
     /**
      */
     protected abstract byte[] readBytes(int off, int len);
+
+    /**
+     */
+    public abstract byte[] rowBytes();
+
+    /**
+     */
+    public abstract byte[] keyChunkBytes();
+
+    /**
+     */
+    public abstract byte[] valueChunkBytes();
 }
diff --git a/modules/schema/src/main/java/org/apache/ignite/internal/schema/TupleAssembler.java b/modules/schema/src/main/java/org/apache/ignite/internal/schema/RowAssembler.java
similarity index 90%
rename from modules/schema/src/main/java/org/apache/ignite/internal/schema/TupleAssembler.java
rename to modules/schema/src/main/java/org/apache/ignite/internal/schema/RowAssembler.java
index be4b4b7..9095596 100644
--- a/modules/schema/src/main/java/org/apache/ignite/internal/schema/TupleAssembler.java
+++ b/modules/schema/src/main/java/org/apache/ignite/internal/schema/RowAssembler.java
@@ -24,17 +24,17 @@ import java.util.BitSet;
 import java.util.UUID;
 
 /**
- * Utility class to build tuples using column appending pattern. The external user of this class must consult
- * with the schema and provide the columns in strict internal column sort order during the tuple construction.
- * Additionally, the user of this class should pre-calculate the resulting tuple size when possible to avoid
- * unnecessary data copies. The assembler provides some utility methods to calculate the resulting tuple size
+ * Utility class to build rows using column appending pattern. The external user of this class must consult
+ * with the schema and provide the columns in strict internal column sort order during the row construction.
+ * Additionally, the user of this class should pre-calculate the resulting row size when possible to avoid
+ * unnecessary data copies. The assembler provides some utility methods to calculate the resulting row size
  * based on the number of null columns and size calculation for strings.
  *
- * @see #tupleSize(Columns, int, int, Columns, int, int)
- * @see #tupleChunkSize(Columns, int, int)
+ * @see #rowChunkSize(Columns, int, int, Columns, int, int)
+ * @see #rowChunkSize(Columns, int, int)
  * @see #utf8EncodedLength(CharSequence)
  */
-public class TupleAssembler {
+public class RowAssembler {
     /**
      *
      */
@@ -106,10 +106,10 @@ public class TupleAssembler {
      * @param cols Columns.
      * @param nonNullVarlenCols Number of non-null varlen columns in chunk.
      * @param nonNullVarlenSize Size of non-null varlen columns in chunk.
-     * @return Tuple's chunk size.
+     * @return Row's chunk size.
      */
-    public static int tupleChunkSize(Columns cols, int nonNullVarlenCols, int nonNullVarlenSize) {
-        int size = Tuple.TOTAL_LEN_FIELD_SIZE + Tuple.VARLEN_TABLE_SIZE_FIELD_SIZE +
+    public static int rowChunkSize(Columns cols, int nonNullVarlenCols, int nonNullVarlenSize) {
+        int size = Row.TOTAL_LEN_FIELD_SIZE + Row.VARLEN_TABLE_SIZE_FIELD_SIZE +
             varlenTableSize(nonNullVarlenCols) + cols.nullMapSize();
 
         for (int i = 0; i < cols.numberOfFixsizeColumns(); i++)
@@ -119,13 +119,13 @@ public class TupleAssembler {
     }
 
     /**
-     * @param schema Tuple schema.
-     * @param size Target tuple size. If the tuple size is known in advance, it should be provided upfront to avoid
+     * @param schema Row schema.
+     * @param size Target row size. If the row size is known in advance, it should be provided upfront to avoid
      * unnecessary arrays copy.
      * @param nonNullVarlenKeyCols Number of non-null varlen columns in key chunk.
      * @param nonNullVarlenValCols Number of non-null varlen columns in value chunk.
      */
-    public TupleAssembler(
+    public RowAssembler(
         SchemaDescriptor schema,
         int size,
         int nonNullVarlenKeyCols,
@@ -139,7 +139,7 @@ public class TupleAssembler {
 
         curCols = schema.keyColumns();
 
-        initOffsets(Tuple.TUPLE_HEADER_SIZE, nonNullVarlenKeyCols);
+        initOffsets(Row.KEY_CHUNK_OFFSET, nonNullVarlenKeyCols);
 
         buf.putShort(0, (short)schema.version());
     }
@@ -151,9 +151,9 @@ public class TupleAssembler {
      * @param valCols Value columns.
      * @param nonNullVarlenValCols Number of non-null varlen columns in value chunk.
      * @param nonNullVarlenValSize Size of non-null varlen columns in value chunk.
-     * @return Total tuple size.
+     * @return Total row size.
      */
-    public static int tupleSize(
+    public static int rowChunkSize(
         Columns keyCols,
         int nonNullVarlenKeyCols,
         int nonNullVarlenKeySize,
@@ -161,9 +161,9 @@ public class TupleAssembler {
         int nonNullVarlenValCols,
         int nonNullVarlenValSize
     ) {
-        return Tuple.TUPLE_HEADER_SIZE +
-            tupleChunkSize(keyCols, nonNullVarlenKeyCols, nonNullVarlenKeySize) +
-            tupleChunkSize(valCols, nonNullVarlenValCols, nonNullVarlenValSize);
+        return Row.KEY_CHUNK_OFFSET +
+            rowChunkSize(keyCols, nonNullVarlenKeyCols, nonNullVarlenKeySize) +
+            rowChunkSize(valCols, nonNullVarlenValCols, nonNullVarlenValSize);
     }
 
     /**
@@ -335,7 +335,7 @@ public class TupleAssembler {
     }
 
     /**
-     * @return Serialized tuple.
+     * @return Serialized row.
      */
     public byte[] build() {
         return buf.toArray();
@@ -444,9 +444,9 @@ public class TupleAssembler {
         curCol = 0;
         curVarlenTblEntry = 0;
 
-        buf.putShort(baseOff + Tuple.TOTAL_LEN_FIELD_SIZE, (short)nonNullVarlenCols);
+        buf.putShort(baseOff + Row.TOTAL_LEN_FIELD_SIZE, (short)nonNullVarlenCols);
 
-        varlenTblOff = baseOff + Tuple.TOTAL_LEN_FIELD_SIZE + Tuple.VARLEN_TABLE_SIZE_FIELD_SIZE;
+        varlenTblOff = baseOff + Row.TOTAL_LEN_FIELD_SIZE + Row.VARLEN_TABLE_SIZE_FIELD_SIZE;
         nullMapOff = varlenTblOff + varlenTableSize(nonNullVarlenCols);
         curOff = nullMapOff + curCols.nullMapSize();
     }
diff --git a/modules/schema/src/main/java/org/apache/ignite/internal/schema/SchemaDescriptor.java b/modules/schema/src/main/java/org/apache/ignite/internal/schema/SchemaDescriptor.java
index f2855b1..7eff0dd 100644
--- a/modules/schema/src/main/java/org/apache/ignite/internal/schema/SchemaDescriptor.java
+++ b/modules/schema/src/main/java/org/apache/ignite/internal/schema/SchemaDescriptor.java
@@ -17,6 +17,12 @@
 
 package org.apache.ignite.internal.schema;
 
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.Map;
+import org.jetbrains.annotations.NotNull;
+import org.jetbrains.annotations.Nullable;
+
 /**
  * Full schema descriptor containing key columns chunk, value columns chunk, and schema version.
  */
@@ -30,6 +36,9 @@ public class SchemaDescriptor {
     /** Value columns in serialization order. */
     private final Columns valCols;
 
+    /** Mapping 'Column name' -> Column. */
+    private final Map<String, Column> colMap;
+
     /**
      * @param ver Schema version.
      * @param keyCols Key columns.
@@ -39,6 +48,11 @@ public class SchemaDescriptor {
         this.ver = ver;
         this.keyCols = new Columns(0, keyCols);
         this.valCols = new Columns(keyCols.length, valCols);
+
+        colMap = new HashMap<>(keyCols.length + valCols.length);
+
+        Arrays.stream(keyCols).forEach(c -> colMap.put(c.name(), c));
+        Arrays.stream(valCols).forEach(c -> colMap.put(c.name(), c));
     }
 
     /**
@@ -84,4 +98,12 @@ public class SchemaDescriptor {
     public int length() {
         return keyCols.length() + valCols.length();
     }
+
+    /**
+     * @param name Column name.
+     * @return Column.
+     */
+    public @Nullable Column column(@NotNull String name) {
+        return colMap.get(name);
+    }
 }
diff --git a/modules/schema/src/main/java/org/apache/ignite/internal/schema/marshaller/AbstractSerializer.java b/modules/schema/src/main/java/org/apache/ignite/internal/schema/marshaller/AbstractSerializer.java
index 5ac127c..02dfe3c 100644
--- a/modules/schema/src/main/java/org/apache/ignite/internal/schema/marshaller/AbstractSerializer.java
+++ b/modules/schema/src/main/java/org/apache/ignite/internal/schema/marshaller/AbstractSerializer.java
@@ -18,10 +18,10 @@
 package org.apache.ignite.internal.schema.marshaller;
 
 import java.util.Objects;
-import org.apache.ignite.internal.schema.ByteBufferTuple;
+import org.apache.ignite.internal.schema.ByteBufferRow;
+import org.apache.ignite.internal.schema.Row;
+import org.apache.ignite.internal.schema.RowAssembler;
 import org.apache.ignite.internal.schema.SchemaDescriptor;
-import org.apache.ignite.internal.schema.Tuple;
-import org.apache.ignite.internal.schema.TupleAssembler;
 import org.apache.ignite.internal.util.Pair;
 import org.jetbrains.annotations.Nullable;
 
@@ -43,66 +43,66 @@ public abstract class AbstractSerializer implements Serializer {
 
     /** {@inheritDoc} */
     @Override public byte[] serialize(Object key, Object val) throws SerializationException {
-        final TupleAssembler assembler = createAssembler(Objects.requireNonNull(key), val);
+        final RowAssembler assembler = createAssembler(Objects.requireNonNull(key), val);
 
         return serialize0(assembler, key, val);
     }
 
     /** {@inheritDoc} */
     @Override public <K> K deserializeKey(byte[] data) throws SerializationException {
-        final Tuple tuple = new ByteBufferTuple(schema, data);
+        final Row row = new ByteBufferRow(schema, data);
 
-        return (K)deserializeKey0(tuple);
+        return (K)deserializeKey0(row);
     }
 
     /** {@inheritDoc} */
     @Override public <V> V deserializeValue(byte[] data) throws SerializationException {
-        final Tuple tuple = new ByteBufferTuple(schema, data);
+        final Row row = new ByteBufferRow(schema, data);
 
-        return (V)deserializeValue0(tuple);
+        return (V)deserializeValue0(row);
     }
 
     /** {@inheritDoc} */
     @Override public <K, V> Pair<K, V> deserialize(byte[] data) throws SerializationException {
-        final Tuple tuple = new ByteBufferTuple(schema, data);
+        final Row row = new ByteBufferRow(schema, data);
 
-        return new Pair<>((K)deserializeKey0(tuple), (V)deserializeValue0(tuple));
+        return new Pair<>((K)deserializeKey0(row), (V)deserializeValue0(row));
     }
 
     /**
-     * Tuple assembler factory method.
+     * Row assembler factory method.
      *
      * @param key Key object.
      * @param val Value object.
      */
-    protected abstract TupleAssembler createAssembler(Object key, @Nullable Object val);
+    protected abstract RowAssembler createAssembler(Object key, @Nullable Object val);
 
     /**
      * Internal serialization method.
      *
-     * @param asm Tuple assembler.
+     * @param asm Row assembler.
      * @param key Key object.
      * @param val Value object.
      * @return Serialized pair.
      * @throws SerializationException If failed.
      */
-    protected abstract byte[] serialize0(TupleAssembler asm, Object key, Object val) throws SerializationException;
+    protected abstract byte[] serialize0(RowAssembler asm, Object key, Object val) throws SerializationException;
 
     /**
-     * Extract key object from tuple.
+     * Extract key object from row.
      *
-     * @param tuple Tuple.
+     * @param row Row.
      * @return Deserialized key object.
      * @throws SerializationException If failed.
      */
-    protected abstract Object deserializeKey0(Tuple tuple) throws SerializationException;
+    protected abstract Object deserializeKey0(Row row) throws SerializationException;
 
     /**
-     * Extract value object from tuple.
+     * Extract value object from row.
      *
-     * @param tuple Tuple.
+     * @param row Row.
      * @return Deserialized value object.
      * @throws SerializationException If failed.
      */
-    protected abstract Object deserializeValue0(Tuple tuple) throws SerializationException;
+    protected abstract Object deserializeValue0(Row row) throws SerializationException;
 }
diff --git a/modules/schema/src/main/java/org/apache/ignite/internal/schema/marshaller/MarshallerUtil.java b/modules/schema/src/main/java/org/apache/ignite/internal/schema/marshaller/MarshallerUtil.java
index 9361fa0..883c986 100644
--- a/modules/schema/src/main/java/org/apache/ignite/internal/schema/marshaller/MarshallerUtil.java
+++ b/modules/schema/src/main/java/org/apache/ignite/internal/schema/marshaller/MarshallerUtil.java
@@ -20,7 +20,7 @@ package org.apache.ignite.internal.schema.marshaller;
 import java.util.BitSet;
 import java.util.UUID;
 import org.apache.ignite.internal.schema.NativeType;
-import org.apache.ignite.internal.schema.TupleAssembler;
+import org.apache.ignite.internal.schema.RowAssembler;
 import org.apache.ignite.internal.util.ObjectFactory;
 
 /**
@@ -40,7 +40,7 @@ public final class MarshallerUtil {
                 return ((byte[])val).length;
 
             case STRING:
-                return TupleAssembler.utf8EncodedLength((CharSequence)val);
+                return RowAssembler.utf8EncodedLength((CharSequence)val);
 
             default:
                 throw new IllegalStateException("Unsupported test varsize type: " + type);
diff --git a/modules/schema/src/main/java/org/apache/ignite/internal/schema/marshaller/Serializer.java b/modules/schema/src/main/java/org/apache/ignite/internal/schema/marshaller/Serializer.java
index e043d0f..c4ffcb2 100644
--- a/modules/schema/src/main/java/org/apache/ignite/internal/schema/marshaller/Serializer.java
+++ b/modules/schema/src/main/java/org/apache/ignite/internal/schema/marshaller/Serializer.java
@@ -24,7 +24,7 @@ import org.apache.ignite.internal.util.Pair;
  */
 public interface Serializer {
     /**
-     * Writes key-value pair to tuple.
+     * Writes key-value pair to row.
      *
      * @param key Key object.
      * @param val Value object.
diff --git a/modules/schema/src/main/java/org/apache/ignite/internal/schema/marshaller/asm/AsmSerializerGenerator.java b/modules/schema/src/main/java/org/apache/ignite/internal/schema/marshaller/asm/AsmSerializerGenerator.java
index d62332d..c365900 100644
--- a/modules/schema/src/main/java/org/apache/ignite/internal/schema/marshaller/asm/AsmSerializerGenerator.java
+++ b/modules/schema/src/main/java/org/apache/ignite/internal/schema/marshaller/asm/AsmSerializerGenerator.java
@@ -40,9 +40,9 @@ import jdk.jfr.Experimental;
 import org.apache.ignite.internal.schema.Column;
 import org.apache.ignite.internal.schema.Columns;
 import org.apache.ignite.internal.schema.NativeType;
+import org.apache.ignite.internal.schema.Row;
+import org.apache.ignite.internal.schema.RowAssembler;
 import org.apache.ignite.internal.schema.SchemaDescriptor;
-import org.apache.ignite.internal.schema.Tuple;
-import org.apache.ignite.internal.schema.TupleAssembler;
 import org.apache.ignite.internal.schema.marshaller.AbstractSerializer;
 import org.apache.ignite.internal.schema.marshaller.BinaryMode;
 import org.apache.ignite.internal.schema.marshaller.MarshallerUtil;
@@ -190,7 +190,7 @@ public class AsmSerializerGenerator implements SerializerFactory {
         if (mode == null)
             return new ObjectMarshallerCodeGenerator(columns, tClass, firstColIdx);
         else
-            return new IdentityMarshallerCodeGenerator(tClass, TupleColumnAccessCodeGenerator.createAccessor(mode, firstColIdx));
+            return new IdentityMarshallerCodeGenerator(tClass, ColumnAccessCodeGenerator.createAccessor(mode, firstColIdx));
     }
 
     /**
@@ -235,7 +235,7 @@ public class AsmSerializerGenerator implements SerializerFactory {
         final MethodDefinition methodDef = classDef.declareMethod(
             EnumSet.of(Access.PROTECTED),
             "createAssembler",
-            ParameterizedType.type(TupleAssembler.class),
+            ParameterizedType.type(RowAssembler.class),
             Parameter.arg("key", Object.class),
             Parameter.arg("val", Object.class)
         );
@@ -299,9 +299,9 @@ public class AsmSerializerGenerator implements SerializerFactory {
             }
         }
 
-        body.append(BytecodeExpressions.newInstance(TupleAssembler.class,
+        body.append(BytecodeExpressions.newInstance(RowAssembler.class,
             methodDef.getThis().getField("schema", SchemaDescriptor.class),
-            BytecodeExpressions.invokeStatic(TupleAssembler.class, "tupleSize", int.class,
+            BytecodeExpressions.invokeStatic(RowAssembler.class, "rowChunkSize", int.class,
                 keyCols, varlenKeyCols, varlenKeyColsSize,
                 valCols, varlenValueCols, varlenValueColsSize),
             varlenKeyCols,
@@ -326,7 +326,7 @@ public class AsmSerializerGenerator implements SerializerFactory {
             EnumSet.of(Access.PROTECTED),
             "serialize0",
             ParameterizedType.type(byte[].class),
-            Parameter.arg("asm", TupleAssembler.class),
+            Parameter.arg("asm", RowAssembler.class),
             Parameter.arg("key", Object.class),
             Parameter.arg("val", Object.class)
         ).addException(SerializationException.class);
@@ -380,7 +380,7 @@ public class AsmSerializerGenerator implements SerializerFactory {
             EnumSet.of(Access.PROTECTED),
             "deserializeKey0",
             ParameterizedType.type(Object.class),
-            Parameter.arg("tuple", Tuple.class)
+            Parameter.arg("row", Row.class)
         ).addException(SerializationException.class);
 
         methodDef.declareAnnotation(Override.class);
@@ -392,7 +392,7 @@ public class AsmSerializerGenerator implements SerializerFactory {
                 .invoke("create", Object.class)));
 
         methodDef.getBody()
-            .append(keyMarsh.unmarshallObject(classDef.getType(), methodDef.getScope().getVariable("tuple"), obj))
+            .append(keyMarsh.unmarshallObject(classDef.getType(), methodDef.getScope().getVariable("row"), obj))
             .append(obj)
             .retObject();
     }
@@ -408,7 +408,7 @@ public class AsmSerializerGenerator implements SerializerFactory {
             EnumSet.of(Access.PROTECTED),
             "deserializeValue0",
             ParameterizedType.type(Object.class),
-            Parameter.arg("tuple", Tuple.class)
+            Parameter.arg("row", Row.class)
         ).addException(SerializationException.class);
 
         methodDef.declareAnnotation(Override.class);
@@ -420,7 +420,7 @@ public class AsmSerializerGenerator implements SerializerFactory {
             block.append(obj.set(methodDef.getThis().getField("valFactory", ObjectFactory.class)
                 .invoke("create", Object.class)));
 
-        block.append(valMarsh.unmarshallObject(classDef.getType(), methodDef.getScope().getVariable("tuple"), obj))
+        block.append(valMarsh.unmarshallObject(classDef.getType(), methodDef.getScope().getVariable("row"), obj))
             .append(obj)
             .retObject();
 
diff --git a/modules/schema/src/main/java/org/apache/ignite/internal/schema/marshaller/asm/TupleColumnAccessCodeGenerator.java b/modules/schema/src/main/java/org/apache/ignite/internal/schema/marshaller/asm/ColumnAccessCodeGenerator.java
similarity index 59%
rename from modules/schema/src/main/java/org/apache/ignite/internal/schema/marshaller/asm/TupleColumnAccessCodeGenerator.java
rename to modules/schema/src/main/java/org/apache/ignite/internal/schema/marshaller/asm/ColumnAccessCodeGenerator.java
index c4a715a..87cb0f2 100644
--- a/modules/schema/src/main/java/org/apache/ignite/internal/schema/marshaller/asm/TupleColumnAccessCodeGenerator.java
+++ b/modules/schema/src/main/java/org/apache/ignite/internal/schema/marshaller/asm/ColumnAccessCodeGenerator.java
@@ -22,48 +22,48 @@ import java.util.UUID;
 import org.apache.ignite.internal.schema.marshaller.BinaryMode;
 
 /**
- * Tuple access code generator.
+ * Row access code generator.
  */
-public class TupleColumnAccessCodeGenerator {
+public class ColumnAccessCodeGenerator {
     /**
      * @param mode Binary mode.
      * @param colIdx Column index in schema.
-     * @return Tuple column access code generator.
+     * @return Row column access code generator.
      */
-    public static TupleColumnAccessCodeGenerator createAccessor(BinaryMode mode, int colIdx) {
+    public static ColumnAccessCodeGenerator createAccessor(BinaryMode mode, int colIdx) {
         switch (mode) {
             case P_BYTE:
-                return new TupleColumnAccessCodeGenerator("byteValue", "appendByte", byte.class, colIdx);
+                return new ColumnAccessCodeGenerator("byteValue", "appendByte", byte.class, colIdx);
             case P_SHORT:
-                return new TupleColumnAccessCodeGenerator("shortValue", "appendShort", short.class, colIdx);
+                return new ColumnAccessCodeGenerator("shortValue", "appendShort", short.class, colIdx);
             case P_INT:
-                return new TupleColumnAccessCodeGenerator("intValue", "appendInt", int.class, colIdx);
+                return new ColumnAccessCodeGenerator("intValue", "appendInt", int.class, colIdx);
             case P_LONG:
-                return new TupleColumnAccessCodeGenerator("longValue", "appendLong", long.class, colIdx);
+                return new ColumnAccessCodeGenerator("longValue", "appendLong", long.class, colIdx);
             case P_FLOAT:
-                return new TupleColumnAccessCodeGenerator("floatValue", "appendFloat", float.class, colIdx);
+                return new ColumnAccessCodeGenerator("floatValue", "appendFloat", float.class, colIdx);
             case P_DOUBLE:
-                return new TupleColumnAccessCodeGenerator("doubleValue", "appendDouble", double.class, colIdx);
+                return new ColumnAccessCodeGenerator("doubleValue", "appendDouble", double.class, colIdx);
             case BYTE:
-                return new TupleColumnAccessCodeGenerator("byteValueBoxed", "appendByte", Byte.class, byte.class, colIdx);
+                return new ColumnAccessCodeGenerator("byteValueBoxed", "appendByte", Byte.class, byte.class, colIdx);
             case SHORT:
-                return new TupleColumnAccessCodeGenerator("shortValueBoxed", "appendShort", Short.class, short.class, colIdx);
+                return new ColumnAccessCodeGenerator("shortValueBoxed", "appendShort", Short.class, short.class, colIdx);
             case INT:
-                return new TupleColumnAccessCodeGenerator("intValueBoxed", "appendInt", Integer.class, int.class, colIdx);
+                return new ColumnAccessCodeGenerator("intValueBoxed", "appendInt", Integer.class, int.class, colIdx);
             case LONG:
-                return new TupleColumnAccessCodeGenerator("longValueBoxed", "appendLong", Long.class, long.class, colIdx);
+                return new ColumnAccessCodeGenerator("longValueBoxed", "appendLong", Long.class, long.class, colIdx);
             case FLOAT:
-                return new TupleColumnAccessCodeGenerator("floatValueBoxed", "appendFloat", Float.class, float.class, colIdx);
+                return new ColumnAccessCodeGenerator("floatValueBoxed", "appendFloat", Float.class, float.class, colIdx);
             case DOUBLE:
-                return new TupleColumnAccessCodeGenerator("doubleValueBoxed", "appendDouble", Double.class, double.class, colIdx);
+                return new ColumnAccessCodeGenerator("doubleValueBoxed", "appendDouble", Double.class, double.class, colIdx);
             case STRING:
-                return new TupleColumnAccessCodeGenerator("stringValue", "appendString", String.class, colIdx);
+                return new ColumnAccessCodeGenerator("stringValue", "appendString", String.class, colIdx);
             case UUID:
-                return new TupleColumnAccessCodeGenerator("uuidValue", "appendUuid", UUID.class, colIdx);
+                return new ColumnAccessCodeGenerator("uuidValue", "appendUuid", UUID.class, colIdx);
             case BYTE_ARR:
-                return new TupleColumnAccessCodeGenerator("bytesValue", "appendBytes", byte[].class, colIdx);
+                return new ColumnAccessCodeGenerator("bytesValue", "appendBytes", byte[].class, colIdx);
             case BITSET:
-                return new TupleColumnAccessCodeGenerator("bitmaskValue", "appendBitmask", BitSet.class, colIdx);
+                return new ColumnAccessCodeGenerator("bitmaskValue", "appendBitmask", BitSet.class, colIdx);
         }
 
         throw new IllegalStateException("Unsupported binary mode: " + mode);
@@ -92,7 +92,7 @@ public class TupleColumnAccessCodeGenerator {
      * @param mappedType Mapped value type.
      * @param colIdx Column index in schema.
      */
-    TupleColumnAccessCodeGenerator(String readMethodName, String writeMethodName, Class<?> mappedType, int colIdx) {
+    ColumnAccessCodeGenerator(String readMethodName, String writeMethodName, Class<?> mappedType, int colIdx) {
         this(readMethodName, writeMethodName, mappedType, mappedType, colIdx);
     }
 
@@ -105,7 +105,7 @@ public class TupleColumnAccessCodeGenerator {
      * @param writeArgType Write method argument type.
      * @param colIdx Column index in schema.
      */
-    TupleColumnAccessCodeGenerator(String readMethodName, String writeMethodName, Class<?> mappedType,
+    ColumnAccessCodeGenerator(String readMethodName, String writeMethodName, Class<?> mappedType,
         Class<?> writeArgType, int colIdx) {
         this.readMethodName = readMethodName;
         this.writeMethodName = writeMethodName;
diff --git a/modules/schema/src/main/java/org/apache/ignite/internal/schema/marshaller/asm/IdentityMarshallerCodeGenerator.java b/modules/schema/src/main/java/org/apache/ignite/internal/schema/marshaller/asm/IdentityMarshallerCodeGenerator.java
index 69511a1..30864fa 100644
--- a/modules/schema/src/main/java/org/apache/ignite/internal/schema/marshaller/asm/IdentityMarshallerCodeGenerator.java
+++ b/modules/schema/src/main/java/org/apache/ignite/internal/schema/marshaller/asm/IdentityMarshallerCodeGenerator.java
@@ -28,7 +28,7 @@ import org.apache.ignite.internal.schema.marshaller.Serializer;
  */
 class IdentityMarshallerCodeGenerator implements MarshallerCodeGenerator {
     /** Object field access expression generator. */
-    private final TupleColumnAccessCodeGenerator columnAccessor;
+    private final ColumnAccessCodeGenerator columnAccessor;
 
     /** Target class. */
     private final Class<?> tClass;
@@ -37,9 +37,9 @@ class IdentityMarshallerCodeGenerator implements MarshallerCodeGenerator {
      * Constructor.
      *
      * @param tClass Target class.
-     * @param columnAccessor Tuple column code generator.
+     * @param columnAccessor Row column code generator.
      */
-    IdentityMarshallerCodeGenerator(Class<?> tClass, TupleColumnAccessCodeGenerator columnAccessor) {
+    IdentityMarshallerCodeGenerator(Class<?> tClass, ColumnAccessCodeGenerator columnAccessor) {
         this.tClass = tClass;
         this.columnAccessor = columnAccessor;
     }
@@ -65,9 +65,9 @@ class IdentityMarshallerCodeGenerator implements MarshallerCodeGenerator {
     }
 
     /** {@inheritDoc} */
-    @Override public BytecodeNode unmarshallObject(ParameterizedType type, Variable tuple, Variable obj) {
+    @Override public BytecodeNode unmarshallObject(ParameterizedType type, Variable row, Variable obj) {
         return obj.set(
-            tuple.invoke(
+            row.invoke(
                 columnAccessor.readMethodName(),
                 columnAccessor.mappedType(),
                 BytecodeExpressions.constantInt(columnAccessor.columnIdx())
diff --git a/modules/schema/src/main/java/org/apache/ignite/internal/schema/marshaller/asm/MarshallerCodeGenerator.java b/modules/schema/src/main/java/org/apache/ignite/internal/schema/marshaller/asm/MarshallerCodeGenerator.java
index 7fdc05d..26aa29e 100644
--- a/modules/schema/src/main/java/org/apache/ignite/internal/schema/marshaller/asm/MarshallerCodeGenerator.java
+++ b/modules/schema/src/main/java/org/apache/ignite/internal/schema/marshaller/asm/MarshallerCodeGenerator.java
@@ -47,7 +47,7 @@ public interface MarshallerCodeGenerator {
 
     /**
      * @param serializerClass Serializer type
-     * @param asm Tuple assembler.
+     * @param asm Row assembler.
      * @param obj Target object variable.
      * @return Unmarshall object code.
      */
@@ -55,11 +55,11 @@ public interface MarshallerCodeGenerator {
 
     /**
      * @param serializerClass Serializer type
-     * @param tuple Tuple.
+     * @param row Row.
      * @param obj Result object variable.
      * @return Unmarshall object code.
      */
-    BytecodeNode unmarshallObject(ParameterizedType serializerClass, Variable tuple, Variable obj);
+    BytecodeNode unmarshallObject(ParameterizedType serializerClass, Variable row, Variable obj);
 
     /**
      * @param classDef Class definition.
diff --git a/modules/schema/src/main/java/org/apache/ignite/internal/schema/marshaller/asm/ObjectMarshallerCodeGenerator.java b/modules/schema/src/main/java/org/apache/ignite/internal/schema/marshaller/asm/ObjectMarshallerCodeGenerator.java
index 28e3fcd..0d2ae13 100644
--- a/modules/schema/src/main/java/org/apache/ignite/internal/schema/marshaller/asm/ObjectMarshallerCodeGenerator.java
+++ b/modules/schema/src/main/java/org/apache/ignite/internal/schema/marshaller/asm/ObjectMarshallerCodeGenerator.java
@@ -48,7 +48,7 @@ class ObjectMarshallerCodeGenerator implements MarshallerCodeGenerator {
     private final Columns columns;
 
     /** Object field access expression generators. */
-    private final TupleColumnAccessCodeGenerator[] columnAccessors;
+    private final ColumnAccessCodeGenerator[] columnAccessors;
 
     ObjectMarshallerCodeGenerator(
         Columns columns,
@@ -57,13 +57,13 @@ class ObjectMarshallerCodeGenerator implements MarshallerCodeGenerator {
     ) {
         this.columns = columns;
         this.tClass = tClass;
-        columnAccessors = new TupleColumnAccessCodeGenerator[columns.length()];
+        columnAccessors = new ColumnAccessCodeGenerator[columns.length()];
 
         try {
             for (int i = 0; i < columns.length(); i++) {
                 final Field field = tClass.getDeclaredField(columns.column(i).name());
 
-                columnAccessors[i] = TupleColumnAccessCodeGenerator.createAccessor(MarshallerUtil.mode(field.getType()), i + firstColIdx);
+                columnAccessors[i] = ColumnAccessCodeGenerator.createAccessor(MarshallerUtil.mode(field.getType()), i + firstColIdx);
             }
         }
         catch (NoSuchFieldException ex) {
@@ -84,7 +84,7 @@ class ObjectMarshallerCodeGenerator implements MarshallerCodeGenerator {
     /** {@inheritDoc} */
     @Override public BytecodeNode getValue(ParameterizedType serializerClass, Variable obj,
         int i) {
-        final TupleColumnAccessCodeGenerator columnAccessor = columnAccessors[i];
+        final ColumnAccessCodeGenerator columnAccessor = columnAccessors[i];
 
         return BytecodeExpressions.getStatic(serializerClass, "FIELD_HANDLER_" + columnAccessor.columnIdx(), ParameterizedType.type(VarHandle.class))
             .invoke("get", columnAccessor.mappedType(), obj);
@@ -95,7 +95,7 @@ class ObjectMarshallerCodeGenerator implements MarshallerCodeGenerator {
         final BytecodeBlock block = new BytecodeBlock();
 
         for (int i = 0; i < columns.length(); i++) {
-            final TupleColumnAccessCodeGenerator columnAccessor = columnAccessors[i];
+            final ColumnAccessCodeGenerator columnAccessor = columnAccessors[i];
 
             final BytecodeExpression fld = BytecodeExpressions.getStatic(serializerClass, "FIELD_HANDLER_" + columnAccessor.columnIdx(), ParameterizedType.type(VarHandle.class))
                 .invoke("get", columnAccessor.mappedType(), obj);
@@ -120,13 +120,13 @@ class ObjectMarshallerCodeGenerator implements MarshallerCodeGenerator {
     }
 
     /** {@inheritDoc} */
-    @Override public BytecodeBlock unmarshallObject(ParameterizedType serializerClass, Variable tuple, Variable obj) {
+    @Override public BytecodeBlock unmarshallObject(ParameterizedType serializerClass, Variable row, Variable obj) {
         final BytecodeBlock block = new BytecodeBlock();
 
         for (int i = 0; i < columns.length(); i++) {
-            final TupleColumnAccessCodeGenerator columnAccessor = columnAccessors[i];
+            final ColumnAccessCodeGenerator columnAccessor = columnAccessors[i];
 
-            final BytecodeExpression val = tuple.invoke(
+            final BytecodeExpression val = row.invoke(
                 columnAccessor.readMethodName(),
                 columnAccessor.mappedType(),
                 BytecodeExpressions.constantInt(columnAccessor.columnIdx())
diff --git a/modules/schema/src/main/java/org/apache/ignite/internal/schema/marshaller/reflection/FieldAccessor.java b/modules/schema/src/main/java/org/apache/ignite/internal/schema/marshaller/reflection/FieldAccessor.java
index d681738..703c62c 100644
--- a/modules/schema/src/main/java/org/apache/ignite/internal/schema/marshaller/reflection/FieldAccessor.java
+++ b/modules/schema/src/main/java/org/apache/ignite/internal/schema/marshaller/reflection/FieldAccessor.java
@@ -25,8 +25,8 @@ import java.util.Objects;
 import java.util.UUID;
 import org.apache.ignite.internal.schema.Column;
 import org.apache.ignite.internal.schema.Columns;
-import org.apache.ignite.internal.schema.Tuple;
-import org.apache.ignite.internal.schema.TupleAssembler;
+import org.apache.ignite.internal.schema.Row;
+import org.apache.ignite.internal.schema.RowAssembler;
 import org.apache.ignite.internal.schema.marshaller.BinaryMode;
 import org.apache.ignite.internal.schema.marshaller.MarshallerUtil;
 import org.apache.ignite.internal.schema.marshaller.SerializationException;
@@ -151,14 +151,14 @@ abstract class FieldAccessor {
     }
 
     /**
-     * Reads value object from tuple.
+     * Reads value object from row.
      *
      * @param reader Reader.
      * @param colIdx Column index.
      * @param mode Binary read mode.
      * @return Read value object.
      */
-    private static Object readRefValue(Tuple reader, int colIdx, BinaryMode mode) {
+    private static Object readRefValue(Row reader, int colIdx, BinaryMode mode) {
         assert reader != null;
         assert colIdx >= 0;
 
@@ -223,13 +223,13 @@ abstract class FieldAccessor {
     }
 
     /**
-     * Writes reference value to tuple.
+     * Writes reference value to row.
      *
      * @param val Value object.
      * @param writer Writer.
      * @param mode Write binary mode.
      */
-    private static void writeRefObject(Object val, TupleAssembler writer, BinaryMode mode) {
+    private static void writeRefObject(Object val, RowAssembler writer, BinaryMode mode) {
         assert writer != null;
 
         if (val == null) {
@@ -325,13 +325,13 @@ abstract class FieldAccessor {
     }
 
     /**
-     * Write object field value to tuple.
+     * Write object field value to row.
      *
-     * @param writer Tuple writer.
+     * @param writer Row writer.
      * @param obj Source object.
      * @throws SerializationException If failed.
      */
-    public void write(TupleAssembler writer, Object obj) throws SerializationException {
+    public void write(RowAssembler writer, Object obj) throws SerializationException {
         try {
             write0(writer, obj);
         }
@@ -341,21 +341,21 @@ abstract class FieldAccessor {
     }
 
     /**
-     * Write object field value to tuple.
+     * Write object field value to row.
      *
-     * @param writer Tuple writer.
+     * @param writer Row writer.
      * @param obj Source object.
      */
-    protected abstract void write0(TupleAssembler writer, Object obj) throws Exception;
+    protected abstract void write0(RowAssembler writer, Object obj) throws Exception;
 
     /**
-     * Reads value fom tuple to object field.
+     * Reads value fom row to object field.
      *
-     * @param reader Tuple reader.
+     * @param reader Row reader.
      * @param obj Target object.
      * @throws SerializationException If failed.
      */
-    public void read(Tuple reader, Object obj) throws SerializationException {
+    public void read(Row reader, Object obj) throws SerializationException {
         try {
             read0(reader, obj);
         }
@@ -365,21 +365,21 @@ abstract class FieldAccessor {
     }
 
     /**
-     * Reads value fom tuple to object field.
+     * Reads value fom row to object field.
      *
-     * @param reader Tuple reader.
+     * @param reader Row reader.
      * @param obj Target object.
      * @throws Exception If failed.
      */
-    protected abstract void read0(Tuple reader, Object obj) throws Exception;
+    protected abstract void read0(Row reader, Object obj) throws Exception;
 
     /**
      * Read value.
      *
-     * @param reader Tuple reader.
+     * @param reader Row reader.
      * @return Object.
      */
-    public Object read(Tuple reader) {
+    public Object read(Row reader) {
         throw new UnsupportedOperationException();
     }
 
@@ -408,17 +408,17 @@ abstract class FieldAccessor {
         }
 
         /** {@inheritDoc} */
-        @Override protected void write0(TupleAssembler writer, Object obj) {
+        @Override protected void write0(RowAssembler writer, Object obj) {
             writeRefObject(obj, writer, mode);
         }
 
         /** {@inheritDoc} */
-        @Override protected void read0(Tuple reader, Object obj) {
+        @Override protected void read0(Row reader, Object obj) {
             throw new UnsupportedOperationException("Called identity accessor for object field.");
         }
 
         /** {@inheritDoc} */
-        @Override public Object read(Tuple reader) {
+        @Override public Object read(Row reader) {
             return readRefValue(reader, colIdx, mode);
         }
 
@@ -443,14 +443,14 @@ abstract class FieldAccessor {
         }
 
         /** {@inheritDoc} */
-        @Override protected void write0(TupleAssembler writer, Object obj) {
+        @Override protected void write0(RowAssembler writer, Object obj) {
             final byte val = (byte)varHandle.get(obj);
 
             writer.appendByte(val);
         }
 
         /** {@inheritDoc} */
-        @Override protected void read0(Tuple reader, Object obj) {
+        @Override protected void read0(Row reader, Object obj) {
             final byte val = reader.byteValue(colIdx);
 
             varHandle.set(obj, val);
@@ -472,14 +472,14 @@ abstract class FieldAccessor {
         }
 
         /** {@inheritDoc} */
-        @Override protected void write0(TupleAssembler writer, Object obj) {
+        @Override protected void write0(RowAssembler writer, Object obj) {
             final short val = (short)varHandle.get(obj);
 
             writer.appendShort(val);
         }
 
         /** {@inheritDoc} */
-        @Override protected void read0(Tuple reader, Object obj) {
+        @Override protected void read0(Row reader, Object obj) {
             final short val = reader.shortValue(colIdx);
 
             varHandle.set(obj, val);
@@ -501,14 +501,14 @@ abstract class FieldAccessor {
         }
 
         /** {@inheritDoc} */
-        @Override protected void write0(TupleAssembler writer, Object obj) {
+        @Override protected void write0(RowAssembler writer, Object obj) {
             final int val = (int)varHandle.get(obj);
 
             writer.appendInt(val);
         }
 
         /** {@inheritDoc} */
-        @Override protected void read0(Tuple reader, Object obj) {
+        @Override protected void read0(Row reader, Object obj) {
             final int val = reader.intValue(colIdx);
 
             varHandle.set(obj, val);
@@ -530,14 +530,14 @@ abstract class FieldAccessor {
         }
 
         /** {@inheritDoc} */
-        @Override protected void write0(TupleAssembler writer, Object obj) {
+        @Override protected void write0(RowAssembler writer, Object obj) {
             final long val = (long)varHandle.get(obj);
 
             writer.appendLong(val);
         }
 
         /** {@inheritDoc} */
-        @Override protected void read0(Tuple reader, Object obj) {
+        @Override protected void read0(Row reader, Object obj) {
             final long val = reader.longValue(colIdx);
 
             varHandle.set(obj, val);
@@ -559,14 +559,14 @@ abstract class FieldAccessor {
         }
 
         /** {@inheritDoc} */
-        @Override protected void write0(TupleAssembler writer, Object obj) {
+        @Override protected void write0(RowAssembler writer, Object obj) {
             final float val = (float)varHandle.get(obj);
 
             writer.appendFloat(val);
         }
 
         /** {@inheritDoc} */
-        @Override protected void read0(Tuple reader, Object obj) {
+        @Override protected void read0(Row reader, Object obj) {
             final float val = reader.floatValue(colIdx);
 
             varHandle.set(obj, val);
@@ -588,14 +588,14 @@ abstract class FieldAccessor {
         }
 
         /** {@inheritDoc} */
-        @Override protected void write0(TupleAssembler writer, Object obj) {
+        @Override protected void write0(RowAssembler writer, Object obj) {
             final double val = (double)varHandle.get(obj);
 
             writer.appendDouble(val);
         }
 
         /** {@inheritDoc} */
-        @Override protected void read0(Tuple reader, Object obj) {
+        @Override protected void read0(Row reader, Object obj) {
             final double val = reader.doubleValue(colIdx);
 
             varHandle.set(obj, val);
@@ -618,7 +618,7 @@ abstract class FieldAccessor {
         }
 
         /** {@inheritDoc} */
-        @Override protected void write0(TupleAssembler writer, Object obj) {
+        @Override protected void write0(RowAssembler writer, Object obj) {
             assert obj != null;
             assert writer != null;
 
@@ -634,7 +634,7 @@ abstract class FieldAccessor {
         }
 
         /** {@inheritDoc} */
-        @Override public void read0(Tuple reader, Object obj) {
+        @Override public void read0(Row reader, Object obj) {
             Object val = readRefValue(reader, colIdx, mode);
 
             varHandle.set(obj, val);
diff --git a/modules/schema/src/main/java/org/apache/ignite/internal/schema/marshaller/reflection/JavaSerializer.java b/modules/schema/src/main/java/org/apache/ignite/internal/schema/marshaller/reflection/JavaSerializer.java
index 1c0efa1..c7a802c 100644
--- a/modules/schema/src/main/java/org/apache/ignite/internal/schema/marshaller/reflection/JavaSerializer.java
+++ b/modules/schema/src/main/java/org/apache/ignite/internal/schema/marshaller/reflection/JavaSerializer.java
@@ -18,9 +18,9 @@
 package org.apache.ignite.internal.schema.marshaller.reflection;
 
 import org.apache.ignite.internal.schema.Columns;
+import org.apache.ignite.internal.schema.Row;
+import org.apache.ignite.internal.schema.RowAssembler;
 import org.apache.ignite.internal.schema.SchemaDescriptor;
-import org.apache.ignite.internal.schema.Tuple;
-import org.apache.ignite.internal.schema.TupleAssembler;
 import org.apache.ignite.internal.schema.marshaller.AbstractSerializer;
 import org.apache.ignite.internal.schema.marshaller.SerializationException;
 import org.jetbrains.annotations.Nullable;
@@ -61,7 +61,7 @@ public class JavaSerializer extends AbstractSerializer {
 
     /** {@inheritDoc} */
     @Override protected byte[] serialize0(
-        TupleAssembler asm,
+        RowAssembler asm,
         Object key,
         @Nullable Object val
     ) throws SerializationException {
@@ -75,21 +75,21 @@ public class JavaSerializer extends AbstractSerializer {
     }
 
     /**
-     * Creates TupleAssebler for key-value pair.
+     * Creates {@link RowAssembler} for key-value pair.
      *
      * @param key Key object.
      * @param val Value object.
-     * @return Tuple assembler.
+     * @return Row assembler.
      */
-    @Override protected TupleAssembler createAssembler(Object key, Object val) {
+    @Override protected RowAssembler createAssembler(Object key, Object val) {
         ObjectStatistic keyStat = collectObjectStats(schema.keyColumns(), keyMarsh, key);
         ObjectStatistic valStat = collectObjectStats(schema.valueColumns(), valMarsh, val);
 
-        int size = TupleAssembler.tupleSize(
+        int size = RowAssembler.rowChunkSize(
             schema.keyColumns(), keyStat.nonNullFields, keyStat.nonNullFieldsSize,
             schema.valueColumns(), valStat.nonNullFields, valStat.nonNullFieldsSize);
 
-        return new TupleAssembler(schema, size, keyStat.nonNullFields, valStat.nonNullFields);
+        return new RowAssembler(schema, size, keyStat.nonNullFields, valStat.nonNullFields);
     }
 
     /**
@@ -121,8 +121,8 @@ public class JavaSerializer extends AbstractSerializer {
     }
 
     /** {@inheritDoc} */
-    @Override protected Object deserializeKey0(Tuple tuple) throws SerializationException {
-        final Object o = keyMarsh.readObject(tuple);
+    @Override protected Object deserializeKey0(Row row) throws SerializationException {
+        final Object o = keyMarsh.readObject(row);
 
         assert keyClass.isInstance(o);
 
@@ -130,8 +130,8 @@ public class JavaSerializer extends AbstractSerializer {
     }
 
     /** {@inheritDoc} */
-    @Override protected Object deserializeValue0(Tuple tuple) throws SerializationException {
-        final Object o = valMarsh.readObject(tuple);
+    @Override protected Object deserializeValue0(Row row) throws SerializationException {
+        final Object o = valMarsh.readObject(row);
 
         assert o == null || valClass.isInstance(o);
 
diff --git a/modules/schema/src/main/java/org/apache/ignite/internal/schema/marshaller/reflection/Marshaller.java b/modules/schema/src/main/java/org/apache/ignite/internal/schema/marshaller/reflection/Marshaller.java
index d420ade..874ace6 100644
--- a/modules/schema/src/main/java/org/apache/ignite/internal/schema/marshaller/reflection/Marshaller.java
+++ b/modules/schema/src/main/java/org/apache/ignite/internal/schema/marshaller/reflection/Marshaller.java
@@ -20,8 +20,8 @@ package org.apache.ignite.internal.schema.marshaller.reflection;
 import java.util.Objects;
 import org.apache.ignite.internal.schema.Column;
 import org.apache.ignite.internal.schema.Columns;
-import org.apache.ignite.internal.schema.Tuple;
-import org.apache.ignite.internal.schema.TupleAssembler;
+import org.apache.ignite.internal.schema.Row;
+import org.apache.ignite.internal.schema.RowAssembler;
 import org.apache.ignite.internal.schema.marshaller.BinaryMode;
 import org.apache.ignite.internal.schema.marshaller.MarshallerUtil;
 import org.apache.ignite.internal.schema.marshaller.SerializationException;
@@ -114,13 +114,13 @@ class Marshaller {
     }
 
     /**
-     * Reads object from tuple.
+     * Reads object from row.
      *
-     * @param reader Tuple reader.
+     * @param reader Row reader.
      * @return Object.
      * @throws SerializationException If failed.
      */
-    public Object readObject(Tuple reader) throws SerializationException {
+    public Object readObject(Row reader) throws SerializationException {
         if (isSimpleTypeMarshaller())
             return fieldAccessors[0].read(reader);
 
@@ -133,13 +133,13 @@ class Marshaller {
     }
 
     /**
-     * Write object to tuple.
+     * Write an object to row.
      *
      * @param obj Object.
-     * @param writer Tuple writer.
+     * @param writer Row writer.
      * @throws SerializationException If failed.
      */
-    public void writeObject(Object obj, TupleAssembler writer) throws SerializationException {
+    public void writeObject(Object obj, RowAssembler writer) throws SerializationException {
         for (int fldIdx = 0; fldIdx < fieldAccessors.length; fldIdx++)
             fieldAccessors[fldIdx].write(writer, obj);
     }
diff --git a/modules/schema/src/main/java/org/apache/ignite/internal/schema/package-info.java b/modules/schema/src/main/java/org/apache/ignite/internal/schema/package-info.java
index fe5f0e3..955effb 100644
--- a/modules/schema/src/main/java/org/apache/ignite/internal/schema/package-info.java
+++ b/modules/schema/src/main/java/org/apache/ignite/internal/schema/package-info.java
@@ -17,6 +17,6 @@
 
 /**
  * <!-- Package description. -->
- * Contains schema description, tuple assembly and field accessor classes.
+ * Contains schema description, row assembly and field accessor classes.
  */
 package org.apache.ignite.internal.schema;
\ No newline at end of file
diff --git a/modules/schema/src/test/java/org/apache/ignite/internal/schema/TupleTest.java b/modules/schema/src/test/java/org/apache/ignite/internal/schema/RowTest.java
similarity index 95%
rename from modules/schema/src/test/java/org/apache/ignite/internal/schema/TupleTest.java
rename to modules/schema/src/test/java/org/apache/ignite/internal/schema/RowTest.java
index 054d905..6168aff 100644
--- a/modules/schema/src/test/java/org/apache/ignite/internal/schema/TupleTest.java
+++ b/modules/schema/src/test/java/org/apache/ignite/internal/schema/RowTest.java
@@ -37,9 +37,9 @@ import static org.junit.jupiter.api.Assertions.assertArrayEquals;
 import static org.junit.jupiter.api.Assertions.assertEquals;
 
 /**
- * Tests tuple assembling and reading.
+ * Tests row assembling and reading.
  */
-public class TupleTest {
+public class RowTest {
     /** */
     private Random rnd;
 
@@ -223,11 +223,11 @@ public class TupleTest {
                 else if (type == NativeTypeSpec.STRING) {
                     if (schema.keyColumn(i)) {
                         nonNullVarLenKeyCols++;
-                        nonNullVarLenKeySize += TupleAssembler.utf8EncodedLength((CharSequence)vals[i]);
+                        nonNullVarLenKeySize += RowAssembler.utf8EncodedLength((CharSequence)vals[i]);
                     }
                     else {
                         nonNullVarLenValCols++;
-                        nonNullVarLenValSize += TupleAssembler.utf8EncodedLength((CharSequence)vals[i]);
+                        nonNullVarLenValSize += RowAssembler.utf8EncodedLength((CharSequence)vals[i]);
                     }
                 }
                 else
@@ -235,11 +235,11 @@ public class TupleTest {
             }
         }
 
-        int size = TupleAssembler.tupleSize(
+        int size = RowAssembler.rowChunkSize(
             schema.keyColumns(), nonNullVarLenKeyCols, nonNullVarLenKeySize,
             schema.valueColumns(), nonNullVarLenValCols, nonNullVarLenValSize);
 
-        TupleAssembler asm = new TupleAssembler(schema, size, nonNullVarLenKeyCols, nonNullVarLenValCols);
+        RowAssembler asm = new RowAssembler(schema, size, nonNullVarLenKeyCols, nonNullVarLenValCols);
 
         for (int i = 0; i < vals.length; i++) {
             if (vals[i] == null)
@@ -296,7 +296,7 @@ public class TupleTest {
 
         byte[] data = asm.build();
 
-        ByteBufferTuple tup = new ByteBufferTuple(schema, data);
+        ByteBufferRow tup = new ByteBufferRow(schema, data);
 
         for (int i = 0; i < vals.length; i++) {
             Column col = schema.column(i);
diff --git a/modules/schema/src/test/java/org/apache/ignite/internal/schema/marshaller/reflection/FieldAccessorTest.java b/modules/schema/src/test/java/org/apache/ignite/internal/schema/marshaller/reflection/FieldAccessorTest.java
index e3933f1..db9a9fd 100644
--- a/modules/schema/src/test/java/org/apache/ignite/internal/schema/marshaller/reflection/FieldAccessorTest.java
+++ b/modules/schema/src/test/java/org/apache/ignite/internal/schema/marshaller/reflection/FieldAccessorTest.java
@@ -25,9 +25,9 @@ import java.util.Random;
 import java.util.UUID;
 import org.apache.ignite.internal.schema.Bitmask;
 import org.apache.ignite.internal.schema.Column;
+import org.apache.ignite.internal.schema.Row;
+import org.apache.ignite.internal.schema.RowAssembler;
 import org.apache.ignite.internal.schema.TestUtils;
-import org.apache.ignite.internal.schema.Tuple;
-import org.apache.ignite.internal.schema.TupleAssembler;
 import org.apache.ignite.internal.schema.marshaller.BinaryMode;
 import org.apache.ignite.internal.schema.marshaller.SerializationException;
 import org.apache.ignite.internal.util.Pair;
@@ -95,17 +95,17 @@ public class FieldAccessorTest {
             new Column("bytesCol", BYTES, false),
         };
 
-        final Pair<TupleAssembler, Tuple> mocks = createMocks();
+        final Pair<RowAssembler, Row> mocks = createMocks();
 
-        final TupleAssembler tupleAssembler = mocks.getFirst();
-        final Tuple tuple = mocks.getSecond();
+        final RowAssembler rowAssembler = mocks.getFirst();
+        final Row row = mocks.getSecond();
 
         final TestObject obj = TestObject.randomObject(rnd);
 
         for (int i = 0; i < cols.length; i++) {
             FieldAccessor accessor = FieldAccessor.create(TestObject.class, cols[i], i);
 
-            accessor.write(tupleAssembler, obj);
+            accessor.write(rowAssembler, obj);
         }
 
         final TestObject restoredObj = new TestObject();
@@ -113,7 +113,7 @@ public class FieldAccessorTest {
         for (int i = 0; i < cols.length; i++) {
             FieldAccessor accessor = FieldAccessor.create(TestObject.class, cols[i], i);
 
-            accessor.read(tuple, restoredObj);
+            accessor.read(row, restoredObj);
         }
 
         assertEquals(obj.pByteCol, restoredObj.pByteCol);
@@ -149,10 +149,10 @@ public class FieldAccessorTest {
             new Column("bytesCol", BYTES, true),
         };
 
-        final Pair<TupleAssembler, Tuple> mocks = createMocks();
+        final Pair<RowAssembler, Row> mocks = createMocks();
 
-        final TupleAssembler tupleAssembler = mocks.getFirst();
-        final Tuple tuple = mocks.getSecond();
+        final RowAssembler rowAssembler = mocks.getFirst();
+        final Row row = mocks.getSecond();
 
         final TestSimpleObject obj = new TestSimpleObject();
         obj.longCol = rnd.nextLong();
@@ -161,7 +161,7 @@ public class FieldAccessorTest {
         for (int i = 0; i < cols.length; i++) {
             FieldAccessor accessor = FieldAccessor.create(TestSimpleObject.class, cols[i], i);
 
-            accessor.write(tupleAssembler, obj);
+            accessor.write(rowAssembler, obj);
         }
 
         final TestSimpleObject restoredObj = new TestSimpleObject();
@@ -169,7 +169,7 @@ public class FieldAccessorTest {
         for (int i = 0; i < cols.length; i++) {
             FieldAccessor accessor = FieldAccessor.create(TestSimpleObject.class, cols[i], i);
 
-            accessor.read(tuple, restoredObj);
+            accessor.read(row, restoredObj);
         }
 
         assertEquals(obj.intCol, restoredObj.intCol);
@@ -191,7 +191,7 @@ public class FieldAccessorTest {
 
         assertEquals("Some string", accessor.value("Some string"));
 
-        final Pair<TupleAssembler, Tuple> mocks = createMocks();
+        final Pair<RowAssembler, Row> mocks = createMocks();
 
         accessor.write(mocks.getFirst(), "Other string");
         assertEquals("Other string", accessor.read(mocks.getSecond()));
@@ -209,7 +209,7 @@ public class FieldAccessorTest {
 
         assertEquals("Some string", accessor.value("Some string"));
 
-        final Pair<TupleAssembler, Tuple> mocks = createMocks();
+        final Pair<RowAssembler, Row> mocks = createMocks();
 
         assertThrows(
             SerializationException.class,
@@ -219,15 +219,15 @@ public class FieldAccessorTest {
     }
 
     /**
-     * Creates mock pair for {@link Tuple} and {@link TupleAssembler).
+     * Creates mock pair for {@link Row} and {@link RowAssembler ).
      *
      * @return Pair of mocks.
      */
-    private Pair<TupleAssembler, Tuple> createMocks() {
+    private Pair<RowAssembler, Row> createMocks() {
         final ArrayList<Object> vals = new ArrayList<>();
 
-        final TupleAssembler mockedAsm = Mockito.mock(TupleAssembler.class);
-        final Tuple mockedTuple = Mockito.mock(Tuple.class);
+        final RowAssembler mockedAsm = Mockito.mock(RowAssembler.class);
+        final Row mockedRow = Mockito.mock(Row.class);
 
         final Answer<Void> asmAnswer = new Answer<>() {
             @Override public Void answer(InvocationOnMock invocation) {
@@ -240,7 +240,7 @@ public class FieldAccessorTest {
             }
         };
 
-        final Answer<Object> tupleAnswer = new Answer<>() {
+        final Answer<Object> rowAnswer = new Answer<>() {
             @Override public Object answer(InvocationOnMock invocation) {
                 final int idx = invocation.getArgument(0, Integer.class);
 
@@ -261,25 +261,25 @@ public class FieldAccessorTest {
         Mockito.doAnswer(asmAnswer).when(mockedAsm).appendString(Mockito.anyString());
         Mockito.doAnswer(asmAnswer).when(mockedAsm).appendBytes(Mockito.any(byte[].class));
 
-        Mockito.doAnswer(tupleAnswer).when(mockedTuple).byteValue(Mockito.anyInt());
-        Mockito.doAnswer(tupleAnswer).when(mockedTuple).byteValueBoxed(Mockito.anyInt());
-        Mockito.doAnswer(tupleAnswer).when(mockedTuple).shortValue(Mockito.anyInt());
-        Mockito.doAnswer(tupleAnswer).when(mockedTuple).shortValueBoxed(Mockito.anyInt());
-        Mockito.doAnswer(tupleAnswer).when(mockedTuple).intValue(Mockito.anyInt());
-        Mockito.doAnswer(tupleAnswer).when(mockedTuple).intValueBoxed(Mockito.anyInt());
-        Mockito.doAnswer(tupleAnswer).when(mockedTuple).longValue(Mockito.anyInt());
-        Mockito.doAnswer(tupleAnswer).when(mockedTuple).longValueBoxed(Mockito.anyInt());
-        Mockito.doAnswer(tupleAnswer).when(mockedTuple).floatValue(Mockito.anyInt());
-        Mockito.doAnswer(tupleAnswer).when(mockedTuple).floatValueBoxed(Mockito.anyInt());
-        Mockito.doAnswer(tupleAnswer).when(mockedTuple).doubleValue(Mockito.anyInt());
-        Mockito.doAnswer(tupleAnswer).when(mockedTuple).doubleValueBoxed(Mockito.anyInt());
-
-        Mockito.doAnswer(tupleAnswer).when(mockedTuple).uuidValue(Mockito.anyInt());
-        Mockito.doAnswer(tupleAnswer).when(mockedTuple).bitmaskValue(Mockito.anyInt());
-        Mockito.doAnswer(tupleAnswer).when(mockedTuple).stringValue(Mockito.anyInt());
-        Mockito.doAnswer(tupleAnswer).when(mockedTuple).bytesValue(Mockito.anyInt());
-
-        return new Pair<>(mockedAsm, mockedTuple);
+        Mockito.doAnswer(rowAnswer).when(mockedRow).byteValue(Mockito.anyInt());
+        Mockito.doAnswer(rowAnswer).when(mockedRow).byteValueBoxed(Mockito.anyInt());
+        Mockito.doAnswer(rowAnswer).when(mockedRow).shortValue(Mockito.anyInt());
+        Mockito.doAnswer(rowAnswer).when(mockedRow).shortValueBoxed(Mockito.anyInt());
+        Mockito.doAnswer(rowAnswer).when(mockedRow).intValue(Mockito.anyInt());
+        Mockito.doAnswer(rowAnswer).when(mockedRow).intValueBoxed(Mockito.anyInt());
+        Mockito.doAnswer(rowAnswer).when(mockedRow).longValue(Mockito.anyInt());
+        Mockito.doAnswer(rowAnswer).when(mockedRow).longValueBoxed(Mockito.anyInt());
+        Mockito.doAnswer(rowAnswer).when(mockedRow).floatValue(Mockito.anyInt());
+        Mockito.doAnswer(rowAnswer).when(mockedRow).floatValueBoxed(Mockito.anyInt());
+        Mockito.doAnswer(rowAnswer).when(mockedRow).doubleValue(Mockito.anyInt());
+        Mockito.doAnswer(rowAnswer).when(mockedRow).doubleValueBoxed(Mockito.anyInt());
+
+        Mockito.doAnswer(rowAnswer).when(mockedRow).uuidValue(Mockito.anyInt());
+        Mockito.doAnswer(rowAnswer).when(mockedRow).bitmaskValue(Mockito.anyInt());
+        Mockito.doAnswer(rowAnswer).when(mockedRow).stringValue(Mockito.anyInt());
+        Mockito.doAnswer(rowAnswer).when(mockedRow).bytesValue(Mockito.anyInt());
+
+        return new Pair<>(mockedAsm, mockedRow);
     }
 
     /**
diff --git a/modules/table/README.md b/modules/table/README.md
new file mode 100644
index 0000000..1d9fdfb
--- /dev/null
+++ b/modules/table/README.md
@@ -0,0 +1,2 @@
+# Ignite table module
+This module provides Table API implementation.
diff --git a/pom.xml b/modules/table/pom.xml
similarity index 57%
copy from pom.xml
copy to modules/table/pom.xml
index 75141f2..5e5e6dc 100644
--- a/pom.xml
+++ b/modules/table/pom.xml
@@ -17,6 +17,9 @@
   limitations under the License.
 -->
 
+<!--
+    POM file.
+-->
 <project xmlns="http://maven.apache.org/POM/4.0.0"
          xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
          xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
@@ -26,25 +29,36 @@
         <groupId>org.apache.ignite</groupId>
         <artifactId>ignite-parent</artifactId>
         <version>1</version>
-        <relativePath>parent</relativePath>
+        <relativePath>../../parent/pom.xml</relativePath>
     </parent>
 
-    <artifactId>apache-ignite</artifactId>
+    <artifactId>ignite-table</artifactId>
     <version>3.0.0-SNAPSHOT</version>
-    <packaging>pom</packaging>
-
-    <modules>
-        <module>modules/api</module>
-        <module>modules/bytecode</module>
-        <module>modules/cli</module>
-        <module>modules/cli-common</module>
-        <module>modules/configuration</module>
-        <module>modules/configuration-annotation-processor</module>
-        <module>modules/core</module>
-        <module>modules/network</module>
-        <module>modules/raft-client</module>
-        <module>modules/rest</module>
-        <module>modules/runner</module>
-        <module>modules/schema</module>
-    </modules>
+
+    <dependencies>
+        <dependency>
+            <groupId>org.apache.ignite</groupId>
+            <artifactId>ignite-api</artifactId>
+            <version>${project.version}</version>
+        </dependency>
+
+        <dependency>
+            <groupId>org.apache.ignite</groupId>
+            <artifactId>ignite-schema</artifactId>
+            <version>${project.version}</version>
+        </dependency>
+
+        <!-- Test dependencies -->
+        <dependency>
+            <groupId>org.junit.jupiter</groupId>
+            <artifactId>junit-jupiter-engine</artifactId>
+            <scope>test</scope>
+        </dependency>
+
+        <dependency>
+            <groupId>org.junit.jupiter</groupId>
+            <artifactId>junit-jupiter-params</artifactId>
+            <scope>test</scope>
+        </dependency>
+    </dependencies>
 </project>
diff --git a/modules/schema/src/main/java/org/apache/ignite/internal/schema/InvalidTypeException.java b/modules/table/src/main/java/org/apache/ignite/binary/BinaryObjects.java
similarity index 56%
copy from modules/schema/src/main/java/org/apache/ignite/internal/schema/InvalidTypeException.java
copy to modules/table/src/main/java/org/apache/ignite/binary/BinaryObjects.java
index d03ca73..2004139 100644
--- a/modules/schema/src/main/java/org/apache/ignite/internal/schema/InvalidTypeException.java
+++ b/modules/table/src/main/java/org/apache/ignite/binary/BinaryObjects.java
@@ -15,16 +15,36 @@
  * limitations under the License.
  */
 
-package org.apache.ignite.internal.schema;
+package org.apache.ignite.binary;
 
 /**
- * An exception thrown when an attempt to read an invalid type from a tuple is performed.
+ * Binary objects utility class.
  */
-public class InvalidTypeException extends IllegalArgumentException {
+public final class BinaryObjects {
     /**
-     * @param msg Error message.
+     * Wraps byte array to BinaryObject.
+     * @param data Object data.
+     * @return Binary object.
      */
-    public InvalidTypeException(String msg) {
-        super(msg);
+    public static BinaryObject wrap(byte[] data) {
+        return null;
+    }
+
+    /**
+     * Deserializes binary object.
+     *
+     * @param obj Object to deserialize.
+     * @param targetCls Target class.
+     * @return Deserialized object.
+     */
+    public static <T> T deserialize(BinaryObject obj, Class<T> targetCls) {
+        return null;
+    }
+
+    /**
+     * Constructor.
+     */
+    private BinaryObjects() {
+        // No-op.
     }
 }
diff --git a/modules/table/src/main/java/org/apache/ignite/internal/schema/marshaller/Marshaller.java b/modules/table/src/main/java/org/apache/ignite/internal/schema/marshaller/Marshaller.java
new file mode 100644
index 0000000..0af432c
--- /dev/null
+++ b/modules/table/src/main/java/org/apache/ignite/internal/schema/marshaller/Marshaller.java
@@ -0,0 +1,74 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.schema.marshaller;
+
+import org.apache.ignite.internal.table.TableRow;
+import org.apache.ignite.table.Tuple;
+import org.jetbrains.annotations.NotNull;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Marshaller interface.
+ */
+public interface Marshaller {
+    /**
+     * @param obj Object to serialize.
+     * @return Table row with columns set from given object.
+     */
+    <T> TableRow serialize(@NotNull T obj);
+
+    /**
+     * @param tuple Record tuple.
+     * @return Table row with columns set from given tuples.
+     */
+    TableRow marshalRecord(@NotNull Tuple tuple);
+
+    /**
+     * @param keyTuple Key tuple.
+     * @param valTuple Value tuple.
+     * @return Table row with columns set from given tuples.
+     */
+    TableRow marshalKVPair(@NotNull Tuple keyTuple, @Nullable Tuple valTuple);
+
+    /**
+     * @param row Table row.
+     * @return Deserialized key object.
+     */
+    <K> @NotNull K deserializeKey(@NotNull TableRow row);
+
+    /**
+     * @param row Table row.
+     * @return Deserialized value object.
+     */
+    <V> @Nullable V deserializeValue(@NotNull TableRow row);
+
+    /**
+     * @param row Table row.
+     * @return Deserialized record object.
+     */
+    <R> R deserializeToRecord(@NotNull TableRow row);
+
+    /**
+     * Deserializes row and fills given record object fields.
+     *
+     * @param row Table row.
+     * @param rec Record object to fill.
+     * @return Given record with filled fields from the given row.
+     */
+    <R> R deserializeToRecord(@NotNull TableRow row, @NotNull R rec);
+}
diff --git a/modules/schema/src/main/java/org/apache/ignite/internal/schema/InvalidTypeException.java b/modules/table/src/main/java/org/apache/ignite/internal/storage/TableStorage.java
similarity index 63%
copy from modules/schema/src/main/java/org/apache/ignite/internal/schema/InvalidTypeException.java
copy to modules/table/src/main/java/org/apache/ignite/internal/storage/TableStorage.java
index d03ca73..1d998b5 100644
--- a/modules/schema/src/main/java/org/apache/ignite/internal/schema/InvalidTypeException.java
+++ b/modules/table/src/main/java/org/apache/ignite/internal/storage/TableStorage.java
@@ -15,16 +15,27 @@
  * limitations under the License.
  */
 
-package org.apache.ignite.internal.schema;
+package org.apache.ignite.internal.storage;
+
+import org.apache.ignite.internal.table.TableRow;
 
 /**
- * An exception thrown when an attempt to read an invalid type from a tuple is performed.
+ * Table storage facade.
  */
-public class InvalidTypeException extends IllegalArgumentException {
+public interface TableStorage {
+    /**
+     * Gets row from storage.
+     *
+     * @param keyRow Row with key columns set.
+     * @return Row with all columns set.
+     */
+    public TableRow get(TableRow keyRow);
+
     /**
-     * @param msg Error message.
+     * Puts row from storage.
+     *
+     * @param row Row.
+     * @return Replaced row or {@code null}.
      */
-    public InvalidTypeException(String msg) {
-        super(msg);
-    }
+    TableRow put(TableRow row);
 }
diff --git a/modules/table/src/main/java/org/apache/ignite/internal/table/KVViewImpl.java b/modules/table/src/main/java/org/apache/ignite/internal/table/KVViewImpl.java
new file mode 100644
index 0000000..c057911
--- /dev/null
+++ b/modules/table/src/main/java/org/apache/ignite/internal/table/KVViewImpl.java
@@ -0,0 +1,228 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.table;
+
+import java.io.Serializable;
+import java.util.Collection;
+import java.util.Map;
+import org.apache.ignite.internal.schema.marshaller.Marshaller;
+import org.apache.ignite.internal.storage.TableStorage;
+import org.apache.ignite.lang.IgniteFuture;
+import org.apache.ignite.table.InvokeProcessor;
+import org.apache.ignite.table.KeyValueView;
+import org.apache.ignite.table.mapper.KeyMapper;
+import org.apache.ignite.table.mapper.ValueMapper;
+import org.jetbrains.annotations.NotNull;
+
+/**
+ * Key-value view implementation.
+ */
+public class KVViewImpl<K, V> implements KeyValueView<K, V> {
+    /** Underlying storage. */
+    private final TableStorage tbl;
+
+    /**
+     * Constructor.
+     *
+     * @param tbl Table storage.
+     * @param keyMapper Key class mapper.
+     * @param valueMapper Value class mapper.
+     */
+    public KVViewImpl(TableStorage tbl, KeyMapper<K> keyMapper, ValueMapper<V> valueMapper) {
+        this.tbl = tbl;
+    }
+
+    /** {@inheritDoc} */
+    @Override public V get(K key) {
+        final Marshaller marsh = marshaller();
+
+        TableRow kRow = marsh.serialize(key);
+
+        TableRow row = tbl.get(kRow);
+
+        return marsh.deserializeValue(row);
+    }
+
+    /** {@inheritDoc} */
+    @Override public @NotNull IgniteFuture<V> getAsync(K key) {
+        return null;
+    }
+
+    /** {@inheritDoc} */
+    @Override public Map<K, V> getAll(Collection<K> keys) {
+        return null;
+    }
+
+    /** {@inheritDoc} */
+    @Override public @NotNull IgniteFuture<Map<K, V>> getAllAsync(Collection<K> keys) {
+        return null;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean contains(K key) {
+        return false;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void put(K key, V val) {
+
+    }
+
+    /** {@inheritDoc} */
+    @Override public @NotNull IgniteFuture<Void> putAsync(K key, V val) {
+        return null;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void putAll(Map<K, V> pairs) {
+
+    }
+
+    /** {@inheritDoc} */
+    @Override public @NotNull IgniteFuture<Void> putAllAsync(Map<K, V> pairs) {
+        return null;
+    }
+
+    /** {@inheritDoc} */
+    @Override public V getAndPut(K key, V val) {
+        return null;
+    }
+
+    /** {@inheritDoc} */
+    @Override public @NotNull IgniteFuture<V> getAndPutAsync(K key, V val) {
+        return null;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean putIfAbsent(K key, V val) {
+        return false;
+    }
+
+    /** {@inheritDoc} */
+    @Override public @NotNull IgniteFuture<Boolean> putIfAbsentAsync(K key, V val) {
+        return null;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean remove(K key) {
+        return false;
+    }
+
+    /** {@inheritDoc} */
+    @Override public @NotNull IgniteFuture<Boolean> removeAsync(K key) {
+        return null;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean remove(K key, V val) {
+        return false;
+    }
+
+    /** {@inheritDoc} */
+    @Override public @NotNull IgniteFuture<Boolean> removeAsync(K key, V val) {
+        return null;
+    }
+
+    /** {@inheritDoc} */
+    @Override public Collection<K> removeAll(Collection<K> keys) {
+        return null;
+    }
+
+    /** {@inheritDoc} */
+    @Override public @NotNull IgniteFuture<K> removeAllAsync(Collection<K> keys) {
+        return null;
+    }
+
+    /** {@inheritDoc} */
+    @Override public V getAndRemove(K key) {
+        return null;
+    }
+
+    /** {@inheritDoc} */
+    @Override public @NotNull IgniteFuture<V> getAndRemoveAsync(K key) {
+        return null;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean replace(K key, V val) {
+        return false;
+    }
+
+    /** {@inheritDoc} */
+    @Override public @NotNull IgniteFuture<Boolean> replaceAsync(K key, V val) {
+        return null;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean replace(K key, V oldVal, V newVal) {
+        return false;
+    }
+
+    /** {@inheritDoc} */
+    @Override public @NotNull IgniteFuture<Boolean> replaceAsync(K key, V oldVal, V newVal) {
+        return null;
+    }
+
+    /** {@inheritDoc} */
+    @Override public V getAndReplace(K key, V val) {
+        return null;
+    }
+
+    /** {@inheritDoc} */
+    @Override public @NotNull IgniteFuture<V> getAndReplaceAsync(K key, V val) {
+        return null;
+    }
+
+    /** {@inheritDoc} */
+    @Override public <R extends Serializable> R invoke(K key, InvokeProcessor<K, V, R> proc, Serializable... args) {
+        return null;
+    }
+
+    /** {@inheritDoc} */
+    @Override public @NotNull <R extends Serializable> IgniteFuture<R> invokeAsync(
+        K key,
+        InvokeProcessor<K, V, R> proc,
+        Serializable... args
+    ) {
+        return null;
+    }
+
+    /** {@inheritDoc} */
+    @Override public <R extends Serializable> Map<K, R> invokeAll(
+        Collection<K> keys,
+        InvokeProcessor<K, V, R> proc,
+        Serializable... args
+    ) {
+        return null;
+    }
+
+    /** {@inheritDoc} */
+    @Override public @NotNull <R extends Serializable> IgniteFuture<Map<K, R>> invokeAllAsync(
+        Collection<K> keys,
+        InvokeProcessor<K, V, R> proc, Serializable... args
+    ) {
+        return null;
+    }
+
+    /**
+     * @return Marshaller.
+     */
+    private Marshaller marshaller() {
+        return null;        // table.schemaManager().marshaller();
+    }
+}
diff --git a/modules/table/src/main/java/org/apache/ignite/internal/table/KeyValueBinaryViewImpl.java b/modules/table/src/main/java/org/apache/ignite/internal/table/KeyValueBinaryViewImpl.java
new file mode 100644
index 0000000..e739985
--- /dev/null
+++ b/modules/table/src/main/java/org/apache/ignite/internal/table/KeyValueBinaryViewImpl.java
@@ -0,0 +1,241 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.table;
+
+import java.io.Serializable;
+import java.util.Collection;
+import java.util.Map;
+import java.util.Objects;
+import org.apache.ignite.internal.schema.marshaller.Marshaller;
+import org.apache.ignite.internal.storage.TableStorage;
+import org.apache.ignite.lang.IgniteFuture;
+import org.apache.ignite.table.InvokeProcessor;
+import org.apache.ignite.table.KeyValueBinaryView;
+import org.apache.ignite.table.Tuple;
+import org.apache.ignite.table.TupleBuilder;
+import org.jetbrains.annotations.NotNull;
+
+/**
+ * Key-value view implementation for binary user-object representation.
+ *
+ * @implNote Key-value {@link Tuple}s represents marshalled user-objects
+ * regarding the binary object concept.
+ */
+public class KeyValueBinaryViewImpl implements KeyValueBinaryView {
+    /** Underlying storage. */
+    private final TableStorage tbl;
+
+    /**
+     * Constructor.
+     *
+     * @param tbl Table storage.
+     */
+    public KeyValueBinaryViewImpl(TableStorage tbl) {
+        this.tbl = tbl;
+    }
+
+    /** {@inheritDoc} */
+    @Override public Tuple get(Tuple key) {
+        Objects.requireNonNull(key);
+
+        return marshaller().marshalKVPair(key, null);
+    }
+
+    /** {@inheritDoc} */
+    @Override public @NotNull IgniteFuture<Tuple> getAsync(Tuple key) {
+        return null;
+    }
+
+    /** {@inheritDoc} */
+    @Override public Map<Tuple, Tuple> getAll(Collection<Tuple> keys) {
+        return null;
+    }
+
+    /** {@inheritDoc} */
+    @Override public @NotNull IgniteFuture<Map<Tuple, Tuple>> getAllAsync(Collection<Tuple> keys) {
+        return null;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean contains(Tuple key) {
+        return false;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void put(Tuple key, Tuple val) {
+        Objects.requireNonNull(key);
+
+        final TableRow row = marshaller().marshalKVPair(key, val);
+
+        tbl.put(row);
+    }
+
+    /** {@inheritDoc} */
+    @Override public @NotNull IgniteFuture<Void> putAsync(Tuple key, Tuple val) {
+        return null;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void putAll(Map<Tuple, Tuple> pairs) {
+
+    }
+
+    /** {@inheritDoc} */
+    @Override public @NotNull IgniteFuture<Void> putAllAsync(Map<Tuple, Tuple> pairs) {
+        return null;
+    }
+
+    /** {@inheritDoc} */
+    @Override public Tuple getAndPut(Tuple key, Tuple val) {
+        return null;
+    }
+
+    /** {@inheritDoc} */
+    @Override public @NotNull IgniteFuture<Tuple> getAndPutAsync(Tuple key, Tuple val) {
+        return null;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean putIfAbsent(Tuple key, Tuple val) {
+        return false;
+    }
+
+    /** {@inheritDoc} */
+    @Override public @NotNull IgniteFuture<Boolean> putIfAbsentAsync(Tuple key, Tuple val) {
+        return null;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean remove(Tuple key) {
+        return false;
+    }
+
+    /** {@inheritDoc} */
+    @Override public @NotNull IgniteFuture<Boolean> removeAsync(Tuple key) {
+        return null;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean remove(Tuple key, Tuple val) {
+        return false;
+    }
+
+    /** {@inheritDoc} */
+    @Override public @NotNull IgniteFuture<Boolean> removeAsync(Tuple key, Tuple val) {
+        return null;
+    }
+
+    /** {@inheritDoc} */
+    @Override public Collection<Tuple> removeAll(Collection<Tuple> keys) {
+        return null;
+    }
+
+    /** {@inheritDoc} */
+    @Override public @NotNull IgniteFuture<Tuple> removeAllAsync(Collection<Tuple> keys) {
+        return null;
+    }
+
+    /** {@inheritDoc} */
+    @Override public Tuple getAndRemove(Tuple key) {
+        return null;
+    }
+
+    /** {@inheritDoc} */
+    @Override public @NotNull IgniteFuture<Tuple> getAndRemoveAsync(Tuple key) {
+        return null;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean replace(Tuple key, Tuple val) {
+        return false;
+    }
+
+    /** {@inheritDoc} */
+    @Override public @NotNull IgniteFuture<Boolean> replaceAsync(Tuple key, Tuple val) {
+        return null;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean replace(Tuple key, Tuple oldVal, Tuple newVal) {
+        return false;
+    }
+
+    /** {@inheritDoc} */
+    @Override public @NotNull IgniteFuture<Boolean> replaceAsync(Tuple key, Tuple oldVal,
+        Tuple newVal) {
+        return null;
+    }
+
+    /** {@inheritDoc} */
+    @Override public Tuple getAndReplace(Tuple key, Tuple val) {
+        return null;
+    }
+
+    /** {@inheritDoc} */
+    @Override public @NotNull IgniteFuture<Tuple> getAndReplaceAsync(Tuple key, Tuple val) {
+        return null;
+    }
+
+    /** {@inheritDoc} */
+    @Override public <R extends Serializable> R invoke(
+        Tuple key,
+        InvokeProcessor<Tuple, Tuple, R> proc,
+        Serializable... args
+    ) {
+        return null;
+    }
+
+    /** {@inheritDoc} */
+    @Override public @NotNull <R extends Serializable> IgniteFuture<R> invokeAsync(
+        Tuple key,
+        InvokeProcessor<Tuple, Tuple, R> proc,
+        Serializable... args
+    ) {
+        return null;
+    }
+
+    /** {@inheritDoc} */
+    @Override public <R extends Serializable> Map<Tuple, R> invokeAll(
+        Collection<Tuple> keys,
+        InvokeProcessor<Tuple, Tuple, R> proc,
+        Serializable... args
+    ) {
+        return null;
+    }
+
+    /** {@inheritDoc} */
+    @Override public @NotNull <R extends Serializable> IgniteFuture<Map<Tuple, R>> invokeAllAsync(
+        Collection<Tuple> keys,
+        InvokeProcessor<Tuple, Tuple, R> proc,
+        Serializable... args
+    ) {
+        return null;
+    }
+
+    /** {@inheritDoc} */
+    @Override public TupleBuilder tupleBuilder() {
+        return null;
+    }
+
+    /**
+     * @return Marshaller.
+     */
+    private Marshaller marshaller() {
+        return null;
+    }
+}
diff --git a/modules/table/src/main/java/org/apache/ignite/internal/table/RecordViewImpl.java b/modules/table/src/main/java/org/apache/ignite/internal/table/RecordViewImpl.java
new file mode 100644
index 0000000..60464fa
--- /dev/null
+++ b/modules/table/src/main/java/org/apache/ignite/internal/table/RecordViewImpl.java
@@ -0,0 +1,249 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.table;
+
+import java.io.Serializable;
+import java.util.Collection;
+import java.util.Map;
+import org.apache.ignite.internal.schema.marshaller.Marshaller;
+import org.apache.ignite.internal.storage.TableStorage;
+import org.apache.ignite.lang.IgniteFuture;
+import org.apache.ignite.table.InvokeProcessor;
+import org.apache.ignite.table.RecordView;
+import org.apache.ignite.table.mapper.RecordMapper;
+import org.jetbrains.annotations.NotNull;
+
+/**
+ * Record view implementation.
+ */
+public class RecordViewImpl<R> implements RecordView<R> {
+    /** Table */
+    private final TableStorage tbl;
+
+    /**
+     * Constructor.
+     *
+     * @param tbl Table.
+     * @param mapper Record class mapper.
+     */
+    public RecordViewImpl(TableStorage tbl, RecordMapper<R> mapper) {
+        this.tbl = tbl;
+    }
+
+    /** {@inheritDoc} */
+    @Override public R get(R keyRec) {
+        Marshaller marsh = marshaller();
+
+        TableRow kRow = marsh.serialize(keyRec);
+
+        TableRow tRow = tbl.get(kRow);
+
+        return marsh.deserializeToRecord(tRow);
+    }
+
+    /** {@inheritDoc} */
+    @Override public R fill(R recObjToFill) {
+        Marshaller marsh = marshaller();
+
+        TableRow kRow = marsh.serialize(recObjToFill);
+
+        TableRow tRow = tbl.get(kRow);
+
+        return marsh.deserializeToRecord(tRow, recObjToFill);
+    }
+
+    /** {@inheritDoc} */
+    @Override public @NotNull IgniteFuture<R> getAsync(R keyRec) {
+        return null;
+    }
+
+    /** {@inheritDoc} */
+    @Override public Collection<R> getAll(Collection<R> keyRecs) {
+        return null;
+    }
+
+    /** {@inheritDoc} */
+    @Override public @NotNull IgniteFuture<Collection<R>> getAllAsync(Collection<R> keyRecs) {
+        return null;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void upsert(R rec) {
+
+    }
+
+    /** {@inheritDoc} */
+    @Override public @NotNull IgniteFuture<Void> upsertAsync(R rec) {
+        return null;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void upsertAll(Collection<R> recs) {
+
+    }
+
+    /** {@inheritDoc} */
+    @Override public @NotNull IgniteFuture<Void> upsertAllAsync(Collection<R> recs) {
+        return null;
+    }
+
+    /** {@inheritDoc} */
+    @Override public R getAndUpsert(R rec) {
+        return null;
+    }
+
+    /** {@inheritDoc} */
+    @Override public @NotNull IgniteFuture<R> getAndUpsertAsync(R rec) {
+        return null;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean insert(R rec) {
+        return false;
+    }
+
+    /** {@inheritDoc} */
+    @Override public @NotNull IgniteFuture<Boolean> insertAsync(R rec) {
+        return null;
+    }
+
+    /** {@inheritDoc} */
+    @Override public Collection<R> insertAll(Collection<R> recs) {
+        return null;
+    }
+
+    /** {@inheritDoc} */
+    @Override public @NotNull IgniteFuture<Collection<R>> insertAllAsync(Collection<R> recs) {
+        return null;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean replace(R rec) {
+        return false;
+    }
+
+    /** {@inheritDoc} */
+    @Override public @NotNull IgniteFuture<Boolean> replaceAsync(R rec) {
+        return null;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean replace(R oldRec, R newRec) {
+        return false;
+    }
+
+    /** {@inheritDoc} */
+    @Override public @NotNull IgniteFuture<Boolean> replaceAsync(R oldRec, R newRec) {
+        return null;
+    }
+
+    /** {@inheritDoc} */
+    @Override public R getAndReplace(R rec) {
+        return null;
+    }
+
+    /** {@inheritDoc} */
+    @Override public @NotNull IgniteFuture<R> getAndReplaceAsync(R rec) {
+        return null;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean delete(R keyRec) {
+        return false;
+    }
+
+    /** {@inheritDoc} */
+    @Override public @NotNull IgniteFuture<Boolean> deleteAsync(R keyRec) {
+        return null;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean deleteExact(R oldRec) {
+        return false;
+    }
+
+    /** {@inheritDoc} */
+    @Override public @NotNull IgniteFuture<Boolean> deleteExactAsync(R oldRec) {
+        return null;
+    }
+
+    /** {@inheritDoc} */
+    @Override public R getAndDelete(R rec) {
+        return null;
+    }
+
+    /** {@inheritDoc} */
+    @Override public @NotNull IgniteFuture<R> getAndDeleteAsync(R rec) {
+        return null;
+    }
+
+    /** {@inheritDoc} */
+    @Override public Collection<R> deleteAll(Collection<R> recs) {
+        return null;
+    }
+
+    /** {@inheritDoc} */
+    @Override public @NotNull IgniteFuture<Collection<R>> deleteAllAsync(Collection<R> recs) {
+        return null;
+    }
+
+    /** {@inheritDoc} */
+    @Override public Collection<R> deleteAllExact(Collection<R> recs) {
+        return null;
+    }
+
+    /** {@inheritDoc} */
+    @Override public @NotNull IgniteFuture<Collection<R>> deleteAllExactAsync(Collection<R> recs) {
+        return null;
+    }
+
+    /** {@inheritDoc} */
+    @Override public <T extends Serializable> T invoke(R keyRec, InvokeProcessor<R, R, T> proc) {
+        return null;
+    }
+
+    /** {@inheritDoc} */
+    @Override public @NotNull <T extends Serializable> IgniteFuture<T> invokeAsync(R keyRec,
+        InvokeProcessor<R, R, T> proc) {
+        return null;
+    }
+
+    /** {@inheritDoc} */
+    @Override public <T extends Serializable> Map<R, T> invokeAll(
+        Collection<R> keyRecs,
+        InvokeProcessor<R, R, T> proc
+    ) {
+        return null;
+    }
+
+    /** {@inheritDoc} */
+    @Override public @NotNull <T extends Serializable> IgniteFuture<Map<R, T>> invokeAllAsync(
+        Collection<R> keyRecs,
+        InvokeProcessor<R, R, T> proc
+    ) {
+        return null;
+    }
+
+    /**
+     * @return Marshaller.
+     */
+    private Marshaller marshaller() {
+        return null;        // table.schemaManager().marshaller();
+    }
+
+}
diff --git a/modules/schema/src/main/java/org/apache/ignite/internal/schema/package-info.java b/modules/table/src/main/java/org/apache/ignite/internal/table/RowChunk.java
similarity index 78%
copy from modules/schema/src/main/java/org/apache/ignite/internal/schema/package-info.java
copy to modules/table/src/main/java/org/apache/ignite/internal/table/RowChunk.java
index fe5f0e3..43ba794 100644
--- a/modules/schema/src/main/java/org/apache/ignite/internal/schema/package-info.java
+++ b/modules/table/src/main/java/org/apache/ignite/internal/table/RowChunk.java
@@ -15,8 +15,16 @@
  * limitations under the License.
  */
 
+package org.apache.ignite.internal.table;
+
+import org.apache.ignite.table.Tuple;
+
 /**
- * <!-- Package description. -->
- * Contains schema description, tuple assembly and field accessor classes.
+ * Table row chunk.
  */
-package org.apache.ignite.internal.schema;
\ No newline at end of file
+public interface RowChunk extends Tuple {
+    /**
+     * @return Row chunk bytes.
+     */
+    public byte[] toBytes();
+}
diff --git a/modules/table/src/main/java/org/apache/ignite/internal/table/RowChunkAdapter.java b/modules/table/src/main/java/org/apache/ignite/internal/table/RowChunkAdapter.java
new file mode 100644
index 0000000..e0ff1a7
--- /dev/null
+++ b/modules/table/src/main/java/org/apache/ignite/internal/table/RowChunkAdapter.java
@@ -0,0 +1,106 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.table;
+
+import org.apache.ignite.binary.BinaryObject;
+import org.apache.ignite.binary.BinaryObjects;
+import org.apache.ignite.internal.schema.Column;
+import org.apache.ignite.internal.schema.Row;
+import org.jetbrains.annotations.NotNull;
+
+/**
+ * Row to RowChunk adapter.
+ */
+public abstract class RowChunkAdapter implements RowChunk {
+    /**
+     * @param colName Column name.
+     * @return Column.
+     */
+    @NotNull protected abstract Column columnByName(@NotNull String colName);
+
+    /**
+     * @return Underlying row.
+     */
+    protected abstract Row row();
+
+    /** {@inheritDoc} */
+    @Override public abstract byte[] toBytes();
+
+    /** {@inheritDoc} */
+    @Override public <T> T value(String colName) {
+        final Column col = columnByName(colName);
+
+        return (T)col.type().spec().objectValue(row(), col.schemaIndex());
+    }
+
+    /** {@inheritDoc} */
+    @Override public BinaryObject binaryObjectField(String colName) {
+        Column col = columnByName(colName);
+
+        return BinaryObjects.wrap(row().bytesValue(col.schemaIndex()));
+    }
+
+    /** {@inheritDoc} */
+    @Override public byte byteValue(String colName) {
+        Column col = columnByName(colName);
+
+        return row().byteValue(col.schemaIndex());
+    }
+
+    /** {@inheritDoc} */
+    @Override public short shortValue(String colName) {
+        Column col = columnByName(colName);
+
+        return row().shortValue(col.schemaIndex());
+    }
+
+    /** {@inheritDoc} */
+    @Override public int intValue(String colName) {
+        Column col = columnByName(colName);
+
+        return row().intValue(col.schemaIndex());
+    }
+
+    /** {@inheritDoc} */
+    @Override public long longValue(String colName) {
+        Column col = columnByName(colName);
+
+        return row().longValue(col.schemaIndex());
+    }
+
+    /** {@inheritDoc} */
+    @Override public float floatValue(String colName) {
+        Column col = columnByName(colName);
+
+        return row().floatValue(col.schemaIndex());
+    }
+
+    /** {@inheritDoc} */
+    @Override public double doubleValue(String colName) {
+        Column col = columnByName(colName);
+
+        return row().doubleValue(col.schemaIndex());
+    }
+
+    /** {@inheritDoc} */
+    @Override public String stringValue(String colName) {
+        Column col = columnByName(colName);
+
+        return row().stringValue(col.schemaIndex());
+    }
+}
diff --git a/modules/table/src/main/java/org/apache/ignite/internal/table/TableImpl.java b/modules/table/src/main/java/org/apache/ignite/internal/table/TableImpl.java
new file mode 100644
index 0000000..f64bb21
--- /dev/null
+++ b/modules/table/src/main/java/org/apache/ignite/internal/table/TableImpl.java
@@ -0,0 +1,265 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.table;
+
+import java.io.Serializable;
+import java.util.Collection;
+import java.util.Map;
+import org.apache.ignite.internal.schema.marshaller.Marshaller;
+import org.apache.ignite.internal.storage.TableStorage;
+import org.apache.ignite.lang.IgniteFuture;
+import org.apache.ignite.table.InvokeProcessor;
+import org.apache.ignite.table.KeyValueBinaryView;
+import org.apache.ignite.table.KeyValueView;
+import org.apache.ignite.table.RecordView;
+import org.apache.ignite.table.Table;
+import org.apache.ignite.table.Tuple;
+import org.apache.ignite.table.TupleBuilder;
+import org.apache.ignite.table.mapper.KeyMapper;
+import org.apache.ignite.table.mapper.RecordMapper;
+import org.apache.ignite.table.mapper.ValueMapper;
+import org.jetbrains.annotations.NotNull;
+
+/**
+ * Table view implementation for binary objects.
+ */
+public class TableImpl implements Table {
+    /** Table. */
+    private final TableStorage tbl;
+
+    /**
+     * Constructor.
+     *
+     * @param tbl Table.
+     */
+    public TableImpl(TableStorage tbl) {
+        this.tbl = tbl;
+    }
+
+    /** {@inheritDoc} */
+    @Override public <R> RecordView<R> recordView(RecordMapper<R> recMapper) {
+        return new RecordViewImpl<>(tbl, recMapper);
+    }
+
+    /** {@inheritDoc} */
+    @Override public <K, V> KeyValueView<K, V> kvView(KeyMapper<K> keyMapper, ValueMapper<V> valMapper) {
+        return new KVViewImpl<>(tbl, keyMapper, valMapper);
+    }
+
+    /** {@inheritDoc} */
+    @Override public KeyValueBinaryView kvView() {
+        return new KeyValueBinaryViewImpl(tbl);
+    }
+
+    /** {@inheritDoc} */
+    @Override public Tuple get(Tuple keyRec) {
+        Marshaller marsh = marshaller();
+
+        return tbl.get(marsh.marshalRecord(keyRec));
+    }
+
+    /** {@inheritDoc} */
+    @Override public @NotNull IgniteFuture<Tuple> getAsync(Tuple keyRec) {
+        return null;
+    }
+
+    /** {@inheritDoc} */
+    @Override public Collection<Tuple> getAll(Collection<Tuple> keyRecs) {
+        return null;
+    }
+
+    /** {@inheritDoc} */
+    @Override public @NotNull IgniteFuture<Collection<Tuple>> getAllAsync(Collection<Tuple> keyRecs) {
+        return null;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void upsert(Tuple rec) {
+
+    }
+
+    /** {@inheritDoc} */
+    @Override public @NotNull IgniteFuture<Void> upsertAsync(Tuple rec) {
+        return null;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void upsertAll(Collection<Tuple> recs) {
+
+    }
+
+    /** {@inheritDoc} */
+    @Override public @NotNull IgniteFuture<Void> upsertAllAsync(Collection<Tuple> recs) {
+        return null;
+    }
+
+    /** {@inheritDoc} */
+    @Override public Tuple getAndUpsert(Tuple rec) {
+        return null;
+    }
+
+    /** {@inheritDoc} */
+    @Override public @NotNull IgniteFuture<Tuple> getAndUpsertAsync(Tuple rec) {
+        return null;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean insert(Tuple rec) {
+        return false;
+    }
+
+    /** {@inheritDoc} */
+    @Override public @NotNull IgniteFuture<Boolean> insertAsync(Tuple rec) {
+        return null;
+    }
+
+    /** {@inheritDoc} */
+    @Override public Collection<Tuple> insertAll(Collection<Tuple> recs) {
+        return null;
+    }
+
+    /** {@inheritDoc} */
+    @Override public @NotNull IgniteFuture<Collection<Tuple>> insertAllAsync(Collection<Tuple> recs) {
+        return null;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean replace(Tuple rec) {
+        return false;
+    }
+
+    /** {@inheritDoc} */
+    @Override public @NotNull IgniteFuture<Boolean> replaceAsync(Tuple rec) {
+        return null;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean replace(Tuple oldRec, Tuple newRec) {
+        return false;
+    }
+
+    /** {@inheritDoc} */
+    @Override public @NotNull IgniteFuture<Boolean> replaceAsync(Tuple oldRec, Tuple newRec) {
+        return null;
+    }
+
+    /** {@inheritDoc} */
+    @Override public Tuple getAndReplace(Tuple rec) {
+        return null;
+    }
+
+    /** {@inheritDoc} */
+    @Override public @NotNull IgniteFuture<Tuple> getAndReplaceAsync(Tuple rec) {
+        return null;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean delete(Tuple keyRec) {
+        return false;
+    }
+
+    /** {@inheritDoc} */
+    @Override public @NotNull IgniteFuture<Boolean> deleteAsync(Tuple keyRec) {
+        return null;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean deleteExact(Tuple oldRec) {
+        return false;
+    }
+
+    /** {@inheritDoc} */
+    @Override public @NotNull IgniteFuture<Boolean> deleteExactAsync(Tuple oldRec) {
+        return null;
+    }
+
+    /** {@inheritDoc} */
+    @Override public Tuple getAndDelete(Tuple rec) {
+        return null;
+    }
+
+    /** {@inheritDoc} */
+    @Override public @NotNull IgniteFuture<Tuple> getAndDeleteAsync(Tuple rec) {
+        return null;
+    }
+
+    /** {@inheritDoc} */
+    @Override public Collection<Tuple> deleteAll(Collection<Tuple> recs) {
+        return null;
+    }
+
+    /** {@inheritDoc} */
+    @Override public @NotNull IgniteFuture<Collection<Tuple>> deleteAllAsync(Collection<Tuple> recs) {
+        return null;
+    }
+
+    /** {@inheritDoc} */
+    @Override public Collection<Tuple> deleteAllExact(Collection<Tuple> recs) {
+        return null;
+    }
+
+    /** {@inheritDoc} */
+    @Override public @NotNull IgniteFuture<Collection<Tuple>> deleteAllExactAsync(
+        Collection<Tuple> recs) {
+        return null;
+    }
+
+    /** {@inheritDoc} */
+    @Override public <T extends Serializable> T invoke(
+        Tuple keyRec,
+        InvokeProcessor<Tuple, Tuple, T> proc
+    ) {
+        return null;
+    }
+
+    /** {@inheritDoc} */
+    @Override public @NotNull <T extends Serializable> IgniteFuture<T> invokeAsync(
+        Tuple keyRec,
+        InvokeProcessor<Tuple, Tuple, T> proc
+    ) {
+        return null;
+    }
+
+    /** {@inheritDoc} */
+    @Override public <T extends Serializable> Map<Tuple, T> invokeAll(
+        Collection<Tuple> keyRecs,
+        InvokeProcessor<Tuple, Tuple, T> proc
+    ) {
+        return null;
+    }
+
+    /** {@inheritDoc} */
+    @Override public @NotNull <T extends Serializable> IgniteFuture<Map<Tuple, T>> invokeAllAsync(
+        Collection<Tuple> keyRecs,
+        InvokeProcessor<Tuple, Tuple, T> proc
+    ) {
+        return null;
+    }
+
+    /** {@inheritDoc} */
+    @Override public TupleBuilder tupleBuilder() {
+        return null;
+    }
+
+    /**
+     * @return Marshaller.
+     */
+    private Marshaller marshaller() {
+        return null;
+    }
+}
diff --git a/modules/schema/src/main/java/org/apache/ignite/internal/schema/InvalidTypeException.java b/modules/table/src/main/java/org/apache/ignite/internal/table/TableRow.java
similarity index 72%
copy from modules/schema/src/main/java/org/apache/ignite/internal/schema/InvalidTypeException.java
copy to modules/table/src/main/java/org/apache/ignite/internal/table/TableRow.java
index d03ca73..e00ca7b 100644
--- a/modules/schema/src/main/java/org/apache/ignite/internal/schema/InvalidTypeException.java
+++ b/modules/table/src/main/java/org/apache/ignite/internal/table/TableRow.java
@@ -15,16 +15,24 @@
  * limitations under the License.
  */
 
-package org.apache.ignite.internal.schema;
+package org.apache.ignite.internal.table;
 
 /**
- * An exception thrown when an attempt to read an invalid type from a tuple is performed.
+ * Table binary row interface.
  */
-public class InvalidTypeException extends IllegalArgumentException {
+public interface TableRow extends RowChunk {
     /**
-     * @param msg Error message.
+     * @return Key chunk.
      */
-    public InvalidTypeException(String msg) {
-        super(msg);
-    }
+    RowChunk keyChunk();
+
+    /**
+     * @return Value chunk.
+     */
+    RowChunk valueChunk();
+
+    /**
+     * @return Row schema version.
+     */
+    long schemaVersion();
 }
diff --git a/modules/table/src/main/java/org/apache/ignite/internal/table/TableRowAdapter.java b/modules/table/src/main/java/org/apache/ignite/internal/table/TableRowAdapter.java
new file mode 100644
index 0000000..75eeede
--- /dev/null
+++ b/modules/table/src/main/java/org/apache/ignite/internal/table/TableRowAdapter.java
@@ -0,0 +1,142 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.table;
+
+import java.util.Objects;
+import org.apache.ignite.internal.schema.Column;
+import org.apache.ignite.internal.schema.Row;
+import org.apache.ignite.internal.schema.SchemaDescriptor;
+import org.jetbrains.annotations.NotNull;
+
+/**
+ * Table row adapter for Row.
+ */
+public class TableRowAdapter extends RowChunkAdapter implements TableRow {
+    /** Schema. */
+    private final SchemaDescriptor schema;
+
+    /** Row. */
+    private final Row row;
+
+    /** Key chunk projection. */
+    private final RowChunk keyChunk;
+
+    /** Value chunk projection. */
+    private final RowChunk valChunk;
+
+    /**
+     * Constructor.
+     *
+     * @param row Row.
+     * @param schema Schema descriptor.
+     */
+    public TableRowAdapter(Row row, SchemaDescriptor schema) {
+        this.schema = schema;
+        this.row = row;
+
+        keyChunk = new KeyRowChunk();
+        valChunk = row.hasValue() ? new ValueRowChunk() : null;
+    }
+
+    /** {@inheritDoc} */
+    @Override public long schemaVersion() {
+        return schema.version();
+    }
+
+    /** {@inheritDoc} */
+    @Override @NotNull protected final Column columnByName(@NotNull String colName) {
+        Objects.requireNonNull(colName);
+
+        final Column col = schema.column(colName);
+
+        if (col == null)
+            throw new IllegalArgumentException("Invalid column name: columnName=" + colName + ", schemaVersion=" + schema.version());
+
+        return col;
+    }
+
+    /** {@inheritDoc} */
+    @Override public byte[] toBytes() {
+        return row.rowBytes();
+    }
+
+    /** {@inheritDoc} */
+    @Override public RowChunk keyChunk() {
+        return keyChunk;
+    }
+
+    /** {@inheritDoc} */
+    @Override public RowChunk valueChunk() {
+        return valChunk;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected Row row() {
+        return row;
+    }
+
+    /** Key column chunk. */
+    private class KeyRowChunk extends RowChunkAdapter {
+        /** {@inheritDoc} */
+        @Override protected Row row() {
+            return row;
+        }
+
+        /** {@inheritDoc} */
+        @Override protected @NotNull Column columnByName(@NotNull String colName) {
+            Objects.requireNonNull(colName);
+
+            final Column col = schema.column(colName);
+
+            if (col == null || !schema.keyColumn(col.schemaIndex()))
+                throw new IllegalArgumentException("Invalid key column name: columnName=" + colName + ", schemaVersion=" + schema.version());
+
+            return col;
+        }
+
+        /** {@inheritDoc} */
+        @Override public byte[] toBytes() {
+            return row.keyChunkBytes();
+        }
+    }
+
+    /** Value column chunk. */
+    private class ValueRowChunk extends RowChunkAdapter {
+        /** {@inheritDoc} */
+        @Override protected Row row() {
+            return row;
+        }
+
+        /** {@inheritDoc} */
+        @Override protected @NotNull Column columnByName(@NotNull String colName) {
+            Objects.requireNonNull(colName);
+
+            final Column col = schema.column(colName);
+
+            if (col == null || schema.keyColumn(col.schemaIndex()))
+                throw new IllegalArgumentException("Invalid key column name: columnName=" + colName + ", schemaVersion=" + schema.version());
+
+            return col;
+        }
+
+        /** {@inheritDoc} */
+        @Override public byte[] toBytes() {
+            return row.valueChunkBytes();
+        }
+    }
+}
diff --git a/modules/schema/src/main/java/org/apache/ignite/internal/schema/package-info.java b/modules/table/src/main/java/org/apache/ignite/internal/table/package-info.java
similarity index 84%
copy from modules/schema/src/main/java/org/apache/ignite/internal/schema/package-info.java
copy to modules/table/src/main/java/org/apache/ignite/internal/table/package-info.java
index fe5f0e3..81ce412 100644
--- a/modules/schema/src/main/java/org/apache/ignite/internal/schema/package-info.java
+++ b/modules/table/src/main/java/org/apache/ignite/internal/table/package-info.java
@@ -16,7 +16,6 @@
  */
 
 /**
- * <!-- Package description. -->
- * Contains schema description, tuple assembly and field accessor classes.
+ * Table access API implementation.
  */
-package org.apache.ignite.internal.schema;
\ No newline at end of file
+package org.apache.ignite.internal.table;
\ No newline at end of file
diff --git a/modules/table/src/test/java/org/apache/ignite/table/Example.java b/modules/table/src/test/java/org/apache/ignite/table/Example.java
new file mode 100644
index 0000000..7fa195f
--- /dev/null
+++ b/modules/table/src/test/java/org/apache/ignite/table/Example.java
@@ -0,0 +1,456 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.table;
+
+import java.math.BigDecimal;
+import java.util.Collections;
+import java.util.List;
+import org.apache.ignite.binary.BinaryObjects;
+import org.apache.ignite.internal.table.TableImpl;
+import org.apache.ignite.binary.BinaryObject;
+import org.apache.ignite.table.impl.TestTableStorageImpl;
+import org.apache.ignite.table.mapper.Mappers;
+import org.junit.jupiter.api.Disabled;
+import org.junit.jupiter.params.ParameterizedTest;
+import org.junit.jupiter.params.provider.MethodSource;
+
+/**
+ *
+ */
+@SuppressWarnings({"PMD.EmptyLineSeparatorCheck", "emptylineseparator",
+    "unused", "UnusedAssignment", "InstanceVariableMayNotBeInitialized", "JoinDeclarationAndAssignmentJava"})
+public class Example {
+    /**
+     * @return Table implementation.
+     */
+    private static List<Table> tableFactory() {
+        return Collections.singletonList(new TableImpl(new TestTableStorageImpl()));
+    }
+
+    /**
+     * Use case 1: a simple one. The table has the structure
+     * [
+     * [id int, orgId int] // key
+     * [name varchar, lastName varchar, decimal salary, int department] // value
+     * ]
+     * We show how to use the raw TableRow and a mapped class.
+     */
+    @Disabled
+    @ParameterizedTest
+    @MethodSource("tableFactory")
+    public void useCase1(Table t) {
+        // Search row will allow nulls even in non-null columns.
+        Tuple res = t.get(t.tupleBuilder().set("id", 1).set("orgId", 1).build());
+
+        String name = res.value("name");
+        String lastName = res.value("latName");
+        BigDecimal salary = res.value("salary");
+        Integer department = res.value("department");
+
+        // We may have primitive-returning methods if needed.
+        int departmentPrimitive = res.intValue("department");
+
+        // Note that schema itself already defined which fields are key field.
+        class Employee {
+            final int id;
+            final int orgId;
+
+            String name;
+            String lastName;
+            BigDecimal salary;
+            int department;
+
+            Employee(int id, int orgId) {
+                this.id = id;
+                this.orgId = orgId;
+            }
+        }
+        RecordView<Employee> employeeView = t.recordView(Employee.class);
+
+        Employee e = employeeView.get(new Employee(1, 1));
+
+        // As described in the IEP-54, we can have a truncated mapping.
+        class TruncatedEmployee {
+            final int id;
+            final int orgId;
+
+            String name;
+            String lastName;
+
+            TruncatedEmployee(int id, int orgId) {
+                this.id = id;
+                this.orgId = orgId;
+            }
+        }
+
+        RecordView<TruncatedEmployee> truncatedEmployeeView = t.recordView(TruncatedEmployee.class);
+
+        // salary and department will not be sent over the network during this call.
+        TruncatedEmployee te = truncatedEmployeeView.get(new TruncatedEmployee(1, 1));
+    }
+
+    /**
+     * Use case 2: using simple KV mappings
+     * The table has structure is
+     * [
+     * [id int, orgId int] // key
+     * [name varchar, lastName varchar, decimal salary, int department] // value
+     * ]
+     */
+    @Disabled
+    @ParameterizedTest
+    @MethodSource("tableFactory")
+    public void useCase2(Table t) {
+        class EmployeeKey {
+            final int id;
+            final int orgId;
+
+            EmployeeKey(int id, int orgId) {
+                this.id = id;
+                this.orgId = orgId;
+            }
+        }
+
+        class Employee {
+            String name;
+            String lastName;
+            BigDecimal salary;
+            int department;
+        }
+
+        KeyValueView<EmployeeKey, Employee> employeeKv = t.kvView(EmployeeKey.class, Employee.class);
+
+        employeeKv.get(new EmployeeKey(1, 1));
+
+        // As described in the IEP-54, we can have a truncated KV mapping.
+        class TruncatedEmployee {
+            String name;
+            String lastName;
+        }
+
+        KeyValueView<EmployeeKey, TruncatedEmployee> truncatedEmployeeKv = t.kvView(EmployeeKey.class, TruncatedEmployee.class);
+
+        TruncatedEmployee te = truncatedEmployeeKv.get(new EmployeeKey(1, 1));
+    }
+
+    /**
+     * Use case 3: Single table strategy for inherited objects.
+     * The table has structure is
+     * [
+     * [id long] // key
+     * [owner varchar, cardNumber long, expYear int, expMonth int, accountNum long, bankName varchar] // value
+     * ]
+     */
+    @Disabled
+    @ParameterizedTest
+    @MethodSource("tableFactory")
+    public void useCase3(Table t) {
+        class BillingDetails {
+            String owner;
+        }
+
+        class CreditCard extends BillingDetails {
+            long cardNumber;
+            int expYear;
+            int expMonth;
+        }
+
+        class BankAccount extends BillingDetails {
+            long account;
+            String bankName;
+        }
+
+        KeyValueView<Long, CreditCard> credCardKvView = t.kvView(Long.class, CreditCard.class);
+        CreditCard creditCard = credCardKvView.get(1L);
+
+        KeyValueView<Long, BankAccount> backAccKvView = t.kvView(Long.class, BankAccount.class);
+        BankAccount bankAccount = backAccKvView.get(2L);
+
+        // Truncated view.
+        KeyValueView<Long, BillingDetails> billingDetailsKVView = t.kvView(Long.class, BillingDetails.class);
+        BillingDetails billingDetails = billingDetailsKVView.get(2L);
+
+        // Without discriminator it is impossible to deserialize to correct type automatically.
+        assert !(billingDetails instanceof CreditCard);
+        assert !(billingDetails instanceof BankAccount);
+
+        // Wide record.
+        class BillingRecord {
+            final long id;
+
+            String owner;
+
+            long cardNumber;
+            int expYear;
+            int expMonth;
+
+            long account;
+            String bankName;
+
+            BillingRecord(long id) {
+                this.id = id;
+            }
+        }
+
+        final RecordView<BillingRecord> billingView = t.recordView(BillingRecord.class);
+
+        final BillingRecord br = billingView.get(new BillingRecord(1));
+    }
+
+    /**
+     * Use case 4: Conditional serialization.
+     * The table has structure is
+     * [
+     * [id int, orgId int] // key
+     * [owner varchar, type int, conditionalDetails byte[]] // value
+     * ]
+     */
+    @Disabled
+    @ParameterizedTest
+    @MethodSource("tableFactory")
+    public void useCase4(Table t) {
+        class OrderKey {
+            final int id;
+            final int orgId;
+
+            OrderKey(int id, int orgId) {
+                this.id = id;
+                this.orgId = orgId;
+            }
+        }
+
+        class OrderValue {
+            String owner;
+            int type; // Discriminator value.
+            /* BillingDetails */ Object billingDetails;
+        }
+
+        class CreditCard /* extends BillingDetails */ {
+            long cardNumber;
+            int expYear;
+            int expMonth;
+        }
+
+        class BankAccount /* extends BillingDetails */ {
+            long account;
+            String bankName;
+        }
+
+        KeyValueView<OrderKey, OrderValue> orderKvView = t.kvView(Mappers.ofKeyClass(OrderKey.class),
+            Mappers.ofValueClassBuilder(OrderValue.class)
+                .map("billingDetails", (row) -> {
+                    BinaryObject bObj = row.binaryObjectField("conditionalDetails");
+                    int type = row.intValue("type");
+
+                    return type == 0 ?
+                        BinaryObjects.deserialize(bObj, CreditCard.class) :
+                        BinaryObjects.deserialize(bObj, BankAccount.class);
+                }).build());
+
+        OrderValue ov = orderKvView.get(new OrderKey(1, 1));
+
+        // Same with direct Row access and BinaryObject wrapper.
+        Tuple res = t.get(t.tupleBuilder().set("id", 1).set("orgId", 1).build());
+
+        byte[] objData = res.value("billingDetails");
+        BinaryObject binObj = BinaryObjects.wrap(objData);
+        // Work with the binary object as in Ignite 2.x
+
+        // Additionally, we may have a shortcut similar to primitive methods.
+        binObj = res.binaryObjectField("billingDetails");
+
+        // Same with RecordAPI.
+        class OrderRecord {
+            final int id;
+            final int orgId;
+
+            String owner;
+            int type;
+            BinaryObject billingDetails;
+
+            OrderRecord(int id, int orgId) {
+                this.id = id;
+                this.orgId = orgId;
+            }
+        }
+
+        final RecordView<OrderRecord> orderRecView = t.recordView(OrderRecord.class);
+
+        OrderRecord orderRecord = orderRecView.get(new OrderRecord(1, 1));
+        binObj = orderRecord.billingDetails;
+
+        // Manual deserialization is possible as well.
+        Object billingDetails = orderRecord.type == 0 ?
+            BinaryObjects.deserialize(binObj, CreditCard.class) :
+            BinaryObjects.deserialize(binObj, BankAccount.class);
+    }
+
+    /**
+     * Use case 5: using byte[] and binary objects in columns.
+     * The table has structure
+     * [
+     * [id int, orgId int] // key
+     * [originalObject byte[], upgradedObject byte[], int department] // value
+     * ]
+     * Where {@code originalObject} is some value that was originally put to the column,
+     * {@code upgradedObject} is a version 2 of the object, and department is an extracted field.
+     */
+    @Disabled
+    @ParameterizedTest
+    @MethodSource("tableFactory")
+    public void useCase5(Table t) {
+        Tuple res = t.get(t.tupleBuilder().set("id", 1).set("orgId", 1).build());
+
+        byte[] objData = res.value("originalObject");
+        BinaryObject binObj = BinaryObjects.wrap(objData);
+        // Work with the binary object as in Ignite 2.x
+
+        // Additionally, we may have a shortcut similar to primitive methods.
+        binObj = res.binaryObjectField("upgradedObject");
+
+        // Plain byte[] and BinaryObject fields in a class are straightforward.
+        class Record {
+            final int id;
+            final int orgId;
+
+            byte[] originalObject;
+            Tuple upgradedObject;
+            int department;
+
+            Record(int id, int orgId) {
+                this.id = id;
+                this.orgId = orgId;
+            }
+        }
+
+        RecordView<Record> recordView = t.recordView(Record.class);
+
+        // Similarly work with the binary objects.
+        Record rec = recordView.get(new Record(1, 1));
+
+        // Now assume that we have some POJO classes to deserialize the binary objects.
+        class JavaPerson {
+            String name;
+            String lastName;
+        }
+
+        class JavaPersonV2 extends JavaPerson {
+            int department;
+        }
+
+        // We can have a compound record deserializing the whole tuple automatically.
+        class JavaPersonRecord {
+            JavaPerson originalObject;
+            JavaPersonV2 upgradedObject;
+            int department;
+        }
+
+        RecordView<JavaPersonRecord> personRecordView = t.recordView(JavaPersonRecord.class);
+
+        // Or we can have an arbitrary record with custom class selection.
+        class TruncatedRecord {
+            JavaPerson upgradedObject;
+            int department;
+        }
+
+        RecordView<TruncatedRecord> truncatedView = t.recordView(
+            Mappers.ofRowClassBuilder(TruncatedRecord.class)
+                .map("upgradedObject", JavaPersonV2.class).build());
+
+        // Or we can have a custom conditional type selection.
+        RecordView<TruncatedRecord> truncatedView2 = t.recordView(
+            Mappers.ofRowClassBuilder(TruncatedRecord.class)
+                .map("upgradedObject", (row) -> {
+                    BinaryObject bObj = row.binaryObjectField("upgradedObject");
+                    int dept = row.intValue("department");
+
+                    return dept == 0 ?
+                        BinaryObjects.deserialize(bObj, JavaPerson.class) :
+                        BinaryObjects.deserialize(bObj, JavaPersonV2.class);
+                }).build());
+    }
+
+    /**
+     * Use case 1: a simple one. The table has the structure
+     * [
+     * [id long] // key
+     * [name varchar, lastName varchar, decimal salary, int department] // value
+     * ]
+     * We show how to use the raw TableRow and a mapped class.
+     */
+    @Disabled
+    @ParameterizedTest
+    @MethodSource("tableFactory")
+    public void useCase6(Table t) {
+        // Search row will allow nulls even in non-null columns.
+        Tuple res = t.get(t.tupleBuilder().set("id", 1).build());
+
+        String name = res.value("name");
+        String lastName = res.value("latName");
+        BigDecimal salary = res.value("salary");
+        Integer department = res.value("department");
+
+        // We may have primitive-returning methods if needed.
+        int departmentPrimitive = res.intValue("department");
+
+        // Note that schema itself already defined which fields are key field.
+        class Employee {
+            String name;
+            String lastName;
+            BigDecimal salary;
+            int department;
+        }
+
+        class Key {
+            long id;
+        }
+
+        KeyValueView<Long, Employee> employeeView = t.kvView(Long.class, Employee.class);
+
+        Employee e = employeeView.get(1L);
+    }
+
+    /**
+     * Use case 1: a simple one. The table has the structure
+     * [
+     * [byte[]] // key
+     * [name varchar, lastName varchar, decimal salary, int department] // value
+     * ]
+     * We show how to use the raw TableRow and a mapped class.
+     */
+    @Disabled
+    @ParameterizedTest
+    @MethodSource("tableFactory")
+    public void useCase7(Table t) {
+        // Note that schema itself already defined which fields are key field.
+        class Employee {
+            String name;
+            String lastName;
+            BigDecimal salary;
+            int department;
+        }
+
+        KeyValueView<Long, BinaryObject> employeeView = t.kvView(Long.class, BinaryObject.class);
+
+        employeeView.put(1L, BinaryObjects.wrap(new byte[0] /* serialized Employee */));
+
+        t.kvView(
+            Mappers.identity(),
+            Mappers.ofValueClassBuilder(BinaryObject.class).deserializeTo(Employee.class).build());
+    }
+}
diff --git a/modules/table/src/test/java/org/apache/ignite/table/impl/TestTableRowImpl.java b/modules/table/src/test/java/org/apache/ignite/table/impl/TestTableRowImpl.java
new file mode 100644
index 0000000..5c64b29
--- /dev/null
+++ b/modules/table/src/test/java/org/apache/ignite/table/impl/TestTableRowImpl.java
@@ -0,0 +1,132 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.table.impl;
+
+import java.nio.ByteBuffer;
+import org.apache.ignite.binary.BinaryObject;
+import org.apache.ignite.internal.schema.Row;
+import org.apache.ignite.internal.table.RowChunk;
+import org.apache.ignite.internal.table.TableRow;
+
+/**
+ * Dummy implementation class.
+ */
+public class TestTableRowImpl implements TableRow {
+    /** Key offset in tuple. */
+    private static final int KEY_OFFSET = Row.KEY_HASH_FIELD_OFFSET;
+
+    /** Payload. */
+    private final byte[] bytes;
+
+    /**
+     * Constructor.
+     *
+     * @param bytes Bytes to wrap.
+     */
+    public TestTableRowImpl(byte[] bytes) {
+        this.bytes = bytes.clone();
+    }
+
+    /** {@inheritDoc} */
+    @Override public byte[] toBytes() {
+        return bytes.clone();
+    }
+
+    /** {@inheritDoc} */
+    @Override public RowChunk keyChunk() {
+        return new TestTableRowImpl(bytes) {
+            @Override public byte[] toBytes() {
+                ByteBuffer buf = ByteBuffer.wrap(bytes());
+
+                int keyLen = buf.getInt(KEY_OFFSET);
+
+                return buf.position(KEY_OFFSET).limit(keyLen).slice().array();
+            }
+        };
+    }
+
+    /** {@inheritDoc} */
+    @Override public RowChunk valueChunk() {
+        return new TestTableRowImpl(bytes) {
+            @Override public byte[] toBytes() {
+                ByteBuffer buf = ByteBuffer.wrap(bytes());
+
+                int valOffset = KEY_OFFSET + buf.getInt(KEY_OFFSET);
+
+                return buf.position(valOffset).slice().array();
+            }
+        };
+    }
+
+    /** {@inheritDoc} */
+    @Override public <T> T value(String colName) {
+        return null;
+    }
+
+    /** {@inheritDoc} */
+    @Override public BinaryObject binaryObjectField(String colName) {
+        return null;
+    }
+
+    /** {@inheritDoc} */
+    @Override public byte byteValue(String colName) {
+        return 0;
+    }
+
+    /** {@inheritDoc} */
+    @Override public short shortValue(String colName) {
+        return 0;
+    }
+
+    /** {@inheritDoc} */
+    @Override public int intValue(String colName) {
+        return 0;
+    }
+
+    /** {@inheritDoc} */
+    @Override public long longValue(String colName) {
+        return 0;
+    }
+
+    /** {@inheritDoc} */
+    @Override public float floatValue(String colName) {
+        return 0;
+    }
+
+    /** {@inheritDoc} */
+    @Override public double doubleValue(String colName) {
+        return 0;
+    }
+
+    /** {@inheritDoc} */
+    @Override public String stringValue(String colName) {
+        return null;
+    }
+
+    /** {@inheritDoc} */
+    @Override public long schemaVersion() {
+        return 0;
+    }
+
+    /**
+     *
+     */
+    private byte[] bytes() {
+        return bytes;
+    }
+}
diff --git a/modules/table/src/test/java/org/apache/ignite/table/impl/TestTableStorageImpl.java b/modules/table/src/test/java/org/apache/ignite/table/impl/TestTableStorageImpl.java
new file mode 100644
index 0000000..7814787
--- /dev/null
+++ b/modules/table/src/test/java/org/apache/ignite/table/impl/TestTableStorageImpl.java
@@ -0,0 +1,82 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.table.impl;
+
+import java.util.Arrays;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import org.apache.ignite.internal.storage.TableStorage;
+import org.apache.ignite.internal.table.TableRow;
+
+/**
+ * Dummy table storage implementation.
+ */
+public class TestTableStorageImpl implements TableStorage {
+    /** In-memory dummy store. */
+    private final Map<KeyChunk, TestTableRowImpl> store = new ConcurrentHashMap<>();
+
+    /** {@inheritDoc} */
+    @Override public TableRow get(TableRow obj) {
+        return store.get(new KeyChunk(obj.keyChunk().toBytes()));
+    }
+
+    /** {@inheritDoc} */
+    @Override public TableRow put(TableRow row) {
+        return store.put(
+            new KeyChunk(row.keyChunk().toBytes()),
+            new TestTableRowImpl(row.toBytes()));
+    }
+
+    /**
+     * Wrapper provides correct byte[] comparison.
+     */
+    private static class KeyChunk {
+        /** Data. */
+        private final byte[] data;
+
+        /** Hash. */
+        private final int hash;
+
+        /**
+         * Constructor.
+         *
+         * @param data Wrapped data.
+         */
+        KeyChunk(byte[] data) {
+            this.data = data;
+            this.hash = Arrays.hashCode(data);
+        }
+
+        /** {@inheritDoc} */
+        @Override public boolean equals(Object o) {
+            if (this == o)
+                return true;
+
+            if (o == null || getClass() != o.getClass())
+                return false;
+
+            KeyChunk wrapper = (KeyChunk)o;
+            return Arrays.equals(data, wrapper.data);
+        }
+
+        /** {@inheritDoc} */
+        @Override public int hashCode() {
+            return hash;
+        }
+    }
+}
diff --git a/pom.xml b/pom.xml
index 75141f2..7910714 100644
--- a/pom.xml
+++ b/pom.xml
@@ -46,5 +46,6 @@
         <module>modules/rest</module>
         <module>modules/runner</module>
         <module>modules/schema</module>
+        <module>modules/table</module>
     </modules>
 </project>