petrov-mg commented on code in PR #12656:
URL: https://github.com/apache/ignite/pull/12656#discussion_r2728000501
##########
modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicAbstractUpdateFuture.java:
##########
@@ -402,8 +405,10 @@ final void map(ClusterNode nearNode,
needReplyToNear = true;
}
- // If there are readers updates then nearNode should not finish before
primary response received.
- sendDhtRequests(nearNode, ret, !readersOnlyNodes);
+ // "Near" node should not finish until it receives a response from
primary node in the following cases:
+ // - only "near cache" is updated
+ // - primary failed to process any keys
+ sendDhtRequests(nearNode, ret, !(readersOnlyNodes ||
hasPrimaryUpdErrors));
Review Comment:
The last argument of `sendDhtRequests` method is `boolean sndRes`. It
controls whether backups, after processing GridDhtAtomicUpdateRequest, will
send a response to the `near` node with `hasResult` flag set to true or false.
If `near` node receives responses from all backups with
`GridDhtAtomicNearResponse#hasResult`set to true - it can complete cache
operation without waiting for a response from the primary node
(GridNearAtomicUpdateFuture#onDhtResponse).
Problem this PR is intended to solve - `near` node revceives responses from
all backups and completes `putAll` operation before primary node sends
GridDhtAtomicNearResponse message with a list of entries that it failed to
process. `near` node receives no information from backup nodes that primary
node failed to process some entries but completes putAll operation
successfully. Near node must rise exception or successfully insert all
inserting entries.
So `!(readersOnlyNodes || hasPrimaryUpdErrors)` makes `near node` not to
complete `putAll` operation even if it receives all responses from backup
nodes, but wait for the responce from primary node (it contains entries which
processing is failed).
##########
modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/IgniteCacheAtomicFullSyncPartialUpdateAllTest.java:
##########
@@ -0,0 +1,163 @@
+/*
+ * 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.distributed.dht.atomic;
+
+import java.util.Map;
+import java.util.TreeMap;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.atomic.AtomicBoolean;
+import javax.cache.Cache;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.cache.CacheInterceptor;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.internal.IgniteEx;
+import org.apache.ignite.internal.IgniteInternalFuture;
+import
org.apache.ignite.internal.processors.cache.distributed.GridCacheModuloAffinityFunction;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.lang.IgniteBiTuple;
+import org.apache.ignite.resources.IgniteInstanceResource;
+import org.apache.ignite.testframework.GridTestUtils;
+import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+import org.jetbrains.annotations.Nullable;
+import org.junit.Test;
+
+import static java.util.Collections.singletonMap;
+import static java.util.concurrent.TimeUnit.MILLISECONDS;
+import static org.apache.ignite.cache.CacheAtomicityMode.ATOMIC;
+import static org.apache.ignite.cache.CacheWriteSynchronizationMode.FULL_SYNC;
+import static
org.apache.ignite.internal.processors.cache.GridCacheUtils.cacheId;
+import static
org.apache.ignite.internal.processors.cache.distributed.GridCacheModuloAffinityFunction.IDX_ATTR;
+
+/** */
+public class IgniteCacheAtomicFullSyncPartialUpdateAllTest extends
GridCommonAbstractTest {
+ /** {@inheritDoc} */
+ @Override protected IgniteConfiguration getConfiguration(String
igniteInstanceName) throws Exception {
+ return super.getConfiguration(igniteInstanceName)
+ .setUserAttributes(singletonMap(IDX_ATTR,
getTestIgniteInstanceIndex(igniteInstanceName)));
+ }
+
+ /** */
+ @Test
+ public void test() throws Exception {
+ startGrid(0);
+ startGrid(1);
+ startGrid(2);
Review Comment:
Done.
--
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
To unsubscribe, e-mail: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]