You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by sb...@apache.org on 2014/12/05 15:07:43 UTC

[42/52] [abbrv] incubator-ignite git commit: # Renaming

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/50d1554c/modules/core/src/main/java/org/gridgain/grid/spi/failover/jobstealing/GridJobStealingFailoverSpi.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/spi/failover/jobstealing/GridJobStealingFailoverSpi.java b/modules/core/src/main/java/org/gridgain/grid/spi/failover/jobstealing/GridJobStealingFailoverSpi.java
deleted file mode 100644
index 98c378f..0000000
--- a/modules/core/src/main/java/org/gridgain/grid/spi/failover/jobstealing/GridJobStealingFailoverSpi.java
+++ /dev/null
@@ -1,344 +0,0 @@
-/* @java.file.header */
-
-/*  _________        _____ __________________        _____
- *  __  ____/___________(_)______  /__  ____/______ ____(_)_______
- *  _  / __  __  ___/__  / _  __  / _  / __  _  __ `/__  / __  __ \
- *  / /_/ /  _  /    _  /  / /_/ /  / /_/ /  / /_/ / _  /  _  / / /
- *  \____/   /_/     /_/   \_,__/   \____/   \__,_/  /_/   /_/ /_/
- */
-
-package org.gridgain.grid.spi.failover.jobstealing;
-
-import org.apache.ignite.*;
-import org.apache.ignite.cluster.*;
-import org.apache.ignite.resources.*;
-import org.apache.ignite.spi.*;
-import org.gridgain.grid.*;
-import org.gridgain.grid.spi.failover.*;
-import org.gridgain.grid.spi.failover.always.*;
-import org.gridgain.grid.util.typedef.*;
-import org.gridgain.grid.util.typedef.internal.*;
-
-import java.util.*;
-
-import static org.apache.ignite.spi.collision.jobstealing.JobStealingCollisionSpi.*;
-
-/**
- * Job stealing failover SPI needs to always be used in conjunction with
- * {@link org.apache.ignite.spi.collision.jobstealing.JobStealingCollisionSpi} SPI. When {@link org.apache.ignite.spi.collision.jobstealing.JobStealingCollisionSpi}
- * receives a <b>steal</b> request and rejects jobs so they can be routed to the
- * appropriate node, it is the responsibility of this {@code GridJobStealingFailoverSpi}
- * SPI to make sure that the job is indeed re-routed to the node that has sent the initial
- * request to <b>steal</b> it.
- * <p>
- * {@code GridJobStealingFailoverSpi} knows where to route a job based on the
- * {@link org.apache.ignite.spi.collision.jobstealing.JobStealingCollisionSpi#THIEF_NODE_ATTR} job context attribute (see {@link org.apache.ignite.compute.ComputeJobContext}).
- * Prior to rejecting a job,  {@link org.apache.ignite.spi.collision.jobstealing.JobStealingCollisionSpi} will populate this
- * attribute with the ID of the node that wants to <b>steal</b> this job.
- * Then {@code GridJobStealingFailoverSpi} will read the value of this attribute and
- * route the job to the node specified.
- * <p>
- * If failure is caused by a node crash, and not by <b>steal</b> request, then this
- * SPI behaves identically to {@link GridAlwaysFailoverSpi}, and tries to find the
- * next balanced node to fail-over a job to.
- * <p>
- * <h1 class="header">Configuration</h1>
- * <h2 class="header">Mandatory</h2>
- * This SPI has no mandatory configuration parameters.
- * <h2 class="header">Optional</h2>
- * This SPI has following optional configuration parameters:
- * <ul>
- * <li>Maximum failover attempts for a single job (see {@link #setMaximumFailoverAttempts(int)}).</li>
- * </ul>
- * Here is a Java example on how to configure grid with {@code GridJobStealingFailoverSpi}.
- * <pre name="code" class="java">
- * GridJobStealingFailoverSpi spi = new GridJobStealingFailoverSpi();
- *
- * // Override maximum failover attempts.
- * spi.setMaximumFailoverAttempts(5);
- *
- * GridConfiguration cfg = new GridConfiguration();
- *
- * // Override default failover SPI.
- * cfg.setFailoverSpiSpi(spi);
- *
- * // Starts grid.
- * G.start(cfg);
- </pre>
- * Here is an example of how to configure {@code GridJobStealingFailoverSpi} from Spring XML configuration file.
- * <pre name="code" class="xml">
- * &lt;property name="failoverSpi"&gt;
- *     &lt;bean class="org.gridgain.grid.spi.failover.jobstealing.GridJobStealingFailoverSpi"&gt;
- *         &lt;property name="maximumFailoverAttempts" value="5"/&gt;
- *     &lt;/bean&gt;
- * &lt;/property&gt;
- * </pre>
- * <p>
- * <img src="http://www.gridgain.com/images/spring-small.png">
- * <br>
- * For information about Spring framework visit <a href="http://www.springframework.org/">www.springframework.org</a>
- * @see GridFailoverSpi
- */
-@IgniteSpiMultipleInstancesSupport(true)
-@IgniteSpiConsistencyChecked(optional = true)
-public class GridJobStealingFailoverSpi extends IgniteSpiAdapter implements GridFailoverSpi,
-    GridJobStealingFailoverSpiMBean {
-    /** Maximum number of attempts to execute a failed job on another node (default is {@code 5}). */
-    public static final int DFLT_MAX_FAILOVER_ATTEMPTS = 5;
-
-    /**
-     * Name of job context attribute containing all nodes a job failed on. Note
-     * that this list does not include nodes that a job was stolen from.
-     *
-     * @see org.apache.ignite.compute.ComputeJobContext
-     */
-    static final String FAILED_NODE_LIST_ATTR = "gg:failover:failednodelist";
-
-    /**
-     * Name of job context attribute containing current failover attempt count.
-     * This count is incremented every time the same job gets failed over to
-     * another node for execution if it was not successfully stolen.
-     *
-     * @see org.apache.ignite.compute.ComputeJobContext
-     */
-    static final String FAILOVER_ATTEMPT_COUNT_ATTR = "gg:failover:attemptcount";
-
-    /** Maximum failover attempts job context attribute name. */
-    private static final String MAX_FAILOVER_ATTEMPT_ATTR = "gg:failover:maxattempts";
-
-    /** Injected grid logger. */
-    @IgniteLoggerResource
-    private IgniteLogger log;
-
-    /** Maximum number of attempts to execute a failed job on another node. */
-    private int maxFailoverAttempts = DFLT_MAX_FAILOVER_ATTEMPTS;
-
-    /** Number of jobs that were failed over. */
-    private int totalFailedOverJobs;
-
-    /** Number of jobs that were stolen. */
-    private int totalStolenJobs;
-
-    /** {@inheritDoc} */
-    @Override public int getMaximumFailoverAttempts() {
-        return maxFailoverAttempts;
-    }
-
-    /**
-     * Sets maximum number of attempts to execute a failed job on another node.
-     * If job gets stolen and thief node exists then it is not considered as
-     * failed job.
-     * If not specified, {@link #DFLT_MAX_FAILOVER_ATTEMPTS} value will be used.
-     * <p>
-     * Note this value must be identical for all grid nodes in the grid.
-     *
-     * @param maxFailoverAttempts Maximum number of attempts to execute a failed
-     *      job on another node.
-     */
-    @IgniteSpiConfiguration(optional = true)
-    public void setMaximumFailoverAttempts(int maxFailoverAttempts) {
-        this.maxFailoverAttempts = maxFailoverAttempts;
-    }
-
-    /** {@inheritDoc} */
-    @Override public int getTotalFailedOverJobsCount() {
-        return totalFailedOverJobs;
-    }
-
-    /** {@inheritDoc} */
-    @Override public int getTotalStolenJobsCount() {
-        return totalStolenJobs;
-    }
-
-    /** {@inheritDoc} */
-    @Override public Map<String, Object> getNodeAttributes() throws IgniteSpiException {
-        return F.<String, Object>asMap(createSpiAttributeName(MAX_FAILOVER_ATTEMPT_ATTR), maxFailoverAttempts);
-    }
-
-    /** {@inheritDoc} */
-    @Override public void spiStart(String gridName) throws IgniteSpiException {
-        // Start SPI start stopwatch.
-        startStopwatch();
-
-        assertParameter(maxFailoverAttempts >= 0, "maximumFailoverAttempts >= 0");
-
-        if (log.isDebugEnabled())
-            log.debug(configInfo("maxFailoverAttempts", maxFailoverAttempts));
-
-        registerMBean(gridName, this, GridJobStealingFailoverSpiMBean.class);
-
-        // Ack ok start.
-        if (log.isDebugEnabled())
-            log.debug(startInfo());
-    }
-
-    /** {@inheritDoc} */
-    @Override public void spiStop() throws IgniteSpiException {
-        unregisterMBean();
-
-        // Ack ok stop.
-        if (log.isDebugEnabled())
-            log.debug(stopInfo());
-    }
-
-    /** {@inheritDoc} */
-    @SuppressWarnings("unchecked")
-    @Override public ClusterNode failover(GridFailoverContext ctx, List<ClusterNode> top) {
-        assert ctx != null;
-        assert top != null;
-
-        if (top.isEmpty()) {
-            U.warn(log, "Received empty subgrid and is forced to fail.");
-
-            // Nowhere to failover to.
-            return null;
-        }
-
-        Integer failoverCnt = ctx.getJobResult().getJobContext().getAttribute(FAILOVER_ATTEMPT_COUNT_ATTR);
-
-        if (failoverCnt == null)
-            failoverCnt = 0;
-
-        if (failoverCnt > maxFailoverAttempts) {
-            U.error(log, "Failover count exceeded maximum failover attempts parameter [failedJob=" +
-                ctx.getJobResult().getJob() + ", maxFailoverAttempts=" + maxFailoverAttempts + ']');
-
-            return null;
-        }
-
-        if (failoverCnt == maxFailoverAttempts) {
-            U.warn(log, "Job failover failed because number of maximum failover attempts is exceeded [failedJob=" +
-                ctx.getJobResult().getJob() + ", maxFailoverAttempts=" + maxFailoverAttempts + ']');
-
-            return null;
-        }
-
-        try {
-            ClusterNode thief = null;
-            boolean isNodeFailed = false;
-
-            UUID thiefId = ctx.getJobResult().getJobContext().getAttribute(THIEF_NODE_ATTR);
-
-            if (thiefId != null) {
-                // Clear attribute.
-                ctx.getJobResult().getJobContext().setAttribute(THIEF_NODE_ATTR, null);
-
-                thief = getSpiContext().node(thiefId);
-
-                if (thief != null) {
-                    // If sender != receiver.
-                    if (thief.equals(ctx.getJobResult().getNode())) {
-                        U.error(log, "Job stealer node is equal to job node (will fail-over using " +
-                            "load-balancing): " + thief.id());
-
-                        isNodeFailed = true;
-
-                        thief = null;
-                    }
-                    else if (!top.contains(thief)) {
-                        U.warn(log, "Thief node is not part of task topology  (will fail-over using load-balancing) " +
-                            "[thief=" + thiefId + ", topSize=" + top.size() + ']');
-
-                        thief = null;
-                    }
-
-                    if (log.isDebugEnabled())
-                        log.debug("Failing-over stolen job [from=" + ctx.getJobResult().getNode() + ", to=" +
-                            thief + ']');
-                }
-                else {
-                    isNodeFailed = true;
-
-                    U.warn(log, "Thief node left grid (will fail-over using load balancing): " + thiefId);
-                }
-            }
-            else
-                isNodeFailed = true;
-
-            // If job was not stolen or stolen node is not part of topology,
-            // then failover the regular way.
-            if (thief == null) {
-                Collection<UUID> failedNodes = ctx.getJobResult().getJobContext().getAttribute(FAILED_NODE_LIST_ATTR);
-
-                if (failedNodes == null)
-                    failedNodes = U.newHashSet(1);
-
-                if (isNodeFailed)
-                    failedNodes.add(ctx.getJobResult().getNode().id());
-
-                // Set updated failed node set into job context.
-                ctx.getJobResult().getJobContext().setAttribute(FAILED_NODE_LIST_ATTR, failedNodes);
-
-                // Copy.
-                List<ClusterNode> newTop = new ArrayList<>(top.size());
-
-                for (ClusterNode n : top) {
-                    // Add non-failed nodes to topology.
-                    if (!failedNodes.contains(n.id()))
-                        newTop.add(n);
-                }
-
-                if (newTop.isEmpty()) {
-                    U.warn(log, "Received topology with only nodes that job had failed on (forced to fail) " +
-                        "[failedNodes=" + failedNodes + ']');
-
-                    // Nowhere to failover to.
-                    return null;
-                }
-
-                thief = ctx.getBalancedNode(newTop);
-
-                if (thief == null)
-                    U.warn(log, "Load balancer returned null node for topology: " + newTop);
-            }
-
-            if (isNodeFailed)
-                // This is a failover, not stealing.
-                failoverCnt++;
-
-            // Even if it was stealing and thief node left grid we assume
-            // that it is failover because of the fail.
-            ctx.getJobResult().getJobContext().setAttribute(FAILOVER_ATTEMPT_COUNT_ATTR, failoverCnt);
-
-            if (thief != null) {
-                totalFailedOverJobs++;
-
-                if (isNodeFailed) {
-                    U.warn(log, "Failed over job to a new node [newNode=" + thief.id() +
-                        ", oldNode=" + ctx.getJobResult().getNode().id() +
-                        ", sesId=" + ctx.getTaskSession().getId() +
-                        ", job=" + ctx.getJobResult().getJob() +
-                        ", jobCtx=" + ctx.getJobResult().getJobContext() +
-                        ", task=" + ctx.getTaskSession().getTaskName() + ']');
-                }
-                else {
-                    totalStolenJobs++;
-                    if (log.isInfoEnabled())
-                        log.info("Stealing job to a new node [newNode=" + thief.id() +
-                            ", oldNode=" + ctx.getJobResult().getNode().id() +
-                            ", sesId=" + ctx.getTaskSession().getId() +
-                            ", job=" + ctx.getJobResult().getJob() +
-                            ", jobCtx=" + ctx.getJobResult().getJobContext() +
-                            ", task=" + ctx.getTaskSession().getTaskName() + ']');
-                }
-            }
-
-            return thief;
-        }
-        catch (GridException e) {
-            U.error(log, "Failed to get next balanced node for failover: " + ctx, e);
-
-            return null;
-        }
-    }
-
-    /** {@inheritDoc} */
-    @Override protected List<String> getConsistentAttributeNames() {
-        return Collections.singletonList(createSpiAttributeName(MAX_FAILOVER_ATTEMPT_ATTR));
-    }
-
-    /** {@inheritDoc} */
-    @Override public String toString() {
-        return S.toString(GridJobStealingFailoverSpi.class, this);
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/50d1554c/modules/core/src/main/java/org/gridgain/grid/spi/failover/jobstealing/GridJobStealingFailoverSpiMBean.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/spi/failover/jobstealing/GridJobStealingFailoverSpiMBean.java b/modules/core/src/main/java/org/gridgain/grid/spi/failover/jobstealing/GridJobStealingFailoverSpiMBean.java
deleted file mode 100644
index 3163d30..0000000
--- a/modules/core/src/main/java/org/gridgain/grid/spi/failover/jobstealing/GridJobStealingFailoverSpiMBean.java
+++ /dev/null
@@ -1,46 +0,0 @@
-/* @java.file.header */
-
-/*  _________        _____ __________________        _____
- *  __  ____/___________(_)______  /__  ____/______ ____(_)_______
- *  _  / __  __  ___/__  / _  __  / _  / __  _  __ `/__  / __  __ \
- *  / /_/ /  _  /    _  /  / /_/ /  / /_/ /  / /_/ / _  /  _  / / /
- *  \____/   /_/     /_/   \_,__/   \____/   \__,_/  /_/   /_/ /_/
- */
-
-package org.gridgain.grid.spi.failover.jobstealing;
-
-import org.apache.ignite.mbean.*;
-import org.apache.ignite.spi.*;
-
-/**
- * Management bean for {@link GridJobStealingFailoverSpi}.
- */
-@IgniteMBeanDescription("MBean that provides access to job stealing failover SPI configuration.")
-public interface GridJobStealingFailoverSpiMBean extends IgniteSpiManagementMBean {
-    /**
-     * Gets maximum number of attempts to execute a failed job on another node.
-     * If job gets stolen and thief node exists then it is not considered as
-     * failed job.
-     * If not specified, {@link GridJobStealingFailoverSpi#DFLT_MAX_FAILOVER_ATTEMPTS} value will be used.
-     *
-     * @return Maximum number of attempts to execute a failed job on another node.
-     */
-    @IgniteMBeanDescription("Maximum number of attempts to execute a failed job on another node.")
-    public int getMaximumFailoverAttempts();
-
-    /**
-     * Get total number of jobs that were failed over including stolen ones.
-     *
-     * @return Total number of failed over jobs.
-     */
-    @IgniteMBeanDescription("Total number of jobs that were failed over including stolen ones.")
-    public int getTotalFailedOverJobsCount();
-
-    /**
-     * Get total number of jobs that were stolen.
-     *
-     * @return Total number of stolen jobs.
-     */
-    @IgniteMBeanDescription("Total number of jobs that were stolen.")
-    public int getTotalStolenJobsCount();
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/50d1554c/modules/core/src/main/java/org/gridgain/grid/spi/failover/jobstealing/JobStealingFailoverSpi.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/spi/failover/jobstealing/JobStealingFailoverSpi.java b/modules/core/src/main/java/org/gridgain/grid/spi/failover/jobstealing/JobStealingFailoverSpi.java
new file mode 100644
index 0000000..e7e7445
--- /dev/null
+++ b/modules/core/src/main/java/org/gridgain/grid/spi/failover/jobstealing/JobStealingFailoverSpi.java
@@ -0,0 +1,343 @@
+/* @java.file.header */
+
+/*  _________        _____ __________________        _____
+ *  __  ____/___________(_)______  /__  ____/______ ____(_)_______
+ *  _  / __  __  ___/__  / _  __  / _  / __  _  __ `/__  / __  __ \
+ *  / /_/ /  _  /    _  /  / /_/ /  / /_/ /  / /_/ / _  /  _  / / /
+ *  \____/   /_/     /_/   \_,__/   \____/   \__,_/  /_/   /_/ /_/
+ */
+
+package org.gridgain.grid.spi.failover.jobstealing;
+
+import org.apache.ignite.*;
+import org.apache.ignite.cluster.*;
+import org.apache.ignite.resources.*;
+import org.apache.ignite.spi.*;
+import org.gridgain.grid.*;
+import org.gridgain.grid.spi.failover.*;
+import org.gridgain.grid.util.typedef.*;
+import org.gridgain.grid.util.typedef.internal.*;
+
+import java.util.*;
+
+import static org.apache.ignite.spi.collision.jobstealing.JobStealingCollisionSpi.*;
+
+/**
+ * Job stealing failover SPI needs to always be used in conjunction with
+ * {@link org.apache.ignite.spi.collision.jobstealing.JobStealingCollisionSpi} SPI. When {@link org.apache.ignite.spi.collision.jobstealing.JobStealingCollisionSpi}
+ * receives a <b>steal</b> request and rejects jobs so they can be routed to the
+ * appropriate node, it is the responsibility of this {@code GridJobStealingFailoverSpi}
+ * SPI to make sure that the job is indeed re-routed to the node that has sent the initial
+ * request to <b>steal</b> it.
+ * <p>
+ * {@code GridJobStealingFailoverSpi} knows where to route a job based on the
+ * {@link org.apache.ignite.spi.collision.jobstealing.JobStealingCollisionSpi#THIEF_NODE_ATTR} job context attribute (see {@link org.apache.ignite.compute.ComputeJobContext}).
+ * Prior to rejecting a job,  {@link org.apache.ignite.spi.collision.jobstealing.JobStealingCollisionSpi} will populate this
+ * attribute with the ID of the node that wants to <b>steal</b> this job.
+ * Then {@code GridJobStealingFailoverSpi} will read the value of this attribute and
+ * route the job to the node specified.
+ * <p>
+ * If failure is caused by a node crash, and not by <b>steal</b> request, then this
+ * SPI behaves identically to {@link org.gridgain.grid.spi.failover.always.AlwaysFailoverSpi}, and tries to find the
+ * next balanced node to fail-over a job to.
+ * <p>
+ * <h1 class="header">Configuration</h1>
+ * <h2 class="header">Mandatory</h2>
+ * This SPI has no mandatory configuration parameters.
+ * <h2 class="header">Optional</h2>
+ * This SPI has following optional configuration parameters:
+ * <ul>
+ * <li>Maximum failover attempts for a single job (see {@link #setMaximumFailoverAttempts(int)}).</li>
+ * </ul>
+ * Here is a Java example on how to configure grid with {@code GridJobStealingFailoverSpi}.
+ * <pre name="code" class="java">
+ * GridJobStealingFailoverSpi spi = new GridJobStealingFailoverSpi();
+ *
+ * // Override maximum failover attempts.
+ * spi.setMaximumFailoverAttempts(5);
+ *
+ * GridConfiguration cfg = new GridConfiguration();
+ *
+ * // Override default failover SPI.
+ * cfg.setFailoverSpiSpi(spi);
+ *
+ * // Starts grid.
+ * G.start(cfg);
+ </pre>
+ * Here is an example of how to configure {@code GridJobStealingFailoverSpi} from Spring XML configuration file.
+ * <pre name="code" class="xml">
+ * &lt;property name="failoverSpi"&gt;
+ *     &lt;bean class="org.gridgain.grid.spi.failover.jobstealing.GridJobStealingFailoverSpi"&gt;
+ *         &lt;property name="maximumFailoverAttempts" value="5"/&gt;
+ *     &lt;/bean&gt;
+ * &lt;/property&gt;
+ * </pre>
+ * <p>
+ * <img src="http://www.gridgain.com/images/spring-small.png">
+ * <br>
+ * For information about Spring framework visit <a href="http://www.springframework.org/">www.springframework.org</a>
+ * @see org.gridgain.grid.spi.failover.FailoverSpi
+ */
+@IgniteSpiMultipleInstancesSupport(true)
+@IgniteSpiConsistencyChecked(optional = true)
+public class JobStealingFailoverSpi extends IgniteSpiAdapter implements FailoverSpi,
+    JobStealingFailoverSpiMBean {
+    /** Maximum number of attempts to execute a failed job on another node (default is {@code 5}). */
+    public static final int DFLT_MAX_FAILOVER_ATTEMPTS = 5;
+
+    /**
+     * Name of job context attribute containing all nodes a job failed on. Note
+     * that this list does not include nodes that a job was stolen from.
+     *
+     * @see org.apache.ignite.compute.ComputeJobContext
+     */
+    static final String FAILED_NODE_LIST_ATTR = "gg:failover:failednodelist";
+
+    /**
+     * Name of job context attribute containing current failover attempt count.
+     * This count is incremented every time the same job gets failed over to
+     * another node for execution if it was not successfully stolen.
+     *
+     * @see org.apache.ignite.compute.ComputeJobContext
+     */
+    static final String FAILOVER_ATTEMPT_COUNT_ATTR = "gg:failover:attemptcount";
+
+    /** Maximum failover attempts job context attribute name. */
+    private static final String MAX_FAILOVER_ATTEMPT_ATTR = "gg:failover:maxattempts";
+
+    /** Injected grid logger. */
+    @IgniteLoggerResource
+    private IgniteLogger log;
+
+    /** Maximum number of attempts to execute a failed job on another node. */
+    private int maxFailoverAttempts = DFLT_MAX_FAILOVER_ATTEMPTS;
+
+    /** Number of jobs that were failed over. */
+    private int totalFailedOverJobs;
+
+    /** Number of jobs that were stolen. */
+    private int totalStolenJobs;
+
+    /** {@inheritDoc} */
+    @Override public int getMaximumFailoverAttempts() {
+        return maxFailoverAttempts;
+    }
+
+    /**
+     * Sets maximum number of attempts to execute a failed job on another node.
+     * If job gets stolen and thief node exists then it is not considered as
+     * failed job.
+     * If not specified, {@link #DFLT_MAX_FAILOVER_ATTEMPTS} value will be used.
+     * <p>
+     * Note this value must be identical for all grid nodes in the grid.
+     *
+     * @param maxFailoverAttempts Maximum number of attempts to execute a failed
+     *      job on another node.
+     */
+    @IgniteSpiConfiguration(optional = true)
+    public void setMaximumFailoverAttempts(int maxFailoverAttempts) {
+        this.maxFailoverAttempts = maxFailoverAttempts;
+    }
+
+    /** {@inheritDoc} */
+    @Override public int getTotalFailedOverJobsCount() {
+        return totalFailedOverJobs;
+    }
+
+    /** {@inheritDoc} */
+    @Override public int getTotalStolenJobsCount() {
+        return totalStolenJobs;
+    }
+
+    /** {@inheritDoc} */
+    @Override public Map<String, Object> getNodeAttributes() throws IgniteSpiException {
+        return F.<String, Object>asMap(createSpiAttributeName(MAX_FAILOVER_ATTEMPT_ATTR), maxFailoverAttempts);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void spiStart(String gridName) throws IgniteSpiException {
+        // Start SPI start stopwatch.
+        startStopwatch();
+
+        assertParameter(maxFailoverAttempts >= 0, "maximumFailoverAttempts >= 0");
+
+        if (log.isDebugEnabled())
+            log.debug(configInfo("maxFailoverAttempts", maxFailoverAttempts));
+
+        registerMBean(gridName, this, JobStealingFailoverSpiMBean.class);
+
+        // Ack ok start.
+        if (log.isDebugEnabled())
+            log.debug(startInfo());
+    }
+
+    /** {@inheritDoc} */
+    @Override public void spiStop() throws IgniteSpiException {
+        unregisterMBean();
+
+        // Ack ok stop.
+        if (log.isDebugEnabled())
+            log.debug(stopInfo());
+    }
+
+    /** {@inheritDoc} */
+    @SuppressWarnings("unchecked")
+    @Override public ClusterNode failover(FailoverContext ctx, List<ClusterNode> top) {
+        assert ctx != null;
+        assert top != null;
+
+        if (top.isEmpty()) {
+            U.warn(log, "Received empty subgrid and is forced to fail.");
+
+            // Nowhere to failover to.
+            return null;
+        }
+
+        Integer failoverCnt = ctx.getJobResult().getJobContext().getAttribute(FAILOVER_ATTEMPT_COUNT_ATTR);
+
+        if (failoverCnt == null)
+            failoverCnt = 0;
+
+        if (failoverCnt > maxFailoverAttempts) {
+            U.error(log, "Failover count exceeded maximum failover attempts parameter [failedJob=" +
+                ctx.getJobResult().getJob() + ", maxFailoverAttempts=" + maxFailoverAttempts + ']');
+
+            return null;
+        }
+
+        if (failoverCnt == maxFailoverAttempts) {
+            U.warn(log, "Job failover failed because number of maximum failover attempts is exceeded [failedJob=" +
+                ctx.getJobResult().getJob() + ", maxFailoverAttempts=" + maxFailoverAttempts + ']');
+
+            return null;
+        }
+
+        try {
+            ClusterNode thief = null;
+            boolean isNodeFailed = false;
+
+            UUID thiefId = ctx.getJobResult().getJobContext().getAttribute(THIEF_NODE_ATTR);
+
+            if (thiefId != null) {
+                // Clear attribute.
+                ctx.getJobResult().getJobContext().setAttribute(THIEF_NODE_ATTR, null);
+
+                thief = getSpiContext().node(thiefId);
+
+                if (thief != null) {
+                    // If sender != receiver.
+                    if (thief.equals(ctx.getJobResult().getNode())) {
+                        U.error(log, "Job stealer node is equal to job node (will fail-over using " +
+                            "load-balancing): " + thief.id());
+
+                        isNodeFailed = true;
+
+                        thief = null;
+                    }
+                    else if (!top.contains(thief)) {
+                        U.warn(log, "Thief node is not part of task topology  (will fail-over using load-balancing) " +
+                            "[thief=" + thiefId + ", topSize=" + top.size() + ']');
+
+                        thief = null;
+                    }
+
+                    if (log.isDebugEnabled())
+                        log.debug("Failing-over stolen job [from=" + ctx.getJobResult().getNode() + ", to=" +
+                            thief + ']');
+                }
+                else {
+                    isNodeFailed = true;
+
+                    U.warn(log, "Thief node left grid (will fail-over using load balancing): " + thiefId);
+                }
+            }
+            else
+                isNodeFailed = true;
+
+            // If job was not stolen or stolen node is not part of topology,
+            // then failover the regular way.
+            if (thief == null) {
+                Collection<UUID> failedNodes = ctx.getJobResult().getJobContext().getAttribute(FAILED_NODE_LIST_ATTR);
+
+                if (failedNodes == null)
+                    failedNodes = U.newHashSet(1);
+
+                if (isNodeFailed)
+                    failedNodes.add(ctx.getJobResult().getNode().id());
+
+                // Set updated failed node set into job context.
+                ctx.getJobResult().getJobContext().setAttribute(FAILED_NODE_LIST_ATTR, failedNodes);
+
+                // Copy.
+                List<ClusterNode> newTop = new ArrayList<>(top.size());
+
+                for (ClusterNode n : top) {
+                    // Add non-failed nodes to topology.
+                    if (!failedNodes.contains(n.id()))
+                        newTop.add(n);
+                }
+
+                if (newTop.isEmpty()) {
+                    U.warn(log, "Received topology with only nodes that job had failed on (forced to fail) " +
+                        "[failedNodes=" + failedNodes + ']');
+
+                    // Nowhere to failover to.
+                    return null;
+                }
+
+                thief = ctx.getBalancedNode(newTop);
+
+                if (thief == null)
+                    U.warn(log, "Load balancer returned null node for topology: " + newTop);
+            }
+
+            if (isNodeFailed)
+                // This is a failover, not stealing.
+                failoverCnt++;
+
+            // Even if it was stealing and thief node left grid we assume
+            // that it is failover because of the fail.
+            ctx.getJobResult().getJobContext().setAttribute(FAILOVER_ATTEMPT_COUNT_ATTR, failoverCnt);
+
+            if (thief != null) {
+                totalFailedOverJobs++;
+
+                if (isNodeFailed) {
+                    U.warn(log, "Failed over job to a new node [newNode=" + thief.id() +
+                        ", oldNode=" + ctx.getJobResult().getNode().id() +
+                        ", sesId=" + ctx.getTaskSession().getId() +
+                        ", job=" + ctx.getJobResult().getJob() +
+                        ", jobCtx=" + ctx.getJobResult().getJobContext() +
+                        ", task=" + ctx.getTaskSession().getTaskName() + ']');
+                }
+                else {
+                    totalStolenJobs++;
+                    if (log.isInfoEnabled())
+                        log.info("Stealing job to a new node [newNode=" + thief.id() +
+                            ", oldNode=" + ctx.getJobResult().getNode().id() +
+                            ", sesId=" + ctx.getTaskSession().getId() +
+                            ", job=" + ctx.getJobResult().getJob() +
+                            ", jobCtx=" + ctx.getJobResult().getJobContext() +
+                            ", task=" + ctx.getTaskSession().getTaskName() + ']');
+                }
+            }
+
+            return thief;
+        }
+        catch (GridException e) {
+            U.error(log, "Failed to get next balanced node for failover: " + ctx, e);
+
+            return null;
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override protected List<String> getConsistentAttributeNames() {
+        return Collections.singletonList(createSpiAttributeName(MAX_FAILOVER_ATTEMPT_ATTR));
+    }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return S.toString(JobStealingFailoverSpi.class, this);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/50d1554c/modules/core/src/main/java/org/gridgain/grid/spi/failover/jobstealing/JobStealingFailoverSpiMBean.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/spi/failover/jobstealing/JobStealingFailoverSpiMBean.java b/modules/core/src/main/java/org/gridgain/grid/spi/failover/jobstealing/JobStealingFailoverSpiMBean.java
new file mode 100644
index 0000000..38d9534
--- /dev/null
+++ b/modules/core/src/main/java/org/gridgain/grid/spi/failover/jobstealing/JobStealingFailoverSpiMBean.java
@@ -0,0 +1,46 @@
+/* @java.file.header */
+
+/*  _________        _____ __________________        _____
+ *  __  ____/___________(_)______  /__  ____/______ ____(_)_______
+ *  _  / __  __  ___/__  / _  __  / _  / __  _  __ `/__  / __  __ \
+ *  / /_/ /  _  /    _  /  / /_/ /  / /_/ /  / /_/ / _  /  _  / / /
+ *  \____/   /_/     /_/   \_,__/   \____/   \__,_/  /_/   /_/ /_/
+ */
+
+package org.gridgain.grid.spi.failover.jobstealing;
+
+import org.apache.ignite.mbean.*;
+import org.apache.ignite.spi.*;
+
+/**
+ * Management bean for {@link JobStealingFailoverSpi}.
+ */
+@IgniteMBeanDescription("MBean that provides access to job stealing failover SPI configuration.")
+public interface JobStealingFailoverSpiMBean extends IgniteSpiManagementMBean {
+    /**
+     * Gets maximum number of attempts to execute a failed job on another node.
+     * If job gets stolen and thief node exists then it is not considered as
+     * failed job.
+     * If not specified, {@link JobStealingFailoverSpi#DFLT_MAX_FAILOVER_ATTEMPTS} value will be used.
+     *
+     * @return Maximum number of attempts to execute a failed job on another node.
+     */
+    @IgniteMBeanDescription("Maximum number of attempts to execute a failed job on another node.")
+    public int getMaximumFailoverAttempts();
+
+    /**
+     * Get total number of jobs that were failed over including stolen ones.
+     *
+     * @return Total number of failed over jobs.
+     */
+    @IgniteMBeanDescription("Total number of jobs that were failed over including stolen ones.")
+    public int getTotalFailedOverJobsCount();
+
+    /**
+     * Get total number of jobs that were stolen.
+     *
+     * @return Total number of stolen jobs.
+     */
+    @IgniteMBeanDescription("Total number of jobs that were stolen.")
+    public int getTotalStolenJobsCount();
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/50d1554c/modules/core/src/main/java/org/gridgain/grid/spi/failover/never/GridNeverFailoverSpi.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/spi/failover/never/GridNeverFailoverSpi.java b/modules/core/src/main/java/org/gridgain/grid/spi/failover/never/GridNeverFailoverSpi.java
deleted file mode 100644
index f596bf3..0000000
--- a/modules/core/src/main/java/org/gridgain/grid/spi/failover/never/GridNeverFailoverSpi.java
+++ /dev/null
@@ -1,94 +0,0 @@
-/* @java.file.header */
-
-/*  _________        _____ __________________        _____
- *  __  ____/___________(_)______  /__  ____/______ ____(_)_______
- *  _  / __  __  ___/__  / _  __  / _  / __  _  __ `/__  / __  __ \
- *  / /_/ /  _  /    _  /  / /_/ /  / /_/ /  / /_/ / _  /  _  / / /
- *  \____/   /_/     /_/   \_,__/   \____/   \__,_/  /_/   /_/ /_/
- */
-
-package org.gridgain.grid.spi.failover.never;
-
-import org.apache.ignite.*;
-import org.apache.ignite.cluster.*;
-import org.apache.ignite.resources.*;
-import org.apache.ignite.spi.*;
-import org.gridgain.grid.spi.failover.*;
-import org.gridgain.grid.util.typedef.internal.*;
-import java.util.*;
-
-/**
- * This class provides failover SPI implementation that never fails over. This implementation
- * never fails over a failed job by always returning {@code null} out of
- * {@link GridFailoverSpi#failover(GridFailoverContext, List)} method.
- * <h1 class="header">Configuration</h1>
- * <h2 class="header">Mandatory</h2>
- * This SPI has no mandatory configuration parameters.
- * <h2 class="header">Optional</h2>
- * This SPI has no optional configuration parameters.
- * <p>
- * Here is a Java example on how to configure grid with {@code GridNeverFailoverSpi}:
- * <pre name="code" class="java">
- * GridNeverFailoverSpi spi = new GridNeverFailoverSpi();
- *
- * GridConfiguration cfg = new GridConfiguration();
- *
- * // Override default failover SPI.
- * cfg.setFailoverSpiSpi(spi);
- *
- * // Starts grid.
- * G.start(cfg);
- * </pre>
- * Here is an example on how to configure grid with {@code GridNeverFailoverSpi} from Spring XML configuration file:
- * <pre name="code" class="xml">
- * &lt;property name="failoverSpi"&gt;
- *     &lt;bean class="org.gridgain.grid.spi.failover.never.GridNeverFailoverSpi"/&gt;
- * &lt;/property&gt;
- * </pre>
- * <p>
- * <img src="http://www.gridgain.com/images/spring-small.png">
- * <br>
- * For information about Spring framework visit <a href="http://www.springframework.org/">www.springframework.org</a>
- * @see GridFailoverSpi
- */
-@IgniteSpiMultipleInstancesSupport(true)
-public class GridNeverFailoverSpi extends IgniteSpiAdapter implements GridFailoverSpi, GridNeverFailoverSpiMBean {
-    /** Injected grid logger. */
-    @IgniteLoggerResource
-    private IgniteLogger log;
-
-    /** {@inheritDoc} */
-    @Override public void spiStart(String gridName) throws IgniteSpiException {
-        // Start SPI start stopwatch.
-        startStopwatch();
-
-        registerMBean(gridName, this, GridNeverFailoverSpiMBean.class);
-
-        // Ack ok start.
-        if (log.isDebugEnabled())
-            log.debug(startInfo());
-    }
-
-    /** {@inheritDoc} */
-    @Override public void spiStop() throws IgniteSpiException {
-        unregisterMBean();
-
-        // Ack ok stop.
-        if (log.isDebugEnabled())
-            log.debug(stopInfo());
-    }
-
-    /** {@inheritDoc} */
-    @Override public ClusterNode failover(GridFailoverContext ctx, List<ClusterNode> top) {
-        U.warn(log, "Returning 'null' node for failed job (failover will not happen) [job=" +
-            ctx.getJobResult().getJob() + ", task=" +  ctx.getTaskSession().getTaskName() +
-            ", sessionId=" + ctx.getTaskSession().getId() + ']');
-
-        return null;
-    }
-
-    /** {@inheritDoc} */
-    @Override public String toString() {
-        return S.toString(GridNeverFailoverSpi.class, this);
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/50d1554c/modules/core/src/main/java/org/gridgain/grid/spi/failover/never/GridNeverFailoverSpiMBean.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/spi/failover/never/GridNeverFailoverSpiMBean.java b/modules/core/src/main/java/org/gridgain/grid/spi/failover/never/GridNeverFailoverSpiMBean.java
deleted file mode 100644
index 5b89564..0000000
--- a/modules/core/src/main/java/org/gridgain/grid/spi/failover/never/GridNeverFailoverSpiMBean.java
+++ /dev/null
@@ -1,21 +0,0 @@
-/* @java.file.header */
-
-/*  _________        _____ __________________        _____
- *  __  ____/___________(_)______  /__  ____/______ ____(_)_______
- *  _  / __  __  ___/__  / _  __  / _  / __  _  __ `/__  / __  __ \
- *  / /_/ /  _  /    _  /  / /_/ /  / /_/ /  / /_/ / _  /  _  / / /
- *  \____/   /_/     /_/   \_,__/   \____/   \__,_/  /_/   /_/ /_/
- */
-
-package org.gridgain.grid.spi.failover.never;
-
-import org.apache.ignite.mbean.*;
-import org.apache.ignite.spi.*;
-
-/**
- * Management bean for {@link GridNeverFailoverSpi}.
- */
-@IgniteMBeanDescription("MBean that provides access to never failover SPI configuration.")
-public interface GridNeverFailoverSpiMBean extends IgniteSpiManagementMBean {
-    // No-op.
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/50d1554c/modules/core/src/main/java/org/gridgain/grid/spi/failover/never/NeverFailoverSpi.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/spi/failover/never/NeverFailoverSpi.java b/modules/core/src/main/java/org/gridgain/grid/spi/failover/never/NeverFailoverSpi.java
new file mode 100644
index 0000000..945b2a5
--- /dev/null
+++ b/modules/core/src/main/java/org/gridgain/grid/spi/failover/never/NeverFailoverSpi.java
@@ -0,0 +1,94 @@
+/* @java.file.header */
+
+/*  _________        _____ __________________        _____
+ *  __  ____/___________(_)______  /__  ____/______ ____(_)_______
+ *  _  / __  __  ___/__  / _  __  / _  / __  _  __ `/__  / __  __ \
+ *  / /_/ /  _  /    _  /  / /_/ /  / /_/ /  / /_/ / _  /  _  / / /
+ *  \____/   /_/     /_/   \_,__/   \____/   \__,_/  /_/   /_/ /_/
+ */
+
+package org.gridgain.grid.spi.failover.never;
+
+import org.apache.ignite.*;
+import org.apache.ignite.cluster.*;
+import org.apache.ignite.resources.*;
+import org.apache.ignite.spi.*;
+import org.gridgain.grid.spi.failover.*;
+import org.gridgain.grid.util.typedef.internal.*;
+import java.util.*;
+
+/**
+ * This class provides failover SPI implementation that never fails over. This implementation
+ * never fails over a failed job by always returning {@code null} out of
+ * {@link org.gridgain.grid.spi.failover.FailoverSpi#failover(org.gridgain.grid.spi.failover.FailoverContext, List)} method.
+ * <h1 class="header">Configuration</h1>
+ * <h2 class="header">Mandatory</h2>
+ * This SPI has no mandatory configuration parameters.
+ * <h2 class="header">Optional</h2>
+ * This SPI has no optional configuration parameters.
+ * <p>
+ * Here is a Java example on how to configure grid with {@code GridNeverFailoverSpi}:
+ * <pre name="code" class="java">
+ * GridNeverFailoverSpi spi = new GridNeverFailoverSpi();
+ *
+ * GridConfiguration cfg = new GridConfiguration();
+ *
+ * // Override default failover SPI.
+ * cfg.setFailoverSpiSpi(spi);
+ *
+ * // Starts grid.
+ * G.start(cfg);
+ * </pre>
+ * Here is an example on how to configure grid with {@code GridNeverFailoverSpi} from Spring XML configuration file:
+ * <pre name="code" class="xml">
+ * &lt;property name="failoverSpi"&gt;
+ *     &lt;bean class="org.gridgain.grid.spi.failover.never.GridNeverFailoverSpi"/&gt;
+ * &lt;/property&gt;
+ * </pre>
+ * <p>
+ * <img src="http://www.gridgain.com/images/spring-small.png">
+ * <br>
+ * For information about Spring framework visit <a href="http://www.springframework.org/">www.springframework.org</a>
+ * @see org.gridgain.grid.spi.failover.FailoverSpi
+ */
+@IgniteSpiMultipleInstancesSupport(true)
+public class NeverFailoverSpi extends IgniteSpiAdapter implements FailoverSpi, NeverFailoverSpiMBean {
+    /** Injected grid logger. */
+    @IgniteLoggerResource
+    private IgniteLogger log;
+
+    /** {@inheritDoc} */
+    @Override public void spiStart(String gridName) throws IgniteSpiException {
+        // Start SPI start stopwatch.
+        startStopwatch();
+
+        registerMBean(gridName, this, NeverFailoverSpiMBean.class);
+
+        // Ack ok start.
+        if (log.isDebugEnabled())
+            log.debug(startInfo());
+    }
+
+    /** {@inheritDoc} */
+    @Override public void spiStop() throws IgniteSpiException {
+        unregisterMBean();
+
+        // Ack ok stop.
+        if (log.isDebugEnabled())
+            log.debug(stopInfo());
+    }
+
+    /** {@inheritDoc} */
+    @Override public ClusterNode failover(FailoverContext ctx, List<ClusterNode> top) {
+        U.warn(log, "Returning 'null' node for failed job (failover will not happen) [job=" +
+            ctx.getJobResult().getJob() + ", task=" +  ctx.getTaskSession().getTaskName() +
+            ", sessionId=" + ctx.getTaskSession().getId() + ']');
+
+        return null;
+    }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return S.toString(NeverFailoverSpi.class, this);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/50d1554c/modules/core/src/main/java/org/gridgain/grid/spi/failover/never/NeverFailoverSpiMBean.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/spi/failover/never/NeverFailoverSpiMBean.java b/modules/core/src/main/java/org/gridgain/grid/spi/failover/never/NeverFailoverSpiMBean.java
new file mode 100644
index 0000000..30651c4
--- /dev/null
+++ b/modules/core/src/main/java/org/gridgain/grid/spi/failover/never/NeverFailoverSpiMBean.java
@@ -0,0 +1,21 @@
+/* @java.file.header */
+
+/*  _________        _____ __________________        _____
+ *  __  ____/___________(_)______  /__  ____/______ ____(_)_______
+ *  _  / __  __  ___/__  / _  __  / _  / __  _  __ `/__  / __  __ \
+ *  / /_/ /  _  /    _  /  / /_/ /  / /_/ /  / /_/ / _  /  _  / / /
+ *  \____/   /_/     /_/   \_,__/   \____/   \__,_/  /_/   /_/ /_/
+ */
+
+package org.gridgain.grid.spi.failover.never;
+
+import org.apache.ignite.mbean.*;
+import org.apache.ignite.spi.*;
+
+/**
+ * Management bean for {@link NeverFailoverSpi}.
+ */
+@IgniteMBeanDescription("MBean that provides access to never failover SPI configuration.")
+public interface NeverFailoverSpiMBean extends IgniteSpiManagementMBean {
+    // No-op.
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/50d1554c/modules/core/src/test/config/job-loadtest/client.xml
----------------------------------------------------------------------
diff --git a/modules/core/src/test/config/job-loadtest/client.xml b/modules/core/src/test/config/job-loadtest/client.xml
index 8f74a41..ce6b718 100644
--- a/modules/core/src/test/config/job-loadtest/client.xml
+++ b/modules/core/src/test/config/job-loadtest/client.xml
@@ -23,7 +23,7 @@
         <property name="deploymentMode" value="CONTINUOUS"/>
 
         <property name="failoverSpi">
-            <bean class="org.gridgain.grid.spi.failover.jobstealing.GridJobStealingFailoverSpi">
+            <bean class="org.gridgain.grid.spi.failover.jobstealing.JobStealingFailoverSpi">
                 <property name="maximumFailoverAttempts" value="10"/>
             </bean>
         </property>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/50d1554c/modules/core/src/test/config/job-loadtest/server.xml
----------------------------------------------------------------------
diff --git a/modules/core/src/test/config/job-loadtest/server.xml b/modules/core/src/test/config/job-loadtest/server.xml
index 74841e9..5546e20 100644
--- a/modules/core/src/test/config/job-loadtest/server.xml
+++ b/modules/core/src/test/config/job-loadtest/server.xml
@@ -36,7 +36,7 @@
         </property>
 
         <property name="failoverSpi">
-            <bean class="org.gridgain.grid.spi.failover.jobstealing.GridJobStealingFailoverSpi">
+            <bean class="org.gridgain.grid.spi.failover.jobstealing.JobStealingFailoverSpi">
                 <property name="maximumFailoverAttempts" value="10"/>
             </bean>
         </property>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/50d1554c/modules/core/src/test/config/load/dsi-load-base.xml
----------------------------------------------------------------------
diff --git a/modules/core/src/test/config/load/dsi-load-base.xml b/modules/core/src/test/config/load/dsi-load-base.xml
index 20548b2..8b49076 100644
--- a/modules/core/src/test/config/load/dsi-load-base.xml
+++ b/modules/core/src/test/config/load/dsi-load-base.xml
@@ -97,7 +97,7 @@
         </property>
 
         <property name="failoverSpi">
-            <bean class="org.gridgain.grid.spi.failover.always.GridAlwaysFailoverSpi">
+            <bean class="org.gridgain.grid.spi.failover.always.AlwaysFailoverSpi">
                 <property name="maximumFailoverAttempts" value="1"/>
             </bean>
         </property>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/50d1554c/modules/core/src/test/java/org/apache/ignite/spi/collision/jobstealing/GridJobStealingCollisionSpiAttributesSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/spi/collision/jobstealing/GridJobStealingCollisionSpiAttributesSelfTest.java b/modules/core/src/test/java/org/apache/ignite/spi/collision/jobstealing/GridJobStealingCollisionSpiAttributesSelfTest.java
index fa73c02..73567e2 100644
--- a/modules/core/src/test/java/org/apache/ignite/spi/collision/jobstealing/GridJobStealingCollisionSpiAttributesSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/spi/collision/jobstealing/GridJobStealingCollisionSpiAttributesSelfTest.java
@@ -81,7 +81,7 @@ public class GridJobStealingCollisionSpiAttributesSelfTest extends GridSpiAbstra
      * @throws Exception If failed.
      */
     private void addSpiDependency(GridTestNode node) throws Exception {
-        node.addAttribute(U.spiAttribute(getSpi(), ATTR_SPI_CLASS), GridJobStealingFailoverSpi.class.getName());
+        node.addAttribute(U.spiAttribute(getSpi(), ATTR_SPI_CLASS), JobStealingFailoverSpi.class.getName());
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/50d1554c/modules/core/src/test/java/org/apache/ignite/spi/collision/jobstealing/GridJobStealingCollisionSpiCustomTopologySelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/spi/collision/jobstealing/GridJobStealingCollisionSpiCustomTopologySelfTest.java b/modules/core/src/test/java/org/apache/ignite/spi/collision/jobstealing/GridJobStealingCollisionSpiCustomTopologySelfTest.java
index ff49b08..e94a937 100644
--- a/modules/core/src/test/java/org/apache/ignite/spi/collision/jobstealing/GridJobStealingCollisionSpiCustomTopologySelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/spi/collision/jobstealing/GridJobStealingCollisionSpiCustomTopologySelfTest.java
@@ -94,7 +94,7 @@ public class GridJobStealingCollisionSpiCustomTopologySelfTest extends
      * @throws Exception If failed.
      */
     private void addSpiDependency(GridTestNode node) throws Exception {
-        node.addAttribute(U.spiAttribute(getSpi(), ATTR_SPI_CLASS), GridJobStealingFailoverSpi.class.getName());
+        node.addAttribute(U.spiAttribute(getSpi(), ATTR_SPI_CLASS), JobStealingFailoverSpi.class.getName());
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/50d1554c/modules/core/src/test/java/org/apache/ignite/spi/collision/jobstealing/GridJobStealingCollisionSpiSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/spi/collision/jobstealing/GridJobStealingCollisionSpiSelfTest.java b/modules/core/src/test/java/org/apache/ignite/spi/collision/jobstealing/GridJobStealingCollisionSpiSelfTest.java
index 5d7c0f1..3040afb 100644
--- a/modules/core/src/test/java/org/apache/ignite/spi/collision/jobstealing/GridJobStealingCollisionSpiSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/spi/collision/jobstealing/GridJobStealingCollisionSpiSelfTest.java
@@ -95,7 +95,7 @@ public class GridJobStealingCollisionSpiSelfTest extends GridSpiAbstractTest<Job
      * @throws Exception If failed.
      */
     private void addSpiDependency(GridTestNode node) throws Exception {
-        node.addAttribute(U.spiAttribute(getSpi(), ATTR_SPI_CLASS), GridJobStealingFailoverSpi.class.getName());
+        node.addAttribute(U.spiAttribute(getSpi(), ATTR_SPI_CLASS), JobStealingFailoverSpi.class.getName());
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/50d1554c/modules/core/src/test/java/org/apache/ignite/spi/collision/jobstealing/GridJobStealingCollisionSpiStartStopSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/spi/collision/jobstealing/GridJobStealingCollisionSpiStartStopSelfTest.java b/modules/core/src/test/java/org/apache/ignite/spi/collision/jobstealing/GridJobStealingCollisionSpiStartStopSelfTest.java
index 48837c7..b00f760 100644
--- a/modules/core/src/test/java/org/apache/ignite/spi/collision/jobstealing/GridJobStealingCollisionSpiStartStopSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/spi/collision/jobstealing/GridJobStealingCollisionSpiStartStopSelfTest.java
@@ -44,6 +44,6 @@ public class GridJobStealingCollisionSpiStartStopSelfTest
      * @throws Exception If failed.
      */
     private void addSpiDependency(GridTestNode node) throws Exception {
-        node.addAttribute(U.spiAttribute(getSpi(), ATTR_SPI_CLASS), GridJobStealingFailoverSpi.class.getName());
+        node.addAttribute(U.spiAttribute(getSpi(), ATTR_SPI_CLASS), JobStealingFailoverSpi.class.getName());
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/50d1554c/modules/core/src/test/java/org/gridgain/grid/kernal/GridAlwaysFailoverSpiFailSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/gridgain/grid/kernal/GridAlwaysFailoverSpiFailSelfTest.java b/modules/core/src/test/java/org/gridgain/grid/kernal/GridAlwaysFailoverSpiFailSelfTest.java
index 03a586a..9167dac 100644
--- a/modules/core/src/test/java/org/gridgain/grid/kernal/GridAlwaysFailoverSpiFailSelfTest.java
+++ b/modules/core/src/test/java/org/gridgain/grid/kernal/GridAlwaysFailoverSpiFailSelfTest.java
@@ -93,9 +93,9 @@ public class GridAlwaysFailoverSpiFailSelfTest extends GridCommonAbstractTest {
     }
 
     /** */
-    private class GridTestFailoverSpi extends GridAlwaysFailoverSpi {
+    private class GridTestFailoverSpi extends AlwaysFailoverSpi {
         /** {@inheritDoc} */
-        @Override public ClusterNode failover(GridFailoverContext ctx, List<ClusterNode> grid) {
+        @Override public ClusterNode failover(FailoverContext ctx, List<ClusterNode> grid) {
             isFailoverCalled = true;
 
             return super.failover(ctx, grid);

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/50d1554c/modules/core/src/test/java/org/gridgain/grid/kernal/GridFailoverCustomTopologySelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/gridgain/grid/kernal/GridFailoverCustomTopologySelfTest.java b/modules/core/src/test/java/org/gridgain/grid/kernal/GridFailoverCustomTopologySelfTest.java
index ddc2741..1535a8c 100644
--- a/modules/core/src/test/java/org/gridgain/grid/kernal/GridFailoverCustomTopologySelfTest.java
+++ b/modules/core/src/test/java/org/gridgain/grid/kernal/GridFailoverCustomTopologySelfTest.java
@@ -46,9 +46,9 @@ public class GridFailoverCustomTopologySelfTest extends GridCommonAbstractTest {
 
         cfg.setNodeId(null);
 
-        cfg.setFailoverSpi(new GridAlwaysFailoverSpi() {
+        cfg.setFailoverSpi(new AlwaysFailoverSpi() {
             /** {@inheritDoc} */
-            @Override public ClusterNode failover(GridFailoverContext ctx, List<ClusterNode> top) {
+            @Override public ClusterNode failover(FailoverContext ctx, List<ClusterNode> top) {
                 failCnt.incrementAndGet();
 
                 return super.failover(ctx, top);

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/50d1554c/modules/core/src/test/java/org/gridgain/grid/kernal/GridFailoverSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/gridgain/grid/kernal/GridFailoverSelfTest.java b/modules/core/src/test/java/org/gridgain/grid/kernal/GridFailoverSelfTest.java
index 2face88..e800413 100644
--- a/modules/core/src/test/java/org/gridgain/grid/kernal/GridFailoverSelfTest.java
+++ b/modules/core/src/test/java/org/gridgain/grid/kernal/GridFailoverSelfTest.java
@@ -39,7 +39,7 @@ public class GridFailoverSelfTest extends GridCommonAbstractTest {
     @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
         IgniteConfiguration cfg = super.getConfiguration(gridName);
 
-        cfg.setFailoverSpi(new GridAlwaysFailoverSpi());
+        cfg.setFailoverSpi(new AlwaysFailoverSpi());
 
         return cfg;
     }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/50d1554c/modules/core/src/test/java/org/gridgain/grid/kernal/GridFailoverTaskWithPredicateSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/gridgain/grid/kernal/GridFailoverTaskWithPredicateSelfTest.java b/modules/core/src/test/java/org/gridgain/grid/kernal/GridFailoverTaskWithPredicateSelfTest.java
index 9b6b79b..284d38b 100644
--- a/modules/core/src/test/java/org/gridgain/grid/kernal/GridFailoverTaskWithPredicateSelfTest.java
+++ b/modules/core/src/test/java/org/gridgain/grid/kernal/GridFailoverTaskWithPredicateSelfTest.java
@@ -48,9 +48,9 @@ public class GridFailoverTaskWithPredicateSelfTest extends GridCommonAbstractTes
     @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
         IgniteConfiguration cfg = super.getConfiguration(gridName);
 
-        cfg.setFailoverSpi(new GridAlwaysFailoverSpi() {
+        cfg.setFailoverSpi(new AlwaysFailoverSpi() {
             /** {@inheritDoc} */
-            @Override public ClusterNode failover(GridFailoverContext ctx, List<ClusterNode> grid) {
+            @Override public ClusterNode failover(FailoverContext ctx, List<ClusterNode> grid) {
                 ClusterNode failoverNode = super.failover(ctx, grid);
 
                 if (failoverNode != null)

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/50d1554c/modules/core/src/test/java/org/gridgain/grid/kernal/GridFailoverTopologySelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/gridgain/grid/kernal/GridFailoverTopologySelfTest.java b/modules/core/src/test/java/org/gridgain/grid/kernal/GridFailoverTopologySelfTest.java
index 567407c..b92bc2f 100644
--- a/modules/core/src/test/java/org/gridgain/grid/kernal/GridFailoverTopologySelfTest.java
+++ b/modules/core/src/test/java/org/gridgain/grid/kernal/GridFailoverTopologySelfTest.java
@@ -42,13 +42,13 @@ public class GridFailoverTopologySelfTest extends GridCommonAbstractTest {
 
         cfg.setNodeId(null);
 
-        cfg.setFailoverSpi(new GridAlwaysFailoverSpi() {
+        cfg.setFailoverSpi(new AlwaysFailoverSpi() {
             /** */
             @IgniteLocalNodeIdResource
             private UUID locNodeId;
 
             /** {@inheritDoc} */
-            @Override public ClusterNode failover(GridFailoverContext ctx, List<ClusterNode> grid) {
+            @Override public ClusterNode failover(FailoverContext ctx, List<ClusterNode> grid) {
                 if (grid.size() != 1) {
                     failed.set(true);
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/50d1554c/modules/core/src/test/java/org/gridgain/grid/kernal/GridJobStealingSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/gridgain/grid/kernal/GridJobStealingSelfTest.java b/modules/core/src/test/java/org/gridgain/grid/kernal/GridJobStealingSelfTest.java
index a71e071..463e14a 100644
--- a/modules/core/src/test/java/org/gridgain/grid/kernal/GridJobStealingSelfTest.java
+++ b/modules/core/src/test/java/org/gridgain/grid/kernal/GridJobStealingSelfTest.java
@@ -275,10 +275,10 @@ public class GridJobStealingSelfTest extends GridCommonAbstractTest {
         colSpi.setActiveJobsThreshold(1);
         colSpi.setWaitJobsThreshold(0);
 
-        GridJobStealingFailoverSpi failSpi = new GridJobStealingFailoverSpi();
+        JobStealingFailoverSpi failSpi = new JobStealingFailoverSpi();
 
         // Verify defaults.
-        assert failSpi.getMaximumFailoverAttempts() == GridJobStealingFailoverSpi.DFLT_MAX_FAILOVER_ATTEMPTS;
+        assert failSpi.getMaximumFailoverAttempts() == JobStealingFailoverSpi.DFLT_MAX_FAILOVER_ATTEMPTS;
 
         cfg.setCollisionSpi(colSpi);
         cfg.setFailoverSpi(failSpi);

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/50d1554c/modules/core/src/test/java/org/gridgain/grid/kernal/GridJobStealingZeroActiveJobsSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/gridgain/grid/kernal/GridJobStealingZeroActiveJobsSelfTest.java b/modules/core/src/test/java/org/gridgain/grid/kernal/GridJobStealingZeroActiveJobsSelfTest.java
index 0eb56c7..4855ae4 100644
--- a/modules/core/src/test/java/org/gridgain/grid/kernal/GridJobStealingZeroActiveJobsSelfTest.java
+++ b/modules/core/src/test/java/org/gridgain/grid/kernal/GridJobStealingZeroActiveJobsSelfTest.java
@@ -72,10 +72,10 @@ public class GridJobStealingZeroActiveJobsSelfTest extends GridCommonAbstractTes
         colSpi.setActiveJobsThreshold(gridName.endsWith("1") ? 0 : 2);
         colSpi.setWaitJobsThreshold(0);
 
-        GridJobStealingFailoverSpi failSpi = new GridJobStealingFailoverSpi();
+        JobStealingFailoverSpi failSpi = new JobStealingFailoverSpi();
 
         // Verify defaults.
-        assert failSpi.getMaximumFailoverAttempts() == GridJobStealingFailoverSpi.DFLT_MAX_FAILOVER_ATTEMPTS;
+        assert failSpi.getMaximumFailoverAttempts() == JobStealingFailoverSpi.DFLT_MAX_FAILOVER_ATTEMPTS;
 
         cfg.setCollisionSpi(colSpi);
         cfg.setFailoverSpi(failSpi);

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/50d1554c/modules/core/src/test/java/org/gridgain/grid/kernal/GridMultipleSpisSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/gridgain/grid/kernal/GridMultipleSpisSelfTest.java b/modules/core/src/test/java/org/gridgain/grid/kernal/GridMultipleSpisSelfTest.java
index 37abc5b..6a999ec 100644
--- a/modules/core/src/test/java/org/gridgain/grid/kernal/GridMultipleSpisSelfTest.java
+++ b/modules/core/src/test/java/org/gridgain/grid/kernal/GridMultipleSpisSelfTest.java
@@ -128,7 +128,7 @@ public class GridMultipleSpisSelfTest extends GridCommonAbstractTest {
     }
 
     /** */
-    private class GridTestFailoverSpi extends GridAlwaysFailoverSpi {
+    private class GridTestFailoverSpi extends AlwaysFailoverSpi {
         /** */
         private String expName;
 
@@ -142,7 +142,7 @@ public class GridMultipleSpisSelfTest extends GridCommonAbstractTest {
         }
 
         /** {@inheritDoc} */
-        @Override public ClusterNode failover(GridFailoverContext ctx, List<ClusterNode> grid) {
+        @Override public ClusterNode failover(FailoverContext ctx, List<ClusterNode> grid) {
             if (getName().equals(expName))
                 isTaskFailoverCalled = true;
             else

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/50d1554c/modules/core/src/test/java/org/gridgain/grid/kernal/GridMultithreadedJobStealingSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/gridgain/grid/kernal/GridMultithreadedJobStealingSelfTest.java b/modules/core/src/test/java/org/gridgain/grid/kernal/GridMultithreadedJobStealingSelfTest.java
index f1ba482..a5da434 100644
--- a/modules/core/src/test/java/org/gridgain/grid/kernal/GridMultithreadedJobStealingSelfTest.java
+++ b/modules/core/src/test/java/org/gridgain/grid/kernal/GridMultithreadedJobStealingSelfTest.java
@@ -126,10 +126,10 @@ public class GridMultithreadedJobStealingSelfTest extends GridCommonAbstractTest
         colSpi.setActiveJobsThreshold(1);
         colSpi.setWaitJobsThreshold(0);
 
-        GridJobStealingFailoverSpi failSpi = new GridJobStealingFailoverSpi();
+        JobStealingFailoverSpi failSpi = new JobStealingFailoverSpi();
 
         // Verify defaults.
-        assert failSpi.getMaximumFailoverAttempts() == GridJobStealingFailoverSpi.DFLT_MAX_FAILOVER_ATTEMPTS;
+        assert failSpi.getMaximumFailoverAttempts() == JobStealingFailoverSpi.DFLT_MAX_FAILOVER_ATTEMPTS;
 
         cfg.setCollisionSpi(colSpi);
         cfg.setFailoverSpi(failSpi);

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/50d1554c/modules/core/src/test/java/org/gridgain/grid/kernal/GridSpiExceptionSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/gridgain/grid/kernal/GridSpiExceptionSelfTest.java b/modules/core/src/test/java/org/gridgain/grid/kernal/GridSpiExceptionSelfTest.java
index 920052f..de78b44 100644
--- a/modules/core/src/test/java/org/gridgain/grid/kernal/GridSpiExceptionSelfTest.java
+++ b/modules/core/src/test/java/org/gridgain/grid/kernal/GridSpiExceptionSelfTest.java
@@ -85,7 +85,7 @@ public class GridSpiExceptionSelfTest extends GridCommonAbstractTest {
      * Test event storage spi that throws an exception on try to query local events.
      */
     @IgniteSpiMultipleInstancesSupport(true)
-    private static class GridTestRuntimeExceptionSpi extends IgniteSpiAdapter implements GridEventStorageSpi {
+    private static class GridTestRuntimeExceptionSpi extends IgniteSpiAdapter implements EventStorageSpi {
         /** {@inheritDoc} */
         @Override public void spiStart(String gridName) throws IgniteSpiException {
             startStopwatch();

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/50d1554c/modules/core/src/test/java/org/gridgain/grid/kernal/GridStopWithWaitSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/gridgain/grid/kernal/GridStopWithWaitSelfTest.java b/modules/core/src/test/java/org/gridgain/grid/kernal/GridStopWithWaitSelfTest.java
index 44722d2..5ca4f9d 100644
--- a/modules/core/src/test/java/org/gridgain/grid/kernal/GridStopWithWaitSelfTest.java
+++ b/modules/core/src/test/java/org/gridgain/grid/kernal/GridStopWithWaitSelfTest.java
@@ -46,7 +46,7 @@ public class GridStopWithWaitSelfTest extends GridCommonAbstractTest {
     @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
         IgniteConfiguration cfg = super.getConfiguration(gridName);
 
-        cfg.setFailoverSpi(new GridAlwaysFailoverSpi());
+        cfg.setFailoverSpi(new AlwaysFailoverSpi());
 
         return cfg;
     }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/50d1554c/modules/core/src/test/java/org/gridgain/grid/kernal/managers/GridManagerStopSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/gridgain/grid/kernal/managers/GridManagerStopSelfTest.java b/modules/core/src/test/java/org/gridgain/grid/kernal/managers/GridManagerStopSelfTest.java
index a832496..053056b 100644
--- a/modules/core/src/test/java/org/gridgain/grid/kernal/managers/GridManagerStopSelfTest.java
+++ b/modules/core/src/test/java/org/gridgain/grid/kernal/managers/GridManagerStopSelfTest.java
@@ -157,7 +157,7 @@ public class GridManagerStopSelfTest extends GridCommonAbstractTest {
      * @throws Exception If failed.
      */
     public void testStopEventStorageManager() throws Exception {
-        GridEventStorageSpi spi = new GridMemoryEventStorageSpi();
+        EventStorageSpi spi = new MemoryEventStorageSpi();
 
         injectLogger(spi);
 
@@ -172,7 +172,7 @@ public class GridManagerStopSelfTest extends GridCommonAbstractTest {
      * @throws Exception If failed.
      */
     public void testStopFailoverManager() throws Exception {
-        GridAlwaysFailoverSpi spi = new GridAlwaysFailoverSpi();
+        AlwaysFailoverSpi spi = new AlwaysFailoverSpi();
 
         injectLogger(spi);
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/50d1554c/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheGroupLockFailoverSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheGroupLockFailoverSelfTest.java b/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheGroupLockFailoverSelfTest.java
index d2039cb..52465fd 100644
--- a/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheGroupLockFailoverSelfTest.java
+++ b/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheGroupLockFailoverSelfTest.java
@@ -455,7 +455,7 @@ public class GridCacheGroupLockFailoverSelfTest extends GridCommonAbstractTest {
     /**
      * Test failover SPI that remembers the job contexts of failed jobs.
      */
-    private class GridTestFailoverSpi extends GridAlwaysFailoverSpi {
+    private class GridTestFailoverSpi extends AlwaysFailoverSpi {
         /** */
         private static final String FAILOVER_NUMBER_ATTR = "failover:number:attr";
 
@@ -479,7 +479,7 @@ public class GridCacheGroupLockFailoverSelfTest extends GridCommonAbstractTest {
         }
 
         /** {@inheritDoc} */
-        @Override public ClusterNode failover(GridFailoverContext ctx, List<ClusterNode> top) {
+        @Override public ClusterNode failover(FailoverContext ctx, List<ClusterNode> top) {
             List<ClusterNode> cp = null;
             if (master) {
                 failedOverJobs.add(ctx.getJobResult().getJobContext());

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/50d1554c/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCachePutAllFailoverSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCachePutAllFailoverSelfTest.java b/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCachePutAllFailoverSelfTest.java
index 7b8fda6..14e38cf 100644
--- a/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCachePutAllFailoverSelfTest.java
+++ b/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCachePutAllFailoverSelfTest.java
@@ -626,7 +626,7 @@ public class GridCachePutAllFailoverSelfTest extends GridCommonAbstractTest {
      * Test failover SPI for master node.
      */
     @IgniteSpiConsistencyChecked(optional = true)
-    private static class MasterFailoverSpi extends GridAlwaysFailoverSpi {
+    private static class MasterFailoverSpi extends AlwaysFailoverSpi {
         /** */
         private static final String FAILOVER_NUMBER_ATTR = "failover:number:attr";
 
@@ -648,7 +648,7 @@ public class GridCachePutAllFailoverSelfTest extends GridCommonAbstractTest {
         }
 
         /** {@inheritDoc} */
-        @Override public ClusterNode failover(GridFailoverContext ctx, List<ClusterNode> top) {
+        @Override public ClusterNode failover(FailoverContext ctx, List<ClusterNode> top) {
             failedOverJobs.add(ctx.getJobResult().getJobContext());
 
             // Clear failed nodes list - allow to failover on the same node.

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/50d1554c/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/distributed/GridCachePreloadEventsAbstractSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/distributed/GridCachePreloadEventsAbstractSelfTest.java b/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/distributed/GridCachePreloadEventsAbstractSelfTest.java
index 9e51616..f03aa0c 100644
--- a/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/distributed/GridCachePreloadEventsAbstractSelfTest.java
+++ b/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/distributed/GridCachePreloadEventsAbstractSelfTest.java
@@ -46,7 +46,7 @@ public abstract class GridCachePreloadEventsAbstractSelfTest extends GridCommonA
 
         cfg.setCacheConfiguration(cacheConfiguration());
 
-        GridMemoryEventStorageSpi evtStorageSpi = new GridMemoryEventStorageSpi();
+        MemoryEventStorageSpi evtStorageSpi = new MemoryEventStorageSpi();
 
         evtStorageSpi.setExpireCount(50_000);
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/50d1554c/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/distributed/replicated/preloader/GridCacheReplicatedPreloadSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/distributed/replicated/preloader/GridCacheReplicatedPreloadSelfTest.java b/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/distributed/replicated/preloader/GridCacheReplicatedPreloadSelfTest.java
index a065a67..bd32dd4 100644
--- a/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/distributed/replicated/preloader/GridCacheReplicatedPreloadSelfTest.java
+++ b/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/distributed/replicated/preloader/GridCacheReplicatedPreloadSelfTest.java
@@ -75,7 +75,7 @@ public class GridCacheReplicatedPreloadSelfTest extends GridCommonAbstractTest {
 
         cfg.setUserAttributes(F.asMap("EVEN", !gridName.endsWith("0") && !gridName.endsWith("2")));
 
-        GridMemoryEventStorageSpi spi = new GridMemoryEventStorageSpi();
+        MemoryEventStorageSpi spi = new MemoryEventStorageSpi();
 
         spi.setExpireCount(50_000);
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/50d1554c/modules/core/src/test/java/org/gridgain/grid/p2p/GridAbstractMultinodeRedeployTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/gridgain/grid/p2p/GridAbstractMultinodeRedeployTest.java b/modules/core/src/test/java/org/gridgain/grid/p2p/GridAbstractMultinodeRedeployTest.java
index dd0638b..0eb7b79 100644
--- a/modules/core/src/test/java/org/gridgain/grid/p2p/GridAbstractMultinodeRedeployTest.java
+++ b/modules/core/src/test/java/org/gridgain/grid/p2p/GridAbstractMultinodeRedeployTest.java
@@ -40,7 +40,7 @@ abstract class GridAbstractMultinodeRedeployTest extends GridCommonAbstractTest
 
         cfg.setDeploymentMode(depMode);
 
-        cfg.setFailoverSpi(new GridNeverFailoverSpi());
+        cfg.setFailoverSpi(new NeverFailoverSpi());
 
         cfg.setNetworkTimeout(10000);
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/50d1554c/modules/core/src/test/java/org/gridgain/grid/session/GridSessionJobFailoverSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/gridgain/grid/session/GridSessionJobFailoverSelfTest.java b/modules/core/src/test/java/org/gridgain/grid/session/GridSessionJobFailoverSelfTest.java
index 5eee826..f61b906 100644
--- a/modules/core/src/test/java/org/gridgain/grid/session/GridSessionJobFailoverSelfTest.java
+++ b/modules/core/src/test/java/org/gridgain/grid/session/GridSessionJobFailoverSelfTest.java
@@ -37,7 +37,7 @@ public class GridSessionJobFailoverSelfTest extends GridCommonAbstractTest {
     @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
         IgniteConfiguration cfg = super.getConfiguration(gridName);
 
-        cfg.setFailoverSpi(new GridAlwaysFailoverSpi());
+        cfg.setFailoverSpi(new AlwaysFailoverSpi());
 
         return cfg;
     }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/50d1554c/modules/core/src/test/java/org/gridgain/grid/spi/eventstorage/memory/GridMemoryEventStorageMultiThreadedSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/gridgain/grid/spi/eventstorage/memory/GridMemoryEventStorageMultiThreadedSelfTest.java b/modules/core/src/test/java/org/gridgain/grid/spi/eventstorage/memory/GridMemoryEventStorageMultiThreadedSelfTest.java
index 9550fb4..5ab8dfd 100644
--- a/modules/core/src/test/java/org/gridgain/grid/spi/eventstorage/memory/GridMemoryEventStorageMultiThreadedSelfTest.java
+++ b/modules/core/src/test/java/org/gridgain/grid/spi/eventstorage/memory/GridMemoryEventStorageMultiThreadedSelfTest.java
@@ -19,8 +19,8 @@ import java.util.concurrent.*;
 /**
  * Memory event storage load test.
  */
-@GridSpiTest(spi = GridMemoryEventStorageSpi.class, group = "EventStorage SPI")
-public class GridMemoryEventStorageMultiThreadedSelfTest extends GridSpiAbstractTest<GridMemoryEventStorageSpi> {
+@GridSpiTest(spi = MemoryEventStorageSpi.class, group = "EventStorage SPI")
+public class GridMemoryEventStorageMultiThreadedSelfTest extends GridSpiAbstractTest<MemoryEventStorageSpi> {
     /**
      * @throws Exception If test failed
      */

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/50d1554c/modules/core/src/test/java/org/gridgain/grid/spi/eventstorage/memory/GridMemoryEventStorageSpiConfigSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/gridgain/grid/spi/eventstorage/memory/GridMemoryEventStorageSpiConfigSelfTest.java b/modules/core/src/test/java/org/gridgain/grid/spi/eventstorage/memory/GridMemoryEventStorageSpiConfigSelfTest.java
index ab14791..dd4ffb2 100644
--- a/modules/core/src/test/java/org/gridgain/grid/spi/eventstorage/memory/GridMemoryEventStorageSpiConfigSelfTest.java
+++ b/modules/core/src/test/java/org/gridgain/grid/spi/eventstorage/memory/GridMemoryEventStorageSpiConfigSelfTest.java
@@ -14,13 +14,13 @@ import org.gridgain.testframework.junits.spi.*;
 /**
  * Memory event storage SPI config test.
  */
-@GridSpiTest(spi = GridMemoryEventStorageSpi.class, group = "Event Storage SPI")
-public class GridMemoryEventStorageSpiConfigSelfTest extends GridSpiAbstractConfigTest<GridMemoryEventStorageSpi> {
+@GridSpiTest(spi = MemoryEventStorageSpi.class, group = "Event Storage SPI")
+public class GridMemoryEventStorageSpiConfigSelfTest extends GridSpiAbstractConfigTest<MemoryEventStorageSpi> {
     /**
      * @throws Exception If failed.
      */
     public void testNegativeConfig() throws Exception {
-        checkNegativeSpiProperty(new GridMemoryEventStorageSpi(), "expireCount", 0);
-        checkNegativeSpiProperty(new GridMemoryEventStorageSpi(), "expireAgeMs", 0);
+        checkNegativeSpiProperty(new MemoryEventStorageSpi(), "expireCount", 0);
+        checkNegativeSpiProperty(new MemoryEventStorageSpi(), "expireAgeMs", 0);
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/50d1554c/modules/core/src/test/java/org/gridgain/grid/spi/eventstorage/memory/GridMemoryEventStorageSpiSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/gridgain/grid/spi/eventstorage/memory/GridMemoryEventStorageSpiSelfTest.java b/modules/core/src/test/java/org/gridgain/grid/spi/eventstorage/memory/GridMemoryEventStorageSpiSelfTest.java
index 4228170..97b5833 100644
--- a/modules/core/src/test/java/org/gridgain/grid/spi/eventstorage/memory/GridMemoryEventStorageSpiSelfTest.java
+++ b/modules/core/src/test/java/org/gridgain/grid/spi/eventstorage/memory/GridMemoryEventStorageSpiSelfTest.java
@@ -19,10 +19,10 @@ import java.util.*;
 import static org.apache.ignite.events.IgniteEventType.*;
 
 /**
- * Tests for {@link GridMemoryEventStorageSpi}.
+ * Tests for {@link MemoryEventStorageSpi}.
  */
-@GridSpiTest(spi = GridMemoryEventStorageSpi.class, group = "Event Storage SPI")
-public class GridMemoryEventStorageSpiSelfTest extends GridSpiAbstractTest<GridMemoryEventStorageSpi> {
+@GridSpiTest(spi = MemoryEventStorageSpi.class, group = "Event Storage SPI")
+public class GridMemoryEventStorageSpiSelfTest extends GridSpiAbstractTest<MemoryEventStorageSpi> {
     /** */
     private static final int EXPIRE_CNT = 100;
 
@@ -46,7 +46,7 @@ public class GridMemoryEventStorageSpiSelfTest extends GridSpiAbstractTest<GridM
      * @throws Exception If failed.
      */
     public void testMemoryEventStorage() throws Exception {
-        GridMemoryEventStorageSpi spi = getSpi();
+        MemoryEventStorageSpi spi = getSpi();
 
         IgnitePredicate<IgniteEvent> filter = F.alwaysTrue();
 
@@ -93,7 +93,7 @@ public class GridMemoryEventStorageSpiSelfTest extends GridSpiAbstractTest<GridM
      */
     @SuppressWarnings({"NullableProblems"})
     public void testFilter() throws Exception {
-        GridMemoryEventStorageSpi spi = getSpi();
+        MemoryEventStorageSpi spi = getSpi();
 
         try {
             spi.clearAll();

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/50d1554c/modules/core/src/test/java/org/gridgain/grid/spi/eventstorage/memory/GridMemoryEventStorageSpiStartStopSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/gridgain/grid/spi/eventstorage/memory/GridMemoryEventStorageSpiStartStopSelfTest.java b/modules/core/src/test/java/org/gridgain/grid/spi/eventstorage/memory/GridMemoryEventStorageSpiStartStopSelfTest.java
index 50826eb..5f5d05d 100644
--- a/modules/core/src/test/java/org/gridgain/grid/spi/eventstorage/memory/GridMemoryEventStorageSpiStartStopSelfTest.java
+++ b/modules/core/src/test/java/org/gridgain/grid/spi/eventstorage/memory/GridMemoryEventStorageSpiStartStopSelfTest.java
@@ -15,7 +15,7 @@ import org.gridgain.testframework.junits.spi.*;
 /**
  * Memory event storage SPI start-stop test.
  */
-@GridSpiTest(spi = GridMemoryEventStorageSpi.class, group = "Event Storage SPI")
-public class GridMemoryEventStorageSpiStartStopSelfTest extends GridSpiStartStopAbstractTest<GridMemoryEventStorageSpi> {
+@GridSpiTest(spi = MemoryEventStorageSpi.class, group = "Event Storage SPI")
+public class GridMemoryEventStorageSpiStartStopSelfTest extends GridSpiStartStopAbstractTest<MemoryEventStorageSpi> {
     // No-op.
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/50d1554c/modules/core/src/test/java/org/gridgain/grid/spi/failover/GridFailoverTestContext.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/gridgain/grid/spi/failover/GridFailoverTestContext.java b/modules/core/src/test/java/org/gridgain/grid/spi/failover/GridFailoverTestContext.java
index dc8ae30..d89a7e8 100644
--- a/modules/core/src/test/java/org/gridgain/grid/spi/failover/GridFailoverTestContext.java
+++ b/modules/core/src/test/java/org/gridgain/grid/spi/failover/GridFailoverTestContext.java
@@ -18,7 +18,7 @@ import java.util.*;
 /**
  * Failover test context.
  */
-public class GridFailoverTestContext implements GridFailoverContext {
+public class GridFailoverTestContext implements FailoverContext {
     /** */
     private static final Random RAND = new Random();
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/50d1554c/modules/core/src/test/java/org/gridgain/grid/spi/failover/always/GridAlwaysFailoverSpiConfigSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/gridgain/grid/spi/failover/always/GridAlwaysFailoverSpiConfigSelfTest.java b/modules/core/src/test/java/org/gridgain/grid/spi/failover/always/GridAlwaysFailoverSpiConfigSelfTest.java
index 720e22a..79d5946 100644
--- a/modules/core/src/test/java/org/gridgain/grid/spi/failover/always/GridAlwaysFailoverSpiConfigSelfTest.java
+++ b/modules/core/src/test/java/org/gridgain/grid/spi/failover/always/GridAlwaysFailoverSpiConfigSelfTest.java
@@ -14,12 +14,12 @@ import org.gridgain.testframework.junits.spi.*;
 /**
  * Always-failover SPI config test.
  */
-@GridSpiTest(spi = GridAlwaysFailoverSpi.class, group = "Collision SPI")
-public class GridAlwaysFailoverSpiConfigSelfTest extends GridSpiAbstractConfigTest<GridAlwaysFailoverSpi> {
+@GridSpiTest(spi = AlwaysFailoverSpi.class, group = "Collision SPI")
+public class GridAlwaysFailoverSpiConfigSelfTest extends GridSpiAbstractConfigTest<AlwaysFailoverSpi> {
     /**
      * @throws Exception If failed.
      */
     public void testNegativeConfig() throws Exception {
-        checkNegativeSpiProperty(new GridAlwaysFailoverSpi(), "maximumFailoverAttempts", -1);
+        checkNegativeSpiProperty(new AlwaysFailoverSpi(), "maximumFailoverAttempts", -1);
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/50d1554c/modules/core/src/test/java/org/gridgain/grid/spi/failover/always/GridAlwaysFailoverSpiSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/gridgain/grid/spi/failover/always/GridAlwaysFailoverSpiSelfTest.java b/modules/core/src/test/java/org/gridgain/grid/spi/failover/always/GridAlwaysFailoverSpiSelfTest.java
index 31e4a73..f8df7b2 100644
--- a/modules/core/src/test/java/org/gridgain/grid/spi/failover/always/GridAlwaysFailoverSpiSelfTest.java
+++ b/modules/core/src/test/java/org/gridgain/grid/spi/failover/always/GridAlwaysFailoverSpiSelfTest.java
@@ -18,18 +18,18 @@ import org.gridgain.testframework.junits.spi.*;
 
 import java.util.*;
 
-import static org.gridgain.grid.spi.failover.always.GridAlwaysFailoverSpi.*;
+import static org.gridgain.grid.spi.failover.always.AlwaysFailoverSpi.*;
 
 /**
  * Always-failover SPI test.
  */
-@GridSpiTest(spi = GridAlwaysFailoverSpi.class, group = "Failover SPI")
-public class GridAlwaysFailoverSpiSelfTest extends GridSpiAbstractTest<GridAlwaysFailoverSpi> {
+@GridSpiTest(spi = AlwaysFailoverSpi.class, group = "Failover SPI")
+public class GridAlwaysFailoverSpiSelfTest extends GridSpiAbstractTest<AlwaysFailoverSpi> {
     /**
      * @throws Exception If failed.
      */
     public void testSingleNode() throws Exception {
-        GridAlwaysFailoverSpi spi = getSpi();
+        AlwaysFailoverSpi spi = getSpi();
 
         List<ClusterNode> nodes = new ArrayList<>();
 
@@ -47,7 +47,7 @@ public class GridAlwaysFailoverSpiSelfTest extends GridSpiAbstractTest<GridAlway
      */
     @SuppressWarnings("unchecked")
     public void testTwoNodes() throws Exception {
-        GridAlwaysFailoverSpi spi = getSpi();
+        AlwaysFailoverSpi spi = getSpi();
 
         List<ClusterNode> nodes = new ArrayList<>();
 
@@ -68,7 +68,7 @@ public class GridAlwaysFailoverSpiSelfTest extends GridSpiAbstractTest<GridAlway
      * @throws Exception If failed.
      */
     public void testMaxAttempts() throws Exception {
-        GridAlwaysFailoverSpi spi = getSpi();
+        AlwaysFailoverSpi spi = getSpi();
 
         spi.setMaximumFailoverAttempts(1);
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/50d1554c/modules/core/src/test/java/org/gridgain/grid/spi/failover/always/GridAlwaysFailoverSpiStartStopSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/gridgain/grid/spi/failover/always/GridAlwaysFailoverSpiStartStopSelfTest.java b/modules/core/src/test/java/org/gridgain/grid/spi/failover/always/GridAlwaysFailoverSpiStartStopSelfTest.java
index 7be414b..b675067 100644
--- a/modules/core/src/test/java/org/gridgain/grid/spi/failover/always/GridAlwaysFailoverSpiStartStopSelfTest.java
+++ b/modules/core/src/test/java/org/gridgain/grid/spi/failover/always/GridAlwaysFailoverSpiStartStopSelfTest.java
@@ -16,7 +16,7 @@ import org.gridgain.testframework.junits.spi.*;
 /**
  * Always-failover SPI start-stop test.
  */
-@GridSpiTest(spi = GridAlwaysFailoverSpi.class, group = "Failover SPI")
-public class GridAlwaysFailoverSpiStartStopSelfTest extends GridSpiStartStopAbstractTest<GridFailoverSpi> {
+@GridSpiTest(spi = AlwaysFailoverSpi.class, group = "Failover SPI")
+public class GridAlwaysFailoverSpiStartStopSelfTest extends GridSpiStartStopAbstractTest<FailoverSpi> {
     // No-op.
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/50d1554c/modules/core/src/test/java/org/gridgain/grid/spi/failover/jobstealing/GridJobStealingFailoverSpiConfigSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/gridgain/grid/spi/failover/jobstealing/GridJobStealingFailoverSpiConfigSelfTest.java b/modules/core/src/test/java/org/gridgain/grid/spi/failover/jobstealing/GridJobStealingFailoverSpiConfigSelfTest.java
index 02bd287..9935dde 100644
--- a/modules/core/src/test/java/org/gridgain/grid/spi/failover/jobstealing/GridJobStealingFailoverSpiConfigSelfTest.java
+++ b/modules/core/src/test/java/org/gridgain/grid/spi/failover/jobstealing/GridJobStealingFailoverSpiConfigSelfTest.java
@@ -14,12 +14,12 @@ import org.gridgain.testframework.junits.spi.*;
 /**
  * Job stealing failover SPI config test.
  */
-@GridSpiTest(spi = GridJobStealingFailoverSpi.class, group = "Collision SPI")
-public class GridJobStealingFailoverSpiConfigSelfTest extends GridSpiAbstractConfigTest<GridJobStealingFailoverSpi> {
+@GridSpiTest(spi = JobStealingFailoverSpi.class, group = "Collision SPI")
+public class GridJobStealingFailoverSpiConfigSelfTest extends GridSpiAbstractConfigTest<JobStealingFailoverSpi> {
     /**
      * @throws Exception If failed.
      */
     public void testNegativeConfig() throws Exception {
-        checkNegativeSpiProperty(new GridJobStealingFailoverSpi(), "maximumFailoverAttempts", -1);
+        checkNegativeSpiProperty(new JobStealingFailoverSpi(), "maximumFailoverAttempts", -1);
     }
 }