Uploaded image for project: 'Ignite'
  1. Ignite
  2. IGNITE-18451

Node stop may hang if an index is not created yet

    XMLWordPrintableJSON

Details

    • Bug
    • Status: Resolved
    • Major
    • Resolution: Fixed
    • None
    • 3.0.0-beta2
    • None

    Description

      1. Start and init a cluster of 3 nodes
      2. Via node 0, create a table with 3 replicas
      3. Do some stuff
      4. Stop the cluster

      The cluster cannot be stoppled, stop of node 2 seems to hang forever. There is a thread:

      "%itrst_tsimwkpn_2%JRaft-FSMCaller-Disruptor-_stripe_3-0" #253053 daemon prio=5 os_prio=0 cpu=0,59ms elapsed=1169,99s tid=0x00007fb4867e3000 nid=0xa1668 waiting on condition  [0x00007fb4832f0000]
         java.lang.Thread.State: WAITING (parking)
          at jdk.internal.misc.Unsafe.park(java.base@11.0.6/Native Method)
          - parking to wait for  <0x0000000707d9a508> (a java.util.concurrent.CompletableFuture$Signaller)
          at java.util.concurrent.locks.LockSupport.park(java.base@11.0.6/LockSupport.java:194)
          at java.util.concurrent.CompletableFuture$Signaller.block(java.base@11.0.6/CompletableFuture.java:1796)
          at java.util.concurrent.ForkJoinPool.managedBlock(java.base@11.0.6/ForkJoinPool.java:3128)
          at java.util.concurrent.CompletableFuture.waitingGet(java.base@11.0.6/CompletableFuture.java:1823)
          at java.util.concurrent.CompletableFuture.join(java.base@11.0.6/CompletableFuture.java:2043)
          at org.apache.ignite.internal.table.TableImpl.pkId(TableImpl.java:126)
          at org.apache.ignite.internal.table.TableImpl.awaitIndexes(TableImpl.java:347)
          at org.apache.ignite.internal.table.TableImpl.lambda$indexStorageAdapters$0(TableImpl.java:237)
          at org.apache.ignite.internal.table.TableImpl$$Lambda$1842/0x0000000800a2fc40.get(Unknown Source)
          at org.apache.ignite.internal.table.distributed.raft.PartitionListener.addToIndexes(PartitionListener.java:435)
          at org.apache.ignite.internal.table.distributed.raft.PartitionListener.lambda$handleUpdateAllCommand$5(PartitionListener.java:234)
          at org.apache.ignite.internal.table.distributed.raft.PartitionListener$$Lambda$2551/0x0000000800ceb440.execute(Unknown Source)
          at org.apache.ignite.internal.storage.rocksdb.RocksDbMvPartitionStorage.runConsistently(RocksDbMvPartitionStorage.java:249)
          at org.apache.ignite.internal.table.distributed.raft.snapshot.outgoing.SnapshotAwarePartitionDataStorage.runConsistently(SnapshotAwarePartitionDataStorage.java:59)
          at org.apache.ignite.internal.table.distributed.raft.PartitionListener.handleUpdateAllCommand(PartitionListener.java:219)
          at org.apache.ignite.internal.table.distributed.raft.PartitionListener.lambda$onWrite$1(PartitionListener.java:153)
          at org.apache.ignite.internal.table.distributed.raft.PartitionListener$$Lambda$1944/0x0000000800a7a440.accept(Unknown Source)
          at java.util.Iterator.forEachRemaining(java.base@11.0.6/Iterator.java:133)
          at org.apache.ignite.internal.table.distributed.raft.PartitionListener.onWrite(PartitionListener.java:123)
          at org.apache.ignite.internal.raft.server.impl.JraftServerImpl$DelegatingStateMachine.onApply(JraftServerImpl.java:563)
          at org.apache.ignite.raft.jraft.core.FSMCallerImpl.doApplyTasks(FSMCallerImpl.java:565)
          at org.apache.ignite.raft.jraft.core.FSMCallerImpl.doCommitted(FSMCallerImpl.java:530)
          at org.apache.ignite.raft.jraft.core.FSMCallerImpl.runApplyTask(FSMCallerImpl.java:449)
          at org.apache.ignite.raft.jraft.core.FSMCallerImpl$ApplyTaskHandler.onEvent(FSMCallerImpl.java:137)
          at org.apache.ignite.raft.jraft.core.FSMCallerImpl$ApplyTaskHandler.onEvent(FSMCallerImpl.java:131)
          at org.apache.ignite.raft.jraft.disruptor.StripedDisruptor$StripeEntryHandler.onEvent(StripedDisruptor.java:217)
          at org.apache.ignite.raft.jraft.disruptor.StripedDisruptor$StripeEntryHandler.onEvent(StripedDisruptor.java:181)
          at com.lmax.disruptor.BatchEventProcessor.run(BatchEventProcessor.java:137)
          at java.lang.Thread.run(java.base@11.0.6/Thread.java:834)

      Probably the node stop was initiated before the indices were created for the table, then index manager was stopped, so the indices never get created. But only index creation completes the future that is being avaited by the shown thread.

      This might have something to do with IGNITE-18203.

      Attachments

        Issue Links

          Activity

            People

              rpuch Roman Puchkovskiy
              rpuch Roman Puchkovskiy
              Aleksandr Polovtsev Aleksandr Polovtsev
              Votes:
              0 Vote for this issue
              Watchers:
              2 Start watching this issue

              Dates

                Created:
                Updated:
                Resolved:

                Time Tracking

                  Estimated:
                  Original Estimate - Not Specified
                  Not Specified
                  Remaining:
                  Remaining Estimate - 0h
                  0h
                  Logged:
                  Time Spent - 1h 10m
                  1h 10m