You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by vp...@apache.org on 2022/06/23 09:54:15 UTC

[ignite] branch master updated: IGNITE-17151 Inconsistent keys after deletion during rebalance (#10083)

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

vpyatkov 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 0a2113ef94d IGNITE-17151 Inconsistent keys after deletion during rebalance (#10083)
0a2113ef94d is described below

commit 0a2113ef94d846e5f4ea97f9fe489abd5e5266de
Author: Vladislav Pyatkov <vl...@gmail.com>
AuthorDate: Thu Jun 23 12:54:04 2022 +0300

    IGNITE-17151 Inconsistent keys after deletion during rebalance (#10083)
---
 .../distributed/dht/GridDhtTxPrepareFuture.java    |   3 -
 .../cache/transactions/IgniteTxEntry.java          |   6 +
 .../testsuites/DeletionDuringRebalanceTest.java    | 220 +++++++++++++++++++++
 .../IgniteBinaryCacheQueryTestSuite.java           |   1 +
 4 files changed, 227 insertions(+), 3 deletions(-)

diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxPrepareFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxPrepareFuture.java
index a20532bce11..6667af42699 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxPrepareFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxPrepareFuture.java
@@ -546,9 +546,6 @@ public final class GridDhtTxPrepareFuture extends GridCacheCompoundFuture<Ignite
                             /*keepBinary*/true);
                     }
 
-                    if (oldVal != null)
-                        oldVal.prepareMarshal(cacheCtx.cacheObjectContext());
-
                     txEntry.oldValue(oldVal);
                 }
             }
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxEntry.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxEntry.java
index 5b8ab389e9d..3780d842fc0 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxEntry.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxEntry.java
@@ -945,6 +945,9 @@ public class IgniteTxEntry implements GridPeerDeployAware, Message {
         }
         else
             expiryPlcBytes = null;
+
+        if (oldVal != null)
+            oldVal.marshal(context());
     }
 
     /**
@@ -1001,6 +1004,9 @@ public class IgniteTxEntry implements GridPeerDeployAware, Message {
 
         if (expiryPlcBytes != null && expiryPlc == null)
             expiryPlc = U.unmarshal(ctx, expiryPlcBytes, U.resolveClassLoader(clsLdr, ctx.gridConfig()));
+
+        if (hasOldValue())
+            oldVal.unmarshal(coctx, clsLdr);
     }
 
     /**
diff --git a/modules/indexing/src/test/java/org/apache/ignite/testsuites/DeletionDuringRebalanceTest.java b/modules/indexing/src/test/java/org/apache/ignite/testsuites/DeletionDuringRebalanceTest.java
new file mode 100644
index 00000000000..8bf232b75df
--- /dev/null
+++ b/modules/indexing/src/test/java/org/apache/ignite/testsuites/DeletionDuringRebalanceTest.java
@@ -0,0 +1,220 @@
+/*
+ * 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.testsuites;
+
+import java.util.Collections;
+import java.util.LinkedHashMap;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.cache.CacheAtomicityMode;
+import org.apache.ignite.cache.QueryEntity;
+import org.apache.ignite.cache.affinity.AffinityKeyMapped;
+import org.apache.ignite.cache.affinity.rendezvous.RendezvousAffinityFunction;
+import org.apache.ignite.cache.query.SqlFieldsQuery;
+import org.apache.ignite.cache.query.annotations.QuerySqlField;
+import org.apache.ignite.cluster.ClusterState;
+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.configuration.SqlConfiguration;
+import org.apache.ignite.indexing.IndexingQueryEngineConfiguration;
+import org.apache.ignite.internal.IgniteEx;
+import org.apache.ignite.internal.TestRecordingCommunicationSpi;
+import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionSupplyMessage;
+import org.apache.ignite.internal.util.typedef.internal.CU;
+import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+import org.junit.Test;
+
+import static org.apache.ignite.internal.TestRecordingCommunicationSpi.spi;
+
+/**
+ * The test checks the SQL delete operation during a rebalance in progress.
+ */
+public class DeletionDuringRebalanceTest extends GridCommonAbstractTest {
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception {
+        return super.getConfiguration(igniteInstanceName)
+            .setConsistentId(igniteInstanceName)
+            .setSqlConfiguration(
+                new SqlConfiguration().setQueryEnginesConfiguration(
+                    new IndexingQueryEngineConfiguration()))
+            .setCommunicationSpi(new TestRecordingCommunicationSpi())
+            .setDataStorageConfiguration(
+                new DataStorageConfiguration().setDefaultDataRegionConfiguration(
+                    new DataRegionConfiguration().setPersistenceEnabled(true)
+                )
+            )
+            .setCacheConfiguration(
+                simpleCacheConfiguration()
+            );
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTestsStarted() throws Exception {
+        super.beforeTestsStarted();
+
+        cleanPersistenceDir();
+    }
+
+    /**
+     * Gets a configuration object for the cahce with default name {@code DEFAULT_CACHE_NAME}.
+     *
+     * @return Cache configuration object.
+     */
+    private CacheConfiguration simpleCacheConfiguration() {
+        CacheConfiguration cfg = new CacheConfiguration()
+            .setName(DEFAULT_CACHE_NAME)
+            .setAffinity(new RendezvousAffinityFunction(false, 16))
+            .setAtomicityMode(CacheAtomicityMode.TRANSACTIONAL)
+            .setBackups(1);
+
+        QueryEntity entity = new QueryEntity()
+            .setKeyType(Integer.class.getName())
+            .setValueType(Subscription.class.getName());
+
+        LinkedHashMap<String, String> fields = new LinkedHashMap<>();
+
+        fields.put("id", Integer.class.getName());
+        fields.put("surname", String.class.getName());
+        fields.put("orgId", Integer.class.getName());
+
+        entity.setFields(fields);
+
+        cfg.setQueryEntities(Collections.singleton(
+            entity
+        ));
+
+        return cfg;
+    }
+
+    /**
+     * The test starts two nodes, preloads data and redtarts one. During the restarted node is rebalancing, SQL delete
+     * operation happens. After the topology got stable, the test checks partition consistency.
+     *
+     * @throws Exception If failed.
+     */
+    @Test
+    public void testSimple() throws Exception {
+        IgniteEx ignite = startGrids(2);
+
+        ignite.cluster().state(ClusterState.ACTIVE);
+
+        IgniteCache<Integer, Subscription> cache = ignite.cache(DEFAULT_CACHE_NAME);
+
+        for (int i = 0; i < 20; i++) {
+            cache.put(i, new Subscription(i, "Ivanov", i % 10));
+        }
+
+        forceCheckpoint();
+
+        stopGrid(1);
+
+        for (int i = 2_000; i < 2_010; i++) {
+            cache.put(i, new Subscription(i, "Ivanov", i % 10));
+        }
+
+        forceCheckpoint();
+
+        spi(ignite).blockMessages((node, message) -> message instanceof GridDhtPartitionSupplyMessage &&
+            ((GridDhtPartitionSupplyMessage)message).groupId() == CU.cacheId(DEFAULT_CACHE_NAME));
+
+        IgniteEx ignite1 = startGrid(1);
+
+        cache.query(new SqlFieldsQuery("delete from Subscription where id = 8"));
+
+        spi(ignite).stopBlock();
+
+        awaitPartitionMapExchange();
+
+        assertNull(cache.get(8));
+
+        assertPartitionsSame(idleVerify(ignite1, DEFAULT_CACHE_NAME));
+    }
+
+    /**
+     * Class for uploading to cache.
+     */
+    private class Subscription {
+        /** Id. */
+        @QuerySqlField
+        private Integer id;
+
+        /** Surname */
+        @QuerySqlField
+        private String surname;
+
+        /** Organization id. */
+        @QuerySqlField
+        @AffinityKeyMapped
+        private Integer orgId;
+
+        /**
+         * The constructor.
+         *
+         * @param id Id.
+         * @param surname Surname.
+         * @param orgId Organization id.
+         */
+        Subscription(Integer id, String surname, Integer orgId) {
+            this.id = id;
+            this.surname = surname;
+            this.orgId = orgId;
+        }
+
+        /**
+         * @return Id.
+         */
+        public Integer getId() {
+            return id;
+        }
+
+        /**
+         * @param id Id.
+         */
+        public void setId(Integer id) {
+            this.id = id;
+        }
+
+        /**
+         * @return Surname.
+         */
+        public String getSurname() {
+            return surname;
+        }
+
+        /**
+         * @param surname Surname.
+         */
+        public void setSurname(String surname) {
+            this.surname = surname;
+        }
+
+        /**
+         * @return Organization id.
+         */
+        public Integer getOrgId() {
+            return orgId;
+        }
+
+        /**
+         * @param orgId Organization id.
+         */
+        public void setOrgId(Integer orgId) {
+            this.orgId = orgId;
+        }
+    }
+}
diff --git a/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteBinaryCacheQueryTestSuite.java b/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteBinaryCacheQueryTestSuite.java
index 7f3db04bda3..413141bda9b 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteBinaryCacheQueryTestSuite.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteBinaryCacheQueryTestSuite.java
@@ -176,6 +176,7 @@ import org.junit.runners.Suite;
     ReplicatedSqlTest.class,
     ReplicatedSqlCustomPartitionsTest.class,
     CheckWarnJoinPartitionedTables.class,
+    DeletionDuringRebalanceTest.class,
 
     SqlParserCreateIndexSelfTest.class,
     SqlParserDropIndexSelfTest.class,