You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@ignite.apache.org by "Stanilovsky Evgeny (Jira)" <ji...@apache.org> on 2021/01/29 04:23:00 UTC
[jira] [Commented] (IGNITE-14076) Exponential putAll performance
degradation in transactional cache
[ https://issues.apache.org/jira/browse/IGNITE-14076?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17274146#comment-17274146 ]
Stanilovsky Evgeny commented on IGNITE-14076:
---------------------------------------------
Ok we really have a kind of amplification here, look :
{code:java}
GridNearOptimisticTxPrepareFuture#prepare{code}
- here we will map all keys into appropriate nodes.
This structure
{code:java}
Queue<GridDistributedTxMapping> mappings{code}
will handle all such mappings.
{code:java}
GridNearOptimisticTxPrepareFuture#proceedPrepare(){code}
will poll and create appropriate
{code:java}
GridNearTxPrepareRequest req = new GridNearTxPrepareRequest({code}
from the other side :
org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtTxLocal#prepareAsync will always append tx items :
{code:java}
if (req.writes() != null) {
for (IgniteTxEntry e : req.writes()) {
addEntry(req.messageId(), e);{code}
and
org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtTxPrepareFuture#createPrepareResponse ->
org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtTxPrepareFuture#localDhtPendingVersions
will exponentially iterate over increasing tx items, for example iterates will cover : 2, 5, 7, 8, .... tx primary enlisted keys
> Exponential putAll performance degradation in transactional cache
> -----------------------------------------------------------------
>
> Key: IGNITE-14076
> URL: https://issues.apache.org/jira/browse/IGNITE-14076
> Project: Ignite
> Issue Type: Bug
> Components: cache
> Affects Versions: 2.9.1
> Reporter: Pavel Tupitsyn
> Priority: Critical
> Fix For: 2.11
>
>
> {{putAll}} execution time grows almost exponentially while the number of keys grows linearly in the following test:
> {code:java}
> public class PutAllTxTest extends GridCommonAbstractTest {
> @Test
> public void testPutAll() throws Exception {
> Ignition.start(getConfiguration("server1"));
> Ignition.start(getConfiguration("server2"));
> Ignite ignite = Ignition.start(getConfiguration("client").setClientMode(true));
> IgniteCache<Integer, Integer> cache = ignite.createCache(
> new CacheConfiguration<Integer, Integer>("c")
> .setAtomicityMode(CacheAtomicityMode.TRANSACTIONAL));
> int count = 50000;
> Map<Integer, Integer> data = new TreeMap<>();
> for (int i = 0; i < count; i++)
> data.put(i, i);
> long begin = System.nanoTime();
> cache.putAll(data);
> long dur = System.nanoTime() - begin;
> System.out.println(">>>>> " + dur / 1000000);
> }
> }
> {code}
> ||Entries||Seconds||
> |1000|0.4|
> |5000|1.9|
> |10000|3.8|
> |20000|10.7|
> |30000|23.5|
> |40000|41|
> |50000|64|
> |60000|90|
> |100000|254|
> This does not reproduce with 1 server node, and does not reproduce on {{ATOMIC}} caches with any number of nodes.
> *Observations:*
> - Not a GC issue (it barely runs)
> - Not a memory issue (heap is under 1GB)
> - GridDhtTxPrepareFuture#localDhtPendingVersions -> GridCacheMapEntry.localCandidates is the bottleneck. For 1K keys, localCandidates gets called 123K times, 2K keys - 484K times, etc.
--
This message was sent by Atlassian Jira
(v8.3.4#803005)