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 2019/03/24 21:41:32 UTC

[ignite] branch master updated: IGNITE-11535 AtomicLong cannot be found after creation - Fixes #6263.

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


The following commit(s) were added to refs/heads/master by this push:
     new df17242  IGNITE-11535 AtomicLong cannot be found after creation - Fixes #6263.
df17242 is described below

commit df17242ea4005ea5a315cd97e1f86b3c746e73aa
Author: Slava Koptilin <sl...@gmail.com>
AuthorDate: Mon Mar 25 00:37:40 2019 +0300

    IGNITE-11535 AtomicLong cannot be found after creation - Fixes #6263.
    
    Signed-off-by: Ivan Rakov <ir...@apache.org>
---
 .../datastructures/AtomicDataStructureProxy.java   |  39 ++-
 .../datastructures/GridCacheAtomicLongImpl.java    |  67 ++--
 .../GridCacheAtomicReferenceImpl.java              |  36 +-
 .../datastructures/GridCacheAtomicStampedImpl.java |  37 +-
 ...ientReconnectAtomicsWithLostPartitionsTest.java | 372 +++++++++++++++++++++
 .../testsuites/IgniteClientReconnectTestSuite.java |   2 +
 6 files changed, 466 insertions(+), 87 deletions(-)

diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/AtomicDataStructureProxy.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/AtomicDataStructureProxy.java
index 1767234..50674b1 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/AtomicDataStructureProxy.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/AtomicDataStructureProxy.java
@@ -19,8 +19,10 @@
 package org.apache.ignite.internal.processors.datastructures;
 
 import java.io.Externalizable;
+import javax.cache.processor.EntryProcessorException;
 import org.apache.ignite.IgniteCacheRestartingException;
 import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteException;
 import org.apache.ignite.IgniteLogger;
 import org.apache.ignite.internal.GridKernalContext;
 import org.apache.ignite.internal.processors.cache.GridCacheContext;
@@ -30,6 +32,9 @@ import org.apache.ignite.internal.util.future.GridFutureAdapter;
 import org.apache.ignite.internal.util.future.IgniteFutureImpl;
 import org.apache.ignite.internal.util.typedef.internal.U;
 
+/**
+ * Represents base class for atomic data structures.
+ */
 public abstract class AtomicDataStructureProxy<V extends AtomicDataStructureValue>
     implements GridCacheRemovable,IgniteChangeGlobalStateSupport {
     /** Logger. */
@@ -42,7 +47,7 @@ public abstract class AtomicDataStructureProxy<V extends AtomicDataStructureValu
     private volatile GridFutureAdapter<Void> suspendFut;
 
     /** Check removed flag. */
-    private boolean rmvCheck;
+    private volatile boolean rmvCheck;
 
     /** Structure name. */
     protected String name;
@@ -139,6 +144,35 @@ public abstract class AtomicDataStructureProxy<V extends AtomicDataStructureValu
     }
 
     /**
+     * Checks removed status after fail.
+     *
+     * @param cause Initial exception.
+     * @return Ignite runtime exception that corresponds the original {@code cause}.
+     */
+    protected IgniteException checkRemovedAfterFail(Exception cause) {
+        assert cause != null: "The original cause must not be null.";
+
+        needCheckNotRemoved();
+
+        try {
+            checkRemoved();
+        }
+        catch (Exception e) {
+            // The original exception should be returned.
+        }
+
+        if (cause instanceof IgniteCheckedException)
+            return U.convertException((IgniteCheckedException) cause);
+        else if (cause instanceof EntryProcessorException)
+            return new IgniteException(cause.getMessage(), cause);
+        else {
+            assert cause instanceof IgniteException;
+
+            return (IgniteException)cause;
+        }
+    }
+
+    /**
      * @return Error.
      */
     private IllegalStateException removedError() {
@@ -188,6 +222,9 @@ public abstract class AtomicDataStructureProxy<V extends AtomicDataStructureValu
         // No-op.
     }
 
+    /**
+     * Invalidates local state.
+     */
     protected void invalidateLocalState() {
         // No-op
     }
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheAtomicLongImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheAtomicLongImpl.java
index 822989e..e76eae3 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheAtomicLongImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheAtomicLongImpl.java
@@ -84,8 +84,8 @@ public final class GridCacheAtomicLongImpl extends AtomicDataStructureProxy<Grid
 
             return val.get();
         }
-        catch (IgniteCheckedException e) {
-            throw U.convertException(e);
+        catch (IgniteException | IgniteCheckedException e) {
+            throw checkRemovedAfterFail(e);
         }
     }
 
@@ -100,11 +100,8 @@ public final class GridCacheAtomicLongImpl extends AtomicDataStructureProxy<Grid
 
             return res.get();
         }
-        catch (EntryProcessorException e) {
-            throw new IgniteException(e.getMessage(), e);
-        }
-        catch (IgniteCheckedException e) {
-            throw U.convertException(e);
+        catch (EntryProcessorException | IgniteCheckedException e) {
+            throw checkRemovedAfterFail(e);
         }
     }
 
@@ -119,11 +116,8 @@ public final class GridCacheAtomicLongImpl extends AtomicDataStructureProxy<Grid
 
             return res.get();
         }
-        catch (EntryProcessorException e) {
-            throw new IgniteException(e.getMessage(), e);
-        }
-        catch (IgniteCheckedException e) {
-            throw U.convertException(e);
+        catch (EntryProcessorException | IgniteCheckedException e) {
+            throw checkRemovedAfterFail(e);
         }
     }
 
@@ -138,11 +132,8 @@ public final class GridCacheAtomicLongImpl extends AtomicDataStructureProxy<Grid
 
             return res.get();
         }
-        catch (EntryProcessorException e) {
-            throw new IgniteException(e.getMessage(), e);
-        }
-        catch (IgniteCheckedException e) {
-            throw U.convertException(e);
+        catch (EntryProcessorException | IgniteCheckedException e) {
+            throw checkRemovedAfterFail(e);
         }
     }
 
@@ -157,11 +148,8 @@ public final class GridCacheAtomicLongImpl extends AtomicDataStructureProxy<Grid
 
             return res.get();
         }
-        catch (EntryProcessorException e) {
-            throw new IgniteException(e.getMessage(), e);
-        }
-        catch (IgniteCheckedException e) {
-            throw U.convertException(e);
+        catch (EntryProcessorException | IgniteCheckedException e) {
+            throw checkRemovedAfterFail(e);
         }
     }
 
@@ -176,11 +164,8 @@ public final class GridCacheAtomicLongImpl extends AtomicDataStructureProxy<Grid
 
             return res.get();
         }
-        catch (EntryProcessorException e) {
-            throw new IgniteException(e.getMessage(), e);
-        }
-        catch (IgniteCheckedException e) {
-            throw U.convertException(e);
+        catch (EntryProcessorException | IgniteCheckedException e) {
+            throw checkRemovedAfterFail(e);
         }
     }
 
@@ -195,11 +180,8 @@ public final class GridCacheAtomicLongImpl extends AtomicDataStructureProxy<Grid
 
             return res.get();
         }
-        catch (EntryProcessorException e) {
-            throw new IgniteException(e.getMessage(), e);
-        }
-        catch (IgniteCheckedException e) {
-            throw U.convertException(e);
+        catch (EntryProcessorException | IgniteCheckedException e) {
+            throw checkRemovedAfterFail(e);
         }
     }
 
@@ -214,11 +196,8 @@ public final class GridCacheAtomicLongImpl extends AtomicDataStructureProxy<Grid
 
             return res.get();
         }
-        catch (EntryProcessorException e) {
-            throw new IgniteException(e.getMessage(), e);
-        }
-        catch (IgniteCheckedException e) {
-            throw U.convertException(e);
+        catch (EntryProcessorException | IgniteCheckedException e) {
+            throw checkRemovedAfterFail(e);
         }
     }
 
@@ -233,11 +212,8 @@ public final class GridCacheAtomicLongImpl extends AtomicDataStructureProxy<Grid
 
             return res.get() == expVal;
         }
-        catch (EntryProcessorException e) {
-            throw new IgniteException(e.getMessage(), e);
-        }
-        catch (IgniteCheckedException e) {
-            throw U.convertException(e);
+        catch (EntryProcessorException | IgniteCheckedException e) {
+            throw checkRemovedAfterFail(e);
         }
     }
 
@@ -256,11 +232,8 @@ public final class GridCacheAtomicLongImpl extends AtomicDataStructureProxy<Grid
 
             return res.get();
         }
-        catch (EntryProcessorException e) {
-            throw new IgniteException(e.getMessage(), e);
-        }
-        catch (IgniteCheckedException e) {
-            throw U.convertException(e);
+        catch (EntryProcessorException | IgniteCheckedException e) {
+            throw checkRemovedAfterFail(e);
         }
     }
 
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheAtomicReferenceImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheAtomicReferenceImpl.java
index 0c98ba1..d21def0 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheAtomicReferenceImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheAtomicReferenceImpl.java
@@ -92,12 +92,12 @@ public final class GridCacheAtomicReferenceImpl<T> extends AtomicDataStructurePr
             GridCacheAtomicReferenceValue<T> ref = cacheView.get(key);
 
             if (ref == null)
-                throw new IgniteCheckedException("Failed to find atomic reference with given name: " + name);
+                throw new IgniteException("Failed to find atomic reference with given name: " + name);
 
             return ref.get();
         }
-        catch (IgniteCheckedException e) {
-            throw U.convertException(e);
+        catch (IgniteException | IgniteCheckedException e) {
+            throw checkRemovedAfterFail(e);
         }
     }
 
@@ -106,8 +106,13 @@ public final class GridCacheAtomicReferenceImpl<T> extends AtomicDataStructurePr
         checkRemoved();
 
         try {
-            if (ctx.dataStructures().knownType(val))
-                cacheView.invoke(key, new ReferenceSetEntryProcessor<>(val));
+            if (ctx.dataStructures().knownType(val)) {
+                EntryProcessorResult res = cacheView.invoke(key, new ReferenceSetEntryProcessor<>(val));
+
+                assert res != null;
+
+                res.get();
+            }
             else {
                 CU.retryTopologySafe(new Callable<Void>() {
                     @Override public Void call() throws Exception {
@@ -127,11 +132,8 @@ public final class GridCacheAtomicReferenceImpl<T> extends AtomicDataStructurePr
                 });
             }
         }
-        catch (EntryProcessorException e) {
-            throw new IgniteException(e.getMessage(), e);
-        }
-        catch (IgniteCheckedException e) {
-            throw U.convertException(e);
+        catch (EntryProcessorException | IgniteException | IgniteCheckedException e) {
+            throw checkRemovedAfterFail(e);
         }
     }
 
@@ -173,11 +175,8 @@ public final class GridCacheAtomicReferenceImpl<T> extends AtomicDataStructurePr
                 });
             }
         }
-        catch (EntryProcessorException e) {
-            throw new IgniteException(e.getMessage(), e);
-        }
-        catch (IgniteCheckedException e) {
-            throw U.convertException(e);
+        catch (EntryProcessorException | IgniteException | IgniteCheckedException e) {
+            throw checkRemovedAfterFail(e);
         }
     }
 
@@ -225,11 +224,8 @@ public final class GridCacheAtomicReferenceImpl<T> extends AtomicDataStructurePr
                 });
             }
         }
-        catch (EntryProcessorException e) {
-            throw new IgniteException(e.getMessage(), e);
-        }
-        catch (IgniteCheckedException e) {
-            throw U.convertException(e);
+        catch (EntryProcessorException | IgniteException | IgniteCheckedException e) {
+            throw checkRemovedAfterFail(e);
         }
     }
 
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheAtomicStampedImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheAtomicStampedImpl.java
index c193918..1453a52 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheAtomicStampedImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheAtomicStampedImpl.java
@@ -91,8 +91,8 @@ public final class GridCacheAtomicStampedImpl<T, S> extends AtomicDataStructureP
 
             return stmp.get();
         }
-        catch (IgniteCheckedException e) {
-            throw U.convertException(e);
+        catch (IgniteException | IgniteCheckedException e) {
+            throw checkRemovedAfterFail(e);
         }
     }
 
@@ -101,8 +101,13 @@ public final class GridCacheAtomicStampedImpl<T, S> extends AtomicDataStructureP
         checkRemoved();
 
         try {
-            if (ctx.dataStructures().knownType(val) && ctx.dataStructures().knownType(stamp))
-                cacheView.invoke(key, new StampedSetEntryProcessor<>(val, stamp));
+            if (ctx.dataStructures().knownType(val) && ctx.dataStructures().knownType(stamp)) {
+                EntryProcessorResult res = cacheView.invoke(key, new StampedSetEntryProcessor<>(val, stamp));
+
+                assert res != null;
+
+                res.get();
+            }
             else {
                 CU.retryTopologySafe(new Callable<Void>() {
                     @Override public Void call() throws Exception {
@@ -122,11 +127,8 @@ public final class GridCacheAtomicStampedImpl<T, S> extends AtomicDataStructureP
                 });
             }
         }
-        catch (EntryProcessorException e) {
-            throw new IgniteException(e.getMessage(), e);
-        }
-        catch (IgniteCheckedException e) {
-            throw U.convertException(e);
+        catch (EntryProcessorException | IgniteException | IgniteCheckedException e) {
+            throw checkRemovedAfterFail(e);
         }
     }
 
@@ -169,11 +171,8 @@ public final class GridCacheAtomicStampedImpl<T, S> extends AtomicDataStructureP
                 });
             }
         }
-        catch (EntryProcessorException e) {
-            throw new IgniteException(e.getMessage(), e);
-        }
-        catch (IgniteCheckedException e) {
-            throw U.convertException(e);
+        catch (EntryProcessorException | IgniteException | IgniteCheckedException e) {
+            throw checkRemovedAfterFail(e);
         }
     }
 
@@ -185,12 +184,12 @@ public final class GridCacheAtomicStampedImpl<T, S> extends AtomicDataStructureP
             GridCacheAtomicStampedValue<T, S> stmp = cacheView.get(key);
 
             if (stmp == null)
-                throw new IgniteCheckedException("Failed to find atomic stamped with given name: " + name);
+                throw new IgniteException("Failed to find atomic stamped with given name: " + name);
 
             return stmp.stamp();
         }
-        catch (IgniteCheckedException e) {
-            throw U.convertException(e);
+        catch (IgniteException | IgniteCheckedException e) {
+            throw checkRemovedAfterFail(e);
         }
     }
 
@@ -206,8 +205,8 @@ public final class GridCacheAtomicStampedImpl<T, S> extends AtomicDataStructureP
 
             return stmp.value();
         }
-        catch (IgniteCheckedException e) {
-            throw U.convertException(e);
+        catch (IgniteException | IgniteCheckedException e) {
+            throw checkRemovedAfterFail(e);
         }
     }
 
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/IgniteClientReconnectAtomicsWithLostPartitionsTest.java b/modules/core/src/test/java/org/apache/ignite/internal/IgniteClientReconnectAtomicsWithLostPartitionsTest.java
new file mode 100644
index 0000000..72ce8ac
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/IgniteClientReconnectAtomicsWithLostPartitionsTest.java
@@ -0,0 +1,372 @@
+/*
+ * 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;
+
+import org.apache.ignite.Ignite;
+import org.apache.ignite.IgniteAtomicLong;
+import org.apache.ignite.IgniteAtomicReference;
+import org.apache.ignite.IgniteAtomicStamped;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.cache.affinity.rendezvous.RendezvousAffinityFunction;
+import org.apache.ignite.configuration.AtomicConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.internal.processors.datastructures.GridCacheAtomicReferenceImpl;
+import org.apache.ignite.lang.IgniteInClosure;
+import org.apache.ignite.testframework.GridTestUtils;
+import org.junit.Test;
+
+/**
+ *
+ */
+public class IgniteClientReconnectAtomicsWithLostPartitionsTest extends IgniteClientReconnectAbstractTest {
+    /** {@inheritDoc} */
+    @Override protected int serverCount() {
+        return 2;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected int clientCount() {
+        return 1;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(igniteInstanceName);
+
+        String consistentId = "consistent-id-" + igniteInstanceName.charAt(igniteInstanceName.length() - 1);
+
+        cfg.setConsistentId(consistentId);
+
+        AtomicConfiguration atomicCfg = new AtomicConfiguration()
+            .setBackups(0)
+            .setAffinity(new RendezvousAffinityFunction(false, 32));
+
+        cfg.setAtomicConfiguration(atomicCfg);
+
+        return cfg;
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    @Test
+    public void testAtomicLongGet() throws Exception {
+        testAtomicLongReconnectClusterRestart("atomic-long-get", IgniteAtomicLong::get);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    @Test
+    public void testAtomicLongIncrementAndGet() throws Exception {
+        testAtomicLongReconnectClusterRestart("atomic-long-incrementAndGet", IgniteAtomicLong::incrementAndGet);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    @Test
+    public void testAtomicLongAddAndGet() throws Exception {
+        testAtomicLongReconnectClusterRestart("atomic-long-addAndGet", atomic -> atomic.addAndGet(1L));
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    @Test
+    public void testAtomicLongGetAndAdd() throws Exception {
+        testAtomicLongReconnectClusterRestart("atomic-long-getAndAdd", atomic -> atomic.getAndAdd(1L));
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    @Test
+    public void testAtomicLongDecrementAndGet() throws Exception {
+        testAtomicLongReconnectClusterRestart("atomic-long-decrementAndGet", IgniteAtomicLong::decrementAndGet);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    @Test
+    public void testAtomicLongGetAndDecrement() throws Exception {
+        testAtomicLongReconnectClusterRestart("atomic-long-getAndDecrement", IgniteAtomicLong::getAndDecrement);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    @Test
+    public void testAtomicLongGetAndSet() throws Exception {
+        testAtomicLongReconnectClusterRestart("atomic-long-getAndSet", atomic -> atomic.getAndSet(1L));
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    @Test
+    public void testAtomicLongCompareAndSet() throws Exception {
+        testAtomicLongReconnectClusterRestart(
+            "atomic-long-compareAndSet",
+            atomic -> atomic.compareAndSet(1L, 2L));
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    @Test
+    public void testAtomicLongGetAndIncrement() throws Exception {
+        testAtomicLongReconnectClusterRestart("atomic-long-getAndIncrement", IgniteAtomicLong::getAndIncrement);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    @Test
+    public void testAtomicReferenceGet() throws Exception {
+        testAtomicReferenceReconnectClusterRestart("atomic-ref-get", IgniteAtomicReference::get);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    @Test
+    public void testAtomicReferenceSet() throws Exception {
+        testAtomicReferenceReconnectClusterRestart("atomic-ref-set", atomic -> atomic.set(50L));
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    @Test
+    public void testAtomicReferenceCompareAndSet() throws Exception {
+        testAtomicReferenceReconnectClusterRestart(
+            "atomic-ref-compareAndSet",
+            atomic -> atomic.compareAndSet(1L, 50L));
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    @Test
+    public void testAtomicReferenceCompareAndSetAndGet() throws Exception {
+        testAtomicReferenceReconnectClusterRestart(
+            "atomic-ref-compareAndSetAndGet",
+            atomic -> ((GridCacheAtomicReferenceImpl)atomic).compareAndSetAndGet(1L, 50L));
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    @Test
+    public void testAtomicStampedGet() throws Exception {
+        testAtomicStampedReconnectClusterRestart(
+            "atomic-stamped-get",
+            IgniteAtomicStamped::get);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    @Test
+    public void testAtomicStampedSet() throws Exception {
+        testAtomicStampedReconnectClusterRestart(
+            "atomic-stamped-set",
+            atomic -> atomic.set("val", "stamp"));
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    @Test
+    public void testAtomicStampedCompareAndSet() throws Exception {
+        testAtomicStampedReconnectClusterRestart(
+            "atomic-stamped-compareAndSet",
+            atomic -> atomic.compareAndSet("val", "stamp", "val", "stamp"));
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    @Test
+    public void testAtomicStampedStamp() throws Exception {
+        testAtomicStampedReconnectClusterRestart(
+            "atomic-stamped-stamp",
+            IgniteAtomicStamped::stamp);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    @Test
+    public void testAtomicStampedValue() throws Exception {
+        testAtomicStampedReconnectClusterRestart(
+            "atomic-stamped-val",
+            IgniteAtomicStamped::value);
+    }
+
+    /**
+     * Tests atomic long operation provided by the the given {@code clo}.
+     *
+     * @param atomicName Name of atomic long.
+     * @param clo Closure that represents an operation.
+     * @throws Exception If failed.
+     */
+    private void testAtomicLongReconnectClusterRestart(
+        String atomicName,
+        final IgniteInClosure<IgniteAtomicLong> clo
+    ) throws Exception {
+        Ignite client = grid(serverCount());
+
+        assertTrue(client.cluster().localNode().isClient());
+
+        final IgniteAtomicLong atomic = client.atomicLong(atomicName, 1L, true);
+
+        assertNotNull(atomic);
+
+        assertEquals("Unexpected initial value.", 1L, atomic.get());
+
+        // Restart the cluster without waiting for rebalancing.
+        // It should lead to data loss because there are no backups in the atomic configuration.
+        for (int i = 0; i < serverCount(); ++i) {
+            grid(i).close();
+
+            startGrid(i);
+        }
+
+        GridTestUtils.assertThrows(
+            log,
+            () -> {
+                clo.apply(atomic);
+
+                return null;
+            },
+            IgniteException.class,
+            "Failed to find atomic long: " + atomicName);
+
+        assertTrue("Atomic long instance should be removed.", atomic.removed());
+
+        IgniteAtomicLong recreatedAtomicLong = client.atomicLong(atomicName, 100L, true);
+
+        assertEquals("Unexpected initial value.", 100L, recreatedAtomicLong.get());
+    }
+
+    /**
+     * Tests atomic reference operation provided by the the given {@code clo}.
+     *
+     * @param atomicName Name of atomic.
+     * @param clo Closure that represents an operation.
+     * @throws Exception If failed.
+     */
+    private void testAtomicReferenceReconnectClusterRestart(
+        String atomicName,
+        final IgniteInClosure<IgniteAtomicReference<Long>> clo
+    ) throws Exception {
+        Ignite client = grid(serverCount());
+
+        assertTrue(client.cluster().localNode().isClient());
+
+        final IgniteAtomicReference atomic = client.atomicReference(atomicName, 1L, true);
+
+        assertNotNull(atomic);
+
+        assertEquals("Unexpected initial value.", 1L, atomic.get());
+
+        // Restart the cluster without waiting for rebalancing.
+        // It should lead to data loss because there are no backups in the atomic configuration.
+        for (int i = 0; i < serverCount(); ++i) {
+            grid(i).close();
+
+            startGrid(i);
+        }
+
+        GridTestUtils.assertThrows(
+            log,
+            () -> {
+                clo.apply(atomic);
+
+                return null;
+            },
+            IgniteException.class,
+            "Failed to find atomic reference with given name: " + atomicName);
+
+        assertTrue("Atomic instance should be removed.", atomic.removed());
+
+        IgniteAtomicReference recreatedAtomic = client.atomicReference(atomicName, 100L, true);
+
+        assertEquals("Unexpected initial value.", 100L, recreatedAtomic.get());
+    }
+
+    /**
+     * Tests atomic stamped operation provided by the the given {@code clo}.
+     *
+     * @param atomicName Name of atomic.
+     * @param op Closure that represents an operation.
+     * @throws Exception If failed.
+     */
+    private void testAtomicStampedReconnectClusterRestart(
+        String atomicName,
+        final IgniteInClosure<IgniteAtomicStamped> op
+    ) throws Exception {
+        Ignite client = grid(serverCount());
+
+        assertTrue(client.cluster().localNode().isClient());
+
+        String initVal = "qwerty";
+        String initStamp = "asdfgh";
+
+        final IgniteAtomicStamped<String, String> atomic = client.atomicStamped(atomicName, initVal, initStamp, true);
+
+        assertNotNull(atomic);
+
+        assertEquals(initVal, atomic.value());
+        assertEquals(initStamp, atomic.stamp());
+        assertEquals(initVal, atomic.get().get1());
+        assertEquals(initStamp, atomic.get().get2());
+
+        // Restart the cluster without waiting for rebalancing.
+        // It should lead to data loss because there are no backups in the atomic configuration.
+        for (int i = 0; i < serverCount(); ++i) {
+            grid(i).close();
+
+            startGrid(i);
+        }
+
+        GridTestUtils.assertThrows(
+            log,
+            () -> {
+                op.apply(atomic);
+
+                return null;
+            },
+            IgniteException.class,
+            "Failed to find atomic stamped with given name: " + atomicName);
+
+        assertTrue("Atomic instance should be removed.", atomic.removed());
+
+        IgniteAtomicStamped<String, String> recreatedAtomic = client.atomicStamped(atomicName, initVal, initStamp, true);
+
+        assertNotNull(recreatedAtomic);
+
+        assertEquals(initVal, recreatedAtomic.value());
+        assertEquals(initStamp, recreatedAtomic.stamp());
+        assertEquals(initVal, recreatedAtomic.get().get1());
+        assertEquals(initStamp, recreatedAtomic.get().get2());
+    }
+}
diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteClientReconnectTestSuite.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteClientReconnectTestSuite.java
index 4c07262..0682ed4 100644
--- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteClientReconnectTestSuite.java
+++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteClientReconnectTestSuite.java
@@ -20,6 +20,7 @@ package org.apache.ignite.testsuites;
 import org.apache.ignite.internal.IgniteClientConnectAfterCommunicationFailureTest;
 import org.apache.ignite.internal.IgniteClientReconnectApiExceptionTest;
 import org.apache.ignite.internal.IgniteClientReconnectAtomicsTest;
+import org.apache.ignite.internal.IgniteClientReconnectAtomicsWithLostPartitionsTest;
 import org.apache.ignite.internal.IgniteClientReconnectBinaryContexTest;
 import org.apache.ignite.internal.IgniteClientReconnectCacheTest;
 import org.apache.ignite.internal.IgniteClientReconnectCollectionsTest;
@@ -48,6 +49,7 @@ import org.junit.runners.Suite;
     IgniteClientReconnectContinuousProcessorTest.class,
     IgniteClientReconnectComputeTest.class,
     IgniteClientReconnectAtomicsTest.class,
+    IgniteClientReconnectAtomicsWithLostPartitionsTest.class,
     IgniteClientReconnectCollectionsTest.class,
     IgniteClientReconnectServicesTest.class,
     IgniteClientReconnectStreamerTest.class,