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/02/26 06:03:40 UTC

[ignite] branch master updated: IGNITE-14206 fix compatibility inline POJO between 2.8.1 and 2.9.0 (#8811)

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 851f650  IGNITE-14206 fix compatibility inline POJO between 2.8.1 and 2.9.0 (#8811)
851f650 is described below

commit 851f650ba03e0b6c081cfe23f411fd2bf6be0228
Author: Maksim Timonin <ti...@gmail.com>
AuthorDate: Fri Feb 26 09:03:16 2021 +0300

    IGNITE-14206 fix compatibility inline POJO between 2.8.1 and 2.9.0 (#8811)
---
 .travis.yml                                        |   2 +-
 .../persistence/InlineIndexCompatibilityTest.java  | 359 +++++++++++++++++++++
 .../IgniteCompatibilityBasicTestSuite.java         |   2 +
 .../processors/query/h2/database/H2Tree.java       |  57 +++-
 .../h2/database/InlineObjectBytesDetector.java     | 196 +++++++++++
 5 files changed, 605 insertions(+), 11 deletions(-)

diff --git a/.travis.yml b/.travis.yml
index a6aa379..c29e3fd 100644
--- a/.travis.yml
+++ b/.travis.yml
@@ -57,6 +57,6 @@ matrix:
       dist: trusty
       install: skip
       jdk: openjdk8
-      script: mvn test -Pcheck-test-suites -B -V
+      script: mvn test -Pcheck-test-suites,all-java,all-scala,scala -B -V
       before_cache: rm -rfv $HOME/.m2/repository/org/apache/ignite
       cache: { directories: [ $HOME/.m2/repository ] }
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/InlineIndexCompatibilityTest.java
new file mode 100644
index 0000000..dbdd2098
--- /dev/null
+++ b/modules/compatibility/src/test/java/org/apache/ignite/compatibility/persistence/InlineIndexCompatibilityTest.java
@@ -0,0 +1,359 @@
+/*
+ * 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.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;
+
+
+/**
+ * Tests that upgrade version on persisted inline index is successfull.
+ */
+@RunWith(Parameterized.class)
+public class InlineIndexCompatibilityTest extends IgnitePersistenceCompatibilityAbstractTest {
+    /** */
+    private static final String TEST_CACHE_NAME = InlineIndexCompatibilityTest.class.getSimpleName();
+
+    /** */
+    private static final int ROWS_CNT = 100;
+
+    /** Index to test. */
+    private static final String INDEX_NAME = "intval1_val_intval2";
+
+    /** Index to test with configured inline size. */
+    private static final String INDEX_SIZED_NAME = "intval1_val_intval2_sized";
+
+    /** Parametrized run param: Ignite version. */
+    @Parameterized.Parameter(0)
+    public String igniteVer;
+
+    /** Parametrized run param: Inline size is configured by user. */
+    @Parameterized.Parameter(1)
+    public boolean cfgInlineSize;
+
+    /** Test run configurations: Ignite version, Inline size configuration. */
+    @Parameterized.Parameters(name = "ver={0}, cfgInlineSize={1}")
+    public static Collection<Object[]> runConfig() {
+        return Arrays.asList(new Object[][] {
+            /** 2.6.0 is a last version where POJO inlining isn't enabled. */
+            {"2.6.0", false},
+            {"2.6.0", true},
+
+            {"2.7.0", false},
+            {"2.7.0", true},
+
+            {"2.7.6", false},
+            {"2.7.6", true},
+
+            {"2.8.0", false},
+            {"2.8.0", true},
+
+            {"2.8.1", false},
+            {"2.8.1", true},
+
+            {"2.9.0", false},
+            {"2.9.0", true},
+
+            {"2.9.1", false},
+            {"2.9.1", true}
+        });
+    }
+
+    /** */
+    @Test
+    public void testQueryOldInlinedIndex() throws Exception {
+        PostStartupClosure closure = cfgInlineSize ? new PostStartupClosureSized() : new PostStartupClosure();
+        String idxName = cfgInlineSize ? INDEX_SIZED_NAME : INDEX_NAME;
+
+        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.
+     *
+     * @param igniteVer 3-digits version of ignite
+     * @throws Exception If failed.
+     */
+    protected void doTestStartupWithOldVersion(String igniteVer, PostStartupClosure closure, String idxName) 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), idxName);
+        }
+        finally {
+            stopAllGrids();
+        }
+    }
+
+    /**
+     * Asserts cache contained all expected values as it was saved before.
+     *
+     * @param cache Cache to check.
+     * @param idxName Name of index to check.
+     */
+    private void validateResultingCacheData(IgniteCache<Object, Object> cache, String idxName) {
+        validateRandomRow(cache, idxName);
+        validateRandomRange(cache, idxName);
+    }
+
+    /** */
+    private void validateRandomRow(IgniteCache<Object, Object> cache, String idxName) {
+        int val = new Random().nextInt(ROWS_CNT);
+
+        // Select by quering complex index.
+        SqlFieldsQuery qry = new SqlFieldsQuery(
+            "SELECT * FROM \"" + TEST_CACHE_NAME + "\".EntityValueValue v " +
+                "WHERE v.intVal1 = ? and v.val = ? and v.intVal2 = ?;")
+            .setArgs(val, new EntityValue(val + 2), val + 1);
+
+        checkIndexUsed(cache, qry, idxName);
+
+        List<List<?>> result = cache.query(qry).getAll();
+
+        assertTrue(result.size() == 1);
+
+        List<?> row = result.get(0);
+
+        assertTrue(row.get(0).equals(new EntityValue(val + 2)));
+        assertTrue(row.get(1).equals(val));
+        assertTrue(row.get(2).equals(val + 1));
+    }
+
+    /** */
+    private void validateRandomRange(IgniteCache<Object, Object> cache, String idxName) {
+        int pivot = new Random().nextInt(ROWS_CNT);
+
+        // Select by quering complex index.
+        SqlFieldsQuery qry = new SqlFieldsQuery(
+            "SELECT * FROM \"" + TEST_CACHE_NAME + "\".EntityValueValue v " +
+                "WHERE v.intVal1 > ? and v.val > ? and v.intVal2 > ? " +
+                "ORDER BY v.val, v.intVal1, v.intVal2;")
+            .setArgs(pivot, new EntityValue(pivot), pivot);
+
+        checkIndexUsed(cache, qry, idxName);
+
+        List<List<?>> result = cache.query(qry).getAll();
+
+        // For strict comparison. There was an issues with >= comparison for some versions.
+        pivot += 1;
+
+        assertTrue(result.size() == ROWS_CNT - pivot);
+
+        for (int i = 0; i < ROWS_CNT - pivot; i++) {
+            List<?> row = result.get(i);
+
+            assertTrue(row.get(0).equals(new EntityValue(pivot + i + 2)));
+            assertTrue(row.get(1).equals(pivot + i));
+            assertTrue(row.get(2).equals(pivot + i + 1));
+        }
+    }
+
+    /** */
+    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) {
+            ignite.active(true);
+
+            CacheConfiguration<Object, Object> cacheCfg = new CacheConfiguration<>();
+            cacheCfg.setName(TEST_CACHE_NAME);
+            cacheCfg.setAtomicityMode(CacheAtomicityMode.TRANSACTIONAL);
+            cacheCfg.setBackups(1);
+
+            cacheCfg.setIndexedTypes(Integer.class, EntityValueValue.class);
+
+            IgniteCache<Object, Object> cache = ignite.createCache(cacheCfg);
+
+            saveCacheData(cache);
+
+            ignite.active(false);
+
+            try {
+                Thread.sleep(1_000);
+            }
+            catch (InterruptedException e) {
+                e.printStackTrace();
+            }
+        }
+
+        /**
+         * Create a complex index (int, pojo, int). Check that middle POJO object is correctly available from inline.
+         *
+         * @param cache to be filled with data. Results may be validated in {@link #validateResultingCacheData(IgniteCache, String)}.
+         */
+        protected void saveCacheData(IgniteCache<Object, Object> cache) {
+            for (int i = 0; i < ROWS_CNT; i++)
+                cache.put(i, new EntityValueValue(new EntityValue(i + 2), i, i + 1));
+
+            // Create index (int, pojo, int).
+            cache.query(new SqlFieldsQuery(
+                    "CREATE INDEX " + INDEX_NAME + " ON \"" + TEST_CACHE_NAME + "\".EntityValueValue " +
+                    "(intVal1, val, intVal2)")).getAll();
+        }
+    }
+
+    /** */
+    public static class PostStartupClosureSized extends PostStartupClosure {
+        /** {@inheritDoc} */
+        @Override protected void saveCacheData(IgniteCache<Object, Object> cache) {
+            for (int i = 0; i < ROWS_CNT; i++)
+                cache.put(i, new EntityValueValue(new EntityValue(i + 2), i, i + 1));
+
+            // Create index (int, pojo, int) with configured inline size.
+            cache.query(new SqlFieldsQuery(
+                "CREATE INDEX " + INDEX_SIZED_NAME + " ON \"" + TEST_CACHE_NAME + "\".EntityValueValue " +
+                    "(intVal1, val, intVal2) " +
+                    "INLINE_SIZE 100")).getAll();
+        }
+    }
+
+    /** POJO object aimed to be inlined. */
+    public static class EntityValue {
+        /** */
+        private final int val;
+
+        /** */
+        public EntityValue(int val) {
+            this.val = val;
+        }
+
+        /** {@inheritDoc} */
+        @Override public String toString() {
+            return "EV[value=" + val + "]";
+        }
+
+        /** {@inheritDoc} */
+        @Override public int hashCode() {
+            return 1 + val;
+        }
+
+        /** {@inheritDoc} */
+        @Override public boolean equals(Object other) {
+            return val == ((EntityValue) other).val;
+        }
+    }
+
+    /** Represents a cache value with 3 fields (POJO, int, int). */
+    public static class EntityValueValue {
+        /** */
+        @QuerySqlField
+        private final EntityValue val;
+
+        /** */
+        @QuerySqlField
+        private final int intVal1;
+
+        /** */
+        @QuerySqlField
+        private final int intVal2;
+
+        /** */
+        public EntityValueValue(EntityValue val, int val1, int val2) {
+            this.val = val;
+            intVal1 = val1;
+            intVal2 = val2;
+        }
+
+        /** {@inheritDoc} */
+        @Override public String toString() {
+            return "EVV[value=" + val + "]";
+        }
+    }
+}
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 4582168..115bf0b 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,6 +21,7 @@ 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.MetaStorageCompatibilityTest;
 import org.apache.ignite.compatibility.persistence.MigratingToWalV2SerializerWithCompactionTest;
 import org.apache.ignite.compatibility.persistence.MoveBinaryMetadataCompatibility;
@@ -34,6 +35,7 @@ import org.junit.runners.Suite;
 @RunWith(Suite.class)
 @Suite.SuiteClasses({
     PersistenceBasicCompatibilityTest.class,
+    InlineIndexCompatibilityTest.class,
     FoldersReuseCompatibilityTest.class,
     MigratingToWalV2SerializerWithCompactionTest.class,
     MetaStorageCompatibilityTest.class,
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/H2Tree.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/H2Tree.java
index 1abb8c3..0c18acd 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/H2Tree.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/H2Tree.java
@@ -24,6 +24,7 @@ import java.util.concurrent.atomic.AtomicInteger;
 import java.util.concurrent.atomic.AtomicLong;
 import java.util.stream.Collectors;
 import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteException;
 import org.apache.ignite.IgniteLogger;
 import org.apache.ignite.IgniteSystemProperties;
 import org.apache.ignite.SystemProperty;
@@ -251,14 +252,26 @@ public class H2Tree extends BPlusTree<H2Row, H2Row> {
 
             inlineSize = metaInfo.inlineSize();
 
+            setIos(
+                H2ExtrasInnerIO.getVersions(inlineSize, mvccEnabled),
+                H2ExtrasLeafIO.getVersions(inlineSize, mvccEnabled)
+            );
+
             List<InlineIndexColumn> inlineIdxs0 = getAvailableInlineColumns(affinityKey, cacheName, idxName, log, pk,
                 table, cols, factory, metaInfo.inlineObjectHash());
 
-            boolean inlineObjSupported = inlineSize > 0 && metaInfo.inlineObjectSupported();
+            boolean inlineObjSupported = inlineSize > 0 && inlineObjectSupported(metaInfo, inlineIdxs0);
+
+            if (inlineObjSupported)
+                inlineIdxs = inlineIdxs0;
+            else {
+                // If an index contains JO type and doesn't support inlining of it then use only prior columns.
+                int objIdx = 0;
+
+                for (; objIdx < inlineIdxs0.size() && inlineIdxs0.get(objIdx).type() != Value.JAVA_OBJECT; ++objIdx);
 
-            inlineIdxs = inlineObjSupported ? inlineIdxs0 : inlineIdxs0.stream()
-                .filter(ih -> ih.type() != Value.JAVA_OBJECT)
-                .collect(Collectors.toList());
+                inlineIdxs = inlineIdxs0.subList(0, objIdx);
+            }
 
             inlineCols = new IndexColumn[inlineIdxs.size()];
 
@@ -269,11 +282,6 @@ public class H2Tree extends BPlusTree<H2Row, H2Row> {
 
             if (!metaInfo.flagsSupported())
                 upgradeMetaPage(inlineObjSupported);
-
-            setIos(
-                H2ExtrasInnerIO.getVersions(inlineSize, mvccEnabled),
-                H2ExtrasLeafIO.getVersions(inlineSize, mvccEnabled)
-            );
         }
         else {
             unwrappedPk = true;
@@ -297,6 +305,35 @@ public class H2Tree extends BPlusTree<H2Row, H2Row> {
     }
 
     /**
+     * Find whether tree supports inlining objects or not.
+     *
+     * @param metaInfo Metapage info.
+     * @param inlineIdxs Base collection of index helpers.
+     * @return {@code true} if inline object is supported by exists tree.
+     */
+    private boolean inlineObjectSupported(MetaPageInfo metaInfo, List<InlineIndexColumn> inlineIdxs) {
+        if (metaInfo.flagsSupported())
+            return metaInfo.inlineObjectSupported();
+        else {
+            try {
+                if (InlineObjectBytesDetector.objectMayBeInlined(inlineSize, inlineIdxs)) {
+                    InlineObjectBytesDetector inlineObjDetector = new InlineObjectBytesDetector(
+                        inlineSize, inlineIdxs, tblName, idxName, log);
+
+                    findFirst(inlineObjDetector);
+
+                    return inlineObjDetector.inlineObjectSupported();
+                }
+                else
+                    return false;
+            }
+            catch (IgniteCheckedException e) {
+                throw new IgniteException("Unexpected exception on detect inline object", e);
+            }
+        }
+    }
+
+    /**
      * Return columns of the index.
      *
      * @return Indexed columns.
@@ -556,7 +593,7 @@ public class H2Tree extends BPlusTree<H2Row, H2Row> {
 
                 for (int i = 0; i < inlineIdxs.size(); i++) {
                     InlineIndexColumn inlineIdx = inlineIdxs.get(i);
-                    
+
                     Value v2 = row.getValue(inlineIdx.columnIndex());
 
                     if (v2 == null)
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/InlineObjectBytesDetector.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/InlineObjectBytesDetector.java
new file mode 100644
index 0000000..1064a73
--- /dev/null
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/InlineObjectBytesDetector.java
@@ -0,0 +1,196 @@
+/*
+ * 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.processors.query.h2.database;
+
+import java.util.List;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteLogger;
+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 detects whether tree contains inline JO type.
+ *
+ * When starting on old Ignite versions it's impossible to discover whether JO type was inlined or not.
+ * Then try to find that with 2 steps:
+ * 1. analyze of inline size;
+ * 2. traverse tree and check stored values.
+ */
+public class InlineObjectBytesDetector implements BPlusTree.TreeRowClosure<H2Row, H2Row> {
+    /** Inline size. */
+    private final int inlineSize;
+
+    /** Inline helpers. */
+    private final List<InlineIndexColumn> inlineCols;
+
+    /** Inline object supported flag. */
+    private boolean inlineObjSupported = true;
+
+    /** */
+    private final String tblName;
+
+    /** */
+    private final String idxName;
+
+    /** */
+    private final IgniteLogger log;
+
+    /**
+     * @param inlineSize Inline size.
+     * @param inlineCols Inline columns.
+     */
+    InlineObjectBytesDetector(int inlineSize, List<InlineIndexColumn> inlineCols, String tblName, String idxName,
+        IgniteLogger log) {
+        this.inlineSize = inlineSize;
+        this.inlineCols = inlineCols;
+        this.tblName = tblName;
+        this.idxName = idxName;
+        this.log = log;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean apply(BPlusTree<H2Row, H2Row> tree, BPlusIO<H2Row> io,
+        long pageAddr,
+        int idx) throws IgniteCheckedException {
+        H2Row r = tree.getRow(io, pageAddr, idx);
+
+        int off = io.offset(idx);
+
+        int fieldOff = 0;
+
+        boolean varLenPresents = false;
+
+        for (InlineIndexColumn ih : inlineCols) {
+            if (fieldOff >= inlineSize)
+                return false;
+
+            if (ih.type() != Value.JAVA_OBJECT) {
+                if (ih.size() < 0)
+                    varLenPresents = true;
+
+                fieldOff += ih.fullSize(pageAddr, off + fieldOff);
+
+                continue;
+            }
+
+            Value val = r.getValue(ih.columnIndex());
+
+            if (val == ValueNull.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) {
+                int len = PageUtils.getShort(pageAddr, off + fieldOff + 1);
+
+                len &= 0x7FFF;
+
+                byte[] originalObjBytes = val.getBytesNoCopy();
+
+                // Read size more then available space or more then origin length.
+                if (len > inlineSize - fieldOff - 3 || len > originalObjBytes.length) {
+                    inlineObjectSupportedDecision(false, "length is big " + len);
+
+                    return true;
+                }
+
+                // Try compare byte by byte for fully or partial inlined object.
+                byte[] inlineBytes = PageUtils.getBytes(pageAddr, off + fieldOff + 3, len);
+
+                for (int i = 0; i < len; i++) {
+                    if (inlineBytes[i] != originalObjBytes[i]) {
+                        inlineObjectSupportedDecision(false, i + " byte compare");
+
+                        return true;
+                    }
+                }
+
+                inlineObjectSupportedDecision(true, len + " bytes compared");
+
+                return true;
+            }
+
+            if (type == Value.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
+                return false;
+            }
+
+            inlineObjectSupportedDecision(false, "inline type " + type);
+
+            return true;
+        }
+
+        inlineObjectSupportedDecision(true, "no java objects for inlining");
+
+        return true;
+    }
+
+    /**
+     * @return {@code true} if inline object is supported on current tree.
+     */
+    public boolean inlineObjectSupported() {
+        return inlineObjSupported;
+    }
+
+    /**
+     * Static analyze inline_size and inline columns set.
+     * e.g.: indexed: (long, obj) and inline_size < 12.
+     * In this case there is no space for inline object.
+     *
+     * @param inlineCols Inline columns.
+     * @param inlineSize Inline size.
+     *
+     * @return {@code true} If the object may be inlined.
+     */
+    public static boolean objectMayBeInlined(int inlineSize, List<InlineIndexColumn> inlineCols) {
+        int remainSize = inlineSize;
+
+        for (InlineIndexColumn ih : inlineCols) {
+            if (ih.type() == Value.JAVA_OBJECT)
+                break;
+
+            // Set size to 1 for variable length columns as that value can be set by user.
+            remainSize -= ih.size() > 0 ? 1 + ih.size() : 1;
+        }
+
+        // For old versions JO type was inlined as byte array.
+        return remainSize >= 4;
+    }
+
+    /**
+     * @param inlineObjSupported {@code true} if inline object is supported on current tree.
+     * @param reason Reason why has been made decision.
+     */
+    private void inlineObjectSupportedDecision(boolean inlineObjSupported, String reason) {
+        this.inlineObjSupported = inlineObjSupported;
+
+        if (inlineObjSupported)
+            log.warning("Index supports JAVA_OBJECT type inlining [tblName=" + tblName + ", idxName=" +
+                idxName + ", reason='" + reason + "']");
+        else
+            log.warning("Index doesn't support JAVA_OBJECT type inlining [tblName=" + tblName + ", idxName=" +
+                idxName + ", reason='" + reason + "']");
+    }
+}