You are viewing a plain text version of this content. The canonical link for it is here.
Posted to user@ignite.apache.org by Andrey Kornev <an...@hotmail.com> on 2018/01/18 23:51:42 UTC

Long activation times with Ignite persistence enabled

Hello,

I'm wondering if there is a way to improve the startup time of Ignite node when the persistence is enabled?

It seems the time is proportional to the size (and number) of the partition files. This is somewhat surprising as I expected the startup time be the same (plus-minus some constant factor) regardless of the amount of data persisted.

The delay looks to be due to Ignite loading *all* partition files for *all* persistence-enabled caches as part of a node's join. Here's an example of the startup log output:

2018-01-18 14:00:40,230 INFO  [exchange-worker-#42%ignite-1%] GridCacheDatabaseSharedManager - Read checkpoint status [startMarker=/tmp/storage/data/1/cp/1516311778910-d56f8ceb-2205-4bef-9ed3-a7446e34aa06-START.bin, endMarker=/tmp/storage/data/1/cp/1516311778910-d56f8ceb-2205-4bef-9ed3-a7446e34aa06-END.bin]
2018-01-18 14:00:40,230 INFO  [exchange-worker-#42%ignite-1%] GridCacheDatabaseSharedManager - Applying lost cache updates since last checkpoint record [lastMarked=FileWALPointer [idx=1693, fileOff=7970054, len=60339], lastCheckpointId=d56f8ceb-2205-4bef-9ed3-a7446e34aa06]
2018-01-18 14:00:57,114 WARN  [exchange-worker-#42%ignite-1%] PageMemoryImpl - Page evictions started, this will affect storage performance (consider increasing DataRegionConfiguration#setMaxSize).
2018-01-18 14:02:05,469 INFO  [exchange-worker-#42%ignite-1%] GridCacheDatabaseSharedManager - Finished applying WAL changes [updatesApplied=0, time=85234ms]

It took ≈1.5 minute to activate a node. To add insult to injury, the eviction kicked in and most of the loaded pages got evicted (in this test, I had the caches sharing a 1GB memory region loading about 10GB of data and index). In general, I think it's not unreasonable to expect 1-to-10 ratio of the data region size to the total persisted data size.

Why load all that data in the first place? It seems like a huge waste of time. Can the data partitions be loaded lazily on demand while the index partition can still be loaded during node startup?

Thanks
Andrey


Re: Long activation times with Ignite persistence enabled

Posted by Naveen <na...@gmail.com>.
Do we have any update on this

Thanks



--
Sent from: http://apache-ignite-users.70518.x6.nabble.com/

Re: Long activation times with Ignite persistence enabled

Posted by Naveen <na...@gmail.com>.
Hi Pavel 

We are using Ignite 2.6 
You were saying, usage of cache groups is definitely needed to improve to
the cluster activation time ? 

I could see below documentation on the usage of cache groups. 
  
Should the grouping be used all the times? 
With all the benefits the cache groups have, they might impact the
performance of read operations and indexes lookups. This is caused by the
fact that all the data and indexes get mixed in shared data structures
(partition maps, B+trees) and it will take more time querying over them. 

Thus, consider using the cache groups if you have a cluster of dozens and
hundreds of nodes and caches, and you spot increased Java heap usage by
internal structures, checkpointing performance drop, slow node connectivity
to the cluster. 


In our case, we do have around 50 caches, and at the max we may have 10
nodes. do you still recommend cache groups for our use case. 

And, ours upserts TPS is very low, may be 1 k per sec, but query or read TPS
is quite high, close to 10 K TPS. So as per the above lines, it says
performance of read operations is impacted since all the caches are gong to
use the shared structures. 
We are looking for a design which can improve the cluster activation time,
but not at the expense of compromising on the query performance, since our
solution being a read intensive, we cant afford to reduce query performance.
In worst scenario, we can live with poor cluster activation process as well,
since it only affects us at the time cluster restart which is performed only
in case of cluster crash or planned maintenance. 

One more thing, if at all we need to change the system pool, below is the
command to change the system pool ?? 
IgniteConfiguration.setSystemThreadPoolSize(...) 
We have 128 CPU machines, what would be the ideal system thread pool size ??
of course, it should be tried and tested but still some number.. 

Regarding the cache groups design, anything I should consider when we are
grouping the cache 

1. We have around 40 caches, no indexes, we only have lookup on primary key,
some of them are simple keys and some of them are having complex primary
key. Some of the caches are queries together, does it help if we group them
into a cache group ? 
2. What if we are trying query the caches which are part of different cache
groups ? 
3. We are going to have close to half a billion records in each cache, so
how do we group them 
4. Some of the caches are independent , does not have any relation with
other caches 

So in case, I am going with cache group, shall I change the partition to 128
or keep the default ? 


Thanks 
Naveen



--
Sent from: http://apache-ignite-users.70518.x6.nabble.com/

Re: Long activation times with Ignite persistence enabled

Posted by Naveen <na...@gmail.com>.
Hi Pavel

We are using Ignite 2.6
You were saying, usage of cache groups is definitely needed to improve to
the cluster activation time ?

I could see below documentation on the usage of cache groups.
 
*Should the grouping be used all the times?
With all the benefits the cache groups have, they might impact the
performance of read operations and indexes lookups. This is caused by the
fact that all the data and indexes get mixed in shared data structures
(partition maps, B+trees) and it will take more time querying over them.

Thus, consider using the cache groups if you have a cluster of dozens and
hundreds of nodes and caches, and you spot increased Java heap usage by
internal structures, checkpointing performance drop, slow node connectivity
to the cluster.
*

In our case, we do have around 50 caches, and at the max we may have 10
nodes. do you still recommend cache groups for our use case. 

And, ours upserts TPS is very low, may be 1 k per sec, but query or read TPS
is quite high, close to 10 K TPS. So as per the above lines, it says
performance of read operations is impacted since all the caches are gong to
use the shared structures. 
We are looking for a design which can improve the cluster activation time,
but not at the expense of compromising on the query performance, since our
solution being a read intensive, we cant afford to reduce query performance.
In worst scenario, we can live with poor cluster activation process as well,
since it only affects us at the time cluster restart which is performed only
in case of cluster crash or planned maintenance.

One more thing, if at all we need to change the system pool, below is the
command to change the system pool ??
IgniteConfiguration.setSystemThreadPoolSize(...)
We have 128 CPU machines, what would be the ideal system thread pool size ??
of course, it should be tried and tested but still some number..

Regarding the cache groups design, anything I should consider when we are
grouping the cache

1. We have around 40 caches, no indexes, we only have lookup on primary key,
some of them are simple keys and some of them are having complex primary
key. Some of the caches are queries together, does it help if we group them
into a cache group ?
2. What if we are trying query the caches which are part of different cache
groups ?
3. We are going to have close to half a billion records in each cache, so
how do we group them
4. Some of the caches are independent , does not have any relation with
other caches

So in case, I am going with cache group, shall I change the partition to 128
or keep the default ?


Thanks
Naveen



--
Sent from: http://apache-ignite-users.70518.x6.nabble.com/

Re: Long activation times with Ignite persistence enabled

Posted by Pavel Kovalenko <jo...@gmail.com>.
Hi Naveen and Andrey,

We've recently done major optimization
https://issues.apache.org/jira/browse/IGNITE-9420 that will speed-up
activation time in your case.
Iteration over WAL now happens only on a node start-up, so it will not
affect activation anymore.
Partitions state restoring (which is the slowest part of the activation
phase as I see in the first message in the thread) was also optimized.
Now it is performed in parallel for each of available cache groups.
Parallelism level of that operation is controlled by System Pool size.
If you have enough CPU cores on your machines (more than the number of
configured cache groups) you can adjust System pool size and your
activation time will be significantly improved.

вт, 6 нояб. 2018 г. в 17:23, Naveen <na...@gmail.com>:

> Hi Denis
>
> We have already reduced the partition to 128, after which activation time
> has come down a bit.
>
> You were saying that, by reducing the partitions, it may lead to uneven
> distribution of data between nodes. Isn't it the same when we go for cache
> groups, group of caches will use the same resources /partitions, so here
> also resource contention may be there right ?? here also same set of
> partitions used by group of caches ?
> If we use cache group, partition size may grow very high since all the
> caches belong to that group will use the same set of partitions, does it
> have any negative effect on the cluster performance ??
>
>
>
> Thanks
> Naveen
>
>
>
> --
> Sent from: http://apache-ignite-users.70518.x6.nabble.com/
>

Re: Long activation times with Ignite persistence enabled

Posted by Naveen <na...@gmail.com>.
Hi Denis

We have already reduced the partition to 128, after which activation time
has come down a bit. 

You were saying that, by reducing the partitions, it may lead to uneven
distribution of data between nodes. Isn't it the same when we go for cache
groups, group of caches will use the same resources /partitions, so here
also resource contention may be there right ?? here also same set of
partitions used by group of caches ? 
If we use cache group, partition size may grow very high since all the
caches belong to that group will use the same set of partitions, does it
have any negative effect on the cluster performance ??



Thanks
Naveen



--
Sent from: http://apache-ignite-users.70518.x6.nabble.com/

Re: Long activation times with Ignite persistence enabled

Posted by Denis Mekhanikov <dm...@gmail.com>.
Naveen,

40 caches is quite a lot. It means, that Ignite needs to handle 40 *
(number of partitions) files.
By default each cache has 1024 partitions.
This is quite a lot, and a disk is the bottleneck here. Changing of thread
pool sizes won't save you.
If you divide your caches into cache groups, then they will share the same
partitions, thus number of files will be reduced.
You can also try reducing the number of partitions, but it may lead to
uneven distribution of data between nodes.
Any of these changes will require reloading of the data.

You can record a *dstat* on the host machine to make sure, that disk is the
weak place.
If its utilization is high, while CPU is not used, then it means, that you
need a faster disk.

Denis


пн, 5 нояб. 2018 г. в 17:10, Naveen <na...@gmail.com>:

> Hi Denis
>
> We have only 40 caches in our cluster.
> If we introduce grouping of caches, guess we need to reload the data from
> scratch, right ??
>
> We do have very powerful machines as part of cluster, they are 128 CPU very
> high end boxes and huge resources available, by increasing any of the below
> thread pools, can we reduce the cluster activation time.
>
> System Pool
> Public Pool
> Striped Pool
> Custom Thread Pools
>
> Thanks
> Naveen
>
>
>
> --
> Sent from: http://apache-ignite-users.70518.x6.nabble.com/
>

Re: Long activation times with Ignite persistence enabled

Posted by Naveen <na...@gmail.com>.
HI  Gianluca

Removing this  -XX:+AlwaysPreTouch did not help us at all, it took the same
time with or without this. 
But I observed without this was, OS has not allocated entire heap memory we
have given as JVM options, we were giving 200G as heap for ignite node, so
when we execute top command, Ignite node used to use 250GB odd. 
Now after removing AlwaysPreTouch, ignite node use only 50G (around). 

Other than, activation time remains same with or without AlwaysPreTouch. 

Thanks
Naveen



--
Sent from: http://apache-ignite-users.70518.x6.nabble.com/

Re: Long activation times with Ignite persistence enabled

Posted by Gianluca Bonetti <gi...@gmail.com>.
Hello

In my case of slow startup, as suggested from a member of this mailing
list, I deleted the -XX:+AlwaysPreTouch command line option from JVM
launch, and the cluster got back to very fast startup.
Don't know if you are using this option, hope it helps.

Cheers
Gianluca

Il giorno lun 5 nov 2018 alle ore 14:10 Naveen <na...@gmail.com>
ha scritto:

> Hi Denis
>
> We have only 40 caches in our cluster.
> If we introduce grouping of caches, guess we need to reload the data from
> scratch, right ??
>
> We do have very powerful machines as part of cluster, they are 128 CPU very
> high end boxes and huge resources available, by increasing any of the below
> thread pools, can we reduce the cluster activation time.
>
> System Pool
> Public Pool
> Striped Pool
> Custom Thread Pools
>
> Thanks
> Naveen
>
>
>
> --
> Sent from: http://apache-ignite-users.70518.x6.nabble.com/
>

Re: Long activation times with Ignite persistence enabled

Posted by Naveen <na...@gmail.com>.
Hi Denis

We have only 40 caches in our cluster.
If we introduce grouping of caches, guess we need to reload the data from
scratch, right ??

We do have very powerful machines as part of cluster, they are 128 CPU very
high end boxes and huge resources available, by increasing any of the below
thread pools, can we reduce the cluster activation time. 

System Pool
Public Pool
Striped Pool
Custom Thread Pools

Thanks
Naveen



--
Sent from: http://apache-ignite-users.70518.x6.nabble.com/

Re: Long activation times with Ignite persistence enabled

Posted by Denis Mekhanikov <dm...@gmail.com>.
Naveen,

How many caches do you have?
As Alexey mentioned, usage of cache groups
<https://apacheignite.readme.io/docs/cache-groups> could reduce the number
of created partitions and improve the startup time.

Denis

сб, 27 окт. 2018 г. в 11:12, Naveen <na...@gmail.com>:

> Do we have any  update long activation times ?
>
> I too face the same issue, am using 2.6.
>
> Cluster with 100 GB of disk size, got activated in 5 minutes, and when I
> tried with a cluster which has 3 TB is taking close to an hour.
>
> Is it the expected behavior OR some configuration I am missing here
>
> Thanks
>
>
>
>
> --
> Sent from: http://apache-ignite-users.70518.x6.nabble.com/
>

Re: Long activation times with Ignite persistence enabled

Posted by Naveen <na...@gmail.com>.
Do we have any  update long activation times ?

I too face the same issue, am using 2.6. 

Cluster with 100 GB of disk size, got activated in 5 minutes, and when I
tried with a cluster which has 3 TB is taking close to an hour.

Is it the expected behavior OR some configuration I am missing here

Thanks
 



--
Sent from: http://apache-ignite-users.70518.x6.nabble.com/

Re: Long activation times with Ignite persistence enabled

Posted by "ilya.kasnacheev" <il...@gmail.com>.
Hello!

Can you please also provide full cache configuration and your hardware specs
(especially for storage)?
After 2.4 release is done, I hope your case will get attention of PDS
developers.

Regards,



--
Sent from: http://apache-ignite-users.70518.x6.nabble.com/

Re: Long activation times with Ignite persistence enabled

Posted by Andrey Kornev <an...@hotmail.com>.
Alexey,

I'm wondering you had a chance to look into this? I'd like to understand what to expect in terms of node activation time and how it's related to the data volume.

Thanks!
Andrey

________________________________
From: Andrey Kornev <an...@hotmail.com>
Sent: Monday, January 22, 2018 11:36 AM
To: Alexey Goncharuk; user@ignite.apache.org
Subject: Re: Long activation times with Ignite persistence enabled

Alexey,

Thanks a lot for looking into this!

My configuration is very basic: 3 caches all using standard 1024 partitions, sharing a 1GB persistent memory region.

Please find below the stack trace of the exchange worker thread captured while the node's activation is in progress (2.4 Ignite branch).

Hope it helps!

Thanks!
Andrey

"exchange-worker-#42%ignite-2%" #82 prio=5 os_prio=31 tid=0x00007ffe8bf1c000 nid=0xc403 waiting on condition [0x000070000ed43000]
   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:177)
at org.apache.ignite.internal.util.future.GridFutureAdapter.getUninterruptibly(GridFutureAdapter.java:145)
at org.apache.ignite.internal.processors.cache.persistence.file.AsyncFileIO.read(AsyncFileIO.java:95)
at org.apache.ignite.internal.processors.cache.persistence.file.FilePageStore.read(FilePageStore.java:324)
at org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.read(FilePageStoreManager.java:306)
at org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.read(FilePageStoreManager.java:291)
at org.apache.ignite.internal.processors.cache.persistence.pagemem.PageMemoryImpl.acquirePage(PageMemoryImpl.java:656)
at org.apache.ignite.internal.processors.cache.persistence.pagemem.PageMemoryImpl.acquirePage(PageMemoryImpl.java:576)
at org.apache.ignite.internal.processors.cache.persistence.DataStructure.acquirePage(DataStructure.java:130)
at org.apache.ignite.internal.processors.cache.persistence.freelist.PagesList.init(PagesList.java:212)
at org.apache.ignite.internal.processors.cache.persistence.freelist.AbstractFreeList.<init>(AbstractFreeList.java:367)
at org.apache.ignite.internal.processors.cache.persistence.freelist.CacheFreeListImpl.<init>(CacheFreeListImpl.java:47)
at org.apache.ignite.internal.processors.cache.persistence.GridCacheOffheapManager$GridCacheDataStore$1.<init>(GridCacheOffheapManager.java:1041)
at org.apache.ignite.internal.processors.cache.persistence.GridCacheOffheapManager$GridCacheDataStore.init0(GridCacheOffheapManager.java:1041)
at org.apache.ignite.internal.processors.cache.persistence.GridCacheOffheapManager$GridCacheDataStore.updateCounter(GridCacheOffheapManager.java:1247)
at org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtLocalPartition.updateCounter(GridDhtLocalPartition.java:835)
at org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtLocalPartition.casState(GridDhtLocalPartition.java:523)
- locked <0x000000077a3d1120> (a org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtLocalPartition)
at org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtLocalPartition.<init>(GridDhtLocalPartition.java:218)
at org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtPartitionTopologyImpl.forceCreatePartition(GridDhtPartitionTopologyImpl.java:804)
at org.apache.ignite.internal.processors.cache.persistence.GridCacheDatabaseSharedManager.restorePartitionState(GridCacheDatabaseSharedManager.java:2196)
at org.apache.ignite.internal.processors.cache.persistence.GridCacheDatabaseSharedManager.applyLastUpdates(GridCacheDatabaseSharedManager.java:2155)
at org.apache.ignite.internal.processors.cache.persistence.GridCacheDatabaseSharedManager.restoreState(GridCacheDatabaseSharedManager.java:1322)
at org.apache.ignite.internal.processors.cache.persistence.GridCacheDatabaseSharedManager.beforeExchange(GridCacheDatabaseSharedManager.java:1113)
at org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionsExchangeFuture.distributedExchange(GridDhtPartitionsExchangeFuture.java:1063)
at org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionsExchangeFuture.init(GridDhtPartitionsExchangeFuture.java:661)
at org.apache.ignite.internal.processors.cache.GridCachePartitionExchangeManager$ExchangeWorker.body(GridCachePartitionExchangeManager.java:2329)
at org.apache.ignite.internal.util.worker.GridWorker.run(GridWorker.java:110)
at java.lang.Thread.run(Thread.java:748)

2018-01-22 11:30:01,049 INFO  [exchange-worker-#42%ContentStore-2%] GridCacheDatabaseSharedManager - Finished applying WAL changes [updatesApplied=0, time=68435ms]
2018-01-22 11:30:01,789 INFO  [main] GridDiscoveryManager - Topology snapshot [ver=4, servers=2, clients=0, CPUs=8, offheap=26.0GB, heap=4.0GB]
2018-01-22 11:30:01,789 INFO  [main] GridDiscoveryManager - Data Regions Configured:
2018-01-22 11:30:01,789 INFO  [main] GridDiscoveryManager -   ^-- default [initSize=256.0 MiB, maxSize=12.0 GiB, persistenceEnabled=false]
2018-01-22 11:30:01,789 INFO  [main] GridDiscoveryManager -   ^-- durable [initSize=256.0 MiB, maxSize=1.0 GiB, persistenceEnabled=true]



Re: Long activation times with Ignite persistence enabled

Posted by Andrey Kornev <an...@hotmail.com>.
Alexey,

Thanks a lot for looking into this!

My configuration is very basic: 3 caches all using standard 1024 partitions, sharing a 1GB persistent memory region.

Please find below the stack trace of the exchange worker thread captured while the node's activation is in progress (2.4 Ignite branch).

Hope it helps!

Thanks!
Andrey

"exchange-worker-#42%ignite-2%" #82 prio=5 os_prio=31 tid=0x00007ffe8bf1c000 nid=0xc403 waiting on condition [0x000070000ed43000]
   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:177)
at org.apache.ignite.internal.util.future.GridFutureAdapter.getUninterruptibly(GridFutureAdapter.java:145)
at org.apache.ignite.internal.processors.cache.persistence.file.AsyncFileIO.read(AsyncFileIO.java:95)
at org.apache.ignite.internal.processors.cache.persistence.file.FilePageStore.read(FilePageStore.java:324)
at org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.read(FilePageStoreManager.java:306)
at org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.read(FilePageStoreManager.java:291)
at org.apache.ignite.internal.processors.cache.persistence.pagemem.PageMemoryImpl.acquirePage(PageMemoryImpl.java:656)
at org.apache.ignite.internal.processors.cache.persistence.pagemem.PageMemoryImpl.acquirePage(PageMemoryImpl.java:576)
at org.apache.ignite.internal.processors.cache.persistence.DataStructure.acquirePage(DataStructure.java:130)
at org.apache.ignite.internal.processors.cache.persistence.freelist.PagesList.init(PagesList.java:212)
at org.apache.ignite.internal.processors.cache.persistence.freelist.AbstractFreeList.<init>(AbstractFreeList.java:367)
at org.apache.ignite.internal.processors.cache.persistence.freelist.CacheFreeListImpl.<init>(CacheFreeListImpl.java:47)
at org.apache.ignite.internal.processors.cache.persistence.GridCacheOffheapManager$GridCacheDataStore$1.<init>(GridCacheOffheapManager.java:1041)
at org.apache.ignite.internal.processors.cache.persistence.GridCacheOffheapManager$GridCacheDataStore.init0(GridCacheOffheapManager.java:1041)
at org.apache.ignite.internal.processors.cache.persistence.GridCacheOffheapManager$GridCacheDataStore.updateCounter(GridCacheOffheapManager.java:1247)
at org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtLocalPartition.updateCounter(GridDhtLocalPartition.java:835)
at org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtLocalPartition.casState(GridDhtLocalPartition.java:523)
- locked <0x000000077a3d1120> (a org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtLocalPartition)
at org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtLocalPartition.<init>(GridDhtLocalPartition.java:218)
at org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtPartitionTopologyImpl.forceCreatePartition(GridDhtPartitionTopologyImpl.java:804)
at org.apache.ignite.internal.processors.cache.persistence.GridCacheDatabaseSharedManager.restorePartitionState(GridCacheDatabaseSharedManager.java:2196)
at org.apache.ignite.internal.processors.cache.persistence.GridCacheDatabaseSharedManager.applyLastUpdates(GridCacheDatabaseSharedManager.java:2155)
at org.apache.ignite.internal.processors.cache.persistence.GridCacheDatabaseSharedManager.restoreState(GridCacheDatabaseSharedManager.java:1322)
at org.apache.ignite.internal.processors.cache.persistence.GridCacheDatabaseSharedManager.beforeExchange(GridCacheDatabaseSharedManager.java:1113)
at org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionsExchangeFuture.distributedExchange(GridDhtPartitionsExchangeFuture.java:1063)
at org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionsExchangeFuture.init(GridDhtPartitionsExchangeFuture.java:661)
at org.apache.ignite.internal.processors.cache.GridCachePartitionExchangeManager$ExchangeWorker.body(GridCachePartitionExchangeManager.java:2329)
at org.apache.ignite.internal.util.worker.GridWorker.run(GridWorker.java:110)
at java.lang.Thread.run(Thread.java:748)

2018-01-22 11:30:01,049 INFO  [exchange-worker-#42%ContentStore-2%] GridCacheDatabaseSharedManager - Finished applying WAL changes [updatesApplied=0, time=68435ms]
2018-01-22 11:30:01,789 INFO  [main] GridDiscoveryManager - Topology snapshot [ver=4, servers=2, clients=0, CPUs=8, offheap=26.0GB, heap=4.0GB]
2018-01-22 11:30:01,789 INFO  [main] GridDiscoveryManager - Data Regions Configured:
2018-01-22 11:30:01,789 INFO  [main] GridDiscoveryManager -   ^-- default [initSize=256.0 MiB, maxSize=12.0 GiB, persistenceEnabled=false]
2018-01-22 11:30:01,789 INFO  [main] GridDiscoveryManager -   ^-- durable [initSize=256.0 MiB, maxSize=1.0 GiB, persistenceEnabled=true]



________________________________
From: Alexey Goncharuk <al...@gmail.com>
Sent: Monday, January 22, 2018 12:30 AM
To: user@ignite.apache.org
Subject: Re: Long activation times with Ignite persistence enabled

Andrey,

Can you please describe in greater detail the configuration of your nodes (specifically, number of caches and number of partitions). Ignite would not load all the partitions into memory on startup simply because there is no such logic. What it does, however, is loading meta pages for each partition in each cache group to determine the correct cluster state and schedule rebalancing, if needed. If the number of caches x number of partitions is high, this may take a while.
If this is the case, you can either reduce the number of partitions or group logical caches with the same affinity into physical cache group, so that those caches will share the same partition file. See CacheConfiguration#setGroupName(String) for more detail.

Last but not least, it looks very suspicious that with 0 pending updates it took almost 90 seconds to read WAL. From the code, I see that this again may be related to partition state recovery, I will need to re-check this and get back to you later.

Thanks,
AG

2018-01-19 2:51 GMT+03:00 Andrey Kornev <an...@hotmail.com>>:
Hello,

I'm wondering if there is a way to improve the startup time of Ignite node when the persistence is enabled?

It seems the time is proportional to the size (and number) of the partition files. This is somewhat surprising as I expected the startup time be the same (plus-minus some constant factor) regardless of the amount of data persisted.

The delay looks to be due to Ignite loading *all* partition files for *all* persistence-enabled caches as part of a node's join. Here's an example of the startup log output:

2018-01-18 14:00:40,230 INFO  [exchange-worker-#42%ignite-1%] GridCacheDatabaseSharedManager - Read checkpoint status [startMarker=/tmp/storage/data/1/cp/1516311778910-d56f8ceb-2205-4bef-9ed3-a7446e34aa06-START.bin, endMarker=/tmp/storage/data/1/cp/1516311778910-d56f8ceb-2205-4bef-9ed3-a7446e34aa06-END.bin]
2018-01-18 14:00:40,230 INFO  [exchange-worker-#42%ignite-1%] GridCacheDatabaseSharedManager - Applying lost cache updates since last checkpoint record [lastMarked=FileWALPointer [idx=1693, fileOff=7970054, len=60339], lastCheckpointId=d56f8ceb-2205-4bef-9ed3-a7446e34aa06]
2018-01-18 14:00:57,114 WARN  [exchange-worker-#42%ignite-1%] PageMemoryImpl - Page evictions started, this will affect storage performance (consider increasing DataRegionConfiguration#setMaxSize).
2018-01-18 14:02:05,469 INFO  [exchange-worker-#42%ignite-1%] GridCacheDatabaseSharedManager - Finished applying WAL changes [updatesApplied=0, time=85234ms]

It took ≈1.5 minute to activate a node. To add insult to injury, the eviction kicked in and most of the loaded pages got evicted (in this test, I had the caches sharing a 1GB memory region loading about 10GB of data and index). In general, I think it's not unreasonable to expect 1-to-10 ratio of the data region size to the total persisted data size.

Why load all that data in the first place? It seems like a huge waste of time. Can the data partitions be loaded lazily on demand while the index partition can still be loaded during node startup?

Thanks
Andrey



Re: Long activation times with Ignite persistence enabled

Posted by Alexey Goncharuk <al...@gmail.com>.
Andrey,

Can you please describe in greater detail the configuration of your nodes
(specifically, number of caches and number of partitions). Ignite would not
load all the partitions into memory on startup simply because there is no
such logic. What it does, however, is loading meta pages for each partition
in each cache group to determine the correct cluster state and schedule
rebalancing, if needed. If the number of caches x number of partitions is
high, this may take a while.
If this is the case, you can either reduce the number of partitions or
group logical caches with the same affinity into physical cache group, so
that those caches will share the same partition file. See
CacheConfiguration#setGroupName(String) for more detail.

Last but not least, it looks very suspicious that with 0 pending updates it
took almost 90 seconds to read WAL. From the code, I see that this again
may be related to partition state recovery, I will need to re-check this
and get back to you later.

Thanks,
AG

2018-01-19 2:51 GMT+03:00 Andrey Kornev <an...@hotmail.com>:

> Hello,
>
> I'm wondering if there is a way to improve the startup time of Ignite node
> when the persistence is enabled?
>
> It seems the time is proportional to the size (and number) of the
> partition files. This is somewhat surprising as I expected the startup
> time be the same (plus-minus some constant factor) regardless of the amount
> of data persisted.
>
> The delay looks to be due to Ignite loading *all* partition files for
> *all* persistence-enabled caches as part of a node's join. Here's an
> example of the startup log output:
>
> 2018-01-18 14:00:40,230 INFO  [exchange-worker-#42%ignite-1%]
> GridCacheDatabaseSharedManager - Read checkpoint status
> [startMarker=/tmp/storage/data/1/cp/1516311778910-d56f8ceb-2205-4bef-9ed3-a7446e34aa06-START.bin,
> endMarker=/tmp/storage/data/1/cp/1516311778910-d56f8ceb-
> 2205-4bef-9ed3-a7446e34aa06-END.bin]
> 2018-01-18 14:00:40,230 INFO  [exchange-worker-#42%ignite-1%]
> GridCacheDatabaseSharedManager - Applying lost cache updates since last
> checkpoint record [lastMarked=FileWALPointer [idx=1693, fileOff=7970054,
> len=60339], lastCheckpointId=d56f8ceb-2205-4bef-9ed3-a7446e34aa06]
> 2018-01-18 14:00:57,114 WARN  [exchange-worker-#42%ignite-1%]
> PageMemoryImpl - Page evictions started, this will affect storage
> performance (consider increasing DataRegionConfiguration#setMaxSize).
> 2018-01-18 14:02:05,469 INFO  [exchange-worker-#42%ignite-1%]
> GridCacheDatabaseSharedManager - Finished applying WAL changes
> [updatesApplied=0, time=85234ms]
>
> It took ≈1.5 minute to activate a node. To add insult to injury, the
> eviction kicked in and most of the loaded pages got evicted (in this
> test, I had the caches sharing a 1GB memory region loading about 10GB of
> data and index). In general, I think it's not unreasonable to expect
> 1-to-10 ratio of the data region size to the total persisted data size.
>
> Why load all that data in the first place? It seems like a huge waste of
> time. Can the data partitions be loaded lazily on demand while the index
> partition can still be loaded during node startup?
>
> Thanks
> Andrey
>
>

Re: Long activation times with Ignite persistence enabled

Posted by Andrey Kornev <an...@hotmail.com>.
Hey Ilya!

In fact I was talking about pre-loading the index *partitions* not pages. But in any case, it's besides the point, just a potential enhancement.

What concerns me the most is how much time the activation takes and its apparent linear growth with the size of the on-disk data. In my case, I had only 10G of data on disk and that took 1.5 minutes from node start to the active state. Now, in production I'm looking at more than 100G of data per node (primary + backup partitions). Should I expect the activation to take 15 minutes? If this is indeed the case, then Ignite Persistence looses a lot of its initial appeal - "Instantaneous Cluster Restarts" - as touted here https://ignite.apache.org/features/durablememory.html. For comparison, on a 10Gpbs network I can transfer 100G in about 90 seconds (optimistically).

Next, to your point about "maybe applying updates from the WAL", I don't think that was the case (as indicated by "Finished applying WAL changes [updatesApplied=0, time=85234ms]" in the log output). Except for the initial load, the data has never changed between multiple restarts of the cluster (as well as of individual nodes). I even tried deleting all WAL files and the WAL archive to see if it makes any difference at all. In every instance the behavior was the same: 1.5 minutes of activation.

So, the question remains: are these activation times expected and the current state of the implementation is as good as it gets, or there is something (configuration-wise or in Ignite code) that can be done to improve the startup time?

Thanks!
Andrey

________________________________
From: ilya.kasnacheev <il...@gmail.com>
Sent: Friday, January 19, 2018 4:29 AM
To: user@ignite.apache.org
Subject: Re: Long activation times with Ignite persistence enabled

Hello!

As far as my understanding goes, there's no distinction in Ignite between
"data pages" and "index pages"
All pages in durable memory are parts of "B+ tree" and are holding both data
and references to downward branches in other pages.

I'm not sure it had to load all the pages. Maybe only ones that needed
applying updates from WAL? Which, in case of infrequent checkpoints, were
plentiful?

I expect most persistent databases would have non-trivial startup times when
full with data.

Regards,



--
Sent from: http://apache-ignite-users.70518.x6.nabble.com/

Re: Long activation times with Ignite persistence enabled

Posted by "ilya.kasnacheev" <il...@gmail.com>.
Hello!

As far as my understanding goes, there's no distinction in Ignite between
"data pages" and "index pages"
All pages in durable memory are parts of "B+ tree" and are holding both data
and references to downward branches in other pages.

I'm not sure it had to load all the pages. Maybe only ones that needed
applying updates from WAL? Which, in case of infrequent checkpoints, were
plentiful?

I expect most persistent databases would have non-trivial startup times when
full with data.

Regards,



--
Sent from: http://apache-ignite-users.70518.x6.nabble.com/