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

[ignite] branch master updated: IGNITE-13056 Move indexes from the indexing to the core module (closes #8490)

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

tledkov pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/ignite.git


The following commit(s) were added to refs/heads/master by this push:
     new 35b3528  IGNITE-13056 Move indexes from the indexing to the core module (closes #8490)
35b3528 is described below

commit 35b3528fa020fd5bf1dce03cd2fc63508c9912a3
Author: Maksim Timonin <ti...@gmail.com>
AuthorDate: Thu Apr 1 18:06:04 2021 +0300

    IGNITE-13056 Move indexes from the indexing to the core module (closes #8490)
---
 .../IndexAbstractCompatibilityTest.java            |  92 ++
 .../persistence/IndexTypesCompatibilityTest.java   | 334 +++++++
 ...java => InlineJavaObjectCompatibilityTest.java} |  93 +-
 .../IgniteCompatibilityBasicTestSuite.java         |   6 +-
 .../GridCommandHandlerIndexForceRebuildTest.java   |  27 +-
 .../GridCommandHandlerIndexRebuildStatusTest.java  |  27 +-
 .../apache/ignite/internal/GridKernalContext.java  |   8 +
 .../ignite/internal/GridKernalContextImpl.java     |  12 +
 .../org/apache/ignite/internal/IgniteKernal.java   |   2 +
 .../internal/cache/query/index/AbstractIndex.java} |  27 +-
 .../ignite/internal/cache/query/index/Index.java   |  69 ++
 .../query/index/IndexDefinition.java}              |  12 +-
 .../query/index/IndexFactory.java}                 |  16 +-
 .../internal/cache/query/index/IndexName.java      |  88 ++
 .../internal/cache/query/index/IndexProcessor.java | 583 ++++++++++++
 .../query/index/NullsOrder.java}                   |  17 +-
 .../ignite/internal/cache/query/index/Order.java}  |  34 +-
 .../internal/cache/query/index/SingleCursor.java   |  58 ++
 .../query/index/SortOrder.java}                    |  17 +-
 .../DurableBackgroundCleanupIndexTreeTask.java     | 108 ++-
 .../query/index/sorted/IndexKeyDefinition.java     |  68 ++
 .../query/index/sorted/IndexKeyTypeSettings.java   |  81 ++
 .../cache/query/index/sorted/IndexKeyTypes.java    | 133 +++
 .../cache/query/index/sorted/IndexRow.java         |  86 ++
 .../cache/query/index/sorted/IndexRowCache.java}   |  19 +-
 .../query/index/sorted/IndexRowCacheRegistry.java} |  22 +-
 .../query/index/sorted/IndexRowComparator.java     |  46 +
 .../query/index/sorted/IndexRowCompartorImpl.java  |  71 ++
 .../cache/query/index/sorted/IndexRowImpl.java     | 176 ++++
 .../query/index/sorted/IndexSearchRowImpl.java     |  78 ++
 .../cache/query/index/sorted/IndexValueCursor.java |  72 ++
 .../query/index/sorted/InlineIndexRowHandler.java  |  69 ++
 .../sorted/InlineIndexRowHandlerFactory.java}      |  23 +-
 .../cache/query/index/sorted/MetaPageInfo.java     |  96 ++
 .../query/index/sorted/SortedIndexDefinition.java  |  62 ++
 .../query/index/sorted/SortedSegmentedIndex.java   | 102 +++
 .../index/sorted/ThreadLocalRowHandlerHolder.java  |  62 ++
 .../sorted/defragmentation/DefragIndexFactory.java | 324 +++++++
 .../defragmentation/IndexingDefragmentation.java   | 269 ++++++
 .../index/sorted/inline/IndexQueryContext.java}    |  37 +-
 .../query/index/sorted/inline/InlineIndex.java}    |  28 +-
 .../index/sorted/inline/InlineIndexFactory.java    | 102 +++
 .../query/index/sorted/inline/InlineIndexImpl.java | 473 ++++++++++
 .../index/sorted/inline/InlineIndexKeyType.java}   |  71 +-
 .../sorted/inline/InlineIndexKeyTypeRegistry.java  | 188 ++++
 .../query/index/sorted/inline/InlineIndexTree.java | 658 ++++++++++++++
 .../sorted/inline}/InlineObjectBytesDetector.java  |  79 +-
 .../index/sorted/inline/InlineRecommender.java     | 141 +++
 .../sorted/inline/InlineTreeFilterClosure.java}    |  47 +-
 .../sorted/inline/JavaObjectKeySerializer.java}    |  27 +-
 .../sorted/inline/io/AbstractInlineInnerIO.java    | 166 ++++
 .../sorted/inline/io/AbstractInlineLeafIO.java     | 166 ++++
 .../index/sorted/inline/io/AbstractInnerIO.java}   |  44 +-
 .../index/sorted/inline/io/AbstractLeafIO.java}    |  50 +-
 .../index/sorted/inline/io/IORowHandler.java}      |  47 +-
 .../query/index/sorted/inline/io/InlineIO.java}    |  19 +-
 .../index/sorted/inline/io/InlineInnerIO.java}     |  15 +-
 .../index/sorted/inline/io/InlineLeafIO.java}      |  16 +-
 .../query/index/sorted/inline/io/InnerIO.java}     |  16 +-
 .../query/index/sorted/inline/io/LeafIO.java}      |  14 +-
 .../query/index/sorted/inline/io/MvccIO.java}      |  25 +-
 .../index/sorted/inline/io/MvccInlineInnerIO.java} |  25 +-
 .../index/sorted/inline/io/MvccInlineLeafIO.java}  |  25 +-
 .../query/index/sorted/inline/io/MvccInnerIO.java} |  18 +-
 .../query/index/sorted/inline/io/MvccLeafIO.java}  |  18 +-
 .../inline/types/BooleanInlineIndexKeyType.java    |  59 ++
 .../inline/types/ByteInlineIndexKeyType.java       |  59 ++
 .../inline/types/BytesInlineIndexKeyType.java}     |  88 +-
 .../inline/types/DateInlineIndexKeyType.java       |  69 ++
 .../sorted/inline/types/DateValueConstants.java    |  46 +
 .../inline/types/DoubleInlineIndexKeyType.java     |  59 ++
 .../inline/types/FloatInlineIndexKeyType.java      |  59 ++
 .../inline/types/IntegerInlineIndexKeyType.java    |  61 ++
 .../inline/types/LongInlineIndexKeyType.java       |  61 ++
 .../inline/types/NullableInlineIndexKeyType.java   | 232 +++++
 .../types/ObjectByteArrayInlineIndexKeyType.java   |  69 ++
 .../inline/types/ObjectHashInlineIndexKeyType.java |  64 ++
 .../inline/types/ShortInlineIndexKeyType.java      |  59 ++
 .../types/SignedBytesInlineIndexKeyType.java}      |  21 +-
 .../inline/types/StringInlineIndexKeyType.java}    | 106 +--
 .../types/StringNoCompareInlineIndexKeyType.java   |  55 ++
 .../inline/types/TimeInlineIndexKeyType.java       |  61 ++
 .../inline/types/TimestampInlineIndexKeyType.java  |  82 ++
 .../inline/types/UuidInlineIndexKeyType.java       |  74 ++
 .../index/sorted/keys/AbstractDateIndexKey.java}   |  27 +-
 .../index/sorted/keys/AbstractTimeIndexKey.java}   |  25 +-
 .../sorted/keys/AbstractTimestampIndexKey.java}    |  29 +-
 .../query/index/sorted/keys/BooleanIndexKey.java}  |  39 +-
 .../query/index/sorted/keys/ByteIndexKey.java}     |  41 +-
 .../query/index/sorted/keys/BytesCompareUtils.java |  59 ++
 .../query/index/sorted/keys/BytesIndexKey.java}    |  39 +-
 .../index/sorted/keys/CacheJavaObjectIndexKey.java |  85 ++
 .../query/index/sorted/keys/DecimalIndexKey.java}  |  40 +-
 .../query/index/sorted/keys/DoubleIndexKey.java}   |  39 +-
 .../query/index/sorted/keys/FloatIndexKey.java}    |  41 +-
 .../cache/query/index/sorted/keys/IndexKey.java}   |  27 +-
 .../query/index/sorted/keys/IndexKeyFactory.java   |  99 ++
 .../query/index/sorted/keys/IntegerIndexKey.java}  |  41 +-
 .../index/sorted/keys/JavaObjectIndexKey.java      | 109 +++
 .../query/index/sorted/keys/LongIndexKey.java}     |  41 +-
 .../query/index/sorted/keys/NullIndexKey.java}     |  33 +-
 .../index/sorted/keys/PlainJavaObjectIndexKey.java |  64 ++
 .../query/index/sorted/keys/ShortIndexKey.java}    |  40 +-
 .../index/sorted/keys/SignedBytesIndexKey.java}    |  25 +-
 .../query/index/sorted/keys/StringIndexKey.java}   |  42 +-
 .../query/index/sorted/keys/UuidIndexKey.java      |  53 ++
 .../managers/indexing/IndexesRebuildTask.java      | 151 ++++
 .../dht/topology/GridDhtLocalPartition.java        |   2 +-
 .../GridCacheDatabaseSharedManager.java            |   2 +-
 .../cache/persistence/GridCacheOffheapManager.java |   6 +-
 .../CachePartitionDefragmentationManager.java      |   4 +-
 .../cache/persistence/pagemem/PageMemoryImpl.java  |   4 +-
 .../wal/reader/StandaloneGridKernalContext.java    |   6 +
 .../cache/tree/mvcc/data/MvccDataRow.java          |   2 +-
 .../processors/query/GridQueryIndexing.java        |  72 +-
 .../processors/query/GridQueryProcessor.java       |  40 +-
 .../processors/query/GridQueryRowCacheCleaner.java |   2 +-
 .../inline/InlineIndexKeyTypeRegistryTest.java     |  89 ++
 .../cache/GridCacheReferenceCleanupSelfTest.java   |   7 +
 .../processors/query/DummyQueryIndexing.java       |  49 +-
 .../development/utils/IgniteWalConverter.java      |  11 -
 .../processors/query/h2/opt/GeoSpatialIndex.java   |  65 ++
 .../query/h2/opt/GeoSpatialIndexDefinition.java    |  58 ++
 .../query/h2/opt/GeoSpatialIndexFactory.java       |  69 ++
 .../query/h2/opt/GeoSpatialIndexImpl.java          | 395 ++++++++
 .../processors/query/h2/opt/GeoSpatialUtils.java   |  71 ++
 .../processors/query/h2/opt/GeometryIndexKey.java} |  40 +-
 .../query/h2/opt/GridH2SpatialIndex.java           | 371 +-------
 .../processors/query/h2/ConnectionManager.java     |  13 +
 .../query/h2/H2JavaObjectSerializer.java           |  27 +-
 .../processors/query/h2/H2TableDescriptor.java     |  18 +-
 .../processors/query/h2/H2TableEngine.java         |  11 +-
 .../internal/processors/query/h2/H2Utils.java      |  38 +-
 .../processors/query/h2/IgniteH2Indexing.java      | 409 ++-------
 .../query/h2/IndexRebuildFullClosure.java          |  49 -
 .../query/h2/IndexRebuildPartialClosure.java       |  91 --
 .../processors/query/h2/SchemaManager.java         |  13 +-
 .../query/h2/database/H2PkHashIndex.java           |  44 +-
 .../processors/query/h2/database/H2Tree.java       | 997 ---------------------
 .../query/h2/database/H2TreeClientIndex.java       |  47 +-
 .../processors/query/h2/database/H2TreeIndex.java  | 612 +++----------
 .../query/h2/database/H2TreeIndexBase.java         |  92 --
 .../inlinecolumn/AbstractInlineIndexColumn.java    | 246 -----
 .../inlinecolumn/BooleanInlineIndexColumn.java     |  68 --
 .../inlinecolumn/ByteInlineIndexColumn.java        |  68 --
 .../inlinecolumn/DateInlineIndexColumn.java        |  69 --
 .../inlinecolumn/DoubleInlineIndexColumn.java      |  68 --
 .../inlinecolumn/FloatInlineIndexColumn.java       |  68 --
 .../inlinecolumn/InlineIndexColumnFactory.java     | 238 -----
 .../inlinecolumn/IntegerInlineIndexColumn.java     |  68 --
 .../inlinecolumn/LongInlineIndexColumn.java        |  68 --
 .../inlinecolumn/ObjectHashInlineIndexColumn.java  | 175 ----
 .../inlinecolumn/ShortInlineIndexColumn.java       |  68 --
 .../inlinecolumn/TimeInlineIndexColumn.java        |  69 --
 .../inlinecolumn/TimestampInlineIndexColumn.java   |  83 --
 .../inlinecolumn/UuidInlineIndexColumn.java        |  81 --
 .../h2/database/io/AbstractH2ExtrasInnerIO.java    | 169 ----
 .../h2/database/io/AbstractH2ExtrasLeafIO.java     | 166 ----
 .../defragmentation/IndexingDefragmentation.java   | 495 ----------
 .../processors/query/h2/index/H2RowComparator.java | 135 +++
 .../query/h2/index/QueryIndexDefinition.java       | 206 +++++
 .../h2/index/QueryIndexKeyDefinitionProvider.java  |  80 ++
 .../query/h2/index/QueryIndexRowHandler.java       | 172 ++++
 .../query/h2/index/QueryRowHandlerFactory.java     |  48 +
 .../h2/index/client/ClientIndexDefinition.java     |  79 ++
 .../query/h2/index/client/ClientIndexFactory.java  |  57 ++
 .../query/h2/index/client/ClientInlineIndex.java   | 153 ++++
 .../query/h2/index/keys/DateIndexKey.java          |  58 ++
 .../query/h2/index/keys/H2ValueWrapperMixin.java   |  44 +
 .../query/h2/index/keys/TimeIndexKey.java          |  58 ++
 .../query/h2/index/keys/TimestampIndexKey.java     |  63 ++
 .../processors/query/h2/opt/GridH2Table.java       | 210 ++---
 .../processors/query/h2/opt/H2CacheRow.java        |   2 +-
 .../internal/processors/query/h2/opt/H2Row.java    |   3 +-
 .../processors/query/h2/opt/join/RangeSource.java  |  12 +-
 .../CacheGroupMetricsWithIndexBuildFailTest.java   |  12 +-
 .../cache/index/AbstractIndexingCommonTest.java    |   7 +-
 .../processors/cache/index/BasicIndexTest.java     |   2 +-
 .../DynamicEnableIndexingConcurrentSelfTest.java   |  21 +-
 .../index/DynamicIndexAbstractBasicSelfTest.java   |   2 +-
 .../cache/index/H2RowCachePageEvictionTest.java    |  14 +-
 .../processors/cache/index/H2RowCacheSelfTest.java |  26 +-
 .../index/H2TreeCorruptedTreeExceptionTest.java    |   5 +-
 .../processors/cache/index/IndexMetricsTest.java   |  10 +-
 .../cache/index/StopRebuildIndexTest.java          |  49 +-
 .../IgnitePdsIndexingDefragmentationTest.java      |  19 +-
 .../db/LongDestroyDurableBackgroundTaskTest.java   | 159 ++--
 .../MultipleParallelCacheDeleteDeadlockTest.java   | 157 ++--
 .../processors/query/SqlSystemViewsSelfTest.java   |   9 +-
 .../query/WrongQueryEntityFieldTypeTest.java       |   3 +
 .../query/h2/GridIndexRebuildSelfTest.java         |  29 +-
 .../inlinecolumn/InlineIndexColumnTest.java        | 166 ++--
 .../IgniteBinaryCacheQueryTestSuite.java           |   5 +-
 193 files changed, 10387 insertions(+), 6358 deletions(-)

diff --git a/modules/compatibility/src/test/java/org/apache/ignite/compatibility/persistence/IndexAbstractCompatibilityTest.java b/modules/compatibility/src/test/java/org/apache/ignite/compatibility/persistence/IndexAbstractCompatibilityTest.java
new file mode 100644
index 0000000..bed0637
--- /dev/null
+++ b/modules/compatibility/src/test/java/org/apache/ignite/compatibility/persistence/IndexAbstractCompatibilityTest.java
@@ -0,0 +1,92 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ *
+ */
+
+package org.apache.ignite.compatibility.persistence;
+
+import java.util.Collection;
+import java.util.Set;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.cache.query.SqlFieldsQuery;
+import org.apache.ignite.compatibility.testframework.junits.Dependency;
+import org.apache.ignite.configuration.BinaryConfiguration;
+import org.apache.ignite.configuration.DataRegionConfiguration;
+import org.apache.ignite.configuration.DataStorageConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.configuration.WALMode;
+import org.jetbrains.annotations.NotNull;
+
+/**
+ * This class contains basic settings for indexes compatibility tests.
+ */
+public abstract class IndexAbstractCompatibilityTest extends IgnitePersistenceCompatibilityAbstractTest {
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(igniteInstanceName);
+
+        cfg.setPeerClassLoadingEnabled(false);
+
+        cfg.setDataStorageConfiguration(
+            new DataStorageConfiguration()
+                .setDefaultDataRegionConfiguration(
+                    new DataRegionConfiguration()
+                        .setPersistenceEnabled(true)
+                        .setMaxSize(DataStorageConfiguration.DFLT_DATA_REGION_INITIAL_SIZE)
+                )
+                // Disable WAL to skip filling index with reading WAL. Instead just start on previous persisted files.
+                .setWalMode(WALMode.NONE));
+
+        cfg.setBinaryConfiguration(
+            new BinaryConfiguration()
+                .setCompactFooter(true)
+        );
+
+        return cfg;
+    }
+
+    /** {@inheritDoc} */
+    @Override @NotNull protected Collection<Dependency> getDependencies(String igniteVer) {
+        Collection<Dependency> dependencies = super.getDependencies(igniteVer);
+
+        if ("2.6.0".equals(igniteVer)) {
+            dependencies.add(new Dependency("h2", "com.h2database", "h2", "1.4.195", false));
+
+            dependencies.add(new Dependency("h2", "org.apache.lucene", "lucene-core", "5.5.2", false));
+            dependencies.add(new Dependency("h2", "org.apache.lucene", "lucene-analyzers-common", "5.5.2", false));
+            dependencies.add(new Dependency("h2", "org.apache.lucene", "lucene-queryparser", "5.5.2", false));
+        }
+
+        dependencies.add(new Dependency("indexing", "ignite-indexing", false));
+
+        return dependencies;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected Set<String> getExcluded(String ver, Collection<Dependency> dependencies) {
+        Set<String> excluded = super.getExcluded(ver, dependencies);
+
+        if ("2.6.0".equals(ver))
+            excluded.add("h2");
+
+        return excluded;
+    }
+
+    /** */
+    protected void checkIndexUsed(IgniteCache<?, ?> cache, SqlFieldsQuery qry, String idxName) {
+        assertTrue("Query does not use index.", queryPlan(cache, qry).toLowerCase().contains(idxName.toLowerCase()));
+    }
+}
diff --git a/modules/compatibility/src/test/java/org/apache/ignite/compatibility/persistence/IndexTypesCompatibilityTest.java b/modules/compatibility/src/test/java/org/apache/ignite/compatibility/persistence/IndexTypesCompatibilityTest.java
new file mode 100644
index 0000000..cc4a650
--- /dev/null
+++ b/modules/compatibility/src/test/java/org/apache/ignite/compatibility/persistence/IndexTypesCompatibilityTest.java
@@ -0,0 +1,334 @@
+/*
+ * 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.compatibility.persistence;
+
+import java.math.BigDecimal;
+import java.nio.ByteBuffer;
+import java.sql.Time;
+import java.sql.Timestamp;
+import java.time.Instant;
+import java.util.Arrays;
+import java.util.Calendar;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.UUID;
+import java.util.function.Function;
+import org.apache.ignite.Ignite;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.cache.CacheAtomicityMode;
+import org.apache.ignite.cache.query.SqlFieldsQuery;
+import org.apache.ignite.cluster.ClusterState;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.internal.IgniteEx;
+import org.apache.ignite.lang.IgniteInClosure;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+/**
+ * Checks all basic sql types work correctly.
+ */
+@RunWith(Parameterized.class)
+public class IndexTypesCompatibilityTest extends IndexAbstractCompatibilityTest {
+    /** */
+    private static final String TEST_CACHE_NAME = IndexTypesCompatibilityTest.class.getSimpleName();
+
+    /** */
+    private static final int ROWS_CNT = 100;
+
+    /** */
+    private static final String TABLE_PREFIX = "TABLE_";
+
+    /**
+     * Key is one of valid Ignite SQL types.
+     * Value is list of functions that produces values with a Java type related to SQL type.
+     * First function is for direct type matching, others are synonims.
+     */
+    private static final Map<String, List<Function<Integer, Object>>> typeProducer = new HashMap<>();
+
+    static {
+        register("Boolean", (i) -> i > 0, Integer::new);
+        register("Tinyint", Integer::byteValue, Integer::shortValue, Integer::new, Long::new);
+        register("Bigint", Integer::longValue, Integer::new);
+        register("Decimal", BigDecimal::new);
+        register("Double", Integer::doubleValue, Float::new);
+        register("Int", Integer::new);
+        register("Real", Integer::floatValue, Integer::doubleValue);
+        register("Smallint", Integer::shortValue, Integer::new);
+        register("Char", (i) -> String.format("%4s", i.toString())); // Padding string for correct comparison.
+        register("Varchar", (i) -> String.format("%4s", i.toString()));
+        register("Date", (i) -> new java.sql.Date(i, Calendar.JANUARY, 1)); // LocalDate is not interchangeable.
+        register("Time", Time::new);  // Time and LocalTime are not interchanheable as differently implement hashCode.
+        register("Timestamp",  // Instant is not supported, LocalDateTime calculates hashCode differently.
+            Timestamp::new, (i) -> java.util.Date.from(Instant.ofEpochMilli(i)));
+        register("Binary", (i) -> ByteBuffer.allocate(4).putInt(i).array());
+        register("UUID", (i) -> UUID.fromString("123e4567-e89b-12d3-a456-" +
+            String.format("%12s", i.toString()).replace(' ', '0')));
+    }
+
+    /** */
+    private static void register(String type, Function<Integer, Object>... funcs) {
+        typeProducer.put(type, Arrays.asList(funcs));
+    }
+
+    /** Parametrized run param: Ignite version. */
+    @Parameterized.Parameter
+    public String igniteVer;
+
+    /** Test run configurations: Ignite version, Inline size configuration. */
+    @Parameterized.Parameters(name = "ver={0}")
+    public static Collection<Object[]> runConfig() {
+        return Arrays.asList(new Object[][] {
+            {"2.6.0"}, {"2.7.0"}, {"2.7.6"}, {"2.8.0"}, {"2.8.1"}, {"2.9.0"}, {"2.9.1"}, {"2.10.0"}
+        });
+    }
+
+        /** */
+    @Test
+    public void testQueryOldIndex() throws Exception {
+        doTestStartupWithOldVersion(igniteVer, new PostStartupClosure());
+    }
+
+    /**
+     * Tests opportunity to read data from previous Ignite DB version.
+     *
+     * @param igniteVer 3-digits version of ignite
+     * @throws Exception If failed.
+     */
+    protected void doTestStartupWithOldVersion(String igniteVer, PostStartupClosure closure) throws Exception {
+        try {
+            startGrid(1, igniteVer,
+                new PersistenceBasicCompatibilityTest.ConfigurationClosure(true),
+                closure);
+
+            stopAllGrids();
+
+            IgniteEx ignite = startGrid(0);
+
+            assertEquals(1, ignite.context().discovery().topologyVersion());
+
+            ignite.cluster().state(ClusterState.ACTIVE);
+
+            validateResultingCacheData(ignite.cache(TEST_CACHE_NAME));
+        }
+        finally {
+            stopAllGrids();
+        }
+    }
+
+    /**
+     * Asserts cache contained all expected values as it was saved before.
+     *
+     * @param cache Cache to check.
+     */
+    private void validateResultingCacheData(IgniteCache<Object, Object> cache) {
+        for (String type: typeProducer.keySet()) {
+            // Byte array is supported as key since 2.8.0.
+            if ("Binary".equals(type) && igniteVer.compareTo("2.8.0") < 0)
+                continue;
+
+            // In versions prior to 2.7.0, UUID must be converted to byte array to work with.
+            if ("UUID".equals(type) && igniteVer.compareTo("2.7.0") < 0)
+                continue;
+
+            int cnt = getRowsCnt(type);
+
+            for (Function<Integer, Object> func: typeProducer.get(type)) {
+                for (int i = 0; i < cnt; i++) {
+                    validateRandomRow(cache, type, func, i);
+                    validateRandomRange(cache, type, func, i);
+                }
+
+                validateNull(cache, type, func);
+            }
+        }
+    }
+
+    /** */
+    private void validateRandomRow(IgniteCache<Object, Object> cache, String type, Function<Integer, Object> func, int inc) {
+        String table = TABLE_PREFIX + type;
+
+        Object val = func.apply(inc);
+
+        // Select by quering complex index.
+        SqlFieldsQuery qry = new SqlFieldsQuery("SELECT * FROM public." + table + " v WHERE id = ?;")
+            .setArgs(val);
+
+        checkIndexUsed(cache, qry, "_key_PK");
+
+        List<List<?>> result = cache.query(qry).getAll();
+
+        assertTrue("Type=" + type + "; inc=" + inc + "; size=" + result.size(), result.size() == 1);
+
+        List<?> row = result.get(0);
+
+        for (int i = 0; i < 2; i++) {
+            if ("Binary".equals(type))
+                assertTrue("Type=" + type + "; exp=" + val + "; act=" + row.get(i),
+                    Arrays.equals((byte[]) val, (byte[]) row.get(i)));
+            else
+                assertTrue("Type=" + type + "; exp=" + val + "; act=" + row.get(i), row.get(i).equals(getBaseValue(type, inc)));
+        }
+    }
+
+    /** */
+    private void validateNull(IgniteCache<Object, Object> cache, String type, Function<Integer, Object> func) {
+        if ("Date".equals(type) || "Boolean".equals(type) || "Tinyint".equals(type))
+            return;
+
+        String table = TABLE_PREFIX + type;
+
+        int inc = getRowsCnt(type);
+        Object val = func.apply(inc);
+
+        // Select by quering complex index.
+        SqlFieldsQuery qry = new SqlFieldsQuery("SELECT * FROM public." + table + " v WHERE id = ?;")
+            .setArgs(val);
+
+        checkIndexUsed(cache, qry, "_key_PK");
+
+        List<List<?>> result = cache.query(qry).getAll();
+
+        assertTrue("Type=" + type + "; inc=" + inc + "; size=" + result.size(), result.size() == 1);
+
+        List<?> row = result.get(0);
+
+        assertNull("Type=" + type, row.get(1));
+    }
+
+    /** */
+    private void validateRandomRange(IgniteCache<Object, Object> cache, String type, Function<Integer, Object> func, int pivot) {
+        String table = TABLE_PREFIX + type;
+
+        int cnt = getRowsCnt(type);
+
+        Object val = func.apply(pivot);
+
+        // Select by quering complex index.
+        SqlFieldsQuery qry = new SqlFieldsQuery(
+            "SELECT * FROM public." + table + " v WHERE id > ? AND val is not null ORDER BY id;").setArgs(val);
+
+        checkIndexUsed(cache, qry, "_key_PK");
+
+        List<List<?>> result = cache.query(qry).getAll();
+
+        // For strict comparison. There was an issues with >= comparison for some versions.
+        pivot += 1;
+
+        assertTrue("Type= " + type + "; exp=" + (cnt - pivot) + "; act=" + result.size(), result.size() == cnt - pivot);
+
+        for (int i = 0; i < cnt - pivot; i++) {
+            List<?> row = result.get(i);
+
+            val = getBaseValue(type, pivot + i);
+
+            if ("Binary".equals(type))
+                assertTrue("Type=" + type + "; exp=" + val + "; act=" + row.get(0),
+                    Arrays.equals((byte[]) val, (byte[]) row.get(0)));
+            else
+                assertTrue("Type=" + type + "; exp=" + val + "; act=" + row.get(0),
+                    row.get(0).equals(val));
+        }
+    }
+
+    /**
+     * @return Count of test rows for every type. Boolean and Tinyint are limited by definition.
+     */
+    private static int getRowsCnt(String type) {
+        if ("Boolean".equals(type))
+            return 2;
+        else if ("Tinyint".equals(type))
+            return 128;
+        else
+            return ROWS_CNT;
+    }
+
+    /**
+     * @return Value produced by function of direct matching of Ignite type to Java class.
+     */
+    private static Object getBaseValue(String type, int val) {
+        return typeProducer.get(type).get(0).apply(val);
+    }
+
+    /** */
+    public static class PostStartupClosure implements IgniteInClosure<Ignite> {
+        /** {@inheritDoc} */
+        @Override public void apply(Ignite ignite) {
+            ignite.active(true);
+
+            CacheConfiguration<Object, Object> cacheCfg = new CacheConfiguration<>();
+            cacheCfg.setName(TEST_CACHE_NAME);
+            cacheCfg.setAtomicityMode(CacheAtomicityMode.TRANSACTIONAL);
+            cacheCfg.setBackups(1);
+
+            IgniteCache<Object, Object> cache = ignite.createCache(cacheCfg);
+
+            saveCacheData(cache);
+
+            ignite.active(false);
+        }
+
+        /**
+         * Create a complex index (int, pojo, int). Check that middle POJO object is correctly available from inline.
+         */
+        protected void saveCacheData(IgniteCache<Object, Object> cache) {
+            // WRAP_VALUE is a hack to make Date column work as PK.
+            String createQry = "CREATE TABLE public.%s (id %s PRIMARY KEY, val %s)";
+
+            String insertQry = "INSERT INTO public.%s (id, val) values (?, ?)";
+
+            for (String type: typeProducer.keySet()) {
+                String table = "TABLE_" + type;
+
+                String create = String.format(createQry, table, type, type);
+                String insert = String.format(insertQry, table);
+
+                // https://issues.apache.org/jira/browse/IGNITE-8552
+                // Date can't be used as PK column without this setting.
+                if ("Date".equals(type))
+                    create += " with \"WRAP_VALUE=false\"";
+
+                cache.query(new SqlFieldsQuery(create));
+
+                for (int i = 0; i < getRowsCnt(type); i++) {
+                    Object val = getBaseValue(type, i);
+
+                    cache.query(new SqlFieldsQuery(insert).setArgs(val, val)).getAll();
+                }
+
+                // Put NULL with last insert. Skip the check for types with limited size (Boolean, Tinyint)
+                // and for Date as it can be created only with WRAP_VALUE=false setting and it does not allow nulls.
+                if (!"Date".equals(type) && !"Boolean".equals(type) && !"Tinyint".equals(type)) {
+                    Object val = getBaseValue(type, getRowsCnt(type));
+
+                    cache.query(new SqlFieldsQuery(insert).setArgs(val, null)).getAll();
+                }
+            }
+        }
+    }
+
+    /**
+     * Too many insert queries for every type. Need wait more time.
+     */
+    @Override protected long getNodeJoinTimeout() {
+        return 60_000;
+    }
+}
diff --git a/modules/compatibility/src/test/java/org/apache/ignite/compatibility/persistence/InlineIndexCompatibilityTest.java b/modules/compatibility/src/test/java/org/apache/ignite/compatibility/persistence/InlineJavaObjectCompatibilityTest.java
similarity index 78%
rename from modules/compatibility/src/test/java/org/apache/ignite/compatibility/persistence/InlineIndexCompatibilityTest.java
rename to modules/compatibility/src/test/java/org/apache/ignite/compatibility/persistence/InlineJavaObjectCompatibilityTest.java
index f178c02..b97f5a1 100644
--- a/modules/compatibility/src/test/java/org/apache/ignite/compatibility/persistence/InlineIndexCompatibilityTest.java
+++ b/modules/compatibility/src/test/java/org/apache/ignite/compatibility/persistence/InlineJavaObjectCompatibilityTest.java
@@ -17,27 +17,23 @@
 
 package org.apache.ignite.compatibility.persistence;
 
+import java.io.IOException;
+import java.io.ObjectInputStream;
+import java.io.ObjectOutputStream;
+import java.io.Serializable;
 import java.util.Arrays;
 import java.util.Collection;
 import java.util.List;
 import java.util.Random;
-import java.util.Set;
 import org.apache.ignite.Ignite;
 import org.apache.ignite.IgniteCache;
 import org.apache.ignite.cache.CacheAtomicityMode;
 import org.apache.ignite.cache.query.SqlFieldsQuery;
 import org.apache.ignite.cache.query.annotations.QuerySqlField;
 import org.apache.ignite.cluster.ClusterState;
-import org.apache.ignite.compatibility.testframework.junits.Dependency;
-import org.apache.ignite.configuration.BinaryConfiguration;
 import org.apache.ignite.configuration.CacheConfiguration;
-import org.apache.ignite.configuration.DataRegionConfiguration;
-import org.apache.ignite.configuration.DataStorageConfiguration;
-import org.apache.ignite.configuration.IgniteConfiguration;
-import org.apache.ignite.configuration.WALMode;
 import org.apache.ignite.internal.IgniteEx;
 import org.apache.ignite.lang.IgniteInClosure;
-import org.jetbrains.annotations.NotNull;
 import org.junit.Test;
 import org.junit.runner.RunWith;
 import org.junit.runners.Parameterized;
@@ -47,12 +43,12 @@ import org.junit.runners.Parameterized;
  * Tests that upgrade version on persisted inline index is successfull.
  */
 @RunWith(Parameterized.class)
-public class InlineIndexCompatibilityTest extends IgnitePersistenceCompatibilityAbstractTest {
+public class InlineJavaObjectCompatibilityTest extends IndexAbstractCompatibilityTest {
     /** */
-    private static final String TEST_CACHE_NAME = InlineIndexCompatibilityTest.class.getSimpleName();
+    private static final String TEST_CACHE_NAME = InlineJavaObjectCompatibilityTest.class.getSimpleName();
 
     /** */
-    private static final int ROWS_CNT = 100;
+    private static final int ROWS_CNT = 1000;
 
     /** Index to test. */
     private static final String INDEX_NAME = "intval1_val_intval2";
@@ -92,7 +88,10 @@ public class InlineIndexCompatibilityTest extends IgnitePersistenceCompatibility
             {"2.9.0", true},
 
             {"2.9.1", false},
-            {"2.9.1", true}
+            {"2.9.1", true},
+
+            {"2.10.0", false},
+            {"2.10.0", true}
         });
     }
 
@@ -105,52 +104,6 @@ public class InlineIndexCompatibilityTest extends IgnitePersistenceCompatibility
         doTestStartupWithOldVersion(igniteVer, closure, idxName);
     }
 
-    /** {@inheritDoc} */
-    @Override @NotNull protected Collection<Dependency> getDependencies(String igniteVer) {
-        Collection<Dependency> dependencies = super.getDependencies(igniteVer);
-
-        if ("2.6.0".equals(igniteVer))
-            dependencies.add(new Dependency("h2", "com.h2database", "h2", "1.4.195", false));
-
-        dependencies.add(new Dependency("indexing", "ignite-indexing", false));
-
-        return dependencies;
-    }
-
-    /** {@inheritDoc} */
-    @Override protected Set<String> getExcluded(String ver, Collection<Dependency> dependencies) {
-        Set<String> excluded = super.getExcluded(ver, dependencies);
-
-        if ("2.6.0".equals(ver))
-            excluded.add("h2");
-
-        return excluded;
-    }
-
-    /** {@inheritDoc} */
-    @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception {
-        IgniteConfiguration cfg = super.getConfiguration(igniteInstanceName);
-
-        cfg.setPeerClassLoadingEnabled(false);
-
-        cfg.setDataStorageConfiguration(
-            new DataStorageConfiguration()
-                .setDefaultDataRegionConfiguration(
-                    new DataRegionConfiguration()
-                        .setPersistenceEnabled(true)
-                        .setMaxSize(DataStorageConfiguration.DFLT_DATA_REGION_INITIAL_SIZE)
-                )
-                // Disable WAL to skip filling index with reading WAL. Instead just start on previous persisted files.
-                .setWalMode(WALMode.NONE));
-
-        cfg.setBinaryConfiguration(
-            new BinaryConfiguration()
-                .setCompactFooter(true)
-        );
-
-        return cfg;
-    }
-
     /**
      * Tests opportunity to read data from previous Ignite DB version.
      *
@@ -230,7 +183,7 @@ public class InlineIndexCompatibilityTest extends IgnitePersistenceCompatibility
         // For strict comparison. There was an issues with >= comparison for some versions.
         pivot += 1;
 
-        assertTrue(result.size() == ROWS_CNT - pivot);
+        assertTrue("Exp=" + (ROWS_CNT - pivot) + "; act=" + result.size(), result.size() == ROWS_CNT - pivot);
 
         for (int i = 0; i < ROWS_CNT - pivot; i++) {
             List<?> row = result.get(i);
@@ -242,11 +195,6 @@ public class InlineIndexCompatibilityTest extends IgnitePersistenceCompatibility
     }
 
     /** */
-    private void checkIndexUsed(IgniteCache<?, ?> cache, SqlFieldsQuery qry, String idxName) {
-        assertTrue("Query does not use index.", queryPlan(cache, qry).toLowerCase().contains(idxName.toLowerCase()));
-    }
-
-    /** */
     public static class PostStartupClosure implements IgniteInClosure<Ignite> {
         /** {@inheritDoc} */
         @Override public void apply(Ignite ignite) {
@@ -305,9 +253,12 @@ public class InlineIndexCompatibilityTest extends IgnitePersistenceCompatibility
     }
 
     /** POJO object aimed to be inlined. */
-    public static class EntityValue {
+    public static class EntityValue implements Serializable {
         /** */
-        private final int val;
+        private static final long serialVersionUID = 0L;
+
+        /** */
+        private int val;
 
         /** */
         public EntityValue(int val) {
@@ -328,6 +279,16 @@ public class InlineIndexCompatibilityTest extends IgnitePersistenceCompatibility
         @Override public boolean equals(Object other) {
             return val == ((EntityValue) other).val;
         }
+
+        /** Enable comparison of EntityValue objects by the {@link #val} field. */
+        private void writeObject(ObjectOutputStream out) throws IOException {
+            out.writeInt(val);
+        }
+
+        /** */
+        private void readObject(ObjectInputStream in) throws IOException {
+            val = in.readInt();
+        }
     }
 
     /** Represents a cache value with 3 fields (POJO, int, int). */
diff --git a/modules/compatibility/src/test/java/org/apache/ignite/compatibility/testsuites/IgniteCompatibilityBasicTestSuite.java b/modules/compatibility/src/test/java/org/apache/ignite/compatibility/testsuites/IgniteCompatibilityBasicTestSuite.java
index 115bf0b..7af1235 100644
--- a/modules/compatibility/src/test/java/org/apache/ignite/compatibility/testsuites/IgniteCompatibilityBasicTestSuite.java
+++ b/modules/compatibility/src/test/java/org/apache/ignite/compatibility/testsuites/IgniteCompatibilityBasicTestSuite.java
@@ -21,7 +21,8 @@ import org.apache.ignite.compatibility.cache.LocalCacheTest;
 import org.apache.ignite.compatibility.clients.JavaThinCompatibilityTest;
 import org.apache.ignite.compatibility.clients.JdbcThinCompatibilityTest;
 import org.apache.ignite.compatibility.persistence.FoldersReuseCompatibilityTest;
-import org.apache.ignite.compatibility.persistence.InlineIndexCompatibilityTest;
+import org.apache.ignite.compatibility.persistence.IndexTypesCompatibilityTest;
+import org.apache.ignite.compatibility.persistence.InlineJavaObjectCompatibilityTest;
 import org.apache.ignite.compatibility.persistence.MetaStorageCompatibilityTest;
 import org.apache.ignite.compatibility.persistence.MigratingToWalV2SerializerWithCompactionTest;
 import org.apache.ignite.compatibility.persistence.MoveBinaryMetadataCompatibility;
@@ -35,7 +36,8 @@ import org.junit.runners.Suite;
 @RunWith(Suite.class)
 @Suite.SuiteClasses({
     PersistenceBasicCompatibilityTest.class,
-    InlineIndexCompatibilityTest.class,
+    InlineJavaObjectCompatibilityTest.class,
+    IndexTypesCompatibilityTest.class,
     FoldersReuseCompatibilityTest.class,
     MigratingToWalV2SerializerWithCompactionTest.class,
     MetaStorageCompatibilityTest.class,
diff --git a/modules/control-utility/src/test/java/org/apache/ignite/util/GridCommandHandlerIndexForceRebuildTest.java b/modules/control-utility/src/test/java/org/apache/ignite/util/GridCommandHandlerIndexForceRebuildTest.java
index 9cce435..904055fd 100644
--- a/modules/control-utility/src/test/java/org/apache/ignite/util/GridCommandHandlerIndexForceRebuildTest.java
+++ b/modules/control-utility/src/test/java/org/apache/ignite/util/GridCommandHandlerIndexForceRebuildTest.java
@@ -30,11 +30,10 @@ import org.apache.ignite.configuration.IgniteConfiguration;
 import org.apache.ignite.internal.IgniteEx;
 import org.apache.ignite.internal.IgniteInternalFuture;
 import org.apache.ignite.internal.IgniteInterruptedCheckedException;
+import org.apache.ignite.internal.cache.query.index.IndexProcessor;
+import org.apache.ignite.internal.managers.indexing.IndexesRebuildTask;
 import org.apache.ignite.internal.processors.cache.GridCacheContext;
 import org.apache.ignite.internal.processors.cache.IgniteCacheProxy;
-import org.apache.ignite.internal.processors.query.GridQueryIndexing;
-import org.apache.ignite.internal.processors.query.GridQueryProcessor;
-import org.apache.ignite.internal.processors.query.h2.IgniteH2Indexing;
 import org.apache.ignite.internal.processors.query.schema.SchemaIndexCacheFuture;
 import org.apache.ignite.internal.processors.query.schema.SchemaIndexCacheVisitorClosure;
 import org.apache.ignite.internal.processors.query.schema.SchemaIndexOperationCancellationToken;
@@ -96,7 +95,7 @@ public class GridCommandHandlerIndexForceRebuildTest extends GridCommandHandlerA
     private static final int LAST_NODE_NUM = GRIDS_NUM - 1;
 
     /**
-     * Map for blocking index rebuilds in a {@link BlockingIndexing}.
+     * Map for blocking index rebuilds in a {@link BlockingIndexesRebuildTask}.
      * To stop blocking, need to delete the entry.
      * Mapping: cache name -> future start blocking rebuilding indexes.
      */
@@ -138,7 +137,7 @@ public class GridCommandHandlerIndexForceRebuildTest extends GridCommandHandlerA
     /** */
     private void startupTestCluster() throws Exception {
         for (int i = 0; i < GRIDS_NUM; i++ ) {
-            GridQueryProcessor.idxCls = BlockingIndexing.class;
+            IndexProcessor.idxRebuildCls = BlockingIndexesRebuildTask.class;
             startGrid(i);
         }
 
@@ -517,7 +516,7 @@ public class GridCommandHandlerIndexForceRebuildTest extends GridCommandHandlerA
 
         GridTestUtils.deleteIndexBin(getTestIgniteInstanceName(2));
 
-        GridQueryProcessor.idxCls = BlockingIndexing.class;
+        IndexProcessor.idxRebuildCls = BlockingIndexesRebuildTask.class;
         final IgniteEx ignite = startGrid(igniteIdx);
 
         resetBaselineTopology();
@@ -591,15 +590,11 @@ public class GridCommandHandlerIndexForceRebuildTest extends GridCommandHandlerA
     /**
      * Indexing that blocks index rebuild until status request is completed.
      */
-    private static class BlockingIndexing extends IgniteH2Indexing {
+    private static class BlockingIndexesRebuildTask extends IndexesRebuildTask {
         /** {@inheritDoc} */
-        @Override protected void rebuildIndexesFromHash0(
-            GridCacheContext cctx,
-            SchemaIndexCacheVisitorClosure clo,
-            GridFutureAdapter<Void> rebuildIdxFut,
-            SchemaIndexOperationCancellationToken cancel
-        ) {
-            super.rebuildIndexesFromHash0(cctx, clo, new BlockingRebuildIdxFuture(rebuildIdxFut, cctx), cancel);
+        @Override protected void startRebuild(GridCacheContext cctx, GridFutureAdapter<Void> fut,
+            SchemaIndexCacheVisitorClosure clo, SchemaIndexOperationCancellationToken cancel) {
+            super.startRebuild(cctx, new BlockingRebuildIdxFuture(fut, cctx), clo, cancel);
         }
     }
 
@@ -647,9 +642,9 @@ public class GridCommandHandlerIndexForceRebuildTest extends GridCommandHandlerA
      * @return Internal index rebuild future.
      */
     @Nullable private SchemaIndexCacheFuture schemaIndexCacheFuture(IgniteEx n, int cacheId) {
-        GridQueryIndexing indexing = n.context().query().getIndexing();
+        IndexesRebuildTask idxRebuild = n.context().indexProcessor().idxRebuild();
 
-        Map<Integer, SchemaIndexCacheFuture> idxRebuildFuts = getFieldValue(indexing, "idxRebuildFuts");
+        Map<Integer, SchemaIndexCacheFuture> idxRebuildFuts = getFieldValue(idxRebuild, "idxRebuildFuts");
 
         return idxRebuildFuts.get(cacheId);
     }
diff --git a/modules/control-utility/src/test/java/org/apache/ignite/util/GridCommandHandlerIndexRebuildStatusTest.java b/modules/control-utility/src/test/java/org/apache/ignite/util/GridCommandHandlerIndexRebuildStatusTest.java
index 9b391cd..aed6f5a 100644
--- a/modules/control-utility/src/test/java/org/apache/ignite/util/GridCommandHandlerIndexRebuildStatusTest.java
+++ b/modules/control-utility/src/test/java/org/apache/ignite/util/GridCommandHandlerIndexRebuildStatusTest.java
@@ -28,11 +28,11 @@ import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.internal.IgniteEx;
 import org.apache.ignite.internal.IgniteInternalFuture;
 import org.apache.ignite.internal.IgniteInterruptedCheckedException;
+import org.apache.ignite.internal.cache.query.index.IndexProcessor;
 import org.apache.ignite.internal.commandline.CommandHandler;
+import org.apache.ignite.internal.managers.indexing.IndexesRebuildTask;
 import org.apache.ignite.internal.processors.cache.GridCacheContext;
 import org.apache.ignite.internal.processors.cache.persistence.CacheDataRow;
-import org.apache.ignite.internal.processors.query.GridQueryProcessor;
-import org.apache.ignite.internal.processors.query.h2.IgniteH2Indexing;
 import org.apache.ignite.internal.processors.query.schema.SchemaIndexCacheVisitorClosure;
 import org.apache.ignite.internal.processors.query.schema.SchemaIndexOperationCancellationToken;
 import org.apache.ignite.internal.util.future.GridFutureAdapter;
@@ -144,10 +144,10 @@ public class GridCommandHandlerIndexRebuildStatusTest extends GridCommandHandler
         deleteIndexBin(getTestIgniteInstanceName(GRIDS_NUM - 1));
         deleteIndexBin(getTestIgniteInstanceName(GRIDS_NUM - 2));
 
-        GridQueryProcessor.idxCls = BlockingIndexing.class;
+        IndexProcessor.idxRebuildCls = BlockingIndexesRebuildTask.class;
         IgniteEx ignite1 = startGrid(GRIDS_NUM - 1);
 
-        GridQueryProcessor.idxCls = BlockingIndexing.class;
+        IndexProcessor.idxRebuildCls = BlockingIndexesRebuildTask.class;
         IgniteEx ignite2 = startGrid(GRIDS_NUM - 2);
 
         final UUID id1 = ignite1.localNode().id();
@@ -179,10 +179,10 @@ public class GridCommandHandlerIndexRebuildStatusTest extends GridCommandHandler
         deleteIndexBin(getTestIgniteInstanceName(GRIDS_NUM - 1));
         deleteIndexBin(getTestIgniteInstanceName(GRIDS_NUM - 2));
 
-        GridQueryProcessor.idxCls = BlockingIndexing.class;
+        IndexProcessor.idxRebuildCls = BlockingIndexesRebuildTask.class;
         IgniteEx ignite1 = startGrid(GRIDS_NUM - 1);
 
-        GridQueryProcessor.idxCls = BlockingIndexing.class;
+        IndexProcessor.idxRebuildCls = BlockingIndexesRebuildTask.class;
         startGrid(GRIDS_NUM - 2);
 
         final UUID id1 = ignite1.localNode().id();
@@ -241,19 +241,14 @@ public class GridCommandHandlerIndexRebuildStatusTest extends GridCommandHandler
     /**
      * Indexing that blocks index rebuild until status request is completed.
      */
-    private static class BlockingIndexing extends IgniteH2Indexing {
-        /** {@inheritDoc} */
-        @Override protected void rebuildIndexesFromHash0(
-            GridCacheContext cctx,
-            SchemaIndexCacheVisitorClosure clo,
-            GridFutureAdapter<Void> rebuildIdxFut,
-            SchemaIndexOperationCancellationToken cancel
-        ) {
+    private static class BlockingIndexesRebuildTask extends IndexesRebuildTask {
+        @Override protected void startRebuild(GridCacheContext cctx, GridFutureAdapter<Void> fut,
+            SchemaIndexCacheVisitorClosure clo, SchemaIndexOperationCancellationToken cancel) {
             idxRebuildsStartedNum.incrementAndGet();
 
-            rebuildIdxFut.listen((CI1<IgniteInternalFuture<?>>)f -> idxRebuildsStartedNum.decrementAndGet());
+            fut.listen((CI1<IgniteInternalFuture<?>>)f -> idxRebuildsStartedNum.decrementAndGet());
 
-            super.rebuildIndexesFromHash0(cctx, new BlockingSchemaIndexCacheVisitorClosure(clo), rebuildIdxFut, cancel);
+            super.startRebuild(cctx, fut, new BlockingSchemaIndexCacheVisitorClosure(clo), cancel);
         }
     }
 
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/GridKernalContext.java b/modules/core/src/main/java/org/apache/ignite/internal/GridKernalContext.java
index 2e2fae6..281b8a0 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/GridKernalContext.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/GridKernalContext.java
@@ -23,6 +23,7 @@ import java.util.UUID;
 import java.util.concurrent.ExecutorService;
 import org.apache.ignite.IgniteLogger;
 import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.internal.cache.query.index.IndexProcessor;
 import org.apache.ignite.internal.managers.checkpoint.GridCheckpointManager;
 import org.apache.ignite.internal.managers.collision.GridCollisionManager;
 import org.apache.ignite.internal.managers.communication.GridIoManager;
@@ -447,6 +448,13 @@ public interface GridKernalContext extends Iterable<GridComponent> {
     public GridIndexingManager indexing();
 
     /**
+     * Indexes processor.
+     *
+     * @return Indexes processor.
+     */
+    public IndexProcessor indexProcessor();
+
+    /**
      * Gets encryption manager.
      *
      * @return Encryption manager.
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/GridKernalContextImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/GridKernalContextImpl.java
index a9f80be..bb8fcf8 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/GridKernalContextImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/GridKernalContextImpl.java
@@ -38,6 +38,7 @@ import org.apache.ignite.IgniteSystemProperties;
 import org.apache.ignite.cluster.ClusterNode;
 import org.apache.ignite.configuration.IgniteConfiguration;
 import org.apache.ignite.failure.FailureType;
+import org.apache.ignite.internal.cache.query.index.IndexProcessor;
 import org.apache.ignite.internal.maintenance.MaintenanceProcessor;
 import org.apache.ignite.internal.managers.checkpoint.GridCheckpointManager;
 import org.apache.ignite.internal.managers.collision.GridCollisionManager;
@@ -174,6 +175,10 @@ public class GridKernalContextImpl implements GridKernalContext, Externalizable
 
     /** */
     @GridToStringExclude
+    private IndexProcessor indexProc;
+
+    /** */
+    @GridToStringExclude
     private GridEncryptionManager encryptionMgr;
 
     /** */
@@ -718,6 +723,8 @@ public class GridKernalContextImpl implements GridKernalContext, Externalizable
             maintenanceProc = (MaintenanceProcessor) comp;
         else if (comp instanceof PerformanceStatisticsProcessor)
             perfStatProc = (PerformanceStatisticsProcessor)comp;
+        else if (comp instanceof IndexProcessor)
+            indexProc = (IndexProcessor)comp;
         else if (!(comp instanceof DiscoveryNodeValidationProcessor
             || comp instanceof PlatformPluginProcessor))
             assert (comp instanceof GridPluginComponent) : "Unknown manager class: " + comp.getClass();
@@ -916,6 +923,11 @@ public class GridKernalContextImpl implements GridKernalContext, Externalizable
     }
 
     /** {@inheritDoc} */
+    @Override public IndexProcessor indexProcessor() {
+        return indexProc;
+    }
+
+    /** {@inheritDoc} */
     @Override public GridEncryptionManager encryption() {
         return encryptionMgr;
     }
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java b/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java
index 0785023..6160333 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java
@@ -109,6 +109,7 @@ import org.apache.ignite.events.EventType;
 import org.apache.ignite.internal.binary.BinaryEnumCache;
 import org.apache.ignite.internal.binary.BinaryMarshaller;
 import org.apache.ignite.internal.binary.BinaryUtils;
+import org.apache.ignite.internal.cache.query.index.IndexProcessor;
 import org.apache.ignite.internal.cluster.ClusterGroupAdapter;
 import org.apache.ignite.internal.cluster.IgniteClusterEx;
 import org.apache.ignite.internal.maintenance.MaintenanceProcessor;
@@ -1251,6 +1252,7 @@ public class IgniteKernal implements IgniteEx, IgniteMXBean, Externalizable {
                 startProcessor(createComponent(IGridClusterStateProcessor.class, ctx));
                 startProcessor(new IgniteAuthenticationProcessor(ctx));
                 startProcessor(new GridCacheProcessor(ctx));
+                startProcessor(new IndexProcessor(ctx));
                 startProcessor(new GridQueryProcessor(ctx));
                 startProcessor(new ClientListenerProcessor(ctx));
                 startProcessor(createServiceProcessor());
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/io/H2InnerIO.java b/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/AbstractIndex.java
similarity index 57%
copy from modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/io/H2InnerIO.java
copy to modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/AbstractIndex.java
index 9baff7a..32e35d1 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/io/H2InnerIO.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/AbstractIndex.java
@@ -15,23 +15,28 @@
  * limitations under the License.
  */
 
-package org.apache.ignite.internal.processors.query.h2.database.io;
+package org.apache.ignite.internal.cache.query.index;
 
-import org.apache.ignite.internal.processors.cache.persistence.tree.io.IOVersions;
+import java.util.concurrent.atomic.AtomicBoolean;
 
 /**
- * Inner page for H2 row references.
+ * Abstract class for all Index implementations.
  */
-public class H2InnerIO extends AbstractH2InnerIO {
-    /** */
-    public static final IOVersions<H2InnerIO> VERSIONS = new IOVersions<>(
-        new H2InnerIO(1)
-    );
+public abstract class AbstractIndex implements Index {
+    /** Whether index is rebuilding now. */
+    private final AtomicBoolean rebuildInProgress = new AtomicBoolean(false);
 
     /**
-     * @param ver Page format version.
+     * @param val Mark or unmark index to rebuild.
      */
-    private H2InnerIO(int ver) {
-        super(T_H2_REF_INNER, ver, 8);
+    public void markIndexRebuild(boolean val) {
+        rebuildInProgress.compareAndSet(!val, val);
+    }
+
+    /**
+     * @return Whether index is rebuilding now.
+     */
+    public boolean rebuildInProgress() {
+        return rebuildInProgress.get();
     }
 }
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/Index.java b/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/Index.java
new file mode 100644
index 0000000..955bc03
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/Index.java
@@ -0,0 +1,69 @@
+/*
+ * 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.cache.query.index;
+
+import java.util.UUID;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.internal.processors.cache.persistence.CacheDataRow;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Basic interface for Ignite indexes.
+ */
+public interface Index {
+    /**
+     * Unique ID.
+     */
+    public UUID id();
+
+    /**
+     * Index name.
+     */
+    public String name();
+
+    /**
+     * Checks whether index handles specified cache row.
+     *
+     * @param row Cache row.
+     * @return Whether index handles specified cache row
+     */
+    public boolean canHandle(CacheDataRow row) throws IgniteCheckedException;
+
+    /**
+     * Callback that runs when the underlying cache is updated.
+     *
+     * @param oldRow Cache row that was replaced with newRow.
+     * @param newRow Cache row that was stored.
+     * @param prevRowAvailable Whether oldRow available.
+     */
+    public void onUpdate(@Nullable CacheDataRow oldRow, @Nullable CacheDataRow newRow, boolean prevRowAvailable)
+        throws IgniteCheckedException;
+
+    /**
+     * Provides a standard way to access the underlying concrete index
+     * implementation to provide access to further, proprietary features.
+     */
+    public <T extends Index> T unwrap(Class<T> clazz);
+
+    /**
+     * Destroy index.
+     *
+     * @param softDelete if {@code true} then perform logical deletion.
+     */
+    public void destroy(boolean softDelete);
+}
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryRowCacheCleaner.java b/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/IndexDefinition.java
similarity index 73%
copy from modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryRowCacheCleaner.java
copy to modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/IndexDefinition.java
index 6275813..35f03ef 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryRowCacheCleaner.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/IndexDefinition.java
@@ -15,16 +15,14 @@
  * limitations under the License.
  */
 
-package org.apache.ignite.internal.processors.query;
+package org.apache.ignite.internal.cache.query.index;
 
 /**
- * Row cache cleaner is used by page memory manager to remove updated / evicted links from rows cache.
+ * Basic interface for index description required to create or destroy index.
  */
-public interface GridQueryRowCacheCleaner {
+public interface IndexDefinition {
     /**
-     * Remove row by link.
-     *
-     * @param link Link to remove.
+     * @return Index name.
      */
-    void remove(long link);
+    public IndexName idxName();
 }
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryRowCacheCleaner.java b/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/IndexFactory.java
similarity index 63%
copy from modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryRowCacheCleaner.java
copy to modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/IndexFactory.java
index 6275813..85329c2 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryRowCacheCleaner.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/IndexFactory.java
@@ -15,16 +15,20 @@
  * limitations under the License.
  */
 
-package org.apache.ignite.internal.processors.query;
+package org.apache.ignite.internal.cache.query.index;
+
+import org.apache.ignite.internal.processors.cache.GridCacheContext;
+import org.jetbrains.annotations.Nullable;
 
 /**
- * Row cache cleaner is used by page memory manager to remove updated / evicted links from rows cache.
+ * Base interface for Ignite index factories.
  */
-public interface GridQueryRowCacheCleaner {
+public interface IndexFactory {
     /**
-     * Remove row by link.
+     * Creates index by specified definition for specified cache.
      *
-     * @param link Link to remove.
+     * @param cctx Cache context.
+     * @param definition Index definition.
      */
-    void remove(long link);
+    public Index createIndex(@Nullable GridCacheContext<?, ?> cctx, IndexDefinition definition);
 }
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/IndexName.java b/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/IndexName.java
new file mode 100644
index 0000000..57f8015
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/IndexName.java
@@ -0,0 +1,88 @@
+/*
+ * 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.cache.query.index;
+
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Represents list of names that fully describes index domain (schema, cache, table, index).
+ */
+public class IndexName {
+    /** Schema name of {@code null} if index is not related to SQL schema. */
+    private final @Nullable String schemaName;
+
+    /** Schema name of {@code null} if index is not related to SQL table. */
+    private final @Nullable String tableName;
+
+    /** Cache name. */
+    private final String cacheName;
+
+    /** Index name. */
+    private final String idxName;
+
+    /** */
+    public IndexName(String cacheName, @Nullable String schemaName, @Nullable String tableName, String idxName) {
+        this.cacheName = cacheName;
+        this.schemaName = schemaName;
+        this.tableName = tableName;
+        this.idxName = idxName;
+    }
+
+    /**
+     * @return Full index name.
+     */
+    public String fullName() {
+        StringBuilder bld = new StringBuilder();
+
+        if (schemaName != null)
+            bld.append(schemaName).append(".");
+
+        if (tableName != null)
+            bld.append(tableName).append(".");
+
+        return bld.append(idxName).toString();
+    }
+
+    /**
+     * @return Index name.
+     */
+    public String idxName() {
+        return idxName;
+    }
+
+    /**
+     * @return Table name.
+     */
+    public String tableName() {
+        return tableName;
+    }
+
+    /**
+     * @return Schema name.
+     */
+    public String schemaName() {
+        return schemaName;
+    }
+
+    /**
+     * @return Cache name.
+     */
+    public String cacheName() {
+        return cacheName;
+    }
+}
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/IndexProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/IndexProcessor.java
new file mode 100644
index 0000000..f10b0f2
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/IndexProcessor.java
@@ -0,0 +1,583 @@
+/*
+ * 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.cache.query.index;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.internal.GridKernalContext;
+import org.apache.ignite.internal.IgniteInternalFuture;
+import org.apache.ignite.internal.cache.query.index.sorted.IndexRow;
+import org.apache.ignite.internal.cache.query.index.sorted.IndexRowCache;
+import org.apache.ignite.internal.cache.query.index.sorted.IndexRowCacheRegistry;
+import org.apache.ignite.internal.cache.query.index.sorted.defragmentation.IndexingDefragmentation;
+import org.apache.ignite.internal.cache.query.index.sorted.inline.InlineIndex;
+import org.apache.ignite.internal.cache.query.index.sorted.inline.JavaObjectKeySerializer;
+import org.apache.ignite.internal.cache.query.index.sorted.inline.io.AbstractInlineInnerIO;
+import org.apache.ignite.internal.cache.query.index.sorted.inline.io.AbstractInlineLeafIO;
+import org.apache.ignite.internal.cache.query.index.sorted.inline.io.InnerIO;
+import org.apache.ignite.internal.cache.query.index.sorted.inline.io.LeafIO;
+import org.apache.ignite.internal.cache.query.index.sorted.inline.io.MvccInnerIO;
+import org.apache.ignite.internal.cache.query.index.sorted.inline.io.MvccLeafIO;
+import org.apache.ignite.internal.managers.indexing.IndexesRebuildTask;
+import org.apache.ignite.internal.pagemem.PageIdAllocator;
+import org.apache.ignite.internal.pagemem.PageMemory;
+import org.apache.ignite.internal.processors.GridProcessorAdapter;
+import org.apache.ignite.internal.processors.cache.CacheGroupContext;
+import org.apache.ignite.internal.processors.cache.GridCacheContext;
+import org.apache.ignite.internal.processors.cache.GridCacheContextInfo;
+import org.apache.ignite.internal.processors.cache.persistence.CacheDataRow;
+import org.apache.ignite.internal.processors.cache.persistence.RootPage;
+import org.apache.ignite.internal.processors.cache.persistence.checkpoint.CheckpointTimeoutLock;
+import org.apache.ignite.internal.processors.cache.persistence.defragmentation.LinkMap;
+import org.apache.ignite.internal.processors.cache.persistence.pagemem.PageMemoryEx;
+import org.apache.ignite.internal.processors.cache.persistence.tree.BPlusTree;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.BPlusIO;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.BPlusMetaIO;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.PageIO;
+import org.apache.ignite.internal.processors.cache.persistence.tree.reuse.ReuseList;
+import org.apache.ignite.internal.processors.cache.persistence.tree.util.PageLockListener;
+import org.apache.ignite.internal.processors.cache.query.IgniteQueryErrorCode;
+import org.apache.ignite.internal.processors.query.IgniteSQLException;
+import org.apache.ignite.internal.processors.query.schema.SchemaIndexCacheVisitor;
+import org.apache.ignite.internal.util.GridAtomicLong;
+import org.apache.ignite.internal.util.collection.IntMap;
+import org.apache.ignite.internal.util.typedef.F;
+import org.apache.ignite.internal.util.typedef.X;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.spi.IgniteSpiException;
+import org.apache.ignite.thread.IgniteThreadPoolExecutor;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Implementation of IndexingSpi that tracks all cache indexes.
+ */
+public class IndexProcessor extends GridProcessorAdapter {
+    /**
+     * Register inline IOs for sorted indexes.
+     */
+    static {
+        PageIO.registerH2(InnerIO.VERSIONS, LeafIO.VERSIONS, MvccInnerIO.VERSIONS, MvccLeafIO.VERSIONS);
+
+        AbstractInlineInnerIO.register();
+        AbstractInlineLeafIO.register();
+    }
+
+    /** For tests to emulate long rebuild process. */
+    public static Class<? extends IndexesRebuildTask> idxRebuildCls;
+
+    /** Indexes rebuild job. */
+    private final IndexesRebuildTask idxRebuild;
+
+    /** Serializer for representing JO as byte array in inline. */
+    public static JavaObjectKeySerializer serializer;
+
+    /** Row cache. */
+    private final IndexRowCacheRegistry idxRowCacheRegistry = new IndexRowCacheRegistry();
+
+    /**
+     * Registry of all indexes. High key is a cache name, lower key is an unique index name.
+     */
+    private final Map<String, Map<String, Index>> cacheToIdx = new ConcurrentHashMap<>();
+
+    /**
+     * Registry of all index definitions. Key is {@link Index#id()}, value is IndexDefinition used for creating index.
+     */
+    private final Map<UUID, IndexDefinition> idxDefs = new ConcurrentHashMap<>();
+
+    /** Exclusive lock for DDL operations. */
+    private final ReentrantReadWriteLock ddlLock = new ReentrantReadWriteLock();
+
+    /**
+     * @param ctx Kernal context.
+     */
+    public IndexProcessor(GridKernalContext ctx) throws IgniteCheckedException {
+        super(ctx);
+
+        if (idxRebuildCls != null) {
+            idxRebuild = U.newInstance(idxRebuildCls);
+
+            idxRebuildCls = null;
+        }
+        else
+            idxRebuild = new IndexesRebuildTask();
+
+        serializer = new JavaObjectKeySerializer(ctx.config());
+    }
+
+    /**
+     * Updates index with new row. Note that key is unique for cache, so if cache contains multiple indexes
+     * the key should be removed from indexes other than one being updated.
+     *
+     * @param cctx Cache context.
+     * @param newRow cache row to store in index.
+     * @param prevRow optional cache row that will be replaced with new row.
+     */
+    public void store(GridCacheContext<?, ?> cctx, CacheDataRow newRow, @Nullable CacheDataRow prevRow,
+        boolean prevRowAvailable)
+        throws IgniteSpiException {
+        try {
+            updateIndexes(cctx.name(), newRow, prevRow, prevRowAvailable);
+
+        } catch (IgniteCheckedException e) {
+            throw new IgniteSpiException("Failed to store row in cache", e);
+        }
+    }
+
+    /**
+     * Updates index with new row. Note that key is unique for cache, so if cache contains multiple indexes
+     * the key should be removed from indexes other than one being updated.
+     *
+     * @param idxs List of indexes to update.
+     * @param newRow cache row to store in index.
+     * @param prevRow optional cache row that will be replaced with new row.
+     */
+    public void store(Collection<? extends Index> idxs, CacheDataRow newRow, @Nullable CacheDataRow prevRow,
+        boolean prevRowAvailable) throws IgniteSpiException {
+        IgniteCheckedException err = null;
+
+        ddlLock.readLock().lock();
+
+        try {
+            for (Index idx : idxs)
+                err = updateIndex(idx, newRow, prevRow, prevRowAvailable, err);
+
+            if (err != null)
+                throw err;
+
+        } catch (IgniteCheckedException e) {
+            throw new IgniteSpiException("Failed to store row in index", e);
+
+        } finally {
+            ddlLock.readLock().unlock();
+        }
+    }
+
+    /**
+     * Delete specified row from index.
+     *
+     * @param cacheName Cache name.
+     * @param prevRow Cache row to delete from index.
+     */
+    public void remove(String cacheName, @Nullable CacheDataRow prevRow) throws IgniteSpiException {
+        try {
+            updateIndexes(cacheName, null, prevRow, true);
+
+        } catch (IgniteCheckedException e) {
+            throw new IgniteSpiException("Failed to remove row in cache", e);
+        }
+    }
+
+    /**
+     * Creates a new index.
+     *
+     * @param cctx Cache context.
+     * @param factory Index factory.
+     * @param definition Description of an index to create.
+     * @param cacheVisitor Enable to cancel dynamic index populating.
+     */
+    public Index createIndexDynamically(GridCacheContext cctx, IndexFactory factory, IndexDefinition definition,
+        SchemaIndexCacheVisitor cacheVisitor) {
+
+        Index idx = createIndex(cctx, factory, definition);
+
+        // Populate index with cache rows.
+        cacheVisitor.visit(row -> {
+            if (idx.canHandle(row))
+                idx.onUpdate(null, row, false);
+        });
+
+        return idx;
+    }
+
+    /**
+     * Creates a new index.
+     *
+     * @param cctx Cache context.
+     * @param factory Index factory.
+     * @param definition Description of an index to create.
+     */
+    public Index createIndex(GridCacheContext<?, ?> cctx, IndexFactory factory, IndexDefinition definition) {
+        ddlLock.writeLock().lock();
+
+        try {
+            String cacheName = definition.idxName().cacheName();
+
+            cacheToIdx.putIfAbsent(cacheName, new ConcurrentHashMap<>());
+
+            String uniqIdxName = definition.idxName().fullName();
+
+            // GridQueryProcessor already checked schema operation for index duplication.
+            assert cacheToIdx.get(cacheName).get(uniqIdxName) == null : "Duplicated index name " + uniqIdxName;
+
+            Index idx = factory.createIndex(cctx, definition);
+
+            cacheToIdx.get(cacheName).put(uniqIdxName, idx);
+
+            idxDefs.put(idx.id(), definition);
+
+            return idx;
+
+        } finally {
+            ddlLock.writeLock().unlock();
+        }
+    }
+
+    /**
+     * Removes an index.
+     *
+     * @param cctx Cache context.
+     * @param idxName Index name.
+     * @param softDelete whether it's required to delete underlying structures.
+     */
+    public void removeIndex(GridCacheContext<?, ?> cctx, IndexName idxName, boolean softDelete) {
+        ddlLock.writeLock().lock();
+
+        try {
+            String cacheName = idxName.cacheName();
+
+            Map<String, Index> idxs = cacheToIdx.get(cacheName);
+
+            assert idxs != null : "Try remove index for non registered cache " + cacheName;
+
+            Index idx = idxs.remove(idxName.fullName());
+
+            if (idx != null) {
+                idx.destroy(softDelete);
+                idxDefs.remove(idx.id());
+            }
+
+        } finally {
+            ddlLock.writeLock().unlock();
+        }
+    }
+
+    /** */
+    private void updateIndexes(String cacheName, CacheDataRow newRow, CacheDataRow prevRow,
+        boolean prevRowAvailable) throws IgniteCheckedException {
+        IgniteCheckedException err = null;
+
+        ddlLock.readLock().lock();
+
+        try {
+            Map<String, Index> indexes = cacheToIdx.get(cacheName);
+
+            if (F.isEmpty(indexes))
+                return;
+
+            for (Index idx: indexes.values())
+                err = updateIndex(idx, newRow, prevRow, prevRowAvailable, err);
+
+        } finally {
+            ddlLock.readLock().unlock();
+        }
+
+        if (err != null)
+            throw err;
+    }
+
+    /**
+     * Index row cache.
+     *
+     * @param grpId Cache group id.
+     * @return Index row cache.
+     */
+    public IndexRowCache rowCacheCleaner(int grpId) {
+        return idxRowCacheRegistry.forGroup(grpId);
+    }
+
+    /**
+     * Index row cache registry.
+     *
+     * @return Index row cache registry.
+     */
+    public IndexRowCacheRegistry idxRowCacheRegistry() {
+        return idxRowCacheRegistry;
+    }
+
+    /**
+     * Mark/unmark for rebuild indexes for a specific cache.
+     */
+    public void markRebuildIndexesForCache(GridCacheContext<?, ?> cctx, boolean val) {
+        ddlLock.readLock().lock();
+
+        try {
+            if (!cacheToIdx.containsKey(cctx.name()))
+                return;
+
+            Collection<Index> idxs = cacheToIdx.get(cctx.name()).values();
+
+            for (Index idx: idxs) {
+                if (idx instanceof AbstractIndex)
+                    ((AbstractIndex) idx).markIndexRebuild(val);
+            }
+
+        } finally {
+            ddlLock.readLock().unlock();
+        }
+    }
+
+    /**
+     * Start rebuild of indexes for specified cache.
+     */
+    public IgniteInternalFuture<?> rebuildIndexesForCache(GridCacheContext<?, ?> cctx) {
+        return idxRebuild.rebuild(cctx);
+    }
+
+    /** */
+    public IndexesRebuildTask idxRebuild() {
+        return idxRebuild;
+    }
+
+    /**
+     * Returns collection of indexes for specified cache.
+     *
+     * @param cctx Cache context.
+     * @return Collection of indexes for specified cache.
+     */
+    public Collection<Index> indexes(GridCacheContext<?, ?> cctx) {
+        ddlLock.readLock().lock();
+
+        try {
+            Map<String, Index> idxs = cacheToIdx.get(cctx.name());
+
+            if (idxs == null)
+                return Collections.emptyList();
+
+            return idxs.values();
+
+        } finally {
+            ddlLock.readLock().unlock();
+        }
+    }
+
+    /**
+     * Returns IndexDefinition used for creating index specified id.
+     *
+     * @param idxId UUID of index.
+     * @return IndexDefinition used for creating index with id {@code idxId}.
+     */
+    public IndexDefinition indexDefinition(UUID idxId) {
+        return idxDefs.get(idxId);
+    }
+
+    /**
+     * Unregisters cache.
+     *
+     * @param cacheInfo Cache context info.
+     */
+    public void unregisterCache(GridCacheContextInfo cacheInfo) {
+        idxRowCacheRegistry.onCacheUnregistered(cacheInfo);
+
+        idxRebuild.stopRebuild(cacheInfo, log);
+    }
+
+    /**
+     * Add row to index.
+     * @param idx Index to add row to.
+     * @param row Row to add to index.
+     * @param prevRow Previous row state, if any.
+     * @param prevRowAvailable Whether previous row is available.
+     * @param prevErr Error on index add.
+     */
+    private IgniteCheckedException updateIndex(
+        Index idx, CacheDataRow row, CacheDataRow prevRow, boolean prevRowAvailable, IgniteCheckedException prevErr
+    ) throws IgniteCheckedException {
+        try {
+            if (row != null && !idx.canHandle(row))
+                return prevErr;
+
+            if (prevRow != null && !idx.canHandle(prevRow))
+                return prevErr;
+
+            idx.onUpdate(prevRow, row, prevRowAvailable);
+
+            return prevErr;
+        }
+        catch (Throwable t) {
+            IgniteSQLException ex = X.cause(t, IgniteSQLException.class);
+
+            if (ex != null && ex.statusCode() == IgniteQueryErrorCode.FIELD_TYPE_MISMATCH) {
+                if (prevErr != null) {
+                    prevErr.addSuppressed(t);
+
+                    return prevErr;
+                }
+                else
+                    return new IgniteCheckedException("Error on add row to index.", t);
+            }
+            else
+                throw t;
+        }
+    }
+
+    /**
+     * Destroy founded index which belongs to stopped cache.
+     *
+     * @param page Root page.
+     * @param indexName Index name.
+     * @param grpId Group id which contains garbage.
+     * @param pageMemory Page memory to work with.
+     * @param removeId Global remove id.
+     * @param reuseList Reuse list where free pages should be stored.
+     * @param mvccEnabled Whether mvcc is enabled.
+     * @throws IgniteCheckedException If failed.
+     */
+    public void destroyOrphanIndex(
+        GridKernalContext ctx,
+        RootPage page,
+        String indexName,
+        int grpId,
+        PageMemory pageMemory,
+        final GridAtomicLong removeId,
+        final ReuseList reuseList,
+        boolean mvccEnabled) throws IgniteCheckedException {
+
+        assert ctx.cache().context().database().checkpointLockIsHeldByThread();
+
+        long metaPageId = page.pageId().pageId();
+
+        int inlineSize = inlineSize(page, grpId, pageMemory);
+
+        String grpName = ctx.cache().cacheGroup(grpId).cacheOrGroupName();
+
+        PageLockListener lockLsnr = ctx.cache().context().diagnostic()
+            .pageLockTracker().createPageLockTracker(grpName + "IndexTree##" + indexName);
+
+        BPlusTree<IndexRow, IndexRow> tree = new BPlusTree<IndexRow, IndexRow>(
+            indexName,
+            grpId,
+            grpName,
+            pageMemory,
+            ctx.cache().context().wal(),
+            removeId,
+            metaPageId,
+            reuseList,
+            AbstractInlineInnerIO.versions(inlineSize, mvccEnabled),
+            AbstractInlineLeafIO.versions(inlineSize, mvccEnabled),
+            PageIdAllocator.FLAG_IDX,
+            ctx.failure(),
+            lockLsnr
+        ) {
+            @Override protected int compare(BPlusIO io, long pageAddr, int idx, IndexRow row) {
+                throw new AssertionError();
+            }
+
+            @Override public IndexRow getRow(BPlusIO io, long pageAddr, int idx, Object x) {
+                throw new AssertionError();
+            }
+        };
+
+        tree.destroy();
+    }
+
+    /**
+     * @param page Root page.
+     * @param grpId Cache group id.
+     * @param pageMemory Page memory.
+     * @return Inline size.
+     * @throws IgniteCheckedException If something went wrong.
+     */
+    private int inlineSize(RootPage page, int grpId, PageMemory pageMemory) throws IgniteCheckedException {
+        long metaPageId = page.pageId().pageId();
+
+        final long metaPage = pageMemory.acquirePage(grpId, metaPageId);
+
+        try {
+            long pageAddr = pageMemory.readLock(grpId, metaPageId, metaPage); // Meta can't be removed.
+
+            assert pageAddr != 0 : "Failed to read lock meta page [metaPageId=" +
+                U.hexLong(metaPageId) + ']';
+
+            try {
+                BPlusMetaIO io = BPlusMetaIO.VERSIONS.forPage(pageAddr);
+
+                return io.getInlineSize(pageAddr);
+            }
+            finally {
+                pageMemory.readUnlock(grpId, metaPageId, metaPage);
+            }
+        }
+        finally {
+            pageMemory.releasePage(grpId, metaPageId, metaPage);
+        }
+    }
+
+    /**
+     * Defragment index partition.
+     *
+     * @param grpCtx Old group context.
+     * @param newCtx New group context.
+     * @param partPageMem Partition page memory.
+     * @param mappingByPart Mapping page memory.
+     * @param cpLock Defragmentation checkpoint read lock.
+     * @param cancellationChecker Cancellation checker.
+     * @param defragmentationThreadPool Thread pool for defragmentation.
+     *
+     * @throws IgniteCheckedException If failed.
+     */
+    public void defragment(
+        CacheGroupContext grpCtx,
+        CacheGroupContext newCtx,
+        PageMemoryEx partPageMem,
+        IntMap<LinkMap> mappingByPart,
+        CheckpointTimeoutLock cpLock,
+        Runnable cancellationChecker,
+        IgniteThreadPoolExecutor defragmentationThreadPool
+    ) throws IgniteCheckedException {
+        new IndexingDefragmentation(this)
+            .defragment(grpCtx, newCtx, partPageMem, mappingByPart, cpLock, cancellationChecker,
+                defragmentationThreadPool, log);
+    }
+
+    /**
+     * Collect indexes for rebuild.
+     *
+     * @param createdOnly Get only created indexes (not restored from dick).
+     */
+    public List<InlineIndex> treeIndexes(GridCacheContext cctx, boolean createdOnly) {
+        Collection<Index> idxs = indexes(cctx);
+
+        List<InlineIndex> treeIdxs = new ArrayList<>();
+
+        for (Index idx: idxs) {
+            if (idx instanceof InlineIndex) {
+                InlineIndex idx0 = (InlineIndex)idx;
+
+                if (!createdOnly || idx0.created())
+                    treeIdxs.add(idx0);
+            }
+        }
+
+        return treeIdxs;
+    }
+
+    /**
+     * @return Logger.
+     */
+    public IgniteLogger logger() {
+        return log;
+    }
+}
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryRowCacheCleaner.java b/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/NullsOrder.java
similarity index 72%
copy from modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryRowCacheCleaner.java
copy to modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/NullsOrder.java
index 6275813..9672219 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryRowCacheCleaner.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/NullsOrder.java
@@ -15,16 +15,15 @@
  * limitations under the License.
  */
 
-package org.apache.ignite.internal.processors.query;
+package org.apache.ignite.internal.cache.query.index;
 
 /**
- * Row cache cleaner is used by page memory manager to remove updated / evicted links from rows cache.
+ * Enum to store possible nulls ordering.
  */
-public interface GridQueryRowCacheCleaner {
-    /**
-     * Remove row by link.
-     *
-     * @param link Link to remove.
-     */
-    void remove(long link);
+public enum NullsOrder {
+    /** */
+    NULLS_LAST,
+
+    /** */
+    NULLS_FIRST
 }
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/io/H2LeafIO.java b/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/Order.java
similarity index 61%
copy from modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/io/H2LeafIO.java
copy to modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/Order.java
index 466cd1c..55e33af 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/io/H2LeafIO.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/Order.java
@@ -15,23 +15,31 @@
  * limitations under the License.
  */
 
-package org.apache.ignite.internal.processors.query.h2.database.io;
-
-import org.apache.ignite.internal.processors.cache.persistence.tree.io.IOVersions;
+package org.apache.ignite.internal.cache.query.index;
 
 /**
- * Leaf page for H2 row references.
+ * Represents ordering of rows within sorted index.
  */
-public class H2LeafIO extends AbstractH2LeafIO {
+public class Order {
+    /** */
+    private final NullsOrder nullsOrder;
+
+    /** */
+    private final SortOrder sortOrder;
+
     /** */
-    public static final IOVersions<H2LeafIO> VERSIONS = new IOVersions<>(
-        new H2LeafIO(1)
-    );
+    public Order(SortOrder sortOrder, NullsOrder nullsOrder) {
+        this.sortOrder = sortOrder;
+        this.nullsOrder = nullsOrder;
+    }
 
-    /**
-     * @param ver Page format version.
-     */
-    public H2LeafIO(int ver) {
-        super(T_H2_REF_LEAF, ver, 8);
+    /** */
+    public SortOrder sortOrder() {
+        return sortOrder;
+    }
+
+    /** */
+    public NullsOrder nullsOrder() {
+        return nullsOrder;
     }
 }
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/SingleCursor.java b/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/SingleCursor.java
new file mode 100644
index 0000000..ce0c2d4
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/SingleCursor.java
@@ -0,0 +1,58 @@
+/*
+ * 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.cache.query.index;
+
+import java.util.concurrent.atomic.AtomicInteger;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.internal.util.lang.GridCursor;
+
+/**
+ * Cursor that holds single value only.
+ * @param <T> class of value to return.
+ */
+public class SingleCursor<T> implements GridCursor<T> {
+    /** Value to return */
+    private final T val;
+
+    /** Counter ot check whether value is already got. */
+    private final AtomicInteger currIdx = new AtomicInteger(-1);
+
+    /** */
+    public SingleCursor(T val) {
+        this.val = val;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean next() {
+        return currIdx.incrementAndGet() == 0;
+    }
+
+    /**
+     * Note that this implementation violates the contract of GridCusror. It must be "before first" but this implementation
+     * handles both cases: "before first" and "on first". Current implementation of SQL with H2 relies on this {@code null}
+     * for queries like "select max(col) from table". This should be fixed for other SQL engines.
+     *
+     * https://issues.apache.org/jira/browse/IGNITE-14303.
+     */
+    @Override public T get() throws IgniteCheckedException {
+        if (currIdx.get() <= 0)
+            return val;
+
+        throw new IgniteCheckedException("No next element.");
+    }
+}
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryRowCacheCleaner.java b/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/SortOrder.java
similarity index 72%
copy from modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryRowCacheCleaner.java
copy to modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/SortOrder.java
index 6275813..9172075 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryRowCacheCleaner.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/SortOrder.java
@@ -15,16 +15,15 @@
  * limitations under the License.
  */
 
-package org.apache.ignite.internal.processors.query;
+package org.apache.ignite.internal.cache.query.index;
 
 /**
- * Row cache cleaner is used by page memory manager to remove updated / evicted links from rows cache.
+ * Enum of possible sort orders.
  */
-public interface GridQueryRowCacheCleaner {
-    /**
-     * Remove row by link.
-     *
-     * @param link Link to remove.
-     */
-    void remove(long link);
+public enum SortOrder {
+    /** */
+    ASC,
+
+    /** */
+    DESC
 }
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/DurableBackgroundCleanupIndexTreeTask.java b/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/DurableBackgroundCleanupIndexTreeTask.java
similarity index 74%
rename from modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/DurableBackgroundCleanupIndexTreeTask.java
rename to modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/DurableBackgroundCleanupIndexTreeTask.java
index 5305c17..4e23c11 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/DurableBackgroundCleanupIndexTreeTask.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/DurableBackgroundCleanupIndexTreeTask.java
@@ -14,27 +14,30 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.ignite.internal.processors.query.h2;
+package org.apache.ignite.internal.cache.query.index.sorted;
 
 import java.util.ArrayList;
 import java.util.Collections;
 import java.util.List;
 import java.util.UUID;
-import java.util.concurrent.atomic.AtomicInteger;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.IgniteException;
 import org.apache.ignite.internal.GridKernalContext;
+import org.apache.ignite.internal.cache.query.index.IndexName;
+import org.apache.ignite.internal.cache.query.index.sorted.inline.InlineIndexKeyType;
+import org.apache.ignite.internal.cache.query.index.sorted.inline.InlineIndexTree;
+import org.apache.ignite.internal.cache.query.index.sorted.keys.IndexKey;
 import org.apache.ignite.internal.metric.IoStatisticsHolderIndex;
 import org.apache.ignite.internal.pagemem.PageIdUtils;
 import org.apache.ignite.internal.pagemem.PageMemory;
 import org.apache.ignite.internal.processors.cache.CacheGroupContext;
 import org.apache.ignite.internal.processors.cache.GridCacheContext;
 import org.apache.ignite.internal.processors.cache.IgniteCacheOffheapManager;
+import org.apache.ignite.internal.processors.cache.persistence.CacheDataRow;
 import org.apache.ignite.internal.processors.cache.persistence.RootPage;
 import org.apache.ignite.internal.processors.cache.persistence.metastorage.pendingtask.DurableBackgroundTask;
 import org.apache.ignite.internal.processors.cache.persistence.tree.BPlusTree;
 import org.apache.ignite.internal.processors.cache.persistence.tree.io.PageIoResolver;
-import org.apache.ignite.internal.processors.query.h2.database.H2Tree;
 import org.apache.ignite.internal.util.typedef.internal.CU;
 import org.apache.ignite.internal.util.typedef.internal.S;
 
@@ -51,7 +54,7 @@ public class DurableBackgroundCleanupIndexTreeTask implements DurableBackgroundT
     private List<Long> rootPages;
 
     /** */
-    private transient List<H2Tree> trees;
+    private transient List<InlineIndexTree> trees;
 
     /** */
     private transient volatile boolean completed;
@@ -60,7 +63,7 @@ public class DurableBackgroundCleanupIndexTreeTask implements DurableBackgroundT
     private String cacheGrpName;
 
     /** */
-    private String cacheName;
+    private final String cacheName;
 
     /** */
     private String schemaName;
@@ -69,30 +72,29 @@ public class DurableBackgroundCleanupIndexTreeTask implements DurableBackgroundT
     private final String treeName;
 
     /** */
-    private String idxName;
+    private final String idxName;
 
     /** */
-    private String id;
+    private final String id;
 
     /** */
     public DurableBackgroundCleanupIndexTreeTask(
         List<Long> rootPages,
-        List<H2Tree> trees,
+        List<InlineIndexTree> trees,
         String cacheGrpName,
         String cacheName,
-        String schemaName,
-        String treeName,
-        String idxName
+        IndexName idxName,
+        String treeName
     ) {
         this.rootPages = rootPages;
         this.trees = trees;
         this.completed = false;
         this.cacheGrpName = cacheGrpName;
         this.cacheName = cacheName;
-        this.schemaName = schemaName;
-        this.treeName = treeName;
-        this.idxName = idxName;
         this.id = UUID.randomUUID().toString();
+        this.idxName = idxName.idxName();
+        this.schemaName = idxName.schemaName();
+        this.treeName = treeName;
     }
 
     /** {@inheritDoc} */
@@ -102,7 +104,7 @@ public class DurableBackgroundCleanupIndexTreeTask implements DurableBackgroundT
 
     /** {@inheritDoc} */
     @Override public void execute(GridKernalContext ctx) {
-        List<H2Tree> trees0 = trees;
+        List<InlineIndexTree> trees0 = trees;
 
         if (trees0 == null) {
             trees0 = new ArrayList<>(rootPages.size());
@@ -173,35 +175,11 @@ public class DurableBackgroundCleanupIndexTreeTask implements DurableBackgroundT
                 try {
                     String treeName = "deletedTree_" + i + "_" + shortName();
 
-                    H2Tree tree = new H2Tree(
-                        cctx,
-                        null,
-                        treeName,
-                        idxName,
-                        cacheName,
-                        null,
-                        offheap.reuseListForIndex(treeName),
-                        grpId,
-                        cacheGrpName,
-                        pageMem,
-                        ctx.cache().context().wal(),
-                        offheap.globalRemoveId(),
-                        rootPage,
-                        false,
-                        Collections.emptyList(),
-                        Collections.emptyList(),
-                        new AtomicInteger(0),
-                        false,
-                        false,
-                        false,
-                        null,
-                        ctx.failure(),
-                        null,
-                        stats,
-                        null,
-                        0,
-                        PageIoResolver.DEFAULT_PAGE_IO_RESOLVER
-                    );
+                    InlineIndexTree tree = new InlineIndexTree(
+                        null, cctx, treeName, cctx.offheap(), cctx.offheap().reuseListForIndex(treeName),
+                        cctx.dataRegion().pageMemory(), PageIoResolver.DEFAULT_PAGE_IO_RESOLVER,
+                        rootPage, false, 0, new IndexKeyTypeSettings(), null,
+                        stats, new NoopRowHandlerFactory(), null);
 
                     trees0.add(tree);
                 }
@@ -279,4 +257,46 @@ public class DurableBackgroundCleanupIndexTreeTask implements DurableBackgroundT
     @Override public String toString() {
         return S.toString(DurableBackgroundCleanupIndexTreeTask.class, this);
     }
+
+    /** */
+    private static class NoopRowHandlerFactory implements InlineIndexRowHandlerFactory {
+        /** {@inheritDoc} */
+        @Override public InlineIndexRowHandler create(SortedIndexDefinition sdef, IndexKeyTypeSettings keyTypeSettings) {
+            return new InlineIndexRowHandler() {
+                /** {@inheritDoc} */
+                @Override public IndexKey indexKey(int idx, CacheDataRow row) {
+                    return null;
+                }
+
+                /** {@inheritDoc} */
+                @Override public List<InlineIndexKeyType> inlineIndexKeyTypes() {
+                    return Collections.emptyList();
+                }
+
+                /** {@inheritDoc} */
+                @Override public List<IndexKeyDefinition> indexKeyDefinitions() {
+                    return Collections.emptyList();
+                }
+
+                @Override public IndexKeyTypeSettings indexKeyTypeSettings() {
+                    return null;
+                }
+
+                /** {@inheritDoc} */
+                @Override public int partition(CacheDataRow row) {
+                    return 0;
+                }
+
+                /** {@inheritDoc} */
+                @Override public Object cacheKey(CacheDataRow row) {
+                    return null;
+                }
+
+                /** {@inheritDoc} */
+                @Override public Object cacheValue(CacheDataRow row) {
+                    return null;
+                }
+            };
+        }
+    }
 }
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/IndexKeyDefinition.java b/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/IndexKeyDefinition.java
new file mode 100644
index 0000000..a7e6c5a
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/IndexKeyDefinition.java
@@ -0,0 +1,68 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.cache.query.index.sorted;
+
+import org.apache.ignite.internal.cache.query.index.Order;
+import org.apache.ignite.internal.cache.query.index.sorted.keys.IndexKey;
+import org.apache.ignite.internal.cache.query.index.sorted.keys.NullIndexKey;
+
+/**
+ * Defines a signle index key.
+ */
+public class IndexKeyDefinition {
+    /** Index key name. */
+    private final String name;
+
+    /** Index key type. {@link IndexKeyTypes}. */
+    private final int idxType;
+
+    /** Order. */
+    private final Order order;
+
+    /** */
+    public IndexKeyDefinition(String name, int idxType, Order order) {
+        this.idxType = idxType;
+        this.order = order;
+        this.name = name;
+    }
+
+    /** */
+    public Order order() {
+        return order;
+    }
+
+    /** */
+    public int idxType() {
+        return idxType;
+    }
+
+    /** */
+    public String name() {
+        return name;
+    }
+
+    /**
+     * @return {@code true} if specified key's type matches to the current type, otherwise {@code false}.
+     */
+    public boolean validate(IndexKey key) {
+        if (key == NullIndexKey.INSTANCE)
+            return true;
+
+        return idxType == key.type();
+    }
+}
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/IndexKeyTypeSettings.java b/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/IndexKeyTypeSettings.java
new file mode 100644
index 0000000..09e2092
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/IndexKeyTypeSettings.java
@@ -0,0 +1,81 @@
+/*
+ * 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.cache.query.index.sorted;
+
+/**
+ * List of settings that affects key types of index keys.
+ */
+public class IndexKeyTypeSettings {
+    /** Whether inlining POJO keys as hash is supported. */
+    private boolean inlineObjHash = true;
+
+    /** Whether inlining of POJO keys is supported. */
+    private boolean inlineObjSupported = true;
+
+    /** Whether optimized algorithm of String comparison is used. */
+    private boolean strOptimizedCompare = true;
+
+    /** Whether use unsigned bytes for storing byte arrays. */
+    private boolean binaryUnsigned = true;
+
+    /** */
+    public boolean inlineObjHash() {
+        return inlineObjHash;
+    }
+
+    /** */
+    public IndexKeyTypeSettings inlineObjHash(boolean inlineObjHash) {
+        this.inlineObjHash = inlineObjHash;
+
+        return this;
+    }
+
+    /** */
+    public boolean inlineObjSupported() {
+        return inlineObjSupported;
+    }
+
+    /** */
+    public IndexKeyTypeSettings inlineObjSupported(boolean inlineObjSupported) {
+        this.inlineObjSupported = inlineObjSupported;
+
+        return this;
+    }
+
+    /** */
+    public boolean stringOptimizedCompare() {
+        return strOptimizedCompare;
+    }
+
+    /** */
+    public IndexKeyTypeSettings stringOptimizedCompare(boolean strOptimizedCompare) {
+        this.strOptimizedCompare = strOptimizedCompare;
+
+        return this;
+    }
+
+    /** */
+    public boolean binaryUnsigned() { return binaryUnsigned; }
+
+    /** */
+    public IndexKeyTypeSettings binaryUnsigned(boolean binaryUnsigned) {
+        this.binaryUnsigned = binaryUnsigned;
+
+        return this;
+    }
+}
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/IndexKeyTypes.java b/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/IndexKeyTypes.java
new file mode 100644
index 0000000..cea3121
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/IndexKeyTypes.java
@@ -0,0 +1,133 @@
+/*
+ * 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.cache.query.index.sorted;
+
+/**
+ * List of available types to use as index key.
+ */
+public class IndexKeyTypes {
+    /** The data type is unknown at this time. */
+    public static final int UNKNOWN = -1;
+
+    /** The value type for NULL. */
+    public static final int NULL = 0;
+
+    /** The value type for BOOLEAN values. */
+    public static final int BOOLEAN = 1;
+
+    /** The value type for BYTE values. */
+    public static final int BYTE = 2;
+
+    /** The value type for SHORT values. */
+    public static final int SHORT = 3;
+
+    /** The value type for INT values. */
+    public static final int INT = 4;
+
+    /** The value type for INT values. */
+    public static final int LONG = 5;
+
+    /** The value type for DECIMAL values. */
+    public static final int DECIMAL = 6;
+
+    /** The value type for DOUBLE values. */
+    public static final int DOUBLE = 7;
+
+    /** The value type for FLOAT values. */
+    public static final int FLOAT = 8;
+
+    /** The value type for TIME values. */
+    public static final int TIME = 9;
+
+    /**
+     * The value type for DATE values.
+     */
+    public static final int DATE = 10;
+
+    /**
+     * The value type for TIMESTAMP values.
+     */
+    public static final int TIMESTAMP = 11;
+
+    /**
+     * The value type for BYTES values.
+     */
+    public static final int BYTES = 12;
+
+    /**
+     * The value type for STRING values.
+     */
+    public static final int STRING = 13;
+
+    /**
+     * The value type for case insensitive STRING values.
+     */
+    public static final int STRING_IGNORECASE = 14;
+
+    /**
+     * The value type for BLOB values.
+     */
+    public static final int BLOB = 15;
+
+    /**
+     * The value type for CLOB values.
+     */
+    public static final int CLOB = 16;
+
+    /**
+     * The value type for ARRAY values.
+     */
+    public static final int ARRAY = 17;
+
+    /**
+     * The value type for RESULT_SET values.
+     */
+    public static final int RESULT_SET = 18;
+
+    /**
+     * The value type for JAVA_OBJECT values.
+     */
+    public static final int JAVA_OBJECT = 19;
+
+    /**
+     * The value type for UUID values.
+     */
+    public static final int UUID = 20;
+
+    /**
+     * The value type for string values with a fixed size.
+     */
+    public static final int STRING_FIXED = 21;
+
+    /**
+     * The value type for string values with a fixed size.
+     */
+    public static final int GEOMETRY = 22;
+
+    // 23 was a short-lived experiment "TIMESTAMP UTC" which has been removed.
+
+    /**
+     * The value type for TIMESTAMP WITH TIME ZONE values.
+     */
+    public static final int TIMESTAMP_TZ = 24;
+
+    /**
+     * The value type for ENUM values.
+     */
+    public static final int ENUM = 25;
+}
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/IndexRow.java b/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/IndexRow.java
new file mode 100644
index 0000000..ac2e8f9
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/IndexRow.java
@@ -0,0 +1,86 @@
+/*
+ * 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.cache.query.index.sorted;
+
+import org.apache.ignite.internal.cache.query.index.sorted.inline.InlineIndexTree;
+import org.apache.ignite.internal.cache.query.index.sorted.keys.IndexKey;
+import org.apache.ignite.internal.processors.cache.mvcc.MvccVersionAware;
+import org.apache.ignite.internal.processors.cache.mvcc.txlog.TxState;
+import org.apache.ignite.internal.processors.cache.persistence.CacheDataRow;
+
+import static org.apache.ignite.internal.processors.cache.mvcc.MvccUtils.MVCC_COUNTER_NA;
+import static org.apache.ignite.internal.processors.cache.mvcc.MvccUtils.MVCC_CRD_COUNTER_NA;
+import static org.apache.ignite.internal.processors.cache.mvcc.MvccUtils.MVCC_OP_COUNTER_NA;
+
+/**
+ * Represents an index row stored in a tree.
+ */
+public interface IndexRow extends MvccVersionAware {
+    /**
+     * @param idx Index of a key.
+     * @return Underlying key by specified index.
+     */
+    public IndexKey key(int idx);
+
+    /**
+     * @return Underlying keys.
+     */
+    public IndexKey[] keys();
+
+    /**
+     * @return Link to a cache row.
+     */
+    public long link();
+
+    /**
+     * @return Schema of an index.
+     */
+    public InlineIndexRowHandler rowHandler();
+
+    /**
+     * @return Cache row.
+     */
+    public CacheDataRow cacheDataRow();
+
+    // MVCC stuff.
+
+    /** {@inheritDoc} */
+    @Override public default long mvccCoordinatorVersion() {
+        return MVCC_CRD_COUNTER_NA;
+    }
+
+    /** {@inheritDoc} */
+    @Override public default long mvccCounter() {
+        return MVCC_COUNTER_NA;
+    }
+
+    /** {@inheritDoc} */
+    @Override public default int mvccOperationCounter() {
+        return MVCC_OP_COUNTER_NA;
+    }
+
+    /** {@inheritDoc} */
+    @Override public default byte mvccTxState() {
+        return TxState.NA;
+    }
+
+    /**
+     * @return {@code True} for rows used for index search (as opposed to rows stored in {@link InlineIndexTree}.
+     */
+    public boolean indexSearchRow();
+}
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/H2RowCache.java b/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/IndexRowCache.java
similarity index 86%
rename from modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/H2RowCache.java
rename to modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/IndexRowCache.java
index c16fd9a..7e714ed 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/H2RowCache.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/IndexRowCache.java
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package org.apache.ignite.internal.processors.query.h2;
+package org.apache.ignite.internal.cache.query.index.sorted;
 
 import java.util.Iterator;
 import java.util.Map;
@@ -25,7 +25,6 @@ import org.apache.ignite.internal.pagemem.PageMemory;
 import org.apache.ignite.internal.processors.cache.CacheGroupContext;
 import org.apache.ignite.internal.processors.cache.GridCacheContextInfo;
 import org.apache.ignite.internal.processors.query.GridQueryRowCacheCleaner;
-import org.apache.ignite.internal.processors.query.h2.opt.H2CacheRow;
 import org.apache.ignite.internal.util.typedef.F;
 import org.jsr166.ConcurrentLinkedHashMap;
 
@@ -35,9 +34,9 @@ import static org.jsr166.ConcurrentLinkedHashMap.DFLT_LOAD_FACTOR;
 /**
  * H2 row cache.
  */
-public class H2RowCache implements GridQueryRowCacheCleaner {
+public class IndexRowCache implements GridQueryRowCacheCleaner {
     /** Cached rows. */
-    private final ConcurrentLinkedHashMap<Long, H2CacheRow> rows;
+    private final ConcurrentLinkedHashMap<Long, IndexRowImpl> rows;
 
     /** Cache group ID. */
     private final CacheGroupContext grpCtx;
@@ -48,7 +47,7 @@ public class H2RowCache implements GridQueryRowCacheCleaner {
     /**
      * @param grpCtx Cache group context.
      */
-    public H2RowCache(CacheGroupContext grpCtx, int maxSize) {
+    public IndexRowCache(CacheGroupContext grpCtx, int maxSize) {
         this.grpCtx = grpCtx;
 
         rows = new ConcurrentLinkedHashMap<>(
@@ -66,8 +65,8 @@ public class H2RowCache implements GridQueryRowCacheCleaner {
      * @return Cached on-heap row.
      * @throws IgniteCheckedException On error.
      */
-    public H2CacheRow get(long link) throws IgniteCheckedException {
-        H2CacheRow row = rows.get(link);
+    public IndexRowImpl get(long link) throws IgniteCheckedException {
+        IndexRowImpl row = rows.get(link);
 
         if (row != null)
             touch(link);
@@ -80,7 +79,7 @@ public class H2RowCache implements GridQueryRowCacheCleaner {
      *
      * @param row Row.
      */
-    public void put(H2CacheRow row) {
+    public void put(IndexRowImpl row) {
         rows.put(row.link(), row);
     }
 
@@ -125,10 +124,10 @@ public class H2RowCache implements GridQueryRowCacheCleaner {
     private void clearForCache(GridCacheContextInfo cacheInfo) {
         int cacheId = cacheInfo.cacheId();
 
-        Iterator<Map.Entry<Long, H2CacheRow>> iter = rows.entrySet().iterator();
+        Iterator<Map.Entry<Long, IndexRowImpl>> iter = rows.entrySet().iterator();
 
         while (iter.hasNext()) {
-            H2CacheRow row = iter.next().getValue();
+            IndexRowImpl row = iter.next().getValue();
 
             if (F.eq(cacheId, row.cacheId()))
                 iter.remove();
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/H2RowCacheRegistry.java b/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/IndexRowCacheRegistry.java
similarity index 84%
rename from modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/H2RowCacheRegistry.java
rename to modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/IndexRowCacheRegistry.java
index 7c06cd0..b482211 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/H2RowCacheRegistry.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/IndexRowCacheRegistry.java
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package org.apache.ignite.internal.processors.query.h2;
+package org.apache.ignite.internal.cache.query.index.sorted;
 
 import java.util.HashMap;
 import java.util.Map;
@@ -25,14 +25,14 @@ import org.apache.ignite.internal.processors.cache.IgniteCacheOffheapManager;
 import org.jetbrains.annotations.Nullable;
 
 /**
- * H2 row cache registry.
+ * Index row cache registry.
  */
-public class H2RowCacheRegistry {
+public class IndexRowCacheRegistry {
     /** Mutex. */
     private final Object mux = new Object();
 
     /** Row caches for specific cache groups. */
-    private volatile Map<Integer, H2RowCache> caches;
+    private volatile Map<Integer, IndexRowCache> caches;
 
     /**
      * Get row cache for the given cache group.
@@ -40,7 +40,7 @@ public class H2RowCacheRegistry {
      * @param grpId Cache group ID.
      * @return Row cache or {@code null} if none available.
      */
-    @Nullable public H2RowCache forGroup(int grpId) {
+    @Nullable public IndexRowCache forGroup(int grpId) {
         return caches != null ? caches.get(grpId) : null;
     }
 
@@ -57,7 +57,7 @@ public class H2RowCacheRegistry {
             int grpId = cacheInfo.groupId();
 
             if (caches != null) {
-                H2RowCache cache = caches.get(grpId);
+                IndexRowCache cache = caches.get(grpId);
 
                 if (cache != null) {
                     cache.onCacheRegistered();
@@ -66,14 +66,14 @@ public class H2RowCacheRegistry {
                 }
             }
 
-            HashMap<Integer, H2RowCache> caches0 = copy();
+            HashMap<Integer, IndexRowCache> caches0 = copy();
 
             if (cacheInfo.affinityNode()) {
                 GridCacheContext cacheCtx = cacheInfo.cacheContext();
 
                 assert cacheCtx != null;
 
-                H2RowCache rowCache = new H2RowCache(cacheCtx.group(), cacheInfo.config().getSqlOnheapCacheMaxSize());
+                IndexRowCache rowCache = new IndexRowCache(cacheCtx.group(), cacheInfo.config().getSqlOnheapCacheMaxSize());
 
                 caches0.put(grpId, rowCache);
 
@@ -102,12 +102,12 @@ public class H2RowCacheRegistry {
 
             assert caches != null;
 
-            H2RowCache cache = caches.get(grpId);
+            IndexRowCache cache = caches.get(grpId);
 
             assert cache != null;
 
             if (cache.onCacheUnregistered(cacheInfo)) {
-                HashMap<Integer, H2RowCache> caches0 = copy();
+                HashMap<Integer, IndexRowCache> caches0 = copy();
 
                 caches0.remove(grpId);
 
@@ -121,7 +121,7 @@ public class H2RowCacheRegistry {
      *
      * @return Copy.
      */
-    private HashMap<Integer, H2RowCache> copy() {
+    private HashMap<Integer, IndexRowCache> copy() {
         assert Thread.holdsLock(mux);
 
         if (caches == null)
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/IndexRowComparator.java b/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/IndexRowComparator.java
new file mode 100644
index 0000000..c388c41
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/IndexRowComparator.java
@@ -0,0 +1,46 @@
+/*
+ * 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.cache.query.index.sorted;
+
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.internal.cache.query.index.sorted.keys.IndexKey;
+
+/**
+ * Comparator for index rows.
+ */
+public interface IndexRowComparator {
+    /**
+     * Compare index keys.
+     *
+     * @param pageAddr address of an index row.
+     * @param off offset of an index key.
+     * @param maxSize max size to read.
+     * @param key key to compare with.
+     * @param curType type of an index key.
+     */
+    public int compareKey(long pageAddr, int off, int maxSize, IndexKey key, int curType) throws IgniteCheckedException;
+
+    /**
+     * Compare index keys.
+     *
+     * @param left index row.
+     * @param right index row.
+     * @param idx offset of index key.
+     */
+    public int compareKey(IndexRow left, IndexRow right, int idx) throws IgniteCheckedException;
+}
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/IndexRowCompartorImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/IndexRowCompartorImpl.java
new file mode 100644
index 0000000..3b2b919
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/IndexRowCompartorImpl.java
@@ -0,0 +1,71 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ *
+ */
+
+package org.apache.ignite.internal.cache.query.index.sorted;
+
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.internal.cache.query.index.sorted.keys.IndexKey;
+import org.apache.ignite.internal.cache.query.index.sorted.keys.NullIndexKey;
+
+import static org.apache.ignite.internal.cache.query.index.sorted.inline.types.NullableInlineIndexKeyType.CANT_BE_COMPARE;
+import static org.apache.ignite.internal.cache.query.index.sorted.inline.types.NullableInlineIndexKeyType.COMPARE_UNSUPPORTED;
+
+/**
+ * Provide default logic of rows comparison.
+ *
+ * Consider:
+ * 1. NULL is the least value.
+ * 2. Comparison of different types is not supported.
+ */
+public class IndexRowCompartorImpl implements IndexRowComparator {
+    /** Key type settings for this index. */
+    protected final IndexKeyTypeSettings keyTypeSettings;
+
+    /** */
+    public IndexRowCompartorImpl(IndexKeyTypeSettings keyTypeSettings) {
+        this.keyTypeSettings = keyTypeSettings;
+    }
+
+    /** {@inheritDoc} */
+    @Override public int compareKey(long pageAddr, int off, int maxSize, IndexKey key, int curType) {
+        if (curType == IndexKeyTypes.UNKNOWN)
+            return CANT_BE_COMPARE;
+
+        if (key == NullIndexKey.INSTANCE)
+            return 1;
+
+        // Check that types are different before that.
+        return COMPARE_UNSUPPORTED;
+    }
+
+    /** {@inheritDoc} */
+    @Override public int compareKey(IndexRow left, IndexRow right, int idx) throws IgniteCheckedException {
+        IndexKey lkey = left.key(idx);
+        IndexKey rkey = right.key(idx);
+
+        if (lkey == NullIndexKey.INSTANCE)
+            return lkey.compare(rkey);
+        else if (rkey == NullIndexKey.INSTANCE)
+            return 1;
+
+        if (lkey.type() == rkey.type())
+            return lkey.compare(rkey);
+
+        return COMPARE_UNSUPPORTED;
+    }
+}
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/IndexRowImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/IndexRowImpl.java
new file mode 100644
index 0000000..2cb2310
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/IndexRowImpl.java
@@ -0,0 +1,176 @@
+/*
+ * 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.cache.query.index.sorted;
+
+import org.apache.ignite.internal.cache.query.index.sorted.keys.IndexKey;
+import org.apache.ignite.internal.processors.cache.CacheObject;
+import org.apache.ignite.internal.processors.cache.persistence.CacheDataRow;
+import org.apache.ignite.internal.processors.query.QueryUtils;
+import org.apache.ignite.internal.util.typedef.internal.S;
+import org.apache.ignite.internal.util.typedef.internal.SB;
+
+import static org.apache.ignite.IgniteSystemProperties.IGNITE_TO_STRING_INCLUDE_SENSITIVE;
+
+/**
+ * This class represents a row in {@link SortedSegmentedIndex}.
+ */
+public class IndexRowImpl implements IndexRow {
+    /** Object that contains info about original IgniteCache row. */
+    private final CacheDataRow cacheRow;
+
+    /** Cache for index row keys. To avoid hit underlying cache for every comparation. */
+    private IndexKey[] keyCache;
+
+    /** Schema of an index. */
+    private final InlineIndexRowHandler rowHnd;
+
+    /** Constructor. */
+    public IndexRowImpl(InlineIndexRowHandler rowHnd, CacheDataRow row) {
+        this(rowHnd, row, null);
+    }
+
+    /**
+     * Constructor with prefilling of keys cache.
+     */
+    public IndexRowImpl(InlineIndexRowHandler rowHnd, CacheDataRow row, IndexKey[] keys) {
+        this.rowHnd = rowHnd;
+        cacheRow = row;
+        keyCache = keys;
+    }
+
+    /**
+     * @return Indexed value.
+     */
+    public CacheObject value() {
+        return cacheRow.value();
+    }
+
+    /** {@inheritDoc} */
+    @Override public IndexKey key(int idx) {
+        if (keyCache != null && keyCache[idx] != null)
+            return keyCache[idx];
+
+        IndexKey key = rowHnd.indexKey(idx, cacheRow);
+
+        if (keyCache != null)
+            keyCache[idx] = key;
+
+        return key;
+    }
+
+    /** {@inheritDoc} */
+    @Override public IndexKey[] keys() {
+        int keysCnt = rowHnd.indexKeyDefinitions().size();
+
+        IndexKey[] keys = new IndexKey[keysCnt];
+
+        for (int i = 0; i < keysCnt; ++i)
+            keys[i] = key(i);
+
+        return keys;
+    }
+
+    /** {@inheritDoc} */
+    @Override public long link() {
+        return cacheRow.link();
+    }
+
+    /** {@inheritDoc} */
+    @Override public InlineIndexRowHandler rowHandler() {
+        return rowHnd;
+    }
+
+    /** {@inheritDoc} */
+    @Override public CacheDataRow cacheDataRow() {
+        return cacheRow;
+    }
+
+    /**
+     * @return Cache ID or {@code 0} if cache ID is not defined.
+     */
+    public int cacheId() {
+        return cacheDataRow().cacheId();
+    }
+
+    /** Initialize a cache for index keys. Useful for inserting rows as there are a lot of comparisons. */
+    public void prepareCache() {
+        keyCache = new IndexKey[rowHnd.indexKeyDefinitions().size()];
+    }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        SB sb = new SB("Row@");
+
+        sb.a(Integer.toHexString(System.identityHashCode(this)));
+
+        Object v = rowHnd.cacheKey(cacheRow);
+
+        sb.a("[ key: ").a(v == null ? "nil" : v.toString());
+
+        v = rowHnd.cacheValue(cacheRow);
+        sb.a(", val: ").a(v == null ? "nil" : (S.includeSensitive() ? v.toString() :
+            "Data hidden due to " + IGNITE_TO_STRING_INCLUDE_SENSITIVE + " flag."));
+
+        sb.a(" ][ ");
+
+        if (v != null) {
+            for (int i = QueryUtils.DEFAULT_COLUMNS_COUNT, cnt = rowHnd.indexKeyDefinitions().size(); i < cnt; i++) {
+                if (i != QueryUtils.DEFAULT_COLUMNS_COUNT)
+                    sb.a(", ");
+
+                try {
+                    v = key(i);
+
+                    sb.a(v == null ? "nil" : (S.includeSensitive() ? v.toString() : "data hidden"));
+                }
+                catch (Exception e) {
+                    sb.a("<value skipped on error: " + e.getMessage() + '>');
+                }
+            }
+        }
+
+        sb.a(" ]");
+
+        return sb.toString();
+    }
+
+    /** {@inheritDoc} */
+    @Override public long mvccCoordinatorVersion() {
+        return cacheRow.mvccCoordinatorVersion();
+    }
+
+    /** {@inheritDoc} */
+    @Override public long mvccCounter() {
+        return cacheRow.mvccCounter();
+    }
+
+    /** {@inheritDoc} */
+    @Override public int mvccOperationCounter() {
+        return cacheRow.mvccOperationCounter();
+    }
+
+    /** {@inheritDoc} */
+    @Override public byte mvccTxState() {
+        return cacheRow.mvccTxState();
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean indexSearchRow() {
+        return false;
+    }
+}
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/IndexSearchRowImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/IndexSearchRowImpl.java
new file mode 100644
index 0000000..a800da0
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/IndexSearchRowImpl.java
@@ -0,0 +1,78 @@
+/*
+ * 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.cache.query.index.sorted;
+
+import org.apache.ignite.internal.cache.query.index.sorted.keys.IndexKey;
+import org.apache.ignite.internal.processors.cache.persistence.CacheDataRow;
+import org.apache.ignite.internal.util.typedef.internal.S;
+
+/**
+ * Represents a search row that used to find a place in a tree.
+ */
+public class IndexSearchRowImpl implements IndexRow {
+    /** */
+    private final IndexKey[] keys;
+
+    /** */
+    private final InlineIndexRowHandler rowHnd;
+
+    /** Constructor. */
+    public IndexSearchRowImpl(IndexKey[] idxKeys, InlineIndexRowHandler rowHnd) {
+        keys = idxKeys;
+        this.rowHnd = rowHnd;
+    }
+
+    /** {@inheritDoc} */
+    @Override public IndexKey key(int idx) {
+        return keys[idx];
+    }
+
+    /** {@inheritDoc} */
+    @Override public IndexKey[] keys() {
+        return keys;
+    }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return S.toString(IndexSearchRowImpl.class, this);
+    }
+
+    /** {@inheritDoc} */
+    @Override public long link() {
+        assert false : "Should not get link by IndexSearchRowImpl";
+
+        return 0;
+    }
+
+    /** {@inheritDoc} */
+    @Override public InlineIndexRowHandler rowHandler() {
+        return rowHnd;
+    }
+
+    /** {@inheritDoc} */
+    @Override public CacheDataRow cacheDataRow() {
+        assert false : "Should not cache data row by IndexSearchRowImpl";
+
+        return null;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean indexSearchRow() {
+        return true;
+    }
+}
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/IndexValueCursor.java b/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/IndexValueCursor.java
new file mode 100644
index 0000000..2272e4f
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/IndexValueCursor.java
@@ -0,0 +1,72 @@
+/*
+ * 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.cache.query.index.sorted;
+
+import java.util.function.Function;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.internal.util.lang.GridCursor;
+
+/**
+ * Cursor over index values.
+ *
+ * @param <V> class represents of value stored in an index.
+ */
+public class IndexValueCursor<V> implements GridCursor<V> {
+    /**
+     * Empty cursor implementation.
+     */
+    public static final GridCursor EMPTY = new GridCursor() {
+        /** {@inheritDoc} */
+        @Override public boolean next() {
+            return false;
+        }
+
+        /**
+         * Note that this implimentation violates the contract of GridCusror. It should throw {@code NoSuchElementException}
+         * instead of returning {@code null}. But current implementation of SQL with H2 relies on this {@code null}
+         * for queries like "select max(col) from table". This should be fixed for other SQL engines.
+         *
+         * https://issues.apache.org/jira/browse/IGNITE-14303.
+         */
+        @Override public Object get() {
+            return null;
+        }
+    };
+
+    /** Underlying cursor over original index rows. */
+    private final GridCursor<IndexRow> delegate;
+
+    /** Map function that transforms index row to index value. */
+    private final Function<IndexRow, V> mapFunc;
+
+    /** */
+    public IndexValueCursor(GridCursor<IndexRow> delegate, Function<IndexRow, V> mapFunc) {
+        this.delegate = delegate;
+        this.mapFunc = mapFunc;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean next() throws IgniteCheckedException {
+        return delegate.next();
+    }
+
+    /** {@inheritDoc} */
+    @Override public V get() throws IgniteCheckedException {
+        return mapFunc.apply(delegate.get());
+    }
+}
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/InlineIndexRowHandler.java b/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/InlineIndexRowHandler.java
new file mode 100644
index 0000000..e9fe0c4
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/InlineIndexRowHandler.java
@@ -0,0 +1,69 @@
+/*
+ * 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.cache.query.index.sorted;
+
+import java.util.List;
+import org.apache.ignite.internal.cache.query.index.sorted.inline.InlineIndexKeyType;
+import org.apache.ignite.internal.cache.query.index.sorted.keys.IndexKey;
+import org.apache.ignite.internal.processors.cache.persistence.CacheDataRow;
+
+/**
+ * Handles InlineIndexRow. Stores information about inlined keys, and rules how to convert CacheDataRow to IndexRow.
+ */
+public interface InlineIndexRowHandler {
+    /**
+     * Returns index key by specified idx.
+     *
+     * @param idx Index of key within index schema.
+     * @param row Original cache data row.
+     * @return Index key.
+     */
+    public IndexKey indexKey(int idx, CacheDataRow row);
+
+    /**
+     * @return List of key types for inlined index keys.
+     */
+    public List<InlineIndexKeyType> inlineIndexKeyTypes();
+
+    /**
+     * @return List of index key definitions.
+     */
+    public List<IndexKeyDefinition> indexKeyDefinitions();
+
+    /**
+     * @return Index key type settings.
+     */
+    public IndexKeyTypeSettings indexKeyTypeSettings();
+
+    /**
+     * @return Parition for specified row.
+     */
+    public int partition(CacheDataRow row);
+
+    /**
+     * @param row Cache row.
+     * @return Cache key.
+     */
+    public Object cacheKey(CacheDataRow row);
+
+    /**
+     * @param row Cache row.
+     * @return Cache value.
+     */
+    public Object cacheValue(CacheDataRow row);
+}
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/io/H2LeafIO.java b/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/InlineIndexRowHandlerFactory.java
similarity index 58%
copy from modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/io/H2LeafIO.java
copy to modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/InlineIndexRowHandlerFactory.java
index 466cd1c..164cac0 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/io/H2LeafIO.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/InlineIndexRowHandlerFactory.java
@@ -15,23 +15,20 @@
  * limitations under the License.
  */
 
-package org.apache.ignite.internal.processors.query.h2.database.io;
+package org.apache.ignite.internal.cache.query.index.sorted;
 
-import org.apache.ignite.internal.processors.cache.persistence.tree.io.IOVersions;
+import org.apache.ignite.IgniteCheckedException;
 
 /**
- * Leaf page for H2 row references.
+ * Basic interface for factories that provide InlineIndexRowHandler based on index definition and optional arguments.
  */
-public class H2LeafIO extends AbstractH2LeafIO {
-    /** */
-    public static final IOVersions<H2LeafIO> VERSIONS = new IOVersions<>(
-        new H2LeafIO(1)
-    );
-
+public interface InlineIndexRowHandlerFactory {
     /**
-     * @param ver Page format version.
+     * Creates instance of row handler.
+     * @param sdef Index definition.
+     * @param keyTypeSettings Index key type settings.
+     * @return Index row handler.
      */
-    public H2LeafIO(int ver) {
-        super(T_H2_REF_LEAF, ver, 8);
-    }
+    public InlineIndexRowHandler create(SortedIndexDefinition sdef, IndexKeyTypeSettings keyTypeSettings)
+        throws IgniteCheckedException;
 }
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/MetaPageInfo.java b/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/MetaPageInfo.java
new file mode 100644
index 0000000..e4057db
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/MetaPageInfo.java
@@ -0,0 +1,96 @@
+/*
+ * 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.cache.query.index.sorted;
+
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.BPlusMetaIO;
+import org.apache.ignite.lang.IgniteProductVersion;
+
+/**
+ * Meta page stores meta data about InlineIndexTree.
+ */
+public class MetaPageInfo {
+    /** Inline size used for a tree. */
+    private final int inlineSize;
+
+    /** Whether index use wrapped / unwrapped PK. */
+    private final boolean useUnwrappedPk;
+
+    /** Whether any flags is supported. */
+    private final boolean flagsSupported;
+
+    /** Whether inlinining of java objects is supported. */
+    private boolean inlineObjSupported;
+
+    /** Whether inlinining of java objects as hash is supported. */
+    private boolean inlineObjHash;
+
+    /** Version of Ignite. */
+    private final IgniteProductVersion createdVer;
+
+    /**
+     * @param io Metapage IO.
+     * @param pageAddr Page address.
+     */
+    public MetaPageInfo(BPlusMetaIO io, long pageAddr) {
+        inlineSize = io.getInlineSize(pageAddr);
+        useUnwrappedPk = io.unwrappedPk(pageAddr);
+        flagsSupported = io.supportFlags();
+
+        if (flagsSupported) {
+            inlineObjSupported = io.inlineObjectSupported(pageAddr);
+            inlineObjHash = io.inlineObjectHash(pageAddr);
+        }
+
+        createdVer = io.createdVersion(pageAddr);
+    }
+
+    /**
+     * @return Inline size.
+     */
+    public int inlineSize() {
+        return inlineSize;
+    }
+
+    /**
+     * @return {@code true} In case use unwrapped PK for indexes.
+     */
+    public boolean useUnwrappedPk() {
+        return useUnwrappedPk;
+    }
+
+    /**
+     * @return {@code true} In case metapage contains flags.
+     */
+    public boolean flagsSupported() {
+        return flagsSupported;
+    }
+
+    /**
+     * @return {@code true} In case inline object is supported.
+     */
+    public boolean inlineObjectSupported() {
+        return inlineObjSupported;
+    }
+
+    /**
+     * @return {@code true} In case inline object is supported.
+     */
+    public boolean inlineObjectHash() {
+        return inlineObjHash;
+    }
+}
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/SortedIndexDefinition.java b/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/SortedIndexDefinition.java
new file mode 100644
index 0000000..5d506b5
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/SortedIndexDefinition.java
@@ -0,0 +1,62 @@
+/*
+ * 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.cache.query.index.sorted;
+
+import java.util.List;
+import org.apache.ignite.internal.cache.query.index.IndexDefinition;
+
+/**
+ * Represents a definition of a sorted index.
+ */
+public interface SortedIndexDefinition extends IndexDefinition {
+    /** Represents an index tree name. */
+    public String treeName();
+
+    /** List of index key definitions. */
+    public List<IndexKeyDefinition> indexKeyDefinitions();
+
+    /** Comparator for comparing index rows. */
+    public IndexRowComparator rowComparator();
+
+    /** Index row handler. */
+    public InlineIndexRowHandlerFactory rowHandlerFactory();
+
+    /** Index key type settings. */
+    public IndexKeyTypeSettings keyTypeSettings();
+
+    /** Cache of index rows. */
+    public IndexRowCache idxRowCache();
+
+    /** Amount of index tree segments.*/
+    public int segments();
+
+    /** Inline size. */
+    public int inlineSize();
+
+    /** Whether this index is primary key (unique) or not. */
+    public boolean primary();
+
+    /** Whether this index is affinity key index or not. */
+    public boolean affinity();
+
+    /**
+     * @param created Whether index is creating or restored from PDS.
+     * @param metaPageInfo Initialize internal state with data from a tree meta.
+     */
+    public void initByMeta(boolean created, MetaPageInfo metaPageInfo);
+}
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/SortedSegmentedIndex.java b/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/SortedSegmentedIndex.java
new file mode 100644
index 0000000..3c6e8c7
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/SortedSegmentedIndex.java
@@ -0,0 +1,102 @@
+/*
+ * 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.cache.query.index.sorted;
+
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.internal.cache.query.index.Index;
+import org.apache.ignite.internal.cache.query.index.sorted.inline.IndexQueryContext;
+import org.apache.ignite.internal.util.lang.GridCursor;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Interface for sorted and segmented Ignite indexes.
+ */
+public interface SortedSegmentedIndex extends Index {
+    /**
+     * Finds index rows by specified range in specified tree segment. Range can be bound or unbound.
+     *
+     * @param lower Nullable lower bound.
+     * @param upper Nullable upper bound.
+     * @param segment Number of tree segment to find.
+     * @return Cursor of found index rows.
+     */
+    public GridCursor<IndexRow> find(@Nullable IndexRow lower, @Nullable IndexRow upper, int segment) throws IgniteCheckedException;
+
+    /**
+     * Finds index rows by specified range in specifed tree segment with cache filtering. Range can be bound or unbound.
+     *
+     * @param lower Nullable lower bound.
+     * @param upper Nullable upper bound.
+     * @param segment Number of tree segment to find.
+     * @param qryCtx External index qyery context.
+     * @return Cursor of found index rows.
+     */
+    public GridCursor<IndexRow> find(IndexRow lower, IndexRow upper, int segment, IndexQueryContext qryCtx)
+        throws IgniteCheckedException;
+
+    /**
+     * Finds first index row for specified tree segment and cache filter.
+     *
+     * @param segment Number of tree segment to find.
+     * @param qryCtx External index qyery context.
+     * @return Cursor of found index rows.
+     */
+    public GridCursor<IndexRow> findFirst(int segment, IndexQueryContext qryCtx)
+        throws IgniteCheckedException;
+
+    /**
+     * Finds last index row for specified tree segment and cache filter.
+     *
+     * @param segment Number of tree segment to find.
+     * @param qryCtx External index qyery context.
+     * @return Cursor of found index rows.
+     */
+    public GridCursor<IndexRow> findLast(int segment, IndexQueryContext qryCtx)
+        throws IgniteCheckedException;
+
+    /**
+     * Counts index rows in specified tree segment.
+     *
+     * @param segment Number of tree segment to find.
+     * @return count of index rows for specified segment.
+     */
+    public long count(int segment) throws IgniteCheckedException;
+
+    /**
+     * Counts index rows in specified tree segment with cache filter.
+     *
+     * @param segment Number of tree segment to find.
+     * @param qryCtx Index query context.
+     * @return count of index rows for specified segment.
+     */
+    public long count(int segment, IndexQueryContext qryCtx) throws IgniteCheckedException;
+
+    /**
+     * Counts index rows for all segments.
+     *
+     * @return total count of index rows.
+     */
+    public long totalCount() throws IgniteCheckedException;
+
+    /**
+     * Returns amount of index tree segments.
+     *
+     * @return amount of index tree segments.
+     */
+    public int segmentsCount();
+}
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/ThreadLocalRowHandlerHolder.java b/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/ThreadLocalRowHandlerHolder.java
new file mode 100644
index 0000000..b952dd0
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/ThreadLocalRowHandlerHolder.java
@@ -0,0 +1,62 @@
+/*
+ * 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.cache.query.index.sorted;
+
+/**
+ * Holds an index row handler during work session with an index tree.
+ */
+public class ThreadLocalRowHandlerHolder {
+    /** */
+    private static final ThreadLocal<RowHandlerHolder> holder = ThreadLocal.withInitial(RowHandlerHolder::new);
+
+    /** Set index row handler for current context. */
+    public static void rowHandler(InlineIndexRowHandler rowHnd) {
+        holder.get().rowHandler(rowHnd);
+    }
+
+    /** Get index row handler for current context. */
+    public static InlineIndexRowHandler rowHandler() {
+        return holder.get().rowHandler();
+    }
+
+    /** Clear index row handler for current context. */
+    public static void clearRowHandler() {
+        holder.get().clear();
+    }
+
+    /** Internal holder to avoid additional lookups of ThreadLocal set. */
+    private static class RowHandlerHolder {
+        /** Actual row handler. */
+        private InlineIndexRowHandler rowHnd;
+
+        /** */
+        private void rowHandler(InlineIndexRowHandler rowHnd) {
+            this.rowHnd = rowHnd;
+        }
+
+        /** */
+        private InlineIndexRowHandler rowHandler() {
+            return rowHnd;
+        }
+
+        /** */
+        private void clear() {
+            rowHnd = null;
+        }
+    }
+}
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/defragmentation/DefragIndexFactory.java b/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/defragmentation/DefragIndexFactory.java
new file mode 100644
index 0000000..05b0a55
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/defragmentation/DefragIndexFactory.java
@@ -0,0 +1,324 @@
+/*
+ * 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.cache.query.index.sorted.defragmentation;
+
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.internal.cache.query.index.sorted.IndexRow;
+import org.apache.ignite.internal.cache.query.index.sorted.IndexRowImpl;
+import org.apache.ignite.internal.cache.query.index.sorted.InlineIndexRowHandler;
+import org.apache.ignite.internal.cache.query.index.sorted.InlineIndexRowHandlerFactory;
+import org.apache.ignite.internal.cache.query.index.sorted.MetaPageInfo;
+import org.apache.ignite.internal.cache.query.index.sorted.SortedIndexDefinition;
+import org.apache.ignite.internal.cache.query.index.sorted.inline.InlineIndex;
+import org.apache.ignite.internal.cache.query.index.sorted.inline.InlineIndexFactory;
+import org.apache.ignite.internal.cache.query.index.sorted.inline.InlineIndexKeyType;
+import org.apache.ignite.internal.cache.query.index.sorted.inline.InlineIndexTree;
+import org.apache.ignite.internal.cache.query.index.sorted.inline.InlineRecommender;
+import org.apache.ignite.internal.cache.query.index.sorted.inline.io.AbstractInlineInnerIO;
+import org.apache.ignite.internal.cache.query.index.sorted.inline.io.AbstractInlineLeafIO;
+import org.apache.ignite.internal.cache.query.index.sorted.inline.io.InlineIO;
+import org.apache.ignite.internal.cache.query.index.sorted.inline.io.InnerIO;
+import org.apache.ignite.internal.cache.query.index.sorted.inline.io.LeafIO;
+import org.apache.ignite.internal.cache.query.index.sorted.keys.IndexKey;
+import org.apache.ignite.internal.metric.IoStatisticsHolder;
+import org.apache.ignite.internal.pagemem.PageIdUtils;
+import org.apache.ignite.internal.pagemem.PageMemory;
+import org.apache.ignite.internal.processors.cache.GridCacheContext;
+import org.apache.ignite.internal.processors.cache.IgniteCacheOffheapManager;
+import org.apache.ignite.internal.processors.cache.persistence.CacheDataRowAdapter;
+import org.apache.ignite.internal.processors.cache.persistence.RootPage;
+import org.apache.ignite.internal.processors.cache.persistence.tree.BPlusTree;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.BPlusIO;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.BPlusInnerIO;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.BPlusLeafIO;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.BPlusMetaIO;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.PageIO;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.PageIoResolver;
+import org.apache.ignite.internal.processors.cache.tree.mvcc.data.MvccDataRow;
+
+/**
+ * Creates temporary index to defragment old index.
+ */
+public class DefragIndexFactory extends InlineIndexFactory {
+    /** Temporary offheap manager. */
+    private final IgniteCacheOffheapManager offheap;
+
+    /** Old index. */
+    private final InlineIndex oldIdx;
+
+    /** Temporary cache page memory. */
+    private final PageMemory newCachePageMemory;
+
+    /** */
+    private final InlineIndexRowHandlerFactory rowHndFactory;
+
+    /** */
+    public DefragIndexFactory(IgniteCacheOffheapManager offheap, PageMemory newCachePageMemory, InlineIndex oldIdx) {
+        // Row handler factory that produces no-op handler.
+        rowHndFactory = (def, settings) -> oldIdx.segment(0).rowHandler();
+
+        this.offheap = offheap;
+        this.oldIdx = oldIdx;
+        this.newCachePageMemory = newCachePageMemory;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected InlineIndexTree createIndexSegment(GridCacheContext<?, ?> cctx, SortedIndexDefinition def,
+        RootPage rootPage, IoStatisticsHolder stats, InlineRecommender recommender, int segmentNum) throws Exception {
+
+        InlineIndexTree tree = new InlineIndexTree(
+            def,
+            cctx,
+            def.treeName(),
+            offheap,
+            offheap.reuseListForIndex(def.treeName()),
+            newCachePageMemory,
+            // Use old row handler to have access to inline index key types.
+            pageIoResolver(),
+            rootPage.pageId().pageId(),
+            rootPage.isAllocated(),
+            oldIdx.inlineSize(),
+            def.keyTypeSettings(),
+            null,
+            stats,
+            rowHndFactory,
+            null
+        );
+
+        final MetaPageInfo oldInfo = oldIdx.segment(segmentNum).metaInfo();
+
+        tree.copyMetaInfo(oldInfo);
+
+        tree.enableSequentialWriteMode();
+
+        return tree;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected RootPage rootPage(GridCacheContext<?, ?> ctx, String treeName, int segment) throws Exception {
+        return offheap.rootPageForIndex(ctx.cacheId(), treeName, segment);
+    }
+
+    /** */
+    private PageIoResolver pageIoResolver() {
+        return pageAddr -> {
+            PageIO io = PageIoResolver.DEFAULT_PAGE_IO_RESOLVER.resolve(pageAddr);
+
+            if (io instanceof BPlusMetaIO)
+                return io;
+
+            //noinspection unchecked,rawtypes,rawtypes
+            return wrap((BPlusIO)io, rowHndFactory.create(null, null));
+        };
+    }
+
+    /** */
+    @SuppressWarnings({"unchecked", "rawtypes"})
+    static BPlusIO<IndexRow> wrap(BPlusIO<IndexRow> io, InlineIndexRowHandler rowHnd) {
+        assert io instanceof InlineIO;
+
+        if (io instanceof BPlusInnerIO) {
+            assert io instanceof AbstractInlineInnerIO
+                || io instanceof InnerIO;
+
+            return new BPlusInnerIoDelegate((BPlusInnerIO<IndexRow>)io, rowHnd);
+        }
+        else {
+            assert io instanceof AbstractInlineLeafIO
+                || io instanceof LeafIO;
+
+            return new BPlusLeafIoDelegate((BPlusLeafIO<IndexRow>)io, rowHnd);
+        }
+    }
+
+    /** */
+    private static <T extends BPlusIO<IndexRow> & InlineIO> IndexRow lookupRow(
+        InlineIndexRowHandler rowHnd,
+        long pageAddr,
+        int idx,
+        T io
+    ) {
+        long link = io.link(pageAddr, idx);
+
+        int off = io.offset(idx);
+
+        IndexKey[] keys = new IndexKey[rowHnd.indexKeyDefinitions().size()];
+
+        int fieldOff = 0;
+
+        for (int i = 0; i < rowHnd.inlineIndexKeyTypes().size(); i++) {
+            InlineIndexKeyType keyType = rowHnd.inlineIndexKeyTypes().get(i);
+
+            IndexKey key = keyType.get(pageAddr, off + fieldOff, io.inlineSize() - fieldOff);
+
+            fieldOff += keyType.inlineSize(key);
+
+            keys[i] = key;
+        }
+
+        if (io.storeMvccInfo()) {
+            long mvccCrdVer = io.mvccCoordinatorVersion(pageAddr, idx);
+            long mvccCntr = io.mvccCounter(pageAddr, idx);
+            int mvccOpCntr = io.mvccOperationCounter(pageAddr, idx);
+
+            MvccDataRow row = new MvccDataRow(
+                null,
+                0,
+                link,
+                PageIdUtils.partId(PageIdUtils.pageId(link)),
+                CacheDataRowAdapter.RowData.LINK_ONLY,
+                mvccCrdVer,
+                mvccCntr,
+                mvccOpCntr,
+                true
+            );
+
+            return new IndexRowImpl(rowHnd, row, keys);
+        }
+
+        return new IndexRowImpl(rowHnd, new CacheDataRowAdapter(link), keys);
+    }
+
+    /** */
+    private static class BPlusInnerIoDelegate<IO extends BPlusInnerIO<IndexRow> & InlineIO>
+        extends BPlusInnerIO<IndexRow> implements InlineIO {
+        /** {@inheritDoc} */
+        private final IO io;
+
+        /** {@inheritDoc} */
+        private final InlineIndexRowHandler rowHnd;
+
+        /** {@inheritDoc} */
+        private BPlusInnerIoDelegate(IO io, InlineIndexRowHandler rowHnd) {
+            super(io.getType(), io.getVersion(), io.canGetRow(), io.getItemSize());
+            this.io = io;
+            this.rowHnd = rowHnd;
+        }
+
+        /** {@inheritDoc} */
+        @Override public void storeByOffset(long pageAddr, int off, IndexRow row) throws IgniteCheckedException {
+            io.storeByOffset(pageAddr, off, row);
+        }
+
+        /** {@inheritDoc} */
+        @Override public void store(long dstPageAddr, int dstIdx, BPlusIO<IndexRow> srcIo, long srcPageAddr, int srcIdx)
+            throws IgniteCheckedException {
+            io.store(dstPageAddr, dstIdx, srcIo, srcPageAddr, srcIdx);
+        }
+
+        /** {@inheritDoc} */
+        @Override public IndexRow getLookupRow(BPlusTree<IndexRow, ?> tree, long pageAddr,
+            int idx) throws IgniteCheckedException {
+            return lookupRow(rowHnd, pageAddr, idx, this);
+        }
+
+        /** {@inheritDoc} */
+        @Override public long link(long pageAddr, int idx) {
+            return io.link(pageAddr, idx);
+        }
+
+        /** {@inheritDoc} */
+        @Override public int inlineSize() {
+            return io.inlineSize();
+        }
+
+        /** {@inheritDoc} */
+        @Override public long mvccCoordinatorVersion(long pageAddr, int idx) {
+            return io.mvccCoordinatorVersion(pageAddr, idx);
+        }
+
+        /** {@inheritDoc} */
+        @Override public long mvccCounter(long pageAddr, int idx) {
+            return io.mvccCounter(pageAddr, idx);
+        }
+
+        /** {@inheritDoc} */
+        @Override public int mvccOperationCounter(long pageAddr, int idx) {
+            return io.mvccOperationCounter(pageAddr, idx);
+        }
+
+        /** {@inheritDoc} */
+        @Override public boolean storeMvccInfo() {
+            return io.storeMvccInfo();
+        }
+    }
+
+    /** */
+    private static class BPlusLeafIoDelegate<IO extends BPlusLeafIO<IndexRow> & InlineIO>
+        extends BPlusLeafIO<IndexRow> implements InlineIO {
+        /** */
+        private final IO io;
+
+        /** */
+        private final InlineIndexRowHandler rowHnd;
+
+        /** */
+        private BPlusLeafIoDelegate(IO io, InlineIndexRowHandler rowHnd) {
+            super(io.getType(), io.getVersion(), io.getItemSize());
+            this.io = io;
+            this.rowHnd = rowHnd;
+        }
+
+        /** {@inheritDoc} */
+        @Override public void storeByOffset(long pageAddr, int off, IndexRow row) throws IgniteCheckedException {
+            io.storeByOffset(pageAddr, off, row);
+        }
+
+        /** {@inheritDoc} */
+        @Override public void store(long dstPageAddr, int dstIdx, BPlusIO<IndexRow> srcIo, long srcPageAddr, int srcIdx)
+            throws IgniteCheckedException
+        {
+            io.store(dstPageAddr, dstIdx, srcIo, srcPageAddr, srcIdx);
+        }
+
+        /** {@inheritDoc} */
+        @Override public IndexRow getLookupRow(BPlusTree<IndexRow, ?> tree, long pageAddr, int idx) throws IgniteCheckedException {
+            return lookupRow(rowHnd, pageAddr, idx, this);
+        }
+
+        /** {@inheritDoc} */
+        @Override public long link(long pageAddr, int idx) {
+            return io.link(pageAddr, idx);
+        }
+
+        /** {@inheritDoc} */
+        @Override public int inlineSize() {
+            return io.inlineSize();
+        }
+
+        /** {@inheritDoc} */
+        @Override public long mvccCoordinatorVersion(long pageAddr, int idx) {
+            return io.mvccCoordinatorVersion(pageAddr, idx);
+        }
+
+        /** {@inheritDoc} */
+        @Override public long mvccCounter(long pageAddr, int idx) {
+            return io.mvccCounter(pageAddr, idx);
+        }
+
+        /** {@inheritDoc} */
+        @Override public int mvccOperationCounter(long pageAddr, int idx) {
+            return io.mvccOperationCounter(pageAddr, idx);
+        }
+
+        /** {@inheritDoc} */
+        @Override public boolean storeMvccInfo() {
+            return io.storeMvccInfo();
+        }
+    }
+}
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/defragmentation/IndexingDefragmentation.java b/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/defragmentation/IndexingDefragmentation.java
new file mode 100644
index 0000000..c53143c
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/defragmentation/IndexingDefragmentation.java
@@ -0,0 +1,269 @@
+/*
+ * 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.cache.query.index.sorted.defragmentation;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.atomic.AtomicLong;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.failure.FailureContext;
+import org.apache.ignite.internal.cache.query.index.IndexProcessor;
+import org.apache.ignite.internal.cache.query.index.sorted.IndexRow;
+import org.apache.ignite.internal.cache.query.index.sorted.IndexRowImpl;
+import org.apache.ignite.internal.cache.query.index.sorted.InlineIndexRowHandler;
+import org.apache.ignite.internal.cache.query.index.sorted.SortedIndexDefinition;
+import org.apache.ignite.internal.cache.query.index.sorted.inline.InlineIndex;
+import org.apache.ignite.internal.cache.query.index.sorted.inline.InlineIndexImpl;
+import org.apache.ignite.internal.cache.query.index.sorted.inline.io.MvccIO;
+import org.apache.ignite.internal.pagemem.PageMemory;
+import org.apache.ignite.internal.processors.cache.CacheGroupContext;
+import org.apache.ignite.internal.processors.cache.GridCacheContext;
+import org.apache.ignite.internal.processors.cache.persistence.CacheDataRow;
+import org.apache.ignite.internal.processors.cache.persistence.CacheDataRowAdapter;
+import org.apache.ignite.internal.processors.cache.persistence.checkpoint.CheckpointTimeoutLock;
+import org.apache.ignite.internal.processors.cache.persistence.defragmentation.LinkMap;
+import org.apache.ignite.internal.processors.cache.persistence.defragmentation.TreeIterator;
+import org.apache.ignite.internal.processors.cache.persistence.pagemem.PageMemoryEx;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.BPlusIO;
+import org.apache.ignite.internal.processors.cache.tree.mvcc.data.MvccDataRow;
+import org.apache.ignite.internal.util.IgniteUtils;
+import org.apache.ignite.internal.util.collection.IntMap;
+import org.apache.ignite.thread.IgniteThreadPoolExecutor;
+import org.jetbrains.annotations.Nullable;
+
+import static org.apache.ignite.failure.FailureType.CRITICAL_ERROR;
+
+/**
+ *
+ */
+public class IndexingDefragmentation {
+    /** Indexing. */
+    private final IndexProcessor indexing;
+
+    /** Constructor. */
+    public IndexingDefragmentation(IndexProcessor indexing) {
+        this.indexing = indexing;
+    }
+
+    /**
+     * Defragment index partition.
+     *
+     * @param grpCtx Old group context.
+     * @param newCtx New group context.
+     * @param partPageMem Partition page memory.
+     * @param mappingByPartition Mapping page memory.
+     * @param cpLock Defragmentation checkpoint read lock.
+     * @param cancellationChecker Cancellation checker.
+     * @param defragmentationThreadPool Thread pool for defragmentation.
+     * @param log Log.
+     *
+     * @throws IgniteCheckedException If failed.
+     */
+    public void defragment(
+        CacheGroupContext grpCtx,
+        CacheGroupContext newCtx,
+        PageMemoryEx partPageMem,
+        IntMap<LinkMap> mappingByPartition,
+        CheckpointTimeoutLock cpLock,
+        Runnable cancellationChecker,
+        IgniteThreadPoolExecutor defragmentationThreadPool,
+        IgniteLogger log
+    ) throws IgniteCheckedException {
+        int pageSize = grpCtx.cacheObjectContext().kernalContext().grid().configuration().getDataStorageConfiguration().getPageSize();
+
+        PageMemoryEx oldCachePageMem = (PageMemoryEx)grpCtx.dataRegion().pageMemory();
+
+        PageMemory newCachePageMemory = partPageMem;
+
+        Collection<TableIndexes> tables = tables(grpCtx);
+
+        long cpLockThreshold = 150L;
+
+        AtomicLong lastCpLockTs = new AtomicLong(System.currentTimeMillis());
+
+        IgniteUtils.doInParallel(
+            defragmentationThreadPool,
+            tables,
+            table -> defragmentTable(
+                newCtx,
+                mappingByPartition,
+                cpLock,
+                cancellationChecker,
+                pageSize,
+                oldCachePageMem,
+                newCachePageMemory,
+                cpLockThreshold,
+                lastCpLockTs,
+                table
+            )
+        );
+
+        if (log.isInfoEnabled())
+            log.info("Defragmentation indexes completed for group '" + grpCtx.groupId() + "'");
+    }
+
+    /**
+     * Defragment one given table.
+     */
+    private boolean defragmentTable(
+        CacheGroupContext newCtx,
+        IntMap<LinkMap> mappingByPartition,
+        CheckpointTimeoutLock cpLock,
+        Runnable cancellationChecker,
+        int pageSize,
+        PageMemoryEx oldCachePageMem,
+        PageMemory newCachePageMemory,
+        long cpLockThreshold,
+        AtomicLong lastCpLockTs,
+        TableIndexes indexes
+    ) throws IgniteCheckedException {
+        cpLock.checkpointReadLock();
+
+        try {
+            TreeIterator treeIterator = new TreeIterator(pageSize);
+
+            GridCacheContext<?, ?> cctx = indexes.cctx;
+
+            cancellationChecker.run();
+
+            for (InlineIndex oldIdx : indexes.idxs) {
+                InlineIndexRowHandler oldRowHnd = oldIdx.segment(0).rowHandler();
+
+                SortedIndexDefinition idxDef = (SortedIndexDefinition) indexing.indexDefinition(oldIdx.id());
+
+                InlineIndexImpl newIdx = new DefragIndexFactory(newCtx.offheap(), newCachePageMemory, oldIdx)
+                    .createIndex(cctx, idxDef)
+                    .unwrap(InlineIndexImpl.class);
+
+                int segments = oldIdx.segmentsCount();
+
+                for (int i = 0; i < segments; ++i) {
+                    treeIterator.iterate(oldIdx.segment(i), oldCachePageMem, (theTree, io, pageAddr, idx) -> {
+                        cancellationChecker.run();
+
+                        if (System.currentTimeMillis() - lastCpLockTs.get() >= cpLockThreshold) {
+                            cpLock.checkpointReadUnlock();
+
+                            cpLock.checkpointReadLock();
+
+                            lastCpLockTs.set(System.currentTimeMillis());
+                        }
+
+                        assert 1 == io.getVersion()
+                            : "IO version " + io.getVersion() + " is not supported by current defragmentation algorithm." +
+                            " Please implement copying of tree in a new format.";
+
+                        BPlusIO<IndexRow> h2IO = DefragIndexFactory.wrap(io, oldRowHnd);
+
+                        IndexRow row = theTree.getRow(h2IO, pageAddr, idx);
+
+                        if (!row.indexSearchRow()) {
+                            IndexRowImpl r = (IndexRowImpl) row;
+
+                            CacheDataRow cacheDataRow = r.cacheDataRow();
+
+                            int partition = cacheDataRow.partition();
+
+                            long link = r.link();
+
+                            LinkMap map = mappingByPartition.get(partition);
+
+                            long newLink = map.get(link);
+
+                            CacheDataRow newDataRow;
+
+                             if (((MvccIO) io).storeMvccInfo()) {
+                                 newDataRow = new MvccDataRow(newLink);
+                                 newDataRow.mvccVersion(row);
+                             } else
+                                 newDataRow = new CacheDataRowAdapter(newLink);
+
+                            // Use old row handler, as MetaInfo is copied from old tree.
+                            IndexRowImpl newRow = new IndexRowImpl(oldRowHnd, newDataRow, r.keys());
+
+                            newIdx.putIndexRow(newRow);
+                        }
+
+                        return true;
+                    });
+                }
+            }
+
+            return true;
+        }
+        catch (Throwable t) {
+            newCtx.cacheObjectContext().kernalContext()
+                .failure().process(new FailureContext(CRITICAL_ERROR, t));
+
+            throw t;
+        }
+        finally {
+            cpLock.checkpointReadUnlock();
+        }
+    }
+
+    /** Returns collection of table indexes. */
+    private Collection<TableIndexes> tables(CacheGroupContext gctx) {
+        Collection<TableIndexes> tables = new ArrayList<>();
+
+        for (GridCacheContext<?, ?> cctx: gctx.caches()) {
+            Map<String, TableIndexes> idxs = new HashMap<>();
+
+            List<InlineIndex> indexes = indexing.treeIndexes(cctx, false);
+
+            for (InlineIndex idx: indexes) {
+                String table = indexing.indexDefinition(idx.id()).idxName().tableName();
+
+                idxs.putIfAbsent(table, new TableIndexes(cctx, table));
+
+                idxs.get(table).addIndex(idx);
+            }
+
+            tables.addAll(idxs.values());
+        }
+
+        return tables;
+    }
+
+    /** Holder for indexes per cache table. */
+    private static class TableIndexes {
+        /** Table name. */
+        final @Nullable String tableName;
+
+        /** Cache context. */
+        final GridCacheContext<?, ?> cctx;
+
+        /** Indexes. */
+        final List<InlineIndex> idxs = new ArrayList<>();
+
+        /** */
+        TableIndexes(GridCacheContext<?, ?> cctx, String tableName) {
+            this.cctx = cctx;
+            this.tableName = tableName;
+        }
+
+        /** */
+        void addIndex(InlineIndex idx) {
+            idxs.add(idx);
+        }
+    }
+}
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/io/H2InnerIO.java b/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/inline/IndexQueryContext.java
similarity index 53%
copy from modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/io/H2InnerIO.java
copy to modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/inline/IndexQueryContext.java
index 9baff7a..99c6b81 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/io/H2InnerIO.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/inline/IndexQueryContext.java
@@ -15,23 +15,36 @@
  * limitations under the License.
  */
 
-package org.apache.ignite.internal.processors.query.h2.database.io;
+package org.apache.ignite.internal.cache.query.index.sorted.inline;
 
-import org.apache.ignite.internal.processors.cache.persistence.tree.io.IOVersions;
+import org.apache.ignite.internal.processors.cache.mvcc.MvccSnapshot;
+import org.apache.ignite.spi.indexing.IndexingQueryFilter;
+
+/** */
+public class IndexQueryContext {
+    /** Cache entry filter. */
+    private final IndexingQueryFilter filter;
 
-/**
- * Inner page for H2 row references.
- */
-public class H2InnerIO extends AbstractH2InnerIO {
     /** */
-    public static final IOVersions<H2InnerIO> VERSIONS = new IOVersions<>(
-        new H2InnerIO(1)
-    );
+    private final MvccSnapshot mvccSnapshot;
+
+    /** */
+    public IndexQueryContext(IndexingQueryFilter filter, MvccSnapshot snapshot) {
+        this.filter = filter;
+        this.mvccSnapshot = snapshot;
+    }
+
+    /**
+     * @return Mvcc snapshot.
+     */
+    public MvccSnapshot mvccSnapshot() {
+        return mvccSnapshot;
+    }
 
     /**
-     * @param ver Page format version.
+     * @return Filter.
      */
-    private H2InnerIO(int ver) {
-        super(T_H2_REF_INNER, ver, 8);
+    public IndexingQueryFilter filter() {
+        return filter;
     }
 }
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/io/H2LeafIO.java b/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/inline/InlineIndex.java
similarity index 56%
copy from modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/io/H2LeafIO.java
copy to modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/inline/InlineIndex.java
index 466cd1c..35a198e 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/io/H2LeafIO.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/inline/InlineIndex.java
@@ -15,23 +15,27 @@
  * limitations under the License.
  */
 
-package org.apache.ignite.internal.processors.query.h2.database.io;
+package org.apache.ignite.internal.cache.query.index.sorted.inline;
 
-import org.apache.ignite.internal.processors.cache.persistence.tree.io.IOVersions;
+import org.apache.ignite.internal.cache.query.index.sorted.SortedSegmentedIndex;
 
 /**
- * Leaf page for H2 row references.
+ * Represents an index that inlines some of index keys.
  */
-public class H2LeafIO extends AbstractH2LeafIO {
-    /** */
-    public static final IOVersions<H2LeafIO> VERSIONS = new IOVersions<>(
-        new H2LeafIO(1)
-    );
+public interface InlineIndex extends SortedSegmentedIndex {
+    /**
+     * @return amount of bytes to store inlined index keys.
+     */
+    public int inlineSize();
+
+    /**
+     * {@code true} if index is created and {@code false} if it is restored from disk.
+     */
+    public boolean created();
 
     /**
-     * @param ver Page format version.
+     * @param segment Number of tree segment.
+     * @return Tree segment for specified number.
      */
-    public H2LeafIO(int ver) {
-        super(T_H2_REF_LEAF, ver, 8);
-    }
+    public InlineIndexTree segment(int segment);
 }
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/inline/InlineIndexFactory.java b/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/inline/InlineIndexFactory.java
new file mode 100644
index 0000000..d3f38a6
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/inline/InlineIndexFactory.java
@@ -0,0 +1,102 @@
+/*
+ * 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.cache.query.index.sorted.inline;
+
+import org.apache.ignite.internal.cache.query.index.Index;
+import org.apache.ignite.internal.cache.query.index.IndexDefinition;
+import org.apache.ignite.internal.cache.query.index.IndexFactory;
+import org.apache.ignite.internal.cache.query.index.sorted.SortedIndexDefinition;
+import org.apache.ignite.internal.metric.IoStatisticsHolder;
+import org.apache.ignite.internal.metric.IoStatisticsHolderIndex;
+import org.apache.ignite.internal.processors.cache.GridCacheContext;
+import org.apache.ignite.internal.processors.cache.persistence.IgniteCacheDatabaseSharedManager;
+import org.apache.ignite.internal.processors.cache.persistence.RootPage;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.PageIoResolver;
+
+import static org.apache.ignite.internal.metric.IoStatisticsType.SORTED_INDEX;
+
+/**
+ * Factory to create {@link InlineIndex}.
+ */
+public class InlineIndexFactory implements IndexFactory {
+    /** */
+    public static final InlineIndexFactory INSTANCE = new InlineIndexFactory();
+
+    /** {@inheritDoc} */
+    @Override public Index createIndex(GridCacheContext<?, ?> cctx, IndexDefinition def) {
+        SortedIndexDefinition sdef = (SortedIndexDefinition) def;
+
+        InlineIndexTree[] trees = new InlineIndexTree[sdef.segments()];
+        InlineRecommender recommender = new InlineRecommender(cctx, sdef);
+
+        IoStatisticsHolderIndex stats = new IoStatisticsHolderIndex(
+            SORTED_INDEX,
+            cctx.name(),
+            sdef.idxName().idxName(),
+            cctx.kernalContext().metric()
+        );
+
+        try {
+            for (int i = 0; i < sdef.segments(); ++i) {
+                // Required for persistence.
+                IgniteCacheDatabaseSharedManager db = cctx.shared().database();
+                db.checkpointReadLock();
+
+                try {
+                    RootPage page = rootPage(cctx, sdef.treeName(), i);
+
+                    trees[i] = createIndexSegment(cctx, sdef, page, stats, recommender, i);
+
+                } finally {
+                    db.checkpointReadUnlock();
+                }
+            }
+
+            return new InlineIndexImpl(cctx, sdef, trees, stats);
+
+        } catch (Exception e) {
+            throw new RuntimeException(e);
+        }
+    }
+
+    /** */
+    protected InlineIndexTree createIndexSegment(GridCacheContext<?, ?> cctx, SortedIndexDefinition def,
+        RootPage rootPage, IoStatisticsHolder stats, InlineRecommender recommender, int segmentNum) throws Exception {
+        return new InlineIndexTree(
+            def,
+            cctx,
+            def.treeName(),
+            cctx.offheap(),
+            cctx.offheap().reuseListForIndex(def.treeName()),
+            cctx.dataRegion().pageMemory(),
+            PageIoResolver.DEFAULT_PAGE_IO_RESOLVER,
+            rootPage.pageId().pageId(),
+            rootPage.isAllocated(),
+            def.inlineSize(),
+            def.keyTypeSettings(),
+            def.idxRowCache(),
+            stats,
+            def.rowHandlerFactory(),
+            recommender);
+    }
+
+    /** */
+    protected RootPage rootPage(GridCacheContext<?, ?> ctx, String treeName, int segment) throws Exception {
+        return ctx.offheap().rootPageForIndex(ctx.cacheId(), treeName, segment);
+    }
+}
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/inline/InlineIndexImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/inline/InlineIndexImpl.java
new file mode 100644
index 0000000..9dd6022
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/inline/InlineIndexImpl.java
@@ -0,0 +1,473 @@
+/*
+ * 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.cache.query.index.sorted.inline;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.UUID;
+import java.util.concurrent.atomic.AtomicBoolean;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.failure.FailureContext;
+import org.apache.ignite.internal.cache.query.index.AbstractIndex;
+import org.apache.ignite.internal.cache.query.index.Index;
+import org.apache.ignite.internal.cache.query.index.SingleCursor;
+import org.apache.ignite.internal.cache.query.index.sorted.DurableBackgroundCleanupIndexTreeTask;
+import org.apache.ignite.internal.cache.query.index.sorted.IndexKeyTypeSettings;
+import org.apache.ignite.internal.cache.query.index.sorted.IndexRow;
+import org.apache.ignite.internal.cache.query.index.sorted.IndexRowImpl;
+import org.apache.ignite.internal.cache.query.index.sorted.IndexValueCursor;
+import org.apache.ignite.internal.cache.query.index.sorted.InlineIndexRowHandler;
+import org.apache.ignite.internal.cache.query.index.sorted.SortedIndexDefinition;
+import org.apache.ignite.internal.cache.query.index.sorted.ThreadLocalRowHandlerHolder;
+import org.apache.ignite.internal.metric.IoStatisticsHolderIndex;
+import org.apache.ignite.internal.processors.cache.GridCacheContext;
+import org.apache.ignite.internal.processors.cache.mvcc.MvccSnapshot;
+import org.apache.ignite.internal.processors.cache.persistence.CacheDataRow;
+import org.apache.ignite.internal.processors.cache.persistence.metastorage.pendingtask.DurableBackgroundTask;
+import org.apache.ignite.internal.util.lang.GridCursor;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.spi.indexing.IndexingQueryCacheFilter;
+import org.jetbrains.annotations.Nullable;
+
+import static org.apache.ignite.failure.FailureType.CRITICAL_ERROR;
+
+/**
+ * Sorted index implementation.
+ */
+public class InlineIndexImpl extends AbstractIndex implements InlineIndex {
+    /** Unique ID. */
+    private final UUID id = UUID.randomUUID();
+
+    /** Segments. */
+    private final InlineIndexTree[] segments;
+
+    /** Index function. */
+    private final SortedIndexDefinition def;
+
+    /** Name of underlying tree name. */
+    private final String treeName;
+
+    /** Cache context. */
+    private final GridCacheContext<?, ?> cctx;
+
+    /** */
+    private final IoStatisticsHolderIndex stats;
+
+    /** Row handler. */
+    private final InlineIndexRowHandler rowHnd;
+
+    /** Constructor. */
+    public InlineIndexImpl(GridCacheContext<?, ?> cctx, SortedIndexDefinition def, InlineIndexTree[] segments,
+        IoStatisticsHolderIndex stats) {
+        this.cctx = cctx;
+        this.segments = segments.clone();
+        this.def = def;
+        treeName = def.treeName();
+        this.stats = stats;
+        rowHnd = segments[0].rowHandler();
+    }
+
+    /** {@inheritDoc} */
+    @Override public GridCursor<IndexRow> find(IndexRow lower, IndexRow upper, int segment) throws IgniteCheckedException {
+        return find(lower, upper, segment, null);
+    }
+
+    /** {@inheritDoc} */
+    @Override public GridCursor<IndexRow> find(IndexRow lower, IndexRow upper, int segment, IndexQueryContext qryCtx) throws IgniteCheckedException {
+        InlineTreeFilterClosure closure = filterClosure(qryCtx);
+
+        // If it is known that only one row will be returned an optimization is employed
+        if (isSingleRowLookup(lower, upper)) {
+            IndexRowImpl row = segments[segment].findOne(lower, closure, null);
+
+            if (row == null || isExpired(row))
+                return IndexValueCursor.EMPTY;
+
+            return new SingleCursor<>(row);
+        }
+
+        return segments[segment].find(lower, upper, closure, null);
+    }
+
+    /** {@inheritDoc} */
+    @Override public long count(int segment) throws IgniteCheckedException {
+        return segments[segment].size();
+    }
+
+    /** {@inheritDoc} */
+    @Override public long count(int segment, IndexQueryContext qryCtx) throws IgniteCheckedException {
+        return segments[segment].size(filterClosure(qryCtx));
+    }
+
+    /**
+     * Returns number of elements in the tree by scanning pages of the bottom (leaf) level.
+     *
+     * @return Number of elements in the tree.
+     * @throws IgniteCheckedException If failed.
+     */
+    @Override public long totalCount() throws IgniteCheckedException {
+        long ret = 0;
+
+        for (int i = 0; i < segmentsCount(); i++)
+            ret += segments[i].size();
+
+        return ret;
+    }
+
+    /** */
+    private boolean isSingleRowLookup(IndexRow lower, IndexRow upper) throws IgniteCheckedException {
+        return !cctx.mvccEnabled() && def.primary() && lower != null && isFullSchemaSearch(lower) && checkRowsTheSame(lower, upper);
+    }
+
+    /**
+     * If {@code true} then length of keys for search must be equal to length of schema, so use full
+     * schema to search. If {@code false} then it's possible to use only part of schema for search.
+     */
+    private boolean isFullSchemaSearch(IndexRow key) {
+        int schemaLength = def.indexKeyDefinitions().size();
+
+        for (int i = 0; i < schemaLength; i++) {
+            // Java null means that column is not specified in a search row, for SQL NULL a special constant is used
+            if (key.key(i) == null)
+                return false;
+        }
+
+        return true;
+    }
+
+    /**
+     * Checks both rows are the same.
+     * <p/>
+     * Primarly used to verify if the single row lookup optimization can be applied.
+     *
+     * @param r1 The first row.
+     * @param r2 Another row.
+     * @return {@code true} in case both rows are efficiently the same, {@code false} otherwise.
+     */
+    private boolean checkRowsTheSame(IndexRow r1, IndexRow r2) throws IgniteCheckedException {
+        if (r1 == r2)
+            return true;
+
+        if (!(r1 != null && r2 != null))
+            return false;
+
+        int keysLen = def.indexKeyDefinitions().size();
+
+        for (int i = 0; i < keysLen; i++) {
+            Object v1 = r1.key(i);
+            Object v2 = r2.key(i);
+
+            if (v1 == null && v2 == null)
+                continue;
+
+            if (!(v1 != null && v2 != null))
+                return false;
+
+            if (def.rowComparator().compareKey((IndexRow) r1, (IndexRow) r2, i) != 0)
+                return false;
+        }
+
+        return true;
+    }
+
+    /** {@inheritDoc} */
+    @Override public GridCursor<IndexRow> findFirst(int segment, IndexQueryContext qryCtx) throws IgniteCheckedException {
+        InlineTreeFilterClosure closure = filterClosure(qryCtx);
+
+        IndexRow found = segments[segment].findFirst(closure);
+
+        if (found == null || isExpired(found))
+            return IndexValueCursor.EMPTY;
+
+        return new SingleCursor<>(found);
+    }
+
+    /** {@inheritDoc} */
+    @Override public GridCursor<IndexRow> findLast(int segment, IndexQueryContext qryCtx) throws IgniteCheckedException {
+        InlineTreeFilterClosure closure = filterClosure(qryCtx);
+
+        IndexRow found = segments[segment].findLast(closure);
+
+        if (found == null || isExpired(found))
+            return IndexValueCursor.EMPTY;
+
+        return new SingleCursor<>(found);
+    }
+
+    /** {@inheritDoc} */
+    @Override public UUID id() {
+        return id;
+    }
+
+    /** {@inheritDoc} */
+    @Override public String name() {
+        return def.idxName().idxName();
+    }
+
+    /** {@inheritDoc} */
+    @Override public void onUpdate(@Nullable CacheDataRow oldRow, @Nullable CacheDataRow newRow,
+        boolean prevRowAvailable) throws IgniteCheckedException {
+        try {
+            if (destroyed.get())
+                return;
+
+            ThreadLocalRowHandlerHolder.rowHandler(rowHnd);
+
+            boolean replaced = false;
+
+            // Create or Update.
+            if (newRow != null) {
+                int segment = segmentForRow(newRow);
+
+                IndexRowImpl row0 = new IndexRowImpl(rowHnd, newRow);
+
+                row0.prepareCache();
+
+                // Validate all keys before an actual put. User may specify wrong data types for an insert query.
+                for (int i = 0; i < def.indexKeyDefinitions().size(); ++i)
+                    row0.key(i);
+
+                replaced = putx(row0, segment, prevRowAvailable && !rebuildInProgress());
+            }
+
+            // Delete.
+            if (!replaced && oldRow != null)
+                remove(oldRow);
+
+        } finally {
+            ThreadLocalRowHandlerHolder.clearRowHandler();
+        }
+    }
+
+    /** */
+    private boolean putx(IndexRowImpl idxRow, int segment, boolean flag) throws IgniteCheckedException {
+        try {
+            boolean replaced;
+
+            if (flag)
+                replaced = segments[segment].putx(idxRow);
+            else {
+                IndexRow prevRow0 = segments[segment].put(idxRow);
+
+                replaced = prevRow0 != null;
+            }
+
+            return replaced;
+
+        } catch (Throwable t) {
+            cctx.kernalContext().failure().process(new FailureContext(CRITICAL_ERROR, t));
+
+            throw t;
+        }
+    }
+
+    /** */
+    private void remove(CacheDataRow row) throws IgniteCheckedException {
+        try {
+            int segment = segmentForRow(row);
+
+            IndexRowImpl idxRow = new IndexRowImpl(rowHnd, row);
+
+            idxRow.prepareCache();
+
+            segments[segment].removex(idxRow);
+
+        } catch (Throwable t) {
+            cctx.kernalContext().failure().process(new FailureContext(CRITICAL_ERROR, t));
+
+            throw t;
+        }
+    }
+
+    /**
+     * Put index row to index. This method is for internal use only.
+     *
+     * @param row Index row.
+     */
+    public void putIndexRow(IndexRowImpl row) throws IgniteCheckedException {
+        int segment = segmentForRow(row.cacheDataRow());
+
+        try {
+            ThreadLocalRowHandlerHolder.rowHandler(rowHnd);
+
+            segments[segment].putx(row);
+        }
+        finally {
+            ThreadLocalRowHandlerHolder.clearRowHandler();
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public <T extends Index> T unwrap(Class<T> clazz) {
+        if (clazz == null)
+            return null;
+
+        if (clazz.isAssignableFrom(getClass()))
+            return clazz.cast(this);
+
+        throw new IllegalArgumentException(
+            String.format("Cannot unwrap [%s] to [%s]", getClass().getName(), clazz.getName())
+        );
+    }
+
+    /** {@inheritDoc} */
+    @Override public int inlineSize() {
+        return segments[0].inlineSize();
+    }
+
+    /** */
+    public IndexKeyTypeSettings keyTypeSettings() {
+        return rowHnd.indexKeyTypeSettings();
+    }
+
+    /** {@inheritDoc} */
+    @Override public int segmentsCount() {
+        return segments.length;
+    }
+
+    /**
+     * @param row cache row.
+     * @return Segment ID for given key
+     */
+    public int segmentForRow(CacheDataRow row) {
+        return segmentsCount() == 1 ? 0 : (rowHnd.partition(row) % segmentsCount());
+    }
+
+    /** */
+    private InlineTreeFilterClosure filterClosure(IndexQueryContext qryCtx) {
+        if (qryCtx == null)
+            return null;
+
+        IndexingQueryCacheFilter cacheFilter = qryCtx.filter() == null ? null
+            : qryCtx.filter().forCache(cctx.cache().name());
+
+        MvccSnapshot v = qryCtx.mvccSnapshot();
+
+        assert !cctx.mvccEnabled() || v != null;
+
+        if (cacheFilter == null && v == null)
+            return null;
+
+        return new InlineTreeFilterClosure(
+            cacheFilter, v, cctx, cctx.kernalContext().config().getGridLogger());
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean created() {
+        assert segments != null;
+
+        for (int i = 0; i < segments.length; i++) {
+            try {
+                InlineIndexTree segment = segments[i];
+
+                if (segment.created())
+                    return true;
+            }
+            catch (Exception e) {
+                throw new IgniteException("Failed to check index tree root page existence [cacheName=" +
+                    cctx.name() + ", tblName=" + def.idxName().tableName() + ", idxName=" + def.idxName().idxName() +
+                    ", segment=" + i + ']');
+            }
+        }
+
+        return false;
+    }
+
+    /** {@inheritDoc} */
+    @Override public InlineIndexTree segment(int segment) {
+        return segments[segment];
+    }
+
+    /**
+     * Determines if provided row can be treated as expired at the current moment.
+     *
+     * @param row row to check.
+     * @throws NullPointerException if provided row is {@code null}.
+     */
+    private static boolean isExpired(IndexRow row) {
+        return row.cacheDataRow().expireTime() > 0 && row.cacheDataRow().expireTime() <= U.currentTimeMillis();
+    }
+
+    /** If {code true} then this index is already marked as destroyed. */
+    private final AtomicBoolean destroyed = new AtomicBoolean();
+
+    /** {@inheritDoc} */
+    @Override public void destroy(boolean softDelete) {
+        // Already destroyed.
+        if (!destroyed.compareAndSet(false, true))
+            return;
+
+        try {
+            if (cctx.affinityNode() && !softDelete) {
+                List<Long> rootPages = new ArrayList<>(segments.length);
+                List<InlineIndexTree> trees = new ArrayList<>(segments.length);
+
+                cctx.shared().database().checkpointReadLock();
+
+                try {
+                    for (int i = 0; i < segments.length; i++) {
+                        InlineIndexTree tree = segments[i];
+
+                        // Just mark it as destroyed. Actual destroy later in background task.
+                        tree.markDestroyed();
+
+                        rootPages.add(tree.getMetaPageId());
+                        trees.add(tree);
+
+                        dropMetaPage(i);
+                    }
+                }
+                finally {
+                    cctx.shared().database().checkpointReadUnlock();
+                }
+
+                cctx.kernalContext().metric().remove(stats.metricRegistryName());
+
+                // Actual destroy index task.
+                DurableBackgroundTask task = new DurableBackgroundCleanupIndexTreeTask(
+                    rootPages,
+                    trees,
+                    cctx.group().name() == null ? cctx.cache().name() : cctx.group().name(),
+                    cctx.cache().name(),
+                    def.idxName(),
+                    treeName
+                );
+
+                cctx.kernalContext().durableBackgroundTasksProcessor().startDurableBackgroundTask(task, cctx.config());
+            }
+        }
+        catch (IgniteCheckedException e) {
+            throw new IgniteException(e);
+        }
+    }
+
+    /**
+     * @param segIdx Segment index.
+     * @throws IgniteCheckedException If failed.
+     */
+    private void dropMetaPage(int segIdx) throws IgniteCheckedException {
+        cctx.offheap().dropRootPageForIndex(cctx.cacheId(), treeName, segIdx);
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean canHandle(CacheDataRow row) throws IgniteCheckedException {
+        return cctx.kernalContext().query().belongsToTable(
+            cctx, def.idxName().cacheName(), def.idxName().tableName(), row.key(), row.value());
+    }
+}
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/InlineIndexColumn.java b/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/inline/InlineIndexKeyType.java
similarity index 60%
rename from modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/InlineIndexColumn.java
rename to modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/inline/InlineIndexKeyType.java
index dba3c39..b61b8a6 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/InlineIndexColumn.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/inline/InlineIndexKeyType.java
@@ -15,87 +15,74 @@
  * limitations under the License.
  */
 
-package org.apache.ignite.internal.processors.query.h2.database;
+package org.apache.ignite.internal.cache.query.index.sorted.inline;
 
-import java.util.Comparator;
-import org.h2.value.Value;
+import org.apache.ignite.internal.cache.query.index.sorted.keys.IndexKey;
+import org.jetbrains.annotations.Nullable;
 
 /**
- * Basic interface for inlined index columns.
+ * Basic interface for inlined index columns. It's not a generic to provide opportunity compare different types.
  */
-public interface InlineIndexColumn {
+public interface InlineIndexKeyType {
     /**
      * Returns type of inlined column.
      *
      * @return Integer code of the column's value type.
-     * @see Value
      */
     public int type();
 
     /**
-     * Size required to inline value of type {@link #type()}.
+     * Returns size of inlined key.
      *
-     * @return Amount of bytes required for fixed-size types or {@code -1}
-     * for types with variable length.
+     * Note: system fields (e.g. type, length) are taken into account as well.
      */
-    public short size();
+    public int inlineSize();
 
     /**
-     * Actual size of inline value. It returns size() + 1 for values with
-     * fixed size and amount of written bytes for values with variable length.
+     * Returns inline size for specified key.
      *
-     * @param pageAddr Page address.
-     * @param off Offset.
-     * @return Returns actual size of inlined value.
+     * Note: system fields (e.g. type, length) are taken into account as well.
      */
-    public int fullSize(long pageAddr, int off);
+    public int inlineSize(IndexKey key);
 
     /**
-     * Compares inlined and given value.
+     * Actual size of inline value. It returns keySize() + 1 for values with
+     * fixed size and amount of written bytes for values with variable length.
+     *
+     * Used for dynamic offset calculation by page for variable length values.
      *
      * @param pageAddr Page address.
      * @param off Offset.
-     * @param maxSize Max size.
-     * @param v Value that should be compare.
-     * @param comp Comparator.
-     *
-     * @return -1, 0 or 1 if inlined value less, equal or greater
-     * than given respectively, or -2 if inlined part is not enough to compare.
-     */
-    public int compare(long pageAddr, int off, int maxSize, Value v, Comparator<Value> comp);
-
-    /**
-     * Calculate size required to inline given value.
-     *
-     * @param val Value to calculate inline size.
-     *
-     * @return Calculated inline size.
+     * @return Returns actual size of inlined value.
      */
-    public int inlineSizeOf(Value val);
+    public int inlineSize(long pageAddr, int off);
 
     /**
      * Puts given value into inline index tree.
      *
      * @param pageAddr Page address.
      * @param off Offset.
-     * @param val Value.
+     * @param key Index key.
      * @param maxSize Max size.
      *
      * @return Amount of bytes actually stored.
      */
-    public int put(long pageAddr, int off, Value val, int maxSize);
+    public int put(long pageAddr, int off, IndexKey key, int maxSize);
 
     /**
-     * Returns name of the inlined column.
-     *
-     * @return Name of the column.
      */
-    public String columnName();
+    @Nullable public IndexKey get(long pageAddr, int off, int maxSize);
 
     /**
-     * Returns index of the inlined column.
+     * Compares inlined and given value.
      *
-     * @return Index of the column.
+     * @param pageAddr Page address.
+     * @param off Offset.
+     * @param maxSize Max size.
+     * @param v Value that should be compare.
+     *
+     * @return -1, 0 or 1 if inlined value less, equal or greater
+     * than given respectively, or -2 if inlined part is not enough to compare.
      */
-    public int columnIndex();
+    public int compare(long pageAddr, int off, int maxSize, IndexKey v);
 }
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/inline/InlineIndexKeyTypeRegistry.java b/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/inline/InlineIndexKeyTypeRegistry.java
new file mode 100644
index 0000000..ad99afd
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/inline/InlineIndexKeyTypeRegistry.java
@@ -0,0 +1,188 @@
+/*
+ * 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.cache.query.index.sorted.inline;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import org.apache.ignite.internal.cache.query.index.sorted.IndexKeyDefinition;
+import org.apache.ignite.internal.cache.query.index.sorted.IndexKeyTypeSettings;
+import org.apache.ignite.internal.cache.query.index.sorted.IndexKeyTypes;
+import org.apache.ignite.internal.cache.query.index.sorted.inline.types.BooleanInlineIndexKeyType;
+import org.apache.ignite.internal.cache.query.index.sorted.inline.types.ByteInlineIndexKeyType;
+import org.apache.ignite.internal.cache.query.index.sorted.inline.types.BytesInlineIndexKeyType;
+import org.apache.ignite.internal.cache.query.index.sorted.inline.types.DateInlineIndexKeyType;
+import org.apache.ignite.internal.cache.query.index.sorted.inline.types.DoubleInlineIndexKeyType;
+import org.apache.ignite.internal.cache.query.index.sorted.inline.types.FloatInlineIndexKeyType;
+import org.apache.ignite.internal.cache.query.index.sorted.inline.types.IntegerInlineIndexKeyType;
+import org.apache.ignite.internal.cache.query.index.sorted.inline.types.LongInlineIndexKeyType;
+import org.apache.ignite.internal.cache.query.index.sorted.inline.types.ObjectByteArrayInlineIndexKeyType;
+import org.apache.ignite.internal.cache.query.index.sorted.inline.types.ObjectHashInlineIndexKeyType;
+import org.apache.ignite.internal.cache.query.index.sorted.inline.types.ShortInlineIndexKeyType;
+import org.apache.ignite.internal.cache.query.index.sorted.inline.types.SignedBytesInlineIndexKeyType;
+import org.apache.ignite.internal.cache.query.index.sorted.inline.types.StringInlineIndexKeyType;
+import org.apache.ignite.internal.cache.query.index.sorted.inline.types.StringNoCompareInlineIndexKeyType;
+import org.apache.ignite.internal.cache.query.index.sorted.inline.types.TimeInlineIndexKeyType;
+import org.apache.ignite.internal.cache.query.index.sorted.inline.types.TimestampInlineIndexKeyType;
+import org.apache.ignite.internal.cache.query.index.sorted.inline.types.UuidInlineIndexKeyType;
+import org.apache.ignite.internal.cache.query.index.sorted.keys.IndexKey;
+import org.apache.ignite.internal.cache.query.index.sorted.keys.NullIndexKey;
+
+/**
+ * Provide mapping for java types and {@link IndexKeyTypes} that supports inlining.
+ */
+public class InlineIndexKeyTypeRegistry {
+    /** Type mapping. */
+    private static final Map<Integer, InlineIndexKeyType> typeMapping = new ConcurrentHashMap<>();
+
+    /** Object key type that maps for custom POJO. Inline stores a hash of the object. */
+    private static final ObjectHashInlineIndexKeyType hashObjectType = new ObjectHashInlineIndexKeyType();
+
+    /** Default String key type use optimized algorithm for comparison. */
+    private static final StringInlineIndexKeyType optimizedCompareStringType = new StringInlineIndexKeyType();
+
+    /** Do not compare inlined String keys. */
+    private static final StringNoCompareInlineIndexKeyType noCompareStringType = new StringNoCompareInlineIndexKeyType();
+
+    /** Default String key type use optimized algorithm for comparison. */
+    private static final BytesInlineIndexKeyType bytesType = new BytesInlineIndexKeyType();
+
+    /** Do not compare inlined String keys. */
+    private static final SignedBytesInlineIndexKeyType signedBytesType = new SignedBytesInlineIndexKeyType();
+
+    /** Object key type that maps for custom POJO. Inline stores a byte array representation of the object. */
+    private static final ObjectByteArrayInlineIndexKeyType bytesObjectType =
+        new ObjectByteArrayInlineIndexKeyType(new BytesInlineIndexKeyType(IndexKeyTypes.JAVA_OBJECT));
+
+    /** Object key type that maps for custom POJO. Inline stores a signed byte array representation of the object. */
+    private static final ObjectByteArrayInlineIndexKeyType signedBytesObjectType =
+        new ObjectByteArrayInlineIndexKeyType(new SignedBytesInlineIndexKeyType(IndexKeyTypes.JAVA_OBJECT));
+
+    static {
+        register(IndexKeyTypes.BOOLEAN, new BooleanInlineIndexKeyType());
+        register(IndexKeyTypes.BYTE, new ByteInlineIndexKeyType());
+        register(IndexKeyTypes.DATE, new DateInlineIndexKeyType());
+        register(IndexKeyTypes.DOUBLE, new DoubleInlineIndexKeyType());
+        register(IndexKeyTypes.FLOAT, new FloatInlineIndexKeyType());
+        register(IndexKeyTypes.INT, new IntegerInlineIndexKeyType());
+        register(IndexKeyTypes.SHORT, new ShortInlineIndexKeyType());
+        register(IndexKeyTypes.LONG, new LongInlineIndexKeyType());
+        register(IndexKeyTypes.TIME, new TimeInlineIndexKeyType());
+        register(IndexKeyTypes.TIMESTAMP, new TimestampInlineIndexKeyType());
+        register(IndexKeyTypes.UUID, new UuidInlineIndexKeyType());
+        // Choice of those types actually depends on IndexKeyTypeSettings.
+        register(IndexKeyTypes.JAVA_OBJECT, hashObjectType);
+        register(IndexKeyTypes.STRING, optimizedCompareStringType);
+        register(IndexKeyTypes.BYTES, bytesType);
+    }
+
+    /** */
+    private static void register(int type, InlineIndexKeyType keyType) {
+        typeMapping.put(type, keyType);
+    }
+
+    /**
+     * Get key type for a class. Used for user queries, where getting type from class.
+     * Type is required for cases when class doesn't have strict type relation (nulls, POJO).
+     *
+     * @param expType Expected type of a key.
+     */
+    public static InlineIndexKeyType get(int expType, IndexKeyTypeSettings keyTypeSettings) {
+        return type(expType, keyTypeSettings);
+    }
+
+    /**
+     * Get key type for specified key. Used for user queries, where getting type from class.
+     * Type is required for cases when class doesn't have strict type relation (nulls, POJO).
+     *
+     * @param key Index key.
+     * @param expType Expected type of a key.
+     * @param keyTypeSettings Index key type settings.
+     */
+    public static InlineIndexKeyType get(IndexKey key, int expType, IndexKeyTypeSettings keyTypeSettings) {
+        return key == NullIndexKey.INSTANCE ?
+            type(expType, keyTypeSettings) :
+            type(key.type(), keyTypeSettings);
+    }
+
+    /** */
+    private static InlineIndexKeyType type(int type, IndexKeyTypeSettings keyTypeSettings) {
+        if (type == IndexKeyTypes.JAVA_OBJECT)
+            return javaObjectType(keyTypeSettings);
+
+        else if (type == IndexKeyTypes.STRING)
+            return stringType(keyTypeSettings);
+
+        else if (type == IndexKeyTypes.BYTES)
+            return bytesType(keyTypeSettings);
+
+        return typeMapping.get(type);
+    }
+
+    /**
+     * Checks whether specified type support inlining.
+     */
+    private static boolean supportInline(int type, IndexKeyTypeSettings keyTypeSettings) {
+        if (type == IndexKeyTypes.JAVA_OBJECT && !keyTypeSettings.inlineObjSupported())
+            return false;
+
+        return typeMapping.containsKey(type);
+    }
+
+    /**
+     * Get key type for the POJO type.
+     */
+    private static InlineIndexKeyType javaObjectType(IndexKeyTypeSettings keyTypeSettings) {
+        if (keyTypeSettings.inlineObjHash())
+            return hashObjectType;
+
+        return keyTypeSettings.binaryUnsigned() ? bytesObjectType : signedBytesObjectType;
+    }
+
+    /**
+     * Get key type for the String type.
+     */
+    private static InlineIndexKeyType stringType(IndexKeyTypeSettings keyTypeSettings) {
+        return keyTypeSettings.stringOptimizedCompare() ? optimizedCompareStringType : noCompareStringType;
+    }
+
+    /**
+     * Get key type for the Bytes type.
+     */
+    private static InlineIndexKeyType bytesType(IndexKeyTypeSettings keyTypeSettings) {
+        return keyTypeSettings.binaryUnsigned() ? bytesType : signedBytesType;
+    }
+
+    /**
+     * Return list of key types for specified key definitions and key type settings.
+     * */
+    public static List<InlineIndexKeyType> types(List<IndexKeyDefinition> keyDefs, IndexKeyTypeSettings settings) {
+        List<InlineIndexKeyType> keyTypes = new ArrayList<>();
+
+        for (IndexKeyDefinition keyDef: keyDefs) {
+            if (!supportInline(keyDef.idxType(), settings))
+                break;
+
+            keyTypes.add(type(keyDef.idxType(), settings));
+        }
+
+        return Collections.unmodifiableList(keyTypes);
+    }
+}
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/inline/InlineIndexTree.java b/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/inline/InlineIndexTree.java
new file mode 100644
index 0000000..6528575
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/inline/InlineIndexTree.java
@@ -0,0 +1,658 @@
+/*
+ * 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.cache.query.index.sorted.inline;
+
+import java.util.List;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.IgniteSystemProperties;
+import org.apache.ignite.failure.FailureType;
+import org.apache.ignite.internal.cache.query.index.SortOrder;
+import org.apache.ignite.internal.cache.query.index.sorted.IndexKeyDefinition;
+import org.apache.ignite.internal.cache.query.index.sorted.IndexKeyTypeSettings;
+import org.apache.ignite.internal.cache.query.index.sorted.IndexKeyTypes;
+import org.apache.ignite.internal.cache.query.index.sorted.IndexRow;
+import org.apache.ignite.internal.cache.query.index.sorted.IndexRowCache;
+import org.apache.ignite.internal.cache.query.index.sorted.IndexRowImpl;
+import org.apache.ignite.internal.cache.query.index.sorted.InlineIndexRowHandler;
+import org.apache.ignite.internal.cache.query.index.sorted.InlineIndexRowHandlerFactory;
+import org.apache.ignite.internal.cache.query.index.sorted.MetaPageInfo;
+import org.apache.ignite.internal.cache.query.index.sorted.SortedIndexDefinition;
+import org.apache.ignite.internal.cache.query.index.sorted.ThreadLocalRowHandlerHolder;
+import org.apache.ignite.internal.cache.query.index.sorted.inline.io.AbstractInlineInnerIO;
+import org.apache.ignite.internal.cache.query.index.sorted.inline.io.AbstractInlineLeafIO;
+import org.apache.ignite.internal.cache.query.index.sorted.inline.io.MvccIO;
+import org.apache.ignite.internal.metric.IoStatisticsHolder;
+import org.apache.ignite.internal.pagemem.FullPageId;
+import org.apache.ignite.internal.pagemem.PageIdAllocator;
+import org.apache.ignite.internal.pagemem.PageIdUtils;
+import org.apache.ignite.internal.pagemem.PageMemory;
+import org.apache.ignite.internal.pagemem.wal.record.PageSnapshot;
+import org.apache.ignite.internal.processors.cache.GridCacheContext;
+import org.apache.ignite.internal.processors.cache.IgniteCacheOffheapManager;
+import org.apache.ignite.internal.processors.cache.mvcc.MvccUtils;
+import org.apache.ignite.internal.processors.cache.persistence.CacheDataRowAdapter;
+import org.apache.ignite.internal.processors.cache.persistence.tree.BPlusTree;
+import org.apache.ignite.internal.processors.cache.persistence.tree.CorruptedTreeException;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.BPlusIO;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.BPlusMetaIO;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.PageIO;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.PageIoResolver;
+import org.apache.ignite.internal.processors.cache.persistence.tree.reuse.ReuseList;
+import org.apache.ignite.internal.processors.cache.tree.mvcc.data.MvccDataRow;
+import org.apache.ignite.internal.util.typedef.F;
+import org.apache.ignite.internal.util.typedef.internal.S;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.jetbrains.annotations.Nullable;
+
+import static org.apache.ignite.internal.cache.query.index.sorted.inline.types.NullableInlineIndexKeyType.CANT_BE_COMPARE;
+import static org.apache.ignite.internal.cache.query.index.sorted.inline.types.NullableInlineIndexKeyType.COMPARE_UNSUPPORTED;
+
+/**
+ * BPlusTree where nodes stores inlined index keys.
+ */
+public class InlineIndexTree extends BPlusTree<IndexRow, IndexRow> {
+    /** Amount of bytes to store inlined index keys. */
+    private final int inlineSize;
+
+    /** Index key type settings for this tree. */
+    private final IndexKeyTypeSettings keyTypeSettings;
+
+    /** Recommends change inline size if needed. */
+    private final InlineRecommender recommender;
+
+    /** Whether tree is created from scratch or reused from underlying store. */
+    private final boolean created;
+
+    /** Definition of index. */
+    private final SortedIndexDefinition def;
+
+    /** */
+    private final InlineIndexRowHandler rowHnd;
+
+    /** Cache context. */
+    private final GridCacheContext<?, ?> cctx;
+
+    /** Statistics holder used by underlying BPlusTree. */
+    private final IoStatisticsHolder stats;
+
+    /** */
+    private final IgniteLogger log;
+
+    /** Row cache. */
+    private final @Nullable IndexRowCache idxRowCache;
+
+    /** Whether MVCC is enabled. */
+    private final boolean mvccEnabled;
+
+    /**
+     * Constructor.
+     */
+    public InlineIndexTree(
+        SortedIndexDefinition def,
+        GridCacheContext<?, ?> cctx,
+        String treeName,
+        IgniteCacheOffheapManager offheap,
+        ReuseList reuseList,
+        PageMemory pageMemory,
+        PageIoResolver pageIoResolver,
+        long metaPageId,
+        boolean initNew,
+        int configuredInlineSize,
+        IndexKeyTypeSettings keyTypeSettings,
+        @Nullable IndexRowCache idxRowCache,
+        @Nullable IoStatisticsHolder stats,
+        InlineIndexRowHandlerFactory rowHndFactory,
+        InlineRecommender recommender) throws IgniteCheckedException {
+        super(
+            treeName,
+            cctx.groupId(),
+            cctx.group().name(),
+            pageMemory,
+            cctx.shared().wal(),
+            offheap.globalRemoveId(),
+            metaPageId,
+            reuseList,
+            PageIdAllocator.FLAG_IDX,
+            cctx.shared().kernalContext().failure(),
+            null,
+            pageIoResolver
+        );
+
+        this.cctx = cctx;
+
+        log = cctx.kernalContext().config().getGridLogger();
+
+        this.stats = stats;
+
+        created = initNew;
+
+        this.def = def;
+
+        this.idxRowCache = idxRowCache;
+
+        mvccEnabled = cctx.mvccEnabled();
+
+        if (!initNew) {
+            // Init from metastore.
+            // Page is ready - read meta information.
+            MetaPageInfo metaInfo = metaInfo();
+
+            if (def != null)
+                def.initByMeta(initNew, metaInfo);
+
+            inlineSize = metaInfo.inlineSize();
+            setIos(inlineSize, mvccEnabled);
+
+            boolean inlineObjSupported = inlineObjectSupported(def, metaInfo, rowHndFactory);
+
+            keyTypeSettings
+                .inlineObjHash(metaInfo.inlineObjectHash())
+                .inlineObjSupported(inlineObjSupported);
+
+            rowHnd = rowHndFactory.create(def, keyTypeSettings);
+
+            if (!metaInfo.flagsSupported())
+                upgradeMetaPage(inlineObjSupported);
+
+        } else {
+            def.initByMeta(initNew, null);
+
+            rowHnd = rowHndFactory.create(def, keyTypeSettings);
+
+            inlineSize = computeInlineSize(
+                rowHnd.inlineIndexKeyTypes(), configuredInlineSize, cctx.config().getSqlIndexMaxInlineSize());
+
+            setIos(inlineSize, mvccEnabled);
+        }
+
+        this.keyTypeSettings = keyTypeSettings;
+
+        initTree(initNew, inlineSize);
+
+        this.recommender = recommender;
+    }
+
+    /** */
+    private void setIos(int inlineSize, boolean mvccEnabled) {
+        setIos(
+            AbstractInlineInnerIO.versions(inlineSize, mvccEnabled),
+            AbstractInlineLeafIO.versions(inlineSize, mvccEnabled)
+        );
+    }
+
+    /**
+     * Find whether tree supports inlining objects or not.
+     *
+     * @param def Index definition.
+     * @param metaInfo Metapage info.
+     * @return {@code true} if inline object is supported by exists tree.
+     */
+    private boolean inlineObjectSupported(SortedIndexDefinition def, MetaPageInfo metaInfo,
+        InlineIndexRowHandlerFactory rowHndFactory) {
+
+        if (metaInfo.flagsSupported())
+            return metaInfo.inlineObjectSupported();
+        else {
+            try {
+                if (InlineObjectBytesDetector.objectMayBeInlined(metaInfo.inlineSize(), def.indexKeyDefinitions())) {
+                    try {
+                        InlineObjectBytesDetector inlineObjDetector = new InlineObjectBytesDetector(
+                            metaInfo.inlineSize(), def.indexKeyDefinitions(), def.idxName(), log);
+
+                        // Create a settings for case where java objects inilned as byte array.
+                        IndexKeyTypeSettings keyTypeSettings = new IndexKeyTypeSettings()
+                            .inlineObjSupported(true)
+                            .inlineObjHash(false);
+
+                        InlineIndexRowHandler rowHnd = rowHndFactory.create(def, keyTypeSettings);
+
+                        ThreadLocalRowHandlerHolder.rowHandler(rowHnd);
+
+                        findFirst(inlineObjDetector);
+
+                        return inlineObjDetector.inlineObjectSupported();
+
+                    } finally {
+                        ThreadLocalRowHandlerHolder.clearRowHandler();
+                    }
+                }
+                else
+                    return false;
+            }
+            catch (IgniteCheckedException e) {
+                throw new IgniteException("Unexpected exception on detect inline object", e);
+            }
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override protected int compare(BPlusIO<IndexRow> io, long pageAddr, int idx, IndexRow row)
+        throws IgniteCheckedException {
+
+        if (inlineSize == 0) {
+            IndexRow currRow = getRow(io, pageAddr, idx);
+
+            int cmp = compareFullRows(currRow, row, 0);
+
+            return cmp == 0 ? mvccCompare(currRow, row) : cmp;
+        }
+
+        int fieldOff = 0;
+
+        // Use it when can't compare values (variable length, for example).
+        int keyIdx;
+
+        IndexRow currRow = null;
+
+        int off = io.offset(idx);
+
+        List<IndexKeyDefinition> keyDefs = def.indexKeyDefinitions();
+
+        List<InlineIndexKeyType> keyTypes = rowHandler().inlineIndexKeyTypes();
+
+        for (keyIdx = 0; keyIdx < keyTypes.size(); keyIdx++) {
+            try {
+                // If a search key is null then skip other keys (consider that null shows that we should get all
+                // possible keys for that comparison).
+                if (row.key(keyIdx) == null)
+                    return 0;
+
+                // Other keys are not inlined. Should compare as rows.
+                if (keyIdx >= keyTypes.size())
+                    break;
+
+                int maxSize = inlineSize - fieldOff;
+
+                InlineIndexKeyType keyType = keyTypes.get(keyIdx);
+
+                IndexKeyDefinition keyDef = keyDefs.get(keyIdx);
+
+                int cmp = COMPARE_UNSUPPORTED;
+
+                // Value can be set up by user in query with different data type.
+                // By default do not compare different types.
+                if (keyDef.validate(row.key(keyIdx))) {
+                    if (keyType.type() != IndexKeyTypes.JAVA_OBJECT || keyTypeSettings.inlineObjSupported()) {
+                        cmp = keyType.compare(pageAddr, off + fieldOff, maxSize, row.key(keyIdx));
+
+                        fieldOff += keyType.inlineSize(pageAddr, off + fieldOff);
+                    }
+                    // If inlining of POJO is not supported then fallback to previous logic.
+                    else
+                        break;
+                }
+
+                // Can't compare as inlined bytes are not enough for comparation.
+                if (cmp == CANT_BE_COMPARE)
+                    break;
+
+                // Try compare stored values for inlined keys with different approach?
+                if (cmp == COMPARE_UNSUPPORTED)
+                    cmp = def.rowComparator().compareKey(
+                        pageAddr, off + fieldOff, maxSize, row.key(keyIdx), keyType.type());
+
+                if (cmp == CANT_BE_COMPARE || cmp == COMPARE_UNSUPPORTED)
+                    break;
+
+                if (cmp != 0)
+                    return applySortOrder(cmp, keyDef.order().sortOrder());
+
+            } catch (Exception e) {
+                throw new IgniteException("Failed to store new index row.", e);
+            }
+        }
+
+        if (keyIdx < keyDefs.size()) {
+            recommender.recommend(row, inlineSize);
+
+            if (currRow == null)
+                currRow = getRow(io, pageAddr, idx);
+
+            int ret = compareFullRows(currRow, row, keyIdx);
+
+            if (ret != 0)
+                return ret;
+        }
+
+        return mvccCompare((MvccIO) io, pageAddr, idx, row);
+    }
+
+    /** */
+    private int compareFullRows(IndexRow currRow, IndexRow row, int from) throws IgniteCheckedException {
+        if (currRow == row)
+            return 0;
+
+        for (int i = from; i < rowHandler().indexKeyDefinitions().size(); i++) {
+            // If a search key is null then skip other keys (consider that null shows that we should get all
+            // possible keys for that comparison).
+            if (row.key(i) == null)
+                return 0;
+
+            int c = def.rowComparator().compareKey(currRow, row, i);
+
+            if (c != 0)
+                return applySortOrder(Integer.signum(c), def.indexKeyDefinitions().get(i).order().sortOrder());
+        }
+
+        return 0;
+    }
+
+    /**
+     * Perform sort order correction.
+     *
+     * @param c Compare result.
+     * @param order Sort order.
+     * @return Fixed compare result.
+     */
+    private static int applySortOrder(int c, SortOrder order) {
+        return order == SortOrder.ASC ? c : -c;
+    }
+
+    /** Creates an index row for this tree. */
+    public IndexRowImpl createIndexRow(long link) throws IgniteCheckedException {
+        IndexRowImpl cachedRow = idxRowCache == null ? null : idxRowCache.get(link);
+
+        if (cachedRow != null)
+            return cachedRow;
+
+        CacheDataRowAdapter row = new CacheDataRowAdapter(link);
+
+        row.initFromLink(cacheContext().group(), CacheDataRowAdapter.RowData.FULL, true);
+
+        IndexRowImpl r = new IndexRowImpl(rowHandler(), row);
+
+        if (idxRowCache != null)
+            idxRowCache.put(r);
+
+        return r;
+    }
+
+    /** Creates an mvcc index row for this tree. */
+    public IndexRowImpl createMvccIndexRow(long link, long mvccCrdVer, long mvccCntr, int mvccOpCntr) throws IgniteCheckedException {
+        IndexRowImpl cachedRow = idxRowCache == null ? null : idxRowCache.get(link);
+
+        if (cachedRow != null)
+            return cachedRow;
+
+        int partId = PageIdUtils.partId(PageIdUtils.pageId(link));
+
+        MvccDataRow row = new MvccDataRow(
+            cacheContext().group(),
+            0,
+            link,
+            partId,
+            null,
+            mvccCrdVer,
+            mvccCntr,
+            mvccOpCntr,
+            true
+        );
+
+        IndexRowImpl r = new IndexRowImpl(rowHandler(), row);
+
+        if (idxRowCache != null)
+            idxRowCache.put(r);
+
+        return r;
+    }
+
+    /** {@inheritDoc} */
+    @Override public IndexRow getRow(BPlusIO<IndexRow> io, long pageAddr, int idx, Object ignore)
+        throws IgniteCheckedException {
+
+        return io.getLookupRow(this, pageAddr, idx);
+    }
+
+    /** */
+    public int inlineSize() {
+        return inlineSize;
+    }
+
+    /**
+     * @param keyTypes Index key types.
+     * @param cfgInlineSize Inline size from index config.
+     * @param maxInlineSize Max inline size from cache config.
+     * @return Inline size.
+     */
+    public static int computeInlineSize(
+        List<InlineIndexKeyType> keyTypes,
+        int cfgInlineSize,
+        int maxInlineSize
+    ) {
+        if (cfgInlineSize == 0)
+            return 0;
+
+        if (F.isEmpty(keyTypes))
+            return 0;
+
+        if (cfgInlineSize != -1)
+            return Math.min(PageIO.MAX_PAYLOAD_SIZE, cfgInlineSize);
+
+        int propSize = maxInlineSize == -1
+            ? IgniteSystemProperties.getInteger(IgniteSystemProperties.IGNITE_MAX_INDEX_PAYLOAD_SIZE, IGNITE_MAX_INDEX_PAYLOAD_SIZE_DEFAULT)
+            : maxInlineSize;
+
+        int size = 0;
+
+        for (InlineIndexKeyType keyType: keyTypes) {
+            if (keyType.inlineSize() <= 0) {
+                size = propSize;
+                break;
+            }
+
+            size += keyType.inlineSize();
+        }
+
+        return Math.min(PageIO.MAX_PAYLOAD_SIZE, size);
+    }
+
+    /** */
+    public GridCacheContext<?, ?> cacheContext() {
+        return cctx;
+    }
+
+    /** Default value for {@code IGNITE_MAX_INDEX_PAYLOAD_SIZE} */
+    public static final int IGNITE_MAX_INDEX_PAYLOAD_SIZE_DEFAULT = 10;
+
+    /**
+     * @return Inline size.
+     * @throws IgniteCheckedException If failed.
+     */
+    public MetaPageInfo metaInfo() throws IgniteCheckedException {
+        final long metaPage = acquirePage(metaPageId);
+
+        try {
+            long pageAddr = readLock(metaPageId, metaPage); // Meta can't be removed.
+
+            assert pageAddr != 0 : "Failed to read lock meta page [metaPageId=" +
+                U.hexLong(metaPageId) + ']';
+
+            try {
+                BPlusMetaIO io = BPlusMetaIO.VERSIONS.forPage(pageAddr);
+
+                return new MetaPageInfo(io, pageAddr);
+            }
+            finally {
+                readUnlock(metaPageId, metaPage, pageAddr);
+            }
+        }
+        finally {
+            releasePage(metaPageId, metaPage);
+        }
+    }
+
+    /**
+     * Update root meta page if need (previous version not supported features flags
+     * and created product version on root meta page).
+     *
+     * @param inlineObjSupported inline POJO by created tree flag.
+     * @throws IgniteCheckedException On error.
+     */
+    private void upgradeMetaPage(boolean inlineObjSupported) throws IgniteCheckedException {
+        final long metaPage = acquirePage(metaPageId);
+
+        try {
+            long pageAddr = writeLock(metaPageId, metaPage); // Meta can't be removed.
+
+            assert pageAddr != 0 : "Failed to read lock meta page [metaPageId=" +
+                U.hexLong(metaPageId) + ']';
+
+            try {
+                BPlusMetaIO.upgradePageVersion(pageAddr, inlineObjSupported, false, pageSize());
+
+                if (wal != null)
+                    wal.log(new PageSnapshot(new FullPageId(metaPageId, grpId),
+                        pageAddr, pageMem.pageSize(), pageMem.realPageSize(grpId)));
+            }
+            finally {
+                writeUnlock(metaPageId, metaPage, pageAddr, true);
+            }
+        }
+        finally {
+            releasePage(metaPageId, metaPage);
+        }
+    }
+
+    /**
+     * Copy info from another meta page.
+     * @param info Meta page info.
+     * @throws IgniteCheckedException If failed.
+     */
+    public void copyMetaInfo(MetaPageInfo info) throws IgniteCheckedException {
+        final long metaPage = acquirePage(metaPageId);
+
+        try {
+            long pageAddr = writeLock(metaPageId, metaPage); // Meta can't be removed.
+
+            assert pageAddr != 0 : "Failed to read lock meta page [metaPageId=" +
+                U.hexLong(metaPageId) + ']';
+
+            try {
+                BPlusMetaIO.setValues(
+                    pageAddr,
+                    info.inlineSize(),
+                    info.useUnwrappedPk(),
+                    info.inlineObjectSupported(),
+                    info.inlineObjectHash()
+                );
+            }
+            finally {
+                writeUnlock(metaPageId, metaPage, pageAddr, true);
+            }
+        }
+        finally {
+            releasePage(metaPageId, metaPage);
+        }
+    }
+
+    /** */
+    public boolean created() {
+        return created;
+    }
+
+    /**
+     * Construct the exception and invoke failure processor.
+     *
+     * @param msg Message.
+     * @param cause Cause.
+     * @param grpId Group id.
+     * @param pageIds Pages ids.
+     * @return New CorruptedTreeException instance.
+     */
+    @Override protected CorruptedTreeException corruptedTreeException(String msg, Throwable cause, int grpId, long... pageIds) {
+        CorruptedTreeException e = new CorruptedTreeException(msg, cause, grpId, grpName, def.idxName().cacheName(),
+            def.idxName().idxName(), pageIds);
+
+        processFailure(FailureType.CRITICAL_ERROR, e);
+
+        return e;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void temporaryReleaseLock() {
+        cctx.kernalContext().cache().context().database().checkpointReadUnlock();
+        cctx.kernalContext().cache().context().database().checkpointReadLock();
+    }
+
+    /** {@inheritDoc} */
+    @Override protected long maxLockHoldTime() {
+        long sysWorkerBlockedTimeout = cctx.kernalContext().workersRegistry().getSystemWorkerBlockedTimeout();
+
+        // Using timeout value reduced by 10 times to increase possibility of lock releasing before timeout.
+        return sysWorkerBlockedTimeout == 0 ? Long.MAX_VALUE : (sysWorkerBlockedTimeout / 10);
+    }
+
+    /** {@inheritDoc} */
+    @Override protected IoStatisticsHolder statisticsHolder() {
+        return stats != null ? stats : super.statisticsHolder();
+    }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return S.toString(InlineIndexTree.class, this, "super", super.toString());
+    }
+
+    /**
+     * @return Index row handler for this tree. Row handler for a tree can be set externally with the holder.
+     */
+    public InlineIndexRowHandler rowHandler() {
+        return rowHnd != null ? rowHnd : ThreadLocalRowHandlerHolder.rowHandler();
+    }
+
+    /**
+     * @param io IO.
+     * @param pageAddr Page address.
+     * @param idx Item index.
+     * @param row Search row.
+     * @return Comparison result.
+     */
+    private int mvccCompare(MvccIO io, long pageAddr, int idx, IndexRow row) {
+        if (!mvccEnabled || row.indexSearchRow())
+            return 0;
+
+        long crd = io.mvccCoordinatorVersion(pageAddr, idx);
+        long cntr = io.mvccCounter(pageAddr, idx);
+        int opCntr = io.mvccOperationCounter(pageAddr, idx);
+
+        assert MvccUtils.mvccVersionIsValid(crd, cntr, opCntr);
+
+        return -MvccUtils.compare(crd, cntr, opCntr, row);  // descending order
+    }
+
+    /**
+     * @param r1 First row.
+     * @param r2 Second row.
+     * @return Comparison result.
+     */
+    private int mvccCompare(IndexRow r1, IndexRow r2) {
+        if (!mvccEnabled || r2.indexSearchRow() || r1 == r2)
+            return 0;
+
+        long crdVer1 = r1.mvccCoordinatorVersion();
+        long crdVer2 = r2.mvccCoordinatorVersion();
+
+        int c = -Long.compare(crdVer1, crdVer2);
+
+        if (c != 0)
+            return c;
+
+        return -Long.compare(r1.mvccCounter(), r2.mvccCounter());
+    }
+}
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/InlineObjectBytesDetector.java b/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/inline/InlineObjectBytesDetector.java
similarity index 67%
rename from modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/InlineObjectBytesDetector.java
rename to modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/inline/InlineObjectBytesDetector.java
index fc77acb..9275749 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/InlineObjectBytesDetector.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/inline/InlineObjectBytesDetector.java
@@ -15,18 +15,23 @@
  * limitations under the License.
  */
 
-package org.apache.ignite.internal.processors.query.h2.database;
+package org.apache.ignite.internal.cache.query.index.sorted.inline;
 
 import java.util.Arrays;
 import java.util.List;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.internal.cache.query.index.IndexName;
+import org.apache.ignite.internal.cache.query.index.sorted.IndexKeyDefinition;
+import org.apache.ignite.internal.cache.query.index.sorted.IndexKeyTypeSettings;
+import org.apache.ignite.internal.cache.query.index.sorted.IndexKeyTypes;
+import org.apache.ignite.internal.cache.query.index.sorted.IndexRow;
+import org.apache.ignite.internal.cache.query.index.sorted.keys.IndexKey;
+import org.apache.ignite.internal.cache.query.index.sorted.keys.JavaObjectIndexKey;
+import org.apache.ignite.internal.cache.query.index.sorted.keys.NullIndexKey;
 import org.apache.ignite.internal.pagemem.PageUtils;
 import org.apache.ignite.internal.processors.cache.persistence.tree.BPlusTree;
 import org.apache.ignite.internal.processors.cache.persistence.tree.io.BPlusIO;
-import org.apache.ignite.internal.processors.query.h2.opt.H2Row;
-import org.h2.value.Value;
-import org.h2.value.ValueNull;
 
 /**
  * This class helps to detect whether tree contains inlined JO type.
@@ -36,45 +41,41 @@ import org.h2.value.ValueNull;
  * 1. analyze of inline size;
  * 2. traverse tree and check stored values.
  */
-public class InlineObjectBytesDetector implements BPlusTree.TreeRowClosure<H2Row, H2Row> {
+public class InlineObjectBytesDetector implements BPlusTree.TreeRowClosure<IndexRow, IndexRow> {
     /** Inline size. */
     private final int inlineSize;
 
     /** Inline helpers. */
-    private final List<InlineIndexColumn> inlineCols;
+    private final List<IndexKeyDefinition> keyDefs;
 
     /** Inline object supported flag. */
     private boolean inlineObjSupported = true;
 
     /** */
-    private final String tblName;
-
-    /** */
-    private final String idxName;
+    private final IndexName idxName;
 
     /** */
     private final IgniteLogger log;
 
     /**
      * @param inlineSize Inline size.
-     * @param inlineCols Inline columns.
+     * @param keyDefs Index key definitions.
      * @param idxName Index name.
      * @param log Ignite logger.
      */
-    InlineObjectBytesDetector(int inlineSize, List<InlineIndexColumn> inlineCols, String tblName, String idxName,
+    public InlineObjectBytesDetector(int inlineSize, List<IndexKeyDefinition> keyDefs, IndexName idxName,
         IgniteLogger log) {
         this.inlineSize = inlineSize;
-        this.inlineCols = inlineCols;
-        this.tblName = tblName;
+        this.keyDefs = keyDefs;
         this.idxName = idxName;
         this.log = log;
     }
 
     /** {@inheritDoc} */
-    @Override public boolean apply(BPlusTree<H2Row, H2Row> tree, BPlusIO<H2Row> io,
+    @Override public boolean apply(BPlusTree<IndexRow, IndexRow> tree, BPlusIO<IndexRow> io,
         long pageAddr,
         int idx) throws IgniteCheckedException {
-        H2Row r = tree.getRow(io, pageAddr, idx);
+        IndexRow r = tree.getRow(io, pageAddr, idx);
 
         int off = io.offset(idx);
 
@@ -82,33 +83,39 @@ public class InlineObjectBytesDetector implements BPlusTree.TreeRowClosure<H2Row
 
         boolean varLenPresents = false;
 
-        for (InlineIndexColumn ih : inlineCols) {
+        IndexKeyTypeSettings keyTypeSettings = new IndexKeyTypeSettings();
+
+        for (int i = 0; i < keyDefs.size(); ++i) {
+            IndexKeyDefinition keyDef = keyDefs.get(i);
+
             if (fieldOff >= inlineSize)
                 return false;
 
-            if (ih.type() != Value.JAVA_OBJECT) {
-                if (ih.size() < 0)
+            if (keyDef.idxType() != IndexKeyTypes.JAVA_OBJECT) {
+                InlineIndexKeyType keyType = InlineIndexKeyTypeRegistry.get(keyDef.idxType(), keyTypeSettings);
+
+                if (keyType.inlineSize() < 0)
                     varLenPresents = true;
 
-                fieldOff += ih.fullSize(pageAddr, off + fieldOff);
+                fieldOff += keyType.inlineSize(pageAddr, off + fieldOff);
 
                 continue;
             }
 
-            Value val = r.getValue(ih.columnIndex());
+            IndexKey key = r.key(i);
 
-            if (val == ValueNull.INSTANCE)
+            if (key == NullIndexKey.INSTANCE)
                 return false;
 
             int type = PageUtils.getByte(pageAddr, off + fieldOff);
 
             // We can have garbage in memory and need to compare data.
-            if (type == Value.JAVA_OBJECT) {
+            if (type == IndexKeyTypes.JAVA_OBJECT) {
                 int len = PageUtils.getShort(pageAddr, off + fieldOff + 1);
 
                 len &= 0x7FFF;
 
-                byte[] originalObjBytes = val.getBytesNoCopy();
+                byte[] originalObjBytes = ((JavaObjectIndexKey) key).bytesNoCopy();
 
                 // Read size more then available space or more then origin length.
                 if (len > inlineSize - fieldOff - 3 || len > originalObjBytes.length) {
@@ -131,7 +138,7 @@ public class InlineObjectBytesDetector implements BPlusTree.TreeRowClosure<H2Row
                 return true;
             }
 
-            if (type == Value.UNKNOWN && varLenPresents) {
+            if (type == IndexKeyTypes.UNKNOWN && varLenPresents) {
                 // We can't guarantee in case unknown type and should check next row:
                 // 1: long string, UNKNOWN for java object.
                 // 2: short string, inlined java object
@@ -160,20 +167,28 @@ public class InlineObjectBytesDetector implements BPlusTree.TreeRowClosure<H2Row
      * e.g.: indexed: (long, obj) and inline_size < 12.
      * In this case there is no space for inline object.
      *
-     * @param inlineCols Inline columns.
+     * @param keyDefs Index key definition.
      * @param inlineSize Inline size.
      *
      * @return {@code true} If the object may be inlined.
      */
-    public static boolean objectMayBeInlined(int inlineSize, List<InlineIndexColumn> inlineCols) {
+    public static boolean objectMayBeInlined(int inlineSize, List<IndexKeyDefinition> keyDefs) {
         int remainSize = inlineSize;
 
-        for (InlineIndexColumn ih : inlineCols) {
-            if (ih.type() == Value.JAVA_OBJECT)
+        // The settings does not affect on inline size.
+        IndexKeyTypeSettings settings = new IndexKeyTypeSettings();
+
+        for (IndexKeyDefinition def: keyDefs) {
+            if (def.idxType() == IndexKeyTypes.JAVA_OBJECT)
                 break;
 
+            InlineIndexKeyType keyType = InlineIndexKeyTypeRegistry.get(def.idxType(), settings);
+
+            if (keyType == null)
+                return false;
+
             // Set size to 1 for variable length columns as that value can be set by user.
-            remainSize -= ih.size() > 0 ? 1 + ih.size() : 1;
+            remainSize -= keyType.inlineSize() > 0 ? 1 + keyType.inlineSize() : 1;
         }
 
         // For old versions JO type was inlined as byte array.
@@ -188,10 +203,10 @@ public class InlineObjectBytesDetector implements BPlusTree.TreeRowClosure<H2Row
         this.inlineObjSupported = inlineObjSupported;
 
         if (inlineObjSupported)
-            log.warning("Index supports JAVA_OBJECT type inlining [tblName=" + tblName + ", idxName=" +
+            log.warning("Index supports JAVA_OBJECT type inlining [tblName=" + idxName.tableName() + ", idxName=" +
                 idxName + ", reason='" + reason + "']");
         else
-            log.warning("Index doesn't support JAVA_OBJECT type inlining [tblName=" + tblName + ", idxName=" +
+            log.warning("Index doesn't support JAVA_OBJECT type inlining [tblName=" + idxName.tableName() + ", idxName=" +
                 idxName + ", reason='" + reason + "']");
     }
 }
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/inline/InlineRecommender.java b/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/inline/InlineRecommender.java
new file mode 100644
index 0000000..16820a7
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/inline/InlineRecommender.java
@@ -0,0 +1,141 @@
+/*
+ * 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.cache.query.index.sorted.inline;
+
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.stream.Collectors;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.IgniteSystemProperties;
+import org.apache.ignite.SystemProperty;
+import org.apache.ignite.internal.cache.query.index.sorted.IndexKeyDefinition;
+import org.apache.ignite.internal.cache.query.index.sorted.IndexRow;
+import org.apache.ignite.internal.cache.query.index.sorted.SortedIndexDefinition;
+import org.apache.ignite.internal.processors.cache.GridCacheContext;
+import org.apache.ignite.internal.util.typedef.internal.U;
+
+/**
+ * Write to a log recommendation for inline size.
+ */
+public class InlineRecommender {
+    /** @see #IGNITE_THROTTLE_INLINE_SIZE_CALCULATION */
+    public static final int DFLT_THROTTLE_INLINE_SIZE_CALCULATION = 1_000;
+
+    /** */
+    @SystemProperty(value = "How often real invocation of inline size calculation will be skipped.", type = Long.class,
+        defaults = "" + DFLT_THROTTLE_INLINE_SIZE_CALCULATION)
+    public static final String IGNITE_THROTTLE_INLINE_SIZE_CALCULATION = "IGNITE_THROTTLE_INLINE_SIZE_CALCULATION";
+
+    /** Counter of inline size calculation for throttling real invocations. */
+    private final AtomicLong inlineSizeCalculationCntr = new AtomicLong();
+
+    /** How often real invocation of inline size calculation will be skipped. */
+    private final int inlineSizeThrottleThreshold =
+        IgniteSystemProperties.getInteger(IGNITE_THROTTLE_INLINE_SIZE_CALCULATION,
+            DFLT_THROTTLE_INLINE_SIZE_CALCULATION);
+
+    /** Keep max calculated inline size for current index. */
+    private final AtomicInteger maxCalculatedInlineSize = new AtomicInteger();
+
+    /** Ignite logger. */
+    private final IgniteLogger log;
+
+    /** Index definition. */
+    private final SortedIndexDefinition def;
+
+    /** Constructor. */
+    public InlineRecommender(GridCacheContext<?, ?> cctx, SortedIndexDefinition def) {
+        log = cctx.kernalContext().indexProcessor().logger();
+        this.def = def;
+    }
+
+    /**
+     * Calculate aggregate inline size for given indexes and log recommendation in case calculated size more than
+     * current inline size.
+     */
+    @SuppressWarnings({"ConditionalBreakInInfiniteLoop", "IfMayBeConditional"})
+    public void recommend(IndexRow row, int currInlineSize) {
+        // Do the check only for put operations.
+        if (row.indexSearchRow())
+            return;
+
+        long invokeCnt = inlineSizeCalculationCntr.get();
+
+        if (!inlineSizeCalculationCntr.compareAndSet(invokeCnt, invokeCnt + 1))
+            return;
+
+        boolean throttle = invokeCnt + 1 % inlineSizeThrottleThreshold != 0;
+
+        if (throttle)
+            return;
+
+        int newSize = 0;
+
+        for (int i = 0; i < row.rowHandler().inlineIndexKeyTypes().size(); i++) {
+            InlineIndexKeyType keyType = row.rowHandler().inlineIndexKeyTypes().get(i);
+
+            newSize += keyType.inlineSize(row.key(i));
+        }
+
+        if (newSize > currInlineSize) {
+            int oldSize;
+
+            while (true) {
+                oldSize = maxCalculatedInlineSize.get();
+
+                if (oldSize >= newSize)
+                    return;
+
+                if (maxCalculatedInlineSize.compareAndSet(oldSize, newSize))
+                    break;
+            }
+
+            String cols = def.indexKeyDefinitions().stream()
+                .map(IndexKeyDefinition::name)
+                .collect(Collectors.joining(", ", "(", ")"));
+
+            String type = def.primary() ? "PRIMARY KEY" : def.affinity() ? "AFFINITY KEY (implicit)" : "SECONDARY";
+
+            String recommendation;
+
+            if (def.primary() || def.affinity()) {
+                recommendation = "set system property "
+                    + IgniteSystemProperties.IGNITE_MAX_INDEX_PAYLOAD_SIZE + " with recommended size " +
+                    "(be aware it will be used by default for all indexes without explicit inline size)";
+            }
+            else {
+                recommendation = "use INLINE_SIZE option for CREATE INDEX command, " +
+                    "QuerySqlField.inlineSize for annotated classes, or QueryIndex.inlineSize for explicit " +
+                    "QueryEntity configuration";
+            }
+
+            String warn = "Indexed columns of a row cannot be fully inlined into index " +
+                "what may lead to slowdown due to additional data page reads, increase index inline size if needed " +
+                "(" + recommendation + ") " +
+                "[cacheName=" + def.idxName().cacheName() +
+                ", tableName=" + def.idxName().tableName() +
+                ", idxName=" + def.idxName().idxName() +
+                ", idxCols=" + cols +
+                ", idxType=" + type +
+                ", curSize=" + currInlineSize +
+                ", recommendedInlineSize=" + newSize + "]";
+
+            U.warn(log, warn);
+        }
+    }
+}
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/H2TreeFilterClosure.java b/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/inline/InlineTreeFilterClosure.java
similarity index 77%
rename from modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/H2TreeFilterClosure.java
rename to modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/inline/InlineTreeFilterClosure.java
index 8cebb1a..cd7bcb4 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/H2TreeFilterClosure.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/inline/InlineTreeFilterClosure.java
@@ -15,10 +15,12 @@
  * limitations under the License.
  */
 
-package org.apache.ignite.internal.processors.query.h2.database;
+package org.apache.ignite.internal.cache.query.index.sorted.inline;
 
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.internal.cache.query.index.sorted.IndexRow;
+import org.apache.ignite.internal.cache.query.index.sorted.inline.io.InlineIO;
 import org.apache.ignite.internal.pagemem.PageIdUtils;
 import org.apache.ignite.internal.processors.cache.GridCacheContext;
 import org.apache.ignite.internal.processors.cache.mvcc.MvccSnapshot;
@@ -26,8 +28,6 @@ import org.apache.ignite.internal.processors.cache.persistence.tree.BPlusTree;
 import org.apache.ignite.internal.processors.cache.persistence.tree.io.BPlusIO;
 import org.apache.ignite.internal.processors.cache.persistence.tree.io.DataPageIO;
 import org.apache.ignite.internal.processors.cache.tree.mvcc.search.MvccDataPageClosure;
-import org.apache.ignite.internal.processors.query.h2.database.io.H2RowLinkIO;
-import org.apache.ignite.internal.processors.query.h2.opt.H2Row;
 import org.apache.ignite.internal.transactions.IgniteTxUnexpectedStateCheckedException;
 import org.apache.ignite.internal.util.typedef.X;
 import org.apache.ignite.internal.util.typedef.internal.S;
@@ -38,9 +38,9 @@ import static org.apache.ignite.internal.processors.cache.mvcc.MvccUtils.isVisib
 import static org.apache.ignite.internal.processors.cache.mvcc.MvccUtils.mvccVersionIsValid;
 
 /**
- *
+ * Reopresents filter that allow query only primary partitions.
  */
-public class H2TreeFilterClosure implements H2Tree.TreeRowClosure<H2Row, H2Row>, MvccDataPageClosure {
+public class InlineTreeFilterClosure implements BPlusTree.TreeRowClosure<IndexRow, IndexRow>, MvccDataPageClosure {
     /** */
     private final MvccSnapshot mvccSnapshot;
 
@@ -53,13 +53,9 @@ public class H2TreeFilterClosure implements H2Tree.TreeRowClosure<H2Row, H2Row>,
     /** */
     private final IgniteLogger log;
 
-    /**
-     * @param filter Cache filter.
-     * @param mvccSnapshot MVCC snapshot.
-     * @param cctx Cache context.
-     */
-    public H2TreeFilterClosure(IndexingQueryCacheFilter filter, MvccSnapshot mvccSnapshot, GridCacheContext cctx,
-        IgniteLogger log) {
+    /** Constructor. */
+    public InlineTreeFilterClosure(IndexingQueryCacheFilter filter, MvccSnapshot mvccSnapshot,
+        GridCacheContext<?, ?> cctx, IgniteLogger log) {
         assert (filter != null || mvccSnapshot != null) && cctx != null;
 
         this.filter = filter;
@@ -69,10 +65,15 @@ public class H2TreeFilterClosure implements H2Tree.TreeRowClosure<H2Row, H2Row>,
     }
 
     /** {@inheritDoc} */
-    @Override public boolean apply(BPlusTree<H2Row, H2Row> tree, BPlusIO<H2Row> io,
+    @Override public boolean apply(BPlusTree<IndexRow, IndexRow> tree, BPlusIO<IndexRow> io,
         long pageAddr, int idx) throws IgniteCheckedException {
-        return (filter == null || applyFilter((H2RowLinkIO)io, pageAddr, idx))
-            && (mvccSnapshot == null || applyMvcc((H2RowLinkIO)io, pageAddr, idx));
+
+        boolean val = filter == null || applyFilter((InlineIO)io, pageAddr, idx);
+
+        if (mvccSnapshot != null)
+            return val && applyMvcc((InlineIO) io, pageAddr, idx);
+
+        return val;
     }
 
     /**
@@ -81,10 +82,10 @@ public class H2TreeFilterClosure implements H2Tree.TreeRowClosure<H2Row, H2Row>,
      * @param idx Item index.
      * @return {@code True} if row passes the filter.
      */
-    private boolean applyFilter(H2RowLinkIO io, long pageAddr, int idx) {
+    private boolean applyFilter(InlineIO io, long pageAddr, int idx) {
         assert filter != null;
 
-        return filter.applyPartition(PageIdUtils.partId(pageId(io.getLink(pageAddr, idx))));
+        return filter.applyPartition(PageIdUtils.partId(pageId(io.link(pageAddr, idx))));
     }
 
     /**
@@ -93,17 +94,17 @@ public class H2TreeFilterClosure implements H2Tree.TreeRowClosure<H2Row, H2Row>,
      * @param idx Item index.
      * @return {@code True} if row passes the filter.
      */
-    private boolean applyMvcc(H2RowLinkIO io, long pageAddr, int idx) throws IgniteCheckedException {
+    private boolean applyMvcc(InlineIO io, long pageAddr, int idx) throws IgniteCheckedException {
         assert io.storeMvccInfo() : io;
 
-        long rowCrdVer = io.getMvccCoordinatorVersion(pageAddr, idx);
-        long rowCntr = io.getMvccCounter(pageAddr, idx);
-        int rowOpCntr = io.getMvccOperationCounter(pageAddr, idx);
+        long rowCrdVer = io.mvccCoordinatorVersion(pageAddr, idx);
+        long rowCntr = io.mvccCounter(pageAddr, idx);
+        int rowOpCntr = io.mvccOperationCounter(pageAddr, idx);
 
         assert mvccVersionIsValid(rowCrdVer, rowCntr, rowOpCntr);
 
         try {
-            return isVisible(cctx, mvccSnapshot, rowCrdVer, rowCntr, rowOpCntr, io.getLink(pageAddr, idx));
+            return isVisible(cctx, mvccSnapshot, rowCrdVer, rowCntr, rowOpCntr, io.link(pageAddr, idx));
         }
         catch (IgniteTxUnexpectedStateCheckedException e) {
             // TODO this catch must not be needed if we switch Vacuum to data page scan
@@ -136,6 +137,6 @@ public class H2TreeFilterClosure implements H2Tree.TreeRowClosure<H2Row, H2Row>,
 
     /** {@inheritDoc} */
     @Override public String toString() {
-        return S.toString(H2TreeFilterClosure.class, this);
+        return S.toString(InlineTreeFilterClosure.class, this);
     }
 }
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/H2JavaObjectSerializer.java b/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/inline/JavaObjectKeySerializer.java
similarity index 63%
copy from modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/H2JavaObjectSerializer.java
copy to modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/inline/JavaObjectKeySerializer.java
index 97abcd2..98ba904 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/H2JavaObjectSerializer.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/inline/JavaObjectKeySerializer.java
@@ -15,18 +15,17 @@
  * limitations under the License.
  */
 
-package org.apache.ignite.internal.processors.query.h2;
+package org.apache.ignite.internal.cache.query.index.sorted.inline;
 
-import org.apache.ignite.internal.GridKernalContext;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.configuration.IgniteConfiguration;
 import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.marshaller.Marshaller;
-import org.h2.api.JavaObjectSerializer;
-import org.jetbrains.annotations.NotNull;
 
 /**
- * Ignite java object serializer implementation for H2.
+ * Serializer for representing JO as byte array in inline.
  */
-class H2JavaObjectSerializer implements JavaObjectSerializer {
+public class JavaObjectKeySerializer {
     /** Class loader. */
     private final ClassLoader clsLdr;
 
@@ -36,20 +35,20 @@ class H2JavaObjectSerializer implements JavaObjectSerializer {
     /**
      * Constructor.
      *
-     * @param ctx Kernal context.
+     * @param cfg Ignite configuration.
      */
-    H2JavaObjectSerializer(@NotNull GridKernalContext ctx) {
-        marshaller = ctx.config().getMarshaller();
-        clsLdr = U.resolveClassLoader(ctx.config());
+    public JavaObjectKeySerializer(IgniteConfiguration cfg) {
+        marshaller = cfg.getMarshaller();
+        clsLdr = U.resolveClassLoader(cfg);
     }
 
-    /** {@inheritDoc} */
-    @Override public byte[] serialize(Object obj) throws Exception {
+    /** */
+    public byte[] serialize(Object obj) throws IgniteCheckedException {
         return U.marshal(marshaller, obj);
     }
 
-    /** {@inheritDoc} */
-    @Override public Object deserialize(byte[] bytes) throws Exception {
+    /** */
+    public Object deserialize(byte[] bytes) throws IgniteCheckedException {
         return U.unmarshal(marshaller, bytes, clsLdr);
     }
 }
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/inline/io/AbstractInlineInnerIO.java b/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/inline/io/AbstractInlineInnerIO.java
new file mode 100644
index 0000000..730eee2
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/inline/io/AbstractInlineInnerIO.java
@@ -0,0 +1,166 @@
+/*
+ * 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.cache.query.index.sorted.inline.io;
+
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.internal.cache.query.index.sorted.IndexRow;
+import org.apache.ignite.internal.cache.query.index.sorted.InlineIndexRowHandler;
+import org.apache.ignite.internal.cache.query.index.sorted.ThreadLocalRowHandlerHolder;
+import org.apache.ignite.internal.cache.query.index.sorted.inline.InlineIndexKeyType;
+import org.apache.ignite.internal.cache.query.index.sorted.inline.InlineIndexTree;
+import org.apache.ignite.internal.pagemem.PageUtils;
+import org.apache.ignite.internal.processors.cache.persistence.tree.BPlusTree;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.BPlusIO;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.BPlusInnerIO;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.IOVersions;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.PageIO;
+
+/**
+ * Inner page to store index rows with inlined keys.
+ */
+public abstract class AbstractInlineInnerIO extends BPlusInnerIO<IndexRow> implements InlineIO {
+    /**
+     * Amount of bytes to store inlined index keys.
+     *
+     * We do not store schema there:
+     * 1. IOs are shared between multiple indexes with the same inlineSize.
+     * 2. For backward compatibility, to restore index from PDS.
+     */
+    private final int inlineSize;
+
+    /**
+     * @param type Page type.
+     * @param ver Page format version.
+     * @param metaSize Size of item - information of a cache row.
+     * @param inlineSize Size of calculated inlined index keys.
+     */
+    AbstractInlineInnerIO(short type, int ver, int metaSize, int inlineSize) {
+        super(type, ver, true, metaSize + inlineSize);
+
+        this.inlineSize = inlineSize;
+    }
+
+    /**
+     * Register IOs for every available {@link #inlineSize} for MVCC and not.
+     */
+    public static void register() {
+        register(false);
+        register(true);
+    }
+
+    /** */
+    private static void register(boolean mvcc) {
+        short type = mvcc ? PageIO.T_H2_EX_REF_MVCC_INNER_START : PageIO.T_H2_EX_REF_INNER_START;
+
+        for (short payload = 1; payload <= PageIO.MAX_PAYLOAD_SIZE; payload++) {
+            short ioType = (short) (type + payload - 1);
+
+            AbstractInlineInnerIO io = mvcc ? new MvccInlineInnerIO(ioType, payload) : new InlineInnerIO(ioType, payload);
+
+            IOVersions<? extends AbstractInlineInnerIO> versions = new IOVersions<>(io);
+
+            PageIO.registerH2ExtraInner(versions, mvcc);
+        }
+    }
+
+    /** {@inheritDoc} */
+    @SuppressWarnings("ForLoopReplaceableByForEach")
+    @Override public final void storeByOffset(long pageAddr, int off, IndexRow row) {
+        assert row.link() != 0 : row;
+
+        int fieldOff = 0;
+
+        InlineIndexRowHandler rowHnd = ThreadLocalRowHandlerHolder.rowHandler();
+
+        for (int i = 0; i < rowHnd.inlineIndexKeyTypes().size(); i++) {
+            try {
+                InlineIndexKeyType keyType = rowHnd.inlineIndexKeyTypes().get(i);
+
+                int size = keyType.put(pageAddr, off + fieldOff, row.key(i), inlineSize - fieldOff);
+
+                // Inline size has exceeded.
+                if (size == 0)
+                    break;
+
+                fieldOff += size;
+
+            } catch (Exception e) {
+                throw new IgniteException("Failed to store new index row.", e);
+            }
+        }
+
+        IORowHandler.store(pageAddr, off + inlineSize, row, storeMvccInfo());
+    }
+
+    /** {@inheritDoc} */
+    @Override public final IndexRow getLookupRow(BPlusTree<IndexRow, ?> tree, long pageAddr, int idx)
+        throws IgniteCheckedException {
+
+        long link = PageUtils.getLong(pageAddr, offset(idx) + inlineSize);
+
+        assert link != 0;
+
+        if (storeMvccInfo()) {
+            long mvccCrdVer = mvccCoordinatorVersion(pageAddr, idx);
+            long mvccCntr = mvccCounter(pageAddr, idx);
+            int mvccOpCntr = mvccOperationCounter(pageAddr, idx);
+
+            return ((InlineIndexTree) tree).createMvccIndexRow(link, mvccCrdVer, mvccCntr, mvccOpCntr);
+        }
+
+        return ((InlineIndexTree) tree).createIndexRow(link);
+    }
+
+    /** {@inheritDoc} */
+    @Override public final void store(long dstPageAddr, int dstIdx, BPlusIO<IndexRow> srcIo, long srcPageAddr, int srcIdx) {
+        int srcOff = srcIo.offset(srcIdx);
+
+        byte[] payload = PageUtils.getBytes(srcPageAddr, srcOff, inlineSize);
+
+        int dstOff = offset(dstIdx);
+
+        PageUtils.putBytes(dstPageAddr, dstOff, payload);
+
+        IORowHandler.store(dstPageAddr, dstOff + inlineSize, (InlineIO) srcIo, srcPageAddr, srcIdx, storeMvccInfo());
+    }
+
+    /** {@inheritDoc} */
+    @Override public long link(long pageAddr, int idx) {
+        return PageUtils.getLong(pageAddr, offset(idx) + inlineSize);
+    }
+
+    /** {@inheritDoc} */
+    @Override public int inlineSize() {
+        return inlineSize;
+    }
+
+    /**
+     * @param payload Payload size.
+     * @param mvccEnabled Whether MVCC is enabled.
+     * @return IOVersions for given payload.
+     */
+    public static IOVersions<? extends BPlusInnerIO<IndexRow>> versions(int payload, boolean mvccEnabled) {
+        assert payload >= 0 && payload <= PageIO.MAX_PAYLOAD_SIZE;
+
+        if (payload == 0)
+            return mvccEnabled ? MvccInnerIO.VERSIONS : InnerIO.VERSIONS;
+        else
+            return (IOVersions<BPlusInnerIO<IndexRow>>)PageIO.getInnerVersions((short)(payload - 1), mvccEnabled);
+    }
+}
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/inline/io/AbstractInlineLeafIO.java b/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/inline/io/AbstractInlineLeafIO.java
new file mode 100644
index 0000000..d8bfecb
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/inline/io/AbstractInlineLeafIO.java
@@ -0,0 +1,166 @@
+/*
+ * 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.cache.query.index.sorted.inline.io;
+
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.internal.cache.query.index.sorted.IndexRow;
+import org.apache.ignite.internal.cache.query.index.sorted.InlineIndexRowHandler;
+import org.apache.ignite.internal.cache.query.index.sorted.ThreadLocalRowHandlerHolder;
+import org.apache.ignite.internal.cache.query.index.sorted.inline.InlineIndexKeyType;
+import org.apache.ignite.internal.cache.query.index.sorted.inline.InlineIndexTree;
+import org.apache.ignite.internal.pagemem.PageUtils;
+import org.apache.ignite.internal.processors.cache.persistence.tree.BPlusTree;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.BPlusIO;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.BPlusLeafIO;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.IOVersions;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.PageIO;
+
+/**
+ * Leaf page to store index rows with inlined keys.
+ */
+public abstract class AbstractInlineLeafIO extends BPlusLeafIO<IndexRow> implements InlineIO {
+    /**
+     * Amount of bytes to store inlined index keys.
+     *
+     * We do not store schema there:
+     * 1. IOs are shared between multiple indexes with the same inlineSize.
+     * 2. For backward compatibility, to restore index from PDS.
+     */
+    private final int inlineSize;
+
+    /**
+     * @param type Page type.
+     * @param ver Page format version.
+     * @param metaSize Size of item - information of a cache row.
+     * @param inlineSize Size of calculated inlined index keys.
+     */
+    AbstractInlineLeafIO(short type, int ver, int metaSize, int inlineSize) {
+        super(type, ver, metaSize + inlineSize);
+
+        this.inlineSize = inlineSize;
+    }
+
+    /**
+     * Register IOs for every available {@link #inlineSize} for MVCC and not.
+     */
+    public static void register() {
+        register(false);
+        register(true);
+    }
+
+    /** */
+    private static void register(boolean mvcc) {
+        short type = mvcc ? PageIO.T_H2_EX_REF_MVCC_LEAF_START : PageIO.T_H2_EX_REF_LEAF_START;
+
+        for (short payload = 1; payload <= PageIO.MAX_PAYLOAD_SIZE; payload++) {
+            short ioType = (short) (type + payload - 1);
+
+            AbstractInlineLeafIO io = mvcc ? new MvccInlineLeafIO(ioType, payload) : new InlineLeafIO(ioType, payload);
+
+            IOVersions<? extends AbstractInlineLeafIO> versions = new IOVersions<>(io);
+
+            PageIO.registerH2ExtraLeaf(versions, mvcc);
+        }
+    }
+
+    /** {@inheritDoc} */
+    @SuppressWarnings("ForLoopReplaceableByForEach")
+    @Override public final void storeByOffset(long pageAddr, int off, IndexRow row) {
+        assert row.link() != 0 : row;
+
+        int fieldOff = 0;
+
+        InlineIndexRowHandler rowHnd = ThreadLocalRowHandlerHolder.rowHandler();
+
+        for (int i = 0; i < rowHnd.inlineIndexKeyTypes().size(); i++) {
+            try {
+                InlineIndexKeyType keyType = rowHnd.inlineIndexKeyTypes().get(i);
+
+                int size = keyType.put(pageAddr, off + fieldOff, row.key(i), inlineSize - fieldOff);
+
+                // Inline size has exceeded.
+                if (size == 0)
+                    break;
+
+                fieldOff += size;
+
+            } catch (Exception e) {
+                throw new IgniteException("Failed to store new index row.", e);
+            }
+        }
+
+        IORowHandler.store(pageAddr, off + inlineSize, row, storeMvccInfo());
+    }
+
+    /** {@inheritDoc} */
+    @Override public final IndexRow getLookupRow(BPlusTree<IndexRow, ?> tree, long pageAddr, int idx)
+        throws IgniteCheckedException {
+
+        long link = PageUtils.getLong(pageAddr, offset(idx) + inlineSize);
+
+        assert link != 0;
+
+        if (storeMvccInfo()) {
+            long mvccCrdVer = mvccCoordinatorVersion(pageAddr, idx);
+            long mvccCntr = mvccCounter(pageAddr, idx);
+            int mvccOpCntr = mvccOperationCounter(pageAddr, idx);
+
+            return ((InlineIndexTree) tree).createMvccIndexRow(link, mvccCrdVer, mvccCntr, mvccOpCntr);
+        }
+
+        return ((InlineIndexTree) tree).createIndexRow(link);
+    }
+
+    /** {@inheritDoc} */
+    @Override public final void store(long dstPageAddr, int dstIdx, BPlusIO<IndexRow> srcIo, long srcPageAddr, int srcIdx) {
+        int srcOff = srcIo.offset(srcIdx);
+
+        byte[] payload = PageUtils.getBytes(srcPageAddr, srcOff, inlineSize);
+
+        int dstOff = offset(dstIdx);
+
+        PageUtils.putBytes(dstPageAddr, dstOff, payload);
+
+        IORowHandler.store(dstPageAddr, dstOff + inlineSize, (InlineIO) srcIo, srcPageAddr, srcIdx, storeMvccInfo());
+    }
+
+    /** {@inheritDoc} */
+    @Override public long link(long pageAddr, int idx) {
+        return PageUtils.getLong(pageAddr, offset(idx) + inlineSize);
+    }
+
+    /** {@inheritDoc} */
+    @Override public int inlineSize() {
+        return inlineSize;
+    }
+
+    /**
+     * @param payload Payload size.
+     * @param mvccEnabled Whether MVCC is enabled.
+     * @return IOVersions for given payload.
+     */
+    public static IOVersions<? extends BPlusLeafIO<IndexRow>> versions(int payload, boolean mvccEnabled) {
+        assert payload >= 0 && payload <= PageIO.MAX_PAYLOAD_SIZE;
+
+        if (payload == 0)
+            return mvccEnabled ? MvccLeafIO.VERSIONS : LeafIO.VERSIONS;
+        else
+            return (IOVersions<BPlusLeafIO<IndexRow>>)PageIO.getLeafVersions((short)(payload - 1), mvccEnabled);
+    }
+}
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/io/AbstractH2InnerIO.java b/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/inline/io/AbstractInnerIO.java
similarity index 57%
rename from modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/io/AbstractH2InnerIO.java
rename to modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/inline/io/AbstractInnerIO.java
index 76cccf3..18e69d9 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/io/AbstractH2InnerIO.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/inline/io/AbstractInnerIO.java
@@ -15,65 +15,67 @@
  * limitations under the License.
  */
 
-package org.apache.ignite.internal.processors.query.h2.database.io;
+package org.apache.ignite.internal.cache.query.index.sorted.inline.io;
 
 import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.internal.cache.query.index.sorted.IndexRow;
+import org.apache.ignite.internal.cache.query.index.sorted.inline.InlineIndexTree;
 import org.apache.ignite.internal.pagemem.PageUtils;
 import org.apache.ignite.internal.processors.cache.persistence.tree.BPlusTree;
 import org.apache.ignite.internal.processors.cache.persistence.tree.io.BPlusIO;
 import org.apache.ignite.internal.processors.cache.persistence.tree.io.BPlusInnerIO;
-import org.apache.ignite.internal.processors.query.h2.database.H2Tree;
-import org.apache.ignite.internal.processors.query.h2.opt.H2CacheRow;
-import org.apache.ignite.internal.processors.query.h2.opt.H2Row;
 
 /**
- * Inner page for H2 row references.
+ * Inner page to store index rows.
  */
-public abstract class AbstractH2InnerIO extends BPlusInnerIO<H2Row> implements H2RowLinkIO {
+public abstract class AbstractInnerIO extends BPlusInnerIO<IndexRow> implements InlineIO {
     /**
      * @param type Page type.
      * @param ver Page format version.
      * @param itemSize Single item size on page.
      */
-    AbstractH2InnerIO(int type, int ver, int itemSize) {
+    AbstractInnerIO(int type, int ver, int itemSize) {
         super(type, ver, true, itemSize);
     }
 
     /** {@inheritDoc} */
-    @Override public void storeByOffset(long pageAddr, int off, H2Row row) {
-        H2CacheRow row0 = (H2CacheRow)row;
+    @Override public void storeByOffset(long pageAddr, int off, IndexRow row) {
+        assert row.link() != 0;
 
-        H2IOUtils.storeRow(row0, pageAddr, off, storeMvccInfo());
+        IORowHandler.store(pageAddr, off, row, storeMvccInfo());
     }
 
     /** {@inheritDoc} */
-    @Override public H2Row getLookupRow(BPlusTree<H2Row, ?> tree, long pageAddr, int idx)
+    @Override public IndexRow getLookupRow(BPlusTree<IndexRow, ?> tree, long pageAddr, int idx)
         throws IgniteCheckedException {
-        long link = getLink(pageAddr, idx);
+
+        long link = PageUtils.getLong(pageAddr, offset(idx));
+
+        assert link != 0;
 
         if (storeMvccInfo()) {
-            long mvccCrdVer = getMvccCoordinatorVersion(pageAddr, idx);
-            long mvccCntr = getMvccCounter(pageAddr, idx);
-            int mvccOpCntr = getMvccOperationCounter(pageAddr, idx);
+            long mvccCrdVer = mvccCoordinatorVersion(pageAddr, idx);
+            long mvccCntr = mvccCounter(pageAddr, idx);
+            int mvccOpCntr = mvccOperationCounter(pageAddr, idx);
 
-            return ((H2Tree)tree).createMvccRow(link, mvccCrdVer, mvccCntr, mvccOpCntr);
+            return ((InlineIndexTree) tree).createMvccIndexRow(link, mvccCrdVer, mvccCntr, mvccOpCntr);
         }
 
-        return ((H2Tree)tree).createRow(link);
+        return ((InlineIndexTree) tree).createIndexRow(link);
     }
 
     /** {@inheritDoc} */
-    @Override public void store(long dstPageAddr, int dstIdx, BPlusIO<H2Row> srcIo, long srcPageAddr, int srcIdx) {
-        H2IOUtils.store(dstPageAddr, offset(dstIdx), srcIo, srcPageAddr, srcIdx, storeMvccInfo());
+    @Override public void store(long dstPageAddr, int dstIdx, BPlusIO<IndexRow> srcIo, long srcPageAddr, int srcIdx) {
+        IORowHandler.store(dstPageAddr, offset(dstIdx), (InlineIO) srcIo, srcPageAddr, srcIdx, storeMvccInfo());
     }
 
     /** {@inheritDoc} */
-    @Override public long getLink(long pageAddr, int idx) {
+    @Override public long link(long pageAddr, int idx) {
         return PageUtils.getLong(pageAddr, offset(idx));
     }
 
     /** {@inheritDoc} */
-    @Override public int getPayloadSize() {
+    @Override public int inlineSize() {
         return 0;
     }
 }
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/io/AbstractH2LeafIO.java b/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/inline/io/AbstractLeafIO.java
similarity index 53%
rename from modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/io/AbstractH2LeafIO.java
rename to modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/inline/io/AbstractLeafIO.java
index 60cbf8c..68a01af 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/io/AbstractH2LeafIO.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/inline/io/AbstractLeafIO.java
@@ -15,67 +15,67 @@
  * limitations under the License.
  */
 
-package org.apache.ignite.internal.processors.query.h2.database.io;
+package org.apache.ignite.internal.cache.query.index.sorted.inline.io;
 
 import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.internal.cache.query.index.sorted.IndexRow;
+import org.apache.ignite.internal.cache.query.index.sorted.inline.InlineIndexTree;
 import org.apache.ignite.internal.pagemem.PageUtils;
 import org.apache.ignite.internal.processors.cache.persistence.tree.BPlusTree;
 import org.apache.ignite.internal.processors.cache.persistence.tree.io.BPlusIO;
 import org.apache.ignite.internal.processors.cache.persistence.tree.io.BPlusLeafIO;
-import org.apache.ignite.internal.processors.query.h2.database.H2Tree;
-import org.apache.ignite.internal.processors.query.h2.opt.H2CacheRow;
-import org.apache.ignite.internal.processors.query.h2.opt.H2Row;
 
 /**
- * Leaf page for H2 row references.
+ * Leaf page to store index rows.
  */
-public abstract class AbstractH2LeafIO extends BPlusLeafIO<H2Row> implements H2RowLinkIO {
+public abstract class AbstractLeafIO extends BPlusLeafIO<IndexRow> implements InlineIO {
     /**
      * @param type Page type.
      * @param ver Page format version.
      * @param itemSize Single item size on page.
      */
-    AbstractH2LeafIO(int type, int ver, int itemSize) {
+    AbstractLeafIO(int type, int ver, int itemSize) {
         super(type, ver, itemSize);
     }
 
     /** {@inheritDoc} */
-    @Override public final void storeByOffset(long pageAddr, int off, H2Row row) {
-        H2CacheRow row0 = (H2CacheRow)row;
+    @Override public void storeByOffset(long pageAddr, int off, IndexRow row) {
+        assert row.link() != 0;
 
-        H2IOUtils.storeRow(row0, pageAddr, off, storeMvccInfo());
+        IORowHandler.store(pageAddr, off, row, storeMvccInfo());
     }
 
     /** {@inheritDoc} */
-    @Override public final void store(long dstPageAddr, int dstIdx, BPlusIO<H2Row> srcIo, long srcPageAddr, int srcIdx) {
-        assert srcIo == this;
+    @Override public IndexRow getLookupRow(BPlusTree<IndexRow, ?> tree, long pageAddr, int idx)
+        throws IgniteCheckedException {
 
-        H2IOUtils.store(dstPageAddr, offset(dstIdx), srcIo, srcPageAddr, srcIdx, storeMvccInfo());
-    }
+        long link = PageUtils.getLong(pageAddr, offset(idx));
 
-    /** {@inheritDoc} */
-    @Override public H2Row getLookupRow(BPlusTree<H2Row, ?> tree, long pageAddr, int idx)
-        throws IgniteCheckedException {
-        long link = getLink(pageAddr, idx);
+        assert link != 0;
 
         if (storeMvccInfo()) {
-            long mvccCrdVer = getMvccCoordinatorVersion(pageAddr, idx);
-            long mvccCntr = getMvccCounter(pageAddr, idx);
-            int mvccOpCntr = getMvccOperationCounter(pageAddr, idx);
+            long mvccCrdVer = mvccCoordinatorVersion(pageAddr, idx);
+            long mvccCntr = mvccCounter(pageAddr, idx);
+            int mvccOpCntr = mvccOperationCounter(pageAddr, idx);
 
-            return ((H2Tree)tree).createMvccRow(link, mvccCrdVer, mvccCntr, mvccOpCntr);
+            return ((InlineIndexTree) tree).createMvccIndexRow(link, mvccCrdVer, mvccCntr, mvccOpCntr);
         }
 
-        return ((H2Tree)tree).createRow(link);
+        return ((InlineIndexTree) tree).createIndexRow(link);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void store(long dstPageAddr, int dstIdx, BPlusIO<IndexRow> srcIo, long srcPageAddr, int srcIdx) {
+        IORowHandler.store(dstPageAddr, offset(dstIdx), (InlineIO) srcIo, srcPageAddr, srcIdx, storeMvccInfo());
     }
 
     /** {@inheritDoc} */
-    @Override public long getLink(long pageAddr, int idx) {
+    @Override public long link(long pageAddr, int idx) {
         return PageUtils.getLong(pageAddr, offset(idx));
     }
 
     /** {@inheritDoc} */
-    @Override public int getPayloadSize() {
+    @Override public int inlineSize() {
         return 0;
     }
 }
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/io/H2IOUtils.java b/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/inline/io/IORowHandler.java
similarity index 64%
rename from modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/io/H2IOUtils.java
rename to modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/inline/io/IORowHandler.java
index c9424ca..98ef338 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/io/H2IOUtils.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/inline/io/IORowHandler.java
@@ -15,35 +15,22 @@
  * limitations under the License.
  */
 
-package org.apache.ignite.internal.processors.query.h2.database.io;
+package org.apache.ignite.internal.cache.query.index.sorted.inline.io;
 
+import org.apache.ignite.internal.cache.query.index.sorted.IndexRow;
 import org.apache.ignite.internal.pagemem.PageUtils;
 import org.apache.ignite.internal.processors.cache.mvcc.MvccUtils;
-import org.apache.ignite.internal.processors.cache.persistence.tree.io.BPlusIO;
-import org.apache.ignite.internal.processors.query.h2.opt.H2CacheRow;
-import org.apache.ignite.internal.processors.query.h2.opt.H2Row;
 
 /**
- *
+ * Class provide a common logic for storing an index row.
  */
-class H2IOUtils {
-    /**
-     *
-     */
-    private H2IOUtils() {}
-
-    /**
-     * @param row Row.
-     * @param pageAddr Page address.
-     * @param off Offset.
-     * @param storeMvcc {@code True} to store mvcc data.
-     */
-    static void storeRow(H2CacheRow row, long pageAddr, int off, boolean storeMvcc) {
-        assert row.link() != 0;
-
+class IORowHandler {
+    /** */
+    static void store(long pageAddr, int off, IndexRow row, boolean storeMvccInfo) {
+        // Write link after all inlined idx keys.
         PageUtils.putLong(pageAddr, off, row.link());
 
-        if (storeMvcc) {
+        if (storeMvccInfo) {
             long mvccCrdVer = row.mvccCoordinatorVersion();
             long mvccCntr = row.mvccCounter();
             int mvccOpCntr = row.mvccOperationCounter();
@@ -64,23 +51,15 @@ class H2IOUtils {
      * @param srcIdx Source index.
      * @param storeMvcc {@code True} to store mvcc data.
      */
-    static void store(long dstPageAddr,
-        int dstOff,
-        BPlusIO<H2Row> srcIo,
-        long srcPageAddr,
-        int srcIdx,
-        boolean storeMvcc)
-    {
-        H2RowLinkIO rowIo = (H2RowLinkIO)srcIo;
-
-        long link = rowIo.getLink(srcPageAddr, srcIdx);
+    static void store(long dstPageAddr, int dstOff, InlineIO srcIo, long srcPageAddr, int srcIdx, boolean storeMvcc) {
+        long link = srcIo.link(srcPageAddr, srcIdx);
 
         PageUtils.putLong(dstPageAddr, dstOff, link);
 
         if (storeMvcc) {
-            long mvccCrdVer = rowIo.getMvccCoordinatorVersion(srcPageAddr, srcIdx);
-            long mvccCntr = rowIo.getMvccCounter(srcPageAddr, srcIdx);
-            int mvccOpCntr = rowIo.getMvccOperationCounter(srcPageAddr, srcIdx);
+            long mvccCrdVer = srcIo.mvccCoordinatorVersion(srcPageAddr, srcIdx);
+            long mvccCntr = srcIo.mvccCounter(srcPageAddr, srcIdx);
+            int mvccOpCntr = srcIo.mvccOperationCounter(srcPageAddr, srcIdx);
 
             assert MvccUtils.mvccVersionIsValid(mvccCrdVer, mvccCntr, mvccOpCntr);
 
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryRowCacheCleaner.java b/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/inline/io/InlineIO.java
similarity index 67%
copy from modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryRowCacheCleaner.java
copy to modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/inline/io/InlineIO.java
index 6275813..3496927 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryRowCacheCleaner.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/inline/io/InlineIO.java
@@ -15,16 +15,21 @@
  * limitations under the License.
  */
 
-package org.apache.ignite.internal.processors.query;
+package org.apache.ignite.internal.cache.query.index.sorted.inline.io;
 
 /**
- * Row cache cleaner is used by page memory manager to remove updated / evicted links from rows cache.
+ * Represents common API for inline IOs.
  */
-public interface GridQueryRowCacheCleaner {
+public interface InlineIO extends MvccIO {
     /**
-     * Remove row by link.
-     *
-     * @param link Link to remove.
+     * @param pageAddr Page address.
+     * @param idx Index.
+     * @return Row link.
      */
-    void remove(long link);
+    public long link(long pageAddr, int idx);
+
+    /**
+     * @return Number of bytes stored in the inline payload.
+     */
+    public int inlineSize();
 }
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/io/H2ExtrasLeafIO.java b/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/inline/io/InlineInnerIO.java
similarity index 68%
rename from modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/io/H2ExtrasLeafIO.java
rename to modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/inline/io/InlineInnerIO.java
index 3c79df7..703850c 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/io/H2ExtrasLeafIO.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/inline/io/InlineInnerIO.java
@@ -15,18 +15,17 @@
  * limitations under the License.
  */
 
-package org.apache.ignite.internal.processors.query.h2.database.io;
+package org.apache.ignite.internal.cache.query.index.sorted.inline.io;
 
 /**
- * Leaf page for H2 row references.
+ * Implementation of {@link AbstractInlineInnerIO}
  */
-public class H2ExtrasLeafIO extends AbstractH2ExtrasLeafIO {
+public final class InlineInnerIO extends AbstractInlineInnerIO {
     /**
-     * @param type Page type.
-     * @param ver Page format version.
-     * @param payloadSize Payload size.
+     * Constructor.
      */
-    public H2ExtrasLeafIO(short type, int ver, int payloadSize) {
-        super(type, ver, 8, payloadSize);
+    public InlineInnerIO(short type, int inlineSize) {
+        // Meta stores link to a cache row.
+        super(type, 1, 8, inlineSize);
     }
 }
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/io/H2ExtrasInnerIO.java b/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/inline/io/InlineLeafIO.java
similarity index 67%
rename from modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/io/H2ExtrasInnerIO.java
rename to modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/inline/io/InlineLeafIO.java
index 8dc8c96..dec5b6d 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/io/H2ExtrasInnerIO.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/inline/io/InlineLeafIO.java
@@ -15,19 +15,17 @@
  * limitations under the License.
  */
 
-package org.apache.ignite.internal.processors.query.h2.database.io;
+package org.apache.ignite.internal.cache.query.index.sorted.inline.io;
 
 /**
- * Inner page for H2 row references.
+ * Represents common API for inline IOs.
  */
-public class H2ExtrasInnerIO extends AbstractH2ExtrasInnerIO implements H2RowLinkIO {
+public final class InlineLeafIO extends AbstractInlineLeafIO {
     /**
-     * @param type Page type.
-     * @param ver Page format version.
-     * @param payloadSize Payload size.
+     * Constructor.
      */
-    H2ExtrasInnerIO(short type, int ver, int payloadSize) {
-        super(type, ver, 8, payloadSize);
+    public InlineLeafIO(short type, int inlineSize) {
+        // Meta stores link to a cache row.
+        super(type, 1, 8, inlineSize);
     }
 }
-
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/io/H2InnerIO.java b/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/inline/io/InnerIO.java
similarity index 66%
copy from modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/io/H2InnerIO.java
copy to modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/inline/io/InnerIO.java
index 9baff7a..20cd1e0c4 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/io/H2InnerIO.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/inline/io/InnerIO.java
@@ -15,23 +15,25 @@
  * limitations under the License.
  */
 
-package org.apache.ignite.internal.processors.query.h2.database.io;
+package org.apache.ignite.internal.cache.query.index.sorted.inline.io;
 
+import org.apache.ignite.internal.cache.query.index.sorted.IndexRow;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.BPlusInnerIO;
 import org.apache.ignite.internal.processors.cache.persistence.tree.io.IOVersions;
 
 /**
- * Inner page for H2 row references.
+ * Inner page for index rows without inline payload.
  */
-public class H2InnerIO extends AbstractH2InnerIO {
-    /** */
-    public static final IOVersions<H2InnerIO> VERSIONS = new IOVersions<>(
-        new H2InnerIO(1)
+public class InnerIO extends AbstractInnerIO {
+    /** Supported versions. */
+    public static final IOVersions<? extends BPlusInnerIO<IndexRow>> VERSIONS = new IOVersions<>(
+        new InnerIO(1)
     );
 
     /**
      * @param ver Page format version.
      */
-    private H2InnerIO(int ver) {
+    private InnerIO(int ver) {
         super(T_H2_REF_INNER, ver, 8);
     }
 }
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/io/H2LeafIO.java b/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/inline/io/LeafIO.java
similarity index 75%
copy from modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/io/H2LeafIO.java
copy to modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/inline/io/LeafIO.java
index 466cd1c..7735513 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/io/H2LeafIO.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/inline/io/LeafIO.java
@@ -15,23 +15,23 @@
  * limitations under the License.
  */
 
-package org.apache.ignite.internal.processors.query.h2.database.io;
+package org.apache.ignite.internal.cache.query.index.sorted.inline.io;
 
 import org.apache.ignite.internal.processors.cache.persistence.tree.io.IOVersions;
 
 /**
- * Leaf page for H2 row references.
+ * Leaf page for index rows without inline payload.
  */
-public class H2LeafIO extends AbstractH2LeafIO {
-    /** */
-    public static final IOVersions<H2LeafIO> VERSIONS = new IOVersions<>(
-        new H2LeafIO(1)
+public class LeafIO extends AbstractLeafIO {
+    /** Supported versions. */
+    public static final IOVersions<LeafIO> VERSIONS = new IOVersions<>(
+        new LeafIO(1)
     );
 
     /**
      * @param ver Page format version.
      */
-    public H2LeafIO(int ver) {
+    private LeafIO(int ver) {
         super(T_H2_REF_LEAF, ver, 8);
     }
 }
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/io/H2RowLinkIO.java b/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/inline/io/MvccIO.java
similarity index 71%
rename from modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/io/H2RowLinkIO.java
rename to modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/inline/io/MvccIO.java
index 55a36d6..0679233 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/io/H2RowLinkIO.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/inline/io/MvccIO.java
@@ -15,25 +15,17 @@
  * limitations under the License.
  */
 
-package org.apache.ignite.internal.processors.query.h2.database.io;
 
-/**
- * Row link IO.
- */
-public interface H2RowLinkIO {
-    /**
-     * @param pageAddr Page address.
-     * @param idx Index.
-     * @return Row link.
-     */
-    public long getLink(long pageAddr, int idx);
+package org.apache.ignite.internal.cache.query.index.sorted.inline.io;
 
+/** This interface represents MVCC aware IO. */
+public interface MvccIO {
     /**
      * @param pageAddr Page address.
      * @param idx Index.
      * @return Mvcc coordinator version.
      */
-    public default long getMvccCoordinatorVersion(long pageAddr, int idx) {
+    public default long mvccCoordinatorVersion(long pageAddr, int idx) {
         throw new UnsupportedOperationException();
     }
 
@@ -42,7 +34,7 @@ public interface H2RowLinkIO {
      * @param idx Index.
      * @return Mvcc counter.
      */
-    public default long getMvccCounter(long pageAddr, int idx) {
+    public default long mvccCounter(long pageAddr, int idx) {
         throw new UnsupportedOperationException();
     }
 
@@ -51,7 +43,7 @@ public interface H2RowLinkIO {
      * @param idx Index.
      * @return Mvcc operation counter.
      */
-    public default int getMvccOperationCounter(long pageAddr, int idx) {
+    public default int mvccOperationCounter(long pageAddr, int idx) {
         throw new UnsupportedOperationException();
     }
 
@@ -61,9 +53,4 @@ public interface H2RowLinkIO {
     public default boolean storeMvccInfo() {
         return false;
     }
-
-    /**
-     * @return Size of reserved data array for data inlining.
-     */
-    public int getPayloadSize();
 }
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/io/H2MvccExtrasLeafIO.java b/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/inline/io/MvccInlineInnerIO.java
similarity index 60%
rename from modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/io/H2MvccExtrasLeafIO.java
rename to modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/inline/io/MvccInlineInnerIO.java
index 7585471..535a5f4 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/io/H2MvccExtrasLeafIO.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/inline/io/MvccInlineInnerIO.java
@@ -15,36 +15,36 @@
  * limitations under the License.
  */
 
-package org.apache.ignite.internal.processors.query.h2.database.io;
+package org.apache.ignite.internal.cache.query.index.sorted.inline.io;
 
 import org.apache.ignite.internal.pagemem.PageUtils;
 
 /**
- * Leaf page for H2 row references.
+ * Inner page for inlined MVCC index rows.
  */
-public class H2MvccExtrasLeafIO extends AbstractH2ExtrasLeafIO {
+public class MvccInlineInnerIO extends AbstractInlineInnerIO {
     /**
      * @param type Page type.
-     * @param ver Page format version.
      * @param payloadSize Payload size.
      */
-    protected H2MvccExtrasLeafIO(short type, int ver, int payloadSize) {
-        super(type, ver, 28, payloadSize);
+    protected MvccInlineInnerIO(short type, int payloadSize) {
+        // Meta stores link and mvcc info about a cache row.
+        super(type, 1, 28, payloadSize);
     }
 
     /** {@inheritDoc} */
-    @Override public long getMvccCoordinatorVersion(long pageAddr, int idx) {
-        return PageUtils.getLong(pageAddr, offset(idx) + payloadSize + 8);
+    @Override public long mvccCoordinatorVersion(long pageAddr, int idx) {
+        return PageUtils.getLong(pageAddr, offset(idx) + inlineSize() + 8);
     }
 
     /** {@inheritDoc} */
-    @Override public long getMvccCounter(long pageAddr, int idx) {
-        return PageUtils.getLong(pageAddr, offset(idx) + payloadSize + 16);
+    @Override public long mvccCounter(long pageAddr, int idx) {
+        return PageUtils.getLong(pageAddr, offset(idx) + inlineSize() + 16);
     }
 
     /** {@inheritDoc} */
-    @Override public int getMvccOperationCounter(long pageAddr, int idx) {
-        return PageUtils.getInt(pageAddr, offset(idx) + payloadSize + 24);
+    @Override public int mvccOperationCounter(long pageAddr, int idx) {
+        return PageUtils.getInt(pageAddr, offset(idx) + inlineSize() + 24);
     }
 
     /** {@inheritDoc} */
@@ -52,3 +52,4 @@ public class H2MvccExtrasLeafIO extends AbstractH2ExtrasLeafIO {
         return true;
     }
 }
+
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/io/H2MvccExtrasInnerIO.java b/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/inline/io/MvccInlineLeafIO.java
similarity index 60%
rename from modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/io/H2MvccExtrasInnerIO.java
rename to modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/inline/io/MvccInlineLeafIO.java
index ee6dc2a..048ec80 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/io/H2MvccExtrasInnerIO.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/inline/io/MvccInlineLeafIO.java
@@ -15,36 +15,36 @@
  * limitations under the License.
  */
 
-package org.apache.ignite.internal.processors.query.h2.database.io;
+package org.apache.ignite.internal.cache.query.index.sorted.inline.io;
 
 import org.apache.ignite.internal.pagemem.PageUtils;
 
 /**
- *
+ * Leaf page for inlined MVCC index rows.
  */
-class H2MvccExtrasInnerIO extends AbstractH2ExtrasInnerIO {
+public class MvccInlineLeafIO extends AbstractInlineLeafIO {
     /**
      * @param type Page type.
-     * @param ver Page format version.
      * @param payloadSize Payload size.
      */
-    H2MvccExtrasInnerIO(short type, int ver, int payloadSize) {
-        super(type, ver, 28, payloadSize);
+    protected MvccInlineLeafIO(short type, int payloadSize) {
+        // Meta stores link and mvcc info about a cache row.
+        super(type, 1, 28, payloadSize);
     }
 
     /** {@inheritDoc} */
-    @Override public long getMvccCoordinatorVersion(long pageAddr, int idx) {
-        return PageUtils.getLong(pageAddr, offset(idx) + payloadSize + 8);
+    @Override public long mvccCoordinatorVersion(long pageAddr, int idx) {
+        return PageUtils.getLong(pageAddr, offset(idx) + inlineSize() + 8);
     }
 
     /** {@inheritDoc} */
-    @Override public long getMvccCounter(long pageAddr, int idx) {
-        return PageUtils.getLong(pageAddr, offset(idx) + payloadSize + 16);
+    @Override public long mvccCounter(long pageAddr, int idx) {
+        return PageUtils.getLong(pageAddr, offset(idx) + inlineSize() + 16);
     }
 
     /** {@inheritDoc} */
-    @Override public int getMvccOperationCounter(long pageAddr, int idx) {
-        return PageUtils.getInt(pageAddr, offset(idx) + payloadSize + 24);
+    @Override public int mvccOperationCounter(long pageAddr, int idx) {
+        return PageUtils.getInt(pageAddr, offset(idx) + inlineSize() + 24);
     }
 
     /** {@inheritDoc} */
@@ -52,4 +52,3 @@ class H2MvccExtrasInnerIO extends AbstractH2ExtrasInnerIO {
         return true;
     }
 }
-
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/io/H2MvccInnerIO.java b/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/inline/io/MvccInnerIO.java
similarity index 74%
rename from modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/io/H2MvccInnerIO.java
rename to modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/inline/io/MvccInnerIO.java
index dbfe784..8eedd11 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/io/H2MvccInnerIO.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/inline/io/MvccInnerIO.java
@@ -15,39 +15,39 @@
  * limitations under the License.
  */
 
-package org.apache.ignite.internal.processors.query.h2.database.io;
+package org.apache.ignite.internal.cache.query.index.sorted.inline.io;
 
 import org.apache.ignite.internal.pagemem.PageUtils;
 import org.apache.ignite.internal.processors.cache.persistence.tree.io.IOVersions;
 
 /**
- * Inner page for H2 row references.
+ * Inner page for MVCC index rows.
  */
-public class H2MvccInnerIO extends AbstractH2InnerIO {
+public class MvccInnerIO extends AbstractInnerIO {
     /** */
-    public static final IOVersions<H2MvccInnerIO> VERSIONS = new IOVersions<>(
-        new H2MvccInnerIO(1)
+    public static final IOVersions<MvccInnerIO> VERSIONS = new IOVersions<>(
+        new MvccInnerIO(1)
     );
 
     /**
      * @param ver Page format version.
      */
-    private H2MvccInnerIO(int ver) {
+    private MvccInnerIO(int ver) {
         super(T_H2_MVCC_REF_INNER, ver, 28);
     }
 
     /** {@inheritDoc} */
-    @Override public long getMvccCoordinatorVersion(long pageAddr, int idx) {
+    @Override public long mvccCoordinatorVersion(long pageAddr, int idx) {
         return PageUtils.getLong(pageAddr, offset(idx) + 8);
     }
 
     /** {@inheritDoc} */
-    @Override public long getMvccCounter(long pageAddr, int idx) {
+    @Override public long mvccCounter(long pageAddr, int idx) {
         return PageUtils.getLong(pageAddr, offset(idx) + 16);
     }
 
     /** {@inheritDoc} */
-    @Override public int getMvccOperationCounter(long pageAddr, int idx) {
+    @Override public int mvccOperationCounter(long pageAddr, int idx) {
         return PageUtils.getInt(pageAddr, offset(idx) + 24);
     }
 
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/io/H2MvccLeafIO.java b/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/inline/io/MvccLeafIO.java
similarity index 74%
rename from modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/io/H2MvccLeafIO.java
rename to modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/inline/io/MvccLeafIO.java
index 5575806..39d5c97 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/io/H2MvccLeafIO.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/inline/io/MvccLeafIO.java
@@ -15,39 +15,39 @@
  * limitations under the License.
  */
 
-package org.apache.ignite.internal.processors.query.h2.database.io;
+package org.apache.ignite.internal.cache.query.index.sorted.inline.io;
 
 import org.apache.ignite.internal.pagemem.PageUtils;
 import org.apache.ignite.internal.processors.cache.persistence.tree.io.IOVersions;
 
 /**
- *
+ * Leaf page for inlined MVCC index rows.
  */
-public class H2MvccLeafIO extends AbstractH2LeafIO {
+public class MvccLeafIO extends AbstractLeafIO {
     /** */
-    public static final IOVersions<H2MvccLeafIO> VERSIONS = new IOVersions<>(
-        new H2MvccLeafIO(1)
+    public static final IOVersions<MvccLeafIO> VERSIONS = new IOVersions<>(
+        new MvccLeafIO(1)
     );
 
     /**
      * @param ver Page format version.
      */
-    protected H2MvccLeafIO(int ver) {
+    protected MvccLeafIO(int ver) {
         super(T_H2_MVCC_REF_LEAF, ver, 28);
     }
 
     /** {@inheritDoc} */
-    @Override public long getMvccCoordinatorVersion(long pageAddr, int idx) {
+    @Override public long mvccCoordinatorVersion(long pageAddr, int idx) {
         return PageUtils.getLong(pageAddr, offset(idx) + 8);
     }
 
     /** {@inheritDoc} */
-    @Override public long getMvccCounter(long pageAddr, int idx) {
+    @Override public long mvccCounter(long pageAddr, int idx) {
         return PageUtils.getLong(pageAddr, offset(idx) + 16);
     }
 
     /** {@inheritDoc} */
-    @Override public int getMvccOperationCounter(long pageAddr, int idx) {
+    @Override public int mvccOperationCounter(long pageAddr, int idx) {
         return PageUtils.getInt(pageAddr, offset(idx) + 24);
     }
 
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/inline/types/BooleanInlineIndexKeyType.java b/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/inline/types/BooleanInlineIndexKeyType.java
new file mode 100644
index 0000000..4386555
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/inline/types/BooleanInlineIndexKeyType.java
@@ -0,0 +1,59 @@
+/*
+ * 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.cache.query.index.sorted.inline.types;
+
+import org.apache.ignite.internal.cache.query.index.sorted.IndexKeyTypes;
+import org.apache.ignite.internal.cache.query.index.sorted.keys.BooleanIndexKey;
+import org.apache.ignite.internal.pagemem.PageUtils;
+
+/**
+ * Inline index key implementation for inlining {@link Boolean} values.
+ */
+public class BooleanInlineIndexKeyType extends NullableInlineIndexKeyType<BooleanIndexKey> {
+    /** */
+    public BooleanInlineIndexKeyType() {
+        super(IndexKeyTypes.BOOLEAN, (short) 1);
+    }
+
+    /** {@inheritDoc} */
+    @Override protected int put0(long pageAddr, int off, BooleanIndexKey key, int maxSize) {
+        PageUtils.putByte(pageAddr, off, (byte) type());
+        PageUtils.putByte(pageAddr, off + 1, (byte)((boolean) key.key() ? 1 : 0));
+
+        return keySize + 1;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected BooleanIndexKey get0(long pageAddr, int off) {
+        boolean key = PageUtils.getByte(pageAddr, off + 1) != 0;
+
+        return new BooleanIndexKey(key);
+    }
+
+    /** {@inheritDoc} */
+    @Override public int compare0(long pageAddr, int off, BooleanIndexKey key) {
+        boolean bool1 = PageUtils.getByte(pageAddr, off + 1) != 0;
+
+        return Integer.signum(Boolean.compare(bool1, (boolean) key.key()));
+    }
+
+    /** {@inheritDoc} */
+    @Override protected int inlineSize0(BooleanIndexKey key) {
+        return keySize + 1;
+    }
+}
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/inline/types/ByteInlineIndexKeyType.java b/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/inline/types/ByteInlineIndexKeyType.java
new file mode 100644
index 0000000..0cefb4f
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/inline/types/ByteInlineIndexKeyType.java
@@ -0,0 +1,59 @@
+/*
+ * 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.cache.query.index.sorted.inline.types;
+
+import org.apache.ignite.internal.cache.query.index.sorted.IndexKeyTypes;
+import org.apache.ignite.internal.cache.query.index.sorted.keys.ByteIndexKey;
+import org.apache.ignite.internal.pagemem.PageUtils;
+
+/**
+ * Inline index key implementation for inlining {@link Byte} values.
+ */
+public class ByteInlineIndexKeyType extends NullableInlineIndexKeyType<ByteIndexKey> {
+    /** */
+    public ByteInlineIndexKeyType() {
+        super(IndexKeyTypes.BYTE, (short) 1);
+    }
+
+    /** {@inheritDoc} */
+    @Override public int compare0(long pageAddr, int off, ByteIndexKey key) {
+        byte byte1 = PageUtils.getByte(pageAddr, off + 1);
+
+        return Integer.signum(byte1 - (byte) key.key());
+    }
+
+    /** {@inheritDoc} */
+    @Override protected int put0(long pageAddr, int off, ByteIndexKey key, int maxSize) {
+        PageUtils.putByte(pageAddr, off, (byte) type());
+        PageUtils.putByte(pageAddr, off + 1, (byte) key.key());
+
+        return keySize + 1;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected ByteIndexKey get0(long pageAddr, int off) {
+        byte b = PageUtils.getByte(pageAddr, off + 1);
+
+        return new ByteIndexKey(b);
+    }
+
+    /** {@inheritDoc} */
+    @Override protected int inlineSize0(ByteIndexKey key) {
+        return keySize + 1;
+    }
+}
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/inlinecolumn/BytesInlineIndexColumn.java b/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/inline/types/BytesInlineIndexKeyType.java
similarity index 60%
rename from modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/inlinecolumn/BytesInlineIndexColumn.java
rename to modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/inline/types/BytesInlineIndexKeyType.java
index 8c9df04..329208a 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/inlinecolumn/BytesInlineIndexColumn.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/inline/types/BytesInlineIndexKeyType.java
@@ -15,61 +15,62 @@
  * limitations under the License.
  */
 
-package org.apache.ignite.internal.processors.query.h2.database.inlinecolumn;
+package org.apache.ignite.internal.cache.query.index.sorted.inline.types;
 
 import java.util.Arrays;
+import org.apache.ignite.internal.cache.query.index.sorted.IndexKeyTypes;
+import org.apache.ignite.internal.cache.query.index.sorted.keys.BytesIndexKey;
+import org.apache.ignite.internal.cache.query.index.sorted.keys.SignedBytesIndexKey;
 import org.apache.ignite.internal.pagemem.PageUtils;
 import org.apache.ignite.internal.util.GridUnsafe;
-import org.h2.table.Column;
-import org.h2.value.Value;
-import org.h2.value.ValueBytes;
 
 /**
- * Inline index column implementation for inlining byte arrays.
+ * Inline index key implementation for inlining byte arrays.
  */
-public class BytesInlineIndexColumn extends AbstractInlineIndexColumn {
+public class BytesInlineIndexKeyType extends NullableInlineIndexKeyType<BytesIndexKey> {
     /** Compare binary unsigned. */
     private final boolean compareBinaryUnsigned;
 
-    /**
-     * @param col Column.
-     */
-    public BytesInlineIndexColumn(Column col, boolean compareBinaryUnsigned) {
-        this(col, Value.BYTES, compareBinaryUnsigned);
+    /** */
+    public BytesInlineIndexKeyType() {
+        this(IndexKeyTypes.BYTES);
     }
 
-    /**
-     * @param col Column.
-     * @param type Type.
-     * @param compareBinaryUnsigned Compare binary unsigned.
-     */
-    BytesInlineIndexColumn(Column col, int type, boolean compareBinaryUnsigned) {
-        super(col, type, (short)-1);
+    /** */
+    public BytesInlineIndexKeyType(int type) {
+        this(type, true);
+    }
 
-        this.compareBinaryUnsigned = compareBinaryUnsigned;
+    /** */
+    public BytesInlineIndexKeyType(boolean compareBinaryUnsigned) {
+        this(IndexKeyTypes.BYTES, compareBinaryUnsigned);
     }
 
-    /** {@inheritDoc} */
-    @Override protected int compare0(long pageAddr, int off, Value v, int type) {
-        if (type() != type)
-            return COMPARE_UNSUPPORTED;
+    /** */
+    public BytesInlineIndexKeyType(int type, boolean compareBinaryUnsigned) {
+        super(type, (short) -1);
 
-        byte[] bytes = v.getBytesNoCopy();
+        this.compareBinaryUnsigned = compareBinaryUnsigned;
+    }
 
+    /** {@inheritDoc} */
+    @Override public int compare0(long pageAddr, int off, BytesIndexKey bytes) {
         long addr = pageAddr + off + 1; // Skip type.
 
         int len1 = PageUtils.getShort(pageAddr, off + 1) & 0x7FFF;
 
         addr += 2; // Skip size.
 
-        int len2 = bytes.length;
+        byte[] arr = (byte[]) bytes.key();
+
+        int len2 = arr.length;
 
         int len = Math.min(len1, len2);
 
         if (compareBinaryUnsigned) {
             for (int i = 0; i < len; i++) {
                 int b1 = GridUnsafe.getByte(addr + i) & 0xff;
-                int b2 = bytes[i] & 0xff;
+                int b2 = arr[i] & 0xff;
 
                 if (b1 != b2)
                     return Integer.signum(b1 - b2);
@@ -78,7 +79,7 @@ public class BytesInlineIndexColumn extends AbstractInlineIndexColumn {
         else {
             for (int i = 0; i < len; i++) {
                 byte b1 = GridUnsafe.getByte(addr + i);
-                byte b2 = bytes[i];
+                byte b2 = arr[i];
 
                 if (b1 != b2)
                     return Integer.signum(b1 - b2);
@@ -100,41 +101,46 @@ public class BytesInlineIndexColumn extends AbstractInlineIndexColumn {
     }
 
     /** {@inheritDoc} */
-    @Override protected int put0(long pageAddr, int off, Value val, int maxSize) {
-        assert type() == val.getType();
-
+    @Override protected int put0(long pageAddr, int off, BytesIndexKey key, int maxSize) {
         short size;
 
-        PageUtils.putByte(pageAddr, off, (byte)val.getType());
+        PageUtils.putByte(pageAddr, off, (byte) type());
 
-        byte[] bytes = val.getBytes();
+        byte[] val = (byte[]) key.key();
 
-        if (bytes.length + 3 <= maxSize) {
-            size = (short)bytes.length;
+        if (val.length + 3 <= maxSize) {
+            size = (short)val.length;
             PageUtils.putShort(pageAddr, off + 1, size);
-            PageUtils.putBytes(pageAddr, off + 3, bytes);
+            PageUtils.putBytes(pageAddr, off + 3, val);
 
             return size + 3;
         }
         else {
             size = (short)((maxSize - 3) | 0x8000);
             PageUtils.putShort(pageAddr, off + 1, size);
-            PageUtils.putBytes(pageAddr, off + 3, Arrays.copyOfRange(bytes, 0, maxSize - 3));
+            PageUtils.putBytes(pageAddr, off + 3, Arrays.copyOfRange(val, 0, maxSize - 3));
 
             return maxSize;
         }
     }
 
     /** {@inheritDoc} */
-    @Override protected Value get0(long pageAddr, int off) {
-        return ValueBytes.get(readBytes(pageAddr, off));
+    @Override protected BytesIndexKey get0(long pageAddr, int off) {
+        byte[] arr = readBytes(pageAddr, off);
+
+        return compareBinaryUnsigned ? new BytesIndexKey(arr) : new SignedBytesIndexKey(arr);
     }
 
     /** {@inheritDoc} */
-    @Override protected int inlineSizeOf0(Value val) {
-        assert val.getType() == type();
+    @Override protected int inlineSize0(BytesIndexKey val) {
+        byte[] arr = (byte[]) val.key();
+
+        return arr.length + 3;
+    }
 
-        return val.getBytes().length + 3;
+    /** */
+    public boolean compareBinaryUnsigned() {
+        return compareBinaryUnsigned;
     }
 
     /**
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/inline/types/DateInlineIndexKeyType.java b/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/inline/types/DateInlineIndexKeyType.java
new file mode 100644
index 0000000..73d9335
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/inline/types/DateInlineIndexKeyType.java
@@ -0,0 +1,69 @@
+/*
+ * 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.cache.query.index.sorted.inline.types;
+
+import org.apache.ignite.internal.cache.query.index.sorted.IndexKeyTypes;
+import org.apache.ignite.internal.cache.query.index.sorted.keys.AbstractDateIndexKey;
+import org.apache.ignite.internal.cache.query.index.sorted.keys.IndexKeyFactory;
+import org.apache.ignite.internal.pagemem.PageUtils;
+
+import static org.apache.ignite.internal.cache.query.index.sorted.inline.types.DateValueConstants.MAX_DATE_VALUE;
+import static org.apache.ignite.internal.cache.query.index.sorted.inline.types.DateValueConstants.MIN_DATE_VALUE;
+
+/**
+ * Inline index key implementation for inlining {@link AbstractDateIndexKey} values.
+ */
+public class DateInlineIndexKeyType extends NullableInlineIndexKeyType<AbstractDateIndexKey> {
+    /** */
+    public DateInlineIndexKeyType() {
+        super(IndexKeyTypes.DATE, (short) 8);
+    }
+
+    /** {@inheritDoc} */
+    @Override public int compare0(long pageAddr, int off, AbstractDateIndexKey key) {
+        long val1 = PageUtils.getLong(pageAddr, off + 1);
+        long val2 = key.dateValue();
+
+        return Integer.signum(Long.compare(val1, val2));
+    }
+
+    /** {@inheritDoc} */
+    @Override protected int put0(long pageAddr, int off, AbstractDateIndexKey key, int maxSize) {
+        PageUtils.putByte(pageAddr, off, (byte) type());
+        PageUtils.putLong(pageAddr, off + 1, key.dateValue());
+
+        return keySize + 1;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected AbstractDateIndexKey get0(long pageAddr, int off) {
+        long dateVal = PageUtils.getLong(pageAddr, off + 1);
+
+        if (dateVal > MAX_DATE_VALUE)
+            dateVal = MAX_DATE_VALUE;
+        else if (dateVal < MIN_DATE_VALUE)
+            dateVal = MIN_DATE_VALUE;
+
+        return (AbstractDateIndexKey) IndexKeyFactory.wrapDateValue(type(), dateVal, 0L);
+    }
+
+    /** {@inheritDoc} */
+    @Override protected int inlineSize0(AbstractDateIndexKey key) {
+        return keySize + 1;
+    }
+}
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/inline/types/DateValueConstants.java b/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/inline/types/DateValueConstants.java
new file mode 100644
index 0000000..227e031
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/inline/types/DateValueConstants.java
@@ -0,0 +1,46 @@
+/*
+ * 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.cache.query.index.sorted.inline.types;
+
+/**
+ * DateValue is a representation of a date in bit form:
+ *
+ * dv = (year << SHIFT_YEAR) | (month << SHIFT_MONTH) | day.
+ */
+public class DateValueConstants {
+    /** Forbid instantiation of this class. Just hold constants there. */
+    private DateValueConstants() {}
+
+    /** */
+    private static final int SHIFT_YEAR = 9;
+
+    /** */
+    private static final int SHIFT_MONTH = 5;
+
+    /** Min date value. */
+    public static final long MIN_DATE_VALUE = (-999_999_999L << SHIFT_YEAR) + (1 << SHIFT_MONTH) + 1;
+
+    /** Max date value. */
+    public static final long MAX_DATE_VALUE = (999_999_999L << SHIFT_YEAR) + (12 << SHIFT_MONTH) + 31;
+
+    /** The number of milliseconds per day. */
+    public static final long MILLIS_PER_DAY = 24 * 60 * 60 * 1000L;
+
+    /** The number of nanoseconds per day. */
+    public static final long NANOS_PER_DAY = MILLIS_PER_DAY * 1_000_000;
+}
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/inline/types/DoubleInlineIndexKeyType.java b/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/inline/types/DoubleInlineIndexKeyType.java
new file mode 100644
index 0000000..03842b2
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/inline/types/DoubleInlineIndexKeyType.java
@@ -0,0 +1,59 @@
+/*
+ * 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.cache.query.index.sorted.inline.types;
+
+import org.apache.ignite.internal.cache.query.index.sorted.IndexKeyTypes;
+import org.apache.ignite.internal.cache.query.index.sorted.keys.DoubleIndexKey;
+import org.apache.ignite.internal.pagemem.PageUtils;
+
+/**
+ * Inline index key implementation for inlining {@link Double} values.
+ */
+public class DoubleInlineIndexKeyType extends NullableInlineIndexKeyType<DoubleIndexKey> {
+    /** */
+    public DoubleInlineIndexKeyType() {
+        super(IndexKeyTypes.DOUBLE, (short) 8);
+    }
+
+    /** {@inheritDoc} */
+    @Override public int compare0(long pageAddr, int off, DoubleIndexKey v) {
+        double val1 = Double.longBitsToDouble(PageUtils.getLong(pageAddr, off + 1));
+
+        return Integer.signum(Double.compare(val1, (double) v.key()));
+    }
+
+    /** {@inheritDoc} */
+    @Override protected int put0(long pageAddr, int off, DoubleIndexKey key, int maxSize) {
+        PageUtils.putByte(pageAddr, off, (byte) type());
+        PageUtils.putLong(pageAddr, off + 1, Double.doubleToLongBits((double) key.key()));
+
+        return keySize + 1;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected DoubleIndexKey get0(long pageAddr, int off) {
+        double key = Double.longBitsToDouble(PageUtils.getLong(pageAddr, off + 1));
+
+        return new DoubleIndexKey(key);
+    }
+
+    /** {@inheritDoc} */
+    @Override protected int inlineSize0(DoubleIndexKey val) {
+        return keySize + 1;
+    }
+}
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/inline/types/FloatInlineIndexKeyType.java b/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/inline/types/FloatInlineIndexKeyType.java
new file mode 100644
index 0000000..3f64389
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/inline/types/FloatInlineIndexKeyType.java
@@ -0,0 +1,59 @@
+/*
+ * 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.cache.query.index.sorted.inline.types;
+
+import org.apache.ignite.internal.cache.query.index.sorted.IndexKeyTypes;
+import org.apache.ignite.internal.cache.query.index.sorted.keys.FloatIndexKey;
+import org.apache.ignite.internal.pagemem.PageUtils;
+
+/**
+ * Inline index key implementation for inlining {@link Float} values.
+ */
+public class FloatInlineIndexKeyType extends NullableInlineIndexKeyType<FloatIndexKey> {
+    /** */
+    public FloatInlineIndexKeyType() {
+        super(IndexKeyTypes.FLOAT, (short) 4);
+    }
+
+    /** {@inheritDoc} */
+    @Override public int compare0(long pageAddr, int off, FloatIndexKey key) {
+        float val1 = Float.intBitsToFloat(PageUtils.getInt(pageAddr, off + 1));
+
+        return Integer.signum(Float.compare(val1, (float) key.key()));
+    }
+
+    /** {@inheritDoc} */
+    @Override protected int put0(long pageAddr, int off, FloatIndexKey key, int maxSize) {
+        PageUtils.putByte(pageAddr, off, (byte) type());
+        PageUtils.putInt(pageAddr, off + 1, Float.floatToIntBits((float) key.key()));
+
+        return keySize + 1;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected FloatIndexKey get0(long pageAddr, int off) {
+        float key = Float.intBitsToFloat(PageUtils.getInt(pageAddr, off + 1));
+
+        return new FloatIndexKey(key);
+    }
+
+    /** {@inheritDoc} */
+    @Override protected int inlineSize0(FloatIndexKey val) {
+        return keySize + 1;
+    }
+}
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/inline/types/IntegerInlineIndexKeyType.java b/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/inline/types/IntegerInlineIndexKeyType.java
new file mode 100644
index 0000000..8a20ff4
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/inline/types/IntegerInlineIndexKeyType.java
@@ -0,0 +1,61 @@
+/*
+ * 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.cache.query.index.sorted.inline.types;
+
+import org.apache.ignite.internal.cache.query.index.sorted.IndexKeyTypes;
+import org.apache.ignite.internal.cache.query.index.sorted.keys.IntegerIndexKey;
+import org.apache.ignite.internal.pagemem.PageUtils;
+
+/**
+ * Inline index key implementation for inlining {@link Integer} values.
+ */
+public class IntegerInlineIndexKeyType extends NullableInlineIndexKeyType<IntegerIndexKey> {
+    /** Constructor. */
+    public IntegerInlineIndexKeyType() {
+        super(IndexKeyTypes.INT, (short) 4);
+    }
+
+    /** {@inheritDoc} */
+    @Override protected int put0(long pageAddr, int off, IntegerIndexKey key, int maxSize) {
+        PageUtils.putByte(pageAddr, off, (byte) type());
+        // +1 shift after type
+        PageUtils.putInt(pageAddr, off + 1, (int) key.key());
+
+        return keySize + 1;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected IntegerIndexKey get0(long pageAddr, int off) {
+        // +1 shift after type
+        int key = PageUtils.getInt(pageAddr, off + 1);
+
+        return new IntegerIndexKey(key);
+    }
+
+    /** {@inheritDoc} */
+    @Override public int compare0(long pageAddr, int off, IntegerIndexKey key) {
+        int val1 = PageUtils.getInt(pageAddr, off + 1);
+
+        return Integer.signum(Integer.compare(val1, (int) key.key()));
+    }
+
+    /** {@inheritDoc} */
+    @Override protected int inlineSize0(IntegerIndexKey key) {
+        return keySize + 1;
+    }
+}
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/inline/types/LongInlineIndexKeyType.java b/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/inline/types/LongInlineIndexKeyType.java
new file mode 100644
index 0000000..ab9c3c4
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/inline/types/LongInlineIndexKeyType.java
@@ -0,0 +1,61 @@
+/*
+ * 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.cache.query.index.sorted.inline.types;
+
+import org.apache.ignite.internal.cache.query.index.sorted.IndexKeyTypes;
+import org.apache.ignite.internal.cache.query.index.sorted.keys.LongIndexKey;
+import org.apache.ignite.internal.pagemem.PageUtils;
+
+/**
+ * Inline index column implementation for inlining {@link Integer} values.
+ */
+public class LongInlineIndexKeyType extends NullableInlineIndexKeyType<LongIndexKey> {
+    /** Constructor. */
+    public LongInlineIndexKeyType() {
+        super(IndexKeyTypes.LONG, (short) 8);
+    }
+
+    /** {@inheritDoc} */
+    @Override protected int put0(long pageAddr, int off, LongIndexKey key, int maxSize) {
+        PageUtils.putByte(pageAddr, off, (byte) type());
+        // +1 shift after type
+        PageUtils.putLong(pageAddr, off + 1, (long) key.key());
+
+        return keySize + 1;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected LongIndexKey get0(long pageAddr, int off) {
+        // +1 shift after type
+        long key = PageUtils.getLong(pageAddr, off + 1);
+
+        return new LongIndexKey(key);
+    }
+
+    /** {@inheritDoc} */
+    @Override public int compare0(long pageAddr, int off, LongIndexKey key) {
+        long val1 = PageUtils.getLong(pageAddr, off + 1);
+
+        return Integer.signum(Long.compare(val1, (long) key.key()));
+    }
+
+    /** {@inheritDoc} */
+    @Override protected int inlineSize0(LongIndexKey key) {
+        return keySize + 1;
+    }
+}
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/inline/types/NullableInlineIndexKeyType.java b/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/inline/types/NullableInlineIndexKeyType.java
new file mode 100644
index 0000000..6b96730
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/inline/types/NullableInlineIndexKeyType.java
@@ -0,0 +1,232 @@
+/*
+ * 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.cache.query.index.sorted.inline.types;
+
+import org.apache.ignite.internal.cache.query.index.sorted.IndexKeyTypes;
+import org.apache.ignite.internal.cache.query.index.sorted.inline.InlineIndexKeyType;
+import org.apache.ignite.internal.cache.query.index.sorted.keys.IndexKey;
+import org.apache.ignite.internal.cache.query.index.sorted.keys.NullIndexKey;
+import org.apache.ignite.internal.pagemem.PageUtils;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Abstract inline key. Store base logic for work with inlined keys. Handle NULL values.
+ */
+public abstract class NullableInlineIndexKeyType<T extends IndexKey> implements InlineIndexKeyType {
+    /** Value for comparison meaning 'Not enough information to compare'. */
+    public static final int CANT_BE_COMPARE = -2;
+
+    /** Value for comparison meaning 'Compare not supported for given value'. */
+    public static final int COMPARE_UNSUPPORTED = Integer.MIN_VALUE;
+
+    /** Type of this key. */
+    private final int type;
+
+    /** Actual size of a key without type field. */
+    protected final short keySize;
+
+    /**
+     * @param type Index key type.
+     * @param keySize Size of value stored in the key.
+     */
+    protected NullableInlineIndexKeyType(int type, short keySize) {
+        this.type = type;
+        this.keySize = keySize;
+    }
+
+    /** {@inheritDoc} */
+    @Override public int type() {
+        return type;
+    }
+
+    /** {@inheritDoc} */
+    @Override public int inlineSize(long pageAddr, int off) {
+        int type = PageUtils.getByte(pageAddr, off);
+
+        if (type == IndexKeyTypes.NULL)
+            return 1;
+
+        if (keySize > 0)
+            // For fixed length types.
+            return keySize + 1;
+        else
+            // For variable length types.
+            return (PageUtils.getShort(pageAddr, off + 1) & 0x7FFF) + 3;
+    }
+
+    /** {@inheritDoc} */
+    @Override public int inlineSize() {
+        if (type == IndexKeyTypes.NULL)
+            return 1;
+
+        // For variable length keys returns -1.
+        return keySize < 0 ? keySize : keySize + 1;
+    }
+
+    /** {@inheritDoc} */
+    @Override public int inlineSize(IndexKey key) {
+        if (key == NullIndexKey.INSTANCE)
+            return 1;
+
+        ensureKeyType(key);
+
+        return inlineSize0((T) key);
+    }
+
+    /**
+     * Restores value from inline, if possible.
+     *
+     * @param pageAddr Address of the page.
+     * @param off Offset on the page.
+     * @param maxSize Max size to read.
+     *
+     * @return Restored value or {@code null} if value can't be restored.
+     */
+    @Override public IndexKey get(long pageAddr, int off, int maxSize) {
+        if (keySize > 0 && keySize + 1 > maxSize)
+            return null;
+
+        if (maxSize < 1)
+            return null;
+
+        int type = PageUtils.getByte(pageAddr, off);
+
+        if (type == IndexKeyTypes.UNKNOWN)
+            return null;
+
+        if (type == IndexKeyTypes.NULL)
+            return NullIndexKey.INSTANCE;
+
+        ensureKeyType(type);
+
+        IndexKey o = get0(pageAddr, off);
+
+        if (o == null)
+            return NullIndexKey.INSTANCE;
+
+        return o;
+    }
+
+    /** {@inheritDoc} */
+    @Override public int put(long pageAddr, int off, IndexKey key, int maxSize) {
+        // +1 is a length of the type byte.
+        if (keySize > 0 && keySize + 1 > maxSize)
+            return 0;
+
+        if (keySize < 0 && maxSize < 4) {
+            // Can't fit vartype field.
+            PageUtils.putByte(pageAddr, off, (byte) IndexKeyTypes.UNKNOWN);
+            return 0;
+        }
+
+        if (key == NullIndexKey.INSTANCE) {
+            PageUtils.putByte(pageAddr, off, (byte) IndexKeyTypes.NULL);
+            return 1;
+        }
+
+        ensureKeyType(key);
+
+        return put0(pageAddr, off, (T) key, maxSize);
+    }
+
+    /**
+     * Puts given value into inline index tree.
+     *
+     * @param pageAddr Page address.
+     * @param off Offset.
+     * @param val Value.
+     * @param maxSize Max size.
+     *
+     * @return Amount of bytes actually stored.
+     */
+    protected abstract int put0(long pageAddr, int off, T val, int maxSize);
+
+    /**
+     * Restores value from inline.
+     *
+     * @param pageAddr Page address.
+     * @param off Offset.
+     *
+     * @return Inline value or {@code null} if value can't be restored.
+     */
+    protected abstract @Nullable T get0(long pageAddr, int off);
+
+    /** Read variable length bytearray */
+    protected byte[] readBytes(long pageAddr, int off) {
+        int size = PageUtils.getShort(pageAddr, off + 1) & 0x7FFF;
+        return PageUtils.getBytes(pageAddr, off + 3, size);
+    }
+
+    /** {@inheritDoc} */
+    @Override public int compare(long pageAddr, int off, int maxSize, IndexKey key) {
+        int type;
+
+        if ((keySize > 0 && keySize + 1 > maxSize)
+            || maxSize < 1
+            || (type = PageUtils.getByte(pageAddr, off)) == (byte) IndexKeyTypes.UNKNOWN)
+            return CANT_BE_COMPARE;
+
+        if (type == IndexKeyTypes.NULL) {
+            if (key == NullIndexKey.INSTANCE)
+                return 0;
+            else
+                return -1;
+        }
+
+        if (type() != type)
+            return COMPARE_UNSUPPORTED;
+
+        if (key == NullIndexKey.INSTANCE)
+            return 1;
+
+        return compare0(pageAddr, off, (T) key);
+    }
+
+    /**
+     * Checks whether specified val corresponds to this key type.
+     */
+    private void ensureKeyType(int type) {
+        if (this.type != type)
+            throw new UnsupportedOperationException("Value type doesn't match: exp=" + this.type + ", act=" + type);
+    }
+
+    /**
+     * Checks whether specified val corresponds to this key type.
+     */
+    private void ensureKeyType(IndexKey key) {
+        if (key != NullIndexKey.INSTANCE && type != key.type())
+            throw new UnsupportedOperationException(key.type() + " cannot be used for inline type " + type());
+    }
+
+    /**
+     * Compares inlined and given value.
+     *
+     * @param pageAddr Page address.
+     * @param off Offset.
+     * @param v Value that should be compare.
+     *
+     * @return -1, 0 or 1 if inlined value less, equal or greater
+     * than given respectively, {@link #CANT_BE_COMPARE} if inlined part
+     * is not enough to compare, or {@link #COMPARE_UNSUPPORTED} if given value
+     * can't be compared with inlined part at all.
+     */
+    public abstract int compare0(long pageAddr, int off, T v);
+
+    /** Return inlined size for specified key. */
+    protected abstract int inlineSize0(T key);
+}
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/inline/types/ObjectByteArrayInlineIndexKeyType.java b/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/inline/types/ObjectByteArrayInlineIndexKeyType.java
new file mode 100644
index 0000000..79eaedb
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/inline/types/ObjectByteArrayInlineIndexKeyType.java
@@ -0,0 +1,69 @@
+/*
+ * 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.cache.query.index.sorted.inline.types;
+
+import org.apache.ignite.internal.cache.query.index.sorted.IndexKeyTypes;
+import org.apache.ignite.internal.cache.query.index.sorted.keys.BytesIndexKey;
+import org.apache.ignite.internal.cache.query.index.sorted.keys.JavaObjectIndexKey;
+import org.apache.ignite.internal.cache.query.index.sorted.keys.PlainJavaObjectIndexKey;
+
+/**
+ * Inline index key implementation for inlining Java Objects as byte array.
+ */
+public class ObjectByteArrayInlineIndexKeyType extends NullableInlineIndexKeyType<JavaObjectIndexKey> {
+    /** */
+    private final BytesInlineIndexKeyType delegate;
+
+    /** */
+    public ObjectByteArrayInlineIndexKeyType(BytesInlineIndexKeyType delegate) {
+        super(IndexKeyTypes.JAVA_OBJECT, (short) -1);
+
+        this.delegate = delegate;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected int put0(long pageAddr, int off, JavaObjectIndexKey key, int maxSize) {
+        byte[] b = key.bytes();
+
+        // Signed or unsigned doesn't matter there.
+        return delegate.put0(pageAddr, off, new BytesIndexKey(b), maxSize);
+    }
+
+    /** {@inheritDoc} */
+    @Override protected JavaObjectIndexKey get0(long pageAddr, int off) {
+        byte[] b = (byte[]) delegate.get0(pageAddr, off).key();
+
+        return new PlainJavaObjectIndexKey(null, b);
+    }
+
+    /** {@inheritDoc} */
+    @Override public int compare0(long pageAddr, int off, JavaObjectIndexKey key) {
+        byte[] b = key.bytesNoCopy();
+
+        // Signed or unsigned doesn't matter there.
+        return delegate.compare0(pageAddr, off, new BytesIndexKey(b));
+    }
+
+    /** {@inheritDoc} */
+    @Override protected int inlineSize0(JavaObjectIndexKey key) {
+        byte[] b = key.bytes();
+
+        // Signed or unsigned doesn't matter there.
+        return delegate.inlineSize0(new BytesIndexKey(b));
+    }
+}
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/inline/types/ObjectHashInlineIndexKeyType.java b/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/inline/types/ObjectHashInlineIndexKeyType.java
new file mode 100644
index 0000000..fa6b4bd
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/inline/types/ObjectHashInlineIndexKeyType.java
@@ -0,0 +1,64 @@
+/*
+ * 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.cache.query.index.sorted.inline.types;
+
+import org.apache.ignite.internal.cache.query.index.sorted.IndexKeyTypes;
+import org.apache.ignite.internal.cache.query.index.sorted.keys.JavaObjectIndexKey;
+import org.apache.ignite.internal.cache.query.index.sorted.keys.PlainJavaObjectIndexKey;
+import org.apache.ignite.internal.pagemem.PageUtils;
+
+/**
+ * Inline index key implementation for inlining hash of Java objects.
+ */
+public class ObjectHashInlineIndexKeyType extends NullableInlineIndexKeyType<JavaObjectIndexKey> {
+    /** */
+    public ObjectHashInlineIndexKeyType() {
+        super(IndexKeyTypes.JAVA_OBJECT, (short) 4);
+    }
+
+    /** {@inheritDoc} */
+    @Override protected int put0(long pageAddr, int off, JavaObjectIndexKey val, int maxSize) {
+        PageUtils.putByte(pageAddr, off, (byte) type());
+        PageUtils.putInt(pageAddr, off + 1, val.hashCode());
+
+        return keySize + 1;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected JavaObjectIndexKey get0(long pageAddr, int off) {
+        // Returns hash code of object.
+        int hash = PageUtils.getInt(pageAddr, off + 1);
+
+        return new PlainJavaObjectIndexKey(hash, null);
+    }
+
+    /** {@inheritDoc} */
+    @Override public int compare0(long pageAddr, int off, JavaObjectIndexKey v) {
+        int val1 = PageUtils.getInt(pageAddr, off + 1);
+        int val2 = v.hashCode();
+
+        int res = Integer.signum(Integer.compare(val1, val2));
+
+        return res == 0 ? CANT_BE_COMPARE : res;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected int inlineSize0(JavaObjectIndexKey key) {
+        return keySize + 1;
+    }
+}
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/inline/types/ShortInlineIndexKeyType.java b/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/inline/types/ShortInlineIndexKeyType.java
new file mode 100644
index 0000000..2502e65
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/inline/types/ShortInlineIndexKeyType.java
@@ -0,0 +1,59 @@
+/*
+ * 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.cache.query.index.sorted.inline.types;
+
+import org.apache.ignite.internal.cache.query.index.sorted.IndexKeyTypes;
+import org.apache.ignite.internal.cache.query.index.sorted.keys.ShortIndexKey;
+import org.apache.ignite.internal.pagemem.PageUtils;
+
+/**
+ * Inline index key implementation for inlining {@link Short} values.
+ */
+public class ShortInlineIndexKeyType extends NullableInlineIndexKeyType<ShortIndexKey> {
+    /** */
+    public ShortInlineIndexKeyType() {
+        super(IndexKeyTypes.SHORT, (short)2);
+    }
+
+    /** {@inheritDoc} */
+    @Override public int compare0(long pageAddr, int off, ShortIndexKey key) {
+        short val1 = PageUtils.getShort(pageAddr, off + 1);
+
+        return Integer.signum(val1 - (short) key.key());
+    }
+
+    /** {@inheritDoc} */
+    @Override protected int put0(long pageAddr, int off, ShortIndexKey key, int maxSize) {
+        PageUtils.putByte(pageAddr, off, (byte) type());
+        PageUtils.putShort(pageAddr, off + 1, (short) key.key());
+
+        return keySize + 1;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected ShortIndexKey get0(long pageAddr, int off) {
+        short key = PageUtils.getShort(pageAddr, off + 1);
+
+        return new ShortIndexKey(key);
+    }
+
+    /** {@inheritDoc} */
+    @Override protected int inlineSize0(ShortIndexKey key) {
+        return keySize + 1;
+    }
+}
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryRowCacheCleaner.java b/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/inline/types/SignedBytesInlineIndexKeyType.java
similarity index 67%
copy from modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryRowCacheCleaner.java
copy to modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/inline/types/SignedBytesInlineIndexKeyType.java
index 6275813..bd26f8b 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryRowCacheCleaner.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/inline/types/SignedBytesInlineIndexKeyType.java
@@ -15,16 +15,19 @@
  * limitations under the License.
  */
 
-package org.apache.ignite.internal.processors.query;
+package org.apache.ignite.internal.cache.query.index.sorted.inline.types;
 
 /**
- * Row cache cleaner is used by page memory manager to remove updated / evicted links from rows cache.
+ * Inline index key implementation for inlining byte arrays.
  */
-public interface GridQueryRowCacheCleaner {
-    /**
-     * Remove row by link.
-     *
-     * @param link Link to remove.
-     */
-    void remove(long link);
+public class SignedBytesInlineIndexKeyType extends BytesInlineIndexKeyType {
+    /** */
+    public SignedBytesInlineIndexKeyType() {
+        super(false);
+    }
+
+    /** */
+    public SignedBytesInlineIndexKeyType(int type) {
+        super(type, false);
+    }
 }
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/inlinecolumn/StringInlineIndexColumn.java b/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/inline/types/StringInlineIndexKeyType.java
similarity index 76%
rename from modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/inlinecolumn/StringInlineIndexColumn.java
rename to modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/inline/types/StringInlineIndexKeyType.java
index 6d5341d..9b74923 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/inlinecolumn/StringInlineIndexColumn.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/inline/types/StringInlineIndexKeyType.java
@@ -15,62 +15,36 @@
  * limitations under the License.
  */
 
-package org.apache.ignite.internal.processors.query.h2.database.inlinecolumn;
+package org.apache.ignite.internal.cache.query.index.sorted.inline.types;
 
 import java.nio.charset.Charset;
 import java.nio.charset.StandardCharsets;
+import org.apache.ignite.internal.cache.query.index.sorted.IndexKeyTypes;
+import org.apache.ignite.internal.cache.query.index.sorted.keys.StringIndexKey;
 import org.apache.ignite.internal.pagemem.PageUtils;
 import org.apache.ignite.internal.util.GridUnsafe;
-import org.h2.table.Column;
-import org.h2.value.Value;
-import org.h2.value.ValueString;
+import org.jetbrains.annotations.Nullable;
 
 /**
- * Inline index column implementation for inlining {@link String} values.
+ * Inline index key implementation for inlining {@link String} values.
  */
-public class StringInlineIndexColumn extends AbstractInlineIndexColumn {
+public class StringInlineIndexKeyType extends NullableInlineIndexKeyType<StringIndexKey> {
     /** Default charset. */
     protected static final Charset CHARSET = StandardCharsets.UTF_8;
 
     /** Whether respect case or not while comparing. */
-    private final boolean compareIgnoreCase;
+    private static final boolean compareIgnoreCase = false;
 
-    /** Whether to use optimized comparison or not. */
-    private final boolean useOptimizedCompare;
-
-    /**
-     * @param col Column.
-     */
-    public StringInlineIndexColumn(Column col, boolean useOptimizedCompare) {
-        this(col, Value.STRING, useOptimizedCompare, false);
-    }
-
-    /**
-     * @param col Column.
-     * @param type Type.
-     */
-    StringInlineIndexColumn(Column col, int type, boolean useOptimizedCompare, boolean compareIgnoreCase) {
-        super(col, type, (short)-1);
-
-        this.compareIgnoreCase = compareIgnoreCase;
-        this.useOptimizedCompare = useOptimizedCompare;
+    /** Constructor. */
+    public StringInlineIndexKeyType() {
+        super(IndexKeyTypes.STRING, (short) -1);  // -1 means variable length.
     }
 
     /** {@inheritDoc} */
-    @Override protected int compare0(long pageAddr, int off, Value v, int type) {
-        if (type() != type || !useOptimizedCompare)
-            return COMPARE_UNSUPPORTED;
-
-        return compareAsString(pageAddr, off, v, compareIgnoreCase);
-    }
-
-    /** {@inheritDoc} */
-    @Override protected int put0(long pageAddr, int off, Value val, int maxSize) {
-        assert type() == val.getType();
-
+    @Override protected int put0(long pageAddr, int off, StringIndexKey str, int maxSize) {
         short size;
 
-        byte[] s = val.getString().getBytes(CHARSET);
+        byte[] s = ((String) str.key()).getBytes(CHARSET);
         if (s.length + 3 <= maxSize)
             size = (short)s.length;
         else {
@@ -80,11 +54,11 @@ public class StringInlineIndexColumn extends AbstractInlineIndexColumn {
 
         if (s == null) {
             // Can't fit anything to
-            PageUtils.putByte(pageAddr, off, (byte)Value.UNKNOWN);
+            PageUtils.putByte(pageAddr, off, (byte) IndexKeyTypes.UNKNOWN);
             return 0;
         }
         else {
-            PageUtils.putByte(pageAddr, off, (byte)val.getType());
+            PageUtils.putByte(pageAddr, off, (byte) type());
             PageUtils.putShort(pageAddr, off + 1, size);
             PageUtils.putBytes(pageAddr, off + 3, s);
             return s.length + 3;
@@ -92,26 +66,15 @@ public class StringInlineIndexColumn extends AbstractInlineIndexColumn {
     }
 
     /** {@inheritDoc} */
-    @Override protected Value get0(long pageAddr, int off) {
-        return ValueString.get(new String(readBytes(pageAddr, off), CHARSET));
-    }
-
-    /** {@inheritDoc} */
-    @Override protected int inlineSizeOf0(Value val) {
-        assert val.getType() == type();
+    @Override protected @Nullable StringIndexKey get0(long pageAddr, int off) {
+        String s = new String(readBytes(pageAddr, off), CHARSET);
 
-        return val.getString().getBytes(CHARSET).length + 3;
+        return new StringIndexKey(s);
     }
 
-    /**
-     * @param pageAddr Page address.
-     * @param off Offset.
-     * @param v Value to compare.
-     * @param ignoreCase {@code True} if a case-insensitive comparison should be used.
-     * @return Compare result ({@code CANT_BE_COMPARE} means we can't compare).
-     */
-    private int compareAsString(long pageAddr, int off, Value v, boolean ignoreCase) {
-        String s = v.getString();
+    /** {@inheritDoc} */
+    @Override public int compare0(long pageAddr, int off, StringIndexKey key) {
+        String s = (String) key.key();
 
         int len1 = PageUtils.getShort(pageAddr, off + 1) & 0x7FFF;
         int len2 = s.length();
@@ -133,7 +96,7 @@ public class StringInlineIndexColumn extends AbstractInlineIndexColumn {
             v1 = (char)c;
             v2 = s.charAt(cntr2++);
 
-            if (ignoreCase) {
+            if (compareIgnoreCase) {
                 v1 = Character.toUpperCase(v1);
                 v2 = Character.toUpperCase(v2);
             }
@@ -255,7 +218,7 @@ public class StringInlineIndexColumn extends AbstractInlineIndexColumn {
 
             v2 = s.charAt(cntr2++);
 
-            if (ignoreCase) {
+            if (compareIgnoreCase) {
                 v1 = Character.toUpperCase(v1);
                 v2 = Character.toUpperCase(v2);
             }
@@ -279,22 +242,13 @@ public class StringInlineIndexColumn extends AbstractInlineIndexColumn {
     }
 
     /**
-     * @param pageAddr Page address.
-     * @param off Offset.
-     * @return {@code True} if string is not truncated on save.
-     */
-    private boolean isValueFull(long pageAddr, int off) {
-        return (PageUtils.getShort(pageAddr, off + 1) & 0x8000) == 0;
-    }
-
-    /**
      * Convert String to byte[] with size limit, according to UTF-8 encoding.
      *
      * @param bytes byte[].
      * @param limit Size limit.
      * @return byte[].
      */
-    static byte[] trimUTF8(byte[] bytes, int limit) {
+    public static byte[] trimUTF8(byte[] bytes, int limit) {
         if (bytes.length <= limit)
             return bytes;
 
@@ -308,4 +262,18 @@ public class StringInlineIndexColumn extends AbstractInlineIndexColumn {
 
         return null;
     }
+
+    /**
+     * @param pageAddr Page address.
+     * @param off Offset.
+     * @return {@code True} if string is not truncated on save.
+     */
+    private boolean isValueFull(long pageAddr, int off) {
+        return (PageUtils.getShort(pageAddr, off + 1) & 0x8000) == 0;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected int inlineSize0(StringIndexKey key) {
+        return ((String) key.key()).getBytes(CHARSET).length + 3;
+    }
 }
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/inline/types/StringNoCompareInlineIndexKeyType.java b/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/inline/types/StringNoCompareInlineIndexKeyType.java
new file mode 100644
index 0000000..b17e4d0
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/inline/types/StringNoCompareInlineIndexKeyType.java
@@ -0,0 +1,55 @@
+/*
+ * 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.cache.query.index.sorted.inline.types;
+
+import org.apache.ignite.internal.cache.query.index.sorted.IndexKeyTypes;
+import org.apache.ignite.internal.cache.query.index.sorted.keys.StringIndexKey;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Skip optimized String comparison implemented in {@link StringInlineIndexKeyType}.
+ */
+public class StringNoCompareInlineIndexKeyType extends NullableInlineIndexKeyType<StringIndexKey> {
+    /** Delegate all String operations except comparison to StringInlineIndexKeyType. */
+    private final StringInlineIndexKeyType delegate = new StringInlineIndexKeyType();
+
+    /** */
+    public StringNoCompareInlineIndexKeyType() {
+        super(IndexKeyTypes.STRING, (short) -1);
+    }
+
+    /** {@inheritDoc} */
+    @Override protected int put0(long pageAddr, int off, StringIndexKey key, int maxSize) {
+        return delegate.put0(pageAddr, off, key, maxSize);
+    }
+
+    /** {@inheritDoc} */
+    @Override protected @Nullable StringIndexKey get0(long pageAddr, int off) {
+        return delegate.get0(pageAddr, off);
+    }
+
+    /** {@inheritDoc} */
+    @Override public int compare0(long pageAddr, int off, StringIndexKey key) {
+        return COMPARE_UNSUPPORTED;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected int inlineSize0(StringIndexKey key) {
+        return delegate.inlineSize0(key);
+    }
+}
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/inline/types/TimeInlineIndexKeyType.java b/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/inline/types/TimeInlineIndexKeyType.java
new file mode 100644
index 0000000..708d0d4
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/inline/types/TimeInlineIndexKeyType.java
@@ -0,0 +1,61 @@
+/*
+ * 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.cache.query.index.sorted.inline.types;
+
+import org.apache.ignite.internal.cache.query.index.sorted.IndexKeyTypes;
+import org.apache.ignite.internal.cache.query.index.sorted.keys.AbstractTimeIndexKey;
+import org.apache.ignite.internal.cache.query.index.sorted.keys.IndexKeyFactory;
+import org.apache.ignite.internal.pagemem.PageUtils;
+
+/**
+ * Inline index key implementation for inlining {@link AbstractTimeIndexKey} values.
+ */
+public class TimeInlineIndexKeyType extends NullableInlineIndexKeyType<AbstractTimeIndexKey> {
+    /** */
+    public TimeInlineIndexKeyType() {
+        super(IndexKeyTypes.TIME, (short) 8);
+    }
+
+    /** {@inheritDoc} */
+    @Override public int compare0(long pageAddr, int off, AbstractTimeIndexKey key) {
+        long val1 = PageUtils.getLong(pageAddr, off + 1);
+        long val2 = key.nanos();
+
+        return Integer.signum(Long.compare(val1, val2));
+    }
+
+    /** {@inheritDoc} */
+    @Override protected int put0(long pageAddr, int off, AbstractTimeIndexKey key, int maxSize) {
+        PageUtils.putByte(pageAddr, off, (byte) type());
+        PageUtils.putLong(pageAddr, off + 1, key.nanos());
+
+        return keySize + 1;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected AbstractTimeIndexKey get0(long pageAddr, int off) {
+        long nanos = PageUtils.getLong(pageAddr, off + 1);
+
+        return (AbstractTimeIndexKey) IndexKeyFactory.wrapDateValue(type(), 0L, nanos);
+    }
+
+    /** {@inheritDoc} */
+    @Override protected int inlineSize0(AbstractTimeIndexKey key) {
+        return keySize + 1;
+    }
+}
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/inline/types/TimestampInlineIndexKeyType.java b/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/inline/types/TimestampInlineIndexKeyType.java
new file mode 100644
index 0000000..c7852ac
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/inline/types/TimestampInlineIndexKeyType.java
@@ -0,0 +1,82 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.cache.query.index.sorted.inline.types;
+
+import org.apache.ignite.internal.cache.query.index.sorted.IndexKeyTypes;
+import org.apache.ignite.internal.cache.query.index.sorted.keys.AbstractTimestampIndexKey;
+import org.apache.ignite.internal.cache.query.index.sorted.keys.IndexKeyFactory;
+import org.apache.ignite.internal.pagemem.PageUtils;
+
+import static org.apache.ignite.internal.cache.query.index.sorted.inline.types.DateValueConstants.MAX_DATE_VALUE;
+import static org.apache.ignite.internal.cache.query.index.sorted.inline.types.DateValueConstants.MIN_DATE_VALUE;
+import static org.apache.ignite.internal.cache.query.index.sorted.inline.types.DateValueConstants.NANOS_PER_DAY;
+
+/**
+ * Inline index key implementation for inlining {@link AbstractTimestampIndexKey} values.
+ */
+public class TimestampInlineIndexKeyType extends NullableInlineIndexKeyType<AbstractTimestampIndexKey> {
+    /** */
+    public TimestampInlineIndexKeyType() {
+        super(IndexKeyTypes.TIMESTAMP, (short) 16);
+    }
+
+    /** {@inheritDoc} */
+    @Override public int compare0(long pageAddr, int off, AbstractTimestampIndexKey key) {
+        long val1 = PageUtils.getLong(pageAddr, off + 1);
+
+        int c = Long.compare(val1, key.dateValue());
+
+        if (c != 0)
+            return Integer.signum(c);
+
+        long nanos1 = PageUtils.getLong(pageAddr, off + 9);
+
+        return Integer.signum(Long.compare(nanos1, key.nanos()));
+    }
+
+    /** {@inheritDoc} */
+    @Override protected int put0(long pageAddr, int off, AbstractTimestampIndexKey key, int maxSize) {
+        PageUtils.putByte(pageAddr, off, (byte) type());
+
+        PageUtils.putLong(pageAddr, off + 1, key.dateValue());
+        PageUtils.putLong(pageAddr, off + 9, key.nanos());
+
+        return keySize + 1;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected AbstractTimestampIndexKey get0(long pageAddr, int off) {
+        long dv = PageUtils.getLong(pageAddr, off + 1);
+        long nanos = PageUtils.getLong(pageAddr, off + 9);
+
+        if (dv > MAX_DATE_VALUE) {
+            dv = MAX_DATE_VALUE;
+            nanos = NANOS_PER_DAY - 1;
+        } else if (dv < MIN_DATE_VALUE) {
+            dv = MIN_DATE_VALUE;
+            nanos = 0;
+        }
+
+        return (AbstractTimestampIndexKey) IndexKeyFactory.wrapDateValue(type(), dv, nanos);
+    }
+
+    /** {@inheritDoc} */
+    @Override protected int inlineSize0(AbstractTimestampIndexKey key) {
+        return keySize + 1;
+    }
+}
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/inline/types/UuidInlineIndexKeyType.java b/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/inline/types/UuidInlineIndexKeyType.java
new file mode 100644
index 0000000..fd2e62f
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/inline/types/UuidInlineIndexKeyType.java
@@ -0,0 +1,74 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.cache.query.index.sorted.inline.types;
+
+import java.util.UUID;
+import org.apache.ignite.internal.cache.query.index.sorted.IndexKeyTypes;
+import org.apache.ignite.internal.cache.query.index.sorted.keys.UuidIndexKey;
+import org.apache.ignite.internal.pagemem.PageUtils;
+
+/**
+ * Inline index key implementation for inlining {@link UUID} values.
+ */
+public class UuidInlineIndexKeyType extends NullableInlineIndexKeyType<UuidIndexKey> {
+    /**
+     */
+    public UuidInlineIndexKeyType() {
+        super(IndexKeyTypes.UUID, (short) 16);
+    }
+
+    /** {@inheritDoc} */
+    @Override public int compare0(long pageAddr, int off, UuidIndexKey key) {
+        UUID v = (UUID) key.key();
+
+        long part1 = PageUtils.getLong(pageAddr, off + 1);
+
+        int c = Integer.signum(Long.compare(part1, v.getMostSignificantBits()));
+
+        if (c != 0)
+            return c;
+
+        long part2 = PageUtils.getLong(pageAddr, off + 9);
+
+        return Integer.signum(Long.compare(part2, v.getLeastSignificantBits()));
+    }
+
+    /** {@inheritDoc} */
+    @Override protected int put0(long pageAddr, int off, UuidIndexKey key, int maxSize) {
+        UUID val = (UUID) key.key();
+
+        PageUtils.putByte(pageAddr, off, (byte) type());
+        PageUtils.putLong(pageAddr, off + 1, val.getMostSignificantBits());
+        PageUtils.putLong(pageAddr, off + 9, val.getLeastSignificantBits());
+
+        return keySize + 1;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected UuidIndexKey get0(long pageAddr, int off) {
+        return new UuidIndexKey(new UUID(
+            PageUtils.getLong(pageAddr, off + 1),
+            PageUtils.getLong(pageAddr, off + 9)
+        ));
+    }
+
+    /** {@inheritDoc} */
+    @Override protected int inlineSize0(UuidIndexKey val) {
+        return keySize + 1;
+    }
+}
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/inlinecolumn/ObjectBytesInlineIndexColumn.java b/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/keys/AbstractDateIndexKey.java
similarity index 54%
rename from modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/inlinecolumn/ObjectBytesInlineIndexColumn.java
rename to modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/keys/AbstractDateIndexKey.java
index 4e790ec..6db91f3 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/inlinecolumn/ObjectBytesInlineIndexColumn.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/keys/AbstractDateIndexKey.java
@@ -15,26 +15,23 @@
  * limitations under the License.
  */
 
-package org.apache.ignite.internal.processors.query.h2.database.inlinecolumn;
+package org.apache.ignite.internal.cache.query.index.sorted.keys;
 
-import org.h2.table.Column;
-import org.h2.value.Value;
-import org.h2.value.ValueJavaObject;
+import org.apache.ignite.internal.cache.query.index.sorted.IndexKeyTypes;
+import org.apache.ignite.internal.cache.query.index.sorted.inline.types.DateInlineIndexKeyType;
+import org.apache.ignite.internal.cache.query.index.sorted.inline.types.DateValueConstants;
 
 /**
- * Inline index column implementation for inlining Java objects as byte array.
+ * Abstract class for representing Date index key.
+ *
+ * {@link DateInlineIndexKeyType} relies on this API to store an object in inline.
  */
-public class ObjectBytesInlineIndexColumn extends BytesInlineIndexColumn {
-    /**
-     * @param col Column.
-     * @param compareBinaryUnsigned Compare binary unsigned.
-     */
-    public ObjectBytesInlineIndexColumn(Column col, boolean compareBinaryUnsigned) {
-        super(col, Value.JAVA_OBJECT, compareBinaryUnsigned);
-    }
+public abstract class AbstractDateIndexKey implements IndexKey {
+    /** @return a date value {@link DateValueConstants}. */
+    public abstract long dateValue();
 
     /** {@inheritDoc} */
-    @Override protected Value get0(long pageAddr, int off) {
-        return ValueJavaObject.getNoCopy(null, readBytes(pageAddr, off), null);
+    @Override public int type() {
+        return IndexKeyTypes.DATE;
     }
 }
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/io/H2LeafIO.java b/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/keys/AbstractTimeIndexKey.java
similarity index 57%
copy from modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/io/H2LeafIO.java
copy to modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/keys/AbstractTimeIndexKey.java
index 466cd1c..16cceeb 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/io/H2LeafIO.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/keys/AbstractTimeIndexKey.java
@@ -15,23 +15,22 @@
  * limitations under the License.
  */
 
-package org.apache.ignite.internal.processors.query.h2.database.io;
+package org.apache.ignite.internal.cache.query.index.sorted.keys;
 
-import org.apache.ignite.internal.processors.cache.persistence.tree.io.IOVersions;
+import org.apache.ignite.internal.cache.query.index.sorted.IndexKeyTypes;
+import org.apache.ignite.internal.cache.query.index.sorted.inline.types.TimeInlineIndexKeyType;
 
 /**
- * Leaf page for H2 row references.
+ * Abstract class for representing Time index key.
+ *
+ * {@link TimeInlineIndexKeyType} relies on this API to store an object in inline.
  */
-public class H2LeafIO extends AbstractH2LeafIO {
-    /** */
-    public static final IOVersions<H2LeafIO> VERSIONS = new IOVersions<>(
-        new H2LeafIO(1)
-    );
+public abstract class AbstractTimeIndexKey implements IndexKey {
+    /** @return nanoseconds since midnight. */
+    public abstract long nanos();
 
-    /**
-     * @param ver Page format version.
-     */
-    public H2LeafIO(int ver) {
-        super(T_H2_REF_LEAF, ver, 8);
+    /** {@inheritDoc} */
+    @Override public int type() {
+        return IndexKeyTypes.TIME;
     }
 }
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/inlinecolumn/FixedStringInlineIndexColumn.java b/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/keys/AbstractTimestampIndexKey.java
similarity index 51%
copy from modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/inlinecolumn/FixedStringInlineIndexColumn.java
copy to modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/keys/AbstractTimestampIndexKey.java
index 6bd039f..25b0e00 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/inlinecolumn/FixedStringInlineIndexColumn.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/keys/AbstractTimestampIndexKey.java
@@ -15,25 +15,26 @@
  * limitations under the License.
  */
 
-package org.apache.ignite.internal.processors.query.h2.database.inlinecolumn;
+package org.apache.ignite.internal.cache.query.index.sorted.keys;
 
-import org.h2.table.Column;
-import org.h2.value.Value;
-import org.h2.value.ValueStringFixed;
+import org.apache.ignite.internal.cache.query.index.sorted.IndexKeyTypes;
+import org.apache.ignite.internal.cache.query.index.sorted.inline.types.DateValueConstants;
+import org.apache.ignite.internal.cache.query.index.sorted.inline.types.TimestampInlineIndexKeyType;
 
 /**
- * Inline index column implementation for inlining strings of fixed length.
+ * Abstract class for representing Timestamp index key.
+ *
+ * {@link TimestampInlineIndexKeyType} relies on this API to store an object in inline.
  */
-public class FixedStringInlineIndexColumn extends StringInlineIndexColumn {
-    /**
-     * @param col Column.
-     */
-    public FixedStringInlineIndexColumn(Column col, boolean useOptimizedCompare) {
-        super(col, Value.STRING_FIXED, useOptimizedCompare, false);
-    }
+public abstract class AbstractTimestampIndexKey implements IndexKey {
+    /** @return a date value {@link DateValueConstants}. */
+    public abstract long dateValue();
+
+    /** @return nanoseconds since midnight. */
+    public abstract long nanos();
 
     /** {@inheritDoc} */
-    @Override protected Value get0(long pageAddr, int off) {
-        return ValueStringFixed.get(new String(readBytes(pageAddr, off), CHARSET));
+    @Override public int type() {
+        return IndexKeyTypes.TIMESTAMP;
     }
 }
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/inlinecolumn/StringIgnoreCaseInlineIndexColumn.java b/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/keys/BooleanIndexKey.java
similarity index 55%
rename from modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/inlinecolumn/StringIgnoreCaseInlineIndexColumn.java
rename to modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/keys/BooleanIndexKey.java
index 79ec285..28bc04d 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/inlinecolumn/StringIgnoreCaseInlineIndexColumn.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/keys/BooleanIndexKey.java
@@ -15,25 +15,34 @@
  * limitations under the License.
  */
 
-package org.apache.ignite.internal.processors.query.h2.database.inlinecolumn;
+package org.apache.ignite.internal.cache.query.index.sorted.keys;
 
-import org.h2.table.Column;
-import org.h2.value.Value;
-import org.h2.value.ValueStringIgnoreCase;
+import org.apache.ignite.internal.cache.query.index.sorted.IndexKeyTypes;
 
-/**
- * Inline index column implementation for inlining strings ignore case.
- */
-public class StringIgnoreCaseInlineIndexColumn extends StringInlineIndexColumn {
-    /**
-     * @param col Column.
-     */
-    public StringIgnoreCaseInlineIndexColumn(Column col, boolean useOptimizedCompare) {
-        super(col, Value.STRING_IGNORECASE, useOptimizedCompare, true);
+/** */
+public class BooleanIndexKey implements IndexKey {
+    /** */
+    private final boolean key;
+
+    /** */
+    public BooleanIndexKey(boolean key) {
+        this.key = key;
+    }
+
+    /** {@inheritDoc} */
+    @Override public Object key() {
+        return key;
     }
 
     /** {@inheritDoc} */
-    @Override protected Value get0(long pageAddr, int off) {
-        return ValueStringIgnoreCase.get(new String(readBytes(pageAddr, off), CHARSET));
+    @Override public int type() {
+        return IndexKeyTypes.BOOLEAN;
+    }
+
+    /** {@inheritDoc} */
+    @Override public int compare(IndexKey o) {
+        boolean okey = (boolean) o.key();
+
+        return Boolean.compare(key, okey);
     }
 }
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/io/H2LeafIO.java b/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/keys/ByteIndexKey.java
similarity index 56%
copy from modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/io/H2LeafIO.java
copy to modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/keys/ByteIndexKey.java
index 466cd1c..cd1842b 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/io/H2LeafIO.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/keys/ByteIndexKey.java
@@ -15,23 +15,34 @@
  * limitations under the License.
  */
 
-package org.apache.ignite.internal.processors.query.h2.database.io;
+package org.apache.ignite.internal.cache.query.index.sorted.keys;
 
-import org.apache.ignite.internal.processors.cache.persistence.tree.io.IOVersions;
+import org.apache.ignite.internal.cache.query.index.sorted.IndexKeyTypes;
+
+/** */
+public class ByteIndexKey implements IndexKey {
+    /** */
+    private final byte key;
 
-/**
- * Leaf page for H2 row references.
- */
-public class H2LeafIO extends AbstractH2LeafIO {
     /** */
-    public static final IOVersions<H2LeafIO> VERSIONS = new IOVersions<>(
-        new H2LeafIO(1)
-    );
-
-    /**
-     * @param ver Page format version.
-     */
-    public H2LeafIO(int ver) {
-        super(T_H2_REF_LEAF, ver, 8);
+    public ByteIndexKey(byte key) {
+        this.key = key;
+    }
+
+    /** {@inheritDoc} */
+    @Override public Object key() {
+        return key;
+    }
+
+    /** {@inheritDoc} */
+    @Override public int type() {
+        return IndexKeyTypes.BYTE;
+    }
+
+    /** {@inheritDoc} */
+    @Override public int compare(IndexKey o) {
+        byte okey = (byte) o.key();
+
+        return Byte.compare(key, okey);
     }
 }
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/keys/BytesCompareUtils.java b/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/keys/BytesCompareUtils.java
new file mode 100644
index 0000000..c2a8310
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/keys/BytesCompareUtils.java
@@ -0,0 +1,59 @@
+/*
+ * 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.cache.query.index.sorted.keys;
+
+/** */
+public final class BytesCompareUtils {
+    /** */
+    public static int compareNotNullSigned(byte[] arr0, byte[] arr1) {
+        if (arr0 == arr1)
+            return 0;
+        else {
+            int commonLen = Math.min(arr0.length, arr1.length);
+
+            for (int i = 0; i < commonLen; ++i) {
+                byte b0 = arr0[i];
+                byte b1 = arr1[i];
+
+                if (b0 != b1)
+                    return b0 > b1 ? 1 : -1;
+            }
+
+            return Integer.signum(arr0.length - arr1.length);
+        }
+    }
+
+    /** */
+    public static int compareNotNullUnsigned(byte[] arr0, byte[] arr1) {
+        if (arr0 == arr1)
+            return 0;
+        else {
+            int commonLen = Math.min(arr0.length, arr1.length);
+
+            for (int i = 0; i < commonLen; ++i) {
+                int unSignArr0 = arr0[i] & 255;
+                int unSignArr1 = arr1[i] & 255;
+
+                if (unSignArr0 != unSignArr1)
+                    return unSignArr0 > unSignArr1 ? 1 : -1;
+            }
+
+            return Integer.signum(arr0.length - arr1.length);
+        }
+    }
+}
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/inlinecolumn/FixedStringInlineIndexColumn.java b/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/keys/BytesIndexKey.java
similarity index 55%
copy from modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/inlinecolumn/FixedStringInlineIndexColumn.java
copy to modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/keys/BytesIndexKey.java
index 6bd039f..8ebf857 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/inlinecolumn/FixedStringInlineIndexColumn.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/keys/BytesIndexKey.java
@@ -15,25 +15,34 @@
  * limitations under the License.
  */
 
-package org.apache.ignite.internal.processors.query.h2.database.inlinecolumn;
+package org.apache.ignite.internal.cache.query.index.sorted.keys;
 
-import org.h2.table.Column;
-import org.h2.value.Value;
-import org.h2.value.ValueStringFixed;
+import org.apache.ignite.internal.cache.query.index.sorted.IndexKeyTypes;
 
-/**
- * Inline index column implementation for inlining strings of fixed length.
- */
-public class FixedStringInlineIndexColumn extends StringInlineIndexColumn {
-    /**
-     * @param col Column.
-     */
-    public FixedStringInlineIndexColumn(Column col, boolean useOptimizedCompare) {
-        super(col, Value.STRING_FIXED, useOptimizedCompare, false);
+/** */
+public class BytesIndexKey implements IndexKey {
+    /** */
+    protected final byte[] key;
+
+    /** */
+    public BytesIndexKey(byte[] key) {
+        this.key = key;
+    }
+
+    /** {@inheritDoc} */
+    @Override public Object key() {
+        return key;
     }
 
     /** {@inheritDoc} */
-    @Override protected Value get0(long pageAddr, int off) {
-        return ValueStringFixed.get(new String(readBytes(pageAddr, off), CHARSET));
+    @Override public int type() {
+        return IndexKeyTypes.BYTES;
+    }
+
+    /** {@inheritDoc} */
+    @Override public int compare(IndexKey o) {
+        byte[] okey = (byte[]) o.key();
+
+        return BytesCompareUtils.compareNotNullUnsigned(key, okey);
     }
 }
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/keys/CacheJavaObjectIndexKey.java b/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/keys/CacheJavaObjectIndexKey.java
new file mode 100644
index 0000000..66e79b1
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/keys/CacheJavaObjectIndexKey.java
@@ -0,0 +1,85 @@
+/*
+ * 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.cache.query.index.sorted.keys;
+
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.internal.binary.BinaryObjectImpl;
+import org.apache.ignite.internal.cache.query.index.IndexProcessor;
+import org.apache.ignite.internal.processors.cache.CacheObject;
+import org.apache.ignite.internal.processors.cache.CacheObjectValueContext;
+
+/** */
+public class CacheJavaObjectIndexKey extends JavaObjectIndexKey {
+    /** */
+    private final CacheObject obj;
+
+    /** Object value context. */
+    private final CacheObjectValueContext valCtx;
+
+    /** */
+    private byte[] serialized;
+
+    /** */
+    private Object key;
+
+    /**
+     * Constructor.
+     *
+     * @param obj Object.
+     * @param valCtx Object value context.
+     */
+    public CacheJavaObjectIndexKey(CacheObject obj, CacheObjectValueContext valCtx) {
+        assert obj != null;
+
+        if (obj instanceof BinaryObjectImpl) {
+            ((BinaryObjectImpl)obj).detachAllowed(true);
+            obj = ((BinaryObjectImpl)obj).detach();
+        }
+
+        this.obj = obj;
+        this.valCtx = valCtx;
+    }
+
+    /** {@inheritDoc} */
+    @Override public Object key() {
+        if (key == null)
+            key = obj.isPlatformType() ? obj.value(valCtx, false) : obj;
+
+        return key;
+    }
+
+    /** {@inheritDoc} */
+    @Override public byte[] bytesNoCopy() {
+        try {
+            if (serialized == null) {
+                // Result must be the same as `marshaller.marshall(obj.value(coctx, false));`
+                if (obj.cacheObjectType() == CacheObject.TYPE_REGULAR)
+                    return obj.valueBytes(valCtx);
+
+                // For user-provided and array types.
+                serialized = IndexProcessor.serializer.serialize(obj);
+            }
+
+            return serialized;
+        }
+        catch (IgniteCheckedException e) {
+            throw new IgniteException(e);
+        }
+    }
+}
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/inlinecolumn/FixedStringInlineIndexColumn.java b/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/keys/DecimalIndexKey.java
similarity index 55%
copy from modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/inlinecolumn/FixedStringInlineIndexColumn.java
copy to modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/keys/DecimalIndexKey.java
index 6bd039f..025b4e6 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/inlinecolumn/FixedStringInlineIndexColumn.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/keys/DecimalIndexKey.java
@@ -15,25 +15,35 @@
  * limitations under the License.
  */
 
-package org.apache.ignite.internal.processors.query.h2.database.inlinecolumn;
+package org.apache.ignite.internal.cache.query.index.sorted.keys;
 
-import org.h2.table.Column;
-import org.h2.value.Value;
-import org.h2.value.ValueStringFixed;
+import java.math.BigDecimal;
+import org.apache.ignite.internal.cache.query.index.sorted.IndexKeyTypes;
 
-/**
- * Inline index column implementation for inlining strings of fixed length.
- */
-public class FixedStringInlineIndexColumn extends StringInlineIndexColumn {
-    /**
-     * @param col Column.
-     */
-    public FixedStringInlineIndexColumn(Column col, boolean useOptimizedCompare) {
-        super(col, Value.STRING_FIXED, useOptimizedCompare, false);
+/** */
+public class DecimalIndexKey implements IndexKey {
+    /** */
+    private final BigDecimal key;
+
+    /** */
+    public DecimalIndexKey(BigDecimal key) {
+        this.key = key;
+    }
+
+    /** {@inheritDoc} */
+    @Override public Object key() {
+        return key;
     }
 
     /** {@inheritDoc} */
-    @Override protected Value get0(long pageAddr, int off) {
-        return ValueStringFixed.get(new String(readBytes(pageAddr, off), CHARSET));
+    @Override public int type() {
+        return IndexKeyTypes.DECIMAL;
+    }
+
+    /** {@inheritDoc} */
+    @Override public int compare(IndexKey o) {
+        BigDecimal okey = (BigDecimal) o.key();
+
+        return key.compareTo(okey);
     }
 }
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/inlinecolumn/FixedStringInlineIndexColumn.java b/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/keys/DoubleIndexKey.java
similarity index 55%
copy from modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/inlinecolumn/FixedStringInlineIndexColumn.java
copy to modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/keys/DoubleIndexKey.java
index 6bd039f..9967189d 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/inlinecolumn/FixedStringInlineIndexColumn.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/keys/DoubleIndexKey.java
@@ -15,25 +15,34 @@
  * limitations under the License.
  */
 
-package org.apache.ignite.internal.processors.query.h2.database.inlinecolumn;
+package org.apache.ignite.internal.cache.query.index.sorted.keys;
 
-import org.h2.table.Column;
-import org.h2.value.Value;
-import org.h2.value.ValueStringFixed;
+import org.apache.ignite.internal.cache.query.index.sorted.IndexKeyTypes;
 
-/**
- * Inline index column implementation for inlining strings of fixed length.
- */
-public class FixedStringInlineIndexColumn extends StringInlineIndexColumn {
-    /**
-     * @param col Column.
-     */
-    public FixedStringInlineIndexColumn(Column col, boolean useOptimizedCompare) {
-        super(col, Value.STRING_FIXED, useOptimizedCompare, false);
+/** */
+public class DoubleIndexKey implements IndexKey {
+    /** */
+    private final double key;
+
+    /** */
+    public DoubleIndexKey(double key) {
+        this.key = key;
+    }
+
+    /** {@inheritDoc} */
+    @Override public Object key() {
+        return key;
     }
 
     /** {@inheritDoc} */
-    @Override protected Value get0(long pageAddr, int off) {
-        return ValueStringFixed.get(new String(readBytes(pageAddr, off), CHARSET));
+    @Override public int type() {
+        return IndexKeyTypes.DOUBLE;
+    }
+
+    /** {@inheritDoc} */
+    @Override public int compare(IndexKey o) {
+        double okey = (double) o.key();
+
+        return Double.compare(key, okey);
     }
 }
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/io/H2LeafIO.java b/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/keys/FloatIndexKey.java
similarity index 56%
copy from modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/io/H2LeafIO.java
copy to modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/keys/FloatIndexKey.java
index 466cd1c..32986a6 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/io/H2LeafIO.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/keys/FloatIndexKey.java
@@ -15,23 +15,34 @@
  * limitations under the License.
  */
 
-package org.apache.ignite.internal.processors.query.h2.database.io;
+package org.apache.ignite.internal.cache.query.index.sorted.keys;
 
-import org.apache.ignite.internal.processors.cache.persistence.tree.io.IOVersions;
+import org.apache.ignite.internal.cache.query.index.sorted.IndexKeyTypes;
+
+/** */
+public class FloatIndexKey implements IndexKey {
+    /** */
+    private final float key;
 
-/**
- * Leaf page for H2 row references.
- */
-public class H2LeafIO extends AbstractH2LeafIO {
     /** */
-    public static final IOVersions<H2LeafIO> VERSIONS = new IOVersions<>(
-        new H2LeafIO(1)
-    );
-
-    /**
-     * @param ver Page format version.
-     */
-    public H2LeafIO(int ver) {
-        super(T_H2_REF_LEAF, ver, 8);
+    public FloatIndexKey(float key) {
+        this.key = key;
+    }
+
+    /** {@inheritDoc} */
+    @Override public Object key() {
+        return key;
+    }
+
+    /** {@inheritDoc} */
+    @Override public int type() {
+        return IndexKeyTypes.FLOAT;
... 11603 lines suppressed ...