sanpwc commented on code in PR #6807:
URL: https://github.com/apache/ignite-3/pull/6807#discussion_r2445694657
##########
modules/transactions/src/main/java/org/apache/ignite/internal/tx/impl/TxMessageSender.java:
##########
@@ -207,18 +208,18 @@ public CompletableFuture<TransactionMeta>
resolveTxStateFromCommitPartition(
/**
* Send TxStateCoordinatorRequest.
*
- * @param primaryConsistentId Node id to send the request to.
+ * @param coordinatorClusterNode Node id to send the request to.
Review Comment:
It's not the "Node **Id**" any longer, just a recipient node.
##########
modules/table/src/test/java/org/apache/ignite/internal/table/distributed/replicator/TransactionStateResolverTest.java:
##########
@@ -0,0 +1,246 @@
+/*
+ * 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.table.distributed.replicator;
+
+import static java.util.concurrent.CompletableFuture.completedFuture;
+import static org.apache.ignite.internal.hlc.HybridTimestamp.hybridTimestamp;
+import static
org.apache.ignite.internal.testframework.matchers.CompletableFutureMatcher.willCompleteSuccessfully;
+import static org.hamcrest.MatcherAssert.assertThat;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.mockito.ArgumentMatchers.any;
+import static org.mockito.ArgumentMatchers.anyLong;
+import static org.mockito.ArgumentMatchers.eq;
+import static org.mockito.Mockito.atLeastOnce;
+import static org.mockito.Mockito.lenient;
+import static org.mockito.Mockito.never;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.when;
+
+import java.util.UUID;
+import java.util.concurrent.CompletableFuture;
+import java.util.function.Function;
+import org.apache.ignite.internal.hlc.ClockService;
+import org.apache.ignite.internal.hlc.HybridClock;
+import org.apache.ignite.internal.hlc.HybridClockImpl;
+import org.apache.ignite.internal.hlc.HybridTimestamp;
+import org.apache.ignite.internal.hlc.TestClockService;
+import org.apache.ignite.internal.network.ClusterNodeImpl;
+import org.apache.ignite.internal.network.ClusterNodeResolver;
+import org.apache.ignite.internal.network.InternalClusterNode;
+import org.apache.ignite.internal.network.MessagingService;
+import org.apache.ignite.internal.network.RecipientLeftException;
+import org.apache.ignite.internal.placementdriver.PlacementDriver;
+import org.apache.ignite.internal.placementdriver.TestReplicaMetaImpl;
+import org.apache.ignite.internal.replicator.TablePartitionId;
+import org.apache.ignite.internal.replicator.message.ReplicaMessagesFactory;
+import org.apache.ignite.internal.testframework.BaseIgniteAbstractTest;
+import org.apache.ignite.internal.tx.TransactionMeta;
+import org.apache.ignite.internal.tx.TxManager;
+import org.apache.ignite.internal.tx.TxState;
+import org.apache.ignite.internal.tx.TxStateMeta;
+import org.apache.ignite.internal.tx.impl.TxMessageSender;
+import org.apache.ignite.internal.tx.message.TransactionMetaMessage;
+import org.apache.ignite.internal.tx.message.TxMessagesFactory;
+import org.apache.ignite.internal.tx.message.TxStateResponse;
+import org.apache.ignite.network.NetworkAddress;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.extension.ExtendWith;
+import org.mockito.ArgumentCaptor;
+import org.mockito.Mock;
+import org.mockito.junit.jupiter.MockitoExtension;
+
+/**
+ * Tests for {@link TransactionStateResolver}.
+ */
+@ExtendWith(MockitoExtension.class)
+public class TransactionStateResolverTest extends BaseIgniteAbstractTest {
+ private static final TxMessagesFactory TX_MESSAGES_FACTORY = new
TxMessagesFactory();
+
+ private static final InternalClusterNode COORDINATOR_NODE =
+ new ClusterNodeImpl(UUID.randomUUID(), "coordinator", new
NetworkAddress("127.0.0.1", 10800), null);
+
+ @Mock
+ private TxManager txManager;
+
+ @Mock
+ private ClusterNodeResolver clusterNodeResolver;
+
+ @Mock
+ private MessagingService messagingService;
+
+ @Mock
+ private PlacementDriver placementDriver;
+
+ @Mock
+ private TxMessageSender txMessageSender;
+
+ private final HybridClock clock = new HybridClockImpl();
+
+ private final ClockService clockService = new TestClockService(clock);
+
+ private TransactionStateResolver resolver;
+
+ @BeforeEach
Review Comment:
Please also add
```
@AfterEach
public void clearMocks() {
Mockito.framework().clearInlineMocks();
}
```
##########
modules/table/src/main/java/org/apache/ignite/internal/table/distributed/replicator/TransactionStateResolver.java:
##########
@@ -213,18 +215,20 @@ private void resolveTxStateFromTxCoordinator(
updateLocalTxMapAfterDistributedStateResolved(txId, txMetaFuture);
InternalClusterNode coordinator = coordinatorId == null ? null :
clusterNodeResolver.getById(coordinatorId);
-
if (coordinator == null) {
// This means the coordinator node have either left the cluster or
restarted.
markAbandoned(txId);
resolveTxStateFromCommitPartition(txId, commitGrpId, txMetaFuture);
} else {
- txMessageSender.resolveTxStateFromCoordinator(coordinator.name(),
txId, timestamp)
+ txMessageSender.resolveTxStateFromCoordinator(coordinator, txId,
timestamp)
.whenComplete((response, e) -> {
- if (e == null) {
-
txMetaFuture.complete(asTransactionMeta(response.txStateMeta()));
+ if (e == null && response.txStateMeta() != null) {
+
txMetaFuture.complete(asTransactionMeta(Objects.requireNonNull(response.txStateMeta())));
Review Comment:
It's better to extract `Objects.requireNonNull` and add a message to the
verification.
##########
modules/table/src/main/java/org/apache/ignite/internal/table/distributed/replicator/TransactionStateResolver.java:
##########
@@ -213,18 +215,20 @@ private void resolveTxStateFromTxCoordinator(
updateLocalTxMapAfterDistributedStateResolved(txId, txMetaFuture);
InternalClusterNode coordinator = coordinatorId == null ? null :
clusterNodeResolver.getById(coordinatorId);
-
if (coordinator == null) {
// This means the coordinator node have either left the cluster or
restarted.
markAbandoned(txId);
resolveTxStateFromCommitPartition(txId, commitGrpId, txMetaFuture);
} else {
- txMessageSender.resolveTxStateFromCoordinator(coordinator.name(),
txId, timestamp)
+ txMessageSender.resolveTxStateFromCoordinator(coordinator, txId,
timestamp)
.whenComplete((response, e) -> {
- if (e == null) {
-
txMetaFuture.complete(asTransactionMeta(response.txStateMeta()));
+ if (e == null && response.txStateMeta() != null) {
+
txMetaFuture.complete(asTransactionMeta(Objects.requireNonNull(response.txStateMeta())));
} else {
+ if (e != null && e.getCause() instanceof
RecipientLeftException) {
Review Comment:
Are you sure that `RecipientLeftException` is the only exception that is
possible here?
As far as I know aforementioned exception will be thrown if there's a node
with given name, however it has another id. What about node absence? Such test
is also missing I guess.
--
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]