Version issue with concurrent cache updates (EntryProcessor)

classic Classic list List threaded Threaded
21 messages Options
12
Myron Chelyada Myron Chelyada
Reply | Threaded
Open this post in threaded view
|

Version issue with concurrent cache updates (EntryProcessor)

Hello team,

I am performing concurrent updates on the same entries in cache by using EntryProcessor.
(Entry processor just sums existing values with provided delta values. I.e. I have List of doubles as cache value).
And after performing such updates N times I expected to have final values equal to N (I used 1.0 as delta values in my testing). But my actual values are less (I could say even much less).
And after digging deeply into that and enabling debug I've captured the following statement in log files:

[cache.GridCacheMapEntry] - Received entry update with smaller version than current (will ignore) [entry=GridDhtAtomicCacheEntry [super=GridDhtCacheEntry [rdrs=[], locPart=GridDhtLocalPartition [id=207, mapPubSize=1, rmvQueue=GridCircularBuffer [sizeMask=511, idxGen=0], cntr=1, state=OWNING, reservations=0, empty=false, createTime=02/08/2016 15:12:26, mapPubSize=1], super=GridDistributedCacheEntry [super=GridCacheMapEntry [key=KeyCacheObjectImpl [val=<myKey>, hasValBytes=true], val=CacheObjectImpl [val=[1.0,1.0], hasValBytes=true], startVer=1454937145741, ver=GridCacheVersion [topVer=66417147, nodeOrderDrId=1, globalTime=1454937165753, order=1454937145734], hash=-1875348639, extras=null, flags=0]]]], newVer=GridCacheVersion [topVer=66417147, nodeOrderDrId=1, globalTime=1454937165753, order=1454937145732]]

My test is being run on single node(i.e. there is only one grid node) and I am using Ignite 1.5.0.final. There might be up to 5 threads performing concurrent updates. My key cache settings (which as far as I understand might affect update behavior) are:

cacheMode=PARTITIONED

atomicityMode=ATOMIC

atomicWriteOrderMode=CLOCK 

writeSynchronizationMode=FULL_SYNC


And my suspicion is that since there is CLOCK write order mode used and as it states:

"
In this mode, write versions are assigned on a sender node which generally leads to better performance in CacheWriteSynchronizationMode.FULL_SYNC synchronization mode, since in this case sender can send write requests to primary and backups at the same time."

, actually one thread assigns/initializes version earlier than another while updates(on entries) are performed in inverse order.

So, could you please clarify am I correct? If not, what is the reason? 

Is it expected behavior? How can It be handled correctly?

Thanks,

alexey.goncharuk alexey.goncharuk
Reply | Threaded
Open this post in threaded view
|

Re: Version issue with concurrent cache updates (EntryProcessor)

Myron,

This is a known usability issue, see [1]. You need to set atomicWriterOrderMode to PRIMARY in order to make entry processors to work correctly. I will cross-post this mail to devlist in order to raise the ticket priority.


--AG
Myron Chelyada Myron Chelyada
Reply | Threaded
Open this post in threaded view
|

Re: Version issue with concurrent cache updates (EntryProcessor)

Hi Alexey,

Thanks for reply.

Are there any other similar limitations with modes related to entry processor?  
Perhaps there is some detailed documentation (more detailed than I see on site)?

2016-02-09 12:09 GMT+02:00 Alexey Goncharuk <[hidden email]>:
Myron,

This is a known usability issue, see [1]. You need to set atomicWriterOrderMode to PRIMARY in order to make entry processors to work correctly. I will cross-post this mail to devlist in order to raise the ticket priority.


--AG

Myron Chelyada Myron Chelyada
Reply | Threaded
Open this post in threaded view
|

Re: Version issue with concurrent cache updates (EntryProcessor)

Hmm, doing the same but with:

atomicWriteOrderMode=PRIMARY

writeSynchronizationMode=PRIMARY_SYNC


causes:

[atomic.GridDhtAtomicCache] - <testCache> Unexpected exception during cache update

java.lang.AssertionError: Invalid version for inner update [entry=GridDhtAtomicCacheEntry [super=GridDhtCacheEntry [rdrs=[], locPart=GridDhtLocalPartition [id=351, mapPubSize=1, rmvQueue=GridCircularBuffer [sizeMask=511, idxGen=0], cntr=0, state=OWNING, reservations=0, empty=false, createTime=02/09/2016 14:52:05, mapPubSize=1], super=GridDistributedCacheEntry [super=GridCacheMapEntry [key=KeyCacheObjectImpl [val=<myKey>, hasValBytes=true], val=CacheObjectImpl [val=[1.0], hasValBytes=true], startVer=1455022325067, ver=GridCacheVersion [topVer=66502326, nodeOrderDrId=1, globalTime=1455022348434, order=1455022325392], hash=1026268892, extras=null, flags=0]]]], newVer=GridCacheVersion [topVer=66502326, nodeOrderDrId=1, globalTime=1455022348422, order=1455022325391]]

        at org.apache.ignite.internal.processors.cache.GridCacheMapEntry.innerUpdate(GridCacheMapEntry.java:2018)

        at org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridDhtAtomicCache.updatePartialBatch(GridDhtAtomicCache.java:2189)

        at org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridDhtAtomicCache.updateWithBatch(GridDhtAtomicCache.java:1747)

        at org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridDhtAtomicCache.updateAllAsyncInternal0(GridDhtAtomicCache.java:1299)

        at org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridDhtAtomicCache.updateAllAsyncInternal(GridDhtAtomicCache.java:1194)

        at org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridNearAtomicUpdateFuture.mapSingle(GridNearAtomicUpdateFuture.java:457)

        at org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridNearAtomicUpdateFuture.access$1400(GridNearAtomicUpdateFuture.java:72)

        at org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridNearAtomicUpdateFuture$UpdateState.map(GridNearAtomicUpdateFuture.java:931)

        at org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridNearAtomicUpdateFuture.mapOnTopology(GridNearAtomicUpdateFuture.java:417)

        at org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridNearAtomicUpdateFuture.map(GridNearAtomicUpdateFuture.java:283)

        at org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridDhtAtomicCache$18.apply(GridDhtAtomicCache.java:925)

        at org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridDhtAtomicCache$18.apply(GridDhtAtomicCache.java:923)

        at org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridDhtAtomicCache.asyncOp(GridDhtAtomicCache.java:700)

        at org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridDhtAtomicCache.updateAllAsync0(GridDhtAtomicCache.java:923)

        at org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridDhtAtomicCache.invokeAllAsync(GridDhtAtomicCache.java:815)

        at org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridDhtAtomicCache.invokeAll(GridDhtAtomicCache.java:739)

        at org.apache.ignite.internal.processors.cache.IgniteCacheProxy.invokeAll(IgniteCacheProxy.java:1537)




2016-02-09 13:18 GMT+02:00 Myron Chelyada <[hidden email]>:
Hi Alexey,

Thanks for reply.

Are there any other similar limitations with modes related to entry processor?  
Perhaps there is some detailed documentation (more detailed than I see on site)?

2016-02-09 12:09 GMT+02:00 Alexey Goncharuk <[hidden email]>:
Myron,

This is a known usability issue, see [1]. You need to set atomicWriterOrderMode to PRIMARY in order to make entry processors to work correctly. I will cross-post this mail to devlist in order to raise the ticket priority.


--AG


Myron Chelyada Myron Chelyada
Reply | Threaded
Open this post in threaded view
|

Re: Version issue with concurrent cache updates (EntryProcessor)

Team,

Could you please provide some input on that? 
I am blocked since it looks like EntryProcessor can't be used at all (at least with PARTITIONED cache).

Thanks,

2016-02-09 15:57 GMT+02:00 Myron Chelyada <[hidden email]>:
Hmm, doing the same but with:

atomicWriteOrderMode=PRIMARY

writeSynchronizationMode=PRIMARY_SYNC


causes:

[atomic.GridDhtAtomicCache] - <testCache> Unexpected exception during cache update

java.lang.AssertionError: Invalid version for inner update [entry=GridDhtAtomicCacheEntry [super=GridDhtCacheEntry [rdrs=[], locPart=GridDhtLocalPartition [id=351, mapPubSize=1, rmvQueue=GridCircularBuffer [sizeMask=511, idxGen=0], cntr=0, state=OWNING, reservations=0, empty=false, createTime=02/09/2016 14:52:05, mapPubSize=1], super=GridDistributedCacheEntry [super=GridCacheMapEntry [key=KeyCacheObjectImpl [val=<myKey>, hasValBytes=true], val=CacheObjectImpl [val=[1.0], hasValBytes=true], startVer=1455022325067, ver=GridCacheVersion [topVer=66502326, nodeOrderDrId=1, globalTime=1455022348434, order=1455022325392], hash=1026268892, extras=null, flags=0]]]], newVer=GridCacheVersion [topVer=66502326, nodeOrderDrId=1, globalTime=1455022348422, order=1455022325391]]

        at org.apache.ignite.internal.processors.cache.GridCacheMapEntry.innerUpdate(GridCacheMapEntry.java:2018)

        at org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridDhtAtomicCache.updatePartialBatch(GridDhtAtomicCache.java:2189)

        at org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridDhtAtomicCache.updateWithBatch(GridDhtAtomicCache.java:1747)

        at org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridDhtAtomicCache.updateAllAsyncInternal0(GridDhtAtomicCache.java:1299)

        at org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridDhtAtomicCache.updateAllAsyncInternal(GridDhtAtomicCache.java:1194)

        at org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridNearAtomicUpdateFuture.mapSingle(GridNearAtomicUpdateFuture.java:457)

        at org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridNearAtomicUpdateFuture.access$1400(GridNearAtomicUpdateFuture.java:72)

        at org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridNearAtomicUpdateFuture$UpdateState.map(GridNearAtomicUpdateFuture.java:931)

        at org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridNearAtomicUpdateFuture.mapOnTopology(GridNearAtomicUpdateFuture.java:417)

        at org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridNearAtomicUpdateFuture.map(GridNearAtomicUpdateFuture.java:283)

        at org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridDhtAtomicCache$18.apply(GridDhtAtomicCache.java:925)

        at org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridDhtAtomicCache$18.apply(GridDhtAtomicCache.java:923)

        at org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridDhtAtomicCache.asyncOp(GridDhtAtomicCache.java:700)

        at org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridDhtAtomicCache.updateAllAsync0(GridDhtAtomicCache.java:923)

        at org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridDhtAtomicCache.invokeAllAsync(GridDhtAtomicCache.java:815)

        at org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridDhtAtomicCache.invokeAll(GridDhtAtomicCache.java:739)

        at org.apache.ignite.internal.processors.cache.IgniteCacheProxy.invokeAll(IgniteCacheProxy.java:1537)




2016-02-09 13:18 GMT+02:00 Myron Chelyada <[hidden email]>:
Hi Alexey,

Thanks for reply.

Are there any other similar limitations with modes related to entry processor?  
Perhaps there is some detailed documentation (more detailed than I see on site)?

2016-02-09 12:09 GMT+02:00 Alexey Goncharuk <[hidden email]>:
Myron,

This is a known usability issue, see [1]. You need to set atomicWriterOrderMode to PRIMARY in order to make entry processors to work correctly. I will cross-post this mail to devlist in order to raise the ticket priority.


--AG



alexey.goncharuk alexey.goncharuk
Reply | Threaded
Open this post in threaded view
|

Re: Version issue with concurrent cache updates (EntryProcessor)

Myron,

I tried to reproduce this assertion on ignite-1.5, but with no luck. Can you share your full cache configuration, the number of nodes in your clusterr and a code snippet allowing to reproduce the issue?

Thanks,
AG
Myron Chelyada Myron Chelyada
Reply | Threaded
Open this post in threaded view
|

Re: Version issue with concurrent cache updates (EntryProcessor)

Hi Alexey,

Will try to extract main logic into some test that would allow to reproduce it.
But in meanwhile I figured out that issue appears only on cache that is store backed. I.e. as soon as I set either "readThrough" or "writeThrough" to "false" (or both) issue disappear.
But actually in my case I need both enabled.

2016-02-10 14:24 GMT+02:00 Alexey Goncharuk <[hidden email]>:
Myron,

I tried to reproduce this assertion on ignite-1.5, but with no luck. Can you share your full cache configuration, the number of nodes in your clusterr and a code snippet allowing to reproduce the issue?

Thanks,
AG

Myron Chelyada Myron Chelyada
Reply | Threaded
Open this post in threaded view
|

Re: Version issue with concurrent cache updates (EntryProcessor)

So,

Please find test in attachment which allows to reproduce the issue. 
It is very plain and much simpler than I initially described. 
I was confused before because I was able to reproduce this issue on one environment and couldn't on another. And the reason for that is that assertion were not enabled there.




2016-02-11 19:31 GMT+02:00 Myron Chelyada <[hidden email]>:
Hi Alexey,

Will try to extract main logic into some test that would allow to reproduce it.
But in meanwhile I figured out that issue appears only on cache that is store backed. I.e. as soon as I set either "readThrough" or "writeThrough" to "false" (or both) issue disappear.
But actually in my case I need both enabled.

2016-02-10 14:24 GMT+02:00 Alexey Goncharuk <[hidden email]>:
Myron,

I tried to reproduce this assertion on ignite-1.5, but with no luck. Can you share your full cache configuration, the number of nodes in your clusterr and a code snippet allowing to reproduce the issue?

Thanks,
AG



CacheEntryProcessorTest.java (7K) Download Attachment
alexey.goncharuk alexey.goncharuk
Reply | Threaded
Open this post in threaded view
|

Re: Version issue with concurrent cache updates (EntryProcessor)

Myron,

Thank you for reporting the issue. The assertion happens when the value is present in the store, absent in the cache and you run invokeAll(). As a temporary solution, you can either call invoke() for each particular key individually, or call getAll() for the keys prior to calling invokeAll() (this will pre-load the values to the cache). 
Since the issue is pretty critical, I believe it will be fixed in 1.6 (if not earlier).

Yakov, Sam,
I created a ticket [1] and suggested a fix there, can you take a look and check if the fix is ok?

Thanks,
AG

Myron Chelyada Myron Chelyada
Reply | Threaded
Open this post in threaded view
|

Re: Version issue with concurrent cache updates (EntryProcessor)

Will try to apply some workaround and looking forward to fix.

2016-02-12 16:58 GMT+02:00 Alexey Goncharuk <[hidden email]>:
Myron,

Thank you for reporting the issue. The assertion happens when the value is present in the store, absent in the cache and you run invokeAll(). As a temporary solution, you can either call invoke() for each particular key individually, or call getAll() for the keys prior to calling invokeAll() (this will pre-load the values to the cache). 
Since the issue is pretty critical, I believe it will be fixed in 1.6 (if not earlier).

Yakov, Sam,
I created a ticket [1] and suggested a fix there, can you take a look and check if the fix is ok?

Thanks,
AG


Myron Chelyada Myron Chelyada
Reply | Threaded
Open this post in threaded view
|

Re: Version issue with concurrent cache updates (EntryProcessor)

Hello Team,

Looks like I am faced to similar issue in case of cache rebalancing. 
I.e. there are few nodes and cache data are being modified/updated via  EntryProcessor. And at some moment additional node is joined to grid.
And finally (after rebalance complete and update complete) I got cache values that are less  than expected for some cache entries(but values are aligned to how they were combined into single update batch).
This time I am not getting assertion error but after enabling debug and etc. I am getting messages like:

... cache.GridCacheMapEntry] - Received entry update with smaller version than current (will ignore) ...

So, I am wondering is it related to versioning issue described previously in this thread or is it another (known?) issue or perhaps it is not an issue at all and I am doing/expecting something wrong?

Thanks,


2016-02-13 9:17 GMT+02:00 Myron Chelyada <[hidden email]>:
Will try to apply some workaround and looking forward to fix.

2016-02-12 16:58 GMT+02:00 Alexey Goncharuk <[hidden email]>:
Myron,

Thank you for reporting the issue. The assertion happens when the value is present in the store, absent in the cache and you run invokeAll(). As a temporary solution, you can either call invoke() for each particular key individually, or call getAll() for the keys prior to calling invokeAll() (this will pre-load the values to the cache). 
Since the issue is pretty critical, I believe it will be fixed in 1.6 (if not earlier).

Yakov, Sam,
I created a ticket [1] and suggested a fix there, can you take a look and check if the fix is ok?

Thanks,
AG



Myron Chelyada Myron Chelyada
Reply | Threaded
Open this post in threaded view
|

Re: Version issue with concurrent cache updates (EntryProcessor)

Hello team,

Could you please provide your input on that?

Thanks,

2016-02-25 16:28 GMT+02:00 Myron Chelyada <[hidden email]>:
Hello Team,

Looks like I am faced to similar issue in case of cache rebalancing. 
I.e. there are few nodes and cache data are being modified/updated via  EntryProcessor. And at some moment additional node is joined to grid.
And finally (after rebalance complete and update complete) I got cache values that are less  than expected for some cache entries(but values are aligned to how they were combined into single update batch).
This time I am not getting assertion error but after enabling debug and etc. I am getting messages like:

... cache.GridCacheMapEntry] - Received entry update with smaller version than current (will ignore) ...

So, I am wondering is it related to versioning issue described previously in this thread or is it another (known?) issue or perhaps it is not an issue at all and I am doing/expecting something wrong?

Thanks,


2016-02-13 9:17 GMT+02:00 Myron Chelyada <[hidden email]>:
Will try to apply some workaround and looking forward to fix.

2016-02-12 16:58 GMT+02:00 Alexey Goncharuk <[hidden email]>:
Myron,

Thank you for reporting the issue. The assertion happens when the value is present in the store, absent in the cache and you run invokeAll(). As a temporary solution, you can either call invoke() for each particular key individually, or call getAll() for the keys prior to calling invokeAll() (this will pre-load the values to the cache). 
Since the issue is pretty critical, I believe it will be fixed in 1.6 (if not earlier).

Yakov, Sam,
I created a ticket [1] and suggested a fix there, can you take a look and check if the fix is ok?

Thanks,
AG




alexey.goncharuk alexey.goncharuk
Reply | Threaded
Open this post in threaded view
|

Re: Version issue with concurrent cache updates (EntryProcessor)

Myron,

We have a specific test for the exact use-case you have described and it passes - see IgniteAtomicCacheEntryProcessorNodeJoinTest. I tried to play with the configuration (added test store, tried different memory modes), but was not able to make the test fail.

Is there any change you can share a reproducible example as you did before for the issue with the assertion you've encountered?

Thanks,
AG
Myron Chelyada Myron Chelyada
Reply | Threaded
Open this post in threaded view
|

Re: Version issue with concurrent cache updates (EntryProcessor)

Alexey,

For now, I've only figured out that problem is on node that is joining the grid and with records/keys for which it becomes primary.
Also I tried different rebalance modes but with no luck. But now I can confirm that "cacheStore" makes no impact on that (tried to disable cacheStore and result is the same).
The other thing I am observing is that "innerUpdate" is being actually called but update is not applied. Will try to capture more information and/or build/extract some test code that would allow to reproduce it.
But meanwhile if provide above information would provide you some hints and allow you to see/find the root cause that would be awesome.

Thanks,
Myron

2016-02-29 1:13 GMT+02:00 Alexey Goncharuk <[hidden email]>:
Myron,

We have a specific test for the exact use-case you have described and it passes - see IgniteAtomicCacheEntryProcessorNodeJoinTest. I tried to play with the configuration (added test store, tried different memory modes), but was not able to make the test fail.

Is there any change you can share a reproducible example as you did before for the issue with the assertion you've encountered?

Thanks,
AG

Myron Chelyada Myron Chelyada
Reply | Threaded
Open this post in threaded view
|

Re: Version issue with concurrent cache updates (EntryProcessor)

Hello Team,

Seems like I've found the root cause of such behavior. 
My EntryProcessor is getting local node resource (actually spring bean). But since resource injection is not supported for EntryProcessor yet (please correct me if I am wrong) I have made a workaround of getting named grid instance and then fetching resource from "nodeLocalMap" there.
And again since there is no possibility to initialize nodeLocalMap via IgniteConfiguration (similar to have lifecycle beans and/or local vent listeners) I've done some workaround to put data there on startup. 
And finally I am faced to issue when nodeLocalMap is not initialized yet but node is already getting EntryProcessors from other nodes (i.e. invokeAll on other nodes). 
And it was a very big surprise for me that there is no any exception nor error/warning within log files (and thats why I spent a lot of time on finding the root cause). So my question is shouldn't such error should be logged?
Also as a fix I tried to use lifecycle bean now which is actually putting resources into nodeLocalMap and my preliminary tests passed. But my question is how reliable is such approach, are there perhaps some edge cases?
Finally (since I've touched resource injection chapter), what are the plans of adding resource injection support for EntryProcessor and nodeLocalMap initialization/availability (I saw some related issues in Jira)?

Thanks,
 

2016-02-29 18:20 GMT+02:00 Myron Chelyada <[hidden email]>:
Alexey,

For now, I've only figured out that problem is on node that is joining the grid and with records/keys for which it becomes primary.
Also I tried different rebalance modes but with no luck. But now I can confirm that "cacheStore" makes no impact on that (tried to disable cacheStore and result is the same).
The other thing I am observing is that "innerUpdate" is being actually called but update is not applied. Will try to capture more information and/or build/extract some test code that would allow to reproduce it.
But meanwhile if provide above information would provide you some hints and allow you to see/find the root cause that would be awesome.

Thanks,
Myron

2016-02-29 1:13 GMT+02:00 Alexey Goncharuk <[hidden email]>:
Myron,

We have a specific test for the exact use-case you have described and it passes - see IgniteAtomicCacheEntryProcessorNodeJoinTest. I tried to play with the configuration (added test store, tried different memory modes), but was not able to make the test fail.

Is there any change you can share a reproducible example as you did before for the issue with the assertion you've encountered?

Thanks,
AG


alexey.goncharuk alexey.goncharuk
Reply | Threaded
Open this post in threaded view
|

Re: Version issue with concurrent cache updates (EntryProcessor)

Myron,

What approach did you use initially to initialize the node local map? IgniteNode is considered to be fully functional as soon as Ignition.start() method returns control, so any operations done on NodeLocalMap after the node start should be considered to be run concurrently with EntryProcessors invocations. If this is a requirement for you, you can put a simple CountDown latch to the NodeLocalMap, wait for it in an EntryProcessor and release it once the NodeLocalMap is in a consistent state.

It is safe to initialize the NodeLocalMap synchronously from LifecycleBean's BEFORE_NODE_START event since the event notification is run at the moment when the discovery SPI is not started yet and no other nodes "know" about this starting node.

As for the EntryProcessor resource injection, you can track this ticket [1]. Looks like it is almost ready and will be merged to master pretty soon.

--
AG

Myron Chelyada Myron Chelyada
Reply | Threaded
Open this post in threaded view
|

Re: Version issue with concurrent cache updates (EntryProcessor)

Alexey,

My initial approach was to put reference into NodeLocalMap after "start" method since I saw no other possibility to do that since I was a little bit bit confused of - https://issues.apache.org/jira/browse/IGNITE-1018.
But then I tried to go with LifecycleBean but with AFTER_NODE_START.

So, could you please clarify do I know can be coupled to BEFORE_NODE_START?

And with regards to:

>> 
 If this is a requirement for you, you can put a simple CountDown latch to the NodeLocalMap, wait for it in an EntryProcessor and release it once the NodeLocalMap is in a consistent state.

Here the question still remains the same - Where/when it is safe to put this CountDown latch into NodeLocalMap?

Thanks,


2016-03-02 0:06 GMT+02:00 Alexey Goncharuk <[hidden email]>:
Myron,

What approach did you use initially to initialize the node local map? IgniteNode is considered to be fully functional as soon as Ignition.start() method returns control, so any operations done on NodeLocalMap after the node start should be considered to be run concurrently with EntryProcessors invocations. If this is a requirement for you, you can put a simple CountDown latch to the NodeLocalMap, wait for it in an EntryProcessor and release it once the NodeLocalMap is in a consistent state.

It is safe to initialize the NodeLocalMap synchronously from LifecycleBean's BEFORE_NODE_START event since the event notification is run at the moment when the discovery SPI is not started yet and no other nodes "know" about this starting node.

As for the EntryProcessor resource injection, you can track this ticket [1]. Looks like it is almost ready and will be merged to master pretty soon.

--
AG


alexey.goncharuk alexey.goncharuk
Reply | Threaded
Open this post in threaded view
|

Re: Version issue with concurrent cache updates (EntryProcessor)

Oh, I see now what you mean, IGNITE-1018 has escaped my view. Then, until IGNITE-1018 is fixed, the only guaranteed approach is to wait on a CDL. Here is the pseudo-code that I have in mind:

LifecycleBean or after Ignition.start():
// Populate your node local map
....
CountDownLatch init = nlm.get(INIT_KEY);

if (init == null) {
CountDownLatch old = nlm.putIfAbsent(INIT_KEY, init = new CountDownLatch(1));

if (old != null)
init = old;
}

init.countDown();

EntryProcessor:

CountDownLatch init = nlm.get(INIT_KEY);

if (init == null) {
CountDownLatch old = nlm.putIfAbsent(INIT_KEY, init = new CountDownLatch(1));

if (old != null)
init = old;
}

init.await();
// Run entry procesor

This approach has only one restriction - you should not call any cache operations on the started node before you release the latch, otherwise this may lead to a distributed deadlock.
Hope this helps!
AG
Myron Chelyada Myron Chelyada
Reply | Threaded
Open this post in threaded view
|

Re: Version issue with concurrent cache updates (EntryProcessor)

Alexey,

Thanks for advice.

But could you also provide any timelines for listed issues?



2016-03-02 1:14 GMT+02:00 Alexey Goncharuk <[hidden email]>:
Oh, I see now what you mean, IGNITE-1018 has escaped my view. Then, until IGNITE-1018 is fixed, the only guaranteed approach is to wait on a CDL. Here is the pseudo-code that I have in mind:

LifecycleBean or after Ignition.start():
// Populate your node local map
....
CountDownLatch init = nlm.get(INIT_KEY);

if (init == null) {
CountDownLatch old = nlm.putIfAbsent(INIT_KEY, init = new CountDownLatch(1));

if (old != null)
init = old;
}

init.countDown();

EntryProcessor:

CountDownLatch init = nlm.get(INIT_KEY);

if (init == null) {
CountDownLatch old = nlm.putIfAbsent(INIT_KEY, init = new CountDownLatch(1));

if (old != null)
init = old;
}

init.await();
// Run entry procesor

This approach has only one restriction - you should not call any cache operations on the started node before you release the latch, otherwise this may lead to a distributed deadlock.
Hope this helps!
AG

alexey.goncharuk alexey.goncharuk
Reply | Threaded
Open this post in threaded view
|

Re: Version issue with concurrent cache updates (EntryProcessor)

Myron,

I believe IGNITE-2645 should be fixed in the near future since the issue is critical, and will definitely be included to 1.6.

As for the IGNITE-1018, I will not speculate on the timelines because the issue has some workarounds, even though it is possible that it will be fixed for 1.6 if someone in the community picks it up.

You can track the discussion on the dev list regarding the 1.6 release and planned features for it. Needless to say that you are always welcome to contribute a patch for any of the tickets you would like to be released :) 

--
AG
12