You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by am...@apache.org on 2020/03/11 13:08:19 UTC

[ignite] branch master updated: IGNITE-12740: Supports feature flags on index meta page (#7493)

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

amashenkov 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 009b0ee  IGNITE-12740: Supports feature flags on index meta page (#7493)
009b0ee is described below

commit 009b0eebe790916465bd00d643d6e3fb8f611c5c
Author: Taras Ledkov <tl...@gridgain.com>
AuthorDate: Wed Mar 11 16:08:00 2020 +0300

    IGNITE-12740: Supports feature flags on index meta page (#7493)
---
 .../internal/pagemem/wal/record/WALRecord.java     |   2 +-
 ...ageInitRootInlineFlagsCreatedVersionRecord.java |  98 ++++++++++++
 .../cache/persistence/tree/BPlusTree.java          |   6 +-
 .../cache/persistence/tree/io/BPlusMetaIO.java     | 175 ++++++++++++++++++---
 .../wal/serializer/RecordDataV1Serializer.java     |  56 +++++++
 .../apache/ignite/lang/IgniteProductVersion.java   |  14 +-
 .../processors/database/BPlusTreeSelfTest.java     |   2 +-
 .../processors/query/h2/database/H2Tree.java       | 150 +++++++++++++++---
 .../processors/query/h2/database/H2TreeIndex.java  |   7 +-
 .../query/h2/database/InlineIndexHelper.java       |   2 +-
 10 files changed, 458 insertions(+), 54 deletions(-)

diff --git a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/WALRecord.java b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/WALRecord.java
index 5a935a9..67106a9 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/WALRecord.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/WALRecord.java
@@ -224,7 +224,7 @@ public abstract class WALRecord {
         PARTITION_META_PAGE_UPDATE_COUNTERS_V2 (58, PHYSICAL),
 
         /** Init root meta page (with flags and created version) */
-        BTREE_META_PAGE_INIT_ROOT_V3(59),
+        BTREE_META_PAGE_INIT_ROOT_V3(59, PHYSICAL),
 
         /** Master key change record. */
         MASTER_KEY_CHANGE_RECORD(60, LOGICAL);
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/MetaPageInitRootInlineFlagsCreatedVersionRecord.java b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/MetaPageInitRootInlineFlagsCreatedVersionRecord.java
new file mode 100644
index 0000000..b1e0c34
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/MetaPageInitRootInlineFlagsCreatedVersionRecord.java
@@ -0,0 +1,98 @@
+/*
+ * 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.pagemem.wal.record.delta;
+
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.internal.IgniteVersionUtils;
+import org.apache.ignite.internal.pagemem.PageMemory;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.BPlusMetaIO;
+import org.apache.ignite.internal.util.typedef.internal.S;
+import org.apache.ignite.lang.IgniteProductVersion;
+
+/**
+ *
+ */
+public class MetaPageInitRootInlineFlagsCreatedVersionRecord extends MetaPageInitRootInlineRecord {
+    /** Created version. */
+    private final long flags;
+
+    /** Created version. */
+    private final IgniteProductVersion createdVer;
+
+    /**
+     * @param grpId Cache group ID.
+     * @param pageId Meta page ID.
+     * @param rootId Root id.
+     * @param inlineSize Inline size.
+     */
+    public MetaPageInitRootInlineFlagsCreatedVersionRecord(int grpId, long pageId, long rootId, int inlineSize) {
+        super(grpId, pageId, rootId, inlineSize);
+
+        createdVer = IgniteVersionUtils.VER;
+        flags = BPlusMetaIO.DEFAULT_FLAGS;
+    }
+
+    /**
+     * @param grpId Cache group ID.
+     * @param pageId Meta page ID.
+     * @param rootId Root id.
+     * @param inlineSize Inline size.
+     * @param flags Flags.
+     * @param createdVer The version of ignite that creates this tree.
+     */
+    public MetaPageInitRootInlineFlagsCreatedVersionRecord(int grpId, long pageId, long rootId, int inlineSize,
+        long flags, IgniteProductVersion createdVer) {
+        super(grpId, pageId, rootId, inlineSize);
+
+        this.flags = flags;
+        this.createdVer = createdVer;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void applyDelta(PageMemory pageMem, long pageAddr) throws IgniteCheckedException {
+        super.applyDelta(pageMem, pageAddr);
+
+        BPlusMetaIO io = BPlusMetaIO.VERSIONS.forPage(pageAddr);
+
+        io.initFlagsAndVersion(pageAddr, flags, createdVer);
+    }
+
+    /** {@inheritDoc} */
+    @Override public RecordType type() {
+        return RecordType.BTREE_META_PAGE_INIT_ROOT_V3;
+    }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return S.toString(MetaPageInitRootInlineFlagsCreatedVersionRecord.class, this, "super", super.toString());
+    }
+
+    /**
+     * @return Created version.
+     */
+    public IgniteProductVersion createdVersion() {
+        return createdVer;
+    }
+
+    /**
+     * @return Meta page flags.
+     */
+    public long flags() {
+        return flags;
+    }
+}
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/tree/BPlusTree.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/tree/BPlusTree.java
index 1095243..7904460 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/tree/BPlusTree.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/tree/BPlusTree.java
@@ -33,6 +33,7 @@ import org.apache.ignite.IgniteSystemProperties;
 import org.apache.ignite.failure.FailureContext;
 import org.apache.ignite.failure.FailureType;
 import org.apache.ignite.internal.IgniteInterruptedCheckedException;
+import org.apache.ignite.internal.IgniteVersionUtils;
 import org.apache.ignite.internal.UnregisteredBinaryTypeException;
 import org.apache.ignite.internal.UnregisteredClassException;
 import org.apache.ignite.internal.metric.IoStatisticsHolder;
@@ -46,7 +47,7 @@ import org.apache.ignite.internal.pagemem.wal.record.delta.FixRemoveId;
 import org.apache.ignite.internal.pagemem.wal.record.delta.InsertRecord;
 import org.apache.ignite.internal.pagemem.wal.record.delta.MetaPageAddRootRecord;
 import org.apache.ignite.internal.pagemem.wal.record.delta.MetaPageCutRootRecord;
-import org.apache.ignite.internal.pagemem.wal.record.delta.MetaPageInitRootInlineRecord;
+import org.apache.ignite.internal.pagemem.wal.record.delta.MetaPageInitRootInlineFlagsCreatedVersionRecord;
 import org.apache.ignite.internal.pagemem.wal.record.delta.NewRootInitRecord;
 import org.apache.ignite.internal.pagemem.wal.record.delta.RemoveRecord;
 import org.apache.ignite.internal.pagemem.wal.record.delta.ReplaceRecord;
@@ -725,9 +726,10 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
 
             io.initRoot(pageAddr, rootId, pageSize());
             io.setInlineSize(pageAddr, inlineSize);
+            io.initFlagsAndVersion(pageAddr, BPlusMetaIO.DEFAULT_FLAGS, IgniteVersionUtils.VER);
 
             if (needWalDeltaRecord(metaId, metaPage, walPlc))
-                wal.log(new MetaPageInitRootInlineRecord(cacheId, metaId, rootId, inlineSize));
+                wal.log(new MetaPageInitRootInlineFlagsCreatedVersionRecord(cacheId, metaId, rootId, inlineSize));
 
             assert io.getRootLevel(pageAddr) == 0;
             assert io.getFirstPageId(pageAddr, 0) == rootId;
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/tree/io/BPlusMetaIO.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/tree/io/BPlusMetaIO.java
index fef58fd..006a130 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/tree/io/BPlusMetaIO.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/tree/io/BPlusMetaIO.java
@@ -19,8 +19,10 @@ package org.apache.ignite.internal.processors.cache.persistence.tree.io;
 
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.IgniteException;
+import org.apache.ignite.internal.IgniteVersionUtils;
 import org.apache.ignite.internal.pagemem.PageUtils;
 import org.apache.ignite.internal.util.GridStringBuilder;
+import org.apache.ignite.lang.IgniteProductVersion;
 
 /**
  * IO routines for B+Tree meta pages.
@@ -28,20 +30,42 @@ import org.apache.ignite.internal.util.GridStringBuilder;
 public class BPlusMetaIO extends PageIO {
     /** */
     public static final IOVersions<BPlusMetaIO> VERSIONS = new IOVersions<>(
-        new BPlusMetaIO(1), new BPlusMetaIO(2), new BPlusMetaIO(3)
+        new BPlusMetaIO(1),
+        new BPlusMetaIO(2),
+
+        // Unwrapped PK & inline POJO
+        new BPlusMetaIO(3),
+
+        // Add feature flags.
+        new BPlusMetaIO(4)
     );
 
     /** */
-    private static final int LVLS_OFF = COMMON_HEADER_END;
+    private static final int LVLS_OFFSET = COMMON_HEADER_END;
 
     /** */
-    private final int refsOff;
+    private static final int INLINE_SIZE_OFFSET = LVLS_OFFSET + 1;
+
+    /** */
+    private static final int FLAGS_OFFSET = INLINE_SIZE_OFFSET + 2;
+
+    /** */
+    private static final int CREATED_VER_OFFSET = FLAGS_OFFSET + 8;
+
+    /** */
+    private static final int REFS_OFFSET = CREATED_VER_OFFSET + IgniteProductVersion.SIZE_IN_BYTES;
 
     /** */
-    private final int inlineSizeOff;
+    private static final long FLAG_UNWRAPPED_PK = 1L;
 
     /** */
-    private boolean unwrappedPk;
+    private static final long FLAG_INLINE_OBJECT_SUPPORTED = 2L;
+
+    /** */
+    public static final long DEFAULT_FLAGS = FLAG_UNWRAPPED_PK | FLAG_INLINE_OBJECT_SUPPORTED ;
+
+    /** */
+    private final int refsOff;
 
     /**
      * @param ver Page format version.
@@ -51,21 +75,19 @@ public class BPlusMetaIO extends PageIO {
 
         switch (ver) {
             case 1:
-                inlineSizeOff = -1;
-                refsOff = LVLS_OFF + 1;
-                unwrappedPk = false;
+                refsOff = LVLS_OFFSET + 1;
                 break;
 
             case 2:
-                inlineSizeOff = LVLS_OFF + 1;
-                refsOff = inlineSizeOff + 2;
-                unwrappedPk = false;
+                refsOff = INLINE_SIZE_OFFSET + 2;
                 break;
 
             case 3:
-                inlineSizeOff = LVLS_OFF + 1;
-                refsOff = inlineSizeOff + 2;
-                unwrappedPk = true;
+                refsOff = INLINE_SIZE_OFFSET + 2;
+                break;
+
+            case 4:
+                refsOff = REFS_OFFSET;
                 break;
 
             default:
@@ -88,7 +110,7 @@ public class BPlusMetaIO extends PageIO {
      * @return Number of levels in this tree.
      */
     public int getLevelsCount(long pageAddr) {
-        return Byte.toUnsignedInt(PageUtils.getByte(pageAddr, LVLS_OFF));
+        return Byte.toUnsignedInt(PageUtils.getByte(pageAddr, LVLS_OFFSET));
     }
 
     /**
@@ -108,7 +130,7 @@ public class BPlusMetaIO extends PageIO {
     private void setLevelsCount(long pageAddr, int lvls, int pageSize) {
         assert lvls >= 0 && lvls <= getMaxLevels(pageAddr, pageSize) : lvls;
 
-        PageUtils.putByte(pageAddr, LVLS_OFF, (byte)lvls);
+        PageUtils.putByte(pageAddr, LVLS_OFFSET, (byte)lvls);
 
         assert getLevelsCount(pageAddr) == lvls;
     }
@@ -183,21 +205,104 @@ public class BPlusMetaIO extends PageIO {
      */
     public void setInlineSize(long pageAddr, int size) {
         if (getVersion() > 1)
-            PageUtils.putShort(pageAddr, inlineSizeOff, (short)size);
+            PageUtils.putShort(pageAddr, INLINE_SIZE_OFFSET, (short)size);
     }
 
     /**
      * @param pageAddr Page address.
+     * @return Inline size.
      */
     public int getInlineSize(long pageAddr) {
-        return getVersion() > 1 ? PageUtils.getShort(pageAddr, inlineSizeOff) : 0;
+        return getVersion() > 1 ? PageUtils.getShort(pageAddr, INLINE_SIZE_OFFSET) : 0;
     }
 
     /**
      * @return {@code true} In case use unwrapped PK.
      */
-    public boolean unwrappedPk() {
-        return unwrappedPk;
+    public boolean unwrappedPk(long pageAddr) {
+        return supportFlags() && (flags(pageAddr) & FLAG_UNWRAPPED_PK) != 0L || getVersion() == 3;
+    }
+
+    /**
+     * @param pageAddr Page address.
+     * @return {@code true} In case inline object is supported by the tree.
+     */
+    public boolean inlineObjectSupported(long pageAddr) {
+        assert supportFlags();
+
+        return (flags(pageAddr) & FLAG_INLINE_OBJECT_SUPPORTED) != 0L;
+    }
+
+    /**
+     * @return {@code true} If flags are supported.
+     */
+    public boolean supportFlags() {
+        return getVersion() > 3;
+    }
+
+    /**
+     * @param pageAddr Page address.
+     * @param flags Flags.
+     * @param createdVer The version of the product that creates the page (b+tree).
+     */
+    public void initFlagsAndVersion(long pageAddr, long flags, IgniteProductVersion createdVer) {
+        PageUtils.putLong(pageAddr, FLAGS_OFFSET, flags);
+
+        setCreatedVersion(pageAddr, createdVer);
+    }
+
+    /**
+     * @param pageAddr Page address.
+     * @param curVer Ignite current version.
+     */
+    public void setCreatedVersion(long pageAddr, IgniteProductVersion curVer) {
+        assert curVer != null;
+
+        PageUtils.putByte(pageAddr, CREATED_VER_OFFSET, curVer.major());
+        PageUtils.putByte(pageAddr, CREATED_VER_OFFSET + 1, curVer.minor());
+        PageUtils.putByte(pageAddr, CREATED_VER_OFFSET + 2, curVer.maintenance());
+        PageUtils.putLong(pageAddr, CREATED_VER_OFFSET + 3, curVer.revisionTimestamp());
+        PageUtils.putBytes(pageAddr, CREATED_VER_OFFSET + 11, curVer.revisionHash());
+    }
+
+    /**
+     * @param pageAddr Page address.
+     * @return The version of product that creates the page.
+     */
+    public IgniteProductVersion createdVersion(long pageAddr) {
+        if (getVersion() < 4)
+            return null;
+
+        return new IgniteProductVersion(
+            PageUtils.getByte(pageAddr, CREATED_VER_OFFSET),
+            PageUtils.getByte(pageAddr, CREATED_VER_OFFSET + 1),
+            PageUtils.getByte(pageAddr, CREATED_VER_OFFSET + 2),
+            PageUtils.getLong(pageAddr, CREATED_VER_OFFSET + 3),
+            PageUtils.getBytes(pageAddr, CREATED_VER_OFFSET + 11, IgniteProductVersion.REV_HASH_SIZE));
+    }
+
+    /**
+     * @param pageAddr Page address.
+     * @return Long with flags.
+     */
+    private long flags(long pageAddr) {
+        assert supportFlags();
+
+        return PageUtils.getLong(pageAddr, FLAGS_OFFSET);
+    }
+
+    /**
+     * @param pageAddr Page address.
+     * @param unwrappedPk unwrapped primary key of this tree flag.
+     * @param inlineObjSupported inline POJO by created tree flag.
+     */
+    public void setFlags(long pageAddr, boolean unwrappedPk, boolean inlineObjSupported) {
+        assert supportFlags();
+
+        long flags = unwrappedPk ? FLAG_UNWRAPPED_PK : 0;
+        flags |= inlineObjSupported ? FLAG_INLINE_OBJECT_SUPPORTED : 0;
+
+        PageUtils.putLong(pageAddr, FLAGS_OFFSET, flags);
     }
 
     /** {@inheritDoc} */
@@ -209,4 +314,34 @@ public class BPlusMetaIO extends PageIO {
         ;
             //TODO print firstPageIds by level
     }
+
+    /**
+     * @param pageAddr Page address.
+     * @param inlineObjSupported Supports inline object flag.
+     * @param unwrappedPk Unwrap PK flag.
+     * @param pageSize Page size.
+     */
+    public static void upgradePageVersion(long pageAddr, boolean inlineObjSupported, boolean unwrappedPk, int pageSize) {
+        BPlusMetaIO ioPrev = VERSIONS.forPage(pageAddr);
+
+        long[] lvls = new long[ioPrev.getLevelsCount(pageAddr)];
+
+        for (int i = 0; i < lvls.length; ++i)
+            lvls[i] = ioPrev.getFirstPageId(pageAddr, i);
+
+        int inlineSize = ioPrev.getInlineSize(pageAddr);
+
+        BPlusMetaIO ioNew = VERSIONS.latest();
+
+        setVersion(pageAddr, VERSIONS.latest().getVersion());
+
+        ioNew.setLevelsCount(pageAddr, lvls.length, pageSize);
+
+        for (int i = 0; i < lvls.length; ++i)
+            ioNew.setFirstPageId(pageAddr, i, lvls[i]);
+
+        ioNew.setInlineSize(pageAddr, inlineSize);
+        ioNew.setCreatedVersion(pageAddr, IgniteVersionUtils.VER);
+        ioNew.setFlags(pageAddr, unwrappedPk, inlineObjSupported);
+    }
 }
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/serializer/RecordDataV1Serializer.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/serializer/RecordDataV1Serializer.java
index 4077c15..829ef58 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/serializer/RecordDataV1Serializer.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/serializer/RecordDataV1Serializer.java
@@ -65,6 +65,7 @@ import org.apache.ignite.internal.pagemem.wal.record.delta.MergeRecord;
 import org.apache.ignite.internal.pagemem.wal.record.delta.MetaPageAddRootRecord;
 import org.apache.ignite.internal.pagemem.wal.record.delta.MetaPageCutRootRecord;
 import org.apache.ignite.internal.pagemem.wal.record.delta.MetaPageInitRecord;
+import org.apache.ignite.internal.pagemem.wal.record.delta.MetaPageInitRootInlineFlagsCreatedVersionRecord;
 import org.apache.ignite.internal.pagemem.wal.record.delta.MetaPageInitRootInlineRecord;
 import org.apache.ignite.internal.pagemem.wal.record.delta.MetaPageInitRootRecord;
 import org.apache.ignite.internal.pagemem.wal.record.delta.MetaPageUpdateLastAllocatedIndex;
@@ -110,6 +111,7 @@ import org.apache.ignite.internal.util.typedef.T2;
 import org.apache.ignite.internal.util.typedef.T3;
 import org.apache.ignite.internal.util.typedef.internal.CU;
 import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.lang.IgniteProductVersion;
 import org.apache.ignite.spi.encryption.EncryptionSpi;
 import org.apache.ignite.spi.encryption.noop.NoopEncryptionSpi;
 import org.jetbrains.annotations.Nullable;
@@ -441,6 +443,9 @@ public class RecordDataV1Serializer implements RecordDataSerializer {
             case BTREE_META_PAGE_INIT_ROOT2:
                 return 4 + 8 + 8 + 2;
 
+            case BTREE_META_PAGE_INIT_ROOT_V3:
+                return 4 + 8 + 8 + 2 + 8 + IgniteProductVersion.SIZE_IN_BYTES;
+
             case BTREE_META_PAGE_ADD_ROOT:
                 return 4 + 8 + 8;
 
@@ -833,6 +838,34 @@ public class RecordDataV1Serializer implements RecordDataSerializer {
 
                 break;
 
+            case BTREE_META_PAGE_INIT_ROOT_V3:
+                cacheId = in.readInt();
+                pageId = in.readLong();
+
+                long rootId3 = in.readLong();
+                int inlineSize3 = in.readShort();
+
+                long flags = in.readLong();
+
+                byte[] revHash = new byte[IgniteProductVersion.REV_HASH_SIZE];
+                byte maj = in.readByte();
+                byte min = in.readByte();
+                byte maint = in.readByte();
+                long verTs = in.readLong();
+                in.readFully(revHash);
+
+                IgniteProductVersion createdVer = new IgniteProductVersion(
+                    maj,
+                    min,
+                    maint,
+                    verTs,
+                    revHash);
+
+                res = new MetaPageInitRootInlineFlagsCreatedVersionRecord(cacheId, pageId, rootId3,
+                    inlineSize3, flags, createdVer);
+
+                break;
+
             case BTREE_META_PAGE_ADD_ROOT:
                 cacheId = in.readInt();
                 pageId = in.readLong();
@@ -1431,6 +1464,29 @@ public class RecordDataV1Serializer implements RecordDataSerializer {
                 buf.putShort((short)imRec2.inlineSize());
                 break;
 
+            case BTREE_META_PAGE_INIT_ROOT_V3:
+                MetaPageInitRootInlineFlagsCreatedVersionRecord imRec3 =
+                    (MetaPageInitRootInlineFlagsCreatedVersionRecord)rec;
+
+                buf.putInt(imRec3.groupId());
+                buf.putLong(imRec3.pageId());
+
+                buf.putLong(imRec3.rootId());
+
+                buf.putShort((short)imRec3.inlineSize());
+
+                buf.putLong(imRec3.flags());
+
+                // Write created version.
+                IgniteProductVersion createdVer = imRec3.createdVersion();
+                buf.put(createdVer.major());
+                buf.put(createdVer.minor());
+                buf.put(createdVer.maintenance());
+                buf.putLong(createdVer.revisionTimestamp());
+                buf.put(createdVer.revisionHash());
+
+                break;
+
             case BTREE_META_PAGE_ADD_ROOT:
                 MetaPageAddRootRecord arRec = (MetaPageAddRootRecord)rec;
 
diff --git a/modules/core/src/main/java/org/apache/ignite/lang/IgniteProductVersion.java b/modules/core/src/main/java/org/apache/ignite/lang/IgniteProductVersion.java
index 82a7758..1c78694 100644
--- a/modules/core/src/main/java/org/apache/ignite/lang/IgniteProductVersion.java
+++ b/modules/core/src/main/java/org/apache/ignite/lang/IgniteProductVersion.java
@@ -41,6 +41,12 @@ public class IgniteProductVersion implements Comparable<IgniteProductVersion>, E
     /** */
     private static final long serialVersionUID = 0L;
 
+    /** Size of the {@link #revHash }*/
+    public static final int REV_HASH_SIZE = 20;
+
+    /** Size in bytes of serialized: 3 bytes (maj, min, maintenance version), 8 bytes - timestamp */
+    public static final int SIZE_IN_BYTES = 3 + 8 + REV_HASH_SIZE;
+
     /** Regexp parse pattern. */
     private static final Pattern VER_PATTERN =
         Pattern.compile("(\\d+)\\.(\\d+)\\.(\\d+)([-.]([^0123456789][^-]+)(-SNAPSHOT)?)?(-(\\d+))?(-([\\da-f]+))?");
@@ -90,15 +96,17 @@ public class IgniteProductVersion implements Comparable<IgniteProductVersion>, E
      * @param revHash Revision hash.
      */
     public IgniteProductVersion(byte major, byte minor, byte maintenance, String stage, long revTs, byte[] revHash) {
-        if (revHash != null && revHash.length != 20)
-            throw new IllegalArgumentException("Invalid length for SHA1 hash (must be 20): " + revHash.length);
+        if (revHash != null && revHash.length != REV_HASH_SIZE) {
+            throw new IllegalArgumentException("Invalid length for SHA1 hash (must be "
+                + REV_HASH_SIZE + "): " + revHash.length);
+        }
 
         this.major = major;
         this.minor = minor;
         this.maintenance = maintenance;
         this.stage = stage;
         this.revTs = revTs;
-        this.revHash = revHash != null ? revHash : new byte[20];
+        this.revHash = revHash != null ? revHash : new byte[REV_HASH_SIZE];
     }
 
     /**
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/database/BPlusTreeSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/database/BPlusTreeSelfTest.java
index b11376c..e4696ca 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/database/BPlusTreeSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/database/BPlusTreeSelfTest.java
@@ -103,7 +103,7 @@ public class BPlusTreeSelfTest extends GridCommonAbstractTest {
     private static final short LONG_LEAF_IO = 30001;
 
     /** */
-    protected static final int PAGE_SIZE = 256;
+    protected static final int PAGE_SIZE = 512;
 
     /** */
     protected static final long MB = 1024 * 1024;
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 93fe93f..ab9c519c 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
@@ -28,9 +28,11 @@ import org.apache.ignite.IgniteLogger;
 import org.apache.ignite.IgniteSystemProperties;
 import org.apache.ignite.failure.FailureType;
 import org.apache.ignite.internal.metric.IoStatisticsHolder;
+import org.apache.ignite.internal.pagemem.FullPageId;
 import org.apache.ignite.internal.pagemem.PageIdUtils;
 import org.apache.ignite.internal.pagemem.PageMemory;
 import org.apache.ignite.internal.pagemem.wal.IgniteWriteAheadLogManager;
+import org.apache.ignite.internal.pagemem.wal.record.PageSnapshot;
 import org.apache.ignite.internal.processors.cache.GridCacheContext;
 import org.apache.ignite.internal.processors.cache.mvcc.MvccUtils;
 import org.apache.ignite.internal.processors.cache.persistence.CacheDataRowAdapter;
@@ -48,8 +50,11 @@ import org.apache.ignite.internal.processors.query.h2.database.io.H2RowLinkIO;
 import org.apache.ignite.internal.processors.query.h2.opt.GridH2Table;
 import org.apache.ignite.internal.processors.query.h2.opt.H2CacheRow;
 import org.apache.ignite.internal.processors.query.h2.opt.H2Row;
+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.apache.ignite.lang.IgnitePredicate;
+import org.apache.ignite.lang.IgniteProductVersion;
 import org.h2.result.SearchRow;
 import org.h2.table.IndexColumn;
 import org.h2.value.Value;
@@ -191,6 +196,15 @@ public class H2Tree extends BPlusTree<H2Row, H2Row> {
         this.cctx = cctx;
         this.table = table;
         this.stats = stats;
+        this.log = log;
+        this.rowCache = rowCache;
+        this.idxName = idxName;
+        this.cacheName = cacheName;
+        this.tblName = tblName;
+        this.maxCalculatedInlineSize = maxCalculatedInlineSize;
+        this.pk = pk;
+        this.affinityKey = affinityKey;
+        this.mvccEnabled = mvccEnabled;
 
         if (!initNew) {
             // Page is ready - read meta information.
@@ -199,34 +213,38 @@ public class H2Tree extends BPlusTree<H2Row, H2Row> {
             inlineSize = metaInfo.inlineSize();
 
             unwrappedPk = metaInfo.useUnwrappedPk();
-        }
-        else {
-            unwrappedPk = true;
 
-            inlineSize = unwrappedColsInfo.inlineSize();
-        }
+            setIos(
+                H2ExtrasInnerIO.getVersions(inlineSize, mvccEnabled),
+                H2ExtrasLeafIO.getVersions(inlineSize, mvccEnabled));
 
-        this.idxName = idxName;
-        this.cacheName = cacheName;
-        this.tblName = tblName;
+            List<InlineIndexHelper> inlineIdxs0 = unwrappedPk ? unwrappedColsInfo.inlineIdx()
+                : wrappedColsInfo.inlineIdx();
 
-        this.maxCalculatedInlineSize = maxCalculatedInlineSize;
+            boolean inlineObjSupported = inlineSize > 0 && metaInfo.inlineObjectSupported();
 
-        this.pk = pk;
-        this.affinityKey = affinityKey;
+            inlineIdxs = inlineObjSupported ? inlineIdxs0 : inlineIdxs0.stream()
+                .filter(ih -> ih.type() != Value.JAVA_OBJECT)
+                .collect(Collectors.toList());
 
-        this.mvccEnabled = mvccEnabled;
+            if (!metaInfo.flagsSupported())
+                upgradeMetaPage(inlineObjSupported);
+        }
+        else {
+            unwrappedPk = true;
 
-        inlineIdxs = unwrappedPk ? unwrappedColsInfo.inlineIdx() : wrappedColsInfo.inlineIdx();
-        cols = unwrappedPk ? unwrappedColsInfo.cols() : wrappedColsInfo.cols();
+            inlineSize = unwrappedColsInfo.inlineSize();
 
-        setIos(H2ExtrasInnerIO.getVersions(inlineSize, mvccEnabled), H2ExtrasLeafIO.getVersions(inlineSize, mvccEnabled));
+            inlineIdxs = unwrappedColsInfo.inlineIdx();
 
-        this.rowCache = rowCache;
+            setIos(
+                H2ExtrasInnerIO.getVersions(inlineSize, mvccEnabled),
+                H2ExtrasLeafIO.getVersions(inlineSize, mvccEnabled));
 
-        this.log = log;
+            initTree(initNew, inlineSize);
+        }
 
-        initTree(initNew, inlineSize);
+        cols = unwrappedPk ? unwrappedColsInfo.cols() : wrappedColsInfo.cols();
 
         created = initNew;
     }
@@ -354,7 +372,7 @@ public class H2Tree extends BPlusTree<H2Row, H2Row> {
             try {
                 BPlusMetaIO io = BPlusMetaIO.VERSIONS.forPage(pageAddr);
 
-                return new MetaPageInfo(io.getInlineSize(pageAddr), io.unwrappedPk());
+                return new MetaPageInfo(io, pageAddr);
             }
             finally {
                 readUnlock(metaPageId, metaPage, pageAddr);
@@ -365,6 +383,38 @@ public class H2Tree extends BPlusTree<H2Row, H2Row> {
         }
     }
 
+    /**
+     * 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);
+        }
+    }
+
     /** {@inheritDoc} */
     @SuppressWarnings("ForLoopReplaceableByForEach")
     @Override protected int compare(BPlusIO<H2Row> io, long pageAddr, int idx,
@@ -599,6 +649,29 @@ public class H2Tree extends BPlusTree<H2Row, H2Row> {
     }
 
     /**
+     * @return Inline indexes for the segment.
+     */
+    public List<InlineIndexHelper> inlineIndexes() {
+        return inlineIdxs;
+    }
+
+    /**
+     * @param idxs Full set of inline helpers.
+     */
+    public void refreshColumnIds(List<InlineIndexHelper> idxs) {
+        assert inlineIdxs.size() <= idxs.size();
+
+        for (int i = 0; i < inlineIdxs.size(); ++i) {
+            final int idx = i;
+
+            inlineIdxs.set(idx, F.find(idxs, null,
+                (IgnitePredicate<InlineIndexHelper>)ih -> ih.colName().equals(inlineIdxs.get(idx).colName())));
+
+            assert inlineIdxs.get(idx) != null;
+        }
+    }
+
+    /**
      *
      */
     private static class MetaPageInfo {
@@ -608,13 +681,28 @@ public class H2Tree extends BPlusTree<H2Row, H2Row> {
         /** */
         boolean useUnwrappedPk;
 
+        /** */
+        boolean flagsSupported;
+
+        /** */
+        Boolean inlineObjectSupported;
+
+        /** */
+        IgniteProductVersion createdVer;
+
         /**
-         * @param inlineSize Inline size.
-         * @param useUnwrappedPk {@code true} In case use unwrapped PK for indexes.
+         * @param io Metapage IO.
+         * @param pageAddr Page address.
          */
-        public MetaPageInfo(int inlineSize, boolean useUnwrappedPk) {
-            this.inlineSize = inlineSize;
-            this.useUnwrappedPk = useUnwrappedPk;
+        public MetaPageInfo(BPlusMetaIO io, long pageAddr) {
+            inlineSize = io.getInlineSize(pageAddr);
+            useUnwrappedPk = io.unwrappedPk(pageAddr);
+            flagsSupported = io.supportFlags();
+
+            if (io.getVersion() >= 3)
+                inlineObjectSupported = io.inlineObjectSupported(pageAddr);
+
+            createdVer = io.createdVersion(pageAddr);
         }
 
         /**
@@ -630,6 +718,20 @@ public class H2Tree extends BPlusTree<H2Row, H2Row> {
         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 inlineObjectSupported;
+        }
     }
 
     /**
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/H2TreeIndex.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/H2TreeIndex.java
index 59bc509..2b57e2b 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/H2TreeIndex.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/H2TreeIndex.java
@@ -472,12 +472,12 @@ public class H2TreeIndex extends H2TreeIndexBase {
     /** {@inheritDoc} */
     @Override public boolean putx(H2CacheRow row) {
         try {
-            InlineIndexHelper.setCurrentInlineIndexes(inlineIdxs);
-
             int seg = segmentForRow(cctx, row);
 
             H2Tree tree = treeForRead(seg);
 
+            InlineIndexHelper.setCurrentInlineIndexes(tree.inlineIndexes());
+
             assert cctx.shared().database().checkpointLockIsHeldByThread();
 
             return tree.putx(row);
@@ -690,6 +690,9 @@ public class H2TreeIndex extends H2TreeIndexBase {
 
         for (int pos = 0; pos < inlineHelpers.size(); ++pos)
             inlineIdxs.set(pos, inlineHelpers.get(pos));
+
+        for (H2Tree seg : segments)
+            seg.refreshColumnIds(inlineIdxs);
     }
 
     /** {@inheritDoc} */
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/InlineIndexHelper.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/InlineIndexHelper.java
index 1defe43..e6bc589 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/InlineIndexHelper.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/InlineIndexHelper.java
@@ -248,7 +248,7 @@ public class InlineIndexHelper {
         if (size > 0)
             return size + 1;
         else
-            return PageUtils.getShort(pageAddr, off + 1) + 3;
+            return (PageUtils.getShort(pageAddr, off + 1) & 0x7FFF) + 3;
     }
 
     /**