You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by ag...@apache.org on 2020/02/13 23:02:29 UTC

[ignite] branch master updated: IGNITE-12638 Distributed meta storage classes are IgniteDataTransferObjects now.

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

agura 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 ffde253  IGNITE-12638 Distributed meta storage classes are IgniteDataTransferObjects now.
ffde253 is described below

commit ffde253144dfb28411a2ded012291756cd447a2d
Author: ibessonov <be...@gmail.com>
AuthorDate: Fri Feb 14 01:56:26 2020 +0300

    IGNITE-12638 Distributed meta storage classes are IgniteDataTransferObjects now.
    
    Signed-off-by: Andrey Gura <ag...@apache.org>
---
 .../DistributedMetaStorageHistoryItem.java         | 76 ++++++++++++++++++----
 .../persistence/DistributedMetaStorageImpl.java    | 64 +++++++++---------
 .../persistence/DistributedMetaStorageVersion.java | 46 +++++++++++--
 .../persistence/DmsDataWriterWorker.java           | 24 +++----
 ...InMemoryCachedDistributedMetaStorageBridge.java | 10 +--
 .../persistence/DmsDataWriterWorkerTest.java       |  6 +-
 6 files changed, 158 insertions(+), 68 deletions(-)

diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/metastorage/persistence/DistributedMetaStorageHistoryItem.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/metastorage/persistence/DistributedMetaStorageHistoryItem.java
index 6a4ec6f..27e5f15 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/metastorage/persistence/DistributedMetaStorageHistoryItem.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/metastorage/persistence/DistributedMetaStorageHistoryItem.java
@@ -17,14 +17,17 @@
 
 package org.apache.ignite.internal.processors.metastorage.persistence;
 
-import java.io.Serializable;
+import java.io.IOException;
+import java.io.ObjectInput;
+import java.io.ObjectOutput;
 import java.util.Arrays;
+import org.apache.ignite.internal.dto.IgniteDataTransferObject;
 import org.apache.ignite.internal.util.tostring.GridToStringInclude;
 import org.apache.ignite.internal.util.typedef.internal.S;
+import org.apache.ignite.internal.util.typedef.internal.U;
 
 /** */
-@SuppressWarnings("PublicField")
-final class DistributedMetaStorageHistoryItem implements Serializable {
+final class DistributedMetaStorageHistoryItem extends IgniteDataTransferObject {
     /** */
     private static final long serialVersionUID = 0L;
 
@@ -33,25 +36,34 @@ final class DistributedMetaStorageHistoryItem implements Serializable {
 
     /** */
     @GridToStringInclude
-    public final String[] keys;
+    private String[] keys;
 
     /** */
     @GridToStringInclude
-    public final byte[][] valBytesArray;
+    private byte[][] valBytesArr;
 
     /** */
     private transient long longHash;
 
+    /** Default constructor for deserialization. */
+    public DistributedMetaStorageHistoryItem() {
+        // No-op.
+    }
+
     /** */
     public DistributedMetaStorageHistoryItem(String key, byte[] valBytes) {
         keys = new String[] {key};
-        valBytesArray = new byte[][] {valBytes};
+        valBytesArr = new byte[][] {valBytes};
     }
 
     /** */
-    public DistributedMetaStorageHistoryItem(String[] keys, byte[][] valBytesArray) {
+    @SuppressWarnings("AssignmentOrReturnOfFieldWithMutableType")
+    public DistributedMetaStorageHistoryItem(String[] keys, byte[][] valBytesArr) {
+        assert keys.length > 0;
+        assert keys.length == valBytesArr.length;
+
         this.keys = keys;
-        this.valBytesArray = valBytesArray;
+        this.valBytesArr = valBytesArr;
     }
 
     /** */
@@ -63,11 +75,51 @@ final class DistributedMetaStorageHistoryItem implements Serializable {
 
         // String encoding is ignored to make estimation faster. 2 "size" values added as well.
         for (int i = 0; i < len; i++)
-            size += keys[i].length() * 2 + (valBytesArray[i] == null ? 0 : valBytesArray[i].length);
+            size += keys[i].length() * 2 + (valBytesArr[i] == null ? 0 : valBytesArr[i].length);
 
         return size;
     }
 
+    /**
+     * Array of keys modified in this update.
+     */
+    public String[] keys() {
+        //noinspection AssignmentOrReturnOfFieldWithMutableType
+        return keys;
+    }
+
+    /**
+     * Array of serialized values corresponded to {@link #keys()} in the same order.
+     */
+    public byte[][] valuesBytesArray() {
+        //noinspection AssignmentOrReturnOfFieldWithMutableType
+        return valBytesArr;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void writeExternalData(ObjectOutput out) throws IOException {
+        out.writeInt(keys.length);
+
+        for (int i = 0; i < keys.length; i++) {
+            U.writeString(out, keys[i]);
+
+            U.writeByteArray(out, valBytesArr[i]);
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void readExternalData(byte protoVer, ObjectInput in) throws IOException {
+        int len = in.readInt();
+
+        keys = new String[len];
+        valBytesArr = new byte[len][];
+
+        for (int i = 0; i < len; i++) {
+            keys[i] = U.readString(in);
+            valBytesArr[i] = U.readByteArray(in);
+        }
+    }
+
     /** {@inheritDoc} */
     @Override public boolean equals(Object o) {
         if (this == o)
@@ -78,12 +130,12 @@ final class DistributedMetaStorageHistoryItem implements Serializable {
 
         DistributedMetaStorageHistoryItem item = (DistributedMetaStorageHistoryItem)o;
 
-        return Arrays.equals(keys, item.keys) && Arrays.deepEquals(valBytesArray, item.valBytesArray);
+        return Arrays.equals(keys, item.keys) && Arrays.deepEquals(valBytesArr, item.valBytesArr);
     }
 
     /** {@inheritDoc} */
     @Override public int hashCode() {
-        return 31 * Arrays.hashCode(keys) + Arrays.deepHashCode(valBytesArray);
+        return Long.hashCode(longHash());
     }
 
     /** Long hash. */
@@ -96,7 +148,7 @@ final class DistributedMetaStorageHistoryItem implements Serializable {
             for (String key : keys)
                 hash = hash * 31L + key.hashCode();
 
-            for (byte[] valBytes : valBytesArray) {
+            for (byte[] valBytes : valBytesArr) {
                 if (valBytes == null)
                     hash *= 31L;
                 else
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/metastorage/persistence/DistributedMetaStorageImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/metastorage/persistence/DistributedMetaStorageImpl.java
index 038d830..82f6d68 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/metastorage/persistence/DistributedMetaStorageImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/metastorage/persistence/DistributedMetaStorageImpl.java
@@ -401,7 +401,7 @@ public class DistributedMetaStorageImpl extends GridProcessorAdapter
 
     /** {@inheritDoc} */
     @Override public long getUpdatesCount() {
-        return ver.id;
+        return ver.id();
     }
 
     /** {@inheritDoc} */
@@ -586,7 +586,7 @@ public class DistributedMetaStorageImpl extends GridProcessorAdapter
             if (!discoData.hasJoiningNodeData()) {
                 // Joining node doesn't support distributed metastorage feature.
 
-                if (isSupported(ctx) && locVer.id > 0 && !(node.isClient() || node.isDaemon())) {
+                if (isSupported(ctx) && locVer.id() > 0 && !(node.isClient() || node.isDaemon())) {
                     String errorMsg = "Node not supporting distributed metastorage feature" +
                         " is not allowed to join the cluster";
 
@@ -623,17 +623,17 @@ public class DistributedMetaStorageImpl extends GridProcessorAdapter
 
             int locHistSize = histCache.size();
 
-            if (remoteVer.id < locVer.id - locHistSize) {
+            if (remoteVer.id() < locVer.id() - locHistSize) {
                 // Remote node is too far behind.
                 // Technicaly this situation should be banned because there's no way to prove data consistency.
                 errorMsg = null;
             }
-            else if (remoteVer.id < locVer.id) {
+            else if (remoteVer.id() < locVer.id()) {
                 // Remote node it behind the cluster version and there's enough history.
                 DistributedMetaStorageVersion newRemoteVer = remoteVer.nextVersion(
                     this::historyItem,
-                    remoteVer.id + 1,
-                    locVer.id
+                    remoteVer.id() + 1,
+                    locVer.id()
                 );
 
                 if (newRemoteVer.equals(locVer))
@@ -641,7 +641,7 @@ public class DistributedMetaStorageImpl extends GridProcessorAdapter
                 else
                     errorMsg = "Joining node has conflicting distributed metastorage data.";
             }
-            else if (remoteVer.id == locVer.id) {
+            else if (remoteVer.id() == locVer.id()) {
                 // Remote and local versions match.
                 if (remoteVer.equals(locVer))
                     errorMsg = null;
@@ -653,7 +653,7 @@ public class DistributedMetaStorageImpl extends GridProcessorAdapter
                     );
                 }
             }
-            else if (remoteVer.id <= locVer.id + remoteHistSize) {
+            else if (remoteVer.id() <= locVer.id() + remoteHistSize) {
                 // Remote node is ahead of the cluster and has enough history.
                 if (clusterIsActive) {
                     errorMsg = "Attempting to join node with larger distributed metastorage version id." +
@@ -664,7 +664,7 @@ public class DistributedMetaStorageImpl extends GridProcessorAdapter
                 else {
                     DistributedMetaStorageVersion newLocVer = locVer.nextVersion(
                         remoteHist,
-                        remoteHistSize - (int)(remoteVer.id - locVer.id),
+                        remoteHistSize - (int)(remoteVer.id() - locVer.id()),
                         remoteHistSize
                     );
 
@@ -675,7 +675,7 @@ public class DistributedMetaStorageImpl extends GridProcessorAdapter
                 }
             }
             else {
-                assert remoteVer.id > locVer.id + remoteHistSize;
+                assert remoteVer.id() > locVer.id() + remoteHistSize;
 
                 // Remote node is too far ahead.
                 if (clusterIsActive) {
@@ -706,12 +706,12 @@ public class DistributedMetaStorageImpl extends GridProcessorAdapter
      */
     private String validatePayload(DistributedMetaStorageJoiningNodeData joiningData) {
         for (DistributedMetaStorageHistoryItem item : joiningData.hist) {
-            for (int i = 0; i < item.keys.length; i++) {
+            for (int i = 0; i < item.keys().length; i++) {
                 try {
-                    unmarshal(marshaller, item.valBytesArray[i]);
+                    unmarshal(marshaller, item.valuesBytesArray()[i]);
                 }
                 catch (IgniteCheckedException e) {
-                    return "Unable to unmarshal key=" + item.keys[i];
+                    return "Unable to unmarshal key=" + item.keys()[i];
                 }
             }
         }
@@ -737,7 +737,7 @@ public class DistributedMetaStorageImpl extends GridProcessorAdapter
 
         DistributedMetaStorageVersion remoteVer = joiningData.ver;
 
-        if (!isSupported(ctx) && remoteVer.id > 0)
+        if (!isSupported(ctx) && remoteVer.id() > 0)
             return;
 
         lock.writeLock().lock();
@@ -745,12 +745,12 @@ public class DistributedMetaStorageImpl extends GridProcessorAdapter
         try {
             DistributedMetaStorageVersion locVer = ver;
 
-            if (remoteVer.id > locVer.id) {
+            if (remoteVer.id() > locVer.id()) {
                 DistributedMetaStorageHistoryItem[] hist = joiningData.hist;
 
-                if (remoteVer.id - locVer.id <= hist.length) {
-                    for (long v = locVer.id + 1; v <= remoteVer.id; v++) {
-                        int hv = (int)(v - remoteVer.id + hist.length - 1);
+                if (remoteVer.id() - locVer.id() <= hist.length) {
+                    for (long v = locVer.id() + 1; v <= remoteVer.id(); v++) {
+                        int hv = (int)(v - remoteVer.id() + hist.length - 1);
 
                         try {
                             completeWrite(hist[hv]);
@@ -806,14 +806,14 @@ public class DistributedMetaStorageImpl extends GridProcessorAdapter
         try {
             DistributedMetaStorageVersion locVer = ver;
 
-            if (remoteVer.id >= locVer.id) {
+            if (remoteVer.id() >= locVer.id()) {
                 Serializable nodeData = new DistributedMetaStorageClusterNodeData(remoteVer, null, null, null);
 
                 dataBag.addGridCommonData(COMPONENT_ID, nodeData);
             }
             else {
-                if (locVer.id - remoteVer.id <= histCache.size() && !dataBag.isJoiningNodeClient()) {
-                    DistributedMetaStorageHistoryItem[] updates = history(remoteVer.id + 1, locVer.id);
+                if (locVer.id() - remoteVer.id() <= histCache.size() && !dataBag.isJoiningNodeClient()) {
+                    DistributedMetaStorageHistoryItem[] updates = history(remoteVer.id() + 1, locVer.id());
 
                     Serializable nodeData = new DistributedMetaStorageClusterNodeData(ver, null, null, updates);
 
@@ -829,7 +829,7 @@ public class DistributedMetaStorageImpl extends GridProcessorAdapter
                     if (dataBag.isJoiningNodeClient())
                         hist = EMPTY_ARRAY;
                     else
-                        hist = history(ver.id - histCache.size() + 1, locVer.id);
+                        hist = history(ver.id() - histCache.size() + 1, locVer.id());
 
                     Serializable nodeData = new DistributedMetaStorageClusterNodeData(ver0, fullData, hist, null);
 
@@ -961,7 +961,7 @@ public class DistributedMetaStorageImpl extends GridProcessorAdapter
                     for (int i = 0, len = nodeData.hist.length; i < len; i++) {
                         DistributedMetaStorageHistoryItem histItem = nodeData.hist[i];
 
-                        addToHistoryCache(ver.id + i - (len - 1), histItem);
+                        addToHistoryCache(ver.id() + i - (len - 1), histItem);
                     }
                 }
 
@@ -973,7 +973,7 @@ public class DistributedMetaStorageImpl extends GridProcessorAdapter
                         completeWrite(update);
                 }
             }
-            else if (!isClient && ver.id > 0) {
+            else if (!isClient && ver.id() > 0) {
                 throw new IgniteException("Cannot join the cluster because it doesn't support distributed metastorage" +
                     " feature and this node has not empty distributed metastorage data");
             }
@@ -1129,13 +1129,13 @@ public class DistributedMetaStorageImpl extends GridProcessorAdapter
 
             ver = ver.nextVersion(histItem);
 
-            for (int i = 0, len = histItem.keys.length; i < len; i++)
-                notifyListeners(histItem.keys[i], bridge.read(histItem.keys[i]), unmarshal(marshaller, histItem.valBytesArray[i]));
+            for (int i = 0, len = histItem.keys().length; i < len; i++)
+                notifyListeners(histItem.keys()[i], bridge.read(histItem.keys()[i]), unmarshal(marshaller, histItem.valuesBytesArray()[i]));
 
-            for (int i = 0, len = histItem.keys.length; i < len; i++)
-                bridge.write(histItem.keys[i], histItem.valBytesArray[i]);
+            for (int i = 0, len = histItem.keys().length; i < len; i++)
+                bridge.write(histItem.keys()[i], histItem.valuesBytesArray()[i]);
 
-            addToHistoryCache(ver.id, histItem);
+            addToHistoryCache(ver.id(), histItem);
         }
         finally {
             lock.writeLock().unlock();
@@ -1156,8 +1156,8 @@ public class DistributedMetaStorageImpl extends GridProcessorAdapter
     @Nullable private DistributedMetaStorageHistoryItem optimizeHistoryItem(
         DistributedMetaStorageHistoryItem histItem
     ) {
-        String[] keys = histItem.keys;
-        byte[][] valBytesArr = histItem.valBytesArray;
+        String[] keys = histItem.keys();
+        byte[][] valBytesArr = histItem.valuesBytesArray();
 
         int len = keys.length;
         int cnt = 0;
@@ -1256,7 +1256,7 @@ public class DistributedMetaStorageImpl extends GridProcessorAdapter
                 histCache.removeOldest();
 
                 if (isPersistenceEnabled)
-                    worker.removeHistItem(ver.id - histCache.size());
+                    worker.removeHistItem(ver.id() - histCache.size());
             }
         }
         finally {
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/metastorage/persistence/DistributedMetaStorageVersion.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/metastorage/persistence/DistributedMetaStorageVersion.java
index e7225a5..65ccf86 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/metastorage/persistence/DistributedMetaStorageVersion.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/metastorage/persistence/DistributedMetaStorageVersion.java
@@ -17,15 +17,18 @@
 
 package org.apache.ignite.internal.processors.metastorage.persistence;
 
-import java.io.Serializable;
+import java.io.IOException;
+import java.io.ObjectInput;
+import java.io.ObjectOutput;
 import java.util.Collection;
 import java.util.List;
 import java.util.function.LongFunction;
+import org.apache.ignite.internal.dto.IgniteDataTransferObject;
 import org.apache.ignite.internal.util.tostring.GridToStringInclude;
 import org.apache.ignite.internal.util.typedef.internal.S;
 
 /** Version class for distributed metastorage. */
-class DistributedMetaStorageVersion implements Serializable {
+final class DistributedMetaStorageVersion extends IgniteDataTransferObject {
     /** Serial version UID. */
     private static final long serialVersionUID = 0L;
 
@@ -44,14 +47,19 @@ class DistributedMetaStorageVersion implements Serializable {
      * @see #INITIAL_VERSION
      */
     @GridToStringInclude
-    public final long id;
+    private long id;
 
     /**
      * Hash of the whole updates list. Hashing algorinthm is almost the same as in {@link List#hashCode()}, but with
      * {@code long} value instead of {@code int}.
      */
     @GridToStringInclude
-    public final long hash;
+    private long hash;
+
+    /** Default constructor for deserialization. */
+    public DistributedMetaStorageVersion() {
+        // No-op.
+    }
 
     /**
      * Constructor with all fields.
@@ -133,6 +141,36 @@ class DistributedMetaStorageVersion implements Serializable {
         return new DistributedMetaStorageVersion(id + toVer + 1 - fromVer, hash);
     }
 
+    /**
+     * Id is basically a total number of distributed metastorage updates in current cluster.
+     * Increases incrementally on every update starting with zero.
+     *
+     * @see #INITIAL_VERSION
+     */
+    public long id() {
+        return id;
+    }
+
+    /**
+     * Hash of the whole updates list. Hashing algorinthm is almost the same as in {@link List#hashCode()}, but with
+     * {@code long} value instead of {@code int}.
+     */
+    public long hash() {
+        return hash;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void writeExternalData(ObjectOutput out) throws IOException {
+        out.writeLong(id);
+        out.writeLong(hash);
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void readExternalData(byte protoVer, ObjectInput in) throws IOException {
+        id = in.readLong();
+        hash = in.readLong();
+    }
+
     /** {@inheritDoc} */
     @Override public boolean equals(Object o) {
         if (this == o)
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/metastorage/persistence/DmsDataWriterWorker.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/metastorage/persistence/DmsDataWriterWorker.java
index 816ff9c..2065784e 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/metastorage/persistence/DmsDataWriterWorker.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/metastorage/persistence/DmsDataWriterWorker.java
@@ -167,7 +167,7 @@ class DmsDataWriterWorker extends GridWorker {
                         for (int i = 0, len = fullNodeData.hist.length; i < len; i++) {
                             DistributedMetaStorageHistoryItem histItem = fullNodeData.hist[i];
 
-                            long histItemVer = fullNodeData.ver.id + i - (len - 1);
+                            long histItemVer = fullNodeData.ver.id() + i - (len - 1);
 
                             metastorage.write(historyItemKey(histItemVer), histItem);
                         }
@@ -201,14 +201,14 @@ class DmsDataWriterWorker extends GridWorker {
 
     /** */
     private void applyUpdate(DistributedMetaStorageHistoryItem histItem) throws IgniteCheckedException {
-        metastorage.write(historyItemKey(workerDmsVer.id + 1), histItem);
+        metastorage.write(historyItemKey(workerDmsVer.id() + 1), histItem);
 
         workerDmsVer = workerDmsVer.nextVersion(histItem);
 
         metastorage.write(versionKey(), workerDmsVer);
 
-        for (int i = 0, len = histItem.keys.length; i < len; i++)
-            write(histItem.keys[i], histItem.valBytesArray[i]);
+        for (int i = 0, len = histItem.keys().length; i < len; i++)
+            write(histItem.keys()[i], histItem.valuesBytesArray()[i]);
     }
 
     /** */
@@ -229,31 +229,31 @@ class DmsDataWriterWorker extends GridWorker {
             }
             else {
                 DistributedMetaStorageHistoryItem histItem =
-                    (DistributedMetaStorageHistoryItem)metastorage.read(historyItemKey(storedVer.id + 1));
+                    (DistributedMetaStorageHistoryItem)metastorage.read(historyItemKey(storedVer.id() + 1));
 
                 if (histItem != null) {
                     workerDmsVer = storedVer.nextVersion(histItem);
 
                     metastorage.write(versionKey(), workerDmsVer);
 
-                    for (int i = 0, len = histItem.keys.length; i < len; i++)
-                        write(histItem.keys[i], histItem.valBytesArray[i]);
+                    for (int i = 0, len = histItem.keys().length; i < len; i++)
+                        write(histItem.keys()[i], histItem.valuesBytesArray()[i]);
                 }
                 else {
                     workerDmsVer = storedVer;
 
-                    histItem = (DistributedMetaStorageHistoryItem)metastorage.read(historyItemKey(storedVer.id));
+                    histItem = (DistributedMetaStorageHistoryItem)metastorage.read(historyItemKey(storedVer.id()));
 
                     if (histItem != null) {
                         boolean equal = true;
 
-                        for (int i = 0, len = histItem.keys.length; i < len; i++) {
-                            byte[] valBytes = metastorage.readRaw(localKey(histItem.keys[i]));
+                        for (int i = 0, len = histItem.keys().length; i < len; i++) {
+                            byte[] valBytes = metastorage.readRaw(localKey(histItem.keys()[i]));
 
-                            if (!equal || !Arrays.equals(valBytes, histItem.valBytesArray[i])) {
+                            if (!equal || !Arrays.equals(valBytes, histItem.valuesBytesArray()[i])) {
                                 equal = false;
 
-                                write(histItem.keys[i], histItem.valBytesArray[i]);
+                                write(histItem.keys()[i], histItem.valuesBytesArray()[i]);
                             }
                         }
                     }
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/metastorage/persistence/InMemoryCachedDistributedMetaStorageBridge.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/metastorage/persistence/InMemoryCachedDistributedMetaStorageBridge.java
index 15cf99d..298d0b5 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/metastorage/persistence/InMemoryCachedDistributedMetaStorageBridge.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/metastorage/persistence/InMemoryCachedDistributedMetaStorageBridge.java
@@ -141,7 +141,7 @@ class InMemoryCachedDistributedMetaStorageBridge {
             DistributedMetaStorageHistoryItem lastHistItem;
 
             DistributedMetaStorageHistoryItem histItem =
-                (DistributedMetaStorageHistoryItem)metastorage.read(historyItemKey(storedVer.id + 1));
+                (DistributedMetaStorageHistoryItem)metastorage.read(historyItemKey(storedVer.id() + 1));
 
             if (histItem != null) {
                 lastHistItem = histItem;
@@ -149,7 +149,7 @@ class InMemoryCachedDistributedMetaStorageBridge {
                 ver = storedVer.nextVersion(histItem);
             }
             else
-                lastHistItem = (DistributedMetaStorageHistoryItem)metastorage.read(historyItemKey(storedVer.id));
+                lastHistItem = (DistributedMetaStorageHistoryItem)metastorage.read(historyItemKey(storedVer.id()));
 
             metastorage.iterate(
                 localKeyPrefix(),
@@ -159,9 +159,9 @@ class InMemoryCachedDistributedMetaStorageBridge {
 
             // Last item rollover.
             if (lastHistItem != null) {
-                for (int i = 0, len = lastHistItem.keys.length; i < len; i++) {
-                    String key = lastHistItem.keys[i];
-                    byte[] valBytes = lastHistItem.valBytesArray[i];
+                for (int i = 0, len = lastHistItem.keys().length; i < len; i++) {
+                    String key = lastHistItem.keys()[i];
+                    byte[] valBytes = lastHistItem.valuesBytesArray()[i];
 
                     if (valBytes == null)
                         cache.remove(key);
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/metastorage/persistence/DmsDataWriterWorkerTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/metastorage/persistence/DmsDataWriterWorkerTest.java
index 40e6408..8a4cccf 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/metastorage/persistence/DmsDataWriterWorkerTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/metastorage/persistence/DmsDataWriterWorkerTest.java
@@ -141,7 +141,7 @@ public class DmsDataWriterWorkerTest {
         assertEquals(6, metastorage.cache.size());
 
         DistributedMetaStorageVersion ver = (DistributedMetaStorageVersion)metastorage.read(versionKey());
-        assertEquals(3, ver.id);
+        assertEquals(3, ver.id());
 
         assertEquals("val3", metastorage.read(localKey("key1")));
         assertEquals("val2", metastorage.read(localKey("key2")));
@@ -335,9 +335,9 @@ public class DmsDataWriterWorkerTest {
 
     /** */
     private DistributedMetaStorageKeyValuePair toKeyValuePair(DistributedMetaStorageHistoryItem histItem) {
-        assertEquals(1, histItem.keys.length);
+        assertEquals(1, histItem.keys().length);
 
-        return new DistributedMetaStorageKeyValuePair(histItem.keys[0], histItem.valBytesArray[0]);
+        return new DistributedMetaStorageKeyValuePair(histItem.keys()[0], histItem.valuesBytesArray()[0]);
     }
 
     /** */