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

infinite loop and 100% cpu in GridDhtPartitionsEvictor: Eviction in progress ...

    XMLWordPrintableJSON

Details

    • Bug
    • Status: Open
    • Blocker
    • Resolution: Unresolved
    • 2.5
    • None
    • cache
    • None

    Description

      Note: RCA was not done:

      Sometimes ignite server nodes fall into infinite loop and consume 100% cpu:

      "sys-#260008" #260285 prio=5 os_prio=0 tid=0x00007fabb020a800 nid=0x1e850 runnable [0x00007fab26fef000]
         java.lang.Thread.State: RUNNABLE
      	at java.util.concurrent.ConcurrentHashMap$Traverser.advance(ConcurrentHashMap.java:3339)
      	at java.util.concurrent.ConcurrentHashMap$ValueIterator.next(ConcurrentHashMap.java:3439)
      	at org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtPartitionsEvictor$1.call(GridDhtPartitionsEvictor.java:84)
      	at org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtPartitionsEvictor$1.call(GridDhtPartitionsEvictor.java:73)
      	at org.apache.ignite.internal.util.IgniteUtils.wrapThreadLoader(IgniteUtils.java:6695)
      	at org.apache.ignite.internal.processors.closure.GridClosureProcessor$2.body(GridClosureProcessor.java:967)
      	at org.apache.ignite.internal.util.worker.GridWorker.run(GridWorker.java:110)
      	at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
      	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
      	at java.lang.Thread.run(Thread.java:748)
      
         Locked ownable synchronizers:
      	- <0x0000000649b9cba0> (a java.util.concurrent.ThreadPoolExecutor$Worker)
      

      the following appears in logs each 2 minutes:

      #### INFO  2019-07-08 12:21:45.081 (1562581305081) [sys-#98168] [GridDhtPartitionsEvictor] > Eviction in progress [grp=CUSTPRODINVOICEDISCUSAGE, remainingCnt=102]
      

      remainingCnt remains the same once it reached 102 (the very first line in the logs was with value equal to 101).

      Some other facts:
      we have a heapdump taken for topVer = 900 . the problem appeared after topVer = 790, but it looks like it was silently waiting from topVer = 641 (about 24 hours back).
      There were 259 topology changes between 900 and 641.

      All 102 GridDhtLocalPartitions can be found in the heapdump:

      select * from "org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtLocalPartition" t where delayedRenting = true
      

      They all have status = 65537 , which means (according to org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtLocalPartition#state):

      reservations(65537) = 1
      getPartState(65537) = OWNING

      There are also 26968 instances of org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtPartitionTopologyImpl$$Lambda$70, that are created by org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtPartitionTopologyImpl#checkEvictions method.
      26418 of 26968 refer to AtomicInteger instance with value = 102:
      26418/102 = 259 = 900 - 641 (see topology info above).

      The key thing seen from the heapdump is that topVer = 641 or topVer = 642 was the last topology where these 102 partitions were assigned to the current ignite server node.

      select
        t.this
       ,t.this['clientEvtChange'] as clientEvtChange
       ,t.this['topVer.topVer'] as topVer
       ,t.this['assignment.elementData'][555]['elementData'][0]['hostNames.elementData'][0] as primary_part
       ,t.this['assignment.elementData'][555]['elementData'][1]['hostNames.elementData'][0] as secondary_part
      from org.apache.ignite.internal.processors.affinity.HistoryAffinityAssignment t where length(t.this['assignment.elementData']) = 1024
      order by topVer
      

      The connection of a client node at topVer = 790 somehow triggered the GridDhtPartitionsEvictor loop to execute.

      Summary:
      1) it is seen that 102 partitions has one reservation and OWNING state.
      2) they were backup partitions.
      3) for some reason their eviction has been silently delaying (because of reservations), but each topology change seemed to trigger eviction attempt.
      4) something managed to make org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtPartitionsEvictor#evictPartitionAsync to run never exiting.

      Additional info:
      topVer = 641 was in chain of sever nodes restarts (not sure if rebalancing actually succeeded):

      Attachments

        1. eviction-in-progress-dumps.zip
          325 kB
          Henrique
        2. image-2019-07-10-16-07-37-185.png
          13 kB
          Igor Kamyshnikov
        3. server-node-restarts-1.png
          47 kB
          Igor Kamyshnikov

        Activity

          People

            Unassigned Unassigned
            kamyshnikov Igor Kamyshnikov
            Votes:
            0 Vote for this issue
            Watchers:
            4 Start watching this issue

            Dates

              Created:
              Updated: