ignite-issues mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From "Alexey Goncharuk (JIRA)" <j...@apache.org>
Subject [jira] [Updated] (IGNITE-9840) Possible deadlock on transactional future on client node in case of network problems or long GC pauses
Date Thu, 08 Nov 2018 09:59:00 GMT

     [ https://issues.apache.org/jira/browse/IGNITE-9840?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]

Alexey Goncharuk updated IGNITE-9840:
-------------------------------------
    Ignite Flags:   (was: Docs Required)

> Possible deadlock on transactional future on client node in case of network problems
or long GC pauses
> ------------------------------------------------------------------------------------------------------
>
>                 Key: IGNITE-9840
>                 URL: https://issues.apache.org/jira/browse/IGNITE-9840
>             Project: Ignite
>          Issue Type: Bug
>          Components: clients
>    Affects Versions: 2.6
>            Reporter: Andrey Aleksandrov
>            Assignee: Ilya Lantukh
>            Priority: Critical
>             Fix For: 2.8
>
>
> Steps to reproduce:
> 1)Start the server node with next timeouts. DefaultTxTimeout should be greater than other:
>  
> {code:java}
> <property name="peerClassLoadingEnabled" value="true"/>
> <property name="failureDetectionTimeout" value="60000"/>
> <property name="clientFailureDetectionTimeout" value="60000"/>
> <property name="networkTimeout" value="60000"/>
> <property name="gridName" value="name"/>
> <!--Transaction timeout setting-->
> <property name="transactionConfiguration">
>     <bean class="org.apache.ignite.configuration.TransactionConfiguration">
>         <property name="DefaultTxTimeout" value="600000"/>
>     </bean>
> </property>
> <property name="idleConnectionTimeout" value="5000"/>
> <property name="connectTimeout" value="5000"/>
> <property name="ackTimeout" value="20000"/>
> {code}
> On the server side you should create a cache with next parameters:
>  
>  
> {code:java}
> <bean class="org.apache.ignite.configuration.CacheConfiguration">
>     <property name="name" value="CACHE"/>
>     <property name="cacheMode" value="PARTITIONED"/>
>     <property name="atomicityMode" value="TRANSACTIONAL"/>
>     <property name="writeSynchronizationMode" value="FULL_SYNC"/>
>     <property name="backups" value="1"/>
>     <property name="statisticsEnabled" value="true"/>{code}
> 2)After that start the client with the next code:
> {code:java}
> IgniteCache<String, Object> cache = ignite.getOrCreateCache("CACHE");
> try (Transaction tx = ignite.transactions().txStart()) {
>     cache.put("Key", new Object());
>     System.out.println("Stop me");
>     //here we will get long GC pause on server side
>     Thread.sleep(10000);
>     // Commit the transaction.
>     tx.commitAsync().get();
> }
> {code}
>  
> On step "Stop me" you should suspend all the thread on the server side to emulate the
networking problem or long GC pause on the server side.
> Finally, you will face in client node next:
> {code:java}
> [2018-10-10 16:46:10,157][ERROR][nio-acceptor-tcp-comm-#28%GRIDC1%][root] Critical system
error detected. Will be handled accordingly to configured handler [hnd=StopNodeOrHaltFailureHandler
[tryStop=false, timeout=0, super=AbstractFailureHandler [ignoredFailureTypes=UnmodifiableSet
[SYSTEM_WORKER_BLOCKED]]], failureCtx=FailureContext [type=SYSTEM_WORKER_BLOCKED, err=class
o.a.i.IgniteException: GridWorker [name=grid-timeout-worker, igniteInstanceName=GRIDC1, finished=false,
heartbeatTs=1539179057570]]]
> {code}
> Also, the similar issue could be reproduced in 2.4. In both cases looks like we have
a deadlock during trying to display the TxEntryValueHolder. Looks like this values are already
used by the transaction with long DefaultTxTimeout .
> {code:java}
> java.lang.Thread.State: WAITING
> at sun.misc.Unsafe.park(Unsafe.java:-1)
> at java.util.concurrent.locks.LockSupport.park(LockSupport.java:304)
> at org.apache.ignite.internal.util.future.GridFutureAdapter.get0(GridFutureAdapter.java:177)
> at org.apache.ignite.internal.util.future.GridFutureAdapter.get(GridFutureAdapter.java:140)
> at org.apache.ignite.internal.processors.cache.binary.CacheObjectBinaryProcessorImpl.metadata0(CacheObjectBinaryProcessorImpl.java:526)
> at org.apache.ignite.internal.processors.cache.binary.CacheObjectBinaryProcessorImpl.metadata(CacheObjectBinaryProcessorImpl.java:510)
> at org.apache.ignite.internal.processors.cache.binary.CacheObjectBinaryProcessorImpl$2.metadata(CacheObjectBinaryProcessorImpl.java:193)
> at org.apache.ignite.internal.binary.BinaryContext.metadata(BinaryContext.java:1265)
> at org.apache.ignite.internal.binary.BinaryUtils.type(BinaryUtils.java:2407)
> at org.apache.ignite.internal.binary.BinaryObjectImpl.rawType(BinaryObjectImpl.java:302)
> at org.apache.ignite.internal.binary.BinaryObjectExImpl.toString(BinaryObjectExImpl.java:205)
> at org.apache.ignite.internal.binary.BinaryObjectExImpl.toString(BinaryObjectExImpl.java:186)
> at org.apache.ignite.internal.binary.BinaryObjectImpl.toString(BinaryObjectImpl.java:919)
> at java.lang.String.valueOf(String.java:2994)
> at java.lang.StringBuilder.append(StringBuilder.java:131)
> at org.apache.ignite.internal.processors.cache.transactions.TxEntryValueHolder.toString(TxEntryValueHolder.java:161)
> ...{code}
> On the client side, it could be looked like a hanging transaction because we waiting
on:
> {code:java}
> tx.commitAsync().get();{code}
>  



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

Mime
View raw message