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

sanpwc pushed a commit to branch main
in repository https://gitbox.apache.org/repos/asf/ignite-3.git


The following commit(s) were added to refs/heads/main by this push:
     new 8867d35821 IGNITE-21418 Add more logs to debug this issue (#3831)
8867d35821 is described below

commit 8867d35821ef12f47d43744bc84a3cfec791649e
Author: Cyrill <cyrill.si...@gmail.com>
AuthorDate: Wed May 29 11:54:51 2024 +0300

    IGNITE-21418 Add more logs to debug this issue (#3831)
---
 .../storage/impl/TestMvPartitionStorage.java       |  18 ++-
 .../apache/ignite/distributed/ItTxTestCluster.java |   8 ++
 .../ignite/internal/table/TxAbstractTest.java      | 142 +++++++++++++++++++--
 .../tx/storage/state/test/TestTxStateStorage.java  |   8 ++
 4 files changed, 159 insertions(+), 17 deletions(-)

diff --git 
a/modules/storage-api/src/testFixtures/java/org/apache/ignite/internal/storage/impl/TestMvPartitionStorage.java
 
b/modules/storage-api/src/testFixtures/java/org/apache/ignite/internal/storage/impl/TestMvPartitionStorage.java
index 3036127766..0ca9ed4aea 100644
--- 
a/modules/storage-api/src/testFixtures/java/org/apache/ignite/internal/storage/impl/TestMvPartitionStorage.java
+++ 
b/modules/storage-api/src/testFixtures/java/org/apache/ignite/internal/storage/impl/TestMvPartitionStorage.java
@@ -44,6 +44,7 @@ import 
org.apache.ignite.internal.storage.TxIdMismatchException;
 import org.apache.ignite.internal.storage.gc.GcEntry;
 import org.apache.ignite.internal.storage.util.LocalLocker;
 import org.apache.ignite.internal.storage.util.LockByRowId;
+import org.apache.ignite.internal.tostring.S;
 import org.apache.ignite.internal.util.Cursor;
 import org.jetbrains.annotations.Nullable;
 
@@ -92,12 +93,12 @@ public class TestMvPartitionStorage implements 
MvPartitionStorage {
     }
 
     private static class VersionChain implements GcEntry {
-        final RowId rowId;
-        final @Nullable BinaryRow row;
-        final @Nullable HybridTimestamp ts;
-        final @Nullable UUID txId;
-        final @Nullable Integer commitTableId;
-        final int commitPartitionId;
+        private final RowId rowId;
+        private final @Nullable BinaryRow row;
+        private final @Nullable HybridTimestamp ts;
+        private final @Nullable UUID txId;
+        private final @Nullable Integer commitTableId;
+        private final int commitPartitionId;
         volatile @Nullable VersionChain next;
 
         VersionChain(
@@ -143,6 +144,11 @@ public class TestMvPartitionStorage implements 
MvPartitionStorage {
 
             return ts;
         }
+
+        @Override
+        public String toString() {
+            return S.toString(VersionChain.class, this);
+        }
     }
 
     @Override
diff --git 
a/modules/table/src/testFixtures/java/org/apache/ignite/distributed/ItTxTestCluster.java
 
b/modules/table/src/testFixtures/java/org/apache/ignite/distributed/ItTxTestCluster.java
index d1a9fcc49b..84030d69bf 100644
--- 
a/modules/table/src/testFixtures/java/org/apache/ignite/distributed/ItTxTestCluster.java
+++ 
b/modules/table/src/testFixtures/java/org/apache/ignite/distributed/ItTxTestCluster.java
@@ -1061,4 +1061,12 @@ public class ItTxTestCluster {
     public Map<String, HybridClock> clocks() {
         return clocks;
     }
+
+    public HybridClock clientClock() {
+        return clientClock;
+    }
+
+    public Map<String, ReplicaManager> replicaManagers() {
+        return replicaManagers;
+    }
 }
diff --git 
a/modules/table/src/testFixtures/java/org/apache/ignite/internal/table/TxAbstractTest.java
 
b/modules/table/src/testFixtures/java/org/apache/ignite/internal/table/TxAbstractTest.java
index 70aa94925b..960d6f2e65 100644
--- 
a/modules/table/src/testFixtures/java/org/apache/ignite/internal/table/TxAbstractTest.java
+++ 
b/modules/table/src/testFixtures/java/org/apache/ignite/internal/table/TxAbstractTest.java
@@ -50,7 +50,9 @@ import java.util.Random;
 import java.util.UUID;
 import java.util.concurrent.CompletableFuture;
 import java.util.concurrent.CompletionException;
+import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.CyclicBarrier;
+import java.util.concurrent.ExecutionException;
 import java.util.concurrent.Flow;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicBoolean;
@@ -58,9 +60,12 @@ import java.util.concurrent.atomic.AtomicReference;
 import java.util.concurrent.atomic.LongAdder;
 import java.util.function.BiConsumer;
 import java.util.function.Consumer;
+import java.util.function.Function;
 import org.apache.ignite.distributed.ItTxTestCluster;
 import 
org.apache.ignite.internal.configuration.testframework.ConfigurationExtension;
 import 
org.apache.ignite.internal.configuration.testframework.InjectConfiguration;
+import org.apache.ignite.internal.hlc.HybridClockImpl;
+import org.apache.ignite.internal.hlc.HybridTimestamp;
 import org.apache.ignite.internal.manager.ComponentContext;
 import org.apache.ignite.internal.network.ClusterService;
 import org.apache.ignite.internal.network.NodeFinder;
@@ -71,7 +76,10 @@ import org.apache.ignite.internal.raft.Peer;
 import org.apache.ignite.internal.raft.RaftNodeId;
 import org.apache.ignite.internal.raft.configuration.RaftConfiguration;
 import org.apache.ignite.internal.raft.server.impl.JraftServerImpl;
+import org.apache.ignite.internal.replicator.Replica;
+import org.apache.ignite.internal.replicator.ReplicaManager;
 import org.apache.ignite.internal.replicator.ReplicaService;
+import org.apache.ignite.internal.replicator.ReplicationGroupId;
 import org.apache.ignite.internal.replicator.TablePartitionId;
 import 
org.apache.ignite.internal.replicator.configuration.ReplicationConfiguration;
 import org.apache.ignite.internal.schema.BinaryRow;
@@ -82,7 +90,13 @@ import 
org.apache.ignite.internal.schema.configuration.StorageUpdateConfiguratio
 import org.apache.ignite.internal.schema.marshaller.TupleMarshallerImpl;
 import org.apache.ignite.internal.schema.row.Row;
 import org.apache.ignite.internal.storage.MvPartitionStorage;
+import org.apache.ignite.internal.storage.RowId;
+import org.apache.ignite.internal.storage.impl.TestMvPartitionStorage;
+import org.apache.ignite.internal.storage.index.IndexStorage;
+import org.apache.ignite.internal.storage.index.impl.TestHashIndexStorage;
+import 
org.apache.ignite.internal.table.distributed.TableSchemaAwareIndexStorage;
 import org.apache.ignite.internal.table.distributed.raft.PartitionListener;
+import 
org.apache.ignite.internal.table.distributed.replicator.PartitionReplicaListener;
 import org.apache.ignite.internal.testframework.IgniteAbstractTest;
 import org.apache.ignite.internal.testframework.IgniteTestUtils;
 import org.apache.ignite.internal.tx.HybridTimestampTracker;
@@ -98,8 +112,12 @@ import org.apache.ignite.internal.tx.TxStateMeta;
 import org.apache.ignite.internal.tx.configuration.TransactionConfiguration;
 import org.apache.ignite.internal.tx.impl.IgniteTransactionsImpl;
 import org.apache.ignite.internal.tx.impl.ReadWriteTransactionImpl;
+import org.apache.ignite.internal.tx.impl.TxManagerImpl;
+import org.apache.ignite.internal.tx.impl.VolatileTxStateMetaStorage;
+import org.apache.ignite.internal.tx.storage.state.TxStateStorage;
 import org.apache.ignite.internal.type.NativeTypes;
 import org.apache.ignite.internal.util.CollectionUtils;
+import org.apache.ignite.internal.util.Lazy;
 import org.apache.ignite.internal.util.Pair;
 import org.apache.ignite.lang.IgniteException;
 import org.apache.ignite.network.ClusterNode;
@@ -276,19 +294,27 @@ public abstract class TxAbstractTest extends 
IgniteAbstractTest {
 
     /** {@inheritDoc} */
     protected TxManager txManager(TableViewInternal t) {
-        CompletableFuture<ReplicaMeta> primaryReplicaFuture = 
txTestCluster.placementDriver().getPrimaryReplica(
-                new TablePartitionId(t.tableId(), 0),
-                
txTestCluster.clocks().get(txTestCluster.localNodeName()).now());
+        String leaseHolder = primaryNode(t);
 
-        assertThat(primaryReplicaFuture, willCompleteSuccessfully());
+        assertNotNull(leaseHolder, "Table primary node should not be null");
 
-        TxManager manager = 
txTestCluster.txManagers().get(primaryReplicaFuture.join().getLeaseholder());
+        TxManager manager = txTestCluster.txManagers().get(leaseHolder);
 
         assertNotNull(manager);
 
         return manager;
     }
 
+    private @Nullable String primaryNode(TableViewInternal t) {
+        CompletableFuture<ReplicaMeta> primaryReplicaFuture = 
txTestCluster.placementDriver().getPrimaryReplica(
+                new TablePartitionId(t.tableId(), 0),
+                
txTestCluster.clocks().get(txTestCluster.localNodeName()).now());
+
+        assertThat(primaryReplicaFuture, willCompleteSuccessfully());
+
+        return primaryReplicaFuture.join().getLeaseholder();
+    }
+
     /**
      * Check the storage of partition is the same across all nodes. The 
checking is based on {@link MvPartitionStorage#lastAppliedIndex()}
      * that is increased on all update storage operation.
@@ -366,7 +392,7 @@ public abstract class TxAbstractTest extends 
IgniteAbstractTest {
     }
 
     @Test
-    public void testRepeatedCommitRollbackAfterUpdateWithException() throws 
Exception {
+    public void testRepeatedCommitRollbackAfterUpdateWithException() {
         injectFailureOnNextOperation(accounts);
 
         InternalTransaction tx = (InternalTransaction) 
igniteTransactions.begin();
@@ -383,7 +409,7 @@ public abstract class TxAbstractTest extends 
IgniteAbstractTest {
     }
 
     @Test
-    public void testRepeatedCommitRollbackAfterRollbackWithException() throws 
Exception {
+    public void testRepeatedCommitRollbackAfterRollbackWithException() {
         InternalTransaction tx = (InternalTransaction) 
igniteTransactions.begin();
 
         accounts.recordView().upsert(tx, makeValue(1, 100.));
@@ -441,9 +467,17 @@ public abstract class TxAbstractTest extends 
IgniteAbstractTest {
         deleteUpsertAll().rollback();
 
         var res1 = accounts.recordView().get(null, makeKey(1));
+
+        // Here we try to catch the flaky NPE problem.
+        checkIfNull(res1, transaction -> 
accounts.recordView().get(transaction, makeKey(1)));
+
         assertEquals(100., res1.doubleValue("balance"), "tuple =[" + res1 + 
"]");
 
         var res2 = accounts.recordView().get(null, makeKey(2));
+
+        // Here we try to catch the flaky NPE problem.
+        checkIfNull(res2, transaction -> 
accounts.recordView().get(transaction, makeKey(2)));
+
         assertEquals(100., res2.doubleValue("balance"), "tuple =[" + res2 + 
"]");
     }
 
@@ -455,9 +489,16 @@ public abstract class TxAbstractTest extends 
IgniteAbstractTest {
         accounts.recordView().upsertAll(null, tuples);
 
         var res1 = accounts.recordView().get(null, makeKey(1));
+
+        // Here we try to catch the flaky NPE problem.
+        checkIfNull(res1, transaction -> 
accounts.recordView().get(transaction, makeKey(1)));
+
         assertEquals(100., res1.doubleValue("balance"), "tuple =[" + res1 + 
"]");
 
         var res2 = accounts.recordView().get(null, makeKey(2));
+        // Here we try to catch the flaky NPE problem.
+        checkIfNull(res2, transaction -> 
accounts.recordView().get(transaction, makeKey(2)));
+
         assertEquals(100., res2.doubleValue("balance"), "tuple =[" + res2 + 
"]");
 
         InternalTransaction tx = (InternalTransaction) 
igniteTransactions.begin();
@@ -477,6 +518,85 @@ public abstract class TxAbstractTest extends 
IgniteAbstractTest {
         return tx;
     }
 
+    private void checkIfNull(Tuple res1, Function<Transaction, Tuple> 
retryFunc) {
+        if (res1 != null) {
+            return;
+        }
+
+        logger().error("Found null value after upsertAll.");
+
+        printDebugInfo();
+
+        var resExplicit = igniteTransactions.runInTransaction(retryFunc);
+
+        logger().info("Same explicit call resulted in {} value", resExplicit);
+
+        fail("Found null value after upsertAll.");
+    }
+
+    private void printDebugInfo() {
+        logger().info("Primary node for accounts table is [node={}]", 
primaryNode(accounts));
+        logger().info("Cluster times:");
+        long latestTime = 
IgniteTestUtils.getFieldValue(txTestCluster.clientClock(), 
HybridClockImpl.class, "latestTime");
+        logger().info("Client clock [time={}]", 
HybridTimestamp.hybridTimestamp(latestTime));
+
+        txTestCluster.clocks().forEach((name, hybridClock) -> {
+            long time = IgniteTestUtils.getFieldValue(hybridClock, 
HybridClockImpl.class, "latestTime");
+
+            logger().info("Cluster clock [node={}, time={}]", name, 
HybridTimestamp.hybridTimestamp(time));
+        });
+
+        logger().info("Replica info:");
+        txTestCluster.replicaManagers().forEach((name, replicaManager) -> {
+            ConcurrentHashMap<ReplicationGroupId, CompletableFuture<Replica>> 
replicas =
+                    IgniteTestUtils.getFieldValue(replicaManager, 
ReplicaManager.class, "replicas");
+
+            replicas.forEach((replicationGroupId, replicaCompletableFuture) ->
+                    printReplicaInfo(name, replicationGroupId, 
replicaCompletableFuture)
+            );
+
+            TxManager txManager = txTestCluster.txManagers().get(name);
+            VolatileTxStateMetaStorage volatileTxState =
+                    IgniteTestUtils.getFieldValue(txManager, 
TxManagerImpl.class, "txStateVolatileStorage");
+
+            ConcurrentHashMap<UUID, TxStateMeta> txStateMap =
+                    IgniteTestUtils.getFieldValue(volatileTxState, 
VolatileTxStateMetaStorage.class, "txStateMap");
+            logger().info("Volatile tx state data [node={}, data={}]", name, 
txStateMap);
+        });
+    }
+
+    private void printReplicaInfo(String name, ReplicationGroupId 
replicationGroupId, CompletableFuture<Replica> replicaCompletableFuture) {
+        if (!replicaCompletableFuture.isDone()) {
+            logger().info("Replica is not ready [node={}, groupId={}]", name, 
replicationGroupId);
+
+            return;
+        }
+
+        Replica replica;
+        try {
+            replica = replicaCompletableFuture.get();
+        } catch (InterruptedException | ExecutionException e) {
+            throw new RuntimeException(e);
+        }
+
+        PartitionReplicaListener listener = 
IgniteTestUtils.getFieldValue(replica, Replica.class, "listener");
+        TestMvPartitionStorage storage = 
IgniteTestUtils.getFieldValue(listener, PartitionReplicaListener.class, 
"mvDataStorage");
+        Map<RowId, ?> map = IgniteTestUtils.getFieldValue(storage, 
TestMvPartitionStorage.class, "map");
+
+        logger().info("Partition data [node={}, groupId={}, data={}]", name, 
replicationGroupId, map);
+
+        Lazy<TableSchemaAwareIndexStorage> indexStorageLazy =
+                IgniteTestUtils.getFieldValue(listener, 
PartitionReplicaListener.class, "pkIndexStorage");
+        IndexStorage indexStorage = indexStorageLazy.get().storage();
+        Map<RowId, ?> indexMap = IgniteTestUtils.getFieldValue(indexStorage, 
TestHashIndexStorage.class, "index");
+
+        logger().info("Index data [node={}, groupId={}, data={}]", name, 
replicationGroupId, indexMap);
+
+        TxStateStorage stateStorage = IgniteTestUtils.getFieldValue(listener, 
PartitionReplicaListener.class, "txStateStorage");
+
+        logger().info("Tx state data [node={}, groupId={}, data={}]", name, 
replicationGroupId, stateStorage);
+    }
+
     @Test
     public void testMixedPutGet() throws TransactionException {
         accounts.recordView().upsert(null, makeValue(1, BALANCE_1));
@@ -810,7 +930,7 @@ public abstract class TxAbstractTest extends 
IgniteAbstractTest {
     }
 
     @Test
-    public void testSimpleConflict() throws Exception {
+    public void testSimpleConflict() {
         accounts.recordView().upsert(null, makeValue(1, 100.));
 
         Transaction tx1 = igniteTransactions.begin();
@@ -1282,7 +1402,7 @@ public abstract class TxAbstractTest extends 
IgniteAbstractTest {
     }
 
     @Test
-    public void testReorder() throws Exception {
+    public void testReorder() {
         accounts.recordView().upsert(null, makeValue(1, 100.));
 
         InternalTransaction tx1 = (InternalTransaction) 
igniteTransactions.begin();
@@ -1502,12 +1622,12 @@ public abstract class TxAbstractTest extends 
IgniteAbstractTest {
     }
 
     @Test
-    public void testScan() throws Exception {
+    public void testScan() {
         doTestScan(null);
     }
 
     @Test
-    public void testScanExplicit() throws Exception {
+    public void testScanExplicit() {
         igniteTransactions.runInTransaction(this::doTestScan);
     }
 
diff --git 
a/modules/transactions/src/testFixtures/java/org/apache/ignite/internal/tx/storage/state/test/TestTxStateStorage.java
 
b/modules/transactions/src/testFixtures/java/org/apache/ignite/internal/tx/storage/state/test/TestTxStateStorage.java
index 13533bcd39..dea779fa2f 100644
--- 
a/modules/transactions/src/testFixtures/java/org/apache/ignite/internal/tx/storage/state/test/TestTxStateStorage.java
+++ 
b/modules/transactions/src/testFixtures/java/org/apache/ignite/internal/tx/storage/state/test/TestTxStateStorage.java
@@ -32,6 +32,8 @@ import java.util.concurrent.ConcurrentSkipListMap;
 import java.util.concurrent.atomic.AtomicReference;
 import org.apache.ignite.internal.lang.IgniteBiTuple;
 import org.apache.ignite.internal.lang.IgniteInternalException;
+import org.apache.ignite.internal.tostring.IgniteToStringInclude;
+import org.apache.ignite.internal.tostring.S;
 import org.apache.ignite.internal.tx.TxMeta;
 import org.apache.ignite.internal.tx.TxState;
 import org.apache.ignite.internal.tx.storage.state.TxStateStorage;
@@ -43,6 +45,7 @@ import org.jetbrains.annotations.Nullable;
  * Test implementation of {@link TxStateStorage} based on {@link 
ConcurrentSkipListMap}.
  */
 public class TestTxStateStorage implements TxStateStorage {
+    @IgniteToStringInclude
     private final ConcurrentSkipListMap<UUID, TxMeta> storage = new 
ConcurrentSkipListMap<>(new UnsignedUuidComparator());
 
     private volatile long lastAppliedIndex;
@@ -265,6 +268,11 @@ public class TestTxStateStorage implements TxStateStorage {
         return nullCompletedFuture();
     }
 
+    @Override
+    public String toString() {
+        return S.toString(TestTxStateStorage.class, this);
+    }
+
     private void checkStorageInProgreesOfRebalance() {
         if (rebalanceFutureReference.get() != null) {
             throwRebalanceInProgressException();

Reply via email to