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 2020/12/10 14:30:10 UTC

[ignite-3] 01/01: IGNITE-13617: Provide an initial implementation for assembling/reading tuples for a given schema.

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

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

commit a8b66dde653ae3894e86bea54da62bdb575eebca
Author: Andrew Mashenkov <an...@gmail.com>
AuthorDate: Thu Dec 10 17:29:53 2020 +0300

    IGNITE-13617: Provide an initial implementation for assembling/reading tuples for a given schema.
---
 modules/commons/pom.xml                            |  46 +++
 .../ignite/internal/schema/AssemblyException.java  |  32 ++
 .../org/apache/ignite/internal/schema/Bitmask.java |  87 +++++
 .../ignite/internal/schema/ByteBufferTuple.java    |  91 +++++
 .../org/apache/ignite/internal/schema/Column.java  | 111 ++++++
 .../org/apache/ignite/internal/schema/Columns.java | 272 +++++++++++++
 .../ignite/internal/schema/ExpandableByteBuf.java  | 253 +++++++++++++
 .../internal/schema/InvalidTypeException.java      |  30 ++
 .../apache/ignite/internal/schema/NativeType.java  | 136 +++++++
 .../ignite/internal/schema/NativeTypeSpec.java     | 178 +++++++++
 .../ignite/internal/schema/SchemaDescriptor.java   |  99 +++++
 .../org/apache/ignite/internal/schema/Tuple.java   | 420 +++++++++++++++++++++
 .../ignite/internal/schema/TupleAssembler.java     | 406 ++++++++++++++++++++
 .../ignite/internal/schema/package-info.java       |  68 ++++
 .../apache/ignite/internal/schema/ColumnTest.java  |  48 +++
 .../apache/ignite/internal/schema/ColumnsTest.java | 399 ++++++++++++++++++++
 .../internal/schema/ExpandableByteBufTest.java     | 153 ++++++++
 .../ignite/internal/schema/NativeTypeTest.java     |  71 ++++
 .../apache/ignite/internal/schema/TupleTest.java   | 382 +++++++++++++++++++
 pom.xml                                            | 117 +++++-
 20 files changed, 3396 insertions(+), 3 deletions(-)

diff --git a/modules/commons/pom.xml b/modules/commons/pom.xml
new file mode 100644
index 0000000..0599be4
--- /dev/null
+++ b/modules/commons/pom.xml
@@ -0,0 +1,46 @@
+<?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.
+-->
+
+<!--
+    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">
+    <modelVersion>4.0.0</modelVersion>
+
+    <parent>
+        <artifactId>apache-ignite</artifactId>
+        <groupId>org.apache.ignite</groupId>
+        <version>3.0.0-SNAPSHOT</version>
+        <relativePath>../..</relativePath>
+    </parent>
+
+    <artifactId>ignite-commons</artifactId>
+    <version>3.0.0-SNAPSHOT</version>
+    <url>http://ignite.apache.org</url>
+
+    <dependencies>
+        <dependency>
+            <groupId>org.jetbrains</groupId>
+            <artifactId>annotations</artifactId>
+            <version>${jetbrains.annotations.version}</version>
+        </dependency>
+    </dependencies>
+</project>
diff --git a/modules/commons/src/main/java/org/apache/ignite/internal/schema/AssemblyException.java b/modules/commons/src/main/java/org/apache/ignite/internal/schema/AssemblyException.java
new file mode 100644
index 0000000..8d01c91
--- /dev/null
+++ b/modules/commons/src/main/java/org/apache/ignite/internal/schema/AssemblyException.java
@@ -0,0 +1,32 @@
+/*
+ * 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;
+
+/**
+ * The exception is thrown when the tuple 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 {
+    /**
+     * @param errMsg Error message
+     * @param cause Cause for this error.
+     */
+    public AssemblyException(String errMsg, Exception cause) {
+        super(errMsg, cause);
+    }
+}
diff --git a/modules/commons/src/main/java/org/apache/ignite/internal/schema/Bitmask.java b/modules/commons/src/main/java/org/apache/ignite/internal/schema/Bitmask.java
new file mode 100644
index 0000000..1314d32
--- /dev/null
+++ b/modules/commons/src/main/java/org/apache/ignite/internal/schema/Bitmask.java
@@ -0,0 +1,87 @@
+/*
+ * 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;
+
+/**
+ * A fixed-sized type representing a bitmask of <code>n</code> bits. The actual size of a bitmask will round up
+ * to the smallest number of bytes required to store <code>n</code> bits.
+ */
+public class Bitmask extends NativeType {
+    /** */
+    private final int bits;
+
+    /**
+     * Factory method for creating the bitmask type.
+     *
+     * @param nBits Maximum number of bits in the bitmask.
+     * @return Bitmask type.
+     */
+    public static Bitmask of(int nBits) {
+        return new Bitmask(nBits);
+    }
+
+    /**
+     * Creates a bitmask type of size <code>bits</code>. In tuple will round up to the closest full byte.
+     *
+     * @param bits The number of bits in the bitmask.
+     */
+    protected Bitmask(int bits) {
+        super(NativeTypeSpec.BITMASK, (bits + 7) / 8);
+
+        this.bits = bits;
+    }
+
+    /**
+     * @return Maximum number of bits to be stored in the bitmask.
+     */
+    public int bits() {
+        return bits;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean equals(Object o) {
+        if (this == o)
+            return true;
+
+        if (o == null || getClass() != o.getClass())
+            return false;
+
+        Bitmask that = (Bitmask)o;
+
+        return bits == that.bits;
+    }
+
+    /** {@inheritDoc} */
+    @Override public int hashCode() {
+        return bits;
+    }
+
+    /** {@inheritDoc} */
+    @Override public int compareTo(NativeType o) {
+        int res = super.compareTo(o);
+
+        if (res == 0) {
+            // The passed in object is also a bitmask, compare the number of bits.
+            Bitmask that = (Bitmask)o;
+
+            return Integer.compare(bits, that.bits);
+        }
+        else
+            return res;
+    }
+}
diff --git a/modules/commons/src/main/java/org/apache/ignite/internal/schema/ByteBufferTuple.java b/modules/commons/src/main/java/org/apache/ignite/internal/schema/ByteBufferTuple.java
new file mode 100644
index 0000000..18adcd1
--- /dev/null
+++ b/modules/commons/src/main/java/org/apache/ignite/internal/schema/ByteBufferTuple.java
@@ -0,0 +1,91 @@
+/*
+ * 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;
+
+import java.nio.ByteBuffer;
+import java.nio.ByteOrder;
+import java.nio.charset.StandardCharsets;
+
+/**
+ * Heap byte buffer-based tuple.
+ */
+public class ByteBufferTuple extends Tuple {
+    /** */
+    private final ByteBuffer buf;
+
+    /**
+     * @param arr Array representation of the tuple.
+     */
+    public ByteBufferTuple(SchemaDescriptor sch, byte[] arr) {
+        super(sch);
+
+        buf = ByteBuffer.wrap(arr);
+        buf.order(ByteOrder.LITTLE_ENDIAN);
+    }
+
+    /** {@inheritDoc} */
+    @Override protected int readByte(int off) {
+        return buf.get(off) & 0xFF;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected int readShort(int off) {
+        return buf.getShort(off) & 0xFFFF;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected int readInteger(int off) {
+        return buf.getInt(off);
+    }
+
+    /** {@inheritDoc} */
+    @Override protected long readLong(int off) {
+        return buf.getLong(off);
+    }
+
+    /** {@inheritDoc} */
+    @Override protected float readFloat(int off) {
+        return buf.getFloat(off);
+    }
+
+    /** {@inheritDoc} */
+    @Override protected double readDouble(int off) {
+        return buf.getDouble(off);
+    }
+
+    /** {@inheritDoc} */
+    @Override protected byte[] readBytes(int off, int len) {
+        try {
+            byte[] res = new byte[len];
+
+            buf.position(off);
+
+            buf.get(res, 0, res.length);
+
+            return res;
+        }
+        finally {
+            buf.position(0);
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override protected String readString(int off, int len) {
+        return new String(buf.array(), off, len, StandardCharsets.UTF_8);
+    }
+}
diff --git a/modules/commons/src/main/java/org/apache/ignite/internal/schema/Column.java b/modules/commons/src/main/java/org/apache/ignite/internal/schema/Column.java
new file mode 100644
index 0000000..b93c4d3
--- /dev/null
+++ b/modules/commons/src/main/java/org/apache/ignite/internal/schema/Column.java
@@ -0,0 +1,111 @@
+/*
+ * 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;
+
+/**
+ * Column description for a type schema. Column contains a column name, a column type and a nullability flag.
+ * <p>
+ * Column instances are comparable in lexicographic order, native type first and then column name. Nullability
+ * flag is not taken into account when columns are compared.
+ */
+public class Column implements Comparable<Column> {
+    /**
+     * Column name.
+     */
+    private final String name;
+
+    /**
+     * An instance of column data type.
+     */
+    private final NativeType type;
+
+    /**
+     * If {@code false}, null values will not be allowed for this column.
+     */
+    private final boolean nullable;
+
+    /**
+     * @param name Column name.
+     * @param type An instance of column data type.
+     * @param nullable If {@code false}, null values will not be allowed for this column.
+     */
+    public Column(
+        String name,
+        NativeType type,
+        boolean nullable
+    ) {
+        this.name = name;
+        this.type = type;
+        this.nullable = nullable;
+    }
+
+    /**
+     * @return Column name.
+     */
+    public String name() {
+        return name;
+    }
+
+    /**
+     * @return An instance of column data type.
+     */
+    public NativeType type() {
+        return type;
+    }
+
+    /**
+     * @return {@code false} if null values will not be allowed for this column.
+     */
+    public boolean nullable() {
+        return nullable;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean equals(Object o) {
+        if (this == o)
+            return true;
+
+        if (o == null || getClass() != o.getClass())
+            return false;
+
+        Column col = (Column)o;
+
+        return name.equals(col.name) &&
+            type.equals(col.type);
+    }
+
+    /** {@inheritDoc} */
+    @Override public int hashCode() {
+        return name.hashCode() + 31 * type.hashCode();
+    }
+
+    /** {@inheritDoc} */
+    @Override public int compareTo(Column o) {
+        int cmp = type.compareTo(o.type);
+
+        if (cmp != 0)
+            return cmp;
+
+        return name.compareTo(o.name);
+    }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return "Column [name=" + name + ", type=" + type + ", nullable=" + nullable + ']';
+    }
+}
diff --git a/modules/commons/src/main/java/org/apache/ignite/internal/schema/Columns.java b/modules/commons/src/main/java/org/apache/ignite/internal/schema/Columns.java
new file mode 100644
index 0000000..eed1918
--- /dev/null
+++ b/modules/commons/src/main/java/org/apache/ignite/internal/schema/Columns.java
@@ -0,0 +1,272 @@
+/*
+ * 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;
+
+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.
+ */
+public class Columns {
+    /** */
+    public static final int[][] EMPTY_FOLDING_TABLE = new int[0][];
+
+    /** */
+    public static final int[] EMPTY_FOLDING_MASK = new int[0];
+
+    /**
+     * Lookup table to speed-up calculation of the number of null/non-null columns based on the null map.
+     * For a given byte {@code b}, {@code NULL_COLUMNS_LOOKUP[b]} will contain the number of {@code null} columns
+     * corresponding to the byte in nullability map.
+     * For example, if nullability map is {@code 0b00100001}, then the map encodes nulls for columns 0 and 5 and
+     * {@code NULL_COLUMNS_LOOKUP[0b00100001] == 2}.
+     */
+    private static final int[] NULL_COLUMNS_LOOKUP;
+
+    /**
+     * Columns in packed order for this chunk.
+     */
+    private final Column[] cols;
+
+    /**
+     * If the type contains varlength columns, this field will contain an index of the first such column.
+     * Otherwise, it will contain {@code -1}.
+     */
+    private final int firstVarlenColIdx;
+
+    /**
+     * Number of bytes required to store the nullability map for this chunk.
+     */
+    private final int nullMapSize;
+
+    /**
+     * Fixed-size column length folding table. The table is used to quickly calculate the offset of a fixed-lengh
+     * column based on the nullability map.
+     */
+    private int[][] foldingTbl;
+
+    /**
+     * Additional mask values for folding table to cut off nullability map for columns with larger indexes.
+     */
+    private int[] foldingMask;
+
+    static {
+        NULL_COLUMNS_LOOKUP = new int[256];
+
+        // Each nonzero bit is a null value.
+        for (int i = 0; i < 255; i++)
+            NULL_COLUMNS_LOOKUP[i] = Integer.bitCount(i);
+    }
+
+    /**
+     * Gets a number of null columns for the given byte from the nullability map (essentially, the number of non-zero
+     * bits in the given byte).
+     *
+     * @param nullmapByte Byte from a nullability map.
+     * @return Number of null columns for the given byte.
+     */
+    public static int numberOfNullColumns(int nullmapByte) {
+        return NULL_COLUMNS_LOOKUP[nullmapByte];
+    }
+
+    /**
+     * Constructs the columns chunk. The columns will be internally sorted in write-effecient order based on
+     * {@link Column} comparison.
+     *
+     * @param cols Array of columns.
+     */
+    public Columns(Column... cols) {
+        this.cols = sortedCopy(cols);
+
+        firstVarlenColIdx = findFirstVarlenColumn();
+
+        nullMapSize = (cols.length + 7) / 8;
+
+        buildFoldingTable();
+    }
+
+    /**
+     * Calculates a sum of fixed-sized columns lengths given the mask of the present columns, assuming that the
+     * {@code maskByte} is an {@code i}-th byte is columns mask.
+     *
+     * @param i Mask byte index in the nullability map.
+     * @param maskByte Mask byte value, where a nonzero bit (counting from LSB to MSB) represents a {@code null} value
+     *      and the corresponding column length should be skipped.
+     * @return Fixed columns length sizes summed wrt to the mask.
+     */
+    public int foldFixedLength(int i, int maskByte) {
+        return foldingTbl[i][maskByte & foldingMask[i]];
+    }
+
+    /**
+     * @return Number of bytes required to store the nullability map for these columns.
+     */
+    public int nullMapSize() {
+        return nullMapSize;
+    }
+
+    /**
+     * @param idx Column index to check.
+     * @return {@code true} if the column with the given index is fixed-size.
+     */
+    public boolean isFixedSize(int idx) {
+        return cols[idx].type().spec().fixedLength();
+    }
+
+    /**
+     * @param idx Column index.
+     * @return Column instance.
+     */
+    public Column column(int idx) {
+        return cols[idx];
+    }
+
+    /**
+     * @return Number of columns in this chunk.
+     */
+    public int length() {
+        return cols.length;
+    }
+
+    /**
+     * @return The number of varlength columns in this chunk.
+     */
+    public int numberOfVarlengthColumns() {
+        return cols.length - numberOfFixsizeColumns();
+    }
+
+    /**
+     * @return The number of fixsize columns in this chunk.
+     */
+    public int numberOfFixsizeColumns() {
+        return firstVarlenColIdx == -1 ? cols.length : firstVarlenColIdx;
+    }
+
+    /**
+     * @return The index of the first varlength column in the sorted order of columns.
+     */
+    public int firstVarlengthColumn() {
+        return firstVarlenColIdx;
+    }
+
+    /**
+     * @param cols User columns.
+     * @return A copy of user columns array sorted in column order.
+     */
+    private Column[] sortedCopy(Column[] cols) {
+        Column[] cp = Arrays.copyOf(cols, cols.length);
+
+        Arrays.sort(cp);
+
+        return cp;
+    }
+
+    /**
+     * @return Index of the first varlength column or {@code -1} if there are none.
+     */
+    private int findFirstVarlenColumn() {
+        for (int i = 0; i < cols.length; i++) {
+            if (!cols[i].type().spec().fixedLength())
+                return i;
+        }
+
+        return -1;
+    }
+
+    /**
+     *
+     */
+    private void buildFoldingTable() {
+        int numFixsize = numberOfFixsizeColumns();
+
+        if (numFixsize == 0) {
+            foldingTbl = EMPTY_FOLDING_TABLE;
+            foldingMask = EMPTY_FOLDING_MASK;
+
+            return;
+        }
+
+        int fixsizeNullMapSize = (numFixsize + 7) / 8;
+
+        int[][] res = new int[fixsizeNullMapSize][];
+        int[] resMask = new int[fixsizeNullMapSize];
+
+        for (int b = 0; b < fixsizeNullMapSize; b++) {
+            int bitsInMask = b == fixsizeNullMapSize - 1 ?
+                (numFixsize - 8 * b) : 8;
+
+            int totalMasks = 1 << bitsInMask;
+
+            resMask[b] = 0xFF >>> (8 - bitsInMask);
+
+            res[b] = new int[totalMasks];
+
+            // Start with all non-nulls.
+            int mask = 0x00;
+
+            for (int i = 0; i < totalMasks; i++) {
+                res[b][mask] = foldManual(b, mask);
+
+                mask++;
+            }
+        }
+
+        foldingTbl = res;
+        foldingMask = resMask;
+    }
+
+    /**
+     * Manually fold the sizes of the fixed-size columns based on the nullability map byte.
+     *
+     * @param b Nullability map byte index.
+     * @param mask Nullability mask from the map.
+     * @return Sum of column sizes based nullability mask.
+     */
+    private int foldManual(int b, int mask) {
+        int size = 0;
+
+        for (int bit = 0; bit < 8; bit++) {
+            boolean hasVal = (mask & (1 << bit)) == 0;
+
+            int idx = b * 8 + bit;
+
+            if (hasVal && idx < numberOfFixsizeColumns()) {
+                assert cols[idx].type().spec().fixedLength() : "Expected fixed-size column [b=" + b +
+                    ", mask=" + mask +
+                    ", cols" + Arrays.toString(cols) + ']';
+
+                size += cols[idx].type().length();
+            }
+        }
+
+        return size;
+    }
+
+    /**
+     */
+    public int columnIndex(String fieldName) {
+        for (int i = 0; i < cols.length; i++) {
+            if (cols[i].name().equalsIgnoreCase(fieldName))
+                return i;
+        }
+
+        throw new NoSuchElementException("No field '" + fieldName + "' defined");
+    }
+}
diff --git a/modules/commons/src/main/java/org/apache/ignite/internal/schema/ExpandableByteBuf.java b/modules/commons/src/main/java/org/apache/ignite/internal/schema/ExpandableByteBuf.java
new file mode 100644
index 0000000..542b539
--- /dev/null
+++ b/modules/commons/src/main/java/org/apache/ignite/internal/schema/ExpandableByteBuf.java
@@ -0,0 +1,253 @@
+/*
+ * 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;
+
+import java.nio.ByteBuffer;
+import java.nio.ByteOrder;
+import java.nio.CharBuffer;
+import java.nio.charset.CharacterCodingException;
+import java.nio.charset.CharsetEncoder;
+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
+ * format.
+ * <p>
+ * Additionally, it tracks the high watermark of the values ever written to the buffer so that only written bytes are
+ * returned from the {@link #toArray()} method. If the current (expanded) buffer size does not match the high watermark,
+ * the {@link #toArray()} method will return a smaller copy of the array to exactly match the watermark.
+ * <p>
+ * All write methods have an absolute position. The buffer will automatically expand to fit the value being written. If
+ * there is a gap between previously written values and the current value, it will be filled with zero bytes:
+ * <pre>
+ * ExpandableByteBuf b = new ExpandableByteBuf(1);
+ * b.put(0, (byte)1); // Does not expand.
+ * b.put(5, (byte)1); // Expands, meaningful bytes are [0..5]
+ *
+ * byte[] data = b.toArray(); // data.length == 6
+ * </pre>
+ */
+@SuppressWarnings("AssignmentOrReturnOfFieldWithMutableType")
+public class ExpandableByteBuf {
+    /** */
+    private static final int MB = 1024 * 1024;
+
+    /** */
+    private byte[] arr;
+
+    /** */
+    private ByteBuffer buf;
+
+    /** */
+    private int len;
+
+    /**
+     * @param size Start buffer size.
+     */
+    public ExpandableByteBuf(int size) {
+        if (size <= 0)
+            size = 16;
+
+        arr = new byte[size];
+        buf = ByteBuffer.wrap(arr);
+        buf.order(ByteOrder.LITTLE_ENDIAN);
+    }
+
+    /**
+     *
+     */
+    public void put(int off, byte val) {
+        ensureCapacity(off + 1);
+
+        buf.put(off, val);
+    }
+
+    /**
+     *
+     */
+    public void putShort(int off, short val) {
+        ensureCapacity(off + 2);
+
+        buf.putShort(off, val);
+    }
+
+    /**
+     *
+     */
+    public void putInt(int off, int val) {
+        ensureCapacity(off + 4);
+
+        buf.putInt(off, val);
+    }
+
+    /**
+     *
+     */
+    public void putFloat(int off, float val) {
+        ensureCapacity(off + 4);
+
+        buf.putFloat(off, val);
+    }
+
+    /**
+     *
+     */
+    public void putLong(int off, long val) {
+        ensureCapacity(off + 8);
+
+        buf.putLong(off, val);
+    }
+
+    /**
+     *
+     */
+    public void putDouble(int off, double val) {
+        ensureCapacity(off + 8);
+
+        buf.putDouble(off, val);
+    }
+
+    /**
+     *
+     */
+    public void putBytes(int curOff, byte[] val) {
+        ensureCapacity(curOff + val.length);
+
+        buf.position(curOff);
+
+        try {
+            buf.put(val);
+        }
+        finally {
+            buf.position(0);
+        }
+    }
+
+    /**
+     *
+     */
+    public int putString(int off, String val, CharsetEncoder encoder) throws CharacterCodingException {
+        ensureCapacity(off);
+
+        encoder.reset();
+
+        buf.position(off);
+
+        try {
+            CharBuffer valBuf = CharBuffer.wrap(val);
+
+            while (true) {
+                CoderResult cr = encoder.encode(valBuf, buf, true);
+
+                len = buf.position();
+
+                if (cr.isUnderflow())
+                    break;
+
+                if (cr.isOverflow()) {
+                    expand(len + 1);
+
+                    continue;
+                }
+
+                if (cr.isError())
+                    cr.throwException();
+
+            }
+
+            while (true) {
+                CoderResult cr = encoder.flush(buf);
+
+                len = buf.position();
+
+                if (cr.isOverflow()) {
+                    expand(len + 1);
+
+                    continue;
+                }
+
+                if (cr.isUnderflow())
+                    break;
+
+                if (cr.isError())
+                    cr.throwException();
+            }
+
+            return len - off;
+        }
+        finally {
+            buf.position(0);
+        }
+    }
+
+    /**
+     *
+     */
+    public byte get(int off) {
+        return buf.get(off);
+    }
+
+    /**
+     * @return The byte array of all bytes written to this array, including gaps.
+     */
+    public byte[] toArray() {
+        if (arr.length == len)
+            return arr;
+        else
+            return Arrays.copyOf(arr, len);
+    }
+
+    /**
+     * If the current capacity is smaller than {@code cap}, will expand the buffer size.
+     *
+     * @param cap Target capacity.
+     */
+    private void ensureCapacity(int cap) {
+        if (arr.length < cap)
+            expand(cap);
+
+        if (cap > len)
+            len = cap;
+    }
+
+    /**
+     * @param cap Capacity to expand.
+     */
+    private void expand(int cap) {
+        int l = arr.length;
+
+        while (l < cap) {
+            if (l < MB)
+                l *= 2;
+            else
+                l += MB;
+        }
+
+        byte[] tmp = new byte[cap];
+
+        System.arraycopy(arr, 0, tmp, 0, arr.length);
+
+        arr = tmp;
+        int oldPos = buf.position();
+        buf = ByteBuffer.wrap(arr);
+        buf.position(oldPos);
+        buf.order(ByteOrder.LITTLE_ENDIAN);
+    }
+}
diff --git a/modules/commons/src/main/java/org/apache/ignite/internal/schema/InvalidTypeException.java b/modules/commons/src/main/java/org/apache/ignite/internal/schema/InvalidTypeException.java
new file mode 100644
index 0000000..d03ca73
--- /dev/null
+++ b/modules/commons/src/main/java/org/apache/ignite/internal/schema/InvalidTypeException.java
@@ -0,0 +1,30 @@
+/*
+ * 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;
+
+/**
+ * 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);
+    }
+}
diff --git a/modules/commons/src/main/java/org/apache/ignite/internal/schema/NativeType.java b/modules/commons/src/main/java/org/apache/ignite/internal/schema/NativeType.java
new file mode 100644
index 0000000..211a1e1
--- /dev/null
+++ b/modules/commons/src/main/java/org/apache/ignite/internal/schema/NativeType.java
@@ -0,0 +1,136 @@
+/*
+ * 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;
+
+/**
+ * A thin wrapper over {@link NativeTypeSpec} to instantiate parameterized constrained types.
+ */
+public class NativeType implements Comparable<NativeType> {
+    /** */
+    public static final NativeType BYTE = new NativeType(NativeTypeSpec.BYTE, 1);
+
+    /** */
+    public static final NativeType SHORT = new NativeType(NativeTypeSpec.SHORT, 2);
+
+    /** */
+    public static final NativeType INTEGER = new NativeType(NativeTypeSpec.INTEGER, 4);
+
+    /** */
+    public static final NativeType LONG = new NativeType(NativeTypeSpec.LONG, 8);
+
+    /** */
+    public static final NativeType FLOAT = new NativeType(NativeTypeSpec.FLOAT, 4);
+
+    /** */
+    public static final NativeType DOUBLE = new NativeType(NativeTypeSpec.DOUBLE, 8);
+
+    /** */
+    public static final NativeType UUID = new NativeType(NativeTypeSpec.UUID, 16);
+
+    /** */
+    public static final NativeType STRING = new NativeType(NativeTypeSpec.STRING);
+
+    /** */
+    public static final NativeType BYTES = new NativeType(NativeTypeSpec.BYTES);
+
+    /** */
+    private final NativeTypeSpec typeSpec;
+
+    /** Type length. */
+    private int len;
+
+    /**
+     */
+    protected NativeType(NativeTypeSpec typeSpec, int len) {
+        if (!typeSpec.fixedLength())
+            throw new IllegalArgumentException("Size must be provided only for fixed-length types: " + typeSpec);
+
+        if (len <= 0)
+            throw new IllegalArgumentException("Size must be positive [typeSpec=" + typeSpec + ", size=" + len + ']');
+
+        this.typeSpec = typeSpec;
+        this.len = len;
+    }
+
+    /**
+     */
+    protected NativeType(NativeTypeSpec typeSpec) {
+        if (typeSpec.fixedLength())
+            throw new IllegalArgumentException("Fixed-length types must be created by the " +
+                "length-aware constructor: " + typeSpec);
+
+        this.typeSpec = typeSpec;
+    }
+
+    /**
+     * @return Length of the type if it is a fixlen type. For varlen types the return value is undefined, so the user
+     * should explicitly check {@code spec().fixedLength()} before using this method.
+     *
+     * @see NativeTypeSpec#fixedLength()
+     */
+    public int length() {
+        return len;
+    }
+
+    /**
+     * @return Type specification enum.
+     */
+    public NativeTypeSpec spec() {
+        return typeSpec;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean equals(Object o) {
+        if (this == o)
+            return true;
+
+        if (o == null || getClass() != o.getClass())
+            return false;
+
+        NativeType that = (NativeType)o;
+
+        return len == that.len && typeSpec == that.typeSpec;
+    }
+
+    /** {@inheritDoc} */
+    @Override public int hashCode() {
+        int res = typeSpec.hashCode();
+
+        res = 31 * res + len;
+
+        return res;
+    }
+
+    /** {@inheritDoc} */
+    @Override public int compareTo(NativeType o) {
+        // Fixed-sized types go first.
+        if (len <= 0 && o.len > 0)
+            return 1;
+
+        if (len > 0 && o.len <= 0)
+            return -1;
+
+        // Either size is -1 for both, or positive for both. Compare sizes, then description.
+        int cmp = Integer.compare(len, o.len);
+
+        if (cmp != 0)
+            return cmp;
+
+        return typeSpec.name().compareTo(o.typeSpec.name());
+    }
+}
diff --git a/modules/commons/src/main/java/org/apache/ignite/internal/schema/NativeTypeSpec.java b/modules/commons/src/main/java/org/apache/ignite/internal/schema/NativeTypeSpec.java
new file mode 100644
index 0000000..49bbaa7
--- /dev/null
+++ b/modules/commons/src/main/java/org/apache/ignite/internal/schema/NativeTypeSpec.java
@@ -0,0 +1,178 @@
+/*
+ * 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;
+
+/**
+ * Base class for storage built-in data types definition. The class contains predefined values
+ * for fixed-sized types and some of the variable-sized types. Parameterized types, such as
+ * 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.
+ */
+public enum NativeTypeSpec {
+    /**
+     * Native type representing a single-byte signed value.
+     */
+    BYTE("byte", true) {
+        /** {@inheritDoc} */
+        @Override public Object objectValue(Tuple tup, int colIdx) {
+            return tup.byteValueBoxed(colIdx);
+        }
+    },
+
+    /**
+     * Native type representing a two-bytes signed value.
+     */
+    SHORT("short", true) {
+        /** {@inheritDoc} */
+        @Override public Object objectValue(Tuple tup, int colIdx) {
+            return tup.shortValueBoxed(colIdx);
+        }
+    },
+
+    /**
+     * Native type representing a four-bytes signed value.
+     */
+    INTEGER ("integer", true) {
+        /** {@inheritDoc} */
+        @Override public Object objectValue(Tuple tup, int colIdx) {
+            return tup.intValueBoxed(colIdx);
+        }
+    },
+
+    /**
+     * Native type representing an eight-bytes signed value.
+     */
+    LONG("long", true) {
+        /** {@inheritDoc} */
+        @Override public Object objectValue(Tuple tup, int colIdx) {
+            return tup.longValueBoxed(colIdx);
+        }
+    },
+
+    /**
+     * Native type representing a four-bytes floating-point value.
+     */
+    FLOAT("float", true) {
+        /** {@inheritDoc} */
+        @Override public Object objectValue(Tuple tup, int colIdx) {
+            return tup.floatValueBoxed(colIdx);
+        }
+    },
+
+    /**
+     * Native type representing an eight-bytes floating-point value.
+     */
+    DOUBLE("double", true) {
+        /** {@inheritDoc} */
+        @Override public Object objectValue(Tuple tup, int colIdx) {
+            return tup.doubleValueBoxed(colIdx);
+        }
+    },
+
+    /**
+     * Native type representing a UUID.
+     */
+    UUID("uuid", true) {
+        /** {@inheritDoc} */
+        @Override public Object objectValue(Tuple tup, int colIdx) {
+            return tup.uuidValue(colIdx);
+        }
+    },
+
+    /**
+     * Native type respresenting a string.
+     */
+    STRING("string") {
+        /** {@inheritDoc} */
+        @Override public Object objectValue(Tuple tup, int colIdx) {
+            return tup.stringValue(colIdx);
+        }
+    },
+
+    /**
+     * Native type representing an arbitrary byte array.
+     */
+    BYTES("blob") {
+        /** {@inheritDoc} */
+        @Override public Object objectValue(Tuple tup, int colIdx) {
+            return tup.bytesValue(colIdx);
+        }
+    },
+
+    /**
+     * Native type representing a bitmask.
+     */
+    BITMASK("bitmask", true) {
+        /** {@inheritDoc} */
+        @Override public Object objectValue(Tuple tup, int colIdx) {
+            return tup.bitmaskValue(colIdx);
+        }
+    };
+
+    /** Flag indicating whether this type specifies a fixed-length type. */
+    private final boolean fixedSize;
+
+    /** Single-token type description. */
+    private final String desc;
+
+    /**
+     * Constructs a varlength type with the given type description.
+     *
+     * @param desc Type description.
+     */
+    NativeTypeSpec(String desc) {
+        this(desc, false);
+    }
+
+    /**
+     * Constructs a type with the given description and size.
+     *
+     * @param desc Type description.
+     * @param fixedSize Flag indicating whether this type specifies a fixed-length type.
+     */
+    NativeTypeSpec(String desc, boolean fixedSize) {
+        this.desc = desc;
+        this.fixedSize = fixedSize;
+    }
+
+    /**
+     * @return {@code true} for fixed-length types, {@code false} otherwise.
+     */
+    public boolean fixedLength() {
+        return fixedSize;
+    }
+
+    /**
+     * Indirection method for getting an Object representation of the given type from the tuple. This method
+     * does do any type conversions and will throw an exception if tuple schema column type differs from this
+     * type.
+     *
+     * @param tup Tuple 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;
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return "NativeType [desc=" + desc + ", size=" + (fixedLength() ? fixedSize : "varlen") + ']';
+    }
+}
diff --git a/modules/commons/src/main/java/org/apache/ignite/internal/schema/SchemaDescriptor.java b/modules/commons/src/main/java/org/apache/ignite/internal/schema/SchemaDescriptor.java
new file mode 100644
index 0000000..247826e
--- /dev/null
+++ b/modules/commons/src/main/java/org/apache/ignite/internal/schema/SchemaDescriptor.java
@@ -0,0 +1,99 @@
+/*
+ * 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;
+
+/**
+ * Full schema descriptor containing key columns chunk, value columns chunk, and schema version.
+ */
+public class SchemaDescriptor {
+    /** Schema version. Incremented on each schema modification. */
+    private final int ver;
+
+    /**
+     * Key columns in serialization order.
+     */
+    private final Columns keyCols;
+
+    /**
+     * Value columns in serialization order.
+     */
+    private final Columns valCols;
+
+    /**
+     * @param ver Schema version.
+     * @param keyCols Key columns.
+     * @param valCols Value columns.
+     */
+    public SchemaDescriptor(int ver, Columns keyCols, Columns valCols) {
+        this.ver = ver;
+        this.keyCols = keyCols;
+        this.valCols = valCols;
+    }
+
+    /**
+     * @return Schema version.
+     */
+    public int version() {
+        return ver;
+    }
+
+    /**
+     * @param idx Index to check.
+     * @return {@code true} if the column belongs to the key chunk.
+     */
+    public boolean keyColumn(int idx) {
+        return idx < keyCols.length();
+    }
+
+    /**
+     * @param col Column index.
+     * @return Column chunk for the given column index.
+     */
+    public Columns columns(int col) {
+        return keyColumn(col) ? keyCols : valCols;
+    }
+
+    /**
+     * @param colIdx Column index.
+     * @return Column instance.
+     */
+    public Column column(int colIdx) {
+        return colIdx < keyCols.length() ? keyCols.column(colIdx) : valCols.column(colIdx - keyCols.length());
+    }
+
+    /**
+     * @return Key columns chunk.
+     */
+    public Columns keyColumns() {
+        return keyCols;
+    }
+
+    /**
+     * @return Value columns chunk.
+     */
+    public Columns valueColumns() {
+        return valCols;
+    }
+
+    /**
+     * @return Total number of columns in schema.
+     */
+    public int length() {
+        return keyCols.length() + valCols.length();
+    }
+}
diff --git a/modules/commons/src/main/java/org/apache/ignite/internal/schema/Tuple.java b/modules/commons/src/main/java/org/apache/ignite/internal/schema/Tuple.java
new file mode 100644
index 0000000..8e22305
--- /dev/null
+++ b/modules/commons/src/main/java/org/apache/ignite/internal/schema/Tuple.java
@@ -0,0 +1,420 @@
+/*
+ * 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;
+
+import java.util.BitSet;
+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 coersions should be implemented outside of the tuple 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 static final int SCHEMA_VERSION_FIELD_SIZE = 2;
+
+    /** */
+    public static final int KEY_HASH_FIELD_SIZE = 4;
+
+    /** */
+    public static final int TOTAL_LEN_FIELD_SIZE = 2;
+
+    /** */
+    public static final int VARSIZE_TABLE_LEN_FIELD_SIZE = 2;
+
+    /** Schema descriptor for which this tuple was created. */
+    private final SchemaDescriptor schema;
+
+    /**
+     * @param schema Schema instance.
+     */
+    protected Tuple(SchemaDescriptor schema) {
+        this.schema = schema;
+    }
+
+    /**
+     */
+    public byte byteValue(int col) {
+        long off = findColumn(col, NativeTypeSpec.BYTE);
+
+        return off < 0 ? 0 : (byte)readByte(offset(off));
+    }
+
+    /**
+     */
+    public Byte byteValueBoxed(int col) {
+        long off = findColumn(col, NativeTypeSpec.BYTE);
+
+        return off < 0 ? null : (byte)readByte(offset(off));
+    }
+
+    /**
+     */
+    public short shortValue(int col) {
+        long off = findColumn(col, NativeTypeSpec.SHORT);
+
+        return off < 0 ? 0 : (short)readShort(offset(off));
+    }
+
+    /**
+     */
+    public Short shortValueBoxed(int col) {
+        long off = findColumn(col, NativeTypeSpec.SHORT);
+
+        return off < 0 ? null : (short)readShort(offset(off));
+    }
+
+    /**
+     */
+    public int intValue(int col) {
+        long off = findColumn(col, NativeTypeSpec.INTEGER);
+
+        return off < 0 ? 0 : readInteger(offset(off));
+    }
+
+    /**
+     */
+    public Integer intValueBoxed(int col) {
+        long off = findColumn(col, NativeTypeSpec.INTEGER);
+
+        return off < 0 ? null : readInteger(offset(off));
+    }
+
+    /**
+     */
+    public long longValue(int col) {
+        long off = findColumn(col, NativeTypeSpec.LONG);
+
+        return off < 0 ? 0 : readLong(offset(off));
+    }
+
+    /**
+     */
+    public Long longValueBoxed(int col) {
+        long off = findColumn(col, NativeTypeSpec.LONG);
+
+        return off < 0 ? null : readLong(offset(off));
+    }
+
+    /**
+     */
+    public float floatValue(int col) {
+        long off = findColumn(col, NativeTypeSpec.FLOAT);
+
+        return off < 0 ? 0.f : readFloat(offset(off));
+    }
+
+    /**
+     */
+    public Float floatValueBoxed(int col) {
+        long off = findColumn(col, NativeTypeSpec.FLOAT);
+
+        return off < 0 ? null : readFloat(offset(off));
+    }
+
+    /**
+     */
+    public double doubleValue(int col) {
+        long off = findColumn(col, NativeTypeSpec.DOUBLE);
+
+        return off < 0 ? 0.d : readDouble(offset(off));
+    }
+
+    /**
+     */
+    public Double doubleValueBoxed(int col) {
+        long off = findColumn(col, NativeTypeSpec.DOUBLE);
+
+        return off < 0 ? null : readDouble(offset(off));
+    }
+
+    /**
+     */
+    public String stringValue(int col) {
+        long offLen = findColumn(col, NativeTypeSpec.STRING);
+
+        if (offLen < 0)
+            return null;
+
+        int off = offset(offLen);
+        int len = length(offLen);
+
+        return readString(off, len);
+    }
+
+    /**
+     */
+    public byte[] bytesValue(int col) {
+        long offLen = findColumn(col, NativeTypeSpec.BYTES);
+
+        if (offLen < 0)
+            return null;
+
+        int off = offset(offLen);
+        int len = length(offLen);
+
+        return readBytes(off, len);
+    }
+
+    /**
+     */
+    public UUID uuidValue(int col) {
+        long found = findColumn(col, NativeTypeSpec.UUID);
+
+        if (found < 0)
+            return null;
+
+        int off = offset(found);
+
+        long lsb = readLong(off);
+        long msb = readLong(off + 8);
+
+        return new UUID(msb, lsb);
+    }
+
+    /**
+     */
+    public BitSet bitmaskValue(int colIdx) {
+        long offLen = findColumn(colIdx, NativeTypeSpec.BITMASK);
+
+        if (offLen < 0)
+            return null;
+
+        int off = offset(offLen);
+
+        Column col = schema.column(colIdx);
+
+        return BitSet.valueOf(readBytes(off, col.type().length()));
+    }
+
+    /**
+     * 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
+     * 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
+     * {@link InvalidTypeException} if the types do not match.
+     *
+     * @param colIdx Column index.
+     * @param type Expected column type.
+     * @return Encoded offset + length of the column.
+     * @see #offset(long)
+     * @see #length(long)
+     * @see InvalidTypeException If actual column type does not match the requested column type.
+     */
+    private long findColumn(int colIdx, NativeTypeSpec type) {
+        // Get base offset (key start or value start) for the given column.
+        boolean keyCol = schema.keyColumn(colIdx);
+        Columns cols = schema.columns(colIdx);
+
+        int off = SCHEMA_VERSION_FIELD_SIZE + KEY_HASH_FIELD_SIZE;
+
+        if (!keyCol) {
+            // Jump to the next chunk, the size of the first chunk is written at the chunk start.
+            off += readShort(off);
+
+            // Adjust the column index according to the number of key columns.
+            colIdx -= schema.keyColumns().length();
+        }
+
+        Column col = cols.column(colIdx);
+
+        if (col.type().spec() != type)
+            throw new InvalidTypeException("Invalid column type requested [requested=" + type +
+                ", column=" + col + ']');
+
+        if (isNull(off, colIdx))
+            return -1;
+
+        return type.fixedLength() ?
+            fixlenColumnOffset(cols, off, colIdx) :
+            varlenColumnOffsetAndLength(cols, off, colIdx);
+    }
+
+    /**
+     * Checks the typle null map for the given column index in the chunk.
+     *
+     * @param baseOff Offset of the chunk start in the tuple.
+     * @param idx Offset of the column in the chunk.
+     * @return {@code true} if the column value is {@code null}.
+     */
+    private boolean isNull(int baseOff, int idx) {
+        int nullMapOff = nullMapOffset(baseOff);
+
+        int nullByte = idx / 8;
+        int posInByte = idx % 8;
+
+        int map = readByte(nullMapOff + nullByte);
+
+        return (map & (1 << posInByte)) != 0;
+    }
+
+    /**
+     * Utility method to extract the column offset from the {@link #findColumn(int, NativeTypeSpec)} result. The
+     * offset is calculated from the beginning of the tuple.
+     *
+     * @param offLen {@code findColumn} invocation result.
+     * @return Column offset from the beginning of the tuple.
+     */
+    private static int offset(long offLen) {
+        return (int)offLen;
+    }
+
+    /**
+     * Utility method to extract the column length from the {@link #findColumn(int, NativeTypeSpec)} result for
+     * varlength columns.
+     *
+     * @param offLen {@code findColumn} invocation result.
+     * @return Length of the column or {@code 0} if the column is fixed-length.
+     */
+    private static int length(long offLen) {
+        return (int)(offLen >>> 32);
+    }
+
+    /**
+     * Calculates the offset and length of varlen column. First, it calculates the number of non-null columns
+     * preceeding the requested column by folding the null map bits. This number is used to adjust the column index
+     * and find the corresponding entry in the varlen table. The length of the column is calculated either by
+     * subtracting two adjacent varlen table offsets, or by subtracting the last varlen table offset from the chunk
+     * length.
+     *
+     * @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.
+     */
+    private long varlenColumnOffsetAndLength(Columns cols, int baseOff, int idx) {
+        int nullMapOff = nullMapOffset(baseOff);
+
+        int nullStartByte = cols.firstVarlengthColumn() / 8;
+        int startBitInByte = cols.firstVarlengthColumn() % 8;
+
+        int nullEndByte = idx / 8;
+        int endBitInByte = idx % 8;
+        int numNullsBefore = 0;
+
+        for (int i = nullStartByte; i <= nullEndByte; i++) {
+            int nullmapByte = readByte(nullMapOff + i);
+
+            if (i == nullStartByte)
+                // We need to clear startBitInByte least significant bits
+                nullmapByte &= (0xFF << startBitInByte);
+
+            if (i == nullEndByte)
+                // We need to clear 8-endBitInByte most significant bits
+                nullmapByte &= (0xFF >> (8 - endBitInByte));
+
+            numNullsBefore += Columns.numberOfNullColumns(nullmapByte);
+        }
+
+        idx -= cols.numberOfFixsizeColumns() + numNullsBefore;
+        int vartableSize = readShort(baseOff + TOTAL_LEN_FIELD_SIZE);
+
+        int vartableOff = vartableOffset(baseOff);
+        // Offset of idx-th column is from base offset.
+        int resOff = readShort(vartableOff + 2 * idx);
+
+        long len = idx == vartableSize - 1 ?
+            // totalLength - columnStartOffset
+            readShort(baseOff) - resOff:
+            // nextColumnStartOffset - columnStartOffset
+            readShort(vartableOff + 2 * (idx + 1)) - resOff;
+
+        return (len << 32) | (resOff + baseOff);
+    }
+
+    /**
+     * Calculates the offset of the fixlen column with the given index in the tuple. It essentially folds the null map
+     * with the column lengths to calculate the size of non-null columns preceeding 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.
+     */
+    int fixlenColumnOffset(Columns cols, int baseOff, int idx) {
+        int nullMapOff = nullMapOffset(baseOff);
+
+        int off = 0;
+        int nullMapIdx = idx / 8;
+
+        // Fold offset based on the whole map bytes in the schema
+        for (int i = 0; i < nullMapIdx; i++)
+            off += cols.foldFixedLength(i, readByte(nullMapOff + i));
+
+        // Set bits starting from posInByte, inclusive, up to either the end of the byte or the last column index, inclusive
+        int startBit = idx % 8;
+        int endBit = nullMapIdx == cols.nullMapSize() - 1 ? ((cols.numberOfFixsizeColumns() - 1) % 8) : 7;
+        int mask = (0xFF >> (7 - endBit)) & (0xFF << startBit);
+
+        off += cols.foldFixedLength(nullMapIdx, readByte(nullMapOff + nullMapIdx) | mask);
+
+        return nullMapOff + cols.nullMapSize() + off;
+    }
+
+    /**
+     * @param baseOff Chunk base offset.
+     * @return Null map offset from the tuple start for the chunk with the given base.
+     */
+    private int nullMapOffset(int baseOff) {
+        int varlenTblLen = readShort(baseOff + TOTAL_LEN_FIELD_SIZE) * 2;
+
+        return vartableOffset(baseOff) + varlenTblLen;
+    }
+
+    /**
+     * @param baseOff Chunk base offset.
+     * @return Offset of the varlen table from the tuple start for the chunk with the given base.
+     */
+    private int vartableOffset(int baseOff) {
+        return baseOff + TOTAL_LEN_FIELD_SIZE + VARSIZE_TABLE_LEN_FIELD_SIZE;
+    }
+
+    /**
+     */
+    protected abstract int readByte(int off);
+
+    /**
+     */
+    protected abstract int readShort(int off);
+
+    /**
+     */
+    protected abstract int readInteger(int off);
+
+    /**
+     */
+    protected abstract long readLong(int off);
+
+    /**
+     */
+    protected abstract float readFloat(int off);
+
+    /**
+     */
+    protected abstract double readDouble(int off);
+
+    /**
+     */
+    protected abstract String readString(int off, int len);
+
+    /**
+     */
+    protected abstract byte[] readBytes(int off, int len);
+}
diff --git a/modules/commons/src/main/java/org/apache/ignite/internal/schema/TupleAssembler.java b/modules/commons/src/main/java/org/apache/ignite/internal/schema/TupleAssembler.java
new file mode 100644
index 0000000..45d3982
--- /dev/null
+++ b/modules/commons/src/main/java/org/apache/ignite/internal/schema/TupleAssembler.java
@@ -0,0 +1,406 @@
+/*
+ * 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;
+
+import java.nio.BufferUnderflowException;
+import java.nio.ByteBuffer;
+import java.nio.CharBuffer;
+import java.nio.charset.CharacterCodingException;
+import java.nio.charset.CharsetEncoder;
+import java.nio.charset.CoderResult;
+import java.nio.charset.StandardCharsets;
+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 must pre-calculate the
+ */
+public class TupleAssembler {
+    /** */
+    private final SchemaDescriptor schema;
+
+    /** The number of non-null varlen columns in values chunk. */
+    private final int nonNullVarlenValCols;
+
+    /** Target byte buffer to write to. */
+    private final ExpandableByteBuf buf;
+
+    /** Current columns chunk. */
+    private Columns curCols;
+
+    /** Current field index (the field is unset). */
+    private int curCol;
+
+    /** Index of the current varlen table entry. Incremented each time non-null varlen column is appended. */
+    private int curVarlenTblEntry;
+
+    /** Current offset for the next column to be appended. */
+    private int curOff;
+
+    /** Base offset of the current chunk */
+    private int baseOff;
+
+    /** Offset of the null map for current chunk. */
+    private int nullMapOff;
+
+    /** Offset of the varlen table for current chunk. */
+    private int varlenTblOff;
+
+    /** Charset encoder for strings. Initialized lazily. */
+    private CharsetEncoder strEncoder;
+
+    /**
+     * @param nonNullVarsizeCols Number of non-null varlen columns.
+     * @return Total size of the varlen table.
+     */
+    public static int varlenTableSize(int nonNullVarsizeCols) {
+        return nonNullVarsizeCols * 2;
+    }
+
+    /**
+     * This implementation is not tolerant to malformed char sequences.
+     */
+    public static int utf8EncodedLength(CharSequence seq) {
+        int cnt = 0;
+
+        for (int i = 0, len = seq.length(); i < len; i++) {
+            char ch = seq.charAt(i);
+
+            if (ch <= 0x7F)
+                cnt++;
+            else if (ch <= 0x7FF)
+                cnt += 2;
+            else if (Character.isHighSurrogate(ch)) {
+                cnt += 4;
+                ++i;
+            }
+            else
+                cnt += 3;
+        }
+
+        return cnt;
+    }
+
+    /**
+     */
+    public static int tupleChunkSize(Columns cols, int nonNullVarsizeCols, int nonNullVarsizeSize) {
+        int size = Tuple.TOTAL_LEN_FIELD_SIZE + Tuple.VARSIZE_TABLE_LEN_FIELD_SIZE +
+            varlenTableSize(nonNullVarsizeCols) + cols.nullMapSize();
+
+        for (int i = 0; i < cols.numberOfFixsizeColumns(); i++)
+            size += cols.column(i).type().length();
+
+        return size + nonNullVarsizeSize;
+    }
+
+    /**
+     * @param schema Tuple schema.
+     * @param size Target tuple size. If the tuple size is known in advance, it should be provided upfront to avoid
+     *      unnccessary arrays copy.
+     * @param nonNullVarsizeKeyCols Number of null varlen columns in key chunk.
+     * @param nonNullVarlenValCols Number of null varlen columns in value chunk.
+     */
+    public TupleAssembler(
+        SchemaDescriptor schema,
+        int size,
+        int nonNullVarsizeKeyCols,
+        int nonNullVarlenValCols
+    ) {
+        this.schema = schema;
+
+        this.nonNullVarlenValCols = nonNullVarlenValCols;
+
+        buf = new ExpandableByteBuf(size);
+
+        curCols = schema.columns(0);
+
+        initOffsets(Tuple.SCHEMA_VERSION_FIELD_SIZE + Tuple.KEY_HASH_FIELD_SIZE, nonNullVarsizeKeyCols);
+
+        buf.putShort(0, (short)schema.version());
+    }
+
+    /**
+     */
+    public static int tupleSize(
+        Columns keyCols,
+        int nonNullVarsizeKeyCols,
+        int nonNullVarsizeKeySize,
+        Columns valCols,
+        int nonNullVarsizeValCols,
+        int nonNullVarsizeValSize
+    ) {
+        return Tuple.SCHEMA_VERSION_FIELD_SIZE + Tuple.KEY_HASH_FIELD_SIZE +
+            tupleChunkSize(keyCols, nonNullVarsizeKeyCols, nonNullVarsizeKeySize) +
+            tupleChunkSize(valCols, nonNullVarsizeValCols, nonNullVarsizeValSize);
+    }
+
+    /**
+     */
+    public void appendNull() {
+        Column col = curCols.column(curCol);
+
+        if (!col.nullable())
+            throw new IllegalArgumentException("Failed to set column (null was passed, but column is not nullable): " +
+                col);
+
+        setNull(curCol);
+
+        shiftColumn(0, false);
+    }
+
+    /**
+     */
+    public void appendByte(byte val) {
+        checkType(NativeType.BYTE);
+
+        buf.put(curOff, val);
+
+        shiftColumn(NativeType.BYTE);
+    }
+
+    /**
+     */
+    public void appendShort(short val) {
+        checkType(NativeType.SHORT);
+
+        buf.putShort(curOff, val);
+
+        shiftColumn(NativeType.SHORT);
+    }
+
+    /**
+     */
+    public void appendInt(int val) {
+        checkType(NativeType.INTEGER);
+
+        buf.putInt(curOff, val);
+
+        shiftColumn(NativeType.INTEGER);
+    }
+
+    /**
+     */
+    public void appendLong(long val) {
+        checkType(NativeType.LONG);
+
+        buf.putLong(curOff, val);
+
+        shiftColumn(NativeType.LONG);
+    }
+
+    /**
+     */
+    public void appendFloat(float val) {
+        checkType(NativeType.FLOAT);
+
+        buf.putFloat(curOff, val);
+
+        shiftColumn(NativeType.FLOAT);
+    }
+
+    /**
+     */
+    public void appendDouble(double val) {
+        checkType(NativeType.DOUBLE);
+
+        buf.putDouble(curOff, val);
+
+        shiftColumn(NativeType.DOUBLE);
+    }
+
+    /**
+     */
+    public void appendUuid(UUID uuid) {
+        checkType(NativeType.UUID);
+
+        buf.putLong(curOff, uuid.getLeastSignificantBits());
+        buf.putLong(curOff + 8, uuid.getMostSignificantBits());
+
+        shiftColumn(NativeType.UUID);
+    }
+
+    /**
+     */
+    public void appendString(String val) {
+        checkType(NativeType.STRING);
+
+        try {
+            int written = buf.putString(curOff, val, encoder());
+
+            writeOffset(curVarlenTblEntry, curOff - baseOff);
+
+            shiftColumn(written, true);
+        }
+        catch (CharacterCodingException e) {
+            throw new AssemblyException("Failed to encode string", e);
+        }
+    }
+
+    /**
+     */
+    public void appendBytes(byte[] val) {
+        checkType(NativeType.BYTES);
+
+        buf.putBytes(curOff, val);
+
+        writeOffset(curVarlenTblEntry, curOff - baseOff);
+
+        shiftColumn(val.length, true);
+    }
+
+    /**
+     */
+    public void appendBitmask(BitSet bitSet) {
+        Column col = curCols.column(curCol);
+
+        checkType(NativeTypeSpec.BITMASK);
+
+        Bitmask maskType = (Bitmask)col.type();
+
+        if (bitSet.length() > maskType.bits())
+            throw new IllegalArgumentException("Failed to set bitmask for column '" + col.name() + "' " +
+                "(mask size exceeds allocated size) [mask=" + bitSet + ", maxSize=" + maskType.bits() + "]");
+
+        byte[] arr = bitSet.toByteArray();
+
+        buf.putBytes(curOff, arr);
+
+        for (int i = 0; i < maskType.length() - arr.length; i++)
+            buf.put(curOff + arr.length + i, (byte)0);
+
+        shiftColumn(maskType);
+    }
+
+    /**
+     */
+    public byte[] build() {
+        return buf.toArray();
+    }
+
+    /**
+     * @return UTF-8 string encoder.
+     */
+    private CharsetEncoder encoder() {
+        if (strEncoder == null)
+            strEncoder = StandardCharsets.UTF_8.newEncoder();
+
+        return strEncoder;
+    }
+
+    /**
+     * Writes the given offset to the varlen table entry with the given index.
+     *
+     * @param tblEntryIdx Varlen table entry index.
+     * @param off Offset to write.
+     */
+    private void writeOffset(int tblEntryIdx, int off) {
+        buf.putShort(varlenTblOff + 2 * tblEntryIdx, (short)off);
+    }
+
+    /**
+     * Checks that the type being appended matches the column type.
+     *
+     * @param type Type spec that is attempted to be appended.
+     */
+    private void checkType(NativeTypeSpec type) {
+        Column col = curCols.column(curCol);
+
+        if (col.type().spec() != type)
+            throw new IllegalArgumentException("Failed to set column (int was passed, but column is of different " +
+                "type): " + col);
+    }
+
+    /**
+     * Checks that the type being appended matches the column type.
+     *
+     * @param type Type that is attempted to be appended.
+     */
+    private void checkType(NativeType type) {
+        checkType(type.spec());
+    }
+
+    /**
+     * Sets null flag in the null map for the given column.
+     * @param colIdx Column index.
+     */
+    private void setNull(int colIdx) {
+        int byteInMap = colIdx / 8;
+        int bitInByte = colIdx % 8;
+
+        buf.put(nullMapOff + byteInMap, (byte)(buf.get(nullMapOff + byteInMap) | (1 << bitInByte)));
+    }
+
+    /**
+     * Must be called after an append of fixlen column.
+     * @param type Type of the appended column.
+     */
+    private void shiftColumn(NativeType type) {
+        assert type.spec().fixedLength() : "Varlen types should provide field length to shift column: " + type;
+
+        shiftColumn(type.length(), false);
+    }
+
+    /**
+     * Shifts current offsets and column indexes as necessary, also changes the chunk base offsets when
+     * moving from key to value columns.
+     *
+     * @param size Size of the appended column.
+     * @param varlen {@code true} if appended column was varlen.
+     */
+    private void shiftColumn(int size, boolean varlen) {
+        curCol++;
+        curOff += size;
+
+        if (varlen)
+            curVarlenTblEntry++;
+
+        if (curCol == curCols.length()) {
+            Columns cols = schema.columns(curCol);
+
+            int keyLen = curOff - baseOff;
+
+            buf.putShort(baseOff, (short)keyLen);
+
+            if (cols == curCols)
+                return;
+
+            curCols = cols;
+
+            initOffsets(baseOff + keyLen, nonNullVarlenValCols);
+        }
+    }
+
+    /**
+     * @param base Chunk base offset.
+     * @param nonNullVarlenCols Number of non-null varlen columns.
+     */
+    private void initOffsets(int base, int nonNullVarlenCols) {
+        baseOff = base;
+
+        curCol = 0;
+        curVarlenTblEntry = 0;
+
+        buf.putShort(baseOff + Tuple.TOTAL_LEN_FIELD_SIZE, (short)nonNullVarlenCols);
+
+        varlenTblOff = baseOff + Tuple.TOTAL_LEN_FIELD_SIZE + Tuple.VARSIZE_TABLE_LEN_FIELD_SIZE;
+        nullMapOff = varlenTblOff + varlenTableSize(nonNullVarlenCols);
+        curOff = nullMapOff + curCols.nullMapSize();
+    }
+}
diff --git a/modules/commons/src/main/java/org/apache/ignite/internal/schema/package-info.java b/modules/commons/src/main/java/org/apache/ignite/internal/schema/package-info.java
new file mode 100644
index 0000000..110317c
--- /dev/null
+++ b/modules/commons/src/main/java/org/apache/ignite/internal/schema/package-info.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 description. -->
+ * Contains schema description, tuple assembly and field accessor classes.
+ * <p>
+ * This package provides necessary infrastructure to create, read, convert to and from POJO classes
+ * schema-defined tuples.
+ * <p>
+ * Schema is defined as a set of columns which are split into key columns chunk and value columns chunk.
+ * Each column defined by a name, nullability flag, and a {@link org.apache.ignite.internal.schema.NativeType}.
+ * Type is a thin wrapper over the {@link org.apache.ignite.internal.schema.NativeTypeSpec} to provide differentiation
+ * between types of one kind with different size (an example of such differentiation is bitmask(n) or number(n)).
+ * {@link org.apache.ignite.internal.schema.NativeTypeSpec} provides necessary indirection to read a column as a
+ * {@code java.lang.Object} without needing to switch over the column type.
+ * <p>
+ * 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 (so that chunk is self-sufficient, and, provided with
+ * the column types can be read independently).
+ * Tuple structure has the following format:
+ *
+ * <pre>
+ * +---------+----------+----------+-------------+
+ * |  Schema |    Key  | Key chunk | Value chunk |
+ * | Version |   Hash  | Bytes     | Bytes       |
+ * +---------+------ --+-----------+-------------+
+ * | 2 bytes | 4 bytes |                         |
+ * +---------+---------+-------------------------+
+ * </pre>
+ * Each bytes section has the following structure:
+ * <pre>
+ * +---------+----------+---------+------+--------+--------+
+ * |   Total | Vartable |  Varlen | Null | Fixlen | Varlen |
+ * |  Length |   Length | Offsets |  Map |  Bytes |  Bytes |
+ * +---------+----------+---------+------+--------+--------+
+ * | 2 bytes |  2 bytes |                                  |
+ * +---------+---------------------------------------------+
+ * </pre>
+ * To assemble a tuple with some schema, an instance of {@link org.apache.ignite.internal.schema.TupleAssembler}
+ * must be used which provides the low-level API for building tuples. When using the tuple 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,
+ * 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.
+ * <p>
+ * To read column values of a tuple, one needs to construct a subclass of
+ * {@link org.apache.ignite.internal.schema.Tuple} which provides necessary logic to read arbitrary columns with
+ * type checking. For primitive types, {@link org.apache.ignite.internal.schema.Tuple} 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).
+ */
+package org.apache.ignite.internal.schema;
\ No newline at end of file
diff --git a/modules/commons/src/test/java/org/apache/ignite/internal/schema/ColumnTest.java b/modules/commons/src/test/java/org/apache/ignite/internal/schema/ColumnTest.java
new file mode 100644
index 0000000..ddc5b74
--- /dev/null
+++ b/modules/commons/src/test/java/org/apache/ignite/internal/schema/ColumnTest.java
@@ -0,0 +1,48 @@
+/*
+ * 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;
+
+import java.util.Arrays;
+import org.junit.jupiter.api.Test;
+
+import static org.junit.jupiter.api.Assertions.assertEquals;
+
+/**
+ *
+ */
+public class ColumnTest {
+    /**
+     *
+     */
+    @Test
+    public void testCompareColumns() {
+        Column[] cols = new Column[] {
+            new Column("C", NativeType.BYTES, false),
+            new Column("B", NativeType.INTEGER, false),
+            new Column("AD", NativeType.STRING, false),
+            new Column("AA", NativeType.STRING, false),
+        };
+
+        Arrays.sort(cols);
+
+        assertEquals("B", cols[0].name());
+        assertEquals("C", cols[1].name());
+        assertEquals("AA", cols[2].name());
+        assertEquals("AD", cols[3].name());
+    }
+}
diff --git a/modules/commons/src/test/java/org/apache/ignite/internal/schema/ColumnsTest.java b/modules/commons/src/test/java/org/apache/ignite/internal/schema/ColumnsTest.java
new file mode 100644
index 0000000..93e5e45
--- /dev/null
+++ b/modules/commons/src/test/java/org/apache/ignite/internal/schema/ColumnsTest.java
@@ -0,0 +1,399 @@
+/*
+ * 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;
+
+import org.junit.jupiter.api.Test;
+
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertFalse;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+
+/**
+ *
+ */
+public class ColumnsTest {
+    /**
+     *
+     */
+    @Test
+    public void testFixsizeIndex() {
+        Columns cols = new Columns(
+            new Column("intCol2", NativeType.INTEGER, false),
+            new Column("intCol1", NativeType.INTEGER, false),
+            new Column("uuidCol", NativeType.UUID, false)
+        );
+
+        assertEquals(3, cols.length());
+        assertEquals(-1, cols.firstVarlengthColumn());
+
+        for (int c = 0; c < cols.length(); c++)
+            assertTrue(cols.isFixedSize(c));
+
+        assertEquals(1, cols.nullMapSize());
+        assertEquals(3, cols.numberOfFixsizeColumns());
+    }
+
+    /**
+     *
+     */
+    @Test
+    public void testVarsizeIndex() {
+        Columns cols = new Columns(
+            new Column("stringCol3", NativeType.STRING, false),
+            new Column("stringCol2", NativeType.STRING, false),
+            new Column("stringCol1", NativeType.STRING, false)
+        );
+
+        assertEquals(3, cols.length());
+        assertEquals(0, cols.firstVarlengthColumn());
+
+        for (int c = 0; c < cols.length(); c++)
+            assertFalse(cols.isFixedSize(c));
+
+        assertEquals(1, cols.nullMapSize());
+        assertEquals(0, cols.numberOfFixsizeColumns());
+    }
+
+    /**
+     *
+     */
+    @Test
+    public void testMixedIndex() {
+        Columns cols = new Columns(
+            new Column("stringCol", NativeType.STRING, false),
+            new Column("intCol2", NativeType.INTEGER, false),
+            new Column("intCol1", NativeType.INTEGER, false),
+            new Column("uuidCol", NativeType.UUID, false)
+        );
+
+        assertEquals(4, cols.length());
+        assertEquals(3, cols.firstVarlengthColumn());
+
+        for (int c = 0; c < cols.length(); c++) {
+            if (c < cols.firstVarlengthColumn())
+                assertTrue(cols.isFixedSize(c));
+            else
+                assertFalse(cols.isFixedSize(c));
+        }
+
+        assertEquals(1, cols.nullMapSize());
+        assertEquals(3, cols.numberOfFixsizeColumns());
+    }
+
+    /**
+     *
+     */
+    @Test
+    public void testNullMapSize() {
+        assertEquals(1, new Columns(columns(1)).nullMapSize());
+        assertEquals(1, new Columns(columns(7)).nullMapSize());
+        assertEquals(1, new Columns(columns(8)).nullMapSize());
+
+        assertEquals(2, new Columns(columns(9)).nullMapSize());
+        assertEquals(2, new Columns(columns(10)).nullMapSize());
+        assertEquals(2, new Columns(columns(15)).nullMapSize());
+        assertEquals(2, new Columns(columns(16)).nullMapSize());
+
+        assertEquals(3, new Columns(columns(17)).nullMapSize());
+        assertEquals(3, new Columns(columns(18)).nullMapSize());
+        assertEquals(3, new Columns(columns(23)).nullMapSize());
+        assertEquals(3, new Columns(columns(24)).nullMapSize());
+    }
+
+    /**
+     *
+     */
+    @Test
+    public void testFoldSizeNoVarlenIncomplete1Byte() {
+        Column[] colDef = {
+            new Column("a", NativeType.SHORT, false),   // 2
+            new Column("b", NativeType.INTEGER, false), // 4
+            new Column("c", NativeType.INTEGER, false), // 4
+            new Column("d", NativeType.LONG, false),    // 8
+            new Column("e", NativeType.LONG, false),    // 8
+            new Column("f", NativeType.LONG, false),    // 8
+            new Column("g", NativeType.UUID, false)     // 16
+        };
+
+        checkColumnFolding(colDef);
+    }
+
+    /**
+     *
+     */
+    @Test
+    public void testFoldSizeNoVarlenFull1Byte() {
+        Column[] colDef = {
+            new Column("a", NativeType.SHORT, false),   // 2
+            new Column("b", NativeType.INTEGER, false), // 4
+            new Column("c", NativeType.INTEGER, false), // 4
+            new Column("d", NativeType.INTEGER, false), // 4
+            new Column("e", NativeType.LONG, false),    // 8
+            new Column("f", NativeType.LONG, false),    // 8
+            new Column("g", NativeType.UUID, false),    // 16
+            new Column("h", NativeType.UUID, false)     // 16
+        };
+
+        checkColumnFolding(colDef);
+    }
+
+    /**
+     *
+     */
+    @Test
+    public void testFoldSizeNoVarlenIncomplete2Bytes() {
+        Column[] colDef = {
+            new Column("a", NativeType.SHORT, false),   // 2
+            new Column("b", NativeType.SHORT, false),   // 2
+            new Column("c", NativeType.INTEGER, false), // 4
+            new Column("d", NativeType.INTEGER, false), // 4
+            new Column("e", NativeType.INTEGER, false), // 4
+            new Column("f", NativeType.INTEGER, false), // 4
+            new Column("g", NativeType.LONG, false),    // 8
+            new Column("h", NativeType.LONG, false),    // 8
+            new Column("i", NativeType.UUID, false),    // 16
+            new Column("j", NativeType.UUID, false)     // 16
+        };
+
+        checkColumnFolding(colDef);
+    }
+
+    /**
+     *
+     */
+    @Test
+    public void testFoldSizeNoVarlenFull2Bytes() {
+        Column[] colDef = {
+            new Column("a", NativeType.SHORT, false),   // 2
+            new Column("b", NativeType.SHORT, false),   // 2
+            new Column("c", NativeType.SHORT, false),   // 2
+            new Column("d", NativeType.SHORT, false),   // 2
+            new Column("e", NativeType.INTEGER, false), // 4
+            new Column("f", NativeType.INTEGER, false), // 4
+            new Column("g", NativeType.INTEGER, false), // 4
+            new Column("h", NativeType.INTEGER, false), // 4
+            new Column("i", NativeType.INTEGER, false), // 4
+            new Column("j", NativeType.INTEGER, false), // 4
+            new Column("k", NativeType.LONG, false),    // 8
+            new Column("l", NativeType.LONG, false),    // 8
+            new Column("m", NativeType.LONG, false),    // 8
+            new Column("n", NativeType.UUID, false),    // 16
+            new Column("o", NativeType.UUID, false),    // 16
+            new Column("p", NativeType.UUID, false)     // 16
+        };
+
+        checkColumnFolding(colDef);
+    }
+
+    /**
+     *
+     */
+    @Test
+    public void testFoldSizeVarlenIncomplete1Byte() {
+        Column[] colDef = {
+            new Column("a", NativeType.SHORT, false),   // 2
+            new Column("b", NativeType.INTEGER, false), // 4
+            new Column("c", NativeType.INTEGER, false), // 4
+            new Column("d", NativeType.INTEGER, false), // 4
+            new Column("e", NativeType.LONG, false),    // 8
+            new Column("f", NativeType.STRING, false),
+            new Column("g", NativeType.BYTES, false)
+        };
+
+        checkColumnFolding(colDef);
+    }
+
+    /**
+     *
+     */
+    @Test
+    public void testFoldSizeVarlenFull1Byte() {
+        Column[] colDef = {
+            new Column("a", NativeType.SHORT, false),   // 2
+            new Column("b", NativeType.INTEGER, false), // 4
+            new Column("c", NativeType.INTEGER, false), // 4
+            new Column("d", NativeType.INTEGER, false), // 4
+            new Column("e", NativeType.LONG, false),    // 8
+            new Column("f", NativeType.STRING, false),
+            new Column("g", NativeType.STRING, false),
+            new Column("h", NativeType.BYTES, false)
+        };
+
+        checkColumnFolding(colDef);
+    }
+
+    /**
+     *
+     */
+    @Test
+    public void testFoldSizeVarlenIncomplete2Bytes1() {
+        Column[] colDef = {
+            new Column("a", NativeType.SHORT, false),   // 2
+            new Column("b", NativeType.INTEGER, false), // 4
+            new Column("c", NativeType.INTEGER, false), // 4
+            new Column("d", NativeType.INTEGER, false), // 4
+            new Column("e", NativeType.INTEGER, false), // 4
+            new Column("f", NativeType.LONG, false),    // 8
+            new Column("g", NativeType.STRING, false),
+            new Column("h", NativeType.STRING, false),
+            new Column("i", NativeType.BYTES, false)
+        };
+
+        checkColumnFolding(colDef);
+    }
+
+    /**
+     *
+     */
+    @Test
+    public void testFoldSizeVarlenIncomplete2Bytes2() {
+        Column[] colDef = {
+            new Column("a", NativeType.SHORT, false),   // 2
+            new Column("b", NativeType.INTEGER, false), // 4
+            new Column("c", NativeType.INTEGER, false), // 4
+            new Column("d", NativeType.INTEGER, false), // 4
+            new Column("e", NativeType.INTEGER, false), // 4
+            new Column("f", NativeType.INTEGER, false), // 4
+            new Column("g", NativeType.INTEGER, false), // 4
+            new Column("h", NativeType.LONG, false),    // 8
+            new Column("i", NativeType.STRING, false),
+            new Column("j", NativeType.STRING, false),
+            new Column("k", NativeType.BYTES, false)
+        };
+
+        checkColumnFolding(colDef);
+    }
+
+    /**
+     *
+     */
+    @Test
+    public void testFoldSizeVarlenIncomplete2Bytes3() {
+        Column[] colDef = {
+            new Column("a", NativeType.SHORT, false),   // 2
+            new Column("b", NativeType.INTEGER, false), // 4
+            new Column("c", NativeType.INTEGER, false), // 4
+            new Column("d", NativeType.INTEGER, false), // 4
+            new Column("e", NativeType.INTEGER, false), // 4
+            new Column("f", NativeType.INTEGER, false), // 4
+            new Column("g", NativeType.INTEGER, false), // 4
+            new Column("h", NativeType.LONG, false),    // 8
+            new Column("i", NativeType.LONG, false),    // 8
+            new Column("j", NativeType.STRING, false),
+            new Column("k", NativeType.BYTES, false)
+        };
+
+        checkColumnFolding(colDef);
+    }
+
+    /**
+     *
+     */
+    @Test
+    public void testFoldSizeVarlenFull2Bytes() {
+        Column[] colDef = {
+            new Column("a", NativeType.SHORT, false),   // 2
+            new Column("b", NativeType.INTEGER, false), // 4
+            new Column("c", NativeType.INTEGER, false), // 4
+            new Column("d", NativeType.INTEGER, false), // 4
+            new Column("e", NativeType.INTEGER, false), // 4
+            new Column("f", NativeType.INTEGER, false), // 4
+            new Column("g", NativeType.INTEGER, false), // 4
+            new Column("h", NativeType.INTEGER, false), // 4
+            new Column("i", NativeType.LONG, false),    // 8
+            new Column("j", NativeType.STRING, false),
+            new Column("k", NativeType.BYTES, false),
+            new Column("l", NativeType.BYTES, false),
+            new Column("m", NativeType.BYTES, false),
+            new Column("n", NativeType.BYTES, false),
+            new Column("o", NativeType.BYTES, false),
+            new Column("p", NativeType.BYTES, false)
+        };
+
+        checkColumnFolding(colDef);
+    }
+
+    /**
+     *
+     */
+    private void checkColumnFolding(Column[] colDef) {
+        Columns cols = new Columns(colDef);
+
+        boolean[] nullMasks = new boolean[cols.numberOfFixsizeColumns()];
+
+        for (int i = 0; i < (1 << cols.numberOfFixsizeColumns()); i++) {
+            checkSize(cols, colDef, nullMasks);
+
+            incrementMask(nullMasks);
+        }
+    }
+
+    /**
+     *
+     */
+    private void incrementMask(boolean[] mask) {
+        boolean add = true;
+
+        for (int i = 0; i < mask.length && add; i++) {
+            add = mask[i];
+            mask[i] = !mask[i];
+        }
+    }
+
+    /**
+     *
+     */
+    private void checkSize(Columns cols, Column[] colDef, boolean[] nullMasks) {
+        // Iterate over bytes first
+        for (int b = 0; b < (cols.numberOfFixsizeColumns() + 7) / 8; b++) {
+            // Start with all non-nulls.
+            int mask = 0x00;
+            int size = 0;
+
+            for (int bit = 0; bit < 8; bit++) {
+                int idx = 8 * b + bit;
+
+                if (idx >= cols.numberOfFixsizeColumns())
+                    break;
+
+                assertTrue(colDef[idx].type().spec().fixedLength());
+
+                if (nullMasks[idx])
+                    // set bit in the mask (indicate null value).
+                    mask |= (1 << bit);
+                else
+                    // non-null, sum the size.
+                    size += colDef[idx].type().length();
+            }
+
+            assertEquals(size, cols.foldFixedLength(b, mask), "Failed [b=" + b + ", mask=" + mask + ']');
+        }
+    }
+
+    /**
+     *
+     */
+    private static Column[] columns(int size) {
+        Column[] ret = new Column[size];
+
+        for (int i = 0; i < ret.length; i++)
+            ret[i] = new Column("column-" + i, NativeType.STRING, true);
+
+        return ret;
+    }
+}
diff --git a/modules/commons/src/test/java/org/apache/ignite/internal/schema/ExpandableByteBufTest.java b/modules/commons/src/test/java/org/apache/ignite/internal/schema/ExpandableByteBufTest.java
new file mode 100644
index 0000000..b6e2135
--- /dev/null
+++ b/modules/commons/src/test/java/org/apache/ignite/internal/schema/ExpandableByteBufTest.java
@@ -0,0 +1,153 @@
+/*
+ * 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;
+
+import java.nio.ByteBuffer;
+import java.nio.ByteOrder;
+import java.nio.charset.StandardCharsets;
+import org.junit.jupiter.api.Test;
+
+import static org.junit.jupiter.api.Assertions.assertArrayEquals;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+
+/**
+ *
+ */
+public class ExpandableByteBufTest {
+    /** */
+    private ExpandableByteBuf buf;
+
+    /**
+     *
+     */
+    @Test
+    public void testAllTypesDirectOrder() throws Exception {
+        buf = new ExpandableByteBuf(5);
+
+        byte[] targetBytes = {1, 2, 3, 4, 5, 6, 7};
+        String targetStr = "abcdefg";
+
+        buf.put(0, (byte)1);
+        buf.putShort(1, (short)2);
+        buf.putInt(3, 3);
+        buf.putLong(7, 4L);
+        buf.putFloat(15, 5.f);
+        buf.putDouble(19, 6.d);
+        buf.putBytes(27, targetBytes);
+        buf.putString(34, targetStr, StandardCharsets.UTF_8.newEncoder());
+
+        byte[] arr = buf.toArray();
+        assertEquals(41, arr.length);
+
+        ByteBuffer b = ByteBuffer.wrap(arr);
+        b.order(ByteOrder.LITTLE_ENDIAN);
+
+        assertEquals((byte)1, b.get(0));
+        assertEquals((short)2, b.getShort(1));
+        assertEquals(3, b.getInt(3));
+        assertEquals(4L, b.getLong(7));
+        assertEquals(5.f, b.getFloat(15));
+        assertEquals(6.d, b.getDouble(19));
+
+        byte[] bytes = new byte[7];
+        b.position(27);
+        b.get(bytes);
+
+        assertArrayEquals(targetBytes, bytes);
+
+        b.position(34);
+        b.get(bytes);
+
+        assertEquals(targetStr, new String(bytes, StandardCharsets.UTF_8));
+    }
+
+    /**
+     *
+     */
+    @Test
+    public void testAllTypesReverseOrder() throws Exception {
+        buf = new ExpandableByteBuf(5);
+
+        byte[] targetBytes = {1, 2, 3, 4, 5, 6, 7};
+        String targetStr = "abcdefg";
+
+        buf.putString(34, targetStr, StandardCharsets.UTF_8.newEncoder());
+        buf.putBytes(27, targetBytes);
+        buf.putDouble(19, 6.d);
+        buf.putFloat(15, 5.f);
+        buf.putLong(7, 4L);
+        buf.putInt(3, 3);
+        buf.putShort(1, (short)2);
+        buf.put(0, (byte)1);
+
+        byte[] arr = buf.toArray();
+        assertEquals(41, arr.length);
+
+        ByteBuffer b = ByteBuffer.wrap(arr);
+        b.order(ByteOrder.LITTLE_ENDIAN);
+
+        assertEquals((byte)1, b.get(0));
+        assertEquals((short)2, b.getShort(1));
+        assertEquals(3, b.getInt(3));
+        assertEquals(4L, b.getLong(7));
+        assertEquals(5.f, b.getFloat(15));
+        assertEquals(6.d, b.getDouble(19));
+
+        byte[] bytes = new byte[7];
+        b.position(27);
+        b.get(bytes);
+
+        assertArrayEquals(targetBytes, bytes);
+
+        b.position(34);
+        b.get(bytes);
+
+        assertEquals(targetStr, new String(bytes, StandardCharsets.UTF_8));
+    }
+
+    /**
+     *
+     */
+    @Test
+    public void testJavadocDesc() {
+        ExpandableByteBuf b = new ExpandableByteBuf(1);
+        b.put(0, (byte)1); // Does not expand.
+        b.put(5, (byte)1); // Expands, meaningful bytes are [0..5]
+        byte[] data = b.toArray();
+
+        assertEquals(6, data.length);
+    }
+
+    /**
+     *
+     */
+    @Test
+    public void testStringExpandMultipleTimes() throws Exception {
+        // Expansion chain 1->2->4->8->16->32.
+        buf = new ExpandableByteBuf(1);
+
+        String str = "abcdefghijklmnopq";
+
+        buf.putString(0, str, StandardCharsets.UTF_8.newEncoder());
+
+        byte[] arr = buf.toArray();
+
+        assertEquals(str.length(), arr.length);
+        assertEquals(str, new String(arr));
+    }
+}
diff --git a/modules/commons/src/test/java/org/apache/ignite/internal/schema/NativeTypeTest.java b/modules/commons/src/test/java/org/apache/ignite/internal/schema/NativeTypeTest.java
new file mode 100644
index 0000000..30b36dd
--- /dev/null
+++ b/modules/commons/src/test/java/org/apache/ignite/internal/schema/NativeTypeTest.java
@@ -0,0 +1,71 @@
+/*
+ * 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;
+
+import org.junit.jupiter.api.Test;
+
+import static org.junit.jupiter.api.Assertions.assertTrue;
+
+/**
+ *
+ */
+public class NativeTypeTest {
+    /**
+     *
+     */
+    @Test
+    public void testCompareFixlenVarlen() {
+        assertTrue(NativeType.BYTE.compareTo(NativeType.STRING) < 0);
+        assertTrue(NativeType.BYTE.compareTo(NativeType.BYTES) < 0);
+
+        assertTrue(NativeType.INTEGER.compareTo(NativeType.STRING) < 0);
+        assertTrue(NativeType.INTEGER.compareTo(NativeType.BYTES) < 0);
+
+        assertTrue(NativeType.LONG.compareTo(NativeType.STRING) < 0);
+        assertTrue(NativeType.LONG.compareTo(NativeType.BYTES) < 0);
+
+        assertTrue(NativeType.UUID.compareTo(NativeType.STRING) < 0);
+        assertTrue(NativeType.UUID.compareTo(NativeType.BYTES) < 0);
+    }
+
+    /**
+     *
+     */
+    @Test
+    public void testCompareFixlenBySize() {
+        assertTrue(NativeType.SHORT.compareTo(NativeType.INTEGER) < 0);
+        assertTrue(NativeType.INTEGER.compareTo(NativeType.LONG) < 0);
+        assertTrue(NativeType.LONG.compareTo(NativeType.UUID) < 0);
+    }
+
+    /**
+     *
+     */
+    @Test
+    public void testCompareFixlenByDesc() {
+        assertTrue(NativeType.FLOAT.compareTo(NativeType.INTEGER) < 0);
+    }
+
+    /**
+     *
+     */
+    @Test
+    public void testCompareVarlenByDesc() {
+        assertTrue(NativeType.BYTES.compareTo(NativeType.STRING) < 0);
+    }
+}
diff --git a/modules/commons/src/test/java/org/apache/ignite/internal/schema/TupleTest.java b/modules/commons/src/test/java/org/apache/ignite/internal/schema/TupleTest.java
new file mode 100644
index 0000000..746e4d5
--- /dev/null
+++ b/modules/commons/src/test/java/org/apache/ignite/internal/schema/TupleTest.java
@@ -0,0 +1,382 @@
+/*
+ * 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;
+
+import java.util.Arrays;
+import java.util.BitSet;
+import java.util.Collections;
+import java.util.Random;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+
+import static org.apache.ignite.internal.schema.NativeType.BYTE;
+import static org.apache.ignite.internal.schema.NativeType.BYTES;
+import static org.apache.ignite.internal.schema.NativeType.DOUBLE;
+import static org.apache.ignite.internal.schema.NativeType.FLOAT;
+import static org.apache.ignite.internal.schema.NativeType.INTEGER;
+import static org.apache.ignite.internal.schema.NativeType.LONG;
+import static org.apache.ignite.internal.schema.NativeType.SHORT;
+import static org.apache.ignite.internal.schema.NativeType.STRING;
+import static org.apache.ignite.internal.schema.NativeType.UUID;
+import static org.junit.jupiter.api.Assertions.assertArrayEquals;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+
+/**
+ * Tests tuple assembling and reading.
+ */
+public class TupleTest {
+    /** */
+    private Random rnd;
+
+    /**
+     *
+     */
+    @BeforeEach
+    public void initRandom() {
+        long seed = System.currentTimeMillis();
+
+        System.out.println("Using seed: " + seed + "L; //");
+
+        rnd = new Random(seed);
+    }
+
+    /**
+     *
+     */
+    @Test
+    public void testFixedSizes() {
+        Column[] keyCols = new Column[] {
+            new Column("keyByteCol", BYTE, true),
+            new Column("keyShortCol", SHORT, true),
+            new Column("keyIntCol", INTEGER, true),
+            new Column("keyLongCol", LONG, true),
+            new Column("keyFloatCol", FLOAT, true),
+            new Column("keyDoubleCol", DOUBLE, true),
+            new Column("keyUuidCol", UUID, true),
+            new Column("keyBitmask1Col", Bitmask.of(4), true),
+            new Column("keyBitmask2Col", Bitmask.of(22), true)
+        };
+
+        Column[] valCols = new Column[] {
+            new Column("valByteCol", BYTE, true),
+            new Column("valShortCol", SHORT, true),
+            new Column("valIntCol", INTEGER, true),
+            new Column("valLongCol", LONG, true),
+            new Column("valFloatCol", FLOAT, true),
+            new Column("valDoubleCol", DOUBLE, true),
+            new Column("valUuidCol", UUID, true),
+            new Column("valBitmask1Col", Bitmask.of(4), true),
+            new Column("valBitmask2Col", Bitmask.of(22), true)
+        };
+
+        checkSchema(keyCols, valCols);
+    }
+
+    /**
+     *
+     */
+    @Test
+    public void testVariableSizes() {
+        Column[] keyCols = new Column[] {
+            new Column("keyByteCol", BYTE, true),
+            new Column("keyShortCol", SHORT, true),
+            new Column("keyIntCol", INTEGER, true),
+            new Column("keyLongCol", LONG, true),
+            new Column("keyBytesCol", BYTES, true),
+            new Column("keyStringCol", STRING, true),
+        };
+
+        Column[] valCols = new Column[] {
+            new Column("keyByteCol", BYTE, true),
+            new Column("keyShortCol", SHORT, true),
+            new Column("keyIntCol", INTEGER, true),
+            new Column("keyLongCol", LONG, true),
+            new Column("valBytesCol", BYTES, true),
+            new Column("valStringCol", STRING, true),
+        };
+
+        checkSchema(keyCols, valCols);
+    }
+
+    /**
+     *
+     */
+    @Test
+    public void testMixedSizes() {
+        Column[] keyCols = new Column[] {
+            new Column("keyBytesCol", BYTES, true),
+            new Column("keyStringCol", STRING, true),
+        };
+
+        Column[] valCols = new Column[] {
+            new Column("valBytesCol", BYTES, true),
+            new Column("valStringCol", STRING, true),
+        };
+
+        checkSchema(keyCols, valCols);
+    }
+
+    /**
+     *
+     */
+    private void checkSchema(Column[] keyCols, Column[] valCols) {
+        checkSchemaShuffled(keyCols, valCols);
+
+        shuffle(keyCols);
+        shuffle(valCols);
+
+        checkSchemaShuffled(keyCols, valCols);
+    }
+
+    /**
+     *
+     */
+    private void checkSchemaShuffled(Column[] keyCols, Column[] valCols) {
+        SchemaDescriptor sch = new SchemaDescriptor(1, new Columns(keyCols), new Columns(valCols));
+
+        Object[] checkArr = sequence(sch);
+
+        checkValues(sch, checkArr);
+
+        while (checkArr[0] != null) {
+            int idx = 0;
+
+            Object prev = checkArr[idx];
+            checkArr[idx] = null;
+
+            checkValues(sch, checkArr);
+
+            while (idx < checkArr.length - 1 && checkArr[idx + 1] != null) {
+                checkArr[idx] = prev;
+                prev = checkArr[idx + 1];
+                checkArr[idx + 1] = null;
+                idx++;
+
+                checkValues(sch, checkArr);
+            }
+        }
+    }
+
+    /**
+     *
+     */
+    private Object[] sequence(SchemaDescriptor schema) {
+        Object[] res = new Object[schema.length()];
+
+        for (int i = 0; i < res.length; i++) {
+            NativeType type = schema.column(i).type();
+
+            res[i] = generateRandomValue(type);
+        }
+
+        return res;
+    }
+
+    /**
+     *
+     */
+    private Object generateRandomValue(NativeType type) {
+        switch (type.spec()) {
+            case BYTE:
+                return (byte)rnd.nextInt(255);
+
+            case SHORT:
+                return (short)rnd.nextInt(65535);
+
+            case INTEGER:
+                return rnd.nextInt();
+
+            case LONG:
+                return rnd.nextLong();
+
+            case FLOAT:
+                return rnd.nextFloat();
+
+            case DOUBLE:
+                return rnd.nextDouble();
+
+            case UUID:
+                return new java.util.UUID(rnd.nextLong(), rnd.nextLong());
+
+            case STRING: {
+                int size = rnd.nextInt(255);
+
+                StringBuilder sb = new StringBuilder();
+
+                while (sb.length() < size) {
+                    char pt = (char)rnd.nextInt(Character.MAX_VALUE + 1);
+
+                    if (Character.isDefined(pt) &&
+                        Character.getType(pt) != Character.PRIVATE_USE &&
+                        !Character.isSurrogate(pt))
+                        sb.append(pt);
+                }
+
+                return sb.toString();
+            }
+
+            case BYTES: {
+                int size = rnd.nextInt(255);
+                byte[] data = new byte[size];
+                rnd.nextBytes(data);
+
+                return data;
+            }
+
+            case BITMASK: {
+                Bitmask maskType = (Bitmask)type;
+
+                BitSet set = new BitSet();
+
+                for (int i = 0; i < maskType.bits(); i++) {
+                    if (rnd.nextBoolean())
+                        set.set(i);
+                }
+
+                return set;
+            }
+
+            default:
+                throw new IllegalStateException("Unsupported type: " + type);
+        }
+    }
+
+    /**
+     *
+     */
+    private void checkValues(SchemaDescriptor schema, Object... vals) {
+        assertEquals(schema.keyColumns().length() + schema.valueColumns().length(), vals.length);
+
+        int nonNullVarsizeKeyCols = 0;
+        int nonNullVarsizeValCols = 0;
+        int nonNullVarsizeKeySize = 0;
+        int nonNullVarsizeValSize = 0;
+
+        for (int i = 0; i < vals.length; i++) {
+            NativeTypeSpec type = schema.column(i).type().spec();
+
+            if (vals[i] != null && !type.fixedLength()) {
+                if (type == NativeTypeSpec.BYTES) {
+                    byte[] val = (byte[])vals[i];
+                    if (schema.keyColumn(i)) {
+                        nonNullVarsizeKeyCols++;
+                        nonNullVarsizeKeySize += val.length;
+                    }
+                    else {
+                        nonNullVarsizeValCols++;
+                        nonNullVarsizeValSize += val.length;
+                    }
+                }
+                else if (type == NativeTypeSpec.STRING) {
+                    if (schema.keyColumn(i)) {
+                        nonNullVarsizeKeyCols++;
+                        nonNullVarsizeKeySize += TupleAssembler.utf8EncodedLength((CharSequence)vals[i]);
+                    }
+                    else {
+                        nonNullVarsizeValCols++;
+                        nonNullVarsizeValSize += TupleAssembler.utf8EncodedLength((CharSequence)vals[i]);
+                    }
+                }
+                else
+                    throw new IllegalStateException("Unsupported test varsize type: " + type);
+            }
+        }
+
+        int size = TupleAssembler.tupleSize(
+            schema.keyColumns(), nonNullVarsizeKeyCols, nonNullVarsizeKeySize,
+            schema.valueColumns(), nonNullVarsizeValCols, nonNullVarsizeValSize);
+
+        TupleAssembler asm = new TupleAssembler(schema, size, nonNullVarsizeKeyCols, nonNullVarsizeValCols);
+
+        for (int i = 0; i < vals.length; i++) {
+            if (vals[i] == null)
+                asm.appendNull();
+            else {
+                NativeType type = schema.column(i).type();
+
+                switch (type.spec()) {
+                    case BYTE:
+                        asm.appendByte((Byte)vals[i]);
+                        break;
+
+                    case SHORT:
+                        asm.appendShort((Short)vals[i]);
+                        break;
+
+                    case INTEGER:
+                        asm.appendInt((Integer)vals[i]);
+                        break;
+
+                    case LONG:
+                        asm.appendLong((Long)vals[i]);
+                        break;
+
+                    case FLOAT:
+                        asm.appendFloat((Float)vals[i]);
+                        break;
+
+                    case DOUBLE:
+                        asm.appendDouble((Double)vals[i]);
+                        break;
+
+                    case UUID:
+                        asm.appendUuid((java.util.UUID)vals[i]);
+                        break;
+
+                    case STRING:
+                        asm.appendString((String)vals[i]);
+                        break;
+
+                    case BYTES:
+                        asm.appendBytes((byte[])vals[i]);
+                        break;
+
+                    case BITMASK:
+                        asm.appendBitmask((BitSet)vals[i]);
+                        break;
+
+                    default:
+                        throw new IllegalStateException("Unsupported test type: " + type);
+                }
+            }
+        }
+
+        byte[] data = asm.build();
+
+        ByteBufferTuple tup = new ByteBufferTuple(schema, data);
+
+        for (int i = 0; i < vals.length; i++) {
+            Column col = schema.column(i);
+
+            NativeTypeSpec type = col.type().spec();
+
+            if (type == NativeTypeSpec.BYTES)
+                assertArrayEquals((byte[])vals[i], (byte[])NativeTypeSpec.BYTES.objectValue(tup, i),
+                    "Failed for column: " + col);
+            else
+                assertEquals(vals[i], type.objectValue(tup, i), "Failed for column: " + col);
+        }
+    }
+
+    /**
+     *
+     */
+    private void shuffle(Column[] cols) {
+        Collections.shuffle(Arrays.asList(cols));
+    }
+}
diff --git a/pom.xml b/pom.xml
index 9a8c2ed..d825ae1 100644
--- a/pom.xml
+++ b/pom.xml
@@ -1,4 +1,25 @@
 <?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.
+-->
+
+<!--
+    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">
@@ -25,16 +46,106 @@
     </licenses>
 
     <properties>
-        <maven.compiler.source>1.8</maven.compiler.source>
-        <maven.compiler.target>1.8</maven.compiler.target>
+        <project.build.sourceEncoding>UTF-8</project.build.sourceEncoding>
+
+        <jetbrains.annotations.version>20.1.0</jetbrains.annotations.version>
+        <junit.jupiter.version>5.7.0</junit.jupiter.version>
+
+        <maven.compiler.plugin.version>3.8.1</maven.compiler.plugin.version>
+        <maven.surefire.plugin.version>3.0.0-M4</maven.surefire.plugin.version>
+
+        <apache.rat.plugin.version>0.13</apache.rat.plugin.version>
     </properties>
 
+    <profiles>
+        <profile>
+            <id>check-licenses</id>
+            <build>
+                <plugins>
+                    <plugin>
+                        <groupId>org.apache.rat</groupId>
+                        <artifactId>apache-rat-plugin</artifactId>
+                        <version>${apache.rat.plugin.version}</version>
+                        <configuration>
+                            <addDefaultLicenseMatchers>true</addDefaultLicenseMatchers>
+                            <licenses>
+                                <license implementation="org.apache.rat.analysis.license.FullTextMatchingLicense">
+                                    <licenseFamilyCategory>IAL20</licenseFamilyCategory>
+                                    <licenseFamilyName>Ignite Apache License 2.0</licenseFamilyName>
+                                    <fullText>
+                                        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.
+                                    </fullText>
+                                </license>
+                            </licenses>
+                            <licenseFamilies>
+                                <licenseFamily implementation="org.apache.rat.license.SimpleLicenseFamily">
+                                    <familyName>Ignite Apache License 2.0</familyName>
+                                </licenseFamily>
+                            </licenseFamilies>
+                        </configuration>
+                        <executions>
+                            <execution>
+                                <phase>validate</phase>
+                                <goals>
+                                    <goal>check</goal>
+                                </goals>
+                                <configuration>
+                                    <excludes>
+                                        <exclude>README.md</exclude>
+                                        <exclude>work/**</exclude>
+                                        <exclude>**/target/**</exclude>
+                                    </excludes>
+                                </configuration>
+                            </execution>
+                        </executions>
+                    </plugin>
+                </plugins>
+            </build>
+        </profile>
+    </profiles>
+
+    <modules>
+        <module>modules/commons</module>
+    </modules>
+
     <dependencies>
         <dependency>
             <groupId>org.junit.jupiter</groupId>
             <artifactId>junit-jupiter-engine</artifactId>
-            <version>5.6.2</version>
+            <version>${junit.jupiter.version}</version>
             <scope>test</scope>
         </dependency>
     </dependencies>
+
+    <build>
+        <plugins>
+            <plugin>
+                <groupId>org.apache.maven.plugins</groupId>
+                <artifactId>maven-compiler-plugin</artifactId>
+                <version>${maven.compiler.plugin.version}</version>
+                <configuration>
+                    <release>11</release>
+                </configuration>
+            </plugin>
+
+            <plugin>
+                <groupId>org.apache.maven.plugins</groupId>
+                <artifactId>maven-surefire-plugin</artifactId>
+                <version>${maven.surefire.plugin.version}</version>
+            </plugin>
+        </plugins>
+    </build>
 </project>