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:22 UTC

[ignite-3] 10/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 d9d414d22871d652cf746c40b6055e077cf35357
Author: Andrew Mashenkov <an...@gmail.com>
AuthorDate: Tue Jun 8 16:56:48 2021 +0300

    WIP.
---
 .../apache/ignite/internal/schema/BinaryRow.java   |   7 +-
 .../ignite/internal/schema/row/ChunkFormat.java    | 289 +++++++++++++++++++++
 .../ignite/internal/schema/row/ChunkReader.java    | 190 ++++++++++++++
 ...TinyChunkWriter.java => MeduimChunkWriter.java} |  30 +--
 .../org/apache/ignite/internal/schema/row/Row.java | 250 +-----------------
 .../ignite/internal/schema/row/RowAssembler.java   |  62 ++---
 .../internal/schema/row/TinyChunkWriter.java       |  49 +---
 .../ignite/internal/schema/row/XXHash32.java       | 243 -----------------
 8 files changed, 522 insertions(+), 598 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 83a73e0..9f13385 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
@@ -157,12 +157,13 @@ public interface BinaryRow {
         /** Flag indicates value chunk omits varlen table. */
         public static final int OMIT_VAL_VARTBL_FLAG = 1 << 4;
 
-        /** Flag indicates key chunk is written in Tiny format. */
-        public static final int KEY_TYNY_FORMAT = 1 << 5;
-
         /** 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;
+
         /** Stub. */
         private RowFlags() {
             // No-op.
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
new file mode 100644
index 0000000..d5e8d17
--- /dev/null
+++ b/modules/schema/src/main/java/org/apache/ignite/internal/schema/row/ChunkFormat.java
@@ -0,0 +1,289 @@
+/*
+ * 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;
+
+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();
+
+    /** Writer factory for med-sized chunks. */
+    private static final ChunkFormat MEDIUM = new MedChunkFormat();
+
+    /** Writer factory for large-sized chunks. */
+    private static final ChunkFormat LARGE = new LargeChunkFormat();
+
+    /**
+     * 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;
+    }
+
+    /**
+     * @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);
+
+    /**
+     * Returns mode flags. First 4-bits are used.
+     *
+     * @return Chunk specific flags.
+     */
+    byte modeFlags() {
+        return 0;
+    }
+
+    /**
+     * Calculates vartable length (in bytes).
+     *
+     * @param items Vartable items.
+     * @return Vartable size in bytes.
+     */
+    protected abstract int vartableLength(int items);
+
+    /**
+     * 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);
+
+    abstract ChunkReader reader(BinaryRow row, int baseOff, int nullMapLen, boolean hasVarTable);
+
+    /**
+     * Writer factory for tiny-sized chunks.
+     */
+    private static class TinyChunkFormat extends ChunkFormat {
+
+        /** {@inheritDoc} */
+        @Override protected int vartableLength(int items) {
+            return items == 0 ? 0 : Byte.BYTES /* Table size */ + items * Byte.BYTES;
+        }
+
+        /** {@inheritDoc} */
+        @Override int chunkSize(int payloadLen, int nullMapLen, int vartblItems) {
+            return Byte.BYTES /* Chunk len. */ + nullMapLen + vartableLength(vartblItems) + payloadLen;
+        }
+
+        /** {@inheritDoc} */
+        @Override byte modeFlags() {
+            return 1;
+        }
+
+        /** {@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));
+        }
+
+        /** {@inheritDoc} */
+        @Override ChunkReader reader(BinaryRow row,int baseOff, int nullMapLen, boolean hasVarTable) {
+            return new TinyChunkReader(row, baseOff,  nullMapLen, hasVarTable);
+        }
+    }
+
+    /**
+     * 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;
+        }
+
+        /** {@inheritDoc} */
+        @Override int chunkSize(int payloadLen, int nullMapLen, int vartblItems) {
+            return Short.BYTES /* Chunk len. */ + nullMapLen + vartableLength(vartblItems) + payloadLen;
+        }
+
+        /** {@inheritDoc} */
+        @Override byte modeFlags() {
+            return 2;
+        }
+
+        /** {@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));
+        }
+
+        /** {@inheritDoc} */
+        @Override ChunkReader reader(BinaryRow row,int baseOff, int nullMapLen, boolean hasVarTable) {
+            return new MediumChunkReader(row, baseOff,  nullMapLen, hasVarTable);
+        }
+    }
+
+    /**
+     * Writer factory for large-sized chunks.
+     */
+    private static class LargeChunkFormat extends ChunkFormat {
+        /** {@inheritDoc} */
+        @Override protected int vartableLength(int items) {
+            return items == 0 ? 0 : Integer.BYTES /* Table size */ + items * Integer.BYTES;
+        }
+
+        /** {@inheritDoc} */
+        @Override int chunkSize(int payloadLen, int nullMapLen, int vartblItems) {
+            return Integer.BYTES /* Chunk len. */ + nullMapLen + vartableLength(vartblItems) + payloadLen;
+        }
+
+        /** {@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);
+        }
+    }
+
+    /**
+     * Tiny chunk format 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;
+        }
+
+        /** {@inheritDoc} */
+        @Override int vartableItems() {
+            return hasVartable() ? (row.readByte(varTableOff) & 0xFF) : 0;
+        }
+
+        /** {@inheritDoc} */
+        @Override protected int varlenItemOffset(int itemIdx) {
+            return dataOff + (row.readByte(varTableOff + Byte.BYTES + itemIdx * Byte.BYTES) & 0xFF);
+        }
+    }
+
+    /**
+     * Medium chunk format reader.
+     */
+    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);
+        }
+
+        /** {@inheritDoc} */
+        @Override int chunkLength() {
+            return row.readShort(baseOff) & 0xFF;
+        }
+
+        /** {@inheritDoc} */
+        @Override int vartableItems() {
+            return hasVartable() ? (row.readShort(varTableOff) & 0xFFFF) : 0;
+        }
+
+        /** {@inheritDoc} */
+        @Override protected int varlenItemOffset(int itemIdx) {
+            return dataOff + (row.readShort(varTableOff + Short.BYTES + itemIdx * Short.BYTES) & 0xFFFF);
+        }
+    }
+
+    /**
+     * Large chunk format reader.
+     */
+    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);
+        }
+
+        /** {@inheritDoc} */
+        @Override public int chunkLength() {
+            return row.readInteger(baseOff);
+        }
+
+        /** {@inheritDoc} */
+        @Override int vartableItems() {
+            return hasVartable() ? row.readInteger(varTableOff) : 0;
+        }
+
+        /** {@inheritDoc} */
+        @Override protected int varlenItemOffset(int itemIdx) {
+            return dataOff + row.readInteger(varTableOff + Integer.BYTES + itemIdx * Integer.BYTES);
+        }
+    }
+}
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
new file mode 100644
index 0000000..fd02415
--- /dev/null
+++ b/modules/schema/src/main/java/org/apache/ignite/internal/schema/row/ChunkReader.java
@@ -0,0 +1,190 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.schema.row;
+
+import org.apache.ignite.internal.schema.BinaryRow;
+import org.apache.ignite.internal.schema.Columns;
+
+/**
+ * Abstract chunk reader.
+ */
+abstract class ChunkReader {
+    /** Row. */
+    protected final BinaryRow row;
+
+    /** Base offset. */
+    protected final int baseOff;
+
+    /** Null-map offset. */
+    protected int nullMapOff;
+
+    /** Vartable offset. */
+    protected int varTableOff;
+
+    /** Payload offset. */
+    protected int dataOff;
+
+    /**
+     * @param baseOff Chunk base offset.
+     */
+    ChunkReader(BinaryRow row, int baseOff) {
+        this.row = row;
+        this.baseOff = baseOff;
+    }
+
+    /**
+     * @return Chunk length in bytes
+     */
+    abstract int chunkLength();
+
+    /**
+     * @return Number of items in vartable.
+     */
+    abstract int vartableItems();
+
+    /**
+     * Checks the row's null map for the given column index in the chunk.
+     *
+     * @param idx Offset of the column in the chunk.
+     * @return {@code true} if the column value is {@code null}.
+     */
+    /** {@inheritDoc} */
+    protected boolean isNull(int idx) {
+        if (!hasNullmap())
+            return false;
+
+        int nullByte = idx / 8;
+        int posInByte = idx % 8;
+
+        int map = row.readByte(nullMapOff + nullByte);
+
+        return (map & (1 << posInByte)) != 0;
+    }
+
+    /**
+     * @return {@code True} if chunk has vartable.
+     */
+    protected boolean hasVartable() {
+        return dataOff > varTableOff;
+    }
+
+    /**
+     * @return {@code True} if chunk has nullmap.
+     */
+    protected boolean hasNullmap() {
+        return varTableOff > nullMapOff;
+    }
+
+    /**
+     * @param itemIdx Varlen table item index.
+     * @return Varlen item offset.
+     */
+    protected abstract int varlenItemOffset(int itemIdx);
+
+    /**
+     * Calculates the offset of the fixlen column with the given index in the row. It essentially folds the null map
+     * with the column lengths to calculate the size of non-null columns preceding the requested column.
+     *
+     * @param cols Columns chunk.
+     * @param idx Column index in the chunk.
+     * @return Encoded offset (from the row start) of the requested fixlen column.
+     */
+    int fixlenColumnOffset(Columns cols, int idx) {
+        int colOff = 0;
+
+        // Calculate fixlen column offset.
+        {
+            int colByteIdx = idx / 8;
+
+            // 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 = colByteIdx == (cols.length() + 7) / 8 - 1 ? ((cols.numberOfFixsizeColumns() - 1) % 8) : 7;
+            int mask = (0xFF >> (7 - endBit)) & (0xFF << startBit);
+
+            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(colByteIdx, row.readByte(nullMapOff + colByteIdx) | mask);
+            }
+            else {
+                for (int i = 0; i < colByteIdx; i++)
+                    colOff += cols.foldFixedLength(i, 0);
+
+                colOff += cols.foldFixedLength(colByteIdx, mask);
+            }
+        }
+
+        return dataOff + colOff;
+    }
+
+    /**
+     * Calculates the offset and length of varlen column. First, it calculates the number of non-null columns
+     * preceding 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 idx Column index in the chunk.
+     * @return Encoded offset (from the row start) and length of the column with the given index.
+     */
+    long varlenColumnOffsetAndLength(Columns cols, int idx) {
+        assert hasVartable() : "Chunk has no vartable: colId=" + idx;
+
+        if (hasNullmap()) {
+            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++) {
+                byte nullmapByte = row.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 -= numNullsBefore;
+        }
+
+        idx -= cols.numberOfFixsizeColumns();
+
+        // Offset of idx-th column is from base offset.
+        int resOff = varlenItemOffset(idx);
+
+        long len = (idx == vartableItems() - 1) ?
+            // totalLength - columnStartOffset
+            (baseOff + chunkLength()) - resOff :
+            // nextColumnStartOffset - columnStartOffset
+            varlenItemOffset(idx + 1) - resOff;
+
+        return (len << 32) | resOff;
+    }
+}
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/MeduimChunkWriter.java
similarity index 72%
copy from modules/schema/src/main/java/org/apache/ignite/internal/schema/row/TinyChunkWriter.java
copy to modules/schema/src/main/java/org/apache/ignite/internal/schema/row/MeduimChunkWriter.java
index 347e257..172dae1 100644
--- 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/MeduimChunkWriter.java
@@ -23,7 +23,7 @@ package org.apache.ignite.internal.schema.row;
  * Uses {@code byte} values for coding sizes/offsets,
  * supports chunks with payload less upt to 255 bytes.
  */
-class TinyChunkWriter extends ChunkWriter {
+class MeduimChunkWriter extends ChunkWriter {
     /**
      * Calculates vartable length (in bytes).
      *
@@ -31,7 +31,7 @@ class TinyChunkWriter extends ChunkWriter {
      * @return Vartable size in bytes.
      */
     static int vartableLength(int items) {
-        return items == 0 ? 0 : Byte.BYTES /* Table size */ + items * Byte.BYTES;
+        return items == 0 ? 0 : Short.BYTES /* Table size */ + items * Short.BYTES;
     }
 
     /**
@@ -43,7 +43,7 @@ class TinyChunkWriter extends ChunkWriter {
      * @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 Byte.BYTES /* Chunk len. */ + nullMapLen + vartableLength(vartblSize) + payloadLen;
+        return Short.BYTES /* Chunk len. */ + nullMapLen + vartableLength(vartblSize) + payloadLen;
     }
 
     /**
@@ -54,8 +54,8 @@ class TinyChunkWriter extends ChunkWriter {
      * @param vartblSize Amount of vartable items.
      * @return {@code true} if a chunk is tiny, {@code false} otherwise.
      */
-    static boolean isTinyChunk(int payloadLen, int nullMapLen, int vartblSize) {
-        return chunkSize(payloadLen, nullMapLen, vartblSize) < 256;
+    static boolean isMediumChunk(int payloadLen, int nullMapLen, int vartblSize) {
+        return chunkSize(payloadLen, nullMapLen, vartblSize) < 64 * 1024;
     }
 
     /**
@@ -66,13 +66,9 @@ class TinyChunkWriter extends ChunkWriter {
      * @param nullMapLen Null-map size in bytes.
      * @param vartblSize Amount of vartable items.
      */
-    TinyChunkWriter(ExpandableByteBuf buf, int baseOff, int nullMapLen, int vartblSize) {
-        super(
-            buf,
-            baseOff,
-            baseOff + Byte.BYTES /* Chunk size */,
-            baseOff + Byte.BYTES + nullMapLen,
-            baseOff + Byte.BYTES + nullMapLen + vartableLength(vartblSize));
+    MeduimChunkWriter(ExpandableByteBuf buf, int baseOff, int nullMapOff, int varTblOff, int dataOff) {
+        super(buf,baseOff, nullMapOff, varTblOff, dataOff);
+
 
         curVartblItem = 0;
     }
@@ -81,21 +77,21 @@ class TinyChunkWriter extends ChunkWriter {
     @Override void flush() {
         final int size = chunkLength();
 
-        assert size < (2 << 8) && size > 0 : "Size field value overflow: " + size;
+        assert size < (2 << 16) && size > 0 : "Size field value overflow: " + size;
 
-        buf.put(baseOff, (byte)size);
+        buf.putShort(baseOff, (short)size);
 
         if (curVartblItem > 0)
-            buf.put(varTblOff, (byte)curVartblItem);
+            buf.putShort(varTblOff, (short)curVartblItem);
     }
 
     /** {@inheritDoc} */
     @Override protected void writeOffset(int tblEntryIdx, int off) {
-        final int itemOff = varTblOff + Byte.BYTES + tblEntryIdx * Byte.BYTES;
+        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.put(itemOff, (byte)off);
+        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 15cb69e..ce80945 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
@@ -46,10 +46,10 @@ public class Row implements BinaryRow {
     private final BinaryRow row;
 
     /** Key reader. */
-    private final AbstractChunkReader keyReader;
+    private final ChunkReader keyReader;
 
     /** Value reader. */
-    private final AbstractChunkReader valReader;
+    private final ChunkReader valReader;
 
     /**
      * Constructor.
@@ -79,20 +79,6 @@ public class Row implements BinaryRow {
             null;
     }
 
-    /**
-     * Chunk reader factory method.
-     *
-     * @param baseOff Chunk base offset.
-     * @param isSmallChunk Small chunk format flag.
-     * @param nullMapLen Null-map length.
-     * @param hasVarTable Vartable presense flag.
-     * @return Chunk reader.
-     */
-    @NotNull private AbstractChunkReader createReader(int baseOff, boolean isSmallChunk, int nullMapLen, boolean hasVarTable) {
-        return isSmallChunk ?
-            new TinyChunkReader(baseOff, nullMapLen, hasVarTable) :
-            new LargeChunkReader(baseOff, nullMapLen, hasVarTable);
-    }
 
     /**
      * @return Row schema.
@@ -384,7 +370,7 @@ public class Row implements BinaryRow {
         if (!isKeyCol)
             colIdx -= schema.keyColumns().length();
 
-        AbstractChunkReader reader = isKeyCol ? keyReader : valReader;
+        ChunkReader reader = isKeyCol ? keyReader : valReader;
         Columns cols = isKeyCol ? schema.keyColumns() : schema.valueColumns();
 
         if (cols.column(colIdx).type().spec() != type)
@@ -500,234 +486,4 @@ public class Row implements BinaryRow {
         return row.readBytes(off, len);
     }
 
-    /**
-     * Abstract chunk reader.
-     */
-    abstract class AbstractChunkReader {
-        /** Base offset. */
-        protected final int baseOff;
-
-        /** Null-map offset. */
-        protected int nullMapOff;
-
-        /** Vartable offset. */
-        protected int varTableOff;
-
-        /** Payload offset. */
-        protected int dataOff;
-
-        /**
-         * @param baseOff Chunk base offset.
-         */
-        AbstractChunkReader(int baseOff) {
-            this.baseOff = baseOff;
-        }
-
-        /**
-         * @return Chunk length in bytes
-         */
-        abstract int chunkLength();
-
-        /**
-         * @return Number of items in vartable.
-         */
-        abstract int vartableItems();
-
-        /**
-         * Checks the row's null map for the given column index in the chunk.
-         *
-         * @param idx Offset of the column in the chunk.
-         * @return {@code true} if the column value is {@code null}.
-         */
-        /** {@inheritDoc} */
-        protected boolean isNull(int idx) {
-            if (!hasNullmap())
-                return false;
-
-            int nullByte = idx / 8;
-            int posInByte = idx % 8;
-
-            int map = readByte(nullMapOff + nullByte);
-
-            return (map & (1 << posInByte)) != 0;
-        }
-
-        /**
-         * @return {@code True} if chunk has vartable.
-         */
-        protected boolean hasVartable() {
-            return dataOff > varTableOff;
-        }
-
-        /**
-         * @return {@code True} if chunk has nullmap.
-         */
-        protected boolean hasNullmap() {
-            return varTableOff > nullMapOff;
-        }
-
-        /**
-         * @param itemIdx Varlen table item index.
-         * @return Varlen item offset.
-         */
-        protected abstract int varlenItemOffset(int itemIdx);
-
-        /**
-         * Calculates the offset of the fixlen column with the given index in the row. It essentially folds the null map
-         * with the column lengths to calculate the size of non-null columns preceding the requested column.
-         *
-         * @param cols Columns chunk.
-         * @param idx Column index in the chunk.
-         * @return Encoded offset (from the row start) of the requested fixlen column.
-         */
-        int fixlenColumnOffset(Columns cols, int idx) {
-            int colOff = 0;
-
-            // Calculate fixlen column offset.
-            {
-                int colByteIdx = idx / 8;
-
-                // 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 = colByteIdx == (cols.length() + 7) / 8 - 1 ? ((cols.numberOfFixsizeColumns() - 1) % 8) : 7;
-                int mask = (0xFF >> (7 - endBit)) & (0xFF << startBit);
-
-                if (hasNullmap()) {
-                    // Fold offset based on the whole map bytes in the schema
-                    for (int i = 0; i < colByteIdx; i++)
-                        colOff += cols.foldFixedLength(i, readByte(nullMapOff + i));
-
-                    colOff += cols.foldFixedLength(colByteIdx, readByte(nullMapOff + colByteIdx) | mask);
-                }
-                else {
-                    for (int i = 0; i < colByteIdx; i++)
-                        colOff += cols.foldFixedLength(i, 0);
-
-                    colOff += cols.foldFixedLength(colByteIdx, mask);
-                }
-            }
-
-            return dataOff + colOff;
-        }
-
-        /**
-         * Calculates the offset and length of varlen column. First, it calculates the number of non-null columns
-         * preceding 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 idx Column index in the chunk.
-         * @return Encoded offset (from the row start) and length of the column with the given index.
-         */
-        long varlenColumnOffsetAndLength(Columns cols, int idx) {
-            assert hasVartable() : "Chunk has no vartable: colId=" + idx;
-
-            if (hasNullmap()) {
-                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++) {
-                    byte 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 -= numNullsBefore;
-            }
-
-            idx -= cols.numberOfFixsizeColumns();
-
-            // Offset of idx-th column is from base offset.
-            int resOff = varlenItemOffset(idx);
-
-            long len = (idx == vartableItems() - 1) ?
-                // totalLength - columnStartOffset
-                (baseOff + chunkLength()) - resOff :
-                // nextColumnStartOffset - columnStartOffset
-                varlenItemOffset(idx + 1) - resOff;
-
-            return (len << 32) | resOff;
-        }
-    }
-
-    /**
-     * Tiny chunk format reader.
-     */
-    class TinyChunkReader extends AbstractChunkReader {
-        /**
-         * @param baseOff Base offset.
-         * @param nullMapLen Null-map length in bytes.
-         * @param hasVarTable Vartable presence flag.
-         */
-        TinyChunkReader(int baseOff, int nullMapLen, boolean hasVarTable) {
-            super(baseOff);
-
-            nullMapOff = baseOff + Byte.BYTES;
-            varTableOff = nullMapOff + nullMapLen;
-            dataOff = varTableOff + (hasVarTable ? Byte.BYTES + (readByte(varTableOff) & 0xFF) * Byte.BYTES : 0);
-        }
-
-        /** {@inheritDoc} */
-        @Override int chunkLength() {
-            return readByte(baseOff) & 0xFF;
-        }
-
-        /** {@inheritDoc} */
-        @Override int vartableItems() {
-            return hasVartable() ? (readByte(varTableOff) & 0xFF) : 0;
-        }
-
-        /** {@inheritDoc} */
-        @Override protected int varlenItemOffset(int itemIdx) {
-            return dataOff + (readByte(varTableOff + Byte.BYTES + itemIdx * Byte.BYTES) & 0xFF);
-        }
-    }
-
-    /**
-     * Large chunk format reader.
-     */
-    class LargeChunkReader extends AbstractChunkReader {
-        /**
-         * @param baseOff Base offset.
-         * @param nullMapLen Null-map length in bytes.
-         * @param hasVarTable Vartable presence flag.
-         */
-        LargeChunkReader(int baseOff, int nullMapLen, boolean hasVarTable) {
-            super(baseOff);
-
-            nullMapOff = baseOff + Integer.BYTES;
-            varTableOff = baseOff + Integer.BYTES + nullMapLen;
-            dataOff = varTableOff + (hasVarTable ? Integer.BYTES + readInteger(varTableOff) * Integer.BYTES : 0);
-        }
-
-        /** {@inheritDoc} */
-        @Override public int chunkLength() {
-            return readInteger(baseOff);
-        }
-
-        /** {@inheritDoc} */
-        @Override int vartableItems() {
-            return hasVartable() ? readInteger(varTableOff) : 0;
-        }
-
-        /** {@inheritDoc} */
-        @Override protected int varlenItemOffset(int itemIdx) {
-            return dataOff + readInteger(varTableOff + Integer.BYTES + itemIdx * Integer.BYTES);
-        }
-    }
 }
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 e555b49..0d43da4 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
@@ -56,9 +56,6 @@ public class RowAssembler {
     /** Target byte buffer to write to. */
     private final ExpandableByteBuf buf;
 
-    /** Val write mode flag. */
-    private final boolean tinyVal;
-
     /** Current columns chunk. */
     private Columns curCols;
 
@@ -77,6 +74,9 @@ public class RowAssembler {
     /** Current chunk writer. */
     private ChunkWriter chunkWriter;
 
+    /** Value write mode. */
+    private ChunkFormat valWriteMode;
+
     /**
      * Calculates encoded string length.
      *
@@ -189,52 +189,25 @@ public class RowAssembler {
         final int keyNullMapSize = keyHasNulls ? schema.keyColumns().nullMapSize() : 0;
         final int valNullMapSize = valHasNulls ? schema.valueColumns().nullMapSize() : 0;
 
-        boolean tinyKey = TinyChunkWriter.isTinyChunk(keyDataSize, keyNullMapSize, keyVarlenCols);
-        tinyVal = TinyChunkWriter.isTinyChunk(valDataSize, keyNullMapSize, valVarlenCols);
+        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;
-        if (tinyKey)
-            flags |= RowFlags.KEY_TYNY_FORMAT;
+
+        flags |= keyWriteMode.modeFlags() & 0x0F << 8;
 
         int size = BinaryRow.HEADER_SIZE +
-            (tinyKey ?
-                TinyChunkWriter.chunkSize(keyDataSize, keyNullMapSize, keyVarlenCols) :
-                LongChunkWriter.chunkSize(keyDataSize, keyNullMapSize, keyVarlenCols)) +
-            (tinyVal ?
-                TinyChunkWriter.chunkSize(valDataSize, valNullMapSize, valVarlenCols) :
-                LongChunkWriter.chunkSize(valDataSize, valNullMapSize, valVarlenCols));
+            keyWriteMode.chunkSize(keyDataSize, keyNullMapSize, keyDataSize) +
+            valWriteMode.chunkSize(valDataSize, valNullMapSize, valDataSize);
 
         buf = new ExpandableByteBuf(size);
         buf.putShort(0, (short)schema.version());
 
-        chunkWriter = createChunkWriter(KEY_CHUNK_OFFSET, keyNullMapSize, keyVarlenCols, tinyKey);
-    }
-
-    /**
-     * Chunk writer factory method.
-     *
-     * @param baseOff Chunk base offset.
-     * @param nullMapLen Null-map size in bytes.
-     * @param vartblSize Amount of vartable items.
-     * @param tiny Tiny format flag.
-     * @return Chunk writer.
-     */
-    private ChunkWriter createChunkWriter(int baseOff, int nullMapLen, int vartblSize, boolean tiny) {
-        return tiny ?
-            new TinyChunkWriter(
-                buf,
-                baseOff,
-                nullMapLen,
-                vartblSize) :
-            new LongChunkWriter(
-                buf,
-                baseOff,
-                nullMapLen,
-                vartblSize);
+        chunkWriter = keyWriteMode.writer(buf, KEY_CHUNK_OFFSET, keyNullMapSize, keyVarlenCols);
     }
 
     /**
@@ -505,17 +478,18 @@ public class RowAssembler {
             // Write value flags.
             if (valVarlenCols == 0)
                 flags |= OMIT_VAL_VARTBL_FLAG;
-            if (schema.valueColumns().nullMapSize() == 0)
-                flags |= RowFlags.OMIT_VAL_NULL_MAP_FLAG;
-            if (tinyVal)
-                flags |= RowFlags.VAL_TYNY_FORMAT;
+            else {
+                if (schema.valueColumns().nullMapSize() == 0)
+                    flags |= RowFlags.OMIT_VAL_NULL_MAP_FLAG;
+
+                flags |= valWriteMode.modeFlags() & 0x0F << 14;
+            }
 
             // Create value chunk writer.
-            chunkWriter = createChunkWriter(
+            chunkWriter = valWriteMode.writer(buf,
                 BinaryRow.HEADER_SIZE + chunkWriter.chunkLength() /* Key chunk size */,
                 schema.valueColumns().nullMapSize(),
-                valVarlenCols,
-                tinyVal);
+                valVarlenCols);
         }
     }
 }
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
index 347e257..25392a9 100644
--- 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
@@ -19,60 +19,21 @@ 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 {
     /**
-     * Calculates vartable length (in bytes).
-     *
-     * @param items Vartable items.
-     * @return Vartable size in bytes.
-     */
-    static int vartableLength(int items) {
-        return items == 0 ? 0 : Byte.BYTES /* Table size */ + items * Byte.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 Byte.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 isTinyChunk(int payloadLen, int nullMapLen, int vartblSize) {
-        return chunkSize(payloadLen, nullMapLen, vartblSize) < 256;
-    }
-
-    /**
      * 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.
+     * @param nullMapOff Null-map offset.
+     * @param varTblOff Vartable offset.
      */
-    TinyChunkWriter(ExpandableByteBuf buf, int baseOff, int nullMapLen, int vartblSize) {
-        super(
-            buf,
-            baseOff,
-            baseOff + Byte.BYTES /* Chunk size */,
-            baseOff + Byte.BYTES + nullMapLen,
-            baseOff + Byte.BYTES + nullMapLen + vartableLength(vartblSize));
+    TinyChunkWriter(ExpandableByteBuf buf, int baseOff, int nullMapOff, int varTblOff, int dataOff) {
+        super(buf,baseOff, nullMapOff, varTblOff, dataOff);
 
         curVartblItem = 0;
     }
diff --git a/modules/schema/src/main/java/org/apache/ignite/internal/schema/row/XXHash32.java b/modules/schema/src/main/java/org/apache/ignite/internal/schema/row/XXHash32.java
deleted file mode 100644
index 2b71c4a..0000000
--- a/modules/schema/src/main/java/org/apache/ignite/internal/schema/row/XXHash32.java
+++ /dev/null
@@ -1,243 +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;
-
-import java.nio.ByteBuffer;
-import java.util.zip.Checksum;
-
-import static java.lang.Integer.rotateLeft;
-
-/**
- * Implementation of the xxhash32 hash algorithm.
- *
- * Copied from Commons Compress 1.14 <a href=
- * "https://git-wip-us.apache.org/repos/asf?p=commons-compress.git;a=blob;f=src/main/java/org/apache/commons/compress/compressors/lz4/XXHash32.java;h=a406ffc197449be594d46f0d2712b2d4786a1e68;hb=HEAD">https://git-wip-us.apache.org/repos/asf?p=commons-compress.git;a=blob;f=src/main/java/org/apache/commons/compress/compressors/lz4/XXHash32.java;h=a406ffc197449be594d46f0d2712b2d4786a1e68;hb=HEAD</a>
- * <p>
- * NotThreadSafe
- *
- * @see <a href="http://cyan4973.github.io/xxHash/">xxHash</a>
- * @since 1.11
- */
-public class XXHash32 implements Checksum {
-    /** Buffer size. */
-    private static final int BUF_SIZE = 16;
-
-    /** Rotation shift. */
-    private static final int ROTATE_BITS = 13;
-
-    /** Prime number. */
-    private static final int PRIME1 = (int) 2654435761L;
-    /** Prime number. */
-    private static final int PRIME2 = (int) 2246822519L;
-    /** Prime number. */
-    private static final int PRIME3 = (int) 3266489917L;
-    /** Prime number. */
-    private static final int PRIME4 =  668265263;
-    /** Prime number. */
-    private static final int PRIME5 =  374761393;
-
-    /** One-byte len buffer. */
-    private final byte[] oneByte = new byte[1];
-
-    /** State. */
-    private final int[] state = new int[4];
-
-    // Note: the code used to use ByteBuffer but the manual method is 50% faster
-    // See: http://git-wip-us.apache.org/repos/asf/commons-compress/diff/2f56fb5c
-    private final byte[] buffer = new byte[BUF_SIZE];
-
-    /** Seed. */
-    private final int seed;
-
-    /** */
-    private int totalLen;
-
-    /** Position. */
-    private int pos;
-
-    /** Set to true when the state array has been updated since the last reset. */
-    private boolean stateUpdated;
-
-    /**
-     * Calculates hash for byte buffer slice.
-     *
-     * @param buf Buffer.
-     * @param off Slice offset.
-     * @param len Slice len.
-     * @return Hash code.
-     */
-    public static int hash(ByteBuffer buf, int off, int len) {
-        final XXHash32 hasher = new XXHash32(0);
-
-        hasher.update(buf.array(), off + buf.arrayOffset(), len);
-
-        return (int)hasher.getValue();
-    }
-
-    /**
-     * Creates an XXHash32 instance with a seed of 0.
-     */
-    public XXHash32() {
-        this(0);
-    }
-
-    /**
-     * Creates an XXHash32 instance.
-     * @param seed the seed to use
-     */
-    public XXHash32(final int seed) {
-        this.seed = seed;
-        initializeState();
-    }
-
-    /** {@inheritDoc} */
-    @Override public void reset() {
-        initializeState();
-        totalLen = 0;
-        pos = 0;
-        stateUpdated = false;
-    }
-
-    /** {@inheritDoc} */
-    @Override public void update(final int b) {
-        oneByte[0] = (byte) (b & 0xff);
-        update(oneByte, 0, 1);
-    }
-
-    /** {@inheritDoc} */
-    @Override public void update(final byte[] b, int off, final int len) {
-        if (len <= 0) {
-            return;
-        }
-        totalLen += len;
-
-        final int end = off + len;
-
-        // Check if the unprocessed bytes and new bytes can fill a block of 16.
-        // Make this overflow safe in the event that len is Integer.MAX_VALUE.
-        // Equivalent to: (pos + len < BUF_SIZE)
-        if (pos + len - BUF_SIZE < 0) {
-            System.arraycopy(b, off, buffer, pos, len);
-            pos += len;
-            return;
-        }
-
-        // Process left-over bytes with new bytes
-        if (pos > 0) {
-            final int size = BUF_SIZE - pos;
-            System.arraycopy(b, off, buffer, pos, size);
-            process(buffer, 0);
-            off += size;
-        }
-
-        final int limit = end - BUF_SIZE;
-        while (off <= limit) {
-            process(b, off);
-            off += BUF_SIZE;
-        }
-
-        // Handle left-over bytes
-        if (off < end) {
-            pos = end - off;
-            System.arraycopy(b, off, buffer, 0, pos);
-        } else {
-            pos = 0;
-        }
-    }
-
-    /** {@inheritDoc} */
-    @Override public long getValue() {
-        int hash;
-        if (stateUpdated) {
-            // Hash with the state
-            hash =
-                rotateLeft(state[0],  1) +
-                    rotateLeft(state[1],  7) +
-                    rotateLeft(state[2], 12) +
-                    rotateLeft(state[3], 18);
-        } else {
-            // Hash using the original seed from position 2
-            hash = state[2] + PRIME5;
-        }
-        hash += totalLen;
-
-        int idx = 0;
-        final int limit = pos - 4;
-        for (; idx <= limit; idx += 4) {
-            hash = rotateLeft(hash + getInt(buffer, idx) * PRIME3, 17) * PRIME4;
-        }
-        while (idx < pos) {
-            hash = rotateLeft(hash + (buffer[idx++] & 0xff) * PRIME5, 11) * PRIME1;
-        }
-
-        hash ^= hash >>> 15;
-        hash *= PRIME2;
-        hash ^= hash >>> 13;
-        hash *= PRIME3;
-        hash ^= hash >>> 16;
-        return hash & 0xffffffffL;
-    }
-
-    /**
-     * Gets the little-endian int from 4 bytes starting at the specified index.
-     *
-     * @param buffer The data
-     * @param idx The index
-     * @return The little-endian int
-     */
-    private static int getInt(final byte[] buffer, final int idx) {
-        return ((buffer[idx    ] & 0xff)      ) |
-            ((buffer[idx + 1] & 0xff) <<  8) |
-            ((buffer[idx + 2] & 0xff) << 16) |
-            ((buffer[idx + 3] & 0xff) << 24);
-    }
-
-    /**
-     * Initialize state.
-     */
-    private void initializeState() {
-        state[0] = seed + PRIME1 + PRIME2;
-        state[1] = seed + PRIME2;
-        state[2] = seed;
-        state[3] = seed - PRIME1;
-    }
-
-    /**
-     * @param b Buffer.
-     * @param offset Offset.
-     */
-    private void process(final byte[] b, final int offset) {
-        // local shadows for performance
-        int s0 = state[0];
-        int s1 = state[1];
-        int s2 = state[2];
-        int s3 = state[3];
-
-        s0 = rotateLeft(s0 + getInt(b, offset) * PRIME2, ROTATE_BITS) * PRIME1;
-        s1 = rotateLeft(s1 + getInt(b, offset + 4) * PRIME2, ROTATE_BITS) * PRIME1;
-        s2 = rotateLeft(s2 + getInt(b, offset + 8) * PRIME2, ROTATE_BITS) * PRIME1;
-        s3 = rotateLeft(s3 + getInt(b, offset + 12) * PRIME2, ROTATE_BITS) * PRIME1;
-
-        state[0] = s0;
-        state[1] = s1;
-        state[2] = s2;
-        state[3] = s3;
-
-        stateUpdated = true;
-    }
-}