ID | IEP-14 | ||||||||
Author | |||||||||
Sponsor | DmitryAndrey Gura | ||||||||
Created | Feb 20 2018 | ||||||||
Status |
|
Table of Contents |
---|
Apache Ignite should have some general engine approach to handle critical failures.
List of The following failures should be covered by this engine:
treated as critical:
OutOfMemoryError
);User should have an ability to define node behavior in case of this failures.
System critical error - error which leads to the system's inoperability.
The following system critical errors should be handled with proposed approach:
IOException
's threw by read/write operations on file system. The following subsystems should be considered as critical:IgniteOutOfMemoryException
OutOfMemoryError
(we should have some memory reserved for this case at node startup to increase chances to handle OOM).The following system workers are critical and ignite node will be inoperative in case of termination one of this workerList of system workers should be covered by this engine:
disco-event-worker
tcp-disco-sock-readertcp-disco-srvr
tcp-disco-msg-worker
tcp-comm-worker
grid-nio-worker
-tcp-commexchange-worker
sys-stripe
grid-timeout-worker
db-checkpoint-thread
wal-file-archiver
wal-write-worker
wal-file-decompressor
ttl-cleanup-worker
nio-acceptor
List of errors to be handled
IgniteConfiguration have to be extended with methods
IgniteConfiguration setIgniteFailureHandler(IgniteFailureHandler igniteFailureHnd)
IgniteFailureHandler getIgniteFailureHandler()
Where
interface IgniteFailureHandler {
IgniteFailureAction onFailure(IgniteFailureContext failureCtx);
}
class IgniteFailureContext {
IgniteFailureType type;
Throwable cause;
}
enum IgniteFailureAction {
RESTART_JVM,
STOP,
NOOP;
}
enum IgniteFailureType {
SEGMENTATION,
SYSTEM_WORKER_CRASHED,
CRITICAL_ERROR
}
So, provided by user subclass of IgniteFailureHandler able to decide what to do (see. IgniteFailureAction) on each registered failure (see. IgniteFailureContext).
// Describe project risks, such as API or binary compatibility issues, major protocol changes, etc.
...
Important notes:
should be extended with methods:IgniteConfiguration
Code Block | ||
---|---|---|
| ||
public IgniteConfiguration setFailureHandler(FailureHandler hnd);
public FailureHandler getFailureHandler(); |
Where:
Code Block | ||
---|---|---|
| ||
interface FailureHandler {
boolean onFailure(Ignite ignite, FailureContext failureCtx);
}
class FailureContext {
FailureType type;
Throwable error;
}
enum FailureType {
SEGMENTATION,
SYSTEM_WORKER_TERMINATION,
CRITICAL_ERROR
} |
FailureHandler
implementation will be able to handle Ignite critical failures accordingly to strategy provided by user.
The following implementations should be provided out of the box:
NoOpFailureHandler
- Just ignores any failure. It's useful for tests and debugging.RestartProcessFailureHandler -
Specific implementation that could be used only with ignite.(sh|bat). Process must be terminated using Ignition.restart(true)
call.StopNodeFailureHandler
- This implementation will stop Ignite node in case of critical error using Ignition.stop(true)
or Ignition.stop(nodeName, true)
call.StopNodeOrHaltFailureHandler(boolean tryStop, long timeout)
- This implementation will try to stop node if tryStop
value is true
. If node can't be stopped during provided timeout
or tryStop
value is false
then JVM process will be terminated forcibly ( Runtime.halt()
).Default failure handler is StopNodeOrHaltFailureProcessor
where tryStop
value is false
.
Critical system worker must catch all exceptions ( Throwable
and derived classes) in high-level try-catch
block and take into account that thread could be terminated due to an programmatic mistake that leads to unintentional worker termination. So basic template should looks like the following code snippet:
Code Block | ||
---|---|---|
| ||
@Override
public void run() {
Throwable err = null;
try {
// Critical worker's code.
}
catch(Throwable e) {
err = e;
}
finally {
// Call failure handler.
FailureContext failureCtx = new FaulureCtx(FailureType.SYSTEM_WORKER_TERMINATION, err);
ctx.failure().process(failureCtx); // Handle failure. Where ctx - kernal context.
}
} |
Example of using FailureHandler
in IgniteConfiguration
via Spring XML:
Code Block | ||
---|---|---|
| ||
<bean class="org.apache.ignite.configuration.IgniteConfiguration">
<property name="failureHandler">
<bean class="org.apache.ignite.failure.StopNodeFailureHandler"/>
</property>
</bean> |
Jira | ||||||||||
---|---|---|---|---|---|---|---|---|---|---|
|