IDIEP-14
Author

Anton Vinogradov

Andrey Gura

SponsorAndrey Gura
CreatedFeb 20 2018
Status

DONE


Motivation

Apache Ignite should have some general approach to handle critical failures.

Description

The following failures should be treated as critical:

  • System critical errors (e.g. OutOfMemoryError);
  • Unintentional system worker termination (e.g. due to an unhandled exception);
  • Cluster node segmentation.

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:

  • File IO errors. Usually IOException's threw by read/write operations on file system. The following subsystems should be considered as critical:
    • WAL
    • Page store
    • Meta store
    • Binary meta store
  • 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 worker:

  • disco-event-worker
  • tcp-disco-srvr
  • tcp-disco-msg-worker
  • tcp-comm-worker
  • grid-nio-worker
  • exchange-worker
  • sys-stripe
  • grid-timeout-worker
  • db-checkpoint-thread
  • wal-file-archiver
  • wal-write-worker
  • wal-file-decompressor
  • ttl-cleanup-worker
  • nio-acceptor

Important notes:

  • More than one Ignite node could be started in one JVM process.
  • Different nodes in one JVM process could belong to different clusters.

Initial design

IgniteConfiguration should be extended with methods:

public IgniteConfiguration setFailureHandler(FailureHandler hnd);

public FailureHandler getFailureHandler();

 

Where:

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:

 

@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:

 

<bean class="org.apache.ignite.configuration.IgniteConfiguration">
    <property name="failureHandler">
        <bean class="org.apache.ignite.failure.StopNodeFailureHandler"/>
    </property>
</bean>

 

Risks and Assumptions

 

Discussion Links

  1. Internal problems requiring graceful node shutdown, reboot, etc.
  2. IEP-14: Ignite failures handling (Discussion)

Reference Links

  1. Apache Ignite documentation: Ignite life cycle
  2. Apache Ignite documentation: Start from command line

Tickets

key summary type created updated due assignee reporter priority status resolution

JQL and issue key arguments for this macro require at least one Jira application link to be configured

  • No labels

5 Comments

  1. Andrey Gura Alexey Goncharuk

    1. we should think of setting default exception handler for internal threads
    2. how about self killing if local node blocks partition map exchange
    3. how about forcibly killing remote node if coordinator detects that it blocks PME
    1. If we could unblock a frozen PME process, this would be AWESOME!

    2. UncaughtExceptionHandler is good solution but it doesn't take into account mistakes that can lead to unintentional thread termination. I think we will use default handlers in the future when we add ability to monitor critical workers and will be able detect worker termination outside.

  2. I would like to propose the following recovery scenarios:

    • stop+kill (default) - this mode will attempt to gracefully stop a frozen node and then, after a certain timeout, kill the process.
    • stop - this mode will attempt to gracefully stop a frozen node without trying to kill the process
    • kill - this mode will attempt to kill the running process without trying to gracefully stop the node.
    • none - do nothing

    Also note that ignite.sh maybe started in auto-restart mode, so the nodes may be automatically restarted after stopped or killed.

     

    1. Dmitriy Setrakyan, I believe that suggested failure handler implementations cover all scenarios. One important thing here that autorestart should use System.exit with specific exit code while process kill should use Runtime.halt call.