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

IgnitePdsPageEvictionDuringPartitionClearTest sometimes hangs on node stop

    XMLWordPrintableJSON

Details

    • Bug
    • Status: Resolved
    • Major
    • Resolution: Fixed
    • 2.7
    • 2.8
    • None

    Description

      It seems that this hang can be explained as follows:

      • 'vacuum-cleaner' thread tries to clean up a cache entry and so it needs to acquire the checkpoint read lock.
        It is important to note that gateway lock for the corresponding cache is already acquired.
        Let's assume the checkpoint read lock cannot be acquired right now because there are many dirty pages, and, therefore, the 'vacuum' has to wait for a next checkpoint.
      "vacuum-cleaner-#484%db.IgnitePdsPageEvictionDuringPartitionClearTest0%" #587 prio=5 os_prio=0 tid=0x00007f5e2c007800 nid=0x1f960e waiting on condition [0x00007f5ee49eb000]
      java.lang.Thread.State: WAITING (parking)
      at sun.misc.Unsafe.park(Native Method)
      at java.util.concurrent.locks.LockSupport.park(LockSupport.java:304)
      at org.apache.ignite.internal.util.future.GridFutureAdapter.get0(GridFutureAdapter.java:178)
      at org.apache.ignite.internal.util.future.GridFutureAdapter.getUninterruptibly(GridFutureAdapter.java:146)
      at org.apache.ignite.internal.processors.cache.persistence.GridCacheDatabaseSharedManager.checkpointReadLock(GridCacheDatabaseSharedManager.java:1565)
      at org.apache.ignite.internal.processors.cache.mvcc.MvccProcessorImpl$VacuumWorker.cleanup(MvccProcessorImpl.java:2395)
      at org.apache.ignite.internal.processors.cache.mvcc.MvccProcessorImpl$VacuumWorker.processPartition(MvccProcessorImpl.java:2270)
      at org.apache.ignite.internal.processors.cache.mvcc.MvccProcessorImpl$VacuumWorker.body(MvccProcessorImpl.java:2188)
      at org.apache.ignite.internal.util.worker.GridWorker.run(GridWorker.java:120)
      at java.lang.Thread.run(Thread.java:748)
      • On the other hand, the checkpoint thread can be already stopped by Ignition#stop(cancel = true)
        In that case, scheduledCp.cpFinishFut is not completed (it seems to me, this is definitely a bug) and leads to a deadlock:
      "test-runner-#302%db.IgnitePdsPageEvictionDuringPartitionClearTest%" #388 prio=5 os_prio=0 tid=0x00007f60f461b800 nid=0x1f954b waiting on condition [0x00007f60338fc000]
      java.lang.Thread.State: TIMED_WAITING (parking)
      at sun.misc.Unsafe.park(Native Method)
      - parking to wait for <0x0000000095fe0948> (a java.util.concurrent.locks.ReentrantReadWriteLock$NonfairSync)
      at java.util.concurrent.locks.LockSupport.parkNanos(LockSupport.java:215)
      at java.util.concurrent.locks.AbstractQueuedSynchronizer.doAcquireNanos(AbstractQueuedSynchronizer.java:934)
      at java.util.concurrent.locks.AbstractQueuedSynchronizer.tryAcquireNanos(AbstractQueuedSynchronizer.java:1247)
      at java.util.concurrent.locks.ReentrantReadWriteLock$WriteLock.tryLock(ReentrantReadWriteLock.java:1115)
      at org.apache.ignite.internal.util.StripedCompositeReadWriteLock$WriteLock.tryLock(StripedCompositeReadWriteLock.java:219)
      at org.apache.ignite.internal.processors.cache.GridCacheGateway.onStopped(GridCacheGateway.java:315)
      at org.apache.ignite.internal.processors.cache.GridCacheProcessor.blockGateways(GridCacheProcessor.java:1142)
      at org.apache.ignite.internal.IgniteKernal.stop0(IgniteKernal.java:2399)
      at org.apache.ignite.internal.IgniteKernal.stop(IgniteKernal.java:2283)
      at org.apache.ignite.internal.IgnitionEx$IgniteNamedInstance.stop0(IgnitionEx.java:2570)
      - locked <0x000000009cd5fcc8> (a org.apache.ignite.internal.IgnitionEx$IgniteNamedInstance)
      at org.apache.ignite.internal.IgnitionEx$IgniteNamedInstance.stop(IgnitionEx.java:2533)
      at org.apache.ignite.internal.IgnitionEx.stop(IgnitionEx.java:330)
      at org.apache.ignite.Ignition.stop(Ignition.java:223)

      Attachments

        Issue Links

          Activity

            People

              slava.koptilin Vyacheslav Koptilin
              slava.koptilin Vyacheslav Koptilin
              Votes:
              0 Vote for this issue
              Watchers:
              3 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 - 20m
                  20m