[
https://issues.apache.org/jira/browse/IGNITE-8166?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16432487#comment-16432487
]
Aleksey Plekhanov commented on IGNITE-8166:
-------------------------------------------
I implement yet another reproducer, now it's not racy and reproduces in 100%
cases. PME hangs when any error during checkpoint begin stage occurs (to
emulate error in my reproducer I used internal listener, but it also can be
external reason, such as IO error when writing to WAL). To fix this, I think,
{{cpBeginFut}} should be completed with error each time corresponding
{{cpFinishFut}} completes with error.
{code:java}
public class PmeHangsTest extends GridCommonAbstractTest {
/** {@inheritDoc} */
@Override protected IgniteConfiguration getConfiguration(String
igniteInstanceName) throws Exception {
IgniteConfiguration cfg = super.getConfiguration(igniteInstanceName);
cfg.setDataStorageConfiguration(
new DataStorageConfiguration().setDefaultDataRegionConfiguration(
new DataRegionConfiguration().setPersistenceEnabled(true)
)
);
return cfg;
}
public void testPmeHangs() throws Exception {
cleanPersistenceDir();
IgniteEx ignite = startGrid(0);
((GridCacheDatabaseSharedManager)ignite.context().cache().context().database()).addCheckpointListener(
new DbCheckpointListener() {
@Override public void onCheckpointBegin(Context ctx) throws
IgniteCheckedException {
throw new IllegalStateException();
}
}
);
ignite.cluster().active(true);
}
}
{code}
> stopGrid() hangs in some cases when node is invalidated and PDS is enabled
> --------------------------------------------------------------------------
>
> Key: IGNITE-8166
> URL: https://issues.apache.org/jira/browse/IGNITE-8166
> Project: Ignite
> Issue Type: Bug
> Affects Versions: 2.5
> Reporter: Aleksey Plekhanov
> Priority: Major
> Labels: iep-14
>
> Node invalidation via FailureProcessor can hang {{exchange-worker}} and
> {{stopGrid()}} when PDS is enabled.
> Reproducer (reproducer is racy, sometimes finished without hang):
> {code:java}
> public class StopNodeHangsTest extends GridCommonAbstractTest {
> /** Offheap size for memory policy. */
> private static final int SIZE = 10 * 1024 * 1024;
> /** Page size. */
> static final int PAGE_SIZE = 2048;
> /** Number of entries. */
> static final int ENTRIES = 2_000;
> /** {@inheritDoc} */
> @Override protected IgniteConfiguration getConfiguration(String
> igniteInstanceName) throws Exception {
> IgniteConfiguration cfg = super.getConfiguration(igniteInstanceName);
> DataStorageConfiguration dsCfg = new DataStorageConfiguration();
> DataRegionConfiguration dfltPlcCfg = new DataRegionConfiguration();
> dfltPlcCfg.setName("dfltPlc");
> dfltPlcCfg.setInitialSize(SIZE);
> dfltPlcCfg.setMaxSize(SIZE);
> dfltPlcCfg.setPersistenceEnabled(true);
> dsCfg.setDefaultDataRegionConfiguration(dfltPlcCfg);
> dsCfg.setPageSize(PAGE_SIZE);
> cfg.setDataStorageConfiguration(dsCfg);
> cfg.setFailureHandler(new FailureHandler() {
> @Override public boolean onFailure(Ignite ignite, FailureContext
> failureCtx) {
> return true;
> }
> });
> return cfg;
> }
> public void testStopNodeHangs() throws Exception {
> cleanPersistenceDir();
> IgniteEx ignite0 = startGrid(0);
> IgniteEx ignite1 = startGrid(1);
> ignite1.cluster().active(true);
> awaitPartitionMapExchange();
> IgniteCache cache = ignite1.getOrCreateCache("TEST");
> Map<Integer, Object> entries = new HashMap<>();
> for (int i = 0; i < ENTRIES; i++)
> entries.put(i, new byte[PAGE_SIZE * 2 / 3]);
> cache.putAll(entries);
> ignite1.context().failure().process(new
> FailureContext(FailureType.CRITICAL_ERROR, null));
> stopGrid(0);
> stopGrid(1);
> }
> }
> {code}
> {{stopGrid(1)}} waiting until exchange finished, {{exchange-worker}} waits on
> method {{GridCacheDatabaseSharedManager#checkpointReadLock}} for
> {{CheckpointProgressSnapshot#cpBeginFut}}, but this future is never done
> because {{db-checkpoint-thread}} got exception at
> {{GridCacheDatabaseSharedManager.Checkpointer#markCheckpointBegin}} thrown by
> {{FileWriteAheadLogManager#checkNode}} and leave method
> {{markCheckpointBegin}} before future is done ({{curr.cpBeginFut.onDone();}})
--
This message was sent by Atlassian JIRA
(v7.6.3#76005)