You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by am...@apache.org on 2021/06/09 19:36:23 UTC

[ignite-3] 11/15: WIP.

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

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

commit cdb9015da5840fd641edc8ec43075c4ee11ac563
Author: Andrew Mashenkov <an...@gmail.com>
AuthorDate: Tue Jun 8 22:11:15 2021 +0300

    WIP.
---
 .../apache/ignite/internal/schema/BinaryRow.java   |  22 +-
 .../ignite/internal/schema/ByteBufferRow.java      |  10 +-
 .../ignite/internal/schema/row/ChunkFormat.java    | 375 +++++++++------------
 .../ignite/internal/schema/row/ChunkReader.java    |  48 ++-
 .../ignite/internal/schema/row/ChunkWriter.java    |  70 +++-
 .../internal/schema/row/LongChunkWriter.java       |  90 -----
 .../internal/schema/row/MeduimChunkWriter.java     |  97 ------
 .../org/apache/ignite/internal/schema/row/Row.java |  38 ++-
 .../ignite/internal/schema/row/RowAssembler.java   |  39 +--
 .../internal/schema/row/TinyChunkWriter.java       |  62 ----
 10 files changed, 295 insertions(+), 556 deletions(-)

diff --git a/modules/schema/src/main/java/org/apache/ignite/internal/schema/BinaryRow.java b/modules/schema/src/main/java/org/apache/ignite/internal/schema/BinaryRow.java
index 9f13385..1826684 100644
--- a/modules/schema/src/main/java/org/apache/ignite/internal/schema/BinaryRow.java
+++ b/modules/schema/src/main/java/org/apache/ignite/internal/schema/BinaryRow.java
@@ -145,24 +145,14 @@ public interface BinaryRow {
         /** Flag indicates row has no value chunk. */
         public static final int NO_VALUE_FLAG = 1;
 
-        /** Flag indicates key chunk omits null map. */
-        public static final int OMIT_KEY_NULL_MAP_FLAG = 1 << 1;
+        /** Chunk flags mask. */
+        public static final int CHUNK_FLAGS_MASK = 0x0F;
 
-        /** Flag indicates value chunk omits null map. */
-        public static final int OMIT_VAL_NULL_MAP_FLAG = 1 << 2;
+        /** Key specific flags. */
+        public static final int KEY_FLAGS_OFFSET = 8;
 
-        /** Flag indicates key chunk omits varlen table. */
-        public static final int OMIT_KEY_VARTBL_FLAG = 1 << 3;
-
-        /** Flag indicates value chunk omits varlen table. */
-        public static final int OMIT_VAL_VARTBL_FLAG = 1 << 4;
-
-        /** Flag indicates value chunk is written in Tiny format. */
-        public static final int VAL_TYNY_FORMAT = 1 << 6;
-
-        /** Flag indicates key chunk is written in Tiny format. */
-        public static final int KEY_FORMAT_MASK = (2 ^ 4 - 1) << 8;
-        public static final int VALUE_FORMAT_MASK = (2 ^ 4 - 1) << 12;
+        /** Value specific flags. */
+        public static final int VAL_FLAGS_OFFSET = 12;
 
         /** Stub. */
         private RowFlags() {
diff --git a/modules/schema/src/main/java/org/apache/ignite/internal/schema/ByteBufferRow.java b/modules/schema/src/main/java/org/apache/ignite/internal/schema/ByteBufferRow.java
index 0f3b484..170028e 100644
--- a/modules/schema/src/main/java/org/apache/ignite/internal/schema/ByteBufferRow.java
+++ b/modules/schema/src/main/java/org/apache/ignite/internal/schema/ByteBufferRow.java
@@ -132,7 +132,7 @@ public class ByteBufferRow implements BinaryRow {
         final short flags = readShort(FLAGS_FIELD_OFFSET);
 
         final int off = KEY_CHUNK_OFFSET;
-        final int len = (flags & RowFlags.KEY_TYNY_FORMAT) == 0 ? readInteger(off) : (readByte(off) & 0xFF);
+        final int len = readInteger(off);
 
         try {
             return buf.limit(off + len).position(off).slice();
@@ -145,12 +145,8 @@ public class ByteBufferRow implements BinaryRow {
 
     /** {@inheritDoc} */
     @Override public ByteBuffer valueSlice() {
-        final short flags = readShort(FLAGS_FIELD_OFFSET);
-
-        int off = KEY_CHUNK_OFFSET +
-            ((flags & RowFlags.KEY_TYNY_FORMAT) == 0 ? readInteger(KEY_CHUNK_OFFSET) : (readByte(KEY_CHUNK_OFFSET) & 0xFF));
-
-        int len = hasValue() ? (flags & RowFlags.VAL_TYNY_FORMAT) == 0 ? readInteger(off) : (readByte(off) & 0xFF) : 0;
+        int off = KEY_CHUNK_OFFSET + readInteger(KEY_CHUNK_OFFSET);
+        int len = hasValue() ? readInteger(off) : 0;
 
         try {
             return buf.limit(off + len).position(off).slice();
diff --git a/modules/schema/src/main/java/org/apache/ignite/internal/schema/row/ChunkFormat.java b/modules/schema/src/main/java/org/apache/ignite/internal/schema/row/ChunkFormat.java
index d5e8d17..270420f 100644
--- a/modules/schema/src/main/java/org/apache/ignite/internal/schema/row/ChunkFormat.java
+++ b/modules/schema/src/main/java/org/apache/ignite/internal/schema/row/ChunkFormat.java
@@ -23,267 +23,222 @@ import org.apache.ignite.internal.schema.BinaryRow;
  * Chunk writers factory.
  */
 abstract class ChunkFormat {
-    /** Writer factory for tiny-sized chunks. */
-    private static final ChunkFormat TINY = new TinyChunkFormat();
+    /** First 2 bits in chunk flags. */
+    public static final int FORMAT_CODE_MASK = 0x03;
 
-    /** Writer factory for med-sized chunks. */
-    private static final ChunkFormat MEDIUM = new MedChunkFormat();
+    /** Flag indicates key chunk omits varlen table. */
+    public static final int OMIT_NULL_MAP_FLAG = 1 << 2;
 
-    /** Writer factory for large-sized chunks. */
-    private static final ChunkFormat LARGE = new LargeChunkFormat();
+    /** Flag indicates value chunk omits null map. */
+    public static final int OMIT_VARTBL_FLAG = 1 << 3;
 
-    /**
-     * Check if chunk fits to max size.
-     *
-     * @param payloadLen Payload size in bytes.
-     * @param nullMapLen Null-map size in bytes.
-     * @param vartblSize Amount of vartable items.
-     * @return {@code true} if a chunk is tiny, {@code false} otherwise.
-     */
-    static ChunkFormat writeMode(int payloadLen, int nullMapLen, int vartblSize) {
-        if (TINY.chunkSize(payloadLen, nullMapLen, vartblSize) < 256)
-            return TINY;
-
-        if (MEDIUM.chunkSize(payloadLen, nullMapLen, vartblSize) < 64 * 1024)
-            return MEDIUM;
-
-        return LARGE;
-    }
+    /** Writer factory for tiny-sized chunks. */
+    private static final ChunkFormat TINY = new ChunkFormat(Byte.BYTES, Byte.BYTES, (byte)1) {
+        @Override void writeOffset(ExpandableByteBuf buf, int itemOff, int off) {
+            assert off < (1 << 8) && off >= 0 : "Varlen offset overflow: offset=" + off;
 
-    /**
-     * @param payloadLen Row payload length in bytes.
-     * @param nullMapLen Null-map length in bytes.
-     * @param vartblItems Number of vartable items.
-     * @return Chunk size.
-     */
-    abstract int chunkSize(int payloadLen, int nullMapLen, int vartblItems);
+            buf.put(itemOff, (byte)off);
+        }
 
-    /**
-     * Returns mode flags. First 4-bits are used.
-     *
-     * @return Chunk specific flags.
-     */
-    byte modeFlags() {
-        return 0;
-    }
+        @Override int readOffset(BinaryRow row, int itemOff) {
+            return row.readByte(itemOff) & 0xFF;
+        }
 
-    /**
-     * Calculates vartable length (in bytes).
-     *
-     * @param items Vartable items.
-     * @return Vartable size in bytes.
-     */
-    protected abstract int vartableLength(int items);
+        @Override void writeVartblSize(ExpandableByteBuf buf, int vartblOff, int size) {
+            assert size < (1 << 8) && size >= 0 : "Vartable size overflow: size=" + size;
 
-    /**
-     * Chunk writer factory method.
-     *
-     * @param buf Row buffer.
-     * @param baseOff Chunk base offset.
-     * @param nullMapLen Null-map length.
-     * @param vartblSize Vartable length.
-     * @return Chunk writer.
-     */
-    abstract ChunkWriter writer(ExpandableByteBuf buf, int baseOff, int nullMapLen, int vartblSize);
+            buf.put(vartblOff, (byte)vartblOff);
+        }
 
-    abstract ChunkReader reader(BinaryRow row, int baseOff, int nullMapLen, boolean hasVarTable);
+        @Override int readVartblSize(BinaryRow row, int vartblOff) {
+            return row.readByte(vartblOff) & 0xFF;
+        }
+    };
 
-    /**
-     * Writer factory for tiny-sized chunks.
-     */
-    private static class TinyChunkFormat extends ChunkFormat {
+    /** Writer factory for med-sized chunks. */
+    private static final ChunkFormat MEDIUM = new ChunkFormat(Short.BYTES, Short.BYTES, (byte)2) {
+        @Override void writeOffset(ExpandableByteBuf buf, int itemOff, int off) {
+            assert off < (1 << 16) && off >= 0 : "Varlen offset overflow: offset=" + off;
 
-        /** {@inheritDoc} */
-        @Override protected int vartableLength(int items) {
-            return items == 0 ? 0 : Byte.BYTES /* Table size */ + items * Byte.BYTES;
+            buf.putShort(itemOff, (short)off);
         }
 
-        /** {@inheritDoc} */
-        @Override int chunkSize(int payloadLen, int nullMapLen, int vartblItems) {
-            return Byte.BYTES /* Chunk len. */ + nullMapLen + vartableLength(vartblItems) + payloadLen;
+        @Override int readOffset(BinaryRow row, int itemOff) {
+            return row.readShort(itemOff) & 0xFFFF;
         }
 
-        /** {@inheritDoc} */
-        @Override byte modeFlags() {
-            return 1;
-        }
+        @Override void writeVartblSize(ExpandableByteBuf buf, int vartblOff, int size) {
+            assert size < (1 << 16) && size >= 0 : "Vartable size overflow: size=" + size;
 
-        /** {@inheritDoc} */
-        @Override ChunkWriter writer(ExpandableByteBuf buf, int baseOff, int nullMapLen, int vartblSize) {
-            return new TinyChunkWriter(buf,
-                baseOff,
-                baseOff + Byte.BYTES /* Chunk size */,
-                baseOff + Byte.BYTES + nullMapLen,
-                baseOff + Byte.BYTES + nullMapLen + vartableLength(vartblSize));
+            buf.putShort(vartblOff, (short)vartblOff);
         }
 
-        /** {@inheritDoc} */
-        @Override ChunkReader reader(BinaryRow row,int baseOff, int nullMapLen, boolean hasVarTable) {
-            return new TinyChunkReader(row, baseOff,  nullMapLen, hasVarTable);
+        @Override int readVartblSize(BinaryRow row, int vartblOff) {
+            return row.readShort(vartblOff) & 0xFFFF;
         }
-    }
+    };
 
-    /**
-     * Writer factory for med-size chunks.
-     */
-    private static class MedChunkFormat extends ChunkFormat {
-        /** {@inheritDoc} */
-        @Override protected int vartableLength(int items) {
-            return items == 0 ? 0 : Short.BYTES /* Table size */ + items * Short.BYTES;
+    /** Writer factory for large-sized chunks. */
+    private static final ChunkFormat LARGE = new ChunkFormat(Short.BYTES, Integer.BYTES, (byte)0) {
+        @Override void writeOffset(ExpandableByteBuf buf, int itemOff, int off) {
+            buf.putInt(itemOff, off);
         }
 
-        /** {@inheritDoc} */
-        @Override int chunkSize(int payloadLen, int nullMapLen, int vartblItems) {
-            return Short.BYTES /* Chunk len. */ + nullMapLen + vartableLength(vartblItems) + payloadLen;
+        @Override int readOffset(BinaryRow row, int itemOff) {
+            return row.readInteger(itemOff);
         }
 
-        /** {@inheritDoc} */
-        @Override byte modeFlags() {
-            return 2;
-        }
+        @Override void writeVartblSize(ExpandableByteBuf buf, int vartblOff, int size) {
+            assert size < (1 << 16) && size >= 0 : "Vartable size overflow: size=" + size;
 
-        /** {@inheritDoc} */
-        @Override ChunkWriter writer(ExpandableByteBuf buf, int baseOff, int nullMapLen, int vartblSize) {
-            return new MeduimChunkWriter(buf,
-                baseOff,
-                baseOff + Short.BYTES /* Chunk size */,
-                baseOff + Short.BYTES + nullMapLen,
-                baseOff + Short.BYTES + nullMapLen + vartableLength(vartblSize));
+            buf.putShort(vartblOff, (short)vartblOff);
         }
 
-        /** {@inheritDoc} */
-        @Override ChunkReader reader(BinaryRow row,int baseOff, int nullMapLen, boolean hasVarTable) {
-            return new MediumChunkReader(row, baseOff,  nullMapLen, hasVarTable);
+        @Override int readVartblSize(BinaryRow row, int vartblOff) {
+            return row.readShort(vartblOff) & 0xFFFF;
         }
-    }
+    };
+
+    /** Chunk length field size. */
+    public static final int CHUNK_LEN_FLD_SIZE = Integer.BYTES;
+
+    private final int vartableItemSize;
+
+    private final int vartableSizeFieldLen;
+
+    private final byte modeFlags;
 
     /**
-     * Writer factory for large-sized chunks.
+     * Return chunk formatter.
+     *
+     * @param payloadLen Payload size in bytes.
+     * @return Chunk formatter.
      */
-    private static class LargeChunkFormat extends ChunkFormat {
-        /** {@inheritDoc} */
-        @Override protected int vartableLength(int items) {
-            return items == 0 ? 0 : Integer.BYTES /* Table size */ + items * Integer.BYTES;
-        }
+    static ChunkFormat formatter(int payloadLen) {
+        if (payloadLen < 256)
+            return TINY;
 
-        /** {@inheritDoc} */
-        @Override int chunkSize(int payloadLen, int nullMapLen, int vartblItems) {
-            return Integer.BYTES /* Chunk len. */ + nullMapLen + vartableLength(vartblItems) + payloadLen;
-        }
+        if (payloadLen < 64 * 1024)
+            return MEDIUM;
 
-        /** {@inheritDoc} */
-        @Override ChunkWriter writer(ExpandableByteBuf buf, int baseOff, int nullMapLen, int vartblSize) {
-            return new TinyChunkWriter(buf,
-                baseOff,
-                baseOff + Integer.BYTES /* Chunk size */,
-                baseOff + Integer.BYTES + nullMapLen,
-                baseOff + Integer.BYTES + nullMapLen + vartableLength(vartblSize));
-        }
-        /** {@inheritDoc} */
-        @Override ChunkReader reader(BinaryRow row,int baseOff, int nullMapLen, boolean hasVarTable) {
-            return new LargeChunkReader(row, baseOff,  nullMapLen, hasVarTable);
-        }
+        return LARGE;
     }
 
     /**
-     * Tiny chunk format reader.
+     * @param row Binary row.
+     * @param offset Offset.
+     * @param nullMapSize Default null-map size.
+     * @param chunkFlags Chunk flags.
+     * @return Reader.
      */
-    private static class TinyChunkReader extends ChunkReader {
-        /**
-         * @param row Row.
-         * @param baseOff Base offset.
-         * @param nullMapLen Null-map length in bytes.
-         * @param hasVarTable Vartable presence flag.
-         */
-        TinyChunkReader(BinaryRow row, int baseOff, int nullMapLen, boolean hasVarTable) {
-            super(row, baseOff);
-
-            nullMapOff = baseOff + Byte.BYTES;
-            varTableOff = nullMapOff + nullMapLen;
-            dataOff = varTableOff + (hasVarTable ? Byte.BYTES + (row.readByte(varTableOff) & 0xFF) * Byte.BYTES : 0);
-        }
-
-        /** {@inheritDoc} */
-        @Override int chunkLength() {
-            return row.readByte(baseOff) & 0xFF;
-        }
+    static ChunkReader createReader(BinaryRow row, int offset, int nullMapSize, byte chunkFlags) {
+        return fromFlags(chunkFlags).reader(row, offset,
+            (chunkFlags & OMIT_NULL_MAP_FLAG) == 0 ? nullMapSize : 0,
+            (chunkFlags & OMIT_VARTBL_FLAG) == 0);
+    }
 
-        /** {@inheritDoc} */
-        @Override int vartableItems() {
-            return hasVartable() ? (row.readByte(varTableOff) & 0xFF) : 0;
-        }
+    /**
+     * Chunk formatter from given flags.
+     *
+     * @param chunkFlags Chunk specific flags.
+     * @return Chunk formatter.
+     */
+    private static ChunkFormat fromFlags(byte chunkFlags) {
+        final int mode = chunkFlags & FORMAT_CODE_MASK;
 
-        /** {@inheritDoc} */
-        @Override protected int varlenItemOffset(int itemIdx) {
-            return dataOff + (row.readByte(varTableOff + Byte.BYTES + itemIdx * Byte.BYTES) & 0xFF);
+        switch (mode) {
+            case 1:
+                return TINY;
+            case 2:
+                return MEDIUM;
+            default:
+                return LARGE;
         }
     }
 
     /**
-     * Medium chunk format reader.
+     * @param vartableSizeFieldLen Vartalble size field length.
+     * @param vartableItemSize Vartable item size.
      */
-    private static class MediumChunkReader extends ChunkReader {
-        /**
-         * @param row Row.
-         * @param baseOff Base offset.
-         * @param nullMapLen Null-map length in bytes.
-         * @param hasVarTable Vartable presence flag.
-         */
-        MediumChunkReader(BinaryRow row, int baseOff, int nullMapLen, boolean hasVarTable) {
-            super(row, baseOff);
-
-            nullMapOff = baseOff + Short.BYTES;
-            varTableOff = nullMapOff + nullMapLen;
-            dataOff = varTableOff + (hasVarTable ? Short.BYTES + (row.readShort(varTableOff) & 0xFFFF) * Short.BYTES : 0);
-        }
+    public ChunkFormat(int vartableSizeFieldLen, int vartableItemSize, byte modeFlags) {
+        this.vartableItemSize = vartableItemSize;
+        this.vartableSizeFieldLen = vartableSizeFieldLen;
+        this.modeFlags = modeFlags;
+    }
 
-        /** {@inheritDoc} */
-        @Override int chunkLength() {
-            return row.readShort(baseOff) & 0xFF;
-        }
+    int vartableSizeFieldLen() {
+        return vartableSizeFieldLen;
+    }
 
-        /** {@inheritDoc} */
-        @Override int vartableItems() {
-            return hasVartable() ? (row.readShort(varTableOff) & 0xFFFF) : 0;
-        }
+    int vartableItemSize() {
+        return vartableItemSize;
+    }
 
-        /** {@inheritDoc} */
-        @Override protected int varlenItemOffset(int itemIdx) {
-            return dataOff + (row.readShort(varTableOff + Short.BYTES + itemIdx * Short.BYTES) & 0xFFFF);
-        }
+    public byte modeFlags() {
+        return modeFlags;
     }
 
     /**
-     * Large chunk format reader.
+     * @param payloadLen Row payload length in bytes.
+     * @param nullMapLen Null-map length in bytes.
+     * @param vartblItems Number of vartable items.
+     * @return Chunk size.
      */
-    private static class LargeChunkReader extends ChunkReader {
-        /**
-         * @param row Row.
-         * @param baseOff Base offset.
-         * @param nullMapLen Null-map length in bytes.
-         * @param hasVarTable Vartable presence flag.
-         */
-        LargeChunkReader(BinaryRow row, int baseOff, int nullMapLen, boolean hasVarTable) {
-            super(row, baseOff);
-
-            nullMapOff = baseOff + Integer.BYTES;
-            varTableOff = baseOff + Integer.BYTES + nullMapLen;
-            dataOff = varTableOff + (hasVarTable ? Integer.BYTES + row.readInteger(varTableOff) * Integer.BYTES : 0);
-        }
+    int chunkSize(int payloadLen, int nullMapLen, int vartblItems) {
+        return CHUNK_LEN_FLD_SIZE /* Chunk len. */ + nullMapLen + vartableLength(vartblItems) + payloadLen;
+    }
 
-        /** {@inheritDoc} */
-        @Override public int chunkLength() {
-            return row.readInteger(baseOff);
-        }
+    /**
+     * Calculates vartable length (in bytes).
+     *
+     * @param items Vartable items.
+     * @return Vartable size in bytes.
+     */
+    protected int vartableLength(int items) {
+        return items == 0 ? 0 : vartableSizeFieldLen /* Table size */ + items * vartableItemSize;
+    }
 
-        /** {@inheritDoc} */
-        @Override int vartableItems() {
-            return hasVartable() ? row.readInteger(varTableOff) : 0;
-        }
+    /**
+     * Calculates vartable item offset.
+     *
+     * @param idx Vartable item idx.
+     * @return Vartable item offset.
+     */
+    int vartblItemOff(int idx) {
+        return vartableSizeFieldLen /* Table size */ + idx * vartableItemSize;
+    }
 
-        /** {@inheritDoc} */
-        @Override protected int varlenItemOffset(int itemIdx) {
-            return dataOff + row.readInteger(varTableOff + Integer.BYTES + itemIdx * Integer.BYTES);
-        }
+    abstract void writeOffset(ExpandableByteBuf buf, int vartblItemOff, int off);
+
+    abstract int readOffset(BinaryRow row, int vartblOff);
+
+    abstract void writeVartblSize(ExpandableByteBuf buf, int vartblOff, int size);
+
+    abstract int readVartblSize(BinaryRow row, int vartblOff);
+
+    /**
+     * Chunk writer factory method.
+     *
+     * @param buf Row buffer.
+     * @param baseOff Chunk base offset.
+     * @param nullMapLen Null-map length.
+     * @param vartblItems Vartable items.
+     * @return Chunk writer.
+     */
+    ChunkWriter writer(ExpandableByteBuf buf, int baseOff, int nullMapLen, int vartblItems) {
+        return new ChunkWriter(buf, baseOff, nullMapLen, vartblItems, this);
+    }
+
+    /**
+     * Chunk reader factory method.
+     *
+     * @param row Row buffer.
+     * @param baseOff Chunk base offset.
+     * @param nullMapLen Null-map length.
+     * @param hasVarTable Has vartable flag.
+     * @return Chunk reader.
+     */
+    ChunkReader reader(BinaryRow row, int baseOff, int nullMapLen, boolean hasVarTable) {
+        return new ChunkReader(row, baseOff, nullMapLen, hasVarTable, this);
     }
-}
+}
\ No newline at end of file
diff --git a/modules/schema/src/main/java/org/apache/ignite/internal/schema/row/ChunkReader.java b/modules/schema/src/main/java/org/apache/ignite/internal/schema/row/ChunkReader.java
index fd02415..30e4db9 100644
--- a/modules/schema/src/main/java/org/apache/ignite/internal/schema/row/ChunkReader.java
+++ b/modules/schema/src/main/java/org/apache/ignite/internal/schema/row/ChunkReader.java
@@ -23,15 +23,15 @@ import org.apache.ignite.internal.schema.Columns;
 /**
  * Abstract chunk reader.
  */
-abstract class ChunkReader {
+class ChunkReader {
     /** Row. */
     protected final BinaryRow row;
 
     /** Base offset. */
     protected final int baseOff;
 
-    /** Null-map offset. */
-    protected int nullMapOff;
+    /** Chunk format. */
+    private final ChunkFormat format;
 
     /** Vartable offset. */
     protected int varTableOff;
@@ -40,22 +40,34 @@ abstract class ChunkReader {
     protected int dataOff;
 
     /**
-     * @param baseOff Chunk base offset.
+     * @param row
+     * @param baseOff
+     * @param nullMapLen
+     * @param hasVarTable
+     * @param format
      */
-    ChunkReader(BinaryRow row, int baseOff) {
+    ChunkReader(BinaryRow row, int baseOff, int nullMapLen, boolean hasVarTable, ChunkFormat format) {
         this.row = row;
         this.baseOff = baseOff;
+        this.format = format;
+        varTableOff = nullmapOff() + nullMapLen;
+        dataOff = varTableOff + (hasVarTable ? format.vartableLength(format.readVartblSize(row, varTableOff)) : 0);
     }
 
     /**
      * @return Chunk length in bytes
      */
-    abstract int chunkLength();
+    /** {@inheritDoc} */
+    int chunkLength() {
+        return row.readInteger(baseOff);
+    }
 
     /**
      * @return Number of items in vartable.
      */
-    abstract int vartableItems();
+    int vartableItems() {
+       return format.readVartblSize(row, varTableOff);
+    }
 
     /**
      * Checks the row's null map for the given column index in the chunk.
@@ -71,11 +83,15 @@ abstract class ChunkReader {
         int nullByte = idx / 8;
         int posInByte = idx % 8;
 
-        int map = row.readByte(nullMapOff + nullByte);
+        int map = row.readByte(nullmapOff() + nullByte);
 
         return (map & (1 << posInByte)) != 0;
     }
 
+    private int nullmapOff() {
+        return baseOff + Integer.BYTES;
+    }
+
     /**
      * @return {@code True} if chunk has vartable.
      */
@@ -87,14 +103,20 @@ abstract class ChunkReader {
      * @return {@code True} if chunk has nullmap.
      */
     protected boolean hasNullmap() {
-        return varTableOff > nullMapOff;
+        return varTableOff > nullmapOff();
     }
 
     /**
      * @param itemIdx Varlen table item index.
      * @return Varlen item offset.
      */
-    protected abstract int varlenItemOffset(int itemIdx);
+    protected int varlenItemOffset(int itemIdx) {
+        assert hasVartable() : "Vartable is ommited.";
+
+        final int off = format.vartblItemOff(itemIdx);
+
+        return format.readOffset(row, off);
+    }
 
     /**
      * Calculates the offset of the fixlen column with the given index in the row. It essentially folds the null map
@@ -119,9 +141,9 @@ abstract class ChunkReader {
             if (hasNullmap()) {
                 // Fold offset based on the whole map bytes in the schema
                 for (int i = 0; i < colByteIdx; i++)
-                    colOff += cols.foldFixedLength(i, row.readByte(nullMapOff + i));
+                    colOff += cols.foldFixedLength(i, row.readByte(nullmapOff() + i));
 
-                colOff += cols.foldFixedLength(colByteIdx, row.readByte(nullMapOff + colByteIdx) | mask);
+                colOff += cols.foldFixedLength(colByteIdx, row.readByte(nullmapOff() + colByteIdx) | mask);
             }
             else {
                 for (int i = 0; i < colByteIdx; i++)
@@ -158,7 +180,7 @@ abstract class ChunkReader {
             int numNullsBefore = 0;
 
             for (int i = nullStartByte; i <= nullEndByte; i++) {
-                byte nullmapByte = row.readByte(nullMapOff + i);
+                byte nullmapByte = row.readByte(nullmapOff() + i);
 
                 if (i == nullStartByte)
                     // We need to clear startBitInByte least significant bits
diff --git a/modules/schema/src/main/java/org/apache/ignite/internal/schema/row/ChunkWriter.java b/modules/schema/src/main/java/org/apache/ignite/internal/schema/row/ChunkWriter.java
index cb49e72..dc65302 100644
--- a/modules/schema/src/main/java/org/apache/ignite/internal/schema/row/ChunkWriter.java
+++ b/modules/schema/src/main/java/org/apache/ignite/internal/schema/row/ChunkWriter.java
@@ -27,16 +27,13 @@ import org.apache.ignite.internal.schema.NativeTypes;
 /**
  * Abstract row chunk writer.
  */
-abstract class ChunkWriter {
+class ChunkWriter {
     /** Chunk buffer. */
     protected final ExpandableByteBuf buf;
 
     /** Base offset of the chunk */
     protected final int baseOff;
 
-    /** Offset of the null map for the chunk. */
-    protected final int nullMapOff;
-
     /** Offset of the varlen table for the chunk. */
     protected final int varTblOff;
 
@@ -49,20 +46,34 @@ abstract class ChunkWriter {
     /** Current offset for the next column to be appended. */
     protected int curOff;
 
+    /** Chunk flags. */
+    private byte flags;
+
+    /** Chunk format. */
+    ChunkFormat format;
+
     /**
      * @param buf Row buffer.
      * @param baseOff Chunk base offset.
-     * @param nullMapOff Null-map offset.
-     * @param varTblOff Vartable offset.
+     * @param nullMapLen Null-map length in bytes.
+     * @param vartblItems Vartable length in bytes.
      */
-    protected ChunkWriter(ExpandableByteBuf buf, int baseOff, int nullMapOff, int varTblOff, int dataOff) {
+    protected ChunkWriter(ExpandableByteBuf buf, int baseOff, int nullMapLen, int vartblItems, ChunkFormat format) {
         this.buf = buf;
         this.baseOff = baseOff;
-        this.nullMapOff = nullMapOff;
-        this.varTblOff = varTblOff;
-        this.dataOff = dataOff;
+        this.format = format;
+        this.flags = format.modeFlags();
+
+        varTblOff = nullmapOff() + nullMapLen;
+        dataOff = varTblOff + format.vartableLength(vartblItems);
         curOff = dataOff;
         curVartblItem = 0;
+
+        if (nullmapOff() == baseOff)
+            this.flags |= ChunkFormat.OMIT_NULL_MAP_FLAG;
+
+        if (dataOff == varTblOff)
+            this.flags |= ChunkFormat.OMIT_VARTBL_FLAG;
     }
 
     /**
@@ -149,6 +160,9 @@ abstract class ChunkWriter {
      * @param val Column value.
      */
     public void appendString(String val, CharsetEncoder encoder) {
+        assert (flags & ChunkFormat.OMIT_VARTBL_FLAG) == 0 :
+            "Illegal writing of varlen when 'omit vartable' flag is set for a chunk.";
+
         try {
             int written = buf.putString(curOff, val, encoder);
 
@@ -168,6 +182,9 @@ abstract class ChunkWriter {
      * @param val Column value.
      */
     public void appendBytes(byte[] val) {
+        assert (flags & ChunkFormat.OMIT_VARTBL_FLAG) == 0 :
+            "Illegal writing of varlen when 'omit vartable' flag is set for a chunk.";
+
         buf.putBytes(curOff, val);
 
         writeOffset(curVartblItem, curOff - dataOff);
@@ -202,15 +219,28 @@ abstract class ChunkWriter {
     /**
      * Post-write action.
      */
-    abstract void flush();
+    void flush() {
+        final int size = chunkLength();
+
+        buf.putInt(baseOff, size);
+
+        if (curVartblItem > 0)
+            format.writeVartblSize(buf, varTblOff, curVartblItem);
+    }
 
     /**
      * Writes the given offset to the varlen table entry with the given index.
      *
-     * @param tblEntryIdx Varlen table entry index.
+     * @param tblItemIdx Varlen table entry index.
      * @param off Offset to write.
      */
-    protected abstract void writeOffset(int tblEntryIdx, int off);
+    protected void writeOffset(int tblItemIdx, int off) {
+        final int itemOff = varTblOff + format.vartblItemOff(tblItemIdx);
+
+        assert itemOff < dataOff : "Vartable overflow: size=" + itemOff;
+
+        format.writeOffset(buf, itemOff, off);
+    }
 
     /**
      * Sets null flag in the null map for the given column.
@@ -218,13 +248,21 @@ abstract class ChunkWriter {
      * @param colIdx Column index.
      */
     protected void setNull(int colIdx) {
-        assert nullMapOff < varTblOff : "Null map is omitted.";
+        assert (flags & ChunkFormat.OMIT_NULL_MAP_FLAG) == 0 : "Null map is omitted.";
 
         int byteInMap = colIdx / 8;
         int bitInByte = colIdx % 8;
 
-        buf.ensureCapacity(nullMapOff + byteInMap + 1);
+        buf.ensureCapacity(nullmapOff() + byteInMap + 1);
+
+        buf.put(nullmapOff() + byteInMap, (byte)(buf.get(nullmapOff() + byteInMap) | (1 << bitInByte)));
+    }
+
+    private int nullmapOff() {
+        return baseOff + Integer.BYTES;
+    }
 
-        buf.put(nullMapOff + byteInMap, (byte)(buf.get(nullMapOff + byteInMap) | (1 << bitInByte)));
+    public short flags() {
+        return flags;
     }
 }
diff --git a/modules/schema/src/main/java/org/apache/ignite/internal/schema/row/LongChunkWriter.java b/modules/schema/src/main/java/org/apache/ignite/internal/schema/row/LongChunkWriter.java
deleted file mode 100644
index a7e94a2..0000000
--- a/modules/schema/src/main/java/org/apache/ignite/internal/schema/row/LongChunkWriter.java
+++ /dev/null
@@ -1,90 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.ignite.internal.schema.row;
-
-/**
- * Row chunk writer for long key/value chunks.
- *
- * Uses {@code int} values for coding sizes/offsets,
- * supports chunks with payload up to 2 GiB.
- */
-class LongChunkWriter extends ChunkWriter {
-    /**
-     * Calculates vartable length (in bytes).
-     *
-     * @param items Vartable items.
-     * @return Vartable size in bytes.
-     */
-    static int vartableLength(int items) {
-        return items == 0 ? 0 : Integer.BYTES /* Table size */ + items * Integer.BYTES;
-    }
-
-    /**
-     * Calculates chunk size.
-     *
-     * @param payloadLen Payload size in bytes.
-     * @param nullMapLen Null-map size in bytes.
-     * @param vartblSize Amount of vartable items.
-     * @return Bytes required to write a chunk or {@code -1} if a chunk is too long.
-     */
-    static int chunkSize(int payloadLen, int nullMapLen, int vartblSize) {
-        return Integer.BYTES /* Chunk len. */ + nullMapLen + vartableLength(vartblSize) + payloadLen;
-    }
-
-    /**
-     * Creates chunk writer for long chunk format.
-     *
-     * @param buf Row buffer.
-     * @param baseOff Chunk base offset.
-     * @param nullMapLen Null-map size in bytes.
-     * @param vartblSize Amount of vartable items.
-     */
-    LongChunkWriter(ExpandableByteBuf buf, int baseOff, int nullMapLen, int vartblSize) {
-        super(
-            buf,
-            baseOff,
-            baseOff + Integer.BYTES /* Chunk size */,
-            baseOff + Integer.BYTES /* Chunk size */ + nullMapLen,
-            baseOff + Integer.BYTES /* Chunk size */ + nullMapLen + vartableLength(vartblSize));
-
-        curVartblItem = 0;
-    }
-
-    /** {@inheritDoc} */
-    @Override void flush() {
-        final int size = chunkLength();
-
-        assert size > 0 : "Size field value overflow: " + size;
-        assert varTblOff + vartableLength(curVartblItem) == dataOff : "Vartable underflowed.";
-
-        buf.putInt(baseOff, size);
-
-        if (curVartblItem > 0)
-            buf.putInt(varTblOff, curVartblItem);
-    }
-
-    /** {@inheritDoc} */
-    @Override protected void writeOffset(int tblEntryIdx, int off) {
-        final int itemOff = varTblOff + Integer.BYTES + tblEntryIdx * Integer.BYTES;
-
-        assert off >= 0 : "Varlen offset overflow: offset=" + off;
-        assert itemOff < dataOff : "Vartable overflow: size=" + itemOff;
-
-        buf.putInt(itemOff, off);
-    }
-}
diff --git a/modules/schema/src/main/java/org/apache/ignite/internal/schema/row/MeduimChunkWriter.java b/modules/schema/src/main/java/org/apache/ignite/internal/schema/row/MeduimChunkWriter.java
deleted file mode 100644
index 172dae1..0000000
--- a/modules/schema/src/main/java/org/apache/ignite/internal/schema/row/MeduimChunkWriter.java
+++ /dev/null
@@ -1,97 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.ignite.internal.schema.row;
-
-/**
- * Row chunk writer for small key/value chunks.
- *
- * Uses {@code byte} values for coding sizes/offsets,
- * supports chunks with payload less upt to 255 bytes.
- */
-class MeduimChunkWriter extends ChunkWriter {
-    /**
-     * Calculates vartable length (in bytes).
-     *
-     * @param items Vartable items.
-     * @return Vartable size in bytes.
-     */
-    static int vartableLength(int items) {
-        return items == 0 ? 0 : Short.BYTES /* Table size */ + items * Short.BYTES;
-    }
-
-    /**
-     * Calculates chunk size.
-     *
-     * @param payloadLen Payload size in bytes.
-     * @param nullMapLen Null-map size in bytes.
-     * @param vartblSize Amount of vartable items.
-     * @return Bytes required to write a chunk or {@code -1} if a chunk is too long.
-     */
-    static int chunkSize(int payloadLen, int nullMapLen, int vartblSize) {
-        return Short.BYTES /* Chunk len. */ + nullMapLen + vartableLength(vartblSize) + payloadLen;
-    }
-
-    /**
-     * Check if chunk fits to max size.
-     *
-     * @param payloadLen Payload size in bytes.
-     * @param nullMapLen Null-map size in bytes.
-     * @param vartblSize Amount of vartable items.
-     * @return {@code true} if a chunk is tiny, {@code false} otherwise.
-     */
-    static boolean isMediumChunk(int payloadLen, int nullMapLen, int vartblSize) {
-        return chunkSize(payloadLen, nullMapLen, vartblSize) < 64 * 1024;
-    }
-
-    /**
-     * Creates chunk writer to write chunk in tiny format.
-     *
-     * @param buf Row buffer.
-     * @param baseOff Chunk base offset.
-     * @param nullMapLen Null-map size in bytes.
-     * @param vartblSize Amount of vartable items.
-     */
-    MeduimChunkWriter(ExpandableByteBuf buf, int baseOff, int nullMapOff, int varTblOff, int dataOff) {
-        super(buf,baseOff, nullMapOff, varTblOff, dataOff);
-
-
-        curVartblItem = 0;
-    }
-
-    /** {@inheritDoc} */
-    @Override void flush() {
-        final int size = chunkLength();
-
-        assert size < (2 << 16) && size > 0 : "Size field value overflow: " + size;
-
-        buf.putShort(baseOff, (short)size);
-
-        if (curVartblItem > 0)
-            buf.putShort(varTblOff, (short)curVartblItem);
-    }
-
-    /** {@inheritDoc} */
-    @Override protected void writeOffset(int tblEntryIdx, int off) {
-        final int itemOff = varTblOff + Short.BYTES + tblEntryIdx * Short.BYTES;
-
-        assert off < (2 << 8) && off >= 0 : "Varlen offset overflow: offset=" + off;
-        assert itemOff < dataOff : "Vartable overflow: size=" + itemOff;
-
-        buf.putShort(itemOff, (short)off);
-    }
-}
diff --git a/modules/schema/src/main/java/org/apache/ignite/internal/schema/row/Row.java b/modules/schema/src/main/java/org/apache/ignite/internal/schema/row/Row.java
index ce80945..c3dd7f7 100644
--- a/modules/schema/src/main/java/org/apache/ignite/internal/schema/row/Row.java
+++ b/modules/schema/src/main/java/org/apache/ignite/internal/schema/row/Row.java
@@ -29,7 +29,6 @@ import org.apache.ignite.internal.schema.Columns;
 import org.apache.ignite.internal.schema.InvalidTypeException;
 import org.apache.ignite.internal.schema.NativeTypeSpec;
 import org.apache.ignite.internal.schema.SchemaDescriptor;
-import org.jetbrains.annotations.NotNull;
 
 /**
  * Schema-aware row.
@@ -49,7 +48,7 @@ public class Row implements BinaryRow {
     private final ChunkReader keyReader;
 
     /** Value reader. */
-    private final ChunkReader valReader;
+    private ChunkReader valReader;
 
     /**
      * Constructor.
@@ -65,21 +64,13 @@ public class Row implements BinaryRow {
 
         final short flags = readShort(FLAGS_FIELD_OFFSET);
 
-        keyReader = createReader(KEY_CHUNK_OFFSET,
-            (flags & RowFlags.KEY_TYNY_FORMAT) != 0,
-            (flags & RowFlags.OMIT_KEY_NULL_MAP_FLAG) == 0 ? schema.keyColumns().nullMapSize() : 0,
-            (flags & RowFlags.OMIT_KEY_VARTBL_FLAG) == 0);
-
-        valReader = ((flags & RowFlags.NO_VALUE_FLAG) == 0) ?
-            createReader(
-                KEY_CHUNK_OFFSET + keyReader.chunkLength(),
-                (flags & RowFlags.VAL_TYNY_FORMAT) != 0,
-                (flags & RowFlags.OMIT_VAL_NULL_MAP_FLAG) == 0 ? schema.valueColumns().nullMapSize() : 0,
-                (flags & RowFlags.OMIT_VAL_VARTBL_FLAG) == 0) :
-            null;
+        keyReader = ChunkFormat.createReader(
+            this,
+            KEY_CHUNK_OFFSET,
+            schema.keyColumns().nullMapSize(),
+            (byte)((flags >>> RowFlags.KEY_FLAGS_OFFSET) & RowFlags.CHUNK_FLAGS_MASK));
     }
 
-
     /**
      * @return Row schema.
      */
@@ -370,7 +361,7 @@ public class Row implements BinaryRow {
         if (!isKeyCol)
             colIdx -= schema.keyColumns().length();
 
-        ChunkReader reader = isKeyCol ? keyReader : valReader;
+        ChunkReader reader = isKeyCol ? keyReader : valueReader();
         Columns cols = isKeyCol ? schema.keyColumns() : schema.valueColumns();
 
         if (cols.column(colIdx).type().spec() != type)
@@ -389,6 +380,21 @@ public class Row implements BinaryRow {
             reader.varlenColumnOffsetAndLength(cols, colIdx);
     }
 
+    private ChunkReader valueReader() {
+        if (valReader != null)
+            return valReader;
+
+        final short flags = readShort(FLAGS_FIELD_OFFSET);
+
+        assert (flags & RowFlags.NO_VALUE_FLAG) == 0 : "Row has no value.";
+
+        return (valReader = ChunkFormat.createReader(
+            this,
+            KEY_CHUNK_OFFSET + keyReader.chunkLength(),
+            schema.keyColumns().nullMapSize(),
+            (byte)(flags >>> RowFlags.VAL_FLAGS_OFFSET)));
+    }
+
     /**
      * @param colIdx Column index.
      * @return Column length.
diff --git a/modules/schema/src/main/java/org/apache/ignite/internal/schema/row/RowAssembler.java b/modules/schema/src/main/java/org/apache/ignite/internal/schema/row/RowAssembler.java
index 0d43da4..c245d9f 100644
--- a/modules/schema/src/main/java/org/apache/ignite/internal/schema/row/RowAssembler.java
+++ b/modules/schema/src/main/java/org/apache/ignite/internal/schema/row/RowAssembler.java
@@ -35,8 +35,8 @@ import org.apache.ignite.internal.schema.SchemaDescriptor;
 
 import static org.apache.ignite.internal.schema.BinaryRow.KEY_CHUNK_OFFSET;
 import static org.apache.ignite.internal.schema.BinaryRow.KEY_HASH_FIELD_OFFSET;
-import static org.apache.ignite.internal.schema.BinaryRow.RowFlags.OMIT_KEY_VARTBL_FLAG;
-import static org.apache.ignite.internal.schema.BinaryRow.RowFlags.OMIT_VAL_VARTBL_FLAG;
+import static org.apache.ignite.internal.schema.BinaryRow.RowFlags.KEY_FLAGS_OFFSET;
+import static org.apache.ignite.internal.schema.BinaryRow.RowFlags.VAL_FLAGS_OFFSET;
 
 /**
  * Utility class to build rows using column appending pattern. The external user of this class must consult
@@ -189,16 +189,8 @@ public class RowAssembler {
         final int keyNullMapSize = keyHasNulls ? schema.keyColumns().nullMapSize() : 0;
         final int valNullMapSize = valHasNulls ? schema.valueColumns().nullMapSize() : 0;
 
-        final ChunkFormat keyWriteMode = ChunkFormat.writeMode(keyDataSize, keyNullMapSize, keyVarlenCols);
-        valWriteMode = ChunkFormat.writeMode(valDataSize, keyNullMapSize, valVarlenCols);
-
-        // Key flags.
-        if (schema.keyColumns().nullMapSize() == 0)
-            flags |= RowFlags.OMIT_KEY_NULL_MAP_FLAG;
-        if (keyVarlenCols == 0)
-            flags |= OMIT_KEY_VARTBL_FLAG;
-
-        flags |= keyWriteMode.modeFlags() & 0x0F << 8;
+        final ChunkFormat keyWriteMode = ChunkFormat.formatter(keyDataSize);
+        valWriteMode = ChunkFormat.formatter(valDataSize);
 
         int size = BinaryRow.HEADER_SIZE +
             keyWriteMode.chunkSize(keyDataSize, keyNullMapSize, keyDataSize) +
@@ -348,9 +340,6 @@ public class RowAssembler {
     public void appendString(String val) {
         checkType(NativeTypes.STRING);
 
-        assert (flags & (schema.keyColumns() == curCols ? OMIT_KEY_VARTBL_FLAG : OMIT_VAL_VARTBL_FLAG)) == 0 :
-            "Illegal writing of varlen when 'omit vartable' flag is set for a chunk.";
-
         if (isKeyColumn())
             hash = 31 * hash + val.hashCode();
 
@@ -367,9 +356,6 @@ public class RowAssembler {
     public void appendBytes(byte[] val) {
         checkType(NativeTypes.BYTES);
 
-        assert (flags & (schema.keyColumns() == curCols ? OMIT_KEY_VARTBL_FLAG : OMIT_VAL_VARTBL_FLAG)) == 0 :
-            "Illegal writing of varlen when 'omit vartable' flag is set for a chunk.";
-
         if (isKeyColumn())
             hash = 31 * hash + Arrays.hashCode(val);
 
@@ -421,7 +407,7 @@ public class RowAssembler {
         return buf.toArray();
     }
 
-    private boolean isKeyColumn () {
+    private boolean isKeyColumn() {
         return curCols == schema.keyColumns();
     }
 
@@ -468,22 +454,17 @@ public class RowAssembler {
             // Write sizes.
             chunkWriter.flush();
 
-            if (schema.valueColumns() == curCols)
+            if (schema.valueColumns() == curCols) {
+                flags |= (chunkWriter.flags() & 0x0F) << VAL_FLAGS_OFFSET;
+
                 return; // No more columns.
+            }
 
             // Switch key->value columns.
             curCols = schema.valueColumns();
             curCol = 0;
 
-            // Write value flags.
-            if (valVarlenCols == 0)
-                flags |= OMIT_VAL_VARTBL_FLAG;
-            else {
-                if (schema.valueColumns().nullMapSize() == 0)
-                    flags |= RowFlags.OMIT_VAL_NULL_MAP_FLAG;
-
-                flags |= valWriteMode.modeFlags() & 0x0F << 14;
-            }
+            flags |= (chunkWriter.flags() & 0x0F) << KEY_FLAGS_OFFSET;
 
             // Create value chunk writer.
             chunkWriter = valWriteMode.writer(buf,
diff --git a/modules/schema/src/main/java/org/apache/ignite/internal/schema/row/TinyChunkWriter.java b/modules/schema/src/main/java/org/apache/ignite/internal/schema/row/TinyChunkWriter.java
deleted file mode 100644
index 25392a9..0000000
--- a/modules/schema/src/main/java/org/apache/ignite/internal/schema/row/TinyChunkWriter.java
+++ /dev/null
@@ -1,62 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.ignite.internal.schema.row;
-
-/**
- * Row chunk writer for small key/value chunks.
- * <p>
- * Uses {@code byte} values for coding sizes/offsets,
- * supports chunks with payload less upt to 255 bytes.
- */
-class TinyChunkWriter extends ChunkWriter {
-    /**
-     * Creates chunk writer to write chunk in tiny format.
-     *
-     * @param buf Row buffer.
-     * @param baseOff Chunk base offset.
-     * @param nullMapOff Null-map offset.
-     * @param varTblOff Vartable offset.
-     */
-    TinyChunkWriter(ExpandableByteBuf buf, int baseOff, int nullMapOff, int varTblOff, int dataOff) {
-        super(buf,baseOff, nullMapOff, varTblOff, dataOff);
-
-        curVartblItem = 0;
-    }
-
-    /** {@inheritDoc} */
-    @Override void flush() {
-        final int size = chunkLength();
-
-        assert size < (2 << 8) && size > 0 : "Size field value overflow: " + size;
-
-        buf.put(baseOff, (byte)size);
-
-        if (curVartblItem > 0)
-            buf.put(varTblOff, (byte)curVartblItem);
-    }
-
-    /** {@inheritDoc} */
-    @Override protected void writeOffset(int tblEntryIdx, int off) {
-        final int itemOff = varTblOff + Byte.BYTES + tblEntryIdx * Byte.BYTES;
-
-        assert off < (2 << 8) && off >= 0 : "Varlen offset overflow: offset=" + off;
-        assert itemOff < dataOff : "Vartable overflow: size=" + itemOff;
-
-        buf.put(itemOff, (byte)off);
-    }
-}