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

[ignite] 02/02: IGNITE-12531 Cluster is unable to change BLT on 2.8 if storage was initially created on 2.7 or less - Fixes #7265.

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

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

commit e31b5300363e2c6a5f51374b8ba4bf6552fd2406
Author: Slava Koptilin <sl...@gmail.com>
AuthorDate: Fri Jan 17 18:14:07 2020 +0300

    IGNITE-12531 Cluster is unable to change BLT on 2.8 if storage was initially created on 2.7 or less - Fixes #7265.
    
    Signed-off-by: Ivan Rakov <ir...@apache.org>
---
 .../IgnitePKIndexesMigrationToUnwrapPkTest.java    |   1 -
 .../persistence/MetaStorageCompatibilityTest.java  | 348 +++++++++++++++++++++
 .../IgniteCompatibilityBasicTestSuite.java         |   6 +-
 .../cache/persistence/metastorage/MetaStorage.java |  46 ++-
 .../metastorage/IgniteMetaStorageBasicTest.java    |   4 +-
 5 files changed, 386 insertions(+), 19 deletions(-)

diff --git a/modules/compatibility/src/test/java/org/apache/ignite/compatibility/persistence/IgnitePKIndexesMigrationToUnwrapPkTest.java b/modules/compatibility/src/test/java/org/apache/ignite/compatibility/persistence/IgnitePKIndexesMigrationToUnwrapPkTest.java
index ed83ecd..3345d7b 100644
--- a/modules/compatibility/src/test/java/org/apache/ignite/compatibility/persistence/IgnitePKIndexesMigrationToUnwrapPkTest.java
+++ b/modules/compatibility/src/test/java/org/apache/ignite/compatibility/persistence/IgnitePKIndexesMigrationToUnwrapPkTest.java
@@ -1,4 +1,3 @@
-
 /*
  * Licensed to the Apache Software Foundation (ASF) under one or more
  * contributor license agreements.  See the NOTICE file distributed with
diff --git a/modules/compatibility/src/test/java/org/apache/ignite/compatibility/persistence/MetaStorageCompatibilityTest.java b/modules/compatibility/src/test/java/org/apache/ignite/compatibility/persistence/MetaStorageCompatibilityTest.java
new file mode 100644
index 0000000..0933808
--- /dev/null
+++ b/modules/compatibility/src/test/java/org/apache/ignite/compatibility/persistence/MetaStorageCompatibilityTest.java
@@ -0,0 +1,348 @@
+/*
+ * 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.io.File;
+import java.io.IOException;
+import java.nio.file.OpenOption;
+import org.apache.ignite.Ignite;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.configuration.DataRegionConfiguration;
+import org.apache.ignite.configuration.DataStorageConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.internal.IgniteEx;
+import org.apache.ignite.internal.IgnitionEx;
+import org.apache.ignite.internal.processors.cache.GridCacheAbstractFullApiSelfTest;
+import org.apache.ignite.internal.processors.cache.persistence.GridCacheDatabaseSharedManager;
+import org.apache.ignite.internal.processors.cache.persistence.file.FileIO;
+import org.apache.ignite.internal.processors.cache.persistence.file.FileIODecorator;
+import org.apache.ignite.internal.processors.cache.persistence.file.FileIOFactory;
+import org.apache.ignite.internal.processors.cache.persistence.file.RandomAccessFileIOFactory;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.lang.IgniteInClosure;
+import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi;
+import org.apache.ignite.testframework.GridTestUtils;
+import org.jetbrains.annotations.Nullable;
+import org.junit.Test;
+
+/**
+ * Tests migration of metastorage.
+ */
+public class MetaStorageCompatibilityTest extends IgnitePersistenceCompatibilityAbstractTest {
+    /** Consistent id. */
+    private static final String CONSISTENT_ID_1 = "node1";
+
+    /** Consistent id. */
+    private static final String CONSISTENT_ID_2 = "node2";
+
+    /** Ignite version. */
+    private static final String IGNITE_VERSION = "2.4.0";
+
+    /** Filename of index. */
+    private static final String INDEX_BIN_FILE = "index.bin";
+
+    /** Filename of partition. */
+    private static final String PART_FILE = "part-0.bin";
+
+    /**
+     * The zero partition hasn't full baseline information. The last changes are read from WAL.
+     *
+     * @throws Exception If failed.
+     */
+    @Test
+    public void testMigrationWithoutFullBaselineIntoPartition() throws Exception {
+        try {
+            U.delete(new File(U.defaultWorkDirectory()));
+
+            startGrid(1, IGNITE_VERSION, new ConfigurationClosure(CONSISTENT_ID_1), new ActivateAndForceCheckpointClosure());
+
+            startGrid(2, IGNITE_VERSION, new ConfigurationClosure(CONSISTENT_ID_2), new NewBaselineTopologyClosure());
+
+            stopAllGrids();
+
+            try (Ignite ig0 = IgnitionEx.start(prepareConfig(getConfiguration(), CONSISTENT_ID_1))) {
+                try (Ignite ig1 = IgnitionEx.start(prepareConfig(getConfiguration(), CONSISTENT_ID_2))) {
+                    // No-op.
+                }
+            }
+
+            assertFalse(metastorageFileExists(INDEX_BIN_FILE));
+            assertFalse(metastorageFileExists(PART_FILE));
+
+            try (Ignite ig0 = IgnitionEx.start(prepareConfig(getConfiguration(), CONSISTENT_ID_1))) {
+                try (Ignite ig1 = IgnitionEx.start(prepareConfig(getConfiguration(), CONSISTENT_ID_2))) {
+                    assertTrue(GridTestUtils.waitForCondition(() -> ig1.cluster().active(), 10_000));
+                }
+            }
+        }
+        finally {
+            stopAllGrids();
+        }
+    }
+
+    /**
+     * Tests that BLT can be changed and persisted after metastorage migration.
+     */
+    @Test
+    public void testMigrationToNewBaselineSetNewBaselineAfterMigration() throws Exception {
+        try {
+            U.delete(new File(U.defaultWorkDirectory()));
+
+            startGrid(1, IGNITE_VERSION, new ConfigurationClosure(CONSISTENT_ID_1), new ActivateAndStopClosure());
+
+            stopAllGrids();
+
+            try (Ignite ig0 = IgnitionEx.start(prepareConfig(getConfiguration(), CONSISTENT_ID_1))) {
+                try (Ignite ig1 = IgnitionEx.start(prepareConfig(getConfiguration(), CONSISTENT_ID_2))) {
+                    ig0.cluster().setBaselineTopology(ig1.cluster().topologyVersion());
+                }
+            }
+
+            assertFalse(metastorageFileExists(INDEX_BIN_FILE));
+            assertFalse(metastorageFileExists(PART_FILE));
+
+            try (Ignite ig0 = IgnitionEx.start(prepareConfig(getConfiguration(), CONSISTENT_ID_1))) {
+                try (Ignite ig1 = IgnitionEx.start(prepareConfig(getConfiguration(), CONSISTENT_ID_2))) {
+                    assertTrue(GridTestUtils.waitForCondition(() -> ig1.cluster().active(), 10_000));
+                }
+            }
+        }
+        finally {
+            stopAllGrids();
+        }
+    }
+
+    /**
+     *
+     */
+    @Test
+    public void testMigrationWithExceptionDuringTheProcess() throws Exception {
+        try {
+            U.delete(new File(U.defaultWorkDirectory()));
+
+            startGrid(1, IGNITE_VERSION, new ConfigurationClosure(CONSISTENT_ID_1), new ActivateAndStopClosure());
+
+            stopAllGrids();
+
+            try (Ignite ig0 = IgnitionEx.start(prepareConfig(getConfiguration(), CONSISTENT_ID_1, PART_FILE))) {
+                ig0.getOrCreateCache("default-cache").put(1, 1); // trigger checkpoint on close()
+            }
+
+            assertTrue(metastorageFileExists(INDEX_BIN_FILE));
+            assertTrue(metastorageFileExists(PART_FILE));
+
+            try (Ignite ig0 = IgnitionEx.start(prepareConfig(getConfiguration(), CONSISTENT_ID_1, INDEX_BIN_FILE))) {
+                ig0.getOrCreateCache("default-cache").put(1, 1); // trigger checkpoint on close()
+            }
+
+            assertTrue(metastorageFileExists(INDEX_BIN_FILE));
+            assertFalse(metastorageFileExists(PART_FILE));
+
+            try (Ignite ig0 = IgnitionEx.start(prepareConfig(getConfiguration(), CONSISTENT_ID_1))) {
+                try (Ignite ig1 = IgnitionEx.start(prepareConfig(getConfiguration(), CONSISTENT_ID_2))) {
+                    ig0.cluster().setBaselineTopology(ig1.cluster().topologyVersion());
+                }
+            }
+
+            try (Ignite ig0 = IgnitionEx.start(prepareConfig(getConfiguration(), CONSISTENT_ID_1))) {
+                try (Ignite ig1 = IgnitionEx.start(prepareConfig(getConfiguration(), CONSISTENT_ID_2))) {
+                    assertTrue(GridTestUtils.waitForCondition(() -> ig1.cluster().active(), 10_000));
+                }
+            }
+        }
+        finally {
+            stopAllGrids();
+        }
+    }
+
+    /**
+     * Checks that file exists.
+     *
+     * @param fileName File name.
+     */
+    private boolean metastorageFileExists(String fileName) throws IgniteCheckedException {
+        return new File(U.defaultWorkDirectory() + "/db/" + U.maskForFileName(CONSISTENT_ID_1) + "/metastorage/" + fileName).exists();
+    }
+
+    /**
+     * Updates the given ignite configuration.
+     *
+     * @param cfg              Ignite configuration to be updated.
+     * @param consistentId     Consistent id.
+     * @return Updated configuration.
+     */
+    private static IgniteConfiguration prepareConfig(IgniteConfiguration cfg, @Nullable String consistentId) {
+        return prepareConfig(cfg, consistentId, null);
+    }
+
+    /**
+     * Updates the given ignite configuration.
+     *
+     * @param cfg Ignite configuration to be updated.
+     * @param consistentId Consistent id.
+     * @param failOnFileRmv Indicates that an exception should be trown when partition/index file is going to be removed.
+     * @return Updated configuration.
+     */
+    private static IgniteConfiguration prepareConfig(
+        IgniteConfiguration cfg,
+        @Nullable String consistentId,
+        @Nullable String failOnFileRmv
+    ) {
+        cfg.setLocalHost("127.0.0.1");
+
+        TcpDiscoverySpi disco = new TcpDiscoverySpi();
+        disco.setIpFinder(GridCacheAbstractFullApiSelfTest.LOCAL_IP_FINDER);
+
+        cfg.setDiscoverySpi(disco);
+
+        cfg.setPeerClassLoadingEnabled(false);
+
+        DataStorageConfiguration memCfg = new DataStorageConfiguration()
+            .setDefaultDataRegionConfiguration(
+                new DataRegionConfiguration()
+                    .setPersistenceEnabled(true)
+                    .setInitialSize(10L * 1024 * 1024)
+                    .setMaxSize(10L * 1024 * 1024))
+            .setPageSize(4096);
+
+        cfg.setDataStorageConfiguration(memCfg);
+
+        if (consistentId != null) {
+            cfg.setIgniteInstanceName(consistentId);
+            cfg.setConsistentId(consistentId);
+        }
+
+        if (failOnFileRmv != null)
+            cfg.getDataStorageConfiguration().setFileIOFactory(new FailingFileIOFactory(failOnFileRmv));
+
+        return cfg;
+    }
+
+    /** */
+    private static class ConfigurationClosure implements IgniteInClosure<IgniteConfiguration> {
+        /** Consistent id. */
+        private final String consistentId;
+
+        /**
+         * Creates a new instance of Configuration closure.
+         *
+         * @param consistentId Consistent id.
+         */
+        public ConfigurationClosure(String consistentId) {
+            this.consistentId = consistentId;
+        }
+
+        /** {@inheritDoc} */
+        @Override public void apply(IgniteConfiguration cfg) {
+            prepareConfig(cfg, consistentId, null);
+        }
+    }
+
+    /**
+     * Post-startup close that activate the grid and force checkpoint.
+     */
+    private static class ActivateAndForceCheckpointClosure implements IgniteInClosure<Ignite> {
+        /** {@inheritDoc} */
+        @Override public void apply(Ignite ignite) {
+            try {
+                ignite.active(true);
+
+                ((IgniteEx)ignite).context().cache().context().database()
+                    .forceCheckpoint("force test checkpoint").finishFuture().get();
+
+                ((GridCacheDatabaseSharedManager)(((IgniteEx)ignite).context().cache().context().database()))
+                    .enableCheckpoints(false);
+            }
+            catch (IgniteCheckedException e) {
+                throw new IgniteException(e);
+            }
+        }
+    }
+
+    /**
+     * Activates the cluster and stops it after that.
+     */
+    private static class ActivateAndStopClosure implements IgniteInClosure<Ignite> {
+        /** {@inheritDoc} */
+        @Override public void apply(Ignite ignite) {
+            ignite.active(true);
+
+            ignite.close();
+        }
+    }
+
+    /**
+     * Disables checkpointer and sets a new baseline topology.
+     */
+    private static class NewBaselineTopologyClosure implements IgniteInClosure<Ignite> {
+        /** {@inheritDoc} */
+        @Override public void apply(Ignite ignite) {
+            ((GridCacheDatabaseSharedManager)(((IgniteEx)ignite).context().cache().context().database()))
+                .enableCheckpoints(false);
+
+            long newTopVer = ignite.cluster().topologyVersion();
+
+            ignite.cluster().setBaselineTopology(newTopVer);
+        }
+    }
+
+    /**
+     * Create File I/O which fails after second attempt to write to File
+     */
+    private static class FailingFileIOFactory implements FileIOFactory {
+        /** Serial version uid. */
+        private static final long serialVersionUID = 0L;
+
+        /** Indicates that an exception should be trown when this file is going to be removed. */
+        private final String failOnFileRmv;
+
+        /** */
+        private final FileIOFactory delegateFactory = new RandomAccessFileIOFactory();
+
+        /**
+         * Creates a new instance of IO factory.
+         */
+        public FailingFileIOFactory() {
+            failOnFileRmv = null;
+        }
+
+        /**
+         * Creates a new instance of IO factory.
+         *
+         * @param failOnFileRmv Indicates that an exception should be trown when the given file is removed.
+         */
+        public FailingFileIOFactory(String failOnFileRmv) {
+            this.failOnFileRmv = failOnFileRmv;
+        }
+
+        /** {@inheritDoc} */
+        @Override public FileIO create(File file, OpenOption... modes) throws IOException {
+            final FileIO delegate = delegateFactory.create(file, modes);
+
+            return new FileIODecorator(delegate) {
+                @Override public void clear() throws IOException {
+                    if (failOnFileRmv != null && failOnFileRmv.equals(file.getName()))
+                        throw new IOException("Test remove fail!");
+
+                    super.clear();
+                }
+            };
+        }
+    }
+}
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 19ad466..e940ff4 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
@@ -18,6 +18,7 @@
 package org.apache.ignite.compatibility.testsuites;
 
 import org.apache.ignite.compatibility.persistence.FoldersReuseCompatibilityTest;
+import org.apache.ignite.compatibility.persistence.MetaStorageCompatibilityTest;
 import org.apache.ignite.compatibility.persistence.MigratingToWalV2SerializerWithCompactionTest;
 import org.apache.ignite.compatibility.persistence.PersistenceBasicCompatibilityTest;
 import org.junit.runner.RunWith;
@@ -29,10 +30,9 @@ import org.junit.runners.Suite;
 @RunWith(Suite.class)
 @Suite.SuiteClasses({
     PersistenceBasicCompatibilityTest.class,
-
     FoldersReuseCompatibilityTest.class,
-
-    MigratingToWalV2SerializerWithCompactionTest.class
+    MigratingToWalV2SerializerWithCompactionTest.class,
+    MetaStorageCompatibilityTest.class
 })
 public class IgniteCompatibilityBasicTestSuite {
 }
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/metastorage/MetaStorage.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/metastorage/MetaStorage.java
index 8db6d2c..f437126 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/metastorage/MetaStorage.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/metastorage/MetaStorage.java
@@ -44,12 +44,12 @@ import org.apache.ignite.internal.pagemem.FullPageId;
 import org.apache.ignite.internal.pagemem.PageIdAllocator;
 import org.apache.ignite.internal.pagemem.PageIdUtils;
 import org.apache.ignite.internal.pagemem.PageMemory;
+import org.apache.ignite.internal.pagemem.store.PageStore;
 import org.apache.ignite.internal.pagemem.wal.IgniteWriteAheadLogManager;
 import org.apache.ignite.internal.pagemem.wal.WALPointer;
 import org.apache.ignite.internal.pagemem.wal.record.MetastoreDataRecord;
 import org.apache.ignite.internal.pagemem.wal.record.delta.MetaPageInitRecord;
 import org.apache.ignite.internal.processors.cache.CacheDiagnosticManager;
-import org.apache.ignite.internal.processors.cache.CacheGroupContext;
 import org.apache.ignite.internal.processors.cache.GridCacheProcessor;
 import org.apache.ignite.internal.processors.cache.GridCacheSharedContext;
 import org.apache.ignite.internal.processors.cache.persistence.DataRegion;
@@ -59,6 +59,7 @@ import org.apache.ignite.internal.processors.cache.persistence.GridCacheDatabase
 import org.apache.ignite.internal.processors.cache.persistence.IgniteCacheDatabaseSharedManager;
 import org.apache.ignite.internal.processors.cache.persistence.RootPage;
 import org.apache.ignite.internal.processors.cache.persistence.StorageException;
+import org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager;
 import org.apache.ignite.internal.processors.cache.persistence.pagemem.PageMemoryEx;
 import org.apache.ignite.internal.processors.cache.persistence.partstorage.PartitionMetaStorageImpl;
 import org.apache.ignite.internal.processors.cache.persistence.tree.io.PageIO;
@@ -177,14 +178,29 @@ public class MetaStorage implements DbCheckpointListener, ReadWriteMetastorage {
 
                 gcProcessor.setTmpStorage(null);
 
-                // remove old partitions
-                CacheGroupContext cgc = cctx.cache().cacheGroup(METASTORAGE_CACHE_ID);
+                db.addCheckpointListener(new DbCheckpointListener() {
+                    /** {@inheritDoc} */
+                    @Override public void onMarkCheckpointBegin(Context ctx) {
+                    }
 
-                if (cgc != null) {
-                    db.schedulePartitionDestroy(METASTORAGE_CACHE_ID, OLD_METASTORE_PARTITION);
+                    /** {@inheritDoc} */
+                    @Override public void onCheckpointBegin(Context ctx) throws IgniteCheckedException {
+                        assert cctx.pageStore() != null;
 
-                    db.schedulePartitionDestroy(METASTORAGE_CACHE_ID, PageIdAllocator.INDEX_PARTITION);
-                }
+                        int partTag = ((PageMemoryEx)dataRegion.pageMemory()).invalidate(METASTORAGE_CACHE_ID, OLD_METASTORE_PARTITION);
+                        cctx.pageStore().onPartitionDestroyed(METASTORAGE_CACHE_ID, OLD_METASTORE_PARTITION, partTag);
+
+                        int idxTag = ((PageMemoryEx)dataRegion.pageMemory()).invalidate(METASTORAGE_CACHE_ID, PageIdAllocator.INDEX_PARTITION);
+                        PageStore store = ((FilePageStoreManager)cctx.pageStore()).getStore(METASTORAGE_CACHE_ID, PageIdAllocator.INDEX_PARTITION);
+                        store.truncate(idxTag);
+
+                        db.removeCheckpointListener(this);
+                    }
+
+                    /** {@inheritDoc} */
+                    @Override public void beforeCheckpointBegin(Context ctx) {
+                    }
+                });
             }
         }
     }
@@ -391,7 +407,7 @@ public class MetaStorage implements DbCheckpointListener, ReadWriteMetastorage {
         while (cur.next()) {
             MetastorageDataRow row = cur.get();
 
-            res.add(new IgniteBiTuple<>(row.key(), row.value()));
+            res.add(new IgniteBiTuple<>(row.key(), marshaller.unmarshal(row.value(), getClass().getClassLoader())));
         }
 
         return res;
@@ -403,7 +419,15 @@ public class MetaStorage implements DbCheckpointListener, ReadWriteMetastorage {
 
         byte[] data = marshaller.marshal(val);
 
-        writeRaw(key, data);
+        final WALPointer ptr;
+
+        synchronized (this) {
+            ptr = wal.log(new MetastoreDataRecord(key, data));
+
+            writeRaw(key, data);
+        }
+
+        wal.flush(ptr, false);
     }
 
     /** {@inheritDoc} */
@@ -414,10 +438,6 @@ public class MetaStorage implements DbCheckpointListener, ReadWriteMetastorage {
     /** {@inheritDoc} */
     @Override public void writeRaw(String key, byte[] data) throws IgniteCheckedException {
         if (!readOnly) {
-            WALPointer ptr = wal.log(new MetastoreDataRecord(key, data));
-
-            wal.flush(ptr, false);
-
             synchronized (this) {
                 MetastorageDataRow oldRow = tree.findOne(new MetastorageDataRow(key, null));
 
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/metastorage/IgniteMetaStorageBasicTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/metastorage/IgniteMetaStorageBasicTest.java
index 730d8e9..4361c4b 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/metastorage/IgniteMetaStorageBasicTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/metastorage/IgniteMetaStorageBasicTest.java
@@ -171,7 +171,7 @@ public class IgniteMetaStorageBasicTest extends GridCommonAbstractTest {
         for (Iterator<IgniteBiTuple<String, byte[]>> it = generateTestData(size, from).iterator(); it.hasNext(); ) {
             IgniteBiTuple<String, byte[]> d = it.next();
 
-            metaStorage.writeRaw(d.getKey(), d.getValue());
+            metaStorage.write(d.getKey(), d.getValue());
 
             res.put(d.getKey(), d.getValue());
         }
@@ -315,7 +315,7 @@ public class IgniteMetaStorageBasicTest extends GridCommonAbstractTest {
 
             try {
                 for (Map.Entry<String, byte[]> v : testData.entrySet())
-                    metaStorage.writeRaw(v.getKey(), v.getValue());
+                    metaStorage.write(v.getKey(), v.getValue());
             }
             finally {
                 db.checkpointReadUnlock();