You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by il...@apache.org on 2021/01/25 14:49:26 UTC

[ignite] branch master updated: IGNITE-14039 Add warning to runtime, javadoc and documentation on WAL enable/disable.

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

ilyak 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 7eb4af3  IGNITE-14039 Add warning to runtime, javadoc and documentation on WAL enable/disable.
7eb4af3 is described below

commit 7eb4af3bec3f8b318494546e9261d2a49f360925
Author: Ilya Kasnacheev <il...@gmail.com>
AuthorDate: Mon Jan 25 17:48:24 2021 +0300

    IGNITE-14039 Add warning to runtime, javadoc and documentation on WAL enable/disable.
    
    Also fixed WAL being incorrectly considered disabled after cache was destroyed and recreated. - Fixes #8688.
    
    Signed-off-by: Ilya Kasnacheev <il...@gmail.com>
---
 docs/_docs/persistence/native-persistence.adoc     |   3 +
 .../main/java/org/apache/ignite/IgniteCluster.java |  12 +
 .../internal/processors/cache/WalStateManager.java |   7 +
 .../GridCacheDatabaseSharedManager.java            |   9 +-
 .../wal/WalEnableDisableWithNodeShutdownTest.java  | 348 +++++++++++++++++++++
 .../ignite/testsuites/IgnitePdsTestSuite.java      |   3 +
 6 files changed, 380 insertions(+), 2 deletions(-)

diff --git a/docs/_docs/persistence/native-persistence.adoc b/docs/_docs/persistence/native-persistence.adoc
index b6c0a23..ee8e652 100644
--- a/docs/_docs/persistence/native-persistence.adoc
+++ b/docs/_docs/persistence/native-persistence.adoc
@@ -238,6 +238,9 @@ tab:C++[unsupported]
 
 
 === Disabling WAL
+
+WARNING: Disabling or enabling WAL should only be done on stable topology: all baseline nodes should be present, no nodes should join or leave the cluster for the duration of this operation. Otherwise, cache may be stuck in inconsistent state. If that happens, it is advised to destroy the affected cache(s).
+
 There are situations when it is reasonable to have the WAL disabled to get better performance. For instance, it is useful to disable WAL during initial data loading and enable it after the pre-loading is complete.
 
 ////
diff --git a/modules/core/src/main/java/org/apache/ignite/IgniteCluster.java b/modules/core/src/main/java/org/apache/ignite/IgniteCluster.java
index f57bece..cf2d2eb 100644
--- a/modules/core/src/main/java/org/apache/ignite/IgniteCluster.java
+++ b/modules/core/src/main/java/org/apache/ignite/IgniteCluster.java
@@ -527,6 +527,12 @@ public interface IgniteCluster extends ClusterGroup, IgniteAsyncSupport {
      * when all dirty pages are prepared for checkpoint, but not necessarily flushed to disk.
      * <p>
      * WAL state can be changed only for persistent caches.
+     * <p>
+     * <b>NOTE:</b>
+     * Currently, this method should only be called on a stable topology when no nodes are leaving or joining cluster,
+     * and all baseline nodes are present.
+     * Cache may be stuck in inconsistent state due to violation of these conditions. It is advised to destroy
+     * such cache.
      *
      * @param cacheName Cache name.
      * @return Whether WAL disabled by this call.
@@ -545,6 +551,12 @@ public interface IgniteCluster extends ClusterGroup, IgniteAsyncSupport {
      * when all data is persisted to disk.
      * <p>
      * WAL state can be changed only for persistent caches.
+     * <p>
+     * <b>NOTE:</b>
+     * Currently, this method should only be called on a stable topology when no nodes are leaving or joining cluster,
+     * and all baseline nodes are present.
+     * Cache may be stuck in inconsistent state due to violation of these conditions. It is advised to destroy
+     * such cache.
      *
      * @param cacheName Cache name.
      * @return Whether WAL enabled by this call.
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/WalStateManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/WalStateManager.java
index a8358ef..a301831 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/WalStateManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/WalStateManager.java
@@ -51,6 +51,7 @@ import org.apache.ignite.internal.util.future.GridFinishedFuture;
 import org.apache.ignite.internal.util.future.GridFutureAdapter;
 import org.apache.ignite.internal.util.typedef.F;
 import org.apache.ignite.internal.util.typedef.T2;
+import org.apache.ignite.internal.util.typedef.internal.LT;
 import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.internal.util.worker.GridWorker;
 import org.apache.ignite.lang.IgniteFuture;
@@ -304,6 +305,12 @@ public class WalStateManager extends GridCacheSharedManagerAdapter {
             String.format("Cache WAL mode cannot be changed within lock or transaction " +
                     "[cacheNames=%s, walEnabled=%s]", cacheNames, enabled));
 
+        LT.warn(log, "Cache WAL mode may only be changed on stable topology: see https://issues.apache.org/jira/browse/IGNITE-13976");
+        LT.warn(log, "  ^-- No nodes may leave or join cluster while changing WAL mode.");
+        LT.warn(log, "  ^-- All baseline nodes should be present.");
+        LT.warn(log, "  ^-- Failure to observe these conditions may cause cache to be stuck in inconsistent state.");
+        LT.warn(log, "  ^-- You may need to destroy affected cache if that happens.");
+
         return init(cacheNames, enabled);
     }
 
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/GridCacheDatabaseSharedManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/GridCacheDatabaseSharedManager.java
index 98ee778..a01ed06 100755
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/GridCacheDatabaseSharedManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/GridCacheDatabaseSharedManager.java
@@ -1471,10 +1471,15 @@ public class GridCacheDatabaseSharedManager extends IgniteCacheDatabaseSharedMan
     ) {
         Map<PageMemoryEx, Collection<Integer>> destroyed = new HashMap<>();
 
-        cctx.snapshotMgr().onCacheGroupsStopped(stoppedGrps.stream()
+        List<Integer> stoppedGrpIds = stoppedGrps.stream()
             .filter(IgniteBiTuple::get2)
             .map(t -> t.get1().groupId())
-            .collect(Collectors.toList()));
+            .collect(Collectors.toList());
+
+        cctx.snapshotMgr().onCacheGroupsStopped(stoppedGrpIds);
+
+        initiallyLocalWalDisabledGrps.removeAll(stoppedGrpIds);
+        initiallyGlobalWalDisabledGrps.removeAll(stoppedGrpIds);
 
         for (IgniteBiTuple<CacheGroupContext, Boolean> tup : stoppedGrps) {
             CacheGroupContext gctx = tup.get1();
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/wal/WalEnableDisableWithNodeShutdownTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/wal/WalEnableDisableWithNodeShutdownTest.java
new file mode 100644
index 0000000..6cae455
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/wal/WalEnableDisableWithNodeShutdownTest.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.internal.processors.cache.persistence.wal;
+
+import java.util.LinkedList;
+import java.util.List;
+import javax.cache.configuration.CompleteConfiguration;
+import org.apache.ignite.Ignite;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.Ignition;
+import org.apache.ignite.cache.CacheAtomicityMode;
+import org.apache.ignite.cache.CacheMode;
+import org.apache.ignite.cache.affinity.rendezvous.RendezvousAffinityFunction;
+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.internal.IgniteEx;
+import org.apache.ignite.internal.processors.cache.persistence.CleanCacheStoresMaintenanceAction;
+import org.apache.ignite.internal.util.typedef.X;
+import org.apache.ignite.lang.IgniteRunnable;
+import org.apache.ignite.maintenance.MaintenanceAction;
+import org.apache.ignite.maintenance.MaintenanceRegistry;
+import org.apache.ignite.resources.IgniteInstanceResource;
+import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+import org.junit.After;
+import org.junit.Test;
+
+import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.CORRUPTED_DATA_FILES_MNTC_TASK_NAME;
+
+/** */
+public class WalEnableDisableWithNodeShutdownTest extends GridCommonAbstractTest {
+    /** */
+    private static final String CACHE_NAME = "MY_CACHE";
+
+    /** */
+    private static final String CACHE_NAME_2 = "MY_CACHE_2";
+
+    /** */
+    private static final int CYCLES = 2;
+
+    /** */
+    public static final int NODES = 4;
+
+    /** */
+    public static final int WAIT_MILLIS = 150;
+
+    /**
+     * Checks whether a node may not be down while WAL is disabled on cache.
+     *
+     * Starts nodes with two caches configured. Activates cluster.
+     *
+     * Remove first node.
+     * Disable WAL on cache.
+     * Bring back first node as new last.
+     * Enable WAL on cache.
+     * Catch expected exception.
+     * Drop and recreate the cache.
+     * Check that new cache is valid and can enable/disable WAL.
+     */
+    @Test
+    public void testDisableWhileNodeOffline() throws Exception {
+        LinkedList<Ignite> nodes = new LinkedList<>();
+
+        for (int i = 0; i < NODES; i++)
+            nodes.add(Ignition.start(igniteCfg(false, "server_" + i)));
+
+        nodes.getFirst().active(true);
+
+        Ignite client = Ignition.start(igniteCfg(true, "client"));
+
+        for (int i = 0; i < CYCLES; i++) {
+            try {
+                Thread.sleep(WAIT_MILLIS);
+
+                Ignite ignite = nodes.removeFirst();
+
+                String consistentId = (String)ignite.cluster().localNode().consistentId();
+
+                ignite.close();
+
+                Thread.sleep(WAIT_MILLIS);
+
+                client.cluster().disableWal(CACHE_NAME);
+
+                Thread.sleep(WAIT_MILLIS);
+
+                nodes.add(Ignition.start(igniteCfg(false, consistentId)));
+
+                Thread.sleep(WAIT_MILLIS);
+
+                client.cluster().enableWal(CACHE_NAME);
+            }
+            catch (IgniteException ex) {
+                if (ex.getMessage().contains("Operation result is unknown because nodes reported different results")) {
+                    log.warning("Expected exception thrown", ex);
+
+                    recreateCacheCheckValid(client);
+
+                    return;
+                }
+                else
+                    throw ex;
+            }
+        }
+
+        fail("Expected exception not thrown");
+    }
+
+    /**
+     * Checks whether a node may be down while WAL is enabled on cache.
+     *
+     * Starts nodes with two caches configured. Activates cluster.
+     *
+     * Disable WAL on cache.
+     * Remove first node.
+     * Enable WAL on cache.
+     * Cleanup WAL by entering maintenance mode.
+     * Bring back first node as new last.
+     * Disable WAL on cache.
+     * Catch expected exception.
+     * Drop and recreate the cache.
+     * Check that new cache is valid and can enable/disable WAL.
+     */
+    @Test
+    public void testEnableWhileNodeOffline() throws Exception {
+        LinkedList<Ignite> nodes = new LinkedList<>();
+
+        for (int i = 0; i < NODES; i++)
+            nodes.add(Ignition.start(igniteCfg(false, "server_" + i)));
+
+        nodes.getFirst().active(true);
+
+        Ignite client = Ignition.start(igniteCfg(true, "client"));
+
+        for (int i = 0; i < CYCLES; i++) {
+            try {
+                Thread.sleep(WAIT_MILLIS);
+
+                client.cluster().disableWal(CACHE_NAME);
+
+                Thread.sleep(WAIT_MILLIS);
+
+                Ignite ignite = nodes.removeFirst();
+
+                String consistentId = (String)ignite.cluster().localNode().consistentId();
+
+                ignite.close();
+
+                Thread.sleep(WAIT_MILLIS);
+
+                client.cluster().enableWal(CACHE_NAME);
+
+                Thread.sleep(WAIT_MILLIS);
+
+                Ignite node = startNodeWithMaintenance(consistentId);
+
+                nodes.add(node);
+            }
+            catch (IgniteException ex) {
+                if (ex.getMessage().contains("Operation result is unknown because nodes reported different results")) {
+                    log.warning("Expected exception thrown", ex);
+
+                    recreateCacheCheckValid(client);
+
+                    return;
+                }
+                else
+                    throw ex;
+            }
+        }
+
+        fail("Expected exception not thrown");
+    }
+
+    /**
+     * Checks whether a node may be down while WAL is off on cache.
+     *
+     * Starts nodes with two caches configured. Activates cluster.
+     *
+     * Repeats the following:
+     * Disable WAL on cache.
+     * Remove first node.
+     * Cleanup WAL by entering maintenance mode.
+     * Bring back first node as new last.
+     * Enable WAL on cache.
+     */
+    @Test
+    public void testOffWhileNodeOffline() throws Exception {
+        LinkedList<Ignite> nodes = new LinkedList<>();
+
+        for (int i = 0; i < NODES; i++)
+            nodes.add(Ignition.start(igniteCfg(false, "server_" + i)));
+
+        nodes.getFirst().active(true);
+
+        Ignite client = Ignition.start(igniteCfg(true, "client"));
+
+        for (int i = 0; i < CYCLES; i++) {
+            try {
+                Thread.sleep(WAIT_MILLIS);
+
+                client.cluster().disableWal(CACHE_NAME);
+
+                Thread.sleep(WAIT_MILLIS);
+
+                Ignite ignite = nodes.removeFirst();
+
+                String consistentId = (String)ignite.cluster().localNode().consistentId();
+
+                ignite.close();
+
+                Thread.sleep(WAIT_MILLIS);
+
+                Ignite node = startNodeWithMaintenance(consistentId);
+
+                nodes.add(node);
+
+                client.cluster().enableWal(CACHE_NAME);
+
+                Thread.sleep(WAIT_MILLIS);
+            }
+            catch (IgniteException ex) {
+                if (ex.getMessage().contains("Operation result is unknown because nodes reported different results")) {
+                    log.error(ex.toString(), ex);
+
+                    fail("WAL is in inconsistent state");
+                }
+                else
+                    throw ex;
+            }
+        }
+    }
+
+    /** */
+    private void recreateCacheCheckValid(Ignite client) {
+        IgniteCache c = client.cache(CACHE_NAME);
+
+        CacheConfiguration ccfg = new CacheConfiguration(
+            (CompleteConfiguration)c.getConfiguration(CacheConfiguration.class));
+
+        c.destroy();
+
+        c = client.createCache(ccfg);
+
+        assertTrue(client.cluster().isWalEnabled(CACHE_NAME));
+
+        c.put(1, "foo");
+
+        client.cluster().disableWal(CACHE_NAME);
+
+        c.put(2, "bar");
+
+        client.cluster().enableWal(CACHE_NAME);
+
+        c.put(1, "baz");
+    }
+
+    /** */
+    private Ignite startNodeWithMaintenance(String consistentId) throws Exception {
+        Ignite node;
+
+        try {
+            node = Ignition.start(igniteCfg(false, consistentId));
+        }
+        catch (Exception ex) {
+            assertTrue(X.hasCause(ex, "Cache groups with potentially corrupted partition files", IgniteException.class));
+
+            node = Ignition.start(igniteCfg(false, consistentId));
+
+            node.compute().run(new IgniteRunnable() {
+                /** */
+                @IgniteInstanceResource
+                private Ignite ignite;
+
+                /** */
+                @Override public void run() {
+                    MaintenanceRegistry mntcRegistry = ((IgniteEx) ignite).context().maintenanceRegistry();
+
+                    List<MaintenanceAction<?>> actions = mntcRegistry
+                        .actionsForMaintenanceTask(CORRUPTED_DATA_FILES_MNTC_TASK_NAME);
+
+                    actions.stream()
+                        .filter(a -> a.name().equals(CleanCacheStoresMaintenanceAction.ACTION_NAME)).findFirst()
+                        .get().execute();
+
+                    mntcRegistry.unregisterMaintenanceTask(CORRUPTED_DATA_FILES_MNTC_TASK_NAME);
+                }
+            });
+
+            node.close();
+
+            node = Ignition.start(igniteCfg(false, consistentId));
+        }
+
+        return node;
+    }
+
+    /** */
+    @After
+    public void cleanUp() throws Exception {
+        stopAllGrids();
+
+        cleanPersistenceDir();
+    }
+
+    /** */
+    private IgniteConfiguration igniteCfg(boolean client, String name) throws Exception {
+        IgniteConfiguration igniteCfg = getConfiguration(name);
+
+        igniteCfg.setConsistentId(name);
+
+        igniteCfg.setClientMode(client);
+
+        CacheConfiguration configuration = new CacheConfiguration(CACHE_NAME);
+        configuration.setAtomicityMode(CacheAtomicityMode.TRANSACTIONAL);
+        configuration.setBackups(0);
+        configuration.setAffinity(new RendezvousAffinityFunction().setPartitions(64));
+        configuration.setCacheMode(CacheMode.PARTITIONED);
+
+        CacheConfiguration configuration2 = new CacheConfiguration(CACHE_NAME_2);
+        configuration2.setAtomicityMode(CacheAtomicityMode.TRANSACTIONAL);
+        configuration2.setBackups(0);
+        configuration.setAffinity(new RendezvousAffinityFunction().setPartitions(64));
+        configuration2.setCacheMode(CacheMode.PARTITIONED);
+
+        igniteCfg.setCacheConfiguration(configuration, configuration2);
+
+        igniteCfg.setDataStorageConfiguration(new DataStorageConfiguration()
+            .setDefaultDataRegionConfiguration(new DataRegionConfiguration().setPersistenceEnabled(true)));
+        return igniteCfg;
+    }
+}
diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgnitePdsTestSuite.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgnitePdsTestSuite.java
index 6895a62..ad60059 100644
--- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgnitePdsTestSuite.java
+++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgnitePdsTestSuite.java
@@ -63,6 +63,7 @@ import org.apache.ignite.internal.processors.cache.persistence.wal.CpTriggeredWa
 import org.apache.ignite.internal.processors.cache.persistence.wal.ExplicitWalDeltaConsistencyTest;
 import org.apache.ignite.internal.processors.cache.persistence.wal.SegmentedRingByteBufferTest;
 import org.apache.ignite.internal.processors.cache.persistence.wal.SysPropWalDeltaConsistencyTest;
+import org.apache.ignite.internal.processors.cache.persistence.wal.WalEnableDisableWithNodeShutdownTest;
 import org.apache.ignite.internal.processors.cache.persistence.wal.aware.SegmentAwareTest;
 import org.apache.ignite.internal.processors.configuration.distributed.DistributedConfigurationPersistentTest;
 import org.apache.ignite.internal.processors.database.IgniteDbDynamicCacheSelfTest;
@@ -132,6 +133,8 @@ public class IgnitePdsTestSuite {
         // Binary meta tests.
         GridTestUtils.addTestIfNeeded(suite, IgnitePdsCacheObjectBinaryProcessorOnDiscoveryTest.class, ignoredTests);
 
+        GridTestUtils.addTestIfNeeded(suite, WalEnableDisableWithNodeShutdownTest.class, ignoredTests);
+
         GridTestUtils.addTestIfNeeded(suite, SegmentAwareTest.class, ignoredTests);
 
         return suite;