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:22:37 UTC

[01/20] incubator-ignite git commit: # Renaming

Repository: incubator-ignite
Updated Branches:
  refs/heads/master 12837f35b -> 8c20bddb3


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/a62862fe/modules/core/src/main/java/org/gridgain/grid/spi/loadbalancing/roundrobin/RoundRobinLoadBalancingSpi.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/spi/loadbalancing/roundrobin/RoundRobinLoadBalancingSpi.java b/modules/core/src/main/java/org/gridgain/grid/spi/loadbalancing/roundrobin/RoundRobinLoadBalancingSpi.java
new file mode 100644
index 0000000..243e20a
--- /dev/null
+++ b/modules/core/src/main/java/org/gridgain/grid/spi/loadbalancing/roundrobin/RoundRobinLoadBalancingSpi.java
@@ -0,0 +1,319 @@
+/* @java.file.header */
+
+/*  _________        _____ __________________        _____
+ *  __  ____/___________(_)______  /__  ____/______ ____(_)_______
+ *  _  / __  __  ___/__  / _  __  / _  / __  _  __ `/__  / __  __ \
+ *  / /_/ /  _  /    _  /  / /_/ /  / /_/ /  / /_/ / _  /  _  / / /
+ *  \____/   /_/     /_/   \_,__/   \____/   \__,_/  /_/   /_/ /_/
+ */
+
+package org.gridgain.grid.spi.loadbalancing.roundrobin;
+
+import org.apache.ignite.*;
+import org.apache.ignite.cluster.*;
+import org.apache.ignite.compute.*;
+import org.apache.ignite.events.*;
+import org.apache.ignite.lang.*;
+import org.apache.ignite.resources.*;
+import org.apache.ignite.spi.*;
+import org.gridgain.grid.*;
+import org.gridgain.grid.kernal.managers.eventstorage.*;
+import org.gridgain.grid.spi.loadbalancing.*;
+import org.gridgain.grid.util.typedef.internal.*;
+import org.jdk8.backport.*;
+import org.jetbrains.annotations.*;
+
+import java.util.*;
+
+import static org.apache.ignite.events.IgniteEventType.*;
+
+/**
+ * This SPI iterates through nodes in round-robin fashion and pick the next
+ * sequential node. Two modes of operation are supported: per-task and global
+ * (see {@link #setPerTask(boolean)} configuration).
+ * <p>
+ * When configured in per-task mode, implementation will pick a random starting
+ * node at the beginning of every task execution and then sequentially iterate through all
+ * nodes in topology starting from the picked node. This is the default configuration
+ * and should fit most of the use cases as it provides a fairly well-distributed
+ * split and also ensures that jobs within a single task are spread out across
+ * nodes to the maximum. For cases when split size is equal to the number of nodes,
+ * this mode guarantees that all nodes will participate in the split.
+ * <p>
+ * When configured in global mode, a single sequential queue of nodes is maintained for
+ * all tasks and the next node in the queue is picked every time. In this mode (unlike in
+ * {@code per-task} mode) it is possible that even if split size may be equal to the
+ * number of nodes, some jobs within the same task will be assigned to the same node if
+ * multiple tasks are executing concurrently.
+ * <h1 class="header">Coding Example</h1>
+ * If you are using {@link org.apache.ignite.compute.ComputeTaskSplitAdapter} then load balancing logic
+ * is transparent to your code and is handled automatically by the adapter.
+ * Here is an example of how your task will look:
+ * <pre name="code" class="java">
+ * public class MyFooBarTask extends GridComputeTaskSplitAdapter&lt;Object, Object&gt; {
+ *    &#64;Override
+ *    protected Collection&lt;? extends GridComputeJob&gt; split(int gridSize, Object arg) throws GridException {
+ *        List&lt;MyFooBarJob&gt; jobs = new ArrayList&lt;MyFooBarJob&gt;(gridSize);
+ *
+ *        for (int i = 0; i &lt; gridSize; i++) {
+ *            jobs.add(new MyFooBarJob(arg));
+ *        }
+ *
+ *        // Node assignment via load balancer
+ *        // happens automatically.
+ *        return jobs;
+ *    }
+ *    ...
+ * }
+ * </pre>
+ * If you need more fine-grained control over how some jobs within task get mapped to a node
+ * and use affinity load balancing for some other jobs within task, then you should use
+ * {@link org.apache.ignite.compute.ComputeTaskAdapter}. Here is an example of how your task will look. Note that in this
+ * case we manually inject load balancer and use it to pick the best node. Doing it in
+ * such way would allow user to map some jobs manually and for others use load balancer.
+ * <pre name="code" class="java">
+ * public class MyFooBarTask extends GridComputeTaskAdapter&lt;String, String&gt; {
+ *    // Inject load balancer.
+ *    &#64;GridLoadBalancerResource
+ *    GridComputeLoadBalancer balancer;
+ *
+ *    // Map jobs to grid nodes.
+ *    public Map&lt;? extends GridComputeJob, GridNode&gt; map(List&lt;GridNode&gt; subgrid, String arg) throws GridException {
+ *        Map&lt;MyFooBarJob, GridNode&gt; jobs = new HashMap&lt;MyFooBarJob, GridNode&gt;(subgrid.size());
+ *
+ *        // In more complex cases, you can actually do
+ *        // more complicated assignments of jobs to nodes.
+ *        for (int i = 0; i &lt; subgrid.size(); i++) {
+ *            // Pick the next best balanced node for the job.
+ *            jobs.put(new MyFooBarJob(arg), balancer.getBalancedNode())
+ *        }
+ *
+ *        return jobs;
+ *    }
+ *
+ *    // Aggregate results into one compound result.
+ *    public String reduce(List&lt;GridComputeJobResult&gt; results) throws GridException {
+ *        // For the purpose of this example we simply
+ *        // concatenate string representation of every
+ *        // job result
+ *        StringBuilder buf = new StringBuilder();
+ *
+ *        for (GridComputeJobResult res : results) {
+ *            // Append string representation of result
+ *            // returned by every job.
+ *            buf.append(res.getData().string());
+ *        }
+ *
+ *        return buf.string();
+ *    }
+ * }
+ * </pre>
+ * <p>
+ * <h1 class="header">Configuration</h1>
+ * In order to use this load balancer, you should configure your grid instance
+ * to use {@code GridRoundRobinLoadBalancingSpi} either from Spring XML file or
+ * directly. The following configuration parameters are supported:
+ * <h2 class="header">Mandatory</h2>
+ * This SPI has no mandatory configuration parameters.
+ * <h2 class="header">Optional</h2>
+ * The following configuration parameters are optional:
+ * <ul>
+ * <li>
+ *      Flag that indicates whether to use {@code per-task} or global
+ *      round-robin modes described above (see {@link #setPerTask(boolean)}).
+ * </li>
+ * </ul>
+ * Below is Java configuration example:
+ * <pre name="code" class="java">
+ * GridRandomLoadBalancingSpi = new GridRandomLoadBalancingSpi();
+ *
+ * // Configure SPI to use global round-robin mode.
+ * spi.setPerTask(false);
+ *
+ * GridConfiguration cfg = new GridConfiguration();
+ *
+ * // Override default load balancing SPI.
+ * cfg.setLoadBalancingSpi(spi);
+ *
+ * // Starts grid.
+ * G.start(cfg);
+ * </pre>
+ * Here is how you can configure {@code GridRandomLoadBalancingSpi} using Spring XML configuration:
+ * <pre name="code" class="xml">
+ * &lt;property name="loadBalancingSpi"&gt;
+ *     &lt;bean class="org.gridgain.grid.spi.loadBalancing.roundrobin.GridRoundRobinLoadBalancingSpi"&gt;
+ *         &lt;!-- Set to global round-robin mode. --&gt;
+ *         &lt;property name="perTask" value="false"/&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>
+ */
+@IgniteSpiMultipleInstancesSupport(true)
+public class RoundRobinLoadBalancingSpi extends IgniteSpiAdapter implements LoadBalancingSpi,
+    RoundRobinLoadBalancingSpiMBean {
+    /** Grid logger. */
+    @IgniteLoggerResource
+    private IgniteLogger log;
+
+    /** */
+    private RoundRobinGlobalLoadBalancer balancer;
+
+    /** */
+    private boolean isPerTask;
+
+    /** */
+    private final Map<IgniteUuid, RoundRobinPerTaskLoadBalancer> perTaskBalancers =
+        new ConcurrentHashMap8<>();
+
+    /** Event listener. */
+    private final GridLocalEventListener lsnr = new GridLocalEventListener() {
+        @Override public void onEvent(IgniteEvent evt) {
+            if (evt.type() == EVT_TASK_FAILED ||
+                evt.type() == EVT_TASK_FINISHED)
+                perTaskBalancers.remove(((IgniteTaskEvent)evt).taskSessionId());
+            else if (evt.type() == EVT_JOB_MAPPED) {
+                RoundRobinPerTaskLoadBalancer balancer =
+                    perTaskBalancers.get(((IgniteJobEvent)evt).taskSessionId());
+
+                if (balancer != null)
+                    balancer.onMapped();
+            }
+        }
+    };
+
+    /** {@inheritDoc} */
+    @Override public boolean isPerTask() {
+        return isPerTask;
+    }
+
+    /**
+     * Configuration parameter indicating whether a new round robin order should be
+     * created for every task. If {@code true} then load balancer is guaranteed
+     * to iterate through nodes sequentially for every task - so as long as number
+     * of jobs is less than or equal to the number of nodes, jobs are guaranteed to
+     * be assigned to unique nodes. If {@code false} then one round-robin order
+     * will be maintained for all tasks, so when tasks execute concurrently, it
+     * is possible for more than one job within task to be assigned to the same
+     * node.
+     * <p>
+     * Default is {@code false}.
+     *
+     * @param isPerTask Configuration parameter indicating whether a new round robin order should
+     *      be created for every task. Default is {@code false}.
+     */
+    @IgniteSpiConfiguration(optional = true)
+    public void setPerTask(boolean isPerTask) {
+        this.isPerTask = isPerTask;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void spiStart(@Nullable String gridName) throws IgniteSpiException {
+        startStopwatch();
+
+        if (log.isDebugEnabled())
+            log.debug(configInfo("isPerTask", isPerTask));
+
+        registerMBean(gridName, this, RoundRobinLoadBalancingSpiMBean.class);
+
+        balancer = new RoundRobinGlobalLoadBalancer(log);
+
+        // Ack ok start.
+        if (log.isDebugEnabled())
+            log.debug(startInfo());
+    }
+
+    /** {@inheritDoc} */
+    @Override public void spiStop() throws IgniteSpiException {
+        balancer = null;
+
+        perTaskBalancers.clear();
+
+        unregisterMBean();
+
+        // Ack ok stop.
+        if (log.isDebugEnabled())
+            log.debug(stopInfo());
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void onContextInitialized0(IgniteSpiContext spiCtx) throws IgniteSpiException {
+        if (!isPerTask)
+            balancer.onContextInitialized(spiCtx);
+        else {
+            if (!getSpiContext().isEventRecordable(EVT_TASK_FAILED, EVT_TASK_FINISHED, EVT_JOB_MAPPED))
+                throw new IgniteSpiException("Required event types are disabled: " +
+                    U.gridEventName(EVT_TASK_FAILED) + ", " +
+                    U.gridEventName(EVT_TASK_FINISHED) + ", " +
+                    U.gridEventName(EVT_JOB_MAPPED));
+
+            getSpiContext().addLocalEventListener(lsnr, EVT_TASK_FAILED, EVT_TASK_FINISHED, EVT_JOB_MAPPED);
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void onContextDestroyed0() {
+        if (!isPerTask) {
+            if (balancer != null)
+                balancer.onContextDestroyed();
+        }
+        else {
+            IgniteSpiContext spiCtx = getSpiContext();
+
+            if (spiCtx != null)
+                spiCtx.removeLocalEventListener(lsnr);
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public ClusterNode getBalancedNode(ComputeTaskSession ses, List<ClusterNode> top, ComputeJob job)
+        throws GridException {
+        A.notNull(ses, "ses", top, "top");
+
+        if (isPerTask) {
+            // Note that every session operates from single thread which
+            // allows us to use concurrent map and avoid synchronization.
+            RoundRobinPerTaskLoadBalancer taskBalancer = perTaskBalancers.get(ses.getId());
+
+            if (taskBalancer == null)
+                perTaskBalancers.put(ses.getId(), taskBalancer = new RoundRobinPerTaskLoadBalancer());
+
+            return taskBalancer.getBalancedNode(top);
+        }
+
+        return balancer.getBalancedNode(top);
+    }
+
+    /**
+     * THIS METHOD IS USED ONLY FOR TESTING.
+     *
+     * @param ses Task session.
+     * @return Internal list of nodes.
+     */
+    List<UUID> getNodeIds(ComputeTaskSession ses) {
+        if (isPerTask) {
+            RoundRobinPerTaskLoadBalancer balancer = perTaskBalancers.get(ses.getId());
+
+            if (balancer == null)
+                return Collections.emptyList();
+
+            List<UUID> ids = new ArrayList<>();
+
+            for (ClusterNode node : balancer.getNodes()) {
+                ids.add(node.id());
+            }
+
+            return ids;
+        }
+
+        return balancer.getNodeIds();
+    }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return S.toString(RoundRobinLoadBalancingSpi.class, this);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/a62862fe/modules/core/src/main/java/org/gridgain/grid/spi/loadbalancing/roundrobin/RoundRobinLoadBalancingSpiMBean.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/spi/loadbalancing/roundrobin/RoundRobinLoadBalancingSpiMBean.java b/modules/core/src/main/java/org/gridgain/grid/spi/loadbalancing/roundrobin/RoundRobinLoadBalancingSpiMBean.java
new file mode 100644
index 0000000..1189677
--- /dev/null
+++ b/modules/core/src/main/java/org/gridgain/grid/spi/loadbalancing/roundrobin/RoundRobinLoadBalancingSpiMBean.java
@@ -0,0 +1,37 @@
+/* @java.file.header */
+
+/*  _________        _____ __________________        _____
+ *  __  ____/___________(_)______  /__  ____/______ ____(_)_______
+ *  _  / __  __  ___/__  / _  __  / _  / __  _  __ `/__  / __  __ \
+ *  / /_/ /  _  /    _  /  / /_/ /  / /_/ /  / /_/ / _  /  _  / / /
+ *  \____/   /_/     /_/   \_,__/   \____/   \__,_/  /_/   /_/ /_/
+ */
+
+package org.gridgain.grid.spi.loadbalancing.roundrobin;
+
+import org.apache.ignite.mbean.*;
+import org.apache.ignite.spi.*;
+
+/**
+ * Management bean for {@link RoundRobinLoadBalancingSpi} SPI.
+ */
+@IgniteMBeanDescription("MBean that provides access to round robin load balancing SPI configuration.")
+public interface RoundRobinLoadBalancingSpiMBean extends IgniteSpiManagementMBean {
+    /**
+     * Configuration parameter indicating whether a new round robin order should be
+     * created for every task. If {@code true} then load balancer is guaranteed
+     * to iterate through nodes sequentially for every task - so as long as number
+     * of jobs is less than or equal to the number of nodes, jobs are guaranteed to
+     * be assigned to unique nodes. If {@code false} then one round-robin order
+     * will be maintained for all tasks, so when tasks execute concurrently, it
+     * is possible for more than one job within task to be assigned to the same
+     * node.
+     * <p>
+     * Default is {@code true}.
+     *
+     * @return Configuration parameter indicating whether a new round robin order should
+     *      be created for every task. Default is {@code true}.
+     */
+    @IgniteMBeanDescription("Configuration parameter indicating whether a new round robin order should be created for every task.")
+    public boolean isPerTask();
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/a62862fe/modules/core/src/main/java/org/gridgain/grid/spi/loadbalancing/roundrobin/RoundRobinPerTaskLoadBalancer.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/spi/loadbalancing/roundrobin/RoundRobinPerTaskLoadBalancer.java b/modules/core/src/main/java/org/gridgain/grid/spi/loadbalancing/roundrobin/RoundRobinPerTaskLoadBalancer.java
new file mode 100644
index 0000000..bf626b4
--- /dev/null
+++ b/modules/core/src/main/java/org/gridgain/grid/spi/loadbalancing/roundrobin/RoundRobinPerTaskLoadBalancer.java
@@ -0,0 +1,96 @@
+/* @java.file.header */
+
+/*  _________        _____ __________________        _____
+ *  __  ____/___________(_)______  /__  ____/______ ____(_)_______
+ *  _  / __  __  ___/__  / _  __  / _  / __  _  __ `/__  / __  __ \
+ *  / /_/ /  _  /    _  /  / /_/ /  / /_/ /  / /_/ / _  /  _  / / /
+ *  \____/   /_/     /_/   \_,__/   \____/   \__,_/  /_/   /_/ /_/
+ */
+
+package org.gridgain.grid.spi.loadbalancing.roundrobin;
+
+import org.apache.ignite.cluster.*;
+
+import java.util.*;
+
+/**
+ * Load balancer for per-task configuration.
+ */
+class RoundRobinPerTaskLoadBalancer {
+    /** Balancing nodes. */
+    private ArrayDeque<ClusterNode> nodeQueue;
+
+    /** Jobs mapped flag. */
+    private volatile boolean isMapped;
+
+    /** Mutex. */
+    private final Object mux = new Object();
+
+    /**
+     * Call back for job mapped event.
+     */
+    void onMapped() {
+        isMapped = true;
+    }
+
+    /**
+     * Gets balanced node for given topology. This implementation
+     * is to be used only from {@link org.apache.ignite.compute.ComputeTask#map(List, Object)} method
+     * and, therefore, does not need to be thread-safe.
+     *
+     * @param top Topology to pick from.
+     * @return Best balanced node.
+     */
+    ClusterNode getBalancedNode(List<ClusterNode> top) {
+        assert top != null;
+        assert !top.isEmpty();
+
+        boolean readjust = isMapped;
+
+        synchronized (mux) {
+            // Populate first time.
+            if (nodeQueue == null)
+                nodeQueue = new ArrayDeque<>(top);
+
+            // If job has been mapped, then it means
+            // that it is most likely being failed over.
+            // In this case topology might have changed
+            // and we need to readjust with every apply.
+            if (readjust)
+                // Add missing nodes.
+                for (ClusterNode node : top)
+                    if (!nodeQueue.contains(node))
+                        nodeQueue.offer(node);
+
+            ClusterNode next = nodeQueue.poll();
+
+            // If jobs have been mapped, we need to make sure
+            // that queued node is still in topology.
+            if (readjust && next != null) {
+                while (!top.contains(next) && !nodeQueue.isEmpty())
+                    next = nodeQueue.poll();
+
+                // No nodes found and queue is empty.
+                if (next != null && !top.contains(next))
+                    return null;
+            }
+
+            if (next != null)
+                // Add to the end.
+                nodeQueue.offer(next);
+
+            return next;
+        }
+    }
+
+    /**
+     * THIS METHOD IS USED ONLY FOR TESTING.
+     *
+     * @return Internal list of nodes.
+     */
+    List<ClusterNode> getNodes() {
+        synchronized (mux) {
+            return Collections.unmodifiableList(new ArrayList<>(nodeQueue));
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/a62862fe/modules/core/src/main/java/org/gridgain/grid/spi/loadbalancing/weightedrandom/GridWeightedRandomLoadBalancingSpi.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/spi/loadbalancing/weightedrandom/GridWeightedRandomLoadBalancingSpi.java b/modules/core/src/main/java/org/gridgain/grid/spi/loadbalancing/weightedrandom/GridWeightedRandomLoadBalancingSpi.java
deleted file mode 100644
index f166da4..0000000
--- a/modules/core/src/main/java/org/gridgain/grid/spi/loadbalancing/weightedrandom/GridWeightedRandomLoadBalancingSpi.java
+++ /dev/null
@@ -1,394 +0,0 @@
-/* @java.file.header */
-
-/*  _________        _____ __________________        _____
- *  __  ____/___________(_)______  /__  ____/______ ____(_)_______
- *  _  / __  __  ___/__  / _  __  / _  / __  _  __ `/__  / __  __ \
- *  / /_/ /  _  /    _  /  / /_/ /  / /_/ /  / /_/ / _  /  _  / / /
- *  \____/   /_/     /_/   \_,__/   \____/   \__,_/  /_/   /_/ /_/
- */
-
-package org.gridgain.grid.spi.loadbalancing.weightedrandom;
-
-import org.apache.ignite.*;
-import org.apache.ignite.cluster.*;
-import org.apache.ignite.compute.*;
-import org.apache.ignite.events.*;
-import org.apache.ignite.lang.*;
-import org.apache.ignite.resources.*;
-import org.apache.ignite.spi.*;
-import org.gridgain.grid.kernal.managers.eventstorage.*;
-import org.gridgain.grid.spi.loadbalancing.*;
-import org.gridgain.grid.util.typedef.*;
-import org.gridgain.grid.util.typedef.internal.*;
-import org.jdk8.backport.*;
-import org.jetbrains.annotations.*;
-
-import java.util.*;
-import java.util.concurrent.*;
-
-import static org.apache.ignite.events.IgniteEventType.*;
-
-/**
- * Load balancing SPI that picks a random node for job execution. Note that you can
- * optionally assign weights to nodes, so nodes with larger weights will end up getting
- * proportionally more jobs routed to them (see {@link #setNodeWeight(int)}
- * configuration property). By default all nodes get equal weight defined by
- * {@link #DFLT_NODE_WEIGHT} (value is {@code 10}).
- * <h1 class="header">Coding Example</h1>
- * If you are using {@link org.apache.ignite.compute.ComputeTaskSplitAdapter} then load balancing logic
- * is transparent to your code and is handled automatically by the adapter.
- * Here is an example of how your task could look:
- * <pre name="code" class="java">
- * public class MyFooBarTask extends GridComputeTaskSplitAdapter&lt;Object, Object&gt; {
- *    &#64;Override
- *    protected Collection&lt;? extends GridComputeJob&gt; split(int gridSize, Object arg) throws GridException {
- *        List&lt;MyFooBarJob&gt; jobs = new ArrayList&lt;MyFooBarJob&gt;(gridSize);
- *
- *        for (int i = 0; i &lt; gridSize; i++) {
- *            jobs.add(new MyFooBarJob(arg));
- *        }
- *
- *        // Node assignment via load balancer
- *        // happens automatically.
- *        return jobs;
- *    }
- *    ...
- * }
- * </pre>
- * If you need more fine-grained control over how some jobs within task get mapped to a node
- * and use affinity load balancing for some other jobs within task, then you should use
- * {@link org.apache.ignite.compute.ComputeTaskAdapter}. Here is an example of how your task will look. Note that in this
- * case we manually inject load balancer and use it to pick the best node. Doing it in
- * such way would allow user to map some jobs manually and for others use load balancer.
- * <pre name="code" class="java">
- * public class MyFooBarTask extends GridComputeTaskAdapter&lt;String, String&gt; {
- *    // Inject load balancer.
- *    &#64;GridLoadBalancerResource
- *    GridComputeLoadBalancer balancer;
- *
- *    // Map jobs to grid nodes.
- *    public Map&lt;? extends GridComputeJob, GridNode&gt; map(List&lt;GridNode&gt; subgrid, String arg) throws GridException {
- *        Map&lt;MyFooBarJob, GridNode&gt; jobs = new HashMap&lt;MyFooBarJob, GridNode&gt;(subgrid.size());
- *
- *        // In more complex cases, you can actually do
- *        // more complicated assignments of jobs to nodes.
- *        for (int i = 0; i &lt; subgrid.size(); i++) {
- *            // Pick the next best balanced node for the job.
- *            jobs.put(new MyFooBarJob(arg), balancer.getBalancedNode())
- *        }
- *
- *        return jobs;
- *    }
- *
- *    // Aggregate results into one compound result.
- *    public String reduce(List&lt;GridComputeJobResult&gt; results) throws GridException {
- *        // For the purpose of this example we simply
- *        // concatenate string representation of every
- *        // job result
- *        StringBuilder buf = new StringBuilder();
- *
- *        for (GridComputeJobResult res : results) {
- *            // Append string representation of result
- *            // returned by every job.
- *            buf.append(res.getData().string());
- *        }
- *
- *        return buf.string();
- *    }
- * }
- * </pre>
- * <p>
- * <h1 class="header">Configuration</h1>
- * In order to use this load balancer, you should configure your grid instance
- * to use {@code GridRandomLoadBalancingSpi} either from Spring XML file or
- * directly. The following configuration parameters are supported:
- * <h2 class="header">Mandatory</h2>
- * This SPI has no mandatory configuration parameters.
- * <h2 class="header">Optional</h2>
- * The following configuration parameters are optional:
- * <ul>
- * <li>
- *      Flag that indicates whether to use weight policy or simple random policy
- *      (see {@link #setUseWeights(boolean)})
- * </li>
- * <li>
- *      Weight of this node (see {@link #setNodeWeight(int)}). This parameter is ignored
- *      if {@link #setUseWeights(boolean)} is set to {@code false}.
- * </li>
- * </ul>
- * Below is Java configuration example:
- * <pre name="code" class="java">
- * GridWeightedRandomLoadBalancingSpi = new GridWeightedLoadBalancingSpi();
- *
- * // Configure SPI to used weighted
- * // random load balancing.
- * spi.setUseWeights(true);
- *
- * // Set weight for the local node.
- * spi.setWeight( *);
- *
- * GridConfiguration cfg = new GridConfiguration();
- *
- * // Override default load balancing SPI.
- * cfg.setLoadBalancingSpi(spi);
- *
- * // Starts grid.
- * G.start(cfg);
- * </pre>
- * Here is how you can configure {@code GridRandomLoadBalancingSpi} using Spring XML configuration:
- * <pre name="code" class="xml">
- * &lt;property name="loadBalancingSpi"&gt;
- *     &lt;bean class="org.gridgain.grid.spi.loadBalancing.weightedrandom.GridWeightedRandomLoadBalancingSpi"&gt;
- *         &lt;property name="useWeights" value="true"/&gt;
- *         &lt;property name="nodeWeight" value="10"/&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>
- */
-@IgniteSpiMultipleInstancesSupport(true)
-@IgniteSpiConsistencyChecked(optional = true)
-public class GridWeightedRandomLoadBalancingSpi extends IgniteSpiAdapter implements GridLoadBalancingSpi,
-    GridWeightedRandomLoadBalancingSpiMBean {
-    /** Random number generator. */
-    private static final Random RAND = new Random();
-
-    /**
-     * Name of node attribute used to indicate load weight of a node
-     * (value is {@code "gridgain.node.weight.attr.name"}).
-     *
-     * @see org.apache.ignite.cluster.ClusterNode#attributes()
-     */
-    public static final String NODE_WEIGHT_ATTR_NAME = "gridgain.node.weight.attr.name";
-
-    /** Default weight assigned to every node if explicit one is not provided (value is {@code 10}). */
-    public static final int DFLT_NODE_WEIGHT = 10;
-
-    /** Grid logger. */
-    @IgniteLoggerResource
-    private IgniteLogger log;
-
-    /** */
-    private boolean isUseWeights;
-
-    /** Local event listener to listen to task completion events. */
-    private GridLocalEventListener evtLsnr;
-
-    /** Weight of this node. */
-    private int nodeWeight = DFLT_NODE_WEIGHT;
-
-    /** Task topologies. First pair value indicates whether or not jobs have been mapped. */
-    private ConcurrentMap<IgniteUuid, IgniteBiTuple<Boolean, WeightedTopology>> taskTops =
-        new ConcurrentHashMap8<>();
-
-    /**
-     * Sets a flag to indicate whether node weights should be checked when
-     * doing random load balancing. Default value is {@code false} which
-     * means that node weights are disregarded for load balancing logic.
-     *
-     * @param isUseWeights If {@code true} then random load is distributed according
-     *      to node weights.
-     */
-    @IgniteSpiConfiguration(optional = true)
-    public void setUseWeights(boolean isUseWeights) {
-        this.isUseWeights = isUseWeights;
-    }
-
-    /** {@inheritDoc} */
-    @Override public boolean isUseWeights() {
-        return isUseWeights;
-    }
-
-    /**
-     * Sets weight of this node. Nodes with more processing capacity
-     * should be assigned proportionally larger weight. Default value
-     * is {@link #DFLT_NODE_WEIGHT} and is equal for all nodes.
-     *
-     * @param nodeWeight Weight of this node.
-     */
-    @IgniteSpiConfiguration(optional = true)
-    public void setNodeWeight(int nodeWeight) {
-        this.nodeWeight = nodeWeight;
-    }
-
-    /** {@inheritDoc} */
-    @Override public int getNodeWeight() {
-        return nodeWeight;
-    }
-
-    /** {@inheritDoc} */
-    @Override public Map<String, Object> getNodeAttributes() throws IgniteSpiException {
-        return F.<String, Object>asMap(createSpiAttributeName(NODE_WEIGHT_ATTR_NAME), nodeWeight);
-    }
-
-    /** {@inheritDoc} */
-    @Override public void spiStart(@Nullable String gridName) throws IgniteSpiException {
-        startStopwatch();
-
-        assertParameter(nodeWeight > 0, "nodeWeight > 0");
-
-        if (log.isDebugEnabled()) {
-            log.debug(configInfo("isUseWeights", isUseWeights));
-            log.debug(configInfo("nodeWeight", nodeWeight));
-        }
-
-        registerMBean(gridName, this, GridWeightedRandomLoadBalancingSpiMBean.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 protected void onContextInitialized0(IgniteSpiContext spiCtx) throws IgniteSpiException {
-        getSpiContext().addLocalEventListener(evtLsnr = new GridLocalEventListener() {
-            @Override public void onEvent(IgniteEvent evt) {
-                assert evt instanceof IgniteTaskEvent || evt instanceof IgniteJobEvent;
-
-                if (evt.type() == EVT_TASK_FINISHED ||
-                    evt.type() == EVT_TASK_FAILED) {
-                    IgniteUuid sesId = ((IgniteTaskEvent)evt).taskSessionId();
-
-                    taskTops.remove(sesId);
-
-                    if (log.isDebugEnabled())
-                        log.debug("Removed task topology from topology cache for session: " + sesId);
-                }
-                // We should keep topology and use cache in GridComputeTask#map() method to
-                // avoid O(n*n/2) complexity, after that we can drop caches.
-                // Here we set mapped property and later cache will be ignored
-                else if (evt.type() == EVT_JOB_MAPPED) {
-                    IgniteUuid sesId = ((IgniteJobEvent)evt).taskSessionId();
-
-                    IgniteBiTuple<Boolean, WeightedTopology> weightedTop = taskTops.get(sesId);
-
-                    if (weightedTop != null)
-                        weightedTop.set1(true);
-
-                    if (log.isDebugEnabled())
-                        log.debug("Job has been mapped. Ignore cache for session: " + sesId);
-                }
-            }
-        },
-            EVT_TASK_FAILED,
-            EVT_TASK_FINISHED,
-            EVT_JOB_MAPPED
-        );
-    }
-
-    /** {@inheritDoc} */
-    @Override protected void onContextDestroyed0() {
-        if (evtLsnr != null) {
-            IgniteSpiContext ctx = getSpiContext();
-
-            if (ctx != null)
-                ctx.removeLocalEventListener(evtLsnr);
-        }
-    }
-
-    /** {@inheritDoc} */
-    @Override public ClusterNode getBalancedNode(ComputeTaskSession ses, List<ClusterNode> top, ComputeJob job) {
-        A.notNull(ses, "ses");
-        A.notNull(top, "top");
-        A.notNull(job, "job");
-
-        // Optimization for non-weighted randomization.
-        if (!isUseWeights)
-            return top.get(RAND.nextInt(top.size()));
-
-        IgniteBiTuple<Boolean, WeightedTopology> weightedTop = taskTops.get(ses.getId());
-
-        // Create new cached topology if there is no one. Do not
-        // use cached topology after task has been mapped.
-        if (weightedTop == null) {
-            // Called from GridComputeTask#map(). Put new topology and false as not mapped yet.
-            taskTops.put(ses.getId(), weightedTop = F.t(false, new WeightedTopology(top)));
-        }
-        // We have topology - check if task has been mapped.
-        else if (weightedTop.get1()) {
-            // Do not use cache after GridComputeTask#map().
-            return new WeightedTopology(top).pickWeightedNode();
-        }
-
-        return weightedTop.get2().pickWeightedNode();
-    }
-
-    /**
-     * @param node Node to get weight for.
-     * @return Node weight
-     */
-    private int getWeight(ClusterNode node) {
-        Integer weight = (Integer)node.attribute(createSpiAttributeName(NODE_WEIGHT_ATTR_NAME));
-
-        if (weight != null && weight == 0)
-            throw new IllegalStateException("Node weight cannot be zero: " + node);
-
-        return weight == null ? DFLT_NODE_WEIGHT : weight;
-    }
-
-    /**
-     * Holder for weighted topology.
-     */
-    private class WeightedTopology {
-        /** Total topology weight. */
-        private final int totalWeight;
-
-        /** Topology sorted by weight. */
-        private final SortedMap<Integer, ClusterNode> circle = new TreeMap<>();
-
-        /**
-         * @param top Topology.
-         */
-        WeightedTopology(Collection<ClusterNode> top) {
-            assert !F.isEmpty(top);
-
-            int totalWeight = 0;
-
-            for (ClusterNode node : top) {
-                totalWeight += getWeight(node);
-
-                // Complexity of this put is O(logN).
-                circle.put(totalWeight, node);
-            }
-
-            this.totalWeight = totalWeight;
-        }
-
-        /**
-         * Gets weighted node in random fashion.
-         *
-         * @return Weighted node.
-         */
-        ClusterNode pickWeightedNode() {
-            int weight = RAND.nextInt(totalWeight) + 1;
-
-            SortedMap<Integer, ClusterNode> pick = circle.tailMap(weight);
-
-            assert !pick.isEmpty();
-
-            return pick.get(pick.firstKey());
-        }
-    }
-
-    /** {@inheritDoc} */
-    @Override protected List<String> getConsistentAttributeNames() {
-        return Collections.singletonList(createSpiAttributeName(NODE_WEIGHT_ATTR_NAME));
-    }
-
-    /** {@inheritDoc} */
-    @Override public String toString() {
-        return S.toString(GridWeightedRandomLoadBalancingSpi.class, this);
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/a62862fe/modules/core/src/main/java/org/gridgain/grid/spi/loadbalancing/weightedrandom/GridWeightedRandomLoadBalancingSpiMBean.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/spi/loadbalancing/weightedrandom/GridWeightedRandomLoadBalancingSpiMBean.java b/modules/core/src/main/java/org/gridgain/grid/spi/loadbalancing/weightedrandom/GridWeightedRandomLoadBalancingSpiMBean.java
deleted file mode 100644
index 2dfe769..0000000
--- a/modules/core/src/main/java/org/gridgain/grid/spi/loadbalancing/weightedrandom/GridWeightedRandomLoadBalancingSpiMBean.java
+++ /dev/null
@@ -1,37 +0,0 @@
-/* @java.file.header */
-
-/*  _________        _____ __________________        _____
- *  __  ____/___________(_)______  /__  ____/______ ____(_)_______
- *  _  / __  __  ___/__  / _  __  / _  / __  _  __ `/__  / __  __ \
- *  / /_/ /  _  /    _  /  / /_/ /  / /_/ /  / /_/ / _  /  _  / / /
- *  \____/   /_/     /_/   \_,__/   \____/   \__,_/  /_/   /_/ /_/
- */
-
-package org.gridgain.grid.spi.loadbalancing.weightedrandom;
-
-import org.apache.ignite.mbean.*;
-import org.apache.ignite.spi.*;
-
-/**
- * Management MBean for {@link GridWeightedRandomLoadBalancingSpi} SPI.
- */
-@IgniteMBeanDescription("MBean that provides access to weighted random load balancing SPI configuration.")
-public interface GridWeightedRandomLoadBalancingSpiMBean extends IgniteSpiManagementMBean {
-    /**
-     * Checks whether node weights are considered when doing
-     * random load balancing.
-     *
-     * @return If {@code true} then random load is distributed according
-     *      to node weights.
-     */
-    @IgniteMBeanDescription("Whether node weights are considered when doing random load balancing.")
-    public boolean isUseWeights();
-
-    /**
-     * Gets weight of this node.
-     *
-     * @return Weight of this node.
-     */
-    @IgniteMBeanDescription("Weight of this node.")
-    public int getNodeWeight();
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/a62862fe/modules/core/src/main/java/org/gridgain/grid/spi/loadbalancing/weightedrandom/WeightedRandomLoadBalancingSpi.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/spi/loadbalancing/weightedrandom/WeightedRandomLoadBalancingSpi.java b/modules/core/src/main/java/org/gridgain/grid/spi/loadbalancing/weightedrandom/WeightedRandomLoadBalancingSpi.java
new file mode 100644
index 0000000..0590d60
--- /dev/null
+++ b/modules/core/src/main/java/org/gridgain/grid/spi/loadbalancing/weightedrandom/WeightedRandomLoadBalancingSpi.java
@@ -0,0 +1,394 @@
+/* @java.file.header */
+
+/*  _________        _____ __________________        _____
+ *  __  ____/___________(_)______  /__  ____/______ ____(_)_______
+ *  _  / __  __  ___/__  / _  __  / _  / __  _  __ `/__  / __  __ \
+ *  / /_/ /  _  /    _  /  / /_/ /  / /_/ /  / /_/ / _  /  _  / / /
+ *  \____/   /_/     /_/   \_,__/   \____/   \__,_/  /_/   /_/ /_/
+ */
+
+package org.gridgain.grid.spi.loadbalancing.weightedrandom;
+
+import org.apache.ignite.*;
+import org.apache.ignite.cluster.*;
+import org.apache.ignite.compute.*;
+import org.apache.ignite.events.*;
+import org.apache.ignite.lang.*;
+import org.apache.ignite.resources.*;
+import org.apache.ignite.spi.*;
+import org.gridgain.grid.kernal.managers.eventstorage.*;
+import org.gridgain.grid.spi.loadbalancing.*;
+import org.gridgain.grid.util.typedef.*;
+import org.gridgain.grid.util.typedef.internal.*;
+import org.jdk8.backport.*;
+import org.jetbrains.annotations.*;
+
+import java.util.*;
+import java.util.concurrent.*;
+
+import static org.apache.ignite.events.IgniteEventType.*;
+
+/**
+ * Load balancing SPI that picks a random node for job execution. Note that you can
+ * optionally assign weights to nodes, so nodes with larger weights will end up getting
+ * proportionally more jobs routed to them (see {@link #setNodeWeight(int)}
+ * configuration property). By default all nodes get equal weight defined by
+ * {@link #DFLT_NODE_WEIGHT} (value is {@code 10}).
+ * <h1 class="header">Coding Example</h1>
+ * If you are using {@link org.apache.ignite.compute.ComputeTaskSplitAdapter} then load balancing logic
+ * is transparent to your code and is handled automatically by the adapter.
+ * Here is an example of how your task could look:
+ * <pre name="code" class="java">
+ * public class MyFooBarTask extends GridComputeTaskSplitAdapter&lt;Object, Object&gt; {
+ *    &#64;Override
+ *    protected Collection&lt;? extends GridComputeJob&gt; split(int gridSize, Object arg) throws GridException {
+ *        List&lt;MyFooBarJob&gt; jobs = new ArrayList&lt;MyFooBarJob&gt;(gridSize);
+ *
+ *        for (int i = 0; i &lt; gridSize; i++) {
+ *            jobs.add(new MyFooBarJob(arg));
+ *        }
+ *
+ *        // Node assignment via load balancer
+ *        // happens automatically.
+ *        return jobs;
+ *    }
+ *    ...
+ * }
+ * </pre>
+ * If you need more fine-grained control over how some jobs within task get mapped to a node
+ * and use affinity load balancing for some other jobs within task, then you should use
+ * {@link org.apache.ignite.compute.ComputeTaskAdapter}. Here is an example of how your task will look. Note that in this
+ * case we manually inject load balancer and use it to pick the best node. Doing it in
+ * such way would allow user to map some jobs manually and for others use load balancer.
+ * <pre name="code" class="java">
+ * public class MyFooBarTask extends GridComputeTaskAdapter&lt;String, String&gt; {
+ *    // Inject load balancer.
+ *    &#64;GridLoadBalancerResource
+ *    GridComputeLoadBalancer balancer;
+ *
+ *    // Map jobs to grid nodes.
+ *    public Map&lt;? extends GridComputeJob, GridNode&gt; map(List&lt;GridNode&gt; subgrid, String arg) throws GridException {
+ *        Map&lt;MyFooBarJob, GridNode&gt; jobs = new HashMap&lt;MyFooBarJob, GridNode&gt;(subgrid.size());
+ *
+ *        // In more complex cases, you can actually do
+ *        // more complicated assignments of jobs to nodes.
+ *        for (int i = 0; i &lt; subgrid.size(); i++) {
+ *            // Pick the next best balanced node for the job.
+ *            jobs.put(new MyFooBarJob(arg), balancer.getBalancedNode())
+ *        }
+ *
+ *        return jobs;
+ *    }
+ *
+ *    // Aggregate results into one compound result.
+ *    public String reduce(List&lt;GridComputeJobResult&gt; results) throws GridException {
+ *        // For the purpose of this example we simply
+ *        // concatenate string representation of every
+ *        // job result
+ *        StringBuilder buf = new StringBuilder();
+ *
+ *        for (GridComputeJobResult res : results) {
+ *            // Append string representation of result
+ *            // returned by every job.
+ *            buf.append(res.getData().string());
+ *        }
+ *
+ *        return buf.string();
+ *    }
+ * }
+ * </pre>
+ * <p>
+ * <h1 class="header">Configuration</h1>
+ * In order to use this load balancer, you should configure your grid instance
+ * to use {@code GridRandomLoadBalancingSpi} either from Spring XML file or
+ * directly. The following configuration parameters are supported:
+ * <h2 class="header">Mandatory</h2>
+ * This SPI has no mandatory configuration parameters.
+ * <h2 class="header">Optional</h2>
+ * The following configuration parameters are optional:
+ * <ul>
+ * <li>
+ *      Flag that indicates whether to use weight policy or simple random policy
+ *      (see {@link #setUseWeights(boolean)})
+ * </li>
+ * <li>
+ *      Weight of this node (see {@link #setNodeWeight(int)}). This parameter is ignored
+ *      if {@link #setUseWeights(boolean)} is set to {@code false}.
+ * </li>
+ * </ul>
+ * Below is Java configuration example:
+ * <pre name="code" class="java">
+ * GridWeightedRandomLoadBalancingSpi = new GridWeightedLoadBalancingSpi();
+ *
+ * // Configure SPI to used weighted
+ * // random load balancing.
+ * spi.setUseWeights(true);
+ *
+ * // Set weight for the local node.
+ * spi.setWeight( *);
+ *
+ * GridConfiguration cfg = new GridConfiguration();
+ *
+ * // Override default load balancing SPI.
+ * cfg.setLoadBalancingSpi(spi);
+ *
+ * // Starts grid.
+ * G.start(cfg);
+ * </pre>
+ * Here is how you can configure {@code GridRandomLoadBalancingSpi} using Spring XML configuration:
+ * <pre name="code" class="xml">
+ * &lt;property name="loadBalancingSpi"&gt;
+ *     &lt;bean class="org.gridgain.grid.spi.loadBalancing.weightedrandom.GridWeightedRandomLoadBalancingSpi"&gt;
+ *         &lt;property name="useWeights" value="true"/&gt;
+ *         &lt;property name="nodeWeight" value="10"/&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>
+ */
+@IgniteSpiMultipleInstancesSupport(true)
+@IgniteSpiConsistencyChecked(optional = true)
+public class WeightedRandomLoadBalancingSpi extends IgniteSpiAdapter implements LoadBalancingSpi,
+    WeightedRandomLoadBalancingSpiMBean {
+    /** Random number generator. */
+    private static final Random RAND = new Random();
+
+    /**
+     * Name of node attribute used to indicate load weight of a node
+     * (value is {@code "gridgain.node.weight.attr.name"}).
+     *
+     * @see org.apache.ignite.cluster.ClusterNode#attributes()
+     */
+    public static final String NODE_WEIGHT_ATTR_NAME = "gridgain.node.weight.attr.name";
+
+    /** Default weight assigned to every node if explicit one is not provided (value is {@code 10}). */
+    public static final int DFLT_NODE_WEIGHT = 10;
+
+    /** Grid logger. */
+    @IgniteLoggerResource
+    private IgniteLogger log;
+
+    /** */
+    private boolean isUseWeights;
+
+    /** Local event listener to listen to task completion events. */
+    private GridLocalEventListener evtLsnr;
+
+    /** Weight of this node. */
+    private int nodeWeight = DFLT_NODE_WEIGHT;
+
+    /** Task topologies. First pair value indicates whether or not jobs have been mapped. */
+    private ConcurrentMap<IgniteUuid, IgniteBiTuple<Boolean, WeightedTopology>> taskTops =
+        new ConcurrentHashMap8<>();
+
+    /**
+     * Sets a flag to indicate whether node weights should be checked when
+     * doing random load balancing. Default value is {@code false} which
+     * means that node weights are disregarded for load balancing logic.
+     *
+     * @param isUseWeights If {@code true} then random load is distributed according
+     *      to node weights.
+     */
+    @IgniteSpiConfiguration(optional = true)
+    public void setUseWeights(boolean isUseWeights) {
+        this.isUseWeights = isUseWeights;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean isUseWeights() {
+        return isUseWeights;
+    }
+
+    /**
+     * Sets weight of this node. Nodes with more processing capacity
+     * should be assigned proportionally larger weight. Default value
+     * is {@link #DFLT_NODE_WEIGHT} and is equal for all nodes.
+     *
+     * @param nodeWeight Weight of this node.
+     */
+    @IgniteSpiConfiguration(optional = true)
+    public void setNodeWeight(int nodeWeight) {
+        this.nodeWeight = nodeWeight;
+    }
+
+    /** {@inheritDoc} */
+    @Override public int getNodeWeight() {
+        return nodeWeight;
+    }
+
+    /** {@inheritDoc} */
+    @Override public Map<String, Object> getNodeAttributes() throws IgniteSpiException {
+        return F.<String, Object>asMap(createSpiAttributeName(NODE_WEIGHT_ATTR_NAME), nodeWeight);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void spiStart(@Nullable String gridName) throws IgniteSpiException {
+        startStopwatch();
+
+        assertParameter(nodeWeight > 0, "nodeWeight > 0");
+
+        if (log.isDebugEnabled()) {
+            log.debug(configInfo("isUseWeights", isUseWeights));
+            log.debug(configInfo("nodeWeight", nodeWeight));
+        }
+
+        registerMBean(gridName, this, WeightedRandomLoadBalancingSpiMBean.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 protected void onContextInitialized0(IgniteSpiContext spiCtx) throws IgniteSpiException {
+        getSpiContext().addLocalEventListener(evtLsnr = new GridLocalEventListener() {
+            @Override public void onEvent(IgniteEvent evt) {
+                assert evt instanceof IgniteTaskEvent || evt instanceof IgniteJobEvent;
+
+                if (evt.type() == EVT_TASK_FINISHED ||
+                    evt.type() == EVT_TASK_FAILED) {
+                    IgniteUuid sesId = ((IgniteTaskEvent)evt).taskSessionId();
+
+                    taskTops.remove(sesId);
+
+                    if (log.isDebugEnabled())
+                        log.debug("Removed task topology from topology cache for session: " + sesId);
+                }
+                // We should keep topology and use cache in GridComputeTask#map() method to
+                // avoid O(n*n/2) complexity, after that we can drop caches.
+                // Here we set mapped property and later cache will be ignored
+                else if (evt.type() == EVT_JOB_MAPPED) {
+                    IgniteUuid sesId = ((IgniteJobEvent)evt).taskSessionId();
+
+                    IgniteBiTuple<Boolean, WeightedTopology> weightedTop = taskTops.get(sesId);
+
+                    if (weightedTop != null)
+                        weightedTop.set1(true);
+
+                    if (log.isDebugEnabled())
+                        log.debug("Job has been mapped. Ignore cache for session: " + sesId);
+                }
+            }
+        },
+            EVT_TASK_FAILED,
+            EVT_TASK_FINISHED,
+            EVT_JOB_MAPPED
+        );
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void onContextDestroyed0() {
+        if (evtLsnr != null) {
+            IgniteSpiContext ctx = getSpiContext();
+
+            if (ctx != null)
+                ctx.removeLocalEventListener(evtLsnr);
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public ClusterNode getBalancedNode(ComputeTaskSession ses, List<ClusterNode> top, ComputeJob job) {
+        A.notNull(ses, "ses");
+        A.notNull(top, "top");
+        A.notNull(job, "job");
+
+        // Optimization for non-weighted randomization.
+        if (!isUseWeights)
+            return top.get(RAND.nextInt(top.size()));
+
+        IgniteBiTuple<Boolean, WeightedTopology> weightedTop = taskTops.get(ses.getId());
+
+        // Create new cached topology if there is no one. Do not
+        // use cached topology after task has been mapped.
+        if (weightedTop == null) {
+            // Called from GridComputeTask#map(). Put new topology and false as not mapped yet.
+            taskTops.put(ses.getId(), weightedTop = F.t(false, new WeightedTopology(top)));
+        }
+        // We have topology - check if task has been mapped.
+        else if (weightedTop.get1()) {
+            // Do not use cache after GridComputeTask#map().
+            return new WeightedTopology(top).pickWeightedNode();
+        }
+
+        return weightedTop.get2().pickWeightedNode();
+    }
+
+    /**
+     * @param node Node to get weight for.
+     * @return Node weight
+     */
+    private int getWeight(ClusterNode node) {
+        Integer weight = (Integer)node.attribute(createSpiAttributeName(NODE_WEIGHT_ATTR_NAME));
+
+        if (weight != null && weight == 0)
+            throw new IllegalStateException("Node weight cannot be zero: " + node);
+
+        return weight == null ? DFLT_NODE_WEIGHT : weight;
+    }
+
+    /**
+     * Holder for weighted topology.
+     */
+    private class WeightedTopology {
+        /** Total topology weight. */
+        private final int totalWeight;
+
+        /** Topology sorted by weight. */
+        private final SortedMap<Integer, ClusterNode> circle = new TreeMap<>();
+
+        /**
+         * @param top Topology.
+         */
+        WeightedTopology(Collection<ClusterNode> top) {
+            assert !F.isEmpty(top);
+
+            int totalWeight = 0;
+
+            for (ClusterNode node : top) {
+                totalWeight += getWeight(node);
+
+                // Complexity of this put is O(logN).
+                circle.put(totalWeight, node);
+            }
+
+            this.totalWeight = totalWeight;
+        }
+
+        /**
+         * Gets weighted node in random fashion.
+         *
+         * @return Weighted node.
+         */
+        ClusterNode pickWeightedNode() {
+            int weight = RAND.nextInt(totalWeight) + 1;
+
+            SortedMap<Integer, ClusterNode> pick = circle.tailMap(weight);
+
+            assert !pick.isEmpty();
+
+            return pick.get(pick.firstKey());
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override protected List<String> getConsistentAttributeNames() {
+        return Collections.singletonList(createSpiAttributeName(NODE_WEIGHT_ATTR_NAME));
+    }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return S.toString(WeightedRandomLoadBalancingSpi.class, this);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/a62862fe/modules/core/src/main/java/org/gridgain/grid/spi/loadbalancing/weightedrandom/WeightedRandomLoadBalancingSpiMBean.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/spi/loadbalancing/weightedrandom/WeightedRandomLoadBalancingSpiMBean.java b/modules/core/src/main/java/org/gridgain/grid/spi/loadbalancing/weightedrandom/WeightedRandomLoadBalancingSpiMBean.java
new file mode 100644
index 0000000..c4daa9e
--- /dev/null
+++ b/modules/core/src/main/java/org/gridgain/grid/spi/loadbalancing/weightedrandom/WeightedRandomLoadBalancingSpiMBean.java
@@ -0,0 +1,37 @@
+/* @java.file.header */
+
+/*  _________        _____ __________________        _____
+ *  __  ____/___________(_)______  /__  ____/______ ____(_)_______
+ *  _  / __  __  ___/__  / _  __  / _  / __  _  __ `/__  / __  __ \
+ *  / /_/ /  _  /    _  /  / /_/ /  / /_/ /  / /_/ / _  /  _  / / /
+ *  \____/   /_/     /_/   \_,__/   \____/   \__,_/  /_/   /_/ /_/
+ */
+
+package org.gridgain.grid.spi.loadbalancing.weightedrandom;
+
+import org.apache.ignite.mbean.*;
+import org.apache.ignite.spi.*;
+
+/**
+ * Management MBean for {@link WeightedRandomLoadBalancingSpi} SPI.
+ */
+@IgniteMBeanDescription("MBean that provides access to weighted random load balancing SPI configuration.")
+public interface WeightedRandomLoadBalancingSpiMBean extends IgniteSpiManagementMBean {
+    /**
+     * Checks whether node weights are considered when doing
+     * random load balancing.
+     *
+     * @return If {@code true} then random load is distributed according
+     *      to node weights.
+     */
+    @IgniteMBeanDescription("Whether node weights are considered when doing random load balancing.")
+    public boolean isUseWeights();
+
+    /**
+     * Gets weight of this node.
+     *
+     * @return Weight of this node.
+     */
+    @IgniteMBeanDescription("Weight of this node.")
+    public int getNodeWeight();
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/a62862fe/modules/core/src/test/config/io-manager-benchmark.xml
----------------------------------------------------------------------
diff --git a/modules/core/src/test/config/io-manager-benchmark.xml b/modules/core/src/test/config/io-manager-benchmark.xml
index 67e2b37..01dc6e2 100644
--- a/modules/core/src/test/config/io-manager-benchmark.xml
+++ b/modules/core/src/test/config/io-manager-benchmark.xml
@@ -50,7 +50,7 @@
 
         <!-- Configure load balancing SPI in the way that do not require extra event subscription. -->
         <property name="loadBalancingSpi">
-            <bean class="org.gridgain.grid.spi.loadbalancing.roundrobin.GridRoundRobinLoadBalancingSpi">
+            <bean class="org.gridgain.grid.spi.loadbalancing.roundrobin.RoundRobinLoadBalancingSpi">
                 <property name="perTask" value="false"/>
             </bean>
         </property>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/a62862fe/modules/core/src/test/config/jobs-load-base.xml
----------------------------------------------------------------------
diff --git a/modules/core/src/test/config/jobs-load-base.xml b/modules/core/src/test/config/jobs-load-base.xml
index d61356d..97325a2 100644
--- a/modules/core/src/test/config/jobs-load-base.xml
+++ b/modules/core/src/test/config/jobs-load-base.xml
@@ -97,7 +97,7 @@
         </property>
 
         <property name="loadBalancingSpi">
-            <bean class="org.gridgain.grid.spi.loadbalancing.roundrobin.GridRoundRobinLoadBalancingSpi">
+            <bean class="org.gridgain.grid.spi.loadbalancing.roundrobin.RoundRobinLoadBalancingSpi">
                 <property name="perTask" value="false"/>
             </bean>
         </property>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/a62862fe/modules/core/src/test/config/load/merge-sort-base.xml
----------------------------------------------------------------------
diff --git a/modules/core/src/test/config/load/merge-sort-base.xml b/modules/core/src/test/config/load/merge-sort-base.xml
index 11a3166..e2293f6 100644
--- a/modules/core/src/test/config/load/merge-sort-base.xml
+++ b/modules/core/src/test/config/load/merge-sort-base.xml
@@ -128,7 +128,7 @@
         </property>
 
         <property name="loadBalancingSpi">
-            <bean class="org.gridgain.grid.spi.loadbalancing.roundrobin.GridRoundRobinLoadBalancingSpi">
+            <bean class="org.gridgain.grid.spi.loadbalancing.roundrobin.RoundRobinLoadBalancingSpi">
                 <property name="perTask" value="false"/>
             </bean>
         </property>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/a62862fe/modules/core/src/test/config/spring-cache-put-remove-load.xml
----------------------------------------------------------------------
diff --git a/modules/core/src/test/config/spring-cache-put-remove-load.xml b/modules/core/src/test/config/spring-cache-put-remove-load.xml
index 103f519..052da54 100644
--- a/modules/core/src/test/config/spring-cache-put-remove-load.xml
+++ b/modules/core/src/test/config/spring-cache-put-remove-load.xml
@@ -42,7 +42,7 @@
         </property>
 
         <property name="loadBalancingSpi">
-            <bean class="org.gridgain.grid.spi.loadbalancing.roundrobin.GridRoundRobinLoadBalancingSpi">
+            <bean class="org.gridgain.grid.spi.loadbalancing.roundrobin.RoundRobinLoadBalancingSpi">
                 <property name="perTask" value="false"/>
             </bean>
         </property>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/a62862fe/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 33b1086..ba5e3b0 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
@@ -153,7 +153,7 @@ public class GridMultipleSpisSelfTest extends GridCommonAbstractTest {
     }
 
     /** */
-    private class GridTestLoadBalancingSpi extends GridRoundRobinLoadBalancingSpi {
+    private class GridTestLoadBalancingSpi extends RoundRobinLoadBalancingSpi {
         /** */
         private String expName;
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/a62862fe/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 1776054..d682e4e 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
@@ -187,7 +187,7 @@ public class GridManagerStopSelfTest extends GridCommonAbstractTest {
      * @throws Exception If failed.
      */
     public void testStopLoadBalancingManager() throws Exception {
-        GridRoundRobinLoadBalancingSpi spi = new GridRoundRobinLoadBalancingSpi();
+        RoundRobinLoadBalancingSpi spi = new RoundRobinLoadBalancingSpi();
 
         injectLogger(spi);
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/a62862fe/modules/core/src/test/java/org/gridgain/grid/spi/loadbalancing/adaptive/GridAdaptiveLoadBalancingSpiConfigSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/gridgain/grid/spi/loadbalancing/adaptive/GridAdaptiveLoadBalancingSpiConfigSelfTest.java b/modules/core/src/test/java/org/gridgain/grid/spi/loadbalancing/adaptive/GridAdaptiveLoadBalancingSpiConfigSelfTest.java
index 210c4df..8e080e8 100644
--- a/modules/core/src/test/java/org/gridgain/grid/spi/loadbalancing/adaptive/GridAdaptiveLoadBalancingSpiConfigSelfTest.java
+++ b/modules/core/src/test/java/org/gridgain/grid/spi/loadbalancing/adaptive/GridAdaptiveLoadBalancingSpiConfigSelfTest.java
@@ -14,13 +14,13 @@ import org.gridgain.testframework.junits.spi.*;
 /**
  *
  */
-@GridSpiTest(spi = GridAdaptiveLoadBalancingSpi.class, group = "LoadBalancing SPI")
+@GridSpiTest(spi = AdaptiveLoadBalancingSpi.class, group = "LoadBalancing SPI")
 public class GridAdaptiveLoadBalancingSpiConfigSelfTest
-    extends GridSpiAbstractConfigTest<GridAdaptiveLoadBalancingSpi> {
+    extends GridSpiAbstractConfigTest<AdaptiveLoadBalancingSpi> {
     /**
      * @throws Exception If failed.
      */
     public void testNegativeConfig() throws Exception {
-        checkNegativeSpiProperty(new GridAdaptiveLoadBalancingSpi(), "loadProbe", null);
+        checkNegativeSpiProperty(new AdaptiveLoadBalancingSpi(), "loadProbe", null);
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/a62862fe/modules/core/src/test/java/org/gridgain/grid/spi/loadbalancing/adaptive/GridAdaptiveLoadBalancingSpiMultipleNodeSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/gridgain/grid/spi/loadbalancing/adaptive/GridAdaptiveLoadBalancingSpiMultipleNodeSelfTest.java b/modules/core/src/test/java/org/gridgain/grid/spi/loadbalancing/adaptive/GridAdaptiveLoadBalancingSpiMultipleNodeSelfTest.java
index fb9d3ff..33e1a18 100644
--- a/modules/core/src/test/java/org/gridgain/grid/spi/loadbalancing/adaptive/GridAdaptiveLoadBalancingSpiMultipleNodeSelfTest.java
+++ b/modules/core/src/test/java/org/gridgain/grid/spi/loadbalancing/adaptive/GridAdaptiveLoadBalancingSpiMultipleNodeSelfTest.java
@@ -19,8 +19,8 @@ import java.util.*;
 /**
  * Tests adaptive load balancing SPI.
  */
-@GridSpiTest(spi = GridAdaptiveLoadBalancingSpi.class, group = "Load Balancing SPI")
-public class GridAdaptiveLoadBalancingSpiMultipleNodeSelfTest extends GridSpiAbstractTest<GridAdaptiveLoadBalancingSpi> {
+@GridSpiTest(spi = AdaptiveLoadBalancingSpi.class, group = "Load Balancing SPI")
+public class GridAdaptiveLoadBalancingSpiMultipleNodeSelfTest extends GridSpiAbstractTest<AdaptiveLoadBalancingSpi> {
     /** */
     private static final int RMT_NODE_CNT = 10;
 
@@ -43,8 +43,8 @@ public class GridAdaptiveLoadBalancingSpiMultipleNodeSelfTest extends GridSpiAbs
      * @return {@code True} if node weights should be considered.
      */
     @GridSpiTestConfig
-    public GridAdaptiveLoadProbe getLoadProbe() {
-        return new GridAdaptiveLoadProbe() {
+    public AdaptiveLoadProbe getLoadProbe() {
+        return new AdaptiveLoadProbe() {
             @Override public double getLoad(ClusterNode node, int jobsSentSinceLastUpdate) {
                 boolean isFirstTime = node.attribute("used") == null;
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/a62862fe/modules/core/src/test/java/org/gridgain/grid/spi/loadbalancing/adaptive/GridAdaptiveLoadBalancingSpiSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/gridgain/grid/spi/loadbalancing/adaptive/GridAdaptiveLoadBalancingSpiSelfTest.java b/modules/core/src/test/java/org/gridgain/grid/spi/loadbalancing/adaptive/GridAdaptiveLoadBalancingSpiSelfTest.java
index b3e7ded..8a3a2c5 100644
--- a/modules/core/src/test/java/org/gridgain/grid/spi/loadbalancing/adaptive/GridAdaptiveLoadBalancingSpiSelfTest.java
+++ b/modules/core/src/test/java/org/gridgain/grid/spi/loadbalancing/adaptive/GridAdaptiveLoadBalancingSpiSelfTest.java
@@ -21,8 +21,8 @@ import java.util.*;
 /**
  * Tests adaptive load balancing SPI.
  */
-@GridSpiTest(spi = GridAdaptiveLoadBalancingSpi.class, group = "Load Balancing SPI")
-public class GridAdaptiveLoadBalancingSpiSelfTest extends GridSpiAbstractTest<GridAdaptiveLoadBalancingSpi> {
+@GridSpiTest(spi = AdaptiveLoadBalancingSpi.class, group = "Load Balancing SPI")
+public class GridAdaptiveLoadBalancingSpiSelfTest extends GridSpiAbstractTest<AdaptiveLoadBalancingSpi> {
     /** {@inheritDoc} */
     @Override protected GridSpiTestContext initSpiContext() throws Exception {
         GridSpiTestContext ctx = super.initSpiContext();
@@ -36,8 +36,8 @@ public class GridAdaptiveLoadBalancingSpiSelfTest extends GridSpiAbstractTest<Gr
      * @return {@code True} if node weights should be considered.
      */
     @GridSpiTestConfig
-    public GridAdaptiveLoadProbe getLoadProbe() {
-        return new GridAdaptiveLoadProbe() {
+    public AdaptiveLoadProbe getLoadProbe() {
+        return new AdaptiveLoadProbe() {
             @Override public double getLoad(ClusterNode node, int jobsSentSinceLastUpdate) {
                 boolean isFirstTime = node.attribute("used") == null;
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/a62862fe/modules/core/src/test/java/org/gridgain/grid/spi/loadbalancing/adaptive/GridAdaptiveLoadBalancingSpiStartStopSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/gridgain/grid/spi/loadbalancing/adaptive/GridAdaptiveLoadBalancingSpiStartStopSelfTest.java b/modules/core/src/test/java/org/gridgain/grid/spi/loadbalancing/adaptive/GridAdaptiveLoadBalancingSpiStartStopSelfTest.java
index 5ffa8f8..19876e7 100644
--- a/modules/core/src/test/java/org/gridgain/grid/spi/loadbalancing/adaptive/GridAdaptiveLoadBalancingSpiStartStopSelfTest.java
+++ b/modules/core/src/test/java/org/gridgain/grid/spi/loadbalancing/adaptive/GridAdaptiveLoadBalancingSpiStartStopSelfTest.java
@@ -16,8 +16,8 @@ import org.gridgain.testframework.junits.spi.*;
  * Adaptive load balancing SPI start-stop test.
  */
 @SuppressWarnings({"JUnitTestCaseWithNoTests"})
-@GridSpiTest(spi = GridAdaptiveLoadBalancingSpi.class, group = "LoadBalancing SPI")
+@GridSpiTest(spi = AdaptiveLoadBalancingSpi.class, group = "LoadBalancing SPI")
 public class GridAdaptiveLoadBalancingSpiStartStopSelfTest extends
-    GridSpiStartStopAbstractTest<GridAdaptiveLoadBalancingSpi> {
+    GridSpiStartStopAbstractTest<AdaptiveLoadBalancingSpi> {
     // No configs.
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/a62862fe/modules/core/src/test/java/org/gridgain/grid/spi/loadbalancing/roundrobin/GridRoundRobinLoadBalancingNotPerTaskMultithreadedSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/gridgain/grid/spi/loadbalancing/roundrobin/GridRoundRobinLoadBalancingNotPerTaskMultithreadedSelfTest.java b/modules/core/src/test/java/org/gridgain/grid/spi/loadbalancing/roundrobin/GridRoundRobinLoadBalancingNotPerTaskMultithreadedSelfTest.java
index 908dff3..ff420e7 100644
--- a/modules/core/src/test/java/org/gridgain/grid/spi/loadbalancing/roundrobin/GridRoundRobinLoadBalancingNotPerTaskMultithreadedSelfTest.java
+++ b/modules/core/src/test/java/org/gridgain/grid/spi/loadbalancing/roundrobin/GridRoundRobinLoadBalancingNotPerTaskMultithreadedSelfTest.java
@@ -23,9 +23,9 @@ import java.util.concurrent.atomic.*;
 /**
  * Multithreaded tests for global load balancer.
  */
-@GridSpiTest(spi = GridRoundRobinLoadBalancingSpi.class, group = "Load Balancing SPI")
+@GridSpiTest(spi = RoundRobinLoadBalancingSpi.class, group = "Load Balancing SPI")
 public class GridRoundRobinLoadBalancingNotPerTaskMultithreadedSelfTest
-    extends GridSpiAbstractTest<GridRoundRobinLoadBalancingSpi> {
+    extends GridSpiAbstractTest<RoundRobinLoadBalancingSpi> {
     /** Thread count. */
     public static final int THREAD_CNT = 8;
 
@@ -60,7 +60,7 @@ public class GridRoundRobinLoadBalancingNotPerTaskMultithreadedSelfTest
      * @throws Exception If failed.
      */
     public void testMultipleTaskSessionsMultithreaded() throws Exception {
-        final GridRoundRobinLoadBalancingSpi spi = getSpi();
+        final RoundRobinLoadBalancingSpi spi = getSpi();
 
         final List<ClusterNode> allNodes = (List<ClusterNode>)getSpiContext().nodes();
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/a62862fe/modules/core/src/test/java/org/gridgain/grid/spi/loadbalancing/roundrobin/GridRoundRobinLoadBalancingSpiLocalNodeSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/gridgain/grid/spi/loadbalancing/roundrobin/GridRoundRobinLoadBalancingSpiLocalNodeSelfTest.java b/modules/core/src/test/java/org/gridgain/grid/spi/loadbalancing/roundrobin/GridRoundRobinLoadBalancingSpiLocalNodeSelfTest.java
index 12f1174..ab992cf 100644
--- a/modules/core/src/test/java/org/gridgain/grid/spi/loadbalancing/roundrobin/GridRoundRobinLoadBalancingSpiLocalNodeSelfTest.java
+++ b/modules/core/src/test/java/org/gridgain/grid/spi/loadbalancing/roundrobin/GridRoundRobinLoadBalancingSpiLocalNodeSelfTest.java
@@ -18,9 +18,9 @@ import java.util.*;
 /**
  * Tests Round Robin load balancing for single node.
  */
-@GridSpiTest(spi = GridRoundRobinLoadBalancingSpi.class, group = "Load Balancing SPI", triggerDiscovery = true)
+@GridSpiTest(spi = RoundRobinLoadBalancingSpi.class, group = "Load Balancing SPI", triggerDiscovery = true)
 public class GridRoundRobinLoadBalancingSpiLocalNodeSelfTest extends
-    GridSpiAbstractTest<GridRoundRobinLoadBalancingSpi> {
+    GridSpiAbstractTest<RoundRobinLoadBalancingSpi> {
     /**
      * @throws Exception If failed.
      */

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/a62862fe/modules/core/src/test/java/org/gridgain/grid/spi/loadbalancing/roundrobin/GridRoundRobinLoadBalancingSpiMultipleNodesSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/gridgain/grid/spi/loadbalancing/roundrobin/GridRoundRobinLoadBalancingSpiMultipleNodesSelfTest.java b/modules/core/src/test/java/org/gridgain/grid/spi/loadbalancing/roundrobin/GridRoundRobinLoadBalancingSpiMultipleNodesSelfTest.java
index 4651db0..0ab487f 100644
--- a/modules/core/src/test/java/org/gridgain/grid/spi/loadbalancing/roundrobin/GridRoundRobinLoadBalancingSpiMultipleNodesSelfTest.java
+++ b/modules/core/src/test/java/org/gridgain/grid/spi/loadbalancing/roundrobin/GridRoundRobinLoadBalancingSpiMultipleNodesSelfTest.java
@@ -24,11 +24,11 @@ import static org.apache.ignite.events.IgniteEventType.*;
 /**
  * Tests round robin load balancing SPI.
  */
-@GridSpiTest(spi = GridRoundRobinLoadBalancingSpi.class, group = "Load Balancing SPI")
+@GridSpiTest(spi = RoundRobinLoadBalancingSpi.class, group = "Load Balancing SPI")
 public class GridRoundRobinLoadBalancingSpiMultipleNodesSelfTest
-    extends GridSpiAbstractTest<GridRoundRobinLoadBalancingSpi> {
+    extends GridSpiAbstractTest<RoundRobinLoadBalancingSpi> {
     /** {@inheritDoc} */
-    @Override protected void spiConfigure(GridRoundRobinLoadBalancingSpi spi) throws Exception {
+    @Override protected void spiConfigure(RoundRobinLoadBalancingSpi spi) throws Exception {
         super.spiConfigure(spi);
 
         spi.setPerTask(true);

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/a62862fe/modules/core/src/test/java/org/gridgain/grid/spi/loadbalancing/roundrobin/GridRoundRobinLoadBalancingSpiNotPerTaskSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/gridgain/grid/spi/loadbalancing/roundrobin/GridRoundRobinLoadBalancingSpiNotPerTaskSelfTest.java b/modules/core/src/test/java/org/gridgain/grid/spi/loadbalancing/roundrobin/GridRoundRobinLoadBalancingSpiNotPerTaskSelfTest.java
index 5e887b0..2d11283 100644
--- a/modules/core/src/test/java/org/gridgain/grid/spi/loadbalancing/roundrobin/GridRoundRobinLoadBalancingSpiNotPerTaskSelfTest.java
+++ b/modules/core/src/test/java/org/gridgain/grid/spi/loadbalancing/roundrobin/GridRoundRobinLoadBalancingSpiNotPerTaskSelfTest.java
@@ -25,9 +25,9 @@ import static org.gridgain.grid.spi.loadbalancing.roundrobin.GridRoundRobinTestU
 /**
  * Tests round robin load balancing.
  */
-@GridSpiTest(spi = GridRoundRobinLoadBalancingSpi.class, group = "Load Balancing SPI")
+@GridSpiTest(spi = RoundRobinLoadBalancingSpi.class, group = "Load Balancing SPI")
 public class GridRoundRobinLoadBalancingSpiNotPerTaskSelfTest
-    extends GridSpiAbstractTest<GridRoundRobinLoadBalancingSpi> {
+    extends GridSpiAbstractTest<RoundRobinLoadBalancingSpi> {
     /**
      * @return Per-task configuration parameter.
      */

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/a62862fe/modules/core/src/test/java/org/gridgain/grid/spi/loadbalancing/roundrobin/GridRoundRobinLoadBalancingSpiStartStopSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/gridgain/grid/spi/loadbalancing/roundrobin/GridRoundRobinLoadBalancingSpiStartStopSelfTest.java b/modules/core/src/test/java/org/gridgain/grid/spi/loadbalancing/roundrobin/GridRoundRobinLoadBalancingSpiStartStopSelfTest.java
index aefb445..bc8baf8 100644
--- a/modules/core/src/test/java/org/gridgain/grid/spi/loadbalancing/roundrobin/GridRoundRobinLoadBalancingSpiStartStopSelfTest.java
+++ b/modules/core/src/test/java/org/gridgain/grid/spi/loadbalancing/roundrobin/GridRoundRobinLoadBalancingSpiStartStopSelfTest.java
@@ -13,11 +13,11 @@ import org.gridgain.grid.spi.*;
 import org.gridgain.testframework.junits.spi.*;
 
 /**
- * Tests correct start of {@link GridRoundRobinLoadBalancingSpi}.
+ * Tests correct start of {@link RoundRobinLoadBalancingSpi}.
  */
 @SuppressWarnings({"JUnitTestCaseWithNoTests"})
-@GridSpiTest(spi = GridRoundRobinLoadBalancingSpi.class, group = "LoadBalancing SPI")
+@GridSpiTest(spi = RoundRobinLoadBalancingSpi.class, group = "LoadBalancing SPI")
 public class GridRoundRobinLoadBalancingSpiStartStopSelfTest
-    extends GridSpiStartStopAbstractTest<GridRoundRobinLoadBalancingSpi> {
+    extends GridSpiStartStopAbstractTest<RoundRobinLoadBalancingSpi> {
     // No configs.
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/a62862fe/modules/core/src/test/java/org/gridgain/grid/spi/loadbalancing/roundrobin/GridRoundRobinLoadBalancingSpiTopologyChangeSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/gridgain/grid/spi/loadbalancing/roundrobin/GridRoundRobinLoadBalancingSpiTopologyChangeSelfTest.java b/modules/core/src/test/java/org/gridgain/grid/spi/loadbalancing/roundrobin/GridRoundRobinLoadBalancingSpiTopologyChangeSelfTest.java
index 589e1e6..54d9d49 100644
--- a/modules/core/src/test/java/org/gridgain/grid/spi/loadbalancing/roundrobin/GridRoundRobinLoadBalancingSpiTopologyChangeSelfTest.java
+++ b/modules/core/src/test/java/org/gridgain/grid/spi/loadbalancing/roundrobin/GridRoundRobinLoadBalancingSpiTopologyChangeSelfTest.java
@@ -23,9 +23,9 @@ import static org.gridgain.grid.spi.loadbalancing.roundrobin.GridRoundRobinTestU
 /**
  * Tests round robin load balancing with topology changes.
  */
-@GridSpiTest(spi = GridRoundRobinLoadBalancingSpi.class, group = "Load Balancing SPI")
+@GridSpiTest(spi = RoundRobinLoadBalancingSpi.class, group = "Load Balancing SPI")
 public class GridRoundRobinLoadBalancingSpiTopologyChangeSelfTest
-    extends GridSpiAbstractTest<GridRoundRobinLoadBalancingSpi> {
+    extends GridSpiAbstractTest<RoundRobinLoadBalancingSpi> {
     /**
      * @return Per-task configuration parameter.
      */

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/a62862fe/modules/core/src/test/java/org/gridgain/grid/spi/loadbalancing/roundrobin/GridRoundRobinTestUtils.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/gridgain/grid/spi/loadbalancing/roundrobin/GridRoundRobinTestUtils.java b/modules/core/src/test/java/org/gridgain/grid/spi/loadbalancing/roundrobin/GridRoundRobinTestUtils.java
index c9ce5ab..71b7744 100644
--- a/modules/core/src/test/java/org/gridgain/grid/spi/loadbalancing/roundrobin/GridRoundRobinTestUtils.java
+++ b/modules/core/src/test/java/org/gridgain/grid/spi/loadbalancing/roundrobin/GridRoundRobinTestUtils.java
@@ -30,7 +30,7 @@ class GridRoundRobinTestUtils {
      * @param ses Task session.
      * @throws GridException If balancer failed.
      */
-    static void checkCyclicBalancing(GridRoundRobinLoadBalancingSpi spi, List<ClusterNode> allNodes,
+    static void checkCyclicBalancing(RoundRobinLoadBalancingSpi spi, List<ClusterNode> allNodes,
         List<UUID> orderedNodes, ComputeTaskSession ses) throws GridException {
 
         ClusterNode firstNode = spi.getBalancedNode(ses, allNodes, new GridTestJob());
@@ -58,7 +58,7 @@ class GridRoundRobinTestUtils {
      * @param ses2 Second task session.
      * @throws GridException If balancer failed.
      */
-    static void checkCyclicBalancing(GridRoundRobinLoadBalancingSpi spi, List<ClusterNode> allNodes,
+    static void checkCyclicBalancing(RoundRobinLoadBalancingSpi spi, List<ClusterNode> allNodes,
         List<UUID> orderedNodes, ComputeTaskSession ses1, ComputeTaskSession ses2) throws GridException {
 
         ClusterNode firstNode = spi.getBalancedNode(ses1, allNodes, new GridTestJob());

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/a62862fe/modules/core/src/test/java/org/gridgain/grid/spi/loadbalancing/weightedrandom/GridWeightedRandomLoadBalancingSpiConfigSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/gridgain/grid/spi/loadbalancing/weightedrandom/GridWeightedRandomLoadBalancingSpiConfigSelfTest.java b/modules/core/src/test/java/org/gridgain/grid/spi/loadbalancing/weightedrandom/GridWeightedRandomLoadBalancingSpiConfigSelfTest.java
index a291f3f..40e24f3 100644
--- a/modules/core/src/test/java/org/gridgain/grid/spi/loadbalancing/weightedrandom/GridWeightedRandomLoadBalancingSpiConfigSelfTest.java
+++ b/modules/core/src/test/java/org/gridgain/grid/spi/loadbalancing/weightedrandom/GridWeightedRandomLoadBalancingSpiConfigSelfTest.java
@@ -14,13 +14,13 @@ import org.gridgain.testframework.junits.spi.*;
 /**
  *
  */
-@GridSpiTest(spi = GridWeightedRandomLoadBalancingSpi.class, group = "Load Balancing SPI")
+@GridSpiTest(spi = WeightedRandomLoadBalancingSpi.class, group = "Load Balancing SPI")
 public class GridWeightedRandomLoadBalancingSpiConfigSelfTest extends
-    GridSpiAbstractConfigTest<GridWeightedRandomLoadBalancingSpi> {
+    GridSpiAbstractConfigTest<WeightedRandomLoadBalancingSpi> {
     /**
      * @throws Exception If failed.
      */
     public void testNegativeConfig() throws Exception {
-        checkNegativeSpiProperty(new GridWeightedRandomLoadBalancingSpi(), "nodeWeight", 0);
+        checkNegativeSpiProperty(new WeightedRandomLoadBalancingSpi(), "nodeWeight", 0);
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/a62862fe/modules/core/src/test/java/org/gridgain/grid/spi/loadbalancing/weightedrandom/GridWeightedRandomLoadBalancingSpiSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/gridgain/grid/spi/loadbalancing/weightedrandom/GridWeightedRandomLoadBalancingSpiSelfTest.java b/modules/core/src/test/java/org/gridgain/grid/spi/loadbalancing/weightedrandom/GridWeightedRandomLoadBalancingSpiSelfTest.java
index 771ceaf..efc21ed 100644
--- a/modules/core/src/test/java/org/gridgain/grid/spi/loadbalancing/weightedrandom/GridWeightedRandomLoadBalancingSpiSelfTest.java
+++ b/modules/core/src/test/java/org/gridgain/grid/spi/loadbalancing/weightedrandom/GridWeightedRandomLoadBalancingSpiSelfTest.java
@@ -18,9 +18,9 @@ import java.util.*;
 /**
  * Weighted random load balancing SPI.
  */
-@GridSpiTest(spi = GridWeightedRandomLoadBalancingSpi.class, group = "Load Balancing SPI")
+@GridSpiTest(spi = WeightedRandomLoadBalancingSpi.class, group = "Load Balancing SPI")
 public class GridWeightedRandomLoadBalancingSpiSelfTest extends
-    GridSpiAbstractTest<GridWeightedRandomLoadBalancingSpi> {
+    GridSpiAbstractTest<WeightedRandomLoadBalancingSpi> {
     /**
      * @throws Exception If failed.
      */

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/a62862fe/modules/core/src/test/java/org/gridgain/grid/spi/loadbalancing/weightedrandom/GridWeightedRandomLoadBalancingSpiStartStopSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/gridgain/grid/spi/loadbalancing/weightedrandom/GridWeightedRandomLoadBalancingSpiStartStopSelfTest.java b/modules/core/src/test/java/org/gridgain/grid/spi/loadbalancing/weightedrandom/GridWeightedRandomLoadBalancingSpiStartStopSelfTest.java
index daee18e..4c9b352 100644
--- a/modules/core/src/test/java/org/gridgain/grid/spi/loadbalancing/weightedrandom/GridWeightedRandomLoadBalancingSpiStartStopSelfTest.java
+++ b/modules/core/src/test/java/org/gridgain/grid/spi/loadbalancing/weightedrandom/GridWeightedRandomLoadBalancingSpiStartStopSelfTest.java
@@ -16,8 +16,8 @@ import org.gridgain.testframework.junits.spi.*;
  * Wighted random load balancing SPI start-stop test.
  */
 @SuppressWarnings({"JUnitTestCaseWithNoTests"})
-@GridSpiTest(spi = GridWeightedRandomLoadBalancingSpi.class, group = "LoadBalancing SPI")
+@GridSpiTest(spi = WeightedRandomLoadBalancingSpi.class, group = "LoadBalancing SPI")
 public class GridWeightedRandomLoadBalancingSpiStartStopSelfTest extends
-    GridSpiStartStopAbstractTest<GridWeightedRandomLoadBalancingSpi> {
+    GridSpiStartStopAbstractTest<WeightedRandomLoadBalancingSpi> {
     // No configs.
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/a62862fe/modules/core/src/test/java/org/gridgain/grid/spi/loadbalancing/weightedrandom/GridWeightedRandomLoadBalancingSpiWeightedSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/gridgain/grid/spi/loadbalancing/weightedrandom/GridWeightedRandomLoadBalancingSpiWeightedSelfTest.java b/modules/core/src/test/java/org/gridgain/grid/spi/loadbalancing/weightedrandom/GridWeightedRandomLoadBalancingSpiWeightedSelfTest.java
index effca68..289be66 100644
--- a/modules/core/src/test/java/org/gridgain/grid/spi/loadbalancing/weightedrandom/GridWeightedRandomLoadBalancingSpiWeightedSelfTest.java
+++ b/modules/core/src/test/java/org/gridgain/grid/spi/loadbalancing/weightedrandom/GridWeightedRandomLoadBalancingSpiWeightedSelfTest.java
@@ -17,14 +17,14 @@ import org.gridgain.testframework.*;
 import org.gridgain.testframework.junits.spi.*;
 import java.util.*;
 
-import static org.gridgain.grid.spi.loadbalancing.weightedrandom.GridWeightedRandomLoadBalancingSpi.*;
+import static org.gridgain.grid.spi.loadbalancing.weightedrandom.WeightedRandomLoadBalancingSpi.*;
 
 /**
- * {@link GridWeightedRandomLoadBalancingSpi} self test.
+ * {@link WeightedRandomLoadBalancingSpi} self test.
  */
-@GridSpiTest(spi = GridWeightedRandomLoadBalancingSpi.class, group = "Load Balancing SPI")
+@GridSpiTest(spi = WeightedRandomLoadBalancingSpi.class, group = "Load Balancing SPI")
 public class GridWeightedRandomLoadBalancingSpiWeightedSelfTest
-    extends GridSpiAbstractTest<GridWeightedRandomLoadBalancingSpi> {
+    extends GridSpiAbstractTest<WeightedRandomLoadBalancingSpi> {
     /**
      * @return {@code True} if node weights should be considered.
      */


[03/20] incubator-ignite git commit: # Renaming

Posted by sb...@apache.org.
# Renaming


Project: http://git-wip-us.apache.org/repos/asf/incubator-ignite/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-ignite/commit/a62862fe
Tree: http://git-wip-us.apache.org/repos/asf/incubator-ignite/tree/a62862fe
Diff: http://git-wip-us.apache.org/repos/asf/incubator-ignite/diff/a62862fe

Branch: refs/heads/master
Commit: a62862fe9bf5d4e329b2531b89b80efed285c847
Parents: 12837f3
Author: sboikov <sb...@gridgain.com>
Authored: Fri Dec 5 17:08:40 2014 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Fri Dec 5 17:08:40 2014 +0300

----------------------------------------------------------------------
 .../java/org/apache/ignite/IgniteCompute.java   |   3 +-
 .../ignite/compute/ComputeLoadBalancer.java     |   3 +-
 .../apache/ignite/compute/ComputeTaskSpis.java  |   4 +-
 .../configuration/IgniteConfiguration.java      |  13 +-
 .../resources/IgniteLoadBalancerResource.java   |   4 +-
 .../ignite/spi/failover/FailoverContext.java    |   4 +-
 .../affinity/GridCacheAffinityKeyMapped.java    |   3 +-
 .../org/gridgain/grid/kernal/GridGainEx.java    |   4 +-
 .../loadbalancer/GridLoadBalancerManager.java   |   2 +-
 .../spi/loadbalancing/GridLoadBalancingSpi.java | 117 ----
 .../spi/loadbalancing/LoadBalancingSpi.java     | 114 ++++
 .../adaptive/AdaptiveCpuLoadProbe.java          | 229 ++++++++
 .../adaptive/AdaptiveJobCountLoadProbe.java     |  96 +++
 .../adaptive/AdaptiveLoadBalancingSpi.java      | 581 +++++++++++++++++++
 .../adaptive/AdaptiveLoadBalancingSpiMBean.java |  27 +
 .../adaptive/AdaptiveLoadProbe.java             |  90 +++
 .../AdaptiveProcessingTimeLoadProbe.java        |  98 ++++
 .../adaptive/GridAdaptiveCpuLoadProbe.java      | 229 --------
 .../adaptive/GridAdaptiveJobCountLoadProbe.java |  96 ---
 .../adaptive/GridAdaptiveLoadBalancingSpi.java  | 581 -------------------
 .../GridAdaptiveLoadBalancingSpiMBean.java      |  27 -
 .../adaptive/GridAdaptiveLoadProbe.java         |  90 ---
 .../GridAdaptiveProcessingTimeLoadProbe.java    |  98 ----
 .../GridRoundRobinGlobalLoadBalancer.java       | 305 ----------
 .../GridRoundRobinLoadBalancingSpi.java         | 319 ----------
 .../GridRoundRobinLoadBalancingSpiMBean.java    |  37 --
 .../GridRoundRobinPerTaskLoadBalancer.java      |  96 ---
 .../RoundRobinGlobalLoadBalancer.java           | 305 ++++++++++
 .../roundrobin/RoundRobinLoadBalancingSpi.java  | 319 ++++++++++
 .../RoundRobinLoadBalancingSpiMBean.java        |  37 ++
 .../RoundRobinPerTaskLoadBalancer.java          |  96 +++
 .../GridWeightedRandomLoadBalancingSpi.java     | 394 -------------
 ...GridWeightedRandomLoadBalancingSpiMBean.java |  37 --
 .../WeightedRandomLoadBalancingSpi.java         | 394 +++++++++++++
 .../WeightedRandomLoadBalancingSpiMBean.java    |  37 ++
 .../src/test/config/io-manager-benchmark.xml    |   2 +-
 modules/core/src/test/config/jobs-load-base.xml |   2 +-
 .../src/test/config/load/merge-sort-base.xml    |   2 +-
 .../config/spring-cache-put-remove-load.xml     |   2 +-
 .../grid/kernal/GridMultipleSpisSelfTest.java   |   2 +-
 .../managers/GridManagerStopSelfTest.java       |   2 +-
 ...dAdaptiveLoadBalancingSpiConfigSelfTest.java |   6 +-
 ...iveLoadBalancingSpiMultipleNodeSelfTest.java |   8 +-
 .../GridAdaptiveLoadBalancingSpiSelfTest.java   |   8 +-
 ...aptiveLoadBalancingSpiStartStopSelfTest.java |   4 +-
 ...alancingNotPerTaskMultithreadedSelfTest.java |   6 +-
 ...dRobinLoadBalancingSpiLocalNodeSelfTest.java |   4 +-
 ...inLoadBalancingSpiMultipleNodesSelfTest.java |   6 +-
 ...RobinLoadBalancingSpiNotPerTaskSelfTest.java |   4 +-
 ...dRobinLoadBalancingSpiStartStopSelfTest.java |   6 +-
 ...nLoadBalancingSpiTopologyChangeSelfTest.java |   4 +-
 .../roundrobin/GridRoundRobinTestUtils.java     |   4 +-
 ...tedRandomLoadBalancingSpiConfigSelfTest.java |   6 +-
 ...dWeightedRandomLoadBalancingSpiSelfTest.java |   4 +-
 ...RandomLoadBalancingSpiStartStopSelfTest.java |   4 +-
 ...dRandomLoadBalancingSpiWeightedSelfTest.java |   8 +-
 56 files changed, 2486 insertions(+), 2497 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/a62862fe/modules/core/src/main/java/org/apache/ignite/IgniteCompute.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/IgniteCompute.java b/modules/core/src/main/java/org/apache/ignite/IgniteCompute.java
index 4a8cd49..875b22a 100644
--- a/modules/core/src/main/java/org/apache/ignite/IgniteCompute.java
+++ b/modules/core/src/main/java/org/apache/ignite/IgniteCompute.java
@@ -13,7 +13,6 @@ import org.apache.ignite.cluster.*;
 import org.apache.ignite.compute.*;
 import org.apache.ignite.lang.*;
 import org.gridgain.grid.*;
-import org.gridgain.grid.spi.loadbalancing.*;
 import org.jetbrains.annotations.*;
 
 import java.io.*;
@@ -52,7 +51,7 @@ import java.util.concurrent.*;
  * {@link Serializable} and should be used to run computations on the grid.
  * <h1 class="header">Load Balancing</h1>
  * In all cases other than {@code broadcast(...)}, GridGain must select a node for a computation
- * to be executed. The node will be selected based on the underlying {@link GridLoadBalancingSpi},
+ * to be executed. The node will be selected based on the underlying {@link org.gridgain.grid.spi.loadbalancing.LoadBalancingSpi},
  * which by default sequentially picks next available node from grid projection. Other load balancing
  * policies, such as {@code random} or {@code adaptive}, can be configured as well by selecting
  * a different load balancing SPI in grid configuration. If your logic requires some custom

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/a62862fe/modules/core/src/main/java/org/apache/ignite/compute/ComputeLoadBalancer.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/compute/ComputeLoadBalancer.java b/modules/core/src/main/java/org/apache/ignite/compute/ComputeLoadBalancer.java
index 20a4e8e..edba366 100644
--- a/modules/core/src/main/java/org/apache/ignite/compute/ComputeLoadBalancer.java
+++ b/modules/core/src/main/java/org/apache/ignite/compute/ComputeLoadBalancer.java
@@ -11,7 +11,6 @@ package org.apache.ignite.compute;
 
 import org.apache.ignite.cluster.*;
 import org.gridgain.grid.*;
-import org.gridgain.grid.spi.loadbalancing.*;
 import org.jetbrains.annotations.*;
 
 import java.util.*;
@@ -19,7 +18,7 @@ import java.util.*;
 /**
  * Load balancer is used for finding the best balanced node according
  * to load balancing policy. Internally load balancer will
- * query the {@link GridLoadBalancingSpi}
+ * query the {@link org.gridgain.grid.spi.loadbalancing.LoadBalancingSpi}
  * to get the balanced node.
  * <p>
  * Load balancer can be used <i>explicitly</i> from inside {@link ComputeTask#map(List, Object)}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/a62862fe/modules/core/src/main/java/org/apache/ignite/compute/ComputeTaskSpis.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/compute/ComputeTaskSpis.java b/modules/core/src/main/java/org/apache/ignite/compute/ComputeTaskSpis.java
index 53941f0..6c1d231 100644
--- a/modules/core/src/main/java/org/apache/ignite/compute/ComputeTaskSpis.java
+++ b/modules/core/src/main/java/org/apache/ignite/compute/ComputeTaskSpis.java
@@ -9,14 +9,12 @@
 
 package org.apache.ignite.compute;
 
-import org.gridgain.grid.spi.loadbalancing.*;
-
 import java.lang.annotation.*;
 
 /**
  * This annotation allows task to specify what SPIs it wants to use.
  * Starting with {@code GridGain 2.1} you can start multiple instances
- * of {@link GridLoadBalancingSpi},
+ * of {@link org.gridgain.grid.spi.loadbalancing.LoadBalancingSpi},
  * {@link org.apache.ignite.spi.failover.FailoverSpi}, and {@link org.apache.ignite.spi.checkpoint.CheckpointSpi}. If you do that,
  * you need to tell a task which SPI to use (by default it will use the fist
  * SPI in the list).

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/a62862fe/modules/core/src/main/java/org/apache/ignite/configuration/IgniteConfiguration.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/configuration/IgniteConfiguration.java b/modules/core/src/main/java/org/apache/ignite/configuration/IgniteConfiguration.java
index 04f7634..d3386d2 100644
--- a/modules/core/src/main/java/org/apache/ignite/configuration/IgniteConfiguration.java
+++ b/modules/core/src/main/java/org/apache/ignite/configuration/IgniteConfiguration.java
@@ -36,7 +36,6 @@ import org.apache.ignite.spi.discovery.*;
 import org.apache.ignite.spi.eventstorage.*;
 import org.apache.ignite.spi.failover.*;
 import org.gridgain.grid.spi.loadbalancing.*;
-import org.gridgain.grid.spi.loadbalancing.roundrobin.*;
 import org.gridgain.grid.spi.securesession.*;
 import org.gridgain.grid.spi.securesession.noop.*;
 import org.gridgain.grid.spi.swapspace.*;
@@ -367,7 +366,7 @@ public class IgniteConfiguration {
     private FailoverSpi[] failSpi;
 
     /** Load balancing SPI. */
-    private GridLoadBalancingSpi[] loadBalancingSpi;
+    private LoadBalancingSpi[] loadBalancingSpi;
 
     /** Checkpoint SPI. */
     private GridSwapSpaceSpi swapSpaceSpi;
@@ -2074,11 +2073,11 @@ public class IgniteConfiguration {
 
     /**
      * Should return fully configured load balancing SPI implementation. If not provided,
-     * {@link GridRoundRobinLoadBalancingSpi} will be used.
+     * {@link org.gridgain.grid.spi.loadbalancing.roundrobin.RoundRobinLoadBalancingSpi} will be used.
      *
      * @return Grid load balancing SPI implementation or {@code null} to use default implementation.
      */
-    public GridLoadBalancingSpi[] getLoadBalancingSpi() {
+    public LoadBalancingSpi[] getLoadBalancingSpi() {
         return loadBalancingSpi;
     }
 
@@ -2115,13 +2114,13 @@ public class IgniteConfiguration {
     }
 
     /**
-     * Sets fully configured instance of {@link GridLoadBalancingSpi}.
+     * Sets fully configured instance of {@link org.gridgain.grid.spi.loadbalancing.LoadBalancingSpi}.
      *
-     * @param loadBalancingSpi Fully configured instance of {@link GridLoadBalancingSpi} or
+     * @param loadBalancingSpi Fully configured instance of {@link org.gridgain.grid.spi.loadbalancing.LoadBalancingSpi} or
      *      {@code null} if no SPI provided.
      * @see IgniteConfiguration#getLoadBalancingSpi()
      */
-    public void setLoadBalancingSpi(GridLoadBalancingSpi... loadBalancingSpi) {
+    public void setLoadBalancingSpi(LoadBalancingSpi... loadBalancingSpi) {
         this.loadBalancingSpi = loadBalancingSpi;
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/a62862fe/modules/core/src/main/java/org/apache/ignite/resources/IgniteLoadBalancerResource.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/resources/IgniteLoadBalancerResource.java b/modules/core/src/main/java/org/apache/ignite/resources/IgniteLoadBalancerResource.java
index 17b0ed1..6479e94 100644
--- a/modules/core/src/main/java/org/apache/ignite/resources/IgniteLoadBalancerResource.java
+++ b/modules/core/src/main/java/org/apache/ignite/resources/IgniteLoadBalancerResource.java
@@ -9,14 +9,12 @@
 
 package org.apache.ignite.resources;
 
-import org.gridgain.grid.spi.loadbalancing.*;
-
 import java.lang.annotation.*;
 
 /**
  * Annotates a field or a setter method for injection of {@link org.apache.ignite.compute.ComputeLoadBalancer}.
  * Specific implementation for grid load balancer is defined by
- * {@link GridLoadBalancingSpi}
+ * {@link org.gridgain.grid.spi.loadbalancing.LoadBalancingSpi}
  * which is provided to grid via {@link org.apache.ignite.configuration.IgniteConfiguration}..
  * <p>
  * Load balancer can be injected into instances of following classes:

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/a62862fe/modules/core/src/main/java/org/apache/ignite/spi/failover/FailoverContext.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/failover/FailoverContext.java b/modules/core/src/main/java/org/apache/ignite/spi/failover/FailoverContext.java
index d66ec56..03603cb 100644
--- a/modules/core/src/main/java/org/apache/ignite/spi/failover/FailoverContext.java
+++ b/modules/core/src/main/java/org/apache/ignite/spi/failover/FailoverContext.java
@@ -12,7 +12,7 @@ package org.apache.ignite.spi.failover;
 import org.apache.ignite.cluster.*;
 import org.apache.ignite.compute.*;
 import org.gridgain.grid.*;
-import org.gridgain.grid.spi.loadbalancing.*;
+
 import java.util.*;
 
 /**
@@ -36,7 +36,7 @@ public interface FailoverContext {
 
     /**
      * Gets the next balanced node for failed job. Internally this method will
-     * delegate to load balancing SPI (see {@link GridLoadBalancingSpi} to
+     * delegate to load balancing SPI (see {@link org.gridgain.grid.spi.loadbalancing.LoadBalancingSpi} to
      * determine the optimal node for execution.
      *
      * @param top Topology to pick balanced node from.

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/a62862fe/modules/core/src/main/java/org/gridgain/grid/cache/affinity/GridCacheAffinityKeyMapped.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/cache/affinity/GridCacheAffinityKeyMapped.java b/modules/core/src/main/java/org/gridgain/grid/cache/affinity/GridCacheAffinityKeyMapped.java
index a0492dc..245d39d 100644
--- a/modules/core/src/main/java/org/gridgain/grid/cache/affinity/GridCacheAffinityKeyMapped.java
+++ b/modules/core/src/main/java/org/gridgain/grid/cache/affinity/GridCacheAffinityKeyMapped.java
@@ -10,7 +10,6 @@
 package org.gridgain.grid.cache.affinity;
 
 import org.gridgain.grid.cache.*;
-import org.gridgain.grid.spi.loadbalancing.*;
 
 import java.lang.annotation.*;
 import java.util.concurrent.*;
@@ -84,7 +83,7 @@ import java.util.concurrent.*;
  * {@link org.apache.ignite.compute.ComputeJob} or any other grid computation, such as {@link Runnable}, {@link Callable}, or
  * {@link org.apache.ignite.lang.IgniteClosure}. It should be attached to a method or field that provides affinity key
  * for the computation. Only one annotation per class is allowed. Whenever such annotation is detected,
- * then {@link GridLoadBalancingSpi} will be bypassed, and computation will be routed to the grid node
+ * then {@link org.gridgain.grid.spi.loadbalancing.LoadBalancingSpi} will be bypassed, and computation will be routed to the grid node
  * where the specified affinity key is cached. You can also use optional {@link GridCacheName @GridCacheName}
  * annotation whenever non-default cache name needs to be specified.
  * <p>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/a62862fe/modules/core/src/main/java/org/gridgain/grid/kernal/GridGainEx.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/kernal/GridGainEx.java b/modules/core/src/main/java/org/gridgain/grid/kernal/GridGainEx.java
index 12b2deb..fbfd071 100644
--- a/modules/core/src/main/java/org/gridgain/grid/kernal/GridGainEx.java
+++ b/modules/core/src/main/java/org/gridgain/grid/kernal/GridGainEx.java
@@ -1472,7 +1472,7 @@ public class GridGainEx {
             DeploymentSpi deploySpi = cfg.getDeploymentSpi();
             CheckpointSpi[] cpSpi = cfg.getCheckpointSpi();
             FailoverSpi[] failSpi = cfg.getFailoverSpi();
-            GridLoadBalancingSpi[] loadBalancingSpi = cfg.getLoadBalancingSpi();
+            LoadBalancingSpi[] loadBalancingSpi = cfg.getLoadBalancingSpi();
             GridSwapSpaceSpi swapspaceSpi = cfg.getSwapSpaceSpi();
             IndexingSpi[] indexingSpi = cfg.getIndexingSpi();
 
@@ -1701,7 +1701,7 @@ public class GridGainEx {
                 failSpi = new FailoverSpi[] {new AlwaysFailoverSpi()};
 
             if (loadBalancingSpi == null)
-                loadBalancingSpi = new GridLoadBalancingSpi[] {new GridRoundRobinLoadBalancingSpi()};
+                loadBalancingSpi = new LoadBalancingSpi[] {new RoundRobinLoadBalancingSpi()};
 
             if (swapspaceSpi == null) {
                 boolean needSwap = false;

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/a62862fe/modules/core/src/main/java/org/gridgain/grid/kernal/managers/loadbalancer/GridLoadBalancerManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/kernal/managers/loadbalancer/GridLoadBalancerManager.java b/modules/core/src/main/java/org/gridgain/grid/kernal/managers/loadbalancer/GridLoadBalancerManager.java
index 26918be..3a89361 100644
--- a/modules/core/src/main/java/org/gridgain/grid/kernal/managers/loadbalancer/GridLoadBalancerManager.java
+++ b/modules/core/src/main/java/org/gridgain/grid/kernal/managers/loadbalancer/GridLoadBalancerManager.java
@@ -27,7 +27,7 @@ import java.util.*;
 /**
  * Load balancing manager.
  */
-public class GridLoadBalancerManager extends GridManagerAdapter<GridLoadBalancingSpi> {
+public class GridLoadBalancerManager extends GridManagerAdapter<LoadBalancingSpi> {
     /**
      * @param ctx Grid kernal context.
      */

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/a62862fe/modules/core/src/main/java/org/gridgain/grid/spi/loadbalancing/GridLoadBalancingSpi.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/spi/loadbalancing/GridLoadBalancingSpi.java b/modules/core/src/main/java/org/gridgain/grid/spi/loadbalancing/GridLoadBalancingSpi.java
deleted file mode 100644
index 1d5f6f9..0000000
--- a/modules/core/src/main/java/org/gridgain/grid/spi/loadbalancing/GridLoadBalancingSpi.java
+++ /dev/null
@@ -1,117 +0,0 @@
-/* @java.file.header */
-
-/*  _________        _____ __________________        _____
- *  __  ____/___________(_)______  /__  ____/______ ____(_)_______
- *  _  / __  __  ___/__  / _  __  / _  / __  _  __ `/__  / __  __ \
- *  / /_/ /  _  /    _  /  / /_/ /  / /_/ /  / /_/ / _  /  _  / / /
- *  \____/   /_/     /_/   \_,__/   \____/   \__,_/  /_/   /_/ /_/
- */
-
-package org.gridgain.grid.spi.loadbalancing;
-
-import org.apache.ignite.cluster.*;
-import org.apache.ignite.compute.*;
-import org.apache.ignite.spi.*;
-import org.gridgain.grid.*;
-import org.gridgain.grid.spi.loadbalancing.adaptive.*;
-import org.gridgain.grid.spi.loadbalancing.roundrobin.*;
-import org.gridgain.grid.spi.loadbalancing.weightedrandom.*;
-
-import java.util.*;
-
-/**
- * Load balancing SPI provides the next best balanced node for job
- * execution. This SPI is used either implicitly or explicitly whenever
- * a job gets mapped to a node during {@link org.apache.ignite.compute.ComputeTask#map(List, Object)}
- * invocation.
- * <h1 class="header">Coding Examples</h1>
- * If you are using {@link org.apache.ignite.compute.ComputeTaskSplitAdapter} then load balancing logic
- * is transparent to your code and is handled automatically by the adapter.
- * Here is an example of how your task could look:
- * <pre name="code" class="java">
- * public class MyFooBarTask extends GridComputeTaskSplitAdapter&lt;Object,Object&gt; {
- *    &#64;Override
- *    protected Collection&lt;? extends GridComputeJob&gt; split(int gridSize, Object arg) throws GridException {
- *        List&lt;MyFooBarJob&gt; jobs = new ArrayList&lt;MyFooBarJob&gt;(gridSize);
- *
- *        for (int i = 0; i &lt; gridSize; i++) {
- *            jobs.add(new MyFooBarJob(arg));
- *        }
- *
- *        // Node assignment via load balancer
- *        // happens automatically.
- *        return jobs;
- *    }
- *    ...
- * }
- * </pre>
- * If you need more fine-grained control over how some jobs within task get mapped to a node
- * <i>and</i> use, for example, affinity load balancing for some other jobs within task, then you should use
- * {@link org.apache.ignite.compute.ComputeTaskAdapter}. Here is an example of how your task could look. Note that in this
- * case we manually inject load balancer and use it to pick the best node. Doing it in
- * such way would allow user to map some jobs manually and for others use load balancer.
- * <pre name="code" class="java">
- * public class MyFooBarTask extends GridComputeTaskAdapter&lt;String,String&gt; {
- *    // Inject load balancer.
- *    &#64;GridLoadBalancerResource
- *    GridComputeLoadBalancer balancer;
- *
- *    // Map jobs to grid nodes.
- *    public Map&lt;? extends GridComputeJob, GridNode&gt; map(List&lt;GridNode&gt; subgrid, String arg) throws GridException {
- *        Map&lt;MyFooBarJob, GridNode&gt; jobs = new HashMap&lt;MyFooBarJob, GridNode&gt;(subgrid.size());
- *
- *        // In more complex cases, you can actually do
- *        // more complicated assignments of jobs to nodes.
- *        for (int i = 0; i &lt; subgrid.size(); i++) {
- *            // Pick the next best balanced node for the job.
- *            GridComputeJob myJob = new MyFooBarJob(arg);
- *
- *            jobs.put(myJob, balancer.getBalancedNode(myJob, null));
- *        }
- *
- *        return jobs;
- *    }
- *
- *    // Aggregate results into one compound result.
- *    public String reduce(List&lt;GridComputeJobResult&gt; results) throws GridException {
- *        // For the purpose of this example we simply
- *        // concatenate string representation of every
- *        // job result
- *        StringBuilder buf = new StringBuilder();
- *
- *        for (GridComputeJobResult res : results) {
- *            // Append string representation of result
- *            // returned by every job.
- *            buf.append(res.getData().toString());
- *        }
- *
- *        return buf.toString();
- *    }
- * }
- * </pre>
- * <p>
- * GridGain comes with the following load balancing SPI implementations out of the box:
- * <ul>
- * <li>{@link GridRoundRobinLoadBalancingSpi} - default</li>
- * <li>{@link GridAdaptiveLoadBalancingSpi}</li>
- * <li>{@link GridWeightedRandomLoadBalancingSpi}</li>
- * </ul>
- * <b>NOTE:</b> this SPI (i.e. methods in this interface) should never be used directly. SPIs provide
- * internal view on the subsystem and is used internally by GridGain kernal. In rare use cases when
- * access to a specific implementation of this SPI is required - an instance of this SPI can be obtained
- * via {@link org.apache.ignite.Ignite#configuration()} method to check its configuration properties or call other non-SPI
- * methods. Note again that calling methods from this interface on the obtained instance can lead
- * to undefined behavior and explicitly not supported.
- */
-public interface GridLoadBalancingSpi extends IgniteSpi {
-    /**
-     * Gets balanced node for specified job within given task session.
-     *
-     * @param ses Grid task session for currently executing task.
-     * @param top Topology of task nodes from which to pick the best balanced node for given job.
-     * @param job Job for which to pick the best balanced node.
-     * @throws GridException If failed to get next balanced node.
-     * @return Best balanced node for the given job within given task session.
-     */
-    public ClusterNode getBalancedNode(ComputeTaskSession ses, List<ClusterNode> top, ComputeJob job) throws GridException;
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/a62862fe/modules/core/src/main/java/org/gridgain/grid/spi/loadbalancing/LoadBalancingSpi.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/spi/loadbalancing/LoadBalancingSpi.java b/modules/core/src/main/java/org/gridgain/grid/spi/loadbalancing/LoadBalancingSpi.java
new file mode 100644
index 0000000..91095fa
--- /dev/null
+++ b/modules/core/src/main/java/org/gridgain/grid/spi/loadbalancing/LoadBalancingSpi.java
@@ -0,0 +1,114 @@
+/* @java.file.header */
+
+/*  _________        _____ __________________        _____
+ *  __  ____/___________(_)______  /__  ____/______ ____(_)_______
+ *  _  / __  __  ___/__  / _  __  / _  / __  _  __ `/__  / __  __ \
+ *  / /_/ /  _  /    _  /  / /_/ /  / /_/ /  / /_/ / _  /  _  / / /
+ *  \____/   /_/     /_/   \_,__/   \____/   \__,_/  /_/   /_/ /_/
+ */
+
+package org.gridgain.grid.spi.loadbalancing;
+
+import org.apache.ignite.cluster.*;
+import org.apache.ignite.compute.*;
+import org.apache.ignite.spi.*;
+import org.gridgain.grid.*;
+
+import java.util.*;
+
+/**
+ * Load balancing SPI provides the next best balanced node for job
+ * execution. This SPI is used either implicitly or explicitly whenever
+ * a job gets mapped to a node during {@link org.apache.ignite.compute.ComputeTask#map(List, Object)}
+ * invocation.
+ * <h1 class="header">Coding Examples</h1>
+ * If you are using {@link org.apache.ignite.compute.ComputeTaskSplitAdapter} then load balancing logic
+ * is transparent to your code and is handled automatically by the adapter.
+ * Here is an example of how your task could look:
+ * <pre name="code" class="java">
+ * public class MyFooBarTask extends GridComputeTaskSplitAdapter&lt;Object,Object&gt; {
+ *    &#64;Override
+ *    protected Collection&lt;? extends GridComputeJob&gt; split(int gridSize, Object arg) throws GridException {
+ *        List&lt;MyFooBarJob&gt; jobs = new ArrayList&lt;MyFooBarJob&gt;(gridSize);
+ *
+ *        for (int i = 0; i &lt; gridSize; i++) {
+ *            jobs.add(new MyFooBarJob(arg));
+ *        }
+ *
+ *        // Node assignment via load balancer
+ *        // happens automatically.
+ *        return jobs;
+ *    }
+ *    ...
+ * }
+ * </pre>
+ * If you need more fine-grained control over how some jobs within task get mapped to a node
+ * <i>and</i> use, for example, affinity load balancing for some other jobs within task, then you should use
+ * {@link org.apache.ignite.compute.ComputeTaskAdapter}. Here is an example of how your task could look. Note that in this
+ * case we manually inject load balancer and use it to pick the best node. Doing it in
+ * such way would allow user to map some jobs manually and for others use load balancer.
+ * <pre name="code" class="java">
+ * public class MyFooBarTask extends GridComputeTaskAdapter&lt;String,String&gt; {
+ *    // Inject load balancer.
+ *    &#64;GridLoadBalancerResource
+ *    GridComputeLoadBalancer balancer;
+ *
+ *    // Map jobs to grid nodes.
+ *    public Map&lt;? extends GridComputeJob, GridNode&gt; map(List&lt;GridNode&gt; subgrid, String arg) throws GridException {
+ *        Map&lt;MyFooBarJob, GridNode&gt; jobs = new HashMap&lt;MyFooBarJob, GridNode&gt;(subgrid.size());
+ *
+ *        // In more complex cases, you can actually do
+ *        // more complicated assignments of jobs to nodes.
+ *        for (int i = 0; i &lt; subgrid.size(); i++) {
+ *            // Pick the next best balanced node for the job.
+ *            GridComputeJob myJob = new MyFooBarJob(arg);
+ *
+ *            jobs.put(myJob, balancer.getBalancedNode(myJob, null));
+ *        }
+ *
+ *        return jobs;
+ *    }
+ *
+ *    // Aggregate results into one compound result.
+ *    public String reduce(List&lt;GridComputeJobResult&gt; results) throws GridException {
+ *        // For the purpose of this example we simply
+ *        // concatenate string representation of every
+ *        // job result
+ *        StringBuilder buf = new StringBuilder();
+ *
+ *        for (GridComputeJobResult res : results) {
+ *            // Append string representation of result
+ *            // returned by every job.
+ *            buf.append(res.getData().toString());
+ *        }
+ *
+ *        return buf.toString();
+ *    }
+ * }
+ * </pre>
+ * <p>
+ * GridGain comes with the following load balancing SPI implementations out of the box:
+ * <ul>
+ * <li>{@link org.gridgain.grid.spi.loadbalancing.roundrobin.RoundRobinLoadBalancingSpi} - default</li>
+ * <li>{@link org.gridgain.grid.spi.loadbalancing.adaptive.AdaptiveLoadBalancingSpi}</li>
+ * <li>{@link org.gridgain.grid.spi.loadbalancing.weightedrandom.WeightedRandomLoadBalancingSpi}</li>
+ * </ul>
+ * <b>NOTE:</b> this SPI (i.e. methods in this interface) should never be used directly. SPIs provide
+ * internal view on the subsystem and is used internally by GridGain kernal. In rare use cases when
+ * access to a specific implementation of this SPI is required - an instance of this SPI can be obtained
+ * via {@link org.apache.ignite.Ignite#configuration()} method to check its configuration properties or call other non-SPI
+ * methods. Note again that calling methods from this interface on the obtained instance can lead
+ * to undefined behavior and explicitly not supported.
+ */
+public interface LoadBalancingSpi extends IgniteSpi {
+    /**
+     * Gets balanced node for specified job within given task session.
+     *
+     * @param ses Grid task session for currently executing task.
+     * @param top Topology of task nodes from which to pick the best balanced node for given job.
+     * @param job Job for which to pick the best balanced node.
+     * @throws GridException If failed to get next balanced node.
+     * @return Best balanced node for the given job within given task session.
+     */
+    public ClusterNode getBalancedNode(ComputeTaskSession ses, List<ClusterNode> top, ComputeJob job) throws GridException;
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/a62862fe/modules/core/src/main/java/org/gridgain/grid/spi/loadbalancing/adaptive/AdaptiveCpuLoadProbe.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/spi/loadbalancing/adaptive/AdaptiveCpuLoadProbe.java b/modules/core/src/main/java/org/gridgain/grid/spi/loadbalancing/adaptive/AdaptiveCpuLoadProbe.java
new file mode 100644
index 0000000..c438e9c
--- /dev/null
+++ b/modules/core/src/main/java/org/gridgain/grid/spi/loadbalancing/adaptive/AdaptiveCpuLoadProbe.java
@@ -0,0 +1,229 @@
+/* @java.file.header */
+
+/*  _________        _____ __________________        _____
+ *  __  ____/___________(_)______  /__  ____/______ ____(_)_______
+ *  _  / __  __  ___/__  / _  __  / _  / __  _  __ `/__  / __  __ \
+ *  / /_/ /  _  /    _  /  / /_/ /  / /_/ /  / /_/ / _  /  _  / / /
+ *  \____/   /_/     /_/   \_,__/   \____/   \__,_/  /_/   /_/ /_/
+ */
+
+package org.gridgain.grid.spi.loadbalancing.adaptive;
+
+import org.apache.ignite.cluster.*;
+import org.gridgain.grid.util.typedef.internal.*;
+
+/**
+ * Implementation of node load probing based on CPU load.
+ * <p>
+ * Based on {@link #setUseAverage(boolean)}
+ * parameter, this implementation will either use average CPU load
+ * values or current (default is to use averages).
+ * <p>
+ * Based on {@link #setUseProcessors(boolean)} parameter, this implementation
+ * will either take number of processors on the node into account or not.
+ * Since CPU load on multi-processor boxes shows medium load of multiple CPU's it
+ * usually means that the remaining capacity is proportional to the number of
+ * CPU's (or cores) on the node. This configuration parameter indicates
+ * whether to divide each node's CPU load by the number of processors on that node
+ * (default is {@code true}).
+ * <p>
+ * Also note that in some environments every processor may not be adding 100% of
+ * processing power. For example, if you are using multi-core CPU's, then addition of
+ * every core would probably result in about 75% of extra CPU power. To account
+ * for that, you should set {@link #setProcessorCoefficient(double)} parameter to
+ * {@code 0.75} .
+ * <p>
+ * Below is an example of how CPU load probe would be configured in GridGain
+ * Spring configuration file:
+ * <pre name="code" class="xml">
+ * &lt;property name="loadBalancingSpi"&gt;
+ *     &lt;bean class="org.gridgain.grid.spi.loadBalancing.adaptive.GridAdaptiveLoadBalancingSpi"&gt;
+ *         &lt;property name="loadProbe"&gt;
+ *             &lt;bean class="org.gridgain.grid.spi.loadBalancing.adaptive.GridAdaptiveCpuLoadProbe"&gt;
+ *                 &lt;property name="useAverage" value="true"/&gt;
+ *                 &lt;property name="useProcessors" value="true"/&gt;
+ *                 &lt;property name="processorCoefficient" value="0.9"/&gt;
+ *             &lt;/bean&gt;
+ *         &lt;/property&gt;
+ *     &lt;/bean&gt;
+ * &lt;/property&gt;
+ * </pre>
+ * <p>
+ * This implementation is used by default by {@link AdaptiveLoadBalancingSpi} SPI.
+ */
+public class AdaptiveCpuLoadProbe implements AdaptiveLoadProbe {
+    /** Flag indicating whether to use average CPU load vs. current. */
+    private boolean useAvg = true;
+
+    /**
+     * Flag indicating whether to divide each node's CPU load
+     * by the number of processors on that node.
+     */
+    private boolean useProcs = true;
+
+    /**
+     * Coefficient of every CPU processor. By default it is {@code 1}, but
+     * in some environments every processor may not be adding 100% of processing
+     * power. For example, if you are using multi-core CPU's, then addition of
+     * every core would probably result in about 75% of extra CPU power, and hence
+     * you would set this coefficient to {@code 0.75} .
+     */
+    private double procCoefficient = 1;
+
+    /**
+     * Initializes CPU load probe to use CPU load average by default.
+     */
+    public AdaptiveCpuLoadProbe() {
+        // No-op.
+    }
+
+    /**
+     * Specifies whether to use average CPU load vs. current and whether or
+     * not to take number of processors into account.
+     * <p>
+     * Since CPU load on multi-processor boxes shows medium load of multiple CPU's it
+     * usually means that the remaining capacity is proportional to the number of
+     * CPU's (or cores) on the node.
+     *
+     * @param useAvg Flag indicating whether to use average CPU load vs. current
+     *      (default is {@code true}).
+     * @param useProcs Flag indicating whether to divide each node's CPU load
+     *      by the number of processors on that node (default is {@code true}).
+     */
+    public AdaptiveCpuLoadProbe(boolean useAvg, boolean useProcs) {
+        this.useAvg = useAvg;
+        this.useProcs = useProcs;
+    }
+
+    /**
+     * Specifies whether to use average CPU load vs. current and whether or
+     * not to take number of processors into account. It also allows to
+     * specify the coefficient of addition power every CPU adds.
+     * <p>
+     * Since CPU load on multi-processor boxes shows medium load of multiple CPU's it
+     * usually means that the remaining capacity is proportional to the number of
+     * CPU's (or cores) on the node.
+     * <p>
+     * Also, in some environments every processor may not be adding 100% of processing
+     * power. For example, if you are using multi-core CPU's, then addition of
+     * every core would probably result in about 75% of extra CPU power, and hence
+     * you would set this coefficient to {@code 0.75} .
+     *
+     * @param useAvg Flag indicating whether to use average CPU load vs. current
+     *      (default is {@code true}).
+     * @param useProcs Flag indicating whether to divide each node's CPU load
+     *      by the number of processors on that node (default is {@code true}).
+     * @param procCoefficient Coefficient of every CPU processor (default value is {@code 1}).
+     */
+    public AdaptiveCpuLoadProbe(boolean useAvg, boolean useProcs, double procCoefficient) {
+        this.useAvg = useAvg;
+        this.useProcs = useProcs;
+        this.procCoefficient = procCoefficient;
+    }
+
+    /**
+     * Gets flag indicating whether to use average CPU load vs. current.
+     *
+     * @return Flag indicating whether to use average CPU load vs. current.
+     */
+    public boolean isUseAverage() {
+        return useAvg;
+    }
+
+    /**
+     * Sets flag indicating whether to use average CPU load vs. current.
+     * If not explicitly set, then default value is {@code true}.
+     *
+     * @param useAvg Flag indicating whether to use average CPU load vs. current.
+     */
+    public void setUseAverage(boolean useAvg) {
+        this.useAvg = useAvg;
+    }
+
+    /**
+     * Gets flag indicating whether to use average CPU load vs. current
+     * (default is {@code true}).
+     * <p>
+     * Since CPU load on multi-processor boxes shows medium load of multiple CPU's it
+     * usually means that the remaining capacity is proportional to the number of
+     * CPU's (or cores) on the node.
+     *
+     * @return Flag indicating whether to divide each node's CPU load
+     *      by the number of processors on that node (default is {@code true}).
+     */
+    public boolean isUseProcessors() {
+        return useProcs;
+    }
+
+    /**
+     * Sets flag indicating whether to use average CPU load vs. current
+     * (default is {@code true}).
+     * <p>
+     * Since CPU load on multi-processor boxes shows medium load of multiple CPU's it
+     * usually means that the remaining capacity is proportional to the number of
+     * CPU's (or cores) on the node.
+     * <p>
+     * If not explicitly set, then default value is {@code true}.
+     *
+     * @param useProcs Flag indicating whether to divide each node's CPU load
+     *      by the number of processors on that node (default is {@code true}).
+     */
+    public void setUseProcessors(boolean useProcs) {
+        this.useProcs = useProcs;
+    }
+
+    /**
+     * Gets coefficient of every CPU processor. By default it is {@code 1}, but
+     * in some environments every processor may not be adding 100% of processing
+     * power. For example, if you are using multi-core CPU's, then addition of
+     * every core would probably result in about 75% of extra CPU power, and hence
+     * you would set this coefficient to {@code 0.75} .
+     * <p>
+     * This value is ignored if {@link #isUseProcessors()} is set to {@code false}.
+     *
+     * @return Coefficient of every CPU processor.
+     */
+    public double getProcessorCoefficient() {
+        return procCoefficient;
+    }
+
+    /**
+     * Sets coefficient of every CPU processor. By default it is {@code 1}, but
+     * in some environments every processor may not be adding 100% of processing
+     * power. For example, if you are using multi-core CPU's, then addition of
+     * every core would probably result in about 75% of extra CPU power, and hence
+     * you would set this coefficient to {@code 0.75} .
+     * <p>
+     * This value is ignored if {@link #isUseProcessors()} is set to {@code false}.
+     *
+     * @param procCoefficient Coefficient of every CPU processor.
+     */
+    public void setProcessorCoefficient(double procCoefficient) {
+        A.ensure(procCoefficient > 0, "procCoefficient > 0");
+
+        this.procCoefficient = procCoefficient;
+    }
+
+    /** {@inheritDoc} */
+    @Override public double getLoad(ClusterNode node, int jobsSentSinceLastUpdate) {
+        ClusterNodeMetrics metrics = node.metrics();
+
+        double k = 1.0d;
+
+        if (useProcs) {
+            int procs = metrics.getTotalCpus();
+
+            if (procs > 1)
+                k = procs * procCoefficient;
+        }
+
+        double load = (useAvg ? metrics.getAverageCpuLoad() : metrics.getCurrentCpuLoad()) / k;
+
+        return load < 0 ? 0 : load;
+    }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return S.toString(AdaptiveCpuLoadProbe.class, this);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/a62862fe/modules/core/src/main/java/org/gridgain/grid/spi/loadbalancing/adaptive/AdaptiveJobCountLoadProbe.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/spi/loadbalancing/adaptive/AdaptiveJobCountLoadProbe.java b/modules/core/src/main/java/org/gridgain/grid/spi/loadbalancing/adaptive/AdaptiveJobCountLoadProbe.java
new file mode 100644
index 0000000..95ef91d
--- /dev/null
+++ b/modules/core/src/main/java/org/gridgain/grid/spi/loadbalancing/adaptive/AdaptiveJobCountLoadProbe.java
@@ -0,0 +1,96 @@
+/* @java.file.header */
+
+/*  _________        _____ __________________        _____
+ *  __  ____/___________(_)______  /__  ____/______ ____(_)_______
+ *  _  / __  __  ___/__  / _  __  / _  / __  _  __ `/__  / __  __ \
+ *  / /_/ /  _  /    _  /  / /_/ /  / /_/ /  / /_/ / _  /  _  / / /
+ *  \____/   /_/     /_/   \_,__/   \____/   \__,_/  /_/   /_/ /_/
+ */
+
+package org.gridgain.grid.spi.loadbalancing.adaptive;
+
+import org.apache.ignite.cluster.*;
+import org.gridgain.grid.util.typedef.internal.*;
+
+/**
+ * Implementation of node load probing based on active and waiting job count.
+ * Based on {@link #setUseAverage(boolean)} parameter, this implementation will
+ * either use average job count values or current (default is to use averages).
+ * <p>
+ * The load of a node is simply calculated by adding active and waiting job counts.
+ * <p>
+ * Below is an example of how CPU load probe would be configured in GridGain
+ * Spring configuration file:
+ * <pre name="code" class="xml">
+ * &lt;property name="loadBalancingSpi"&gt;
+ *     &lt;bean class="org.gridgain.grid.spi.loadBalancing.adaptive.GridAdaptiveLoadBalancingSpi"&gt;
+ *         &lt;property name="loadProbe"&gt;
+ *             &lt;bean class="org.gridgain.grid.spi.loadBalancing.adaptive.GridAdaptiveJobCountLoadProbe"&gt;
+ *                 &lt;property name="useAverage" value="true"/&gt;
+ *             &lt;/bean&gt;
+ *         &lt;/property&gt;
+ *     &lt;/bean&gt;
+ * &lt;/property&gt;
+ * </pre>
+ */
+public class AdaptiveJobCountLoadProbe implements AdaptiveLoadProbe {
+    /** Flag indicating whether to use average CPU load vs. current. */
+    private boolean useAvg = true;
+
+    /**
+     * Initializes active job probe.
+     */
+    public AdaptiveJobCountLoadProbe() {
+        // No-op.
+    }
+
+    /**
+     * Creates new active job prove specifying whether to use average
+     * job counts vs. current.
+     *
+     * @param useAvg Flag indicating whether to use average job counts vs. current.
+     */
+    public AdaptiveJobCountLoadProbe(boolean useAvg) {
+        this.useAvg = useAvg;
+    }
+
+    /**
+     * Gets flag indicating whether to use average job counts vs. current.
+     *
+     * @return Flag indicating whether to use average job counts vs. current.
+     */
+    public boolean isUseAverage() {
+        return useAvg;
+    }
+
+    /**
+     * Sets flag indicating whether to use average job counts vs. current.
+     *
+     * @param useAvg Flag indicating whether to use average job counts vs. current.
+     */
+    public void setUseAverage(boolean useAvg) {
+        this.useAvg = useAvg;
+    }
+
+
+    /** {@inheritDoc} */
+    @Override public double getLoad(ClusterNode node, int jobsSentSinceLastUpdate) {
+        ClusterNodeMetrics metrics = node.metrics();
+
+        if (useAvg) {
+            double load = metrics.getAverageActiveJobs() + metrics.getAverageWaitingJobs();
+
+            if (load > 0)
+                return load;
+        }
+
+        double load = metrics.getCurrentActiveJobs() + metrics.getCurrentWaitingJobs();
+
+        return load < 0 ? 0 : load;
+    }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return S.toString(AdaptiveJobCountLoadProbe.class, this);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/a62862fe/modules/core/src/main/java/org/gridgain/grid/spi/loadbalancing/adaptive/AdaptiveLoadBalancingSpi.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/spi/loadbalancing/adaptive/AdaptiveLoadBalancingSpi.java b/modules/core/src/main/java/org/gridgain/grid/spi/loadbalancing/adaptive/AdaptiveLoadBalancingSpi.java
new file mode 100644
index 0000000..069d269
--- /dev/null
+++ b/modules/core/src/main/java/org/gridgain/grid/spi/loadbalancing/adaptive/AdaptiveLoadBalancingSpi.java
@@ -0,0 +1,581 @@
+/* @java.file.header */
+
+/*  _________        _____ __________________        _____
+ *  __  ____/___________(_)______  /__  ____/______ ____(_)_______
+ *  _  / __  __  ___/__  / _  __  / _  / __  _  __ `/__  / __  __ \
+ *  / /_/ /  _  /    _  /  / /_/ /  / /_/ /  / /_/ / _  /  _  / / /
+ *  \____/   /_/     /_/   \_,__/   \____/   \__,_/  /_/   /_/ /_/
+ */
+
+package org.gridgain.grid.spi.loadbalancing.adaptive;
+
+import org.apache.ignite.*;
+import org.apache.ignite.cluster.*;
+import org.apache.ignite.compute.*;
+import org.apache.ignite.events.*;
+import org.apache.ignite.lang.*;
+import org.apache.ignite.resources.*;
+import org.apache.ignite.spi.*;
+import org.gridgain.grid.*;
+import org.gridgain.grid.kernal.managers.eventstorage.*;
+import org.gridgain.grid.spi.loadbalancing.*;
+import org.gridgain.grid.util.typedef.*;
+import org.gridgain.grid.util.typedef.internal.*;
+import org.jdk8.backport.*;
+import org.jetbrains.annotations.*;
+
+import java.util.*;
+import java.util.concurrent.*;
+import java.util.concurrent.atomic.*;
+import java.util.concurrent.locks.*;
+
+import static org.apache.ignite.events.IgniteEventType.*;
+
+/**
+ * Load balancing SPI that adapts to overall node performance. It
+ * proportionally distributes more jobs to more performant nodes based
+ * on a pluggable and dynamic node load probing.
+ * <p>
+ * <h1 class="header">Adaptive Node Probe</h1>
+ * This SPI comes with pluggable algorithm to calculate a node load
+ * at any given point of time. The algorithm is defined by
+ * {@link AdaptiveLoadProbe} interface and user is
+ * free to provide custom implementations. By default
+ * {@link AdaptiveCpuLoadProbe} implementation is used
+ * which distributes jobs to nodes based on average CPU load
+ * on every node.
+ * <p>
+ * The following load probes are available with the product:
+ * <ul>
+ * <li>{@link AdaptiveCpuLoadProbe} - default</li>
+ * <li>{@link AdaptiveProcessingTimeLoadProbe}</li>
+ * <li>{@link AdaptiveJobCountLoadProbe}</li>
+ * </ul>
+ * Note that if {@link AdaptiveLoadProbe#getLoad(org.apache.ignite.cluster.ClusterNode, int)} returns a value of {@code 0},
+ * then implementation will assume that load value is simply not available and
+ * will try to calculate an average of load values for other nodes. If such
+ * average cannot be obtained (all node load values are {@code 0}), then a value
+ * of {@code 1} will be used.
+ * <p>
+ * When working with node metrics, take into account that all averages are
+ * calculated over metrics history size defined by {@link org.apache.ignite.configuration.IgniteConfiguration#getMetricsExpireTime()}
+ * and {@link org.apache.ignite.configuration.IgniteConfiguration#getMetricsHistorySize()} grid configuration parameters.
+ * Generally the larger these configuration parameter values are, the more precise the metrics are.
+ * You should tune these values based on the level of accuracy needed vs. the additional memory
+ * that would be required for storing metrics.
+ * <p>
+ * You should also keep in mind that metrics for remote nodes are delayed (usually by the
+ * heartbeat frequency). So if it is acceptable in your environment, set the heartbeat frequency
+ * to be more inline with job execution time. Generally, the more often heartbeats between nodes
+ * are exchanged, the more precise the metrics are. However, you should keep in mind that if
+ * heartbeats are exchanged too often then it may create unnecessary traffic in the network.
+ * Heartbeats (or metrics update frequency) can be configured via underlying
+ * {@link org.apache.ignite.spi.discovery.DiscoverySpi} used in your grid.
+ * <p>
+ * Here is an example of how probing can be implemented to use
+ * number of active and waiting jobs as probing mechanism:
+ * <pre name="code" class="java">
+ * public class FooBarLoadProbe implements GridAdaptiveLoadProbe {
+ *     // Flag indicating whether to use average value or current.
+ *     private int useAvg = true;
+ *
+ *     public FooBarLoadProbe(boolean useAvg) {
+ *         this.useAvg = useAvg;
+ *     }
+ *
+ *     // Calculate load based on number of active and waiting jobs.
+ *     public double getLoad(GridNode node, int jobsSentSinceLastUpdate) {
+ *         GridNodeMetrics metrics = node.getMetrics();
+ *
+ *         if (useAvg) {
+ *             double load = metrics.getAverageActiveJobs() + metrics.getAverageWaitingJobs();
+ *
+ *             if (load > 0) {
+ *                 return load;
+ *             }
+ *         }
+ *
+ *         return metrics.getCurrentActiveJobs() + metrics.getCurrentWaitingJobs();
+ *     }
+ * }
+ * </pre>
+ * <h1 class="header">Which Node Probe To Use</h1>
+ * There is no correct answer here. Every single node probe will work better or worse in
+ * different environments. CPU load probe (default option) is the safest approach to start
+ * with as it simply attempts to utilize every CPU on the grid to the maximum. However, you should
+ * experiment with other probes by executing load tests in your environment and observing
+ * which probe gives you best performance and load balancing.
+ * <p>
+ * <h1 class="header">Task Coding Example</h1>
+ * If you are using {@link org.apache.ignite.compute.ComputeTaskSplitAdapter} then load balancing logic
+ * is transparent to your code and is handled automatically by the adapter.
+ * Here is an example of how your task will look:
+ * <pre name="code" class="java">
+ * public class MyFooBarTask extends GridComputeTaskSplitAdapter&lt;Object, Object&gt; {
+ *    &#64;Override
+ *    protected Collection&lt;? extends GridComputeJob&gt; split(int gridSize, Object arg) throws GridException {
+ *        List&lt;MyFooBarJob&gt; jobs = new ArrayList&lt;MyFooBarJob&gt;(gridSize);
+ *
+ *        for (int i = 0; i &lt; gridSize; i++) {
+ *            jobs.add(new MyFooBarJob(arg));
+ *        }
+ *
+ *        // Node assignment via load balancer
+ *        // happens automatically.
+ *        return jobs;
+ *    }
+ *    ...
+ * }
+ * </pre>
+ * If you need more fine-grained control over how some jobs within task get mapped to a node
+ * and use affinity load balancing for some other jobs within task, then you should use
+ * {@link org.apache.ignite.compute.ComputeTaskAdapter}. Here is an example of how your task will look. Note that in this
+ * case we manually inject load balancer and use it to pick the best node. Doing it in
+ * such way would allow user to map some jobs manually and for others use load balancer.
+ * <pre name="code" class="java">
+ * public class MyFooBarTask extends GridComputeTaskAdapter&lt;String, String&gt; {
+ *    // Inject load balancer.
+ *    &#64;GridLoadBalancerResource
+ *    GridComputeLoadBalancer balancer;
+ *
+ *    // Map jobs to grid nodes.
+ *    public Map&lt;? extends GridComputeJob, GridNode&gt; map(List&lt;GridNode&gt; subgrid, String arg) throws GridException {
+ *        Map&lt;MyFooBarJob, GridNode&gt; jobs = new HashMap&lt;MyFooBarJob, GridNode&gt;(subgrid.size());
+ *
+ *        // In more complex cases, you can actually do
+ *        // more complicated assignments of jobs to nodes.
+ *        for (int i = 0; i &lt; subgrid.size(); i++) {
+ *            // Pick the next best balanced node for the job.
+ *            jobs.put(new MyFooBarJob(arg), balancer.getBalancedNode())
+ *        }
+ *
+ *        return jobs;
+ *    }
+ *
+ *    // Aggregate results into one compound result.
+ *    public String reduce(List&lt;GridComputeJobResult&gt; results) throws GridException {
+ *        // For the purpose of this example we simply
+ *        // concatenate string representation of every
+ *        // job result
+ *        StringBuilder buf = new StringBuilder();
+ *
+ *        for (GridComputeJobResult res : results) {
+ *            // Append string representation of result
+ *            // returned by every job.
+ *            buf.append(res.getData().string());
+ *        }
+ *
+ *        return buf.string();
+ *    }
+ * }
+ * </pre>
+ * <p>
+ * <h1 class="header">Configuration</h1>
+ * In order to use this load balancer, you should configure your grid instance
+ * to use {@code GridJobsLoadBalancingSpi} either from Spring XML file or
+ * directly. The following configuration parameters are supported:
+ * <h2 class="header">Mandatory</h2>
+ * This SPI has no mandatory configuration parameters.
+ * <h2 class="header">Optional</h2>
+ * This SPI has the following optional configuration parameters:
+ * <ul>
+ * <li>
+ *      Adaptive node load probing implementation (see {@link #setLoadProbe(AdaptiveLoadProbe)}).
+ *      This configuration parameter supplies a custom algorithm for probing a node's load.
+ *      By default, {@link AdaptiveCpuLoadProbe} implementation is used which
+ *      takes every node's CPU load and tries to send proportionally more jobs to less loaded nodes.
+ * </li>
+ * </ul>
+ * <p>
+ * Below is Java configuration example:
+ * <pre name="code" class="java">
+ * GridAdaptiveLoadBalancingSpi spi = new GridAdaptiveLoadBalancingSpi();
+ *
+ * // Configure probe to use latest job execution time vs. average.
+ * GridAdaptiveProcessingTimeLoadProbe probe = new GridAdaptiveProcessingTimeLoadProbe(false);
+ *
+ * spi.setLoadProbe(probe);
+ *
+ * GridConfiguration cfg = new GridConfiguration();
+ *
+ * // Override default load balancing SPI.
+ * cfg.setLoadBalancingSpi(spi);
+ *
+ * // Starts grid.
+ * G.start(cfg);
+ * </pre>
+ * Here is how you can configure {@code GridJobsLoadBalancingSpi} using Spring XML configuration:
+ * <pre name="code" class="xml">
+ * &lt;property name="loadBalancingSpi"&gt;
+ *     &lt;bean class="org.gridgain.grid.spi.loadBalancing.adaptive.GridAdaptiveLoadBalancingSpi"&gt;
+ *         &lt;property name="loadProbe"&gt;
+ *             &lt;bean class="org.gridgain.grid.spi.loadBalancing.adaptive.GridAdaptiveProcessingTimeLoadProbe"&gt;
+ *                 &lt;constructor-arg value="false"/&gt;
+ *             &lt;/bean&gt;
+ *         &lt;/property&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>
+ */
+@IgniteSpiMultipleInstancesSupport(true)
+public class AdaptiveLoadBalancingSpi extends IgniteSpiAdapter implements LoadBalancingSpi,
+    AdaptiveLoadBalancingSpiMBean {
+    /** Random number generator. */
+    private static final Random RAND = new Random();
+
+    /** Grid logger. */
+    @IgniteLoggerResource
+    private IgniteLogger log;
+
+    /** */
+    private AdaptiveLoadProbe probe = new AdaptiveCpuLoadProbe();
+
+    /** Local event listener to listen to task completion events. */
+    private GridLocalEventListener evtLsnr;
+
+    /** Task topologies. First pair value indicates whether or not jobs have been mapped. */
+    private ConcurrentMap<IgniteUuid, IgniteBiTuple<Boolean, WeightedTopology>> taskTops =
+        new ConcurrentHashMap8<>();
+
+    /** */
+    private final Map<UUID, AtomicInteger> nodeJobs = new HashMap<>();
+
+    /** */
+    private final ReadWriteLock rwLock = new ReentrantReadWriteLock();
+
+    /** {@inheritDoc} */
+    @Override public String getLoadProbeFormatted() {
+        return probe.toString();
+    }
+
+    /**
+     * Sets implementation of node load probe. By default {@link AdaptiveProcessingTimeLoadProbe}
+     * is used which proportionally distributes load based on the average job execution
+     * time on every node.
+     *
+     * @param probe Implementation of node load probe
+     */
+    @IgniteSpiConfiguration(optional = true)
+    public void setLoadProbe(AdaptiveLoadProbe probe) {
+        A.ensure(probe != null, "probe != null");
+
+        this.probe = probe;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void spiStart(@Nullable String gridName) throws IgniteSpiException {
+        startStopwatch();
+
+        assertParameter(probe != null, "loadProbe != null");
+
+        if (log.isDebugEnabled())
+            log.debug(configInfo("loadProbe", probe));
+
+        registerMBean(gridName, this, AdaptiveLoadBalancingSpiMBean.class);
+
+        // Ack ok start.
+        if (log.isDebugEnabled())
+            log.debug(startInfo());
+    }
+
+    /** {@inheritDoc} */
+    @Override public void spiStop() throws IgniteSpiException {
+        rwLock.writeLock().lock();
+
+        try {
+            nodeJobs.clear();
+        }
+        finally {
+            rwLock.writeLock().unlock();
+        }
+
+        unregisterMBean();
+
+        // Ack ok stop.
+        if (log.isDebugEnabled())
+            log.debug(stopInfo());
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void onContextInitialized0(IgniteSpiContext spiCtx) throws IgniteSpiException {
+        getSpiContext().addLocalEventListener(evtLsnr = new GridLocalEventListener() {
+            @Override public void onEvent(IgniteEvent evt) {
+                switch (evt.type()) {
+                    case EVT_TASK_FINISHED:
+                    case EVT_TASK_FAILED: {
+                        IgniteTaskEvent taskEvt = (IgniteTaskEvent)evt;
+
+                        taskTops.remove(taskEvt.taskSessionId());
+
+                        if (log.isDebugEnabled())
+                            log.debug("Removed task topology from topology cache for session: " +
+                                taskEvt.taskSessionId());
+
+                        break;
+                    }
+
+                    case EVT_JOB_MAPPED: {
+                        // We should keep topology and use cache in GridComputeTask#map() method to
+                        // avoid O(n*n/2) complexity, after that we can drop caches.
+                        // Here we set mapped property and later cache will be ignored
+                        IgniteJobEvent jobEvt = (IgniteJobEvent)evt;
+
+                        IgniteBiTuple<Boolean, WeightedTopology> weightedTop = taskTops.get(jobEvt.taskSessionId());
+
+                        if (weightedTop != null)
+                            weightedTop.set1(true);
+
+                        if (log.isDebugEnabled())
+                            log.debug("Job has been mapped. Ignore cache for session: " + jobEvt.taskSessionId());
+
+                        break;
+                    }
+
+                    case EVT_NODE_METRICS_UPDATED:
+                    case EVT_NODE_FAILED:
+                    case EVT_NODE_JOINED:
+                    case EVT_NODE_LEFT: {
+                        IgniteDiscoveryEvent discoEvt = (IgniteDiscoveryEvent)evt;
+
+                        rwLock.writeLock().lock();
+
+                        try {
+                            switch (evt.type()) {
+                                case EVT_NODE_JOINED: {
+                                    nodeJobs.put(discoEvt.eventNode().id(), new AtomicInteger(0));
+
+                                    break;
+                                }
+
+                                case EVT_NODE_LEFT:
+                                case EVT_NODE_FAILED: {
+                                    nodeJobs.remove(discoEvt.eventNode().id());
+
+                                    break;
+                                }
+
+                                case EVT_NODE_METRICS_UPDATED: {
+                                    // Reset counter.
+                                    nodeJobs.put(discoEvt.eventNode().id(), new AtomicInteger(0));
+
+                                    break;
+                                }
+                            }
+                        }
+                        finally {
+                            rwLock.writeLock().unlock();
+                        }
+                    }
+
+                }
+            }
+        },
+            EVT_NODE_METRICS_UPDATED,
+            EVT_NODE_FAILED,
+            EVT_NODE_JOINED,
+            EVT_NODE_LEFT,
+            EVT_TASK_FINISHED,
+            EVT_TASK_FAILED,
+            EVT_JOB_MAPPED
+        );
+
+        // Put all known nodes.
+        rwLock.writeLock().lock();
+
+        try {
+            for (ClusterNode node : getSpiContext().nodes())
+                nodeJobs.put(node.id(), new AtomicInteger(0));
+        }
+        finally {
+            rwLock.writeLock().unlock();
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void onContextDestroyed0() {
+        if (evtLsnr != null) {
+            IgniteSpiContext ctx = getSpiContext();
+
+            if (ctx != null)
+                ctx.removeLocalEventListener(evtLsnr);
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public ClusterNode getBalancedNode(ComputeTaskSession ses, List<ClusterNode> top, ComputeJob job)
+    throws GridException {
+        A.notNull(ses, "ses");
+        A.notNull(top, "top");
+        A.notNull(job, "job");
+
+        IgniteBiTuple<Boolean, WeightedTopology> weightedTop = taskTops.get(ses.getId());
+
+        // Create new cached topology if there is no one. Do not
+        // use cached topology after task has been mapped.
+        if (weightedTop == null)
+            // Called from GridComputeTask#map(). Put new topology and false as not mapped yet.
+            taskTops.put(ses.getId(), weightedTop = F.t(false, new WeightedTopology(top)));
+        // We have topology - check if task has been mapped.
+        else if (weightedTop.get1())
+            // Do not use cache after GridComputeTask#map().
+            return new WeightedTopology(top).pickWeightedNode();
+
+        return weightedTop.get2().pickWeightedNode();
+    }
+
+    /**
+     * Calculates node load based on set probe.
+     *
+     * @param top List of all nodes.
+     * @param node Node to get load for.
+     * @return Node load.
+     * @throws GridException If returned load is negative.
+     */
+    @SuppressWarnings({"TooBroadScope"})
+    private double getLoad(Collection<ClusterNode> top, ClusterNode node) throws GridException {
+        assert !F.isEmpty(top);
+
+        int jobsSentSinceLastUpdate = 0;
+
+        rwLock.readLock().lock();
+
+        try {
+            AtomicInteger cnt = nodeJobs.get(node.id());
+
+            jobsSentSinceLastUpdate = cnt == null ? 0 : cnt.get();
+        }
+        finally {
+            rwLock.readLock().unlock();
+        }
+
+        double load = probe.getLoad(node, jobsSentSinceLastUpdate);
+
+        if (load < 0)
+            throw new GridException("Failed to obtain non-negative load from adaptive load probe: " + load);
+
+        return load;
+    }
+
+    /**
+     * Holder for weighted topology.
+     */
+    private class WeightedTopology {
+        /** Topology sorted by weight. */
+        private final SortedMap<Double, ClusterNode> circle = new TreeMap<>();
+
+        /**
+         * @param top Task topology.
+         * @throws GridException If any load was negative.
+         */
+        WeightedTopology(List<ClusterNode> top) throws GridException {
+            assert !F.isEmpty(top);
+
+            double totalLoad = 0;
+
+            // We need to cache loads here to avoid calls later as load might be
+            // changed between the calls.
+            double[] nums = new double[top.size()];
+
+            int zeroCnt = 0;
+
+            // Compute loads.
+            for (int i = 0; i < top.size(); i++) {
+                double load = getLoad(top, top.get(i));
+
+                nums[i] = load;
+
+                if (load == 0)
+                    zeroCnt++;
+
+                totalLoad += load;
+            }
+
+            // Take care of zero loads.
+            if (zeroCnt > 0) {
+                double newTotal = totalLoad;
+
+                int nonZeroCnt = top.size() - zeroCnt;
+
+                for (int i = 0; i < nums.length; i++) {
+                    double load = nums[i];
+
+                    if (load == 0) {
+                        if (nonZeroCnt > 0)
+                            load = totalLoad / nonZeroCnt;
+
+                        if (load == 0)
+                            load = 1;
+
+                        nums[i] = load;
+
+                        newTotal += load;
+                    }
+                }
+
+                totalLoad = newTotal;
+            }
+
+            double totalWeight = 0;
+
+            // Calculate weights and total weight.
+            for (int i = 0; i < nums.length; i++) {
+                assert nums[i] > 0 : "Invalid load: " + nums[i];
+
+                double weight = totalLoad / nums[i];
+
+                // Convert to weight.
+                nums[i] = weight;
+
+                totalWeight += weight;
+            }
+
+            double weight = 0;
+
+            // Enforce range from 0 to 1.
+            for (int i = 0; i < nums.length; i++) {
+                weight = i == nums.length - 1 ? 1.0d : weight + nums[i] / totalWeight;
+
+                assert weight < 2 : "Invalid weight: " + weight;
+
+                // Complexity of this put is O(logN).
+                circle.put(weight, top.get(i));
+            }
+        }
+
+        /**
+         * Gets weighted node in random fashion.
+         *
+         * @return Weighted node.
+         */
+        ClusterNode pickWeightedNode() {
+            double weight = RAND.nextDouble();
+
+            SortedMap<Double, ClusterNode> pick = circle.tailMap(weight);
+
+            ClusterNode node = pick.get(pick.firstKey());
+
+            rwLock.readLock().lock();
+
+            try {
+                AtomicInteger cnt = nodeJobs.get(node.id());
+
+                if (cnt != null)
+                    cnt.incrementAndGet();
+            }
+            finally {
+                rwLock.readLock().unlock();
+            }
+
+            return node;
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return S.toString(AdaptiveLoadBalancingSpi.class, this);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/a62862fe/modules/core/src/main/java/org/gridgain/grid/spi/loadbalancing/adaptive/AdaptiveLoadBalancingSpiMBean.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/spi/loadbalancing/adaptive/AdaptiveLoadBalancingSpiMBean.java b/modules/core/src/main/java/org/gridgain/grid/spi/loadbalancing/adaptive/AdaptiveLoadBalancingSpiMBean.java
new file mode 100644
index 0000000..5553635
--- /dev/null
+++ b/modules/core/src/main/java/org/gridgain/grid/spi/loadbalancing/adaptive/AdaptiveLoadBalancingSpiMBean.java
@@ -0,0 +1,27 @@
+/* @java.file.header */
+
+/*  _________        _____ __________________        _____
+ *  __  ____/___________(_)______  /__  ____/______ ____(_)_______
+ *  _  / __  __  ___/__  / _  __  / _  / __  _  __ `/__  / __  __ \
+ *  / /_/ /  _  /    _  /  / /_/ /  / /_/ /  / /_/ / _  /  _  / / /
+ *  \____/   /_/     /_/   \_,__/   \____/   \__,_/  /_/   /_/ /_/
+ */
+
+package org.gridgain.grid.spi.loadbalancing.adaptive;
+
+import org.apache.ignite.mbean.*;
+import org.apache.ignite.spi.*;
+
+/**
+ * Management MBean for {@link AdaptiveLoadBalancingSpi} SPI.
+ */
+@IgniteMBeanDescription("MBean that provides access to adaptive load balancing SPI configuration.")
+public interface AdaptiveLoadBalancingSpiMBean extends IgniteSpiManagementMBean {
+    /**
+     * Gets text description of current load probing implementation used.
+     *
+     * @return Text description of current load probing implementation used.
+     */
+    @IgniteMBeanDescription("Text description of current load probing implementation used.")
+    public String getLoadProbeFormatted();
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/a62862fe/modules/core/src/main/java/org/gridgain/grid/spi/loadbalancing/adaptive/AdaptiveLoadProbe.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/spi/loadbalancing/adaptive/AdaptiveLoadProbe.java b/modules/core/src/main/java/org/gridgain/grid/spi/loadbalancing/adaptive/AdaptiveLoadProbe.java
new file mode 100644
index 0000000..512536f
--- /dev/null
+++ b/modules/core/src/main/java/org/gridgain/grid/spi/loadbalancing/adaptive/AdaptiveLoadProbe.java
@@ -0,0 +1,90 @@
+/* @java.file.header */
+
+/*  _________        _____ __________________        _____
+ *  __  ____/___________(_)______  /__  ____/______ ____(_)_______
+ *  _  / __  __  ___/__  / _  __  / _  / __  _  __ `/__  / __  __ \
+ *  / /_/ /  _  /    _  /  / /_/ /  / /_/ /  / /_/ / _  /  _  / / /
+ *  \____/   /_/     /_/   \_,__/   \____/   \__,_/  /_/   /_/ /_/
+ */
+
+package org.gridgain.grid.spi.loadbalancing.adaptive;
+
+import org.apache.ignite.cluster.*;
+
+/**
+ * Pluggable implementation of node load probing. Implementations
+ * of this can be configured to be used with {@link AdaptiveLoadBalancingSpi}
+ * by setting {@link AdaptiveLoadBalancingSpi#setLoadProbe(AdaptiveLoadProbe)}
+ * configuration parameter.
+ * <p>
+ * Note that if {@link #getLoad(org.apache.ignite.cluster.ClusterNode, int)} returns a value of {@code 0},
+ * then implementation will assume that load value is simply not available and
+ * will try to calculate an average of load values for other nodes. If such
+ * average cannot be obtained (all node load values are {@code 0}), then a value
+ * of {@code 1} will be used.
+ * <p>
+ * By default, {@link AdaptiveCpuLoadProbe} probing implementation is used.
+ * <p>
+ * <h1 class="header">Example</h1>
+ * Here is an example of how probing can be implemented to use
+ * number of active and waiting jobs as probing mechanism:
+ * <pre name="code" class="java">
+ * public class FooBarLoadProbe implements GridAdaptiveLoadProbe {
+ *     // Flag indicating whether to use average value or current.
+ *     private int useAvg = true;
+ *
+ *     public FooBarLoadProbe(boolean useAvg) {
+ *         this.useAvg = useAvg;
+ *     }
+ *
+ *     // Calculate load based on number of active and waiting jobs.
+ *     public double getLoad(GridNode node, int jobsSentSinceLastUpdate) {
+ *         GridNodeMetrics metrics = node.getMetrics();
+ *
+ *         if (useAvg) {
+ *             double load = metrics.getAverageActiveJobs() + metrics.getAverageWaitingJobs();
+ *
+ *             if (load > 0) {
+ *                 return load;
+ *             }
+ *         }
+ *
+ *         return metrics.getCurrentActiveJobs() + metrics.getCurrentWaitingJobs();
+ *     }
+ * }
+ * </pre>
+ * Below is an example of how a probe shown above would be configured with {@link AdaptiveLoadBalancingSpi}
+ * SPI:
+ * <pre name="code" class="xml">
+ * &lt;property name="loadBalancingSpi"&gt;
+ *     &lt;bean class="org.gridgain.grid.spi.loadBalancing.adaptive.GridAdaptiveLoadBalancingSpi"&gt;
+ *         &lt;property name="loadProbe"&gt;
+ *             &lt;bean class="foo.bar.FooBarLoadProbe"&gt;
+ *                 &lt;constructor-arg value="true"/&gt;
+ *             &lt;/bean&gt;
+ *         &lt;/property&gt;
+ *     &lt;/bean&gt;
+ * &lt;/property&gt;
+ * </pre>
+ */
+public interface AdaptiveLoadProbe {
+    /**
+     * Calculates load value for a given node. Specific implementations would
+     * usually take into account some of the values provided by
+     * {@link org.apache.ignite.cluster.ClusterNode#metrics()} method. For example, load can be calculated
+     * based on job execution time or number of active jobs, or CPU/Heap utilization.
+     * <p>
+     * Note that if this method returns a value of {@code 0},
+     * then implementation will assume that load value is simply not available and
+     * will try to calculate an average of load values for other nodes. If such
+     * average cannot be obtained (all node load values are {@code 0}), then a value
+     * of {@code 1} will be used.
+     *
+     * @param node Grid node to calculate load for.
+     * @param jobsSentSinceLastUpdate Number of jobs sent to this node since
+     *      last metrics update. This parameter may be useful when
+     *      implementation takes into account the current job count on a node.
+     * @return Non-negative load value for the node (zero and above).
+     */
+    public double getLoad(ClusterNode node, int jobsSentSinceLastUpdate);
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/a62862fe/modules/core/src/main/java/org/gridgain/grid/spi/loadbalancing/adaptive/AdaptiveProcessingTimeLoadProbe.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/spi/loadbalancing/adaptive/AdaptiveProcessingTimeLoadProbe.java b/modules/core/src/main/java/org/gridgain/grid/spi/loadbalancing/adaptive/AdaptiveProcessingTimeLoadProbe.java
new file mode 100644
index 0000000..dc9e250
--- /dev/null
+++ b/modules/core/src/main/java/org/gridgain/grid/spi/loadbalancing/adaptive/AdaptiveProcessingTimeLoadProbe.java
@@ -0,0 +1,98 @@
+/* @java.file.header */
+
+/*  _________        _____ __________________        _____
+ *  __  ____/___________(_)______  /__  ____/______ ____(_)_______
+ *  _  / __  __  ___/__  / _  __  / _  / __  _  __ `/__  / __  __ \
+ *  / /_/ /  _  /    _  /  / /_/ /  / /_/ /  / /_/ / _  /  _  / / /
+ *  \____/   /_/     /_/   \_,__/   \____/   \__,_/  /_/   /_/ /_/
+ */
+
+package org.gridgain.grid.spi.loadbalancing.adaptive;
+
+import org.apache.ignite.cluster.*;
+import org.gridgain.grid.util.typedef.internal.*;
+
+/**
+ * Implementation of node load probing based on total job processing time.
+ * Based on {@link #setUseAverage(boolean)}
+ * parameter, this implementation will either use average job execution
+ * time values or current (default is to use averages). The algorithm
+ * returns a sum of job wait time and job execution time.
+ * <p>
+ * Below is an example of how CPU load probe would be configured in GridGain
+ * Spring configuration file:
+ * <pre name="code" class="xml">
+ * &lt;property name="loadBalancingSpi"&gt;
+ *     &lt;bean class="org.gridgain.grid.spi.loadBalancing.adaptive.GridAdaptiveLoadBalancingSpi"&gt;
+ *         &lt;property name="loadProbe"&gt;
+ *             &lt;bean class="org.gridgain.grid.spi.loadBalancing.adaptive.GridAdaptiveProcessingTimeLoadProbe"&gt;
+ *                 &lt;property name="useAverage" value="true"/&gt;
+ *             &lt;/bean&gt;
+ *         &lt;/property&gt;
+ *     &lt;/bean&gt;
+ * &lt;/property&gt;
+ * </pre>
+ */
+public class AdaptiveProcessingTimeLoadProbe implements AdaptiveLoadProbe {
+    /** Flag indicating whether to use average execution time vs. current. */
+    private boolean useAvg = true;
+
+    /**
+     * Initializes execution time load probe to use
+     * execution time average by default.
+     */
+    public AdaptiveProcessingTimeLoadProbe() {
+        // No-op.
+    }
+
+    /**
+     * Specifies whether to use average execution time vs. current.
+     *
+     * @param useAvg Flag indicating whether to use average execution time vs. current.
+     */
+    public AdaptiveProcessingTimeLoadProbe(boolean useAvg) {
+        this.useAvg = useAvg;
+    }
+
+    /**
+     * Gets flag indicating whether to use average execution time vs. current.
+     *
+     * @return Flag indicating whether to use average execution time vs. current.
+     */
+    public boolean isUseAverage() {
+        return useAvg;
+    }
+
+    /**
+     * Sets flag indicating whether to use average execution time vs. current.
+     *
+     * @param useAvg Flag indicating whether to use average execution time vs. current.
+     */
+    public void setUseAverage(boolean useAvg) {
+        this.useAvg = useAvg;
+    }
+
+
+    /** {@inheritDoc} */
+    @Override public double getLoad(ClusterNode node, int jobsSentSinceLastUpdate) {
+        ClusterNodeMetrics metrics = node.metrics();
+
+        if (useAvg) {
+            double load = metrics.getAverageJobExecuteTime() + metrics.getAverageJobWaitTime();
+
+            // If load is greater than 0, then we can use average times.
+            // Otherwise, we will proceed to using current times.
+            if (load > 0)
+                return load;
+        }
+
+        double load = metrics.getCurrentJobExecuteTime() + metrics.getCurrentJobWaitTime();
+
+        return load < 0 ? 0 : load;
+    }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return S.toString(AdaptiveProcessingTimeLoadProbe.class, this);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/a62862fe/modules/core/src/main/java/org/gridgain/grid/spi/loadbalancing/adaptive/GridAdaptiveCpuLoadProbe.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/spi/loadbalancing/adaptive/GridAdaptiveCpuLoadProbe.java b/modules/core/src/main/java/org/gridgain/grid/spi/loadbalancing/adaptive/GridAdaptiveCpuLoadProbe.java
deleted file mode 100644
index a9c929c..0000000
--- a/modules/core/src/main/java/org/gridgain/grid/spi/loadbalancing/adaptive/GridAdaptiveCpuLoadProbe.java
+++ /dev/null
@@ -1,229 +0,0 @@
-/* @java.file.header */
-
-/*  _________        _____ __________________        _____
- *  __  ____/___________(_)______  /__  ____/______ ____(_)_______
- *  _  / __  __  ___/__  / _  __  / _  / __  _  __ `/__  / __  __ \
- *  / /_/ /  _  /    _  /  / /_/ /  / /_/ /  / /_/ / _  /  _  / / /
- *  \____/   /_/     /_/   \_,__/   \____/   \__,_/  /_/   /_/ /_/
- */
-
-package org.gridgain.grid.spi.loadbalancing.adaptive;
-
-import org.apache.ignite.cluster.*;
-import org.gridgain.grid.util.typedef.internal.*;
-
-/**
- * Implementation of node load probing based on CPU load.
- * <p>
- * Based on {@link #setUseAverage(boolean)}
- * parameter, this implementation will either use average CPU load
- * values or current (default is to use averages).
- * <p>
- * Based on {@link #setUseProcessors(boolean)} parameter, this implementation
- * will either take number of processors on the node into account or not.
- * Since CPU load on multi-processor boxes shows medium load of multiple CPU's it
- * usually means that the remaining capacity is proportional to the number of
- * CPU's (or cores) on the node. This configuration parameter indicates
- * whether to divide each node's CPU load by the number of processors on that node
- * (default is {@code true}).
- * <p>
- * Also note that in some environments every processor may not be adding 100% of
- * processing power. For example, if you are using multi-core CPU's, then addition of
- * every core would probably result in about 75% of extra CPU power. To account
- * for that, you should set {@link #setProcessorCoefficient(double)} parameter to
- * {@code 0.75} .
- * <p>
- * Below is an example of how CPU load probe would be configured in GridGain
- * Spring configuration file:
- * <pre name="code" class="xml">
- * &lt;property name="loadBalancingSpi"&gt;
- *     &lt;bean class="org.gridgain.grid.spi.loadBalancing.adaptive.GridAdaptiveLoadBalancingSpi"&gt;
- *         &lt;property name="loadProbe"&gt;
- *             &lt;bean class="org.gridgain.grid.spi.loadBalancing.adaptive.GridAdaptiveCpuLoadProbe"&gt;
- *                 &lt;property name="useAverage" value="true"/&gt;
- *                 &lt;property name="useProcessors" value="true"/&gt;
- *                 &lt;property name="processorCoefficient" value="0.9"/&gt;
- *             &lt;/bean&gt;
- *         &lt;/property&gt;
- *     &lt;/bean&gt;
- * &lt;/property&gt;
- * </pre>
- * <p>
- * This implementation is used by default by {@link GridAdaptiveLoadBalancingSpi} SPI.
- */
-public class GridAdaptiveCpuLoadProbe implements GridAdaptiveLoadProbe {
-    /** Flag indicating whether to use average CPU load vs. current. */
-    private boolean useAvg = true;
-
-    /**
-     * Flag indicating whether to divide each node's CPU load
-     * by the number of processors on that node.
-     */
-    private boolean useProcs = true;
-
-    /**
-     * Coefficient of every CPU processor. By default it is {@code 1}, but
-     * in some environments every processor may not be adding 100% of processing
-     * power. For example, if you are using multi-core CPU's, then addition of
-     * every core would probably result in about 75% of extra CPU power, and hence
-     * you would set this coefficient to {@code 0.75} .
-     */
-    private double procCoefficient = 1;
-
-    /**
-     * Initializes CPU load probe to use CPU load average by default.
-     */
-    public GridAdaptiveCpuLoadProbe() {
-        // No-op.
-    }
-
-    /**
-     * Specifies whether to use average CPU load vs. current and whether or
-     * not to take number of processors into account.
-     * <p>
-     * Since CPU load on multi-processor boxes shows medium load of multiple CPU's it
-     * usually means that the remaining capacity is proportional to the number of
-     * CPU's (or cores) on the node.
-     *
-     * @param useAvg Flag indicating whether to use average CPU load vs. current
-     *      (default is {@code true}).
-     * @param useProcs Flag indicating whether to divide each node's CPU load
-     *      by the number of processors on that node (default is {@code true}).
-     */
-    public GridAdaptiveCpuLoadProbe(boolean useAvg, boolean useProcs) {
-        this.useAvg = useAvg;
-        this.useProcs = useProcs;
-    }
-
-    /**
-     * Specifies whether to use average CPU load vs. current and whether or
-     * not to take number of processors into account. It also allows to
-     * specify the coefficient of addition power every CPU adds.
-     * <p>
-     * Since CPU load on multi-processor boxes shows medium load of multiple CPU's it
-     * usually means that the remaining capacity is proportional to the number of
-     * CPU's (or cores) on the node.
-     * <p>
-     * Also, in some environments every processor may not be adding 100% of processing
-     * power. For example, if you are using multi-core CPU's, then addition of
-     * every core would probably result in about 75% of extra CPU power, and hence
-     * you would set this coefficient to {@code 0.75} .
-     *
-     * @param useAvg Flag indicating whether to use average CPU load vs. current
-     *      (default is {@code true}).
-     * @param useProcs Flag indicating whether to divide each node's CPU load
-     *      by the number of processors on that node (default is {@code true}).
-     * @param procCoefficient Coefficient of every CPU processor (default value is {@code 1}).
-     */
-    public GridAdaptiveCpuLoadProbe(boolean useAvg, boolean useProcs, double procCoefficient) {
-        this.useAvg = useAvg;
-        this.useProcs = useProcs;
-        this.procCoefficient = procCoefficient;
-    }
-
-    /**
-     * Gets flag indicating whether to use average CPU load vs. current.
-     *
-     * @return Flag indicating whether to use average CPU load vs. current.
-     */
-    public boolean isUseAverage() {
-        return useAvg;
-    }
-
-    /**
-     * Sets flag indicating whether to use average CPU load vs. current.
-     * If not explicitly set, then default value is {@code true}.
-     *
-     * @param useAvg Flag indicating whether to use average CPU load vs. current.
-     */
-    public void setUseAverage(boolean useAvg) {
-        this.useAvg = useAvg;
-    }
-
-    /**
-     * Gets flag indicating whether to use average CPU load vs. current
-     * (default is {@code true}).
-     * <p>
-     * Since CPU load on multi-processor boxes shows medium load of multiple CPU's it
-     * usually means that the remaining capacity is proportional to the number of
-     * CPU's (or cores) on the node.
-     *
-     * @return Flag indicating whether to divide each node's CPU load
-     *      by the number of processors on that node (default is {@code true}).
-     */
-    public boolean isUseProcessors() {
-        return useProcs;
-    }
-
-    /**
-     * Sets flag indicating whether to use average CPU load vs. current
-     * (default is {@code true}).
-     * <p>
-     * Since CPU load on multi-processor boxes shows medium load of multiple CPU's it
-     * usually means that the remaining capacity is proportional to the number of
-     * CPU's (or cores) on the node.
-     * <p>
-     * If not explicitly set, then default value is {@code true}.
-     *
-     * @param useProcs Flag indicating whether to divide each node's CPU load
-     *      by the number of processors on that node (default is {@code true}).
-     */
-    public void setUseProcessors(boolean useProcs) {
-        this.useProcs = useProcs;
-    }
-
-    /**
-     * Gets coefficient of every CPU processor. By default it is {@code 1}, but
-     * in some environments every processor may not be adding 100% of processing
-     * power. For example, if you are using multi-core CPU's, then addition of
-     * every core would probably result in about 75% of extra CPU power, and hence
-     * you would set this coefficient to {@code 0.75} .
-     * <p>
-     * This value is ignored if {@link #isUseProcessors()} is set to {@code false}.
-     *
-     * @return Coefficient of every CPU processor.
-     */
-    public double getProcessorCoefficient() {
-        return procCoefficient;
-    }
-
-    /**
-     * Sets coefficient of every CPU processor. By default it is {@code 1}, but
-     * in some environments every processor may not be adding 100% of processing
-     * power. For example, if you are using multi-core CPU's, then addition of
-     * every core would probably result in about 75% of extra CPU power, and hence
-     * you would set this coefficient to {@code 0.75} .
-     * <p>
-     * This value is ignored if {@link #isUseProcessors()} is set to {@code false}.
-     *
-     * @param procCoefficient Coefficient of every CPU processor.
-     */
-    public void setProcessorCoefficient(double procCoefficient) {
-        A.ensure(procCoefficient > 0, "procCoefficient > 0");
-
-        this.procCoefficient = procCoefficient;
-    }
-
-    /** {@inheritDoc} */
-    @Override public double getLoad(ClusterNode node, int jobsSentSinceLastUpdate) {
-        ClusterNodeMetrics metrics = node.metrics();
-
-        double k = 1.0d;
-
-        if (useProcs) {
-            int procs = metrics.getTotalCpus();
-
-            if (procs > 1)
-                k = procs * procCoefficient;
-        }
-
-        double load = (useAvg ? metrics.getAverageCpuLoad() : metrics.getCurrentCpuLoad()) / k;
-
-        return load < 0 ? 0 : load;
-    }
-
-    /** {@inheritDoc} */
-    @Override public String toString() {
-        return S.toString(GridAdaptiveCpuLoadProbe.class, this);
-    }
-}


[15/20] incubator-ignite git commit: # Renaming

Posted by sb...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/ef258ece/modules/core/src/test/java/org/apache/ignite/spi/swapspace/file/GridFileSwapCompactionSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/spi/swapspace/file/GridFileSwapCompactionSelfTest.java b/modules/core/src/test/java/org/apache/ignite/spi/swapspace/file/GridFileSwapCompactionSelfTest.java
new file mode 100644
index 0000000..5191085
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/spi/swapspace/file/GridFileSwapCompactionSelfTest.java
@@ -0,0 +1,119 @@
+/* @java.file.header */
+
+/*  _________        _____ __________________        _____
+ *  __  ____/___________(_)______  /__  ____/______ ____(_)_______
+ *  _  / __  __  ___/__  / _  __  / _  / __  _  __ `/__  / __  __ \
+ *  / /_/ /  _  /    _  /  / /_/ /  / /_/ /  / /_/ / _  /  _  / / /
+ *  \____/   /_/     /_/   \_,__/   \____/   \__,_/  /_/   /_/ /_/
+ */
+
+package org.apache.ignite.spi.swapspace.file;
+
+import org.gridgain.grid.util.typedef.*;
+import org.gridgain.testframework.junits.common.*;
+
+import java.io.File;
+import java.nio.ByteBuffer;
+import java.util.*;
+
+/**
+ * Test for {@link FileSwapSpaceSpi}.
+ */
+public class GridFileSwapCompactionSelfTest extends GridCommonAbstractTest {
+    /**
+     * @throws Exception If failed.
+     */
+    public void testCompact() throws Exception {
+        File file = new File(UUID.randomUUID().toString());
+
+        X.println("file: " + file.getPath());
+
+        FileSwapSpaceSpi.SwapFile f = new FileSwapSpaceSpi.SwapFile(file, 8);
+
+        Random rnd = new Random();
+
+        ArrayList<FileSwapSpaceSpi.SwapValue> arr = new ArrayList<>();
+
+        int size = 0;
+
+        for (int a = 0; a < 100; a++) {
+            FileSwapSpaceSpi.SwapValue[] vals = new FileSwapSpaceSpi.SwapValue[1 + rnd.nextInt(10)];
+
+            int size0 = 0;
+
+            for (int i = 0; i < vals.length; i++) {
+                byte[] bytes = new byte[1 + rnd.nextInt(49)];
+
+                rnd.nextBytes(bytes);
+
+                size0 += bytes.length;
+
+                vals[i] = new FileSwapSpaceSpi.SwapValue(bytes);
+
+                arr.add(vals[i]);
+            }
+
+            f.write(new FileSwapSpaceSpi.SwapValues(vals, size0), 1);
+
+            size += size0;
+
+            assertEquals(f.length(), size);
+            assertEquals(file.length(), size);
+        }
+
+        int i = 0;
+
+        for (FileSwapSpaceSpi.SwapValue val : arr)
+            assertEquals(val.idx(), ++i);
+
+        i = 0;
+
+        for (int cnt = arr.size() / 2; i < cnt; i++) {
+
+            FileSwapSpaceSpi.SwapValue v = arr.remove(rnd.nextInt(arr.size()));
+
+            assertTrue(f.tryRemove(v.idx(), v));
+        }
+
+        int hash0 = 0;
+
+        for (FileSwapSpaceSpi.SwapValue val : arr)
+            hash0 += Arrays.hashCode(val.readValue(f.readCh));
+
+        ArrayList<T2<ByteBuffer, ArrayDeque<FileSwapSpaceSpi.SwapValue>>> bufs = new ArrayList();
+
+        for (;;) {
+            ArrayDeque<FileSwapSpaceSpi.SwapValue> que = new ArrayDeque<>();
+
+            ByteBuffer buf = f.compact(que, 1024);
+
+            if (buf == null)
+                break;
+
+            bufs.add(new T2(buf, que));
+        }
+
+        f.delete();
+
+        int hash1 = 0;
+
+        for (FileSwapSpaceSpi.SwapValue val : arr)
+            hash1 += Arrays.hashCode(val.value(null));
+
+        assertEquals(hash0, hash1);
+
+        File file0 = new File(UUID.randomUUID().toString());
+
+        FileSwapSpaceSpi.SwapFile f0 = new FileSwapSpaceSpi.SwapFile(file0, 8);
+
+        for (T2<ByteBuffer, ArrayDeque<FileSwapSpaceSpi.SwapValue>> t : bufs)
+            f0.write(t.get2(), t.get1(), 1);
+
+        int hash2 = 0;
+
+        for (FileSwapSpaceSpi.SwapValue val : arr)
+            hash2 += Arrays.hashCode(val.readValue(f0.readCh));
+
+        assertEquals(hash2, hash1);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/ef258ece/modules/core/src/test/java/org/apache/ignite/spi/swapspace/file/GridFileSwapSpaceSpiSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/spi/swapspace/file/GridFileSwapSpaceSpiSelfTest.java b/modules/core/src/test/java/org/apache/ignite/spi/swapspace/file/GridFileSwapSpaceSpiSelfTest.java
new file mode 100644
index 0000000..566759e
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/spi/swapspace/file/GridFileSwapSpaceSpiSelfTest.java
@@ -0,0 +1,345 @@
+/* @java.file.header */
+
+/*  _________        _____ __________________        _____
+ *  __  ____/___________(_)______  /__  ____/______ ____(_)_______
+ *  _  / __  __  ___/__  / _  __  / _  / __  _  __ `/__  / __  __ \
+ *  / /_/ /  _  /    _  /  / /_/ /  / /_/ /  / /_/ / _  /  _  / / /
+ *  \____/   /_/     /_/   \_,__/   \____/   \__,_/  /_/   /_/ /_/
+ */
+
+package org.apache.ignite.spi.swapspace.file;
+
+import org.apache.ignite.lang.*;
+import org.apache.ignite.spi.*;
+import org.apache.ignite.spi.swapspace.*;
+import org.gridgain.grid.util.typedef.*;
+import org.gridgain.grid.util.typedef.internal.*;
+import org.jdk8.backport.*;
+import org.jetbrains.annotations.*;
+
+import java.util.*;
+import java.util.concurrent.*;
+import java.util.concurrent.atomic.*;
+
+/**
+ * Test for {@link FileSwapSpaceSpi}.
+ */
+public class GridFileSwapSpaceSpiSelfTest extends GridSwapSpaceSpiAbstractSelfTest {
+    /** {@inheritDoc} */
+    @Override protected SwapSpaceSpi spi() {
+        FileSwapSpaceSpi s = new FileSwapSpaceSpi();
+
+        s.setMaximumSparsity(0.05f);
+        s.setWriteBufferSize(8 * 1024);
+
+        return s;
+    }
+
+    /**
+     * Tests if SPI works correctly with multithreaded writes.
+     *
+     * @throws Exception If failed.
+     */
+    public void testMultithreadedWrite() throws Exception {
+        final AtomicLong valCntr = new AtomicLong();
+
+        final SwapKey key = new SwapKey("key");
+
+        final CountDownLatch wLatch = new CountDownLatch(1);
+
+        final AtomicBoolean done = new AtomicBoolean();
+
+        IgniteFuture<?> wFut = multithreadedAsync(new Callable<Object>() {
+            @Nullable @Override public Object call() throws Exception {
+                while (!done.get()) {
+                    long val = valCntr.incrementAndGet();
+
+                    spi.store(null, key, Long.toString(val).getBytes(), context());
+
+                    if (val == 1)
+                        wLatch.countDown();
+                }
+
+                return null;
+            }
+        }, 8);
+
+        wLatch.await();
+
+        IgniteFuture<?> rFut = multithreadedAsync(new Callable<Object>() {
+            @Nullable @Override public Object call() throws Exception {
+                while (valCntr.get() < 1000) {
+                    byte[] val = spi.read(null, key, context());
+
+                    assertNotNull(val);
+
+                    long lval = Long.parseLong(new String(val));
+
+                    assertTrue(lval <= valCntr.get());
+                }
+
+                return null;
+            }
+        }, 8);
+
+        rFut.get();
+
+        done.set(true);
+
+        wFut.get();
+    }
+
+    /**
+     * @param i Integer.
+     * @return Swap key.
+     */
+    private SwapKey key(int i) {
+        return new SwapKey(i, i % 11, U.intToBytes(i));
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testMultithreadedOperations() throws Exception {
+        final ConcurrentHashMap8<SwapKey, byte[]> map = new ConcurrentHashMap8<>();
+
+        Random rnd = new Random();
+
+        final int keys = 25000;
+
+        int hash0 = 0;
+
+        final int minValSize = 5;
+        final int maxValSize = 9000; // More than write buffer size.
+
+        for (int i = 0; i < keys; i++) {
+            byte[] val = new byte[minValSize + rnd.nextInt(maxValSize - minValSize)];
+
+            rnd.nextBytes(val);
+
+            hash0 += i * Arrays.hashCode(val);
+
+            assertNull(map.put(key(i), val));
+        }
+
+        assertEquals(keys, map.size());
+
+        for (int i = 0; i < keys; i++)
+            assertTrue(map.containsKey(key(i)));
+
+        final String space = "test_space";
+
+        final AtomicBoolean fin = new AtomicBoolean();
+
+        final IgniteFuture<?> fut = multithreadedAsync(new Callable<Object>() {
+            @Override public Object call() throws Exception {
+                Random rnd = new Random();
+
+                while (!fin.get()) {
+                    final SwapKey key = key(rnd.nextInt(keys));
+
+                    switch(rnd.nextInt(13)) {
+                        case 0: // store
+                            byte[] val = map.remove(key);
+
+                            if (val != null)
+                                spi.store(space, key, val, context());
+
+                            break;
+
+                        case 1: // remove
+                            spi.remove(space, key, new CIX1<byte[]>() {
+                                @Override public void applyx(byte[] bytes) {
+                                    if (bytes != null)
+                                        assertNull(map.putIfAbsent(key, bytes));
+                                }
+                            }, context());
+
+                            break;
+
+                        case 2: // read
+                            for (;;) {
+                                val = spi.read(space, key, context());
+
+                                if (val != null)
+                                    break;
+
+                                val = map.get(key);
+
+                                if (val != null)
+                                    break;
+                            }
+
+                            break;
+
+                        case 3: // storeAll
+                        case 4:
+                        case 9:
+                            Map<SwapKey, byte[]> m = new HashMap<>();
+
+                            int cnt = 1 + rnd.nextInt(25);
+
+                            for (int i = 0; i < cnt; i++) {
+                                SwapKey k = key(rnd.nextInt(keys));
+
+                                val = map.remove(k);
+
+                                if (val != null)
+                                    assertNull(m.put(k, val));
+                            }
+
+                            if (m.isEmpty())
+                                break;
+
+                            spi.storeAll(space, m, context());
+
+                            break;
+
+                        case 5: // readAll
+                            HashSet<SwapKey> s = new HashSet<>();
+
+                            cnt = 1 + rnd.nextInt(25);
+
+                            for (int i = 0; i < cnt; i++) {
+                                SwapKey k = key(rnd.nextInt(keys));
+
+                                val = map.get(k);
+
+                                if (val == null)
+                                    s.add(k);
+                            }
+
+                            while (!s.isEmpty()) {
+                                m = spi.readAll(space, s, context());
+
+                                s.removeAll(m.keySet());
+
+                                Iterator<SwapKey> iter = s.iterator();
+
+                                while (iter.hasNext()) {
+                                    SwapKey k = iter.next();
+
+                                    if (map.containsKey(k))
+                                        iter.remove();
+                                }
+                            }
+
+                            break;
+
+                        case 6: // iterateKeys
+                            IgniteSpiCloseableIterator<Integer> kIt = spi.keyIterator(space, context());
+
+                            if (kIt == null)
+                                break;
+
+                            while (kIt.hasNext())
+                                assertNotNull(kIt.next());
+
+                            kIt.close();
+
+                            break;
+
+                        case 7: // iterate
+                            IgniteSpiCloseableIterator<Map.Entry<byte[], byte[]>> iter = spi.rawIterator(space);
+
+                            if (iter == null)
+                                break;
+
+                            while (iter.hasNext()) {
+                                Map.Entry<byte[], byte[]> entry = iter.next();
+
+                                assertEquals(4, entry.getKey().length);
+
+                                byte[] v = entry.getValue();
+
+                                assertTrue(v.length >= minValSize && v.length < maxValSize);
+                            }
+
+                            iter.close();
+
+                            break;
+
+                        case 8: // iterate partitions
+                            iter = spi.rawIterator(space, rnd.nextInt(11));
+
+                            if (iter == null)
+                                break;
+
+                            while ( iter.hasNext()) {
+                                Map.Entry<byte[], byte[]> entry = iter.next();
+
+                                assertEquals(4, entry.getKey().length);
+
+                                byte[] v = entry.getValue();
+
+                                assertTrue(v.length >= minValSize && v.length < maxValSize);
+                            }
+
+                            iter.close();
+
+                            break;
+
+                        default: // removeAll
+                            s = new HashSet<>();
+
+                            cnt = 1 + rnd.nextInt(25);
+
+                            for (int i = 0; i < cnt; i++) {
+                                SwapKey k = key(rnd.nextInt(keys));
+
+                                val = map.get(k);
+
+                                if (val == null)
+                                    s.add(k);
+                            }
+
+                            if (s.isEmpty())
+                                break;
+
+                            spi.removeAll(space, s, new IgniteBiInClosure<SwapKey, byte[]>() {
+                                @Override public void apply(SwapKey k, byte[] bytes) {
+                                    if (bytes != null)
+                                        assertNull(map.putIfAbsent(k, bytes));
+                                }
+                            }, context());
+
+                            break;
+                    }
+                }
+
+                return null;
+            }
+        }, 39);
+
+        Thread.sleep(60000);
+
+        System.out.println("stopping");
+
+        fin.set(true);
+
+        fut.get();
+
+        assertEquals(keys, map.size() + spi.count(space));
+
+        int hash1 = 0;
+
+        int cnt = 0;
+
+        IgniteSpiCloseableIterator<Map.Entry<byte[], byte[]>> iter = spi.rawIterator(space);
+
+        while (iter.hasNext()) {
+            Map.Entry<byte[], byte[]> entry = iter.next();
+
+            hash1 += U.bytesToInt(entry.getKey(), 0) * Arrays.hashCode(entry.getValue());
+
+            cnt++;
+        }
+
+        assertEquals(cnt, spi.count(space));
+
+        for (Map.Entry<SwapKey, byte[]> entry : map.entrySet())
+            hash1 += (Integer)entry.getKey().key() * Arrays.hashCode(entry.getValue());
+
+        assertEquals(hash0, hash1);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/ef258ece/modules/core/src/test/java/org/apache/ignite/spi/swapspace/inmemory/GridTestSwapSpaceSpi.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/spi/swapspace/inmemory/GridTestSwapSpaceSpi.java b/modules/core/src/test/java/org/apache/ignite/spi/swapspace/inmemory/GridTestSwapSpaceSpi.java
new file mode 100644
index 0000000..e6ea5b6
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/spi/swapspace/inmemory/GridTestSwapSpaceSpi.java
@@ -0,0 +1,444 @@
+/* @java.file.header */
+
+/*  _________        _____ __________________        _____
+ *  __  ____/___________(_)______  /__  ____/______ ____(_)_______
+ *  _  / __  __  ___/__  / _  __  / _  / __  _  __ `/__  / __  __ \
+ *  / /_/ /  _  /    _  /  / /_/ /  / /_/ /  / /_/ / _  /  _  / / /
+ *  \____/   /_/     /_/   \_,__/   \____/   \__,_/  /_/   /_/ /_/
+ */
+
+package org.apache.ignite.spi.swapspace.inmemory;
+
+import org.apache.ignite.lang.*;
+import org.apache.ignite.spi.*;
+import org.apache.ignite.spi.swapspace.*;
+import org.gridgain.grid.util.typedef.*;
+import org.jdk8.backport.*;
+import org.jetbrains.annotations.*;
+
+import java.util.*;
+import java.util.concurrent.*;
+
+import static org.apache.ignite.events.IgniteEventType.*;
+
+/**
+ * Test swap space SPI that stores values in map.
+ */
+@IgniteSpiMultipleInstancesSupport(true)
+public class GridTestSwapSpaceSpi extends IgniteSpiAdapter implements SwapSpaceSpi {
+    /** Listener. */
+    private SwapSpaceSpiListener lsnr;
+
+    /** Spaces map. */
+    private ConcurrentMap<String, Space> spaces = new ConcurrentHashMap8<>();
+
+    /** {@inheritDoc} */
+    @Override public void spiStart(@Nullable String gridName) throws IgniteSpiException {
+        // No-op.
+    }
+
+    /** {@inheritDoc} */
+    @Override public void spiStop() throws IgniteSpiException {
+        // No-op.
+    }
+
+    /** {@inheritDoc} */
+    @Override public void clear(@Nullable String spaceName) throws IgniteSpiException {
+        Space space = space(spaceName);
+
+        if (space != null)
+            space.clear();
+    }
+
+    /** {@inheritDoc} */
+    @Override public long size(@Nullable String spaceName) throws IgniteSpiException {
+        Space space = space(spaceName);
+
+        return space != null ? space.size() : 0;
+    }
+
+    /** {@inheritDoc} */
+    @Override public long count(@Nullable String spaceName) throws IgniteSpiException {
+        Space space = space(spaceName);
+
+        return space != null ? space.count() : 0;
+    }
+
+    /** {@inheritDoc} */
+    @Override public byte[] read(@Nullable String spaceName, SwapKey key, SwapContext ctx)
+        throws IgniteSpiException {
+        Space space = space(spaceName);
+
+        return space != null ? space.read(key) : null;
+    }
+
+    /** {@inheritDoc} */
+    @Override public Map<SwapKey, byte[]> readAll(@Nullable String spaceName, Iterable<SwapKey> keys,
+        SwapContext ctx) throws IgniteSpiException {
+        Space space = space(spaceName);
+
+        return space != null ? space.readAll(keys) : Collections.<SwapKey, byte[]>emptyMap();
+    }
+
+    /** {@inheritDoc} */
+    @Override public void remove(@Nullable String spaceName, SwapKey key, @Nullable IgniteInClosure<byte[]> c,
+        SwapContext ctx) throws IgniteSpiException {
+        Space space = space(spaceName);
+
+        if (space != null)
+            space.remove(key, c);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void removeAll(@Nullable String spaceName, Collection<SwapKey> keys,
+        @Nullable IgniteBiInClosure<SwapKey, byte[]> c, SwapContext ctx) throws IgniteSpiException {
+        Space space = space(spaceName);
+
+        if (space != null)
+            space.removeAll(keys, c);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void store(@Nullable String spaceName, SwapKey key, @Nullable byte[] val, SwapContext ctx)
+        throws IgniteSpiException {
+        ensureSpace(spaceName).store(key, val);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void storeAll(@Nullable String spaceName, Map<SwapKey, byte[]> pairs, SwapContext ctx)
+        throws IgniteSpiException {
+        ensureSpace(spaceName).storeAll(pairs);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void setListener(@Nullable SwapSpaceSpiListener evictLsnr) {
+        lsnr = evictLsnr;
+    }
+
+    /** {@inheritDoc} */
+    @Override public Collection<Integer> partitions(@Nullable String spaceName) throws IgniteSpiException {
+        Space space = space(spaceName);
+
+        return space != null ? space.partitions() : Collections.<Integer>emptyList();
+    }
+
+    /** {@inheritDoc} */
+    @Override public <K> IgniteSpiCloseableIterator<K> keyIterator(@Nullable String spaceName, SwapContext ctx)
+        throws IgniteSpiException {
+        return ensureSpace(spaceName).keyIterator();
+    }
+
+    /** {@inheritDoc} */
+    @Override public IgniteSpiCloseableIterator<Map.Entry<byte[], byte[]>> rawIterator(@Nullable String spaceName)
+        throws IgniteSpiException {
+        return ensureSpace(spaceName).rawIterator();
+    }
+
+    /** {@inheritDoc} */
+    @Override public IgniteSpiCloseableIterator<Map.Entry<byte[], byte[]>> rawIterator(@Nullable String spaceName, int part)
+        throws IgniteSpiException {
+        return ensureSpace(spaceName).rawIterator(part);
+    }
+
+    /**
+     * @param spaceName Space name.
+     * @return Space object.
+     */
+    @Nullable private Space space(String spaceName) {
+        return spaces.get(spaceName);
+    }
+
+    /**
+     * Gets space, creates if does not exist.
+     *
+     * @param spaceName Space name.
+     * @return Space.
+     */
+    private Space ensureSpace(String spaceName) {
+        Space space = spaces.get(spaceName);
+
+        if (space == null)
+            space = F.addIfAbsent(spaces, spaceName, new Space(spaceName));
+
+        return space;
+    }
+
+    private void fireEvent(int evtType, String spaceName, @Nullable byte[] key) {
+        SwapSpaceSpiListener lsnr0 = lsnr;
+
+        if (lsnr0 != null)
+            lsnr0.onSwapEvent(evtType, spaceName, key);
+    }
+
+    private class Space {
+        /** Data storage. */
+        private ConcurrentMap<SwapKey, byte[]> data = new ConcurrentHashMap8<>();
+
+        private final String name;
+
+        /**
+         * @param name Space name.
+         */
+        private Space(String name) {
+            this.name = name;
+        }
+
+        /**
+         * Clears space.
+         */
+        public void clear() {
+            data.clear();
+
+            fireEvent(EVT_SWAP_SPACE_CLEARED, name, null);
+        }
+
+        /**
+         * @return Space size.
+         */
+        public long size() {
+            return data.size();
+        }
+
+        /**
+         * @return Space size.
+         */
+        public long count() {
+            return data.size();
+        }
+
+        /**
+         * @param key Key to read.
+         * @return Read bytes.
+         */
+        public byte[] read(SwapKey key) {
+            byte[] bytes = data.get(key);
+
+            fireEvent(EVT_SWAP_SPACE_DATA_READ, name, key.keyBytes());
+
+            return bytes;
+        }
+
+        /**
+         * @param keys Keys to read.
+         * @return Read keys.
+         */
+        public Map<SwapKey, byte[]> readAll(Iterable<SwapKey> keys) {
+            Map<SwapKey, byte[]> res = new HashMap<>();
+
+            for (SwapKey key : keys) {
+                byte[] val = data.get(key);
+
+                if (val != null) {
+                    res.put(key, val);
+
+                    fireEvent(EVT_SWAP_SPACE_DATA_READ, name, key.keyBytes());
+                }
+            }
+
+            return res;
+        }
+
+        /**
+         * @param key Key to remove.
+         * @param c Closure.
+         */
+        public void remove(SwapKey key, IgniteInClosure<byte[]> c) {
+            byte[] val = data.remove(key);
+
+            if (val != null) {
+                c.apply(val);
+
+                fireEvent(EVT_SWAP_SPACE_DATA_REMOVED, name, key.keyBytes());
+            }
+        }
+
+        /**
+         * @param keys Keys to remove.
+         * @param c Closure to apply for removed values.
+         */
+        public void removeAll(Iterable<SwapKey> keys, IgniteBiInClosure<SwapKey, byte[]> c) {
+            for (SwapKey key : keys) {
+                byte[] val = data.remove(key);
+
+                if (val != null) {
+                    c.apply(key, val);
+
+                    fireEvent(EVT_SWAP_SPACE_DATA_REMOVED, name, key.keyBytes());
+                }
+            }
+        }
+
+        /**
+         * @param key Key to store.
+         * @param val Value to store.
+         */
+        public void store(SwapKey key, byte[] val) {
+            if (val != null) {
+                data.put(key, val);
+
+                fireEvent(EVT_SWAP_SPACE_DATA_STORED, name, key.keyBytes());
+            }
+            else {
+                val = data.remove(key);
+
+                if (val != null)
+                    fireEvent(EVT_SWAP_SPACE_DATA_REMOVED, name, key.keyBytes());
+            }
+        }
+
+        /**
+         * @param pairs Values to store.
+         */
+        public void storeAll(Map<SwapKey, byte[]> pairs) {
+            for (Map.Entry<SwapKey, byte[]> entry : pairs.entrySet()) {
+                SwapKey key = entry.getKey();
+                byte[] val = entry.getValue();
+
+                store(key, val);
+            }
+        }
+
+        /**
+         * @return Partitions in space.
+         */
+        public Collection<Integer> partitions() {
+            Collection<Integer> parts = new HashSet<>();
+
+            for (SwapKey key : data.keySet())
+                parts.add(key.partition());
+
+            return parts;
+        }
+
+        public <K> IgniteSpiCloseableIterator<K> keyIterator() {
+            final Iterator<SwapKey> it = data.keySet().iterator();
+
+            return new IgniteSpiCloseableIterator<K>() {
+                @Override public void close() {
+                    // No-op.
+                }
+
+                @Override public boolean hasNext() {
+                    return it.hasNext();
+                }
+
+                @Override public K next() {
+                    SwapKey next = it.next();
+
+                    return (K)next.key();
+                }
+
+                @Override public void remove() {
+                    it.remove();
+                }
+            };
+        }
+
+        public IgniteSpiCloseableIterator<Map.Entry<byte[], byte[]>> rawIterator() {
+            final Iterator<Map.Entry<SwapKey, byte[]>> it = data.entrySet().iterator();
+
+            return new IgniteSpiCloseableIterator<Map.Entry<byte[], byte[]>>() {
+                @Override public void close() {
+                    // No-op.
+                }
+
+                @Override public boolean hasNext() {
+                    return it.hasNext();
+                }
+
+                @Override public Map.Entry<byte[], byte[]> next() {
+                    final Map.Entry<SwapKey, byte[]> next = it.next();
+
+                    return new Map.Entry<byte[], byte[]>() {
+                        @Override public byte[] getKey() {
+                            return next.getKey().keyBytes();
+                        }
+
+                        @Override public byte[] getValue() {
+                            return next.getValue();
+                        }
+
+                        @Override public byte[] setValue(byte[] val) {
+                            return data.put(next.getKey(), val);
+                        }
+                    };
+                }
+
+                @Override public void remove() {
+                    it.remove();
+                }
+            };
+        }
+
+        public IgniteSpiCloseableIterator<Map.Entry<byte[], byte[]>> rawIterator(final int part) {
+            final Iterator<Map.Entry<SwapKey, byte[]>> it = data.entrySet().iterator();
+
+            return new IgniteSpiCloseableIterator<Map.Entry<byte[], byte[]>>() {
+                /** Next entry in this iterator. */
+                private Map.Entry<SwapKey, byte[]> next;
+
+                private Map.Entry<SwapKey, byte[]> cur;
+
+                {
+                    advance();
+                }
+
+                @Override public void close() {
+                    // No-op.
+                }
+
+                @Override public boolean hasNext() {
+                    return next != null;
+                }
+
+                @Override public Map.Entry<byte[], byte[]> next() {
+                    if (next == null)
+                        throw new NoSuchElementException();
+
+                    final Map.Entry<SwapKey, byte[]> ret = next;
+
+                    cur = ret;
+
+                    advance();
+
+                    return new Map.Entry<byte[], byte[]>() {
+                        @Override public byte[] getKey() {
+                            return ret.getKey().keyBytes();
+                        }
+
+                        @Override public byte[] getValue() {
+                            return ret.getValue();
+                        }
+
+                        @Override public byte[] setValue(byte[] val) {
+                            return data.put(ret.getKey(), val);
+                        }
+                    };
+                }
+
+                @Override public void remove() {
+                    if (cur == null)
+                        throw new IllegalStateException();
+
+                    data.remove(cur.getKey(), cur.getValue());
+                }
+
+                private void advance() {
+                    while (it.hasNext()) {
+                        Map.Entry<SwapKey, byte[]> entry = it.next();
+
+                        if(entry.getKey().partition() == part) {
+                            cur = next;
+
+                            next = entry;
+
+                            return;
+                        }
+                    }
+
+                    next = null;
+                }
+            };
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/ef258ece/modules/core/src/test/java/org/apache/ignite/spi/swapspace/noop/GridNoopSwapSpaceSpiSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/spi/swapspace/noop/GridNoopSwapSpaceSpiSelfTest.java b/modules/core/src/test/java/org/apache/ignite/spi/swapspace/noop/GridNoopSwapSpaceSpiSelfTest.java
new file mode 100644
index 0000000..654b305
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/spi/swapspace/noop/GridNoopSwapSpaceSpiSelfTest.java
@@ -0,0 +1,53 @@
+/* @java.file.header */
+
+/*  _________        _____ __________________        _____
+ *  __  ____/___________(_)______  /__  ____/______ ____(_)_______
+ *  _  / __  __  ___/__  / _  __  / _  / __  _  __ `/__  / __  __ \
+ *  / /_/ /  _  /    _  /  / /_/ /  / /_/ /  / /_/ / _  /  _  / / /
+ *  \____/   /_/     /_/   \_,__/   \____/   \__,_/  /_/   /_/ /_/
+ */
+
+package org.apache.ignite.spi.swapspace.noop;
+
+import org.apache.ignite.*;
+import org.apache.ignite.configuration.*;
+import org.apache.ignite.spi.discovery.tcp.*;
+import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.*;
+import org.apache.ignite.spi.swapspace.*;
+import org.gridgain.testframework.junits.common.*;
+
+/**
+ * Tests for "noop" realization of {@link org.apache.ignite.spi.swapspace.SwapSpaceSpi}.
+ */
+public class GridNoopSwapSpaceSpiSelfTest extends GridCommonAbstractTest {
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(gridName);
+
+        TcpDiscoverySpi disco = new TcpDiscoverySpi();
+
+        disco.setIpFinder(new TcpDiscoveryVmIpFinder(true));
+
+        cfg.setDiscoverySpi(disco);
+
+        return cfg;
+    }
+
+    /**
+     * @throws Exception If test failed.
+     */
+    public void testWithoutCacheUseNoopSwapSapce() throws Exception {
+        try {
+            Ignite ignite = startGrid(1);
+
+            SwapSpaceSpi spi = ignite.configuration().getSwapSpaceSpi();
+
+            assertNotNull(spi);
+
+            assertTrue(spi instanceof NoopSwapSpaceSpi);
+        }
+        finally {
+            stopAllGrids();
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/ef258ece/modules/core/src/test/java/org/apache/ignite/spi/swapspace/package.html
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/spi/swapspace/package.html b/modules/core/src/test/java/org/apache/ignite/spi/swapspace/package.html
new file mode 100644
index 0000000..5cad80a
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/spi/swapspace/package.html
@@ -0,0 +1,15 @@
+<!DOCTYPE html PUBLIC "-//W3C//DTD HTML 4.01 Transitional//EN" "http://www.w3.org/TR/html4/loose.dtd">
+<!--
+    @html.file.header
+    _________        _____ __________________        _____
+    __  ____/___________(_)______  /__  ____/______ ____(_)_______
+    _  / __  __  ___/__  / _  __  / _  / __  _  __ `/__  / __  __ \
+    / /_/ /  _  /    _  /  / /_/ /  / /_/ /  / /_/ / _  /  _  / / /
+    \____/   /_/     /_/   \_,__/   \____/   \__,_/  /_/   /_/ /_/
+-->
+<html>
+<body>
+    <!-- Package description. -->
+    Contains internal tests or test related classes and interfaces.
+</body>
+</html>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/ef258ece/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 aa89e2a..28863d5 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
@@ -36,8 +36,8 @@ import org.apache.ignite.spi.eventstorage.*;
 import org.apache.ignite.spi.eventstorage.memory.*;
 import org.apache.ignite.spi.failover.always.*;
 import org.apache.ignite.spi.loadbalancing.roundrobin.*;
-import org.gridgain.grid.spi.swapspace.*;
-import org.gridgain.grid.spi.swapspace.file.*;
+import org.apache.ignite.spi.swapspace.*;
+import org.apache.ignite.spi.swapspace.file.*;
 import org.gridgain.testframework.junits.*;
 import org.gridgain.testframework.junits.common.*;
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/ef258ece/modules/core/src/test/java/org/gridgain/grid/kernal/managers/swapspace/GridSwapSpaceManagerSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/gridgain/grid/kernal/managers/swapspace/GridSwapSpaceManagerSelfTest.java b/modules/core/src/test/java/org/gridgain/grid/kernal/managers/swapspace/GridSwapSpaceManagerSelfTest.java
index db45129..5d404ea 100644
--- a/modules/core/src/test/java/org/gridgain/grid/kernal/managers/swapspace/GridSwapSpaceManagerSelfTest.java
+++ b/modules/core/src/test/java/org/gridgain/grid/kernal/managers/swapspace/GridSwapSpaceManagerSelfTest.java
@@ -14,8 +14,8 @@ import org.apache.ignite.configuration.*;
 import org.apache.ignite.events.*;
 import org.apache.ignite.lang.*;
 import org.gridgain.grid.kernal.*;
-import org.gridgain.grid.spi.swapspace.*;
-import org.gridgain.grid.spi.swapspace.file.*;
+import org.apache.ignite.spi.swapspace.*;
+import org.apache.ignite.spi.swapspace.file.*;
 import org.gridgain.grid.util.typedef.*;
 import org.gridgain.grid.util.lang.*;
 import org.gridgain.testframework.junits.common.*;

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/ef258ece/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheAbstractFullApiSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheAbstractFullApiSelfTest.java b/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheAbstractFullApiSelfTest.java
index c246fbb..fe7436a 100644
--- a/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheAbstractFullApiSelfTest.java
+++ b/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheAbstractFullApiSelfTest.java
@@ -16,7 +16,7 @@ import org.apache.ignite.events.*;
 import org.apache.ignite.lang.*;
 import org.gridgain.grid.*;
 import org.gridgain.grid.cache.*;
-import org.gridgain.grid.spi.swapspace.inmemory.*;
+import org.apache.ignite.spi.swapspace.inmemory.*;
 import org.gridgain.grid.util.lang.*;
 import org.gridgain.grid.util.typedef.*;
 import org.gridgain.grid.util.typedef.internal.*;

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/ef258ece/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheMemoryModeSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheMemoryModeSelfTest.java b/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheMemoryModeSelfTest.java
index 84b3132..11befd4 100644
--- a/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheMemoryModeSelfTest.java
+++ b/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheMemoryModeSelfTest.java
@@ -18,7 +18,7 @@ import org.gridgain.grid.cache.eviction.lru.*;
 import org.apache.ignite.spi.discovery.tcp.*;
 import org.apache.ignite.spi.discovery.tcp.ipfinder.*;
 import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.*;
-import org.gridgain.grid.spi.swapspace.file.*;
+import org.apache.ignite.spi.swapspace.file.*;
 import org.gridgain.grid.util.typedef.*;
 import org.gridgain.testframework.junits.common.*;
 import org.junit.*;

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/ef258ece/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheOffHeapSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheOffHeapSelfTest.java b/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheOffHeapSelfTest.java
index f741cbd..68a1b0b 100644
--- a/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheOffHeapSelfTest.java
+++ b/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheOffHeapSelfTest.java
@@ -19,7 +19,7 @@ import org.gridgain.grid.cache.query.*;
 import org.apache.ignite.spi.discovery.tcp.*;
 import org.apache.ignite.spi.discovery.tcp.ipfinder.*;
 import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.*;
-import org.gridgain.grid.spi.swapspace.file.*;
+import org.apache.ignite.spi.swapspace.file.*;
 import org.gridgain.grid.util.typedef.*;
 import org.gridgain.grid.util.typedef.internal.*;
 import org.gridgain.testframework.junits.common.*;

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/ef258ece/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheOffHeapTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheOffHeapTest.java b/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheOffHeapTest.java
index 66e2adb..7b03c6e 100644
--- a/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheOffHeapTest.java
+++ b/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheOffHeapTest.java
@@ -16,7 +16,7 @@ import org.gridgain.grid.cache.eviction.fifo.*;
 import org.apache.ignite.spi.discovery.tcp.*;
 import org.apache.ignite.spi.discovery.tcp.ipfinder.*;
 import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.*;
-import org.gridgain.grid.spi.swapspace.noop.*;
+import org.apache.ignite.spi.swapspace.noop.*;
 import org.gridgain.testframework.junits.common.*;
 
 import java.util.concurrent.*;

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/ef258ece/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheP2PUndeploySelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheP2PUndeploySelfTest.java b/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheP2PUndeploySelfTest.java
index 6954e2a..1f99504 100644
--- a/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheP2PUndeploySelfTest.java
+++ b/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheP2PUndeploySelfTest.java
@@ -19,7 +19,7 @@ import org.gridgain.grid.kernal.processors.cache.distributed.*;
 import org.apache.ignite.spi.discovery.tcp.*;
 import org.apache.ignite.spi.discovery.tcp.ipfinder.*;
 import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.*;
-import org.gridgain.grid.spi.swapspace.file.*;
+import org.apache.ignite.spi.swapspace.file.*;
 import org.gridgain.grid.util.typedef.*;
 import org.gridgain.grid.util.typedef.internal.*;
 import org.gridgain.testframework.junits.common.*;

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/ef258ece/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheSwapReloadSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheSwapReloadSelfTest.java b/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheSwapReloadSelfTest.java
index 18ddd46..063d3ac 100644
--- a/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheSwapReloadSelfTest.java
+++ b/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheSwapReloadSelfTest.java
@@ -19,8 +19,8 @@ import org.gridgain.grid.kernal.*;
 import org.apache.ignite.spi.discovery.tcp.*;
 import org.apache.ignite.spi.discovery.tcp.ipfinder.*;
 import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.*;
-import org.gridgain.grid.spi.swapspace.*;
-import org.gridgain.grid.spi.swapspace.file.*;
+import org.apache.ignite.spi.swapspace.*;
+import org.apache.ignite.spi.swapspace.file.*;
 import org.gridgain.grid.util.typedef.*;
 import org.gridgain.grid.util.typedef.internal.*;
 import org.gridgain.testframework.junits.common.*;

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/ef258ece/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/distributed/GridCacheAbstractDistributedByteArrayValuesSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/distributed/GridCacheAbstractDistributedByteArrayValuesSelfTest.java b/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/distributed/GridCacheAbstractDistributedByteArrayValuesSelfTest.java
index 0852e09..7029753 100644
--- a/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/distributed/GridCacheAbstractDistributedByteArrayValuesSelfTest.java
+++ b/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/distributed/GridCacheAbstractDistributedByteArrayValuesSelfTest.java
@@ -13,7 +13,7 @@ import org.apache.ignite.*;
 import org.apache.ignite.configuration.*;
 import org.gridgain.grid.cache.*;
 import org.gridgain.grid.kernal.processors.cache.*;
-import org.gridgain.grid.spi.swapspace.file.*;
+import org.apache.ignite.spi.swapspace.file.*;
 import org.jetbrains.annotations.*;
 
 import java.util.*;

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/ef258ece/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/distributed/dht/GridCacheAbstractPartitionedOnlyByteArrayValuesSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/distributed/dht/GridCacheAbstractPartitionedOnlyByteArrayValuesSelfTest.java b/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/distributed/dht/GridCacheAbstractPartitionedOnlyByteArrayValuesSelfTest.java
index 66d1af2..ac23a7e 100644
--- a/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/distributed/dht/GridCacheAbstractPartitionedOnlyByteArrayValuesSelfTest.java
+++ b/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/distributed/dht/GridCacheAbstractPartitionedOnlyByteArrayValuesSelfTest.java
@@ -12,7 +12,7 @@ package org.gridgain.grid.kernal.processors.cache.distributed.dht;
 import org.apache.ignite.configuration.*;
 import org.gridgain.grid.cache.*;
 import org.gridgain.grid.kernal.processors.cache.distributed.*;
-import org.gridgain.grid.spi.swapspace.file.*;
+import org.apache.ignite.spi.swapspace.file.*;
 
 import static org.gridgain.grid.cache.GridCacheAtomicWriteOrderMode.*;
 import static org.gridgain.grid.cache.GridCacheAtomicityMode.*;

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/ef258ece/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/distributed/dht/GridCacheColocatedOptimisticTransactionSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/distributed/dht/GridCacheColocatedOptimisticTransactionSelfTest.java b/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/distributed/dht/GridCacheColocatedOptimisticTransactionSelfTest.java
index bbc76cd..7fefb15 100644
--- a/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/distributed/dht/GridCacheColocatedOptimisticTransactionSelfTest.java
+++ b/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/distributed/dht/GridCacheColocatedOptimisticTransactionSelfTest.java
@@ -15,7 +15,7 @@ import org.gridgain.grid.cache.*;
 import org.apache.ignite.spi.discovery.tcp.*;
 import org.apache.ignite.spi.discovery.tcp.ipfinder.*;
 import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.*;
-import org.gridgain.grid.spi.swapspace.file.*;
+import org.apache.ignite.spi.swapspace.file.*;
 import org.gridgain.grid.util.typedef.*;
 import org.gridgain.testframework.junits.common.*;
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/ef258ece/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/distributed/replicated/GridCacheReplicatedUnswapAdvancedSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/distributed/replicated/GridCacheReplicatedUnswapAdvancedSelfTest.java b/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/distributed/replicated/GridCacheReplicatedUnswapAdvancedSelfTest.java
index a2c721c..9abeada 100644
--- a/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/distributed/replicated/GridCacheReplicatedUnswapAdvancedSelfTest.java
+++ b/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/distributed/replicated/GridCacheReplicatedUnswapAdvancedSelfTest.java
@@ -17,7 +17,7 @@ import org.gridgain.grid.cache.*;
 import org.apache.ignite.spi.discovery.tcp.*;
 import org.apache.ignite.spi.discovery.tcp.ipfinder.*;
 import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.*;
-import org.gridgain.grid.spi.swapspace.file.*;
+import org.apache.ignite.spi.swapspace.file.*;
 import org.gridgain.grid.util.typedef.internal.*;
 import org.gridgain.testframework.*;
 import org.gridgain.testframework.junits.common.*;

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/ef258ece/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/local/GridCacheLocalByteArrayValuesSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/local/GridCacheLocalByteArrayValuesSelfTest.java b/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/local/GridCacheLocalByteArrayValuesSelfTest.java
index 0abba7b..f9eb36b 100644
--- a/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/local/GridCacheLocalByteArrayValuesSelfTest.java
+++ b/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/local/GridCacheLocalByteArrayValuesSelfTest.java
@@ -13,7 +13,7 @@ import org.apache.ignite.*;
 import org.apache.ignite.configuration.*;
 import org.gridgain.grid.cache.*;
 import org.gridgain.grid.kernal.processors.cache.*;
-import org.gridgain.grid.spi.swapspace.file.*;
+import org.apache.ignite.spi.swapspace.file.*;
 import org.gridgain.grid.util.typedef.*;
 import org.jetbrains.annotations.*;
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/ef258ece/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/query/GridCacheSwapScanQueryAbstractSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/query/GridCacheSwapScanQueryAbstractSelfTest.java b/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/query/GridCacheSwapScanQueryAbstractSelfTest.java
index 37c3707..1f74051 100644
--- a/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/query/GridCacheSwapScanQueryAbstractSelfTest.java
+++ b/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/query/GridCacheSwapScanQueryAbstractSelfTest.java
@@ -18,7 +18,7 @@ import org.gridgain.grid.cache.query.*;
 import org.apache.ignite.spi.discovery.tcp.*;
 import org.apache.ignite.spi.discovery.tcp.ipfinder.*;
 import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.*;
-import org.gridgain.grid.spi.swapspace.file.*;
+import org.apache.ignite.spi.swapspace.file.*;
 import org.gridgain.testframework.*;
 import org.gridgain.testframework.junits.common.*;
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/ef258ece/modules/core/src/test/java/org/gridgain/grid/loadtest/swapspace/GridFileSwapSpaceSpiMultithreadedLoadTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/gridgain/grid/loadtest/swapspace/GridFileSwapSpaceSpiMultithreadedLoadTest.java b/modules/core/src/test/java/org/gridgain/grid/loadtest/swapspace/GridFileSwapSpaceSpiMultithreadedLoadTest.java
index 68d0d4d..4f57454 100644
--- a/modules/core/src/test/java/org/gridgain/grid/loadtest/swapspace/GridFileSwapSpaceSpiMultithreadedLoadTest.java
+++ b/modules/core/src/test/java/org/gridgain/grid/loadtest/swapspace/GridFileSwapSpaceSpiMultithreadedLoadTest.java
@@ -12,8 +12,8 @@ package org.gridgain.grid.loadtest.swapspace;
 import org.apache.ignite.lang.*;
 import org.apache.ignite.spi.*;
 import org.gridgain.grid.*;
-import org.gridgain.grid.spi.swapspace.*;
-import org.gridgain.grid.spi.swapspace.file.*;
+import org.apache.ignite.spi.swapspace.*;
+import org.apache.ignite.spi.swapspace.file.*;
 import org.gridgain.grid.util.typedef.*;
 import org.gridgain.loadtests.util.*;
 import org.gridgain.testframework.*;

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/ef258ece/modules/core/src/test/java/org/gridgain/grid/spi/swapspace/GridSwapSpaceSpiAbstractSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/gridgain/grid/spi/swapspace/GridSwapSpaceSpiAbstractSelfTest.java b/modules/core/src/test/java/org/gridgain/grid/spi/swapspace/GridSwapSpaceSpiAbstractSelfTest.java
deleted file mode 100644
index 0a5f28a..0000000
--- a/modules/core/src/test/java/org/gridgain/grid/spi/swapspace/GridSwapSpaceSpiAbstractSelfTest.java
+++ /dev/null
@@ -1,634 +0,0 @@
-/* @java.file.header */
-
-/*  _________        _____ __________________        _____
- *  __  ____/___________(_)______  /__  ____/______ ____(_)_______
- *  _  / __  __  ___/__  / _  __  / _  / __  _  __ `/__  / __  __ \
- *  / /_/ /  _  /    _  /  / /_/ /  / /_/ /  / /_/ / _  /  _  / / /
- *  \____/   /_/     /_/   \_,__/   \____/   \__,_/  /_/   /_/ /_/
- */
-
-package org.gridgain.grid.spi.swapspace;
-
-import org.apache.ignite.lang.*;
-import org.apache.ignite.spi.*;
-import org.gridgain.grid.util.typedef.*;
-import org.gridgain.grid.util.typedef.internal.*;
-import org.gridgain.testframework.junits.common.*;
-import org.jetbrains.annotations.*;
-
-import java.io.*;
-import java.util.*;
-import java.util.concurrent.*;
-
-import static java.util.concurrent.TimeUnit.*;
-import static org.apache.ignite.events.IgniteEventType.*;
-import static org.junit.Assert.*;
-
-/**
- * Test for various {@link SwapSpaceSpi} implementations.
- */
-public abstract class GridSwapSpaceSpiAbstractSelfTest extends GridCommonAbstractTest {
-    /** Default swap space name. */
-    private static final String DFLT_SPACE_NAME = "dflt-space";
-
-    /** */
-    protected static final String SPACE1 = "space1";
-
-    /** */
-    protected static final String SPACE2 = "space2";
-
-    /** SPI to test. */
-    protected SwapSpaceSpi spi;
-
-    /**
-     * @return New {@link SwapSpaceSpi} instance.
-     */
-    protected abstract SwapSpaceSpi spi();
-
-    /** {@inheritDoc} */
-    @Override protected void beforeTest() throws Exception {
-        U.setWorkDirectory(null, U.getGridGainHome());
-
-        spi = spi();
-
-        getTestResources().inject(spi);
-
-        spi.spiStart("");
-
-        spi.clear(DFLT_SPACE_NAME);
-    }
-
-    /** @throws Exception If failed. */
-    @Override protected void afterTest() throws Exception {
-        spi.spiStop();
-    }
-
-    /**
-     * @return Swap context.
-     */
-    protected SwapContext context() {
-        return context(null);
-    }
-
-    /**
-     * @param clsLdr Class loader.
-     * @return Swap context.
-     */
-    private SwapContext context(@Nullable ClassLoader clsLdr) {
-        SwapContext ctx = new SwapContext();
-
-        ctx.classLoader(clsLdr != null ? clsLdr : getClass().getClassLoader());
-
-        return ctx;
-    }
-
-    /**
-     * @param s String.
-     * @return Byte array.
-     */
-    protected byte[] str2ByteArray(String s) {
-        return s.getBytes();
-    }
-
-    /**
-     * Tests the Create-Read-Update-Delete operations with a simple key.
-     *
-     * @throws Exception If failed.
-     */
-    public void testSimpleCrud() throws Exception {
-        assertEquals(0, spi.count(DFLT_SPACE_NAME));
-
-        long key1 = 1;
-
-        byte[] val1 = Long.toString(key1).getBytes();
-
-        spi.store(DFLT_SPACE_NAME, new SwapKey(key1), val1, context());
-
-        assertEquals(1, spi.count(DFLT_SPACE_NAME));
-
-        assertArrayEquals(spi.read(DFLT_SPACE_NAME, new SwapKey(key1), context()), val1);
-
-        final byte[] val2 = "newValue".getBytes();
-
-        spi.store(DFLT_SPACE_NAME, new SwapKey(key1), val2, context());
-
-        assertEquals(1, spi.count(DFLT_SPACE_NAME));
-
-        assertArrayEquals(spi.read(DFLT_SPACE_NAME, new SwapKey(key1), context()), val2);
-
-        spi.remove(DFLT_SPACE_NAME, new SwapKey(key1), new IgniteInClosure<byte[]>() {
-            @Override public void apply(byte[] old) {
-                assertArrayEquals(val2, old);
-            }
-        }, context());
-
-        assertEquals(0, spi.count(DFLT_SPACE_NAME));
-    }
-
-    /**
-     * Tests the Create-Read-Update-Delete operations with a simple key
-     * and different spaces.
-     *
-     * @throws Exception If failed.
-     */
-    public void testSimpleCrudDifferentSpaces() throws Exception {
-        String space1 = SPACE1;
-
-        spi.clear(space1);
-
-        String space2 = SPACE2;
-
-        spi.clear(space2);
-
-        assertEquals(0, spi.count(space1));
-
-        assertEquals(0, spi.count(space2));
-
-        long key1 = 1;
-
-        final byte[] val1 = Long.toString(key1).getBytes();
-
-        spi.store(space1, new SwapKey(key1), val1, context());
-
-        assertEquals(1, spi.count(space1));
-
-        assertEquals(0, spi.count(space2));
-
-        spi.store(space2, new SwapKey(key1), val1, context());
-
-        assertEquals(1, spi.count(space1));
-
-        assertEquals(1, spi.count(space2));
-
-        assertArrayEquals(spi.read(space1, new SwapKey(key1), context()), val1);
-
-        assertArrayEquals(spi.read(space2, new SwapKey(key1), context()), val1);
-
-        long key2 = 2;
-
-        byte[] val2 = Long.toString(key2).getBytes();
-
-        spi.store(space1, new SwapKey(key2), val2, context());
-
-        assertEquals(2, spi.count(space1));
-
-        assertEquals(1, spi.count(space2));
-
-        assertArrayEquals(spi.read(space1, new SwapKey(key2), context()), val2);
-
-        assertNull(spi.read(space2, new SwapKey(key2), context()));
-
-        final byte[] val12 = "newValue".getBytes();
-
-        spi.store(space1, new SwapKey(key1), val12, context());
-
-        assertEquals(2, spi.count(space1));
-
-        assertEquals(1, spi.count(space2));
-
-        assertArrayEquals(spi.read(space1, new SwapKey(key1), context()), val12);
-
-        assertArrayEquals(spi.read(space2, new SwapKey(key1), context()), val1);
-
-        spi.remove(space1, new SwapKey(key1), new IgniteInClosure<byte[]>() {
-            @Override public void apply(byte[] old) {
-                assertArrayEquals(val12, old);
-            }
-        }, context());
-
-        assertEquals(1, spi.count(space1));
-
-        assertEquals(1, spi.count(space2));
-
-        spi.remove(space2, new SwapKey(key1), new IgniteInClosure<byte[]>() {
-            @Override public void apply(byte[] old) {
-                assertArrayEquals(val1, old);
-            }
-        }, context());
-
-        assertEquals(1, spi.count(space1));
-
-        assertEquals(0, spi.count(space2));
-    }
-
-    /**
-     * Tests the Create-Update-Delete operations with a key batches.
-     *
-     * @throws Exception If failed.
-     */
-    public void testBatchCrud() throws Exception {
-        assertEquals(0, spi.count(DFLT_SPACE_NAME));
-
-        final Map<SwapKey, byte[]> batch = new HashMap<>();
-
-        int batchSize = 10;
-
-        // Generate initial values.
-        for (int i = 0; i < batchSize; i++)
-            batch.put(new SwapKey(i), Integer.toString(i).getBytes());
-
-        spi.storeAll(DFLT_SPACE_NAME, batch, context());
-
-        assertEquals(batchSize, spi.count(DFLT_SPACE_NAME));
-
-        Map<SwapKey, byte[]> read = spi.readAll(DFLT_SPACE_NAME, batch.keySet(), context());
-
-        // Check all entries are as expected.
-        assertTrue(F.forAll(read, new P1<Map.Entry<SwapKey, byte[]>>() {
-            @Override public boolean apply(Map.Entry<SwapKey, byte[]> e) {
-                return Arrays.equals(batch.get(e.getKey()), e.getValue());
-            }
-        }));
-
-        // Generate new values.
-        for (int i = 0; i < batchSize; i++)
-            batch.put(new SwapKey(i), Integer.toString(i + 1).getBytes());
-
-        spi.storeAll(DFLT_SPACE_NAME, batch, context());
-
-        assertEquals(batchSize, spi.count(DFLT_SPACE_NAME));
-
-        read = spi.readAll(DFLT_SPACE_NAME, batch.keySet(), context());
-
-        // Check all entries are as expected.
-        assertTrue(F.forAll(read, new P1<Map.Entry<SwapKey, byte[]>>() {
-            @Override public boolean apply(Map.Entry<SwapKey, byte[]> e) {
-                return Arrays.equals(batch.get(e.getKey()), e.getValue());
-            }
-        }));
-
-        spi.removeAll(DFLT_SPACE_NAME, batch.keySet(), null, context());
-
-        assertEquals(0, spi.count(DFLT_SPACE_NAME));
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testDeleteIfNotPersist() throws Exception {
-        spi.store(SPACE1, new SwapKey("key1"), "value1".getBytes(), context());
-
-        assertArrayEquals("value1".getBytes(), spi.read(SPACE1, new SwapKey("key1"), context()));
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testStoreReadRemove() throws Exception {
-        int cnt = 5;
-
-        final CountDownLatch storeLatch = new CountDownLatch(cnt);
-        final CountDownLatch readLatch = new CountDownLatch(cnt);
-        final CountDownLatch rmvLatch = new CountDownLatch(cnt);
-
-        spi.setListener(new SwapSpaceSpiListener() {
-            @Override public void onSwapEvent(int evtType, @Nullable String spaceName, @Nullable byte[] keyBytes) {
-                info("Received event: " + evtType);
-
-                if (evtType == EVT_SWAP_SPACE_DATA_STORED)
-                    storeLatch.countDown();
-
-                else if (evtType == EVT_SWAP_SPACE_DATA_READ)
-                    readLatch.countDown();
-
-                else if (evtType == EVT_SWAP_SPACE_DATA_REMOVED)
-                    rmvLatch.countDown();
-
-                else
-                    assert false : "Unexpected event type: " + evtType;
-            }
-        });
-
-        for (int i = 0; i < cnt; i++)
-            assertNull(spi.read(SPACE1, new SwapKey("key" + i), context()));
-
-        for (int i = 0; i < cnt; i++)
-            spi.store(SPACE1, new SwapKey("key" + i), str2ByteArray("value" + i), context());
-
-        assert storeLatch.await(5000, MILLISECONDS);
-
-        for (int i = 0; i < cnt; i++)
-            assertArrayEquals(str2ByteArray("value" + i), spi.read(SPACE1, new SwapKey("key" + i), context()));
-
-        assert readLatch.await(5000, MILLISECONDS);
-
-        for (int i = 0; i < cnt; i++) {
-            final int tmp = i;
-
-            spi.remove(SPACE1, new SwapKey("key" + i), new CI1<byte[]>() {
-                @Override public void apply(byte[] arr) {
-                    assertArrayEquals(str2ByteArray("value" + tmp), arr);
-
-                    info("Removed correct value for: key" + tmp);
-                }
-            }, context());
-        }
-
-        assert rmvLatch.await(10000, MILLISECONDS);
-
-        for (int i = 0; i < cnt; i++)
-            assertNull(spi.read(SPACE1, new SwapKey("key" + i), context()));
-    }
-
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testStoreReadRemoveNulls() throws Exception {
-        int cnt = 5;
-
-        final CountDownLatch storeLatch = new CountDownLatch(cnt);
-        final CountDownLatch readLatch = new CountDownLatch(cnt);
-        final CountDownLatch rmvLatch = new CountDownLatch(cnt);
-
-        spi.setListener(new SwapSpaceSpiListener() {
-            @Override public void onSwapEvent(int evtType, @Nullable String spaceName, @Nullable byte[] keyBytes) {
-                info("Received event: " + evtType);
-
-                if (evtType == EVT_SWAP_SPACE_DATA_STORED)
-                    storeLatch.countDown();
-
-                else if (evtType == EVT_SWAP_SPACE_DATA_READ)
-                    readLatch.countDown();
-
-                else if (evtType == EVT_SWAP_SPACE_DATA_REMOVED)
-                    rmvLatch.countDown();
-
-                else
-                    assert false : "Unexpected event type: " + evtType;
-            }
-        });
-
-        for (int i = 0; i < cnt; i++)
-            assertNull(spi.read(SPACE1, new SwapKey("key" + i), context()));
-
-        for (int i = 0; i < cnt; i++)
-            spi.store(SPACE1, new SwapKey("key" + i), null, context());
-
-        assert storeLatch.await(5000, MILLISECONDS);
-
-        for (int i = 0; i < cnt; i++)
-            assertNull(spi.read(SPACE1, new SwapKey("key" + i), context()));
-
-        assert readLatch.await(5000, MILLISECONDS);
-
-        for (int i = 0; i < cnt; i++) {
-            final int tmp = i;
-
-            spi.remove(SPACE1, new SwapKey("key" + i), new CI1<byte[]>() {
-                @Override public void apply(byte[] arr) {
-                    assertNull(arr);
-
-                    info("Removed correct value for: key" + tmp);
-                }
-            }, context());
-        }
-
-        assert rmvLatch.await(10000, MILLISECONDS);
-
-        for (int i = 0; i < cnt; i++)
-            assertNull(spi.read(SPACE1, new SwapKey("key" + i), context()));
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testCollisions() throws Exception {
-        int cnt = 5;
-
-        final CountDownLatch storeLatch = new CountDownLatch(cnt);
-        final CountDownLatch readLatch = new CountDownLatch(cnt);
-        final CountDownLatch rmvLatch = new CountDownLatch(cnt);
-
-        spi.setListener(new SwapSpaceSpiListener() {
-            @Override public void onSwapEvent(int evtType, @Nullable String spaceName, @Nullable byte[] keyBytes) {
-                info("Received event: " + evtType);
-
-                if (evtType == EVT_SWAP_SPACE_DATA_STORED)
-                    storeLatch.countDown();
-
-                else if (evtType == EVT_SWAP_SPACE_DATA_READ)
-                    readLatch.countDown();
-
-                else if (evtType == EVT_SWAP_SPACE_DATA_REMOVED)
-                    rmvLatch.countDown();
-
-                else
-                    assert false : "Unexpected event type: " + evtType;
-            }
-        });
-
-        List<Integer> keys = new ArrayList<>(cnt);
-
-        final Map<Integer, String> entries = new HashMap<>();
-
-        for (int i = 0; i < cnt; i++) {
-            String val = "value" + i;
-
-            spi.store(SPACE1, new SwapKey(new Key(i)), str2ByteArray(val), context());
-
-            keys.add(i);
-
-            entries.put(i, val);
-        }
-
-        assert storeLatch.await(5000, MILLISECONDS) : "Count: " + storeLatch.getCount();
-
-        for (int i = 0; i < cnt; i++)
-            assertArrayEquals(entries.get(i).getBytes(),
-                spi.read(SPACE1, new SwapKey(new Key(i)), context()));
-
-        assert readLatch.await(5000, MILLISECONDS) : "Count: " + readLatch.getCount();
-
-        Collections.shuffle(keys);
-
-        for (final Integer key : keys) {
-            spi.remove(SPACE1, new SwapKey(new Key(key)), new CI1<byte[]>() {
-                @Override public void apply(byte[] arr) {
-                    assertArrayEquals(entries.get(key).getBytes(), arr);
-
-                    info("Removed correct entry for key: " + key);
-                }
-            }, context());
-        }
-
-        assert rmvLatch.await(5000, MILLISECONDS) : "Count: " + rmvLatch.getCount();
-
-        for (final Integer key : keys)
-            assertNull(spi.read(SPACE1, new SwapKey(new Key(key)), context()));
-    }
-
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testIteration() throws Exception {
-        spi.clear(SPACE1);
-
-        int cnt = 10;
-
-        for (int i = 0; i < cnt; i++)
-            spi.store(SPACE1, new SwapKey("key" + i, i), str2ByteArray("value" + i), context());
-
-        for (int i = 0; i < cnt; i++)
-            assertArrayEquals(str2ByteArray("value" + i),
-                spi.read(SPACE1, new SwapKey("key" + i, i), context()));
-
-        try (IgniteSpiCloseableIterator<Map.Entry<byte[], byte[]>> iter = spi.rawIterator(SPACE1)) {
-            assertNotNull(iter);
-
-            int i = 0;
-
-            while (iter.hasNext()) {
-                Map.Entry<byte[], byte[]> next = iter.next();
-
-                String key = getTestResources().getMarshaller().unmarshal(next.getKey(), null);
-
-                info("Got from iterator [key=" + key + ", val=" + new String(next.getValue()));
-
-                i++;
-
-                iter.remove();
-            }
-
-            assertEquals(10, i);
-        }
-
-        assertEquals(0, spi.count(SPACE1));
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testIterationOverPartition() throws Exception {
-        spi.store(SPACE1, new SwapKey("key", 0), str2ByteArray("value"), context());
-
-        spi.clear(SPACE1);
-
-        int cnt = 10;
-
-        for (int i = 0; i < cnt; i++)
-            spi.store(SPACE1, new SwapKey("key" + i, i), str2ByteArray("value" + i), context());
-
-        for (int i = 0; i < cnt; i++)
-            assertArrayEquals(str2ByteArray("value" + i),
-                spi.read(SPACE1, new SwapKey("key" + i, i), context()));
-
-        try (IgniteSpiCloseableIterator<Map.Entry<byte[], byte[]>> iter = spi.rawIterator(SPACE1, 5)) {
-            assertNotNull(iter);
-
-            int i = 0;
-
-            while (iter.hasNext()) {
-                Map.Entry<byte[], byte[]> next = iter.next();
-
-                String key = getTestResources().getMarshaller().unmarshal(next.getKey(), null);
-
-                info("Got from iterator [key=" + key + ", val=" + new String(next.getValue()));
-
-                assert "key5".equals(key);
-
-                iter.remove();
-
-                assertNull(spi.read(SPACE1, new SwapKey(key, 5), context()));
-
-                i++;
-            }
-
-            assertEquals(1, i);
-        }
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testSwapIterator() throws Exception {
-        spi.store(SPACE1, new SwapKey("key", 0), str2ByteArray("value"), context());
-
-        spi.clear(SPACE1);
-
-        int cnt = 10;
-
-        for (int i = 0; i < cnt; i++)
-            spi.store(SPACE1, new SwapKey("key" + i, i), str2ByteArray("value" + i), context());
-
-        IgniteSpiCloseableIterator<Map.Entry<byte[], byte[]>> iter = spi.rawIterator(SPACE1);
-
-        assertNotNull(iter);
-
-        iter.close();
-
-        try {
-            iter.next();
-
-            assert false;
-        }
-        catch (NoSuchElementException e) {
-            info("Caught expected exception (illegal state): " + e);
-        }
-    }
-
-    /**
-     *
-     */
-    @SuppressWarnings({"PublicInnerClass"})
-    public static class TestValue implements Serializable {
-        /** */
-        private String val = "test-" + System.currentTimeMillis();
-
-        /**
-         * @return Value
-         */
-        public String getValue() {
-            return val;
-        }
-
-        /** {@inheritDoc} */
-        @Override public boolean equals(Object obj) {
-            return obj instanceof TestValue && val.equals(((TestValue)obj).val);
-        }
-
-        /** {@inheritDoc} */
-        @Override public String toString() {
-            return S.toString(TestValue.class, this);
-        }
-    }
-
-    /**
-     * Key.
-     */
-    private static class Key {
-        /** Index. */
-        private final int i;
-
-        /**
-         * @param i Index.
-         */
-        Key(int i) {
-            this.i = i;
-        }
-
-        /** {@inheritDoc} */
-        @Override public boolean equals(Object o) {
-            if (this == o)
-                return true;
-
-            if (!(o instanceof Key))
-                return false;
-
-            Key key = (Key)o;
-
-            return i == key.i;
-        }
-
-        /** {@inheritDoc} */
-        @Override public int hashCode() {
-            return 1; // 100% collision.
-        }
-
-        /** {@inheritDoc} */
-        @Override public String toString() {
-            return "Key: " + i;
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/ef258ece/modules/core/src/test/java/org/gridgain/grid/spi/swapspace/file/GridFileSwapCompactionSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/gridgain/grid/spi/swapspace/file/GridFileSwapCompactionSelfTest.java b/modules/core/src/test/java/org/gridgain/grid/spi/swapspace/file/GridFileSwapCompactionSelfTest.java
deleted file mode 100644
index 73b2491..0000000
--- a/modules/core/src/test/java/org/gridgain/grid/spi/swapspace/file/GridFileSwapCompactionSelfTest.java
+++ /dev/null
@@ -1,119 +0,0 @@
-/* @java.file.header */
-
-/*  _________        _____ __________________        _____
- *  __  ____/___________(_)______  /__  ____/______ ____(_)_______
- *  _  / __  __  ___/__  / _  __  / _  / __  _  __ `/__  / __  __ \
- *  / /_/ /  _  /    _  /  / /_/ /  / /_/ /  / /_/ / _  /  _  / / /
- *  \____/   /_/     /_/   \_,__/   \____/   \__,_/  /_/   /_/ /_/
- */
-
-package org.gridgain.grid.spi.swapspace.file;
-
-import org.gridgain.grid.util.typedef.*;
-import org.gridgain.testframework.junits.common.*;
-
-import java.io.File;
-import java.nio.ByteBuffer;
-import java.util.*;
-
-/**
- * Test for {@link FileSwapSpaceSpi}.
- */
-public class GridFileSwapCompactionSelfTest extends GridCommonAbstractTest {
-    /**
-     * @throws Exception If failed.
-     */
-    public void testCompact() throws Exception {
-        File file = new File(UUID.randomUUID().toString());
-
-        X.println("file: " + file.getPath());
-
-        FileSwapSpaceSpi.SwapFile f = new FileSwapSpaceSpi.SwapFile(file, 8);
-
-        Random rnd = new Random();
-
-        ArrayList<FileSwapSpaceSpi.SwapValue> arr = new ArrayList<>();
-
-        int size = 0;
-
-        for (int a = 0; a < 100; a++) {
-            FileSwapSpaceSpi.SwapValue[] vals = new FileSwapSpaceSpi.SwapValue[1 + rnd.nextInt(10)];
-
-            int size0 = 0;
-
-            for (int i = 0; i < vals.length; i++) {
-                byte[] bytes = new byte[1 + rnd.nextInt(49)];
-
-                rnd.nextBytes(bytes);
-
-                size0 += bytes.length;
-
-                vals[i] = new FileSwapSpaceSpi.SwapValue(bytes);
-
-                arr.add(vals[i]);
-            }
-
-            f.write(new FileSwapSpaceSpi.SwapValues(vals, size0), 1);
-
-            size += size0;
-
-            assertEquals(f.length(), size);
-            assertEquals(file.length(), size);
-        }
-
-        int i = 0;
-
-        for (FileSwapSpaceSpi.SwapValue val : arr)
-            assertEquals(val.idx(), ++i);
-
-        i = 0;
-
-        for (int cnt = arr.size() / 2; i < cnt; i++) {
-
-            FileSwapSpaceSpi.SwapValue v = arr.remove(rnd.nextInt(arr.size()));
-
-            assertTrue(f.tryRemove(v.idx(), v));
-        }
-
-        int hash0 = 0;
-
-        for (FileSwapSpaceSpi.SwapValue val : arr)
-            hash0 += Arrays.hashCode(val.readValue(f.readCh));
-
-        ArrayList<T2<ByteBuffer, ArrayDeque<FileSwapSpaceSpi.SwapValue>>> bufs = new ArrayList();
-
-        for (;;) {
-            ArrayDeque<FileSwapSpaceSpi.SwapValue> que = new ArrayDeque<>();
-
-            ByteBuffer buf = f.compact(que, 1024);
-
-            if (buf == null)
-                break;
-
-            bufs.add(new T2(buf, que));
-        }
-
-        f.delete();
-
-        int hash1 = 0;
-
-        for (FileSwapSpaceSpi.SwapValue val : arr)
-            hash1 += Arrays.hashCode(val.value(null));
-
-        assertEquals(hash0, hash1);
-
-        File file0 = new File(UUID.randomUUID().toString());
-
-        FileSwapSpaceSpi.SwapFile f0 = new FileSwapSpaceSpi.SwapFile(file0, 8);
-
-        for (T2<ByteBuffer, ArrayDeque<FileSwapSpaceSpi.SwapValue>> t : bufs)
-            f0.write(t.get2(), t.get1(), 1);
-
-        int hash2 = 0;
-
-        for (FileSwapSpaceSpi.SwapValue val : arr)
-            hash2 += Arrays.hashCode(val.readValue(f0.readCh));
-
-        assertEquals(hash2, hash1);
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/ef258ece/modules/core/src/test/java/org/gridgain/grid/spi/swapspace/file/GridFileSwapSpaceSpiSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/gridgain/grid/spi/swapspace/file/GridFileSwapSpaceSpiSelfTest.java b/modules/core/src/test/java/org/gridgain/grid/spi/swapspace/file/GridFileSwapSpaceSpiSelfTest.java
deleted file mode 100644
index a7b7f85..0000000
--- a/modules/core/src/test/java/org/gridgain/grid/spi/swapspace/file/GridFileSwapSpaceSpiSelfTest.java
+++ /dev/null
@@ -1,345 +0,0 @@
-/* @java.file.header */
-
-/*  _________        _____ __________________        _____
- *  __  ____/___________(_)______  /__  ____/______ ____(_)_______
- *  _  / __  __  ___/__  / _  __  / _  / __  _  __ `/__  / __  __ \
- *  / /_/ /  _  /    _  /  / /_/ /  / /_/ /  / /_/ / _  /  _  / / /
- *  \____/   /_/     /_/   \_,__/   \____/   \__,_/  /_/   /_/ /_/
- */
-
-package org.gridgain.grid.spi.swapspace.file;
-
-import org.apache.ignite.lang.*;
-import org.apache.ignite.spi.*;
-import org.gridgain.grid.spi.swapspace.*;
-import org.gridgain.grid.util.typedef.*;
-import org.gridgain.grid.util.typedef.internal.*;
-import org.jdk8.backport.*;
-import org.jetbrains.annotations.*;
-
-import java.util.*;
-import java.util.concurrent.*;
-import java.util.concurrent.atomic.*;
-
-/**
- * Test for {@link FileSwapSpaceSpi}.
- */
-public class GridFileSwapSpaceSpiSelfTest extends GridSwapSpaceSpiAbstractSelfTest {
-    /** {@inheritDoc} */
-    @Override protected SwapSpaceSpi spi() {
-        FileSwapSpaceSpi s = new FileSwapSpaceSpi();
-
-        s.setMaximumSparsity(0.05f);
-        s.setWriteBufferSize(8 * 1024);
-
-        return s;
-    }
-
-    /**
-     * Tests if SPI works correctly with multithreaded writes.
-     *
-     * @throws Exception If failed.
-     */
-    public void testMultithreadedWrite() throws Exception {
-        final AtomicLong valCntr = new AtomicLong();
-
-        final SwapKey key = new SwapKey("key");
-
-        final CountDownLatch wLatch = new CountDownLatch(1);
-
-        final AtomicBoolean done = new AtomicBoolean();
-
-        IgniteFuture<?> wFut = multithreadedAsync(new Callable<Object>() {
-            @Nullable @Override public Object call() throws Exception {
-                while (!done.get()) {
-                    long val = valCntr.incrementAndGet();
-
-                    spi.store(null, key, Long.toString(val).getBytes(), context());
-
-                    if (val == 1)
-                        wLatch.countDown();
-                }
-
-                return null;
-            }
-        }, 8);
-
-        wLatch.await();
-
-        IgniteFuture<?> rFut = multithreadedAsync(new Callable<Object>() {
-            @Nullable @Override public Object call() throws Exception {
-                while (valCntr.get() < 1000) {
-                    byte[] val = spi.read(null, key, context());
-
-                    assertNotNull(val);
-
-                    long lval = Long.parseLong(new String(val));
-
-                    assertTrue(lval <= valCntr.get());
-                }
-
-                return null;
-            }
-        }, 8);
-
-        rFut.get();
-
-        done.set(true);
-
-        wFut.get();
-    }
-
-    /**
-     * @param i Integer.
-     * @return Swap key.
-     */
-    private SwapKey key(int i) {
-        return new SwapKey(i, i % 11, U.intToBytes(i));
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testMultithreadedOperations() throws Exception {
-        final ConcurrentHashMap8<SwapKey, byte[]> map = new ConcurrentHashMap8<>();
-
-        Random rnd = new Random();
-
-        final int keys = 25000;
-
-        int hash0 = 0;
-
-        final int minValSize = 5;
-        final int maxValSize = 9000; // More than write buffer size.
-
-        for (int i = 0; i < keys; i++) {
-            byte[] val = new byte[minValSize + rnd.nextInt(maxValSize - minValSize)];
-
-            rnd.nextBytes(val);
-
-            hash0 += i * Arrays.hashCode(val);
-
-            assertNull(map.put(key(i), val));
-        }
-
-        assertEquals(keys, map.size());
-
-        for (int i = 0; i < keys; i++)
-            assertTrue(map.containsKey(key(i)));
-
-        final String space = "test_space";
-
-        final AtomicBoolean fin = new AtomicBoolean();
-
-        final IgniteFuture<?> fut = multithreadedAsync(new Callable<Object>() {
-            @Override public Object call() throws Exception {
-                Random rnd = new Random();
-
-                while (!fin.get()) {
-                    final SwapKey key = key(rnd.nextInt(keys));
-
-                    switch(rnd.nextInt(13)) {
-                        case 0: // store
-                            byte[] val = map.remove(key);
-
-                            if (val != null)
-                                spi.store(space, key, val, context());
-
-                            break;
-
-                        case 1: // remove
-                            spi.remove(space, key, new CIX1<byte[]>() {
-                                @Override public void applyx(byte[] bytes) {
-                                    if (bytes != null)
-                                        assertNull(map.putIfAbsent(key, bytes));
-                                }
-                            }, context());
-
-                            break;
-
-                        case 2: // read
-                            for (;;) {
-                                val = spi.read(space, key, context());
-
-                                if (val != null)
-                                    break;
-
-                                val = map.get(key);
-
-                                if (val != null)
-                                    break;
-                            }
-
-                            break;
-
-                        case 3: // storeAll
-                        case 4:
-                        case 9:
-                            Map<SwapKey, byte[]> m = new HashMap<>();
-
-                            int cnt = 1 + rnd.nextInt(25);
-
-                            for (int i = 0; i < cnt; i++) {
-                                SwapKey k = key(rnd.nextInt(keys));
-
-                                val = map.remove(k);
-
-                                if (val != null)
-                                    assertNull(m.put(k, val));
-                            }
-
-                            if (m.isEmpty())
-                                break;
-
-                            spi.storeAll(space, m, context());
-
-                            break;
-
-                        case 5: // readAll
-                            HashSet<SwapKey> s = new HashSet<>();
-
-                            cnt = 1 + rnd.nextInt(25);
-
-                            for (int i = 0; i < cnt; i++) {
-                                SwapKey k = key(rnd.nextInt(keys));
-
-                                val = map.get(k);
-
-                                if (val == null)
-                                    s.add(k);
-                            }
-
-                            while (!s.isEmpty()) {
-                                m = spi.readAll(space, s, context());
-
-                                s.removeAll(m.keySet());
-
-                                Iterator<SwapKey> iter = s.iterator();
-
-                                while (iter.hasNext()) {
-                                    SwapKey k = iter.next();
-
-                                    if (map.containsKey(k))
-                                        iter.remove();
-                                }
-                            }
-
-                            break;
-
-                        case 6: // iterateKeys
-                            IgniteSpiCloseableIterator<Integer> kIt = spi.keyIterator(space, context());
-
-                            if (kIt == null)
-                                break;
-
-                            while (kIt.hasNext())
-                                assertNotNull(kIt.next());
-
-                            kIt.close();
-
-                            break;
-
-                        case 7: // iterate
-                            IgniteSpiCloseableIterator<Map.Entry<byte[], byte[]>> iter = spi.rawIterator(space);
-
-                            if (iter == null)
-                                break;
-
-                            while (iter.hasNext()) {
-                                Map.Entry<byte[], byte[]> entry = iter.next();
-
-                                assertEquals(4, entry.getKey().length);
-
-                                byte[] v = entry.getValue();
-
-                                assertTrue(v.length >= minValSize && v.length < maxValSize);
-                            }
-
-                            iter.close();
-
-                            break;
-
-                        case 8: // iterate partitions
-                            iter = spi.rawIterator(space, rnd.nextInt(11));
-
-                            if (iter == null)
-                                break;
-
-                            while ( iter.hasNext()) {
-                                Map.Entry<byte[], byte[]> entry = iter.next();
-
-                                assertEquals(4, entry.getKey().length);
-
-                                byte[] v = entry.getValue();
-
-                                assertTrue(v.length >= minValSize && v.length < maxValSize);
-                            }
-
-                            iter.close();
-
-                            break;
-
-                        default: // removeAll
-                            s = new HashSet<>();
-
-                            cnt = 1 + rnd.nextInt(25);
-
-                            for (int i = 0; i < cnt; i++) {
-                                SwapKey k = key(rnd.nextInt(keys));
-
-                                val = map.get(k);
-
-                                if (val == null)
-                                    s.add(k);
-                            }
-
-                            if (s.isEmpty())
-                                break;
-
-                            spi.removeAll(space, s, new IgniteBiInClosure<SwapKey, byte[]>() {
-                                @Override public void apply(SwapKey k, byte[] bytes) {
-                                    if (bytes != null)
-                                        assertNull(map.putIfAbsent(k, bytes));
-                                }
-                            }, context());
-
-                            break;
-                    }
-                }
-
-                return null;
-            }
-        }, 39);
-
-        Thread.sleep(60000);
-
-        System.out.println("stopping");
-
-        fin.set(true);
-
-        fut.get();
-
-        assertEquals(keys, map.size() + spi.count(space));
-
-        int hash1 = 0;
-
-        int cnt = 0;
-
-        IgniteSpiCloseableIterator<Map.Entry<byte[], byte[]>> iter = spi.rawIterator(space);
-
-        while (iter.hasNext()) {
-            Map.Entry<byte[], byte[]> entry = iter.next();
-
-            hash1 += U.bytesToInt(entry.getKey(), 0) * Arrays.hashCode(entry.getValue());
-
-            cnt++;
-        }
-
-        assertEquals(cnt, spi.count(space));
-
-        for (Map.Entry<SwapKey, byte[]> entry : map.entrySet())
-            hash1 += (Integer)entry.getKey().key() * Arrays.hashCode(entry.getValue());
-
-        assertEquals(hash0, hash1);
-    }
-}


[12/20] incubator-ignite git commit: # Renaming

Posted by sb...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c31cec7c/modules/core/src/main/java/org/gridgain/grid/spi/swapspace/file/FileSwapSpaceSpi.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/spi/swapspace/file/FileSwapSpaceSpi.java b/modules/core/src/main/java/org/gridgain/grid/spi/swapspace/file/FileSwapSpaceSpi.java
new file mode 100644
index 0000000..7bb91d2
--- /dev/null
+++ b/modules/core/src/main/java/org/gridgain/grid/spi/swapspace/file/FileSwapSpaceSpi.java
@@ -0,0 +1,1843 @@
+/* @java.file.header */
+
+/*  _________        _____ __________________        _____
+ *  __  ____/___________(_)______  /__  ____/______ ____(_)_______
+ *  _  / __  __  ___/__  / _  __  / _  / __  _  __ `/__  / __  __ \
+ *  / /_/ /  _  /    _  /  / /_/ /  / /_/ /  / /_/ / _  /  _  / / /
+ *  \____/   /_/     /_/   \_,__/   \____/   \__,_/  /_/   /_/ /_/
+ */
+
+package org.gridgain.grid.spi.swapspace.file;
+
+import org.apache.ignite.*;
+import org.apache.ignite.lang.*;
+import org.apache.ignite.marshaller.*;
+import org.apache.ignite.resources.*;
+import org.apache.ignite.spi.*;
+import org.gridgain.grid.*;
+import org.gridgain.grid.spi.swapspace.*;
+import org.gridgain.grid.util.*;
+import org.gridgain.grid.util.typedef.*;
+import org.gridgain.grid.util.typedef.internal.*;
+import org.jdk8.backport.*;
+import org.jetbrains.annotations.*;
+
+import java.io.*;
+import java.nio.*;
+import java.nio.channels.*;
+import java.util.*;
+import java.util.concurrent.*;
+import java.util.concurrent.atomic.*;
+import java.util.concurrent.locks.*;
+
+import static org.apache.ignite.events.IgniteEventType.*;
+
+/**
+ * File-based swap space SPI implementation which holds keys in memory. This SPI is used by default.
+ * It is intended for use in cases when value size is bigger than {@code 100} bytes, otherwise it will not
+ * have any positive effect.
+ * <p>
+ * <b>NOTE: This SPI does not support swap eviction currently, manual removes needed to reduce disk space
+ * consumption.</b>
+ * <p>
+ * Every space has a name and when used in combination with in-memory data grid name and local node ID,
+ * space name represents the actual cache name associated with this swap space. Default name is {@code null}
+ * which is represented by {@link #DFLT_SPACE_NAME}.
+ *
+ * <h1 class="header">Configuration</h1>
+ * <h2 class="header">Mandatory</h2>
+ * This SPI has no mandatory configuration parameters.
+ * <h2 class="header">Optional SPI configuration.</h2>
+ * <ul>
+ *     <li>Base directory path (see {@link #setBaseDirectory(String)}).</li>
+ *     <li>Maximum sparsity (see {@link #setMaximumSparsity(float)}).</li>
+ *     <li>Write buffer size in bytes (see {@link #setWriteBufferSize(int)}).</li>
+ *     <li>Max write queue size in bytes (see {@link #setMaxWriteQueueSize(int)}).</li>
+ *     <li>Read stripes number. (see {@link #setReadStripesNumber(int)}).</li>
+ * </ul>
+ *
+ * <h2 class="header">Java Example</h2>
+ * GridFileSwapSpaceSpi is configured by default and should be explicitly configured
+ * only if some SPI configuration parameters need to be overridden.
+ * <pre name="code" class="java">
+ * GridFileSwapSpaceSpi spi = new GridFileSwapSpaceSpi();
+ *
+ * // Configure root folder path.
+ * spi.setBaseDirectory("/path/to/swap/folder");
+ *
+ * GridConfiguration cfg = new GridConfiguration();
+ *
+ * // Override default swap space SPI.
+ * cfg.setSwapSpaceSpi(spi);
+ *
+ * // Starts grid.
+ * G.start(cfg);
+ * </pre>
+ * <h2 class="header">Spring Example</h2>
+ * GridFileSwapSpaceSpi can be configured from Spring XML configuration file:
+ * <pre name="code" class="xml">
+ * &lt;bean id=&quot;grid.cfg&quot; class=&quot;org.gridgain.grid.GridConfiguration&quot; scope=&quot;singleton&quot;&gt;
+ *     ...
+ *     &lt;property name=&quot;swapSpaceSpi&quot;&gt;
+ *         &lt;bean class=&quot;org.gridgain.grid.spi.swapspace.file.GridFileSwapSpaceSpi&quot;&gt;
+ *             &lt;property name=&quot;baseDirectory&quot; value=&quot;/path/to/swap/folder&quot;/&gt;
+ *         &lt;/bean&gt;
+ *     &lt;/property&gt;
+ *     ...
+ * &lt;/bean&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.swapspace.SwapSpaceSpi
+ */
+@IgniteSpiMultipleInstancesSupport(true)
+@SuppressWarnings({"PackageVisibleInnerClass", "PackageVisibleField"})
+public class FileSwapSpaceSpi extends IgniteSpiAdapter implements SwapSpaceSpi, FileSwapSpaceSpiMBean {
+    /**
+     * Default base directory. Note that this path is relative to {@code GRIDGAIN_HOME/work} folder
+     * if {@code GRIDGAIN_HOME} system or environment variable specified, otherwise it is relative to
+     * {@code work} folder under system {@code java.io.tmpdir} folder.
+     *
+     * @see org.apache.ignite.configuration.IgniteConfiguration#getWorkDirectory()
+     */
+    public static final String DFLT_BASE_DIR = "swapspace";
+
+    /** Default maximum sparsity. */
+    public static final float DFLT_MAX_SPARSITY = 0.5f;
+
+    /** Default write buffer size in bytes. */
+    public static final int DFLT_BUF_SIZE = 64 * 1024;
+
+    /** Default write queue size in bytes. */
+    public static final int DFLT_QUE_SIZE = 1024 * 1024;
+
+    /** Name for {@code null} space. */
+    public static final String DFLT_SPACE_NAME = "gg-dflt-space";
+
+    /** Spaces. */
+    private final ConcurrentMap<String, Space> spaces = new ConcurrentHashMap<>();
+
+    /** Base directory. */
+    private String baseDir = DFLT_BASE_DIR;
+
+    /** Maximum sparsity. */
+    private float maxSparsity = DFLT_MAX_SPARSITY;
+
+    /** Eviction listener. */
+    private volatile SwapSpaceSpiListener evictLsnr;
+
+    /** Directory. */
+    private File dir;
+
+    /** Write buffer size. */
+    private int writeBufSize = DFLT_BUF_SIZE;
+
+    /** Max write queue size in bytes. */
+    private int maxWriteQueSize = DFLT_QUE_SIZE;
+
+    /** Read stripes number. */
+    private int readStripesNum = -1;
+
+    /** Logger. */
+    @IgniteLoggerResource
+    private IgniteLogger log;
+
+    /** Local node ID. */
+    @IgniteLocalNodeIdResource
+    private UUID locNodeId;
+
+    /** Name of the grid. */
+    @IgniteNameResource
+    private String gridName;
+
+    /** Marshaller. */
+    @IgniteMarshallerResource
+    private IgniteMarshaller marsh;
+
+    /** {@inheritDoc} */
+    @Override public String getBaseDirectory() {
+        return baseDir;
+    }
+
+    /**
+     * Sets base directory.
+     *
+     * @param baseDir Base directory.
+     */
+    @IgniteSpiConfiguration(optional = true)
+    public void setBaseDirectory(String baseDir) {
+        this.baseDir = baseDir;
+    }
+
+    /** {@inheritDoc} */
+    @Override public float getMaximumSparsity() {
+        return maxSparsity;
+    }
+
+    /**
+     * Sets maximum sparsity. This property defines maximum acceptable wasted file space to whole file size ratio.
+     * When this ratio becomes higher than specified number compacting thread starts working.
+     *
+     * @param maxSparsity Maximum sparsity. Must be between 0 and 1, default is {@link #DFLT_MAX_SPARSITY}.
+     */
+    public void setMaximumSparsity(float maxSparsity) {
+        this.maxSparsity = maxSparsity;
+    }
+
+    /** {@inheritDoc} */
+    @Override public int getWriteBufferSize() {
+        return writeBufSize;
+    }
+
+    /**
+     * Sets write buffer size in bytes. Write to disk occurs only when this buffer is full. Default is
+     * {@link #DFLT_BUF_SIZE}.
+     *
+     * @param writeBufSize Write buffer size in bytes.
+     */
+    public void setWriteBufferSize(int writeBufSize) {
+        this.writeBufSize = writeBufSize;
+    }
+
+    /** {@inheritDoc} */
+    @Override public int getMaxWriteQueueSize() {
+        return maxWriteQueSize;
+    }
+
+    /**
+     * Sets max write queue size in bytes. If there are more values are waiting for being written to disk then specified
+     * size, SPI will block on {@link #store(String, org.gridgain.grid.spi.swapspace.SwapKey, byte[], org.gridgain.grid.spi.swapspace.SwapContext)} operation. Default is
+     * {@link #DFLT_QUE_SIZE}.
+     *
+     * @param maxWriteQueSize Max write queue size in bytes.
+     */
+    public void setMaxWriteQueueSize(int maxWriteQueSize) {
+        this.maxWriteQueSize = maxWriteQueSize;
+    }
+
+    /** {@inheritDoc} */
+    @Override public int getReadStripesNumber() {
+        return readStripesNum;
+    }
+
+    /**
+     * Sets read stripe size. Defines number of file channels to be used concurrently. Default is equal to number of
+     * CPU cores available to this JVM.
+     *
+     * @param readStripesNum Read stripe number.
+     */
+    public void setReadStripesNumber(int readStripesNum) {
+        A.ensure(readStripesNum == -1 || (readStripesNum & (readStripesNum - 1)) == 0,
+            "readStripesNum must be positive and power of two");
+
+        this.readStripesNum = readStripesNum;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void spiStart(@Nullable String gridName) throws IgniteSpiException {
+        assertParameter(!F.isEmpty(baseDir), "!F.isEmpty(baseDir)");
+        assertParameter(maxSparsity >= 0 && maxSparsity < 1, "maxSparsity >= 0 && maxSparsity < 1");
+        assertParameter(readStripesNum == -1 || (readStripesNum & (readStripesNum - 1)) == 0,
+            "readStripesNum must be positive and power of two.");
+
+        if (readStripesNum == -1) {
+            // User has not configured the number.
+            int readStripesNum0 = 1;
+            int cpuCnt = Runtime.getRuntime().availableProcessors();
+
+            while (readStripesNum0 <= cpuCnt)
+                readStripesNum0 <<= 1;
+
+            if (readStripesNum0 > cpuCnt)
+                readStripesNum0 >>= 1;
+
+            assert readStripesNum0 > 0 && (readStripesNum0 & readStripesNum0 - 1) == 0;
+
+            readStripesNum = readStripesNum0;
+        }
+
+        startStopwatch();
+
+        registerMBean(gridName, this, FileSwapSpaceSpiMBean.class);
+
+        String path = baseDir + File.separator + gridName + File.separator + locNodeId;
+
+        try {
+            dir = U.resolveWorkDirectory(path, true);
+        }
+        catch (GridException e) {
+            throw new IgniteSpiException(e);
+        }
+
+        if (log.isDebugEnabled())
+            log.debug(startInfo());
+    }
+
+    /** {@inheritDoc} */
+    @Override public void spiStop() throws IgniteSpiException {
+        unregisterMBean();
+
+        for (Space space : spaces.values()) {
+            space.initialize();
+
+            try {
+                space.stop();
+            }
+            catch (GridInterruptedException e) {
+                U.error(log, "Interrupted.", e);
+            }
+        }
+
+        if (dir != null && dir.exists() && !U.delete(dir))
+            U.warn(log, "Failed to delete swap directory: " + dir.getAbsolutePath());
+
+        if (log.isDebugEnabled())
+            log.debug(stopInfo());
+    }
+
+    /** {@inheritDoc} */
+    @Override public void clear(@Nullable String spaceName) throws IgniteSpiException {
+        Space space = space(spaceName, false);
+
+        if (space == null)
+            return;
+
+        space.clear();
+
+        notifyListener(EVT_SWAP_SPACE_CLEARED, spaceName);
+    }
+
+    /** {@inheritDoc} */
+    @Override public long size(@Nullable String spaceName) throws IgniteSpiException {
+        Space space = space(spaceName, false);
+
+        if (space == null)
+            return 0;
+
+        return space.size();
+    }
+
+    /** {@inheritDoc} */
+    @Override public long count(@Nullable String spaceName) throws IgniteSpiException {
+        Space space = space(spaceName, false);
+
+        if (space == null)
+            return 0;
+
+        return space.count();
+    }
+
+    /** {@inheritDoc} */
+    @Nullable @Override public byte[] read(@Nullable String spaceName, SwapKey key, SwapContext ctx)
+        throws IgniteSpiException {
+        assert key != null;
+        assert ctx != null;
+
+        Space space = space(spaceName, false);
+
+        if (space == null)
+            return null;
+
+        byte[] val = space.read(key);
+
+        notifyListener(EVT_SWAP_SPACE_DATA_READ, spaceName);
+
+        return val;
+    }
+
+    /** {@inheritDoc} */
+    @Override public Map<SwapKey, byte[]> readAll(@Nullable String spaceName, Iterable<SwapKey> keys,
+        SwapContext ctx) throws IgniteSpiException {
+        assert keys != null;
+        assert ctx != null;
+
+        Space space = space(spaceName, false);
+
+        if (space == null)
+            return Collections.emptyMap();
+
+        Map<SwapKey, byte[]> res = new HashMap<>();
+
+        for (SwapKey key : keys) {
+            if (key != null) {
+                byte[] val = space.read(key);
+
+                if (val != null)
+                    res.put(key, val);
+
+                notifyListener(EVT_SWAP_SPACE_DATA_READ, spaceName);
+            }
+        }
+
+        return res;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void remove(@Nullable String spaceName, SwapKey key, @Nullable IgniteInClosure<byte[]> c,
+        SwapContext ctx) throws IgniteSpiException {
+        assert key != null;
+        assert ctx != null;
+
+        Space space = space(spaceName, false);
+
+        if (space == null)
+            return;
+
+        byte[] val = space.remove(key, c != null);
+
+        if (c != null)
+            c.apply(val);
+
+        notifyListener(EVT_SWAP_SPACE_DATA_REMOVED, spaceName);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void removeAll(@Nullable String spaceName, Collection<SwapKey> keys,
+        @Nullable IgniteBiInClosure<SwapKey, byte[]> c, SwapContext ctx) throws IgniteSpiException {
+        assert keys != null;
+        assert ctx != null;
+
+        Space space = space(spaceName, false);
+
+        if (space == null)
+            return;
+
+        for (SwapKey key : keys) {
+            if (key != null) {
+                byte[] val = space.remove(key, c != null);
+
+                if (c != null)
+                    c.apply(key, val);
+
+                notifyListener(EVT_SWAP_SPACE_DATA_REMOVED, spaceName);
+            }
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public void store(@Nullable String spaceName, SwapKey key, @Nullable byte[] val,
+        SwapContext ctx) throws IgniteSpiException {
+        assert key != null;
+        assert ctx != null;
+
+        Space space = space(spaceName, true);
+
+        assert space != null;
+
+        space.store(key, val);
+
+        notifyListener(EVT_SWAP_SPACE_DATA_STORED, spaceName);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void storeAll(@Nullable String spaceName, Map<SwapKey, byte[]> pairs,
+        SwapContext ctx) throws IgniteSpiException {
+        assert pairs != null;
+        assert ctx != null;
+
+        Space space = space(spaceName, true);
+
+        assert space != null;
+
+        for (Map.Entry<SwapKey, byte[]> pair : pairs.entrySet()) {
+            SwapKey key = pair.getKey();
+
+            if (key != null) {
+                space.store(key, pair.getValue());
+
+                notifyListener(EVT_SWAP_SPACE_DATA_STORED, spaceName);
+            }
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public void setListener(@Nullable SwapSpaceSpiListener evictLsnr) {
+        this.evictLsnr = evictLsnr;
+    }
+
+    /** {@inheritDoc} */
+    @Nullable @Override public Collection<Integer> partitions(@Nullable String spaceName)
+        throws IgniteSpiException {
+        Space space = space(spaceName, false);
+
+        if (space == null)
+            return null;
+
+        return space.partitions();
+    }
+
+    /** {@inheritDoc} */
+    @Nullable @Override public <K> IgniteSpiCloseableIterator<K> keyIterator(@Nullable String spaceName,
+        SwapContext ctx) throws IgniteSpiException {
+        final Space space = space(spaceName, false);
+
+        if (space == null)
+            return null;
+
+        final Iterator<Map.Entry<SwapKey, byte[]>> iter = space.entriesIterator();
+
+        return new GridCloseableIteratorAdapter<K>() {
+            @Override protected boolean onHasNext() {
+                return iter.hasNext();
+            }
+
+            @Override protected K onNext() {
+                return (K)iter.next().getKey().key();
+            }
+
+            @Override protected void onRemove() {
+                iter.remove();
+            }
+        };
+    }
+
+    /** {@inheritDoc} */
+    @Nullable @Override public IgniteSpiCloseableIterator<Map.Entry<byte[], byte[]>> rawIterator(
+        @Nullable String spaceName) throws IgniteSpiException {
+        Space space = space(spaceName, false);
+
+        if (space == null)
+            return null;
+
+        return rawIterator(space.entriesIterator());
+    }
+
+    /** {@inheritDoc} */
+    @Nullable @Override public IgniteSpiCloseableIterator<Map.Entry<byte[], byte[]>> rawIterator(
+        @Nullable String spaceName, int part) throws IgniteSpiException {
+        Space space = space(spaceName, false);
+
+        if (space == null)
+            return null;
+
+        return rawIterator(space.entriesIterator(part));
+    }
+
+    /**
+     * Creates raw iterator based on provided entries iterator.
+     *
+     * @param iter Entries iterator.
+     * @return Raw iterator.
+     */
+    private IgniteSpiCloseableIterator<Map.Entry<byte[], byte[]>> rawIterator(
+        final Iterator<Map.Entry<SwapKey, byte[]>> iter) {
+        return new GridCloseableIteratorAdapter<Map.Entry<byte[], byte[]>>() {
+            @Override protected Map.Entry<byte[], byte[]> onNext() throws GridException {
+                Map.Entry<SwapKey, byte[]> x = iter.next();
+
+                return new T2<>(keyBytes(x.getKey()), x.getValue());
+            }
+
+            @Override protected boolean onHasNext() {
+                return iter.hasNext();
+            }
+
+            @Override protected void onRemove() {
+                iter.remove();
+            }
+        };
+    }
+
+    /**
+     * Gets key bytes.
+     *
+     * @param key Swap key.
+     * @return Key bytes.
+     * @throws org.apache.ignite.spi.IgniteSpiException In case of error.
+     */
+    private byte[] keyBytes(SwapKey key) throws IgniteSpiException {
+        assert key != null;
+
+        byte[] keyBytes = key.keyBytes();
+
+        if (keyBytes == null) {
+            try {
+                keyBytes = marsh.marshal(key.key());
+            }
+            catch (GridException e) {
+                throw new IgniteSpiException("Failed to marshal key: " + key.key(), e);
+            }
+
+            key.keyBytes(keyBytes);
+        }
+
+        return keyBytes;
+    }
+
+    /**
+     * Notifies eviction listener.
+     *
+     * @param evtType Event type.
+     * @param spaceName Space name.
+     */
+    private void notifyListener(int evtType, @Nullable String spaceName) {
+        SwapSpaceSpiListener lsnr = evictLsnr;
+
+        if (lsnr != null)
+            lsnr.onSwapEvent(evtType, spaceName, null);
+    }
+
+    /**
+     * Gets space by name.
+     *
+     * @param name Space name.
+     * @param create Whether to create space if it doesn't exist.
+     * @return Space.
+     * @throws org.apache.ignite.spi.IgniteSpiException In case of error.
+     */
+    @Nullable private Space space(@Nullable String name, boolean create) throws IgniteSpiException {
+        String masked = name != null ? name : DFLT_SPACE_NAME;
+
+        assert masked != null;
+
+        Space space = spaces.get(masked);
+
+        if (space == null && create) {
+            validateName(name);
+
+            Space old = spaces.putIfAbsent(masked, space = new Space(masked));
+
+            if (old != null)
+                space = old;
+        }
+
+        if (space != null)
+            space.initialize();
+
+        return space;
+    }
+
+    /**
+     * Validates space name.
+     *
+     * @param name Space name.
+     * @throws org.apache.ignite.spi.IgniteSpiException If name is invalid.
+     */
+    private void validateName(@Nullable String name) throws IgniteSpiException {
+        if (name == null)
+            return;
+
+        if (name.isEmpty())
+            throw new IgniteSpiException("Space name cannot be empty: " + name);
+        else if (DFLT_SPACE_NAME.equalsIgnoreCase(name))
+            throw new IgniteSpiException("Space name is reserved for default space: " + name);
+        else if (name.contains("/") || name.contains("\\"))
+            throw new IgniteSpiException("Space name contains invalid characters: " + name);
+    }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return S.toString(FileSwapSpaceSpi.class, this);
+    }
+
+    /**
+     * Swap value.
+     */
+    static class SwapValue {
+        /** */
+        private static final int NEW = 0;
+
+        /** */
+        private static final int DELETED = Integer.MIN_VALUE;
+
+        /** */
+        private static final AtomicIntegerFieldUpdater<SwapValue> idxUpdater = AtomicIntegerFieldUpdater.
+            newUpdater(SwapValue.class, "idx");
+
+        /** */
+        private byte[] val;
+
+        /** */
+        private final int len;
+
+        /** */
+        @SuppressWarnings("FieldAccessedSynchronizedAndUnsynchronized")
+        private long pos = -1;
+
+        /** */
+        @SuppressWarnings("UnusedDeclaration")
+        private volatile int idx;
+
+        /**
+         * @param val Value.
+         */
+        SwapValue(byte[] val) {
+            assert val != null;
+
+            this.val = val;
+            len = val.length;
+        }
+
+        /**
+         * @param space Space.
+         * @return Value.
+         * @throws org.apache.ignite.spi.IgniteSpiException If failed.
+         */
+        @SuppressWarnings("NonPrivateFieldAccessedInSynchronizedContext")
+        @Nullable public synchronized byte[] value(Space space) throws IgniteSpiException {
+            byte[] v = val;
+
+            if (v == null) { // Read value from file.
+                int i = idx;
+
+                assert i != NEW;
+
+                if (i != DELETED) {
+                    StripedFileChannel ch = i < 0 ? space.left.readCh : space.right.readCh;
+
+                    if (idx != DELETED) // Double check works in pair with striped channel reopening.
+                        v = readValue(ch);
+                }
+            }
+            else if (v.length != len) {
+                int p = (int)pos;
+
+                v = Arrays.copyOfRange(v, p, p + len); // In case of compaction.
+            }
+
+            return v;
+        }
+
+        /**
+         * @param ch File channel.
+         * @return Bytes.
+         * @throws org.apache.ignite.spi.IgniteSpiException if failed.
+         */
+        @Nullable byte[] readValue(StripedFileChannel ch) throws IgniteSpiException {
+            byte[] v = new byte[len];
+
+            int res = 0;
+
+            try {
+                res = ch.read(ByteBuffer.wrap(v), pos);
+            }
+            catch (ClosedChannelException ignore) {
+                assert idx == DELETED;
+            }
+            catch (IOException e) {
+                throw new IgniteSpiException("Failed to read value.", e);
+            }
+
+            if (res < len)
+                return null; // When concurrent compaction occurs this may happen.
+
+            return v;
+        }
+
+        /**
+         * @param pos Position.
+         * @param val Value.
+         */
+        public synchronized void set(long pos, byte[] val) {
+            if (pos != -1)
+                this.pos = pos;
+
+            this.val = val;
+        }
+
+        /**
+         * @param exp Expected.
+         * @param idx New index.
+         * @return {@code true} if succeeded.
+         */
+        public boolean casIdx(int exp, int idx) {
+            return idxUpdater.compareAndSet(this, exp, idx);
+        }
+
+        /**
+         * @return Index in file array.
+         */
+        int idx() {
+            return idx;
+        }
+
+        /** {@inheritDoc} */
+        @Override public String toString() {
+            return pos + " " + len;
+        }
+    }
+
+    /**
+     * Queue of swap values.
+     */
+    private static class SwapValuesQueue {
+        /** */
+        private final ArrayDeque<SwapValue> deq = new ArrayDeque<>();
+
+        /** */
+        @SuppressWarnings("TypeMayBeWeakened")
+        private final ReentrantLock lock = new ReentrantLock();
+
+        /** */
+        private final Condition mayAdd = lock.newCondition();
+
+        /** */
+        private final Condition mayTake = lock.newCondition();
+
+        /** */
+        private int size;
+
+        /** */
+        private final int minTakeSize;
+
+        /** */
+        private final int maxSize;
+
+        /**
+         * @param minTakeSize Min size.
+         * @param maxSize Max size.
+         */
+        private SwapValuesQueue(int minTakeSize, int maxSize) {
+            this.minTakeSize = minTakeSize;
+            this.maxSize = maxSize;
+        }
+
+        /**
+         * Adds to queue.
+         *
+         * @param val Swap value.
+         * @throws org.apache.ignite.spi.IgniteSpiException If failed.
+         */
+        public void add(SwapValue val) throws IgniteSpiException {
+            lock.lock();
+
+            try {
+                while (size + val.len > maxSize)
+                    mayAdd.await();
+
+                size += val.len;
+
+                deq.addLast(val);
+
+                if (size >= minTakeSize)
+                    mayTake.signalAll();
+            }
+            catch (InterruptedException e) {
+                throw new IgniteSpiException(e);
+            }
+            finally {
+                lock.unlock();
+            }
+        }
+
+        /**
+         * Takes swap values from queue.
+         *
+         * @return Swap values.
+         * @throws InterruptedException If interrupted.
+         */
+        public SwapValues take() throws InterruptedException {
+            lock.lock();
+
+            try {
+                while (size < minTakeSize)
+                    mayTake.await();
+
+                int size = 0;
+                int cnt = 0;
+
+                for (SwapValue val : deq) {
+                    size += val.len;
+                    cnt++;
+
+                    if (size >= minTakeSize)
+                        break;
+                }
+
+                SwapValue[] vals = new SwapValue[cnt];
+
+                for (int i = 0; i < cnt; i++) {
+                    SwapValue val = deq.pollFirst();
+
+                    vals[i] = val;
+                }
+
+                if ((this.size -= size) < maxSize)
+                    mayAdd.signalAll();
+
+                return new SwapValues(vals, size);
+            }
+            finally {
+                lock.unlock();
+            }
+        }
+    }
+
+    /**
+     * Array of swap values and their size in bytes.
+     */
+    static class SwapValues {
+        /** */
+        private final SwapValue[] vals;
+
+        /** Size in bytes. */
+        private final int size;
+
+        /**
+         * @param vals Values.
+         * @param size Size.
+         */
+        SwapValues(SwapValue[] vals, int size) {
+            this.vals = vals;
+            this.size = size;
+        }
+    }
+
+    /**
+     * Readable striped file channel.
+     */
+    private static class StripedFileChannel {
+        /** */
+        private final AtomicInteger enter = new AtomicInteger();
+
+        /** */
+        private final RandomAccessFile[] rafs;
+
+        /** */
+        private final FileChannel[] chs;
+
+        /**
+         * @param f File.
+         * @param stripes Stripes.
+         * @throws FileNotFoundException If failed.
+         */
+        StripedFileChannel(File f, int stripes) throws FileNotFoundException {
+            assert stripes > 0 && (stripes & (stripes - 1)) == 0 : "stripes must be positive and power of two.";
+
+            rafs = new RandomAccessFile[stripes];
+            chs = new FileChannel[stripes];
+
+            for (int i = 0; i < stripes; i++) {
+                RandomAccessFile raf = new RandomAccessFile(f, "r");
+
+                rafs[i] = raf;
+                chs[i] = raf.getChannel();
+            }
+        }
+
+        /**
+         * Reads data from file channel to buffer.
+         *
+         * @param buf Buffer.
+         * @param pos Position.
+         * @return Read bytes count.
+         * @throws IOException If failed.
+         */
+        int read(ByteBuffer buf, long pos) throws IOException {
+            int i = enter.getAndIncrement() & (chs.length - 1);
+
+            return chs[i].read(buf, pos);
+        }
+
+        /**
+         * Closes channel.
+         */
+        void close() {
+            for (RandomAccessFile raf : rafs)
+                U.closeQuiet(raf);
+        }
+    }
+
+    /**
+     * Swap file.
+     */
+    static class SwapFile {
+        /** */
+        private static final long MIN_TRUNK_SIZE = 10 * 1024 * 1024;
+
+        /** */
+        private final File file;
+
+        /** */
+        private final RandomAccessFile raf;
+
+        /** */
+        private final FileChannel writeCh;
+
+        /** */
+        volatile StripedFileChannel readCh;
+
+        /** */
+        private volatile long len;
+
+        /** */
+        private final FileSwapArray<SwapValue> arr = new FileSwapArray<>();
+
+        /**
+         * @param file File.
+         * @param readerStripes Reader stripes number.
+         * @throws IOException In case of error.
+         */
+        SwapFile(File file, int readerStripes) throws IOException {
+            assert file != null;
+
+            file.delete();
+
+            if (!file.createNewFile())
+                throw new IllegalStateException("Failed to create file: " + file.getAbsolutePath());
+
+            this.file = file;
+
+            raf = new RandomAccessFile(file, "rw");
+
+            writeCh = raf.getChannel();
+
+            readCh = new StripedFileChannel(file, readerStripes);
+        }
+
+        /**
+         * Reopens read channel.
+         *
+         * @throws FileNotFoundException If failed.
+         */
+        void reopenReadChannel() throws FileNotFoundException {
+            readCh.close();
+
+            readCh = new StripedFileChannel(file, readCh.chs.length);
+        }
+
+        /**
+         * @param vals Values.
+         * @param buf Duffer.
+         * @param sign Indicates where should we write value, to the left or to the right.
+         * @throws Exception If failed.
+         */
+        public void write(Iterable<SwapValue> vals, ByteBuffer buf, int sign) throws Exception {
+            for (SwapValue val : vals) {
+                int oldIdx = val.idx;
+
+                if (oldIdx == SwapValue.DELETED)
+                    continue;
+
+                int idx = arr.add(val);
+
+                if (!val.casIdx(oldIdx, sign * idx)) {
+                    assert val.idx == SwapValue.DELETED;
+
+                    boolean res = tryRemove(idx, val);
+
+                    assert res;
+                }
+            }
+
+            final int size = buf.remaining();
+
+            if (size == 0)
+                return;
+
+            long pos = len;
+
+            len = pos + size;
+
+            long res = writeCh.write(buf, pos);
+
+            if (res != size)
+                throw new IllegalStateException(res + " != " + size);
+
+            // Nullify bytes in values ans set pos.
+            for (SwapValue val : vals) {
+                val.set(pos, null);
+
+                pos += val.len;
+            }
+        }
+
+        /**
+         * @param vals Values.
+         * @param sign Sign: 1 or -1.
+         * @throws Exception If failed.
+         */
+        public void write(SwapValues vals, int sign) throws Exception {
+            ByteBuffer buf = ByteBuffer.allocateDirect(vals.size);
+
+            for (int i = 0, len = vals.vals.length; i < len; i++) {
+                SwapValue val = vals.vals[i];
+
+                if (val.idx == SwapValue.DELETED) {
+                    vals.vals[i] = null;
+
+                    continue;
+                }
+
+                int idx = arr.add(val);
+
+                if (!val.casIdx(SwapValue.NEW, sign * idx)) {
+                    assert val.idx == SwapValue.DELETED;
+
+                    tryRemove(idx, val);
+
+                    vals.vals[i] = null;
+                }
+                else
+                    buf.put(val.value(null));
+            }
+
+            buf.flip();
+
+            final int size = buf.remaining();
+
+            if (size == 0)
+                return;
+
+            long pos = len;
+
+            len = pos + size;
+
+            long res = writeCh.write(buf, pos);
+
+            if (res != size)
+                throw new IllegalStateException(res + " != " + size);
+
+            // Nullify bytes in values ans set pos.
+            for (SwapValue val : vals.vals) {
+                if (val == null)
+                    continue;
+
+                val.set(pos, null);
+
+                pos += val.len;
+            }
+        }
+
+        /**
+         * Gets file path.
+         *
+         * @return File path.
+         */
+        public String path() {
+            return file.getAbsolutePath();
+        }
+
+        /**
+         * Gets file length.
+         *
+         * @return File length.
+         */
+        public long length() {
+            return len;
+        }
+
+        /**
+         * Deletes file.
+         *
+         * @return Whether file was actually deleted.
+         */
+        public boolean delete() {
+            U.closeQuiet(raf);
+
+            readCh.close();
+
+            return U.delete(file);
+        }
+
+        /**
+         * @param idx Index.
+         * @param exp Expected value.
+         * @return {@code true} If succeeded.
+         */
+        public boolean tryRemove(int idx, SwapValue exp) {
+            assert idx > 0 : idx;
+
+            FileSwapArray.Slot<SwapValue> s = arr.slot(idx);
+
+            return s != null && s.cas(exp, null);
+        }
+
+        /**
+         * Does compaction for one buffer.
+         *
+         * @param vals Values.
+         * @param bufSize Buffer size.
+         * @return Buffer.
+         * @throws IOException If failed.
+         * @throws InterruptedException If interrupted.
+         */
+        public ByteBuffer compact(ArrayDeque<SwapValue> vals, final int bufSize) throws IOException,
+            InterruptedException {
+            assert vals.isEmpty();
+
+            Compact c = new Compact(vals, bufSize);
+
+            c.doCompact();
+
+            return c.result();
+        }
+
+        /**
+         * Single compaction operation.
+         */
+        private class Compact {
+            /** */
+            private final ArrayDeque<SwapValue> vals;
+
+            /** */
+            private final int bufSize;
+
+            /** */
+            private byte[] bytes;
+
+            /** */
+            private ByteBuffer buf;
+
+            /** */
+            private long beg = -1;
+
+            /** */
+            private long end = -1;
+
+            /** */
+            private int compacted;
+
+            /**
+             * @param vals Values.
+             * @param bufSize Buffer size.
+             */
+            private Compact(ArrayDeque<SwapValue> vals, final int bufSize) {
+                assert vals.isEmpty();
+
+                this.vals = vals;
+                this.bufSize = bufSize;
+            }
+
+            /**
+             * Reads buffer and compacts it.
+             *
+             * @throws IOException if failed.
+             */
+            private void readAndCompact() throws IOException {
+                assert beg != -1;
+
+                if (buf == null) {
+                    bytes = new byte[bufSize];
+
+                    buf = ByteBuffer.wrap(bytes);
+                }
+
+                final int pos = buf.position();
+
+                final int lim = (int)(end - beg + pos);
+
+                assert pos >= 0;
+                assert pos < lim : pos + " " + lim;
+                assert lim <= buf.capacity();
+
+                buf.limit(lim);
+
+                int res = writeCh.read(buf, beg);
+
+                assert res == lim - pos;
+
+                int prevEnd = pos;
+                long delta = beg - pos; // To translate from file based positions to buffer based.
+
+                for (int j = vals.size(); j > compacted; j--) {
+                    SwapValue val = vals.pollFirst();
+
+                    int valPos = (int)(val.pos - delta);
+
+                    if (prevEnd != valPos) {
+                        assert prevEnd < valPos : prevEnd + " " + valPos;
+
+                        U.arrayCopy(bytes, valPos, bytes, prevEnd, val.len);
+                    }
+
+                    prevEnd += val.len;
+
+                    vals.addLast(val); // To have values in the same order as in byte buffer.
+                }
+
+                assert prevEnd > 0 : prevEnd;
+
+                buf.position(prevEnd);
+
+                end = -1;
+
+                compacted = vals.size();
+            }
+
+            /**
+             * Compacts.
+             *
+             * @throws IOException If failed.
+             */
+            private void doCompact() throws IOException {
+                int idx = arr.size();
+
+                while (--idx > 0) {
+                    FileSwapArray.Slot<SwapValue> s = arr.slot(idx);
+
+                    assert s != null;
+
+                    SwapValue v = s.get();
+
+                    if (v == null || v.idx == SwapValue.DELETED)
+                        continue;
+
+                    if (end == -1)
+                        end = v.pos + v.len;
+
+                    long size = end - v.pos;
+
+                    if ((buf == null ? bufSize : buf.remaining()) < size) {
+                        if (vals.isEmpty()) {  // Too big single value.
+                            assert bytes == null && buf == null;
+
+                            bytes = new byte[(int)size];
+
+                            buf = ByteBuffer.wrap(bytes);
+                        }
+                        else if (compacted == vals.size())
+                            break; // Finish current compaction, nothing new collected.
+                        else { // Read region and compact values in buffer.
+                            readAndCompact();
+
+                            // Retry the same value.
+                            idx++;
+
+                            continue;
+                        }
+                    }
+
+                    beg = v.pos;
+
+                    vals.addFirst(v);
+
+                    s.cas(v, null);
+                }
+
+                if (vals.isEmpty()) {
+                    arr.truncate(1);
+
+                    writeCh.truncate(0);
+
+                    len = 0;
+
+                    reopenReadChannel(); // Make sure that value can be read only from right file but not after switch.
+
+                    return;
+                }
+
+                if (compacted != vals.size())
+                    readAndCompact();
+
+                int pos = 0;
+
+                for (SwapValue val : vals) { // The values will share one byte array with different offsets while moving.
+                    val.set(pos, bytes);
+
+                    pos += val.len;
+                }
+
+                buf.flip();
+
+                assert buf.limit() == pos : buf.limit() + " " + pos;
+
+                arr.truncate(idx + 1);
+
+                if (len - beg > MIN_TRUNK_SIZE) {
+                    writeCh.truncate(beg);
+
+                    len = beg;
+                }
+            }
+
+            /**
+             * @return Buffer.
+             */
+            public ByteBuffer result() {
+                return buf;
+            }
+        }
+    }
+
+    /**
+     * Space.
+     */
+    private class Space {
+        /** Space name. */
+        private final String name;
+
+        /** */
+        private final GridAtomicInitializer<Void> initializer = new GridAtomicInitializer<>();
+
+        /** Swap file left. */
+        @SuppressWarnings("FieldAccessedSynchronizedAndUnsynchronized")
+        private SwapFile left;
+
+        /** Swap file right. */
+        @SuppressWarnings("FieldAccessedSynchronizedAndUnsynchronized")
+        private SwapFile right;
+
+        /** */
+        private final SwapValuesQueue que = new SwapValuesQueue(writeBufSize, maxWriteQueSize);
+
+        /** Partitions. */
+        private final ConcurrentMap<Integer, ConcurrentMap<SwapKey, SwapValue>> parts =
+            new ConcurrentHashMap8<>();
+
+        /** Total size. */
+        private final AtomicLong size = new AtomicLong();
+
+        /** Total count. */
+        private final AtomicLong cnt = new AtomicLong();
+
+        /** */
+        private int sign = 1;
+
+        /** Writer thread. */
+        private Thread writer;
+
+        /** */
+        private Thread compactor;
+
+        /**
+         * @param name Space name.
+         */
+        private Space(String name) {
+            assert name != null;
+
+            this.name = name;
+        }
+
+        /**
+         * Initializes space.
+         *
+         * @throws org.apache.ignite.spi.IgniteSpiException If initialization failed.
+         */
+        public void initialize() throws IgniteSpiException {
+            if (initializer.succeeded())
+                return;
+
+            assert dir.exists();
+            assert dir.isDirectory();
+
+            try {
+                initializer.init(new Callable<Void>(){
+                    @Override public Void call() throws Exception {
+                        left = new SwapFile(new File(dir, name + ".left"), readStripesNum);
+
+                        right = new SwapFile(new File(dir, name + ".right"), readStripesNum);
+
+                        final Object mux = new Object();
+
+                        writer = new IgniteSpiThread(gridName,  "Swap writer: " + name, log) {
+                            @Override protected void body() throws InterruptedException {
+                                while (!isInterrupted()) {
+                                    SwapValues vals = que.take();
+
+                                    synchronized (mux) {
+                                        SwapFile f = sign == 1 ? right : left;
+
+                                        try {
+                                            f.write(vals, sign);
+                                        }
+                                        catch (Exception e) {
+                                            throw new GridRuntimeException(e);
+                                        }
+                                    }
+                                }
+                            }
+                        };
+
+                        compactor = new IgniteSpiThread(gridName, "Swap compactor: " + name, log) {
+                            @Override protected void body() throws InterruptedException {
+                                SwapFile w = null;
+                                SwapFile c = null;
+
+                                ArrayDeque<SwapValue> vals = null;
+
+                                while (!isInterrupted()) {
+                                    while(!needCompact()) {
+                                        LockSupport.park();
+
+                                        if (isInterrupted())
+                                            return;
+                                    }
+
+                                    ByteBuffer buf = null;
+
+                                    if (vals == null)
+                                        vals = new ArrayDeque<>();
+                                    else {
+                                        vals.clear();
+
+                                        try {
+                                            buf = c.compact(vals, writeBufSize);
+                                        }
+                                        catch (IOException e) {
+                                            throw new GridRuntimeException(e);
+                                        }
+                                    }
+
+                                    if (vals.isEmpty()) {
+                                        synchronized (mux) {
+                                            sign = -sign;
+
+                                            if (sign == 1) {
+                                                w = right;
+                                                c = left;
+                                            }
+                                            else {
+                                                w = left;
+                                                c = right;
+                                            }
+                                        }
+                                    }
+                                    else {
+                                        assert buf != null && buf.remaining() != 0;
+
+                                        synchronized (mux) {
+                                            try {
+                                                w.write(vals, buf, sign);
+                                            }
+                                            catch (Exception e) {
+                                                throw new GridRuntimeException(e);
+                                            }
+                                        }
+                                    }
+                                }
+                            }
+                        };
+
+                        writer.start();
+                        compactor.start();
+
+                        return null;
+                    }
+                });
+            }
+            catch (GridException e) {
+                throw new IgniteSpiException(e);
+            }
+        }
+
+        /**
+         * Gets total space size in bytes.
+         *
+         * @return Total size.
+         */
+        public long size() {
+            return left.length() + right.length();
+        }
+
+        /**
+         * Gets total space count.
+         *
+         * @return Total count.
+         */
+        public long count() {
+            return cnt.get();
+        }
+
+        /**
+         * Clears space.
+         *
+         * @throws org.apache.ignite.spi.IgniteSpiException If failed.
+         */
+        public void clear() throws IgniteSpiException {
+            Iterator<Map.Entry<SwapKey, byte[]>> iter = entriesIterator();
+
+            while (iter.hasNext())
+                remove(iter.next().getKey(), false);
+        }
+
+        /**
+         * Stops space.
+         *
+         * @throws GridInterruptedException If interrupted.
+         */
+        public void stop() throws GridInterruptedException {
+            U.interrupt(writer);
+            U.interrupt(compactor);
+
+            U.join(writer);
+            U.join(compactor);
+
+            left.delete();
+            right.delete();
+        }
+
+        /**
+         * Stores value in space.
+         *
+         * @param key Key.
+         * @param val Value.
+         * @throws org.apache.ignite.spi.IgniteSpiException In case of error.
+         */
+        public void store(final SwapKey key, @Nullable final byte[] val) throws IgniteSpiException {
+            assert key != null;
+
+            final ConcurrentMap<SwapKey, SwapValue> part = partition(key.partition(), true);
+
+            assert part != null;
+
+            if (val == null) {
+                SwapValue swapVal = part.remove(key);
+
+                if (swapVal != null) {
+                    removeFromFile(swapVal);
+
+                    size.addAndGet(-swapVal.len);
+                    cnt.decrementAndGet();
+                }
+
+                return;
+            }
+
+            final SwapValue swapVal = new SwapValue(val);
+
+            SwapValue old = part.put(key, swapVal);
+
+            if (old != null) {
+                size.addAndGet(val.length - old.len);
+
+                removeFromFile(old);
+            }
+            else {
+                size.addAndGet(val.length);
+                cnt.incrementAndGet();
+            }
+
+            que.add(swapVal);
+        }
+
+        /**
+         * Reads value from space.
+         *
+         * @param key Key.
+         * @return Value.
+         * @throws org.apache.ignite.spi.IgniteSpiException In case of error.
+         */
+        @Nullable public byte[] read(SwapKey key) throws IgniteSpiException {
+            assert key != null;
+
+            final Map<SwapKey, SwapValue> part = partition(key.partition(), false);
+
+            if (part == null)
+                return null;
+
+            SwapValue swapVal = part.get(key);
+
+            if (swapVal == null)
+                return null;
+
+            return swapVal.value(this);
+        }
+
+        /**
+         * Removes value from space.
+         *
+         * @param key Key.
+         * @param read If value has to be read.
+         * @return Value.
+         * @throws org.apache.ignite.spi.IgniteSpiException In case of error.
+         */
+        @Nullable public byte[] remove(SwapKey key, boolean read) throws IgniteSpiException {
+            assert key != null;
+
+            final Map<SwapKey, SwapValue> part = partition(key.partition(), false);
+
+            if (part == null)
+                return null;
+
+            SwapValue val = part.remove(key);
+
+            if (val == null)
+                return null;
+
+            size.addAndGet(-val.len);
+
+            cnt.decrementAndGet();
+
+            byte[] bytes = null;
+
+            if (read) {
+                bytes = val.value(this);
+
+                assert bytes != null; // Value bytes were read before removal from file, so compaction can't happen.
+            }
+
+            removeFromFile(val);
+
+            return bytes;
+        }
+
+        /**
+         * @param val Value.
+         */
+        private void removeFromFile(SwapValue val) {
+            for (;;) {
+                int idx = val.idx;
+
+                assert idx != SwapValue.DELETED;
+
+                if (val.casIdx(idx, SwapValue.DELETED)) {
+                    if (idx != SwapValue.NEW) {
+                        SwapFile f = idx > 0 ? right : left;
+
+                        f.tryRemove(Math.abs(idx), val);
+                    }
+
+                    break;
+                }
+            }
+
+            if (needCompact())
+                LockSupport.unpark(compactor);
+        }
+
+        /**
+         * @return {@code true} If compaction needed.
+         */
+        private boolean needCompact() {
+            long fileLen = size();
+
+            return fileLen > writeBufSize && (fileLen - size.get()) / (float)fileLen > maxSparsity;
+        }
+
+        /**
+         * Gets numbers of partitioned stored in this space.
+         *
+         * @return Partition numbers.
+         */
+        public Collection<Integer> partitions() {
+            return parts.keySet();
+        }
+
+        /**
+         * Gets partition map by its number.
+         *
+         * @param part Partition number.
+         * @param create Whether to create partition if it doesn't exist.
+         * @return Partition map.
+         */
+        @Nullable private ConcurrentMap<SwapKey, SwapValue> partition(int part, boolean create) {
+            ConcurrentMap<SwapKey, SwapValue> map = parts.get(part);
+
+            if (map == null && create) {
+                ConcurrentMap<SwapKey, SwapValue> old = parts.putIfAbsent(part,
+                    map = new ConcurrentHashMap<>());
+
+                if (old != null)
+                    map = old;
+            }
+
+            return map;
+        }
+
+        /**
+         * @param part Partition.
+         * @return Iterator over partition.
+         */
+        public Iterator<Map.Entry<SwapKey, byte[]>> entriesIterator(int part) {
+            Map<SwapKey, SwapValue> partMap = partition(part, false);
+
+            if (partMap == null)
+                return Collections.<Map.Entry<SwapKey, byte[]>>emptySet().iterator();
+
+            return transform(partMap.entrySet().iterator());
+        }
+
+        /**
+         * @return Iterator over all entries.
+         */
+        public Iterator<Map.Entry<SwapKey, byte[]>> entriesIterator() {
+            final Iterator<ConcurrentMap<SwapKey, SwapValue>> iter = parts.values().iterator();
+
+            return transform(F.concat(new Iterator<Iterator<Map.Entry<SwapKey, SwapValue>>>() {
+                @Override public boolean hasNext() {
+                    return iter.hasNext();
+                }
+
+                @Override public Iterator<Map.Entry<SwapKey, SwapValue>> next() {
+                    return iter.next().entrySet().iterator();
+                }
+
+                @Override public void remove() {
+                    throw new UnsupportedOperationException();
+                }
+            }));
+        }
+
+        /**
+         * Gets iterator for all entries in space.
+         *
+         * @param iter Iterator with {@link SwapValue} to transform.
+         * @return Entries iterator.
+         */
+        private Iterator<Map.Entry<SwapKey, byte[]>> transform(final Iterator<Map.Entry<SwapKey,
+            SwapValue>> iter) {
+            return new Iterator<Map.Entry<SwapKey, byte[]>>() {
+                /** */
+                private Map.Entry<SwapKey, byte[]> next;
+
+                /** */
+                private Map.Entry<SwapKey, byte[]> last;
+
+                {
+                    advance();
+                }
+
+                @Override public boolean hasNext() {
+                    return next != null;
+                }
+
+                /**
+                 * Gets next entry.
+                 */
+                private void advance() {
+                    while (iter.hasNext()) {
+                        Map.Entry<SwapKey, SwapValue> entry = iter.next();
+
+                        byte[] bytes;
+
+                        try {
+                            bytes = entry.getValue().value(Space.this);
+                        }
+                        catch (IgniteSpiException e) {
+                            throw new GridRuntimeException(e);
+                        }
+
+                        if (bytes != null) {
+                            next = new T2<>(entry.getKey(), bytes);
+
+                            break;
+                        }
+                    }
+                }
+
+                @Override public Map.Entry<SwapKey, byte[]> next() {
+                    final Map.Entry<SwapKey, byte[]> res = next;
+
+                    if (res == null)
+                        throw new NoSuchElementException();
+
+                    next = null;
+
+                    advance();
+
+                    last = res;
+
+                    return res;
+                }
+
+                @Override public void remove() {
+                    if (last == null)
+                        throw new IllegalStateException();
+
+                    try {
+                        Space.this.remove(last.getKey(), false);
+                    }
+                    catch (IgniteSpiException e) {
+                        throw new GridRuntimeException(e);
+                    }
+                    finally {
+                        last = null;
+                    }
+                }
+            };
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c31cec7c/modules/core/src/main/java/org/gridgain/grid/spi/swapspace/file/FileSwapSpaceSpiMBean.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/spi/swapspace/file/FileSwapSpaceSpiMBean.java b/modules/core/src/main/java/org/gridgain/grid/spi/swapspace/file/FileSwapSpaceSpiMBean.java
new file mode 100644
index 0000000..12b6a67
--- /dev/null
+++ b/modules/core/src/main/java/org/gridgain/grid/spi/swapspace/file/FileSwapSpaceSpiMBean.java
@@ -0,0 +1,59 @@
+/* @java.file.header */
+
+/*  _________        _____ __________________        _____
+ *  __  ____/___________(_)______  /__  ____/______ ____(_)_______
+ *  _  / __  __  ___/__  / _  __  / _  / __  _  __ `/__  / __  __ \
+ *  / /_/ /  _  /    _  /  / /_/ /  / /_/ /  / /_/ / _  /  _  / / /
+ *  \____/   /_/     /_/   \_,__/   \____/   \__,_/  /_/   /_/ /_/
+ */
+
+package org.gridgain.grid.spi.swapspace.file;
+
+import org.apache.ignite.mbean.*;
+import org.apache.ignite.spi.*;
+
+/**
+ * Management bean for {@link FileSwapSpaceSpi}.
+ */
+@IgniteMBeanDescription("MBean that provides configuration information on file-based swapspace SPI.")
+public interface FileSwapSpaceSpiMBean extends IgniteSpiManagementMBean {
+    /**
+     * Gets base directory.
+     *
+     * @return Base directory.
+     */
+    @IgniteMBeanDescription("Base directory.")
+    public String getBaseDirectory();
+
+    /**
+     * Gets maximum sparsity.
+     *
+     * @return Maximum sparsity.
+     */
+    @IgniteMBeanDescription("Maximum sparsity.")
+    public float getMaximumSparsity();
+
+    /**
+     * Gets write buffer size in bytes.
+     *
+     * @return Write buffer size in bytes.
+     */
+    @IgniteMBeanDescription("Write buffer size in bytes.")
+    public int getWriteBufferSize();
+
+    /**
+     * Gets max write queue size in bytes.
+     *
+     * @return Max write queue size in bytes.
+     */
+    @IgniteMBeanDescription("Max write queue size in bytes.")
+    public int getMaxWriteQueueSize();
+
+    /**
+     * Gets read pool size.
+     *
+     * @return Read pool size.
+     */
+    @IgniteMBeanDescription("Read pool size.")
+    public int getReadStripesNumber();
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c31cec7c/modules/core/src/main/java/org/gridgain/grid/spi/swapspace/file/GridFileSwapArray.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/spi/swapspace/file/GridFileSwapArray.java b/modules/core/src/main/java/org/gridgain/grid/spi/swapspace/file/GridFileSwapArray.java
deleted file mode 100644
index 8796baf..0000000
--- a/modules/core/src/main/java/org/gridgain/grid/spi/swapspace/file/GridFileSwapArray.java
+++ /dev/null
@@ -1,181 +0,0 @@
-/* @java.file.header */
-
-/*  _________        _____ __________________        _____
- *  __  ____/___________(_)______  /__  ____/______ ____(_)_______
- *  _  / __  __  ___/__  / _  __  / _  / __  _  __ `/__  / __  __ \
- *  / /_/ /  _  /    _  /  / /_/ /  / /_/ /  / /_/ / _  /  _  / / /
- *  \____/   /_/     /_/   \_,__/   \____/   \__,_/  /_/   /_/ /_/
- */
-
-package org.gridgain.grid.spi.swapspace.file;
-
-import org.jetbrains.annotations.*;
-
-import java.util.concurrent.atomic.*;
-
-/**
- * Growing array.
- */
-class GridFileSwapArray<X> {
-    /** First partition size must be power of two. */
-    private static final int FIRST_ARRAY_SIZE = 4096;
-
-    /** */
-    private static final int LADDER_SIZE = Integer.numberOfLeadingZeros(FIRST_ARRAY_SIZE) + 1;
-
-    /** */
-    @SuppressWarnings("unchecked")
-    private final AtomicReferenceArray<X>[] ladder = new AtomicReferenceArray[LADDER_SIZE];
-
-    /** */
-    private int idx = 1;
-
-    /**
-     *
-     */
-    GridFileSwapArray() {
-        synchronized (ladder) {
-            ladder[0] = new AtomicReferenceArray<>(FIRST_ARRAY_SIZE);
-        }
-    }
-
-    /**
-     * @return Size.
-     */
-    public int size() {
-        return idx;
-    }
-
-    /**
-     * Adds value to the end.
-     *
-     * @param x Value.
-     * @return Index where it was added.
-     */
-    int add(X x) {
-        int i = idx++;
-
-        assert i >= 0 && i != Integer.MAX_VALUE : "Integer overflow";
-
-        Slot<X> s = slot(i);
-
-        assert s != null; // We should add always in one thread.
-
-        s.set(x);
-
-        int len = s.arr.length();
-
-        if (s.idx + 1 == len) {
-            synchronized (ladder) {
-                ladder[s.arrIdx + 1] = new AtomicReferenceArray<>(s.arrIdx == 0 ? len : len << 1);
-            }
-        }
-
-        return i;
-    }
-
-    /**
-     * @param size New size.
-     */
-    void truncate(int size) {
-        assert size > 0;
-
-        idx = size;
-
-        int arrIdx = arrayIndex(idx) + 1;
-
-        if (arrIdx < ladder.length && ladder[arrIdx] != null) {
-            synchronized (ladder) {
-                do {
-                    ladder[arrIdx++] = null;
-                }
-                while (arrIdx < ladder.length && ladder[arrIdx] != null);
-            }
-        }
-    }
-
-    /**
-     * @param idx Absolute slot index.
-     * @return Array index in {@link #ladder}.
-     */
-    static int arrayIndex(int idx) {
-        if (idx < FIRST_ARRAY_SIZE)
-            return 0;
-
-        return LADDER_SIZE - Integer.numberOfLeadingZeros(idx);
-    }
-
-    /**
-     * Slot for given absolute index.
-     *
-     * @param idx Absolute index.
-     * @return Slot.
-     */
-    @Nullable Slot<X> slot(int idx) {
-        assert idx > 0 : idx;
-
-        int arrIdx = arrayIndex(idx);
-
-        AtomicReferenceArray<X> arr = ladder[arrIdx];
-
-        if (arr == null) {
-            synchronized (ladder) { // Ensure visibility.
-                arr = ladder[arrIdx];
-            }
-
-            if (arr == null)
-                return null;
-        }
-
-        return new Slot<>(arrIdx, arr, arrIdx == 0 ? idx : idx - arr.length());
-    }
-
-    /**
-     * Slot in array.
-     */
-    @SuppressWarnings("PublicInnerClass")
-    static final class Slot<X> {
-        /** */
-        private final int arrIdx;
-
-        /** */
-        private final AtomicReferenceArray<X> arr;
-
-        /** */
-        private final int idx;
-
-        /**
-         * @param arrIdx Index of array.
-         * @param arr Array.
-         * @param idx Index within the array.
-         */
-        private Slot(int arrIdx, AtomicReferenceArray<X> arr, int idx) {
-            this.arrIdx = arrIdx;
-            this.arr = arr;
-            this.idx = idx;
-        }
-
-        /**
-         * @return Value.
-         */
-        public X get() {
-            return arr.get(idx);
-        }
-
-        /**
-         * @param exp Expected.
-         * @param x New value.
-         * @return {@code true} If succeeded.
-         */
-        public boolean cas(@Nullable X exp, @Nullable X x) {
-            return exp == x || arr.compareAndSet(idx, exp, x);
-        }
-
-        /**
-         * @param x value.
-         */
-        private void set(X x) {
-            arr.lazySet(idx, x);
-        }
-    }
-}


[02/20] incubator-ignite git commit: # Renaming

Posted by sb...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/a62862fe/modules/core/src/main/java/org/gridgain/grid/spi/loadbalancing/adaptive/GridAdaptiveJobCountLoadProbe.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/spi/loadbalancing/adaptive/GridAdaptiveJobCountLoadProbe.java b/modules/core/src/main/java/org/gridgain/grid/spi/loadbalancing/adaptive/GridAdaptiveJobCountLoadProbe.java
deleted file mode 100644
index b98108e..0000000
--- a/modules/core/src/main/java/org/gridgain/grid/spi/loadbalancing/adaptive/GridAdaptiveJobCountLoadProbe.java
+++ /dev/null
@@ -1,96 +0,0 @@
-/* @java.file.header */
-
-/*  _________        _____ __________________        _____
- *  __  ____/___________(_)______  /__  ____/______ ____(_)_______
- *  _  / __  __  ___/__  / _  __  / _  / __  _  __ `/__  / __  __ \
- *  / /_/ /  _  /    _  /  / /_/ /  / /_/ /  / /_/ / _  /  _  / / /
- *  \____/   /_/     /_/   \_,__/   \____/   \__,_/  /_/   /_/ /_/
- */
-
-package org.gridgain.grid.spi.loadbalancing.adaptive;
-
-import org.apache.ignite.cluster.*;
-import org.gridgain.grid.util.typedef.internal.*;
-
-/**
- * Implementation of node load probing based on active and waiting job count.
- * Based on {@link #setUseAverage(boolean)} parameter, this implementation will
- * either use average job count values or current (default is to use averages).
- * <p>
- * The load of a node is simply calculated by adding active and waiting job counts.
- * <p>
- * Below is an example of how CPU load probe would be configured in GridGain
- * Spring configuration file:
- * <pre name="code" class="xml">
- * &lt;property name="loadBalancingSpi"&gt;
- *     &lt;bean class="org.gridgain.grid.spi.loadBalancing.adaptive.GridAdaptiveLoadBalancingSpi"&gt;
- *         &lt;property name="loadProbe"&gt;
- *             &lt;bean class="org.gridgain.grid.spi.loadBalancing.adaptive.GridAdaptiveJobCountLoadProbe"&gt;
- *                 &lt;property name="useAverage" value="true"/&gt;
- *             &lt;/bean&gt;
- *         &lt;/property&gt;
- *     &lt;/bean&gt;
- * &lt;/property&gt;
- * </pre>
- */
-public class GridAdaptiveJobCountLoadProbe implements GridAdaptiveLoadProbe {
-    /** Flag indicating whether to use average CPU load vs. current. */
-    private boolean useAvg = true;
-
-    /**
-     * Initializes active job probe.
-     */
-    public GridAdaptiveJobCountLoadProbe() {
-        // No-op.
-    }
-
-    /**
-     * Creates new active job prove specifying whether to use average
-     * job counts vs. current.
-     *
-     * @param useAvg Flag indicating whether to use average job counts vs. current.
-     */
-    public GridAdaptiveJobCountLoadProbe(boolean useAvg) {
-        this.useAvg = useAvg;
-    }
-
-    /**
-     * Gets flag indicating whether to use average job counts vs. current.
-     *
-     * @return Flag indicating whether to use average job counts vs. current.
-     */
-    public boolean isUseAverage() {
-        return useAvg;
-    }
-
-    /**
-     * Sets flag indicating whether to use average job counts vs. current.
-     *
-     * @param useAvg Flag indicating whether to use average job counts vs. current.
-     */
-    public void setUseAverage(boolean useAvg) {
-        this.useAvg = useAvg;
-    }
-
-
-    /** {@inheritDoc} */
-    @Override public double getLoad(ClusterNode node, int jobsSentSinceLastUpdate) {
-        ClusterNodeMetrics metrics = node.metrics();
-
-        if (useAvg) {
-            double load = metrics.getAverageActiveJobs() + metrics.getAverageWaitingJobs();
-
-            if (load > 0)
-                return load;
-        }
-
-        double load = metrics.getCurrentActiveJobs() + metrics.getCurrentWaitingJobs();
-
-        return load < 0 ? 0 : load;
-    }
-
-    /** {@inheritDoc} */
-    @Override public String toString() {
-        return S.toString(GridAdaptiveJobCountLoadProbe.class, this);
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/a62862fe/modules/core/src/main/java/org/gridgain/grid/spi/loadbalancing/adaptive/GridAdaptiveLoadBalancingSpi.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/spi/loadbalancing/adaptive/GridAdaptiveLoadBalancingSpi.java b/modules/core/src/main/java/org/gridgain/grid/spi/loadbalancing/adaptive/GridAdaptiveLoadBalancingSpi.java
deleted file mode 100644
index d02529b..0000000
--- a/modules/core/src/main/java/org/gridgain/grid/spi/loadbalancing/adaptive/GridAdaptiveLoadBalancingSpi.java
+++ /dev/null
@@ -1,581 +0,0 @@
-/* @java.file.header */
-
-/*  _________        _____ __________________        _____
- *  __  ____/___________(_)______  /__  ____/______ ____(_)_______
- *  _  / __  __  ___/__  / _  __  / _  / __  _  __ `/__  / __  __ \
- *  / /_/ /  _  /    _  /  / /_/ /  / /_/ /  / /_/ / _  /  _  / / /
- *  \____/   /_/     /_/   \_,__/   \____/   \__,_/  /_/   /_/ /_/
- */
-
-package org.gridgain.grid.spi.loadbalancing.adaptive;
-
-import org.apache.ignite.*;
-import org.apache.ignite.cluster.*;
-import org.apache.ignite.compute.*;
-import org.apache.ignite.events.*;
-import org.apache.ignite.lang.*;
-import org.apache.ignite.resources.*;
-import org.apache.ignite.spi.*;
-import org.gridgain.grid.*;
-import org.gridgain.grid.kernal.managers.eventstorage.*;
-import org.gridgain.grid.spi.loadbalancing.*;
-import org.gridgain.grid.util.typedef.*;
-import org.gridgain.grid.util.typedef.internal.*;
-import org.jdk8.backport.*;
-import org.jetbrains.annotations.*;
-
-import java.util.*;
-import java.util.concurrent.*;
-import java.util.concurrent.atomic.*;
-import java.util.concurrent.locks.*;
-
-import static org.apache.ignite.events.IgniteEventType.*;
-
-/**
- * Load balancing SPI that adapts to overall node performance. It
- * proportionally distributes more jobs to more performant nodes based
- * on a pluggable and dynamic node load probing.
- * <p>
- * <h1 class="header">Adaptive Node Probe</h1>
- * This SPI comes with pluggable algorithm to calculate a node load
- * at any given point of time. The algorithm is defined by
- * {@link GridAdaptiveLoadProbe} interface and user is
- * free to provide custom implementations. By default
- * {@link GridAdaptiveCpuLoadProbe} implementation is used
- * which distributes jobs to nodes based on average CPU load
- * on every node.
- * <p>
- * The following load probes are available with the product:
- * <ul>
- * <li>{@link GridAdaptiveCpuLoadProbe} - default</li>
- * <li>{@link GridAdaptiveProcessingTimeLoadProbe}</li>
- * <li>{@link GridAdaptiveJobCountLoadProbe}</li>
- * </ul>
- * Note that if {@link GridAdaptiveLoadProbe#getLoad(org.apache.ignite.cluster.ClusterNode, int)} returns a value of {@code 0},
- * then implementation will assume that load value is simply not available and
- * will try to calculate an average of load values for other nodes. If such
- * average cannot be obtained (all node load values are {@code 0}), then a value
- * of {@code 1} will be used.
- * <p>
- * When working with node metrics, take into account that all averages are
- * calculated over metrics history size defined by {@link org.apache.ignite.configuration.IgniteConfiguration#getMetricsExpireTime()}
- * and {@link org.apache.ignite.configuration.IgniteConfiguration#getMetricsHistorySize()} grid configuration parameters.
- * Generally the larger these configuration parameter values are, the more precise the metrics are.
- * You should tune these values based on the level of accuracy needed vs. the additional memory
- * that would be required for storing metrics.
- * <p>
- * You should also keep in mind that metrics for remote nodes are delayed (usually by the
- * heartbeat frequency). So if it is acceptable in your environment, set the heartbeat frequency
- * to be more inline with job execution time. Generally, the more often heartbeats between nodes
- * are exchanged, the more precise the metrics are. However, you should keep in mind that if
- * heartbeats are exchanged too often then it may create unnecessary traffic in the network.
- * Heartbeats (or metrics update frequency) can be configured via underlying
- * {@link org.apache.ignite.spi.discovery.DiscoverySpi} used in your grid.
- * <p>
- * Here is an example of how probing can be implemented to use
- * number of active and waiting jobs as probing mechanism:
- * <pre name="code" class="java">
- * public class FooBarLoadProbe implements GridAdaptiveLoadProbe {
- *     // Flag indicating whether to use average value or current.
- *     private int useAvg = true;
- *
- *     public FooBarLoadProbe(boolean useAvg) {
- *         this.useAvg = useAvg;
- *     }
- *
- *     // Calculate load based on number of active and waiting jobs.
- *     public double getLoad(GridNode node, int jobsSentSinceLastUpdate) {
- *         GridNodeMetrics metrics = node.getMetrics();
- *
- *         if (useAvg) {
- *             double load = metrics.getAverageActiveJobs() + metrics.getAverageWaitingJobs();
- *
- *             if (load > 0) {
- *                 return load;
- *             }
- *         }
- *
- *         return metrics.getCurrentActiveJobs() + metrics.getCurrentWaitingJobs();
- *     }
- * }
- * </pre>
- * <h1 class="header">Which Node Probe To Use</h1>
- * There is no correct answer here. Every single node probe will work better or worse in
- * different environments. CPU load probe (default option) is the safest approach to start
- * with as it simply attempts to utilize every CPU on the grid to the maximum. However, you should
- * experiment with other probes by executing load tests in your environment and observing
- * which probe gives you best performance and load balancing.
- * <p>
- * <h1 class="header">Task Coding Example</h1>
- * If you are using {@link org.apache.ignite.compute.ComputeTaskSplitAdapter} then load balancing logic
- * is transparent to your code and is handled automatically by the adapter.
- * Here is an example of how your task will look:
- * <pre name="code" class="java">
- * public class MyFooBarTask extends GridComputeTaskSplitAdapter&lt;Object, Object&gt; {
- *    &#64;Override
- *    protected Collection&lt;? extends GridComputeJob&gt; split(int gridSize, Object arg) throws GridException {
- *        List&lt;MyFooBarJob&gt; jobs = new ArrayList&lt;MyFooBarJob&gt;(gridSize);
- *
- *        for (int i = 0; i &lt; gridSize; i++) {
- *            jobs.add(new MyFooBarJob(arg));
- *        }
- *
- *        // Node assignment via load balancer
- *        // happens automatically.
- *        return jobs;
- *    }
- *    ...
- * }
- * </pre>
- * If you need more fine-grained control over how some jobs within task get mapped to a node
- * and use affinity load balancing for some other jobs within task, then you should use
- * {@link org.apache.ignite.compute.ComputeTaskAdapter}. Here is an example of how your task will look. Note that in this
- * case we manually inject load balancer and use it to pick the best node. Doing it in
- * such way would allow user to map some jobs manually and for others use load balancer.
- * <pre name="code" class="java">
- * public class MyFooBarTask extends GridComputeTaskAdapter&lt;String, String&gt; {
- *    // Inject load balancer.
- *    &#64;GridLoadBalancerResource
- *    GridComputeLoadBalancer balancer;
- *
- *    // Map jobs to grid nodes.
- *    public Map&lt;? extends GridComputeJob, GridNode&gt; map(List&lt;GridNode&gt; subgrid, String arg) throws GridException {
- *        Map&lt;MyFooBarJob, GridNode&gt; jobs = new HashMap&lt;MyFooBarJob, GridNode&gt;(subgrid.size());
- *
- *        // In more complex cases, you can actually do
- *        // more complicated assignments of jobs to nodes.
- *        for (int i = 0; i &lt; subgrid.size(); i++) {
- *            // Pick the next best balanced node for the job.
- *            jobs.put(new MyFooBarJob(arg), balancer.getBalancedNode())
- *        }
- *
- *        return jobs;
- *    }
- *
- *    // Aggregate results into one compound result.
- *    public String reduce(List&lt;GridComputeJobResult&gt; results) throws GridException {
- *        // For the purpose of this example we simply
- *        // concatenate string representation of every
- *        // job result
- *        StringBuilder buf = new StringBuilder();
- *
- *        for (GridComputeJobResult res : results) {
- *            // Append string representation of result
- *            // returned by every job.
- *            buf.append(res.getData().string());
- *        }
- *
- *        return buf.string();
- *    }
- * }
- * </pre>
- * <p>
- * <h1 class="header">Configuration</h1>
- * In order to use this load balancer, you should configure your grid instance
- * to use {@code GridJobsLoadBalancingSpi} either from Spring XML file or
- * directly. The following configuration parameters are supported:
- * <h2 class="header">Mandatory</h2>
- * This SPI has no mandatory configuration parameters.
- * <h2 class="header">Optional</h2>
- * This SPI has the following optional configuration parameters:
- * <ul>
- * <li>
- *      Adaptive node load probing implementation (see {@link #setLoadProbe(GridAdaptiveLoadProbe)}).
- *      This configuration parameter supplies a custom algorithm for probing a node's load.
- *      By default, {@link GridAdaptiveCpuLoadProbe} implementation is used which
- *      takes every node's CPU load and tries to send proportionally more jobs to less loaded nodes.
- * </li>
- * </ul>
- * <p>
- * Below is Java configuration example:
- * <pre name="code" class="java">
- * GridAdaptiveLoadBalancingSpi spi = new GridAdaptiveLoadBalancingSpi();
- *
- * // Configure probe to use latest job execution time vs. average.
- * GridAdaptiveProcessingTimeLoadProbe probe = new GridAdaptiveProcessingTimeLoadProbe(false);
- *
- * spi.setLoadProbe(probe);
- *
- * GridConfiguration cfg = new GridConfiguration();
- *
- * // Override default load balancing SPI.
- * cfg.setLoadBalancingSpi(spi);
- *
- * // Starts grid.
- * G.start(cfg);
- * </pre>
- * Here is how you can configure {@code GridJobsLoadBalancingSpi} using Spring XML configuration:
- * <pre name="code" class="xml">
- * &lt;property name="loadBalancingSpi"&gt;
- *     &lt;bean class="org.gridgain.grid.spi.loadBalancing.adaptive.GridAdaptiveLoadBalancingSpi"&gt;
- *         &lt;property name="loadProbe"&gt;
- *             &lt;bean class="org.gridgain.grid.spi.loadBalancing.adaptive.GridAdaptiveProcessingTimeLoadProbe"&gt;
- *                 &lt;constructor-arg value="false"/&gt;
- *             &lt;/bean&gt;
- *         &lt;/property&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>
- */
-@IgniteSpiMultipleInstancesSupport(true)
-public class GridAdaptiveLoadBalancingSpi extends IgniteSpiAdapter implements GridLoadBalancingSpi,
-    GridAdaptiveLoadBalancingSpiMBean {
-    /** Random number generator. */
-    private static final Random RAND = new Random();
-
-    /** Grid logger. */
-    @IgniteLoggerResource
-    private IgniteLogger log;
-
-    /** */
-    private GridAdaptiveLoadProbe probe = new GridAdaptiveCpuLoadProbe();
-
-    /** Local event listener to listen to task completion events. */
-    private GridLocalEventListener evtLsnr;
-
-    /** Task topologies. First pair value indicates whether or not jobs have been mapped. */
-    private ConcurrentMap<IgniteUuid, IgniteBiTuple<Boolean, WeightedTopology>> taskTops =
-        new ConcurrentHashMap8<>();
-
-    /** */
-    private final Map<UUID, AtomicInteger> nodeJobs = new HashMap<>();
-
-    /** */
-    private final ReadWriteLock rwLock = new ReentrantReadWriteLock();
-
-    /** {@inheritDoc} */
-    @Override public String getLoadProbeFormatted() {
-        return probe.toString();
-    }
-
-    /**
-     * Sets implementation of node load probe. By default {@link GridAdaptiveProcessingTimeLoadProbe}
-     * is used which proportionally distributes load based on the average job execution
-     * time on every node.
-     *
-     * @param probe Implementation of node load probe
-     */
-    @IgniteSpiConfiguration(optional = true)
-    public void setLoadProbe(GridAdaptiveLoadProbe probe) {
-        A.ensure(probe != null, "probe != null");
-
-        this.probe = probe;
-    }
-
-    /** {@inheritDoc} */
-    @Override public void spiStart(@Nullable String gridName) throws IgniteSpiException {
-        startStopwatch();
-
-        assertParameter(probe != null, "loadProbe != null");
-
-        if (log.isDebugEnabled())
-            log.debug(configInfo("loadProbe", probe));
-
-        registerMBean(gridName, this, GridAdaptiveLoadBalancingSpiMBean.class);
-
-        // Ack ok start.
-        if (log.isDebugEnabled())
-            log.debug(startInfo());
-    }
-
-    /** {@inheritDoc} */
-    @Override public void spiStop() throws IgniteSpiException {
-        rwLock.writeLock().lock();
-
-        try {
-            nodeJobs.clear();
-        }
-        finally {
-            rwLock.writeLock().unlock();
-        }
-
-        unregisterMBean();
-
-        // Ack ok stop.
-        if (log.isDebugEnabled())
-            log.debug(stopInfo());
-    }
-
-    /** {@inheritDoc} */
-    @Override protected void onContextInitialized0(IgniteSpiContext spiCtx) throws IgniteSpiException {
-        getSpiContext().addLocalEventListener(evtLsnr = new GridLocalEventListener() {
-            @Override public void onEvent(IgniteEvent evt) {
-                switch (evt.type()) {
-                    case EVT_TASK_FINISHED:
-                    case EVT_TASK_FAILED: {
-                        IgniteTaskEvent taskEvt = (IgniteTaskEvent)evt;
-
-                        taskTops.remove(taskEvt.taskSessionId());
-
-                        if (log.isDebugEnabled())
-                            log.debug("Removed task topology from topology cache for session: " +
-                                taskEvt.taskSessionId());
-
-                        break;
-                    }
-
-                    case EVT_JOB_MAPPED: {
-                        // We should keep topology and use cache in GridComputeTask#map() method to
-                        // avoid O(n*n/2) complexity, after that we can drop caches.
-                        // Here we set mapped property and later cache will be ignored
-                        IgniteJobEvent jobEvt = (IgniteJobEvent)evt;
-
-                        IgniteBiTuple<Boolean, WeightedTopology> weightedTop = taskTops.get(jobEvt.taskSessionId());
-
-                        if (weightedTop != null)
-                            weightedTop.set1(true);
-
-                        if (log.isDebugEnabled())
-                            log.debug("Job has been mapped. Ignore cache for session: " + jobEvt.taskSessionId());
-
-                        break;
-                    }
-
-                    case EVT_NODE_METRICS_UPDATED:
-                    case EVT_NODE_FAILED:
-                    case EVT_NODE_JOINED:
-                    case EVT_NODE_LEFT: {
-                        IgniteDiscoveryEvent discoEvt = (IgniteDiscoveryEvent)evt;
-
-                        rwLock.writeLock().lock();
-
-                        try {
-                            switch (evt.type()) {
-                                case EVT_NODE_JOINED: {
-                                    nodeJobs.put(discoEvt.eventNode().id(), new AtomicInteger(0));
-
-                                    break;
-                                }
-
-                                case EVT_NODE_LEFT:
-                                case EVT_NODE_FAILED: {
-                                    nodeJobs.remove(discoEvt.eventNode().id());
-
-                                    break;
-                                }
-
-                                case EVT_NODE_METRICS_UPDATED: {
-                                    // Reset counter.
-                                    nodeJobs.put(discoEvt.eventNode().id(), new AtomicInteger(0));
-
-                                    break;
-                                }
-                            }
-                        }
-                        finally {
-                            rwLock.writeLock().unlock();
-                        }
-                    }
-
-                }
-            }
-        },
-            EVT_NODE_METRICS_UPDATED,
-            EVT_NODE_FAILED,
-            EVT_NODE_JOINED,
-            EVT_NODE_LEFT,
-            EVT_TASK_FINISHED,
-            EVT_TASK_FAILED,
-            EVT_JOB_MAPPED
-        );
-
-        // Put all known nodes.
-        rwLock.writeLock().lock();
-
-        try {
-            for (ClusterNode node : getSpiContext().nodes())
-                nodeJobs.put(node.id(), new AtomicInteger(0));
-        }
-        finally {
-            rwLock.writeLock().unlock();
-        }
-    }
-
-    /** {@inheritDoc} */
-    @Override protected void onContextDestroyed0() {
-        if (evtLsnr != null) {
-            IgniteSpiContext ctx = getSpiContext();
-
-            if (ctx != null)
-                ctx.removeLocalEventListener(evtLsnr);
-        }
-    }
-
-    /** {@inheritDoc} */
-    @Override public ClusterNode getBalancedNode(ComputeTaskSession ses, List<ClusterNode> top, ComputeJob job)
-    throws GridException {
-        A.notNull(ses, "ses");
-        A.notNull(top, "top");
-        A.notNull(job, "job");
-
-        IgniteBiTuple<Boolean, WeightedTopology> weightedTop = taskTops.get(ses.getId());
-
-        // Create new cached topology if there is no one. Do not
-        // use cached topology after task has been mapped.
-        if (weightedTop == null)
-            // Called from GridComputeTask#map(). Put new topology and false as not mapped yet.
-            taskTops.put(ses.getId(), weightedTop = F.t(false, new WeightedTopology(top)));
-        // We have topology - check if task has been mapped.
-        else if (weightedTop.get1())
-            // Do not use cache after GridComputeTask#map().
-            return new WeightedTopology(top).pickWeightedNode();
-
-        return weightedTop.get2().pickWeightedNode();
-    }
-
-    /**
-     * Calculates node load based on set probe.
-     *
-     * @param top List of all nodes.
-     * @param node Node to get load for.
-     * @return Node load.
-     * @throws GridException If returned load is negative.
-     */
-    @SuppressWarnings({"TooBroadScope"})
-    private double getLoad(Collection<ClusterNode> top, ClusterNode node) throws GridException {
-        assert !F.isEmpty(top);
-
-        int jobsSentSinceLastUpdate = 0;
-
-        rwLock.readLock().lock();
-
-        try {
-            AtomicInteger cnt = nodeJobs.get(node.id());
-
-            jobsSentSinceLastUpdate = cnt == null ? 0 : cnt.get();
-        }
-        finally {
-            rwLock.readLock().unlock();
-        }
-
-        double load = probe.getLoad(node, jobsSentSinceLastUpdate);
-
-        if (load < 0)
-            throw new GridException("Failed to obtain non-negative load from adaptive load probe: " + load);
-
-        return load;
-    }
-
-    /**
-     * Holder for weighted topology.
-     */
-    private class WeightedTopology {
-        /** Topology sorted by weight. */
-        private final SortedMap<Double, ClusterNode> circle = new TreeMap<>();
-
-        /**
-         * @param top Task topology.
-         * @throws GridException If any load was negative.
-         */
-        WeightedTopology(List<ClusterNode> top) throws GridException {
-            assert !F.isEmpty(top);
-
-            double totalLoad = 0;
-
-            // We need to cache loads here to avoid calls later as load might be
-            // changed between the calls.
-            double[] nums = new double[top.size()];
-
-            int zeroCnt = 0;
-
-            // Compute loads.
-            for (int i = 0; i < top.size(); i++) {
-                double load = getLoad(top, top.get(i));
-
-                nums[i] = load;
-
-                if (load == 0)
-                    zeroCnt++;
-
-                totalLoad += load;
-            }
-
-            // Take care of zero loads.
-            if (zeroCnt > 0) {
-                double newTotal = totalLoad;
-
-                int nonZeroCnt = top.size() - zeroCnt;
-
-                for (int i = 0; i < nums.length; i++) {
-                    double load = nums[i];
-
-                    if (load == 0) {
-                        if (nonZeroCnt > 0)
-                            load = totalLoad / nonZeroCnt;
-
-                        if (load == 0)
-                            load = 1;
-
-                        nums[i] = load;
-
-                        newTotal += load;
-                    }
-                }
-
-                totalLoad = newTotal;
-            }
-
-            double totalWeight = 0;
-
-            // Calculate weights and total weight.
-            for (int i = 0; i < nums.length; i++) {
-                assert nums[i] > 0 : "Invalid load: " + nums[i];
-
-                double weight = totalLoad / nums[i];
-
-                // Convert to weight.
-                nums[i] = weight;
-
-                totalWeight += weight;
-            }
-
-            double weight = 0;
-
-            // Enforce range from 0 to 1.
-            for (int i = 0; i < nums.length; i++) {
-                weight = i == nums.length - 1 ? 1.0d : weight + nums[i] / totalWeight;
-
-                assert weight < 2 : "Invalid weight: " + weight;
-
-                // Complexity of this put is O(logN).
-                circle.put(weight, top.get(i));
-            }
-        }
-
-        /**
-         * Gets weighted node in random fashion.
-         *
-         * @return Weighted node.
-         */
-        ClusterNode pickWeightedNode() {
-            double weight = RAND.nextDouble();
-
-            SortedMap<Double, ClusterNode> pick = circle.tailMap(weight);
-
-            ClusterNode node = pick.get(pick.firstKey());
-
-            rwLock.readLock().lock();
-
-            try {
-                AtomicInteger cnt = nodeJobs.get(node.id());
-
-                if (cnt != null)
-                    cnt.incrementAndGet();
-            }
-            finally {
-                rwLock.readLock().unlock();
-            }
-
-            return node;
-        }
-    }
-
-    /** {@inheritDoc} */
-    @Override public String toString() {
-        return S.toString(GridAdaptiveLoadBalancingSpi.class, this);
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/a62862fe/modules/core/src/main/java/org/gridgain/grid/spi/loadbalancing/adaptive/GridAdaptiveLoadBalancingSpiMBean.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/spi/loadbalancing/adaptive/GridAdaptiveLoadBalancingSpiMBean.java b/modules/core/src/main/java/org/gridgain/grid/spi/loadbalancing/adaptive/GridAdaptiveLoadBalancingSpiMBean.java
deleted file mode 100644
index a473d59..0000000
--- a/modules/core/src/main/java/org/gridgain/grid/spi/loadbalancing/adaptive/GridAdaptiveLoadBalancingSpiMBean.java
+++ /dev/null
@@ -1,27 +0,0 @@
-/* @java.file.header */
-
-/*  _________        _____ __________________        _____
- *  __  ____/___________(_)______  /__  ____/______ ____(_)_______
- *  _  / __  __  ___/__  / _  __  / _  / __  _  __ `/__  / __  __ \
- *  / /_/ /  _  /    _  /  / /_/ /  / /_/ /  / /_/ / _  /  _  / / /
- *  \____/   /_/     /_/   \_,__/   \____/   \__,_/  /_/   /_/ /_/
- */
-
-package org.gridgain.grid.spi.loadbalancing.adaptive;
-
-import org.apache.ignite.mbean.*;
-import org.apache.ignite.spi.*;
-
-/**
- * Management MBean for {@link GridAdaptiveLoadBalancingSpi} SPI.
- */
-@IgniteMBeanDescription("MBean that provides access to adaptive load balancing SPI configuration.")
-public interface GridAdaptiveLoadBalancingSpiMBean extends IgniteSpiManagementMBean {
-    /**
-     * Gets text description of current load probing implementation used.
-     *
-     * @return Text description of current load probing implementation used.
-     */
-    @IgniteMBeanDescription("Text description of current load probing implementation used.")
-    public String getLoadProbeFormatted();
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/a62862fe/modules/core/src/main/java/org/gridgain/grid/spi/loadbalancing/adaptive/GridAdaptiveLoadProbe.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/spi/loadbalancing/adaptive/GridAdaptiveLoadProbe.java b/modules/core/src/main/java/org/gridgain/grid/spi/loadbalancing/adaptive/GridAdaptiveLoadProbe.java
deleted file mode 100644
index 88226cc..0000000
--- a/modules/core/src/main/java/org/gridgain/grid/spi/loadbalancing/adaptive/GridAdaptiveLoadProbe.java
+++ /dev/null
@@ -1,90 +0,0 @@
-/* @java.file.header */
-
-/*  _________        _____ __________________        _____
- *  __  ____/___________(_)______  /__  ____/______ ____(_)_______
- *  _  / __  __  ___/__  / _  __  / _  / __  _  __ `/__  / __  __ \
- *  / /_/ /  _  /    _  /  / /_/ /  / /_/ /  / /_/ / _  /  _  / / /
- *  \____/   /_/     /_/   \_,__/   \____/   \__,_/  /_/   /_/ /_/
- */
-
-package org.gridgain.grid.spi.loadbalancing.adaptive;
-
-import org.apache.ignite.cluster.*;
-
-/**
- * Pluggable implementation of node load probing. Implementations
- * of this can be configured to be used with {@link GridAdaptiveLoadBalancingSpi}
- * by setting {@link GridAdaptiveLoadBalancingSpi#setLoadProbe(GridAdaptiveLoadProbe)}
- * configuration parameter.
- * <p>
- * Note that if {@link #getLoad(org.apache.ignite.cluster.ClusterNode, int)} returns a value of {@code 0},
- * then implementation will assume that load value is simply not available and
- * will try to calculate an average of load values for other nodes. If such
- * average cannot be obtained (all node load values are {@code 0}), then a value
- * of {@code 1} will be used.
- * <p>
- * By default, {@link GridAdaptiveCpuLoadProbe} probing implementation is used.
- * <p>
- * <h1 class="header">Example</h1>
- * Here is an example of how probing can be implemented to use
- * number of active and waiting jobs as probing mechanism:
- * <pre name="code" class="java">
- * public class FooBarLoadProbe implements GridAdaptiveLoadProbe {
- *     // Flag indicating whether to use average value or current.
- *     private int useAvg = true;
- *
- *     public FooBarLoadProbe(boolean useAvg) {
- *         this.useAvg = useAvg;
- *     }
- *
- *     // Calculate load based on number of active and waiting jobs.
- *     public double getLoad(GridNode node, int jobsSentSinceLastUpdate) {
- *         GridNodeMetrics metrics = node.getMetrics();
- *
- *         if (useAvg) {
- *             double load = metrics.getAverageActiveJobs() + metrics.getAverageWaitingJobs();
- *
- *             if (load > 0) {
- *                 return load;
- *             }
- *         }
- *
- *         return metrics.getCurrentActiveJobs() + metrics.getCurrentWaitingJobs();
- *     }
- * }
- * </pre>
- * Below is an example of how a probe shown above would be configured with {@link GridAdaptiveLoadBalancingSpi}
- * SPI:
- * <pre name="code" class="xml">
- * &lt;property name="loadBalancingSpi"&gt;
- *     &lt;bean class="org.gridgain.grid.spi.loadBalancing.adaptive.GridAdaptiveLoadBalancingSpi"&gt;
- *         &lt;property name="loadProbe"&gt;
- *             &lt;bean class="foo.bar.FooBarLoadProbe"&gt;
- *                 &lt;constructor-arg value="true"/&gt;
- *             &lt;/bean&gt;
- *         &lt;/property&gt;
- *     &lt;/bean&gt;
- * &lt;/property&gt;
- * </pre>
- */
-public interface GridAdaptiveLoadProbe {
-    /**
-     * Calculates load value for a given node. Specific implementations would
-     * usually take into account some of the values provided by
-     * {@link org.apache.ignite.cluster.ClusterNode#metrics()} method. For example, load can be calculated
-     * based on job execution time or number of active jobs, or CPU/Heap utilization.
-     * <p>
-     * Note that if this method returns a value of {@code 0},
-     * then implementation will assume that load value is simply not available and
-     * will try to calculate an average of load values for other nodes. If such
-     * average cannot be obtained (all node load values are {@code 0}), then a value
-     * of {@code 1} will be used.
-     *
-     * @param node Grid node to calculate load for.
-     * @param jobsSentSinceLastUpdate Number of jobs sent to this node since
-     *      last metrics update. This parameter may be useful when
-     *      implementation takes into account the current job count on a node.
-     * @return Non-negative load value for the node (zero and above).
-     */
-    public double getLoad(ClusterNode node, int jobsSentSinceLastUpdate);
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/a62862fe/modules/core/src/main/java/org/gridgain/grid/spi/loadbalancing/adaptive/GridAdaptiveProcessingTimeLoadProbe.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/spi/loadbalancing/adaptive/GridAdaptiveProcessingTimeLoadProbe.java b/modules/core/src/main/java/org/gridgain/grid/spi/loadbalancing/adaptive/GridAdaptiveProcessingTimeLoadProbe.java
deleted file mode 100644
index bc6f745..0000000
--- a/modules/core/src/main/java/org/gridgain/grid/spi/loadbalancing/adaptive/GridAdaptiveProcessingTimeLoadProbe.java
+++ /dev/null
@@ -1,98 +0,0 @@
-/* @java.file.header */
-
-/*  _________        _____ __________________        _____
- *  __  ____/___________(_)______  /__  ____/______ ____(_)_______
- *  _  / __  __  ___/__  / _  __  / _  / __  _  __ `/__  / __  __ \
- *  / /_/ /  _  /    _  /  / /_/ /  / /_/ /  / /_/ / _  /  _  / / /
- *  \____/   /_/     /_/   \_,__/   \____/   \__,_/  /_/   /_/ /_/
- */
-
-package org.gridgain.grid.spi.loadbalancing.adaptive;
-
-import org.apache.ignite.cluster.*;
-import org.gridgain.grid.util.typedef.internal.*;
-
-/**
- * Implementation of node load probing based on total job processing time.
- * Based on {@link #setUseAverage(boolean)}
- * parameter, this implementation will either use average job execution
- * time values or current (default is to use averages). The algorithm
- * returns a sum of job wait time and job execution time.
- * <p>
- * Below is an example of how CPU load probe would be configured in GridGain
- * Spring configuration file:
- * <pre name="code" class="xml">
- * &lt;property name="loadBalancingSpi"&gt;
- *     &lt;bean class="org.gridgain.grid.spi.loadBalancing.adaptive.GridAdaptiveLoadBalancingSpi"&gt;
- *         &lt;property name="loadProbe"&gt;
- *             &lt;bean class="org.gridgain.grid.spi.loadBalancing.adaptive.GridAdaptiveProcessingTimeLoadProbe"&gt;
- *                 &lt;property name="useAverage" value="true"/&gt;
- *             &lt;/bean&gt;
- *         &lt;/property&gt;
- *     &lt;/bean&gt;
- * &lt;/property&gt;
- * </pre>
- */
-public class GridAdaptiveProcessingTimeLoadProbe implements GridAdaptiveLoadProbe {
-    /** Flag indicating whether to use average execution time vs. current. */
-    private boolean useAvg = true;
-
-    /**
-     * Initializes execution time load probe to use
-     * execution time average by default.
-     */
-    public GridAdaptiveProcessingTimeLoadProbe() {
-        // No-op.
-    }
-
-    /**
-     * Specifies whether to use average execution time vs. current.
-     *
-     * @param useAvg Flag indicating whether to use average execution time vs. current.
-     */
-    public GridAdaptiveProcessingTimeLoadProbe(boolean useAvg) {
-        this.useAvg = useAvg;
-    }
-
-    /**
-     * Gets flag indicating whether to use average execution time vs. current.
-     *
-     * @return Flag indicating whether to use average execution time vs. current.
-     */
-    public boolean isUseAverage() {
-        return useAvg;
-    }
-
-    /**
-     * Sets flag indicating whether to use average execution time vs. current.
-     *
-     * @param useAvg Flag indicating whether to use average execution time vs. current.
-     */
-    public void setUseAverage(boolean useAvg) {
-        this.useAvg = useAvg;
-    }
-
-
-    /** {@inheritDoc} */
-    @Override public double getLoad(ClusterNode node, int jobsSentSinceLastUpdate) {
-        ClusterNodeMetrics metrics = node.metrics();
-
-        if (useAvg) {
-            double load = metrics.getAverageJobExecuteTime() + metrics.getAverageJobWaitTime();
-
-            // If load is greater than 0, then we can use average times.
-            // Otherwise, we will proceed to using current times.
-            if (load > 0)
-                return load;
-        }
-
-        double load = metrics.getCurrentJobExecuteTime() + metrics.getCurrentJobWaitTime();
-
-        return load < 0 ? 0 : load;
-    }
-
-    /** {@inheritDoc} */
-    @Override public String toString() {
-        return S.toString(GridAdaptiveProcessingTimeLoadProbe.class, this);
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/a62862fe/modules/core/src/main/java/org/gridgain/grid/spi/loadbalancing/roundrobin/GridRoundRobinGlobalLoadBalancer.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/spi/loadbalancing/roundrobin/GridRoundRobinGlobalLoadBalancer.java b/modules/core/src/main/java/org/gridgain/grid/spi/loadbalancing/roundrobin/GridRoundRobinGlobalLoadBalancer.java
deleted file mode 100644
index 9234e3c..0000000
--- a/modules/core/src/main/java/org/gridgain/grid/spi/loadbalancing/roundrobin/GridRoundRobinGlobalLoadBalancer.java
+++ /dev/null
@@ -1,305 +0,0 @@
-/* @java.file.header */
-
-/*  _________        _____ __________________        _____
- *  __  ____/___________(_)______  /__  ____/______ ____(_)_______
- *  _  / __  __  ___/__  / _  __  / _  / __  _  __ `/__  / __  __ \
- *  / /_/ /  _  /    _  /  / /_/ /  / /_/ /  / /_/ / _  /  _  / / /
- *  \____/   /_/     /_/   \_,__/   \____/   \__,_/  /_/   /_/ /_/
- */
-
-package org.gridgain.grid.spi.loadbalancing.roundrobin;
-
-import org.apache.ignite.*;
-import org.apache.ignite.cluster.*;
-import org.apache.ignite.events.*;
-import org.apache.ignite.spi.*;
-import org.gridgain.grid.*;
-import org.gridgain.grid.kernal.managers.eventstorage.*;
-import org.gridgain.grid.util.typedef.*;
-import org.gridgain.grid.util.typedef.internal.*;
-
-import java.util.*;
-import java.util.concurrent.*;
-import java.util.concurrent.atomic.*;
-
-import static org.apache.ignite.events.IgniteEventType.*;
-
-/**
- * Load balancer that works in global (not-per-task) mode.
- */
-class GridRoundRobinGlobalLoadBalancer {
-    /** SPI context. */
-    private IgniteSpiContext ctx;
-
-    /** Listener for node's events. */
-    private GridLocalEventListener lsnr;
-
-    /** Logger. */
-    private final IgniteLogger log;
-
-    /** Current snapshot of nodes which participated in load balancing. */
-    private volatile GridNodeList nodeList = new GridNodeList(0, null);
-
-    /** Mutex for updating current topology. */
-    private final Object mux = new Object();
-
-    /** Barrier for separating initialization callback and load balancing routine. */
-    private final CountDownLatch initLatch = new CountDownLatch(1);
-
-    /**
-     * @param log Grid logger.
-     */
-    GridRoundRobinGlobalLoadBalancer(IgniteLogger log) {
-        assert log != null;
-
-        this.log = log;
-    }
-
-    /**
-     * @param ctx Load balancing context.
-     */
-    void onContextInitialized(final IgniteSpiContext ctx) {
-        this.ctx = ctx;
-
-        ctx.addLocalEventListener(
-            lsnr = new GridLocalEventListener() {
-                @Override public void onEvent(IgniteEvent evt) {
-                    assert evt instanceof IgniteDiscoveryEvent;
-
-                    UUID nodeId = ((IgniteDiscoveryEvent)evt).eventNode().id();
-
-                    synchronized (mux) {
-                        if (evt.type() == EVT_NODE_JOINED) {
-                            List<UUID> oldNodes = nodeList.getNodes();
-
-                            if (!oldNodes.contains(nodeId)) {
-                                List<UUID> newNodes = new ArrayList<>(oldNodes.size() + 1);
-
-                                newNodes.add(nodeId);
-
-                                for (UUID node : oldNodes)
-                                    newNodes.add(node);
-
-                                nodeList = new GridNodeList(0, newNodes);
-                            }
-                        }
-                        else {
-                            assert evt.type() == EVT_NODE_LEFT || evt.type() == EVT_NODE_FAILED;
-
-                            List<UUID> oldNodes = nodeList.getNodes();
-
-                            if (oldNodes.contains(nodeId)) {
-                                List<UUID> newNodes = new ArrayList<>(oldNodes.size() - 1);
-
-                                for (UUID node : oldNodes)
-                                    if (!nodeId.equals(node))
-                                        newNodes.add(node);
-
-                                nodeList = new GridNodeList(0, newNodes);
-                            }
-                        }
-                    }
-                }
-            },
-            EVT_NODE_FAILED, EVT_NODE_JOINED, EVT_NODE_LEFT
-        );
-
-        synchronized (mux) {
-            List<UUID> oldNodes = nodeList.getNodes();
-
-            Collection<UUID> set = oldNodes == null ? new HashSet<UUID>() : new HashSet<>(oldNodes);
-
-            for (ClusterNode node : ctx.nodes())
-                set.add(node.id());
-
-            nodeList = new GridNodeList(0, new ArrayList<>(set));
-        }
-
-        initLatch.countDown();
-    }
-
-    /** */
-    void onContextDestroyed() {
-        if (ctx != null)
-            ctx.removeLocalEventListener(lsnr);
-    }
-
-    /**
-     * Gets balanced node for given topology.
-     *
-     * @param top Topology to pick from.
-     * @return Best balanced node.
-     * @throws GridException Thrown in case of any error.
-     */
-    ClusterNode getBalancedNode(Collection<ClusterNode> top) throws GridException {
-        assert !F.isEmpty(top);
-
-        awaitInitializationCompleted();
-
-        Map<UUID, ClusterNode> topMap = null;
-
-        ClusterNode found;
-
-        int misses = 0;
-
-        do {
-            GridNodeList nodeList = this.nodeList;
-
-            List<UUID> nodes = nodeList.getNodes();
-
-            int cycleSize = nodes.size();
-
-            if (cycleSize == 0)
-                throw new GridException("Task topology does not have any alive nodes.");
-
-            AtomicInteger idx;
-
-            int curIdx, nextIdx;
-
-            do {
-                idx = nodeList.getCurrentIdx();
-
-                curIdx = idx.get();
-
-                nextIdx = (idx.get() + 1) % cycleSize;
-            }
-            while (!idx.compareAndSet(curIdx, nextIdx));
-
-            found = findNodeById(top, nodes.get(nextIdx));
-
-            if (found == null) {
-                misses++;
-
-                // For optimization purposes checks balancer can return at least one node with specified
-                // request topology only after full cycle (approximately).
-                if (misses >= cycleSize) {
-                    if (topMap == null) {
-                        topMap = U.newHashMap(top.size());
-
-                        for (ClusterNode node : top)
-                            topMap.put(node.id(), node);
-                    }
-
-                    checkBalancerNodes(top, topMap, nodes);
-
-                    // Zero miss counter so next topology check will be performed once again after full cycle.
-                    misses = 0;
-                }
-            }
-        }
-        while (found == null);
-
-        if (log.isDebugEnabled())
-            log.debug("Found round-robin node: " + found);
-
-        return found;
-    }
-
-    /**
-     * Finds node by id. Returns null in case of absence of specified id in request topology.
-     *
-     * @param top Topology for current request.
-     * @param foundNodeId Node id.
-     * @return Found node or null in case of absence of specified id in request topology.
-     */
-    private static ClusterNode findNodeById(Iterable<ClusterNode> top, UUID foundNodeId) {
-        for (ClusterNode node : top)
-            if (foundNodeId.equals(node.id()))
-                return node;
-
-        return null;
-    }
-
-    /**
-     * Checks if balancer can return at least one node,
-     * throw exception otherwise.
-     *
-     * @param top Topology for current request.
-     * @param topMap Topology map.
-     * @param nodes Current balanced nodes.
-     * @throws GridException If balancer can not return any node.
-     */
-    private static void checkBalancerNodes(Collection<ClusterNode> top, Map<UUID, ClusterNode> topMap, Iterable<UUID> nodes)
-        throws GridException {
-
-        boolean contains = false;
-
-        for (UUID nodeId : nodes) {
-            if (topMap.get(nodeId) != null) {
-                contains = true;
-
-                break;
-            }
-        }
-
-        if (!contains)
-            throw new GridException("Task topology does not have alive nodes: " + top);
-    }
-
-    /**
-     * Awaits initialization of balancing nodes to be completed.
-     *
-     * @throws GridException Thrown in case of thread interruption.
-     */
-    private void awaitInitializationCompleted() throws GridException {
-        try {
-            if (initLatch.getCount() > 0)
-                initLatch.await();
-        }
-        catch (InterruptedException e) {
-            Thread.currentThread().interrupt();
-
-            throw new GridException("Global balancer was interrupted.", e);
-        }
-    }
-
-    /**
-     * Snapshot of nodes which participated in load balancing.
-     */
-    private static final class GridNodeList {
-        /** Cyclic pointer for selecting next node. */
-        private final AtomicInteger curIdx;
-
-        /** Node ids. */
-        private final List<UUID> nodes;
-
-        /**
-         * @param curIdx Initial index of current node.
-         * @param nodes Initial node ids.
-         */
-        private GridNodeList(int curIdx, List<UUID> nodes) {
-            this.curIdx = new AtomicInteger(curIdx);
-            this.nodes = nodes;
-        }
-
-        /**
-         * @return Index of current node.
-         */
-        private AtomicInteger getCurrentIdx() {
-            return curIdx;
-        }
-
-        /**
-         * @return Node ids.
-         */
-        private List<UUID> getNodes() {
-            return nodes;
-        }
-    }
-
-    /**
-     * THIS METHOD IS USED ONLY FOR TESTING.
-     *
-     * @return Internal list of nodes.
-     */
-    List<UUID> getNodeIds() {
-        List<UUID> nodes = nodeList.getNodes();
-
-        return Collections.unmodifiableList(nodes);
-    }
-
-    /** {@inheritDoc} */
-    @Override public String toString() {
-        return S.toString(GridRoundRobinGlobalLoadBalancer.class, this);
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/a62862fe/modules/core/src/main/java/org/gridgain/grid/spi/loadbalancing/roundrobin/GridRoundRobinLoadBalancingSpi.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/spi/loadbalancing/roundrobin/GridRoundRobinLoadBalancingSpi.java b/modules/core/src/main/java/org/gridgain/grid/spi/loadbalancing/roundrobin/GridRoundRobinLoadBalancingSpi.java
deleted file mode 100644
index 7a85098..0000000
--- a/modules/core/src/main/java/org/gridgain/grid/spi/loadbalancing/roundrobin/GridRoundRobinLoadBalancingSpi.java
+++ /dev/null
@@ -1,319 +0,0 @@
-/* @java.file.header */
-
-/*  _________        _____ __________________        _____
- *  __  ____/___________(_)______  /__  ____/______ ____(_)_______
- *  _  / __  __  ___/__  / _  __  / _  / __  _  __ `/__  / __  __ \
- *  / /_/ /  _  /    _  /  / /_/ /  / /_/ /  / /_/ / _  /  _  / / /
- *  \____/   /_/     /_/   \_,__/   \____/   \__,_/  /_/   /_/ /_/
- */
-
-package org.gridgain.grid.spi.loadbalancing.roundrobin;
-
-import org.apache.ignite.*;
-import org.apache.ignite.cluster.*;
-import org.apache.ignite.compute.*;
-import org.apache.ignite.events.*;
-import org.apache.ignite.lang.*;
-import org.apache.ignite.resources.*;
-import org.apache.ignite.spi.*;
-import org.gridgain.grid.*;
-import org.gridgain.grid.kernal.managers.eventstorage.*;
-import org.gridgain.grid.spi.loadbalancing.*;
-import org.gridgain.grid.util.typedef.internal.*;
-import org.jdk8.backport.*;
-import org.jetbrains.annotations.*;
-
-import java.util.*;
-
-import static org.apache.ignite.events.IgniteEventType.*;
-
-/**
- * This SPI iterates through nodes in round-robin fashion and pick the next
- * sequential node. Two modes of operation are supported: per-task and global
- * (see {@link #setPerTask(boolean)} configuration).
- * <p>
- * When configured in per-task mode, implementation will pick a random starting
- * node at the beginning of every task execution and then sequentially iterate through all
- * nodes in topology starting from the picked node. This is the default configuration
- * and should fit most of the use cases as it provides a fairly well-distributed
- * split and also ensures that jobs within a single task are spread out across
- * nodes to the maximum. For cases when split size is equal to the number of nodes,
- * this mode guarantees that all nodes will participate in the split.
- * <p>
- * When configured in global mode, a single sequential queue of nodes is maintained for
- * all tasks and the next node in the queue is picked every time. In this mode (unlike in
- * {@code per-task} mode) it is possible that even if split size may be equal to the
- * number of nodes, some jobs within the same task will be assigned to the same node if
- * multiple tasks are executing concurrently.
- * <h1 class="header">Coding Example</h1>
- * If you are using {@link org.apache.ignite.compute.ComputeTaskSplitAdapter} then load balancing logic
- * is transparent to your code and is handled automatically by the adapter.
- * Here is an example of how your task will look:
- * <pre name="code" class="java">
- * public class MyFooBarTask extends GridComputeTaskSplitAdapter&lt;Object, Object&gt; {
- *    &#64;Override
- *    protected Collection&lt;? extends GridComputeJob&gt; split(int gridSize, Object arg) throws GridException {
- *        List&lt;MyFooBarJob&gt; jobs = new ArrayList&lt;MyFooBarJob&gt;(gridSize);
- *
- *        for (int i = 0; i &lt; gridSize; i++) {
- *            jobs.add(new MyFooBarJob(arg));
- *        }
- *
- *        // Node assignment via load balancer
- *        // happens automatically.
- *        return jobs;
- *    }
- *    ...
- * }
- * </pre>
- * If you need more fine-grained control over how some jobs within task get mapped to a node
- * and use affinity load balancing for some other jobs within task, then you should use
- * {@link org.apache.ignite.compute.ComputeTaskAdapter}. Here is an example of how your task will look. Note that in this
- * case we manually inject load balancer and use it to pick the best node. Doing it in
- * such way would allow user to map some jobs manually and for others use load balancer.
- * <pre name="code" class="java">
- * public class MyFooBarTask extends GridComputeTaskAdapter&lt;String, String&gt; {
- *    // Inject load balancer.
- *    &#64;GridLoadBalancerResource
- *    GridComputeLoadBalancer balancer;
- *
- *    // Map jobs to grid nodes.
- *    public Map&lt;? extends GridComputeJob, GridNode&gt; map(List&lt;GridNode&gt; subgrid, String arg) throws GridException {
- *        Map&lt;MyFooBarJob, GridNode&gt; jobs = new HashMap&lt;MyFooBarJob, GridNode&gt;(subgrid.size());
- *
- *        // In more complex cases, you can actually do
- *        // more complicated assignments of jobs to nodes.
- *        for (int i = 0; i &lt; subgrid.size(); i++) {
- *            // Pick the next best balanced node for the job.
- *            jobs.put(new MyFooBarJob(arg), balancer.getBalancedNode())
- *        }
- *
- *        return jobs;
- *    }
- *
- *    // Aggregate results into one compound result.
- *    public String reduce(List&lt;GridComputeJobResult&gt; results) throws GridException {
- *        // For the purpose of this example we simply
- *        // concatenate string representation of every
- *        // job result
- *        StringBuilder buf = new StringBuilder();
- *
- *        for (GridComputeJobResult res : results) {
- *            // Append string representation of result
- *            // returned by every job.
- *            buf.append(res.getData().string());
- *        }
- *
- *        return buf.string();
- *    }
- * }
- * </pre>
- * <p>
- * <h1 class="header">Configuration</h1>
- * In order to use this load balancer, you should configure your grid instance
- * to use {@code GridRoundRobinLoadBalancingSpi} either from Spring XML file or
- * directly. The following configuration parameters are supported:
- * <h2 class="header">Mandatory</h2>
- * This SPI has no mandatory configuration parameters.
- * <h2 class="header">Optional</h2>
- * The following configuration parameters are optional:
- * <ul>
- * <li>
- *      Flag that indicates whether to use {@code per-task} or global
- *      round-robin modes described above (see {@link #setPerTask(boolean)}).
- * </li>
- * </ul>
- * Below is Java configuration example:
- * <pre name="code" class="java">
- * GridRandomLoadBalancingSpi = new GridRandomLoadBalancingSpi();
- *
- * // Configure SPI to use global round-robin mode.
- * spi.setPerTask(false);
- *
- * GridConfiguration cfg = new GridConfiguration();
- *
- * // Override default load balancing SPI.
- * cfg.setLoadBalancingSpi(spi);
- *
- * // Starts grid.
- * G.start(cfg);
- * </pre>
- * Here is how you can configure {@code GridRandomLoadBalancingSpi} using Spring XML configuration:
- * <pre name="code" class="xml">
- * &lt;property name="loadBalancingSpi"&gt;
- *     &lt;bean class="org.gridgain.grid.spi.loadBalancing.roundrobin.GridRoundRobinLoadBalancingSpi"&gt;
- *         &lt;!-- Set to global round-robin mode. --&gt;
- *         &lt;property name="perTask" value="false"/&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>
- */
-@IgniteSpiMultipleInstancesSupport(true)
-public class GridRoundRobinLoadBalancingSpi extends IgniteSpiAdapter implements GridLoadBalancingSpi,
-    GridRoundRobinLoadBalancingSpiMBean {
-    /** Grid logger. */
-    @IgniteLoggerResource
-    private IgniteLogger log;
-
-    /** */
-    private GridRoundRobinGlobalLoadBalancer balancer;
-
-    /** */
-    private boolean isPerTask;
-
-    /** */
-    private final Map<IgniteUuid, GridRoundRobinPerTaskLoadBalancer> perTaskBalancers =
-        new ConcurrentHashMap8<>();
-
-    /** Event listener. */
-    private final GridLocalEventListener lsnr = new GridLocalEventListener() {
-        @Override public void onEvent(IgniteEvent evt) {
-            if (evt.type() == EVT_TASK_FAILED ||
-                evt.type() == EVT_TASK_FINISHED)
-                perTaskBalancers.remove(((IgniteTaskEvent)evt).taskSessionId());
-            else if (evt.type() == EVT_JOB_MAPPED) {
-                GridRoundRobinPerTaskLoadBalancer balancer =
-                    perTaskBalancers.get(((IgniteJobEvent)evt).taskSessionId());
-
-                if (balancer != null)
-                    balancer.onMapped();
-            }
-        }
-    };
-
-    /** {@inheritDoc} */
-    @Override public boolean isPerTask() {
-        return isPerTask;
-    }
-
-    /**
-     * Configuration parameter indicating whether a new round robin order should be
-     * created for every task. If {@code true} then load balancer is guaranteed
-     * to iterate through nodes sequentially for every task - so as long as number
-     * of jobs is less than or equal to the number of nodes, jobs are guaranteed to
-     * be assigned to unique nodes. If {@code false} then one round-robin order
-     * will be maintained for all tasks, so when tasks execute concurrently, it
-     * is possible for more than one job within task to be assigned to the same
-     * node.
-     * <p>
-     * Default is {@code false}.
-     *
-     * @param isPerTask Configuration parameter indicating whether a new round robin order should
-     *      be created for every task. Default is {@code false}.
-     */
-    @IgniteSpiConfiguration(optional = true)
-    public void setPerTask(boolean isPerTask) {
-        this.isPerTask = isPerTask;
-    }
-
-    /** {@inheritDoc} */
-    @Override public void spiStart(@Nullable String gridName) throws IgniteSpiException {
-        startStopwatch();
-
-        if (log.isDebugEnabled())
-            log.debug(configInfo("isPerTask", isPerTask));
-
-        registerMBean(gridName, this, GridRoundRobinLoadBalancingSpiMBean.class);
-
-        balancer = new GridRoundRobinGlobalLoadBalancer(log);
-
-        // Ack ok start.
-        if (log.isDebugEnabled())
-            log.debug(startInfo());
-    }
-
-    /** {@inheritDoc} */
-    @Override public void spiStop() throws IgniteSpiException {
-        balancer = null;
-
-        perTaskBalancers.clear();
-
-        unregisterMBean();
-
-        // Ack ok stop.
-        if (log.isDebugEnabled())
-            log.debug(stopInfo());
-    }
-
-    /** {@inheritDoc} */
-    @Override protected void onContextInitialized0(IgniteSpiContext spiCtx) throws IgniteSpiException {
-        if (!isPerTask)
-            balancer.onContextInitialized(spiCtx);
-        else {
-            if (!getSpiContext().isEventRecordable(EVT_TASK_FAILED, EVT_TASK_FINISHED, EVT_JOB_MAPPED))
-                throw new IgniteSpiException("Required event types are disabled: " +
-                    U.gridEventName(EVT_TASK_FAILED) + ", " +
-                    U.gridEventName(EVT_TASK_FINISHED) + ", " +
-                    U.gridEventName(EVT_JOB_MAPPED));
-
-            getSpiContext().addLocalEventListener(lsnr, EVT_TASK_FAILED, EVT_TASK_FINISHED, EVT_JOB_MAPPED);
-        }
-    }
-
-    /** {@inheritDoc} */
-    @Override protected void onContextDestroyed0() {
-        if (!isPerTask) {
-            if (balancer != null)
-                balancer.onContextDestroyed();
-        }
-        else {
-            IgniteSpiContext spiCtx = getSpiContext();
-
-            if (spiCtx != null)
-                spiCtx.removeLocalEventListener(lsnr);
-        }
-    }
-
-    /** {@inheritDoc} */
-    @Override public ClusterNode getBalancedNode(ComputeTaskSession ses, List<ClusterNode> top, ComputeJob job)
-        throws GridException {
-        A.notNull(ses, "ses", top, "top");
-
-        if (isPerTask) {
-            // Note that every session operates from single thread which
-            // allows us to use concurrent map and avoid synchronization.
-            GridRoundRobinPerTaskLoadBalancer taskBalancer = perTaskBalancers.get(ses.getId());
-
-            if (taskBalancer == null)
-                perTaskBalancers.put(ses.getId(), taskBalancer = new GridRoundRobinPerTaskLoadBalancer());
-
-            return taskBalancer.getBalancedNode(top);
-        }
-
-        return balancer.getBalancedNode(top);
-    }
-
-    /**
-     * THIS METHOD IS USED ONLY FOR TESTING.
-     *
-     * @param ses Task session.
-     * @return Internal list of nodes.
-     */
-    List<UUID> getNodeIds(ComputeTaskSession ses) {
-        if (isPerTask) {
-            GridRoundRobinPerTaskLoadBalancer balancer = perTaskBalancers.get(ses.getId());
-
-            if (balancer == null)
-                return Collections.emptyList();
-
-            List<UUID> ids = new ArrayList<>();
-
-            for (ClusterNode node : balancer.getNodes()) {
-                ids.add(node.id());
-            }
-
-            return ids;
-        }
-
-        return balancer.getNodeIds();
-    }
-
-    /** {@inheritDoc} */
-    @Override public String toString() {
-        return S.toString(GridRoundRobinLoadBalancingSpi.class, this);
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/a62862fe/modules/core/src/main/java/org/gridgain/grid/spi/loadbalancing/roundrobin/GridRoundRobinLoadBalancingSpiMBean.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/spi/loadbalancing/roundrobin/GridRoundRobinLoadBalancingSpiMBean.java b/modules/core/src/main/java/org/gridgain/grid/spi/loadbalancing/roundrobin/GridRoundRobinLoadBalancingSpiMBean.java
deleted file mode 100644
index 2f561ae..0000000
--- a/modules/core/src/main/java/org/gridgain/grid/spi/loadbalancing/roundrobin/GridRoundRobinLoadBalancingSpiMBean.java
+++ /dev/null
@@ -1,37 +0,0 @@
-/* @java.file.header */
-
-/*  _________        _____ __________________        _____
- *  __  ____/___________(_)______  /__  ____/______ ____(_)_______
- *  _  / __  __  ___/__  / _  __  / _  / __  _  __ `/__  / __  __ \
- *  / /_/ /  _  /    _  /  / /_/ /  / /_/ /  / /_/ / _  /  _  / / /
- *  \____/   /_/     /_/   \_,__/   \____/   \__,_/  /_/   /_/ /_/
- */
-
-package org.gridgain.grid.spi.loadbalancing.roundrobin;
-
-import org.apache.ignite.mbean.*;
-import org.apache.ignite.spi.*;
-
-/**
- * Management bean for {@link GridRoundRobinLoadBalancingSpi} SPI.
- */
-@IgniteMBeanDescription("MBean that provides access to round robin load balancing SPI configuration.")
-public interface GridRoundRobinLoadBalancingSpiMBean extends IgniteSpiManagementMBean {
-    /**
-     * Configuration parameter indicating whether a new round robin order should be
-     * created for every task. If {@code true} then load balancer is guaranteed
-     * to iterate through nodes sequentially for every task - so as long as number
-     * of jobs is less than or equal to the number of nodes, jobs are guaranteed to
-     * be assigned to unique nodes. If {@code false} then one round-robin order
-     * will be maintained for all tasks, so when tasks execute concurrently, it
-     * is possible for more than one job within task to be assigned to the same
-     * node.
-     * <p>
-     * Default is {@code true}.
-     *
-     * @return Configuration parameter indicating whether a new round robin order should
-     *      be created for every task. Default is {@code true}.
-     */
-    @IgniteMBeanDescription("Configuration parameter indicating whether a new round robin order should be created for every task.")
-    public boolean isPerTask();
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/a62862fe/modules/core/src/main/java/org/gridgain/grid/spi/loadbalancing/roundrobin/GridRoundRobinPerTaskLoadBalancer.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/spi/loadbalancing/roundrobin/GridRoundRobinPerTaskLoadBalancer.java b/modules/core/src/main/java/org/gridgain/grid/spi/loadbalancing/roundrobin/GridRoundRobinPerTaskLoadBalancer.java
deleted file mode 100644
index 4544955..0000000
--- a/modules/core/src/main/java/org/gridgain/grid/spi/loadbalancing/roundrobin/GridRoundRobinPerTaskLoadBalancer.java
+++ /dev/null
@@ -1,96 +0,0 @@
-/* @java.file.header */
-
-/*  _________        _____ __________________        _____
- *  __  ____/___________(_)______  /__  ____/______ ____(_)_______
- *  _  / __  __  ___/__  / _  __  / _  / __  _  __ `/__  / __  __ \
- *  / /_/ /  _  /    _  /  / /_/ /  / /_/ /  / /_/ / _  /  _  / / /
- *  \____/   /_/     /_/   \_,__/   \____/   \__,_/  /_/   /_/ /_/
- */
-
-package org.gridgain.grid.spi.loadbalancing.roundrobin;
-
-import org.apache.ignite.cluster.*;
-
-import java.util.*;
-
-/**
- * Load balancer for per-task configuration.
- */
-class GridRoundRobinPerTaskLoadBalancer {
-    /** Balancing nodes. */
-    private ArrayDeque<ClusterNode> nodeQueue;
-
-    /** Jobs mapped flag. */
-    private volatile boolean isMapped;
-
-    /** Mutex. */
-    private final Object mux = new Object();
-
-    /**
-     * Call back for job mapped event.
-     */
-    void onMapped() {
-        isMapped = true;
-    }
-
-    /**
-     * Gets balanced node for given topology. This implementation
-     * is to be used only from {@link org.apache.ignite.compute.ComputeTask#map(List, Object)} method
-     * and, therefore, does not need to be thread-safe.
-     *
-     * @param top Topology to pick from.
-     * @return Best balanced node.
-     */
-    ClusterNode getBalancedNode(List<ClusterNode> top) {
-        assert top != null;
-        assert !top.isEmpty();
-
-        boolean readjust = isMapped;
-
-        synchronized (mux) {
-            // Populate first time.
-            if (nodeQueue == null)
-                nodeQueue = new ArrayDeque<>(top);
-
-            // If job has been mapped, then it means
-            // that it is most likely being failed over.
-            // In this case topology might have changed
-            // and we need to readjust with every apply.
-            if (readjust)
-                // Add missing nodes.
-                for (ClusterNode node : top)
-                    if (!nodeQueue.contains(node))
-                        nodeQueue.offer(node);
-
-            ClusterNode next = nodeQueue.poll();
-
-            // If jobs have been mapped, we need to make sure
-            // that queued node is still in topology.
-            if (readjust && next != null) {
-                while (!top.contains(next) && !nodeQueue.isEmpty())
-                    next = nodeQueue.poll();
-
-                // No nodes found and queue is empty.
-                if (next != null && !top.contains(next))
-                    return null;
-            }
-
-            if (next != null)
-                // Add to the end.
-                nodeQueue.offer(next);
-
-            return next;
-        }
-    }
-
-    /**
-     * THIS METHOD IS USED ONLY FOR TESTING.
-     *
-     * @return Internal list of nodes.
-     */
-    List<ClusterNode> getNodes() {
-        synchronized (mux) {
-            return Collections.unmodifiableList(new ArrayList<>(nodeQueue));
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/a62862fe/modules/core/src/main/java/org/gridgain/grid/spi/loadbalancing/roundrobin/RoundRobinGlobalLoadBalancer.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/spi/loadbalancing/roundrobin/RoundRobinGlobalLoadBalancer.java b/modules/core/src/main/java/org/gridgain/grid/spi/loadbalancing/roundrobin/RoundRobinGlobalLoadBalancer.java
new file mode 100644
index 0000000..e467bac
--- /dev/null
+++ b/modules/core/src/main/java/org/gridgain/grid/spi/loadbalancing/roundrobin/RoundRobinGlobalLoadBalancer.java
@@ -0,0 +1,305 @@
+/* @java.file.header */
+
+/*  _________        _____ __________________        _____
+ *  __  ____/___________(_)______  /__  ____/______ ____(_)_______
+ *  _  / __  __  ___/__  / _  __  / _  / __  _  __ `/__  / __  __ \
+ *  / /_/ /  _  /    _  /  / /_/ /  / /_/ /  / /_/ / _  /  _  / / /
+ *  \____/   /_/     /_/   \_,__/   \____/   \__,_/  /_/   /_/ /_/
+ */
+
+package org.gridgain.grid.spi.loadbalancing.roundrobin;
+
+import org.apache.ignite.*;
+import org.apache.ignite.cluster.*;
+import org.apache.ignite.events.*;
+import org.apache.ignite.spi.*;
+import org.gridgain.grid.*;
+import org.gridgain.grid.kernal.managers.eventstorage.*;
+import org.gridgain.grid.util.typedef.*;
+import org.gridgain.grid.util.typedef.internal.*;
+
+import java.util.*;
+import java.util.concurrent.*;
+import java.util.concurrent.atomic.*;
+
+import static org.apache.ignite.events.IgniteEventType.*;
+
+/**
+ * Load balancer that works in global (not-per-task) mode.
+ */
+class RoundRobinGlobalLoadBalancer {
+    /** SPI context. */
+    private IgniteSpiContext ctx;
+
+    /** Listener for node's events. */
+    private GridLocalEventListener lsnr;
+
+    /** Logger. */
+    private final IgniteLogger log;
+
+    /** Current snapshot of nodes which participated in load balancing. */
+    private volatile GridNodeList nodeList = new GridNodeList(0, null);
+
+    /** Mutex for updating current topology. */
+    private final Object mux = new Object();
+
+    /** Barrier for separating initialization callback and load balancing routine. */
+    private final CountDownLatch initLatch = new CountDownLatch(1);
+
+    /**
+     * @param log Grid logger.
+     */
+    RoundRobinGlobalLoadBalancer(IgniteLogger log) {
+        assert log != null;
+
+        this.log = log;
+    }
+
+    /**
+     * @param ctx Load balancing context.
+     */
+    void onContextInitialized(final IgniteSpiContext ctx) {
+        this.ctx = ctx;
+
+        ctx.addLocalEventListener(
+            lsnr = new GridLocalEventListener() {
+                @Override public void onEvent(IgniteEvent evt) {
+                    assert evt instanceof IgniteDiscoveryEvent;
+
+                    UUID nodeId = ((IgniteDiscoveryEvent)evt).eventNode().id();
+
+                    synchronized (mux) {
+                        if (evt.type() == EVT_NODE_JOINED) {
+                            List<UUID> oldNodes = nodeList.getNodes();
+
+                            if (!oldNodes.contains(nodeId)) {
+                                List<UUID> newNodes = new ArrayList<>(oldNodes.size() + 1);
+
+                                newNodes.add(nodeId);
+
+                                for (UUID node : oldNodes)
+                                    newNodes.add(node);
+
+                                nodeList = new GridNodeList(0, newNodes);
+                            }
+                        }
+                        else {
+                            assert evt.type() == EVT_NODE_LEFT || evt.type() == EVT_NODE_FAILED;
+
+                            List<UUID> oldNodes = nodeList.getNodes();
+
+                            if (oldNodes.contains(nodeId)) {
+                                List<UUID> newNodes = new ArrayList<>(oldNodes.size() - 1);
+
+                                for (UUID node : oldNodes)
+                                    if (!nodeId.equals(node))
+                                        newNodes.add(node);
+
+                                nodeList = new GridNodeList(0, newNodes);
+                            }
+                        }
+                    }
+                }
+            },
+            EVT_NODE_FAILED, EVT_NODE_JOINED, EVT_NODE_LEFT
+        );
+
+        synchronized (mux) {
+            List<UUID> oldNodes = nodeList.getNodes();
+
+            Collection<UUID> set = oldNodes == null ? new HashSet<UUID>() : new HashSet<>(oldNodes);
+
+            for (ClusterNode node : ctx.nodes())
+                set.add(node.id());
+
+            nodeList = new GridNodeList(0, new ArrayList<>(set));
+        }
+
+        initLatch.countDown();
+    }
+
+    /** */
+    void onContextDestroyed() {
+        if (ctx != null)
+            ctx.removeLocalEventListener(lsnr);
+    }
+
+    /**
+     * Gets balanced node for given topology.
+     *
+     * @param top Topology to pick from.
+     * @return Best balanced node.
+     * @throws GridException Thrown in case of any error.
+     */
+    ClusterNode getBalancedNode(Collection<ClusterNode> top) throws GridException {
+        assert !F.isEmpty(top);
+
+        awaitInitializationCompleted();
+
+        Map<UUID, ClusterNode> topMap = null;
+
+        ClusterNode found;
+
+        int misses = 0;
+
+        do {
+            GridNodeList nodeList = this.nodeList;
+
+            List<UUID> nodes = nodeList.getNodes();
+
+            int cycleSize = nodes.size();
+
+            if (cycleSize == 0)
+                throw new GridException("Task topology does not have any alive nodes.");
+
+            AtomicInteger idx;
+
+            int curIdx, nextIdx;
+
+            do {
+                idx = nodeList.getCurrentIdx();
+
+                curIdx = idx.get();
+
+                nextIdx = (idx.get() + 1) % cycleSize;
+            }
+            while (!idx.compareAndSet(curIdx, nextIdx));
+
+            found = findNodeById(top, nodes.get(nextIdx));
+
+            if (found == null) {
+                misses++;
+
+                // For optimization purposes checks balancer can return at least one node with specified
+                // request topology only after full cycle (approximately).
+                if (misses >= cycleSize) {
+                    if (topMap == null) {
+                        topMap = U.newHashMap(top.size());
+
+                        for (ClusterNode node : top)
+                            topMap.put(node.id(), node);
+                    }
+
+                    checkBalancerNodes(top, topMap, nodes);
+
+                    // Zero miss counter so next topology check will be performed once again after full cycle.
+                    misses = 0;
+                }
+            }
+        }
+        while (found == null);
+
+        if (log.isDebugEnabled())
+            log.debug("Found round-robin node: " + found);
+
+        return found;
+    }
+
+    /**
+     * Finds node by id. Returns null in case of absence of specified id in request topology.
+     *
+     * @param top Topology for current request.
+     * @param foundNodeId Node id.
+     * @return Found node or null in case of absence of specified id in request topology.
+     */
+    private static ClusterNode findNodeById(Iterable<ClusterNode> top, UUID foundNodeId) {
+        for (ClusterNode node : top)
+            if (foundNodeId.equals(node.id()))
+                return node;
+
+        return null;
+    }
+
+    /**
+     * Checks if balancer can return at least one node,
+     * throw exception otherwise.
+     *
+     * @param top Topology for current request.
+     * @param topMap Topology map.
+     * @param nodes Current balanced nodes.
+     * @throws GridException If balancer can not return any node.
+     */
+    private static void checkBalancerNodes(Collection<ClusterNode> top, Map<UUID, ClusterNode> topMap, Iterable<UUID> nodes)
+        throws GridException {
+
+        boolean contains = false;
+
+        for (UUID nodeId : nodes) {
+            if (topMap.get(nodeId) != null) {
+                contains = true;
+
+                break;
+            }
+        }
+
+        if (!contains)
+            throw new GridException("Task topology does not have alive nodes: " + top);
+    }
+
+    /**
+     * Awaits initialization of balancing nodes to be completed.
+     *
+     * @throws GridException Thrown in case of thread interruption.
+     */
+    private void awaitInitializationCompleted() throws GridException {
+        try {
+            if (initLatch.getCount() > 0)
+                initLatch.await();
+        }
+        catch (InterruptedException e) {
+            Thread.currentThread().interrupt();
+
+            throw new GridException("Global balancer was interrupted.", e);
+        }
+    }
+
+    /**
+     * Snapshot of nodes which participated in load balancing.
+     */
+    private static final class GridNodeList {
+        /** Cyclic pointer for selecting next node. */
+        private final AtomicInteger curIdx;
+
+        /** Node ids. */
+        private final List<UUID> nodes;
+
+        /**
+         * @param curIdx Initial index of current node.
+         * @param nodes Initial node ids.
+         */
+        private GridNodeList(int curIdx, List<UUID> nodes) {
+            this.curIdx = new AtomicInteger(curIdx);
+            this.nodes = nodes;
+        }
+
+        /**
+         * @return Index of current node.
+         */
+        private AtomicInteger getCurrentIdx() {
+            return curIdx;
+        }
+
+        /**
+         * @return Node ids.
+         */
+        private List<UUID> getNodes() {
+            return nodes;
+        }
+    }
+
+    /**
+     * THIS METHOD IS USED ONLY FOR TESTING.
+     *
+     * @return Internal list of nodes.
+     */
+    List<UUID> getNodeIds() {
+        List<UUID> nodes = nodeList.getNodes();
+
+        return Collections.unmodifiableList(nodes);
+    }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return S.toString(RoundRobinGlobalLoadBalancer.class, this);
+    }
+}


[08/20] incubator-ignite git commit: # Renaming

Posted by sb...@apache.org.
# Renaming


Project: http://git-wip-us.apache.org/repos/asf/incubator-ignite/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-ignite/commit/b69a23cd
Tree: http://git-wip-us.apache.org/repos/asf/incubator-ignite/tree/b69a23cd
Diff: http://git-wip-us.apache.org/repos/asf/incubator-ignite/diff/b69a23cd

Branch: refs/heads/master
Commit: b69a23cd226056307fc288af69e7863dbfe19181
Parents: a62862f
Author: sboikov <sb...@gridgain.com>
Authored: Fri Dec 5 17:09:19 2014 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Fri Dec 5 17:09:21 2014 +0300

----------------------------------------------------------------------
 .../java/org/apache/ignite/IgniteCompute.java   |   2 +-
 .../ignite/compute/ComputeLoadBalancer.java     |   2 +-
 .../apache/ignite/compute/ComputeTaskSpis.java  |   2 +-
 .../configuration/IgniteConfiguration.java      |   8 +-
 .../resources/IgniteLoadBalancerResource.java   |   2 +-
 .../ignite/spi/failover/FailoverContext.java    |   2 +-
 .../spi/loadbalancing/LoadBalancingSpi.java     | 114 ++++
 .../adaptive/AdaptiveCpuLoadProbe.java          | 229 ++++++++
 .../adaptive/AdaptiveJobCountLoadProbe.java     |  96 +++
 .../adaptive/AdaptiveLoadBalancingSpi.java      | 581 +++++++++++++++++++
 .../adaptive/AdaptiveLoadBalancingSpiMBean.java |  27 +
 .../adaptive/AdaptiveLoadProbe.java             |  90 +++
 .../AdaptiveProcessingTimeLoadProbe.java        |  98 ++++
 .../spi/loadbalancing/adaptive/package.html     |  15 +
 .../ignite/spi/loadbalancing/package.html       |  15 +
 .../RoundRobinGlobalLoadBalancer.java           | 305 ++++++++++
 .../roundrobin/RoundRobinLoadBalancingSpi.java  | 319 ++++++++++
 .../RoundRobinLoadBalancingSpiMBean.java        |  37 ++
 .../RoundRobinPerTaskLoadBalancer.java          |  96 +++
 .../spi/loadbalancing/roundrobin/package.html   |  15 +
 .../WeightedRandomLoadBalancingSpi.java         | 394 +++++++++++++
 .../WeightedRandomLoadBalancingSpiMBean.java    |  37 ++
 .../loadbalancing/weightedrandom/package.html   |  15 +
 .../affinity/GridCacheAffinityKeyMapped.java    |   2 +-
 .../org/gridgain/grid/kernal/GridGainEx.java    |   4 +-
 .../loadbalancer/GridLoadBalancerManager.java   |   2 +-
 .../spi/loadbalancing/LoadBalancingSpi.java     | 114 ----
 .../adaptive/AdaptiveCpuLoadProbe.java          | 229 --------
 .../adaptive/AdaptiveJobCountLoadProbe.java     |  96 ---
 .../adaptive/AdaptiveLoadBalancingSpi.java      | 581 -------------------
 .../adaptive/AdaptiveLoadBalancingSpiMBean.java |  27 -
 .../adaptive/AdaptiveLoadProbe.java             |  90 ---
 .../AdaptiveProcessingTimeLoadProbe.java        |  98 ----
 .../spi/loadbalancing/adaptive/package.html     |  15 -
 .../grid/spi/loadbalancing/package.html         |  15 -
 .../RoundRobinGlobalLoadBalancer.java           | 305 ----------
 .../roundrobin/RoundRobinLoadBalancingSpi.java  | 319 ----------
 .../RoundRobinLoadBalancingSpiMBean.java        |  37 --
 .../RoundRobinPerTaskLoadBalancer.java          |  96 ---
 .../spi/loadbalancing/roundrobin/package.html   |  15 -
 .../WeightedRandomLoadBalancingSpi.java         | 394 -------------
 .../WeightedRandomLoadBalancingSpiMBean.java    |  37 --
 .../loadbalancing/weightedrandom/package.html   |  15 -
 .../src/test/config/io-manager-benchmark.xml    |   2 +-
 modules/core/src/test/config/jobs-load-base.xml |   2 +-
 .../src/test/config/load/merge-sort-base.xml    |   2 +-
 .../config/spring-cache-put-remove-load.xml     |   2 +-
 ...dAdaptiveLoadBalancingSpiConfigSelfTest.java |  26 +
 ...iveLoadBalancingSpiMultipleNodeSelfTest.java |  87 +++
 .../GridAdaptiveLoadBalancingSpiSelfTest.java   | 125 ++++
 ...aptiveLoadBalancingSpiStartStopSelfTest.java |  23 +
 .../spi/loadbalancing/adaptive/package.html     |  15 +
 .../ignite/spi/loadbalancing/package.html       |  15 +
 ...alancingNotPerTaskMultithreadedSelfTest.java | 115 ++++
 ...dRobinLoadBalancingSpiLocalNodeSelfTest.java |  44 ++
 ...inLoadBalancingSpiMultipleNodesSelfTest.java | 126 ++++
 ...RobinLoadBalancingSpiNotPerTaskSelfTest.java | 121 ++++
 ...dRobinLoadBalancingSpiStartStopSelfTest.java |  23 +
 ...nLoadBalancingSpiTopologyChangeSelfTest.java |  98 ++++
 .../roundrobin/GridRoundRobinTestUtils.java     |  95 +++
 .../spi/loadbalancing/roundrobin/package.html   |  15 +
 ...tedRandomLoadBalancingSpiConfigSelfTest.java |  26 +
 ...dWeightedRandomLoadBalancingSpiSelfTest.java |  58 ++
 ...RandomLoadBalancingSpiStartStopSelfTest.java |  23 +
 ...dRandomLoadBalancingSpiWeightedSelfTest.java |  73 +++
 .../loadbalancing/weightedrandom/package.html   |  15 +
 .../grid/kernal/GridMultipleSpisSelfTest.java   |   2 +-
 .../managers/GridManagerStopSelfTest.java       |   2 +-
 ...dAdaptiveLoadBalancingSpiConfigSelfTest.java |  26 -
 ...iveLoadBalancingSpiMultipleNodeSelfTest.java |  87 ---
 .../GridAdaptiveLoadBalancingSpiSelfTest.java   | 125 ----
 ...aptiveLoadBalancingSpiStartStopSelfTest.java |  23 -
 .../spi/loadbalancing/adaptive/package.html     |  15 -
 .../grid/spi/loadbalancing/package.html         |  15 -
 ...alancingNotPerTaskMultithreadedSelfTest.java | 115 ----
 ...dRobinLoadBalancingSpiLocalNodeSelfTest.java |  44 --
 ...inLoadBalancingSpiMultipleNodesSelfTest.java | 126 ----
 ...RobinLoadBalancingSpiNotPerTaskSelfTest.java | 121 ----
 ...dRobinLoadBalancingSpiStartStopSelfTest.java |  23 -
 ...nLoadBalancingSpiTopologyChangeSelfTest.java |  98 ----
 .../roundrobin/GridRoundRobinTestUtils.java     |  95 ---
 .../spi/loadbalancing/roundrobin/package.html   |  15 -
 ...tedRandomLoadBalancingSpiConfigSelfTest.java |  26 -
 ...dWeightedRandomLoadBalancingSpiSelfTest.java |  58 --
 ...RandomLoadBalancingSpiStartStopSelfTest.java |  23 -
 ...dRandomLoadBalancingSpiWeightedSelfTest.java |  73 ---
 .../loadbalancing/weightedrandom/package.html   |  15 -
 .../GridSpiLoadBalancingSelfTestSuite.java      |   6 +-
 pom.xml                                         |   4 +-
 89 files changed, 3630 insertions(+), 3630 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/b69a23cd/modules/core/src/main/java/org/apache/ignite/IgniteCompute.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/IgniteCompute.java b/modules/core/src/main/java/org/apache/ignite/IgniteCompute.java
index 875b22a..39818ee 100644
--- a/modules/core/src/main/java/org/apache/ignite/IgniteCompute.java
+++ b/modules/core/src/main/java/org/apache/ignite/IgniteCompute.java
@@ -51,7 +51,7 @@ import java.util.concurrent.*;
  * {@link Serializable} and should be used to run computations on the grid.
  * <h1 class="header">Load Balancing</h1>
  * In all cases other than {@code broadcast(...)}, GridGain must select a node for a computation
- * to be executed. The node will be selected based on the underlying {@link org.gridgain.grid.spi.loadbalancing.LoadBalancingSpi},
+ * to be executed. The node will be selected based on the underlying {@link org.apache.ignite.spi.loadbalancing.LoadBalancingSpi},
  * which by default sequentially picks next available node from grid projection. Other load balancing
  * policies, such as {@code random} or {@code adaptive}, can be configured as well by selecting
  * a different load balancing SPI in grid configuration. If your logic requires some custom

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/b69a23cd/modules/core/src/main/java/org/apache/ignite/compute/ComputeLoadBalancer.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/compute/ComputeLoadBalancer.java b/modules/core/src/main/java/org/apache/ignite/compute/ComputeLoadBalancer.java
index edba366..54a265c 100644
--- a/modules/core/src/main/java/org/apache/ignite/compute/ComputeLoadBalancer.java
+++ b/modules/core/src/main/java/org/apache/ignite/compute/ComputeLoadBalancer.java
@@ -18,7 +18,7 @@ import java.util.*;
 /**
  * Load balancer is used for finding the best balanced node according
  * to load balancing policy. Internally load balancer will
- * query the {@link org.gridgain.grid.spi.loadbalancing.LoadBalancingSpi}
+ * query the {@link org.apache.ignite.spi.loadbalancing.LoadBalancingSpi}
  * to get the balanced node.
  * <p>
  * Load balancer can be used <i>explicitly</i> from inside {@link ComputeTask#map(List, Object)}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/b69a23cd/modules/core/src/main/java/org/apache/ignite/compute/ComputeTaskSpis.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/compute/ComputeTaskSpis.java b/modules/core/src/main/java/org/apache/ignite/compute/ComputeTaskSpis.java
index 6c1d231..33dd020 100644
--- a/modules/core/src/main/java/org/apache/ignite/compute/ComputeTaskSpis.java
+++ b/modules/core/src/main/java/org/apache/ignite/compute/ComputeTaskSpis.java
@@ -14,7 +14,7 @@ import java.lang.annotation.*;
 /**
  * This annotation allows task to specify what SPIs it wants to use.
  * Starting with {@code GridGain 2.1} you can start multiple instances
- * of {@link org.gridgain.grid.spi.loadbalancing.LoadBalancingSpi},
+ * of {@link org.apache.ignite.spi.loadbalancing.LoadBalancingSpi},
  * {@link org.apache.ignite.spi.failover.FailoverSpi}, and {@link org.apache.ignite.spi.checkpoint.CheckpointSpi}. If you do that,
  * you need to tell a task which SPI to use (by default it will use the fist
  * SPI in the list).

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/b69a23cd/modules/core/src/main/java/org/apache/ignite/configuration/IgniteConfiguration.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/configuration/IgniteConfiguration.java b/modules/core/src/main/java/org/apache/ignite/configuration/IgniteConfiguration.java
index d3386d2..f8b2547 100644
--- a/modules/core/src/main/java/org/apache/ignite/configuration/IgniteConfiguration.java
+++ b/modules/core/src/main/java/org/apache/ignite/configuration/IgniteConfiguration.java
@@ -35,7 +35,7 @@ import org.apache.ignite.spi.deployment.*;
 import org.apache.ignite.spi.discovery.*;
 import org.apache.ignite.spi.eventstorage.*;
 import org.apache.ignite.spi.failover.*;
-import org.gridgain.grid.spi.loadbalancing.*;
+import org.apache.ignite.spi.loadbalancing.*;
 import org.gridgain.grid.spi.securesession.*;
 import org.gridgain.grid.spi.securesession.noop.*;
 import org.gridgain.grid.spi.swapspace.*;
@@ -2073,7 +2073,7 @@ public class IgniteConfiguration {
 
     /**
      * Should return fully configured load balancing SPI implementation. If not provided,
-     * {@link org.gridgain.grid.spi.loadbalancing.roundrobin.RoundRobinLoadBalancingSpi} will be used.
+     * {@link org.apache.ignite.spi.loadbalancing.roundrobin.RoundRobinLoadBalancingSpi} will be used.
      *
      * @return Grid load balancing SPI implementation or {@code null} to use default implementation.
      */
@@ -2114,9 +2114,9 @@ public class IgniteConfiguration {
     }
 
     /**
-     * Sets fully configured instance of {@link org.gridgain.grid.spi.loadbalancing.LoadBalancingSpi}.
+     * Sets fully configured instance of {@link org.apache.ignite.spi.loadbalancing.LoadBalancingSpi}.
      *
-     * @param loadBalancingSpi Fully configured instance of {@link org.gridgain.grid.spi.loadbalancing.LoadBalancingSpi} or
+     * @param loadBalancingSpi Fully configured instance of {@link org.apache.ignite.spi.loadbalancing.LoadBalancingSpi} or
      *      {@code null} if no SPI provided.
      * @see IgniteConfiguration#getLoadBalancingSpi()
      */

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/b69a23cd/modules/core/src/main/java/org/apache/ignite/resources/IgniteLoadBalancerResource.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/resources/IgniteLoadBalancerResource.java b/modules/core/src/main/java/org/apache/ignite/resources/IgniteLoadBalancerResource.java
index 6479e94..2edea7b 100644
--- a/modules/core/src/main/java/org/apache/ignite/resources/IgniteLoadBalancerResource.java
+++ b/modules/core/src/main/java/org/apache/ignite/resources/IgniteLoadBalancerResource.java
@@ -14,7 +14,7 @@ import java.lang.annotation.*;
 /**
  * Annotates a field or a setter method for injection of {@link org.apache.ignite.compute.ComputeLoadBalancer}.
  * Specific implementation for grid load balancer is defined by
- * {@link org.gridgain.grid.spi.loadbalancing.LoadBalancingSpi}
+ * {@link org.apache.ignite.spi.loadbalancing.LoadBalancingSpi}
  * which is provided to grid via {@link org.apache.ignite.configuration.IgniteConfiguration}..
  * <p>
  * Load balancer can be injected into instances of following classes:

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/b69a23cd/modules/core/src/main/java/org/apache/ignite/spi/failover/FailoverContext.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/failover/FailoverContext.java b/modules/core/src/main/java/org/apache/ignite/spi/failover/FailoverContext.java
index 03603cb..64c6af7 100644
--- a/modules/core/src/main/java/org/apache/ignite/spi/failover/FailoverContext.java
+++ b/modules/core/src/main/java/org/apache/ignite/spi/failover/FailoverContext.java
@@ -36,7 +36,7 @@ public interface FailoverContext {
 
     /**
      * Gets the next balanced node for failed job. Internally this method will
-     * delegate to load balancing SPI (see {@link org.gridgain.grid.spi.loadbalancing.LoadBalancingSpi} to
+     * delegate to load balancing SPI (see {@link org.apache.ignite.spi.loadbalancing.LoadBalancingSpi} to
      * determine the optimal node for execution.
      *
      * @param top Topology to pick balanced node from.

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/b69a23cd/modules/core/src/main/java/org/apache/ignite/spi/loadbalancing/LoadBalancingSpi.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/loadbalancing/LoadBalancingSpi.java b/modules/core/src/main/java/org/apache/ignite/spi/loadbalancing/LoadBalancingSpi.java
new file mode 100644
index 0000000..e3713c5
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/spi/loadbalancing/LoadBalancingSpi.java
@@ -0,0 +1,114 @@
+/* @java.file.header */
+
+/*  _________        _____ __________________        _____
+ *  __  ____/___________(_)______  /__  ____/______ ____(_)_______
+ *  _  / __  __  ___/__  / _  __  / _  / __  _  __ `/__  / __  __ \
+ *  / /_/ /  _  /    _  /  / /_/ /  / /_/ /  / /_/ / _  /  _  / / /
+ *  \____/   /_/     /_/   \_,__/   \____/   \__,_/  /_/   /_/ /_/
+ */
+
+package org.apache.ignite.spi.loadbalancing;
+
+import org.apache.ignite.cluster.*;
+import org.apache.ignite.compute.*;
+import org.apache.ignite.spi.*;
+import org.gridgain.grid.*;
+
+import java.util.*;
+
+/**
+ * Load balancing SPI provides the next best balanced node for job
+ * execution. This SPI is used either implicitly or explicitly whenever
+ * a job gets mapped to a node during {@link org.apache.ignite.compute.ComputeTask#map(List, Object)}
+ * invocation.
+ * <h1 class="header">Coding Examples</h1>
+ * If you are using {@link org.apache.ignite.compute.ComputeTaskSplitAdapter} then load balancing logic
+ * is transparent to your code and is handled automatically by the adapter.
+ * Here is an example of how your task could look:
+ * <pre name="code" class="java">
+ * public class MyFooBarTask extends GridComputeTaskSplitAdapter&lt;Object,Object&gt; {
+ *    &#64;Override
+ *    protected Collection&lt;? extends GridComputeJob&gt; split(int gridSize, Object arg) throws GridException {
+ *        List&lt;MyFooBarJob&gt; jobs = new ArrayList&lt;MyFooBarJob&gt;(gridSize);
+ *
+ *        for (int i = 0; i &lt; gridSize; i++) {
+ *            jobs.add(new MyFooBarJob(arg));
+ *        }
+ *
+ *        // Node assignment via load balancer
+ *        // happens automatically.
+ *        return jobs;
+ *    }
+ *    ...
+ * }
+ * </pre>
+ * If you need more fine-grained control over how some jobs within task get mapped to a node
+ * <i>and</i> use, for example, affinity load balancing for some other jobs within task, then you should use
+ * {@link org.apache.ignite.compute.ComputeTaskAdapter}. Here is an example of how your task could look. Note that in this
+ * case we manually inject load balancer and use it to pick the best node. Doing it in
+ * such way would allow user to map some jobs manually and for others use load balancer.
+ * <pre name="code" class="java">
+ * public class MyFooBarTask extends GridComputeTaskAdapter&lt;String,String&gt; {
+ *    // Inject load balancer.
+ *    &#64;GridLoadBalancerResource
+ *    GridComputeLoadBalancer balancer;
+ *
+ *    // Map jobs to grid nodes.
+ *    public Map&lt;? extends GridComputeJob, GridNode&gt; map(List&lt;GridNode&gt; subgrid, String arg) throws GridException {
+ *        Map&lt;MyFooBarJob, GridNode&gt; jobs = new HashMap&lt;MyFooBarJob, GridNode&gt;(subgrid.size());
+ *
+ *        // In more complex cases, you can actually do
+ *        // more complicated assignments of jobs to nodes.
+ *        for (int i = 0; i &lt; subgrid.size(); i++) {
+ *            // Pick the next best balanced node for the job.
+ *            GridComputeJob myJob = new MyFooBarJob(arg);
+ *
+ *            jobs.put(myJob, balancer.getBalancedNode(myJob, null));
+ *        }
+ *
+ *        return jobs;
+ *    }
+ *
+ *    // Aggregate results into one compound result.
+ *    public String reduce(List&lt;GridComputeJobResult&gt; results) throws GridException {
+ *        // For the purpose of this example we simply
+ *        // concatenate string representation of every
+ *        // job result
+ *        StringBuilder buf = new StringBuilder();
+ *
+ *        for (GridComputeJobResult res : results) {
+ *            // Append string representation of result
+ *            // returned by every job.
+ *            buf.append(res.getData().toString());
+ *        }
+ *
+ *        return buf.toString();
+ *    }
+ * }
+ * </pre>
+ * <p>
+ * GridGain comes with the following load balancing SPI implementations out of the box:
+ * <ul>
+ * <li>{@link org.apache.ignite.spi.loadbalancing.roundrobin.RoundRobinLoadBalancingSpi} - default</li>
+ * <li>{@link org.apache.ignite.spi.loadbalancing.adaptive.AdaptiveLoadBalancingSpi}</li>
+ * <li>{@link org.apache.ignite.spi.loadbalancing.weightedrandom.WeightedRandomLoadBalancingSpi}</li>
+ * </ul>
+ * <b>NOTE:</b> this SPI (i.e. methods in this interface) should never be used directly. SPIs provide
+ * internal view on the subsystem and is used internally by GridGain kernal. In rare use cases when
+ * access to a specific implementation of this SPI is required - an instance of this SPI can be obtained
+ * via {@link org.apache.ignite.Ignite#configuration()} method to check its configuration properties or call other non-SPI
+ * methods. Note again that calling methods from this interface on the obtained instance can lead
+ * to undefined behavior and explicitly not supported.
+ */
+public interface LoadBalancingSpi extends IgniteSpi {
+    /**
+     * Gets balanced node for specified job within given task session.
+     *
+     * @param ses Grid task session for currently executing task.
+     * @param top Topology of task nodes from which to pick the best balanced node for given job.
+     * @param job Job for which to pick the best balanced node.
+     * @throws GridException If failed to get next balanced node.
+     * @return Best balanced node for the given job within given task session.
+     */
+    public ClusterNode getBalancedNode(ComputeTaskSession ses, List<ClusterNode> top, ComputeJob job) throws GridException;
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/b69a23cd/modules/core/src/main/java/org/apache/ignite/spi/loadbalancing/adaptive/AdaptiveCpuLoadProbe.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/loadbalancing/adaptive/AdaptiveCpuLoadProbe.java b/modules/core/src/main/java/org/apache/ignite/spi/loadbalancing/adaptive/AdaptiveCpuLoadProbe.java
new file mode 100644
index 0000000..9db2d84
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/spi/loadbalancing/adaptive/AdaptiveCpuLoadProbe.java
@@ -0,0 +1,229 @@
+/* @java.file.header */
+
+/*  _________        _____ __________________        _____
+ *  __  ____/___________(_)______  /__  ____/______ ____(_)_______
+ *  _  / __  __  ___/__  / _  __  / _  / __  _  __ `/__  / __  __ \
+ *  / /_/ /  _  /    _  /  / /_/ /  / /_/ /  / /_/ / _  /  _  / / /
+ *  \____/   /_/     /_/   \_,__/   \____/   \__,_/  /_/   /_/ /_/
+ */
+
+package org.apache.ignite.spi.loadbalancing.adaptive;
+
+import org.apache.ignite.cluster.*;
+import org.gridgain.grid.util.typedef.internal.*;
+
+/**
+ * Implementation of node load probing based on CPU load.
+ * <p>
+ * Based on {@link #setUseAverage(boolean)}
+ * parameter, this implementation will either use average CPU load
+ * values or current (default is to use averages).
+ * <p>
+ * Based on {@link #setUseProcessors(boolean)} parameter, this implementation
+ * will either take number of processors on the node into account or not.
+ * Since CPU load on multi-processor boxes shows medium load of multiple CPU's it
+ * usually means that the remaining capacity is proportional to the number of
+ * CPU's (or cores) on the node. This configuration parameter indicates
+ * whether to divide each node's CPU load by the number of processors on that node
+ * (default is {@code true}).
+ * <p>
+ * Also note that in some environments every processor may not be adding 100% of
+ * processing power. For example, if you are using multi-core CPU's, then addition of
+ * every core would probably result in about 75% of extra CPU power. To account
+ * for that, you should set {@link #setProcessorCoefficient(double)} parameter to
+ * {@code 0.75} .
+ * <p>
+ * Below is an example of how CPU load probe would be configured in GridGain
+ * Spring configuration file:
+ * <pre name="code" class="xml">
+ * &lt;property name="loadBalancingSpi"&gt;
+ *     &lt;bean class="org.gridgain.grid.spi.loadBalancing.adaptive.GridAdaptiveLoadBalancingSpi"&gt;
+ *         &lt;property name="loadProbe"&gt;
+ *             &lt;bean class="org.gridgain.grid.spi.loadBalancing.adaptive.GridAdaptiveCpuLoadProbe"&gt;
+ *                 &lt;property name="useAverage" value="true"/&gt;
+ *                 &lt;property name="useProcessors" value="true"/&gt;
+ *                 &lt;property name="processorCoefficient" value="0.9"/&gt;
+ *             &lt;/bean&gt;
+ *         &lt;/property&gt;
+ *     &lt;/bean&gt;
+ * &lt;/property&gt;
+ * </pre>
+ * <p>
+ * This implementation is used by default by {@link AdaptiveLoadBalancingSpi} SPI.
+ */
+public class AdaptiveCpuLoadProbe implements AdaptiveLoadProbe {
+    /** Flag indicating whether to use average CPU load vs. current. */
+    private boolean useAvg = true;
+
+    /**
+     * Flag indicating whether to divide each node's CPU load
+     * by the number of processors on that node.
+     */
+    private boolean useProcs = true;
+
+    /**
+     * Coefficient of every CPU processor. By default it is {@code 1}, but
+     * in some environments every processor may not be adding 100% of processing
+     * power. For example, if you are using multi-core CPU's, then addition of
+     * every core would probably result in about 75% of extra CPU power, and hence
+     * you would set this coefficient to {@code 0.75} .
+     */
+    private double procCoefficient = 1;
+
+    /**
+     * Initializes CPU load probe to use CPU load average by default.
+     */
+    public AdaptiveCpuLoadProbe() {
+        // No-op.
+    }
+
+    /**
+     * Specifies whether to use average CPU load vs. current and whether or
+     * not to take number of processors into account.
+     * <p>
+     * Since CPU load on multi-processor boxes shows medium load of multiple CPU's it
+     * usually means that the remaining capacity is proportional to the number of
+     * CPU's (or cores) on the node.
+     *
+     * @param useAvg Flag indicating whether to use average CPU load vs. current
+     *      (default is {@code true}).
+     * @param useProcs Flag indicating whether to divide each node's CPU load
+     *      by the number of processors on that node (default is {@code true}).
+     */
+    public AdaptiveCpuLoadProbe(boolean useAvg, boolean useProcs) {
+        this.useAvg = useAvg;
+        this.useProcs = useProcs;
+    }
+
+    /**
+     * Specifies whether to use average CPU load vs. current and whether or
+     * not to take number of processors into account. It also allows to
+     * specify the coefficient of addition power every CPU adds.
+     * <p>
+     * Since CPU load on multi-processor boxes shows medium load of multiple CPU's it
+     * usually means that the remaining capacity is proportional to the number of
+     * CPU's (or cores) on the node.
+     * <p>
+     * Also, in some environments every processor may not be adding 100% of processing
+     * power. For example, if you are using multi-core CPU's, then addition of
+     * every core would probably result in about 75% of extra CPU power, and hence
+     * you would set this coefficient to {@code 0.75} .
+     *
+     * @param useAvg Flag indicating whether to use average CPU load vs. current
+     *      (default is {@code true}).
+     * @param useProcs Flag indicating whether to divide each node's CPU load
+     *      by the number of processors on that node (default is {@code true}).
+     * @param procCoefficient Coefficient of every CPU processor (default value is {@code 1}).
+     */
+    public AdaptiveCpuLoadProbe(boolean useAvg, boolean useProcs, double procCoefficient) {
+        this.useAvg = useAvg;
+        this.useProcs = useProcs;
+        this.procCoefficient = procCoefficient;
+    }
+
+    /**
+     * Gets flag indicating whether to use average CPU load vs. current.
+     *
+     * @return Flag indicating whether to use average CPU load vs. current.
+     */
+    public boolean isUseAverage() {
+        return useAvg;
+    }
+
+    /**
+     * Sets flag indicating whether to use average CPU load vs. current.
+     * If not explicitly set, then default value is {@code true}.
+     *
+     * @param useAvg Flag indicating whether to use average CPU load vs. current.
+     */
+    public void setUseAverage(boolean useAvg) {
+        this.useAvg = useAvg;
+    }
+
+    /**
+     * Gets flag indicating whether to use average CPU load vs. current
+     * (default is {@code true}).
+     * <p>
+     * Since CPU load on multi-processor boxes shows medium load of multiple CPU's it
+     * usually means that the remaining capacity is proportional to the number of
+     * CPU's (or cores) on the node.
+     *
+     * @return Flag indicating whether to divide each node's CPU load
+     *      by the number of processors on that node (default is {@code true}).
+     */
+    public boolean isUseProcessors() {
+        return useProcs;
+    }
+
+    /**
+     * Sets flag indicating whether to use average CPU load vs. current
+     * (default is {@code true}).
+     * <p>
+     * Since CPU load on multi-processor boxes shows medium load of multiple CPU's it
+     * usually means that the remaining capacity is proportional to the number of
+     * CPU's (or cores) on the node.
+     * <p>
+     * If not explicitly set, then default value is {@code true}.
+     *
+     * @param useProcs Flag indicating whether to divide each node's CPU load
+     *      by the number of processors on that node (default is {@code true}).
+     */
+    public void setUseProcessors(boolean useProcs) {
+        this.useProcs = useProcs;
+    }
+
+    /**
+     * Gets coefficient of every CPU processor. By default it is {@code 1}, but
+     * in some environments every processor may not be adding 100% of processing
+     * power. For example, if you are using multi-core CPU's, then addition of
+     * every core would probably result in about 75% of extra CPU power, and hence
+     * you would set this coefficient to {@code 0.75} .
+     * <p>
+     * This value is ignored if {@link #isUseProcessors()} is set to {@code false}.
+     *
+     * @return Coefficient of every CPU processor.
+     */
+    public double getProcessorCoefficient() {
+        return procCoefficient;
+    }
+
+    /**
+     * Sets coefficient of every CPU processor. By default it is {@code 1}, but
+     * in some environments every processor may not be adding 100% of processing
+     * power. For example, if you are using multi-core CPU's, then addition of
+     * every core would probably result in about 75% of extra CPU power, and hence
+     * you would set this coefficient to {@code 0.75} .
+     * <p>
+     * This value is ignored if {@link #isUseProcessors()} is set to {@code false}.
+     *
+     * @param procCoefficient Coefficient of every CPU processor.
+     */
+    public void setProcessorCoefficient(double procCoefficient) {
+        A.ensure(procCoefficient > 0, "procCoefficient > 0");
+
+        this.procCoefficient = procCoefficient;
+    }
+
+    /** {@inheritDoc} */
+    @Override public double getLoad(ClusterNode node, int jobsSentSinceLastUpdate) {
+        ClusterNodeMetrics metrics = node.metrics();
+
+        double k = 1.0d;
+
+        if (useProcs) {
+            int procs = metrics.getTotalCpus();
+
+            if (procs > 1)
+                k = procs * procCoefficient;
+        }
+
+        double load = (useAvg ? metrics.getAverageCpuLoad() : metrics.getCurrentCpuLoad()) / k;
+
+        return load < 0 ? 0 : load;
+    }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return S.toString(AdaptiveCpuLoadProbe.class, this);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/b69a23cd/modules/core/src/main/java/org/apache/ignite/spi/loadbalancing/adaptive/AdaptiveJobCountLoadProbe.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/loadbalancing/adaptive/AdaptiveJobCountLoadProbe.java b/modules/core/src/main/java/org/apache/ignite/spi/loadbalancing/adaptive/AdaptiveJobCountLoadProbe.java
new file mode 100644
index 0000000..d5c99b7
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/spi/loadbalancing/adaptive/AdaptiveJobCountLoadProbe.java
@@ -0,0 +1,96 @@
+/* @java.file.header */
+
+/*  _________        _____ __________________        _____
+ *  __  ____/___________(_)______  /__  ____/______ ____(_)_______
+ *  _  / __  __  ___/__  / _  __  / _  / __  _  __ `/__  / __  __ \
+ *  / /_/ /  _  /    _  /  / /_/ /  / /_/ /  / /_/ / _  /  _  / / /
+ *  \____/   /_/     /_/   \_,__/   \____/   \__,_/  /_/   /_/ /_/
+ */
+
+package org.apache.ignite.spi.loadbalancing.adaptive;
+
+import org.apache.ignite.cluster.*;
+import org.gridgain.grid.util.typedef.internal.*;
+
+/**
+ * Implementation of node load probing based on active and waiting job count.
+ * Based on {@link #setUseAverage(boolean)} parameter, this implementation will
+ * either use average job count values or current (default is to use averages).
+ * <p>
+ * The load of a node is simply calculated by adding active and waiting job counts.
+ * <p>
+ * Below is an example of how CPU load probe would be configured in GridGain
+ * Spring configuration file:
+ * <pre name="code" class="xml">
+ * &lt;property name="loadBalancingSpi"&gt;
+ *     &lt;bean class="org.gridgain.grid.spi.loadBalancing.adaptive.GridAdaptiveLoadBalancingSpi"&gt;
+ *         &lt;property name="loadProbe"&gt;
+ *             &lt;bean class="org.gridgain.grid.spi.loadBalancing.adaptive.GridAdaptiveJobCountLoadProbe"&gt;
+ *                 &lt;property name="useAverage" value="true"/&gt;
+ *             &lt;/bean&gt;
+ *         &lt;/property&gt;
+ *     &lt;/bean&gt;
+ * &lt;/property&gt;
+ * </pre>
+ */
+public class AdaptiveJobCountLoadProbe implements AdaptiveLoadProbe {
+    /** Flag indicating whether to use average CPU load vs. current. */
+    private boolean useAvg = true;
+
+    /**
+     * Initializes active job probe.
+     */
+    public AdaptiveJobCountLoadProbe() {
+        // No-op.
+    }
+
+    /**
+     * Creates new active job prove specifying whether to use average
+     * job counts vs. current.
+     *
+     * @param useAvg Flag indicating whether to use average job counts vs. current.
+     */
+    public AdaptiveJobCountLoadProbe(boolean useAvg) {
+        this.useAvg = useAvg;
+    }
+
+    /**
+     * Gets flag indicating whether to use average job counts vs. current.
+     *
+     * @return Flag indicating whether to use average job counts vs. current.
+     */
+    public boolean isUseAverage() {
+        return useAvg;
+    }
+
+    /**
+     * Sets flag indicating whether to use average job counts vs. current.
+     *
+     * @param useAvg Flag indicating whether to use average job counts vs. current.
+     */
+    public void setUseAverage(boolean useAvg) {
+        this.useAvg = useAvg;
+    }
+
+
+    /** {@inheritDoc} */
+    @Override public double getLoad(ClusterNode node, int jobsSentSinceLastUpdate) {
+        ClusterNodeMetrics metrics = node.metrics();
+
+        if (useAvg) {
+            double load = metrics.getAverageActiveJobs() + metrics.getAverageWaitingJobs();
+
+            if (load > 0)
+                return load;
+        }
+
+        double load = metrics.getCurrentActiveJobs() + metrics.getCurrentWaitingJobs();
+
+        return load < 0 ? 0 : load;
+    }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return S.toString(AdaptiveJobCountLoadProbe.class, this);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/b69a23cd/modules/core/src/main/java/org/apache/ignite/spi/loadbalancing/adaptive/AdaptiveLoadBalancingSpi.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/loadbalancing/adaptive/AdaptiveLoadBalancingSpi.java b/modules/core/src/main/java/org/apache/ignite/spi/loadbalancing/adaptive/AdaptiveLoadBalancingSpi.java
new file mode 100644
index 0000000..5bb4501
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/spi/loadbalancing/adaptive/AdaptiveLoadBalancingSpi.java
@@ -0,0 +1,581 @@
+/* @java.file.header */
+
+/*  _________        _____ __________________        _____
+ *  __  ____/___________(_)______  /__  ____/______ ____(_)_______
+ *  _  / __  __  ___/__  / _  __  / _  / __  _  __ `/__  / __  __ \
+ *  / /_/ /  _  /    _  /  / /_/ /  / /_/ /  / /_/ / _  /  _  / / /
+ *  \____/   /_/     /_/   \_,__/   \____/   \__,_/  /_/   /_/ /_/
+ */
+
+package org.apache.ignite.spi.loadbalancing.adaptive;
+
+import org.apache.ignite.*;
+import org.apache.ignite.cluster.*;
+import org.apache.ignite.compute.*;
+import org.apache.ignite.events.*;
+import org.apache.ignite.lang.*;
+import org.apache.ignite.resources.*;
+import org.apache.ignite.spi.*;
+import org.gridgain.grid.*;
+import org.gridgain.grid.kernal.managers.eventstorage.*;
+import org.apache.ignite.spi.loadbalancing.*;
+import org.gridgain.grid.util.typedef.*;
+import org.gridgain.grid.util.typedef.internal.*;
+import org.jdk8.backport.*;
+import org.jetbrains.annotations.*;
+
+import java.util.*;
+import java.util.concurrent.*;
+import java.util.concurrent.atomic.*;
+import java.util.concurrent.locks.*;
+
+import static org.apache.ignite.events.IgniteEventType.*;
+
+/**
+ * Load balancing SPI that adapts to overall node performance. It
+ * proportionally distributes more jobs to more performant nodes based
+ * on a pluggable and dynamic node load probing.
+ * <p>
+ * <h1 class="header">Adaptive Node Probe</h1>
+ * This SPI comes with pluggable algorithm to calculate a node load
+ * at any given point of time. The algorithm is defined by
+ * {@link AdaptiveLoadProbe} interface and user is
+ * free to provide custom implementations. By default
+ * {@link AdaptiveCpuLoadProbe} implementation is used
+ * which distributes jobs to nodes based on average CPU load
+ * on every node.
+ * <p>
+ * The following load probes are available with the product:
+ * <ul>
+ * <li>{@link AdaptiveCpuLoadProbe} - default</li>
+ * <li>{@link AdaptiveProcessingTimeLoadProbe}</li>
+ * <li>{@link AdaptiveJobCountLoadProbe}</li>
+ * </ul>
+ * Note that if {@link AdaptiveLoadProbe#getLoad(org.apache.ignite.cluster.ClusterNode, int)} returns a value of {@code 0},
+ * then implementation will assume that load value is simply not available and
+ * will try to calculate an average of load values for other nodes. If such
+ * average cannot be obtained (all node load values are {@code 0}), then a value
+ * of {@code 1} will be used.
+ * <p>
+ * When working with node metrics, take into account that all averages are
+ * calculated over metrics history size defined by {@link org.apache.ignite.configuration.IgniteConfiguration#getMetricsExpireTime()}
+ * and {@link org.apache.ignite.configuration.IgniteConfiguration#getMetricsHistorySize()} grid configuration parameters.
+ * Generally the larger these configuration parameter values are, the more precise the metrics are.
+ * You should tune these values based on the level of accuracy needed vs. the additional memory
+ * that would be required for storing metrics.
+ * <p>
+ * You should also keep in mind that metrics for remote nodes are delayed (usually by the
+ * heartbeat frequency). So if it is acceptable in your environment, set the heartbeat frequency
+ * to be more inline with job execution time. Generally, the more often heartbeats between nodes
+ * are exchanged, the more precise the metrics are. However, you should keep in mind that if
+ * heartbeats are exchanged too often then it may create unnecessary traffic in the network.
+ * Heartbeats (or metrics update frequency) can be configured via underlying
+ * {@link org.apache.ignite.spi.discovery.DiscoverySpi} used in your grid.
+ * <p>
+ * Here is an example of how probing can be implemented to use
+ * number of active and waiting jobs as probing mechanism:
+ * <pre name="code" class="java">
+ * public class FooBarLoadProbe implements GridAdaptiveLoadProbe {
+ *     // Flag indicating whether to use average value or current.
+ *     private int useAvg = true;
+ *
+ *     public FooBarLoadProbe(boolean useAvg) {
+ *         this.useAvg = useAvg;
+ *     }
+ *
+ *     // Calculate load based on number of active and waiting jobs.
+ *     public double getLoad(GridNode node, int jobsSentSinceLastUpdate) {
+ *         GridNodeMetrics metrics = node.getMetrics();
+ *
+ *         if (useAvg) {
+ *             double load = metrics.getAverageActiveJobs() + metrics.getAverageWaitingJobs();
+ *
+ *             if (load > 0) {
+ *                 return load;
+ *             }
+ *         }
+ *
+ *         return metrics.getCurrentActiveJobs() + metrics.getCurrentWaitingJobs();
+ *     }
+ * }
+ * </pre>
+ * <h1 class="header">Which Node Probe To Use</h1>
+ * There is no correct answer here. Every single node probe will work better or worse in
+ * different environments. CPU load probe (default option) is the safest approach to start
+ * with as it simply attempts to utilize every CPU on the grid to the maximum. However, you should
+ * experiment with other probes by executing load tests in your environment and observing
+ * which probe gives you best performance and load balancing.
+ * <p>
+ * <h1 class="header">Task Coding Example</h1>
+ * If you are using {@link org.apache.ignite.compute.ComputeTaskSplitAdapter} then load balancing logic
+ * is transparent to your code and is handled automatically by the adapter.
+ * Here is an example of how your task will look:
+ * <pre name="code" class="java">
+ * public class MyFooBarTask extends GridComputeTaskSplitAdapter&lt;Object, Object&gt; {
+ *    &#64;Override
+ *    protected Collection&lt;? extends GridComputeJob&gt; split(int gridSize, Object arg) throws GridException {
+ *        List&lt;MyFooBarJob&gt; jobs = new ArrayList&lt;MyFooBarJob&gt;(gridSize);
+ *
+ *        for (int i = 0; i &lt; gridSize; i++) {
+ *            jobs.add(new MyFooBarJob(arg));
+ *        }
+ *
+ *        // Node assignment via load balancer
+ *        // happens automatically.
+ *        return jobs;
+ *    }
+ *    ...
+ * }
+ * </pre>
+ * If you need more fine-grained control over how some jobs within task get mapped to a node
+ * and use affinity load balancing for some other jobs within task, then you should use
+ * {@link org.apache.ignite.compute.ComputeTaskAdapter}. Here is an example of how your task will look. Note that in this
+ * case we manually inject load balancer and use it to pick the best node. Doing it in
+ * such way would allow user to map some jobs manually and for others use load balancer.
+ * <pre name="code" class="java">
+ * public class MyFooBarTask extends GridComputeTaskAdapter&lt;String, String&gt; {
+ *    // Inject load balancer.
+ *    &#64;GridLoadBalancerResource
+ *    GridComputeLoadBalancer balancer;
+ *
+ *    // Map jobs to grid nodes.
+ *    public Map&lt;? extends GridComputeJob, GridNode&gt; map(List&lt;GridNode&gt; subgrid, String arg) throws GridException {
+ *        Map&lt;MyFooBarJob, GridNode&gt; jobs = new HashMap&lt;MyFooBarJob, GridNode&gt;(subgrid.size());
+ *
+ *        // In more complex cases, you can actually do
+ *        // more complicated assignments of jobs to nodes.
+ *        for (int i = 0; i &lt; subgrid.size(); i++) {
+ *            // Pick the next best balanced node for the job.
+ *            jobs.put(new MyFooBarJob(arg), balancer.getBalancedNode())
+ *        }
+ *
+ *        return jobs;
+ *    }
+ *
+ *    // Aggregate results into one compound result.
+ *    public String reduce(List&lt;GridComputeJobResult&gt; results) throws GridException {
+ *        // For the purpose of this example we simply
+ *        // concatenate string representation of every
+ *        // job result
+ *        StringBuilder buf = new StringBuilder();
+ *
+ *        for (GridComputeJobResult res : results) {
+ *            // Append string representation of result
+ *            // returned by every job.
+ *            buf.append(res.getData().string());
+ *        }
+ *
+ *        return buf.string();
+ *    }
+ * }
+ * </pre>
+ * <p>
+ * <h1 class="header">Configuration</h1>
+ * In order to use this load balancer, you should configure your grid instance
+ * to use {@code GridJobsLoadBalancingSpi} either from Spring XML file or
+ * directly. The following configuration parameters are supported:
+ * <h2 class="header">Mandatory</h2>
+ * This SPI has no mandatory configuration parameters.
+ * <h2 class="header">Optional</h2>
+ * This SPI has the following optional configuration parameters:
+ * <ul>
+ * <li>
+ *      Adaptive node load probing implementation (see {@link #setLoadProbe(AdaptiveLoadProbe)}).
+ *      This configuration parameter supplies a custom algorithm for probing a node's load.
+ *      By default, {@link AdaptiveCpuLoadProbe} implementation is used which
+ *      takes every node's CPU load and tries to send proportionally more jobs to less loaded nodes.
+ * </li>
+ * </ul>
+ * <p>
+ * Below is Java configuration example:
+ * <pre name="code" class="java">
+ * GridAdaptiveLoadBalancingSpi spi = new GridAdaptiveLoadBalancingSpi();
+ *
+ * // Configure probe to use latest job execution time vs. average.
+ * GridAdaptiveProcessingTimeLoadProbe probe = new GridAdaptiveProcessingTimeLoadProbe(false);
+ *
+ * spi.setLoadProbe(probe);
+ *
+ * GridConfiguration cfg = new GridConfiguration();
+ *
+ * // Override default load balancing SPI.
+ * cfg.setLoadBalancingSpi(spi);
+ *
+ * // Starts grid.
+ * G.start(cfg);
+ * </pre>
+ * Here is how you can configure {@code GridJobsLoadBalancingSpi} using Spring XML configuration:
+ * <pre name="code" class="xml">
+ * &lt;property name="loadBalancingSpi"&gt;
+ *     &lt;bean class="org.gridgain.grid.spi.loadBalancing.adaptive.GridAdaptiveLoadBalancingSpi"&gt;
+ *         &lt;property name="loadProbe"&gt;
+ *             &lt;bean class="org.gridgain.grid.spi.loadBalancing.adaptive.GridAdaptiveProcessingTimeLoadProbe"&gt;
+ *                 &lt;constructor-arg value="false"/&gt;
+ *             &lt;/bean&gt;
+ *         &lt;/property&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>
+ */
+@IgniteSpiMultipleInstancesSupport(true)
+public class AdaptiveLoadBalancingSpi extends IgniteSpiAdapter implements LoadBalancingSpi,
+    AdaptiveLoadBalancingSpiMBean {
+    /** Random number generator. */
+    private static final Random RAND = new Random();
+
+    /** Grid logger. */
+    @IgniteLoggerResource
+    private IgniteLogger log;
+
+    /** */
+    private AdaptiveLoadProbe probe = new AdaptiveCpuLoadProbe();
+
+    /** Local event listener to listen to task completion events. */
+    private GridLocalEventListener evtLsnr;
+
+    /** Task topologies. First pair value indicates whether or not jobs have been mapped. */
+    private ConcurrentMap<IgniteUuid, IgniteBiTuple<Boolean, WeightedTopology>> taskTops =
+        new ConcurrentHashMap8<>();
+
+    /** */
+    private final Map<UUID, AtomicInteger> nodeJobs = new HashMap<>();
+
+    /** */
+    private final ReadWriteLock rwLock = new ReentrantReadWriteLock();
+
+    /** {@inheritDoc} */
+    @Override public String getLoadProbeFormatted() {
+        return probe.toString();
+    }
+
+    /**
+     * Sets implementation of node load probe. By default {@link AdaptiveProcessingTimeLoadProbe}
+     * is used which proportionally distributes load based on the average job execution
+     * time on every node.
+     *
+     * @param probe Implementation of node load probe
+     */
+    @IgniteSpiConfiguration(optional = true)
+    public void setLoadProbe(AdaptiveLoadProbe probe) {
+        A.ensure(probe != null, "probe != null");
+
+        this.probe = probe;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void spiStart(@Nullable String gridName) throws IgniteSpiException {
+        startStopwatch();
+
+        assertParameter(probe != null, "loadProbe != null");
+
+        if (log.isDebugEnabled())
+            log.debug(configInfo("loadProbe", probe));
+
+        registerMBean(gridName, this, AdaptiveLoadBalancingSpiMBean.class);
+
+        // Ack ok start.
+        if (log.isDebugEnabled())
+            log.debug(startInfo());
+    }
+
+    /** {@inheritDoc} */
+    @Override public void spiStop() throws IgniteSpiException {
+        rwLock.writeLock().lock();
+
+        try {
+            nodeJobs.clear();
+        }
+        finally {
+            rwLock.writeLock().unlock();
+        }
+
+        unregisterMBean();
+
+        // Ack ok stop.
+        if (log.isDebugEnabled())
+            log.debug(stopInfo());
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void onContextInitialized0(IgniteSpiContext spiCtx) throws IgniteSpiException {
+        getSpiContext().addLocalEventListener(evtLsnr = new GridLocalEventListener() {
+            @Override public void onEvent(IgniteEvent evt) {
+                switch (evt.type()) {
+                    case EVT_TASK_FINISHED:
+                    case EVT_TASK_FAILED: {
+                        IgniteTaskEvent taskEvt = (IgniteTaskEvent)evt;
+
+                        taskTops.remove(taskEvt.taskSessionId());
+
+                        if (log.isDebugEnabled())
+                            log.debug("Removed task topology from topology cache for session: " +
+                                taskEvt.taskSessionId());
+
+                        break;
+                    }
+
+                    case EVT_JOB_MAPPED: {
+                        // We should keep topology and use cache in GridComputeTask#map() method to
+                        // avoid O(n*n/2) complexity, after that we can drop caches.
+                        // Here we set mapped property and later cache will be ignored
+                        IgniteJobEvent jobEvt = (IgniteJobEvent)evt;
+
+                        IgniteBiTuple<Boolean, WeightedTopology> weightedTop = taskTops.get(jobEvt.taskSessionId());
+
+                        if (weightedTop != null)
+                            weightedTop.set1(true);
+
+                        if (log.isDebugEnabled())
+                            log.debug("Job has been mapped. Ignore cache for session: " + jobEvt.taskSessionId());
+
+                        break;
+                    }
+
+                    case EVT_NODE_METRICS_UPDATED:
+                    case EVT_NODE_FAILED:
+                    case EVT_NODE_JOINED:
+                    case EVT_NODE_LEFT: {
+                        IgniteDiscoveryEvent discoEvt = (IgniteDiscoveryEvent)evt;
+
+                        rwLock.writeLock().lock();
+
+                        try {
+                            switch (evt.type()) {
+                                case EVT_NODE_JOINED: {
+                                    nodeJobs.put(discoEvt.eventNode().id(), new AtomicInteger(0));
+
+                                    break;
+                                }
+
+                                case EVT_NODE_LEFT:
+                                case EVT_NODE_FAILED: {
+                                    nodeJobs.remove(discoEvt.eventNode().id());
+
+                                    break;
+                                }
+
+                                case EVT_NODE_METRICS_UPDATED: {
+                                    // Reset counter.
+                                    nodeJobs.put(discoEvt.eventNode().id(), new AtomicInteger(0));
+
+                                    break;
+                                }
+                            }
+                        }
+                        finally {
+                            rwLock.writeLock().unlock();
+                        }
+                    }
+
+                }
+            }
+        },
+            EVT_NODE_METRICS_UPDATED,
+            EVT_NODE_FAILED,
+            EVT_NODE_JOINED,
+            EVT_NODE_LEFT,
+            EVT_TASK_FINISHED,
+            EVT_TASK_FAILED,
+            EVT_JOB_MAPPED
+        );
+
+        // Put all known nodes.
+        rwLock.writeLock().lock();
+
+        try {
+            for (ClusterNode node : getSpiContext().nodes())
+                nodeJobs.put(node.id(), new AtomicInteger(0));
+        }
+        finally {
+            rwLock.writeLock().unlock();
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void onContextDestroyed0() {
+        if (evtLsnr != null) {
+            IgniteSpiContext ctx = getSpiContext();
+
+            if (ctx != null)
+                ctx.removeLocalEventListener(evtLsnr);
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public ClusterNode getBalancedNode(ComputeTaskSession ses, List<ClusterNode> top, ComputeJob job)
+    throws GridException {
+        A.notNull(ses, "ses");
+        A.notNull(top, "top");
+        A.notNull(job, "job");
+
+        IgniteBiTuple<Boolean, WeightedTopology> weightedTop = taskTops.get(ses.getId());
+
+        // Create new cached topology if there is no one. Do not
+        // use cached topology after task has been mapped.
+        if (weightedTop == null)
+            // Called from GridComputeTask#map(). Put new topology and false as not mapped yet.
+            taskTops.put(ses.getId(), weightedTop = F.t(false, new WeightedTopology(top)));
+        // We have topology - check if task has been mapped.
+        else if (weightedTop.get1())
+            // Do not use cache after GridComputeTask#map().
+            return new WeightedTopology(top).pickWeightedNode();
+
+        return weightedTop.get2().pickWeightedNode();
+    }
+
+    /**
+     * Calculates node load based on set probe.
+     *
+     * @param top List of all nodes.
+     * @param node Node to get load for.
+     * @return Node load.
+     * @throws GridException If returned load is negative.
+     */
+    @SuppressWarnings({"TooBroadScope"})
+    private double getLoad(Collection<ClusterNode> top, ClusterNode node) throws GridException {
+        assert !F.isEmpty(top);
+
+        int jobsSentSinceLastUpdate = 0;
+
+        rwLock.readLock().lock();
+
+        try {
+            AtomicInteger cnt = nodeJobs.get(node.id());
+
+            jobsSentSinceLastUpdate = cnt == null ? 0 : cnt.get();
+        }
+        finally {
+            rwLock.readLock().unlock();
+        }
+
+        double load = probe.getLoad(node, jobsSentSinceLastUpdate);
+
+        if (load < 0)
+            throw new GridException("Failed to obtain non-negative load from adaptive load probe: " + load);
+
+        return load;
+    }
+
+    /**
+     * Holder for weighted topology.
+     */
+    private class WeightedTopology {
+        /** Topology sorted by weight. */
+        private final SortedMap<Double, ClusterNode> circle = new TreeMap<>();
+
+        /**
+         * @param top Task topology.
+         * @throws GridException If any load was negative.
+         */
+        WeightedTopology(List<ClusterNode> top) throws GridException {
+            assert !F.isEmpty(top);
+
+            double totalLoad = 0;
+
+            // We need to cache loads here to avoid calls later as load might be
+            // changed between the calls.
+            double[] nums = new double[top.size()];
+
+            int zeroCnt = 0;
+
+            // Compute loads.
+            for (int i = 0; i < top.size(); i++) {
+                double load = getLoad(top, top.get(i));
+
+                nums[i] = load;
+
+                if (load == 0)
+                    zeroCnt++;
+
+                totalLoad += load;
+            }
+
+            // Take care of zero loads.
+            if (zeroCnt > 0) {
+                double newTotal = totalLoad;
+
+                int nonZeroCnt = top.size() - zeroCnt;
+
+                for (int i = 0; i < nums.length; i++) {
+                    double load = nums[i];
+
+                    if (load == 0) {
+                        if (nonZeroCnt > 0)
+                            load = totalLoad / nonZeroCnt;
+
+                        if (load == 0)
+                            load = 1;
+
+                        nums[i] = load;
+
+                        newTotal += load;
+                    }
+                }
+
+                totalLoad = newTotal;
+            }
+
+            double totalWeight = 0;
+
+            // Calculate weights and total weight.
+            for (int i = 0; i < nums.length; i++) {
+                assert nums[i] > 0 : "Invalid load: " + nums[i];
+
+                double weight = totalLoad / nums[i];
+
+                // Convert to weight.
+                nums[i] = weight;
+
+                totalWeight += weight;
+            }
+
+            double weight = 0;
+
+            // Enforce range from 0 to 1.
+            for (int i = 0; i < nums.length; i++) {
+                weight = i == nums.length - 1 ? 1.0d : weight + nums[i] / totalWeight;
+
+                assert weight < 2 : "Invalid weight: " + weight;
+
+                // Complexity of this put is O(logN).
+                circle.put(weight, top.get(i));
+            }
+        }
+
+        /**
+         * Gets weighted node in random fashion.
+         *
+         * @return Weighted node.
+         */
+        ClusterNode pickWeightedNode() {
+            double weight = RAND.nextDouble();
+
+            SortedMap<Double, ClusterNode> pick = circle.tailMap(weight);
+
+            ClusterNode node = pick.get(pick.firstKey());
+
+            rwLock.readLock().lock();
+
+            try {
+                AtomicInteger cnt = nodeJobs.get(node.id());
+
+                if (cnt != null)
+                    cnt.incrementAndGet();
+            }
+            finally {
+                rwLock.readLock().unlock();
+            }
+
+            return node;
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return S.toString(AdaptiveLoadBalancingSpi.class, this);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/b69a23cd/modules/core/src/main/java/org/apache/ignite/spi/loadbalancing/adaptive/AdaptiveLoadBalancingSpiMBean.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/loadbalancing/adaptive/AdaptiveLoadBalancingSpiMBean.java b/modules/core/src/main/java/org/apache/ignite/spi/loadbalancing/adaptive/AdaptiveLoadBalancingSpiMBean.java
new file mode 100644
index 0000000..14b4ed8
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/spi/loadbalancing/adaptive/AdaptiveLoadBalancingSpiMBean.java
@@ -0,0 +1,27 @@
+/* @java.file.header */
+
+/*  _________        _____ __________________        _____
+ *  __  ____/___________(_)______  /__  ____/______ ____(_)_______
+ *  _  / __  __  ___/__  / _  __  / _  / __  _  __ `/__  / __  __ \
+ *  / /_/ /  _  /    _  /  / /_/ /  / /_/ /  / /_/ / _  /  _  / / /
+ *  \____/   /_/     /_/   \_,__/   \____/   \__,_/  /_/   /_/ /_/
+ */
+
+package org.apache.ignite.spi.loadbalancing.adaptive;
+
+import org.apache.ignite.mbean.*;
+import org.apache.ignite.spi.*;
+
+/**
+ * Management MBean for {@link AdaptiveLoadBalancingSpi} SPI.
+ */
+@IgniteMBeanDescription("MBean that provides access to adaptive load balancing SPI configuration.")
+public interface AdaptiveLoadBalancingSpiMBean extends IgniteSpiManagementMBean {
+    /**
+     * Gets text description of current load probing implementation used.
+     *
+     * @return Text description of current load probing implementation used.
+     */
+    @IgniteMBeanDescription("Text description of current load probing implementation used.")
+    public String getLoadProbeFormatted();
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/b69a23cd/modules/core/src/main/java/org/apache/ignite/spi/loadbalancing/adaptive/AdaptiveLoadProbe.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/loadbalancing/adaptive/AdaptiveLoadProbe.java b/modules/core/src/main/java/org/apache/ignite/spi/loadbalancing/adaptive/AdaptiveLoadProbe.java
new file mode 100644
index 0000000..8027281
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/spi/loadbalancing/adaptive/AdaptiveLoadProbe.java
@@ -0,0 +1,90 @@
+/* @java.file.header */
+
+/*  _________        _____ __________________        _____
+ *  __  ____/___________(_)______  /__  ____/______ ____(_)_______
+ *  _  / __  __  ___/__  / _  __  / _  / __  _  __ `/__  / __  __ \
+ *  / /_/ /  _  /    _  /  / /_/ /  / /_/ /  / /_/ / _  /  _  / / /
+ *  \____/   /_/     /_/   \_,__/   \____/   \__,_/  /_/   /_/ /_/
+ */
+
+package org.apache.ignite.spi.loadbalancing.adaptive;
+
+import org.apache.ignite.cluster.*;
+
+/**
+ * Pluggable implementation of node load probing. Implementations
+ * of this can be configured to be used with {@link AdaptiveLoadBalancingSpi}
+ * by setting {@link AdaptiveLoadBalancingSpi#setLoadProbe(AdaptiveLoadProbe)}
+ * configuration parameter.
+ * <p>
+ * Note that if {@link #getLoad(org.apache.ignite.cluster.ClusterNode, int)} returns a value of {@code 0},
+ * then implementation will assume that load value is simply not available and
+ * will try to calculate an average of load values for other nodes. If such
+ * average cannot be obtained (all node load values are {@code 0}), then a value
+ * of {@code 1} will be used.
+ * <p>
+ * By default, {@link AdaptiveCpuLoadProbe} probing implementation is used.
+ * <p>
+ * <h1 class="header">Example</h1>
+ * Here is an example of how probing can be implemented to use
+ * number of active and waiting jobs as probing mechanism:
+ * <pre name="code" class="java">
+ * public class FooBarLoadProbe implements GridAdaptiveLoadProbe {
+ *     // Flag indicating whether to use average value or current.
+ *     private int useAvg = true;
+ *
+ *     public FooBarLoadProbe(boolean useAvg) {
+ *         this.useAvg = useAvg;
+ *     }
+ *
+ *     // Calculate load based on number of active and waiting jobs.
+ *     public double getLoad(GridNode node, int jobsSentSinceLastUpdate) {
+ *         GridNodeMetrics metrics = node.getMetrics();
+ *
+ *         if (useAvg) {
+ *             double load = metrics.getAverageActiveJobs() + metrics.getAverageWaitingJobs();
+ *
+ *             if (load > 0) {
+ *                 return load;
+ *             }
+ *         }
+ *
+ *         return metrics.getCurrentActiveJobs() + metrics.getCurrentWaitingJobs();
+ *     }
+ * }
+ * </pre>
+ * Below is an example of how a probe shown above would be configured with {@link AdaptiveLoadBalancingSpi}
+ * SPI:
+ * <pre name="code" class="xml">
+ * &lt;property name="loadBalancingSpi"&gt;
+ *     &lt;bean class="org.gridgain.grid.spi.loadBalancing.adaptive.GridAdaptiveLoadBalancingSpi"&gt;
+ *         &lt;property name="loadProbe"&gt;
+ *             &lt;bean class="foo.bar.FooBarLoadProbe"&gt;
+ *                 &lt;constructor-arg value="true"/&gt;
+ *             &lt;/bean&gt;
+ *         &lt;/property&gt;
+ *     &lt;/bean&gt;
+ * &lt;/property&gt;
+ * </pre>
+ */
+public interface AdaptiveLoadProbe {
+    /**
+     * Calculates load value for a given node. Specific implementations would
+     * usually take into account some of the values provided by
+     * {@link org.apache.ignite.cluster.ClusterNode#metrics()} method. For example, load can be calculated
+     * based on job execution time or number of active jobs, or CPU/Heap utilization.
+     * <p>
+     * Note that if this method returns a value of {@code 0},
+     * then implementation will assume that load value is simply not available and
+     * will try to calculate an average of load values for other nodes. If such
+     * average cannot be obtained (all node load values are {@code 0}), then a value
+     * of {@code 1} will be used.
+     *
+     * @param node Grid node to calculate load for.
+     * @param jobsSentSinceLastUpdate Number of jobs sent to this node since
+     *      last metrics update. This parameter may be useful when
+     *      implementation takes into account the current job count on a node.
+     * @return Non-negative load value for the node (zero and above).
+     */
+    public double getLoad(ClusterNode node, int jobsSentSinceLastUpdate);
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/b69a23cd/modules/core/src/main/java/org/apache/ignite/spi/loadbalancing/adaptive/AdaptiveProcessingTimeLoadProbe.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/loadbalancing/adaptive/AdaptiveProcessingTimeLoadProbe.java b/modules/core/src/main/java/org/apache/ignite/spi/loadbalancing/adaptive/AdaptiveProcessingTimeLoadProbe.java
new file mode 100644
index 0000000..30474ad
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/spi/loadbalancing/adaptive/AdaptiveProcessingTimeLoadProbe.java
@@ -0,0 +1,98 @@
+/* @java.file.header */
+
+/*  _________        _____ __________________        _____
+ *  __  ____/___________(_)______  /__  ____/______ ____(_)_______
+ *  _  / __  __  ___/__  / _  __  / _  / __  _  __ `/__  / __  __ \
+ *  / /_/ /  _  /    _  /  / /_/ /  / /_/ /  / /_/ / _  /  _  / / /
+ *  \____/   /_/     /_/   \_,__/   \____/   \__,_/  /_/   /_/ /_/
+ */
+
+package org.apache.ignite.spi.loadbalancing.adaptive;
+
+import org.apache.ignite.cluster.*;
+import org.gridgain.grid.util.typedef.internal.*;
+
+/**
+ * Implementation of node load probing based on total job processing time.
+ * Based on {@link #setUseAverage(boolean)}
+ * parameter, this implementation will either use average job execution
+ * time values or current (default is to use averages). The algorithm
+ * returns a sum of job wait time and job execution time.
+ * <p>
+ * Below is an example of how CPU load probe would be configured in GridGain
+ * Spring configuration file:
+ * <pre name="code" class="xml">
+ * &lt;property name="loadBalancingSpi"&gt;
+ *     &lt;bean class="org.gridgain.grid.spi.loadBalancing.adaptive.GridAdaptiveLoadBalancingSpi"&gt;
+ *         &lt;property name="loadProbe"&gt;
+ *             &lt;bean class="org.gridgain.grid.spi.loadBalancing.adaptive.GridAdaptiveProcessingTimeLoadProbe"&gt;
+ *                 &lt;property name="useAverage" value="true"/&gt;
+ *             &lt;/bean&gt;
+ *         &lt;/property&gt;
+ *     &lt;/bean&gt;
+ * &lt;/property&gt;
+ * </pre>
+ */
+public class AdaptiveProcessingTimeLoadProbe implements AdaptiveLoadProbe {
+    /** Flag indicating whether to use average execution time vs. current. */
+    private boolean useAvg = true;
+
+    /**
+     * Initializes execution time load probe to use
+     * execution time average by default.
+     */
+    public AdaptiveProcessingTimeLoadProbe() {
+        // No-op.
+    }
+
+    /**
+     * Specifies whether to use average execution time vs. current.
+     *
+     * @param useAvg Flag indicating whether to use average execution time vs. current.
+     */
+    public AdaptiveProcessingTimeLoadProbe(boolean useAvg) {
+        this.useAvg = useAvg;
+    }
+
+    /**
+     * Gets flag indicating whether to use average execution time vs. current.
+     *
+     * @return Flag indicating whether to use average execution time vs. current.
+     */
+    public boolean isUseAverage() {
+        return useAvg;
+    }
+
+    /**
+     * Sets flag indicating whether to use average execution time vs. current.
+     *
+     * @param useAvg Flag indicating whether to use average execution time vs. current.
+     */
+    public void setUseAverage(boolean useAvg) {
+        this.useAvg = useAvg;
+    }
+
+
+    /** {@inheritDoc} */
+    @Override public double getLoad(ClusterNode node, int jobsSentSinceLastUpdate) {
+        ClusterNodeMetrics metrics = node.metrics();
+
+        if (useAvg) {
+            double load = metrics.getAverageJobExecuteTime() + metrics.getAverageJobWaitTime();
+
+            // If load is greater than 0, then we can use average times.
+            // Otherwise, we will proceed to using current times.
+            if (load > 0)
+                return load;
+        }
+
+        double load = metrics.getCurrentJobExecuteTime() + metrics.getCurrentJobWaitTime();
+
+        return load < 0 ? 0 : load;
+    }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return S.toString(AdaptiveProcessingTimeLoadProbe.class, this);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/b69a23cd/modules/core/src/main/java/org/apache/ignite/spi/loadbalancing/adaptive/package.html
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/loadbalancing/adaptive/package.html b/modules/core/src/main/java/org/apache/ignite/spi/loadbalancing/adaptive/package.html
new file mode 100644
index 0000000..ee3a5eb
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/spi/loadbalancing/adaptive/package.html
@@ -0,0 +1,15 @@
+<!DOCTYPE html PUBLIC "-//W3C//DTD HTML 4.01 Transitional//EN" "http://www.w3.org/TR/html4/loose.dtd">
+<!--
+    @html.file.header
+    _________        _____ __________________        _____
+    __  ____/___________(_)______  /__  ____/______ ____(_)_______
+    _  / __  __  ___/__  / _  __  / _  / __  _  __ `/__  / __  __ \
+    / /_/ /  _  /    _  /  / /_/ /  / /_/ /  / /_/ / _  /  _  / / /
+    \____/   /_/     /_/   \_,__/   \____/   \__,_/  /_/   /_/ /_/
+-->
+<html>
+<body>
+    <!-- Package description. -->
+    Contains adaptive load balancing SPI.
+</body>
+</html>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/b69a23cd/modules/core/src/main/java/org/apache/ignite/spi/loadbalancing/package.html
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/loadbalancing/package.html b/modules/core/src/main/java/org/apache/ignite/spi/loadbalancing/package.html
new file mode 100644
index 0000000..fd879b9
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/spi/loadbalancing/package.html
@@ -0,0 +1,15 @@
+<!DOCTYPE html PUBLIC "-//W3C//DTD HTML 4.01 Transitional//EN" "http://www.w3.org/TR/html4/loose.dtd">
+<!--
+    @html.file.header
+    _________        _____ __________________        _____
+    __  ____/___________(_)______  /__  ____/______ ____(_)_______
+    _  / __  __  ___/__  / _  __  / _  / __  _  __ `/__  / __  __ \
+    / /_/ /  _  /    _  /  / /_/ /  / /_/ /  / /_/ / _  /  _  / / /
+    \____/   /_/     /_/   \_,__/   \____/   \__,_/  /_/   /_/ /_/
+-->
+<html>
+<body>
+    <!-- Package description. -->
+    Contains APIs for load balancing SPI.
+</body>
+</html>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/b69a23cd/modules/core/src/main/java/org/apache/ignite/spi/loadbalancing/roundrobin/RoundRobinGlobalLoadBalancer.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/loadbalancing/roundrobin/RoundRobinGlobalLoadBalancer.java b/modules/core/src/main/java/org/apache/ignite/spi/loadbalancing/roundrobin/RoundRobinGlobalLoadBalancer.java
new file mode 100644
index 0000000..e17231a
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/spi/loadbalancing/roundrobin/RoundRobinGlobalLoadBalancer.java
@@ -0,0 +1,305 @@
+/* @java.file.header */
+
+/*  _________        _____ __________________        _____
+ *  __  ____/___________(_)______  /__  ____/______ ____(_)_______
+ *  _  / __  __  ___/__  / _  __  / _  / __  _  __ `/__  / __  __ \
+ *  / /_/ /  _  /    _  /  / /_/ /  / /_/ /  / /_/ / _  /  _  / / /
+ *  \____/   /_/     /_/   \_,__/   \____/   \__,_/  /_/   /_/ /_/
+ */
+
+package org.apache.ignite.spi.loadbalancing.roundrobin;
+
+import org.apache.ignite.*;
+import org.apache.ignite.cluster.*;
+import org.apache.ignite.events.*;
+import org.apache.ignite.spi.*;
+import org.gridgain.grid.*;
+import org.gridgain.grid.kernal.managers.eventstorage.*;
+import org.gridgain.grid.util.typedef.*;
+import org.gridgain.grid.util.typedef.internal.*;
+
+import java.util.*;
+import java.util.concurrent.*;
+import java.util.concurrent.atomic.*;
+
+import static org.apache.ignite.events.IgniteEventType.*;
+
+/**
+ * Load balancer that works in global (not-per-task) mode.
+ */
+class RoundRobinGlobalLoadBalancer {
+    /** SPI context. */
+    private IgniteSpiContext ctx;
+
+    /** Listener for node's events. */
+    private GridLocalEventListener lsnr;
+
+    /** Logger. */
+    private final IgniteLogger log;
+
+    /** Current snapshot of nodes which participated in load balancing. */
+    private volatile GridNodeList nodeList = new GridNodeList(0, null);
+
+    /** Mutex for updating current topology. */
+    private final Object mux = new Object();
+
+    /** Barrier for separating initialization callback and load balancing routine. */
+    private final CountDownLatch initLatch = new CountDownLatch(1);
+
+    /**
+     * @param log Grid logger.
+     */
+    RoundRobinGlobalLoadBalancer(IgniteLogger log) {
+        assert log != null;
+
+        this.log = log;
+    }
+
+    /**
+     * @param ctx Load balancing context.
+     */
+    void onContextInitialized(final IgniteSpiContext ctx) {
+        this.ctx = ctx;
+
+        ctx.addLocalEventListener(
+            lsnr = new GridLocalEventListener() {
+                @Override public void onEvent(IgniteEvent evt) {
+                    assert evt instanceof IgniteDiscoveryEvent;
+
+                    UUID nodeId = ((IgniteDiscoveryEvent)evt).eventNode().id();
+
+                    synchronized (mux) {
+                        if (evt.type() == EVT_NODE_JOINED) {
+                            List<UUID> oldNodes = nodeList.getNodes();
+
+                            if (!oldNodes.contains(nodeId)) {
+                                List<UUID> newNodes = new ArrayList<>(oldNodes.size() + 1);
+
+                                newNodes.add(nodeId);
+
+                                for (UUID node : oldNodes)
+                                    newNodes.add(node);
+
+                                nodeList = new GridNodeList(0, newNodes);
+                            }
+                        }
+                        else {
+                            assert evt.type() == EVT_NODE_LEFT || evt.type() == EVT_NODE_FAILED;
+
+                            List<UUID> oldNodes = nodeList.getNodes();
+
+                            if (oldNodes.contains(nodeId)) {
+                                List<UUID> newNodes = new ArrayList<>(oldNodes.size() - 1);
+
+                                for (UUID node : oldNodes)
+                                    if (!nodeId.equals(node))
+                                        newNodes.add(node);
+
+                                nodeList = new GridNodeList(0, newNodes);
+                            }
+                        }
+                    }
+                }
+            },
+            EVT_NODE_FAILED, EVT_NODE_JOINED, EVT_NODE_LEFT
+        );
+
+        synchronized (mux) {
+            List<UUID> oldNodes = nodeList.getNodes();
+
+            Collection<UUID> set = oldNodes == null ? new HashSet<UUID>() : new HashSet<>(oldNodes);
+
+            for (ClusterNode node : ctx.nodes())
+                set.add(node.id());
+
+            nodeList = new GridNodeList(0, new ArrayList<>(set));
+        }
+
+        initLatch.countDown();
+    }
+
+    /** */
+    void onContextDestroyed() {
+        if (ctx != null)
+            ctx.removeLocalEventListener(lsnr);
+    }
+
+    /**
+     * Gets balanced node for given topology.
+     *
+     * @param top Topology to pick from.
+     * @return Best balanced node.
+     * @throws GridException Thrown in case of any error.
+     */
+    ClusterNode getBalancedNode(Collection<ClusterNode> top) throws GridException {
+        assert !F.isEmpty(top);
+
+        awaitInitializationCompleted();
+
+        Map<UUID, ClusterNode> topMap = null;
+
+        ClusterNode found;
+
+        int misses = 0;
+
+        do {
+            GridNodeList nodeList = this.nodeList;
+
+            List<UUID> nodes = nodeList.getNodes();
+
+            int cycleSize = nodes.size();
+
+            if (cycleSize == 0)
+                throw new GridException("Task topology does not have any alive nodes.");
+
+            AtomicInteger idx;
+
+            int curIdx, nextIdx;
+
+            do {
+                idx = nodeList.getCurrentIdx();
+
+                curIdx = idx.get();
+
+                nextIdx = (idx.get() + 1) % cycleSize;
+            }
+            while (!idx.compareAndSet(curIdx, nextIdx));
+
+            found = findNodeById(top, nodes.get(nextIdx));
+
+            if (found == null) {
+                misses++;
+
+                // For optimization purposes checks balancer can return at least one node with specified
+                // request topology only after full cycle (approximately).
+                if (misses >= cycleSize) {
+                    if (topMap == null) {
+                        topMap = U.newHashMap(top.size());
+
+                        for (ClusterNode node : top)
+                            topMap.put(node.id(), node);
+                    }
+
+                    checkBalancerNodes(top, topMap, nodes);
+
+                    // Zero miss counter so next topology check will be performed once again after full cycle.
+                    misses = 0;
+                }
+            }
+        }
+        while (found == null);
+
+        if (log.isDebugEnabled())
+            log.debug("Found round-robin node: " + found);
+
+        return found;
+    }
+
+    /**
+     * Finds node by id. Returns null in case of absence of specified id in request topology.
+     *
+     * @param top Topology for current request.
+     * @param foundNodeId Node id.
+     * @return Found node or null in case of absence of specified id in request topology.
+     */
+    private static ClusterNode findNodeById(Iterable<ClusterNode> top, UUID foundNodeId) {
+        for (ClusterNode node : top)
+            if (foundNodeId.equals(node.id()))
+                return node;
+
+        return null;
+    }
+
+    /**
+     * Checks if balancer can return at least one node,
+     * throw exception otherwise.
+     *
+     * @param top Topology for current request.
+     * @param topMap Topology map.
+     * @param nodes Current balanced nodes.
+     * @throws GridException If balancer can not return any node.
+     */
+    private static void checkBalancerNodes(Collection<ClusterNode> top, Map<UUID, ClusterNode> topMap, Iterable<UUID> nodes)
+        throws GridException {
+
+        boolean contains = false;
+
+        for (UUID nodeId : nodes) {
+            if (topMap.get(nodeId) != null) {
+                contains = true;
+
+                break;
+            }
+        }
+
+        if (!contains)
+            throw new GridException("Task topology does not have alive nodes: " + top);
+    }
+
+    /**
+     * Awaits initialization of balancing nodes to be completed.
+     *
+     * @throws GridException Thrown in case of thread interruption.
+     */
+    private void awaitInitializationCompleted() throws GridException {
+        try {
+            if (initLatch.getCount() > 0)
+                initLatch.await();
+        }
+        catch (InterruptedException e) {
+            Thread.currentThread().interrupt();
+
+            throw new GridException("Global balancer was interrupted.", e);
+        }
+    }
+
+    /**
+     * Snapshot of nodes which participated in load balancing.
+     */
+    private static final class GridNodeList {
+        /** Cyclic pointer for selecting next node. */
+        private final AtomicInteger curIdx;
+
+        /** Node ids. */
+        private final List<UUID> nodes;
+
+        /**
+         * @param curIdx Initial index of current node.
+         * @param nodes Initial node ids.
+         */
+        private GridNodeList(int curIdx, List<UUID> nodes) {
+            this.curIdx = new AtomicInteger(curIdx);
+            this.nodes = nodes;
+        }
+
+        /**
+         * @return Index of current node.
+         */
+        private AtomicInteger getCurrentIdx() {
+            return curIdx;
+        }
+
+        /**
+         * @return Node ids.
+         */
+        private List<UUID> getNodes() {
+            return nodes;
+        }
+    }
+
+    /**
+     * THIS METHOD IS USED ONLY FOR TESTING.
+     *
+     * @return Internal list of nodes.
+     */
+    List<UUID> getNodeIds() {
+        List<UUID> nodes = nodeList.getNodes();
+
+        return Collections.unmodifiableList(nodes);
+    }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return S.toString(RoundRobinGlobalLoadBalancer.class, this);
+    }
+}


[04/20] incubator-ignite git commit: # Renaming

Posted by sb...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/b69a23cd/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 80c806a..15de6ec 100644
--- a/pom.xml
+++ b/pom.xml
@@ -659,7 +659,7 @@
                                         </group>
                                         <group>
                                             <title>SPI: Load Balancing</title>
-                                            <packages>org.gridgain.grid.spi.loadbalancing:org.gridgain.grid.spi.loadbalancing.adaptive:org.gridgain.grid.spi.loadbalancing.weightedrandom:org.gridgain.grid.spi.loadbalancing.roundrobin</packages>
+                                            <packages>org.gridgain.grid.spi.loadbalancing:org.apache.ignite.spi.loadbalancing.adaptive:org.apache.ignite.spi.loadbalancing.weightedrandom:org.apache.ignite.spi.loadbalancing.roundrobin</packages>
                                         </group>
                                         <group>
                                             <title>SPI: Communication</title>
@@ -861,7 +861,7 @@
                                         </group>
                                         <group>
                                             <title>SPI: Load Balancing</title>
-                                            <packages>org.gridgain.grid.spi.loadbalancing:org.gridgain.grid.spi.loadbalancing.adaptive:org.gridgain.grid.spi.loadbalancing.weightedrandom:org.gridgain.grid.spi.loadbalancing.roundrobin</packages>
+                                            <packages>org.gridgain.grid.spi.loadbalancing:org.apache.ignite.spi.loadbalancing.adaptive:org.apache.ignite.spi.loadbalancing.weightedrandom:org.apache.ignite.spi.loadbalancing.roundrobin</packages>
                                         </group>
                                         <group>
                                             <title>SPI: Communication</title>


[05/20] incubator-ignite git commit: # Renaming

Posted by sb...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/b69a23cd/modules/core/src/test/java/org/apache/ignite/spi/loadbalancing/roundrobin/GridRoundRobinLoadBalancingSpiMultipleNodesSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/spi/loadbalancing/roundrobin/GridRoundRobinLoadBalancingSpiMultipleNodesSelfTest.java b/modules/core/src/test/java/org/apache/ignite/spi/loadbalancing/roundrobin/GridRoundRobinLoadBalancingSpiMultipleNodesSelfTest.java
new file mode 100644
index 0000000..232db9e
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/spi/loadbalancing/roundrobin/GridRoundRobinLoadBalancingSpiMultipleNodesSelfTest.java
@@ -0,0 +1,126 @@
+/* @java.file.header */
+
+/*  _________        _____ __________________        _____
+ *  __  ____/___________(_)______  /__  ____/______ ____(_)_______
+ *  _  / __  __  ___/__  / _  __  / _  / __  _  __ `/__  / __  __ \
+ *  / /_/ /  _  /    _  /  / /_/ /  / /_/ /  / /_/ / _  /  _  / / /
+ *  \____/   /_/     /_/   \_,__/   \____/   \__,_/  /_/   /_/ /_/
+ */
+
+package org.apache.ignite.spi.loadbalancing.roundrobin;
+
+import org.apache.ignite.cluster.*;
+import org.apache.ignite.compute.*;
+import org.apache.ignite.events.*;
+import org.apache.ignite.lang.*;
+import org.gridgain.grid.*;
+import org.gridgain.testframework.*;
+import org.gridgain.testframework.junits.spi.*;
+
+import java.util.*;
+
+import static org.apache.ignite.events.IgniteEventType.*;
+
+/**
+ * Tests round robin load balancing SPI.
+ */
+@GridSpiTest(spi = RoundRobinLoadBalancingSpi.class, group = "Load Balancing SPI")
+public class GridRoundRobinLoadBalancingSpiMultipleNodesSelfTest
+    extends GridSpiAbstractTest<RoundRobinLoadBalancingSpi> {
+    /** {@inheritDoc} */
+    @Override protected void spiConfigure(RoundRobinLoadBalancingSpi spi) throws Exception {
+        super.spiConfigure(spi);
+
+        spi.setPerTask(true);
+    }
+
+    /** {@inheritDoc} */
+    @Override protected GridSpiTestContext initSpiContext() throws Exception {
+        GridSpiTestContext spiCtx = super.initSpiContext();
+
+        spiCtx.createLocalNode();
+        spiCtx.createRemoteNodes(10);
+
+        return spiCtx;
+    }
+
+    /**
+     * @throws Exception If test failed.
+     */
+    @SuppressWarnings({"ObjectEquality"})
+    public void testMultipleNodes() throws Exception {
+        List<ClusterNode> allNodes = (List<ClusterNode>)getSpiContext().nodes();
+
+        ComputeTaskSession ses = new GridTestTaskSession(IgniteUuid.randomUuid());
+
+        // Initialize.
+        getSpi().getBalancedNode(ses, allNodes, new GridTestJob());
+
+        List<UUID> orderedNodes = new ArrayList<>(getSpi().getNodeIds(ses));
+
+        // Check the round-robin actually did circle.
+        for (int i = 0; i < allNodes.size(); i++) {
+            ClusterNode node = getSpi().getBalancedNode(ses, allNodes, new GridTestJob());
+
+            assert orderedNodes.get(i) == node.id();
+        }
+
+        // Double-check.
+        for (int i = 0; i < allNodes.size(); i++) {
+            ClusterNode node = getSpi().getBalancedNode(ses, allNodes, new GridTestJob());
+
+            assert orderedNodes.get(i) == node.id();
+        }
+    }
+
+    /**
+     * @throws Exception If test failed.
+     */
+    @SuppressWarnings({"ObjectEquality"})
+    public void testMultipleTasks() throws Exception {
+        ComputeTaskSession ses1 = new GridTestTaskSession(IgniteUuid.randomUuid());
+        ComputeTaskSession ses2 = new GridTestTaskSession(IgniteUuid.randomUuid());
+
+        List<ClusterNode> allNodes = (List<ClusterNode>)getSpiContext().nodes();
+
+        // Initialize.
+        getSpi().getBalancedNode(ses1, allNodes, new GridTestJob());
+        getSpi().getBalancedNode(ses2, allNodes, new GridTestJob());
+
+        List<UUID> orderedNodes1 = getSpi().getNodeIds(ses1);
+        List<UUID> orderedNodes2 = getSpi().getNodeIds(ses2);
+
+        assert orderedNodes1 != orderedNodes2;
+
+        // Check the round-robin actually did circle.
+        for (int i = 0; i < allNodes.size(); i++) {
+            ClusterNode node1 = getSpi().getBalancedNode(ses1, allNodes, new GridTestJob());
+
+            assert orderedNodes1.get(i) == node1.id();
+
+            ClusterNode node2 = getSpi().getBalancedNode(ses2, allNodes, new GridTestJob());
+
+            assert orderedNodes2.get(i) == node2.id();
+
+            assert orderedNodes1.get(i) == orderedNodes2.get(i);
+        }
+
+        // Double-check.
+        for (int i = 0; i < allNodes.size(); i++) {
+            ClusterNode node1 = getSpi().getBalancedNode(ses1, allNodes, new GridTestJob());
+
+            assert orderedNodes1.get(i) == node1.id();
+
+            ClusterNode node2 = getSpi().getBalancedNode(ses2, allNodes, new GridTestJob());
+
+            assert orderedNodes2.get(i) == node2.id();
+
+            assert orderedNodes1.get(i) == orderedNodes2.get(i);
+        }
+
+        getSpiContext().triggerEvent(new IgniteTaskEvent(
+            null, null, EVT_TASK_FINISHED, ses1.getId(), null, null, false, null));
+        getSpiContext().triggerEvent(new IgniteTaskEvent(
+            null, null, EVT_TASK_FAILED, ses2.getId(), null, null, false, null));
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/b69a23cd/modules/core/src/test/java/org/apache/ignite/spi/loadbalancing/roundrobin/GridRoundRobinLoadBalancingSpiNotPerTaskSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/spi/loadbalancing/roundrobin/GridRoundRobinLoadBalancingSpiNotPerTaskSelfTest.java b/modules/core/src/test/java/org/apache/ignite/spi/loadbalancing/roundrobin/GridRoundRobinLoadBalancingSpiNotPerTaskSelfTest.java
new file mode 100644
index 0000000..a46b75f
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/spi/loadbalancing/roundrobin/GridRoundRobinLoadBalancingSpiNotPerTaskSelfTest.java
@@ -0,0 +1,121 @@
+/* @java.file.header */
+
+/*  _________        _____ __________________        _____
+ *  __  ____/___________(_)______  /__  ____/______ ____(_)_______
+ *  _  / __  __  ___/__  / _  __  / _  / __  _  __ `/__  / __  __ \
+ *  / /_/ /  _  /    _  /  / /_/ /  / /_/ /  / /_/ / _  /  _  / / /
+ *  \____/   /_/     /_/   \_,__/   \____/   \__,_/  /_/   /_/ /_/
+ */
+
+package org.apache.ignite.spi.loadbalancing.roundrobin;
+
+import org.apache.ignite.cluster.*;
+import org.apache.ignite.compute.*;
+import org.apache.ignite.events.*;
+import org.apache.ignite.lang.*;
+import org.gridgain.grid.*;
+import org.gridgain.testframework.*;
+import org.gridgain.testframework.junits.spi.*;
+
+import java.util.*;
+
+import static org.apache.ignite.events.IgniteEventType.*;
+import static org.apache.ignite.spi.loadbalancing.roundrobin.GridRoundRobinTestUtils.*;
+
+/**
+ * Tests round robin load balancing.
+ */
+@GridSpiTest(spi = RoundRobinLoadBalancingSpi.class, group = "Load Balancing SPI")
+public class GridRoundRobinLoadBalancingSpiNotPerTaskSelfTest
+    extends GridSpiAbstractTest<RoundRobinLoadBalancingSpi> {
+    /**
+     * @return Per-task configuration parameter.
+     */
+    @GridSpiTestConfig
+    public boolean getPerTask() {
+        return false;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected GridSpiTestContext initSpiContext() throws Exception {
+        GridSpiTestContext spiCtx = super.initSpiContext();
+
+        spiCtx.createLocalNode();
+        spiCtx.createRemoteNodes(10);
+
+        return spiCtx;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTest() throws Exception {
+        assert !getSpi().isPerTask() : "Invalid SPI configuration.";
+    }
+
+    /**
+     * @throws Exception If test failed.
+     */
+    public void testMultipleNodes() throws Exception {
+        List<ClusterNode> allNodes = (List<ClusterNode>)getSpiContext().nodes();
+
+        ComputeTaskSession ses = new GridTestTaskSession();
+
+        List<UUID> orderedNodes = new ArrayList<>(getSpi().getNodeIds(ses));
+
+        assertEquals("Balancer doesn't use all available nodes", orderedNodes.size(), allNodes.size());
+
+        checkCyclicBalancing(getSpi(), allNodes, orderedNodes, ses);
+    }
+
+    /**
+     * @throws Exception If test failed.
+     */
+    public void testMultipleTaskSessions() throws Exception {
+        ComputeTaskSession ses1 = new GridTestTaskSession(IgniteUuid.randomUuid());
+        ComputeTaskSession ses2 = new GridTestTaskSession(IgniteUuid.randomUuid());
+
+        List<ClusterNode> allNodes = (List<ClusterNode>)getSpiContext().nodes();
+
+        List<UUID> orderedNodes = getSpi().getNodeIds(ses1);
+
+        assertEquals("Balancer doesn't use all available nodes", orderedNodes.size(), allNodes.size());
+
+        checkCyclicBalancing(getSpi(), allNodes, orderedNodes, ses1, ses2);
+
+        getSpiContext().triggerEvent(new IgniteTaskEvent(
+            null, null, EVT_TASK_FINISHED, ses1.getId(), null, null, false, null));
+        getSpiContext().triggerEvent(new IgniteTaskEvent(
+            null, null, EVT_TASK_FAILED, ses2.getId(), null, null, false, null));
+    }
+
+    /**
+     * @throws Exception If test failed.
+     */
+    public void testBalancingOneNode() throws Exception {
+        ComputeTaskSession ses = new GridTestTaskSession();
+
+        List<ClusterNode> allNodes = (List<ClusterNode>)getSpiContext().nodes();
+
+        List<ClusterNode> balancedNode = Arrays.asList(allNodes.get(0));
+
+        ClusterNode firstNode = getSpi().getBalancedNode(ses, balancedNode, new GridTestJob());
+        ClusterNode secondNode = getSpi().getBalancedNode(ses, balancedNode, new GridTestJob());
+
+        assertEquals(firstNode, secondNode);
+    }
+
+    /** */
+    public void testNodeNotInTopology() {
+        ComputeTaskSession ses = new GridTestTaskSession();
+
+        ClusterNode node = new GridTestNode(UUID.randomUUID());
+
+        List<ClusterNode> notInTop = Arrays.asList(node);
+
+        try {
+            getSpi().getBalancedNode(ses, notInTop, new GridTestJob());
+        }
+        catch (GridException e) {
+            assertTrue(e.getMessage().contains("Task topology does not have alive nodes"));
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/b69a23cd/modules/core/src/test/java/org/apache/ignite/spi/loadbalancing/roundrobin/GridRoundRobinLoadBalancingSpiStartStopSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/spi/loadbalancing/roundrobin/GridRoundRobinLoadBalancingSpiStartStopSelfTest.java b/modules/core/src/test/java/org/apache/ignite/spi/loadbalancing/roundrobin/GridRoundRobinLoadBalancingSpiStartStopSelfTest.java
new file mode 100644
index 0000000..6aa1c92
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/spi/loadbalancing/roundrobin/GridRoundRobinLoadBalancingSpiStartStopSelfTest.java
@@ -0,0 +1,23 @@
+/* @java.file.header */
+
+/*  _________        _____ __________________        _____
+ *  __  ____/___________(_)______  /__  ____/______ ____(_)_______
+ *  _  / __  __  ___/__  / _  __  / _  / __  _  __ `/__  / __  __ \
+ *  / /_/ /  _  /    _  /  / /_/ /  / /_/ /  / /_/ / _  /  _  / / /
+ *  \____/   /_/     /_/   \_,__/   \____/   \__,_/  /_/   /_/ /_/
+ */
+
+package org.apache.ignite.spi.loadbalancing.roundrobin;
+
+import org.gridgain.grid.spi.*;
+import org.gridgain.testframework.junits.spi.*;
+
+/**
+ * Tests correct start of {@link RoundRobinLoadBalancingSpi}.
+ */
+@SuppressWarnings({"JUnitTestCaseWithNoTests"})
+@GridSpiTest(spi = RoundRobinLoadBalancingSpi.class, group = "LoadBalancing SPI")
+public class GridRoundRobinLoadBalancingSpiStartStopSelfTest
+    extends GridSpiStartStopAbstractTest<RoundRobinLoadBalancingSpi> {
+    // No configs.
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/b69a23cd/modules/core/src/test/java/org/apache/ignite/spi/loadbalancing/roundrobin/GridRoundRobinLoadBalancingSpiTopologyChangeSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/spi/loadbalancing/roundrobin/GridRoundRobinLoadBalancingSpiTopologyChangeSelfTest.java b/modules/core/src/test/java/org/apache/ignite/spi/loadbalancing/roundrobin/GridRoundRobinLoadBalancingSpiTopologyChangeSelfTest.java
new file mode 100644
index 0000000..3a148ed
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/spi/loadbalancing/roundrobin/GridRoundRobinLoadBalancingSpiTopologyChangeSelfTest.java
@@ -0,0 +1,98 @@
+/* @java.file.header */
+
+/*  _________        _____ __________________        _____
+ *  __  ____/___________(_)______  /__  ____/______ ____(_)_______
+ *  _  / __  __  ___/__  / _  __  / _  / __  _  __ `/__  / __  __ \
+ *  / /_/ /  _  /    _  /  / /_/ /  / /_/ /  / /_/ / _  /  _  / / /
+ *  \____/   /_/     /_/   \_,__/   \____/   \__,_/  /_/   /_/ /_/
+ */
+
+package org.apache.ignite.spi.loadbalancing.roundrobin;
+
+import org.apache.ignite.cluster.*;
+import org.apache.ignite.compute.*;
+import org.apache.ignite.lang.*;
+import org.gridgain.grid.*;
+import org.gridgain.testframework.*;
+import org.gridgain.testframework.junits.spi.*;
+
+import java.util.*;
+
+import static org.apache.ignite.spi.loadbalancing.roundrobin.GridRoundRobinTestUtils.*;
+
+/**
+ * Tests round robin load balancing with topology changes.
+ */
+@GridSpiTest(spi = RoundRobinLoadBalancingSpi.class, group = "Load Balancing SPI")
+public class GridRoundRobinLoadBalancingSpiTopologyChangeSelfTest
+    extends GridSpiAbstractTest<RoundRobinLoadBalancingSpi> {
+    /**
+     * @return Per-task configuration parameter.
+     */
+    @GridSpiTestConfig
+    public boolean getPerTask() { return false; }
+
+    /** {@inheritDoc} */
+    @Override protected GridSpiTestContext initSpiContext() throws Exception {
+        GridSpiTestContext spiCtx = super.initSpiContext();
+
+        spiCtx.createLocalNode();
+        spiCtx.createRemoteNodes(10);
+
+        return spiCtx;
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testTopologyChange() throws Exception {
+        ComputeTaskSession ses = new GridTestTaskSession(IgniteUuid.randomUuid());
+
+        // Warm up.
+        List<ClusterNode> allNodes = (List<ClusterNode>)getSpiContext().nodes();
+
+        List<UUID> orderedNodes = getSpi().getNodeIds(ses);
+
+        checkCyclicBalancing(getSpi(), allNodes, orderedNodes, ses);
+
+        // Remove node.
+        UUID doomed = orderedNodes.get(0);
+
+        if (getSpiContext().localNode().id().equals(doomed))
+            doomed = orderedNodes.get(1);
+
+        getSpiContext().removeNode(doomed);
+
+        assertTrue(allNodes.remove(new GridTestNode(doomed)));
+
+        orderedNodes = getSpi().getNodeIds(ses);
+
+        assertFalse("Balancer uses removed node", orderedNodes.contains(doomed));
+
+        checkCyclicBalancing(getSpi(), allNodes, orderedNodes, ses);
+
+        // Add node.
+        ClusterNode newNode = new GridTestNode(UUID.randomUUID());
+
+        getSpiContext().addNode(newNode);
+
+        assertTrue(allNodes.add(newNode));
+
+        // Check that new node was added to balancing.
+        boolean foundNewNode = false;
+
+        for (int i = 0; i < allNodes.size(); i++) {
+            ClusterNode node = getSpi().getBalancedNode(ses, allNodes, new GridTestJob());
+            if (newNode.id().equals(node.id())) {
+                foundNewNode = true;
+                break;
+            }
+        }
+
+        assertTrue("Balancer doesn't use added node", foundNewNode);
+
+        orderedNodes = getSpi().getNodeIds(ses);
+
+        checkCyclicBalancing(getSpi(), allNodes, orderedNodes, ses);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/b69a23cd/modules/core/src/test/java/org/apache/ignite/spi/loadbalancing/roundrobin/GridRoundRobinTestUtils.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/spi/loadbalancing/roundrobin/GridRoundRobinTestUtils.java b/modules/core/src/test/java/org/apache/ignite/spi/loadbalancing/roundrobin/GridRoundRobinTestUtils.java
new file mode 100644
index 0000000..fc3fbc4
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/spi/loadbalancing/roundrobin/GridRoundRobinTestUtils.java
@@ -0,0 +1,95 @@
+/* @java.file.header */
+
+/*  _________        _____ __________________        _____
+ *  __  ____/___________(_)______  /__  ____/______ ____(_)_______
+ *  _  / __  __  ___/__  / _  __  / _  / __  _  __ `/__  / __  __ \
+ *  / /_/ /  _  /    _  /  / /_/ /  / /_/ /  / /_/ / _  /  _  / / /
+ *  \____/   /_/     /_/   \_,__/   \____/   \__,_/  /_/   /_/ /_/
+ */
+
+package org.apache.ignite.spi.loadbalancing.roundrobin;
+
+import org.apache.ignite.cluster.*;
+import org.apache.ignite.compute.*;
+import org.gridgain.grid.*;
+
+import java.util.*;
+
+import static org.junit.Assert.*;
+
+/**
+ * Helper class for balancer tests.
+ */
+class GridRoundRobinTestUtils {
+    /**
+     * Performs two full cycles by round robin routine for check correct order.
+     *
+     * @param spi Load balancing SPI.
+     * @param allNodes Topology nodes.
+     * @param orderedNodes Balancing nodes.
+     * @param ses Task session.
+     * @throws GridException If balancer failed.
+     */
+    static void checkCyclicBalancing(RoundRobinLoadBalancingSpi spi, List<ClusterNode> allNodes,
+        List<UUID> orderedNodes, ComputeTaskSession ses) throws GridException {
+
+        ClusterNode firstNode = spi.getBalancedNode(ses, allNodes, new GridTestJob());
+
+        int startIdx = firstBalancedNodeIndex(firstNode, orderedNodes);
+
+        // Two full cycles by round robin routine.
+        for (int i = 0; i < allNodes.size() * 2; i++) {
+            int actualIdx = (startIdx + i + 1) % allNodes.size();
+
+            ClusterNode nextNode = spi.getBalancedNode(ses, allNodes, new GridTestJob());
+
+            assertEquals("Balancer returns node out of order", nextNode.id(), orderedNodes.get(actualIdx));
+        }
+    }
+
+    /**
+     * Performs two full cycles by round robin routine for check correct order.
+     * Switches between two task sessions by turns.
+     *
+     * @param spi Load balancing SPI.
+     * @param allNodes Topology nodes.
+     * @param orderedNodes Balancing nodes.
+     * @param ses1 First task session.
+     * @param ses2 Second task session.
+     * @throws GridException If balancer failed.
+     */
+    static void checkCyclicBalancing(RoundRobinLoadBalancingSpi spi, List<ClusterNode> allNodes,
+        List<UUID> orderedNodes, ComputeTaskSession ses1, ComputeTaskSession ses2) throws GridException {
+
+        ClusterNode firstNode = spi.getBalancedNode(ses1, allNodes, new GridTestJob());
+
+        int startIdx = firstBalancedNodeIndex(firstNode, orderedNodes);
+
+        // Two full cycles by round robin routine.
+        for (int i = 0; i < allNodes.size() * 2; i++) {
+            int actualIdx = (startIdx + i + 1) % allNodes.size();
+
+            ClusterNode nextNode = spi.getBalancedNode(i % 2 == 0 ? ses1 : ses2, allNodes, new GridTestJob());
+
+            assertEquals("Balancer returns node out of order", nextNode.id(), orderedNodes.get(actualIdx));
+        }
+    }
+
+    /**
+     * @param firstNode First node which was return by balancer.
+     * @param orderedNodes Balancing nodes.
+     * @return Index of first node which was return by balancer.
+     */
+    static int firstBalancedNodeIndex(ClusterNode firstNode, List<UUID> orderedNodes) {
+        int startIdx = -1;
+
+        for (int i = 0; i < orderedNodes.size(); i++) {
+            if (firstNode.id() == orderedNodes.get(i))
+                startIdx = i;
+        }
+
+        assertTrue("Can't find position of first balanced node", startIdx >= 0);
+
+        return startIdx;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/b69a23cd/modules/core/src/test/java/org/apache/ignite/spi/loadbalancing/roundrobin/package.html
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/spi/loadbalancing/roundrobin/package.html b/modules/core/src/test/java/org/apache/ignite/spi/loadbalancing/roundrobin/package.html
new file mode 100644
index 0000000..5cad80a
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/spi/loadbalancing/roundrobin/package.html
@@ -0,0 +1,15 @@
+<!DOCTYPE html PUBLIC "-//W3C//DTD HTML 4.01 Transitional//EN" "http://www.w3.org/TR/html4/loose.dtd">
+<!--
+    @html.file.header
+    _________        _____ __________________        _____
+    __  ____/___________(_)______  /__  ____/______ ____(_)_______
+    _  / __  __  ___/__  / _  __  / _  / __  _  __ `/__  / __  __ \
+    / /_/ /  _  /    _  /  / /_/ /  / /_/ /  / /_/ / _  /  _  / / /
+    \____/   /_/     /_/   \_,__/   \____/   \__,_/  /_/   /_/ /_/
+-->
+<html>
+<body>
+    <!-- Package description. -->
+    Contains internal tests or test related classes and interfaces.
+</body>
+</html>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/b69a23cd/modules/core/src/test/java/org/apache/ignite/spi/loadbalancing/weightedrandom/GridWeightedRandomLoadBalancingSpiConfigSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/spi/loadbalancing/weightedrandom/GridWeightedRandomLoadBalancingSpiConfigSelfTest.java b/modules/core/src/test/java/org/apache/ignite/spi/loadbalancing/weightedrandom/GridWeightedRandomLoadBalancingSpiConfigSelfTest.java
new file mode 100644
index 0000000..ec55882
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/spi/loadbalancing/weightedrandom/GridWeightedRandomLoadBalancingSpiConfigSelfTest.java
@@ -0,0 +1,26 @@
+/* @java.file.header */
+
+/*  _________        _____ __________________        _____
+ *  __  ____/___________(_)______  /__  ____/______ ____(_)_______
+ *  _  / __  __  ___/__  / _  __  / _  / __  _  __ `/__  / __  __ \
+ *  / /_/ /  _  /    _  /  / /_/ /  / /_/ /  / /_/ / _  /  _  / / /
+ *  \____/   /_/     /_/   \_,__/   \____/   \__,_/  /_/   /_/ /_/
+ */
+
+package org.apache.ignite.spi.loadbalancing.weightedrandom;
+
+import org.gridgain.testframework.junits.spi.*;
+
+/**
+ *
+ */
+@GridSpiTest(spi = WeightedRandomLoadBalancingSpi.class, group = "Load Balancing SPI")
+public class GridWeightedRandomLoadBalancingSpiConfigSelfTest extends
+    GridSpiAbstractConfigTest<WeightedRandomLoadBalancingSpi> {
+    /**
+     * @throws Exception If failed.
+     */
+    public void testNegativeConfig() throws Exception {
+        checkNegativeSpiProperty(new WeightedRandomLoadBalancingSpi(), "nodeWeight", 0);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/b69a23cd/modules/core/src/test/java/org/apache/ignite/spi/loadbalancing/weightedrandom/GridWeightedRandomLoadBalancingSpiSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/spi/loadbalancing/weightedrandom/GridWeightedRandomLoadBalancingSpiSelfTest.java b/modules/core/src/test/java/org/apache/ignite/spi/loadbalancing/weightedrandom/GridWeightedRandomLoadBalancingSpiSelfTest.java
new file mode 100644
index 0000000..3658582
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/spi/loadbalancing/weightedrandom/GridWeightedRandomLoadBalancingSpiSelfTest.java
@@ -0,0 +1,58 @@
+/* @java.file.header */
+
+/*  _________        _____ __________________        _____
+ *  __  ____/___________(_)______  /__  ____/______ ____(_)_______
+ *  _  / __  __  ___/__  / _  __  / _  / __  _  __ `/__  / __  __ \
+ *  / /_/ /  _  /    _  /  / /_/ /  / /_/ /  / /_/ / _  /  _  / / /
+ *  \____/   /_/     /_/   \_,__/   \____/   \__,_/  /_/   /_/ /_/
+ */
+
+package org.apache.ignite.spi.loadbalancing.weightedrandom;
+
+import org.apache.ignite.cluster.*;
+import org.gridgain.grid.*;
+import org.gridgain.testframework.*;
+import org.gridgain.testframework.junits.spi.*;
+import java.util.*;
+
+/**
+ * Weighted random load balancing SPI.
+ */
+@GridSpiTest(spi = WeightedRandomLoadBalancingSpi.class, group = "Load Balancing SPI")
+public class GridWeightedRandomLoadBalancingSpiSelfTest extends
+    GridSpiAbstractTest<WeightedRandomLoadBalancingSpi> {
+    /**
+     * @throws Exception If failed.
+     */
+    @SuppressWarnings({"ObjectEquality"})
+    public void testSingleNode() throws Exception {
+        List<ClusterNode> nodes = Collections.singletonList((ClusterNode)new GridTestNode(UUID.randomUUID()));
+
+        ClusterNode node = getSpi().getBalancedNode(new GridTestTaskSession(), nodes, new GridTestJob());
+
+        assert nodes.contains(node);
+
+        // Verify that same instance is returned every time.
+        ClusterNode balancedNode = getSpi().getBalancedNode(new GridTestTaskSession(), nodes, new GridTestJob());
+
+        assert node == balancedNode;
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testMultipleNodes() throws Exception {
+        List<ClusterNode> nodes = new ArrayList<>();
+
+        for (int i = 0; i < 10; i++)
+            nodes.add(new GridTestNode(UUID.randomUUID()));
+
+        // Seal it.
+        nodes = Collections.unmodifiableList(nodes);
+
+        ClusterNode node = getSpi().getBalancedNode(new GridTestTaskSession(), nodes, new GridTestJob());
+
+        assert node != null;
+        assert nodes.contains(node);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/b69a23cd/modules/core/src/test/java/org/apache/ignite/spi/loadbalancing/weightedrandom/GridWeightedRandomLoadBalancingSpiStartStopSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/spi/loadbalancing/weightedrandom/GridWeightedRandomLoadBalancingSpiStartStopSelfTest.java b/modules/core/src/test/java/org/apache/ignite/spi/loadbalancing/weightedrandom/GridWeightedRandomLoadBalancingSpiStartStopSelfTest.java
new file mode 100644
index 0000000..c05a33e
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/spi/loadbalancing/weightedrandom/GridWeightedRandomLoadBalancingSpiStartStopSelfTest.java
@@ -0,0 +1,23 @@
+/* @java.file.header */
+
+/*  _________        _____ __________________        _____
+ *  __  ____/___________(_)______  /__  ____/______ ____(_)_______
+ *  _  / __  __  ___/__  / _  __  / _  / __  _  __ `/__  / __  __ \
+ *  / /_/ /  _  /    _  /  / /_/ /  / /_/ /  / /_/ / _  /  _  / / /
+ *  \____/   /_/     /_/   \_,__/   \____/   \__,_/  /_/   /_/ /_/
+ */
+
+package org.apache.ignite.spi.loadbalancing.weightedrandom;
+
+import org.gridgain.grid.spi.*;
+import org.gridgain.testframework.junits.spi.*;
+
+/**
+ * Wighted random load balancing SPI start-stop test.
+ */
+@SuppressWarnings({"JUnitTestCaseWithNoTests"})
+@GridSpiTest(spi = WeightedRandomLoadBalancingSpi.class, group = "LoadBalancing SPI")
+public class GridWeightedRandomLoadBalancingSpiStartStopSelfTest extends
+    GridSpiStartStopAbstractTest<WeightedRandomLoadBalancingSpi> {
+    // No configs.
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/b69a23cd/modules/core/src/test/java/org/apache/ignite/spi/loadbalancing/weightedrandom/GridWeightedRandomLoadBalancingSpiWeightedSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/spi/loadbalancing/weightedrandom/GridWeightedRandomLoadBalancingSpiWeightedSelfTest.java b/modules/core/src/test/java/org/apache/ignite/spi/loadbalancing/weightedrandom/GridWeightedRandomLoadBalancingSpiWeightedSelfTest.java
new file mode 100644
index 0000000..835c163
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/spi/loadbalancing/weightedrandom/GridWeightedRandomLoadBalancingSpiWeightedSelfTest.java
@@ -0,0 +1,73 @@
+/* @java.file.header */
+
+/*  _________        _____ __________________        _____
+ *  __  ____/___________(_)______  /__  ____/______ ____(_)_______
+ *  _  / __  __  ___/__  / _  __  / _  / __  _  __ `/__  / __  __ \
+ *  / /_/ /  _  /    _  /  / /_/ /  / /_/ /  / /_/ / _  /  _  / / /
+ *  \____/   /_/     /_/   \_,__/   \____/   \__,_/  /_/   /_/ /_/
+ */
+
+package org.apache.ignite.spi.loadbalancing.weightedrandom;
+
+import org.apache.ignite.cluster.*;
+import org.apache.ignite.lang.*;
+import org.gridgain.grid.*;
+import org.gridgain.grid.util.typedef.internal.*;
+import org.gridgain.testframework.*;
+import org.gridgain.testframework.junits.spi.*;
+import java.util.*;
+
+import static org.apache.ignite.spi.loadbalancing.weightedrandom.WeightedRandomLoadBalancingSpi.*;
+
+/**
+ * {@link WeightedRandomLoadBalancingSpi} self test.
+ */
+@GridSpiTest(spi = WeightedRandomLoadBalancingSpi.class, group = "Load Balancing SPI")
+public class GridWeightedRandomLoadBalancingSpiWeightedSelfTest
+    extends GridSpiAbstractTest<WeightedRandomLoadBalancingSpi> {
+    /**
+     * @return {@code True} if node weights should be considered.
+     */
+    @GridSpiTestConfig
+    public boolean getUseWeights() {
+        return true;
+    }
+
+    /**
+     * @throws Exception If test failed.
+     */
+    public void testWeights() throws Exception {
+        List<ClusterNode> nodes = new ArrayList<>();
+
+        for (int i = 0; i < 10; i++) {
+            GridTestNode node = new GridTestNode(UUID.randomUUID());
+
+            node.addAttribute(U.spiAttribute(getSpi(), NODE_WEIGHT_ATTR_NAME), i + 1);
+
+            nodes.add(node);
+        }
+
+        // Seal it.
+        nodes = Collections.unmodifiableList(nodes);
+
+        int[] cnts = new int[10];
+
+        // Invoke load balancer a large number of times, so statistics won't lie.
+        for (int i = 0; i < 100000; i++) {
+            ClusterNode node = getSpi().getBalancedNode(new GridTestTaskSession(IgniteUuid.randomUuid()), nodes,
+                new GridTestJob());
+
+            int weight = (Integer)node.attribute(U.spiAttribute(getSpi(), NODE_WEIGHT_ATTR_NAME));
+
+            // Increment number of times a node was picked.
+            cnts[weight - 1]++;
+        }
+
+        for (int i = 0; i < cnts.length - 1; i++) {
+            assert cnts[i] < cnts[i + 1] : "Invalid node counts for index [idx=" + i + ", cnts[i]=" + cnts[i] +
+                ", cnts[i+1]=" + cnts[i + 1] + ']';
+        }
+
+        info("Node counts: " + Arrays.toString(cnts));
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/b69a23cd/modules/core/src/test/java/org/apache/ignite/spi/loadbalancing/weightedrandom/package.html
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/spi/loadbalancing/weightedrandom/package.html b/modules/core/src/test/java/org/apache/ignite/spi/loadbalancing/weightedrandom/package.html
new file mode 100644
index 0000000..5cad80a
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/spi/loadbalancing/weightedrandom/package.html
@@ -0,0 +1,15 @@
+<!DOCTYPE html PUBLIC "-//W3C//DTD HTML 4.01 Transitional//EN" "http://www.w3.org/TR/html4/loose.dtd">
+<!--
+    @html.file.header
+    _________        _____ __________________        _____
+    __  ____/___________(_)______  /__  ____/______ ____(_)_______
+    _  / __  __  ___/__  / _  __  / _  / __  _  __ `/__  / __  __ \
+    / /_/ /  _  /    _  /  / /_/ /  / /_/ /  / /_/ / _  /  _  / / /
+    \____/   /_/     /_/   \_,__/   \____/   \__,_/  /_/   /_/ /_/
+-->
+<html>
+<body>
+    <!-- Package description. -->
+    Contains internal tests or test related classes and interfaces.
+</body>
+</html>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/b69a23cd/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 ba5e3b0..3585fc5 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
@@ -19,7 +19,7 @@ import org.gridgain.grid.*;
 import org.apache.ignite.spi.checkpoint.sharedfs.*;
 import org.apache.ignite.spi.failover.*;
 import org.apache.ignite.spi.failover.always.*;
-import org.gridgain.grid.spi.loadbalancing.roundrobin.*;
+import org.apache.ignite.spi.loadbalancing.roundrobin.*;
 import org.gridgain.testframework.junits.common.*;
 
 import java.util.*;

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/b69a23cd/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 d682e4e..9209a34 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
@@ -35,7 +35,7 @@ import org.apache.ignite.spi.discovery.tcp.*;
 import org.apache.ignite.spi.eventstorage.*;
 import org.apache.ignite.spi.eventstorage.memory.*;
 import org.apache.ignite.spi.failover.always.*;
-import org.gridgain.grid.spi.loadbalancing.roundrobin.*;
+import org.apache.ignite.spi.loadbalancing.roundrobin.*;
 import org.gridgain.grid.spi.swapspace.*;
 import org.gridgain.grid.spi.swapspace.file.*;
 import org.gridgain.testframework.junits.*;

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/b69a23cd/modules/core/src/test/java/org/gridgain/grid/spi/loadbalancing/adaptive/GridAdaptiveLoadBalancingSpiConfigSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/gridgain/grid/spi/loadbalancing/adaptive/GridAdaptiveLoadBalancingSpiConfigSelfTest.java b/modules/core/src/test/java/org/gridgain/grid/spi/loadbalancing/adaptive/GridAdaptiveLoadBalancingSpiConfigSelfTest.java
deleted file mode 100644
index 8e080e8..0000000
--- a/modules/core/src/test/java/org/gridgain/grid/spi/loadbalancing/adaptive/GridAdaptiveLoadBalancingSpiConfigSelfTest.java
+++ /dev/null
@@ -1,26 +0,0 @@
-/* @java.file.header */
-
-/*  _________        _____ __________________        _____
- *  __  ____/___________(_)______  /__  ____/______ ____(_)_______
- *  _  / __  __  ___/__  / _  __  / _  / __  _  __ `/__  / __  __ \
- *  / /_/ /  _  /    _  /  / /_/ /  / /_/ /  / /_/ / _  /  _  / / /
- *  \____/   /_/     /_/   \_,__/   \____/   \__,_/  /_/   /_/ /_/
- */
-
-package org.gridgain.grid.spi.loadbalancing.adaptive;
-
-import org.gridgain.testframework.junits.spi.*;
-
-/**
- *
- */
-@GridSpiTest(spi = AdaptiveLoadBalancingSpi.class, group = "LoadBalancing SPI")
-public class GridAdaptiveLoadBalancingSpiConfigSelfTest
-    extends GridSpiAbstractConfigTest<AdaptiveLoadBalancingSpi> {
-    /**
-     * @throws Exception If failed.
-     */
-    public void testNegativeConfig() throws Exception {
-        checkNegativeSpiProperty(new AdaptiveLoadBalancingSpi(), "loadProbe", null);
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/b69a23cd/modules/core/src/test/java/org/gridgain/grid/spi/loadbalancing/adaptive/GridAdaptiveLoadBalancingSpiMultipleNodeSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/gridgain/grid/spi/loadbalancing/adaptive/GridAdaptiveLoadBalancingSpiMultipleNodeSelfTest.java b/modules/core/src/test/java/org/gridgain/grid/spi/loadbalancing/adaptive/GridAdaptiveLoadBalancingSpiMultipleNodeSelfTest.java
deleted file mode 100644
index 33e1a18..0000000
--- a/modules/core/src/test/java/org/gridgain/grid/spi/loadbalancing/adaptive/GridAdaptiveLoadBalancingSpiMultipleNodeSelfTest.java
+++ /dev/null
@@ -1,87 +0,0 @@
-/* @java.file.header */
-
-/*  _________        _____ __________________        _____
- *  __  ____/___________(_)______  /__  ____/______ ____(_)_______
- *  _  / __  __  ___/__  / _  __  / _  / __  _  __ `/__  / __  __ \
- *  / /_/ /  _  /    _  /  / /_/ /  / /_/ /  / /_/ / _  /  _  / / /
- *  \____/   /_/     /_/   \_,__/   \____/   \__,_/  /_/   /_/ /_/
- */
-
-package org.gridgain.grid.spi.loadbalancing.adaptive;
-
-import org.apache.ignite.cluster.*;
-import org.apache.ignite.lang.*;
-import org.gridgain.grid.*;
-import org.gridgain.testframework.*;
-import org.gridgain.testframework.junits.spi.*;
-import java.util.*;
-
-/**
- * Tests adaptive load balancing SPI.
- */
-@GridSpiTest(spi = AdaptiveLoadBalancingSpi.class, group = "Load Balancing SPI")
-public class GridAdaptiveLoadBalancingSpiMultipleNodeSelfTest extends GridSpiAbstractTest<AdaptiveLoadBalancingSpi> {
-    /** */
-    private static final int RMT_NODE_CNT = 10;
-
-    /** {@inheritDoc} */
-    @Override protected GridSpiTestContext initSpiContext() throws Exception {
-        GridSpiTestContext ctx = super.initSpiContext();
-
-        for (int i = 0; i < RMT_NODE_CNT; i++) {
-            GridTestNode node = new GridTestNode(UUID.randomUUID());
-
-            node.setAttribute("load", (double)(i + 1));
-
-            ctx.addNode(node);
-        }
-
-        return ctx;
-    }
-
-    /**
-     * @return {@code True} if node weights should be considered.
-     */
-    @GridSpiTestConfig
-    public AdaptiveLoadProbe getLoadProbe() {
-        return new AdaptiveLoadProbe() {
-            @Override public double getLoad(ClusterNode node, int jobsSentSinceLastUpdate) {
-                boolean isFirstTime = node.attribute("used") == null;
-
-                assert isFirstTime ? jobsSentSinceLastUpdate == 0 : jobsSentSinceLastUpdate > 0;
-
-                return (Double)node.attribute("load");
-            }
-        };
-    }
-    /**
-     * @throws Exception If failed.
-     */
-    public void testWeights() throws Exception {
-        // Seal it.
-        List<ClusterNode> nodes = new ArrayList<>(getSpiContext().remoteNodes());
-
-        int[] cnts = new int[RMT_NODE_CNT];
-
-        // Invoke load balancer a large number of times, so statistics won't lie.
-        for (int i = 0; i < 50000; i++) {
-            GridTestNode node = (GridTestNode)getSpi().getBalancedNode(new GridTestTaskSession(IgniteUuid.randomUuid()),
-                nodes, new GridTestJob());
-
-            int idx = ((Double)node.attribute("load")).intValue() - 1;
-
-            if (cnts[idx] == 0)
-                node.setAttribute("used", true);
-
-            // Increment number of times a node was picked.
-            cnts[idx]++;
-        }
-
-        info("Node counts: " + Arrays.toString(cnts));
-
-        for (int i = 0; i < cnts.length - 1; i++) {
-            assert cnts[i] > cnts[i + 1] : "Invalid node counts for index [idx=" + i + ", cnts[i]=" + cnts[i] +
-                ", cnts[i+1]=" + cnts[i + 1] + ']';
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/b69a23cd/modules/core/src/test/java/org/gridgain/grid/spi/loadbalancing/adaptive/GridAdaptiveLoadBalancingSpiSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/gridgain/grid/spi/loadbalancing/adaptive/GridAdaptiveLoadBalancingSpiSelfTest.java b/modules/core/src/test/java/org/gridgain/grid/spi/loadbalancing/adaptive/GridAdaptiveLoadBalancingSpiSelfTest.java
deleted file mode 100644
index 8a3a2c5..0000000
--- a/modules/core/src/test/java/org/gridgain/grid/spi/loadbalancing/adaptive/GridAdaptiveLoadBalancingSpiSelfTest.java
+++ /dev/null
@@ -1,125 +0,0 @@
-/* @java.file.header */
-
-/*  _________        _____ __________________        _____
- *  __  ____/___________(_)______  /__  ____/______ ____(_)_______
- *  _  / __  __  ___/__  / _  __  / _  / __  _  __ `/__  / __  __ \
- *  / /_/ /  _  /    _  /  / /_/ /  / /_/ /  / /_/ / _  /  _  / / /
- *  \____/   /_/     /_/   \_,__/   \____/   \__,_/  /_/   /_/ /_/
- */
-
-package org.gridgain.grid.spi.loadbalancing.adaptive;
-
-import org.apache.ignite.cluster.*;
-import org.apache.ignite.compute.*;
-import org.apache.ignite.lang.*;
-import org.gridgain.grid.*;
-import org.gridgain.testframework.*;
-import org.gridgain.testframework.junits.spi.*;
-
-import java.util.*;
-
-/**
- * Tests adaptive load balancing SPI.
- */
-@GridSpiTest(spi = AdaptiveLoadBalancingSpi.class, group = "Load Balancing SPI")
-public class GridAdaptiveLoadBalancingSpiSelfTest extends GridSpiAbstractTest<AdaptiveLoadBalancingSpi> {
-    /** {@inheritDoc} */
-    @Override protected GridSpiTestContext initSpiContext() throws Exception {
-        GridSpiTestContext ctx = super.initSpiContext();
-
-        ctx.setLocalNode(new GridTestNode(UUID.randomUUID()));
-
-        return ctx;
-    }
-
-    /**
-     * @return {@code True} if node weights should be considered.
-     */
-    @GridSpiTestConfig
-    public AdaptiveLoadProbe getLoadProbe() {
-        return new AdaptiveLoadProbe() {
-            @Override public double getLoad(ClusterNode node, int jobsSentSinceLastUpdate) {
-                boolean isFirstTime = node.attribute("used") == null;
-
-                assert isFirstTime ? jobsSentSinceLastUpdate == 0 : jobsSentSinceLastUpdate > 0;
-
-                return (Double)node.attribute("load");
-            }
-        };
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    @SuppressWarnings({"ObjectEquality"})
-    public void testSingleNodeZeroWeight() throws Exception {
-        GridTestNode node = (GridTestNode)getSpiContext().nodes().iterator().next();
-
-        node.addAttribute("load", 0d);
-
-        List<ClusterNode> nodes = Collections.singletonList((ClusterNode)node);
-
-        ComputeTaskSession ses = new GridTestTaskSession(IgniteUuid.randomUuid());
-
-        GridTestNode pick1 = (GridTestNode)getSpi().getBalancedNode(ses, nodes, new GridTestJob());
-
-        pick1.setAttribute("used", true);
-
-        assert nodes.contains(pick1);
-
-        // Verify that same instance is returned every time.
-        ClusterNode pick2 = getSpi().getBalancedNode(ses, nodes, new GridTestJob());
-
-        assert pick1 == pick2;
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    @SuppressWarnings({"ObjectEquality"})
-    public void testSingleNodeSameSession() throws Exception {
-        GridTestNode node = (GridTestNode)getSpiContext().nodes().iterator().next();
-
-        node.addAttribute("load", 1d);
-
-        List<ClusterNode> nodes = Collections.singletonList((ClusterNode)node);
-
-        ComputeTaskSession ses = new GridTestTaskSession(IgniteUuid.randomUuid());
-
-        GridTestNode pick1 = (GridTestNode)getSpi().getBalancedNode(ses, nodes, new GridTestJob());
-
-        pick1.setAttribute("used", true);
-
-        assert nodes.contains(pick1);
-
-        // Verify that same instance is returned every time.
-        ClusterNode pick2 = getSpi().getBalancedNode(ses, nodes, new GridTestJob());
-
-        assert pick1 == pick2;
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    @SuppressWarnings({"ObjectEquality"})
-    public void testSingleNodeDifferentSession() throws Exception {
-        GridTestNode node = (GridTestNode)getSpiContext().nodes().iterator().next();
-
-        node.addAttribute("load", 2d);
-
-        List<ClusterNode> nodes = Collections.singletonList((ClusterNode)node);
-
-        GridTestNode pick1 = (GridTestNode)getSpi().getBalancedNode(new GridTestTaskSession(IgniteUuid.randomUuid()),
-            nodes, new GridTestJob());
-
-        pick1.setAttribute("used", true);
-
-        assert nodes.contains(pick1);
-
-        // Verify that same instance is returned every time.
-        ClusterNode pick2 = getSpi().getBalancedNode(new GridTestTaskSession(IgniteUuid.randomUuid()), nodes,
-            new GridTestJob());
-
-        assert pick1 == pick2;
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/b69a23cd/modules/core/src/test/java/org/gridgain/grid/spi/loadbalancing/adaptive/GridAdaptiveLoadBalancingSpiStartStopSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/gridgain/grid/spi/loadbalancing/adaptive/GridAdaptiveLoadBalancingSpiStartStopSelfTest.java b/modules/core/src/test/java/org/gridgain/grid/spi/loadbalancing/adaptive/GridAdaptiveLoadBalancingSpiStartStopSelfTest.java
deleted file mode 100644
index 19876e7..0000000
--- a/modules/core/src/test/java/org/gridgain/grid/spi/loadbalancing/adaptive/GridAdaptiveLoadBalancingSpiStartStopSelfTest.java
+++ /dev/null
@@ -1,23 +0,0 @@
-/* @java.file.header */
-
-/*  _________        _____ __________________        _____
- *  __  ____/___________(_)______  /__  ____/______ ____(_)_______
- *  _  / __  __  ___/__  / _  __  / _  / __  _  __ `/__  / __  __ \
- *  / /_/ /  _  /    _  /  / /_/ /  / /_/ /  / /_/ / _  /  _  / / /
- *  \____/   /_/     /_/   \_,__/   \____/   \__,_/  /_/   /_/ /_/
- */
-
-package org.gridgain.grid.spi.loadbalancing.adaptive;
-
-import org.gridgain.grid.spi.*;
-import org.gridgain.testframework.junits.spi.*;
-
-/**
- * Adaptive load balancing SPI start-stop test.
- */
-@SuppressWarnings({"JUnitTestCaseWithNoTests"})
-@GridSpiTest(spi = AdaptiveLoadBalancingSpi.class, group = "LoadBalancing SPI")
-public class GridAdaptiveLoadBalancingSpiStartStopSelfTest extends
-    GridSpiStartStopAbstractTest<AdaptiveLoadBalancingSpi> {
-    // No configs.
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/b69a23cd/modules/core/src/test/java/org/gridgain/grid/spi/loadbalancing/adaptive/package.html
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/gridgain/grid/spi/loadbalancing/adaptive/package.html b/modules/core/src/test/java/org/gridgain/grid/spi/loadbalancing/adaptive/package.html
deleted file mode 100644
index 5cad80a..0000000
--- a/modules/core/src/test/java/org/gridgain/grid/spi/loadbalancing/adaptive/package.html
+++ /dev/null
@@ -1,15 +0,0 @@
-<!DOCTYPE html PUBLIC "-//W3C//DTD HTML 4.01 Transitional//EN" "http://www.w3.org/TR/html4/loose.dtd">
-<!--
-    @html.file.header
-    _________        _____ __________________        _____
-    __  ____/___________(_)______  /__  ____/______ ____(_)_______
-    _  / __  __  ___/__  / _  __  / _  / __  _  __ `/__  / __  __ \
-    / /_/ /  _  /    _  /  / /_/ /  / /_/ /  / /_/ / _  /  _  / / /
-    \____/   /_/     /_/   \_,__/   \____/   \__,_/  /_/   /_/ /_/
--->
-<html>
-<body>
-    <!-- Package description. -->
-    Contains internal tests or test related classes and interfaces.
-</body>
-</html>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/b69a23cd/modules/core/src/test/java/org/gridgain/grid/spi/loadbalancing/package.html
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/gridgain/grid/spi/loadbalancing/package.html b/modules/core/src/test/java/org/gridgain/grid/spi/loadbalancing/package.html
deleted file mode 100644
index 5cad80a..0000000
--- a/modules/core/src/test/java/org/gridgain/grid/spi/loadbalancing/package.html
+++ /dev/null
@@ -1,15 +0,0 @@
-<!DOCTYPE html PUBLIC "-//W3C//DTD HTML 4.01 Transitional//EN" "http://www.w3.org/TR/html4/loose.dtd">
-<!--
-    @html.file.header
-    _________        _____ __________________        _____
-    __  ____/___________(_)______  /__  ____/______ ____(_)_______
-    _  / __  __  ___/__  / _  __  / _  / __  _  __ `/__  / __  __ \
-    / /_/ /  _  /    _  /  / /_/ /  / /_/ /  / /_/ / _  /  _  / / /
-    \____/   /_/     /_/   \_,__/   \____/   \__,_/  /_/   /_/ /_/
--->
-<html>
-<body>
-    <!-- Package description. -->
-    Contains internal tests or test related classes and interfaces.
-</body>
-</html>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/b69a23cd/modules/core/src/test/java/org/gridgain/grid/spi/loadbalancing/roundrobin/GridRoundRobinLoadBalancingNotPerTaskMultithreadedSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/gridgain/grid/spi/loadbalancing/roundrobin/GridRoundRobinLoadBalancingNotPerTaskMultithreadedSelfTest.java b/modules/core/src/test/java/org/gridgain/grid/spi/loadbalancing/roundrobin/GridRoundRobinLoadBalancingNotPerTaskMultithreadedSelfTest.java
deleted file mode 100644
index ff420e7..0000000
--- a/modules/core/src/test/java/org/gridgain/grid/spi/loadbalancing/roundrobin/GridRoundRobinLoadBalancingNotPerTaskMultithreadedSelfTest.java
+++ /dev/null
@@ -1,115 +0,0 @@
-/* @java.file.header */
-
-/*  _________        _____ __________________        _____
- *  __  ____/___________(_)______  /__  ____/______ ____(_)_______
- *  _  / __  __  ___/__  / _  __  / _  / __  _  __ `/__  / __  __ \
- *  / /_/ /  _  /    _  /  / /_/ /  / /_/ /  / /_/ / _  /  _  / / /
- *  \____/   /_/     /_/   \_,__/   \____/   \__,_/  /_/   /_/ /_/
- */
-
-package org.gridgain.grid.spi.loadbalancing.roundrobin;
-
-import org.apache.ignite.cluster.*;
-import org.apache.ignite.compute.*;
-import org.apache.ignite.lang.*;
-import org.gridgain.grid.*;
-import org.gridgain.testframework.*;
-import org.gridgain.testframework.junits.spi.*;
-
-import java.util.*;
-import java.util.concurrent.*;
-import java.util.concurrent.atomic.*;
-
-/**
- * Multithreaded tests for global load balancer.
- */
-@GridSpiTest(spi = RoundRobinLoadBalancingSpi.class, group = "Load Balancing SPI")
-public class GridRoundRobinLoadBalancingNotPerTaskMultithreadedSelfTest
-    extends GridSpiAbstractTest<RoundRobinLoadBalancingSpi> {
-    /** Thread count. */
-    public static final int THREAD_CNT = 8;
-
-    /** Per-thread iteration count. */
-    public static final int ITER_CNT = 4_000_000;
-
-    /**
-     * @return Per-task configuration parameter.
-     */
-    @GridSpiTestConfig
-    public boolean getPerTask() {
-        return false;
-    }
-
-    /** {@inheritDoc} */
-    @Override protected GridSpiTestContext initSpiContext() throws Exception {
-        GridSpiTestContext spiCtx = super.initSpiContext();
-
-        spiCtx.createLocalNode();
-        spiCtx.createRemoteNodes(10);
-
-        return spiCtx;
-    }
-
-    /** {@inheritDoc} */
-    @Override protected void beforeTest() throws Exception {
-        assert !getSpi().isPerTask() : "Invalid SPI configuration.";
-    }
-
-    /**
-     *
-     * @throws Exception If failed.
-     */
-    public void testMultipleTaskSessionsMultithreaded() throws Exception {
-        final RoundRobinLoadBalancingSpi spi = getSpi();
-
-        final List<ClusterNode> allNodes = (List<ClusterNode>)getSpiContext().nodes();
-
-        GridTestUtils.runMultiThreaded(new Callable<Object>() {
-            @Override public Object call() throws Exception {
-                ComputeTaskSession ses = new GridTestTaskSession(IgniteUuid.randomUuid());
-
-                Map<UUID, AtomicInteger> nodeCnts = new HashMap<>();
-
-                for (int i = 1; i <= ITER_CNT; i++) {
-                    ClusterNode node = spi.getBalancedNode(ses, allNodes, new GridTestJob());
-
-                    if (!nodeCnts.containsKey(node.id()))
-                        nodeCnts.put(node.id(), new AtomicInteger(1));
-                    else
-                        nodeCnts.get(node.id()).incrementAndGet();
-                }
-
-                int predictCnt = ITER_CNT / allNodes.size();
-
-                // Consider +-20% is permissible spread for single node measure.
-                int floor = (int)(predictCnt * 0.8);
-
-                double avgSpread = 0;
-
-                for (ClusterNode n : allNodes) {
-                    int curCnt = nodeCnts.get(n.id()).intValue();
-
-                    avgSpread += Math.abs(predictCnt - curCnt);
-
-                    String msg = "Node stats [id=" + n.id() + ", cnt=" + curCnt + ", floor=" + floor +
-                        ", predictCnt=" + predictCnt + ']';
-
-                    info(msg);
-
-                    assertTrue(msg, curCnt >= floor);
-                }
-
-                avgSpread /= allNodes.size();
-
-                avgSpread = 100.0 * avgSpread / predictCnt;
-
-                info("Average spread for " + allNodes.size() + " nodes is " + avgSpread + " percents");
-
-                // Consider +-10% is permissible average spread for all nodes.
-                assertTrue("Average spread is too big: " + avgSpread, avgSpread <= 10);
-
-                return null;
-            }
-        }, THREAD_CNT, "balancer-test-worker");
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/b69a23cd/modules/core/src/test/java/org/gridgain/grid/spi/loadbalancing/roundrobin/GridRoundRobinLoadBalancingSpiLocalNodeSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/gridgain/grid/spi/loadbalancing/roundrobin/GridRoundRobinLoadBalancingSpiLocalNodeSelfTest.java b/modules/core/src/test/java/org/gridgain/grid/spi/loadbalancing/roundrobin/GridRoundRobinLoadBalancingSpiLocalNodeSelfTest.java
deleted file mode 100644
index ab992cf..0000000
--- a/modules/core/src/test/java/org/gridgain/grid/spi/loadbalancing/roundrobin/GridRoundRobinLoadBalancingSpiLocalNodeSelfTest.java
+++ /dev/null
@@ -1,44 +0,0 @@
-/* @java.file.header */
-
-/*  _________        _____ __________________        _____
- *  __  ____/___________(_)______  /__  ____/______ ____(_)_______
- *  _  / __  __  ___/__  / _  __  / _  / __  _  __ `/__  / __  __ \
- *  / /_/ /  _  /    _  /  / /_/ /  / /_/ /  / /_/ / _  /  _  / / /
- *  \____/   /_/     /_/   \_,__/   \____/   \__,_/  /_/   /_/ /_/
- */
-
-package org.gridgain.grid.spi.loadbalancing.roundrobin;
-
-import org.apache.ignite.cluster.*;
-import org.apache.ignite.lang.*;
-import org.gridgain.grid.*;
-import org.gridgain.testframework.junits.spi.*;
-import java.util.*;
-
-/**
- * Tests Round Robin load balancing for single node.
- */
-@GridSpiTest(spi = RoundRobinLoadBalancingSpi.class, group = "Load Balancing SPI", triggerDiscovery = true)
-public class GridRoundRobinLoadBalancingSpiLocalNodeSelfTest extends
-    GridSpiAbstractTest<RoundRobinLoadBalancingSpi> {
-    /**
-     * @throws Exception If failed.
-     */
-    @SuppressWarnings({"ObjectEquality"})
-    public void testLocalNode() throws Exception {
-        assert getDiscoverySpi().getRemoteNodes().isEmpty();
-
-        ClusterNode locNode = getDiscoverySpi().getLocalNode();
-
-        ClusterNode node = getSpi().getBalancedNode(new GridTestTaskSession(IgniteUuid.randomUuid()),
-            Collections.singletonList(locNode), new GridTestJob());
-
-        assert  node == locNode;
-
-        // Double check.
-        node = getSpi().getBalancedNode(new GridTestTaskSession(IgniteUuid.randomUuid()),
-            Collections.singletonList(locNode), new GridTestJob());
-
-        assert node == locNode;
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/b69a23cd/modules/core/src/test/java/org/gridgain/grid/spi/loadbalancing/roundrobin/GridRoundRobinLoadBalancingSpiMultipleNodesSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/gridgain/grid/spi/loadbalancing/roundrobin/GridRoundRobinLoadBalancingSpiMultipleNodesSelfTest.java b/modules/core/src/test/java/org/gridgain/grid/spi/loadbalancing/roundrobin/GridRoundRobinLoadBalancingSpiMultipleNodesSelfTest.java
deleted file mode 100644
index 0ab487f..0000000
--- a/modules/core/src/test/java/org/gridgain/grid/spi/loadbalancing/roundrobin/GridRoundRobinLoadBalancingSpiMultipleNodesSelfTest.java
+++ /dev/null
@@ -1,126 +0,0 @@
-/* @java.file.header */
-
-/*  _________        _____ __________________        _____
- *  __  ____/___________(_)______  /__  ____/______ ____(_)_______
- *  _  / __  __  ___/__  / _  __  / _  / __  _  __ `/__  / __  __ \
- *  / /_/ /  _  /    _  /  / /_/ /  / /_/ /  / /_/ / _  /  _  / / /
- *  \____/   /_/     /_/   \_,__/   \____/   \__,_/  /_/   /_/ /_/
- */
-
-package org.gridgain.grid.spi.loadbalancing.roundrobin;
-
-import org.apache.ignite.cluster.*;
-import org.apache.ignite.compute.*;
-import org.apache.ignite.events.*;
-import org.apache.ignite.lang.*;
-import org.gridgain.grid.*;
-import org.gridgain.testframework.*;
-import org.gridgain.testframework.junits.spi.*;
-
-import java.util.*;
-
-import static org.apache.ignite.events.IgniteEventType.*;
-
-/**
- * Tests round robin load balancing SPI.
- */
-@GridSpiTest(spi = RoundRobinLoadBalancingSpi.class, group = "Load Balancing SPI")
-public class GridRoundRobinLoadBalancingSpiMultipleNodesSelfTest
-    extends GridSpiAbstractTest<RoundRobinLoadBalancingSpi> {
-    /** {@inheritDoc} */
-    @Override protected void spiConfigure(RoundRobinLoadBalancingSpi spi) throws Exception {
-        super.spiConfigure(spi);
-
-        spi.setPerTask(true);
-    }
-
-    /** {@inheritDoc} */
-    @Override protected GridSpiTestContext initSpiContext() throws Exception {
-        GridSpiTestContext spiCtx = super.initSpiContext();
-
-        spiCtx.createLocalNode();
-        spiCtx.createRemoteNodes(10);
-
-        return spiCtx;
-    }
-
-    /**
-     * @throws Exception If test failed.
-     */
-    @SuppressWarnings({"ObjectEquality"})
-    public void testMultipleNodes() throws Exception {
-        List<ClusterNode> allNodes = (List<ClusterNode>)getSpiContext().nodes();
-
-        ComputeTaskSession ses = new GridTestTaskSession(IgniteUuid.randomUuid());
-
-        // Initialize.
-        getSpi().getBalancedNode(ses, allNodes, new GridTestJob());
-
-        List<UUID> orderedNodes = new ArrayList<>(getSpi().getNodeIds(ses));
-
-        // Check the round-robin actually did circle.
-        for (int i = 0; i < allNodes.size(); i++) {
-            ClusterNode node = getSpi().getBalancedNode(ses, allNodes, new GridTestJob());
-
-            assert orderedNodes.get(i) == node.id();
-        }
-
-        // Double-check.
-        for (int i = 0; i < allNodes.size(); i++) {
-            ClusterNode node = getSpi().getBalancedNode(ses, allNodes, new GridTestJob());
-
-            assert orderedNodes.get(i) == node.id();
-        }
-    }
-
-    /**
-     * @throws Exception If test failed.
-     */
-    @SuppressWarnings({"ObjectEquality"})
-    public void testMultipleTasks() throws Exception {
-        ComputeTaskSession ses1 = new GridTestTaskSession(IgniteUuid.randomUuid());
-        ComputeTaskSession ses2 = new GridTestTaskSession(IgniteUuid.randomUuid());
-
-        List<ClusterNode> allNodes = (List<ClusterNode>)getSpiContext().nodes();
-
-        // Initialize.
-        getSpi().getBalancedNode(ses1, allNodes, new GridTestJob());
-        getSpi().getBalancedNode(ses2, allNodes, new GridTestJob());
-
-        List<UUID> orderedNodes1 = getSpi().getNodeIds(ses1);
-        List<UUID> orderedNodes2 = getSpi().getNodeIds(ses2);
-
-        assert orderedNodes1 != orderedNodes2;
-
-        // Check the round-robin actually did circle.
-        for (int i = 0; i < allNodes.size(); i++) {
-            ClusterNode node1 = getSpi().getBalancedNode(ses1, allNodes, new GridTestJob());
-
-            assert orderedNodes1.get(i) == node1.id();
-
-            ClusterNode node2 = getSpi().getBalancedNode(ses2, allNodes, new GridTestJob());
-
-            assert orderedNodes2.get(i) == node2.id();
-
-            assert orderedNodes1.get(i) == orderedNodes2.get(i);
-        }
-
-        // Double-check.
-        for (int i = 0; i < allNodes.size(); i++) {
-            ClusterNode node1 = getSpi().getBalancedNode(ses1, allNodes, new GridTestJob());
-
-            assert orderedNodes1.get(i) == node1.id();
-
-            ClusterNode node2 = getSpi().getBalancedNode(ses2, allNodes, new GridTestJob());
-
-            assert orderedNodes2.get(i) == node2.id();
-
-            assert orderedNodes1.get(i) == orderedNodes2.get(i);
-        }
-
-        getSpiContext().triggerEvent(new IgniteTaskEvent(
-            null, null, EVT_TASK_FINISHED, ses1.getId(), null, null, false, null));
-        getSpiContext().triggerEvent(new IgniteTaskEvent(
-            null, null, EVT_TASK_FAILED, ses2.getId(), null, null, false, null));
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/b69a23cd/modules/core/src/test/java/org/gridgain/grid/spi/loadbalancing/roundrobin/GridRoundRobinLoadBalancingSpiNotPerTaskSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/gridgain/grid/spi/loadbalancing/roundrobin/GridRoundRobinLoadBalancingSpiNotPerTaskSelfTest.java b/modules/core/src/test/java/org/gridgain/grid/spi/loadbalancing/roundrobin/GridRoundRobinLoadBalancingSpiNotPerTaskSelfTest.java
deleted file mode 100644
index 2d11283..0000000
--- a/modules/core/src/test/java/org/gridgain/grid/spi/loadbalancing/roundrobin/GridRoundRobinLoadBalancingSpiNotPerTaskSelfTest.java
+++ /dev/null
@@ -1,121 +0,0 @@
-/* @java.file.header */
-
-/*  _________        _____ __________________        _____
- *  __  ____/___________(_)______  /__  ____/______ ____(_)_______
- *  _  / __  __  ___/__  / _  __  / _  / __  _  __ `/__  / __  __ \
- *  / /_/ /  _  /    _  /  / /_/ /  / /_/ /  / /_/ / _  /  _  / / /
- *  \____/   /_/     /_/   \_,__/   \____/   \__,_/  /_/   /_/ /_/
- */
-
-package org.gridgain.grid.spi.loadbalancing.roundrobin;
-
-import org.apache.ignite.cluster.*;
-import org.apache.ignite.compute.*;
-import org.apache.ignite.events.*;
-import org.apache.ignite.lang.*;
-import org.gridgain.grid.*;
-import org.gridgain.testframework.*;
-import org.gridgain.testframework.junits.spi.*;
-
-import java.util.*;
-
-import static org.apache.ignite.events.IgniteEventType.*;
-import static org.gridgain.grid.spi.loadbalancing.roundrobin.GridRoundRobinTestUtils.*;
-
-/**
- * Tests round robin load balancing.
- */
-@GridSpiTest(spi = RoundRobinLoadBalancingSpi.class, group = "Load Balancing SPI")
-public class GridRoundRobinLoadBalancingSpiNotPerTaskSelfTest
-    extends GridSpiAbstractTest<RoundRobinLoadBalancingSpi> {
-    /**
-     * @return Per-task configuration parameter.
-     */
-    @GridSpiTestConfig
-    public boolean getPerTask() {
-        return false;
-    }
-
-    /** {@inheritDoc} */
-    @Override protected GridSpiTestContext initSpiContext() throws Exception {
-        GridSpiTestContext spiCtx = super.initSpiContext();
-
-        spiCtx.createLocalNode();
-        spiCtx.createRemoteNodes(10);
-
-        return spiCtx;
-    }
-
-    /** {@inheritDoc} */
-    @Override protected void beforeTest() throws Exception {
-        assert !getSpi().isPerTask() : "Invalid SPI configuration.";
-    }
-
-    /**
-     * @throws Exception If test failed.
-     */
-    public void testMultipleNodes() throws Exception {
-        List<ClusterNode> allNodes = (List<ClusterNode>)getSpiContext().nodes();
-
-        ComputeTaskSession ses = new GridTestTaskSession();
-
-        List<UUID> orderedNodes = new ArrayList<>(getSpi().getNodeIds(ses));
-
-        assertEquals("Balancer doesn't use all available nodes", orderedNodes.size(), allNodes.size());
-
-        checkCyclicBalancing(getSpi(), allNodes, orderedNodes, ses);
-    }
-
-    /**
-     * @throws Exception If test failed.
-     */
-    public void testMultipleTaskSessions() throws Exception {
-        ComputeTaskSession ses1 = new GridTestTaskSession(IgniteUuid.randomUuid());
-        ComputeTaskSession ses2 = new GridTestTaskSession(IgniteUuid.randomUuid());
-
-        List<ClusterNode> allNodes = (List<ClusterNode>)getSpiContext().nodes();
-
-        List<UUID> orderedNodes = getSpi().getNodeIds(ses1);
-
-        assertEquals("Balancer doesn't use all available nodes", orderedNodes.size(), allNodes.size());
-
-        checkCyclicBalancing(getSpi(), allNodes, orderedNodes, ses1, ses2);
-
-        getSpiContext().triggerEvent(new IgniteTaskEvent(
-            null, null, EVT_TASK_FINISHED, ses1.getId(), null, null, false, null));
-        getSpiContext().triggerEvent(new IgniteTaskEvent(
-            null, null, EVT_TASK_FAILED, ses2.getId(), null, null, false, null));
-    }
-
-    /**
-     * @throws Exception If test failed.
-     */
-    public void testBalancingOneNode() throws Exception {
-        ComputeTaskSession ses = new GridTestTaskSession();
-
-        List<ClusterNode> allNodes = (List<ClusterNode>)getSpiContext().nodes();
-
-        List<ClusterNode> balancedNode = Arrays.asList(allNodes.get(0));
-
-        ClusterNode firstNode = getSpi().getBalancedNode(ses, balancedNode, new GridTestJob());
-        ClusterNode secondNode = getSpi().getBalancedNode(ses, balancedNode, new GridTestJob());
-
-        assertEquals(firstNode, secondNode);
-    }
-
-    /** */
-    public void testNodeNotInTopology() {
-        ComputeTaskSession ses = new GridTestTaskSession();
-
-        ClusterNode node = new GridTestNode(UUID.randomUUID());
-
-        List<ClusterNode> notInTop = Arrays.asList(node);
-
-        try {
-            getSpi().getBalancedNode(ses, notInTop, new GridTestJob());
-        }
-        catch (GridException e) {
-            assertTrue(e.getMessage().contains("Task topology does not have alive nodes"));
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/b69a23cd/modules/core/src/test/java/org/gridgain/grid/spi/loadbalancing/roundrobin/GridRoundRobinLoadBalancingSpiStartStopSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/gridgain/grid/spi/loadbalancing/roundrobin/GridRoundRobinLoadBalancingSpiStartStopSelfTest.java b/modules/core/src/test/java/org/gridgain/grid/spi/loadbalancing/roundrobin/GridRoundRobinLoadBalancingSpiStartStopSelfTest.java
deleted file mode 100644
index bc8baf8..0000000
--- a/modules/core/src/test/java/org/gridgain/grid/spi/loadbalancing/roundrobin/GridRoundRobinLoadBalancingSpiStartStopSelfTest.java
+++ /dev/null
@@ -1,23 +0,0 @@
-/* @java.file.header */
-
-/*  _________        _____ __________________        _____
- *  __  ____/___________(_)______  /__  ____/______ ____(_)_______
- *  _  / __  __  ___/__  / _  __  / _  / __  _  __ `/__  / __  __ \
- *  / /_/ /  _  /    _  /  / /_/ /  / /_/ /  / /_/ / _  /  _  / / /
- *  \____/   /_/     /_/   \_,__/   \____/   \__,_/  /_/   /_/ /_/
- */
-
-package org.gridgain.grid.spi.loadbalancing.roundrobin;
-
-import org.gridgain.grid.spi.*;
-import org.gridgain.testframework.junits.spi.*;
-
-/**
- * Tests correct start of {@link RoundRobinLoadBalancingSpi}.
- */
-@SuppressWarnings({"JUnitTestCaseWithNoTests"})
-@GridSpiTest(spi = RoundRobinLoadBalancingSpi.class, group = "LoadBalancing SPI")
-public class GridRoundRobinLoadBalancingSpiStartStopSelfTest
-    extends GridSpiStartStopAbstractTest<RoundRobinLoadBalancingSpi> {
-    // No configs.
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/b69a23cd/modules/core/src/test/java/org/gridgain/grid/spi/loadbalancing/roundrobin/GridRoundRobinLoadBalancingSpiTopologyChangeSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/gridgain/grid/spi/loadbalancing/roundrobin/GridRoundRobinLoadBalancingSpiTopologyChangeSelfTest.java b/modules/core/src/test/java/org/gridgain/grid/spi/loadbalancing/roundrobin/GridRoundRobinLoadBalancingSpiTopologyChangeSelfTest.java
deleted file mode 100644
index 54d9d49..0000000
--- a/modules/core/src/test/java/org/gridgain/grid/spi/loadbalancing/roundrobin/GridRoundRobinLoadBalancingSpiTopologyChangeSelfTest.java
+++ /dev/null
@@ -1,98 +0,0 @@
-/* @java.file.header */
-
-/*  _________        _____ __________________        _____
- *  __  ____/___________(_)______  /__  ____/______ ____(_)_______
- *  _  / __  __  ___/__  / _  __  / _  / __  _  __ `/__  / __  __ \
- *  / /_/ /  _  /    _  /  / /_/ /  / /_/ /  / /_/ / _  /  _  / / /
- *  \____/   /_/     /_/   \_,__/   \____/   \__,_/  /_/   /_/ /_/
- */
-
-package org.gridgain.grid.spi.loadbalancing.roundrobin;
-
-import org.apache.ignite.cluster.*;
-import org.apache.ignite.compute.*;
-import org.apache.ignite.lang.*;
-import org.gridgain.grid.*;
-import org.gridgain.testframework.*;
-import org.gridgain.testframework.junits.spi.*;
-
-import java.util.*;
-
-import static org.gridgain.grid.spi.loadbalancing.roundrobin.GridRoundRobinTestUtils.*;
-
-/**
- * Tests round robin load balancing with topology changes.
- */
-@GridSpiTest(spi = RoundRobinLoadBalancingSpi.class, group = "Load Balancing SPI")
-public class GridRoundRobinLoadBalancingSpiTopologyChangeSelfTest
-    extends GridSpiAbstractTest<RoundRobinLoadBalancingSpi> {
-    /**
-     * @return Per-task configuration parameter.
-     */
-    @GridSpiTestConfig
-    public boolean getPerTask() { return false; }
-
-    /** {@inheritDoc} */
-    @Override protected GridSpiTestContext initSpiContext() throws Exception {
-        GridSpiTestContext spiCtx = super.initSpiContext();
-
-        spiCtx.createLocalNode();
-        spiCtx.createRemoteNodes(10);
-
-        return spiCtx;
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testTopologyChange() throws Exception {
-        ComputeTaskSession ses = new GridTestTaskSession(IgniteUuid.randomUuid());
-
-        // Warm up.
-        List<ClusterNode> allNodes = (List<ClusterNode>)getSpiContext().nodes();
-
-        List<UUID> orderedNodes = getSpi().getNodeIds(ses);
-
-        checkCyclicBalancing(getSpi(), allNodes, orderedNodes, ses);
-
-        // Remove node.
-        UUID doomed = orderedNodes.get(0);
-
-        if (getSpiContext().localNode().id().equals(doomed))
-            doomed = orderedNodes.get(1);
-
-        getSpiContext().removeNode(doomed);
-
-        assertTrue(allNodes.remove(new GridTestNode(doomed)));
-
-        orderedNodes = getSpi().getNodeIds(ses);
-
-        assertFalse("Balancer uses removed node", orderedNodes.contains(doomed));
-
-        checkCyclicBalancing(getSpi(), allNodes, orderedNodes, ses);
-
-        // Add node.
-        ClusterNode newNode = new GridTestNode(UUID.randomUUID());
-
-        getSpiContext().addNode(newNode);
-
-        assertTrue(allNodes.add(newNode));
-
-        // Check that new node was added to balancing.
-        boolean foundNewNode = false;
-
-        for (int i = 0; i < allNodes.size(); i++) {
-            ClusterNode node = getSpi().getBalancedNode(ses, allNodes, new GridTestJob());
-            if (newNode.id().equals(node.id())) {
-                foundNewNode = true;
-                break;
-            }
-        }
-
-        assertTrue("Balancer doesn't use added node", foundNewNode);
-
-        orderedNodes = getSpi().getNodeIds(ses);
-
-        checkCyclicBalancing(getSpi(), allNodes, orderedNodes, ses);
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/b69a23cd/modules/core/src/test/java/org/gridgain/grid/spi/loadbalancing/roundrobin/GridRoundRobinTestUtils.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/gridgain/grid/spi/loadbalancing/roundrobin/GridRoundRobinTestUtils.java b/modules/core/src/test/java/org/gridgain/grid/spi/loadbalancing/roundrobin/GridRoundRobinTestUtils.java
deleted file mode 100644
index 71b7744..0000000
--- a/modules/core/src/test/java/org/gridgain/grid/spi/loadbalancing/roundrobin/GridRoundRobinTestUtils.java
+++ /dev/null
@@ -1,95 +0,0 @@
-/* @java.file.header */
-
-/*  _________        _____ __________________        _____
- *  __  ____/___________(_)______  /__  ____/______ ____(_)_______
- *  _  / __  __  ___/__  / _  __  / _  / __  _  __ `/__  / __  __ \
- *  / /_/ /  _  /    _  /  / /_/ /  / /_/ /  / /_/ / _  /  _  / / /
- *  \____/   /_/     /_/   \_,__/   \____/   \__,_/  /_/   /_/ /_/
- */
-
-package org.gridgain.grid.spi.loadbalancing.roundrobin;
-
-import org.apache.ignite.cluster.*;
-import org.apache.ignite.compute.*;
-import org.gridgain.grid.*;
-
-import java.util.*;
-
-import static org.junit.Assert.*;
-
-/**
- * Helper class for balancer tests.
- */
-class GridRoundRobinTestUtils {
-    /**
-     * Performs two full cycles by round robin routine for check correct order.
-     *
-     * @param spi Load balancing SPI.
-     * @param allNodes Topology nodes.
-     * @param orderedNodes Balancing nodes.
-     * @param ses Task session.
-     * @throws GridException If balancer failed.
-     */
-    static void checkCyclicBalancing(RoundRobinLoadBalancingSpi spi, List<ClusterNode> allNodes,
-        List<UUID> orderedNodes, ComputeTaskSession ses) throws GridException {
-
-        ClusterNode firstNode = spi.getBalancedNode(ses, allNodes, new GridTestJob());
-
-        int startIdx = firstBalancedNodeIndex(firstNode, orderedNodes);
-
-        // Two full cycles by round robin routine.
-        for (int i = 0; i < allNodes.size() * 2; i++) {
-            int actualIdx = (startIdx + i + 1) % allNodes.size();
-
-            ClusterNode nextNode = spi.getBalancedNode(ses, allNodes, new GridTestJob());
-
-            assertEquals("Balancer returns node out of order", nextNode.id(), orderedNodes.get(actualIdx));
-        }
-    }
-
-    /**
-     * Performs two full cycles by round robin routine for check correct order.
-     * Switches between two task sessions by turns.
-     *
-     * @param spi Load balancing SPI.
-     * @param allNodes Topology nodes.
-     * @param orderedNodes Balancing nodes.
-     * @param ses1 First task session.
-     * @param ses2 Second task session.
-     * @throws GridException If balancer failed.
-     */
-    static void checkCyclicBalancing(RoundRobinLoadBalancingSpi spi, List<ClusterNode> allNodes,
-        List<UUID> orderedNodes, ComputeTaskSession ses1, ComputeTaskSession ses2) throws GridException {
-
-        ClusterNode firstNode = spi.getBalancedNode(ses1, allNodes, new GridTestJob());
-
-        int startIdx = firstBalancedNodeIndex(firstNode, orderedNodes);
-
-        // Two full cycles by round robin routine.
-        for (int i = 0; i < allNodes.size() * 2; i++) {
-            int actualIdx = (startIdx + i + 1) % allNodes.size();
-
-            ClusterNode nextNode = spi.getBalancedNode(i % 2 == 0 ? ses1 : ses2, allNodes, new GridTestJob());
-
-            assertEquals("Balancer returns node out of order", nextNode.id(), orderedNodes.get(actualIdx));
-        }
-    }
-
-    /**
-     * @param firstNode First node which was return by balancer.
-     * @param orderedNodes Balancing nodes.
-     * @return Index of first node which was return by balancer.
-     */
-    static int firstBalancedNodeIndex(ClusterNode firstNode, List<UUID> orderedNodes) {
-        int startIdx = -1;
-
-        for (int i = 0; i < orderedNodes.size(); i++) {
-            if (firstNode.id() == orderedNodes.get(i))
-                startIdx = i;
-        }
-
-        assertTrue("Can't find position of first balanced node", startIdx >= 0);
-
-        return startIdx;
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/b69a23cd/modules/core/src/test/java/org/gridgain/grid/spi/loadbalancing/roundrobin/package.html
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/gridgain/grid/spi/loadbalancing/roundrobin/package.html b/modules/core/src/test/java/org/gridgain/grid/spi/loadbalancing/roundrobin/package.html
deleted file mode 100644
index 5cad80a..0000000
--- a/modules/core/src/test/java/org/gridgain/grid/spi/loadbalancing/roundrobin/package.html
+++ /dev/null
@@ -1,15 +0,0 @@
-<!DOCTYPE html PUBLIC "-//W3C//DTD HTML 4.01 Transitional//EN" "http://www.w3.org/TR/html4/loose.dtd">
-<!--
-    @html.file.header
-    _________        _____ __________________        _____
-    __  ____/___________(_)______  /__  ____/______ ____(_)_______
-    _  / __  __  ___/__  / _  __  / _  / __  _  __ `/__  / __  __ \
-    / /_/ /  _  /    _  /  / /_/ /  / /_/ /  / /_/ / _  /  _  / / /
-    \____/   /_/     /_/   \_,__/   \____/   \__,_/  /_/   /_/ /_/
--->
-<html>
-<body>
-    <!-- Package description. -->
-    Contains internal tests or test related classes and interfaces.
-</body>
-</html>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/b69a23cd/modules/core/src/test/java/org/gridgain/grid/spi/loadbalancing/weightedrandom/GridWeightedRandomLoadBalancingSpiConfigSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/gridgain/grid/spi/loadbalancing/weightedrandom/GridWeightedRandomLoadBalancingSpiConfigSelfTest.java b/modules/core/src/test/java/org/gridgain/grid/spi/loadbalancing/weightedrandom/GridWeightedRandomLoadBalancingSpiConfigSelfTest.java
deleted file mode 100644
index 40e24f3..0000000
--- a/modules/core/src/test/java/org/gridgain/grid/spi/loadbalancing/weightedrandom/GridWeightedRandomLoadBalancingSpiConfigSelfTest.java
+++ /dev/null
@@ -1,26 +0,0 @@
-/* @java.file.header */
-
-/*  _________        _____ __________________        _____
- *  __  ____/___________(_)______  /__  ____/______ ____(_)_______
- *  _  / __  __  ___/__  / _  __  / _  / __  _  __ `/__  / __  __ \
- *  / /_/ /  _  /    _  /  / /_/ /  / /_/ /  / /_/ / _  /  _  / / /
- *  \____/   /_/     /_/   \_,__/   \____/   \__,_/  /_/   /_/ /_/
- */
-
-package org.gridgain.grid.spi.loadbalancing.weightedrandom;
-
-import org.gridgain.testframework.junits.spi.*;
-
-/**
- *
- */
-@GridSpiTest(spi = WeightedRandomLoadBalancingSpi.class, group = "Load Balancing SPI")
-public class GridWeightedRandomLoadBalancingSpiConfigSelfTest extends
-    GridSpiAbstractConfigTest<WeightedRandomLoadBalancingSpi> {
-    /**
-     * @throws Exception If failed.
-     */
-    public void testNegativeConfig() throws Exception {
-        checkNegativeSpiProperty(new WeightedRandomLoadBalancingSpi(), "nodeWeight", 0);
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/b69a23cd/modules/core/src/test/java/org/gridgain/grid/spi/loadbalancing/weightedrandom/GridWeightedRandomLoadBalancingSpiSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/gridgain/grid/spi/loadbalancing/weightedrandom/GridWeightedRandomLoadBalancingSpiSelfTest.java b/modules/core/src/test/java/org/gridgain/grid/spi/loadbalancing/weightedrandom/GridWeightedRandomLoadBalancingSpiSelfTest.java
deleted file mode 100644
index efc21ed..0000000
--- a/modules/core/src/test/java/org/gridgain/grid/spi/loadbalancing/weightedrandom/GridWeightedRandomLoadBalancingSpiSelfTest.java
+++ /dev/null
@@ -1,58 +0,0 @@
-/* @java.file.header */
-
-/*  _________        _____ __________________        _____
- *  __  ____/___________(_)______  /__  ____/______ ____(_)_______
- *  _  / __  __  ___/__  / _  __  / _  / __  _  __ `/__  / __  __ \
- *  / /_/ /  _  /    _  /  / /_/ /  / /_/ /  / /_/ / _  /  _  / / /
- *  \____/   /_/     /_/   \_,__/   \____/   \__,_/  /_/   /_/ /_/
- */
-
-package org.gridgain.grid.spi.loadbalancing.weightedrandom;
-
-import org.apache.ignite.cluster.*;
-import org.gridgain.grid.*;
-import org.gridgain.testframework.*;
-import org.gridgain.testframework.junits.spi.*;
-import java.util.*;
-
-/**
- * Weighted random load balancing SPI.
- */
-@GridSpiTest(spi = WeightedRandomLoadBalancingSpi.class, group = "Load Balancing SPI")
-public class GridWeightedRandomLoadBalancingSpiSelfTest extends
-    GridSpiAbstractTest<WeightedRandomLoadBalancingSpi> {
-    /**
-     * @throws Exception If failed.
-     */
-    @SuppressWarnings({"ObjectEquality"})
-    public void testSingleNode() throws Exception {
-        List<ClusterNode> nodes = Collections.singletonList((ClusterNode)new GridTestNode(UUID.randomUUID()));
-
-        ClusterNode node = getSpi().getBalancedNode(new GridTestTaskSession(), nodes, new GridTestJob());
-
-        assert nodes.contains(node);
-
-        // Verify that same instance is returned every time.
-        ClusterNode balancedNode = getSpi().getBalancedNode(new GridTestTaskSession(), nodes, new GridTestJob());
-
-        assert node == balancedNode;
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testMultipleNodes() throws Exception {
-        List<ClusterNode> nodes = new ArrayList<>();
-
-        for (int i = 0; i < 10; i++)
-            nodes.add(new GridTestNode(UUID.randomUUID()));
-
-        // Seal it.
-        nodes = Collections.unmodifiableList(nodes);
-
-        ClusterNode node = getSpi().getBalancedNode(new GridTestTaskSession(), nodes, new GridTestJob());
-
-        assert node != null;
-        assert nodes.contains(node);
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/b69a23cd/modules/core/src/test/java/org/gridgain/grid/spi/loadbalancing/weightedrandom/GridWeightedRandomLoadBalancingSpiStartStopSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/gridgain/grid/spi/loadbalancing/weightedrandom/GridWeightedRandomLoadBalancingSpiStartStopSelfTest.java b/modules/core/src/test/java/org/gridgain/grid/spi/loadbalancing/weightedrandom/GridWeightedRandomLoadBalancingSpiStartStopSelfTest.java
deleted file mode 100644
index 4c9b352..0000000
--- a/modules/core/src/test/java/org/gridgain/grid/spi/loadbalancing/weightedrandom/GridWeightedRandomLoadBalancingSpiStartStopSelfTest.java
+++ /dev/null
@@ -1,23 +0,0 @@
-/* @java.file.header */
-
-/*  _________        _____ __________________        _____
- *  __  ____/___________(_)______  /__  ____/______ ____(_)_______
- *  _  / __  __  ___/__  / _  __  / _  / __  _  __ `/__  / __  __ \
- *  / /_/ /  _  /    _  /  / /_/ /  / /_/ /  / /_/ / _  /  _  / / /
- *  \____/   /_/     /_/   \_,__/   \____/   \__,_/  /_/   /_/ /_/
- */
-
-package org.gridgain.grid.spi.loadbalancing.weightedrandom;
-
-import org.gridgain.grid.spi.*;
-import org.gridgain.testframework.junits.spi.*;
-
-/**
- * Wighted random load balancing SPI start-stop test.
- */
-@SuppressWarnings({"JUnitTestCaseWithNoTests"})
-@GridSpiTest(spi = WeightedRandomLoadBalancingSpi.class, group = "LoadBalancing SPI")
-public class GridWeightedRandomLoadBalancingSpiStartStopSelfTest extends
-    GridSpiStartStopAbstractTest<WeightedRandomLoadBalancingSpi> {
-    // No configs.
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/b69a23cd/modules/core/src/test/java/org/gridgain/grid/spi/loadbalancing/weightedrandom/GridWeightedRandomLoadBalancingSpiWeightedSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/gridgain/grid/spi/loadbalancing/weightedrandom/GridWeightedRandomLoadBalancingSpiWeightedSelfTest.java b/modules/core/src/test/java/org/gridgain/grid/spi/loadbalancing/weightedrandom/GridWeightedRandomLoadBalancingSpiWeightedSelfTest.java
deleted file mode 100644
index 289be66..0000000
--- a/modules/core/src/test/java/org/gridgain/grid/spi/loadbalancing/weightedrandom/GridWeightedRandomLoadBalancingSpiWeightedSelfTest.java
+++ /dev/null
@@ -1,73 +0,0 @@
-/* @java.file.header */
-
-/*  _________        _____ __________________        _____
- *  __  ____/___________(_)______  /__  ____/______ ____(_)_______
- *  _  / __  __  ___/__  / _  __  / _  / __  _  __ `/__  / __  __ \
- *  / /_/ /  _  /    _  /  / /_/ /  / /_/ /  / /_/ / _  /  _  / / /
- *  \____/   /_/     /_/   \_,__/   \____/   \__,_/  /_/   /_/ /_/
- */
-
-package org.gridgain.grid.spi.loadbalancing.weightedrandom;
-
-import org.apache.ignite.cluster.*;
-import org.apache.ignite.lang.*;
-import org.gridgain.grid.*;
-import org.gridgain.grid.util.typedef.internal.*;
-import org.gridgain.testframework.*;
-import org.gridgain.testframework.junits.spi.*;
-import java.util.*;
-
-import static org.gridgain.grid.spi.loadbalancing.weightedrandom.WeightedRandomLoadBalancingSpi.*;
-
-/**
- * {@link WeightedRandomLoadBalancingSpi} self test.
- */
-@GridSpiTest(spi = WeightedRandomLoadBalancingSpi.class, group = "Load Balancing SPI")
-public class GridWeightedRandomLoadBalancingSpiWeightedSelfTest
-    extends GridSpiAbstractTest<WeightedRandomLoadBalancingSpi> {
-    /**
-     * @return {@code True} if node weights should be considered.
-     */
-    @GridSpiTestConfig
-    public boolean getUseWeights() {
-        return true;
-    }
-
-    /**
-     * @throws Exception If test failed.
-     */
-    public void testWeights() throws Exception {
-        List<ClusterNode> nodes = new ArrayList<>();
-
-        for (int i = 0; i < 10; i++) {
-            GridTestNode node = new GridTestNode(UUID.randomUUID());
-
-            node.addAttribute(U.spiAttribute(getSpi(), NODE_WEIGHT_ATTR_NAME), i + 1);
-
-            nodes.add(node);
-        }
-
-        // Seal it.
-        nodes = Collections.unmodifiableList(nodes);
-
-        int[] cnts = new int[10];
-
-        // Invoke load balancer a large number of times, so statistics won't lie.
-        for (int i = 0; i < 100000; i++) {
-            ClusterNode node = getSpi().getBalancedNode(new GridTestTaskSession(IgniteUuid.randomUuid()), nodes,
-                new GridTestJob());
-
-            int weight = (Integer)node.attribute(U.spiAttribute(getSpi(), NODE_WEIGHT_ATTR_NAME));
-
-            // Increment number of times a node was picked.
-            cnts[weight - 1]++;
-        }
-
-        for (int i = 0; i < cnts.length - 1; i++) {
-            assert cnts[i] < cnts[i + 1] : "Invalid node counts for index [idx=" + i + ", cnts[i]=" + cnts[i] +
-                ", cnts[i+1]=" + cnts[i + 1] + ']';
-        }
-
-        info("Node counts: " + Arrays.toString(cnts));
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/b69a23cd/modules/core/src/test/java/org/gridgain/grid/spi/loadbalancing/weightedrandom/package.html
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/gridgain/grid/spi/loadbalancing/weightedrandom/package.html b/modules/core/src/test/java/org/gridgain/grid/spi/loadbalancing/weightedrandom/package.html
deleted file mode 100644
index 5cad80a..0000000
--- a/modules/core/src/test/java/org/gridgain/grid/spi/loadbalancing/weightedrandom/package.html
+++ /dev/null
@@ -1,15 +0,0 @@
-<!DOCTYPE html PUBLIC "-//W3C//DTD HTML 4.01 Transitional//EN" "http://www.w3.org/TR/html4/loose.dtd">
-<!--
-    @html.file.header
-    _________        _____ __________________        _____
-    __  ____/___________(_)______  /__  ____/______ ____(_)_______
-    _  / __  __  ___/__  / _  __  / _  / __  _  __ `/__  / __  __ \
-    / /_/ /  _  /    _  /  / /_/ /  / /_/ /  / /_/ / _  /  _  / / /
-    \____/   /_/     /_/   \_,__/   \____/   \__,_/  /_/   /_/ /_/
--->
-<html>
-<body>
-    <!-- Package description. -->
-    Contains internal tests or test related classes and interfaces.
-</body>
-</html>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/b69a23cd/modules/core/src/test/java/org/gridgain/testsuites/GridSpiLoadBalancingSelfTestSuite.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/gridgain/testsuites/GridSpiLoadBalancingSelfTestSuite.java b/modules/core/src/test/java/org/gridgain/testsuites/GridSpiLoadBalancingSelfTestSuite.java
index 21f89fb..80c2bff 100644
--- a/modules/core/src/test/java/org/gridgain/testsuites/GridSpiLoadBalancingSelfTestSuite.java
+++ b/modules/core/src/test/java/org/gridgain/testsuites/GridSpiLoadBalancingSelfTestSuite.java
@@ -10,9 +10,9 @@
 package org.gridgain.testsuites;
 
 import junit.framework.*;
-import org.gridgain.grid.spi.loadbalancing.adaptive.*;
-import org.gridgain.grid.spi.loadbalancing.roundrobin.*;
-import org.gridgain.grid.spi.loadbalancing.weightedrandom.*;
+import org.apache.ignite.spi.loadbalancing.adaptive.*;
+import org.apache.ignite.spi.loadbalancing.roundrobin.*;
+import org.apache.ignite.spi.loadbalancing.weightedrandom.*;
 
 /**
  * Load balancing SPI self-test suite.


[16/20] incubator-ignite git commit: # Renaming

Posted by sb...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/ef258ece/modules/core/src/main/java/org/gridgain/grid/spi/swapspace/file/FileSwapSpaceSpi.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/spi/swapspace/file/FileSwapSpaceSpi.java b/modules/core/src/main/java/org/gridgain/grid/spi/swapspace/file/FileSwapSpaceSpi.java
deleted file mode 100644
index 7bb91d2..0000000
--- a/modules/core/src/main/java/org/gridgain/grid/spi/swapspace/file/FileSwapSpaceSpi.java
+++ /dev/null
@@ -1,1843 +0,0 @@
-/* @java.file.header */
-
-/*  _________        _____ __________________        _____
- *  __  ____/___________(_)______  /__  ____/______ ____(_)_______
- *  _  / __  __  ___/__  / _  __  / _  / __  _  __ `/__  / __  __ \
- *  / /_/ /  _  /    _  /  / /_/ /  / /_/ /  / /_/ / _  /  _  / / /
- *  \____/   /_/     /_/   \_,__/   \____/   \__,_/  /_/   /_/ /_/
- */
-
-package org.gridgain.grid.spi.swapspace.file;
-
-import org.apache.ignite.*;
-import org.apache.ignite.lang.*;
-import org.apache.ignite.marshaller.*;
-import org.apache.ignite.resources.*;
-import org.apache.ignite.spi.*;
-import org.gridgain.grid.*;
-import org.gridgain.grid.spi.swapspace.*;
-import org.gridgain.grid.util.*;
-import org.gridgain.grid.util.typedef.*;
-import org.gridgain.grid.util.typedef.internal.*;
-import org.jdk8.backport.*;
-import org.jetbrains.annotations.*;
-
-import java.io.*;
-import java.nio.*;
-import java.nio.channels.*;
-import java.util.*;
-import java.util.concurrent.*;
-import java.util.concurrent.atomic.*;
-import java.util.concurrent.locks.*;
-
-import static org.apache.ignite.events.IgniteEventType.*;
-
-/**
- * File-based swap space SPI implementation which holds keys in memory. This SPI is used by default.
- * It is intended for use in cases when value size is bigger than {@code 100} bytes, otherwise it will not
- * have any positive effect.
- * <p>
- * <b>NOTE: This SPI does not support swap eviction currently, manual removes needed to reduce disk space
- * consumption.</b>
- * <p>
- * Every space has a name and when used in combination with in-memory data grid name and local node ID,
- * space name represents the actual cache name associated with this swap space. Default name is {@code null}
- * which is represented by {@link #DFLT_SPACE_NAME}.
- *
- * <h1 class="header">Configuration</h1>
- * <h2 class="header">Mandatory</h2>
- * This SPI has no mandatory configuration parameters.
- * <h2 class="header">Optional SPI configuration.</h2>
- * <ul>
- *     <li>Base directory path (see {@link #setBaseDirectory(String)}).</li>
- *     <li>Maximum sparsity (see {@link #setMaximumSparsity(float)}).</li>
- *     <li>Write buffer size in bytes (see {@link #setWriteBufferSize(int)}).</li>
- *     <li>Max write queue size in bytes (see {@link #setMaxWriteQueueSize(int)}).</li>
- *     <li>Read stripes number. (see {@link #setReadStripesNumber(int)}).</li>
- * </ul>
- *
- * <h2 class="header">Java Example</h2>
- * GridFileSwapSpaceSpi is configured by default and should be explicitly configured
- * only if some SPI configuration parameters need to be overridden.
- * <pre name="code" class="java">
- * GridFileSwapSpaceSpi spi = new GridFileSwapSpaceSpi();
- *
- * // Configure root folder path.
- * spi.setBaseDirectory("/path/to/swap/folder");
- *
- * GridConfiguration cfg = new GridConfiguration();
- *
- * // Override default swap space SPI.
- * cfg.setSwapSpaceSpi(spi);
- *
- * // Starts grid.
- * G.start(cfg);
- * </pre>
- * <h2 class="header">Spring Example</h2>
- * GridFileSwapSpaceSpi can be configured from Spring XML configuration file:
- * <pre name="code" class="xml">
- * &lt;bean id=&quot;grid.cfg&quot; class=&quot;org.gridgain.grid.GridConfiguration&quot; scope=&quot;singleton&quot;&gt;
- *     ...
- *     &lt;property name=&quot;swapSpaceSpi&quot;&gt;
- *         &lt;bean class=&quot;org.gridgain.grid.spi.swapspace.file.GridFileSwapSpaceSpi&quot;&gt;
- *             &lt;property name=&quot;baseDirectory&quot; value=&quot;/path/to/swap/folder&quot;/&gt;
- *         &lt;/bean&gt;
- *     &lt;/property&gt;
- *     ...
- * &lt;/bean&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.swapspace.SwapSpaceSpi
- */
-@IgniteSpiMultipleInstancesSupport(true)
-@SuppressWarnings({"PackageVisibleInnerClass", "PackageVisibleField"})
-public class FileSwapSpaceSpi extends IgniteSpiAdapter implements SwapSpaceSpi, FileSwapSpaceSpiMBean {
-    /**
-     * Default base directory. Note that this path is relative to {@code GRIDGAIN_HOME/work} folder
-     * if {@code GRIDGAIN_HOME} system or environment variable specified, otherwise it is relative to
-     * {@code work} folder under system {@code java.io.tmpdir} folder.
-     *
-     * @see org.apache.ignite.configuration.IgniteConfiguration#getWorkDirectory()
-     */
-    public static final String DFLT_BASE_DIR = "swapspace";
-
-    /** Default maximum sparsity. */
-    public static final float DFLT_MAX_SPARSITY = 0.5f;
-
-    /** Default write buffer size in bytes. */
-    public static final int DFLT_BUF_SIZE = 64 * 1024;
-
-    /** Default write queue size in bytes. */
-    public static final int DFLT_QUE_SIZE = 1024 * 1024;
-
-    /** Name for {@code null} space. */
-    public static final String DFLT_SPACE_NAME = "gg-dflt-space";
-
-    /** Spaces. */
-    private final ConcurrentMap<String, Space> spaces = new ConcurrentHashMap<>();
-
-    /** Base directory. */
-    private String baseDir = DFLT_BASE_DIR;
-
-    /** Maximum sparsity. */
-    private float maxSparsity = DFLT_MAX_SPARSITY;
-
-    /** Eviction listener. */
-    private volatile SwapSpaceSpiListener evictLsnr;
-
-    /** Directory. */
-    private File dir;
-
-    /** Write buffer size. */
-    private int writeBufSize = DFLT_BUF_SIZE;
-
-    /** Max write queue size in bytes. */
-    private int maxWriteQueSize = DFLT_QUE_SIZE;
-
-    /** Read stripes number. */
-    private int readStripesNum = -1;
-
-    /** Logger. */
-    @IgniteLoggerResource
-    private IgniteLogger log;
-
-    /** Local node ID. */
-    @IgniteLocalNodeIdResource
-    private UUID locNodeId;
-
-    /** Name of the grid. */
-    @IgniteNameResource
-    private String gridName;
-
-    /** Marshaller. */
-    @IgniteMarshallerResource
-    private IgniteMarshaller marsh;
-
-    /** {@inheritDoc} */
-    @Override public String getBaseDirectory() {
-        return baseDir;
-    }
-
-    /**
-     * Sets base directory.
-     *
-     * @param baseDir Base directory.
-     */
-    @IgniteSpiConfiguration(optional = true)
-    public void setBaseDirectory(String baseDir) {
-        this.baseDir = baseDir;
-    }
-
-    /** {@inheritDoc} */
-    @Override public float getMaximumSparsity() {
-        return maxSparsity;
-    }
-
-    /**
-     * Sets maximum sparsity. This property defines maximum acceptable wasted file space to whole file size ratio.
-     * When this ratio becomes higher than specified number compacting thread starts working.
-     *
-     * @param maxSparsity Maximum sparsity. Must be between 0 and 1, default is {@link #DFLT_MAX_SPARSITY}.
-     */
-    public void setMaximumSparsity(float maxSparsity) {
-        this.maxSparsity = maxSparsity;
-    }
-
-    /** {@inheritDoc} */
-    @Override public int getWriteBufferSize() {
-        return writeBufSize;
-    }
-
-    /**
-     * Sets write buffer size in bytes. Write to disk occurs only when this buffer is full. Default is
-     * {@link #DFLT_BUF_SIZE}.
-     *
-     * @param writeBufSize Write buffer size in bytes.
-     */
-    public void setWriteBufferSize(int writeBufSize) {
-        this.writeBufSize = writeBufSize;
-    }
-
-    /** {@inheritDoc} */
-    @Override public int getMaxWriteQueueSize() {
-        return maxWriteQueSize;
-    }
-
-    /**
-     * Sets max write queue size in bytes. If there are more values are waiting for being written to disk then specified
-     * size, SPI will block on {@link #store(String, org.gridgain.grid.spi.swapspace.SwapKey, byte[], org.gridgain.grid.spi.swapspace.SwapContext)} operation. Default is
-     * {@link #DFLT_QUE_SIZE}.
-     *
-     * @param maxWriteQueSize Max write queue size in bytes.
-     */
-    public void setMaxWriteQueueSize(int maxWriteQueSize) {
-        this.maxWriteQueSize = maxWriteQueSize;
-    }
-
-    /** {@inheritDoc} */
-    @Override public int getReadStripesNumber() {
-        return readStripesNum;
-    }
-
-    /**
-     * Sets read stripe size. Defines number of file channels to be used concurrently. Default is equal to number of
-     * CPU cores available to this JVM.
-     *
-     * @param readStripesNum Read stripe number.
-     */
-    public void setReadStripesNumber(int readStripesNum) {
-        A.ensure(readStripesNum == -1 || (readStripesNum & (readStripesNum - 1)) == 0,
-            "readStripesNum must be positive and power of two");
-
-        this.readStripesNum = readStripesNum;
-    }
-
-    /** {@inheritDoc} */
-    @Override public void spiStart(@Nullable String gridName) throws IgniteSpiException {
-        assertParameter(!F.isEmpty(baseDir), "!F.isEmpty(baseDir)");
-        assertParameter(maxSparsity >= 0 && maxSparsity < 1, "maxSparsity >= 0 && maxSparsity < 1");
-        assertParameter(readStripesNum == -1 || (readStripesNum & (readStripesNum - 1)) == 0,
-            "readStripesNum must be positive and power of two.");
-
-        if (readStripesNum == -1) {
-            // User has not configured the number.
-            int readStripesNum0 = 1;
-            int cpuCnt = Runtime.getRuntime().availableProcessors();
-
-            while (readStripesNum0 <= cpuCnt)
-                readStripesNum0 <<= 1;
-
-            if (readStripesNum0 > cpuCnt)
-                readStripesNum0 >>= 1;
-
-            assert readStripesNum0 > 0 && (readStripesNum0 & readStripesNum0 - 1) == 0;
-
-            readStripesNum = readStripesNum0;
-        }
-
-        startStopwatch();
-
-        registerMBean(gridName, this, FileSwapSpaceSpiMBean.class);
-
-        String path = baseDir + File.separator + gridName + File.separator + locNodeId;
-
-        try {
-            dir = U.resolveWorkDirectory(path, true);
-        }
-        catch (GridException e) {
-            throw new IgniteSpiException(e);
-        }
-
-        if (log.isDebugEnabled())
-            log.debug(startInfo());
-    }
-
-    /** {@inheritDoc} */
-    @Override public void spiStop() throws IgniteSpiException {
-        unregisterMBean();
-
-        for (Space space : spaces.values()) {
-            space.initialize();
-
-            try {
-                space.stop();
-            }
-            catch (GridInterruptedException e) {
-                U.error(log, "Interrupted.", e);
-            }
-        }
-
-        if (dir != null && dir.exists() && !U.delete(dir))
-            U.warn(log, "Failed to delete swap directory: " + dir.getAbsolutePath());
-
-        if (log.isDebugEnabled())
-            log.debug(stopInfo());
-    }
-
-    /** {@inheritDoc} */
-    @Override public void clear(@Nullable String spaceName) throws IgniteSpiException {
-        Space space = space(spaceName, false);
-
-        if (space == null)
-            return;
-
-        space.clear();
-
-        notifyListener(EVT_SWAP_SPACE_CLEARED, spaceName);
-    }
-
-    /** {@inheritDoc} */
-    @Override public long size(@Nullable String spaceName) throws IgniteSpiException {
-        Space space = space(spaceName, false);
-
-        if (space == null)
-            return 0;
-
-        return space.size();
-    }
-
-    /** {@inheritDoc} */
-    @Override public long count(@Nullable String spaceName) throws IgniteSpiException {
-        Space space = space(spaceName, false);
-
-        if (space == null)
-            return 0;
-
-        return space.count();
-    }
-
-    /** {@inheritDoc} */
-    @Nullable @Override public byte[] read(@Nullable String spaceName, SwapKey key, SwapContext ctx)
-        throws IgniteSpiException {
-        assert key != null;
-        assert ctx != null;
-
-        Space space = space(spaceName, false);
-
-        if (space == null)
-            return null;
-
-        byte[] val = space.read(key);
-
-        notifyListener(EVT_SWAP_SPACE_DATA_READ, spaceName);
-
-        return val;
-    }
-
-    /** {@inheritDoc} */
-    @Override public Map<SwapKey, byte[]> readAll(@Nullable String spaceName, Iterable<SwapKey> keys,
-        SwapContext ctx) throws IgniteSpiException {
-        assert keys != null;
-        assert ctx != null;
-
-        Space space = space(spaceName, false);
-
-        if (space == null)
-            return Collections.emptyMap();
-
-        Map<SwapKey, byte[]> res = new HashMap<>();
-
-        for (SwapKey key : keys) {
-            if (key != null) {
-                byte[] val = space.read(key);
-
-                if (val != null)
-                    res.put(key, val);
-
-                notifyListener(EVT_SWAP_SPACE_DATA_READ, spaceName);
-            }
-        }
-
-        return res;
-    }
-
-    /** {@inheritDoc} */
-    @Override public void remove(@Nullable String spaceName, SwapKey key, @Nullable IgniteInClosure<byte[]> c,
-        SwapContext ctx) throws IgniteSpiException {
-        assert key != null;
-        assert ctx != null;
-
-        Space space = space(spaceName, false);
-
-        if (space == null)
-            return;
-
-        byte[] val = space.remove(key, c != null);
-
-        if (c != null)
-            c.apply(val);
-
-        notifyListener(EVT_SWAP_SPACE_DATA_REMOVED, spaceName);
-    }
-
-    /** {@inheritDoc} */
-    @Override public void removeAll(@Nullable String spaceName, Collection<SwapKey> keys,
-        @Nullable IgniteBiInClosure<SwapKey, byte[]> c, SwapContext ctx) throws IgniteSpiException {
-        assert keys != null;
-        assert ctx != null;
-
-        Space space = space(spaceName, false);
-
-        if (space == null)
-            return;
-
-        for (SwapKey key : keys) {
-            if (key != null) {
-                byte[] val = space.remove(key, c != null);
-
-                if (c != null)
-                    c.apply(key, val);
-
-                notifyListener(EVT_SWAP_SPACE_DATA_REMOVED, spaceName);
-            }
-        }
-    }
-
-    /** {@inheritDoc} */
-    @Override public void store(@Nullable String spaceName, SwapKey key, @Nullable byte[] val,
-        SwapContext ctx) throws IgniteSpiException {
-        assert key != null;
-        assert ctx != null;
-
-        Space space = space(spaceName, true);
-
-        assert space != null;
-
-        space.store(key, val);
-
-        notifyListener(EVT_SWAP_SPACE_DATA_STORED, spaceName);
-    }
-
-    /** {@inheritDoc} */
-    @Override public void storeAll(@Nullable String spaceName, Map<SwapKey, byte[]> pairs,
-        SwapContext ctx) throws IgniteSpiException {
-        assert pairs != null;
-        assert ctx != null;
-
-        Space space = space(spaceName, true);
-
-        assert space != null;
-
-        for (Map.Entry<SwapKey, byte[]> pair : pairs.entrySet()) {
-            SwapKey key = pair.getKey();
-
-            if (key != null) {
-                space.store(key, pair.getValue());
-
-                notifyListener(EVT_SWAP_SPACE_DATA_STORED, spaceName);
-            }
-        }
-    }
-
-    /** {@inheritDoc} */
-    @Override public void setListener(@Nullable SwapSpaceSpiListener evictLsnr) {
-        this.evictLsnr = evictLsnr;
-    }
-
-    /** {@inheritDoc} */
-    @Nullable @Override public Collection<Integer> partitions(@Nullable String spaceName)
-        throws IgniteSpiException {
-        Space space = space(spaceName, false);
-
-        if (space == null)
-            return null;
-
-        return space.partitions();
-    }
-
-    /** {@inheritDoc} */
-    @Nullable @Override public <K> IgniteSpiCloseableIterator<K> keyIterator(@Nullable String spaceName,
-        SwapContext ctx) throws IgniteSpiException {
-        final Space space = space(spaceName, false);
-
-        if (space == null)
-            return null;
-
-        final Iterator<Map.Entry<SwapKey, byte[]>> iter = space.entriesIterator();
-
-        return new GridCloseableIteratorAdapter<K>() {
-            @Override protected boolean onHasNext() {
-                return iter.hasNext();
-            }
-
-            @Override protected K onNext() {
-                return (K)iter.next().getKey().key();
-            }
-
-            @Override protected void onRemove() {
-                iter.remove();
-            }
-        };
-    }
-
-    /** {@inheritDoc} */
-    @Nullable @Override public IgniteSpiCloseableIterator<Map.Entry<byte[], byte[]>> rawIterator(
-        @Nullable String spaceName) throws IgniteSpiException {
-        Space space = space(spaceName, false);
-
-        if (space == null)
-            return null;
-
-        return rawIterator(space.entriesIterator());
-    }
-
-    /** {@inheritDoc} */
-    @Nullable @Override public IgniteSpiCloseableIterator<Map.Entry<byte[], byte[]>> rawIterator(
-        @Nullable String spaceName, int part) throws IgniteSpiException {
-        Space space = space(spaceName, false);
-
-        if (space == null)
-            return null;
-
-        return rawIterator(space.entriesIterator(part));
-    }
-
-    /**
-     * Creates raw iterator based on provided entries iterator.
-     *
-     * @param iter Entries iterator.
-     * @return Raw iterator.
-     */
-    private IgniteSpiCloseableIterator<Map.Entry<byte[], byte[]>> rawIterator(
-        final Iterator<Map.Entry<SwapKey, byte[]>> iter) {
-        return new GridCloseableIteratorAdapter<Map.Entry<byte[], byte[]>>() {
-            @Override protected Map.Entry<byte[], byte[]> onNext() throws GridException {
-                Map.Entry<SwapKey, byte[]> x = iter.next();
-
-                return new T2<>(keyBytes(x.getKey()), x.getValue());
-            }
-
-            @Override protected boolean onHasNext() {
-                return iter.hasNext();
-            }
-
-            @Override protected void onRemove() {
-                iter.remove();
-            }
-        };
-    }
-
-    /**
-     * Gets key bytes.
-     *
-     * @param key Swap key.
-     * @return Key bytes.
-     * @throws org.apache.ignite.spi.IgniteSpiException In case of error.
-     */
-    private byte[] keyBytes(SwapKey key) throws IgniteSpiException {
-        assert key != null;
-
-        byte[] keyBytes = key.keyBytes();
-
-        if (keyBytes == null) {
-            try {
-                keyBytes = marsh.marshal(key.key());
-            }
-            catch (GridException e) {
-                throw new IgniteSpiException("Failed to marshal key: " + key.key(), e);
-            }
-
-            key.keyBytes(keyBytes);
-        }
-
-        return keyBytes;
-    }
-
-    /**
-     * Notifies eviction listener.
-     *
-     * @param evtType Event type.
-     * @param spaceName Space name.
-     */
-    private void notifyListener(int evtType, @Nullable String spaceName) {
-        SwapSpaceSpiListener lsnr = evictLsnr;
-
-        if (lsnr != null)
-            lsnr.onSwapEvent(evtType, spaceName, null);
-    }
-
-    /**
-     * Gets space by name.
-     *
-     * @param name Space name.
-     * @param create Whether to create space if it doesn't exist.
-     * @return Space.
-     * @throws org.apache.ignite.spi.IgniteSpiException In case of error.
-     */
-    @Nullable private Space space(@Nullable String name, boolean create) throws IgniteSpiException {
-        String masked = name != null ? name : DFLT_SPACE_NAME;
-
-        assert masked != null;
-
-        Space space = spaces.get(masked);
-
-        if (space == null && create) {
-            validateName(name);
-
-            Space old = spaces.putIfAbsent(masked, space = new Space(masked));
-
-            if (old != null)
-                space = old;
-        }
-
-        if (space != null)
-            space.initialize();
-
-        return space;
-    }
-
-    /**
-     * Validates space name.
-     *
-     * @param name Space name.
-     * @throws org.apache.ignite.spi.IgniteSpiException If name is invalid.
-     */
-    private void validateName(@Nullable String name) throws IgniteSpiException {
-        if (name == null)
-            return;
-
-        if (name.isEmpty())
-            throw new IgniteSpiException("Space name cannot be empty: " + name);
-        else if (DFLT_SPACE_NAME.equalsIgnoreCase(name))
-            throw new IgniteSpiException("Space name is reserved for default space: " + name);
-        else if (name.contains("/") || name.contains("\\"))
-            throw new IgniteSpiException("Space name contains invalid characters: " + name);
-    }
-
-    /** {@inheritDoc} */
-    @Override public String toString() {
-        return S.toString(FileSwapSpaceSpi.class, this);
-    }
-
-    /**
-     * Swap value.
-     */
-    static class SwapValue {
-        /** */
-        private static final int NEW = 0;
-
-        /** */
-        private static final int DELETED = Integer.MIN_VALUE;
-
-        /** */
-        private static final AtomicIntegerFieldUpdater<SwapValue> idxUpdater = AtomicIntegerFieldUpdater.
-            newUpdater(SwapValue.class, "idx");
-
-        /** */
-        private byte[] val;
-
-        /** */
-        private final int len;
-
-        /** */
-        @SuppressWarnings("FieldAccessedSynchronizedAndUnsynchronized")
-        private long pos = -1;
-
-        /** */
-        @SuppressWarnings("UnusedDeclaration")
-        private volatile int idx;
-
-        /**
-         * @param val Value.
-         */
-        SwapValue(byte[] val) {
-            assert val != null;
-
-            this.val = val;
-            len = val.length;
-        }
-
-        /**
-         * @param space Space.
-         * @return Value.
-         * @throws org.apache.ignite.spi.IgniteSpiException If failed.
-         */
-        @SuppressWarnings("NonPrivateFieldAccessedInSynchronizedContext")
-        @Nullable public synchronized byte[] value(Space space) throws IgniteSpiException {
-            byte[] v = val;
-
-            if (v == null) { // Read value from file.
-                int i = idx;
-
-                assert i != NEW;
-
-                if (i != DELETED) {
-                    StripedFileChannel ch = i < 0 ? space.left.readCh : space.right.readCh;
-
-                    if (idx != DELETED) // Double check works in pair with striped channel reopening.
-                        v = readValue(ch);
-                }
-            }
-            else if (v.length != len) {
-                int p = (int)pos;
-
-                v = Arrays.copyOfRange(v, p, p + len); // In case of compaction.
-            }
-
-            return v;
-        }
-
-        /**
-         * @param ch File channel.
-         * @return Bytes.
-         * @throws org.apache.ignite.spi.IgniteSpiException if failed.
-         */
-        @Nullable byte[] readValue(StripedFileChannel ch) throws IgniteSpiException {
-            byte[] v = new byte[len];
-
-            int res = 0;
-
-            try {
-                res = ch.read(ByteBuffer.wrap(v), pos);
-            }
-            catch (ClosedChannelException ignore) {
-                assert idx == DELETED;
-            }
-            catch (IOException e) {
-                throw new IgniteSpiException("Failed to read value.", e);
-            }
-
-            if (res < len)
-                return null; // When concurrent compaction occurs this may happen.
-
-            return v;
-        }
-
-        /**
-         * @param pos Position.
-         * @param val Value.
-         */
-        public synchronized void set(long pos, byte[] val) {
-            if (pos != -1)
-                this.pos = pos;
-
-            this.val = val;
-        }
-
-        /**
-         * @param exp Expected.
-         * @param idx New index.
-         * @return {@code true} if succeeded.
-         */
-        public boolean casIdx(int exp, int idx) {
-            return idxUpdater.compareAndSet(this, exp, idx);
-        }
-
-        /**
-         * @return Index in file array.
-         */
-        int idx() {
-            return idx;
-        }
-
-        /** {@inheritDoc} */
-        @Override public String toString() {
-            return pos + " " + len;
-        }
-    }
-
-    /**
-     * Queue of swap values.
-     */
-    private static class SwapValuesQueue {
-        /** */
-        private final ArrayDeque<SwapValue> deq = new ArrayDeque<>();
-
-        /** */
-        @SuppressWarnings("TypeMayBeWeakened")
-        private final ReentrantLock lock = new ReentrantLock();
-
-        /** */
-        private final Condition mayAdd = lock.newCondition();
-
-        /** */
-        private final Condition mayTake = lock.newCondition();
-
-        /** */
-        private int size;
-
-        /** */
-        private final int minTakeSize;
-
-        /** */
-        private final int maxSize;
-
-        /**
-         * @param minTakeSize Min size.
-         * @param maxSize Max size.
-         */
-        private SwapValuesQueue(int minTakeSize, int maxSize) {
-            this.minTakeSize = minTakeSize;
-            this.maxSize = maxSize;
-        }
-
-        /**
-         * Adds to queue.
-         *
-         * @param val Swap value.
-         * @throws org.apache.ignite.spi.IgniteSpiException If failed.
-         */
-        public void add(SwapValue val) throws IgniteSpiException {
-            lock.lock();
-
-            try {
-                while (size + val.len > maxSize)
-                    mayAdd.await();
-
-                size += val.len;
-
-                deq.addLast(val);
-
-                if (size >= minTakeSize)
-                    mayTake.signalAll();
-            }
-            catch (InterruptedException e) {
-                throw new IgniteSpiException(e);
-            }
-            finally {
-                lock.unlock();
-            }
-        }
-
-        /**
-         * Takes swap values from queue.
-         *
-         * @return Swap values.
-         * @throws InterruptedException If interrupted.
-         */
-        public SwapValues take() throws InterruptedException {
-            lock.lock();
-
-            try {
-                while (size < minTakeSize)
-                    mayTake.await();
-
-                int size = 0;
-                int cnt = 0;
-
-                for (SwapValue val : deq) {
-                    size += val.len;
-                    cnt++;
-
-                    if (size >= minTakeSize)
-                        break;
-                }
-
-                SwapValue[] vals = new SwapValue[cnt];
-
-                for (int i = 0; i < cnt; i++) {
-                    SwapValue val = deq.pollFirst();
-
-                    vals[i] = val;
-                }
-
-                if ((this.size -= size) < maxSize)
-                    mayAdd.signalAll();
-
-                return new SwapValues(vals, size);
-            }
-            finally {
-                lock.unlock();
-            }
-        }
-    }
-
-    /**
-     * Array of swap values and their size in bytes.
-     */
-    static class SwapValues {
-        /** */
-        private final SwapValue[] vals;
-
-        /** Size in bytes. */
-        private final int size;
-
-        /**
-         * @param vals Values.
-         * @param size Size.
-         */
-        SwapValues(SwapValue[] vals, int size) {
-            this.vals = vals;
-            this.size = size;
-        }
-    }
-
-    /**
-     * Readable striped file channel.
-     */
-    private static class StripedFileChannel {
-        /** */
-        private final AtomicInteger enter = new AtomicInteger();
-
-        /** */
-        private final RandomAccessFile[] rafs;
-
-        /** */
-        private final FileChannel[] chs;
-
-        /**
-         * @param f File.
-         * @param stripes Stripes.
-         * @throws FileNotFoundException If failed.
-         */
-        StripedFileChannel(File f, int stripes) throws FileNotFoundException {
-            assert stripes > 0 && (stripes & (stripes - 1)) == 0 : "stripes must be positive and power of two.";
-
-            rafs = new RandomAccessFile[stripes];
-            chs = new FileChannel[stripes];
-
-            for (int i = 0; i < stripes; i++) {
-                RandomAccessFile raf = new RandomAccessFile(f, "r");
-
-                rafs[i] = raf;
-                chs[i] = raf.getChannel();
-            }
-        }
-
-        /**
-         * Reads data from file channel to buffer.
-         *
-         * @param buf Buffer.
-         * @param pos Position.
-         * @return Read bytes count.
-         * @throws IOException If failed.
-         */
-        int read(ByteBuffer buf, long pos) throws IOException {
-            int i = enter.getAndIncrement() & (chs.length - 1);
-
-            return chs[i].read(buf, pos);
-        }
-
-        /**
-         * Closes channel.
-         */
-        void close() {
-            for (RandomAccessFile raf : rafs)
-                U.closeQuiet(raf);
-        }
-    }
-
-    /**
-     * Swap file.
-     */
-    static class SwapFile {
-        /** */
-        private static final long MIN_TRUNK_SIZE = 10 * 1024 * 1024;
-
-        /** */
-        private final File file;
-
-        /** */
-        private final RandomAccessFile raf;
-
-        /** */
-        private final FileChannel writeCh;
-
-        /** */
-        volatile StripedFileChannel readCh;
-
-        /** */
-        private volatile long len;
-
-        /** */
-        private final FileSwapArray<SwapValue> arr = new FileSwapArray<>();
-
-        /**
-         * @param file File.
-         * @param readerStripes Reader stripes number.
-         * @throws IOException In case of error.
-         */
-        SwapFile(File file, int readerStripes) throws IOException {
-            assert file != null;
-
-            file.delete();
-
-            if (!file.createNewFile())
-                throw new IllegalStateException("Failed to create file: " + file.getAbsolutePath());
-
-            this.file = file;
-
-            raf = new RandomAccessFile(file, "rw");
-
-            writeCh = raf.getChannel();
-
-            readCh = new StripedFileChannel(file, readerStripes);
-        }
-
-        /**
-         * Reopens read channel.
-         *
-         * @throws FileNotFoundException If failed.
-         */
-        void reopenReadChannel() throws FileNotFoundException {
-            readCh.close();
-
-            readCh = new StripedFileChannel(file, readCh.chs.length);
-        }
-
-        /**
-         * @param vals Values.
-         * @param buf Duffer.
-         * @param sign Indicates where should we write value, to the left or to the right.
-         * @throws Exception If failed.
-         */
-        public void write(Iterable<SwapValue> vals, ByteBuffer buf, int sign) throws Exception {
-            for (SwapValue val : vals) {
-                int oldIdx = val.idx;
-
-                if (oldIdx == SwapValue.DELETED)
-                    continue;
-
-                int idx = arr.add(val);
-
-                if (!val.casIdx(oldIdx, sign * idx)) {
-                    assert val.idx == SwapValue.DELETED;
-
-                    boolean res = tryRemove(idx, val);
-
-                    assert res;
-                }
-            }
-
-            final int size = buf.remaining();
-
-            if (size == 0)
-                return;
-
-            long pos = len;
-
-            len = pos + size;
-
-            long res = writeCh.write(buf, pos);
-
-            if (res != size)
-                throw new IllegalStateException(res + " != " + size);
-
-            // Nullify bytes in values ans set pos.
-            for (SwapValue val : vals) {
-                val.set(pos, null);
-
-                pos += val.len;
-            }
-        }
-
-        /**
-         * @param vals Values.
-         * @param sign Sign: 1 or -1.
-         * @throws Exception If failed.
-         */
-        public void write(SwapValues vals, int sign) throws Exception {
-            ByteBuffer buf = ByteBuffer.allocateDirect(vals.size);
-
-            for (int i = 0, len = vals.vals.length; i < len; i++) {
-                SwapValue val = vals.vals[i];
-
-                if (val.idx == SwapValue.DELETED) {
-                    vals.vals[i] = null;
-
-                    continue;
-                }
-
-                int idx = arr.add(val);
-
-                if (!val.casIdx(SwapValue.NEW, sign * idx)) {
-                    assert val.idx == SwapValue.DELETED;
-
-                    tryRemove(idx, val);
-
-                    vals.vals[i] = null;
-                }
-                else
-                    buf.put(val.value(null));
-            }
-
-            buf.flip();
-
-            final int size = buf.remaining();
-
-            if (size == 0)
-                return;
-
-            long pos = len;
-
-            len = pos + size;
-
-            long res = writeCh.write(buf, pos);
-
-            if (res != size)
-                throw new IllegalStateException(res + " != " + size);
-
-            // Nullify bytes in values ans set pos.
-            for (SwapValue val : vals.vals) {
-                if (val == null)
-                    continue;
-
-                val.set(pos, null);
-
-                pos += val.len;
-            }
-        }
-
-        /**
-         * Gets file path.
-         *
-         * @return File path.
-         */
-        public String path() {
-            return file.getAbsolutePath();
-        }
-
-        /**
-         * Gets file length.
-         *
-         * @return File length.
-         */
-        public long length() {
-            return len;
-        }
-
-        /**
-         * Deletes file.
-         *
-         * @return Whether file was actually deleted.
-         */
-        public boolean delete() {
-            U.closeQuiet(raf);
-
-            readCh.close();
-
-            return U.delete(file);
-        }
-
-        /**
-         * @param idx Index.
-         * @param exp Expected value.
-         * @return {@code true} If succeeded.
-         */
-        public boolean tryRemove(int idx, SwapValue exp) {
-            assert idx > 0 : idx;
-
-            FileSwapArray.Slot<SwapValue> s = arr.slot(idx);
-
-            return s != null && s.cas(exp, null);
-        }
-
-        /**
-         * Does compaction for one buffer.
-         *
-         * @param vals Values.
-         * @param bufSize Buffer size.
-         * @return Buffer.
-         * @throws IOException If failed.
-         * @throws InterruptedException If interrupted.
-         */
-        public ByteBuffer compact(ArrayDeque<SwapValue> vals, final int bufSize) throws IOException,
-            InterruptedException {
-            assert vals.isEmpty();
-
-            Compact c = new Compact(vals, bufSize);
-
-            c.doCompact();
-
-            return c.result();
-        }
-
-        /**
-         * Single compaction operation.
-         */
-        private class Compact {
-            /** */
-            private final ArrayDeque<SwapValue> vals;
-
-            /** */
-            private final int bufSize;
-
-            /** */
-            private byte[] bytes;
-
-            /** */
-            private ByteBuffer buf;
-
-            /** */
-            private long beg = -1;
-
-            /** */
-            private long end = -1;
-
-            /** */
-            private int compacted;
-
-            /**
-             * @param vals Values.
-             * @param bufSize Buffer size.
-             */
-            private Compact(ArrayDeque<SwapValue> vals, final int bufSize) {
-                assert vals.isEmpty();
-
-                this.vals = vals;
-                this.bufSize = bufSize;
-            }
-
-            /**
-             * Reads buffer and compacts it.
-             *
-             * @throws IOException if failed.
-             */
-            private void readAndCompact() throws IOException {
-                assert beg != -1;
-
-                if (buf == null) {
-                    bytes = new byte[bufSize];
-
-                    buf = ByteBuffer.wrap(bytes);
-                }
-
-                final int pos = buf.position();
-
-                final int lim = (int)(end - beg + pos);
-
-                assert pos >= 0;
-                assert pos < lim : pos + " " + lim;
-                assert lim <= buf.capacity();
-
-                buf.limit(lim);
-
-                int res = writeCh.read(buf, beg);
-
-                assert res == lim - pos;
-
-                int prevEnd = pos;
-                long delta = beg - pos; // To translate from file based positions to buffer based.
-
-                for (int j = vals.size(); j > compacted; j--) {
-                    SwapValue val = vals.pollFirst();
-
-                    int valPos = (int)(val.pos - delta);
-
-                    if (prevEnd != valPos) {
-                        assert prevEnd < valPos : prevEnd + " " + valPos;
-
-                        U.arrayCopy(bytes, valPos, bytes, prevEnd, val.len);
-                    }
-
-                    prevEnd += val.len;
-
-                    vals.addLast(val); // To have values in the same order as in byte buffer.
-                }
-
-                assert prevEnd > 0 : prevEnd;
-
-                buf.position(prevEnd);
-
-                end = -1;
-
-                compacted = vals.size();
-            }
-
-            /**
-             * Compacts.
-             *
-             * @throws IOException If failed.
-             */
-            private void doCompact() throws IOException {
-                int idx = arr.size();
-
-                while (--idx > 0) {
-                    FileSwapArray.Slot<SwapValue> s = arr.slot(idx);
-
-                    assert s != null;
-
-                    SwapValue v = s.get();
-
-                    if (v == null || v.idx == SwapValue.DELETED)
-                        continue;
-
-                    if (end == -1)
-                        end = v.pos + v.len;
-
-                    long size = end - v.pos;
-
-                    if ((buf == null ? bufSize : buf.remaining()) < size) {
-                        if (vals.isEmpty()) {  // Too big single value.
-                            assert bytes == null && buf == null;
-
-                            bytes = new byte[(int)size];
-
-                            buf = ByteBuffer.wrap(bytes);
-                        }
-                        else if (compacted == vals.size())
-                            break; // Finish current compaction, nothing new collected.
-                        else { // Read region and compact values in buffer.
-                            readAndCompact();
-
-                            // Retry the same value.
-                            idx++;
-
-                            continue;
-                        }
-                    }
-
-                    beg = v.pos;
-
-                    vals.addFirst(v);
-
-                    s.cas(v, null);
-                }
-
-                if (vals.isEmpty()) {
-                    arr.truncate(1);
-
-                    writeCh.truncate(0);
-
-                    len = 0;
-
-                    reopenReadChannel(); // Make sure that value can be read only from right file but not after switch.
-
-                    return;
-                }
-
-                if (compacted != vals.size())
-                    readAndCompact();
-
-                int pos = 0;
-
-                for (SwapValue val : vals) { // The values will share one byte array with different offsets while moving.
-                    val.set(pos, bytes);
-
-                    pos += val.len;
-                }
-
-                buf.flip();
-
-                assert buf.limit() == pos : buf.limit() + " " + pos;
-
-                arr.truncate(idx + 1);
-
-                if (len - beg > MIN_TRUNK_SIZE) {
-                    writeCh.truncate(beg);
-
-                    len = beg;
-                }
-            }
-
-            /**
-             * @return Buffer.
-             */
-            public ByteBuffer result() {
-                return buf;
-            }
-        }
-    }
-
-    /**
-     * Space.
-     */
-    private class Space {
-        /** Space name. */
-        private final String name;
-
-        /** */
-        private final GridAtomicInitializer<Void> initializer = new GridAtomicInitializer<>();
-
-        /** Swap file left. */
-        @SuppressWarnings("FieldAccessedSynchronizedAndUnsynchronized")
-        private SwapFile left;
-
-        /** Swap file right. */
-        @SuppressWarnings("FieldAccessedSynchronizedAndUnsynchronized")
-        private SwapFile right;
-
-        /** */
-        private final SwapValuesQueue que = new SwapValuesQueue(writeBufSize, maxWriteQueSize);
-
-        /** Partitions. */
-        private final ConcurrentMap<Integer, ConcurrentMap<SwapKey, SwapValue>> parts =
-            new ConcurrentHashMap8<>();
-
-        /** Total size. */
-        private final AtomicLong size = new AtomicLong();
-
-        /** Total count. */
-        private final AtomicLong cnt = new AtomicLong();
-
-        /** */
-        private int sign = 1;
-
-        /** Writer thread. */
-        private Thread writer;
-
-        /** */
-        private Thread compactor;
-
-        /**
-         * @param name Space name.
-         */
-        private Space(String name) {
-            assert name != null;
-
-            this.name = name;
-        }
-
-        /**
-         * Initializes space.
-         *
-         * @throws org.apache.ignite.spi.IgniteSpiException If initialization failed.
-         */
-        public void initialize() throws IgniteSpiException {
-            if (initializer.succeeded())
-                return;
-
-            assert dir.exists();
-            assert dir.isDirectory();
-
-            try {
-                initializer.init(new Callable<Void>(){
-                    @Override public Void call() throws Exception {
-                        left = new SwapFile(new File(dir, name + ".left"), readStripesNum);
-
-                        right = new SwapFile(new File(dir, name + ".right"), readStripesNum);
-
-                        final Object mux = new Object();
-
-                        writer = new IgniteSpiThread(gridName,  "Swap writer: " + name, log) {
-                            @Override protected void body() throws InterruptedException {
-                                while (!isInterrupted()) {
-                                    SwapValues vals = que.take();
-
-                                    synchronized (mux) {
-                                        SwapFile f = sign == 1 ? right : left;
-
-                                        try {
-                                            f.write(vals, sign);
-                                        }
-                                        catch (Exception e) {
-                                            throw new GridRuntimeException(e);
-                                        }
-                                    }
-                                }
-                            }
-                        };
-
-                        compactor = new IgniteSpiThread(gridName, "Swap compactor: " + name, log) {
-                            @Override protected void body() throws InterruptedException {
-                                SwapFile w = null;
-                                SwapFile c = null;
-
-                                ArrayDeque<SwapValue> vals = null;
-
-                                while (!isInterrupted()) {
-                                    while(!needCompact()) {
-                                        LockSupport.park();
-
-                                        if (isInterrupted())
-                                            return;
-                                    }
-
-                                    ByteBuffer buf = null;
-
-                                    if (vals == null)
-                                        vals = new ArrayDeque<>();
-                                    else {
-                                        vals.clear();
-
-                                        try {
-                                            buf = c.compact(vals, writeBufSize);
-                                        }
-                                        catch (IOException e) {
-                                            throw new GridRuntimeException(e);
-                                        }
-                                    }
-
-                                    if (vals.isEmpty()) {
-                                        synchronized (mux) {
-                                            sign = -sign;
-
-                                            if (sign == 1) {
-                                                w = right;
-                                                c = left;
-                                            }
-                                            else {
-                                                w = left;
-                                                c = right;
-                                            }
-                                        }
-                                    }
-                                    else {
-                                        assert buf != null && buf.remaining() != 0;
-
-                                        synchronized (mux) {
-                                            try {
-                                                w.write(vals, buf, sign);
-                                            }
-                                            catch (Exception e) {
-                                                throw new GridRuntimeException(e);
-                                            }
-                                        }
-                                    }
-                                }
-                            }
-                        };
-
-                        writer.start();
-                        compactor.start();
-
-                        return null;
-                    }
-                });
-            }
-            catch (GridException e) {
-                throw new IgniteSpiException(e);
-            }
-        }
-
-        /**
-         * Gets total space size in bytes.
-         *
-         * @return Total size.
-         */
-        public long size() {
-            return left.length() + right.length();
-        }
-
-        /**
-         * Gets total space count.
-         *
-         * @return Total count.
-         */
-        public long count() {
-            return cnt.get();
-        }
-
-        /**
-         * Clears space.
-         *
-         * @throws org.apache.ignite.spi.IgniteSpiException If failed.
-         */
-        public void clear() throws IgniteSpiException {
-            Iterator<Map.Entry<SwapKey, byte[]>> iter = entriesIterator();
-
-            while (iter.hasNext())
-                remove(iter.next().getKey(), false);
-        }
-
-        /**
-         * Stops space.
-         *
-         * @throws GridInterruptedException If interrupted.
-         */
-        public void stop() throws GridInterruptedException {
-            U.interrupt(writer);
-            U.interrupt(compactor);
-
-            U.join(writer);
-            U.join(compactor);
-
-            left.delete();
-            right.delete();
-        }
-
-        /**
-         * Stores value in space.
-         *
-         * @param key Key.
-         * @param val Value.
-         * @throws org.apache.ignite.spi.IgniteSpiException In case of error.
-         */
-        public void store(final SwapKey key, @Nullable final byte[] val) throws IgniteSpiException {
-            assert key != null;
-
-            final ConcurrentMap<SwapKey, SwapValue> part = partition(key.partition(), true);
-
-            assert part != null;
-
-            if (val == null) {
-                SwapValue swapVal = part.remove(key);
-
-                if (swapVal != null) {
-                    removeFromFile(swapVal);
-
-                    size.addAndGet(-swapVal.len);
-                    cnt.decrementAndGet();
-                }
-
-                return;
-            }
-
-            final SwapValue swapVal = new SwapValue(val);
-
-            SwapValue old = part.put(key, swapVal);
-
-            if (old != null) {
-                size.addAndGet(val.length - old.len);
-
-                removeFromFile(old);
-            }
-            else {
-                size.addAndGet(val.length);
-                cnt.incrementAndGet();
-            }
-
-            que.add(swapVal);
-        }
-
-        /**
-         * Reads value from space.
-         *
-         * @param key Key.
-         * @return Value.
-         * @throws org.apache.ignite.spi.IgniteSpiException In case of error.
-         */
-        @Nullable public byte[] read(SwapKey key) throws IgniteSpiException {
-            assert key != null;
-
-            final Map<SwapKey, SwapValue> part = partition(key.partition(), false);
-
-            if (part == null)
-                return null;
-
-            SwapValue swapVal = part.get(key);
-
-            if (swapVal == null)
-                return null;
-
-            return swapVal.value(this);
-        }
-
-        /**
-         * Removes value from space.
-         *
-         * @param key Key.
-         * @param read If value has to be read.
-         * @return Value.
-         * @throws org.apache.ignite.spi.IgniteSpiException In case of error.
-         */
-        @Nullable public byte[] remove(SwapKey key, boolean read) throws IgniteSpiException {
-            assert key != null;
-
-            final Map<SwapKey, SwapValue> part = partition(key.partition(), false);
-
-            if (part == null)
-                return null;
-
-            SwapValue val = part.remove(key);
-
-            if (val == null)
-                return null;
-
-            size.addAndGet(-val.len);
-
-            cnt.decrementAndGet();
-
-            byte[] bytes = null;
-
-            if (read) {
-                bytes = val.value(this);
-
-                assert bytes != null; // Value bytes were read before removal from file, so compaction can't happen.
-            }
-
-            removeFromFile(val);
-
-            return bytes;
-        }
-
-        /**
-         * @param val Value.
-         */
-        private void removeFromFile(SwapValue val) {
-            for (;;) {
-                int idx = val.idx;
-
-                assert idx != SwapValue.DELETED;
-
-                if (val.casIdx(idx, SwapValue.DELETED)) {
-                    if (idx != SwapValue.NEW) {
-                        SwapFile f = idx > 0 ? right : left;
-
-                        f.tryRemove(Math.abs(idx), val);
-                    }
-
-                    break;
-                }
-            }
-
-            if (needCompact())
-                LockSupport.unpark(compactor);
-        }
-
-        /**
-         * @return {@code true} If compaction needed.
-         */
-        private boolean needCompact() {
-            long fileLen = size();
-
-            return fileLen > writeBufSize && (fileLen - size.get()) / (float)fileLen > maxSparsity;
-        }
-
-        /**
-         * Gets numbers of partitioned stored in this space.
-         *
-         * @return Partition numbers.
-         */
-        public Collection<Integer> partitions() {
-            return parts.keySet();
-        }
-
-        /**
-         * Gets partition map by its number.
-         *
-         * @param part Partition number.
-         * @param create Whether to create partition if it doesn't exist.
-         * @return Partition map.
-         */
-        @Nullable private ConcurrentMap<SwapKey, SwapValue> partition(int part, boolean create) {
-            ConcurrentMap<SwapKey, SwapValue> map = parts.get(part);
-
-            if (map == null && create) {
-                ConcurrentMap<SwapKey, SwapValue> old = parts.putIfAbsent(part,
-                    map = new ConcurrentHashMap<>());
-
-                if (old != null)
-                    map = old;
-            }
-
-            return map;
-        }
-
-        /**
-         * @param part Partition.
-         * @return Iterator over partition.
-         */
-        public Iterator<Map.Entry<SwapKey, byte[]>> entriesIterator(int part) {
-            Map<SwapKey, SwapValue> partMap = partition(part, false);
-
-            if (partMap == null)
-                return Collections.<Map.Entry<SwapKey, byte[]>>emptySet().iterator();
-
-            return transform(partMap.entrySet().iterator());
-        }
-
-        /**
-         * @return Iterator over all entries.
-         */
-        public Iterator<Map.Entry<SwapKey, byte[]>> entriesIterator() {
-            final Iterator<ConcurrentMap<SwapKey, SwapValue>> iter = parts.values().iterator();
-
-            return transform(F.concat(new Iterator<Iterator<Map.Entry<SwapKey, SwapValue>>>() {
-                @Override public boolean hasNext() {
-                    return iter.hasNext();
-                }
-
-                @Override public Iterator<Map.Entry<SwapKey, SwapValue>> next() {
-                    return iter.next().entrySet().iterator();
-                }
-
-                @Override public void remove() {
-                    throw new UnsupportedOperationException();
-                }
-            }));
-        }
-
-        /**
-         * Gets iterator for all entries in space.
-         *
-         * @param iter Iterator with {@link SwapValue} to transform.
-         * @return Entries iterator.
-         */
-        private Iterator<Map.Entry<SwapKey, byte[]>> transform(final Iterator<Map.Entry<SwapKey,
-            SwapValue>> iter) {
-            return new Iterator<Map.Entry<SwapKey, byte[]>>() {
-                /** */
-                private Map.Entry<SwapKey, byte[]> next;
-
-                /** */
-                private Map.Entry<SwapKey, byte[]> last;
-
-                {
-                    advance();
-                }
-
-                @Override public boolean hasNext() {
-                    return next != null;
-                }
-
-                /**
-                 * Gets next entry.
-                 */
-                private void advance() {
-                    while (iter.hasNext()) {
-                        Map.Entry<SwapKey, SwapValue> entry = iter.next();
-
-                        byte[] bytes;
-
-                        try {
-                            bytes = entry.getValue().value(Space.this);
-                        }
-                        catch (IgniteSpiException e) {
-                            throw new GridRuntimeException(e);
-                        }
-
-                        if (bytes != null) {
-                            next = new T2<>(entry.getKey(), bytes);
-
-                            break;
-                        }
-                    }
-                }
-
-                @Override public Map.Entry<SwapKey, byte[]> next() {
-                    final Map.Entry<SwapKey, byte[]> res = next;
-
-                    if (res == null)
-                        throw new NoSuchElementException();
-
-                    next = null;
-
-                    advance();
-
-                    last = res;
-
-                    return res;
-                }
-
-                @Override public void remove() {
-                    if (last == null)
-                        throw new IllegalStateException();
-
-                    try {
-                        Space.this.remove(last.getKey(), false);
-                    }
-                    catch (IgniteSpiException e) {
-                        throw new GridRuntimeException(e);
-                    }
-                    finally {
-                        last = null;
-                    }
-                }
-            };
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/ef258ece/modules/core/src/main/java/org/gridgain/grid/spi/swapspace/file/FileSwapSpaceSpiMBean.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/spi/swapspace/file/FileSwapSpaceSpiMBean.java b/modules/core/src/main/java/org/gridgain/grid/spi/swapspace/file/FileSwapSpaceSpiMBean.java
deleted file mode 100644
index 12b6a67..0000000
--- a/modules/core/src/main/java/org/gridgain/grid/spi/swapspace/file/FileSwapSpaceSpiMBean.java
+++ /dev/null
@@ -1,59 +0,0 @@
-/* @java.file.header */
-
-/*  _________        _____ __________________        _____
- *  __  ____/___________(_)______  /__  ____/______ ____(_)_______
- *  _  / __  __  ___/__  / _  __  / _  / __  _  __ `/__  / __  __ \
- *  / /_/ /  _  /    _  /  / /_/ /  / /_/ /  / /_/ / _  /  _  / / /
- *  \____/   /_/     /_/   \_,__/   \____/   \__,_/  /_/   /_/ /_/
- */
-
-package org.gridgain.grid.spi.swapspace.file;
-
-import org.apache.ignite.mbean.*;
-import org.apache.ignite.spi.*;
-
-/**
- * Management bean for {@link FileSwapSpaceSpi}.
- */
-@IgniteMBeanDescription("MBean that provides configuration information on file-based swapspace SPI.")
-public interface FileSwapSpaceSpiMBean extends IgniteSpiManagementMBean {
-    /**
-     * Gets base directory.
-     *
-     * @return Base directory.
-     */
-    @IgniteMBeanDescription("Base directory.")
-    public String getBaseDirectory();
-
-    /**
-     * Gets maximum sparsity.
-     *
-     * @return Maximum sparsity.
-     */
-    @IgniteMBeanDescription("Maximum sparsity.")
-    public float getMaximumSparsity();
-
-    /**
-     * Gets write buffer size in bytes.
-     *
-     * @return Write buffer size in bytes.
-     */
-    @IgniteMBeanDescription("Write buffer size in bytes.")
-    public int getWriteBufferSize();
-
-    /**
-     * Gets max write queue size in bytes.
-     *
-     * @return Max write queue size in bytes.
-     */
-    @IgniteMBeanDescription("Max write queue size in bytes.")
-    public int getMaxWriteQueueSize();
-
-    /**
-     * Gets read pool size.
-     *
-     * @return Read pool size.
-     */
-    @IgniteMBeanDescription("Read pool size.")
-    public int getReadStripesNumber();
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/ef258ece/modules/core/src/main/java/org/gridgain/grid/spi/swapspace/file/package.html
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/spi/swapspace/file/package.html b/modules/core/src/main/java/org/gridgain/grid/spi/swapspace/file/package.html
deleted file mode 100644
index cc92a66..0000000
--- a/modules/core/src/main/java/org/gridgain/grid/spi/swapspace/file/package.html
+++ /dev/null
@@ -1,15 +0,0 @@
-<!DOCTYPE html PUBLIC "-//W3C//DTD HTML 4.01 Transitional//EN" "http://www.w3.org/TR/html4/loose.dtd">
-<!--
-    @html.file.header
-    _________        _____ __________________        _____
-    __  ____/___________(_)______  /__  ____/______ ____(_)_______
-    _  / __  __  ___/__  / _  __  / _  / __  _  __ `/__  / __  __ \
-    / /_/ /  _  /    _  /  / /_/ /  / /_/ /  / /_/ / _  /  _  / / /
-    \____/   /_/     /_/   \_,__/   \____/   \__,_/  /_/   /_/ /_/
--->
-<html>
-<body>
-    <!-- Package description. -->
-    Contains file-based swap space SPI.
-</body>
-</html>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/ef258ece/modules/core/src/main/java/org/gridgain/grid/spi/swapspace/noop/NoopSwapSpaceSpi.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/spi/swapspace/noop/NoopSwapSpaceSpi.java b/modules/core/src/main/java/org/gridgain/grid/spi/swapspace/noop/NoopSwapSpaceSpi.java
deleted file mode 100644
index f7f2668..0000000
--- a/modules/core/src/main/java/org/gridgain/grid/spi/swapspace/noop/NoopSwapSpaceSpi.java
+++ /dev/null
@@ -1,126 +0,0 @@
-/* @java.file.header */
-
-/*  _________        _____ __________________        _____
- *  __  ____/___________(_)______  /__  ____/______ ____(_)_______
- *  _  / __  __  ___/__  / _  __  / _  / __  _  __ `/__  / __  __ \
- *  / /_/ /  _  /    _  /  / /_/ /  / /_/ /  / /_/ / _  /  _  / / /
- *  \____/   /_/     /_/   \_,__/   \____/   \__,_/  /_/   /_/ /_/
- */
-
-package org.gridgain.grid.spi.swapspace.noop;
-
-import org.apache.ignite.*;
-import org.apache.ignite.lang.*;
-import org.apache.ignite.resources.*;
-import org.apache.ignite.spi.*;
-import org.gridgain.grid.spi.swapspace.*;
-import org.gridgain.grid.util.*;
-import org.gridgain.grid.util.typedef.internal.*;
-import org.jetbrains.annotations.*;
-
-import java.util.*;
-
-/**
- * No-op implementation of {@link org.gridgain.grid.spi.swapspace.SwapSpaceSpi}. Exists for testing and benchmarking purposes.
- */
-@IgniteSpiNoop
-@IgniteSpiMultipleInstancesSupport(true)
-public class NoopSwapSpaceSpi extends IgniteSpiAdapter implements SwapSpaceSpi {
-    /** Logger. */
-    @IgniteLoggerResource
-    private IgniteLogger log;
-
-    /** {@inheritDoc} */
-    @Override public void spiStart(@Nullable String gridName) throws IgniteSpiException {
-        U.warn(log, "Swap space is disabled. To enable use GridFileSwapSpaceSpi.");
-    }
-
-    /** {@inheritDoc} */
-    @Override public void spiStop() throws IgniteSpiException {
-        // No-op.
-    }
-
-    /** {@inheritDoc} */
-    @Override public void clear(@Nullable String space) throws IgniteSpiException {
-        // No-op.
-    }
-
-    /** {@inheritDoc} */
-    @Override public long size(@Nullable String space) throws IgniteSpiException {
-        return 0;
-    }
-
-    /** {@inheritDoc} */
-    @Override public long count(@Nullable String space) throws IgniteSpiException {
-        return 0;
-    }
-
-    /** {@inheritDoc} */
-    @Override @Nullable public byte[] read(@Nullable String spaceName, SwapKey key, SwapContext ctx)
-        throws IgniteSpiException {
-        return null;
-    }
-
-    /** {@inheritDoc} */
-    @Override public Map<SwapKey, byte[]> readAll(@Nullable String spaceName, Iterable<SwapKey> keys,
-        SwapContext ctx) throws IgniteSpiException {
-        return Collections.emptyMap();
-    }
-
-    /** {@inheritDoc} */
-    @Override public void remove(@Nullable String spaceName, SwapKey key, @Nullable IgniteInClosure<byte[]> c,
-        SwapContext ctx) throws IgniteSpiException {
-        // No-op.
-    }
-
-    /** {@inheritDoc} */
-    @Override public void removeAll(@Nullable String spaceName, Collection<SwapKey> keys,
-        @Nullable IgniteBiInClosure<SwapKey, byte[]> c, SwapContext ctx) throws IgniteSpiException {
-        // No-op.
-    }
-
-    /** {@inheritDoc} */
-    @Override public void store(@Nullable String spaceName, SwapKey key, @Nullable byte[] val,
-        SwapContext ctx) throws IgniteSpiException {
-        // No-op.
-    }
-
-    /** {@inheritDoc} */
-    @Override public void storeAll(@Nullable String spaceName, Map<SwapKey, byte[]> pairs,
-        SwapContext ctx) throws IgniteSpiException {
-        // No-op.
-    }
-
-    /** {@inheritDoc} */
-    @Override public void setListener(@Nullable SwapSpaceSpiListener evictLsnr) {
-        // No-op.
-    }
-
-    /** {@inheritDoc} */
-    @Override public Collection<Integer> partitions(@Nullable String spaceName) throws IgniteSpiException {
-        return Collections.emptyList();
-    }
-
-    /** {@inheritDoc} */
-    @Override public <K> IgniteSpiCloseableIterator<K> keyIterator(@Nullable String spaceName,
-        SwapContext ctx) throws IgniteSpiException {
-        return new GridEmptyCloseableIterator<>();
-    }
-
-    /** {@inheritDoc} */
-    @Override public IgniteSpiCloseableIterator<Map.Entry<byte[], byte[]>> rawIterator(
-        @Nullable String spaceName) throws IgniteSpiException {
-        return new GridEmptyCloseableIterator<>();
-    }
-
-    /** {@inheritDoc} */
-    @Override public IgniteSpiCloseableIterator<Map.Entry<byte[], byte[]>> rawIterator(@Nullable String spaceName,
-        int part) throws IgniteSpiException {
-        return new GridEmptyCloseableIterator<>();
-    }
-
-    /** {@inheritDoc} */
-    @Override public String toString() {
-        return S.toString(NoopSwapSpaceSpi.class, this);
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/ef258ece/modules/core/src/main/java/org/gridgain/grid/spi/swapspace/noop/package.html
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/spi/swapspace/noop/package.html b/modules/core/src/main/java/org/gridgain/grid/spi/swapspace/noop/package.html
deleted file mode 100644
index d003e95..0000000
--- a/modules/core/src/main/java/org/gridgain/grid/spi/swapspace/noop/package.html
+++ /dev/null
@@ -1,15 +0,0 @@
-<!DOCTYPE html PUBLIC "-//W3C//DTD HTML 4.01 Transitional//EN" "http://www.w3.org/TR/html4/loose.dtd">
-<!--
-    @html.file.header
-    _________        _____ __________________        _____
-    __  ____/___________(_)______  /__  ____/______ ____(_)_______
-    _  / __  __  ___/__  / _  __  / _  / __  _  __ `/__  / __  __ \
-    / /_/ /  _  /    _  /  / /_/ /  / /_/ /  / /_/ / _  /  _  / / /
-    \____/   /_/     /_/   \_,__/   \____/   \__,_/  /_/   /_/ /_/
--->
-<html>
-<body>
-    <!-- Package description. -->
-    Contains <b>default</b> no-op swap space SPI implementation.
-</body>
-</html>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/ef258ece/modules/core/src/main/java/org/gridgain/grid/spi/swapspace/package.html
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/spi/swapspace/package.html b/modules/core/src/main/java/org/gridgain/grid/spi/swapspace/package.html
deleted file mode 100644
index 2106182..0000000
--- a/modules/core/src/main/java/org/gridgain/grid/spi/swapspace/package.html
+++ /dev/null
@@ -1,15 +0,0 @@
-<!DOCTYPE html PUBLIC "-//W3C//DTD HTML 4.01 Transitional//EN" "http://www.w3.org/TR/html4/loose.dtd">
-<!--
-    @html.file.header
-    _________        _____ __________________        _____
-    __  ____/___________(_)______  /__  ____/______ ____(_)_______
-    _  / __  __  ___/__  / _  __  / _  / __  _  __ `/__  / __  __ \
-    / /_/ /  _  /    _  /  / /_/ /  / /_/ /  / /_/ / _  /  _  / / /
-    \____/   /_/     /_/   \_,__/   \____/   \__,_/  /_/   /_/ /_/
--->
-<html>
-<body>
-    <!-- Package description. -->
-    Contains APIs for swap space SPI.
-</body>
-</html>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/ef258ece/modules/core/src/test/config/spring-cache-swap.xml
----------------------------------------------------------------------
diff --git a/modules/core/src/test/config/spring-cache-swap.xml b/modules/core/src/test/config/spring-cache-swap.xml
index 1aa7122..cd36e67 100644
--- a/modules/core/src/test/config/spring-cache-swap.xml
+++ b/modules/core/src/test/config/spring-cache-swap.xml
@@ -39,7 +39,7 @@
         <property name="networkTimeout" value="10000"/>
 
         <property name="swapSpaceSpi">
-            <bean class="org.gridgain.grid.spi.swapspace.file.FileSwapSpaceSpi">
+            <bean class="org.apache.ignite.spi.swapspace.file.FileSwapSpaceSpi">
                 <property name="rootFolderPath" value="/Users/yzhdanov/tmp/swap-test"/>
                 <property name="taskQueueCapacity" value="100000"/>
                 <property name="poolSize" value="2"/>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/ef258ece/modules/core/src/test/java/org/apache/ignite/spi/swapspace/GridSwapSpaceSpiAbstractSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/spi/swapspace/GridSwapSpaceSpiAbstractSelfTest.java b/modules/core/src/test/java/org/apache/ignite/spi/swapspace/GridSwapSpaceSpiAbstractSelfTest.java
new file mode 100644
index 0000000..d11c1d6
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/spi/swapspace/GridSwapSpaceSpiAbstractSelfTest.java
@@ -0,0 +1,634 @@
+/* @java.file.header */
+
+/*  _________        _____ __________________        _____
+ *  __  ____/___________(_)______  /__  ____/______ ____(_)_______
+ *  _  / __  __  ___/__  / _  __  / _  / __  _  __ `/__  / __  __ \
+ *  / /_/ /  _  /    _  /  / /_/ /  / /_/ /  / /_/ / _  /  _  / / /
+ *  \____/   /_/     /_/   \_,__/   \____/   \__,_/  /_/   /_/ /_/
+ */
+
+package org.apache.ignite.spi.swapspace;
+
+import org.apache.ignite.lang.*;
+import org.apache.ignite.spi.*;
+import org.gridgain.grid.util.typedef.*;
+import org.gridgain.grid.util.typedef.internal.*;
+import org.gridgain.testframework.junits.common.*;
+import org.jetbrains.annotations.*;
+
+import java.io.*;
+import java.util.*;
+import java.util.concurrent.*;
+
+import static java.util.concurrent.TimeUnit.*;
+import static org.apache.ignite.events.IgniteEventType.*;
+import static org.junit.Assert.*;
+
+/**
+ * Test for various {@link SwapSpaceSpi} implementations.
+ */
+public abstract class GridSwapSpaceSpiAbstractSelfTest extends GridCommonAbstractTest {
+    /** Default swap space name. */
+    private static final String DFLT_SPACE_NAME = "dflt-space";
+
+    /** */
+    protected static final String SPACE1 = "space1";
+
+    /** */
+    protected static final String SPACE2 = "space2";
+
+    /** SPI to test. */
+    protected SwapSpaceSpi spi;
+
+    /**
+     * @return New {@link SwapSpaceSpi} instance.
+     */
+    protected abstract SwapSpaceSpi spi();
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTest() throws Exception {
+        U.setWorkDirectory(null, U.getGridGainHome());
+
+        spi = spi();
+
+        getTestResources().inject(spi);
+
+        spi.spiStart("");
+
+        spi.clear(DFLT_SPACE_NAME);
+    }
+
+    /** @throws Exception If failed. */
+    @Override protected void afterTest() throws Exception {
+        spi.spiStop();
+    }
+
+    /**
+     * @return Swap context.
+     */
+    protected SwapContext context() {
+        return context(null);
+    }
+
+    /**
+     * @param clsLdr Class loader.
+     * @return Swap context.
+     */
+    private SwapContext context(@Nullable ClassLoader clsLdr) {
+        SwapContext ctx = new SwapContext();
+
+        ctx.classLoader(clsLdr != null ? clsLdr : getClass().getClassLoader());
+
+        return ctx;
+    }
+
+    /**
+     * @param s String.
+     * @return Byte array.
+     */
+    protected byte[] str2ByteArray(String s) {
+        return s.getBytes();
+    }
+
+    /**
+     * Tests the Create-Read-Update-Delete operations with a simple key.
+     *
+     * @throws Exception If failed.
+     */
+    public void testSimpleCrud() throws Exception {
+        assertEquals(0, spi.count(DFLT_SPACE_NAME));
+
+        long key1 = 1;
+
+        byte[] val1 = Long.toString(key1).getBytes();
+
+        spi.store(DFLT_SPACE_NAME, new SwapKey(key1), val1, context());
+
+        assertEquals(1, spi.count(DFLT_SPACE_NAME));
+
+        assertArrayEquals(spi.read(DFLT_SPACE_NAME, new SwapKey(key1), context()), val1);
+
+        final byte[] val2 = "newValue".getBytes();
+
+        spi.store(DFLT_SPACE_NAME, new SwapKey(key1), val2, context());
+
+        assertEquals(1, spi.count(DFLT_SPACE_NAME));
+
+        assertArrayEquals(spi.read(DFLT_SPACE_NAME, new SwapKey(key1), context()), val2);
+
+        spi.remove(DFLT_SPACE_NAME, new SwapKey(key1), new IgniteInClosure<byte[]>() {
+            @Override public void apply(byte[] old) {
+                assertArrayEquals(val2, old);
+            }
+        }, context());
+
+        assertEquals(0, spi.count(DFLT_SPACE_NAME));
+    }
+
+    /**
+     * Tests the Create-Read-Update-Delete operations with a simple key
+     * and different spaces.
+     *
+     * @throws Exception If failed.
+     */
+    public void testSimpleCrudDifferentSpaces() throws Exception {
+        String space1 = SPACE1;
+
+        spi.clear(space1);
+
+        String space2 = SPACE2;
+
+        spi.clear(space2);
+
+        assertEquals(0, spi.count(space1));
+
+        assertEquals(0, spi.count(space2));
+
+        long key1 = 1;
+
+        final byte[] val1 = Long.toString(key1).getBytes();
+
+        spi.store(space1, new SwapKey(key1), val1, context());
+
+        assertEquals(1, spi.count(space1));
+
+        assertEquals(0, spi.count(space2));
+
+        spi.store(space2, new SwapKey(key1), val1, context());
+
+        assertEquals(1, spi.count(space1));
+
+        assertEquals(1, spi.count(space2));
+
+        assertArrayEquals(spi.read(space1, new SwapKey(key1), context()), val1);
+
+        assertArrayEquals(spi.read(space2, new SwapKey(key1), context()), val1);
+
+        long key2 = 2;
+
+        byte[] val2 = Long.toString(key2).getBytes();
+
+        spi.store(space1, new SwapKey(key2), val2, context());
+
+        assertEquals(2, spi.count(space1));
+
+        assertEquals(1, spi.count(space2));
+
+        assertArrayEquals(spi.read(space1, new SwapKey(key2), context()), val2);
+
+        assertNull(spi.read(space2, new SwapKey(key2), context()));
+
+        final byte[] val12 = "newValue".getBytes();
+
+        spi.store(space1, new SwapKey(key1), val12, context());
+
+        assertEquals(2, spi.count(space1));
+
+        assertEquals(1, spi.count(space2));
+
+        assertArrayEquals(spi.read(space1, new SwapKey(key1), context()), val12);
+
+        assertArrayEquals(spi.read(space2, new SwapKey(key1), context()), val1);
+
+        spi.remove(space1, new SwapKey(key1), new IgniteInClosure<byte[]>() {
+            @Override public void apply(byte[] old) {
+                assertArrayEquals(val12, old);
+            }
+        }, context());
+
+        assertEquals(1, spi.count(space1));
+
+        assertEquals(1, spi.count(space2));
+
+        spi.remove(space2, new SwapKey(key1), new IgniteInClosure<byte[]>() {
+            @Override public void apply(byte[] old) {
+                assertArrayEquals(val1, old);
+            }
+        }, context());
+
+        assertEquals(1, spi.count(space1));
+
+        assertEquals(0, spi.count(space2));
+    }
+
+    /**
+     * Tests the Create-Update-Delete operations with a key batches.
+     *
+     * @throws Exception If failed.
+     */
+    public void testBatchCrud() throws Exception {
+        assertEquals(0, spi.count(DFLT_SPACE_NAME));
+
+        final Map<SwapKey, byte[]> batch = new HashMap<>();
+
+        int batchSize = 10;
+
+        // Generate initial values.
+        for (int i = 0; i < batchSize; i++)
+            batch.put(new SwapKey(i), Integer.toString(i).getBytes());
+
+        spi.storeAll(DFLT_SPACE_NAME, batch, context());
+
+        assertEquals(batchSize, spi.count(DFLT_SPACE_NAME));
+
+        Map<SwapKey, byte[]> read = spi.readAll(DFLT_SPACE_NAME, batch.keySet(), context());
+
+        // Check all entries are as expected.
+        assertTrue(F.forAll(read, new P1<Map.Entry<SwapKey, byte[]>>() {
+            @Override public boolean apply(Map.Entry<SwapKey, byte[]> e) {
+                return Arrays.equals(batch.get(e.getKey()), e.getValue());
+            }
+        }));
+
+        // Generate new values.
+        for (int i = 0; i < batchSize; i++)
+            batch.put(new SwapKey(i), Integer.toString(i + 1).getBytes());
+
+        spi.storeAll(DFLT_SPACE_NAME, batch, context());
+
+        assertEquals(batchSize, spi.count(DFLT_SPACE_NAME));
+
+        read = spi.readAll(DFLT_SPACE_NAME, batch.keySet(), context());
+
+        // Check all entries are as expected.
+        assertTrue(F.forAll(read, new P1<Map.Entry<SwapKey, byte[]>>() {
+            @Override public boolean apply(Map.Entry<SwapKey, byte[]> e) {
+                return Arrays.equals(batch.get(e.getKey()), e.getValue());
+            }
+        }));
+
+        spi.removeAll(DFLT_SPACE_NAME, batch.keySet(), null, context());
+
+        assertEquals(0, spi.count(DFLT_SPACE_NAME));
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testDeleteIfNotPersist() throws Exception {
+        spi.store(SPACE1, new SwapKey("key1"), "value1".getBytes(), context());
+
+        assertArrayEquals("value1".getBytes(), spi.read(SPACE1, new SwapKey("key1"), context()));
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testStoreReadRemove() throws Exception {
+        int cnt = 5;
+
+        final CountDownLatch storeLatch = new CountDownLatch(cnt);
+        final CountDownLatch readLatch = new CountDownLatch(cnt);
+        final CountDownLatch rmvLatch = new CountDownLatch(cnt);
+
+        spi.setListener(new SwapSpaceSpiListener() {
+            @Override public void onSwapEvent(int evtType, @Nullable String spaceName, @Nullable byte[] keyBytes) {
+                info("Received event: " + evtType);
+
+                if (evtType == EVT_SWAP_SPACE_DATA_STORED)
+                    storeLatch.countDown();
+
+                else if (evtType == EVT_SWAP_SPACE_DATA_READ)
+                    readLatch.countDown();
+
+                else if (evtType == EVT_SWAP_SPACE_DATA_REMOVED)
+                    rmvLatch.countDown();
+
+                else
+                    assert false : "Unexpected event type: " + evtType;
+            }
+        });
+
+        for (int i = 0; i < cnt; i++)
+            assertNull(spi.read(SPACE1, new SwapKey("key" + i), context()));
+
+        for (int i = 0; i < cnt; i++)
+            spi.store(SPACE1, new SwapKey("key" + i), str2ByteArray("value" + i), context());
+
+        assert storeLatch.await(5000, MILLISECONDS);
+
+        for (int i = 0; i < cnt; i++)
+            assertArrayEquals(str2ByteArray("value" + i), spi.read(SPACE1, new SwapKey("key" + i), context()));
+
+        assert readLatch.await(5000, MILLISECONDS);
+
+        for (int i = 0; i < cnt; i++) {
+            final int tmp = i;
+
+            spi.remove(SPACE1, new SwapKey("key" + i), new CI1<byte[]>() {
+                @Override public void apply(byte[] arr) {
+                    assertArrayEquals(str2ByteArray("value" + tmp), arr);
+
+                    info("Removed correct value for: key" + tmp);
+                }
+            }, context());
+        }
+
+        assert rmvLatch.await(10000, MILLISECONDS);
+
+        for (int i = 0; i < cnt; i++)
+            assertNull(spi.read(SPACE1, new SwapKey("key" + i), context()));
+    }
+
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testStoreReadRemoveNulls() throws Exception {
+        int cnt = 5;
+
+        final CountDownLatch storeLatch = new CountDownLatch(cnt);
+        final CountDownLatch readLatch = new CountDownLatch(cnt);
+        final CountDownLatch rmvLatch = new CountDownLatch(cnt);
+
+        spi.setListener(new SwapSpaceSpiListener() {
+            @Override public void onSwapEvent(int evtType, @Nullable String spaceName, @Nullable byte[] keyBytes) {
+                info("Received event: " + evtType);
+
+                if (evtType == EVT_SWAP_SPACE_DATA_STORED)
+                    storeLatch.countDown();
+
+                else if (evtType == EVT_SWAP_SPACE_DATA_READ)
+                    readLatch.countDown();
+
+                else if (evtType == EVT_SWAP_SPACE_DATA_REMOVED)
+                    rmvLatch.countDown();
+
+                else
+                    assert false : "Unexpected event type: " + evtType;
+            }
+        });
+
+        for (int i = 0; i < cnt; i++)
+            assertNull(spi.read(SPACE1, new SwapKey("key" + i), context()));
+
+        for (int i = 0; i < cnt; i++)
+            spi.store(SPACE1, new SwapKey("key" + i), null, context());
+
+        assert storeLatch.await(5000, MILLISECONDS);
+
+        for (int i = 0; i < cnt; i++)
+            assertNull(spi.read(SPACE1, new SwapKey("key" + i), context()));
+
+        assert readLatch.await(5000, MILLISECONDS);
+
+        for (int i = 0; i < cnt; i++) {
+            final int tmp = i;
+
+            spi.remove(SPACE1, new SwapKey("key" + i), new CI1<byte[]>() {
+                @Override public void apply(byte[] arr) {
+                    assertNull(arr);
+
+                    info("Removed correct value for: key" + tmp);
+                }
+            }, context());
+        }
+
+        assert rmvLatch.await(10000, MILLISECONDS);
+
+        for (int i = 0; i < cnt; i++)
+            assertNull(spi.read(SPACE1, new SwapKey("key" + i), context()));
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testCollisions() throws Exception {
+        int cnt = 5;
+
+        final CountDownLatch storeLatch = new CountDownLatch(cnt);
+        final CountDownLatch readLatch = new CountDownLatch(cnt);
+        final CountDownLatch rmvLatch = new CountDownLatch(cnt);
+
+        spi.setListener(new SwapSpaceSpiListener() {
+            @Override public void onSwapEvent(int evtType, @Nullable String spaceName, @Nullable byte[] keyBytes) {
+                info("Received event: " + evtType);
+
+                if (evtType == EVT_SWAP_SPACE_DATA_STORED)
+                    storeLatch.countDown();
+
+                else if (evtType == EVT_SWAP_SPACE_DATA_READ)
+                    readLatch.countDown();
+
+                else if (evtType == EVT_SWAP_SPACE_DATA_REMOVED)
+                    rmvLatch.countDown();
+
+                else
+                    assert false : "Unexpected event type: " + evtType;
+            }
+        });
+
+        List<Integer> keys = new ArrayList<>(cnt);
+
+        final Map<Integer, String> entries = new HashMap<>();
+
+        for (int i = 0; i < cnt; i++) {
+            String val = "value" + i;
+
+            spi.store(SPACE1, new SwapKey(new Key(i)), str2ByteArray(val), context());
+
+            keys.add(i);
+
+            entries.put(i, val);
+        }
+
+        assert storeLatch.await(5000, MILLISECONDS) : "Count: " + storeLatch.getCount();
+
+        for (int i = 0; i < cnt; i++)
+            assertArrayEquals(entries.get(i).getBytes(),
+                spi.read(SPACE1, new SwapKey(new Key(i)), context()));
+
+        assert readLatch.await(5000, MILLISECONDS) : "Count: " + readLatch.getCount();
+
+        Collections.shuffle(keys);
+
+        for (final Integer key : keys) {
+            spi.remove(SPACE1, new SwapKey(new Key(key)), new CI1<byte[]>() {
+                @Override public void apply(byte[] arr) {
+                    assertArrayEquals(entries.get(key).getBytes(), arr);
+
+                    info("Removed correct entry for key: " + key);
+                }
+            }, context());
+        }
+
+        assert rmvLatch.await(5000, MILLISECONDS) : "Count: " + rmvLatch.getCount();
+
+        for (final Integer key : keys)
+            assertNull(spi.read(SPACE1, new SwapKey(new Key(key)), context()));
+    }
+
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testIteration() throws Exception {
+        spi.clear(SPACE1);
+
+        int cnt = 10;
+
+        for (int i = 0; i < cnt; i++)
+            spi.store(SPACE1, new SwapKey("key" + i, i), str2ByteArray("value" + i), context());
+
+        for (int i = 0; i < cnt; i++)
+            assertArrayEquals(str2ByteArray("value" + i),
+                spi.read(SPACE1, new SwapKey("key" + i, i), context()));
+
+        try (IgniteSpiCloseableIterator<Map.Entry<byte[], byte[]>> iter = spi.rawIterator(SPACE1)) {
+            assertNotNull(iter);
+
+            int i = 0;
+
+            while (iter.hasNext()) {
+                Map.Entry<byte[], byte[]> next = iter.next();
+
+                String key = getTestResources().getMarshaller().unmarshal(next.getKey(), null);
+
+                info("Got from iterator [key=" + key + ", val=" + new String(next.getValue()));
+
+                i++;
+
+                iter.remove();
+            }
+
+            assertEquals(10, i);
+        }
+
+        assertEquals(0, spi.count(SPACE1));
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testIterationOverPartition() throws Exception {
+        spi.store(SPACE1, new SwapKey("key", 0), str2ByteArray("value"), context());
+
+        spi.clear(SPACE1);
+
+        int cnt = 10;
+
+        for (int i = 0; i < cnt; i++)
+            spi.store(SPACE1, new SwapKey("key" + i, i), str2ByteArray("value" + i), context());
+
+        for (int i = 0; i < cnt; i++)
+            assertArrayEquals(str2ByteArray("value" + i),
+                spi.read(SPACE1, new SwapKey("key" + i, i), context()));
+
+        try (IgniteSpiCloseableIterator<Map.Entry<byte[], byte[]>> iter = spi.rawIterator(SPACE1, 5)) {
+            assertNotNull(iter);
+
+            int i = 0;
+
+            while (iter.hasNext()) {
+                Map.Entry<byte[], byte[]> next = iter.next();
+
+                String key = getTestResources().getMarshaller().unmarshal(next.getKey(), null);
+
+                info("Got from iterator [key=" + key + ", val=" + new String(next.getValue()));
+
+                assert "key5".equals(key);
+
+                iter.remove();
+
+                assertNull(spi.read(SPACE1, new SwapKey(key, 5), context()));
+
+                i++;
+            }
+
+            assertEquals(1, i);
+        }
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testSwapIterator() throws Exception {
+        spi.store(SPACE1, new SwapKey("key", 0), str2ByteArray("value"), context());
+
+        spi.clear(SPACE1);
+
+        int cnt = 10;
+
+        for (int i = 0; i < cnt; i++)
+            spi.store(SPACE1, new SwapKey("key" + i, i), str2ByteArray("value" + i), context());
+
+        IgniteSpiCloseableIterator<Map.Entry<byte[], byte[]>> iter = spi.rawIterator(SPACE1);
+
+        assertNotNull(iter);
+
+        iter.close();
+
+        try {
+            iter.next();
+
+            assert false;
+        }
+        catch (NoSuchElementException e) {
+            info("Caught expected exception (illegal state): " + e);
+        }
+    }
+
+    /**
+     *
+     */
+    @SuppressWarnings({"PublicInnerClass"})
+    public static class TestValue implements Serializable {
+        /** */
+        private String val = "test-" + System.currentTimeMillis();
+
+        /**
+         * @return Value
+         */
+        public String getValue() {
+            return val;
+        }
+
+        /** {@inheritDoc} */
+        @Override public boolean equals(Object obj) {
+            return obj instanceof TestValue && val.equals(((TestValue)obj).val);
+        }
+
+        /** {@inheritDoc} */
+        @Override public String toString() {
+            return S.toString(TestValue.class, this);
+        }
+    }
+
+    /**
+     * Key.
+     */
+    private static class Key {
+        /** Index. */
+        private final int i;
+
+        /**
+         * @param i Index.
+         */
+        Key(int i) {
+            this.i = i;
+        }
+
+        /** {@inheritDoc} */
+        @Override public boolean equals(Object o) {
+            if (this == o)
+                return true;
+
+            if (!(o instanceof Key))
+                return false;
+
+            Key key = (Key)o;
+
+            return i == key.i;
+        }
+
+        /** {@inheritDoc} */
+        @Override public int hashCode() {
+            return 1; // 100% collision.
+        }
+
+        /** {@inheritDoc} */
+        @Override public String toString() {
+            return "Key: " + i;
+        }
+    }
+}


[19/20] incubator-ignite git commit: # Renaming

Posted by sb...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/8c20bddb/modules/core/src/main/java/org/gridgain/grid/kernal/visor/event/VisorGridAuthenticationEvent.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/kernal/visor/event/VisorGridAuthenticationEvent.java b/modules/core/src/main/java/org/gridgain/grid/kernal/visor/event/VisorGridAuthenticationEvent.java
index 0ddb437..2e1224a 100644
--- a/modules/core/src/main/java/org/gridgain/grid/kernal/visor/event/VisorGridAuthenticationEvent.java
+++ b/modules/core/src/main/java/org/gridgain/grid/kernal/visor/event/VisorGridAuthenticationEvent.java
@@ -10,7 +10,7 @@
 package org.gridgain.grid.kernal.visor.event;
 
 import org.apache.ignite.lang.*;
-import org.gridgain.grid.security.*;
+import org.apache.ignite.plugin.security.*;
 import org.gridgain.grid.util.typedef.internal.*;
 
 import java.util.*;

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/8c20bddb/modules/core/src/main/java/org/gridgain/grid/kernal/visor/event/VisorGridAuthorizationEvent.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/kernal/visor/event/VisorGridAuthorizationEvent.java b/modules/core/src/main/java/org/gridgain/grid/kernal/visor/event/VisorGridAuthorizationEvent.java
index 43217d8..36a53f9 100644
--- a/modules/core/src/main/java/org/gridgain/grid/kernal/visor/event/VisorGridAuthorizationEvent.java
+++ b/modules/core/src/main/java/org/gridgain/grid/kernal/visor/event/VisorGridAuthorizationEvent.java
@@ -10,7 +10,7 @@
 package org.gridgain.grid.kernal.visor.event;
 
 import org.apache.ignite.lang.*;
-import org.gridgain.grid.security.*;
+import org.apache.ignite.plugin.security.*;
 import org.gridgain.grid.util.typedef.internal.*;
 
 import java.util.*;

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/8c20bddb/modules/core/src/main/java/org/gridgain/grid/kernal/visor/event/VisorGridSecuritySessionEvent.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/kernal/visor/event/VisorGridSecuritySessionEvent.java b/modules/core/src/main/java/org/gridgain/grid/kernal/visor/event/VisorGridSecuritySessionEvent.java
index 7f9b682..9511b48 100644
--- a/modules/core/src/main/java/org/gridgain/grid/kernal/visor/event/VisorGridSecuritySessionEvent.java
+++ b/modules/core/src/main/java/org/gridgain/grid/kernal/visor/event/VisorGridSecuritySessionEvent.java
@@ -10,7 +10,7 @@
 package org.gridgain.grid.kernal.visor.event;
 
 import org.apache.ignite.lang.*;
-import org.gridgain.grid.security.*;
+import org.apache.ignite.plugin.security.*;
 import org.gridgain.grid.util.typedef.internal.*;
 
 import java.util.*;

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/8c20bddb/modules/core/src/main/java/org/gridgain/grid/kernal/visor/node/VisorSegmentationConfiguration.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/kernal/visor/node/VisorSegmentationConfiguration.java b/modules/core/src/main/java/org/gridgain/grid/kernal/visor/node/VisorSegmentationConfiguration.java
index 9980483..e381ef7 100644
--- a/modules/core/src/main/java/org/gridgain/grid/kernal/visor/node/VisorSegmentationConfiguration.java
+++ b/modules/core/src/main/java/org/gridgain/grid/kernal/visor/node/VisorSegmentationConfiguration.java
@@ -10,7 +10,7 @@
 package org.gridgain.grid.kernal.visor.node;
 
 import org.apache.ignite.configuration.*;
-import org.gridgain.grid.segmentation.*;
+import org.apache.ignite.plugin.segmentation.*;
 import org.gridgain.grid.util.typedef.internal.*;
 import org.jetbrains.annotations.*;
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/8c20bddb/modules/core/src/main/java/org/gridgain/grid/security/GridSecurity.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/security/GridSecurity.java b/modules/core/src/main/java/org/gridgain/grid/security/GridSecurity.java
deleted file mode 100644
index 1d7b05f..0000000
--- a/modules/core/src/main/java/org/gridgain/grid/security/GridSecurity.java
+++ /dev/null
@@ -1,89 +0,0 @@
-/* @java.file.header */
-
-/*  _________        _____ __________________        _____
- *  __  ____/___________(_)______  /__  ____/______ ____(_)_______
- *  _  / __  __  ___/__  / _  __  / _  / __  _  __ `/__  / __  __ \
- *  / /_/ /  _  /    _  /  / /_/ /  / /_/ /  / /_/ / _  /  _  / / /
- *  \____/   /_/     /_/   \_,__/   \____/   \__,_/  /_/   /_/ /_/
- */
-
-package org.gridgain.grid.security;
-
-import org.gridgain.grid.*;
-
-import java.util.*;
-
-/**
- * Grid security facade. This facade contains information about authenticated subjects
- * currently logged in to grid together with their permission sets.
- * <p>
- * You can get an instance of security facade from {@link org.apache.ignite.Ignite#security()} method.
- * <h1 class="header">Grid Nodes vs Remote Clients</h1>
- * When security is enabled, both grid nodes and remote clients must be authenticated.
- * For grid nodes, authentication parameters are specified in grid configuration via
- * {@link org.apache.ignite.configuration.IgniteConfiguration#getSecurityCredentialsProvider()} provider. Here is an example
- * of how a simple user name and password may be provided:
- * <pre class="brush: java">
- *     GridConfiguration cfg = new GridConfiguration();
- *
- *     GridSecurityCredentials creds = new GridSecurityCredentials("username", "password");
- *
- *     cfg.setSecurityCredentialsProvider(new GridSecurityCredentialsBasicProvider(creds));
- *
- *     Grid grid = GridGain.start(cfg);
- * </pre>
- * For remote Java client, configuration is provided in a similar way by specifying
- * {@code GridClientConfiguration.setSecurityCredentialsProvider(...)} property.
- * <p>
- * For remote C++ and .NET clients, security credentials are provided in configuration
- * as well in the form of {@code "username:password"} string.
- * <h1 class="header">Authentication And Authorization</h1>
- * Node or client authentication happens in {@link org.apache.ignite.spi.authentication.AuthenticationSpi}. Upon successful
- * authentication, the SPI will return list of permissions for authenticated subject.
- * <p>
- * GridGain ships with following authentication SPIs out of the box:
- * <ul>
- * <li>{@code GridJaasAuthenticationSpi} - provides authentication based on JAAS standard.</li>
- * <li>{@code GridPasscodeAuthenticationSpi} - basic username and password authentication.</li>
- * </ul>
- * All permissions supported by GridGain are provided in {@link GridSecurityPermission} enum. Permissions
- * are specified on per-cache or per-task level (wildcards are allowed). Authentication SPIs should usually
- * (although not required) specify security permissions in the following JSON format:
- * <pre class="brush: text">
- * {
- *     {
- *         "cache":"partitioned",
- *         "permissions":["CACHE_PUT", "CACHE_REMOVE", "CACHE_READ"]
- *     },
- *     {
- *         "cache":"*",
- *         "permissions":["CACHE_READ"]
- *     },
- *     {
- *         "task":"org.mytasks.*",
- *         "permissions":["TASK_EXECUTE"]
- *     },
- *     {
- *         "system":["EVENTS_ENABLE", "ADMIN_VIEW"]
- *     }
- *     "defaultAllow":"false"
- * }
- * </pre>
- * Refer to documentation of available authentication SPIs for more information.
- */
-public interface GridSecurity {
-    /**
-     * Gets collection of authenticated subjects together with their permissions.
-     *
-     * @return Collection of authenticated subjects.
-     */
-    public Collection<GridSecuritySubject> authenticatedSubjects() throws GridException;
-
-    /**
-     * Gets security subject based on subject ID.
-     *
-     * @param subjId Subject ID.
-     * @return Authorized security subject.
-     */
-    public GridSecuritySubject authenticatedSubject(UUID subjId) throws GridException;
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/8c20bddb/modules/core/src/main/java/org/gridgain/grid/security/GridSecurityCredentials.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/security/GridSecurityCredentials.java b/modules/core/src/main/java/org/gridgain/grid/security/GridSecurityCredentials.java
deleted file mode 100644
index 2cccfbc..0000000
--- a/modules/core/src/main/java/org/gridgain/grid/security/GridSecurityCredentials.java
+++ /dev/null
@@ -1,196 +0,0 @@
-/* @java.file.header */
-
-/*  _________        _____ __________________        _____
- *  __  ____/___________(_)______  /__  ____/______ ____(_)_______
- *  _  / __  __  ___/__  / _  __  / _  / __  _  __ `/__  / __  __ \
- *  / /_/ /  _  /    _  /  / /_/ /  / /_/ /  / /_/ / _  /  _  / / /
- *  \____/   /_/     /_/   \_,__/   \____/   \__,_/  /_/   /_/ /_/
- */
-
-package org.gridgain.grid.security;
-
-import org.apache.ignite.portables.*;
-import org.gridgain.grid.util.tostring.*;
-import org.gridgain.grid.util.typedef.*;
-import org.gridgain.grid.util.typedef.internal.*;
-import org.jetbrains.annotations.*;
-
-import java.io.*;
-
-/**
- * Security credentials used for client or node authentication. Security credentials
- * are provided by {@link GridSecurityCredentialsProvider} which is specified on
- * client or node startup in configuration.
- * <p>
- * For grid node, security credentials provider is specified in
- * {@link org.apache.ignite.configuration.IgniteConfiguration#setSecurityCredentialsProvider(GridSecurityCredentialsProvider)}
- * configuration property. For Java clients, you can provide credentials in
- * {@code GridClientConfiguration.setSecurityCredentialsProvider(...)} method.
- * <p>
- * Getting credentials through {@link GridSecurityCredentialsProvider} abstraction allows
- * users to provide custom implementations for storing user names and passwords in their
- * environment, possibly in encrypted format. GridGain comes with
- * {@link GridSecurityCredentialsBasicProvider} which simply provides
- * the passed in {@code login} and {@code password} when encryption or custom logic is not required.
- * <p>
- * In addition to {@code login} and {@code password}, security credentials allow for
- * specifying {@link #setUserObject(Object) userObject} as well, which can be used
- * to pass in any additional information required for authentication.
- */
-public class GridSecurityCredentials implements Externalizable, PortableMarshalAware {
-    /** */
-    private static final long serialVersionUID = -2655741071578326256L;
-
-    /** Login. */
-    private Object login;
-
-    /** Password. */
-    @GridToStringExclude
-    private Object password;
-
-    /** Additional user object. */
-    @GridToStringExclude
-    private Object userObj;
-
-    /**
-     * Empty constructor required by {@link Externalizable}.
-     */
-    public GridSecurityCredentials() {
-        // No-op.
-    }
-
-    /**
-     * Constructs security credentials based on {@code login} and {@code password}.
-     *
-     * @param login Login.
-     * @param password Password.
-     */
-    public GridSecurityCredentials(String login, String password) {
-        this.login = login;
-        this.password = password;
-    }
-
-    /**
-     * Constructs security credentials based on {@code login}, {@code password},
-     * and custom user object.
-     *
-     * @param login Login.
-     * @param password Password.
-     * @param userObj User object.
-     */
-    public GridSecurityCredentials(String login, String password, @Nullable Object userObj) {
-        this.login = login;
-        this.password = password;
-        this.userObj = userObj;
-    }
-
-    /**
-     * Gets login.
-     *
-     * @return Login.
-     */
-    public Object getLogin() {
-        return login;
-    }
-
-    /**
-     * Sets login.
-     *
-     * @param login Login.
-     */
-    public void setLogin(Object login) {
-        this.login = login;
-    }
-
-    /**
-     * Gets password.
-     *
-     * @return Password.
-     */
-    public Object getPassword() {
-        return password;
-    }
-
-    /**
-     * Sets password.
-     *
-     * @param password Password.
-     */
-    public void setPassword(Object password) {
-        this.password = password;
-    }
-
-    /**
-     * Gets user-specific object.
-     *
-     * @return User object.
-     */
-    @Nullable public Object getUserObject() {
-        return userObj;
-    }
-
-    /**
-     * Sets user-specific object.
-     *
-     * @param userObj User object.
-     */
-    public void setUserObject(@Nullable Object userObj) {
-        this.userObj = userObj;
-    }
-
-    /** {@inheritDoc} */
-    @Override public void writeExternal(ObjectOutput out) throws IOException {
-        out.writeObject(login);
-        out.writeObject(password);
-        out.writeObject(userObj);
-    }
-
-    /** {@inheritDoc} */
-    @Override public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException {
-        login = in.readObject();
-        password = in.readObject();
-        userObj = in.readObject();
-    }
-
-    /** {@inheritDoc} */
-    @Override public boolean equals(Object o) {
-        if (this == o)
-            return true;
-
-        if (!(o instanceof GridSecurityCredentials))
-            return false;
-
-        GridSecurityCredentials that = (GridSecurityCredentials)o;
-
-        return F.eq(login, that.login) && F.eq(password, that.password) && F.eq(userObj, that.userObj);
-    }
-
-    /** {@inheritDoc} */
-    @Override public int hashCode() {
-        int res = login != null ? login.hashCode() : 0;
-
-        res = 31 * res + (password != null ? password.hashCode() : 0);
-        res = 31 * res + (userObj != null ? userObj.hashCode() : 0);
-
-        return res;
-    }
-
-    /** {@inheritDoc} */
-    @Override public void writePortable(PortableWriter writer) throws PortableException {
-        writer.rawWriter().writeObject(login);
-        writer.rawWriter().writeObject(password);
-        writer.rawWriter().writeObject(userObj);
-    }
-
-    /** {@inheritDoc} */
-    @Override public void readPortable(PortableReader reader) throws PortableException {
-        login = reader.rawReader().readObject();
-        password = reader.rawReader().readObject();
-        userObj = reader.rawReader().readObject();
-    }
-
-    /** {@inheritDoc} */
-    @Override public String toString() {
-        return S.toString(GridSecurityCredentials.class, this);
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/8c20bddb/modules/core/src/main/java/org/gridgain/grid/security/GridSecurityCredentialsBasicProvider.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/security/GridSecurityCredentialsBasicProvider.java b/modules/core/src/main/java/org/gridgain/grid/security/GridSecurityCredentialsBasicProvider.java
deleted file mode 100644
index 1a33152..0000000
--- a/modules/core/src/main/java/org/gridgain/grid/security/GridSecurityCredentialsBasicProvider.java
+++ /dev/null
@@ -1,36 +0,0 @@
-/* @java.file.header */
-
-/*  _________        _____ __________________        _____
- *  __  ____/___________(_)______  /__  ____/______ ____(_)_______
- *  _  / __  __  ___/__  / _  __  / _  / __  _  __ `/__  / __  __ \
- *  / /_/ /  _  /    _  /  / /_/ /  / /_/ /  / /_/ / _  /  _  / / /
- *  \____/   /_/     /_/   \_,__/   \____/   \__,_/  /_/   /_/ /_/
- */
-
-package org.gridgain.grid.security;
-
-import org.gridgain.grid.*;
-
-/**
- * Basic implementation for {@link GridSecurityCredentialsProvider}. Use it
- * when custom logic for storing security credentials is not required and it
- * is OK to specify credentials directly in configuration.
- */
-public class GridSecurityCredentialsBasicProvider implements GridSecurityCredentialsProvider {
-    /** */
-    private GridSecurityCredentials cred;
-
-    /**
-     * Constructs security credentials provider based on security credentials passed in.
-     *
-     * @param cred Security credentials.
-     */
-    public GridSecurityCredentialsBasicProvider(GridSecurityCredentials cred) {
-        this.cred = cred;
-    }
-
-    /** {@inheritDoc} */
-    @Override public GridSecurityCredentials credentials() throws GridException {
-        return cred;
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/8c20bddb/modules/core/src/main/java/org/gridgain/grid/security/GridSecurityCredentialsProvider.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/security/GridSecurityCredentialsProvider.java b/modules/core/src/main/java/org/gridgain/grid/security/GridSecurityCredentialsProvider.java
deleted file mode 100644
index afdfece..0000000
--- a/modules/core/src/main/java/org/gridgain/grid/security/GridSecurityCredentialsProvider.java
+++ /dev/null
@@ -1,41 +0,0 @@
-/* @java.file.header */
-
-/*  _________        _____ __________________        _____
- *  __  ____/___________(_)______  /__  ____/______ ____(_)_______
- *  _  / __  __  ___/__  / _  __  / _  / __  _  __ `/__  / __  __ \
- *  / /_/ /  _  /    _  /  / /_/ /  / /_/ /  / /_/ / _  /  _  / / /
- *  \____/   /_/     /_/   \_,__/   \____/   \__,_/  /_/   /_/ /_/
- */
-
-package org.gridgain.grid.security;
-
-import org.gridgain.grid.*;
-
-/**
- * Security credentials provider for specifying security credentials.
- * Security credentials used for client or node authentication.
- * <p>
- * For grid node, security credentials provider is specified in
- * {@link org.apache.ignite.configuration.IgniteConfiguration#setSecurityCredentialsProvider(GridSecurityCredentialsProvider)}
- * configuration property. For Java clients, you can provide credentials in
- * {@code GridClientConfiguration.setSecurityCredentialsProvider(...)} method.
- * <p>
- * Getting credentials through {@link GridSecurityCredentialsProvider} abstraction allows
- * users to provide custom implementations for storing user names and passwords in their
- * environment, possibly in encrypted format. GridGain comes with
- * {@link GridSecurityCredentialsBasicProvider} which simply provides
- * the passed in {@code login} and {@code password} when encryption or custom logic is not required.
- * <p>
- * In addition to {@code login} and {@code password}, security credentials allow for
- * specifying {@link GridSecurityCredentials#setUserObject(Object) userObject} as well, which can be used
- * to pass in any additional information required for authentication.
- */
-public interface GridSecurityCredentialsProvider {
-    /**
-     * Gets security credentials.
-     *
-     * @return Security credentials.
-     * @throws GridException If failed.
-     */
-    public GridSecurityCredentials credentials() throws GridException;
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/8c20bddb/modules/core/src/main/java/org/gridgain/grid/security/GridSecurityException.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/security/GridSecurityException.java b/modules/core/src/main/java/org/gridgain/grid/security/GridSecurityException.java
deleted file mode 100644
index a5d13fd..0000000
--- a/modules/core/src/main/java/org/gridgain/grid/security/GridSecurityException.java
+++ /dev/null
@@ -1,50 +0,0 @@
-/* @java.file.header */
-
-/*  _________        _____ __________________        _____
- *  __  ____/___________(_)______  /__  ____/______ ____(_)_______
- *  _  / __  __  ___/__  / _  __  / _  / __  _  __ `/__  / __  __ \
- *  / /_/ /  _  /    _  /  / /_/ /  / /_/ /  / /_/ / _  /  _  / / /
- *  \____/   /_/     /_/   \_,__/   \____/   \__,_/  /_/   /_/ /_/
- */
-
-package org.gridgain.grid.security;
-
-import org.gridgain.grid.*;
-import org.jetbrains.annotations.*;
-
-/**
- * Common security exception for the grid.
- */
-public class GridSecurityException extends GridRuntimeException {
-    /** */
-    private static final long serialVersionUID = 0L;
-
-    /**
-     * Constructs security grid exception with given message and cause.
-     *
-     * @param msg Exception message.
-     * @param cause Exception cause.
-     */
-    public GridSecurityException(String msg, @Nullable Throwable cause) {
-        super(msg, cause);
-    }
-
-    /**
-     * Creates new security grid exception given throwable as a cause and
-     * source of error message.
-     *
-     * @param cause Non-null throwable cause.
-     */
-    public GridSecurityException(Throwable cause) {
-        this(cause.getMessage(), cause);
-    }
-
-    /**
-     * Constructs security grid exception with given message.
-     *
-     * @param msg Exception message.
-     */
-    public GridSecurityException(String msg) {
-        super(msg);
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/8c20bddb/modules/core/src/main/java/org/gridgain/grid/security/GridSecurityPermission.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/security/GridSecurityPermission.java b/modules/core/src/main/java/org/gridgain/grid/security/GridSecurityPermission.java
deleted file mode 100644
index 4cfabec..0000000
--- a/modules/core/src/main/java/org/gridgain/grid/security/GridSecurityPermission.java
+++ /dev/null
@@ -1,61 +0,0 @@
-/* @java.file.header */
-
-/*  _________        _____ __________________        _____
- *  __  ____/___________(_)______  /__  ____/______ ____(_)_______
- *  _  / __  __  ___/__  / _  __  / _  / __  _  __ `/__  / __  __ \
- *  / /_/ /  _  /    _  /  / /_/ /  / /_/ /  / /_/ / _  /  _  / / /
- *  \____/   /_/     /_/   \_,__/   \____/   \__,_/  /_/   /_/ /_/
- */
-
-package org.gridgain.grid.security;
-
-import org.jetbrains.annotations.*;
-
-/**
- * Supported security permissions within grid. Permissions
- * are specified on per-cache or per-task level.
- */
-public enum GridSecurityPermission {
-    /** Cache {@code read} permission. */
-    CACHE_READ,
-
-    /** Cache {@code put} permission. */
-    CACHE_PUT,
-
-    /** Cache {@code remove} permission. */
-    CACHE_REMOVE,
-
-    /** Task {@code execute} permission. */
-    TASK_EXECUTE,
-
-    /** Task {@code cancel} permission. */
-    TASK_CANCEL,
-
-    /** Events {@code enable} permission. */
-    EVENTS_ENABLE,
-
-    /** Events {@code disable} permission. */
-    EVENTS_DISABLE,
-
-    /** Common visor tasks permission. */
-    ADMIN_VIEW,
-
-    /** Visor cache read (query) permission. */
-    ADMIN_QUERY,
-
-    /** Visor cache load permission. */
-    ADMIN_CACHE;
-
-    /** Enumerated values. */
-    private static final GridSecurityPermission[] VALS = values();
-
-    /**
-     * Efficiently gets enumerated value from its ordinal.
-     *
-     * @param ord Ordinal value.
-     * @return Enumerated value or {@code null} if ordinal out of range.
-     */
-    @Nullable public static GridSecurityPermission fromOrdinal(int ord) {
-        return ord >= 0 && ord < VALS.length ? VALS[ord] : null;
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/8c20bddb/modules/core/src/main/java/org/gridgain/grid/security/GridSecurityPermissionSet.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/security/GridSecurityPermissionSet.java b/modules/core/src/main/java/org/gridgain/grid/security/GridSecurityPermissionSet.java
deleted file mode 100644
index 0a89704..0000000
--- a/modules/core/src/main/java/org/gridgain/grid/security/GridSecurityPermissionSet.java
+++ /dev/null
@@ -1,58 +0,0 @@
-/* @java.file.header */
-
-/*  _________        _____ __________________        _____
- *  __  ____/___________(_)______  /__  ____/______ ____(_)_______
- *  _  / __  __  ___/__  / _  __  / _  / __  _  __ `/__  / __  __ \
- *  / /_/ /  _  /    _  /  / /_/ /  / /_/ /  / /_/ / _  /  _  / / /
- *  \____/   /_/     /_/   \_,__/   \____/   \__,_/  /_/   /_/ /_/
- */
-
-package org.gridgain.grid.security;
-
-import org.jetbrains.annotations.*;
-
-import java.io.*;
-import java.util.*;
-
-/**
- * Security permission set for authorized security subjects. Permission set
- * allows to specify task permissions for every task and cache permissions
- * for every cache. While cards are supported at the end of task or
- * cache name.
- * <p>
- * Property {@link #defaultAllowAll()} specifies whether to allow or deny
- * cache and task operations if they were not explicitly specified.
- */
-public interface GridSecurityPermissionSet extends Serializable {
-    /**
-     * Flag indicating whether to allow or deny cache and task operations
-     * if they were not explicitly specified.
-     *
-     * @return {@code True} to allow all cache task operations if they were
-     *      not explicitly specified, {@code false} otherwise.
-     */
-    public boolean defaultAllowAll();
-
-    /**
-     * Map of task names to task permissions. Wildcards are allowed at the
-     * end of task names.
-     *
-     * @return Map of task names to task permissions.
-     */
-    public Map<String, Collection<GridSecurityPermission>> taskPermissions();
-
-    /**
-     * Map of cache names to cache permissions. Wildcards are allowed at the
-     * end of cache names.
-     *
-     * @return Map of cache names to cache permissions.
-     */
-    public Map<String, Collection<GridSecurityPermission>> cachePermissions();
-
-    /**
-     * Collection of system-wide permissions (events enable/disable, Visor task execution).
-     *
-     * @return Collection of system-wide permissions.
-     */
-    @Nullable public Collection<GridSecurityPermission> systemPermissions();
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/8c20bddb/modules/core/src/main/java/org/gridgain/grid/security/GridSecuritySubject.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/security/GridSecuritySubject.java b/modules/core/src/main/java/org/gridgain/grid/security/GridSecuritySubject.java
deleted file mode 100644
index c289ec8..0000000
--- a/modules/core/src/main/java/org/gridgain/grid/security/GridSecuritySubject.java
+++ /dev/null
@@ -1,55 +0,0 @@
-/* @java.file.header */
-
-/*  _________        _____ __________________        _____
- *  __  ____/___________(_)______  /__  ____/______ ____(_)_______
- *  _  / __  __  ___/__  / _  __  / _  / __  _  __ `/__  / __  __ \
- *  / /_/ /  _  /    _  /  / /_/ /  / /_/ /  / /_/ / _  /  _  / / /
- *  \____/   /_/     /_/   \_,__/   \____/   \__,_/  /_/   /_/ /_/
- */
-
-package org.gridgain.grid.security;
-
-import java.io.*;
-import java.net.*;
-import java.util.*;
-
-/**
- * Security subject representing authenticated node or client with a set of permissions.
- * List of authenticated subjects can be retrieved from {@link GridSecurity#authenticatedSubjects()} method.
- */
-public interface GridSecuritySubject extends Serializable {
-    /**
-     * Gets subject ID.
-     *
-     * @return Subject ID.
-     */
-    public UUID id();
-
-    /**
-     * Gets subject type, either node or client.
-     *
-     * @return Subject type.
-     */
-    public GridSecuritySubjectType type();
-
-    /**
-     * Login provided via subject security credentials.
-     *
-     * @return Login object.
-     */
-    public Object login();
-
-    /**
-     * Gets subject connection address. Usually {@link InetSocketAddress} representing connection IP and port.
-     *
-     * @return Subject connection address.
-     */
-    public InetSocketAddress address();
-
-    /**
-     * Authorized permission set for the subject.
-     *
-     * @return Authorized permission set for the subject.
-     */
-    public GridSecurityPermissionSet permissions();
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/8c20bddb/modules/core/src/main/java/org/gridgain/grid/security/GridSecuritySubjectType.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/security/GridSecuritySubjectType.java b/modules/core/src/main/java/org/gridgain/grid/security/GridSecuritySubjectType.java
deleted file mode 100644
index 8af3476..0000000
--- a/modules/core/src/main/java/org/gridgain/grid/security/GridSecuritySubjectType.java
+++ /dev/null
@@ -1,40 +0,0 @@
-/* @java.file.header */
-
-/*  _________        _____ __________________        _____
- *  __  ____/___________(_)______  /__  ____/______ ____(_)_______
- *  _  / __  __  ___/__  / _  __  / _  / __  _  __ `/__  / __  __ \
- *  / /_/ /  _  /    _  /  / /_/ /  / /_/ /  / /_/ / _  /  _  / / /
- *  \____/   /_/     /_/   \_,__/   \____/   \__,_/  /_/   /_/ /_/
- */
-
-package org.gridgain.grid.security;
-
-import org.jetbrains.annotations.*;
-
-/**
- * Supported security subject types. Subject type can be retrieved form {@link GridSecuritySubject#type()} method.
- */
-public enum GridSecuritySubjectType {
-    /**
-     * Subject type for a remote {@link org.apache.ignite.cluster.ClusterNode}.
-     */
-    REMOTE_NODE,
-
-    /**
-     * Subject type for remote client.
-     */
-    REMOTE_CLIENT;
-
-    /** Enumerated values. */
-    private static final GridSecuritySubjectType[] VALS = values();
-
-    /**
-     * Efficiently gets enumerated value from its ordinal.
-     *
-     * @param ord Ordinal value.
-     * @return Enumerated value.
-     */
-    @Nullable public static GridSecuritySubjectType fromOrdinal(byte ord) {
-        return ord >= 0 && ord < VALS.length ? VALS[ord] : null;
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/8c20bddb/modules/core/src/main/java/org/gridgain/grid/security/package.html
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/security/package.html b/modules/core/src/main/java/org/gridgain/grid/security/package.html
deleted file mode 100644
index bd468a3..0000000
--- a/modules/core/src/main/java/org/gridgain/grid/security/package.html
+++ /dev/null
@@ -1,15 +0,0 @@
-<!DOCTYPE html PUBLIC "-//W3C//DTD HTML 4.01 Transitional//EN" "http://www.w3.org/TR/html4/loose.dtd">
-<!--
-    @html.file.header
-    _________        _____ __________________        _____
-    __  ____/___________(_)______  /__  ____/______ ____(_)_______
-    _  / __  __  ___/__  / _  __  / _  / __  _  __ `/__  / __  __ \
-    / /_/ /  _  /    _  /  / /_/ /  / /_/ /  / /_/ / _  /  _  / / /
-    \____/   /_/     /_/   \_,__/   \____/   \__,_/  /_/   /_/ /_/
--->
-<html>
-<body>
-    <!-- Package description. -->
-    Contains APIs for security, authentication, and authorization.
-</body>
-</html>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/8c20bddb/modules/core/src/main/java/org/gridgain/grid/segmentation/GridSegmentationPolicy.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/segmentation/GridSegmentationPolicy.java b/modules/core/src/main/java/org/gridgain/grid/segmentation/GridSegmentationPolicy.java
deleted file mode 100644
index 21c6d14..0000000
--- a/modules/core/src/main/java/org/gridgain/grid/segmentation/GridSegmentationPolicy.java
+++ /dev/null
@@ -1,40 +0,0 @@
-/* @java.file.header */
-
-/*  _________        _____ __________________        _____
- *  __  ____/___________(_)______  /__  ____/______ ____(_)_______
- *  _  / __  __  ___/__  / _  __  / _  / __  _  __ `/__  / __  __ \
- *  / /_/ /  _  /    _  /  / /_/ /  / /_/ /  / /_/ / _  /  _  / / /
- *  \____/   /_/     /_/   \_,__/   \____/   \__,_/  /_/   /_/ /_/
- */
-
-package org.gridgain.grid.segmentation;
-
-/**
- * Policy that defines how node will react on topology segmentation. Note that default
- * segmentation policy is defined by {@link org.apache.ignite.configuration.IgniteConfiguration#DFLT_SEG_PLC} property.
- * @see GridSegmentationResolver
- */
-public enum GridSegmentationPolicy {
-    /**
-     * When segmentation policy is {@code RESTART_JVM}, all listeners will receive
-     * {@link org.apache.ignite.events.IgniteEventType#EVT_NODE_SEGMENTED} event and then JVM will be restarted.
-     * Note, that this will work <b>only</b> if GridGain is started with {@link org.apache.ignite.startup.cmdline.CommandLineStartup}
-     * via standard {@code ggstart.{sh|bat}} shell script.
-     */
-    RESTART_JVM,
-
-    /**
-     * When segmentation policy is {@code STOP}, all listeners will receive
-     * {@link org.apache.ignite.events.IgniteEventType#EVT_NODE_SEGMENTED} event and then particular grid node
-     * will be stopped via call to {@link org.apache.ignite.Ignition#stop(String, boolean)}.
-     */
-    STOP,
-
-    /**
-     * When segmentation policy is {@code NOOP}, all listeners will receive
-     * {@link org.apache.ignite.events.IgniteEventType#EVT_NODE_SEGMENTED} event and it is up to user to
-     * implement logic to handle this event.
-     */
-    NOOP
-}
-

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/8c20bddb/modules/core/src/main/java/org/gridgain/grid/segmentation/GridSegmentationResolver.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/segmentation/GridSegmentationResolver.java b/modules/core/src/main/java/org/gridgain/grid/segmentation/GridSegmentationResolver.java
deleted file mode 100644
index c18fb56..0000000
--- a/modules/core/src/main/java/org/gridgain/grid/segmentation/GridSegmentationResolver.java
+++ /dev/null
@@ -1,56 +0,0 @@
-/* @java.file.header */
-
-/*  _________        _____ __________________        _____
- *  __  ____/___________(_)______  /__  ____/______ ____(_)_______
- *  _  / __  __  ___/__  / _  __  / _  / __  _  __ `/__  / __  __ \
- *  / /_/ /  _  /    _  /  / /_/ /  / /_/ /  / /_/ / _  /  _  / / /
- *  \____/   /_/     /_/   \_,__/   \____/   \__,_/  /_/   /_/ /_/
- */
-
-package org.gridgain.grid.segmentation;
-
-import org.gridgain.grid.*;
-
-import java.io.*;
-
-/**
- * This is interface for segmentation (a.k.a "split-brain" problem) resolvers.
- * <p>
- * Each segmentation resolver checks segment for validity, using its inner logic.
- * Typically, resolver should run light-weight single check (i.e. one IP address or
- * one shared folder). Compound segment checks may be performed using several
- * resolvers.
- * <p>
- * Note that GridGain support a logical segmentation and not limited to network
- * related segmentation only. For example, a particular segmentation resolver
- * can check for specific application or service present on the network and
- * mark the topology as segmented in case it is not available. In other words
- * you can equate the service outage with network outage via segmentation resolution
- * and employ the unified approach in dealing with these types of problems.
- * @see org.apache.ignite.configuration.IgniteConfiguration#getSegmentationResolvers()
- * @see org.apache.ignite.configuration.IgniteConfiguration#getSegmentationPolicy()
- * @see org.apache.ignite.configuration.IgniteConfiguration#getSegmentCheckFrequency()
- * @see org.apache.ignite.configuration.IgniteConfiguration#isAllSegmentationResolversPassRequired()
- * @see org.apache.ignite.configuration.IgniteConfiguration#isWaitForSegmentOnStart()
- * @see GridSegmentationPolicy
- */
-public interface GridSegmentationResolver extends Serializable {
-    /**
-     * Checks whether segment is valid.
-     * <p>
-     * When segmentation happens every node ends up in either one of two segments:
-     * <ul>
-     *     <li>Correct segment</li>
-     *     <li>Invalid segment</li>
-     * </ul>
-     * Nodes in correct segment will continue operate as if nodes in the invalid segment
-     * simply left the topology (i.e. the topology just got "smaller"). Nodes in the
-     * invalid segment will realized that were "left out or disconnected" from the correct segment
-     * and will try to reconnect via {@link GridSegmentationPolicy segmentation policy} set
-     * in configuration.
-     *
-     * @return {@code True} if segment is correct, {@code false} otherwise.
-     * @throws GridException If an error occurred.
-     */
-    public abstract boolean isValidSegment() throws GridException;
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/8c20bddb/modules/core/src/main/java/org/gridgain/grid/segmentation/package.html
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/segmentation/package.html b/modules/core/src/main/java/org/gridgain/grid/segmentation/package.html
deleted file mode 100644
index a4754a1..0000000
--- a/modules/core/src/main/java/org/gridgain/grid/segmentation/package.html
+++ /dev/null
@@ -1,15 +0,0 @@
-<!DOCTYPE html PUBLIC "-//W3C//DTD HTML 4.01 Transitional//EN" "http://www.w3.org/TR/html4/loose.dtd">
-<!--
-    @html.file.header
-    _________        _____ __________________        _____
-    __  ____/___________(_)______  /__  ____/______ ____(_)_______
-    _  / __  __  ___/__  / _  __  / _  / __  _  __ `/__  / __  __ \
-    / /_/ /  _  /    _  /  / /_/ /  / /_/ /  / /_/ / _  /  _  / / /
-    \____/   /_/     /_/   \_,__/   \____/   \__,_/  /_/   /_/ /_/
--->
-<html>
-<body>
-    <!-- Package description. -->
-    Contains segmentation "split-brain" related classes.
-</body>
-</html>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/8c20bddb/modules/core/src/main/java/org/gridgain/grid/spi/package.html
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/spi/package.html b/modules/core/src/main/java/org/gridgain/grid/spi/package.html
deleted file mode 100644
index 72117f8..0000000
--- a/modules/core/src/main/java/org/gridgain/grid/spi/package.html
+++ /dev/null
@@ -1,15 +0,0 @@
-<!DOCTYPE html PUBLIC "-//W3C//DTD HTML 4.01 Transitional//EN" "http://www.w3.org/TR/html4/loose.dtd">
-<!--
-    @html.file.header
-    _________        _____ __________________        _____
-    __  ____/___________(_)______  /__  ____/______ ____(_)_______
-    _  / __  __  ___/__  / _  __  / _  / __  _  __ `/__  / __  __ \
-    / /_/ /  _  /    _  /  / /_/ /  / /_/ /  / /_/ / _  /  _  / / /
-    \____/   /_/     /_/   \_,__/   \____/   \__,_/  /_/   /_/ /_/
--->
-<html>
-<body>
-    <!-- Package description. -->
-    Contains common classes and interfaces for SPI implementations.
-</body>
-</html>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/8c20bddb/modules/core/src/test/java/org/apache/ignite/spi/discovery/GridAbstractDiscoverySelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/spi/discovery/GridAbstractDiscoverySelfTest.java b/modules/core/src/test/java/org/apache/ignite/spi/discovery/GridAbstractDiscoverySelfTest.java
index 338ce9e..efbc8c6 100644
--- a/modules/core/src/test/java/org/apache/ignite/spi/discovery/GridAbstractDiscoverySelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/spi/discovery/GridAbstractDiscoverySelfTest.java
@@ -14,7 +14,7 @@ import org.apache.ignite.cluster.*;
 import org.apache.ignite.marshaller.*;
 import org.apache.ignite.spi.*;
 import org.gridgain.grid.kernal.managers.security.*;
-import org.gridgain.grid.security.*;
+import org.apache.ignite.plugin.security.*;
 import org.gridgain.grid.util.typedef.internal.*;
 import org.gridgain.testframework.config.*;
 import org.gridgain.testframework.junits.*;

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/8c20bddb/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/GridTcpDiscoverySpiStartStopSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/GridTcpDiscoverySpiStartStopSelfTest.java b/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/GridTcpDiscoverySpiStartStopSelfTest.java
index 5b7386d..2b4af2e 100644
--- a/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/GridTcpDiscoverySpiStartStopSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/GridTcpDiscoverySpiStartStopSelfTest.java
@@ -11,7 +11,7 @@ package org.apache.ignite.spi.discovery.tcp;
 
 import org.apache.ignite.cluster.*;
 import org.gridgain.grid.kernal.managers.security.*;
-import org.gridgain.grid.security.*;
+import org.apache.ignite.plugin.security.*;
 import org.gridgain.grid.spi.*;
 import org.apache.ignite.spi.discovery.*;
 import org.apache.ignite.spi.discovery.tcp.ipfinder.*;

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/8c20bddb/modules/core/src/test/java/org/gridgain/grid/kernal/GridLifecycleAwareSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/gridgain/grid/kernal/GridLifecycleAwareSelfTest.java b/modules/core/src/test/java/org/gridgain/grid/kernal/GridLifecycleAwareSelfTest.java
index 8bde51f..72aee8d 100644
--- a/modules/core/src/test/java/org/gridgain/grid/kernal/GridLifecycleAwareSelfTest.java
+++ b/modules/core/src/test/java/org/gridgain/grid/kernal/GridLifecycleAwareSelfTest.java
@@ -15,7 +15,7 @@ import org.apache.ignite.logger.java.*;
 import org.apache.ignite.marshaller.optimized.*;
 import org.gridgain.client.ssl.*;
 import org.gridgain.grid.*;
-import org.gridgain.grid.segmentation.*;
+import org.apache.ignite.plugin.segmentation.*;
 import org.gridgain.testframework.junits.common.*;
 import org.jetbrains.annotations.*;
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/8c20bddb/modules/core/src/test/java/org/gridgain/testframework/GridSpiTestContext.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/gridgain/testframework/GridSpiTestContext.java b/modules/core/src/test/java/org/gridgain/testframework/GridSpiTestContext.java
index a0be5f1..679976f 100644
--- a/modules/core/src/test/java/org/gridgain/testframework/GridSpiTestContext.java
+++ b/modules/core/src/test/java/org/gridgain/testframework/GridSpiTestContext.java
@@ -16,7 +16,7 @@ import org.apache.ignite.spi.*;
 import org.gridgain.grid.*;
 import org.gridgain.grid.kernal.managers.communication.*;
 import org.gridgain.grid.kernal.managers.eventstorage.*;
-import org.gridgain.grid.security.*;
+import org.apache.ignite.plugin.security.*;
 import org.apache.ignite.spi.discovery.*;
 import org.apache.ignite.spi.swapspace.*;
 import org.gridgain.grid.util.direct.*;

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/8c20bddb/modules/core/src/test/java/org/gridgain/testframework/junits/spi/GridSpiAbstractTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/gridgain/testframework/junits/spi/GridSpiAbstractTest.java b/modules/core/src/test/java/org/gridgain/testframework/junits/spi/GridSpiAbstractTest.java
index 798c1dd..767b0f2 100644
--- a/modules/core/src/test/java/org/gridgain/testframework/junits/spi/GridSpiAbstractTest.java
+++ b/modules/core/src/test/java/org/gridgain/testframework/junits/spi/GridSpiAbstractTest.java
@@ -14,7 +14,7 @@ import org.apache.ignite.product.*;
 import org.apache.ignite.spi.*;
 import org.gridgain.grid.kernal.*;
 import org.gridgain.grid.kernal.managers.security.*;
-import org.gridgain.grid.security.*;
+import org.apache.ignite.plugin.security.*;
 import org.apache.ignite.spi.communication.*;
 import org.apache.ignite.spi.communication.tcp.*;
 import org.apache.ignite.spi.discovery.*;

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/8c20bddb/modules/rest-http/src/main/java/org/gridgain/grid/kernal/processors/rest/protocols/http/jetty/GridJettyRestHandler.java
----------------------------------------------------------------------
diff --git a/modules/rest-http/src/main/java/org/gridgain/grid/kernal/processors/rest/protocols/http/jetty/GridJettyRestHandler.java b/modules/rest-http/src/main/java/org/gridgain/grid/kernal/processors/rest/protocols/http/jetty/GridJettyRestHandler.java
index 277e1cf..85d1e58 100644
--- a/modules/rest-http/src/main/java/org/gridgain/grid/kernal/processors/rest/protocols/http/jetty/GridJettyRestHandler.java
+++ b/modules/rest-http/src/main/java/org/gridgain/grid/kernal/processors/rest/protocols/http/jetty/GridJettyRestHandler.java
@@ -17,7 +17,7 @@ import org.eclipse.jetty.server.handler.*;
 import org.gridgain.grid.*;
 import org.gridgain.grid.kernal.processors.rest.*;
 import org.gridgain.grid.kernal.processors.rest.request.*;
-import org.gridgain.grid.security.*;
+import org.apache.ignite.plugin.security.*;
 import org.gridgain.grid.util.typedef.*;
 import org.gridgain.grid.util.typedef.internal.*;
 import org.jetbrains.annotations.*;

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/8c20bddb/modules/spring/src/main/java/org/gridgain/grid/GridSpringBean.java
----------------------------------------------------------------------
diff --git a/modules/spring/src/main/java/org/gridgain/grid/GridSpringBean.java b/modules/spring/src/main/java/org/gridgain/grid/GridSpringBean.java
index 8378260..28efd6c 100644
--- a/modules/spring/src/main/java/org/gridgain/grid/GridSpringBean.java
+++ b/modules/spring/src/main/java/org/gridgain/grid/GridSpringBean.java
@@ -16,7 +16,7 @@ import org.apache.ignite.plugin.*;
 import org.apache.ignite.product.*;
 import org.gridgain.grid.cache.*;
 import org.gridgain.grid.hadoop.*;
-import org.gridgain.grid.security.*;
+import org.apache.ignite.plugin.security.*;
 import org.gridgain.grid.util.lang.*;
 import org.gridgain.grid.util.typedef.*;
 import org.gridgain.grid.util.typedef.internal.*;

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/8c20bddb/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index e00d6f3..a50b5e0 100644
--- a/pom.xml
+++ b/pom.xml
@@ -635,7 +635,7 @@
                                         </group>
                                         <group>
                                             <title>Security APIs</title>
-                                            <packages>org.gridgain.grid.security</packages>
+                                            <packages>org.apache.ignite.plugin.security</packages>
                                         </group>
                                         <group>
                                             <title>Distributed Services APIs</title>
@@ -687,7 +687,7 @@
                                         </group>
                                         <group>
                                             <title>Segmentation Detection</title>
-                                            <packages>org.gridgain.grid.segmentation</packages>
+                                            <packages>org.apache.ignite.plugin.segmentation</packages>
                                         </group>
                                         <group>
                                             <title>Logging</title>
@@ -837,7 +837,7 @@
                                         </group>
                                         <group>
                                             <title>Security APIs</title>
-                                            <packages>org.gridgain.grid.security</packages>
+                                            <packages>org.apache.ignite.plugin.security</packages>
                                         </group>
                                         <group>
                                             <title>Distributed Services APIs</title>
@@ -889,7 +889,7 @@
                                         </group>
                                         <group>
                                             <title>Segmentation Detection</title>
-                                            <packages>org.gridgain.grid.segmentation</packages>
+                                            <packages>org.apache.ignite.plugin.segmentation</packages>
                                         </group>
                                         <group>
                                             <title>Logging</title>


[06/20] incubator-ignite git commit: # Renaming

Posted by sb...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/b69a23cd/modules/core/src/main/java/org/gridgain/grid/spi/loadbalancing/adaptive/AdaptiveLoadProbe.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/spi/loadbalancing/adaptive/AdaptiveLoadProbe.java b/modules/core/src/main/java/org/gridgain/grid/spi/loadbalancing/adaptive/AdaptiveLoadProbe.java
deleted file mode 100644
index 512536f..0000000
--- a/modules/core/src/main/java/org/gridgain/grid/spi/loadbalancing/adaptive/AdaptiveLoadProbe.java
+++ /dev/null
@@ -1,90 +0,0 @@
-/* @java.file.header */
-
-/*  _________        _____ __________________        _____
- *  __  ____/___________(_)______  /__  ____/______ ____(_)_______
- *  _  / __  __  ___/__  / _  __  / _  / __  _  __ `/__  / __  __ \
- *  / /_/ /  _  /    _  /  / /_/ /  / /_/ /  / /_/ / _  /  _  / / /
- *  \____/   /_/     /_/   \_,__/   \____/   \__,_/  /_/   /_/ /_/
- */
-
-package org.gridgain.grid.spi.loadbalancing.adaptive;
-
-import org.apache.ignite.cluster.*;
-
-/**
- * Pluggable implementation of node load probing. Implementations
- * of this can be configured to be used with {@link AdaptiveLoadBalancingSpi}
- * by setting {@link AdaptiveLoadBalancingSpi#setLoadProbe(AdaptiveLoadProbe)}
- * configuration parameter.
- * <p>
- * Note that if {@link #getLoad(org.apache.ignite.cluster.ClusterNode, int)} returns a value of {@code 0},
- * then implementation will assume that load value is simply not available and
- * will try to calculate an average of load values for other nodes. If such
- * average cannot be obtained (all node load values are {@code 0}), then a value
- * of {@code 1} will be used.
- * <p>
- * By default, {@link AdaptiveCpuLoadProbe} probing implementation is used.
- * <p>
- * <h1 class="header">Example</h1>
- * Here is an example of how probing can be implemented to use
- * number of active and waiting jobs as probing mechanism:
- * <pre name="code" class="java">
- * public class FooBarLoadProbe implements GridAdaptiveLoadProbe {
- *     // Flag indicating whether to use average value or current.
- *     private int useAvg = true;
- *
- *     public FooBarLoadProbe(boolean useAvg) {
- *         this.useAvg = useAvg;
- *     }
- *
- *     // Calculate load based on number of active and waiting jobs.
- *     public double getLoad(GridNode node, int jobsSentSinceLastUpdate) {
- *         GridNodeMetrics metrics = node.getMetrics();
- *
- *         if (useAvg) {
- *             double load = metrics.getAverageActiveJobs() + metrics.getAverageWaitingJobs();
- *
- *             if (load > 0) {
- *                 return load;
- *             }
- *         }
- *
- *         return metrics.getCurrentActiveJobs() + metrics.getCurrentWaitingJobs();
- *     }
- * }
- * </pre>
- * Below is an example of how a probe shown above would be configured with {@link AdaptiveLoadBalancingSpi}
- * SPI:
- * <pre name="code" class="xml">
- * &lt;property name="loadBalancingSpi"&gt;
- *     &lt;bean class="org.gridgain.grid.spi.loadBalancing.adaptive.GridAdaptiveLoadBalancingSpi"&gt;
- *         &lt;property name="loadProbe"&gt;
- *             &lt;bean class="foo.bar.FooBarLoadProbe"&gt;
- *                 &lt;constructor-arg value="true"/&gt;
- *             &lt;/bean&gt;
- *         &lt;/property&gt;
- *     &lt;/bean&gt;
- * &lt;/property&gt;
- * </pre>
- */
-public interface AdaptiveLoadProbe {
-    /**
-     * Calculates load value for a given node. Specific implementations would
-     * usually take into account some of the values provided by
-     * {@link org.apache.ignite.cluster.ClusterNode#metrics()} method. For example, load can be calculated
-     * based on job execution time or number of active jobs, or CPU/Heap utilization.
-     * <p>
-     * Note that if this method returns a value of {@code 0},
-     * then implementation will assume that load value is simply not available and
-     * will try to calculate an average of load values for other nodes. If such
-     * average cannot be obtained (all node load values are {@code 0}), then a value
-     * of {@code 1} will be used.
-     *
-     * @param node Grid node to calculate load for.
-     * @param jobsSentSinceLastUpdate Number of jobs sent to this node since
-     *      last metrics update. This parameter may be useful when
-     *      implementation takes into account the current job count on a node.
-     * @return Non-negative load value for the node (zero and above).
-     */
-    public double getLoad(ClusterNode node, int jobsSentSinceLastUpdate);
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/b69a23cd/modules/core/src/main/java/org/gridgain/grid/spi/loadbalancing/adaptive/AdaptiveProcessingTimeLoadProbe.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/spi/loadbalancing/adaptive/AdaptiveProcessingTimeLoadProbe.java b/modules/core/src/main/java/org/gridgain/grid/spi/loadbalancing/adaptive/AdaptiveProcessingTimeLoadProbe.java
deleted file mode 100644
index dc9e250..0000000
--- a/modules/core/src/main/java/org/gridgain/grid/spi/loadbalancing/adaptive/AdaptiveProcessingTimeLoadProbe.java
+++ /dev/null
@@ -1,98 +0,0 @@
-/* @java.file.header */
-
-/*  _________        _____ __________________        _____
- *  __  ____/___________(_)______  /__  ____/______ ____(_)_______
- *  _  / __  __  ___/__  / _  __  / _  / __  _  __ `/__  / __  __ \
- *  / /_/ /  _  /    _  /  / /_/ /  / /_/ /  / /_/ / _  /  _  / / /
- *  \____/   /_/     /_/   \_,__/   \____/   \__,_/  /_/   /_/ /_/
- */
-
-package org.gridgain.grid.spi.loadbalancing.adaptive;
-
-import org.apache.ignite.cluster.*;
-import org.gridgain.grid.util.typedef.internal.*;
-
-/**
- * Implementation of node load probing based on total job processing time.
- * Based on {@link #setUseAverage(boolean)}
- * parameter, this implementation will either use average job execution
- * time values or current (default is to use averages). The algorithm
- * returns a sum of job wait time and job execution time.
- * <p>
- * Below is an example of how CPU load probe would be configured in GridGain
- * Spring configuration file:
- * <pre name="code" class="xml">
- * &lt;property name="loadBalancingSpi"&gt;
- *     &lt;bean class="org.gridgain.grid.spi.loadBalancing.adaptive.GridAdaptiveLoadBalancingSpi"&gt;
- *         &lt;property name="loadProbe"&gt;
- *             &lt;bean class="org.gridgain.grid.spi.loadBalancing.adaptive.GridAdaptiveProcessingTimeLoadProbe"&gt;
- *                 &lt;property name="useAverage" value="true"/&gt;
- *             &lt;/bean&gt;
- *         &lt;/property&gt;
- *     &lt;/bean&gt;
- * &lt;/property&gt;
- * </pre>
- */
-public class AdaptiveProcessingTimeLoadProbe implements AdaptiveLoadProbe {
-    /** Flag indicating whether to use average execution time vs. current. */
-    private boolean useAvg = true;
-
-    /**
-     * Initializes execution time load probe to use
-     * execution time average by default.
-     */
-    public AdaptiveProcessingTimeLoadProbe() {
-        // No-op.
-    }
-
-    /**
-     * Specifies whether to use average execution time vs. current.
-     *
-     * @param useAvg Flag indicating whether to use average execution time vs. current.
-     */
-    public AdaptiveProcessingTimeLoadProbe(boolean useAvg) {
-        this.useAvg = useAvg;
-    }
-
-    /**
-     * Gets flag indicating whether to use average execution time vs. current.
-     *
-     * @return Flag indicating whether to use average execution time vs. current.
-     */
-    public boolean isUseAverage() {
-        return useAvg;
-    }
-
-    /**
-     * Sets flag indicating whether to use average execution time vs. current.
-     *
-     * @param useAvg Flag indicating whether to use average execution time vs. current.
-     */
-    public void setUseAverage(boolean useAvg) {
-        this.useAvg = useAvg;
-    }
-
-
-    /** {@inheritDoc} */
-    @Override public double getLoad(ClusterNode node, int jobsSentSinceLastUpdate) {
-        ClusterNodeMetrics metrics = node.metrics();
-
-        if (useAvg) {
-            double load = metrics.getAverageJobExecuteTime() + metrics.getAverageJobWaitTime();
-
-            // If load is greater than 0, then we can use average times.
-            // Otherwise, we will proceed to using current times.
-            if (load > 0)
-                return load;
-        }
-
-        double load = metrics.getCurrentJobExecuteTime() + metrics.getCurrentJobWaitTime();
-
-        return load < 0 ? 0 : load;
-    }
-
-    /** {@inheritDoc} */
-    @Override public String toString() {
-        return S.toString(AdaptiveProcessingTimeLoadProbe.class, this);
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/b69a23cd/modules/core/src/main/java/org/gridgain/grid/spi/loadbalancing/adaptive/package.html
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/spi/loadbalancing/adaptive/package.html b/modules/core/src/main/java/org/gridgain/grid/spi/loadbalancing/adaptive/package.html
deleted file mode 100644
index ee3a5eb..0000000
--- a/modules/core/src/main/java/org/gridgain/grid/spi/loadbalancing/adaptive/package.html
+++ /dev/null
@@ -1,15 +0,0 @@
-<!DOCTYPE html PUBLIC "-//W3C//DTD HTML 4.01 Transitional//EN" "http://www.w3.org/TR/html4/loose.dtd">
-<!--
-    @html.file.header
-    _________        _____ __________________        _____
-    __  ____/___________(_)______  /__  ____/______ ____(_)_______
-    _  / __  __  ___/__  / _  __  / _  / __  _  __ `/__  / __  __ \
-    / /_/ /  _  /    _  /  / /_/ /  / /_/ /  / /_/ / _  /  _  / / /
-    \____/   /_/     /_/   \_,__/   \____/   \__,_/  /_/   /_/ /_/
--->
-<html>
-<body>
-    <!-- Package description. -->
-    Contains adaptive load balancing SPI.
-</body>
-</html>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/b69a23cd/modules/core/src/main/java/org/gridgain/grid/spi/loadbalancing/package.html
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/spi/loadbalancing/package.html b/modules/core/src/main/java/org/gridgain/grid/spi/loadbalancing/package.html
deleted file mode 100644
index fd879b9..0000000
--- a/modules/core/src/main/java/org/gridgain/grid/spi/loadbalancing/package.html
+++ /dev/null
@@ -1,15 +0,0 @@
-<!DOCTYPE html PUBLIC "-//W3C//DTD HTML 4.01 Transitional//EN" "http://www.w3.org/TR/html4/loose.dtd">
-<!--
-    @html.file.header
-    _________        _____ __________________        _____
-    __  ____/___________(_)______  /__  ____/______ ____(_)_______
-    _  / __  __  ___/__  / _  __  / _  / __  _  __ `/__  / __  __ \
-    / /_/ /  _  /    _  /  / /_/ /  / /_/ /  / /_/ / _  /  _  / / /
-    \____/   /_/     /_/   \_,__/   \____/   \__,_/  /_/   /_/ /_/
--->
-<html>
-<body>
-    <!-- Package description. -->
-    Contains APIs for load balancing SPI.
-</body>
-</html>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/b69a23cd/modules/core/src/main/java/org/gridgain/grid/spi/loadbalancing/roundrobin/RoundRobinGlobalLoadBalancer.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/spi/loadbalancing/roundrobin/RoundRobinGlobalLoadBalancer.java b/modules/core/src/main/java/org/gridgain/grid/spi/loadbalancing/roundrobin/RoundRobinGlobalLoadBalancer.java
deleted file mode 100644
index e467bac..0000000
--- a/modules/core/src/main/java/org/gridgain/grid/spi/loadbalancing/roundrobin/RoundRobinGlobalLoadBalancer.java
+++ /dev/null
@@ -1,305 +0,0 @@
-/* @java.file.header */
-
-/*  _________        _____ __________________        _____
- *  __  ____/___________(_)______  /__  ____/______ ____(_)_______
- *  _  / __  __  ___/__  / _  __  / _  / __  _  __ `/__  / __  __ \
- *  / /_/ /  _  /    _  /  / /_/ /  / /_/ /  / /_/ / _  /  _  / / /
- *  \____/   /_/     /_/   \_,__/   \____/   \__,_/  /_/   /_/ /_/
- */
-
-package org.gridgain.grid.spi.loadbalancing.roundrobin;
-
-import org.apache.ignite.*;
-import org.apache.ignite.cluster.*;
-import org.apache.ignite.events.*;
-import org.apache.ignite.spi.*;
-import org.gridgain.grid.*;
-import org.gridgain.grid.kernal.managers.eventstorage.*;
-import org.gridgain.grid.util.typedef.*;
-import org.gridgain.grid.util.typedef.internal.*;
-
-import java.util.*;
-import java.util.concurrent.*;
-import java.util.concurrent.atomic.*;
-
-import static org.apache.ignite.events.IgniteEventType.*;
-
-/**
- * Load balancer that works in global (not-per-task) mode.
- */
-class RoundRobinGlobalLoadBalancer {
-    /** SPI context. */
-    private IgniteSpiContext ctx;
-
-    /** Listener for node's events. */
-    private GridLocalEventListener lsnr;
-
-    /** Logger. */
-    private final IgniteLogger log;
-
-    /** Current snapshot of nodes which participated in load balancing. */
-    private volatile GridNodeList nodeList = new GridNodeList(0, null);
-
-    /** Mutex for updating current topology. */
-    private final Object mux = new Object();
-
-    /** Barrier for separating initialization callback and load balancing routine. */
-    private final CountDownLatch initLatch = new CountDownLatch(1);
-
-    /**
-     * @param log Grid logger.
-     */
-    RoundRobinGlobalLoadBalancer(IgniteLogger log) {
-        assert log != null;
-
-        this.log = log;
-    }
-
-    /**
-     * @param ctx Load balancing context.
-     */
-    void onContextInitialized(final IgniteSpiContext ctx) {
-        this.ctx = ctx;
-
-        ctx.addLocalEventListener(
-            lsnr = new GridLocalEventListener() {
-                @Override public void onEvent(IgniteEvent evt) {
-                    assert evt instanceof IgniteDiscoveryEvent;
-
-                    UUID nodeId = ((IgniteDiscoveryEvent)evt).eventNode().id();
-
-                    synchronized (mux) {
-                        if (evt.type() == EVT_NODE_JOINED) {
-                            List<UUID> oldNodes = nodeList.getNodes();
-
-                            if (!oldNodes.contains(nodeId)) {
-                                List<UUID> newNodes = new ArrayList<>(oldNodes.size() + 1);
-
-                                newNodes.add(nodeId);
-
-                                for (UUID node : oldNodes)
-                                    newNodes.add(node);
-
-                                nodeList = new GridNodeList(0, newNodes);
-                            }
-                        }
-                        else {
-                            assert evt.type() == EVT_NODE_LEFT || evt.type() == EVT_NODE_FAILED;
-
-                            List<UUID> oldNodes = nodeList.getNodes();
-
-                            if (oldNodes.contains(nodeId)) {
-                                List<UUID> newNodes = new ArrayList<>(oldNodes.size() - 1);
-
-                                for (UUID node : oldNodes)
-                                    if (!nodeId.equals(node))
-                                        newNodes.add(node);
-
-                                nodeList = new GridNodeList(0, newNodes);
-                            }
-                        }
-                    }
-                }
-            },
-            EVT_NODE_FAILED, EVT_NODE_JOINED, EVT_NODE_LEFT
-        );
-
-        synchronized (mux) {
-            List<UUID> oldNodes = nodeList.getNodes();
-
-            Collection<UUID> set = oldNodes == null ? new HashSet<UUID>() : new HashSet<>(oldNodes);
-
-            for (ClusterNode node : ctx.nodes())
-                set.add(node.id());
-
-            nodeList = new GridNodeList(0, new ArrayList<>(set));
-        }
-
-        initLatch.countDown();
-    }
-
-    /** */
-    void onContextDestroyed() {
-        if (ctx != null)
-            ctx.removeLocalEventListener(lsnr);
-    }
-
-    /**
-     * Gets balanced node for given topology.
-     *
-     * @param top Topology to pick from.
-     * @return Best balanced node.
-     * @throws GridException Thrown in case of any error.
-     */
-    ClusterNode getBalancedNode(Collection<ClusterNode> top) throws GridException {
-        assert !F.isEmpty(top);
-
-        awaitInitializationCompleted();
-
-        Map<UUID, ClusterNode> topMap = null;
-
-        ClusterNode found;
-
-        int misses = 0;
-
-        do {
-            GridNodeList nodeList = this.nodeList;
-
-            List<UUID> nodes = nodeList.getNodes();
-
-            int cycleSize = nodes.size();
-
-            if (cycleSize == 0)
-                throw new GridException("Task topology does not have any alive nodes.");
-
-            AtomicInteger idx;
-
-            int curIdx, nextIdx;
-
-            do {
-                idx = nodeList.getCurrentIdx();
-
-                curIdx = idx.get();
-
-                nextIdx = (idx.get() + 1) % cycleSize;
-            }
-            while (!idx.compareAndSet(curIdx, nextIdx));
-
-            found = findNodeById(top, nodes.get(nextIdx));
-
-            if (found == null) {
-                misses++;
-
-                // For optimization purposes checks balancer can return at least one node with specified
-                // request topology only after full cycle (approximately).
-                if (misses >= cycleSize) {
-                    if (topMap == null) {
-                        topMap = U.newHashMap(top.size());
-
-                        for (ClusterNode node : top)
-                            topMap.put(node.id(), node);
-                    }
-
-                    checkBalancerNodes(top, topMap, nodes);
-
-                    // Zero miss counter so next topology check will be performed once again after full cycle.
-                    misses = 0;
-                }
-            }
-        }
-        while (found == null);
-
-        if (log.isDebugEnabled())
-            log.debug("Found round-robin node: " + found);
-
-        return found;
-    }
-
-    /**
-     * Finds node by id. Returns null in case of absence of specified id in request topology.
-     *
-     * @param top Topology for current request.
-     * @param foundNodeId Node id.
-     * @return Found node or null in case of absence of specified id in request topology.
-     */
-    private static ClusterNode findNodeById(Iterable<ClusterNode> top, UUID foundNodeId) {
-        for (ClusterNode node : top)
-            if (foundNodeId.equals(node.id()))
-                return node;
-
-        return null;
-    }
-
-    /**
-     * Checks if balancer can return at least one node,
-     * throw exception otherwise.
-     *
-     * @param top Topology for current request.
-     * @param topMap Topology map.
-     * @param nodes Current balanced nodes.
-     * @throws GridException If balancer can not return any node.
-     */
-    private static void checkBalancerNodes(Collection<ClusterNode> top, Map<UUID, ClusterNode> topMap, Iterable<UUID> nodes)
-        throws GridException {
-
-        boolean contains = false;
-
-        for (UUID nodeId : nodes) {
-            if (topMap.get(nodeId) != null) {
-                contains = true;
-
-                break;
-            }
-        }
-
-        if (!contains)
-            throw new GridException("Task topology does not have alive nodes: " + top);
-    }
-
-    /**
-     * Awaits initialization of balancing nodes to be completed.
-     *
-     * @throws GridException Thrown in case of thread interruption.
-     */
-    private void awaitInitializationCompleted() throws GridException {
-        try {
-            if (initLatch.getCount() > 0)
-                initLatch.await();
-        }
-        catch (InterruptedException e) {
-            Thread.currentThread().interrupt();
-
-            throw new GridException("Global balancer was interrupted.", e);
-        }
-    }
-
-    /**
-     * Snapshot of nodes which participated in load balancing.
-     */
-    private static final class GridNodeList {
-        /** Cyclic pointer for selecting next node. */
-        private final AtomicInteger curIdx;
-
-        /** Node ids. */
-        private final List<UUID> nodes;
-
-        /**
-         * @param curIdx Initial index of current node.
-         * @param nodes Initial node ids.
-         */
-        private GridNodeList(int curIdx, List<UUID> nodes) {
-            this.curIdx = new AtomicInteger(curIdx);
-            this.nodes = nodes;
-        }
-
-        /**
-         * @return Index of current node.
-         */
-        private AtomicInteger getCurrentIdx() {
-            return curIdx;
-        }
-
-        /**
-         * @return Node ids.
-         */
-        private List<UUID> getNodes() {
-            return nodes;
-        }
-    }
-
-    /**
-     * THIS METHOD IS USED ONLY FOR TESTING.
-     *
-     * @return Internal list of nodes.
-     */
-    List<UUID> getNodeIds() {
-        List<UUID> nodes = nodeList.getNodes();
-
-        return Collections.unmodifiableList(nodes);
-    }
-
-    /** {@inheritDoc} */
-    @Override public String toString() {
-        return S.toString(RoundRobinGlobalLoadBalancer.class, this);
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/b69a23cd/modules/core/src/main/java/org/gridgain/grid/spi/loadbalancing/roundrobin/RoundRobinLoadBalancingSpi.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/spi/loadbalancing/roundrobin/RoundRobinLoadBalancingSpi.java b/modules/core/src/main/java/org/gridgain/grid/spi/loadbalancing/roundrobin/RoundRobinLoadBalancingSpi.java
deleted file mode 100644
index 243e20a..0000000
--- a/modules/core/src/main/java/org/gridgain/grid/spi/loadbalancing/roundrobin/RoundRobinLoadBalancingSpi.java
+++ /dev/null
@@ -1,319 +0,0 @@
-/* @java.file.header */
-
-/*  _________        _____ __________________        _____
- *  __  ____/___________(_)______  /__  ____/______ ____(_)_______
- *  _  / __  __  ___/__  / _  __  / _  / __  _  __ `/__  / __  __ \
- *  / /_/ /  _  /    _  /  / /_/ /  / /_/ /  / /_/ / _  /  _  / / /
- *  \____/   /_/     /_/   \_,__/   \____/   \__,_/  /_/   /_/ /_/
- */
-
-package org.gridgain.grid.spi.loadbalancing.roundrobin;
-
-import org.apache.ignite.*;
-import org.apache.ignite.cluster.*;
-import org.apache.ignite.compute.*;
-import org.apache.ignite.events.*;
-import org.apache.ignite.lang.*;
-import org.apache.ignite.resources.*;
-import org.apache.ignite.spi.*;
-import org.gridgain.grid.*;
-import org.gridgain.grid.kernal.managers.eventstorage.*;
-import org.gridgain.grid.spi.loadbalancing.*;
-import org.gridgain.grid.util.typedef.internal.*;
-import org.jdk8.backport.*;
-import org.jetbrains.annotations.*;
-
-import java.util.*;
-
-import static org.apache.ignite.events.IgniteEventType.*;
-
-/**
- * This SPI iterates through nodes in round-robin fashion and pick the next
- * sequential node. Two modes of operation are supported: per-task and global
- * (see {@link #setPerTask(boolean)} configuration).
- * <p>
- * When configured in per-task mode, implementation will pick a random starting
- * node at the beginning of every task execution and then sequentially iterate through all
- * nodes in topology starting from the picked node. This is the default configuration
- * and should fit most of the use cases as it provides a fairly well-distributed
- * split and also ensures that jobs within a single task are spread out across
- * nodes to the maximum. For cases when split size is equal to the number of nodes,
- * this mode guarantees that all nodes will participate in the split.
- * <p>
- * When configured in global mode, a single sequential queue of nodes is maintained for
- * all tasks and the next node in the queue is picked every time. In this mode (unlike in
- * {@code per-task} mode) it is possible that even if split size may be equal to the
- * number of nodes, some jobs within the same task will be assigned to the same node if
- * multiple tasks are executing concurrently.
- * <h1 class="header">Coding Example</h1>
- * If you are using {@link org.apache.ignite.compute.ComputeTaskSplitAdapter} then load balancing logic
- * is transparent to your code and is handled automatically by the adapter.
- * Here is an example of how your task will look:
- * <pre name="code" class="java">
- * public class MyFooBarTask extends GridComputeTaskSplitAdapter&lt;Object, Object&gt; {
- *    &#64;Override
- *    protected Collection&lt;? extends GridComputeJob&gt; split(int gridSize, Object arg) throws GridException {
- *        List&lt;MyFooBarJob&gt; jobs = new ArrayList&lt;MyFooBarJob&gt;(gridSize);
- *
- *        for (int i = 0; i &lt; gridSize; i++) {
- *            jobs.add(new MyFooBarJob(arg));
- *        }
- *
- *        // Node assignment via load balancer
- *        // happens automatically.
- *        return jobs;
- *    }
- *    ...
- * }
- * </pre>
- * If you need more fine-grained control over how some jobs within task get mapped to a node
- * and use affinity load balancing for some other jobs within task, then you should use
- * {@link org.apache.ignite.compute.ComputeTaskAdapter}. Here is an example of how your task will look. Note that in this
- * case we manually inject load balancer and use it to pick the best node. Doing it in
- * such way would allow user to map some jobs manually and for others use load balancer.
- * <pre name="code" class="java">
- * public class MyFooBarTask extends GridComputeTaskAdapter&lt;String, String&gt; {
- *    // Inject load balancer.
- *    &#64;GridLoadBalancerResource
- *    GridComputeLoadBalancer balancer;
- *
- *    // Map jobs to grid nodes.
- *    public Map&lt;? extends GridComputeJob, GridNode&gt; map(List&lt;GridNode&gt; subgrid, String arg) throws GridException {
- *        Map&lt;MyFooBarJob, GridNode&gt; jobs = new HashMap&lt;MyFooBarJob, GridNode&gt;(subgrid.size());
- *
- *        // In more complex cases, you can actually do
- *        // more complicated assignments of jobs to nodes.
- *        for (int i = 0; i &lt; subgrid.size(); i++) {
- *            // Pick the next best balanced node for the job.
- *            jobs.put(new MyFooBarJob(arg), balancer.getBalancedNode())
- *        }
- *
- *        return jobs;
- *    }
- *
- *    // Aggregate results into one compound result.
- *    public String reduce(List&lt;GridComputeJobResult&gt; results) throws GridException {
- *        // For the purpose of this example we simply
- *        // concatenate string representation of every
- *        // job result
- *        StringBuilder buf = new StringBuilder();
- *
- *        for (GridComputeJobResult res : results) {
- *            // Append string representation of result
- *            // returned by every job.
- *            buf.append(res.getData().string());
- *        }
- *
- *        return buf.string();
- *    }
- * }
- * </pre>
- * <p>
- * <h1 class="header">Configuration</h1>
- * In order to use this load balancer, you should configure your grid instance
- * to use {@code GridRoundRobinLoadBalancingSpi} either from Spring XML file or
- * directly. The following configuration parameters are supported:
- * <h2 class="header">Mandatory</h2>
- * This SPI has no mandatory configuration parameters.
- * <h2 class="header">Optional</h2>
- * The following configuration parameters are optional:
- * <ul>
- * <li>
- *      Flag that indicates whether to use {@code per-task} or global
- *      round-robin modes described above (see {@link #setPerTask(boolean)}).
- * </li>
- * </ul>
- * Below is Java configuration example:
- * <pre name="code" class="java">
- * GridRandomLoadBalancingSpi = new GridRandomLoadBalancingSpi();
- *
- * // Configure SPI to use global round-robin mode.
- * spi.setPerTask(false);
- *
- * GridConfiguration cfg = new GridConfiguration();
- *
- * // Override default load balancing SPI.
- * cfg.setLoadBalancingSpi(spi);
- *
- * // Starts grid.
- * G.start(cfg);
- * </pre>
- * Here is how you can configure {@code GridRandomLoadBalancingSpi} using Spring XML configuration:
- * <pre name="code" class="xml">
- * &lt;property name="loadBalancingSpi"&gt;
- *     &lt;bean class="org.gridgain.grid.spi.loadBalancing.roundrobin.GridRoundRobinLoadBalancingSpi"&gt;
- *         &lt;!-- Set to global round-robin mode. --&gt;
- *         &lt;property name="perTask" value="false"/&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>
- */
-@IgniteSpiMultipleInstancesSupport(true)
-public class RoundRobinLoadBalancingSpi extends IgniteSpiAdapter implements LoadBalancingSpi,
-    RoundRobinLoadBalancingSpiMBean {
-    /** Grid logger. */
-    @IgniteLoggerResource
-    private IgniteLogger log;
-
-    /** */
-    private RoundRobinGlobalLoadBalancer balancer;
-
-    /** */
-    private boolean isPerTask;
-
-    /** */
-    private final Map<IgniteUuid, RoundRobinPerTaskLoadBalancer> perTaskBalancers =
-        new ConcurrentHashMap8<>();
-
-    /** Event listener. */
-    private final GridLocalEventListener lsnr = new GridLocalEventListener() {
-        @Override public void onEvent(IgniteEvent evt) {
-            if (evt.type() == EVT_TASK_FAILED ||
-                evt.type() == EVT_TASK_FINISHED)
-                perTaskBalancers.remove(((IgniteTaskEvent)evt).taskSessionId());
-            else if (evt.type() == EVT_JOB_MAPPED) {
-                RoundRobinPerTaskLoadBalancer balancer =
-                    perTaskBalancers.get(((IgniteJobEvent)evt).taskSessionId());
-
-                if (balancer != null)
-                    balancer.onMapped();
-            }
-        }
-    };
-
-    /** {@inheritDoc} */
-    @Override public boolean isPerTask() {
-        return isPerTask;
-    }
-
-    /**
-     * Configuration parameter indicating whether a new round robin order should be
-     * created for every task. If {@code true} then load balancer is guaranteed
-     * to iterate through nodes sequentially for every task - so as long as number
-     * of jobs is less than or equal to the number of nodes, jobs are guaranteed to
-     * be assigned to unique nodes. If {@code false} then one round-robin order
-     * will be maintained for all tasks, so when tasks execute concurrently, it
-     * is possible for more than one job within task to be assigned to the same
-     * node.
-     * <p>
-     * Default is {@code false}.
-     *
-     * @param isPerTask Configuration parameter indicating whether a new round robin order should
-     *      be created for every task. Default is {@code false}.
-     */
-    @IgniteSpiConfiguration(optional = true)
-    public void setPerTask(boolean isPerTask) {
-        this.isPerTask = isPerTask;
-    }
-
-    /** {@inheritDoc} */
-    @Override public void spiStart(@Nullable String gridName) throws IgniteSpiException {
-        startStopwatch();
-
-        if (log.isDebugEnabled())
-            log.debug(configInfo("isPerTask", isPerTask));
-
-        registerMBean(gridName, this, RoundRobinLoadBalancingSpiMBean.class);
-
-        balancer = new RoundRobinGlobalLoadBalancer(log);
-
-        // Ack ok start.
-        if (log.isDebugEnabled())
-            log.debug(startInfo());
-    }
-
-    /** {@inheritDoc} */
-    @Override public void spiStop() throws IgniteSpiException {
-        balancer = null;
-
-        perTaskBalancers.clear();
-
-        unregisterMBean();
-
-        // Ack ok stop.
-        if (log.isDebugEnabled())
-            log.debug(stopInfo());
-    }
-
-    /** {@inheritDoc} */
-    @Override protected void onContextInitialized0(IgniteSpiContext spiCtx) throws IgniteSpiException {
-        if (!isPerTask)
-            balancer.onContextInitialized(spiCtx);
-        else {
-            if (!getSpiContext().isEventRecordable(EVT_TASK_FAILED, EVT_TASK_FINISHED, EVT_JOB_MAPPED))
-                throw new IgniteSpiException("Required event types are disabled: " +
-                    U.gridEventName(EVT_TASK_FAILED) + ", " +
-                    U.gridEventName(EVT_TASK_FINISHED) + ", " +
-                    U.gridEventName(EVT_JOB_MAPPED));
-
-            getSpiContext().addLocalEventListener(lsnr, EVT_TASK_FAILED, EVT_TASK_FINISHED, EVT_JOB_MAPPED);
-        }
-    }
-
-    /** {@inheritDoc} */
-    @Override protected void onContextDestroyed0() {
-        if (!isPerTask) {
-            if (balancer != null)
-                balancer.onContextDestroyed();
-        }
-        else {
-            IgniteSpiContext spiCtx = getSpiContext();
-
-            if (spiCtx != null)
-                spiCtx.removeLocalEventListener(lsnr);
-        }
-    }
-
-    /** {@inheritDoc} */
-    @Override public ClusterNode getBalancedNode(ComputeTaskSession ses, List<ClusterNode> top, ComputeJob job)
-        throws GridException {
-        A.notNull(ses, "ses", top, "top");
-
-        if (isPerTask) {
-            // Note that every session operates from single thread which
-            // allows us to use concurrent map and avoid synchronization.
-            RoundRobinPerTaskLoadBalancer taskBalancer = perTaskBalancers.get(ses.getId());
-
-            if (taskBalancer == null)
-                perTaskBalancers.put(ses.getId(), taskBalancer = new RoundRobinPerTaskLoadBalancer());
-
-            return taskBalancer.getBalancedNode(top);
-        }
-
-        return balancer.getBalancedNode(top);
-    }
-
-    /**
-     * THIS METHOD IS USED ONLY FOR TESTING.
-     *
-     * @param ses Task session.
-     * @return Internal list of nodes.
-     */
-    List<UUID> getNodeIds(ComputeTaskSession ses) {
-        if (isPerTask) {
-            RoundRobinPerTaskLoadBalancer balancer = perTaskBalancers.get(ses.getId());
-
-            if (balancer == null)
-                return Collections.emptyList();
-
-            List<UUID> ids = new ArrayList<>();
-
-            for (ClusterNode node : balancer.getNodes()) {
-                ids.add(node.id());
-            }
-
-            return ids;
-        }
-
-        return balancer.getNodeIds();
-    }
-
-    /** {@inheritDoc} */
-    @Override public String toString() {
-        return S.toString(RoundRobinLoadBalancingSpi.class, this);
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/b69a23cd/modules/core/src/main/java/org/gridgain/grid/spi/loadbalancing/roundrobin/RoundRobinLoadBalancingSpiMBean.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/spi/loadbalancing/roundrobin/RoundRobinLoadBalancingSpiMBean.java b/modules/core/src/main/java/org/gridgain/grid/spi/loadbalancing/roundrobin/RoundRobinLoadBalancingSpiMBean.java
deleted file mode 100644
index 1189677..0000000
--- a/modules/core/src/main/java/org/gridgain/grid/spi/loadbalancing/roundrobin/RoundRobinLoadBalancingSpiMBean.java
+++ /dev/null
@@ -1,37 +0,0 @@
-/* @java.file.header */
-
-/*  _________        _____ __________________        _____
- *  __  ____/___________(_)______  /__  ____/______ ____(_)_______
- *  _  / __  __  ___/__  / _  __  / _  / __  _  __ `/__  / __  __ \
- *  / /_/ /  _  /    _  /  / /_/ /  / /_/ /  / /_/ / _  /  _  / / /
- *  \____/   /_/     /_/   \_,__/   \____/   \__,_/  /_/   /_/ /_/
- */
-
-package org.gridgain.grid.spi.loadbalancing.roundrobin;
-
-import org.apache.ignite.mbean.*;
-import org.apache.ignite.spi.*;
-
-/**
- * Management bean for {@link RoundRobinLoadBalancingSpi} SPI.
- */
-@IgniteMBeanDescription("MBean that provides access to round robin load balancing SPI configuration.")
-public interface RoundRobinLoadBalancingSpiMBean extends IgniteSpiManagementMBean {
-    /**
-     * Configuration parameter indicating whether a new round robin order should be
-     * created for every task. If {@code true} then load balancer is guaranteed
-     * to iterate through nodes sequentially for every task - so as long as number
-     * of jobs is less than or equal to the number of nodes, jobs are guaranteed to
-     * be assigned to unique nodes. If {@code false} then one round-robin order
-     * will be maintained for all tasks, so when tasks execute concurrently, it
-     * is possible for more than one job within task to be assigned to the same
-     * node.
-     * <p>
-     * Default is {@code true}.
-     *
-     * @return Configuration parameter indicating whether a new round robin order should
-     *      be created for every task. Default is {@code true}.
-     */
-    @IgniteMBeanDescription("Configuration parameter indicating whether a new round robin order should be created for every task.")
-    public boolean isPerTask();
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/b69a23cd/modules/core/src/main/java/org/gridgain/grid/spi/loadbalancing/roundrobin/RoundRobinPerTaskLoadBalancer.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/spi/loadbalancing/roundrobin/RoundRobinPerTaskLoadBalancer.java b/modules/core/src/main/java/org/gridgain/grid/spi/loadbalancing/roundrobin/RoundRobinPerTaskLoadBalancer.java
deleted file mode 100644
index bf626b4..0000000
--- a/modules/core/src/main/java/org/gridgain/grid/spi/loadbalancing/roundrobin/RoundRobinPerTaskLoadBalancer.java
+++ /dev/null
@@ -1,96 +0,0 @@
-/* @java.file.header */
-
-/*  _________        _____ __________________        _____
- *  __  ____/___________(_)______  /__  ____/______ ____(_)_______
- *  _  / __  __  ___/__  / _  __  / _  / __  _  __ `/__  / __  __ \
- *  / /_/ /  _  /    _  /  / /_/ /  / /_/ /  / /_/ / _  /  _  / / /
- *  \____/   /_/     /_/   \_,__/   \____/   \__,_/  /_/   /_/ /_/
- */
-
-package org.gridgain.grid.spi.loadbalancing.roundrobin;
-
-import org.apache.ignite.cluster.*;
-
-import java.util.*;
-
-/**
- * Load balancer for per-task configuration.
- */
-class RoundRobinPerTaskLoadBalancer {
-    /** Balancing nodes. */
-    private ArrayDeque<ClusterNode> nodeQueue;
-
-    /** Jobs mapped flag. */
-    private volatile boolean isMapped;
-
-    /** Mutex. */
-    private final Object mux = new Object();
-
-    /**
-     * Call back for job mapped event.
-     */
-    void onMapped() {
-        isMapped = true;
-    }
-
-    /**
-     * Gets balanced node for given topology. This implementation
-     * is to be used only from {@link org.apache.ignite.compute.ComputeTask#map(List, Object)} method
-     * and, therefore, does not need to be thread-safe.
-     *
-     * @param top Topology to pick from.
-     * @return Best balanced node.
-     */
-    ClusterNode getBalancedNode(List<ClusterNode> top) {
-        assert top != null;
-        assert !top.isEmpty();
-
-        boolean readjust = isMapped;
-
-        synchronized (mux) {
-            // Populate first time.
-            if (nodeQueue == null)
-                nodeQueue = new ArrayDeque<>(top);
-
-            // If job has been mapped, then it means
-            // that it is most likely being failed over.
-            // In this case topology might have changed
-            // and we need to readjust with every apply.
-            if (readjust)
-                // Add missing nodes.
-                for (ClusterNode node : top)
-                    if (!nodeQueue.contains(node))
-                        nodeQueue.offer(node);
-
-            ClusterNode next = nodeQueue.poll();
-
-            // If jobs have been mapped, we need to make sure
-            // that queued node is still in topology.
-            if (readjust && next != null) {
-                while (!top.contains(next) && !nodeQueue.isEmpty())
-                    next = nodeQueue.poll();
-
-                // No nodes found and queue is empty.
-                if (next != null && !top.contains(next))
-                    return null;
-            }
-
-            if (next != null)
-                // Add to the end.
-                nodeQueue.offer(next);
-
-            return next;
-        }
-    }
-
-    /**
-     * THIS METHOD IS USED ONLY FOR TESTING.
-     *
-     * @return Internal list of nodes.
-     */
-    List<ClusterNode> getNodes() {
-        synchronized (mux) {
-            return Collections.unmodifiableList(new ArrayList<>(nodeQueue));
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/b69a23cd/modules/core/src/main/java/org/gridgain/grid/spi/loadbalancing/roundrobin/package.html
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/spi/loadbalancing/roundrobin/package.html b/modules/core/src/main/java/org/gridgain/grid/spi/loadbalancing/roundrobin/package.html
deleted file mode 100644
index 9909144..0000000
--- a/modules/core/src/main/java/org/gridgain/grid/spi/loadbalancing/roundrobin/package.html
+++ /dev/null
@@ -1,15 +0,0 @@
-<!DOCTYPE html PUBLIC "-//W3C//DTD HTML 4.01 Transitional//EN" "http://www.w3.org/TR/html4/loose.dtd">
-<!--
-    @html.file.header
-    _________        _____ __________________        _____
-    __  ____/___________(_)______  /__  ____/______ ____(_)_______
-    _  / __  __  ___/__  / _  __  / _  / __  _  __ `/__  / __  __ \
-    / /_/ /  _  /    _  /  / /_/ /  / /_/ /  / /_/ / _  /  _  / / /
-    \____/   /_/     /_/   \_,__/   \____/   \__,_/  /_/   /_/ /_/
--->
-<html>
-<body>
-    <!-- Package description. -->
-    Contains <b>default</b> round-robin implementation for load balancing SPI.
-</body>
-</html>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/b69a23cd/modules/core/src/main/java/org/gridgain/grid/spi/loadbalancing/weightedrandom/WeightedRandomLoadBalancingSpi.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/spi/loadbalancing/weightedrandom/WeightedRandomLoadBalancingSpi.java b/modules/core/src/main/java/org/gridgain/grid/spi/loadbalancing/weightedrandom/WeightedRandomLoadBalancingSpi.java
deleted file mode 100644
index 0590d60..0000000
--- a/modules/core/src/main/java/org/gridgain/grid/spi/loadbalancing/weightedrandom/WeightedRandomLoadBalancingSpi.java
+++ /dev/null
@@ -1,394 +0,0 @@
-/* @java.file.header */
-
-/*  _________        _____ __________________        _____
- *  __  ____/___________(_)______  /__  ____/______ ____(_)_______
- *  _  / __  __  ___/__  / _  __  / _  / __  _  __ `/__  / __  __ \
- *  / /_/ /  _  /    _  /  / /_/ /  / /_/ /  / /_/ / _  /  _  / / /
- *  \____/   /_/     /_/   \_,__/   \____/   \__,_/  /_/   /_/ /_/
- */
-
-package org.gridgain.grid.spi.loadbalancing.weightedrandom;
-
-import org.apache.ignite.*;
-import org.apache.ignite.cluster.*;
-import org.apache.ignite.compute.*;
-import org.apache.ignite.events.*;
-import org.apache.ignite.lang.*;
-import org.apache.ignite.resources.*;
-import org.apache.ignite.spi.*;
-import org.gridgain.grid.kernal.managers.eventstorage.*;
-import org.gridgain.grid.spi.loadbalancing.*;
-import org.gridgain.grid.util.typedef.*;
-import org.gridgain.grid.util.typedef.internal.*;
-import org.jdk8.backport.*;
-import org.jetbrains.annotations.*;
-
-import java.util.*;
-import java.util.concurrent.*;
-
-import static org.apache.ignite.events.IgniteEventType.*;
-
-/**
- * Load balancing SPI that picks a random node for job execution. Note that you can
- * optionally assign weights to nodes, so nodes with larger weights will end up getting
- * proportionally more jobs routed to them (see {@link #setNodeWeight(int)}
- * configuration property). By default all nodes get equal weight defined by
- * {@link #DFLT_NODE_WEIGHT} (value is {@code 10}).
- * <h1 class="header">Coding Example</h1>
- * If you are using {@link org.apache.ignite.compute.ComputeTaskSplitAdapter} then load balancing logic
- * is transparent to your code and is handled automatically by the adapter.
- * Here is an example of how your task could look:
- * <pre name="code" class="java">
- * public class MyFooBarTask extends GridComputeTaskSplitAdapter&lt;Object, Object&gt; {
- *    &#64;Override
- *    protected Collection&lt;? extends GridComputeJob&gt; split(int gridSize, Object arg) throws GridException {
- *        List&lt;MyFooBarJob&gt; jobs = new ArrayList&lt;MyFooBarJob&gt;(gridSize);
- *
- *        for (int i = 0; i &lt; gridSize; i++) {
- *            jobs.add(new MyFooBarJob(arg));
- *        }
- *
- *        // Node assignment via load balancer
- *        // happens automatically.
- *        return jobs;
- *    }
- *    ...
- * }
- * </pre>
- * If you need more fine-grained control over how some jobs within task get mapped to a node
- * and use affinity load balancing for some other jobs within task, then you should use
- * {@link org.apache.ignite.compute.ComputeTaskAdapter}. Here is an example of how your task will look. Note that in this
- * case we manually inject load balancer and use it to pick the best node. Doing it in
- * such way would allow user to map some jobs manually and for others use load balancer.
- * <pre name="code" class="java">
- * public class MyFooBarTask extends GridComputeTaskAdapter&lt;String, String&gt; {
- *    // Inject load balancer.
- *    &#64;GridLoadBalancerResource
- *    GridComputeLoadBalancer balancer;
- *
- *    // Map jobs to grid nodes.
- *    public Map&lt;? extends GridComputeJob, GridNode&gt; map(List&lt;GridNode&gt; subgrid, String arg) throws GridException {
- *        Map&lt;MyFooBarJob, GridNode&gt; jobs = new HashMap&lt;MyFooBarJob, GridNode&gt;(subgrid.size());
- *
- *        // In more complex cases, you can actually do
- *        // more complicated assignments of jobs to nodes.
- *        for (int i = 0; i &lt; subgrid.size(); i++) {
- *            // Pick the next best balanced node for the job.
- *            jobs.put(new MyFooBarJob(arg), balancer.getBalancedNode())
- *        }
- *
- *        return jobs;
- *    }
- *
- *    // Aggregate results into one compound result.
- *    public String reduce(List&lt;GridComputeJobResult&gt; results) throws GridException {
- *        // For the purpose of this example we simply
- *        // concatenate string representation of every
- *        // job result
- *        StringBuilder buf = new StringBuilder();
- *
- *        for (GridComputeJobResult res : results) {
- *            // Append string representation of result
- *            // returned by every job.
- *            buf.append(res.getData().string());
- *        }
- *
- *        return buf.string();
- *    }
- * }
- * </pre>
- * <p>
- * <h1 class="header">Configuration</h1>
- * In order to use this load balancer, you should configure your grid instance
- * to use {@code GridRandomLoadBalancingSpi} either from Spring XML file or
- * directly. The following configuration parameters are supported:
- * <h2 class="header">Mandatory</h2>
- * This SPI has no mandatory configuration parameters.
- * <h2 class="header">Optional</h2>
- * The following configuration parameters are optional:
- * <ul>
- * <li>
- *      Flag that indicates whether to use weight policy or simple random policy
- *      (see {@link #setUseWeights(boolean)})
- * </li>
- * <li>
- *      Weight of this node (see {@link #setNodeWeight(int)}). This parameter is ignored
- *      if {@link #setUseWeights(boolean)} is set to {@code false}.
- * </li>
- * </ul>
- * Below is Java configuration example:
- * <pre name="code" class="java">
- * GridWeightedRandomLoadBalancingSpi = new GridWeightedLoadBalancingSpi();
- *
- * // Configure SPI to used weighted
- * // random load balancing.
- * spi.setUseWeights(true);
- *
- * // Set weight for the local node.
- * spi.setWeight( *);
- *
- * GridConfiguration cfg = new GridConfiguration();
- *
- * // Override default load balancing SPI.
- * cfg.setLoadBalancingSpi(spi);
- *
- * // Starts grid.
- * G.start(cfg);
- * </pre>
- * Here is how you can configure {@code GridRandomLoadBalancingSpi} using Spring XML configuration:
- * <pre name="code" class="xml">
- * &lt;property name="loadBalancingSpi"&gt;
- *     &lt;bean class="org.gridgain.grid.spi.loadBalancing.weightedrandom.GridWeightedRandomLoadBalancingSpi"&gt;
- *         &lt;property name="useWeights" value="true"/&gt;
- *         &lt;property name="nodeWeight" value="10"/&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>
- */
-@IgniteSpiMultipleInstancesSupport(true)
-@IgniteSpiConsistencyChecked(optional = true)
-public class WeightedRandomLoadBalancingSpi extends IgniteSpiAdapter implements LoadBalancingSpi,
-    WeightedRandomLoadBalancingSpiMBean {
-    /** Random number generator. */
-    private static final Random RAND = new Random();
-
-    /**
-     * Name of node attribute used to indicate load weight of a node
-     * (value is {@code "gridgain.node.weight.attr.name"}).
-     *
-     * @see org.apache.ignite.cluster.ClusterNode#attributes()
-     */
-    public static final String NODE_WEIGHT_ATTR_NAME = "gridgain.node.weight.attr.name";
-
-    /** Default weight assigned to every node if explicit one is not provided (value is {@code 10}). */
-    public static final int DFLT_NODE_WEIGHT = 10;
-
-    /** Grid logger. */
-    @IgniteLoggerResource
-    private IgniteLogger log;
-
-    /** */
-    private boolean isUseWeights;
-
-    /** Local event listener to listen to task completion events. */
-    private GridLocalEventListener evtLsnr;
-
-    /** Weight of this node. */
-    private int nodeWeight = DFLT_NODE_WEIGHT;
-
-    /** Task topologies. First pair value indicates whether or not jobs have been mapped. */
-    private ConcurrentMap<IgniteUuid, IgniteBiTuple<Boolean, WeightedTopology>> taskTops =
-        new ConcurrentHashMap8<>();
-
-    /**
-     * Sets a flag to indicate whether node weights should be checked when
-     * doing random load balancing. Default value is {@code false} which
-     * means that node weights are disregarded for load balancing logic.
-     *
-     * @param isUseWeights If {@code true} then random load is distributed according
-     *      to node weights.
-     */
-    @IgniteSpiConfiguration(optional = true)
-    public void setUseWeights(boolean isUseWeights) {
-        this.isUseWeights = isUseWeights;
-    }
-
-    /** {@inheritDoc} */
-    @Override public boolean isUseWeights() {
-        return isUseWeights;
-    }
-
-    /**
-     * Sets weight of this node. Nodes with more processing capacity
-     * should be assigned proportionally larger weight. Default value
-     * is {@link #DFLT_NODE_WEIGHT} and is equal for all nodes.
-     *
-     * @param nodeWeight Weight of this node.
-     */
-    @IgniteSpiConfiguration(optional = true)
-    public void setNodeWeight(int nodeWeight) {
-        this.nodeWeight = nodeWeight;
-    }
-
-    /** {@inheritDoc} */
-    @Override public int getNodeWeight() {
-        return nodeWeight;
-    }
-
-    /** {@inheritDoc} */
-    @Override public Map<String, Object> getNodeAttributes() throws IgniteSpiException {
-        return F.<String, Object>asMap(createSpiAttributeName(NODE_WEIGHT_ATTR_NAME), nodeWeight);
-    }
-
-    /** {@inheritDoc} */
-    @Override public void spiStart(@Nullable String gridName) throws IgniteSpiException {
-        startStopwatch();
-
-        assertParameter(nodeWeight > 0, "nodeWeight > 0");
-
-        if (log.isDebugEnabled()) {
-            log.debug(configInfo("isUseWeights", isUseWeights));
-            log.debug(configInfo("nodeWeight", nodeWeight));
-        }
-
-        registerMBean(gridName, this, WeightedRandomLoadBalancingSpiMBean.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 protected void onContextInitialized0(IgniteSpiContext spiCtx) throws IgniteSpiException {
-        getSpiContext().addLocalEventListener(evtLsnr = new GridLocalEventListener() {
-            @Override public void onEvent(IgniteEvent evt) {
-                assert evt instanceof IgniteTaskEvent || evt instanceof IgniteJobEvent;
-
-                if (evt.type() == EVT_TASK_FINISHED ||
-                    evt.type() == EVT_TASK_FAILED) {
-                    IgniteUuid sesId = ((IgniteTaskEvent)evt).taskSessionId();
-
-                    taskTops.remove(sesId);
-
-                    if (log.isDebugEnabled())
-                        log.debug("Removed task topology from topology cache for session: " + sesId);
-                }
-                // We should keep topology and use cache in GridComputeTask#map() method to
-                // avoid O(n*n/2) complexity, after that we can drop caches.
-                // Here we set mapped property and later cache will be ignored
-                else if (evt.type() == EVT_JOB_MAPPED) {
-                    IgniteUuid sesId = ((IgniteJobEvent)evt).taskSessionId();
-
-                    IgniteBiTuple<Boolean, WeightedTopology> weightedTop = taskTops.get(sesId);
-
-                    if (weightedTop != null)
-                        weightedTop.set1(true);
-
-                    if (log.isDebugEnabled())
-                        log.debug("Job has been mapped. Ignore cache for session: " + sesId);
-                }
-            }
-        },
-            EVT_TASK_FAILED,
-            EVT_TASK_FINISHED,
-            EVT_JOB_MAPPED
-        );
-    }
-
-    /** {@inheritDoc} */
-    @Override protected void onContextDestroyed0() {
-        if (evtLsnr != null) {
-            IgniteSpiContext ctx = getSpiContext();
-
-            if (ctx != null)
-                ctx.removeLocalEventListener(evtLsnr);
-        }
-    }
-
-    /** {@inheritDoc} */
-    @Override public ClusterNode getBalancedNode(ComputeTaskSession ses, List<ClusterNode> top, ComputeJob job) {
-        A.notNull(ses, "ses");
-        A.notNull(top, "top");
-        A.notNull(job, "job");
-
-        // Optimization for non-weighted randomization.
-        if (!isUseWeights)
-            return top.get(RAND.nextInt(top.size()));
-
-        IgniteBiTuple<Boolean, WeightedTopology> weightedTop = taskTops.get(ses.getId());
-
-        // Create new cached topology if there is no one. Do not
-        // use cached topology after task has been mapped.
-        if (weightedTop == null) {
-            // Called from GridComputeTask#map(). Put new topology and false as not mapped yet.
-            taskTops.put(ses.getId(), weightedTop = F.t(false, new WeightedTopology(top)));
-        }
-        // We have topology - check if task has been mapped.
-        else if (weightedTop.get1()) {
-            // Do not use cache after GridComputeTask#map().
-            return new WeightedTopology(top).pickWeightedNode();
-        }
-
-        return weightedTop.get2().pickWeightedNode();
-    }
-
-    /**
-     * @param node Node to get weight for.
-     * @return Node weight
-     */
-    private int getWeight(ClusterNode node) {
-        Integer weight = (Integer)node.attribute(createSpiAttributeName(NODE_WEIGHT_ATTR_NAME));
-
-        if (weight != null && weight == 0)
-            throw new IllegalStateException("Node weight cannot be zero: " + node);
-
-        return weight == null ? DFLT_NODE_WEIGHT : weight;
-    }
-
-    /**
-     * Holder for weighted topology.
-     */
-    private class WeightedTopology {
-        /** Total topology weight. */
-        private final int totalWeight;
-
-        /** Topology sorted by weight. */
-        private final SortedMap<Integer, ClusterNode> circle = new TreeMap<>();
-
-        /**
-         * @param top Topology.
-         */
-        WeightedTopology(Collection<ClusterNode> top) {
-            assert !F.isEmpty(top);
-
-            int totalWeight = 0;
-
-            for (ClusterNode node : top) {
-                totalWeight += getWeight(node);
-
-                // Complexity of this put is O(logN).
-                circle.put(totalWeight, node);
-            }
-
-            this.totalWeight = totalWeight;
-        }
-
-        /**
-         * Gets weighted node in random fashion.
-         *
-         * @return Weighted node.
-         */
-        ClusterNode pickWeightedNode() {
-            int weight = RAND.nextInt(totalWeight) + 1;
-
-            SortedMap<Integer, ClusterNode> pick = circle.tailMap(weight);
-
-            assert !pick.isEmpty();
-
-            return pick.get(pick.firstKey());
-        }
-    }
-
-    /** {@inheritDoc} */
-    @Override protected List<String> getConsistentAttributeNames() {
-        return Collections.singletonList(createSpiAttributeName(NODE_WEIGHT_ATTR_NAME));
-    }
-
-    /** {@inheritDoc} */
-    @Override public String toString() {
-        return S.toString(WeightedRandomLoadBalancingSpi.class, this);
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/b69a23cd/modules/core/src/main/java/org/gridgain/grid/spi/loadbalancing/weightedrandom/WeightedRandomLoadBalancingSpiMBean.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/spi/loadbalancing/weightedrandom/WeightedRandomLoadBalancingSpiMBean.java b/modules/core/src/main/java/org/gridgain/grid/spi/loadbalancing/weightedrandom/WeightedRandomLoadBalancingSpiMBean.java
deleted file mode 100644
index c4daa9e..0000000
--- a/modules/core/src/main/java/org/gridgain/grid/spi/loadbalancing/weightedrandom/WeightedRandomLoadBalancingSpiMBean.java
+++ /dev/null
@@ -1,37 +0,0 @@
-/* @java.file.header */
-
-/*  _________        _____ __________________        _____
- *  __  ____/___________(_)______  /__  ____/______ ____(_)_______
- *  _  / __  __  ___/__  / _  __  / _  / __  _  __ `/__  / __  __ \
- *  / /_/ /  _  /    _  /  / /_/ /  / /_/ /  / /_/ / _  /  _  / / /
- *  \____/   /_/     /_/   \_,__/   \____/   \__,_/  /_/   /_/ /_/
- */
-
-package org.gridgain.grid.spi.loadbalancing.weightedrandom;
-
-import org.apache.ignite.mbean.*;
-import org.apache.ignite.spi.*;
-
-/**
- * Management MBean for {@link WeightedRandomLoadBalancingSpi} SPI.
- */
-@IgniteMBeanDescription("MBean that provides access to weighted random load balancing SPI configuration.")
-public interface WeightedRandomLoadBalancingSpiMBean extends IgniteSpiManagementMBean {
-    /**
-     * Checks whether node weights are considered when doing
-     * random load balancing.
-     *
-     * @return If {@code true} then random load is distributed according
-     *      to node weights.
-     */
-    @IgniteMBeanDescription("Whether node weights are considered when doing random load balancing.")
-    public boolean isUseWeights();
-
-    /**
-     * Gets weight of this node.
-     *
-     * @return Weight of this node.
-     */
-    @IgniteMBeanDescription("Weight of this node.")
-    public int getNodeWeight();
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/b69a23cd/modules/core/src/main/java/org/gridgain/grid/spi/loadbalancing/weightedrandom/package.html
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/spi/loadbalancing/weightedrandom/package.html b/modules/core/src/main/java/org/gridgain/grid/spi/loadbalancing/weightedrandom/package.html
deleted file mode 100644
index 2da3d3a..0000000
--- a/modules/core/src/main/java/org/gridgain/grid/spi/loadbalancing/weightedrandom/package.html
+++ /dev/null
@@ -1,15 +0,0 @@
-<!DOCTYPE html PUBLIC "-//W3C//DTD HTML 4.01 Transitional//EN" "http://www.w3.org/TR/html4/loose.dtd">
-<!--
-    @html.file.header
-    _________        _____ __________________        _____
-    __  ____/___________(_)______  /__  ____/______ ____(_)_______
-    _  / __  __  ___/__  / _  __  / _  / __  _  __ `/__  / __  __ \
-    / /_/ /  _  /    _  /  / /_/ /  / /_/ /  / /_/ / _  /  _  / / /
-    \____/   /_/     /_/   \_,__/   \____/   \__,_/  /_/   /_/ /_/
--->
-<html>
-<body>
-    <!-- Package description. -->
-    Contains weighted random-base implementation for load balancing SPI.
-</body>
-</html>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/b69a23cd/modules/core/src/test/config/io-manager-benchmark.xml
----------------------------------------------------------------------
diff --git a/modules/core/src/test/config/io-manager-benchmark.xml b/modules/core/src/test/config/io-manager-benchmark.xml
index 01dc6e2..a58a89c 100644
--- a/modules/core/src/test/config/io-manager-benchmark.xml
+++ b/modules/core/src/test/config/io-manager-benchmark.xml
@@ -50,7 +50,7 @@
 
         <!-- Configure load balancing SPI in the way that do not require extra event subscription. -->
         <property name="loadBalancingSpi">
-            <bean class="org.gridgain.grid.spi.loadbalancing.roundrobin.RoundRobinLoadBalancingSpi">
+            <bean class="org.apache.ignite.spi.loadbalancing.roundrobin.RoundRobinLoadBalancingSpi">
                 <property name="perTask" value="false"/>
             </bean>
         </property>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/b69a23cd/modules/core/src/test/config/jobs-load-base.xml
----------------------------------------------------------------------
diff --git a/modules/core/src/test/config/jobs-load-base.xml b/modules/core/src/test/config/jobs-load-base.xml
index 97325a2..4ee2a39 100644
--- a/modules/core/src/test/config/jobs-load-base.xml
+++ b/modules/core/src/test/config/jobs-load-base.xml
@@ -97,7 +97,7 @@
         </property>
 
         <property name="loadBalancingSpi">
-            <bean class="org.gridgain.grid.spi.loadbalancing.roundrobin.RoundRobinLoadBalancingSpi">
+            <bean class="org.apache.ignite.spi.loadbalancing.roundrobin.RoundRobinLoadBalancingSpi">
                 <property name="perTask" value="false"/>
             </bean>
         </property>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/b69a23cd/modules/core/src/test/config/load/merge-sort-base.xml
----------------------------------------------------------------------
diff --git a/modules/core/src/test/config/load/merge-sort-base.xml b/modules/core/src/test/config/load/merge-sort-base.xml
index e2293f6..3613dae 100644
--- a/modules/core/src/test/config/load/merge-sort-base.xml
+++ b/modules/core/src/test/config/load/merge-sort-base.xml
@@ -128,7 +128,7 @@
         </property>
 
         <property name="loadBalancingSpi">
-            <bean class="org.gridgain.grid.spi.loadbalancing.roundrobin.RoundRobinLoadBalancingSpi">
+            <bean class="org.apache.ignite.spi.loadbalancing.roundrobin.RoundRobinLoadBalancingSpi">
                 <property name="perTask" value="false"/>
             </bean>
         </property>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/b69a23cd/modules/core/src/test/config/spring-cache-put-remove-load.xml
----------------------------------------------------------------------
diff --git a/modules/core/src/test/config/spring-cache-put-remove-load.xml b/modules/core/src/test/config/spring-cache-put-remove-load.xml
index 052da54..a2eb339 100644
--- a/modules/core/src/test/config/spring-cache-put-remove-load.xml
+++ b/modules/core/src/test/config/spring-cache-put-remove-load.xml
@@ -42,7 +42,7 @@
         </property>
 
         <property name="loadBalancingSpi">
-            <bean class="org.gridgain.grid.spi.loadbalancing.roundrobin.RoundRobinLoadBalancingSpi">
+            <bean class="org.apache.ignite.spi.loadbalancing.roundrobin.RoundRobinLoadBalancingSpi">
                 <property name="perTask" value="false"/>
             </bean>
         </property>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/b69a23cd/modules/core/src/test/java/org/apache/ignite/spi/loadbalancing/adaptive/GridAdaptiveLoadBalancingSpiConfigSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/spi/loadbalancing/adaptive/GridAdaptiveLoadBalancingSpiConfigSelfTest.java b/modules/core/src/test/java/org/apache/ignite/spi/loadbalancing/adaptive/GridAdaptiveLoadBalancingSpiConfigSelfTest.java
new file mode 100644
index 0000000..54479e9
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/spi/loadbalancing/adaptive/GridAdaptiveLoadBalancingSpiConfigSelfTest.java
@@ -0,0 +1,26 @@
+/* @java.file.header */
+
+/*  _________        _____ __________________        _____
+ *  __  ____/___________(_)______  /__  ____/______ ____(_)_______
+ *  _  / __  __  ___/__  / _  __  / _  / __  _  __ `/__  / __  __ \
+ *  / /_/ /  _  /    _  /  / /_/ /  / /_/ /  / /_/ / _  /  _  / / /
+ *  \____/   /_/     /_/   \_,__/   \____/   \__,_/  /_/   /_/ /_/
+ */
+
+package org.apache.ignite.spi.loadbalancing.adaptive;
+
+import org.gridgain.testframework.junits.spi.*;
+
+/**
+ *
+ */
+@GridSpiTest(spi = AdaptiveLoadBalancingSpi.class, group = "LoadBalancing SPI")
+public class GridAdaptiveLoadBalancingSpiConfigSelfTest
+    extends GridSpiAbstractConfigTest<AdaptiveLoadBalancingSpi> {
+    /**
+     * @throws Exception If failed.
+     */
+    public void testNegativeConfig() throws Exception {
+        checkNegativeSpiProperty(new AdaptiveLoadBalancingSpi(), "loadProbe", null);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/b69a23cd/modules/core/src/test/java/org/apache/ignite/spi/loadbalancing/adaptive/GridAdaptiveLoadBalancingSpiMultipleNodeSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/spi/loadbalancing/adaptive/GridAdaptiveLoadBalancingSpiMultipleNodeSelfTest.java b/modules/core/src/test/java/org/apache/ignite/spi/loadbalancing/adaptive/GridAdaptiveLoadBalancingSpiMultipleNodeSelfTest.java
new file mode 100644
index 0000000..efef13d
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/spi/loadbalancing/adaptive/GridAdaptiveLoadBalancingSpiMultipleNodeSelfTest.java
@@ -0,0 +1,87 @@
+/* @java.file.header */
+
+/*  _________        _____ __________________        _____
+ *  __  ____/___________(_)______  /__  ____/______ ____(_)_______
+ *  _  / __  __  ___/__  / _  __  / _  / __  _  __ `/__  / __  __ \
+ *  / /_/ /  _  /    _  /  / /_/ /  / /_/ /  / /_/ / _  /  _  / / /
+ *  \____/   /_/     /_/   \_,__/   \____/   \__,_/  /_/   /_/ /_/
+ */
+
+package org.apache.ignite.spi.loadbalancing.adaptive;
+
+import org.apache.ignite.cluster.*;
+import org.apache.ignite.lang.*;
+import org.gridgain.grid.*;
+import org.gridgain.testframework.*;
+import org.gridgain.testframework.junits.spi.*;
+import java.util.*;
+
+/**
+ * Tests adaptive load balancing SPI.
+ */
+@GridSpiTest(spi = AdaptiveLoadBalancingSpi.class, group = "Load Balancing SPI")
+public class GridAdaptiveLoadBalancingSpiMultipleNodeSelfTest extends GridSpiAbstractTest<AdaptiveLoadBalancingSpi> {
+    /** */
+    private static final int RMT_NODE_CNT = 10;
+
+    /** {@inheritDoc} */
+    @Override protected GridSpiTestContext initSpiContext() throws Exception {
+        GridSpiTestContext ctx = super.initSpiContext();
+
+        for (int i = 0; i < RMT_NODE_CNT; i++) {
+            GridTestNode node = new GridTestNode(UUID.randomUUID());
+
+            node.setAttribute("load", (double)(i + 1));
+
+            ctx.addNode(node);
+        }
+
+        return ctx;
+    }
+
+    /**
+     * @return {@code True} if node weights should be considered.
+     */
+    @GridSpiTestConfig
+    public AdaptiveLoadProbe getLoadProbe() {
+        return new AdaptiveLoadProbe() {
+            @Override public double getLoad(ClusterNode node, int jobsSentSinceLastUpdate) {
+                boolean isFirstTime = node.attribute("used") == null;
+
+                assert isFirstTime ? jobsSentSinceLastUpdate == 0 : jobsSentSinceLastUpdate > 0;
+
+                return (Double)node.attribute("load");
+            }
+        };
+    }
+    /**
+     * @throws Exception If failed.
+     */
+    public void testWeights() throws Exception {
+        // Seal it.
+        List<ClusterNode> nodes = new ArrayList<>(getSpiContext().remoteNodes());
+
+        int[] cnts = new int[RMT_NODE_CNT];
+
+        // Invoke load balancer a large number of times, so statistics won't lie.
+        for (int i = 0; i < 50000; i++) {
+            GridTestNode node = (GridTestNode)getSpi().getBalancedNode(new GridTestTaskSession(IgniteUuid.randomUuid()),
+                nodes, new GridTestJob());
+
+            int idx = ((Double)node.attribute("load")).intValue() - 1;
+
+            if (cnts[idx] == 0)
+                node.setAttribute("used", true);
+
+            // Increment number of times a node was picked.
+            cnts[idx]++;
+        }
+
+        info("Node counts: " + Arrays.toString(cnts));
+
+        for (int i = 0; i < cnts.length - 1; i++) {
+            assert cnts[i] > cnts[i + 1] : "Invalid node counts for index [idx=" + i + ", cnts[i]=" + cnts[i] +
+                ", cnts[i+1]=" + cnts[i + 1] + ']';
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/b69a23cd/modules/core/src/test/java/org/apache/ignite/spi/loadbalancing/adaptive/GridAdaptiveLoadBalancingSpiSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/spi/loadbalancing/adaptive/GridAdaptiveLoadBalancingSpiSelfTest.java b/modules/core/src/test/java/org/apache/ignite/spi/loadbalancing/adaptive/GridAdaptiveLoadBalancingSpiSelfTest.java
new file mode 100644
index 0000000..2cc4555
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/spi/loadbalancing/adaptive/GridAdaptiveLoadBalancingSpiSelfTest.java
@@ -0,0 +1,125 @@
+/* @java.file.header */
+
+/*  _________        _____ __________________        _____
+ *  __  ____/___________(_)______  /__  ____/______ ____(_)_______
+ *  _  / __  __  ___/__  / _  __  / _  / __  _  __ `/__  / __  __ \
+ *  / /_/ /  _  /    _  /  / /_/ /  / /_/ /  / /_/ / _  /  _  / / /
+ *  \____/   /_/     /_/   \_,__/   \____/   \__,_/  /_/   /_/ /_/
+ */
+
+package org.apache.ignite.spi.loadbalancing.adaptive;
+
+import org.apache.ignite.cluster.*;
+import org.apache.ignite.compute.*;
+import org.apache.ignite.lang.*;
+import org.gridgain.grid.*;
+import org.gridgain.testframework.*;
+import org.gridgain.testframework.junits.spi.*;
+
+import java.util.*;
+
+/**
+ * Tests adaptive load balancing SPI.
+ */
+@GridSpiTest(spi = AdaptiveLoadBalancingSpi.class, group = "Load Balancing SPI")
+public class GridAdaptiveLoadBalancingSpiSelfTest extends GridSpiAbstractTest<AdaptiveLoadBalancingSpi> {
+    /** {@inheritDoc} */
+    @Override protected GridSpiTestContext initSpiContext() throws Exception {
+        GridSpiTestContext ctx = super.initSpiContext();
+
+        ctx.setLocalNode(new GridTestNode(UUID.randomUUID()));
+
+        return ctx;
+    }
+
+    /**
+     * @return {@code True} if node weights should be considered.
+     */
+    @GridSpiTestConfig
+    public AdaptiveLoadProbe getLoadProbe() {
+        return new AdaptiveLoadProbe() {
+            @Override public double getLoad(ClusterNode node, int jobsSentSinceLastUpdate) {
+                boolean isFirstTime = node.attribute("used") == null;
+
+                assert isFirstTime ? jobsSentSinceLastUpdate == 0 : jobsSentSinceLastUpdate > 0;
+
+                return (Double)node.attribute("load");
+            }
+        };
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    @SuppressWarnings({"ObjectEquality"})
+    public void testSingleNodeZeroWeight() throws Exception {
+        GridTestNode node = (GridTestNode)getSpiContext().nodes().iterator().next();
+
+        node.addAttribute("load", 0d);
+
+        List<ClusterNode> nodes = Collections.singletonList((ClusterNode)node);
+
+        ComputeTaskSession ses = new GridTestTaskSession(IgniteUuid.randomUuid());
+
+        GridTestNode pick1 = (GridTestNode)getSpi().getBalancedNode(ses, nodes, new GridTestJob());
+
+        pick1.setAttribute("used", true);
+
+        assert nodes.contains(pick1);
+
+        // Verify that same instance is returned every time.
+        ClusterNode pick2 = getSpi().getBalancedNode(ses, nodes, new GridTestJob());
+
+        assert pick1 == pick2;
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    @SuppressWarnings({"ObjectEquality"})
+    public void testSingleNodeSameSession() throws Exception {
+        GridTestNode node = (GridTestNode)getSpiContext().nodes().iterator().next();
+
+        node.addAttribute("load", 1d);
+
+        List<ClusterNode> nodes = Collections.singletonList((ClusterNode)node);
+
+        ComputeTaskSession ses = new GridTestTaskSession(IgniteUuid.randomUuid());
+
+        GridTestNode pick1 = (GridTestNode)getSpi().getBalancedNode(ses, nodes, new GridTestJob());
+
+        pick1.setAttribute("used", true);
+
+        assert nodes.contains(pick1);
+
+        // Verify that same instance is returned every time.
+        ClusterNode pick2 = getSpi().getBalancedNode(ses, nodes, new GridTestJob());
+
+        assert pick1 == pick2;
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    @SuppressWarnings({"ObjectEquality"})
+    public void testSingleNodeDifferentSession() throws Exception {
+        GridTestNode node = (GridTestNode)getSpiContext().nodes().iterator().next();
+
+        node.addAttribute("load", 2d);
+
+        List<ClusterNode> nodes = Collections.singletonList((ClusterNode)node);
+
+        GridTestNode pick1 = (GridTestNode)getSpi().getBalancedNode(new GridTestTaskSession(IgniteUuid.randomUuid()),
+            nodes, new GridTestJob());
+
+        pick1.setAttribute("used", true);
+
+        assert nodes.contains(pick1);
+
+        // Verify that same instance is returned every time.
+        ClusterNode pick2 = getSpi().getBalancedNode(new GridTestTaskSession(IgniteUuid.randomUuid()), nodes,
+            new GridTestJob());
+
+        assert pick1 == pick2;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/b69a23cd/modules/core/src/test/java/org/apache/ignite/spi/loadbalancing/adaptive/GridAdaptiveLoadBalancingSpiStartStopSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/spi/loadbalancing/adaptive/GridAdaptiveLoadBalancingSpiStartStopSelfTest.java b/modules/core/src/test/java/org/apache/ignite/spi/loadbalancing/adaptive/GridAdaptiveLoadBalancingSpiStartStopSelfTest.java
new file mode 100644
index 0000000..69c70fa
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/spi/loadbalancing/adaptive/GridAdaptiveLoadBalancingSpiStartStopSelfTest.java
@@ -0,0 +1,23 @@
+/* @java.file.header */
+
+/*  _________        _____ __________________        _____
+ *  __  ____/___________(_)______  /__  ____/______ ____(_)_______
+ *  _  / __  __  ___/__  / _  __  / _  / __  _  __ `/__  / __  __ \
+ *  / /_/ /  _  /    _  /  / /_/ /  / /_/ /  / /_/ / _  /  _  / / /
+ *  \____/   /_/     /_/   \_,__/   \____/   \__,_/  /_/   /_/ /_/
+ */
+
+package org.apache.ignite.spi.loadbalancing.adaptive;
+
+import org.gridgain.grid.spi.*;
+import org.gridgain.testframework.junits.spi.*;
+
+/**
+ * Adaptive load balancing SPI start-stop test.
+ */
+@SuppressWarnings({"JUnitTestCaseWithNoTests"})
+@GridSpiTest(spi = AdaptiveLoadBalancingSpi.class, group = "LoadBalancing SPI")
+public class GridAdaptiveLoadBalancingSpiStartStopSelfTest extends
+    GridSpiStartStopAbstractTest<AdaptiveLoadBalancingSpi> {
+    // No configs.
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/b69a23cd/modules/core/src/test/java/org/apache/ignite/spi/loadbalancing/adaptive/package.html
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/spi/loadbalancing/adaptive/package.html b/modules/core/src/test/java/org/apache/ignite/spi/loadbalancing/adaptive/package.html
new file mode 100644
index 0000000..5cad80a
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/spi/loadbalancing/adaptive/package.html
@@ -0,0 +1,15 @@
+<!DOCTYPE html PUBLIC "-//W3C//DTD HTML 4.01 Transitional//EN" "http://www.w3.org/TR/html4/loose.dtd">
+<!--
+    @html.file.header
+    _________        _____ __________________        _____
+    __  ____/___________(_)______  /__  ____/______ ____(_)_______
+    _  / __  __  ___/__  / _  __  / _  / __  _  __ `/__  / __  __ \
+    / /_/ /  _  /    _  /  / /_/ /  / /_/ /  / /_/ / _  /  _  / / /
+    \____/   /_/     /_/   \_,__/   \____/   \__,_/  /_/   /_/ /_/
+-->
+<html>
+<body>
+    <!-- Package description. -->
+    Contains internal tests or test related classes and interfaces.
+</body>
+</html>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/b69a23cd/modules/core/src/test/java/org/apache/ignite/spi/loadbalancing/package.html
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/spi/loadbalancing/package.html b/modules/core/src/test/java/org/apache/ignite/spi/loadbalancing/package.html
new file mode 100644
index 0000000..5cad80a
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/spi/loadbalancing/package.html
@@ -0,0 +1,15 @@
+<!DOCTYPE html PUBLIC "-//W3C//DTD HTML 4.01 Transitional//EN" "http://www.w3.org/TR/html4/loose.dtd">
+<!--
+    @html.file.header
+    _________        _____ __________________        _____
+    __  ____/___________(_)______  /__  ____/______ ____(_)_______
+    _  / __  __  ___/__  / _  __  / _  / __  _  __ `/__  / __  __ \
+    / /_/ /  _  /    _  /  / /_/ /  / /_/ /  / /_/ / _  /  _  / / /
+    \____/   /_/     /_/   \_,__/   \____/   \__,_/  /_/   /_/ /_/
+-->
+<html>
+<body>
+    <!-- Package description. -->
+    Contains internal tests or test related classes and interfaces.
+</body>
+</html>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/b69a23cd/modules/core/src/test/java/org/apache/ignite/spi/loadbalancing/roundrobin/GridRoundRobinLoadBalancingNotPerTaskMultithreadedSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/spi/loadbalancing/roundrobin/GridRoundRobinLoadBalancingNotPerTaskMultithreadedSelfTest.java b/modules/core/src/test/java/org/apache/ignite/spi/loadbalancing/roundrobin/GridRoundRobinLoadBalancingNotPerTaskMultithreadedSelfTest.java
new file mode 100644
index 0000000..544da32
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/spi/loadbalancing/roundrobin/GridRoundRobinLoadBalancingNotPerTaskMultithreadedSelfTest.java
@@ -0,0 +1,115 @@
+/* @java.file.header */
+
+/*  _________        _____ __________________        _____
+ *  __  ____/___________(_)______  /__  ____/______ ____(_)_______
+ *  _  / __  __  ___/__  / _  __  / _  / __  _  __ `/__  / __  __ \
+ *  / /_/ /  _  /    _  /  / /_/ /  / /_/ /  / /_/ / _  /  _  / / /
+ *  \____/   /_/     /_/   \_,__/   \____/   \__,_/  /_/   /_/ /_/
+ */
+
+package org.apache.ignite.spi.loadbalancing.roundrobin;
+
+import org.apache.ignite.cluster.*;
+import org.apache.ignite.compute.*;
+import org.apache.ignite.lang.*;
+import org.gridgain.grid.*;
+import org.gridgain.testframework.*;
+import org.gridgain.testframework.junits.spi.*;
+
+import java.util.*;
+import java.util.concurrent.*;
+import java.util.concurrent.atomic.*;
+
+/**
+ * Multithreaded tests for global load balancer.
+ */
+@GridSpiTest(spi = RoundRobinLoadBalancingSpi.class, group = "Load Balancing SPI")
+public class GridRoundRobinLoadBalancingNotPerTaskMultithreadedSelfTest
+    extends GridSpiAbstractTest<RoundRobinLoadBalancingSpi> {
+    /** Thread count. */
+    public static final int THREAD_CNT = 8;
+
+    /** Per-thread iteration count. */
+    public static final int ITER_CNT = 4_000_000;
+
+    /**
+     * @return Per-task configuration parameter.
+     */
+    @GridSpiTestConfig
+    public boolean getPerTask() {
+        return false;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected GridSpiTestContext initSpiContext() throws Exception {
+        GridSpiTestContext spiCtx = super.initSpiContext();
+
+        spiCtx.createLocalNode();
+        spiCtx.createRemoteNodes(10);
+
+        return spiCtx;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTest() throws Exception {
+        assert !getSpi().isPerTask() : "Invalid SPI configuration.";
+    }
+
+    /**
+     *
+     * @throws Exception If failed.
+     */
+    public void testMultipleTaskSessionsMultithreaded() throws Exception {
+        final RoundRobinLoadBalancingSpi spi = getSpi();
+
+        final List<ClusterNode> allNodes = (List<ClusterNode>)getSpiContext().nodes();
+
+        GridTestUtils.runMultiThreaded(new Callable<Object>() {
+            @Override public Object call() throws Exception {
+                ComputeTaskSession ses = new GridTestTaskSession(IgniteUuid.randomUuid());
+
+                Map<UUID, AtomicInteger> nodeCnts = new HashMap<>();
+
+                for (int i = 1; i <= ITER_CNT; i++) {
+                    ClusterNode node = spi.getBalancedNode(ses, allNodes, new GridTestJob());
+
+                    if (!nodeCnts.containsKey(node.id()))
+                        nodeCnts.put(node.id(), new AtomicInteger(1));
+                    else
+                        nodeCnts.get(node.id()).incrementAndGet();
+                }
+
+                int predictCnt = ITER_CNT / allNodes.size();
+
+                // Consider +-20% is permissible spread for single node measure.
+                int floor = (int)(predictCnt * 0.8);
+
+                double avgSpread = 0;
+
+                for (ClusterNode n : allNodes) {
+                    int curCnt = nodeCnts.get(n.id()).intValue();
+
+                    avgSpread += Math.abs(predictCnt - curCnt);
+
+                    String msg = "Node stats [id=" + n.id() + ", cnt=" + curCnt + ", floor=" + floor +
+                        ", predictCnt=" + predictCnt + ']';
+
+                    info(msg);
+
+                    assertTrue(msg, curCnt >= floor);
+                }
+
+                avgSpread /= allNodes.size();
+
+                avgSpread = 100.0 * avgSpread / predictCnt;
+
+                info("Average spread for " + allNodes.size() + " nodes is " + avgSpread + " percents");
+
+                // Consider +-10% is permissible average spread for all nodes.
+                assertTrue("Average spread is too big: " + avgSpread, avgSpread <= 10);
+
+                return null;
+            }
+        }, THREAD_CNT, "balancer-test-worker");
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/b69a23cd/modules/core/src/test/java/org/apache/ignite/spi/loadbalancing/roundrobin/GridRoundRobinLoadBalancingSpiLocalNodeSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/spi/loadbalancing/roundrobin/GridRoundRobinLoadBalancingSpiLocalNodeSelfTest.java b/modules/core/src/test/java/org/apache/ignite/spi/loadbalancing/roundrobin/GridRoundRobinLoadBalancingSpiLocalNodeSelfTest.java
new file mode 100644
index 0000000..c546c1f
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/spi/loadbalancing/roundrobin/GridRoundRobinLoadBalancingSpiLocalNodeSelfTest.java
@@ -0,0 +1,44 @@
+/* @java.file.header */
+
+/*  _________        _____ __________________        _____
+ *  __  ____/___________(_)______  /__  ____/______ ____(_)_______
+ *  _  / __  __  ___/__  / _  __  / _  / __  _  __ `/__  / __  __ \
+ *  / /_/ /  _  /    _  /  / /_/ /  / /_/ /  / /_/ / _  /  _  / / /
+ *  \____/   /_/     /_/   \_,__/   \____/   \__,_/  /_/   /_/ /_/
+ */
+
+package org.apache.ignite.spi.loadbalancing.roundrobin;
+
+import org.apache.ignite.cluster.*;
+import org.apache.ignite.lang.*;
+import org.gridgain.grid.*;
+import org.gridgain.testframework.junits.spi.*;
+import java.util.*;
+
+/**
+ * Tests Round Robin load balancing for single node.
+ */
+@GridSpiTest(spi = RoundRobinLoadBalancingSpi.class, group = "Load Balancing SPI", triggerDiscovery = true)
+public class GridRoundRobinLoadBalancingSpiLocalNodeSelfTest extends
+    GridSpiAbstractTest<RoundRobinLoadBalancingSpi> {
+    /**
+     * @throws Exception If failed.
+     */
+    @SuppressWarnings({"ObjectEquality"})
+    public void testLocalNode() throws Exception {
+        assert getDiscoverySpi().getRemoteNodes().isEmpty();
+
+        ClusterNode locNode = getDiscoverySpi().getLocalNode();
+
+        ClusterNode node = getSpi().getBalancedNode(new GridTestTaskSession(IgniteUuid.randomUuid()),
+            Collections.singletonList(locNode), new GridTestJob());
+
+        assert  node == locNode;
+
+        // Double check.
+        node = getSpi().getBalancedNode(new GridTestTaskSession(IgniteUuid.randomUuid()),
+            Collections.singletonList(locNode), new GridTestJob());
+
+        assert node == locNode;
+    }
+}


[18/20] incubator-ignite git commit: # Renaming

Posted by sb...@apache.org.
# Renaming


Project: http://git-wip-us.apache.org/repos/asf/incubator-ignite/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-ignite/commit/ef258ece
Tree: http://git-wip-us.apache.org/repos/asf/incubator-ignite/tree/ef258ece
Diff: http://git-wip-us.apache.org/repos/asf/incubator-ignite/diff/ef258ece

Branch: refs/heads/master
Commit: ef258ece69a3a172a39b81561359a971c8919a07
Parents: c31cec7
Author: sboikov <sb...@gridgain.com>
Authored: Fri Dec 5 17:14:40 2014 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Fri Dec 5 17:14:40 2014 +0300

----------------------------------------------------------------------
 .../integration/GridClientAbstractSelfTest.java |    2 +-
 .../configuration/IgniteConfiguration.java      |    8 +-
 .../org/apache/ignite/spi/IgniteSpiAdapter.java |    2 +-
 .../org/apache/ignite/spi/IgniteSpiContext.java |    2 +-
 .../ignite/spi/swapspace/SwapContext.java       |   39 +
 .../apache/ignite/spi/swapspace/SwapKey.java    |  116 ++
 .../ignite/spi/swapspace/SwapSpaceSpi.java      |  202 ++
 .../spi/swapspace/SwapSpaceSpiListener.java     |   26 +
 .../spi/swapspace/file/FileSwapArray.java       |  181 ++
 .../spi/swapspace/file/FileSwapSpaceSpi.java    | 1843 ++++++++++++++++++
 .../swapspace/file/FileSwapSpaceSpiMBean.java   |   59 +
 .../ignite/spi/swapspace/file/package.html      |   15 +
 .../spi/swapspace/noop/NoopSwapSpaceSpi.java    |  126 ++
 .../ignite/spi/swapspace/noop/package.html      |   15 +
 .../apache/ignite/spi/swapspace/package.html    |   15 +
 .../org/gridgain/grid/kernal/GridGainEx.java    |    6 +-
 .../kernal/managers/GridManagerAdapter.java     |    2 +-
 .../swapspace/GridSwapSpaceManager.java         |    2 +-
 .../processors/cache/GridCacheSwapManager.java  |    2 +-
 .../grid/spi/swapspace/SwapContext.java         |   39 -
 .../gridgain/grid/spi/swapspace/SwapKey.java    |  116 --
 .../grid/spi/swapspace/SwapSpaceSpi.java        |  202 --
 .../spi/swapspace/SwapSpaceSpiListener.java     |   26 -
 .../grid/spi/swapspace/file/FileSwapArray.java  |  181 --
 .../spi/swapspace/file/FileSwapSpaceSpi.java    | 1843 ------------------
 .../swapspace/file/FileSwapSpaceSpiMBean.java   |   59 -
 .../grid/spi/swapspace/file/package.html        |   15 -
 .../spi/swapspace/noop/NoopSwapSpaceSpi.java    |  126 --
 .../grid/spi/swapspace/noop/package.html        |   15 -
 .../gridgain/grid/spi/swapspace/package.html    |   15 -
 .../core/src/test/config/spring-cache-swap.xml  |    2 +-
 .../GridSwapSpaceSpiAbstractSelfTest.java       |  634 ++++++
 .../file/GridFileSwapCompactionSelfTest.java    |  119 ++
 .../file/GridFileSwapSpaceSpiSelfTest.java      |  345 ++++
 .../inmemory/GridTestSwapSpaceSpi.java          |  444 +++++
 .../noop/GridNoopSwapSpaceSpiSelfTest.java      |   53 +
 .../apache/ignite/spi/swapspace/package.html    |   15 +
 .../managers/GridManagerStopSelfTest.java       |    4 +-
 .../swapspace/GridSwapSpaceManagerSelfTest.java |    4 +-
 .../cache/GridCacheAbstractFullApiSelfTest.java |    2 +-
 .../cache/GridCacheMemoryModeSelfTest.java      |    2 +-
 .../cache/GridCacheOffHeapSelfTest.java         |    2 +-
 .../processors/cache/GridCacheOffHeapTest.java  |    2 +-
 .../cache/GridCacheP2PUndeploySelfTest.java     |    2 +-
 .../cache/GridCacheSwapReloadSelfTest.java      |    4 +-
 ...tractDistributedByteArrayValuesSelfTest.java |    2 +-
 ...tPartitionedOnlyByteArrayValuesSelfTest.java |    2 +-
 ...eColocatedOptimisticTransactionSelfTest.java |    2 +-
 ...idCacheReplicatedUnswapAdvancedSelfTest.java |    2 +-
 .../GridCacheLocalByteArrayValuesSelfTest.java  |    2 +-
 .../GridCacheSwapScanQueryAbstractSelfTest.java |    2 +-
 ...idFileSwapSpaceSpiMultithreadedLoadTest.java |    4 +-
 .../GridSwapSpaceSpiAbstractSelfTest.java       |  634 ------
 .../file/GridFileSwapCompactionSelfTest.java    |  119 --
 .../file/GridFileSwapSpaceSpiSelfTest.java      |  345 ----
 .../inmemory/GridTestSwapSpaceSpi.java          |  444 -----
 .../noop/GridNoopSwapSpaceSpiSelfTest.java      |   53 -
 .../gridgain/grid/spi/swapspace/package.html    |   15 -
 .../swap/GridSwapEvictAllBenchmark.java         |    2 +-
 .../testframework/GridSpiTestContext.java       |    2 +-
 .../GridSpiSwapSpaceSelfTestSuite.java          |    4 +-
 .../cache/GridCacheAbstractQuerySelfTest.java   |    2 +-
 .../cache/GridCacheOffHeapAndSwapSelfTest.java  |    2 +-
 .../GridCacheQueryMultiThreadedSelfTest.java    |    2 +-
 .../processors/cache/GridCacheSwapSelfTest.java |    4 +-
 .../cache/GridIndexingWithNoopSwapSelfTest.java |    2 +-
 pom.xml                                         |    4 +-
 67 files changed, 4292 insertions(+), 4292 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/ef258ece/modules/clients/src/test/java/org/gridgain/client/integration/GridClientAbstractSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/clients/src/test/java/org/gridgain/client/integration/GridClientAbstractSelfTest.java b/modules/clients/src/test/java/org/gridgain/client/integration/GridClientAbstractSelfTest.java
index 36feaad..9e46977 100644
--- a/modules/clients/src/test/java/org/gridgain/client/integration/GridClientAbstractSelfTest.java
+++ b/modules/clients/src/test/java/org/gridgain/client/integration/GridClientAbstractSelfTest.java
@@ -23,7 +23,7 @@ import org.gridgain.grid.cache.store.*;
 import org.apache.ignite.spi.discovery.tcp.*;
 import org.apache.ignite.spi.discovery.tcp.ipfinder.*;
 import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.*;
-import org.gridgain.grid.spi.swapspace.file.*;
+import org.apache.ignite.spi.swapspace.file.*;
 import org.gridgain.grid.util.typedef.*;
 import org.gridgain.grid.util.typedef.internal.*;
 import org.gridgain.testframework.junits.common.*;

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/ef258ece/modules/core/src/main/java/org/apache/ignite/configuration/IgniteConfiguration.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/configuration/IgniteConfiguration.java b/modules/core/src/main/java/org/apache/ignite/configuration/IgniteConfiguration.java
index dc2102d..73233e4 100644
--- a/modules/core/src/main/java/org/apache/ignite/configuration/IgniteConfiguration.java
+++ b/modules/core/src/main/java/org/apache/ignite/configuration/IgniteConfiguration.java
@@ -37,7 +37,7 @@ import org.apache.ignite.spi.eventstorage.*;
 import org.apache.ignite.spi.failover.*;
 import org.apache.ignite.spi.loadbalancing.*;
 import org.apache.ignite.spi.securesession.*;
-import org.gridgain.grid.spi.swapspace.*;
+import org.apache.ignite.spi.swapspace.*;
 import org.gridgain.grid.util.typedef.internal.*;
 import org.jetbrains.annotations.*;
 
@@ -2123,9 +2123,9 @@ public class IgniteConfiguration {
     }
 
     /**
-     * Sets fully configured instances of {@link org.gridgain.grid.spi.swapspace.SwapSpaceSpi}.
+     * Sets fully configured instances of {@link org.apache.ignite.spi.swapspace.SwapSpaceSpi}.
      *
-     * @param swapSpaceSpi Fully configured instances of {@link org.gridgain.grid.spi.swapspace.SwapSpaceSpi} or
+     * @param swapSpaceSpi Fully configured instances of {@link org.apache.ignite.spi.swapspace.SwapSpaceSpi} or
      *      <tt>null</tt> if no SPI provided.
      * @see IgniteConfiguration#getSwapSpaceSpi()
      */
@@ -2135,7 +2135,7 @@ public class IgniteConfiguration {
 
     /**
      * Should return fully configured swap space SPI implementation. If not provided,
-     * {@link org.gridgain.grid.spi.swapspace.file.FileSwapSpaceSpi} will be used.
+     * {@link org.apache.ignite.spi.swapspace.file.FileSwapSpaceSpi} will be used.
      * <p>
      * Note that user can provide one or multiple instances of this SPI (and select later which one
      * is used in a particular context).

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/ef258ece/modules/core/src/main/java/org/apache/ignite/spi/IgniteSpiAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/IgniteSpiAdapter.java b/modules/core/src/main/java/org/apache/ignite/spi/IgniteSpiAdapter.java
index 328ac82..561414b 100644
--- a/modules/core/src/main/java/org/apache/ignite/spi/IgniteSpiAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/spi/IgniteSpiAdapter.java
@@ -20,7 +20,7 @@ import org.gridgain.grid.kernal.managers.communication.*;
 import org.gridgain.grid.kernal.managers.eventstorage.*;
 import org.gridgain.grid.security.*;
 import org.apache.ignite.spi.securesession.*;
-import org.gridgain.grid.spi.swapspace.*;
+import org.apache.ignite.spi.swapspace.*;
 import org.gridgain.grid.util.direct.*;
 import org.gridgain.grid.util.typedef.*;
 import org.gridgain.grid.util.typedef.internal.*;

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/ef258ece/modules/core/src/main/java/org/apache/ignite/spi/IgniteSpiContext.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/IgniteSpiContext.java b/modules/core/src/main/java/org/apache/ignite/spi/IgniteSpiContext.java
index a4b2c8d..39b281f 100644
--- a/modules/core/src/main/java/org/apache/ignite/spi/IgniteSpiContext.java
+++ b/modules/core/src/main/java/org/apache/ignite/spi/IgniteSpiContext.java
@@ -15,7 +15,7 @@ import org.gridgain.grid.*;
 import org.gridgain.grid.kernal.managers.communication.*;
 import org.gridgain.grid.kernal.managers.eventstorage.*;
 import org.gridgain.grid.security.*;
-import org.gridgain.grid.spi.swapspace.*;
+import org.apache.ignite.spi.swapspace.*;
 import org.gridgain.grid.util.direct.*;
 import org.jetbrains.annotations.*;
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/ef258ece/modules/core/src/main/java/org/apache/ignite/spi/swapspace/SwapContext.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/swapspace/SwapContext.java b/modules/core/src/main/java/org/apache/ignite/spi/swapspace/SwapContext.java
new file mode 100644
index 0000000..7d50d9b
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/spi/swapspace/SwapContext.java
@@ -0,0 +1,39 @@
+/* @java.file.header */
+
+/*  _________        _____ __________________        _____
+ *  __  ____/___________(_)______  /__  ____/______ ____(_)_______
+ *  _  / __  __  ___/__  / _  __  / _  / __  _  __ `/__  / __  __ \
+ *  / /_/ /  _  /    _  /  / /_/ /  / /_/ /  / /_/ / _  /  _  / / /
+ *  \____/   /_/     /_/   \_,__/   \____/   \__,_/  /_/   /_/ /_/
+ */
+
+package org.apache.ignite.spi.swapspace;
+
+import org.gridgain.grid.util.typedef.internal.*;
+
+/**
+ * Context for swap operations.
+ */
+public class SwapContext {
+    /** */
+    private ClassLoader clsLdr;
+
+    /**
+     * @return Class loader.
+     */
+    public ClassLoader classLoader() {
+        return clsLdr;
+    }
+
+    /**
+     * @param clsLdr Class loader.
+     */
+    public void classLoader(ClassLoader clsLdr) {
+        this.clsLdr = clsLdr;
+    }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return S.toString(SwapContext.class, this);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/ef258ece/modules/core/src/main/java/org/apache/ignite/spi/swapspace/SwapKey.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/swapspace/SwapKey.java b/modules/core/src/main/java/org/apache/ignite/spi/swapspace/SwapKey.java
new file mode 100644
index 0000000..b9ed06e
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/spi/swapspace/SwapKey.java
@@ -0,0 +1,116 @@
+/* @java.file.header */
+
+/*  _________        _____ __________________        _____
+ *  __  ____/___________(_)______  /__  ____/______ ____(_)_______
+ *  _  / __  __  ___/__  / _  __  / _  / __  _  __ `/__  / __  __ \
+ *  / /_/ /  _  /    _  /  / /_/ /  / /_/ /  / /_/ / _  /  _  / / /
+ *  \____/   /_/     /_/   \_,__/   \____/   \__,_/  /_/   /_/ /_/
+ */
+
+package org.apache.ignite.spi.swapspace;
+
+import org.gridgain.grid.util.typedef.internal.*;
+import org.gridgain.grid.util.tostring.*;
+import org.jetbrains.annotations.*;
+
+/**
+ * Utility wrapper class that represents swap key.
+ * <p>
+ * This class also holds information about partition this key belongs to
+ * (if needed for caches).
+ */
+public class SwapKey {
+    /** */
+    @GridToStringInclude
+    private final Object key;
+
+    /** */
+    private final int part;
+
+    /** Serialized key. */
+    @GridToStringExclude
+    private byte[] keyBytes;
+
+    /**
+     * @param key Key.
+     */
+    public SwapKey(Object key) {
+        this(key, Integer.MAX_VALUE, null);
+    }
+
+    /**
+     * @param key Key.
+     * @param part Partition.
+     */
+    public SwapKey(Object key, int part) {
+        this(key, part, null);
+    }
+
+    /**
+     * @param key Key.
+     * @param part Part.
+     * @param keyBytes Key bytes.
+     */
+    public SwapKey(Object key, int part, @Nullable byte[] keyBytes) {
+        assert key != null;
+        assert part >= 0;
+
+        this.key = key;
+        this.part = part;
+        this.keyBytes = keyBytes;
+    }
+
+    /**
+     * @return Key.
+     */
+    public Object key() {
+        return key;
+    }
+
+    /**
+     * @return Partition this key belongs to.
+     */
+    public int partition() {
+        return part;
+    }
+
+    /**
+     * @return Serialized key.
+     */
+    @Nullable public byte[] keyBytes() {
+        return keyBytes;
+    }
+
+    /**
+     * @param keyBytes Serialized key.
+     */
+    public void keyBytes(byte[] keyBytes) {
+        assert keyBytes != null;
+
+        this.keyBytes = keyBytes;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean equals(Object obj) {
+        if (obj == this)
+            return true;
+
+        if (obj instanceof SwapKey) {
+            SwapKey other = (SwapKey)obj;
+
+            return part == other.part && key.equals(other.key);
+        }
+
+        return false;
+    }
+
+    /** {@inheritDoc} */
+    @Override public int hashCode() {
+        return key.hashCode();
+    }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return S.toString(SwapKey.class, this);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/ef258ece/modules/core/src/main/java/org/apache/ignite/spi/swapspace/SwapSpaceSpi.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/swapspace/SwapSpaceSpi.java b/modules/core/src/main/java/org/apache/ignite/spi/swapspace/SwapSpaceSpi.java
new file mode 100644
index 0000000..17b2350
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/spi/swapspace/SwapSpaceSpi.java
@@ -0,0 +1,202 @@
+/* @java.file.header */
+
+/*  _________        _____ __________________        _____
+ *  __  ____/___________(_)______  /__  ____/______ ____(_)_______
+ *  _  / __  __  ___/__  / _  __  / _  / __  _  __ `/__  / __  __ \
+ *  / /_/ /  _  /    _  /  / /_/ /  / /_/ /  / /_/ / _  /  _  / / /
+ *  \____/   /_/     /_/   \_,__/   \____/   \__,_/  /_/   /_/ /_/
+ */
+
+package org.apache.ignite.spi.swapspace;
+
+import org.apache.ignite.lang.*;
+import org.apache.ignite.spi.*;
+import org.jetbrains.annotations.*;
+
+import java.util.*;
+
+/**
+ * Provides a mechanism in grid for storing data on disk. GridGain cache uses swap space to overflow
+ * data to disk if it cannot fit in memory. It's also possible to use swap space directly
+ * by calling {@link org.apache.ignite.Ignite} API swap-related methods. Logically storage is organized into
+ * independent 'spaces' in which data is stored.
+ * <p>
+ * All swap space implementations can be configured to prevent infinite growth and evict oldest entries.
+ * <p>
+ * The default swap space SPI is {@link org.apache.ignite.spi.swapspace.file.FileSwapSpaceSpi} which stores values on disk in files and keeps keys in
+ * memory.
+ * <p>
+ * Gridgain provides the following {@code GridSwapSpaceSpi} implementations:
+ * <ul>
+ * <li>
+ *     {@link org.apache.ignite.spi.swapspace.file.FileSwapSpaceSpi} - pure Java implementation with in-memory keys. This SPI is used by default.
+ * </li>
+ * <li>
+ *     {@link org.apache.ignite.spi.swapspace.noop.NoopSwapSpaceSpi} - no-op SPI mainly for testing.
+ * </li>
+ * </ul>
+ * <p>
+ * <p>
+ * <b>NOTE:</b> this SPI (i.e. methods in this interface) should never be used directly. SPIs provide
+ * internal view on the subsystem and is used internally by GridGain kernal. In rare use cases when
+ * access to a specific implementation of this SPI is required - an instance of this SPI can be obtained
+ * via {@link org.apache.ignite.Ignite#configuration()} method to check its configuration properties or call other non-SPI
+ * methods. Note again that calling methods from this interface on the obtained instance can lead
+ * to undefined behavior and explicitly not supported.
+ */
+public interface SwapSpaceSpi extends IgniteSpi {
+    /**
+     * Entirely clears data space with given name, if any.
+     *
+     * @param spaceName Space name to clear.
+     * @throws org.apache.ignite.spi.IgniteSpiException In case of any errors.
+     */
+    public void clear(@Nullable String spaceName) throws IgniteSpiException;
+
+    /**
+     * Gets size in bytes for data space with given name. If specified space does
+     * not exist this method returns {@code 0}.
+     *
+     * @param spaceName Space name to get size for.
+     * @return Size in bytes.
+     * @throws org.apache.ignite.spi.IgniteSpiException In case of any errors.
+     */
+    public long size(@Nullable String spaceName) throws IgniteSpiException;
+
+    /**
+     * Gets number of stored entries (keys) in data space with given name. If specified
+     * space does not exist this method returns {@code 0}.
+     *
+     * @param spaceName Space name to get number of entries for.
+     * @return Number of stored entries in specified space.
+     * @throws org.apache.ignite.spi.IgniteSpiException In case of any errors.
+     */
+    public long count(@Nullable String spaceName) throws IgniteSpiException;
+
+    /**
+     * Reads stored value as array of bytes by key from data space with given name.
+     * If specified space does not exist this method returns {@code null}.
+     *
+     * @param spaceName Name of the data space to read from.
+     * @param key Key used to read value from data space.
+     * @param ctx Swap context.
+     * @return Value as array of bytes stored in specified data space that matches
+     *      to given key.
+     * @throws org.apache.ignite.spi.IgniteSpiException In case of any errors.
+     */
+    @Nullable public byte[] read(@Nullable String spaceName, SwapKey key, SwapContext ctx)
+        throws IgniteSpiException;
+
+    /**
+     * Reads stored values as array of bytes by all passed keys from data space with
+     * given name. If specified space does not exist this method returns empty map.
+     *
+     * @param spaceName Name of the data space to read from.
+     * @param keys Keys used to read values from data space.
+     * @param ctx Swap context.
+     * @return Map in which keys are the ones passed into method and values are
+     *      corresponding values read from swap storage.
+     * @throws org.apache.ignite.spi.IgniteSpiException In case of any errors.
+     */
+    public Map<SwapKey, byte[]> readAll(@Nullable String spaceName,
+        Iterable<SwapKey> keys, SwapContext ctx) throws IgniteSpiException;
+
+    /**
+     * Removes value stored in data space with given name corresponding to specified key.
+     *
+     * @param spaceName Space name to remove value from.
+     * @param key Key to remove value in the specified space for.
+     * @param c Optional closure that takes removed value and executes after actual
+     *      removing. If there was no value in storage the closure is not executed.
+     * @param ctx Swap context.
+     * @throws org.apache.ignite.spi.IgniteSpiException In case of any errors.
+     */
+    public void remove(@Nullable String spaceName, SwapKey key,
+        @Nullable IgniteInClosure<byte[]> c, SwapContext ctx) throws IgniteSpiException;
+
+    /**
+     * Removes values stored in data space with given name corresponding to specified keys.
+     *
+     * @param spaceName Space name to remove values from.
+     * @param keys Keys to remove value in the specified space for.
+     * @param c Optional closure that takes removed value and executes after actual
+     *      removing. If there was no value in storage the closure is not executed.
+     * @param ctx Swap context.
+     * @throws org.apache.ignite.spi.IgniteSpiException In case of any errors.
+     */
+    public void removeAll(@Nullable String spaceName, Collection<SwapKey> keys,
+        @Nullable IgniteBiInClosure<SwapKey, byte[]> c, SwapContext ctx) throws IgniteSpiException;
+
+    /**
+     * Stores value as array of bytes with given key into data space with given name.
+     *
+     * @param spaceName Space name to store key-value pair into.
+     * @param key Key to store given value for. This key can be used further to
+     *      read or remove stored value.
+     * @param val Some value as array of bytes to store into specified data space.
+     * @param ctx Swap context.
+     * @throws org.apache.ignite.spi.IgniteSpiException In case of any errors.
+     */
+    public void store(@Nullable String spaceName, SwapKey key, @Nullable byte[] val, SwapContext ctx)
+        throws IgniteSpiException;
+
+    /**
+     * Stores key-value pairs (both keys and values are arrays of bytes) into data
+     * space with given name.
+     *
+     * @param spaceName Space name to store key-value pairs into.
+     * @param pairs Map of stored key-value pairs where each one is an array of bytes.
+     * @param ctx Swap context.
+     * @throws org.apache.ignite.spi.IgniteSpiException In case of any errors.
+     */
+    public void storeAll(@Nullable String spaceName, Map<SwapKey, byte[]> pairs, SwapContext ctx)
+        throws IgniteSpiException;
+
+    /**
+     * Sets eviction listener to receive notifications on evicted swap entries.
+     *
+     * @param evictLsnr Eviction listener ({@code null} to stop receiving notifications).
+     */
+    public void setListener(@Nullable SwapSpaceSpiListener evictLsnr);
+
+    /**
+     * Gets partitions IDs that are stored in the passed in space.
+     *
+     * @param spaceName Space name.
+     * @return Partitions IDs or {@code null} if space is unknown.
+     * @throws org.apache.ignite.spi.IgniteSpiException If failed.
+     */
+    @Nullable public Collection<Integer> partitions(@Nullable String spaceName) throws IgniteSpiException;
+
+    /**
+     * Gets iterator over space keys.
+     *
+     * @param spaceName Space name.
+     * @param ctx Swap context.
+     * @return Iterator over space entries or {@code null} if space is unknown.
+     * @throws org.apache.ignite.spi.IgniteSpiException If failed.
+     */
+    @Nullable <K> IgniteSpiCloseableIterator<K> keyIterator(@Nullable String spaceName, SwapContext ctx)
+        throws IgniteSpiException;
+
+    /**
+     * Gets raw iterator over space entries.
+     *
+     * @param spaceName Space name.
+     * @return Iterator over space entries or {@code null} if space is unknown.
+     * @throws org.apache.ignite.spi.IgniteSpiException If failed.
+     */
+    @Nullable public IgniteSpiCloseableIterator<Map.Entry<byte[], byte[]>> rawIterator(@Nullable String spaceName)
+        throws IgniteSpiException;
+
+    /**
+     * Gets raw iterator over space entries.
+     *
+     * @param spaceName Space name.
+     * @param part Partition.
+     * @return Iterator over space entries or {@code null} if space is unknown.
+     * @throws org.apache.ignite.spi.IgniteSpiException If failed.
+     */
+    @Nullable public IgniteSpiCloseableIterator<Map.Entry<byte[], byte[]>> rawIterator(@Nullable String spaceName,
+        int part) throws IgniteSpiException;
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/ef258ece/modules/core/src/main/java/org/apache/ignite/spi/swapspace/SwapSpaceSpiListener.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/swapspace/SwapSpaceSpiListener.java b/modules/core/src/main/java/org/apache/ignite/spi/swapspace/SwapSpaceSpiListener.java
new file mode 100644
index 0000000..54f24d6
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/spi/swapspace/SwapSpaceSpiListener.java
@@ -0,0 +1,26 @@
+/* @java.file.header */
+
+/*  _________        _____ __________________        _____
+ *  __  ____/___________(_)______  /__  ____/______ ____(_)_______
+ *  _  / __  __  ___/__  / _  __  / _  / __  _  __ `/__  / __  __ \
+ *  / /_/ /  _  /    _  /  / /_/ /  / /_/ /  / /_/ / _  /  _  / / /
+ *  \____/   /_/     /_/   \_,__/   \____/   \__,_/  /_/   /_/ /_/
+ */
+
+package org.apache.ignite.spi.swapspace;
+
+import org.jetbrains.annotations.*;
+
+/**
+ * Swap space SPI eviction listener.
+ */
+public interface SwapSpaceSpiListener {
+    /**
+     * Notification for swap space events.
+     *
+     * @param evtType Event type. See {@link org.apache.ignite.events.IgniteSwapSpaceEvent}
+     * @param spaceName Space name for this event or {@code null} for default space.
+     * @param keyBytes Key bytes of affected entry. Not {@code null} only for evict notifications.
+     */
+    public void onSwapEvent(int evtType, @Nullable String spaceName, @Nullable byte[] keyBytes);
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/ef258ece/modules/core/src/main/java/org/apache/ignite/spi/swapspace/file/FileSwapArray.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/swapspace/file/FileSwapArray.java b/modules/core/src/main/java/org/apache/ignite/spi/swapspace/file/FileSwapArray.java
new file mode 100644
index 0000000..374700b
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/spi/swapspace/file/FileSwapArray.java
@@ -0,0 +1,181 @@
+/* @java.file.header */
+
+/*  _________        _____ __________________        _____
+ *  __  ____/___________(_)______  /__  ____/______ ____(_)_______
+ *  _  / __  __  ___/__  / _  __  / _  / __  _  __ `/__  / __  __ \
+ *  / /_/ /  _  /    _  /  / /_/ /  / /_/ /  / /_/ / _  /  _  / / /
+ *  \____/   /_/     /_/   \_,__/   \____/   \__,_/  /_/   /_/ /_/
+ */
+
+package org.apache.ignite.spi.swapspace.file;
+
+import org.jetbrains.annotations.*;
+
+import java.util.concurrent.atomic.*;
+
+/**
+ * Growing array.
+ */
+class FileSwapArray<X> {
+    /** First partition size must be power of two. */
+    private static final int FIRST_ARRAY_SIZE = 4096;
+
+    /** */
+    private static final int LADDER_SIZE = Integer.numberOfLeadingZeros(FIRST_ARRAY_SIZE) + 1;
+
+    /** */
+    @SuppressWarnings("unchecked")
+    private final AtomicReferenceArray<X>[] ladder = new AtomicReferenceArray[LADDER_SIZE];
+
+    /** */
+    private int idx = 1;
+
+    /**
+     *
+     */
+    FileSwapArray() {
+        synchronized (ladder) {
+            ladder[0] = new AtomicReferenceArray<>(FIRST_ARRAY_SIZE);
+        }
+    }
+
+    /**
+     * @return Size.
+     */
+    public int size() {
+        return idx;
+    }
+
+    /**
+     * Adds value to the end.
+     *
+     * @param x Value.
+     * @return Index where it was added.
+     */
+    int add(X x) {
+        int i = idx++;
+
+        assert i >= 0 && i != Integer.MAX_VALUE : "Integer overflow";
+
+        Slot<X> s = slot(i);
+
+        assert s != null; // We should add always in one thread.
+
+        s.set(x);
+
+        int len = s.arr.length();
+
+        if (s.idx + 1 == len) {
+            synchronized (ladder) {
+                ladder[s.arrIdx + 1] = new AtomicReferenceArray<>(s.arrIdx == 0 ? len : len << 1);
+            }
+        }
+
+        return i;
+    }
+
+    /**
+     * @param size New size.
+     */
+    void truncate(int size) {
+        assert size > 0;
+
+        idx = size;
+
+        int arrIdx = arrayIndex(idx) + 1;
+
+        if (arrIdx < ladder.length && ladder[arrIdx] != null) {
+            synchronized (ladder) {
+                do {
+                    ladder[arrIdx++] = null;
+                }
+                while (arrIdx < ladder.length && ladder[arrIdx] != null);
+            }
+        }
+    }
+
+    /**
+     * @param idx Absolute slot index.
+     * @return Array index in {@link #ladder}.
+     */
+    static int arrayIndex(int idx) {
+        if (idx < FIRST_ARRAY_SIZE)
+            return 0;
+
+        return LADDER_SIZE - Integer.numberOfLeadingZeros(idx);
+    }
+
+    /**
+     * Slot for given absolute index.
+     *
+     * @param idx Absolute index.
+     * @return Slot.
+     */
+    @Nullable Slot<X> slot(int idx) {
+        assert idx > 0 : idx;
+
+        int arrIdx = arrayIndex(idx);
+
+        AtomicReferenceArray<X> arr = ladder[arrIdx];
+
+        if (arr == null) {
+            synchronized (ladder) { // Ensure visibility.
+                arr = ladder[arrIdx];
+            }
+
+            if (arr == null)
+                return null;
+        }
+
+        return new Slot<>(arrIdx, arr, arrIdx == 0 ? idx : idx - arr.length());
+    }
+
+    /**
+     * Slot in array.
+     */
+    @SuppressWarnings("PublicInnerClass")
+    static final class Slot<X> {
+        /** */
+        private final int arrIdx;
+
+        /** */
+        private final AtomicReferenceArray<X> arr;
+
+        /** */
+        private final int idx;
+
+        /**
+         * @param arrIdx Index of array.
+         * @param arr Array.
+         * @param idx Index within the array.
+         */
+        private Slot(int arrIdx, AtomicReferenceArray<X> arr, int idx) {
+            this.arrIdx = arrIdx;
+            this.arr = arr;
+            this.idx = idx;
+        }
+
+        /**
+         * @return Value.
+         */
+        public X get() {
+            return arr.get(idx);
+        }
+
+        /**
+         * @param exp Expected.
+         * @param x New value.
+         * @return {@code true} If succeeded.
+         */
+        public boolean cas(@Nullable X exp, @Nullable X x) {
+            return exp == x || arr.compareAndSet(idx, exp, x);
+        }
+
+        /**
+         * @param x value.
+         */
+        private void set(X x) {
+            arr.lazySet(idx, x);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/ef258ece/modules/core/src/main/java/org/apache/ignite/spi/swapspace/file/FileSwapSpaceSpi.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/swapspace/file/FileSwapSpaceSpi.java b/modules/core/src/main/java/org/apache/ignite/spi/swapspace/file/FileSwapSpaceSpi.java
new file mode 100644
index 0000000..1af22f3
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/spi/swapspace/file/FileSwapSpaceSpi.java
@@ -0,0 +1,1843 @@
+/* @java.file.header */
+
+/*  _________        _____ __________________        _____
+ *  __  ____/___________(_)______  /__  ____/______ ____(_)_______
+ *  _  / __  __  ___/__  / _  __  / _  / __  _  __ `/__  / __  __ \
+ *  / /_/ /  _  /    _  /  / /_/ /  / /_/ /  / /_/ / _  /  _  / / /
+ *  \____/   /_/     /_/   \_,__/   \____/   \__,_/  /_/   /_/ /_/
+ */
+
+package org.apache.ignite.spi.swapspace.file;
+
+import org.apache.ignite.*;
+import org.apache.ignite.lang.*;
+import org.apache.ignite.marshaller.*;
+import org.apache.ignite.resources.*;
+import org.apache.ignite.spi.*;
+import org.gridgain.grid.*;
+import org.apache.ignite.spi.swapspace.*;
+import org.gridgain.grid.util.*;
+import org.gridgain.grid.util.typedef.*;
+import org.gridgain.grid.util.typedef.internal.*;
+import org.jdk8.backport.*;
+import org.jetbrains.annotations.*;
+
+import java.io.*;
+import java.nio.*;
+import java.nio.channels.*;
+import java.util.*;
+import java.util.concurrent.*;
+import java.util.concurrent.atomic.*;
+import java.util.concurrent.locks.*;
+
+import static org.apache.ignite.events.IgniteEventType.*;
+
+/**
+ * File-based swap space SPI implementation which holds keys in memory. This SPI is used by default.
+ * It is intended for use in cases when value size is bigger than {@code 100} bytes, otherwise it will not
+ * have any positive effect.
+ * <p>
+ * <b>NOTE: This SPI does not support swap eviction currently, manual removes needed to reduce disk space
+ * consumption.</b>
+ * <p>
+ * Every space has a name and when used in combination with in-memory data grid name and local node ID,
+ * space name represents the actual cache name associated with this swap space. Default name is {@code null}
+ * which is represented by {@link #DFLT_SPACE_NAME}.
+ *
+ * <h1 class="header">Configuration</h1>
+ * <h2 class="header">Mandatory</h2>
+ * This SPI has no mandatory configuration parameters.
+ * <h2 class="header">Optional SPI configuration.</h2>
+ * <ul>
+ *     <li>Base directory path (see {@link #setBaseDirectory(String)}).</li>
+ *     <li>Maximum sparsity (see {@link #setMaximumSparsity(float)}).</li>
+ *     <li>Write buffer size in bytes (see {@link #setWriteBufferSize(int)}).</li>
+ *     <li>Max write queue size in bytes (see {@link #setMaxWriteQueueSize(int)}).</li>
+ *     <li>Read stripes number. (see {@link #setReadStripesNumber(int)}).</li>
+ * </ul>
+ *
+ * <h2 class="header">Java Example</h2>
+ * GridFileSwapSpaceSpi is configured by default and should be explicitly configured
+ * only if some SPI configuration parameters need to be overridden.
+ * <pre name="code" class="java">
+ * GridFileSwapSpaceSpi spi = new GridFileSwapSpaceSpi();
+ *
+ * // Configure root folder path.
+ * spi.setBaseDirectory("/path/to/swap/folder");
+ *
+ * GridConfiguration cfg = new GridConfiguration();
+ *
+ * // Override default swap space SPI.
+ * cfg.setSwapSpaceSpi(spi);
+ *
+ * // Starts grid.
+ * G.start(cfg);
+ * </pre>
+ * <h2 class="header">Spring Example</h2>
+ * GridFileSwapSpaceSpi can be configured from Spring XML configuration file:
+ * <pre name="code" class="xml">
+ * &lt;bean id=&quot;grid.cfg&quot; class=&quot;org.gridgain.grid.GridConfiguration&quot; scope=&quot;singleton&quot;&gt;
+ *     ...
+ *     &lt;property name=&quot;swapSpaceSpi&quot;&gt;
+ *         &lt;bean class=&quot;org.gridgain.grid.spi.swapspace.file.GridFileSwapSpaceSpi&quot;&gt;
+ *             &lt;property name=&quot;baseDirectory&quot; value=&quot;/path/to/swap/folder&quot;/&gt;
+ *         &lt;/bean&gt;
+ *     &lt;/property&gt;
+ *     ...
+ * &lt;/bean&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.apache.ignite.spi.swapspace.SwapSpaceSpi
+ */
+@IgniteSpiMultipleInstancesSupport(true)
+@SuppressWarnings({"PackageVisibleInnerClass", "PackageVisibleField"})
+public class FileSwapSpaceSpi extends IgniteSpiAdapter implements SwapSpaceSpi, FileSwapSpaceSpiMBean {
+    /**
+     * Default base directory. Note that this path is relative to {@code GRIDGAIN_HOME/work} folder
+     * if {@code GRIDGAIN_HOME} system or environment variable specified, otherwise it is relative to
+     * {@code work} folder under system {@code java.io.tmpdir} folder.
+     *
+     * @see org.apache.ignite.configuration.IgniteConfiguration#getWorkDirectory()
+     */
+    public static final String DFLT_BASE_DIR = "swapspace";
+
+    /** Default maximum sparsity. */
+    public static final float DFLT_MAX_SPARSITY = 0.5f;
+
+    /** Default write buffer size in bytes. */
+    public static final int DFLT_BUF_SIZE = 64 * 1024;
+
+    /** Default write queue size in bytes. */
+    public static final int DFLT_QUE_SIZE = 1024 * 1024;
+
+    /** Name for {@code null} space. */
+    public static final String DFLT_SPACE_NAME = "gg-dflt-space";
+
+    /** Spaces. */
+    private final ConcurrentMap<String, Space> spaces = new ConcurrentHashMap<>();
+
+    /** Base directory. */
+    private String baseDir = DFLT_BASE_DIR;
+
+    /** Maximum sparsity. */
+    private float maxSparsity = DFLT_MAX_SPARSITY;
+
+    /** Eviction listener. */
+    private volatile SwapSpaceSpiListener evictLsnr;
+
+    /** Directory. */
+    private File dir;
+
+    /** Write buffer size. */
+    private int writeBufSize = DFLT_BUF_SIZE;
+
+    /** Max write queue size in bytes. */
+    private int maxWriteQueSize = DFLT_QUE_SIZE;
+
+    /** Read stripes number. */
+    private int readStripesNum = -1;
+
+    /** Logger. */
+    @IgniteLoggerResource
+    private IgniteLogger log;
+
+    /** Local node ID. */
+    @IgniteLocalNodeIdResource
+    private UUID locNodeId;
+
+    /** Name of the grid. */
+    @IgniteNameResource
+    private String gridName;
+
+    /** Marshaller. */
+    @IgniteMarshallerResource
+    private IgniteMarshaller marsh;
+
+    /** {@inheritDoc} */
+    @Override public String getBaseDirectory() {
+        return baseDir;
+    }
+
+    /**
+     * Sets base directory.
+     *
+     * @param baseDir Base directory.
+     */
+    @IgniteSpiConfiguration(optional = true)
+    public void setBaseDirectory(String baseDir) {
+        this.baseDir = baseDir;
+    }
+
+    /** {@inheritDoc} */
+    @Override public float getMaximumSparsity() {
+        return maxSparsity;
+    }
+
+    /**
+     * Sets maximum sparsity. This property defines maximum acceptable wasted file space to whole file size ratio.
+     * When this ratio becomes higher than specified number compacting thread starts working.
+     *
+     * @param maxSparsity Maximum sparsity. Must be between 0 and 1, default is {@link #DFLT_MAX_SPARSITY}.
+     */
+    public void setMaximumSparsity(float maxSparsity) {
+        this.maxSparsity = maxSparsity;
+    }
+
+    /** {@inheritDoc} */
+    @Override public int getWriteBufferSize() {
+        return writeBufSize;
+    }
+
+    /**
+     * Sets write buffer size in bytes. Write to disk occurs only when this buffer is full. Default is
+     * {@link #DFLT_BUF_SIZE}.
+     *
+     * @param writeBufSize Write buffer size in bytes.
+     */
+    public void setWriteBufferSize(int writeBufSize) {
+        this.writeBufSize = writeBufSize;
+    }
+
+    /** {@inheritDoc} */
+    @Override public int getMaxWriteQueueSize() {
+        return maxWriteQueSize;
+    }
+
+    /**
+     * Sets max write queue size in bytes. If there are more values are waiting for being written to disk then specified
+     * size, SPI will block on {@link #store(String, org.apache.ignite.spi.swapspace.SwapKey, byte[], org.apache.ignite.spi.swapspace.SwapContext)} operation. Default is
+     * {@link #DFLT_QUE_SIZE}.
+     *
+     * @param maxWriteQueSize Max write queue size in bytes.
+     */
+    public void setMaxWriteQueueSize(int maxWriteQueSize) {
+        this.maxWriteQueSize = maxWriteQueSize;
+    }
+
+    /** {@inheritDoc} */
+    @Override public int getReadStripesNumber() {
+        return readStripesNum;
+    }
+
+    /**
+     * Sets read stripe size. Defines number of file channels to be used concurrently. Default is equal to number of
+     * CPU cores available to this JVM.
+     *
+     * @param readStripesNum Read stripe number.
+     */
+    public void setReadStripesNumber(int readStripesNum) {
+        A.ensure(readStripesNum == -1 || (readStripesNum & (readStripesNum - 1)) == 0,
+            "readStripesNum must be positive and power of two");
+
+        this.readStripesNum = readStripesNum;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void spiStart(@Nullable String gridName) throws IgniteSpiException {
+        assertParameter(!F.isEmpty(baseDir), "!F.isEmpty(baseDir)");
+        assertParameter(maxSparsity >= 0 && maxSparsity < 1, "maxSparsity >= 0 && maxSparsity < 1");
+        assertParameter(readStripesNum == -1 || (readStripesNum & (readStripesNum - 1)) == 0,
+            "readStripesNum must be positive and power of two.");
+
+        if (readStripesNum == -1) {
+            // User has not configured the number.
+            int readStripesNum0 = 1;
+            int cpuCnt = Runtime.getRuntime().availableProcessors();
+
+            while (readStripesNum0 <= cpuCnt)
+                readStripesNum0 <<= 1;
+
+            if (readStripesNum0 > cpuCnt)
+                readStripesNum0 >>= 1;
+
+            assert readStripesNum0 > 0 && (readStripesNum0 & readStripesNum0 - 1) == 0;
+
+            readStripesNum = readStripesNum0;
+        }
+
+        startStopwatch();
+
+        registerMBean(gridName, this, FileSwapSpaceSpiMBean.class);
+
+        String path = baseDir + File.separator + gridName + File.separator + locNodeId;
+
+        try {
+            dir = U.resolveWorkDirectory(path, true);
+        }
+        catch (GridException e) {
+            throw new IgniteSpiException(e);
+        }
+
+        if (log.isDebugEnabled())
+            log.debug(startInfo());
+    }
+
+    /** {@inheritDoc} */
+    @Override public void spiStop() throws IgniteSpiException {
+        unregisterMBean();
+
+        for (Space space : spaces.values()) {
+            space.initialize();
+
+            try {
+                space.stop();
+            }
+            catch (GridInterruptedException e) {
+                U.error(log, "Interrupted.", e);
+            }
+        }
+
+        if (dir != null && dir.exists() && !U.delete(dir))
+            U.warn(log, "Failed to delete swap directory: " + dir.getAbsolutePath());
+
+        if (log.isDebugEnabled())
+            log.debug(stopInfo());
+    }
+
+    /** {@inheritDoc} */
+    @Override public void clear(@Nullable String spaceName) throws IgniteSpiException {
+        Space space = space(spaceName, false);
+
+        if (space == null)
+            return;
+
+        space.clear();
+
+        notifyListener(EVT_SWAP_SPACE_CLEARED, spaceName);
+    }
+
+    /** {@inheritDoc} */
+    @Override public long size(@Nullable String spaceName) throws IgniteSpiException {
+        Space space = space(spaceName, false);
+
+        if (space == null)
+            return 0;
+
+        return space.size();
+    }
+
+    /** {@inheritDoc} */
+    @Override public long count(@Nullable String spaceName) throws IgniteSpiException {
+        Space space = space(spaceName, false);
+
+        if (space == null)
+            return 0;
+
+        return space.count();
+    }
+
+    /** {@inheritDoc} */
+    @Nullable @Override public byte[] read(@Nullable String spaceName, SwapKey key, SwapContext ctx)
+        throws IgniteSpiException {
+        assert key != null;
+        assert ctx != null;
+
+        Space space = space(spaceName, false);
+
+        if (space == null)
+            return null;
+
+        byte[] val = space.read(key);
+
+        notifyListener(EVT_SWAP_SPACE_DATA_READ, spaceName);
+
+        return val;
+    }
+
+    /** {@inheritDoc} */
+    @Override public Map<SwapKey, byte[]> readAll(@Nullable String spaceName, Iterable<SwapKey> keys,
+        SwapContext ctx) throws IgniteSpiException {
+        assert keys != null;
+        assert ctx != null;
+
+        Space space = space(spaceName, false);
+
+        if (space == null)
+            return Collections.emptyMap();
+
+        Map<SwapKey, byte[]> res = new HashMap<>();
+
+        for (SwapKey key : keys) {
+            if (key != null) {
+                byte[] val = space.read(key);
+
+                if (val != null)
+                    res.put(key, val);
+
+                notifyListener(EVT_SWAP_SPACE_DATA_READ, spaceName);
+            }
+        }
+
+        return res;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void remove(@Nullable String spaceName, SwapKey key, @Nullable IgniteInClosure<byte[]> c,
+        SwapContext ctx) throws IgniteSpiException {
+        assert key != null;
+        assert ctx != null;
+
+        Space space = space(spaceName, false);
+
+        if (space == null)
+            return;
+
+        byte[] val = space.remove(key, c != null);
+
+        if (c != null)
+            c.apply(val);
+
+        notifyListener(EVT_SWAP_SPACE_DATA_REMOVED, spaceName);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void removeAll(@Nullable String spaceName, Collection<SwapKey> keys,
+        @Nullable IgniteBiInClosure<SwapKey, byte[]> c, SwapContext ctx) throws IgniteSpiException {
+        assert keys != null;
+        assert ctx != null;
+
+        Space space = space(spaceName, false);
+
+        if (space == null)
+            return;
+
+        for (SwapKey key : keys) {
+            if (key != null) {
+                byte[] val = space.remove(key, c != null);
+
+                if (c != null)
+                    c.apply(key, val);
+
+                notifyListener(EVT_SWAP_SPACE_DATA_REMOVED, spaceName);
+            }
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public void store(@Nullable String spaceName, SwapKey key, @Nullable byte[] val,
+        SwapContext ctx) throws IgniteSpiException {
+        assert key != null;
+        assert ctx != null;
+
+        Space space = space(spaceName, true);
+
+        assert space != null;
+
+        space.store(key, val);
+
+        notifyListener(EVT_SWAP_SPACE_DATA_STORED, spaceName);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void storeAll(@Nullable String spaceName, Map<SwapKey, byte[]> pairs,
+        SwapContext ctx) throws IgniteSpiException {
+        assert pairs != null;
+        assert ctx != null;
+
+        Space space = space(spaceName, true);
+
+        assert space != null;
+
+        for (Map.Entry<SwapKey, byte[]> pair : pairs.entrySet()) {
+            SwapKey key = pair.getKey();
+
+            if (key != null) {
+                space.store(key, pair.getValue());
+
+                notifyListener(EVT_SWAP_SPACE_DATA_STORED, spaceName);
+            }
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public void setListener(@Nullable SwapSpaceSpiListener evictLsnr) {
+        this.evictLsnr = evictLsnr;
+    }
+
+    /** {@inheritDoc} */
+    @Nullable @Override public Collection<Integer> partitions(@Nullable String spaceName)
+        throws IgniteSpiException {
+        Space space = space(spaceName, false);
+
+        if (space == null)
+            return null;
+
+        return space.partitions();
+    }
+
+    /** {@inheritDoc} */
+    @Nullable @Override public <K> IgniteSpiCloseableIterator<K> keyIterator(@Nullable String spaceName,
+        SwapContext ctx) throws IgniteSpiException {
+        final Space space = space(spaceName, false);
+
+        if (space == null)
+            return null;
+
+        final Iterator<Map.Entry<SwapKey, byte[]>> iter = space.entriesIterator();
+
+        return new GridCloseableIteratorAdapter<K>() {
+            @Override protected boolean onHasNext() {
+                return iter.hasNext();
+            }
+
+            @Override protected K onNext() {
+                return (K)iter.next().getKey().key();
+            }
+
+            @Override protected void onRemove() {
+                iter.remove();
+            }
+        };
+    }
+
+    /** {@inheritDoc} */
+    @Nullable @Override public IgniteSpiCloseableIterator<Map.Entry<byte[], byte[]>> rawIterator(
+        @Nullable String spaceName) throws IgniteSpiException {
+        Space space = space(spaceName, false);
+
+        if (space == null)
+            return null;
+
+        return rawIterator(space.entriesIterator());
+    }
+
+    /** {@inheritDoc} */
+    @Nullable @Override public IgniteSpiCloseableIterator<Map.Entry<byte[], byte[]>> rawIterator(
+        @Nullable String spaceName, int part) throws IgniteSpiException {
+        Space space = space(spaceName, false);
+
+        if (space == null)
+            return null;
+
+        return rawIterator(space.entriesIterator(part));
+    }
+
+    /**
+     * Creates raw iterator based on provided entries iterator.
+     *
+     * @param iter Entries iterator.
+     * @return Raw iterator.
+     */
+    private IgniteSpiCloseableIterator<Map.Entry<byte[], byte[]>> rawIterator(
+        final Iterator<Map.Entry<SwapKey, byte[]>> iter) {
+        return new GridCloseableIteratorAdapter<Map.Entry<byte[], byte[]>>() {
+            @Override protected Map.Entry<byte[], byte[]> onNext() throws GridException {
+                Map.Entry<SwapKey, byte[]> x = iter.next();
+
+                return new T2<>(keyBytes(x.getKey()), x.getValue());
+            }
+
+            @Override protected boolean onHasNext() {
+                return iter.hasNext();
+            }
+
+            @Override protected void onRemove() {
+                iter.remove();
+            }
+        };
+    }
+
+    /**
+     * Gets key bytes.
+     *
+     * @param key Swap key.
+     * @return Key bytes.
+     * @throws org.apache.ignite.spi.IgniteSpiException In case of error.
+     */
+    private byte[] keyBytes(SwapKey key) throws IgniteSpiException {
+        assert key != null;
+
+        byte[] keyBytes = key.keyBytes();
+
+        if (keyBytes == null) {
+            try {
+                keyBytes = marsh.marshal(key.key());
+            }
+            catch (GridException e) {
+                throw new IgniteSpiException("Failed to marshal key: " + key.key(), e);
+            }
+
+            key.keyBytes(keyBytes);
+        }
+
+        return keyBytes;
+    }
+
+    /**
+     * Notifies eviction listener.
+     *
+     * @param evtType Event type.
+     * @param spaceName Space name.
+     */
+    private void notifyListener(int evtType, @Nullable String spaceName) {
+        SwapSpaceSpiListener lsnr = evictLsnr;
+
+        if (lsnr != null)
+            lsnr.onSwapEvent(evtType, spaceName, null);
+    }
+
+    /**
+     * Gets space by name.
+     *
+     * @param name Space name.
+     * @param create Whether to create space if it doesn't exist.
+     * @return Space.
+     * @throws org.apache.ignite.spi.IgniteSpiException In case of error.
+     */
+    @Nullable private Space space(@Nullable String name, boolean create) throws IgniteSpiException {
+        String masked = name != null ? name : DFLT_SPACE_NAME;
+
+        assert masked != null;
+
+        Space space = spaces.get(masked);
+
+        if (space == null && create) {
+            validateName(name);
+
+            Space old = spaces.putIfAbsent(masked, space = new Space(masked));
+
+            if (old != null)
+                space = old;
+        }
+
+        if (space != null)
+            space.initialize();
+
+        return space;
+    }
+
+    /**
+     * Validates space name.
+     *
+     * @param name Space name.
+     * @throws org.apache.ignite.spi.IgniteSpiException If name is invalid.
+     */
+    private void validateName(@Nullable String name) throws IgniteSpiException {
+        if (name == null)
+            return;
+
+        if (name.isEmpty())
+            throw new IgniteSpiException("Space name cannot be empty: " + name);
+        else if (DFLT_SPACE_NAME.equalsIgnoreCase(name))
+            throw new IgniteSpiException("Space name is reserved for default space: " + name);
+        else if (name.contains("/") || name.contains("\\"))
+            throw new IgniteSpiException("Space name contains invalid characters: " + name);
+    }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return S.toString(FileSwapSpaceSpi.class, this);
+    }
+
+    /**
+     * Swap value.
+     */
+    static class SwapValue {
+        /** */
+        private static final int NEW = 0;
+
+        /** */
+        private static final int DELETED = Integer.MIN_VALUE;
+
+        /** */
+        private static final AtomicIntegerFieldUpdater<SwapValue> idxUpdater = AtomicIntegerFieldUpdater.
+            newUpdater(SwapValue.class, "idx");
+
+        /** */
+        private byte[] val;
+
+        /** */
+        private final int len;
+
+        /** */
+        @SuppressWarnings("FieldAccessedSynchronizedAndUnsynchronized")
+        private long pos = -1;
+
+        /** */
+        @SuppressWarnings("UnusedDeclaration")
+        private volatile int idx;
+
+        /**
+         * @param val Value.
+         */
+        SwapValue(byte[] val) {
+            assert val != null;
+
+            this.val = val;
+            len = val.length;
+        }
+
+        /**
+         * @param space Space.
+         * @return Value.
+         * @throws org.apache.ignite.spi.IgniteSpiException If failed.
+         */
+        @SuppressWarnings("NonPrivateFieldAccessedInSynchronizedContext")
+        @Nullable public synchronized byte[] value(Space space) throws IgniteSpiException {
+            byte[] v = val;
+
+            if (v == null) { // Read value from file.
+                int i = idx;
+
+                assert i != NEW;
+
+                if (i != DELETED) {
+                    StripedFileChannel ch = i < 0 ? space.left.readCh : space.right.readCh;
+
+                    if (idx != DELETED) // Double check works in pair with striped channel reopening.
+                        v = readValue(ch);
+                }
+            }
+            else if (v.length != len) {
+                int p = (int)pos;
+
+                v = Arrays.copyOfRange(v, p, p + len); // In case of compaction.
+            }
+
+            return v;
+        }
+
+        /**
+         * @param ch File channel.
+         * @return Bytes.
+         * @throws org.apache.ignite.spi.IgniteSpiException if failed.
+         */
+        @Nullable byte[] readValue(StripedFileChannel ch) throws IgniteSpiException {
+            byte[] v = new byte[len];
+
+            int res = 0;
+
+            try {
+                res = ch.read(ByteBuffer.wrap(v), pos);
+            }
+            catch (ClosedChannelException ignore) {
+                assert idx == DELETED;
+            }
+            catch (IOException e) {
+                throw new IgniteSpiException("Failed to read value.", e);
+            }
+
+            if (res < len)
+                return null; // When concurrent compaction occurs this may happen.
+
+            return v;
+        }
+
+        /**
+         * @param pos Position.
+         * @param val Value.
+         */
+        public synchronized void set(long pos, byte[] val) {
+            if (pos != -1)
+                this.pos = pos;
+
+            this.val = val;
+        }
+
+        /**
+         * @param exp Expected.
+         * @param idx New index.
+         * @return {@code true} if succeeded.
+         */
+        public boolean casIdx(int exp, int idx) {
+            return idxUpdater.compareAndSet(this, exp, idx);
+        }
+
+        /**
+         * @return Index in file array.
+         */
+        int idx() {
+            return idx;
+        }
+
+        /** {@inheritDoc} */
+        @Override public String toString() {
+            return pos + " " + len;
+        }
+    }
+
+    /**
+     * Queue of swap values.
+     */
+    private static class SwapValuesQueue {
+        /** */
+        private final ArrayDeque<SwapValue> deq = new ArrayDeque<>();
+
+        /** */
+        @SuppressWarnings("TypeMayBeWeakened")
+        private final ReentrantLock lock = new ReentrantLock();
+
+        /** */
+        private final Condition mayAdd = lock.newCondition();
+
+        /** */
+        private final Condition mayTake = lock.newCondition();
+
+        /** */
+        private int size;
+
+        /** */
+        private final int minTakeSize;
+
+        /** */
+        private final int maxSize;
+
+        /**
+         * @param minTakeSize Min size.
+         * @param maxSize Max size.
+         */
+        private SwapValuesQueue(int minTakeSize, int maxSize) {
+            this.minTakeSize = minTakeSize;
+            this.maxSize = maxSize;
+        }
+
+        /**
+         * Adds to queue.
+         *
+         * @param val Swap value.
+         * @throws org.apache.ignite.spi.IgniteSpiException If failed.
+         */
+        public void add(SwapValue val) throws IgniteSpiException {
+            lock.lock();
+
+            try {
+                while (size + val.len > maxSize)
+                    mayAdd.await();
+
+                size += val.len;
+
+                deq.addLast(val);
+
+                if (size >= minTakeSize)
+                    mayTake.signalAll();
+            }
+            catch (InterruptedException e) {
+                throw new IgniteSpiException(e);
+            }
+            finally {
+                lock.unlock();
+            }
+        }
+
+        /**
+         * Takes swap values from queue.
+         *
+         * @return Swap values.
+         * @throws InterruptedException If interrupted.
+         */
+        public SwapValues take() throws InterruptedException {
+            lock.lock();
+
+            try {
+                while (size < minTakeSize)
+                    mayTake.await();
+
+                int size = 0;
+                int cnt = 0;
+
+                for (SwapValue val : deq) {
+                    size += val.len;
+                    cnt++;
+
+                    if (size >= minTakeSize)
+                        break;
+                }
+
+                SwapValue[] vals = new SwapValue[cnt];
+
+                for (int i = 0; i < cnt; i++) {
+                    SwapValue val = deq.pollFirst();
+
+                    vals[i] = val;
+                }
+
+                if ((this.size -= size) < maxSize)
+                    mayAdd.signalAll();
+
+                return new SwapValues(vals, size);
+            }
+            finally {
+                lock.unlock();
+            }
+        }
+    }
+
+    /**
+     * Array of swap values and their size in bytes.
+     */
+    static class SwapValues {
+        /** */
+        private final SwapValue[] vals;
+
+        /** Size in bytes. */
+        private final int size;
+
+        /**
+         * @param vals Values.
+         * @param size Size.
+         */
+        SwapValues(SwapValue[] vals, int size) {
+            this.vals = vals;
+            this.size = size;
+        }
+    }
+
+    /**
+     * Readable striped file channel.
+     */
+    private static class StripedFileChannel {
+        /** */
+        private final AtomicInteger enter = new AtomicInteger();
+
+        /** */
+        private final RandomAccessFile[] rafs;
+
+        /** */
+        private final FileChannel[] chs;
+
+        /**
+         * @param f File.
+         * @param stripes Stripes.
+         * @throws FileNotFoundException If failed.
+         */
+        StripedFileChannel(File f, int stripes) throws FileNotFoundException {
+            assert stripes > 0 && (stripes & (stripes - 1)) == 0 : "stripes must be positive and power of two.";
+
+            rafs = new RandomAccessFile[stripes];
+            chs = new FileChannel[stripes];
+
+            for (int i = 0; i < stripes; i++) {
+                RandomAccessFile raf = new RandomAccessFile(f, "r");
+
+                rafs[i] = raf;
+                chs[i] = raf.getChannel();
+            }
+        }
+
+        /**
+         * Reads data from file channel to buffer.
+         *
+         * @param buf Buffer.
+         * @param pos Position.
+         * @return Read bytes count.
+         * @throws IOException If failed.
+         */
+        int read(ByteBuffer buf, long pos) throws IOException {
+            int i = enter.getAndIncrement() & (chs.length - 1);
+
+            return chs[i].read(buf, pos);
+        }
+
+        /**
+         * Closes channel.
+         */
+        void close() {
+            for (RandomAccessFile raf : rafs)
+                U.closeQuiet(raf);
+        }
+    }
+
+    /**
+     * Swap file.
+     */
+    static class SwapFile {
+        /** */
+        private static final long MIN_TRUNK_SIZE = 10 * 1024 * 1024;
+
+        /** */
+        private final File file;
+
+        /** */
+        private final RandomAccessFile raf;
+
+        /** */
+        private final FileChannel writeCh;
+
+        /** */
+        volatile StripedFileChannel readCh;
+
+        /** */
+        private volatile long len;
+
+        /** */
+        private final FileSwapArray<SwapValue> arr = new FileSwapArray<>();
+
+        /**
+         * @param file File.
+         * @param readerStripes Reader stripes number.
+         * @throws IOException In case of error.
+         */
+        SwapFile(File file, int readerStripes) throws IOException {
+            assert file != null;
+
+            file.delete();
+
+            if (!file.createNewFile())
+                throw new IllegalStateException("Failed to create file: " + file.getAbsolutePath());
+
+            this.file = file;
+
+            raf = new RandomAccessFile(file, "rw");
+
+            writeCh = raf.getChannel();
+
+            readCh = new StripedFileChannel(file, readerStripes);
+        }
+
+        /**
+         * Reopens read channel.
+         *
+         * @throws FileNotFoundException If failed.
+         */
+        void reopenReadChannel() throws FileNotFoundException {
+            readCh.close();
+
+            readCh = new StripedFileChannel(file, readCh.chs.length);
+        }
+
+        /**
+         * @param vals Values.
+         * @param buf Duffer.
+         * @param sign Indicates where should we write value, to the left or to the right.
+         * @throws Exception If failed.
+         */
+        public void write(Iterable<SwapValue> vals, ByteBuffer buf, int sign) throws Exception {
+            for (SwapValue val : vals) {
+                int oldIdx = val.idx;
+
+                if (oldIdx == SwapValue.DELETED)
+                    continue;
+
+                int idx = arr.add(val);
+
+                if (!val.casIdx(oldIdx, sign * idx)) {
+                    assert val.idx == SwapValue.DELETED;
+
+                    boolean res = tryRemove(idx, val);
+
+                    assert res;
+                }
+            }
+
+            final int size = buf.remaining();
+
+            if (size == 0)
+                return;
+
+            long pos = len;
+
+            len = pos + size;
+
+            long res = writeCh.write(buf, pos);
+
+            if (res != size)
+                throw new IllegalStateException(res + " != " + size);
+
+            // Nullify bytes in values ans set pos.
+            for (SwapValue val : vals) {
+                val.set(pos, null);
+
+                pos += val.len;
+            }
+        }
+
+        /**
+         * @param vals Values.
+         * @param sign Sign: 1 or -1.
+         * @throws Exception If failed.
+         */
+        public void write(SwapValues vals, int sign) throws Exception {
+            ByteBuffer buf = ByteBuffer.allocateDirect(vals.size);
+
+            for (int i = 0, len = vals.vals.length; i < len; i++) {
+                SwapValue val = vals.vals[i];
+
+                if (val.idx == SwapValue.DELETED) {
+                    vals.vals[i] = null;
+
+                    continue;
+                }
+
+                int idx = arr.add(val);
+
+                if (!val.casIdx(SwapValue.NEW, sign * idx)) {
+                    assert val.idx == SwapValue.DELETED;
+
+                    tryRemove(idx, val);
+
+                    vals.vals[i] = null;
+                }
+                else
+                    buf.put(val.value(null));
+            }
+
+            buf.flip();
+
+            final int size = buf.remaining();
+
+            if (size == 0)
+                return;
+
+            long pos = len;
+
+            len = pos + size;
+
+            long res = writeCh.write(buf, pos);
+
+            if (res != size)
+                throw new IllegalStateException(res + " != " + size);
+
+            // Nullify bytes in values ans set pos.
+            for (SwapValue val : vals.vals) {
+                if (val == null)
+                    continue;
+
+                val.set(pos, null);
+
+                pos += val.len;
+            }
+        }
+
+        /**
+         * Gets file path.
+         *
+         * @return File path.
+         */
+        public String path() {
+            return file.getAbsolutePath();
+        }
+
+        /**
+         * Gets file length.
+         *
+         * @return File length.
+         */
+        public long length() {
+            return len;
+        }
+
+        /**
+         * Deletes file.
+         *
+         * @return Whether file was actually deleted.
+         */
+        public boolean delete() {
+            U.closeQuiet(raf);
+
+            readCh.close();
+
+            return U.delete(file);
+        }
+
+        /**
+         * @param idx Index.
+         * @param exp Expected value.
+         * @return {@code true} If succeeded.
+         */
+        public boolean tryRemove(int idx, SwapValue exp) {
+            assert idx > 0 : idx;
+
+            FileSwapArray.Slot<SwapValue> s = arr.slot(idx);
+
+            return s != null && s.cas(exp, null);
+        }
+
+        /**
+         * Does compaction for one buffer.
+         *
+         * @param vals Values.
+         * @param bufSize Buffer size.
+         * @return Buffer.
+         * @throws IOException If failed.
+         * @throws InterruptedException If interrupted.
+         */
+        public ByteBuffer compact(ArrayDeque<SwapValue> vals, final int bufSize) throws IOException,
+            InterruptedException {
+            assert vals.isEmpty();
+
+            Compact c = new Compact(vals, bufSize);
+
+            c.doCompact();
+
+            return c.result();
+        }
+
+        /**
+         * Single compaction operation.
+         */
+        private class Compact {
+            /** */
+            private final ArrayDeque<SwapValue> vals;
+
+            /** */
+            private final int bufSize;
+
+            /** */
+            private byte[] bytes;
+
+            /** */
+            private ByteBuffer buf;
+
+            /** */
+            private long beg = -1;
+
+            /** */
+            private long end = -1;
+
+            /** */
+            private int compacted;
+
+            /**
+             * @param vals Values.
+             * @param bufSize Buffer size.
+             */
+            private Compact(ArrayDeque<SwapValue> vals, final int bufSize) {
+                assert vals.isEmpty();
+
+                this.vals = vals;
+                this.bufSize = bufSize;
+            }
+
+            /**
+             * Reads buffer and compacts it.
+             *
+             * @throws IOException if failed.
+             */
+            private void readAndCompact() throws IOException {
+                assert beg != -1;
+
+                if (buf == null) {
+                    bytes = new byte[bufSize];
+
+                    buf = ByteBuffer.wrap(bytes);
+                }
+
+                final int pos = buf.position();
+
+                final int lim = (int)(end - beg + pos);
+
+                assert pos >= 0;
+                assert pos < lim : pos + " " + lim;
+                assert lim <= buf.capacity();
+
+                buf.limit(lim);
+
+                int res = writeCh.read(buf, beg);
+
+                assert res == lim - pos;
+
+                int prevEnd = pos;
+                long delta = beg - pos; // To translate from file based positions to buffer based.
+
+                for (int j = vals.size(); j > compacted; j--) {
+                    SwapValue val = vals.pollFirst();
+
+                    int valPos = (int)(val.pos - delta);
+
+                    if (prevEnd != valPos) {
+                        assert prevEnd < valPos : prevEnd + " " + valPos;
+
+                        U.arrayCopy(bytes, valPos, bytes, prevEnd, val.len);
+                    }
+
+                    prevEnd += val.len;
+
+                    vals.addLast(val); // To have values in the same order as in byte buffer.
+                }
+
+                assert prevEnd > 0 : prevEnd;
+
+                buf.position(prevEnd);
+
+                end = -1;
+
+                compacted = vals.size();
+            }
+
+            /**
+             * Compacts.
+             *
+             * @throws IOException If failed.
+             */
+            private void doCompact() throws IOException {
+                int idx = arr.size();
+
+                while (--idx > 0) {
+                    FileSwapArray.Slot<SwapValue> s = arr.slot(idx);
+
+                    assert s != null;
+
+                    SwapValue v = s.get();
+
+                    if (v == null || v.idx == SwapValue.DELETED)
+                        continue;
+
+                    if (end == -1)
+                        end = v.pos + v.len;
+
+                    long size = end - v.pos;
+
+                    if ((buf == null ? bufSize : buf.remaining()) < size) {
+                        if (vals.isEmpty()) {  // Too big single value.
+                            assert bytes == null && buf == null;
+
+                            bytes = new byte[(int)size];
+
+                            buf = ByteBuffer.wrap(bytes);
+                        }
+                        else if (compacted == vals.size())
+                            break; // Finish current compaction, nothing new collected.
+                        else { // Read region and compact values in buffer.
+                            readAndCompact();
+
+                            // Retry the same value.
+                            idx++;
+
+                            continue;
+                        }
+                    }
+
+                    beg = v.pos;
+
+                    vals.addFirst(v);
+
+                    s.cas(v, null);
+                }
+
+                if (vals.isEmpty()) {
+                    arr.truncate(1);
+
+                    writeCh.truncate(0);
+
+                    len = 0;
+
+                    reopenReadChannel(); // Make sure that value can be read only from right file but not after switch.
+
+                    return;
+                }
+
+                if (compacted != vals.size())
+                    readAndCompact();
+
+                int pos = 0;
+
+                for (SwapValue val : vals) { // The values will share one byte array with different offsets while moving.
+                    val.set(pos, bytes);
+
+                    pos += val.len;
+                }
+
+                buf.flip();
+
+                assert buf.limit() == pos : buf.limit() + " " + pos;
+
+                arr.truncate(idx + 1);
+
+                if (len - beg > MIN_TRUNK_SIZE) {
+                    writeCh.truncate(beg);
+
+                    len = beg;
+                }
+            }
+
+            /**
+             * @return Buffer.
+             */
+            public ByteBuffer result() {
+                return buf;
+            }
+        }
+    }
+
+    /**
+     * Space.
+     */
+    private class Space {
+        /** Space name. */
+        private final String name;
+
+        /** */
+        private final GridAtomicInitializer<Void> initializer = new GridAtomicInitializer<>();
+
+        /** Swap file left. */
+        @SuppressWarnings("FieldAccessedSynchronizedAndUnsynchronized")
+        private SwapFile left;
+
+        /** Swap file right. */
+        @SuppressWarnings("FieldAccessedSynchronizedAndUnsynchronized")
+        private SwapFile right;
+
+        /** */
+        private final SwapValuesQueue que = new SwapValuesQueue(writeBufSize, maxWriteQueSize);
+
+        /** Partitions. */
+        private final ConcurrentMap<Integer, ConcurrentMap<SwapKey, SwapValue>> parts =
+            new ConcurrentHashMap8<>();
+
+        /** Total size. */
+        private final AtomicLong size = new AtomicLong();
+
+        /** Total count. */
+        private final AtomicLong cnt = new AtomicLong();
+
+        /** */
+        private int sign = 1;
+
+        /** Writer thread. */
+        private Thread writer;
+
+        /** */
+        private Thread compactor;
+
+        /**
+         * @param name Space name.
+         */
+        private Space(String name) {
+            assert name != null;
+
+            this.name = name;
+        }
+
+        /**
+         * Initializes space.
+         *
+         * @throws org.apache.ignite.spi.IgniteSpiException If initialization failed.
+         */
+        public void initialize() throws IgniteSpiException {
+            if (initializer.succeeded())
+                return;
+
+            assert dir.exists();
+            assert dir.isDirectory();
+
+            try {
+                initializer.init(new Callable<Void>(){
+                    @Override public Void call() throws Exception {
+                        left = new SwapFile(new File(dir, name + ".left"), readStripesNum);
+
+                        right = new SwapFile(new File(dir, name + ".right"), readStripesNum);
+
+                        final Object mux = new Object();
+
+                        writer = new IgniteSpiThread(gridName,  "Swap writer: " + name, log) {
+                            @Override protected void body() throws InterruptedException {
+                                while (!isInterrupted()) {
+                                    SwapValues vals = que.take();
+
+                                    synchronized (mux) {
+                                        SwapFile f = sign == 1 ? right : left;
+
+                                        try {
+                                            f.write(vals, sign);
+                                        }
+                                        catch (Exception e) {
+                                            throw new GridRuntimeException(e);
+                                        }
+                                    }
+                                }
+                            }
+                        };
+
+                        compactor = new IgniteSpiThread(gridName, "Swap compactor: " + name, log) {
+                            @Override protected void body() throws InterruptedException {
+                                SwapFile w = null;
+                                SwapFile c = null;
+
+                                ArrayDeque<SwapValue> vals = null;
+
+                                while (!isInterrupted()) {
+                                    while(!needCompact()) {
+                                        LockSupport.park();
+
+                                        if (isInterrupted())
+                                            return;
+                                    }
+
+                                    ByteBuffer buf = null;
+
+                                    if (vals == null)
+                                        vals = new ArrayDeque<>();
+                                    else {
+                                        vals.clear();
+
+                                        try {
+                                            buf = c.compact(vals, writeBufSize);
+                                        }
+                                        catch (IOException e) {
+                                            throw new GridRuntimeException(e);
+                                        }
+                                    }
+
+                                    if (vals.isEmpty()) {
+                                        synchronized (mux) {
+                                            sign = -sign;
+
+                                            if (sign == 1) {
+                                                w = right;
+                                                c = left;
+                                            }
+                                            else {
+                                                w = left;
+                                                c = right;
+                                            }
+                                        }
+                                    }
+                                    else {
+                                        assert buf != null && buf.remaining() != 0;
+
+                                        synchronized (mux) {
+                                            try {
+                                                w.write(vals, buf, sign);
+                                            }
+                                            catch (Exception e) {
+                                                throw new GridRuntimeException(e);
+                                            }
+                                        }
+                                    }
+                                }
+                            }
+                        };
+
+                        writer.start();
+                        compactor.start();
+
+                        return null;
+                    }
+                });
+            }
+            catch (GridException e) {
+                throw new IgniteSpiException(e);
+            }
+        }
+
+        /**
+         * Gets total space size in bytes.
+         *
+         * @return Total size.
+         */
+        public long size() {
+            return left.length() + right.length();
+        }
+
+        /**
+         * Gets total space count.
+         *
+         * @return Total count.
+         */
+        public long count() {
+            return cnt.get();
+        }
+
+        /**
+         * Clears space.
+         *
+         * @throws org.apache.ignite.spi.IgniteSpiException If failed.
+         */
+        public void clear() throws IgniteSpiException {
+            Iterator<Map.Entry<SwapKey, byte[]>> iter = entriesIterator();
+
+            while (iter.hasNext())
+                remove(iter.next().getKey(), false);
+        }
+
+        /**
+         * Stops space.
+         *
+         * @throws GridInterruptedException If interrupted.
+         */
+        public void stop() throws GridInterruptedException {
+            U.interrupt(writer);
+            U.interrupt(compactor);
+
+            U.join(writer);
+            U.join(compactor);
+
+            left.delete();
+            right.delete();
+        }
+
+        /**
+         * Stores value in space.
+         *
+         * @param key Key.
+         * @param val Value.
+         * @throws org.apache.ignite.spi.IgniteSpiException In case of error.
+         */
+        public void store(final SwapKey key, @Nullable final byte[] val) throws IgniteSpiException {
+            assert key != null;
+
+            final ConcurrentMap<SwapKey, SwapValue> part = partition(key.partition(), true);
+
+            assert part != null;
+
+            if (val == null) {
+                SwapValue swapVal = part.remove(key);
+
+                if (swapVal != null) {
+                    removeFromFile(swapVal);
+
+                    size.addAndGet(-swapVal.len);
+                    cnt.decrementAndGet();
+                }
+
+                return;
+            }
+
+            final SwapValue swapVal = new SwapValue(val);
+
+            SwapValue old = part.put(key, swapVal);
+
+            if (old != null) {
+                size.addAndGet(val.length - old.len);
+
+                removeFromFile(old);
+            }
+            else {
+                size.addAndGet(val.length);
+                cnt.incrementAndGet();
+            }
+
+            que.add(swapVal);
+        }
+
+        /**
+         * Reads value from space.
+         *
+         * @param key Key.
+         * @return Value.
+         * @throws org.apache.ignite.spi.IgniteSpiException In case of error.
+         */
+        @Nullable public byte[] read(SwapKey key) throws IgniteSpiException {
+            assert key != null;
+
+            final Map<SwapKey, SwapValue> part = partition(key.partition(), false);
+
+            if (part == null)
+                return null;
+
+            SwapValue swapVal = part.get(key);
+
+            if (swapVal == null)
+                return null;
+
+            return swapVal.value(this);
+        }
+
+        /**
+         * Removes value from space.
+         *
+         * @param key Key.
+         * @param read If value has to be read.
+         * @return Value.
+         * @throws org.apache.ignite.spi.IgniteSpiException In case of error.
+         */
+        @Nullable public byte[] remove(SwapKey key, boolean read) throws IgniteSpiException {
+            assert key != null;
+
+            final Map<SwapKey, SwapValue> part = partition(key.partition(), false);
+
+            if (part == null)
+                return null;
+
+            SwapValue val = part.remove(key);
+
+            if (val == null)
+                return null;
+
+            size.addAndGet(-val.len);
+
+            cnt.decrementAndGet();
+
+            byte[] bytes = null;
+
+            if (read) {
+                bytes = val.value(this);
+
+                assert bytes != null; // Value bytes were read before removal from file, so compaction can't happen.
+            }
+
+            removeFromFile(val);
+
+            return bytes;
+        }
+
+        /**
+         * @param val Value.
+         */
+        private void removeFromFile(SwapValue val) {
+            for (;;) {
+                int idx = val.idx;
+
+                assert idx != SwapValue.DELETED;
+
+                if (val.casIdx(idx, SwapValue.DELETED)) {
+                    if (idx != SwapValue.NEW) {
+                        SwapFile f = idx > 0 ? right : left;
+
+                        f.tryRemove(Math.abs(idx), val);
+                    }
+
+                    break;
+                }
+            }
+
+            if (needCompact())
+                LockSupport.unpark(compactor);
+        }
+
+        /**
+         * @return {@code true} If compaction needed.
+         */
+        private boolean needCompact() {
+            long fileLen = size();
+
+            return fileLen > writeBufSize && (fileLen - size.get()) / (float)fileLen > maxSparsity;
+        }
+
+        /**
+         * Gets numbers of partitioned stored in this space.
+         *
+         * @return Partition numbers.
+         */
+        public Collection<Integer> partitions() {
+            return parts.keySet();
+        }
+
+        /**
+         * Gets partition map by its number.
+         *
+         * @param part Partition number.
+         * @param create Whether to create partition if it doesn't exist.
+         * @return Partition map.
+         */
+        @Nullable private ConcurrentMap<SwapKey, SwapValue> partition(int part, boolean create) {
+            ConcurrentMap<SwapKey, SwapValue> map = parts.get(part);
+
+            if (map == null && create) {
+                ConcurrentMap<SwapKey, SwapValue> old = parts.putIfAbsent(part,
+                    map = new ConcurrentHashMap<>());
+
+                if (old != null)
+                    map = old;
+            }
+
+            return map;
+        }
+
+        /**
+         * @param part Partition.
+         * @return Iterator over partition.
+         */
+        public Iterator<Map.Entry<SwapKey, byte[]>> entriesIterator(int part) {
+            Map<SwapKey, SwapValue> partMap = partition(part, false);
+
+            if (partMap == null)
+                return Collections.<Map.Entry<SwapKey, byte[]>>emptySet().iterator();
+
+            return transform(partMap.entrySet().iterator());
+        }
+
+        /**
+         * @return Iterator over all entries.
+         */
+        public Iterator<Map.Entry<SwapKey, byte[]>> entriesIterator() {
+            final Iterator<ConcurrentMap<SwapKey, SwapValue>> iter = parts.values().iterator();
+
+            return transform(F.concat(new Iterator<Iterator<Map.Entry<SwapKey, SwapValue>>>() {
+                @Override public boolean hasNext() {
+                    return iter.hasNext();
+                }
+
+                @Override public Iterator<Map.Entry<SwapKey, SwapValue>> next() {
+                    return iter.next().entrySet().iterator();
+                }
+
+                @Override public void remove() {
+                    throw new UnsupportedOperationException();
+                }
+            }));
+        }
+
+        /**
+         * Gets iterator for all entries in space.
+         *
+         * @param iter Iterator with {@link SwapValue} to transform.
+         * @return Entries iterator.
+         */
+        private Iterator<Map.Entry<SwapKey, byte[]>> transform(final Iterator<Map.Entry<SwapKey,
+            SwapValue>> iter) {
+            return new Iterator<Map.Entry<SwapKey, byte[]>>() {
+                /** */
+                private Map.Entry<SwapKey, byte[]> next;
+
+                /** */
+                private Map.Entry<SwapKey, byte[]> last;
+
+                {
+                    advance();
+                }
+
+                @Override public boolean hasNext() {
+                    return next != null;
+                }
+
+                /**
+                 * Gets next entry.
+                 */
+                private void advance() {
+                    while (iter.hasNext()) {
+                        Map.Entry<SwapKey, SwapValue> entry = iter.next();
+
+                        byte[] bytes;
+
+                        try {
+                            bytes = entry.getValue().value(Space.this);
+                        }
+                        catch (IgniteSpiException e) {
+                            throw new GridRuntimeException(e);
+                        }
+
+                        if (bytes != null) {
+                            next = new T2<>(entry.getKey(), bytes);
+
+                            break;
+                        }
+                    }
+                }
+
+                @Override public Map.Entry<SwapKey, byte[]> next() {
+                    final Map.Entry<SwapKey, byte[]> res = next;
+
+                    if (res == null)
+                        throw new NoSuchElementException();
+
+                    next = null;
+
+                    advance();
+
+                    last = res;
+
+                    return res;
+                }
+
+                @Override public void remove() {
+                    if (last == null)
+                        throw new IllegalStateException();
+
+                    try {
+                        Space.this.remove(last.getKey(), false);
+                    }
+                    catch (IgniteSpiException e) {
+                        throw new GridRuntimeException(e);
+                    }
+                    finally {
+                        last = null;
+                    }
+                }
+            };
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/ef258ece/modules/core/src/main/java/org/apache/ignite/spi/swapspace/file/FileSwapSpaceSpiMBean.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/swapspace/file/FileSwapSpaceSpiMBean.java b/modules/core/src/main/java/org/apache/ignite/spi/swapspace/file/FileSwapSpaceSpiMBean.java
new file mode 100644
index 0000000..8d0483c
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/spi/swapspace/file/FileSwapSpaceSpiMBean.java
@@ -0,0 +1,59 @@
+/* @java.file.header */
+
+/*  _________        _____ __________________        _____
+ *  __  ____/___________(_)______  /__  ____/______ ____(_)_______
+ *  _  / __  __  ___/__  / _  __  / _  / __  _  __ `/__  / __  __ \
+ *  / /_/ /  _  /    _  /  / /_/ /  / /_/ /  / /_/ / _  /  _  / / /
+ *  \____/   /_/     /_/   \_,__/   \____/   \__,_/  /_/   /_/ /_/
+ */
+
+package org.apache.ignite.spi.swapspace.file;
+
+import org.apache.ignite.mbean.*;
+import org.apache.ignite.spi.*;
+
+/**
+ * Management bean for {@link FileSwapSpaceSpi}.
+ */
+@IgniteMBeanDescription("MBean that provides configuration information on file-based swapspace SPI.")
+public interface FileSwapSpaceSpiMBean extends IgniteSpiManagementMBean {
+    /**
+     * Gets base directory.
+     *
+     * @return Base directory.
+     */
+    @IgniteMBeanDescription("Base directory.")
+    public String getBaseDirectory();
+
+    /**
+     * Gets maximum sparsity.
+     *
+     * @return Maximum sparsity.
+     */
+    @IgniteMBeanDescription("Maximum sparsity.")
+    public float getMaximumSparsity();
+
+    /**
+     * Gets write buffer size in bytes.
+     *
+     * @return Write buffer size in bytes.
+     */
+    @IgniteMBeanDescription("Write buffer size in bytes.")
+    public int getWriteBufferSize();
+
+    /**
+     * Gets max write queue size in bytes.
+     *
+     * @return Max write queue size in bytes.
+     */
+    @IgniteMBeanDescription("Max write queue size in bytes.")
+    public int getMaxWriteQueueSize();
+
+    /**
+     * Gets read pool size.
+     *
+     * @return Read pool size.
+     */
+    @IgniteMBeanDescription("Read pool size.")
+    public int getReadStripesNumber();
+}


[11/20] incubator-ignite git commit: # Renaming

Posted by sb...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c31cec7c/modules/core/src/main/java/org/gridgain/grid/spi/swapspace/file/GridFileSwapSpaceSpi.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/spi/swapspace/file/GridFileSwapSpaceSpi.java b/modules/core/src/main/java/org/gridgain/grid/spi/swapspace/file/GridFileSwapSpaceSpi.java
deleted file mode 100644
index 65209bb..0000000
--- a/modules/core/src/main/java/org/gridgain/grid/spi/swapspace/file/GridFileSwapSpaceSpi.java
+++ /dev/null
@@ -1,1843 +0,0 @@
-/* @java.file.header */
-
-/*  _________        _____ __________________        _____
- *  __  ____/___________(_)______  /__  ____/______ ____(_)_______
- *  _  / __  __  ___/__  / _  __  / _  / __  _  __ `/__  / __  __ \
- *  / /_/ /  _  /    _  /  / /_/ /  / /_/ /  / /_/ / _  /  _  / / /
- *  \____/   /_/     /_/   \_,__/   \____/   \__,_/  /_/   /_/ /_/
- */
-
-package org.gridgain.grid.spi.swapspace.file;
-
-import org.apache.ignite.*;
-import org.apache.ignite.lang.*;
-import org.apache.ignite.marshaller.*;
-import org.apache.ignite.resources.*;
-import org.apache.ignite.spi.*;
-import org.gridgain.grid.*;
-import org.gridgain.grid.spi.swapspace.*;
-import org.gridgain.grid.util.*;
-import org.gridgain.grid.util.typedef.*;
-import org.gridgain.grid.util.typedef.internal.*;
-import org.jdk8.backport.*;
-import org.jetbrains.annotations.*;
-
-import java.io.*;
-import java.nio.*;
-import java.nio.channels.*;
-import java.util.*;
-import java.util.concurrent.*;
-import java.util.concurrent.atomic.*;
-import java.util.concurrent.locks.*;
-
-import static org.apache.ignite.events.IgniteEventType.*;
-
-/**
- * File-based swap space SPI implementation which holds keys in memory. This SPI is used by default.
- * It is intended for use in cases when value size is bigger than {@code 100} bytes, otherwise it will not
- * have any positive effect.
- * <p>
- * <b>NOTE: This SPI does not support swap eviction currently, manual removes needed to reduce disk space
- * consumption.</b>
- * <p>
- * Every space has a name and when used in combination with in-memory data grid name and local node ID,
- * space name represents the actual cache name associated with this swap space. Default name is {@code null}
- * which is represented by {@link #DFLT_SPACE_NAME}.
- *
- * <h1 class="header">Configuration</h1>
- * <h2 class="header">Mandatory</h2>
- * This SPI has no mandatory configuration parameters.
- * <h2 class="header">Optional SPI configuration.</h2>
- * <ul>
- *     <li>Base directory path (see {@link #setBaseDirectory(String)}).</li>
- *     <li>Maximum sparsity (see {@link #setMaximumSparsity(float)}).</li>
- *     <li>Write buffer size in bytes (see {@link #setWriteBufferSize(int)}).</li>
- *     <li>Max write queue size in bytes (see {@link #setMaxWriteQueueSize(int)}).</li>
- *     <li>Read stripes number. (see {@link #setReadStripesNumber(int)}).</li>
- * </ul>
- *
- * <h2 class="header">Java Example</h2>
- * GridFileSwapSpaceSpi is configured by default and should be explicitly configured
- * only if some SPI configuration parameters need to be overridden.
- * <pre name="code" class="java">
- * GridFileSwapSpaceSpi spi = new GridFileSwapSpaceSpi();
- *
- * // Configure root folder path.
- * spi.setBaseDirectory("/path/to/swap/folder");
- *
- * GridConfiguration cfg = new GridConfiguration();
- *
- * // Override default swap space SPI.
- * cfg.setSwapSpaceSpi(spi);
- *
- * // Starts grid.
- * G.start(cfg);
- * </pre>
- * <h2 class="header">Spring Example</h2>
- * GridFileSwapSpaceSpi can be configured from Spring XML configuration file:
- * <pre name="code" class="xml">
- * &lt;bean id=&quot;grid.cfg&quot; class=&quot;org.gridgain.grid.GridConfiguration&quot; scope=&quot;singleton&quot;&gt;
- *     ...
- *     &lt;property name=&quot;swapSpaceSpi&quot;&gt;
- *         &lt;bean class=&quot;org.gridgain.grid.spi.swapspace.file.GridFileSwapSpaceSpi&quot;&gt;
- *             &lt;property name=&quot;baseDirectory&quot; value=&quot;/path/to/swap/folder&quot;/&gt;
- *         &lt;/bean&gt;
- *     &lt;/property&gt;
- *     ...
- * &lt;/bean&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 GridSwapSpaceSpi
- */
-@IgniteSpiMultipleInstancesSupport(true)
-@SuppressWarnings({"PackageVisibleInnerClass", "PackageVisibleField"})
-public class GridFileSwapSpaceSpi extends IgniteSpiAdapter implements GridSwapSpaceSpi, GridFileSwapSpaceSpiMBean {
-    /**
-     * Default base directory. Note that this path is relative to {@code GRIDGAIN_HOME/work} folder
-     * if {@code GRIDGAIN_HOME} system or environment variable specified, otherwise it is relative to
-     * {@code work} folder under system {@code java.io.tmpdir} folder.
-     *
-     * @see org.apache.ignite.configuration.IgniteConfiguration#getWorkDirectory()
-     */
-    public static final String DFLT_BASE_DIR = "swapspace";
-
-    /** Default maximum sparsity. */
-    public static final float DFLT_MAX_SPARSITY = 0.5f;
-
-    /** Default write buffer size in bytes. */
-    public static final int DFLT_BUF_SIZE = 64 * 1024;
-
-    /** Default write queue size in bytes. */
-    public static final int DFLT_QUE_SIZE = 1024 * 1024;
-
-    /** Name for {@code null} space. */
-    public static final String DFLT_SPACE_NAME = "gg-dflt-space";
-
-    /** Spaces. */
-    private final ConcurrentMap<String, Space> spaces = new ConcurrentHashMap<>();
-
-    /** Base directory. */
-    private String baseDir = DFLT_BASE_DIR;
-
-    /** Maximum sparsity. */
-    private float maxSparsity = DFLT_MAX_SPARSITY;
-
-    /** Eviction listener. */
-    private volatile GridSwapSpaceSpiListener evictLsnr;
-
-    /** Directory. */
-    private File dir;
-
-    /** Write buffer size. */
-    private int writeBufSize = DFLT_BUF_SIZE;
-
-    /** Max write queue size in bytes. */
-    private int maxWriteQueSize = DFLT_QUE_SIZE;
-
-    /** Read stripes number. */
-    private int readStripesNum = -1;
-
-    /** Logger. */
-    @IgniteLoggerResource
-    private IgniteLogger log;
-
-    /** Local node ID. */
-    @IgniteLocalNodeIdResource
-    private UUID locNodeId;
-
-    /** Name of the grid. */
-    @IgniteNameResource
-    private String gridName;
-
-    /** Marshaller. */
-    @IgniteMarshallerResource
-    private IgniteMarshaller marsh;
-
-    /** {@inheritDoc} */
-    @Override public String getBaseDirectory() {
-        return baseDir;
-    }
-
-    /**
-     * Sets base directory.
-     *
-     * @param baseDir Base directory.
-     */
-    @IgniteSpiConfiguration(optional = true)
-    public void setBaseDirectory(String baseDir) {
-        this.baseDir = baseDir;
-    }
-
-    /** {@inheritDoc} */
-    @Override public float getMaximumSparsity() {
-        return maxSparsity;
-    }
-
-    /**
-     * Sets maximum sparsity. This property defines maximum acceptable wasted file space to whole file size ratio.
-     * When this ratio becomes higher than specified number compacting thread starts working.
-     *
-     * @param maxSparsity Maximum sparsity. Must be between 0 and 1, default is {@link #DFLT_MAX_SPARSITY}.
-     */
-    public void setMaximumSparsity(float maxSparsity) {
-        this.maxSparsity = maxSparsity;
-    }
-
-    /** {@inheritDoc} */
-    @Override public int getWriteBufferSize() {
-        return writeBufSize;
-    }
-
-    /**
-     * Sets write buffer size in bytes. Write to disk occurs only when this buffer is full. Default is
-     * {@link #DFLT_BUF_SIZE}.
-     *
-     * @param writeBufSize Write buffer size in bytes.
-     */
-    public void setWriteBufferSize(int writeBufSize) {
-        this.writeBufSize = writeBufSize;
-    }
-
-    /** {@inheritDoc} */
-    @Override public int getMaxWriteQueueSize() {
-        return maxWriteQueSize;
-    }
-
-    /**
-     * Sets max write queue size in bytes. If there are more values are waiting for being written to disk then specified
-     * size, SPI will block on {@link #store(String, GridSwapKey, byte[], GridSwapContext)} operation. Default is
-     * {@link #DFLT_QUE_SIZE}.
-     *
-     * @param maxWriteQueSize Max write queue size in bytes.
-     */
-    public void setMaxWriteQueueSize(int maxWriteQueSize) {
-        this.maxWriteQueSize = maxWriteQueSize;
-    }
-
-    /** {@inheritDoc} */
-    @Override public int getReadStripesNumber() {
-        return readStripesNum;
-    }
-
-    /**
-     * Sets read stripe size. Defines number of file channels to be used concurrently. Default is equal to number of
-     * CPU cores available to this JVM.
-     *
-     * @param readStripesNum Read stripe number.
-     */
-    public void setReadStripesNumber(int readStripesNum) {
-        A.ensure(readStripesNum == -1 || (readStripesNum & (readStripesNum - 1)) == 0,
-            "readStripesNum must be positive and power of two");
-
-        this.readStripesNum = readStripesNum;
-    }
-
-    /** {@inheritDoc} */
-    @Override public void spiStart(@Nullable String gridName) throws IgniteSpiException {
-        assertParameter(!F.isEmpty(baseDir), "!F.isEmpty(baseDir)");
-        assertParameter(maxSparsity >= 0 && maxSparsity < 1, "maxSparsity >= 0 && maxSparsity < 1");
-        assertParameter(readStripesNum == -1 || (readStripesNum & (readStripesNum - 1)) == 0,
-            "readStripesNum must be positive and power of two.");
-
-        if (readStripesNum == -1) {
-            // User has not configured the number.
-            int readStripesNum0 = 1;
-            int cpuCnt = Runtime.getRuntime().availableProcessors();
-
-            while (readStripesNum0 <= cpuCnt)
-                readStripesNum0 <<= 1;
-
-            if (readStripesNum0 > cpuCnt)
-                readStripesNum0 >>= 1;
-
-            assert readStripesNum0 > 0 && (readStripesNum0 & readStripesNum0 - 1) == 0;
-
-            readStripesNum = readStripesNum0;
-        }
-
-        startStopwatch();
-
-        registerMBean(gridName, this, GridFileSwapSpaceSpiMBean.class);
-
-        String path = baseDir + File.separator + gridName + File.separator + locNodeId;
-
-        try {
-            dir = U.resolveWorkDirectory(path, true);
-        }
-        catch (GridException e) {
-            throw new IgniteSpiException(e);
-        }
-
-        if (log.isDebugEnabled())
-            log.debug(startInfo());
-    }
-
-    /** {@inheritDoc} */
-    @Override public void spiStop() throws IgniteSpiException {
-        unregisterMBean();
-
-        for (Space space : spaces.values()) {
-            space.initialize();
-
-            try {
-                space.stop();
-            }
-            catch (GridInterruptedException e) {
-                U.error(log, "Interrupted.", e);
-            }
-        }
-
-        if (dir != null && dir.exists() && !U.delete(dir))
-            U.warn(log, "Failed to delete swap directory: " + dir.getAbsolutePath());
-
-        if (log.isDebugEnabled())
-            log.debug(stopInfo());
-    }
-
-    /** {@inheritDoc} */
-    @Override public void clear(@Nullable String spaceName) throws IgniteSpiException {
-        Space space = space(spaceName, false);
-
-        if (space == null)
-            return;
-
-        space.clear();
-
-        notifyListener(EVT_SWAP_SPACE_CLEARED, spaceName);
-    }
-
-    /** {@inheritDoc} */
-    @Override public long size(@Nullable String spaceName) throws IgniteSpiException {
-        Space space = space(spaceName, false);
-
-        if (space == null)
-            return 0;
-
-        return space.size();
-    }
-
-    /** {@inheritDoc} */
-    @Override public long count(@Nullable String spaceName) throws IgniteSpiException {
-        Space space = space(spaceName, false);
-
-        if (space == null)
-            return 0;
-
-        return space.count();
-    }
-
-    /** {@inheritDoc} */
-    @Nullable @Override public byte[] read(@Nullable String spaceName, GridSwapKey key, GridSwapContext ctx)
-        throws IgniteSpiException {
-        assert key != null;
-        assert ctx != null;
-
-        Space space = space(spaceName, false);
-
-        if (space == null)
-            return null;
-
-        byte[] val = space.read(key);
-
-        notifyListener(EVT_SWAP_SPACE_DATA_READ, spaceName);
-
-        return val;
-    }
-
-    /** {@inheritDoc} */
-    @Override public Map<GridSwapKey, byte[]> readAll(@Nullable String spaceName, Iterable<GridSwapKey> keys,
-        GridSwapContext ctx) throws IgniteSpiException {
-        assert keys != null;
-        assert ctx != null;
-
-        Space space = space(spaceName, false);
-
-        if (space == null)
-            return Collections.emptyMap();
-
-        Map<GridSwapKey, byte[]> res = new HashMap<>();
-
-        for (GridSwapKey key : keys) {
-            if (key != null) {
-                byte[] val = space.read(key);
-
-                if (val != null)
-                    res.put(key, val);
-
-                notifyListener(EVT_SWAP_SPACE_DATA_READ, spaceName);
-            }
-        }
-
-        return res;
-    }
-
-    /** {@inheritDoc} */
-    @Override public void remove(@Nullable String spaceName, GridSwapKey key, @Nullable IgniteInClosure<byte[]> c,
-        GridSwapContext ctx) throws IgniteSpiException {
-        assert key != null;
-        assert ctx != null;
-
-        Space space = space(spaceName, false);
-
-        if (space == null)
-            return;
-
-        byte[] val = space.remove(key, c != null);
-
-        if (c != null)
-            c.apply(val);
-
-        notifyListener(EVT_SWAP_SPACE_DATA_REMOVED, spaceName);
-    }
-
-    /** {@inheritDoc} */
-    @Override public void removeAll(@Nullable String spaceName, Collection<GridSwapKey> keys,
-        @Nullable IgniteBiInClosure<GridSwapKey, byte[]> c, GridSwapContext ctx) throws IgniteSpiException {
-        assert keys != null;
-        assert ctx != null;
-
-        Space space = space(spaceName, false);
-
-        if (space == null)
-            return;
-
-        for (GridSwapKey key : keys) {
-            if (key != null) {
-                byte[] val = space.remove(key, c != null);
-
-                if (c != null)
-                    c.apply(key, val);
-
-                notifyListener(EVT_SWAP_SPACE_DATA_REMOVED, spaceName);
-            }
-        }
-    }
-
-    /** {@inheritDoc} */
-    @Override public void store(@Nullable String spaceName, GridSwapKey key, @Nullable byte[] val,
-        GridSwapContext ctx) throws IgniteSpiException {
-        assert key != null;
-        assert ctx != null;
-
-        Space space = space(spaceName, true);
-
-        assert space != null;
-
-        space.store(key, val);
-
-        notifyListener(EVT_SWAP_SPACE_DATA_STORED, spaceName);
-    }
-
-    /** {@inheritDoc} */
-    @Override public void storeAll(@Nullable String spaceName, Map<GridSwapKey, byte[]> pairs,
-        GridSwapContext ctx) throws IgniteSpiException {
-        assert pairs != null;
-        assert ctx != null;
-
-        Space space = space(spaceName, true);
-
-        assert space != null;
-
-        for (Map.Entry<GridSwapKey, byte[]> pair : pairs.entrySet()) {
-            GridSwapKey key = pair.getKey();
-
-            if (key != null) {
-                space.store(key, pair.getValue());
-
-                notifyListener(EVT_SWAP_SPACE_DATA_STORED, spaceName);
-            }
-        }
-    }
-
-    /** {@inheritDoc} */
-    @Override public void setListener(@Nullable GridSwapSpaceSpiListener evictLsnr) {
-        this.evictLsnr = evictLsnr;
-    }
-
-    /** {@inheritDoc} */
-    @Nullable @Override public Collection<Integer> partitions(@Nullable String spaceName)
-        throws IgniteSpiException {
-        Space space = space(spaceName, false);
-
-        if (space == null)
-            return null;
-
-        return space.partitions();
-    }
-
-    /** {@inheritDoc} */
-    @Nullable @Override public <K> IgniteSpiCloseableIterator<K> keyIterator(@Nullable String spaceName,
-        GridSwapContext ctx) throws IgniteSpiException {
-        final Space space = space(spaceName, false);
-
-        if (space == null)
-            return null;
-
-        final Iterator<Map.Entry<GridSwapKey, byte[]>> iter = space.entriesIterator();
-
-        return new GridCloseableIteratorAdapter<K>() {
-            @Override protected boolean onHasNext() {
-                return iter.hasNext();
-            }
-
-            @Override protected K onNext() {
-                return (K)iter.next().getKey().key();
-            }
-
-            @Override protected void onRemove() {
-                iter.remove();
-            }
-        };
-    }
-
-    /** {@inheritDoc} */
-    @Nullable @Override public IgniteSpiCloseableIterator<Map.Entry<byte[], byte[]>> rawIterator(
-        @Nullable String spaceName) throws IgniteSpiException {
-        Space space = space(spaceName, false);
-
-        if (space == null)
-            return null;
-
-        return rawIterator(space.entriesIterator());
-    }
-
-    /** {@inheritDoc} */
-    @Nullable @Override public IgniteSpiCloseableIterator<Map.Entry<byte[], byte[]>> rawIterator(
-        @Nullable String spaceName, int part) throws IgniteSpiException {
-        Space space = space(spaceName, false);
-
-        if (space == null)
-            return null;
-
-        return rawIterator(space.entriesIterator(part));
-    }
-
-    /**
-     * Creates raw iterator based on provided entries iterator.
-     *
-     * @param iter Entries iterator.
-     * @return Raw iterator.
-     */
-    private IgniteSpiCloseableIterator<Map.Entry<byte[], byte[]>> rawIterator(
-        final Iterator<Map.Entry<GridSwapKey, byte[]>> iter) {
-        return new GridCloseableIteratorAdapter<Map.Entry<byte[], byte[]>>() {
-            @Override protected Map.Entry<byte[], byte[]> onNext() throws GridException {
-                Map.Entry<GridSwapKey, byte[]> x = iter.next();
-
-                return new T2<>(keyBytes(x.getKey()), x.getValue());
-            }
-
-            @Override protected boolean onHasNext() {
-                return iter.hasNext();
-            }
-
-            @Override protected void onRemove() {
-                iter.remove();
-            }
-        };
-    }
-
-    /**
-     * Gets key bytes.
-     *
-     * @param key Swap key.
-     * @return Key bytes.
-     * @throws org.apache.ignite.spi.IgniteSpiException In case of error.
-     */
-    private byte[] keyBytes(GridSwapKey key) throws IgniteSpiException {
-        assert key != null;
-
-        byte[] keyBytes = key.keyBytes();
-
-        if (keyBytes == null) {
-            try {
-                keyBytes = marsh.marshal(key.key());
-            }
-            catch (GridException e) {
-                throw new IgniteSpiException("Failed to marshal key: " + key.key(), e);
-            }
-
-            key.keyBytes(keyBytes);
-        }
-
-        return keyBytes;
-    }
-
-    /**
-     * Notifies eviction listener.
-     *
-     * @param evtType Event type.
-     * @param spaceName Space name.
-     */
-    private void notifyListener(int evtType, @Nullable String spaceName) {
-        GridSwapSpaceSpiListener lsnr = evictLsnr;
-
-        if (lsnr != null)
-            lsnr.onSwapEvent(evtType, spaceName, null);
-    }
-
-    /**
-     * Gets space by name.
-     *
-     * @param name Space name.
-     * @param create Whether to create space if it doesn't exist.
-     * @return Space.
-     * @throws org.apache.ignite.spi.IgniteSpiException In case of error.
-     */
-    @Nullable private Space space(@Nullable String name, boolean create) throws IgniteSpiException {
-        String masked = name != null ? name : DFLT_SPACE_NAME;
-
-        assert masked != null;
-
-        Space space = spaces.get(masked);
-
-        if (space == null && create) {
-            validateName(name);
-
-            Space old = spaces.putIfAbsent(masked, space = new Space(masked));
-
-            if (old != null)
-                space = old;
-        }
-
-        if (space != null)
-            space.initialize();
-
-        return space;
-    }
-
-    /**
-     * Validates space name.
-     *
-     * @param name Space name.
-     * @throws org.apache.ignite.spi.IgniteSpiException If name is invalid.
-     */
-    private void validateName(@Nullable String name) throws IgniteSpiException {
-        if (name == null)
-            return;
-
-        if (name.isEmpty())
-            throw new IgniteSpiException("Space name cannot be empty: " + name);
-        else if (DFLT_SPACE_NAME.equalsIgnoreCase(name))
-            throw new IgniteSpiException("Space name is reserved for default space: " + name);
-        else if (name.contains("/") || name.contains("\\"))
-            throw new IgniteSpiException("Space name contains invalid characters: " + name);
-    }
-
-    /** {@inheritDoc} */
-    @Override public String toString() {
-        return S.toString(GridFileSwapSpaceSpi.class, this);
-    }
-
-    /**
-     * Swap value.
-     */
-    static class SwapValue {
-        /** */
-        private static final int NEW = 0;
-
-        /** */
-        private static final int DELETED = Integer.MIN_VALUE;
-
-        /** */
-        private static final AtomicIntegerFieldUpdater<SwapValue> idxUpdater = AtomicIntegerFieldUpdater.
-            newUpdater(SwapValue.class, "idx");
-
-        /** */
-        private byte[] val;
-
-        /** */
-        private final int len;
-
-        /** */
-        @SuppressWarnings("FieldAccessedSynchronizedAndUnsynchronized")
-        private long pos = -1;
-
-        /** */
-        @SuppressWarnings("UnusedDeclaration")
-        private volatile int idx;
-
-        /**
-         * @param val Value.
-         */
-        SwapValue(byte[] val) {
-            assert val != null;
-
-            this.val = val;
-            len = val.length;
-        }
-
-        /**
-         * @param space Space.
-         * @return Value.
-         * @throws org.apache.ignite.spi.IgniteSpiException If failed.
-         */
-        @SuppressWarnings("NonPrivateFieldAccessedInSynchronizedContext")
-        @Nullable public synchronized byte[] value(Space space) throws IgniteSpiException {
-            byte[] v = val;
-
-            if (v == null) { // Read value from file.
-                int i = idx;
-
-                assert i != NEW;
-
-                if (i != DELETED) {
-                    StripedFileChannel ch = i < 0 ? space.left.readCh : space.right.readCh;
-
-                    if (idx != DELETED) // Double check works in pair with striped channel reopening.
-                        v = readValue(ch);
-                }
-            }
-            else if (v.length != len) {
-                int p = (int)pos;
-
-                v = Arrays.copyOfRange(v, p, p + len); // In case of compaction.
-            }
-
-            return v;
-        }
-
-        /**
-         * @param ch File channel.
-         * @return Bytes.
-         * @throws org.apache.ignite.spi.IgniteSpiException if failed.
-         */
-        @Nullable byte[] readValue(StripedFileChannel ch) throws IgniteSpiException {
-            byte[] v = new byte[len];
-
-            int res = 0;
-
-            try {
-                res = ch.read(ByteBuffer.wrap(v), pos);
-            }
-            catch (ClosedChannelException ignore) {
-                assert idx == DELETED;
-            }
-            catch (IOException e) {
-                throw new IgniteSpiException("Failed to read value.", e);
-            }
-
-            if (res < len)
-                return null; // When concurrent compaction occurs this may happen.
-
-            return v;
-        }
-
-        /**
-         * @param pos Position.
-         * @param val Value.
-         */
-        public synchronized void set(long pos, byte[] val) {
-            if (pos != -1)
-                this.pos = pos;
-
-            this.val = val;
-        }
-
-        /**
-         * @param exp Expected.
-         * @param idx New index.
-         * @return {@code true} if succeeded.
-         */
-        public boolean casIdx(int exp, int idx) {
-            return idxUpdater.compareAndSet(this, exp, idx);
-        }
-
-        /**
-         * @return Index in file array.
-         */
-        int idx() {
-            return idx;
-        }
-
-        /** {@inheritDoc} */
-        @Override public String toString() {
-            return pos + " " + len;
-        }
-    }
-
-    /**
-     * Queue of swap values.
-     */
-    private static class SwapValuesQueue {
-        /** */
-        private final ArrayDeque<SwapValue> deq = new ArrayDeque<>();
-
-        /** */
-        @SuppressWarnings("TypeMayBeWeakened")
-        private final ReentrantLock lock = new ReentrantLock();
-
-        /** */
-        private final Condition mayAdd = lock.newCondition();
-
-        /** */
-        private final Condition mayTake = lock.newCondition();
-
-        /** */
-        private int size;
-
-        /** */
-        private final int minTakeSize;
-
-        /** */
-        private final int maxSize;
-
-        /**
-         * @param minTakeSize Min size.
-         * @param maxSize Max size.
-         */
-        private SwapValuesQueue(int minTakeSize, int maxSize) {
-            this.minTakeSize = minTakeSize;
-            this.maxSize = maxSize;
-        }
-
-        /**
-         * Adds to queue.
-         *
-         * @param val Swap value.
-         * @throws org.apache.ignite.spi.IgniteSpiException If failed.
-         */
-        public void add(SwapValue val) throws IgniteSpiException {
-            lock.lock();
-
-            try {
-                while (size + val.len > maxSize)
-                    mayAdd.await();
-
-                size += val.len;
-
-                deq.addLast(val);
-
-                if (size >= minTakeSize)
-                    mayTake.signalAll();
-            }
-            catch (InterruptedException e) {
-                throw new IgniteSpiException(e);
-            }
-            finally {
-                lock.unlock();
-            }
-        }
-
-        /**
-         * Takes swap values from queue.
-         *
-         * @return Swap values.
-         * @throws InterruptedException If interrupted.
-         */
-        public SwapValues take() throws InterruptedException {
-            lock.lock();
-
-            try {
-                while (size < minTakeSize)
-                    mayTake.await();
-
-                int size = 0;
-                int cnt = 0;
-
-                for (SwapValue val : deq) {
-                    size += val.len;
-                    cnt++;
-
-                    if (size >= minTakeSize)
-                        break;
-                }
-
-                SwapValue[] vals = new SwapValue[cnt];
-
-                for (int i = 0; i < cnt; i++) {
-                    SwapValue val = deq.pollFirst();
-
-                    vals[i] = val;
-                }
-
-                if ((this.size -= size) < maxSize)
-                    mayAdd.signalAll();
-
-                return new SwapValues(vals, size);
-            }
-            finally {
-                lock.unlock();
-            }
-        }
-    }
-
-    /**
-     * Array of swap values and their size in bytes.
-     */
-    static class SwapValues {
-        /** */
-        private final SwapValue[] vals;
-
-        /** Size in bytes. */
-        private final int size;
-
-        /**
-         * @param vals Values.
-         * @param size Size.
-         */
-        SwapValues(SwapValue[] vals, int size) {
-            this.vals = vals;
-            this.size = size;
-        }
-    }
-
-    /**
-     * Readable striped file channel.
-     */
-    private static class StripedFileChannel {
-        /** */
-        private final AtomicInteger enter = new AtomicInteger();
-
-        /** */
-        private final RandomAccessFile[] rafs;
-
-        /** */
-        private final FileChannel[] chs;
-
-        /**
-         * @param f File.
-         * @param stripes Stripes.
-         * @throws FileNotFoundException If failed.
-         */
-        StripedFileChannel(File f, int stripes) throws FileNotFoundException {
-            assert stripes > 0 && (stripes & (stripes - 1)) == 0 : "stripes must be positive and power of two.";
-
-            rafs = new RandomAccessFile[stripes];
-            chs = new FileChannel[stripes];
-
-            for (int i = 0; i < stripes; i++) {
-                RandomAccessFile raf = new RandomAccessFile(f, "r");
-
-                rafs[i] = raf;
-                chs[i] = raf.getChannel();
-            }
-        }
-
-        /**
-         * Reads data from file channel to buffer.
-         *
-         * @param buf Buffer.
-         * @param pos Position.
-         * @return Read bytes count.
-         * @throws IOException If failed.
-         */
-        int read(ByteBuffer buf, long pos) throws IOException {
-            int i = enter.getAndIncrement() & (chs.length - 1);
-
-            return chs[i].read(buf, pos);
-        }
-
-        /**
-         * Closes channel.
-         */
-        void close() {
-            for (RandomAccessFile raf : rafs)
-                U.closeQuiet(raf);
-        }
-    }
-
-    /**
-     * Swap file.
-     */
-    static class SwapFile {
-        /** */
-        private static final long MIN_TRUNK_SIZE = 10 * 1024 * 1024;
-
-        /** */
-        private final File file;
-
-        /** */
-        private final RandomAccessFile raf;
-
-        /** */
-        private final FileChannel writeCh;
-
-        /** */
-        volatile StripedFileChannel readCh;
-
-        /** */
-        private volatile long len;
-
-        /** */
-        private final GridFileSwapArray<SwapValue> arr = new GridFileSwapArray<>();
-
-        /**
-         * @param file File.
-         * @param readerStripes Reader stripes number.
-         * @throws IOException In case of error.
-         */
-        SwapFile(File file, int readerStripes) throws IOException {
-            assert file != null;
-
-            file.delete();
-
-            if (!file.createNewFile())
-                throw new IllegalStateException("Failed to create file: " + file.getAbsolutePath());
-
-            this.file = file;
-
-            raf = new RandomAccessFile(file, "rw");
-
-            writeCh = raf.getChannel();
-
-            readCh = new StripedFileChannel(file, readerStripes);
-        }
-
-        /**
-         * Reopens read channel.
-         *
-         * @throws FileNotFoundException If failed.
-         */
-        void reopenReadChannel() throws FileNotFoundException {
-            readCh.close();
-
-            readCh = new StripedFileChannel(file, readCh.chs.length);
-        }
-
-        /**
-         * @param vals Values.
-         * @param buf Duffer.
-         * @param sign Indicates where should we write value, to the left or to the right.
-         * @throws Exception If failed.
-         */
-        public void write(Iterable<SwapValue> vals, ByteBuffer buf, int sign) throws Exception {
-            for (SwapValue val : vals) {
-                int oldIdx = val.idx;
-
-                if (oldIdx == SwapValue.DELETED)
-                    continue;
-
-                int idx = arr.add(val);
-
-                if (!val.casIdx(oldIdx, sign * idx)) {
-                    assert val.idx == SwapValue.DELETED;
-
-                    boolean res = tryRemove(idx, val);
-
-                    assert res;
-                }
-            }
-
-            final int size = buf.remaining();
-
-            if (size == 0)
-                return;
-
-            long pos = len;
-
-            len = pos + size;
-
-            long res = writeCh.write(buf, pos);
-
-            if (res != size)
-                throw new IllegalStateException(res + " != " + size);
-
-            // Nullify bytes in values ans set pos.
-            for (SwapValue val : vals) {
-                val.set(pos, null);
-
-                pos += val.len;
-            }
-        }
-
-        /**
-         * @param vals Values.
-         * @param sign Sign: 1 or -1.
-         * @throws Exception If failed.
-         */
-        public void write(SwapValues vals, int sign) throws Exception {
-            ByteBuffer buf = ByteBuffer.allocateDirect(vals.size);
-
-            for (int i = 0, len = vals.vals.length; i < len; i++) {
-                SwapValue val = vals.vals[i];
-
-                if (val.idx == SwapValue.DELETED) {
-                    vals.vals[i] = null;
-
-                    continue;
-                }
-
-                int idx = arr.add(val);
-
-                if (!val.casIdx(SwapValue.NEW, sign * idx)) {
-                    assert val.idx == SwapValue.DELETED;
-
-                    tryRemove(idx, val);
-
-                    vals.vals[i] = null;
-                }
-                else
-                    buf.put(val.value(null));
-            }
-
-            buf.flip();
-
-            final int size = buf.remaining();
-
-            if (size == 0)
-                return;
-
-            long pos = len;
-
-            len = pos + size;
-
-            long res = writeCh.write(buf, pos);
-
-            if (res != size)
-                throw new IllegalStateException(res + " != " + size);
-
-            // Nullify bytes in values ans set pos.
-            for (SwapValue val : vals.vals) {
-                if (val == null)
-                    continue;
-
-                val.set(pos, null);
-
-                pos += val.len;
-            }
-        }
-
-        /**
-         * Gets file path.
-         *
-         * @return File path.
-         */
-        public String path() {
-            return file.getAbsolutePath();
-        }
-
-        /**
-         * Gets file length.
-         *
-         * @return File length.
-         */
-        public long length() {
-            return len;
-        }
-
-        /**
-         * Deletes file.
-         *
-         * @return Whether file was actually deleted.
-         */
-        public boolean delete() {
-            U.closeQuiet(raf);
-
-            readCh.close();
-
-            return U.delete(file);
-        }
-
-        /**
-         * @param idx Index.
-         * @param exp Expected value.
-         * @return {@code true} If succeeded.
-         */
-        public boolean tryRemove(int idx, SwapValue exp) {
-            assert idx > 0 : idx;
-
-            GridFileSwapArray.Slot<SwapValue> s = arr.slot(idx);
-
-            return s != null && s.cas(exp, null);
-        }
-
-        /**
-         * Does compaction for one buffer.
-         *
-         * @param vals Values.
-         * @param bufSize Buffer size.
-         * @return Buffer.
-         * @throws IOException If failed.
-         * @throws InterruptedException If interrupted.
-         */
-        public ByteBuffer compact(ArrayDeque<SwapValue> vals, final int bufSize) throws IOException,
-            InterruptedException {
-            assert vals.isEmpty();
-
-            Compact c = new Compact(vals, bufSize);
-
-            c.doCompact();
-
-            return c.result();
-        }
-
-        /**
-         * Single compaction operation.
-         */
-        private class Compact {
-            /** */
-            private final ArrayDeque<SwapValue> vals;
-
-            /** */
-            private final int bufSize;
-
-            /** */
-            private byte[] bytes;
-
-            /** */
-            private ByteBuffer buf;
-
-            /** */
-            private long beg = -1;
-
-            /** */
-            private long end = -1;
-
-            /** */
-            private int compacted;
-
-            /**
-             * @param vals Values.
-             * @param bufSize Buffer size.
-             */
-            private Compact(ArrayDeque<SwapValue> vals, final int bufSize) {
-                assert vals.isEmpty();
-
-                this.vals = vals;
-                this.bufSize = bufSize;
-            }
-
-            /**
-             * Reads buffer and compacts it.
-             *
-             * @throws IOException if failed.
-             */
-            private void readAndCompact() throws IOException {
-                assert beg != -1;
-
-                if (buf == null) {
-                    bytes = new byte[bufSize];
-
-                    buf = ByteBuffer.wrap(bytes);
-                }
-
-                final int pos = buf.position();
-
-                final int lim = (int)(end - beg + pos);
-
-                assert pos >= 0;
-                assert pos < lim : pos + " " + lim;
-                assert lim <= buf.capacity();
-
-                buf.limit(lim);
-
-                int res = writeCh.read(buf, beg);
-
-                assert res == lim - pos;
-
-                int prevEnd = pos;
-                long delta = beg - pos; // To translate from file based positions to buffer based.
-
-                for (int j = vals.size(); j > compacted; j--) {
-                    SwapValue val = vals.pollFirst();
-
-                    int valPos = (int)(val.pos - delta);
-
-                    if (prevEnd != valPos) {
-                        assert prevEnd < valPos : prevEnd + " " + valPos;
-
-                        U.arrayCopy(bytes, valPos, bytes, prevEnd, val.len);
-                    }
-
-                    prevEnd += val.len;
-
-                    vals.addLast(val); // To have values in the same order as in byte buffer.
-                }
-
-                assert prevEnd > 0 : prevEnd;
-
-                buf.position(prevEnd);
-
-                end = -1;
-
-                compacted = vals.size();
-            }
-
-            /**
-             * Compacts.
-             *
-             * @throws IOException If failed.
-             */
-            private void doCompact() throws IOException {
-                int idx = arr.size();
-
-                while (--idx > 0) {
-                    GridFileSwapArray.Slot<SwapValue> s = arr.slot(idx);
-
-                    assert s != null;
-
-                    SwapValue v = s.get();
-
-                    if (v == null || v.idx == SwapValue.DELETED)
-                        continue;
-
-                    if (end == -1)
-                        end = v.pos + v.len;
-
-                    long size = end - v.pos;
-
-                    if ((buf == null ? bufSize : buf.remaining()) < size) {
-                        if (vals.isEmpty()) {  // Too big single value.
-                            assert bytes == null && buf == null;
-
-                            bytes = new byte[(int)size];
-
-                            buf = ByteBuffer.wrap(bytes);
-                        }
-                        else if (compacted == vals.size())
-                            break; // Finish current compaction, nothing new collected.
-                        else { // Read region and compact values in buffer.
-                            readAndCompact();
-
-                            // Retry the same value.
-                            idx++;
-
-                            continue;
-                        }
-                    }
-
-                    beg = v.pos;
-
-                    vals.addFirst(v);
-
-                    s.cas(v, null);
-                }
-
-                if (vals.isEmpty()) {
-                    arr.truncate(1);
-
-                    writeCh.truncate(0);
-
-                    len = 0;
-
-                    reopenReadChannel(); // Make sure that value can be read only from right file but not after switch.
-
-                    return;
-                }
-
-                if (compacted != vals.size())
-                    readAndCompact();
-
-                int pos = 0;
-
-                for (SwapValue val : vals) { // The values will share one byte array with different offsets while moving.
-                    val.set(pos, bytes);
-
-                    pos += val.len;
-                }
-
-                buf.flip();
-
-                assert buf.limit() == pos : buf.limit() + " " + pos;
-
-                arr.truncate(idx + 1);
-
-                if (len - beg > MIN_TRUNK_SIZE) {
-                    writeCh.truncate(beg);
-
-                    len = beg;
-                }
-            }
-
-            /**
-             * @return Buffer.
-             */
-            public ByteBuffer result() {
-                return buf;
-            }
-        }
-    }
-
-    /**
-     * Space.
-     */
-    private class Space {
-        /** Space name. */
-        private final String name;
-
-        /** */
-        private final GridAtomicInitializer<Void> initializer = new GridAtomicInitializer<>();
-
-        /** Swap file left. */
-        @SuppressWarnings("FieldAccessedSynchronizedAndUnsynchronized")
-        private SwapFile left;
-
-        /** Swap file right. */
-        @SuppressWarnings("FieldAccessedSynchronizedAndUnsynchronized")
-        private SwapFile right;
-
-        /** */
-        private final SwapValuesQueue que = new SwapValuesQueue(writeBufSize, maxWriteQueSize);
-
-        /** Partitions. */
-        private final ConcurrentMap<Integer, ConcurrentMap<GridSwapKey, SwapValue>> parts =
-            new ConcurrentHashMap8<>();
-
-        /** Total size. */
-        private final AtomicLong size = new AtomicLong();
-
-        /** Total count. */
-        private final AtomicLong cnt = new AtomicLong();
-
-        /** */
-        private int sign = 1;
-
-        /** Writer thread. */
-        private Thread writer;
-
-        /** */
-        private Thread compactor;
-
-        /**
-         * @param name Space name.
-         */
-        private Space(String name) {
-            assert name != null;
-
-            this.name = name;
-        }
-
-        /**
-         * Initializes space.
-         *
-         * @throws org.apache.ignite.spi.IgniteSpiException If initialization failed.
-         */
-        public void initialize() throws IgniteSpiException {
-            if (initializer.succeeded())
-                return;
-
-            assert dir.exists();
-            assert dir.isDirectory();
-
-            try {
-                initializer.init(new Callable<Void>(){
-                    @Override public Void call() throws Exception {
-                        left = new SwapFile(new File(dir, name + ".left"), readStripesNum);
-
-                        right = new SwapFile(new File(dir, name + ".right"), readStripesNum);
-
-                        final Object mux = new Object();
-
-                        writer = new IgniteSpiThread(gridName,  "Swap writer: " + name, log) {
-                            @Override protected void body() throws InterruptedException {
-                                while (!isInterrupted()) {
-                                    SwapValues vals = que.take();
-
-                                    synchronized (mux) {
-                                        SwapFile f = sign == 1 ? right : left;
-
-                                        try {
-                                            f.write(vals, sign);
-                                        }
-                                        catch (Exception e) {
-                                            throw new GridRuntimeException(e);
-                                        }
-                                    }
-                                }
-                            }
-                        };
-
-                        compactor = new IgniteSpiThread(gridName, "Swap compactor: " + name, log) {
-                            @Override protected void body() throws InterruptedException {
-                                SwapFile w = null;
-                                SwapFile c = null;
-
-                                ArrayDeque<SwapValue> vals = null;
-
-                                while (!isInterrupted()) {
-                                    while(!needCompact()) {
-                                        LockSupport.park();
-
-                                        if (isInterrupted())
-                                            return;
-                                    }
-
-                                    ByteBuffer buf = null;
-
-                                    if (vals == null)
-                                        vals = new ArrayDeque<>();
-                                    else {
-                                        vals.clear();
-
-                                        try {
-                                            buf = c.compact(vals, writeBufSize);
-                                        }
-                                        catch (IOException e) {
-                                            throw new GridRuntimeException(e);
-                                        }
-                                    }
-
-                                    if (vals.isEmpty()) {
-                                        synchronized (mux) {
-                                            sign = -sign;
-
-                                            if (sign == 1) {
-                                                w = right;
-                                                c = left;
-                                            }
-                                            else {
-                                                w = left;
-                                                c = right;
-                                            }
-                                        }
-                                    }
-                                    else {
-                                        assert buf != null && buf.remaining() != 0;
-
-                                        synchronized (mux) {
-                                            try {
-                                                w.write(vals, buf, sign);
-                                            }
-                                            catch (Exception e) {
-                                                throw new GridRuntimeException(e);
-                                            }
-                                        }
-                                    }
-                                }
-                            }
-                        };
-
-                        writer.start();
-                        compactor.start();
-
-                        return null;
-                    }
-                });
-            }
-            catch (GridException e) {
-                throw new IgniteSpiException(e);
-            }
-        }
-
-        /**
-         * Gets total space size in bytes.
-         *
-         * @return Total size.
-         */
-        public long size() {
-            return left.length() + right.length();
-        }
-
-        /**
-         * Gets total space count.
-         *
-         * @return Total count.
-         */
-        public long count() {
-            return cnt.get();
-        }
-
-        /**
-         * Clears space.
-         *
-         * @throws org.apache.ignite.spi.IgniteSpiException If failed.
-         */
-        public void clear() throws IgniteSpiException {
-            Iterator<Map.Entry<GridSwapKey, byte[]>> iter = entriesIterator();
-
-            while (iter.hasNext())
-                remove(iter.next().getKey(), false);
-        }
-
-        /**
-         * Stops space.
-         *
-         * @throws GridInterruptedException If interrupted.
-         */
-        public void stop() throws GridInterruptedException {
-            U.interrupt(writer);
-            U.interrupt(compactor);
-
-            U.join(writer);
-            U.join(compactor);
-
-            left.delete();
-            right.delete();
-        }
-
-        /**
-         * Stores value in space.
-         *
-         * @param key Key.
-         * @param val Value.
-         * @throws org.apache.ignite.spi.IgniteSpiException In case of error.
-         */
-        public void store(final GridSwapKey key, @Nullable final byte[] val) throws IgniteSpiException {
-            assert key != null;
-
-            final ConcurrentMap<GridSwapKey, SwapValue> part = partition(key.partition(), true);
-
-            assert part != null;
-
-            if (val == null) {
-                SwapValue swapVal = part.remove(key);
-
-                if (swapVal != null) {
-                    removeFromFile(swapVal);
-
-                    size.addAndGet(-swapVal.len);
-                    cnt.decrementAndGet();
-                }
-
-                return;
-            }
-
-            final SwapValue swapVal = new SwapValue(val);
-
-            SwapValue old = part.put(key, swapVal);
-
-            if (old != null) {
-                size.addAndGet(val.length - old.len);
-
-                removeFromFile(old);
-            }
-            else {
-                size.addAndGet(val.length);
-                cnt.incrementAndGet();
-            }
-
-            que.add(swapVal);
-        }
-
-        /**
-         * Reads value from space.
-         *
-         * @param key Key.
-         * @return Value.
-         * @throws org.apache.ignite.spi.IgniteSpiException In case of error.
-         */
-        @Nullable public byte[] read(GridSwapKey key) throws IgniteSpiException {
-            assert key != null;
-
-            final Map<GridSwapKey, SwapValue> part = partition(key.partition(), false);
-
-            if (part == null)
-                return null;
-
-            SwapValue swapVal = part.get(key);
-
-            if (swapVal == null)
-                return null;
-
-            return swapVal.value(this);
-        }
-
-        /**
-         * Removes value from space.
-         *
-         * @param key Key.
-         * @param read If value has to be read.
-         * @return Value.
-         * @throws org.apache.ignite.spi.IgniteSpiException In case of error.
-         */
-        @Nullable public byte[] remove(GridSwapKey key, boolean read) throws IgniteSpiException {
-            assert key != null;
-
-            final Map<GridSwapKey, SwapValue> part = partition(key.partition(), false);
-
-            if (part == null)
-                return null;
-
-            SwapValue val = part.remove(key);
-
-            if (val == null)
-                return null;
-
-            size.addAndGet(-val.len);
-
-            cnt.decrementAndGet();
-
-            byte[] bytes = null;
-
-            if (read) {
-                bytes = val.value(this);
-
-                assert bytes != null; // Value bytes were read before removal from file, so compaction can't happen.
-            }
-
-            removeFromFile(val);
-
-            return bytes;
-        }
-
-        /**
-         * @param val Value.
-         */
-        private void removeFromFile(SwapValue val) {
-            for (;;) {
-                int idx = val.idx;
-
-                assert idx != SwapValue.DELETED;
-
-                if (val.casIdx(idx, SwapValue.DELETED)) {
-                    if (idx != SwapValue.NEW) {
-                        SwapFile f = idx > 0 ? right : left;
-
-                        f.tryRemove(Math.abs(idx), val);
-                    }
-
-                    break;
-                }
-            }
-
-            if (needCompact())
-                LockSupport.unpark(compactor);
-        }
-
-        /**
-         * @return {@code true} If compaction needed.
-         */
-        private boolean needCompact() {
-            long fileLen = size();
-
-            return fileLen > writeBufSize && (fileLen - size.get()) / (float)fileLen > maxSparsity;
-        }
-
-        /**
-         * Gets numbers of partitioned stored in this space.
-         *
-         * @return Partition numbers.
-         */
-        public Collection<Integer> partitions() {
-            return parts.keySet();
-        }
-
-        /**
-         * Gets partition map by its number.
-         *
-         * @param part Partition number.
-         * @param create Whether to create partition if it doesn't exist.
-         * @return Partition map.
-         */
-        @Nullable private ConcurrentMap<GridSwapKey, SwapValue> partition(int part, boolean create) {
-            ConcurrentMap<GridSwapKey, SwapValue> map = parts.get(part);
-
-            if (map == null && create) {
-                ConcurrentMap<GridSwapKey, SwapValue> old = parts.putIfAbsent(part,
-                    map = new ConcurrentHashMap<>());
-
-                if (old != null)
-                    map = old;
-            }
-
-            return map;
-        }
-
-        /**
-         * @param part Partition.
-         * @return Iterator over partition.
-         */
-        public Iterator<Map.Entry<GridSwapKey, byte[]>> entriesIterator(int part) {
-            Map<GridSwapKey, SwapValue> partMap = partition(part, false);
-
-            if (partMap == null)
-                return Collections.<Map.Entry<GridSwapKey, byte[]>>emptySet().iterator();
-
-            return transform(partMap.entrySet().iterator());
-        }
-
-        /**
-         * @return Iterator over all entries.
-         */
-        public Iterator<Map.Entry<GridSwapKey, byte[]>> entriesIterator() {
-            final Iterator<ConcurrentMap<GridSwapKey, SwapValue>> iter = parts.values().iterator();
-
-            return transform(F.concat(new Iterator<Iterator<Map.Entry<GridSwapKey, SwapValue>>>() {
-                @Override public boolean hasNext() {
-                    return iter.hasNext();
-                }
-
-                @Override public Iterator<Map.Entry<GridSwapKey, SwapValue>> next() {
-                    return iter.next().entrySet().iterator();
-                }
-
-                @Override public void remove() {
-                    throw new UnsupportedOperationException();
-                }
-            }));
-        }
-
-        /**
-         * Gets iterator for all entries in space.
-         *
-         * @param iter Iterator with {@link SwapValue} to transform.
-         * @return Entries iterator.
-         */
-        private Iterator<Map.Entry<GridSwapKey, byte[]>> transform(final Iterator<Map.Entry<GridSwapKey,
-            SwapValue>> iter) {
-            return new Iterator<Map.Entry<GridSwapKey, byte[]>>() {
-                /** */
-                private Map.Entry<GridSwapKey, byte[]> next;
-
-                /** */
-                private Map.Entry<GridSwapKey, byte[]> last;
-
-                {
-                    advance();
-                }
-
-                @Override public boolean hasNext() {
-                    return next != null;
-                }
-
-                /**
-                 * Gets next entry.
-                 */
-                private void advance() {
-                    while (iter.hasNext()) {
-                        Map.Entry<GridSwapKey, SwapValue> entry = iter.next();
-
-                        byte[] bytes;
-
-                        try {
-                            bytes = entry.getValue().value(Space.this);
-                        }
-                        catch (IgniteSpiException e) {
-                            throw new GridRuntimeException(e);
-                        }
-
-                        if (bytes != null) {
-                            next = new T2<>(entry.getKey(), bytes);
-
-                            break;
-                        }
-                    }
-                }
-
-                @Override public Map.Entry<GridSwapKey, byte[]> next() {
-                    final Map.Entry<GridSwapKey, byte[]> res = next;
-
-                    if (res == null)
-                        throw new NoSuchElementException();
-
-                    next = null;
-
-                    advance();
-
-                    last = res;
-
-                    return res;
-                }
-
-                @Override public void remove() {
-                    if (last == null)
-                        throw new IllegalStateException();
-
-                    try {
-                        Space.this.remove(last.getKey(), false);
-                    }
-                    catch (IgniteSpiException e) {
-                        throw new GridRuntimeException(e);
-                    }
-                    finally {
-                        last = null;
-                    }
-                }
-            };
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c31cec7c/modules/core/src/main/java/org/gridgain/grid/spi/swapspace/file/GridFileSwapSpaceSpiMBean.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/spi/swapspace/file/GridFileSwapSpaceSpiMBean.java b/modules/core/src/main/java/org/gridgain/grid/spi/swapspace/file/GridFileSwapSpaceSpiMBean.java
deleted file mode 100644
index 9c11b5c..0000000
--- a/modules/core/src/main/java/org/gridgain/grid/spi/swapspace/file/GridFileSwapSpaceSpiMBean.java
+++ /dev/null
@@ -1,59 +0,0 @@
-/* @java.file.header */
-
-/*  _________        _____ __________________        _____
- *  __  ____/___________(_)______  /__  ____/______ ____(_)_______
- *  _  / __  __  ___/__  / _  __  / _  / __  _  __ `/__  / __  __ \
- *  / /_/ /  _  /    _  /  / /_/ /  / /_/ /  / /_/ / _  /  _  / / /
- *  \____/   /_/     /_/   \_,__/   \____/   \__,_/  /_/   /_/ /_/
- */
-
-package org.gridgain.grid.spi.swapspace.file;
-
-import org.apache.ignite.mbean.*;
-import org.apache.ignite.spi.*;
-
-/**
- * Management bean for {@link GridFileSwapSpaceSpi}.
- */
-@IgniteMBeanDescription("MBean that provides configuration information on file-based swapspace SPI.")
-public interface GridFileSwapSpaceSpiMBean extends IgniteSpiManagementMBean {
-    /**
-     * Gets base directory.
-     *
-     * @return Base directory.
-     */
-    @IgniteMBeanDescription("Base directory.")
-    public String getBaseDirectory();
-
-    /**
-     * Gets maximum sparsity.
-     *
-     * @return Maximum sparsity.
-     */
-    @IgniteMBeanDescription("Maximum sparsity.")
-    public float getMaximumSparsity();
-
-    /**
-     * Gets write buffer size in bytes.
-     *
-     * @return Write buffer size in bytes.
-     */
-    @IgniteMBeanDescription("Write buffer size in bytes.")
-    public int getWriteBufferSize();
-
-    /**
-     * Gets max write queue size in bytes.
-     *
-     * @return Max write queue size in bytes.
-     */
-    @IgniteMBeanDescription("Max write queue size in bytes.")
-    public int getMaxWriteQueueSize();
-
-    /**
-     * Gets read pool size.
-     *
-     * @return Read pool size.
-     */
-    @IgniteMBeanDescription("Read pool size.")
-    public int getReadStripesNumber();
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c31cec7c/modules/core/src/main/java/org/gridgain/grid/spi/swapspace/noop/GridNoopSwapSpaceSpi.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/spi/swapspace/noop/GridNoopSwapSpaceSpi.java b/modules/core/src/main/java/org/gridgain/grid/spi/swapspace/noop/GridNoopSwapSpaceSpi.java
deleted file mode 100644
index cc8545f..0000000
--- a/modules/core/src/main/java/org/gridgain/grid/spi/swapspace/noop/GridNoopSwapSpaceSpi.java
+++ /dev/null
@@ -1,126 +0,0 @@
-/* @java.file.header */
-
-/*  _________        _____ __________________        _____
- *  __  ____/___________(_)______  /__  ____/______ ____(_)_______
- *  _  / __  __  ___/__  / _  __  / _  / __  _  __ `/__  / __  __ \
- *  / /_/ /  _  /    _  /  / /_/ /  / /_/ /  / /_/ / _  /  _  / / /
- *  \____/   /_/     /_/   \_,__/   \____/   \__,_/  /_/   /_/ /_/
- */
-
-package org.gridgain.grid.spi.swapspace.noop;
-
-import org.apache.ignite.*;
-import org.apache.ignite.lang.*;
-import org.apache.ignite.resources.*;
-import org.apache.ignite.spi.*;
-import org.gridgain.grid.spi.swapspace.*;
-import org.gridgain.grid.util.*;
-import org.gridgain.grid.util.typedef.internal.*;
-import org.jetbrains.annotations.*;
-
-import java.util.*;
-
-/**
- * No-op implementation of {@link GridSwapSpaceSpi}. Exists for testing and benchmarking purposes.
- */
-@IgniteSpiNoop
-@IgniteSpiMultipleInstancesSupport(true)
-public class GridNoopSwapSpaceSpi extends IgniteSpiAdapter implements GridSwapSpaceSpi {
-    /** Logger. */
-    @IgniteLoggerResource
-    private IgniteLogger log;
-
-    /** {@inheritDoc} */
-    @Override public void spiStart(@Nullable String gridName) throws IgniteSpiException {
-        U.warn(log, "Swap space is disabled. To enable use GridFileSwapSpaceSpi.");
-    }
-
-    /** {@inheritDoc} */
-    @Override public void spiStop() throws IgniteSpiException {
-        // No-op.
-    }
-
-    /** {@inheritDoc} */
-    @Override public void clear(@Nullable String space) throws IgniteSpiException {
-        // No-op.
-    }
-
-    /** {@inheritDoc} */
-    @Override public long size(@Nullable String space) throws IgniteSpiException {
-        return 0;
-    }
-
-    /** {@inheritDoc} */
-    @Override public long count(@Nullable String space) throws IgniteSpiException {
-        return 0;
-    }
-
-    /** {@inheritDoc} */
-    @Override @Nullable public byte[] read(@Nullable String spaceName, GridSwapKey key, GridSwapContext ctx)
-        throws IgniteSpiException {
-        return null;
-    }
-
-    /** {@inheritDoc} */
-    @Override public Map<GridSwapKey, byte[]> readAll(@Nullable String spaceName, Iterable<GridSwapKey> keys,
-        GridSwapContext ctx) throws IgniteSpiException {
-        return Collections.emptyMap();
-    }
-
-    /** {@inheritDoc} */
-    @Override public void remove(@Nullable String spaceName, GridSwapKey key, @Nullable IgniteInClosure<byte[]> c,
-        GridSwapContext ctx) throws IgniteSpiException {
-        // No-op.
-    }
-
-    /** {@inheritDoc} */
-    @Override public void removeAll(@Nullable String spaceName, Collection<GridSwapKey> keys,
-        @Nullable IgniteBiInClosure<GridSwapKey, byte[]> c, GridSwapContext ctx) throws IgniteSpiException {
-        // No-op.
-    }
-
-    /** {@inheritDoc} */
-    @Override public void store(@Nullable String spaceName, GridSwapKey key, @Nullable byte[] val,
-        GridSwapContext ctx) throws IgniteSpiException {
-        // No-op.
-    }
-
-    /** {@inheritDoc} */
-    @Override public void storeAll(@Nullable String spaceName, Map<GridSwapKey, byte[]> pairs,
-        GridSwapContext ctx) throws IgniteSpiException {
-        // No-op.
-    }
-
-    /** {@inheritDoc} */
-    @Override public void setListener(@Nullable GridSwapSpaceSpiListener evictLsnr) {
-        // No-op.
-    }
-
-    /** {@inheritDoc} */
-    @Override public Collection<Integer> partitions(@Nullable String spaceName) throws IgniteSpiException {
-        return Collections.emptyList();
-    }
-
-    /** {@inheritDoc} */
-    @Override public <K> IgniteSpiCloseableIterator<K> keyIterator(@Nullable String spaceName,
-        GridSwapContext ctx) throws IgniteSpiException {
-        return new GridEmptyCloseableIterator<>();
-    }
-
-    /** {@inheritDoc} */
-    @Override public IgniteSpiCloseableIterator<Map.Entry<byte[], byte[]>> rawIterator(
-        @Nullable String spaceName) throws IgniteSpiException {
-        return new GridEmptyCloseableIterator<>();
-    }
-
-    /** {@inheritDoc} */
-    @Override public IgniteSpiCloseableIterator<Map.Entry<byte[], byte[]>> rawIterator(@Nullable String spaceName,
-        int part) throws IgniteSpiException {
-        return new GridEmptyCloseableIterator<>();
-    }
-
-    /** {@inheritDoc} */
-    @Override public String toString() {
-        return S.toString(GridNoopSwapSpaceSpi.class, this);
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c31cec7c/modules/core/src/main/java/org/gridgain/grid/spi/swapspace/noop/NoopSwapSpaceSpi.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/spi/swapspace/noop/NoopSwapSpaceSpi.java b/modules/core/src/main/java/org/gridgain/grid/spi/swapspace/noop/NoopSwapSpaceSpi.java
new file mode 100644
index 0000000..f7f2668
--- /dev/null
+++ b/modules/core/src/main/java/org/gridgain/grid/spi/swapspace/noop/NoopSwapSpaceSpi.java
@@ -0,0 +1,126 @@
+/* @java.file.header */
+
+/*  _________        _____ __________________        _____
+ *  __  ____/___________(_)______  /__  ____/______ ____(_)_______
+ *  _  / __  __  ___/__  / _  __  / _  / __  _  __ `/__  / __  __ \
+ *  / /_/ /  _  /    _  /  / /_/ /  / /_/ /  / /_/ / _  /  _  / / /
+ *  \____/   /_/     /_/   \_,__/   \____/   \__,_/  /_/   /_/ /_/
+ */
+
+package org.gridgain.grid.spi.swapspace.noop;
+
+import org.apache.ignite.*;
+import org.apache.ignite.lang.*;
+import org.apache.ignite.resources.*;
+import org.apache.ignite.spi.*;
+import org.gridgain.grid.spi.swapspace.*;
+import org.gridgain.grid.util.*;
+import org.gridgain.grid.util.typedef.internal.*;
+import org.jetbrains.annotations.*;
+
+import java.util.*;
+
+/**
+ * No-op implementation of {@link org.gridgain.grid.spi.swapspace.SwapSpaceSpi}. Exists for testing and benchmarking purposes.
+ */
+@IgniteSpiNoop
+@IgniteSpiMultipleInstancesSupport(true)
+public class NoopSwapSpaceSpi extends IgniteSpiAdapter implements SwapSpaceSpi {
+    /** Logger. */
+    @IgniteLoggerResource
+    private IgniteLogger log;
+
+    /** {@inheritDoc} */
+    @Override public void spiStart(@Nullable String gridName) throws IgniteSpiException {
+        U.warn(log, "Swap space is disabled. To enable use GridFileSwapSpaceSpi.");
+    }
+
+    /** {@inheritDoc} */
+    @Override public void spiStop() throws IgniteSpiException {
+        // No-op.
+    }
+
+    /** {@inheritDoc} */
+    @Override public void clear(@Nullable String space) throws IgniteSpiException {
+        // No-op.
+    }
+
+    /** {@inheritDoc} */
+    @Override public long size(@Nullable String space) throws IgniteSpiException {
+        return 0;
+    }
+
+    /** {@inheritDoc} */
+    @Override public long count(@Nullable String space) throws IgniteSpiException {
+        return 0;
+    }
+
+    /** {@inheritDoc} */
+    @Override @Nullable public byte[] read(@Nullable String spaceName, SwapKey key, SwapContext ctx)
+        throws IgniteSpiException {
+        return null;
+    }
+
+    /** {@inheritDoc} */
+    @Override public Map<SwapKey, byte[]> readAll(@Nullable String spaceName, Iterable<SwapKey> keys,
+        SwapContext ctx) throws IgniteSpiException {
+        return Collections.emptyMap();
+    }
+
+    /** {@inheritDoc} */
+    @Override public void remove(@Nullable String spaceName, SwapKey key, @Nullable IgniteInClosure<byte[]> c,
+        SwapContext ctx) throws IgniteSpiException {
+        // No-op.
+    }
+
+    /** {@inheritDoc} */
+    @Override public void removeAll(@Nullable String spaceName, Collection<SwapKey> keys,
+        @Nullable IgniteBiInClosure<SwapKey, byte[]> c, SwapContext ctx) throws IgniteSpiException {
+        // No-op.
+    }
+
+    /** {@inheritDoc} */
+    @Override public void store(@Nullable String spaceName, SwapKey key, @Nullable byte[] val,
+        SwapContext ctx) throws IgniteSpiException {
+        // No-op.
+    }
+
+    /** {@inheritDoc} */
+    @Override public void storeAll(@Nullable String spaceName, Map<SwapKey, byte[]> pairs,
+        SwapContext ctx) throws IgniteSpiException {
+        // No-op.
+    }
+
+    /** {@inheritDoc} */
+    @Override public void setListener(@Nullable SwapSpaceSpiListener evictLsnr) {
+        // No-op.
+    }
+
+    /** {@inheritDoc} */
+    @Override public Collection<Integer> partitions(@Nullable String spaceName) throws IgniteSpiException {
+        return Collections.emptyList();
+    }
+
+    /** {@inheritDoc} */
+    @Override public <K> IgniteSpiCloseableIterator<K> keyIterator(@Nullable String spaceName,
+        SwapContext ctx) throws IgniteSpiException {
+        return new GridEmptyCloseableIterator<>();
+    }
+
+    /** {@inheritDoc} */
+    @Override public IgniteSpiCloseableIterator<Map.Entry<byte[], byte[]>> rawIterator(
+        @Nullable String spaceName) throws IgniteSpiException {
+        return new GridEmptyCloseableIterator<>();
+    }
+
+    /** {@inheritDoc} */
+    @Override public IgniteSpiCloseableIterator<Map.Entry<byte[], byte[]>> rawIterator(@Nullable String spaceName,
+        int part) throws IgniteSpiException {
+        return new GridEmptyCloseableIterator<>();
+    }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return S.toString(NoopSwapSpaceSpi.class, this);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c31cec7c/modules/core/src/test/config/spring-cache-swap.xml
----------------------------------------------------------------------
diff --git a/modules/core/src/test/config/spring-cache-swap.xml b/modules/core/src/test/config/spring-cache-swap.xml
index 3b2bf6f..1aa7122 100644
--- a/modules/core/src/test/config/spring-cache-swap.xml
+++ b/modules/core/src/test/config/spring-cache-swap.xml
@@ -39,7 +39,7 @@
         <property name="networkTimeout" value="10000"/>
 
         <property name="swapSpaceSpi">
-            <bean class="org.gridgain.grid.spi.swapspace.file.GridFileSwapSpaceSpi">
+            <bean class="org.gridgain.grid.spi.swapspace.file.FileSwapSpaceSpi">
                 <property name="rootFolderPath" value="/Users/yzhdanov/tmp/swap-test"/>
                 <property name="taskQueueCapacity" value="100000"/>
                 <property name="poolSize" value="2"/>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c31cec7c/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 9209a34..aa89e2a 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
@@ -202,7 +202,7 @@ public class GridManagerStopSelfTest extends GridCommonAbstractTest {
      * @throws Exception If failed.
      */
     public void testStopSwapSpaceManager() throws Exception {
-        GridSwapSpaceSpi spi = new GridFileSwapSpaceSpi();
+        SwapSpaceSpi spi = new FileSwapSpaceSpi();
 
         injectLogger(spi);
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c31cec7c/modules/core/src/test/java/org/gridgain/grid/kernal/managers/swapspace/GridSwapSpaceManagerSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/gridgain/grid/kernal/managers/swapspace/GridSwapSpaceManagerSelfTest.java b/modules/core/src/test/java/org/gridgain/grid/kernal/managers/swapspace/GridSwapSpaceManagerSelfTest.java
index 5c15bb3..db45129 100644
--- a/modules/core/src/test/java/org/gridgain/grid/kernal/managers/swapspace/GridSwapSpaceManagerSelfTest.java
+++ b/modules/core/src/test/java/org/gridgain/grid/kernal/managers/swapspace/GridSwapSpaceManagerSelfTest.java
@@ -45,7 +45,7 @@ public class GridSwapSpaceManagerSelfTest extends GridCommonAbstractTest {
     @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
         IgniteConfiguration c = super.getConfiguration(gridName);
 
-        c.setSwapSpaceSpi(new GridFileSwapSpaceSpi());
+        c.setSwapSpaceSpi(new FileSwapSpaceSpi());
 
         return c;
     }
@@ -120,13 +120,13 @@ public class GridSwapSpaceManagerSelfTest extends GridCommonAbstractTest {
         // Empty data space.
         assertEquals(0, mgr.swapSize(spaceName));
 
-        GridSwapKey key = new GridSwapKey("key1");
+        SwapKey key = new SwapKey("key1");
 
         String val = "value";
 
         mgr.write(spaceName, key, val.getBytes(), null);
 
-        mgr.write(spaceName, new GridSwapKey("key2"), val.getBytes(), null);
+        mgr.write(spaceName, new SwapKey("key2"), val.getBytes(), null);
 
         assert storeCnt.await(10, SECONDS);
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c31cec7c/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheMemoryModeSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheMemoryModeSelfTest.java b/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheMemoryModeSelfTest.java
index 9a4ffb8..84b3132 100644
--- a/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheMemoryModeSelfTest.java
+++ b/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheMemoryModeSelfTest.java
@@ -66,7 +66,7 @@ public class GridCacheMemoryModeSelfTest extends GridCommonAbstractTest {
 
         cfg.setNetworkTimeout(2000);
 
-        cfg.setSwapSpaceSpi(new GridFileSwapSpaceSpi());
+        cfg.setSwapSpaceSpi(new FileSwapSpaceSpi());
 
         GridCacheConfiguration cacheCfg = defaultCacheConfiguration();
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c31cec7c/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheOffHeapSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheOffHeapSelfTest.java b/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheOffHeapSelfTest.java
index bc671da..f741cbd 100644
--- a/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheOffHeapSelfTest.java
+++ b/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheOffHeapSelfTest.java
@@ -66,7 +66,7 @@ public class GridCacheOffHeapSelfTest extends GridCommonAbstractTest {
 
         cfg.setNetworkTimeout(2000);
 
-        cfg.setSwapSpaceSpi(new GridFileSwapSpaceSpi());
+        cfg.setSwapSpaceSpi(new FileSwapSpaceSpi());
 
         GridCacheConfiguration cacheCfg = defaultCacheConfiguration();
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c31cec7c/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheOffHeapTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheOffHeapTest.java b/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheOffHeapTest.java
index 27cbf0e..66e2adb 100644
--- a/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheOffHeapTest.java
+++ b/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheOffHeapTest.java
@@ -57,7 +57,7 @@ public class GridCacheOffHeapTest extends GridCommonAbstractTest {
 
         cfg.setDiscoverySpi(disco);
 
-        cfg.setSwapSpaceSpi(new GridNoopSwapSpaceSpi());
+        cfg.setSwapSpaceSpi(new NoopSwapSpaceSpi());
 
         GridCacheConfiguration cacheCfg = defaultCacheConfiguration();
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c31cec7c/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheP2PUndeploySelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheP2PUndeploySelfTest.java b/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheP2PUndeploySelfTest.java
index 6429469..6954e2a 100644
--- a/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheP2PUndeploySelfTest.java
+++ b/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheP2PUndeploySelfTest.java
@@ -68,7 +68,7 @@ public class GridCacheP2PUndeploySelfTest extends GridCommonAbstractTest {
 
         cfg.setMarshaller(new IgniteJdkMarshaller());
 
-        cfg.setSwapSpaceSpi(new GridFileSwapSpaceSpi());
+        cfg.setSwapSpaceSpi(new FileSwapSpaceSpi());
 
         GridCacheConfiguration repCacheCfg = defaultCacheConfiguration();
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c31cec7c/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheSwapReloadSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheSwapReloadSelfTest.java b/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheSwapReloadSelfTest.java
index 2489e5c..18ddd46 100644
--- a/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheSwapReloadSelfTest.java
+++ b/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheSwapReloadSelfTest.java
@@ -51,7 +51,7 @@ public class GridCacheSwapReloadSelfTest extends GridCommonAbstractTest {
 
         cfg.setDiscoverySpi(disco);
 
-        GridFileSwapSpaceSpi swap = new GridFileSwapSpaceSpi();
+        FileSwapSpaceSpi swap = new FileSwapSpaceSpi();
 
         swap.setWriteBufferSize(1);
 
@@ -187,7 +187,7 @@ public class GridCacheSwapReloadSelfTest extends GridCommonAbstractTest {
     /**
      * @return Swap space SPI.
      */
-    private GridSwapSpaceSpi swap() {
+    private SwapSpaceSpi swap() {
         return grid().configuration().getSwapSpaceSpi();
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c31cec7c/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/distributed/GridCacheAbstractDistributedByteArrayValuesSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/distributed/GridCacheAbstractDistributedByteArrayValuesSelfTest.java b/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/distributed/GridCacheAbstractDistributedByteArrayValuesSelfTest.java
index df788cb..0852e09 100644
--- a/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/distributed/GridCacheAbstractDistributedByteArrayValuesSelfTest.java
+++ b/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/distributed/GridCacheAbstractDistributedByteArrayValuesSelfTest.java
@@ -47,7 +47,7 @@ public abstract class GridCacheAbstractDistributedByteArrayValuesSelfTest extend
             offheapCacheConfiguration(),
             offheapTieredCacheConfiguration());
 
-        c.setSwapSpaceSpi(new GridFileSwapSpaceSpi());
+        c.setSwapSpaceSpi(new FileSwapSpaceSpi());
 
         c.setPeerClassLoadingEnabled(peerClassLoading());
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c31cec7c/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/distributed/dht/GridCacheAbstractPartitionedOnlyByteArrayValuesSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/distributed/dht/GridCacheAbstractPartitionedOnlyByteArrayValuesSelfTest.java b/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/distributed/dht/GridCacheAbstractPartitionedOnlyByteArrayValuesSelfTest.java
index 54cb34f..66d1af2 100644
--- a/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/distributed/dht/GridCacheAbstractPartitionedOnlyByteArrayValuesSelfTest.java
+++ b/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/distributed/dht/GridCacheAbstractPartitionedOnlyByteArrayValuesSelfTest.java
@@ -71,7 +71,7 @@ public abstract class GridCacheAbstractPartitionedOnlyByteArrayValuesSelfTest ex
             atomicOffheapCacheCfg,
             atomicOffheapTieredCacheCfg);
 
-        c.setSwapSpaceSpi(new GridFileSwapSpaceSpi());
+        c.setSwapSpaceSpi(new FileSwapSpaceSpi());
 
         c.setPeerClassLoadingEnabled(peerClassLoading());
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c31cec7c/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/distributed/dht/GridCacheColocatedOptimisticTransactionSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/distributed/dht/GridCacheColocatedOptimisticTransactionSelfTest.java b/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/distributed/dht/GridCacheColocatedOptimisticTransactionSelfTest.java
index 9043871..bbc76cd 100644
--- a/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/distributed/dht/GridCacheColocatedOptimisticTransactionSelfTest.java
+++ b/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/distributed/dht/GridCacheColocatedOptimisticTransactionSelfTest.java
@@ -75,7 +75,7 @@ public class GridCacheColocatedOptimisticTransactionSelfTest extends GridCommonA
 
         c.setDiscoverySpi(disco);
         c.setCacheConfiguration(cc);
-        c.setSwapSpaceSpi(new GridFileSwapSpaceSpi());
+        c.setSwapSpaceSpi(new FileSwapSpaceSpi());
 
         return c;
     }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c31cec7c/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/distributed/replicated/GridCacheReplicatedUnswapAdvancedSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/distributed/replicated/GridCacheReplicatedUnswapAdvancedSelfTest.java b/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/distributed/replicated/GridCacheReplicatedUnswapAdvancedSelfTest.java
index 9e6f039..a2c721c 100644
--- a/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/distributed/replicated/GridCacheReplicatedUnswapAdvancedSelfTest.java
+++ b/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/distributed/replicated/GridCacheReplicatedUnswapAdvancedSelfTest.java
@@ -56,7 +56,7 @@ public class GridCacheReplicatedUnswapAdvancedSelfTest extends GridCommonAbstrac
 
         cfg.setCacheConfiguration(cacheCfg);
 
-        cfg.setSwapSpaceSpi(new GridFileSwapSpaceSpi());
+        cfg.setSwapSpaceSpi(new FileSwapSpaceSpi());
 
         return cfg;
     }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c31cec7c/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/local/GridCacheLocalByteArrayValuesSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/local/GridCacheLocalByteArrayValuesSelfTest.java b/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/local/GridCacheLocalByteArrayValuesSelfTest.java
index cf60f4b..0abba7b 100644
--- a/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/local/GridCacheLocalByteArrayValuesSelfTest.java
+++ b/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/local/GridCacheLocalByteArrayValuesSelfTest.java
@@ -67,7 +67,7 @@ public class GridCacheLocalByteArrayValuesSelfTest extends GridCacheAbstractByte
 
         c.setCacheConfiguration(cc1, cc2);
 
-        c.setSwapSpaceSpi(new GridFileSwapSpaceSpi());
+        c.setSwapSpaceSpi(new FileSwapSpaceSpi());
 
         return c;
     }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c31cec7c/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/query/GridCacheSwapScanQueryAbstractSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/query/GridCacheSwapScanQueryAbstractSelfTest.java b/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/query/GridCacheSwapScanQueryAbstractSelfTest.java
index 541954f..37c3707 100644
--- a/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/query/GridCacheSwapScanQueryAbstractSelfTest.java
+++ b/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/query/GridCacheSwapScanQueryAbstractSelfTest.java
@@ -56,7 +56,7 @@ public abstract class GridCacheSwapScanQueryAbstractSelfTest extends GridCommonA
 
         cfg.setDiscoverySpi(disco);
 
-        cfg.setSwapSpaceSpi(new GridFileSwapSpaceSpi());
+        cfg.setSwapSpaceSpi(new FileSwapSpaceSpi());
 
         cfg.setCacheConfiguration(cacheConfiguration(ATOMIC_CACHE_NAME, ATOMIC),
             cacheConfiguration(TRANSACTIONAL_CACHE_NAME, TRANSACTIONAL));

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c31cec7c/modules/core/src/test/java/org/gridgain/grid/loadtest/swapspace/GridFileSwapSpaceSpiMultithreadedLoadTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/gridgain/grid/loadtest/swapspace/GridFileSwapSpaceSpiMultithreadedLoadTest.java b/modules/core/src/test/java/org/gridgain/grid/loadtest/swapspace/GridFileSwapSpaceSpiMultithreadedLoadTest.java
index bca027c..68d0d4d 100644
--- a/modules/core/src/test/java/org/gridgain/grid/loadtest/swapspace/GridFileSwapSpaceSpiMultithreadedLoadTest.java
+++ b/modules/core/src/test/java/org/gridgain/grid/loadtest/swapspace/GridFileSwapSpaceSpiMultithreadedLoadTest.java
@@ -44,10 +44,10 @@ public class GridFileSwapSpaceSpiMultithreadedLoadTest extends GridCommonAbstrac
     private static final long DURATION = 10 * 60 * 1000;
 
     /** Swap context. */
-    private final GridSwapContext swapCtx = new GridSwapContext();
+    private final SwapContext swapCtx = new SwapContext();
 
     /** SPI to test. */
-    private GridSwapSpaceSpi spi;
+    private SwapSpaceSpi spi;
 
     /**
      * Starts the daemon thread.
@@ -65,8 +65,8 @@ public class GridFileSwapSpaceSpiMultithreadedLoadTest extends GridCommonAbstrac
     /**
      * @return An SPI instance to test.
      */
-    private GridSwapSpaceSpi spi() {
-        GridFileSwapSpaceSpi spi = new GridFileSwapSpaceSpi();
+    private SwapSpaceSpi spi() {
+        FileSwapSpaceSpi spi = new FileSwapSpaceSpi();
 
 //        spi.setConcurrencyLevel(N_THREADS);
 //        spi.setWriterThreadsCount(N_THREADS);
@@ -78,7 +78,7 @@ public class GridFileSwapSpaceSpiMultithreadedLoadTest extends GridCommonAbstrac
      * @return Swap context for swap operations.
      */
     @SuppressWarnings("ConstantConditions")
-    private GridSwapContext context() {
+    private SwapContext context() {
         return swapCtx;
     }
 
@@ -144,12 +144,12 @@ public class GridFileSwapSpaceSpiMultithreadedLoadTest extends GridCommonAbstrac
                 try {
                     ThreadLocalRandom8 rnd = ThreadLocalRandom8.current();
 
-                    Map<GridSwapKey, byte[]> entries = new HashMap<>(BATCH_SIZE);
+                    Map<SwapKey, byte[]> entries = new HashMap<>(BATCH_SIZE);
 
                     while (!done.get()) {
                         long l = rnd.nextLong(0, MAX_ENTRIES);
 
-                        entries.put(new GridSwapKey(l), Long.toString(l).getBytes());
+                        entries.put(new SwapKey(l), Long.toString(l).getBytes());
 
                         if (entries.size() == BATCH_SIZE) {
                             spi.storeAll(SPACE_NAME, entries, context());
@@ -206,10 +206,10 @@ public class GridFileSwapSpaceSpiMultithreadedLoadTest extends GridCommonAbstrac
                 try {
                     ThreadLocalRandom8 rnd = ThreadLocalRandom8.current();
 
-                    Collection<GridSwapKey> keys = new ArrayList<>(BATCH_SIZE);
+                    Collection<SwapKey> keys = new ArrayList<>(BATCH_SIZE);
 
                     while (!done.get()) {
-                        keys.add(new GridSwapKey(rnd.nextLong(0, MAX_ENTRIES)));
+                        keys.add(new SwapKey(rnd.nextLong(0, MAX_ENTRIES)));
 
                         if (keys.size() == BATCH_SIZE) {
                             spi.readAll(SPACE_NAME, keys, context());


[14/20] incubator-ignite git commit: # Renaming

Posted by sb...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/ef258ece/modules/core/src/test/java/org/gridgain/grid/spi/swapspace/inmemory/GridTestSwapSpaceSpi.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/gridgain/grid/spi/swapspace/inmemory/GridTestSwapSpaceSpi.java b/modules/core/src/test/java/org/gridgain/grid/spi/swapspace/inmemory/GridTestSwapSpaceSpi.java
deleted file mode 100644
index 7e96007..0000000
--- a/modules/core/src/test/java/org/gridgain/grid/spi/swapspace/inmemory/GridTestSwapSpaceSpi.java
+++ /dev/null
@@ -1,444 +0,0 @@
-/* @java.file.header */
-
-/*  _________        _____ __________________        _____
- *  __  ____/___________(_)______  /__  ____/______ ____(_)_______
- *  _  / __  __  ___/__  / _  __  / _  / __  _  __ `/__  / __  __ \
- *  / /_/ /  _  /    _  /  / /_/ /  / /_/ /  / /_/ / _  /  _  / / /
- *  \____/   /_/     /_/   \_,__/   \____/   \__,_/  /_/   /_/ /_/
- */
-
-package org.gridgain.grid.spi.swapspace.inmemory;
-
-import org.apache.ignite.lang.*;
-import org.apache.ignite.spi.*;
-import org.gridgain.grid.spi.swapspace.*;
-import org.gridgain.grid.util.typedef.*;
-import org.jdk8.backport.*;
-import org.jetbrains.annotations.*;
-
-import java.util.*;
-import java.util.concurrent.*;
-
-import static org.apache.ignite.events.IgniteEventType.*;
-
-/**
- * Test swap space SPI that stores values in map.
- */
-@IgniteSpiMultipleInstancesSupport(true)
-public class GridTestSwapSpaceSpi extends IgniteSpiAdapter implements SwapSpaceSpi {
-    /** Listener. */
-    private SwapSpaceSpiListener lsnr;
-
-    /** Spaces map. */
-    private ConcurrentMap<String, Space> spaces = new ConcurrentHashMap8<>();
-
-    /** {@inheritDoc} */
-    @Override public void spiStart(@Nullable String gridName) throws IgniteSpiException {
-        // No-op.
-    }
-
-    /** {@inheritDoc} */
-    @Override public void spiStop() throws IgniteSpiException {
-        // No-op.
-    }
-
-    /** {@inheritDoc} */
-    @Override public void clear(@Nullable String spaceName) throws IgniteSpiException {
-        Space space = space(spaceName);
-
-        if (space != null)
-            space.clear();
-    }
-
-    /** {@inheritDoc} */
-    @Override public long size(@Nullable String spaceName) throws IgniteSpiException {
-        Space space = space(spaceName);
-
-        return space != null ? space.size() : 0;
-    }
-
-    /** {@inheritDoc} */
-    @Override public long count(@Nullable String spaceName) throws IgniteSpiException {
-        Space space = space(spaceName);
-
-        return space != null ? space.count() : 0;
-    }
-
-    /** {@inheritDoc} */
-    @Override public byte[] read(@Nullable String spaceName, SwapKey key, SwapContext ctx)
-        throws IgniteSpiException {
-        Space space = space(spaceName);
-
-        return space != null ? space.read(key) : null;
-    }
-
-    /** {@inheritDoc} */
-    @Override public Map<SwapKey, byte[]> readAll(@Nullable String spaceName, Iterable<SwapKey> keys,
-        SwapContext ctx) throws IgniteSpiException {
-        Space space = space(spaceName);
-
-        return space != null ? space.readAll(keys) : Collections.<SwapKey, byte[]>emptyMap();
-    }
-
-    /** {@inheritDoc} */
-    @Override public void remove(@Nullable String spaceName, SwapKey key, @Nullable IgniteInClosure<byte[]> c,
-        SwapContext ctx) throws IgniteSpiException {
-        Space space = space(spaceName);
-
-        if (space != null)
-            space.remove(key, c);
-    }
-
-    /** {@inheritDoc} */
-    @Override public void removeAll(@Nullable String spaceName, Collection<SwapKey> keys,
-        @Nullable IgniteBiInClosure<SwapKey, byte[]> c, SwapContext ctx) throws IgniteSpiException {
-        Space space = space(spaceName);
-
-        if (space != null)
-            space.removeAll(keys, c);
-    }
-
-    /** {@inheritDoc} */
-    @Override public void store(@Nullable String spaceName, SwapKey key, @Nullable byte[] val, SwapContext ctx)
-        throws IgniteSpiException {
-        ensureSpace(spaceName).store(key, val);
-    }
-
-    /** {@inheritDoc} */
-    @Override public void storeAll(@Nullable String spaceName, Map<SwapKey, byte[]> pairs, SwapContext ctx)
-        throws IgniteSpiException {
-        ensureSpace(spaceName).storeAll(pairs);
-    }
-
-    /** {@inheritDoc} */
-    @Override public void setListener(@Nullable SwapSpaceSpiListener evictLsnr) {
-        lsnr = evictLsnr;
-    }
-
-    /** {@inheritDoc} */
-    @Override public Collection<Integer> partitions(@Nullable String spaceName) throws IgniteSpiException {
-        Space space = space(spaceName);
-
-        return space != null ? space.partitions() : Collections.<Integer>emptyList();
-    }
-
-    /** {@inheritDoc} */
-    @Override public <K> IgniteSpiCloseableIterator<K> keyIterator(@Nullable String spaceName, SwapContext ctx)
-        throws IgniteSpiException {
-        return ensureSpace(spaceName).keyIterator();
-    }
-
-    /** {@inheritDoc} */
-    @Override public IgniteSpiCloseableIterator<Map.Entry<byte[], byte[]>> rawIterator(@Nullable String spaceName)
-        throws IgniteSpiException {
-        return ensureSpace(spaceName).rawIterator();
-    }
-
-    /** {@inheritDoc} */
-    @Override public IgniteSpiCloseableIterator<Map.Entry<byte[], byte[]>> rawIterator(@Nullable String spaceName, int part)
-        throws IgniteSpiException {
-        return ensureSpace(spaceName).rawIterator(part);
-    }
-
-    /**
-     * @param spaceName Space name.
-     * @return Space object.
-     */
-    @Nullable private Space space(String spaceName) {
-        return spaces.get(spaceName);
-    }
-
-    /**
-     * Gets space, creates if does not exist.
-     *
-     * @param spaceName Space name.
-     * @return Space.
-     */
-    private Space ensureSpace(String spaceName) {
-        Space space = spaces.get(spaceName);
-
-        if (space == null)
-            space = F.addIfAbsent(spaces, spaceName, new Space(spaceName));
-
-        return space;
-    }
-
-    private void fireEvent(int evtType, String spaceName, @Nullable byte[] key) {
-        SwapSpaceSpiListener lsnr0 = lsnr;
-
-        if (lsnr0 != null)
-            lsnr0.onSwapEvent(evtType, spaceName, key);
-    }
-
-    private class Space {
-        /** Data storage. */
-        private ConcurrentMap<SwapKey, byte[]> data = new ConcurrentHashMap8<>();
-
-        private final String name;
-
-        /**
-         * @param name Space name.
-         */
-        private Space(String name) {
-            this.name = name;
-        }
-
-        /**
-         * Clears space.
-         */
-        public void clear() {
-            data.clear();
-
-            fireEvent(EVT_SWAP_SPACE_CLEARED, name, null);
-        }
-
-        /**
-         * @return Space size.
-         */
-        public long size() {
-            return data.size();
-        }
-
-        /**
-         * @return Space size.
-         */
-        public long count() {
-            return data.size();
-        }
-
-        /**
-         * @param key Key to read.
-         * @return Read bytes.
-         */
-        public byte[] read(SwapKey key) {
-            byte[] bytes = data.get(key);
-
-            fireEvent(EVT_SWAP_SPACE_DATA_READ, name, key.keyBytes());
-
-            return bytes;
-        }
-
-        /**
-         * @param keys Keys to read.
-         * @return Read keys.
-         */
-        public Map<SwapKey, byte[]> readAll(Iterable<SwapKey> keys) {
-            Map<SwapKey, byte[]> res = new HashMap<>();
-
-            for (SwapKey key : keys) {
-                byte[] val = data.get(key);
-
-                if (val != null) {
-                    res.put(key, val);
-
-                    fireEvent(EVT_SWAP_SPACE_DATA_READ, name, key.keyBytes());
-                }
-            }
-
-            return res;
-        }
-
-        /**
-         * @param key Key to remove.
-         * @param c Closure.
-         */
-        public void remove(SwapKey key, IgniteInClosure<byte[]> c) {
-            byte[] val = data.remove(key);
-
-            if (val != null) {
-                c.apply(val);
-
-                fireEvent(EVT_SWAP_SPACE_DATA_REMOVED, name, key.keyBytes());
-            }
-        }
-
-        /**
-         * @param keys Keys to remove.
-         * @param c Closure to apply for removed values.
-         */
-        public void removeAll(Iterable<SwapKey> keys, IgniteBiInClosure<SwapKey, byte[]> c) {
-            for (SwapKey key : keys) {
-                byte[] val = data.remove(key);
-
-                if (val != null) {
-                    c.apply(key, val);
-
-                    fireEvent(EVT_SWAP_SPACE_DATA_REMOVED, name, key.keyBytes());
-                }
-            }
-        }
-
-        /**
-         * @param key Key to store.
-         * @param val Value to store.
-         */
-        public void store(SwapKey key, byte[] val) {
-            if (val != null) {
-                data.put(key, val);
-
-                fireEvent(EVT_SWAP_SPACE_DATA_STORED, name, key.keyBytes());
-            }
-            else {
-                val = data.remove(key);
-
-                if (val != null)
-                    fireEvent(EVT_SWAP_SPACE_DATA_REMOVED, name, key.keyBytes());
-            }
-        }
-
-        /**
-         * @param pairs Values to store.
-         */
-        public void storeAll(Map<SwapKey, byte[]> pairs) {
-            for (Map.Entry<SwapKey, byte[]> entry : pairs.entrySet()) {
-                SwapKey key = entry.getKey();
-                byte[] val = entry.getValue();
-
-                store(key, val);
-            }
-        }
-
-        /**
-         * @return Partitions in space.
-         */
-        public Collection<Integer> partitions() {
-            Collection<Integer> parts = new HashSet<>();
-
-            for (SwapKey key : data.keySet())
-                parts.add(key.partition());
-
-            return parts;
-        }
-
-        public <K> IgniteSpiCloseableIterator<K> keyIterator() {
-            final Iterator<SwapKey> it = data.keySet().iterator();
-
-            return new IgniteSpiCloseableIterator<K>() {
-                @Override public void close() {
-                    // No-op.
-                }
-
-                @Override public boolean hasNext() {
-                    return it.hasNext();
-                }
-
-                @Override public K next() {
-                    SwapKey next = it.next();
-
-                    return (K)next.key();
-                }
-
-                @Override public void remove() {
-                    it.remove();
-                }
-            };
-        }
-
-        public IgniteSpiCloseableIterator<Map.Entry<byte[], byte[]>> rawIterator() {
-            final Iterator<Map.Entry<SwapKey, byte[]>> it = data.entrySet().iterator();
-
-            return new IgniteSpiCloseableIterator<Map.Entry<byte[], byte[]>>() {
-                @Override public void close() {
-                    // No-op.
-                }
-
-                @Override public boolean hasNext() {
-                    return it.hasNext();
-                }
-
-                @Override public Map.Entry<byte[], byte[]> next() {
-                    final Map.Entry<SwapKey, byte[]> next = it.next();
-
-                    return new Map.Entry<byte[], byte[]>() {
-                        @Override public byte[] getKey() {
-                            return next.getKey().keyBytes();
-                        }
-
-                        @Override public byte[] getValue() {
-                            return next.getValue();
-                        }
-
-                        @Override public byte[] setValue(byte[] val) {
-                            return data.put(next.getKey(), val);
-                        }
-                    };
-                }
-
-                @Override public void remove() {
-                    it.remove();
-                }
-            };
-        }
-
-        public IgniteSpiCloseableIterator<Map.Entry<byte[], byte[]>> rawIterator(final int part) {
-            final Iterator<Map.Entry<SwapKey, byte[]>> it = data.entrySet().iterator();
-
-            return new IgniteSpiCloseableIterator<Map.Entry<byte[], byte[]>>() {
-                /** Next entry in this iterator. */
-                private Map.Entry<SwapKey, byte[]> next;
-
-                private Map.Entry<SwapKey, byte[]> cur;
-
-                {
-                    advance();
-                }
-
-                @Override public void close() {
-                    // No-op.
-                }
-
-                @Override public boolean hasNext() {
-                    return next != null;
-                }
-
-                @Override public Map.Entry<byte[], byte[]> next() {
-                    if (next == null)
-                        throw new NoSuchElementException();
-
-                    final Map.Entry<SwapKey, byte[]> ret = next;
-
-                    cur = ret;
-
-                    advance();
-
-                    return new Map.Entry<byte[], byte[]>() {
-                        @Override public byte[] getKey() {
-                            return ret.getKey().keyBytes();
-                        }
-
-                        @Override public byte[] getValue() {
-                            return ret.getValue();
-                        }
-
-                        @Override public byte[] setValue(byte[] val) {
-                            return data.put(ret.getKey(), val);
-                        }
-                    };
-                }
-
-                @Override public void remove() {
-                    if (cur == null)
-                        throw new IllegalStateException();
-
-                    data.remove(cur.getKey(), cur.getValue());
-                }
-
-                private void advance() {
-                    while (it.hasNext()) {
-                        Map.Entry<SwapKey, byte[]> entry = it.next();
-
-                        if(entry.getKey().partition() == part) {
-                            cur = next;
-
-                            next = entry;
-
-                            return;
-                        }
-                    }
-
-                    next = null;
-                }
-            };
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/ef258ece/modules/core/src/test/java/org/gridgain/grid/spi/swapspace/noop/GridNoopSwapSpaceSpiSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/gridgain/grid/spi/swapspace/noop/GridNoopSwapSpaceSpiSelfTest.java b/modules/core/src/test/java/org/gridgain/grid/spi/swapspace/noop/GridNoopSwapSpaceSpiSelfTest.java
deleted file mode 100644
index d7b90de..0000000
--- a/modules/core/src/test/java/org/gridgain/grid/spi/swapspace/noop/GridNoopSwapSpaceSpiSelfTest.java
+++ /dev/null
@@ -1,53 +0,0 @@
-/* @java.file.header */
-
-/*  _________        _____ __________________        _____
- *  __  ____/___________(_)______  /__  ____/______ ____(_)_______
- *  _  / __  __  ___/__  / _  __  / _  / __  _  __ `/__  / __  __ \
- *  / /_/ /  _  /    _  /  / /_/ /  / /_/ /  / /_/ / _  /  _  / / /
- *  \____/   /_/     /_/   \_,__/   \____/   \__,_/  /_/   /_/ /_/
- */
-
-package org.gridgain.grid.spi.swapspace.noop;
-
-import org.apache.ignite.*;
-import org.apache.ignite.configuration.*;
-import org.apache.ignite.spi.discovery.tcp.*;
-import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.*;
-import org.gridgain.grid.spi.swapspace.*;
-import org.gridgain.testframework.junits.common.*;
-
-/**
- * Tests for "noop" realization of {@link org.gridgain.grid.spi.swapspace.SwapSpaceSpi}.
- */
-public class GridNoopSwapSpaceSpiSelfTest extends GridCommonAbstractTest {
-    /** {@inheritDoc} */
-    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
-        IgniteConfiguration cfg = super.getConfiguration(gridName);
-
-        TcpDiscoverySpi disco = new TcpDiscoverySpi();
-
-        disco.setIpFinder(new TcpDiscoveryVmIpFinder(true));
-
-        cfg.setDiscoverySpi(disco);
-
-        return cfg;
-    }
-
-    /**
-     * @throws Exception If test failed.
-     */
-    public void testWithoutCacheUseNoopSwapSapce() throws Exception {
-        try {
-            Ignite ignite = startGrid(1);
-
-            SwapSpaceSpi spi = ignite.configuration().getSwapSpaceSpi();
-
-            assertNotNull(spi);
-
-            assertTrue(spi instanceof NoopSwapSpaceSpi);
-        }
-        finally {
-            stopAllGrids();
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/ef258ece/modules/core/src/test/java/org/gridgain/grid/spi/swapspace/package.html
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/gridgain/grid/spi/swapspace/package.html b/modules/core/src/test/java/org/gridgain/grid/spi/swapspace/package.html
deleted file mode 100644
index 5cad80a..0000000
--- a/modules/core/src/test/java/org/gridgain/grid/spi/swapspace/package.html
+++ /dev/null
@@ -1,15 +0,0 @@
-<!DOCTYPE html PUBLIC "-//W3C//DTD HTML 4.01 Transitional//EN" "http://www.w3.org/TR/html4/loose.dtd">
-<!--
-    @html.file.header
-    _________        _____ __________________        _____
-    __  ____/___________(_)______  /__  ____/______ ____(_)_______
-    _  / __  __  ___/__  / _  __  / _  / __  _  __ `/__  / __  __ \
-    / /_/ /  _  /    _  /  / /_/ /  / /_/ /  / /_/ / _  /  _  / / /
-    \____/   /_/     /_/   \_,__/   \____/   \__,_/  /_/   /_/ /_/
--->
-<html>
-<body>
-    <!-- Package description. -->
-    Contains internal tests or test related classes and interfaces.
-</body>
-</html>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/ef258ece/modules/core/src/test/java/org/gridgain/loadtests/swap/GridSwapEvictAllBenchmark.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/gridgain/loadtests/swap/GridSwapEvictAllBenchmark.java b/modules/core/src/test/java/org/gridgain/loadtests/swap/GridSwapEvictAllBenchmark.java
index 0f5a559..6d7a0e3 100644
--- a/modules/core/src/test/java/org/gridgain/loadtests/swap/GridSwapEvictAllBenchmark.java
+++ b/modules/core/src/test/java/org/gridgain/loadtests/swap/GridSwapEvictAllBenchmark.java
@@ -19,7 +19,7 @@ import org.gridgain.grid.cache.store.*;
 import org.apache.ignite.spi.discovery.tcp.*;
 import org.apache.ignite.spi.discovery.tcp.ipfinder.*;
 import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.*;
-import org.gridgain.grid.spi.swapspace.file.*;
+import org.apache.ignite.spi.swapspace.file.*;
 import org.gridgain.grid.util.typedef.*;
 import org.gridgain.loadtests.util.*;
 import org.gridgain.testframework.*;

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/ef258ece/modules/core/src/test/java/org/gridgain/testframework/GridSpiTestContext.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/gridgain/testframework/GridSpiTestContext.java b/modules/core/src/test/java/org/gridgain/testframework/GridSpiTestContext.java
index bc5384d..a0be5f1 100644
--- a/modules/core/src/test/java/org/gridgain/testframework/GridSpiTestContext.java
+++ b/modules/core/src/test/java/org/gridgain/testframework/GridSpiTestContext.java
@@ -18,7 +18,7 @@ import org.gridgain.grid.kernal.managers.communication.*;
 import org.gridgain.grid.kernal.managers.eventstorage.*;
 import org.gridgain.grid.security.*;
 import org.apache.ignite.spi.discovery.*;
-import org.gridgain.grid.spi.swapspace.*;
+import org.apache.ignite.spi.swapspace.*;
 import org.gridgain.grid.util.direct.*;
 import org.gridgain.grid.util.typedef.*;
 import org.jetbrains.annotations.*;

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/ef258ece/modules/core/src/test/java/org/gridgain/testsuites/GridSpiSwapSpaceSelfTestSuite.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/gridgain/testsuites/GridSpiSwapSpaceSelfTestSuite.java b/modules/core/src/test/java/org/gridgain/testsuites/GridSpiSwapSpaceSelfTestSuite.java
index e5017d5..42afc7f 100644
--- a/modules/core/src/test/java/org/gridgain/testsuites/GridSpiSwapSpaceSelfTestSuite.java
+++ b/modules/core/src/test/java/org/gridgain/testsuites/GridSpiSwapSpaceSelfTestSuite.java
@@ -10,8 +10,8 @@
 package org.gridgain.testsuites;
 
 import junit.framework.*;
-import org.gridgain.grid.spi.swapspace.file.*;
-import org.gridgain.grid.spi.swapspace.noop.*;
+import org.apache.ignite.spi.swapspace.file.*;
+import org.apache.ignite.spi.swapspace.noop.*;
 
 /**
  *

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/ef258ece/modules/indexing/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheAbstractQuerySelfTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheAbstractQuerySelfTest.java b/modules/indexing/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheAbstractQuerySelfTest.java
index 4efc310..3e20845 100644
--- a/modules/indexing/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheAbstractQuerySelfTest.java
+++ b/modules/indexing/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheAbstractQuerySelfTest.java
@@ -24,7 +24,7 @@ import org.apache.ignite.spi.discovery.tcp.*;
 import org.apache.ignite.spi.discovery.tcp.ipfinder.*;
 import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.*;
 import org.gridgain.grid.spi.indexing.h2.*;
-import org.gridgain.grid.spi.swapspace.file.*;
+import org.apache.ignite.spi.swapspace.file.*;
 import org.gridgain.grid.util.tostring.*;
 import org.gridgain.grid.util.typedef.*;
 import org.gridgain.grid.util.typedef.internal.*;

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/ef258ece/modules/indexing/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheOffHeapAndSwapSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheOffHeapAndSwapSelfTest.java b/modules/indexing/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheOffHeapAndSwapSelfTest.java
index e5a72f2..88a18fe 100644
--- a/modules/indexing/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheOffHeapAndSwapSelfTest.java
+++ b/modules/indexing/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheOffHeapAndSwapSelfTest.java
@@ -20,7 +20,7 @@ import org.apache.ignite.spi.discovery.tcp.*;
 import org.apache.ignite.spi.discovery.tcp.ipfinder.*;
 import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.*;
 import org.gridgain.grid.spi.indexing.h2.*;
-import org.gridgain.grid.spi.swapspace.file.*;
+import org.apache.ignite.spi.swapspace.file.*;
 import org.gridgain.grid.util.lang.*;
 import org.gridgain.grid.util.typedef.*;
 import org.gridgain.testframework.junits.common.*;

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/ef258ece/modules/indexing/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheQueryMultiThreadedSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheQueryMultiThreadedSelfTest.java b/modules/indexing/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheQueryMultiThreadedSelfTest.java
index 8a7522f..70a4f3e 100644
--- a/modules/indexing/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheQueryMultiThreadedSelfTest.java
+++ b/modules/indexing/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheQueryMultiThreadedSelfTest.java
@@ -24,7 +24,7 @@ import org.apache.ignite.spi.discovery.tcp.*;
 import org.apache.ignite.spi.discovery.tcp.ipfinder.*;
 import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.*;
 import org.gridgain.grid.spi.indexing.h2.*;
-import org.gridgain.grid.spi.swapspace.file.*;
+import org.apache.ignite.spi.swapspace.file.*;
 import org.gridgain.grid.util.typedef.*;
 import org.gridgain.testframework.junits.common.*;
 import org.jetbrains.annotations.*;

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/ef258ece/modules/indexing/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheSwapSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheSwapSelfTest.java b/modules/indexing/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheSwapSelfTest.java
index dfa298a..1df68e9 100644
--- a/modules/indexing/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheSwapSelfTest.java
+++ b/modules/indexing/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheSwapSelfTest.java
@@ -19,8 +19,8 @@ import org.gridgain.grid.cache.query.*;
 import org.apache.ignite.spi.discovery.tcp.*;
 import org.apache.ignite.spi.discovery.tcp.ipfinder.*;
 import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.*;
-import org.gridgain.grid.spi.swapspace.*;
-import org.gridgain.grid.spi.swapspace.noop.*;
+import org.apache.ignite.spi.swapspace.*;
+import org.apache.ignite.spi.swapspace.noop.*;
 import org.gridgain.grid.util.typedef.*;
 import org.gridgain.testframework.junits.common.*;
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/ef258ece/modules/indexing/src/test/java/org/gridgain/grid/kernal/processors/cache/GridIndexingWithNoopSwapSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/gridgain/grid/kernal/processors/cache/GridIndexingWithNoopSwapSelfTest.java b/modules/indexing/src/test/java/org/gridgain/grid/kernal/processors/cache/GridIndexingWithNoopSwapSelfTest.java
index 02355e5..9cdb1c7 100644
--- a/modules/indexing/src/test/java/org/gridgain/grid/kernal/processors/cache/GridIndexingWithNoopSwapSelfTest.java
+++ b/modules/indexing/src/test/java/org/gridgain/grid/kernal/processors/cache/GridIndexingWithNoopSwapSelfTest.java
@@ -19,7 +19,7 @@ import org.apache.ignite.spi.discovery.tcp.*;
 import org.apache.ignite.spi.discovery.tcp.ipfinder.*;
 import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.*;
 import org.gridgain.grid.spi.indexing.h2.*;
-import org.gridgain.grid.spi.swapspace.noop.*;
+import org.apache.ignite.spi.swapspace.noop.*;
 import org.gridgain.testframework.junits.common.*;
 
 import java.util.*;

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/ef258ece/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 15de6ec..e00d6f3 100644
--- a/pom.xml
+++ b/pom.xml
@@ -671,7 +671,7 @@
                                         </group>
                                         <group>
                                             <title>SPI: Swap Space</title>
-                                            <packages>org.gridgain.grid.spi.swapspace:org.gridgain.grid.spi.swapspace.file:org.gridgain.grid.spi.swapspace.noop</packages>
+                                            <packages>org.gridgain.grid.spi.swapspace:org.apache.ignite.spi.swapspace.file:org.apache.ignite.spi.swapspace.noop</packages>
                                         </group>
                                         <group>
                                             <title>SPI: Discovery</title>
@@ -873,7 +873,7 @@
                                         </group>
                                         <group>
                                             <title>SPI: Swap Space</title>
-                                            <packages>org.gridgain.grid.spi.swapspace:org.gridgain.grid.spi.swapspace.file:org.gridgain.grid.spi.swapspace.noop</packages>
+                                            <packages>org.gridgain.grid.spi.swapspace:org.apache.ignite.spi.swapspace.file:org.apache.ignite.spi.swapspace.noop</packages>
                                         </group>
                                         <group>
                                             <title>SPI: Discovery</title>


[20/20] incubator-ignite git commit: # Renaming

Posted by sb...@apache.org.
# Renaming


Project: http://git-wip-us.apache.org/repos/asf/incubator-ignite/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-ignite/commit/8c20bddb
Tree: http://git-wip-us.apache.org/repos/asf/incubator-ignite/tree/8c20bddb
Diff: http://git-wip-us.apache.org/repos/asf/incubator-ignite/diff/8c20bddb

Branch: refs/heads/master
Commit: 8c20bddb3ddc4beaaace880e5dc744e59a878bc5
Parents: ef258ec
Author: sboikov <sb...@gridgain.com>
Authored: Fri Dec 5 17:17:20 2014 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Fri Dec 5 17:17:20 2014 +0300

----------------------------------------------------------------------
 .../src/main/java/org/apache/ignite/Ignite.java |   2 +-
 .../java/org/apache/ignite/IgniteState.java     |   2 +-
 .../configuration/IgniteConfiguration.java      |   6 +-
 .../events/IgniteAuthenticationEvent.java       |   2 +-
 .../ignite/events/IgniteAuthorizationEvent.java |   2 +-
 .../ignite/events/IgniteSecureSessionEvent.java |   2 +-
 .../ignite/plugin/security/GridSecurity.java    |  89 +++++++++
 .../security/GridSecurityCredentials.java       | 196 +++++++++++++++++++
 .../GridSecurityCredentialsBasicProvider.java   |  36 ++++
 .../GridSecurityCredentialsProvider.java        |  41 ++++
 .../plugin/security/GridSecurityException.java  |  50 +++++
 .../plugin/security/GridSecurityPermission.java |  61 ++++++
 .../security/GridSecurityPermissionSet.java     |  58 ++++++
 .../plugin/security/GridSecuritySubject.java    |  55 ++++++
 .../security/GridSecuritySubjectType.java       |  40 ++++
 .../apache/ignite/plugin/security/package.html  |  15 ++
 .../segmentation/GridSegmentationPolicy.java    |  40 ++++
 .../segmentation/GridSegmentationResolver.java  |  56 ++++++
 .../ignite/plugin/segmentation/package.html     |  15 ++
 .../org/apache/ignite/spi/IgniteSpiAdapter.java |   2 +-
 .../org/apache/ignite/spi/IgniteSpiContext.java |   2 +-
 .../AuthenticationAclProvider.java              |   2 +-
 .../authentication/AuthenticationContext.java   |   2 +-
 .../AuthenticationContextAdapter.java           |   2 +-
 .../spi/authentication/AuthenticationSpi.java   |   2 +-
 .../noop/NoopAuthenticationSpi.java             |   2 +-
 .../DiscoverySpiNodeAuthenticator.java          |   2 +-
 .../spi/discovery/tcp/TcpDiscoverySpi.java      |   2 +-
 .../java/org/apache/ignite/spi/package.html     |  15 ++
 .../spi/securesession/SecureSessionSpi.java     |   2 +-
 .../noop/NoopSecureSessionSpi.java              |   2 +-
 .../client/GridClientConfiguration.java         |   2 +-
 .../GridClientConnectionManagerAdapter.java     |   2 +-
 .../router/GridTcpRouterConfiguration.java      |   2 +-
 .../org/gridgain/grid/kernal/GridGainEx.java    |   4 +-
 .../org/gridgain/grid/kernal/GridKernal.java    |   2 +-
 .../grid/kernal/GridTaskFutureImpl.java         |   2 +-
 .../kernal/managers/GridManagerAdapter.java     |   2 +-
 .../discovery/GridDiscoveryManager.java         |   6 +-
 .../eventstorage/GridEventStorageManager.java   |   2 +-
 .../securesession/GridSecureSessionManager.java |   2 +-
 .../os/GridOsSecureSessionManager.java          |   2 +-
 .../security/GridAllowAllPermissionSet.java     |   2 +-
 .../managers/security/GridSecurityContext.java  |   2 +-
 .../managers/security/GridSecurityImpl.java     |   2 +-
 .../managers/security/GridSecurityManager.java  |   2 +-
 .../security/GridSecuritySubjectAdapter.java    |   2 +-
 .../security/os/GridOsSecurityManager.java      |   2 +-
 .../processors/cache/GridCacheAdapter.java      |   2 +-
 .../processors/cache/GridCacheContext.java      |   2 +-
 .../cache/GridCacheTxLocalAdapter.java          |   2 +-
 .../dht/atomic/GridDhtAtomicCache.java          |   2 +-
 .../dht/colocated/GridDhtColocatedCache.java    |   2 +-
 .../distributed/near/GridNearAtomicCache.java   |   2 +-
 .../near/GridNearTransactionalCache.java        |   2 +-
 .../local/atomic/GridLocalAtomicCache.java      |   2 +-
 .../cache/query/GridCacheQueryAdapter.java      |   2 +-
 .../GridCacheContinuousQueryAdapter.java        |   2 +-
 .../processors/rest/GridRestProcessor.java      |   4 +-
 .../processors/task/GridTaskProcessor.java      |   2 +-
 .../event/VisorGridAuthenticationEvent.java     |   2 +-
 .../event/VisorGridAuthorizationEvent.java      |   2 +-
 .../event/VisorGridSecuritySessionEvent.java    |   2 +-
 .../node/VisorSegmentationConfiguration.java    |   2 +-
 .../gridgain/grid/security/GridSecurity.java    |  89 ---------
 .../grid/security/GridSecurityCredentials.java  | 196 -------------------
 .../GridSecurityCredentialsBasicProvider.java   |  36 ----
 .../GridSecurityCredentialsProvider.java        |  41 ----
 .../grid/security/GridSecurityException.java    |  50 -----
 .../grid/security/GridSecurityPermission.java   |  61 ------
 .../security/GridSecurityPermissionSet.java     |  58 ------
 .../grid/security/GridSecuritySubject.java      |  55 ------
 .../grid/security/GridSecuritySubjectType.java  |  40 ----
 .../org/gridgain/grid/security/package.html     |  15 --
 .../segmentation/GridSegmentationPolicy.java    |  40 ----
 .../segmentation/GridSegmentationResolver.java  |  56 ------
 .../org/gridgain/grid/segmentation/package.html |  15 --
 .../java/org/gridgain/grid/spi/package.html     |  15 --
 .../GridAbstractDiscoverySelfTest.java          |   2 +-
 .../GridTcpDiscoverySpiStartStopSelfTest.java   |   2 +-
 .../grid/kernal/GridLifecycleAwareSelfTest.java |   2 +-
 .../testframework/GridSpiTestContext.java       |   2 +-
 .../junits/spi/GridSpiAbstractTest.java         |   2 +-
 .../http/jetty/GridJettyRestHandler.java        |   2 +-
 .../java/org/gridgain/grid/GridSpringBean.java  |   2 +-
 pom.xml                                         |   8 +-
 86 files changed, 834 insertions(+), 834 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/8c20bddb/modules/core/src/main/java/org/apache/ignite/Ignite.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/Ignite.java b/modules/core/src/main/java/org/apache/ignite/Ignite.java
index a6ff544..8b550f1 100644
--- a/modules/core/src/main/java/org/apache/ignite/Ignite.java
+++ b/modules/core/src/main/java/org/apache/ignite/Ignite.java
@@ -16,7 +16,7 @@ import org.apache.ignite.product.*;
 import org.gridgain.grid.*;
 import org.gridgain.grid.cache.*;
 import org.gridgain.grid.hadoop.*;
-import org.gridgain.grid.security.*;
+import org.apache.ignite.plugin.security.*;
 import org.gridgain.grid.util.typedef.*;
 import org.jetbrains.annotations.*;
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/8c20bddb/modules/core/src/main/java/org/apache/ignite/IgniteState.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/IgniteState.java b/modules/core/src/main/java/org/apache/ignite/IgniteState.java
index 5a067ae..4c3905e 100644
--- a/modules/core/src/main/java/org/apache/ignite/IgniteState.java
+++ b/modules/core/src/main/java/org/apache/ignite/IgniteState.java
@@ -9,7 +9,7 @@
 
 package org.apache.ignite;
 
-import org.gridgain.grid.segmentation.*;
+import org.apache.ignite.plugin.segmentation.*;
 import org.jetbrains.annotations.*;
 
 /**

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/8c20bddb/modules/core/src/main/java/org/apache/ignite/configuration/IgniteConfiguration.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/configuration/IgniteConfiguration.java b/modules/core/src/main/java/org/apache/ignite/configuration/IgniteConfiguration.java
index 73233e4..92bcdab 100644
--- a/modules/core/src/main/java/org/apache/ignite/configuration/IgniteConfiguration.java
+++ b/modules/core/src/main/java/org/apache/ignite/configuration/IgniteConfiguration.java
@@ -26,8 +26,8 @@ import org.gridgain.grid.cache.*;
 import org.gridgain.grid.dotnet.*;
 import org.gridgain.grid.hadoop.*;
 import org.gridgain.grid.kernal.managers.eventstorage.*;
-import org.gridgain.grid.security.*;
-import org.gridgain.grid.segmentation.*;
+import org.apache.ignite.plugin.security.*;
+import org.apache.ignite.plugin.segmentation.*;
 import org.apache.ignite.spi.checkpoint.*;
 import org.apache.ignite.spi.collision.*;
 import org.apache.ignite.spi.communication.*;
@@ -47,7 +47,7 @@ import java.net.*;
 import java.util.*;
 import java.util.concurrent.*;
 
-import static org.gridgain.grid.segmentation.GridSegmentationPolicy.*;
+import static org.apache.ignite.plugin.segmentation.GridSegmentationPolicy.*;
 
 /**
  * This class defines grid runtime configuration. This configuration is passed to

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/8c20bddb/modules/core/src/main/java/org/apache/ignite/events/IgniteAuthenticationEvent.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/events/IgniteAuthenticationEvent.java b/modules/core/src/main/java/org/apache/ignite/events/IgniteAuthenticationEvent.java
index 3c7e8d1..08922e8 100644
--- a/modules/core/src/main/java/org/apache/ignite/events/IgniteAuthenticationEvent.java
+++ b/modules/core/src/main/java/org/apache/ignite/events/IgniteAuthenticationEvent.java
@@ -10,7 +10,7 @@
 package org.apache.ignite.events;
 
 import org.apache.ignite.cluster.*;
-import org.gridgain.grid.security.*;
+import org.apache.ignite.plugin.security.*;
 import org.gridgain.grid.util.tostring.*;
 import org.gridgain.grid.util.typedef.internal.*;
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/8c20bddb/modules/core/src/main/java/org/apache/ignite/events/IgniteAuthorizationEvent.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/events/IgniteAuthorizationEvent.java b/modules/core/src/main/java/org/apache/ignite/events/IgniteAuthorizationEvent.java
index 7722c02..3156a79 100644
--- a/modules/core/src/main/java/org/apache/ignite/events/IgniteAuthorizationEvent.java
+++ b/modules/core/src/main/java/org/apache/ignite/events/IgniteAuthorizationEvent.java
@@ -10,7 +10,7 @@
 package org.apache.ignite.events;
 
 import org.apache.ignite.cluster.*;
-import org.gridgain.grid.security.*;
+import org.apache.ignite.plugin.security.*;
 import org.gridgain.grid.util.typedef.internal.*;
 
 /**

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/8c20bddb/modules/core/src/main/java/org/apache/ignite/events/IgniteSecureSessionEvent.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/events/IgniteSecureSessionEvent.java b/modules/core/src/main/java/org/apache/ignite/events/IgniteSecureSessionEvent.java
index 482ea8f..87aa34d 100644
--- a/modules/core/src/main/java/org/apache/ignite/events/IgniteSecureSessionEvent.java
+++ b/modules/core/src/main/java/org/apache/ignite/events/IgniteSecureSessionEvent.java
@@ -10,7 +10,7 @@
 package org.apache.ignite.events;
 
 import org.apache.ignite.cluster.*;
-import org.gridgain.grid.security.*;
+import org.apache.ignite.plugin.security.*;
 import org.gridgain.grid.util.typedef.internal.*;
 
 import java.util.*;

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/8c20bddb/modules/core/src/main/java/org/apache/ignite/plugin/security/GridSecurity.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/plugin/security/GridSecurity.java b/modules/core/src/main/java/org/apache/ignite/plugin/security/GridSecurity.java
new file mode 100644
index 0000000..bff63e9
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/plugin/security/GridSecurity.java
@@ -0,0 +1,89 @@
+/* @java.file.header */
+
+/*  _________        _____ __________________        _____
+ *  __  ____/___________(_)______  /__  ____/______ ____(_)_______
+ *  _  / __  __  ___/__  / _  __  / _  / __  _  __ `/__  / __  __ \
+ *  / /_/ /  _  /    _  /  / /_/ /  / /_/ /  / /_/ / _  /  _  / / /
+ *  \____/   /_/     /_/   \_,__/   \____/   \__,_/  /_/   /_/ /_/
+ */
+
+package org.apache.ignite.plugin.security;
+
+import org.gridgain.grid.*;
+
+import java.util.*;
+
+/**
+ * Grid security facade. This facade contains information about authenticated subjects
+ * currently logged in to grid together with their permission sets.
+ * <p>
+ * You can get an instance of security facade from {@link org.apache.ignite.Ignite#security()} method.
+ * <h1 class="header">Grid Nodes vs Remote Clients</h1>
+ * When security is enabled, both grid nodes and remote clients must be authenticated.
+ * For grid nodes, authentication parameters are specified in grid configuration via
+ * {@link org.apache.ignite.configuration.IgniteConfiguration#getSecurityCredentialsProvider()} provider. Here is an example
+ * of how a simple user name and password may be provided:
+ * <pre class="brush: java">
+ *     GridConfiguration cfg = new GridConfiguration();
+ *
+ *     GridSecurityCredentials creds = new GridSecurityCredentials("username", "password");
+ *
+ *     cfg.setSecurityCredentialsProvider(new GridSecurityCredentialsBasicProvider(creds));
+ *
+ *     Grid grid = GridGain.start(cfg);
+ * </pre>
+ * For remote Java client, configuration is provided in a similar way by specifying
+ * {@code GridClientConfiguration.setSecurityCredentialsProvider(...)} property.
+ * <p>
+ * For remote C++ and .NET clients, security credentials are provided in configuration
+ * as well in the form of {@code "username:password"} string.
+ * <h1 class="header">Authentication And Authorization</h1>
+ * Node or client authentication happens in {@link org.apache.ignite.spi.authentication.AuthenticationSpi}. Upon successful
+ * authentication, the SPI will return list of permissions for authenticated subject.
+ * <p>
+ * GridGain ships with following authentication SPIs out of the box:
+ * <ul>
+ * <li>{@code GridJaasAuthenticationSpi} - provides authentication based on JAAS standard.</li>
+ * <li>{@code GridPasscodeAuthenticationSpi} - basic username and password authentication.</li>
+ * </ul>
+ * All permissions supported by GridGain are provided in {@link GridSecurityPermission} enum. Permissions
+ * are specified on per-cache or per-task level (wildcards are allowed). Authentication SPIs should usually
+ * (although not required) specify security permissions in the following JSON format:
+ * <pre class="brush: text">
+ * {
+ *     {
+ *         "cache":"partitioned",
+ *         "permissions":["CACHE_PUT", "CACHE_REMOVE", "CACHE_READ"]
+ *     },
+ *     {
+ *         "cache":"*",
+ *         "permissions":["CACHE_READ"]
+ *     },
+ *     {
+ *         "task":"org.mytasks.*",
+ *         "permissions":["TASK_EXECUTE"]
+ *     },
+ *     {
+ *         "system":["EVENTS_ENABLE", "ADMIN_VIEW"]
+ *     }
+ *     "defaultAllow":"false"
+ * }
+ * </pre>
+ * Refer to documentation of available authentication SPIs for more information.
+ */
+public interface GridSecurity {
+    /**
+     * Gets collection of authenticated subjects together with their permissions.
+     *
+     * @return Collection of authenticated subjects.
+     */
+    public Collection<GridSecuritySubject> authenticatedSubjects() throws GridException;
+
+    /**
+     * Gets security subject based on subject ID.
+     *
+     * @param subjId Subject ID.
+     * @return Authorized security subject.
+     */
+    public GridSecuritySubject authenticatedSubject(UUID subjId) throws GridException;
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/8c20bddb/modules/core/src/main/java/org/apache/ignite/plugin/security/GridSecurityCredentials.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/plugin/security/GridSecurityCredentials.java b/modules/core/src/main/java/org/apache/ignite/plugin/security/GridSecurityCredentials.java
new file mode 100644
index 0000000..2f8a885
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/plugin/security/GridSecurityCredentials.java
@@ -0,0 +1,196 @@
+/* @java.file.header */
+
+/*  _________        _____ __________________        _____
+ *  __  ____/___________(_)______  /__  ____/______ ____(_)_______
+ *  _  / __  __  ___/__  / _  __  / _  / __  _  __ `/__  / __  __ \
+ *  / /_/ /  _  /    _  /  / /_/ /  / /_/ /  / /_/ / _  /  _  / / /
+ *  \____/   /_/     /_/   \_,__/   \____/   \__,_/  /_/   /_/ /_/
+ */
+
+package org.apache.ignite.plugin.security;
+
+import org.apache.ignite.portables.*;
+import org.gridgain.grid.util.tostring.*;
+import org.gridgain.grid.util.typedef.*;
+import org.gridgain.grid.util.typedef.internal.*;
+import org.jetbrains.annotations.*;
+
+import java.io.*;
+
+/**
+ * Security credentials used for client or node authentication. Security credentials
+ * are provided by {@link GridSecurityCredentialsProvider} which is specified on
+ * client or node startup in configuration.
+ * <p>
+ * For grid node, security credentials provider is specified in
+ * {@link org.apache.ignite.configuration.IgniteConfiguration#setSecurityCredentialsProvider(GridSecurityCredentialsProvider)}
+ * configuration property. For Java clients, you can provide credentials in
+ * {@code GridClientConfiguration.setSecurityCredentialsProvider(...)} method.
+ * <p>
+ * Getting credentials through {@link GridSecurityCredentialsProvider} abstraction allows
+ * users to provide custom implementations for storing user names and passwords in their
+ * environment, possibly in encrypted format. GridGain comes with
+ * {@link GridSecurityCredentialsBasicProvider} which simply provides
+ * the passed in {@code login} and {@code password} when encryption or custom logic is not required.
+ * <p>
+ * In addition to {@code login} and {@code password}, security credentials allow for
+ * specifying {@link #setUserObject(Object) userObject} as well, which can be used
+ * to pass in any additional information required for authentication.
+ */
+public class GridSecurityCredentials implements Externalizable, PortableMarshalAware {
+    /** */
+    private static final long serialVersionUID = -2655741071578326256L;
+
+    /** Login. */
+    private Object login;
+
+    /** Password. */
+    @GridToStringExclude
+    private Object password;
+
+    /** Additional user object. */
+    @GridToStringExclude
+    private Object userObj;
+
+    /**
+     * Empty constructor required by {@link Externalizable}.
+     */
+    public GridSecurityCredentials() {
+        // No-op.
+    }
+
+    /**
+     * Constructs security credentials based on {@code login} and {@code password}.
+     *
+     * @param login Login.
+     * @param password Password.
+     */
+    public GridSecurityCredentials(String login, String password) {
+        this.login = login;
+        this.password = password;
+    }
+
+    /**
+     * Constructs security credentials based on {@code login}, {@code password},
+     * and custom user object.
+     *
+     * @param login Login.
+     * @param password Password.
+     * @param userObj User object.
+     */
+    public GridSecurityCredentials(String login, String password, @Nullable Object userObj) {
+        this.login = login;
+        this.password = password;
+        this.userObj = userObj;
+    }
+
+    /**
+     * Gets login.
+     *
+     * @return Login.
+     */
+    public Object getLogin() {
+        return login;
+    }
+
+    /**
+     * Sets login.
+     *
+     * @param login Login.
+     */
+    public void setLogin(Object login) {
+        this.login = login;
+    }
+
+    /**
+     * Gets password.
+     *
+     * @return Password.
+     */
+    public Object getPassword() {
+        return password;
+    }
+
+    /**
+     * Sets password.
+     *
+     * @param password Password.
+     */
+    public void setPassword(Object password) {
+        this.password = password;
+    }
+
+    /**
+     * Gets user-specific object.
+     *
+     * @return User object.
+     */
+    @Nullable public Object getUserObject() {
+        return userObj;
+    }
+
+    /**
+     * Sets user-specific object.
+     *
+     * @param userObj User object.
+     */
+    public void setUserObject(@Nullable Object userObj) {
+        this.userObj = userObj;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeExternal(ObjectOutput out) throws IOException {
+        out.writeObject(login);
+        out.writeObject(password);
+        out.writeObject(userObj);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException {
+        login = in.readObject();
+        password = in.readObject();
+        userObj = in.readObject();
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean equals(Object o) {
+        if (this == o)
+            return true;
+
+        if (!(o instanceof GridSecurityCredentials))
+            return false;
+
+        GridSecurityCredentials that = (GridSecurityCredentials)o;
+
+        return F.eq(login, that.login) && F.eq(password, that.password) && F.eq(userObj, that.userObj);
+    }
+
+    /** {@inheritDoc} */
+    @Override public int hashCode() {
+        int res = login != null ? login.hashCode() : 0;
+
+        res = 31 * res + (password != null ? password.hashCode() : 0);
+        res = 31 * res + (userObj != null ? userObj.hashCode() : 0);
+
+        return res;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writePortable(PortableWriter writer) throws PortableException {
+        writer.rawWriter().writeObject(login);
+        writer.rawWriter().writeObject(password);
+        writer.rawWriter().writeObject(userObj);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void readPortable(PortableReader reader) throws PortableException {
+        login = reader.rawReader().readObject();
+        password = reader.rawReader().readObject();
+        userObj = reader.rawReader().readObject();
+    }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return S.toString(GridSecurityCredentials.class, this);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/8c20bddb/modules/core/src/main/java/org/apache/ignite/plugin/security/GridSecurityCredentialsBasicProvider.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/plugin/security/GridSecurityCredentialsBasicProvider.java b/modules/core/src/main/java/org/apache/ignite/plugin/security/GridSecurityCredentialsBasicProvider.java
new file mode 100644
index 0000000..1cc184b
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/plugin/security/GridSecurityCredentialsBasicProvider.java
@@ -0,0 +1,36 @@
+/* @java.file.header */
+
+/*  _________        _____ __________________        _____
+ *  __  ____/___________(_)______  /__  ____/______ ____(_)_______
+ *  _  / __  __  ___/__  / _  __  / _  / __  _  __ `/__  / __  __ \
+ *  / /_/ /  _  /    _  /  / /_/ /  / /_/ /  / /_/ / _  /  _  / / /
+ *  \____/   /_/     /_/   \_,__/   \____/   \__,_/  /_/   /_/ /_/
+ */
+
+package org.apache.ignite.plugin.security;
+
+import org.gridgain.grid.*;
+
+/**
+ * Basic implementation for {@link GridSecurityCredentialsProvider}. Use it
+ * when custom logic for storing security credentials is not required and it
+ * is OK to specify credentials directly in configuration.
+ */
+public class GridSecurityCredentialsBasicProvider implements GridSecurityCredentialsProvider {
+    /** */
+    private GridSecurityCredentials cred;
+
+    /**
+     * Constructs security credentials provider based on security credentials passed in.
+     *
+     * @param cred Security credentials.
+     */
+    public GridSecurityCredentialsBasicProvider(GridSecurityCredentials cred) {
+        this.cred = cred;
+    }
+
+    /** {@inheritDoc} */
+    @Override public GridSecurityCredentials credentials() throws GridException {
+        return cred;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/8c20bddb/modules/core/src/main/java/org/apache/ignite/plugin/security/GridSecurityCredentialsProvider.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/plugin/security/GridSecurityCredentialsProvider.java b/modules/core/src/main/java/org/apache/ignite/plugin/security/GridSecurityCredentialsProvider.java
new file mode 100644
index 0000000..2cfb328
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/plugin/security/GridSecurityCredentialsProvider.java
@@ -0,0 +1,41 @@
+/* @java.file.header */
+
+/*  _________        _____ __________________        _____
+ *  __  ____/___________(_)______  /__  ____/______ ____(_)_______
+ *  _  / __  __  ___/__  / _  __  / _  / __  _  __ `/__  / __  __ \
+ *  / /_/ /  _  /    _  /  / /_/ /  / /_/ /  / /_/ / _  /  _  / / /
+ *  \____/   /_/     /_/   \_,__/   \____/   \__,_/  /_/   /_/ /_/
+ */
+
+package org.apache.ignite.plugin.security;
+
+import org.gridgain.grid.*;
+
+/**
+ * Security credentials provider for specifying security credentials.
+ * Security credentials used for client or node authentication.
+ * <p>
+ * For grid node, security credentials provider is specified in
+ * {@link org.apache.ignite.configuration.IgniteConfiguration#setSecurityCredentialsProvider(GridSecurityCredentialsProvider)}
+ * configuration property. For Java clients, you can provide credentials in
+ * {@code GridClientConfiguration.setSecurityCredentialsProvider(...)} method.
+ * <p>
+ * Getting credentials through {@link GridSecurityCredentialsProvider} abstraction allows
+ * users to provide custom implementations for storing user names and passwords in their
+ * environment, possibly in encrypted format. GridGain comes with
+ * {@link GridSecurityCredentialsBasicProvider} which simply provides
+ * the passed in {@code login} and {@code password} when encryption or custom logic is not required.
+ * <p>
+ * In addition to {@code login} and {@code password}, security credentials allow for
+ * specifying {@link GridSecurityCredentials#setUserObject(Object) userObject} as well, which can be used
+ * to pass in any additional information required for authentication.
+ */
+public interface GridSecurityCredentialsProvider {
+    /**
+     * Gets security credentials.
+     *
+     * @return Security credentials.
+     * @throws GridException If failed.
+     */
+    public GridSecurityCredentials credentials() throws GridException;
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/8c20bddb/modules/core/src/main/java/org/apache/ignite/plugin/security/GridSecurityException.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/plugin/security/GridSecurityException.java b/modules/core/src/main/java/org/apache/ignite/plugin/security/GridSecurityException.java
new file mode 100644
index 0000000..b8899df
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/plugin/security/GridSecurityException.java
@@ -0,0 +1,50 @@
+/* @java.file.header */
+
+/*  _________        _____ __________________        _____
+ *  __  ____/___________(_)______  /__  ____/______ ____(_)_______
+ *  _  / __  __  ___/__  / _  __  / _  / __  _  __ `/__  / __  __ \
+ *  / /_/ /  _  /    _  /  / /_/ /  / /_/ /  / /_/ / _  /  _  / / /
+ *  \____/   /_/     /_/   \_,__/   \____/   \__,_/  /_/   /_/ /_/
+ */
+
+package org.apache.ignite.plugin.security;
+
+import org.gridgain.grid.*;
+import org.jetbrains.annotations.*;
+
+/**
+ * Common security exception for the grid.
+ */
+public class GridSecurityException extends GridRuntimeException {
+    /** */
+    private static final long serialVersionUID = 0L;
+
+    /**
+     * Constructs security grid exception with given message and cause.
+     *
+     * @param msg Exception message.
+     * @param cause Exception cause.
+     */
+    public GridSecurityException(String msg, @Nullable Throwable cause) {
+        super(msg, cause);
+    }
+
+    /**
+     * Creates new security grid exception given throwable as a cause and
+     * source of error message.
+     *
+     * @param cause Non-null throwable cause.
+     */
+    public GridSecurityException(Throwable cause) {
+        this(cause.getMessage(), cause);
+    }
+
+    /**
+     * Constructs security grid exception with given message.
+     *
+     * @param msg Exception message.
+     */
+    public GridSecurityException(String msg) {
+        super(msg);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/8c20bddb/modules/core/src/main/java/org/apache/ignite/plugin/security/GridSecurityPermission.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/plugin/security/GridSecurityPermission.java b/modules/core/src/main/java/org/apache/ignite/plugin/security/GridSecurityPermission.java
new file mode 100644
index 0000000..fcb43fe
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/plugin/security/GridSecurityPermission.java
@@ -0,0 +1,61 @@
+/* @java.file.header */
+
+/*  _________        _____ __________________        _____
+ *  __  ____/___________(_)______  /__  ____/______ ____(_)_______
+ *  _  / __  __  ___/__  / _  __  / _  / __  _  __ `/__  / __  __ \
+ *  / /_/ /  _  /    _  /  / /_/ /  / /_/ /  / /_/ / _  /  _  / / /
+ *  \____/   /_/     /_/   \_,__/   \____/   \__,_/  /_/   /_/ /_/
+ */
+
+package org.apache.ignite.plugin.security;
+
+import org.jetbrains.annotations.*;
+
+/**
+ * Supported security permissions within grid. Permissions
+ * are specified on per-cache or per-task level.
+ */
+public enum GridSecurityPermission {
+    /** Cache {@code read} permission. */
+    CACHE_READ,
+
+    /** Cache {@code put} permission. */
+    CACHE_PUT,
+
+    /** Cache {@code remove} permission. */
+    CACHE_REMOVE,
+
+    /** Task {@code execute} permission. */
+    TASK_EXECUTE,
+
+    /** Task {@code cancel} permission. */
+    TASK_CANCEL,
+
+    /** Events {@code enable} permission. */
+    EVENTS_ENABLE,
+
+    /** Events {@code disable} permission. */
+    EVENTS_DISABLE,
+
+    /** Common visor tasks permission. */
+    ADMIN_VIEW,
+
+    /** Visor cache read (query) permission. */
+    ADMIN_QUERY,
+
+    /** Visor cache load permission. */
+    ADMIN_CACHE;
+
+    /** Enumerated values. */
+    private static final GridSecurityPermission[] VALS = values();
+
+    /**
+     * Efficiently gets enumerated value from its ordinal.
+     *
+     * @param ord Ordinal value.
+     * @return Enumerated value or {@code null} if ordinal out of range.
+     */
+    @Nullable public static GridSecurityPermission fromOrdinal(int ord) {
+        return ord >= 0 && ord < VALS.length ? VALS[ord] : null;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/8c20bddb/modules/core/src/main/java/org/apache/ignite/plugin/security/GridSecurityPermissionSet.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/plugin/security/GridSecurityPermissionSet.java b/modules/core/src/main/java/org/apache/ignite/plugin/security/GridSecurityPermissionSet.java
new file mode 100644
index 0000000..8e7a534
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/plugin/security/GridSecurityPermissionSet.java
@@ -0,0 +1,58 @@
+/* @java.file.header */
+
+/*  _________        _____ __________________        _____
+ *  __  ____/___________(_)______  /__  ____/______ ____(_)_______
+ *  _  / __  __  ___/__  / _  __  / _  / __  _  __ `/__  / __  __ \
+ *  / /_/ /  _  /    _  /  / /_/ /  / /_/ /  / /_/ / _  /  _  / / /
+ *  \____/   /_/     /_/   \_,__/   \____/   \__,_/  /_/   /_/ /_/
+ */
+
+package org.apache.ignite.plugin.security;
+
+import org.jetbrains.annotations.*;
+
+import java.io.*;
+import java.util.*;
+
+/**
+ * Security permission set for authorized security subjects. Permission set
+ * allows to specify task permissions for every task and cache permissions
+ * for every cache. While cards are supported at the end of task or
+ * cache name.
+ * <p>
+ * Property {@link #defaultAllowAll()} specifies whether to allow or deny
+ * cache and task operations if they were not explicitly specified.
+ */
+public interface GridSecurityPermissionSet extends Serializable {
+    /**
+     * Flag indicating whether to allow or deny cache and task operations
+     * if they were not explicitly specified.
+     *
+     * @return {@code True} to allow all cache task operations if they were
+     *      not explicitly specified, {@code false} otherwise.
+     */
+    public boolean defaultAllowAll();
+
+    /**
+     * Map of task names to task permissions. Wildcards are allowed at the
+     * end of task names.
+     *
+     * @return Map of task names to task permissions.
+     */
+    public Map<String, Collection<GridSecurityPermission>> taskPermissions();
+
+    /**
+     * Map of cache names to cache permissions. Wildcards are allowed at the
+     * end of cache names.
+     *
+     * @return Map of cache names to cache permissions.
+     */
+    public Map<String, Collection<GridSecurityPermission>> cachePermissions();
+
+    /**
+     * Collection of system-wide permissions (events enable/disable, Visor task execution).
+     *
+     * @return Collection of system-wide permissions.
+     */
+    @Nullable public Collection<GridSecurityPermission> systemPermissions();
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/8c20bddb/modules/core/src/main/java/org/apache/ignite/plugin/security/GridSecuritySubject.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/plugin/security/GridSecuritySubject.java b/modules/core/src/main/java/org/apache/ignite/plugin/security/GridSecuritySubject.java
new file mode 100644
index 0000000..74ca013
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/plugin/security/GridSecuritySubject.java
@@ -0,0 +1,55 @@
+/* @java.file.header */
+
+/*  _________        _____ __________________        _____
+ *  __  ____/___________(_)______  /__  ____/______ ____(_)_______
+ *  _  / __  __  ___/__  / _  __  / _  / __  _  __ `/__  / __  __ \
+ *  / /_/ /  _  /    _  /  / /_/ /  / /_/ /  / /_/ / _  /  _  / / /
+ *  \____/   /_/     /_/   \_,__/   \____/   \__,_/  /_/   /_/ /_/
+ */
+
+package org.apache.ignite.plugin.security;
+
+import java.io.*;
+import java.net.*;
+import java.util.*;
+
+/**
+ * Security subject representing authenticated node or client with a set of permissions.
+ * List of authenticated subjects can be retrieved from {@link GridSecurity#authenticatedSubjects()} method.
+ */
+public interface GridSecuritySubject extends Serializable {
+    /**
+     * Gets subject ID.
+     *
+     * @return Subject ID.
+     */
+    public UUID id();
+
+    /**
+     * Gets subject type, either node or client.
+     *
+     * @return Subject type.
+     */
+    public GridSecuritySubjectType type();
+
+    /**
+     * Login provided via subject security credentials.
+     *
+     * @return Login object.
+     */
+    public Object login();
+
+    /**
+     * Gets subject connection address. Usually {@link InetSocketAddress} representing connection IP and port.
+     *
+     * @return Subject connection address.
+     */
+    public InetSocketAddress address();
+
+    /**
+     * Authorized permission set for the subject.
+     *
+     * @return Authorized permission set for the subject.
+     */
+    public GridSecurityPermissionSet permissions();
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/8c20bddb/modules/core/src/main/java/org/apache/ignite/plugin/security/GridSecuritySubjectType.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/plugin/security/GridSecuritySubjectType.java b/modules/core/src/main/java/org/apache/ignite/plugin/security/GridSecuritySubjectType.java
new file mode 100644
index 0000000..5b912c2
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/plugin/security/GridSecuritySubjectType.java
@@ -0,0 +1,40 @@
+/* @java.file.header */
+
+/*  _________        _____ __________________        _____
+ *  __  ____/___________(_)______  /__  ____/______ ____(_)_______
+ *  _  / __  __  ___/__  / _  __  / _  / __  _  __ `/__  / __  __ \
+ *  / /_/ /  _  /    _  /  / /_/ /  / /_/ /  / /_/ / _  /  _  / / /
+ *  \____/   /_/     /_/   \_,__/   \____/   \__,_/  /_/   /_/ /_/
+ */
+
+package org.apache.ignite.plugin.security;
+
+import org.jetbrains.annotations.*;
+
+/**
+ * Supported security subject types. Subject type can be retrieved form {@link GridSecuritySubject#type()} method.
+ */
+public enum GridSecuritySubjectType {
+    /**
+     * Subject type for a remote {@link org.apache.ignite.cluster.ClusterNode}.
+     */
+    REMOTE_NODE,
+
+    /**
+     * Subject type for remote client.
+     */
+    REMOTE_CLIENT;
+
+    /** Enumerated values. */
+    private static final GridSecuritySubjectType[] VALS = values();
+
+    /**
+     * Efficiently gets enumerated value from its ordinal.
+     *
+     * @param ord Ordinal value.
+     * @return Enumerated value.
+     */
+    @Nullable public static GridSecuritySubjectType fromOrdinal(byte ord) {
+        return ord >= 0 && ord < VALS.length ? VALS[ord] : null;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/8c20bddb/modules/core/src/main/java/org/apache/ignite/plugin/security/package.html
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/plugin/security/package.html b/modules/core/src/main/java/org/apache/ignite/plugin/security/package.html
new file mode 100644
index 0000000..bd468a3
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/plugin/security/package.html
@@ -0,0 +1,15 @@
+<!DOCTYPE html PUBLIC "-//W3C//DTD HTML 4.01 Transitional//EN" "http://www.w3.org/TR/html4/loose.dtd">
+<!--
+    @html.file.header
+    _________        _____ __________________        _____
+    __  ____/___________(_)______  /__  ____/______ ____(_)_______
+    _  / __  __  ___/__  / _  __  / _  / __  _  __ `/__  / __  __ \
+    / /_/ /  _  /    _  /  / /_/ /  / /_/ /  / /_/ / _  /  _  / / /
+    \____/   /_/     /_/   \_,__/   \____/   \__,_/  /_/   /_/ /_/
+-->
+<html>
+<body>
+    <!-- Package description. -->
+    Contains APIs for security, authentication, and authorization.
+</body>
+</html>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/8c20bddb/modules/core/src/main/java/org/apache/ignite/plugin/segmentation/GridSegmentationPolicy.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/plugin/segmentation/GridSegmentationPolicy.java b/modules/core/src/main/java/org/apache/ignite/plugin/segmentation/GridSegmentationPolicy.java
new file mode 100644
index 0000000..fb7afda
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/plugin/segmentation/GridSegmentationPolicy.java
@@ -0,0 +1,40 @@
+/* @java.file.header */
+
+/*  _________        _____ __________________        _____
+ *  __  ____/___________(_)______  /__  ____/______ ____(_)_______
+ *  _  / __  __  ___/__  / _  __  / _  / __  _  __ `/__  / __  __ \
+ *  / /_/ /  _  /    _  /  / /_/ /  / /_/ /  / /_/ / _  /  _  / / /
+ *  \____/   /_/     /_/   \_,__/   \____/   \__,_/  /_/   /_/ /_/
+ */
+
+package org.apache.ignite.plugin.segmentation;
+
+/**
+ * Policy that defines how node will react on topology segmentation. Note that default
+ * segmentation policy is defined by {@link org.apache.ignite.configuration.IgniteConfiguration#DFLT_SEG_PLC} property.
+ * @see GridSegmentationResolver
+ */
+public enum GridSegmentationPolicy {
+    /**
+     * When segmentation policy is {@code RESTART_JVM}, all listeners will receive
+     * {@link org.apache.ignite.events.IgniteEventType#EVT_NODE_SEGMENTED} event and then JVM will be restarted.
+     * Note, that this will work <b>only</b> if GridGain is started with {@link org.apache.ignite.startup.cmdline.CommandLineStartup}
+     * via standard {@code ggstart.{sh|bat}} shell script.
+     */
+    RESTART_JVM,
+
+    /**
+     * When segmentation policy is {@code STOP}, all listeners will receive
+     * {@link org.apache.ignite.events.IgniteEventType#EVT_NODE_SEGMENTED} event and then particular grid node
+     * will be stopped via call to {@link org.apache.ignite.Ignition#stop(String, boolean)}.
+     */
+    STOP,
+
+    /**
+     * When segmentation policy is {@code NOOP}, all listeners will receive
+     * {@link org.apache.ignite.events.IgniteEventType#EVT_NODE_SEGMENTED} event and it is up to user to
+     * implement logic to handle this event.
+     */
+    NOOP
+}
+

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/8c20bddb/modules/core/src/main/java/org/apache/ignite/plugin/segmentation/GridSegmentationResolver.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/plugin/segmentation/GridSegmentationResolver.java b/modules/core/src/main/java/org/apache/ignite/plugin/segmentation/GridSegmentationResolver.java
new file mode 100644
index 0000000..6b8418b
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/plugin/segmentation/GridSegmentationResolver.java
@@ -0,0 +1,56 @@
+/* @java.file.header */
+
+/*  _________        _____ __________________        _____
+ *  __  ____/___________(_)______  /__  ____/______ ____(_)_______
+ *  _  / __  __  ___/__  / _  __  / _  / __  _  __ `/__  / __  __ \
+ *  / /_/ /  _  /    _  /  / /_/ /  / /_/ /  / /_/ / _  /  _  / / /
+ *  \____/   /_/     /_/   \_,__/   \____/   \__,_/  /_/   /_/ /_/
+ */
+
+package org.apache.ignite.plugin.segmentation;
+
+import org.gridgain.grid.*;
+
+import java.io.*;
+
+/**
+ * This is interface for segmentation (a.k.a "split-brain" problem) resolvers.
+ * <p>
+ * Each segmentation resolver checks segment for validity, using its inner logic.
+ * Typically, resolver should run light-weight single check (i.e. one IP address or
+ * one shared folder). Compound segment checks may be performed using several
+ * resolvers.
+ * <p>
+ * Note that GridGain support a logical segmentation and not limited to network
+ * related segmentation only. For example, a particular segmentation resolver
+ * can check for specific application or service present on the network and
+ * mark the topology as segmented in case it is not available. In other words
+ * you can equate the service outage with network outage via segmentation resolution
+ * and employ the unified approach in dealing with these types of problems.
+ * @see org.apache.ignite.configuration.IgniteConfiguration#getSegmentationResolvers()
+ * @see org.apache.ignite.configuration.IgniteConfiguration#getSegmentationPolicy()
+ * @see org.apache.ignite.configuration.IgniteConfiguration#getSegmentCheckFrequency()
+ * @see org.apache.ignite.configuration.IgniteConfiguration#isAllSegmentationResolversPassRequired()
+ * @see org.apache.ignite.configuration.IgniteConfiguration#isWaitForSegmentOnStart()
+ * @see GridSegmentationPolicy
+ */
+public interface GridSegmentationResolver extends Serializable {
+    /**
+     * Checks whether segment is valid.
+     * <p>
+     * When segmentation happens every node ends up in either one of two segments:
+     * <ul>
+     *     <li>Correct segment</li>
+     *     <li>Invalid segment</li>
+     * </ul>
+     * Nodes in correct segment will continue operate as if nodes in the invalid segment
+     * simply left the topology (i.e. the topology just got "smaller"). Nodes in the
+     * invalid segment will realized that were "left out or disconnected" from the correct segment
+     * and will try to reconnect via {@link GridSegmentationPolicy segmentation policy} set
+     * in configuration.
+     *
+     * @return {@code True} if segment is correct, {@code false} otherwise.
+     * @throws GridException If an error occurred.
+     */
+    public abstract boolean isValidSegment() throws GridException;
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/8c20bddb/modules/core/src/main/java/org/apache/ignite/plugin/segmentation/package.html
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/plugin/segmentation/package.html b/modules/core/src/main/java/org/apache/ignite/plugin/segmentation/package.html
new file mode 100644
index 0000000..a4754a1
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/plugin/segmentation/package.html
@@ -0,0 +1,15 @@
+<!DOCTYPE html PUBLIC "-//W3C//DTD HTML 4.01 Transitional//EN" "http://www.w3.org/TR/html4/loose.dtd">
+<!--
+    @html.file.header
+    _________        _____ __________________        _____
+    __  ____/___________(_)______  /__  ____/______ ____(_)_______
+    _  / __  __  ___/__  / _  __  / _  / __  _  __ `/__  / __  __ \
+    / /_/ /  _  /    _  /  / /_/ /  / /_/ /  / /_/ / _  /  _  / / /
+    \____/   /_/     /_/   \_,__/   \____/   \__,_/  /_/   /_/ /_/
+-->
+<html>
+<body>
+    <!-- Package description. -->
+    Contains segmentation "split-brain" related classes.
+</body>
+</html>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/8c20bddb/modules/core/src/main/java/org/apache/ignite/spi/IgniteSpiAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/IgniteSpiAdapter.java b/modules/core/src/main/java/org/apache/ignite/spi/IgniteSpiAdapter.java
index 561414b..3d73650 100644
--- a/modules/core/src/main/java/org/apache/ignite/spi/IgniteSpiAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/spi/IgniteSpiAdapter.java
@@ -18,7 +18,7 @@ import org.gridgain.grid.*;
 import org.gridgain.grid.kernal.*;
 import org.gridgain.grid.kernal.managers.communication.*;
 import org.gridgain.grid.kernal.managers.eventstorage.*;
-import org.gridgain.grid.security.*;
+import org.apache.ignite.plugin.security.*;
 import org.apache.ignite.spi.securesession.*;
 import org.apache.ignite.spi.swapspace.*;
 import org.gridgain.grid.util.direct.*;

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/8c20bddb/modules/core/src/main/java/org/apache/ignite/spi/IgniteSpiContext.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/IgniteSpiContext.java b/modules/core/src/main/java/org/apache/ignite/spi/IgniteSpiContext.java
index 39b281f..5d5dfb7 100644
--- a/modules/core/src/main/java/org/apache/ignite/spi/IgniteSpiContext.java
+++ b/modules/core/src/main/java/org/apache/ignite/spi/IgniteSpiContext.java
@@ -14,7 +14,7 @@ import org.apache.ignite.events.*;
 import org.gridgain.grid.*;
 import org.gridgain.grid.kernal.managers.communication.*;
 import org.gridgain.grid.kernal.managers.eventstorage.*;
-import org.gridgain.grid.security.*;
+import org.apache.ignite.plugin.security.*;
 import org.apache.ignite.spi.swapspace.*;
 import org.gridgain.grid.util.direct.*;
 import org.jetbrains.annotations.*;

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/8c20bddb/modules/core/src/main/java/org/apache/ignite/spi/authentication/AuthenticationAclProvider.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/authentication/AuthenticationAclProvider.java b/modules/core/src/main/java/org/apache/ignite/spi/authentication/AuthenticationAclProvider.java
index c05e5db..e8c2b84 100644
--- a/modules/core/src/main/java/org/apache/ignite/spi/authentication/AuthenticationAclProvider.java
+++ b/modules/core/src/main/java/org/apache/ignite/spi/authentication/AuthenticationAclProvider.java
@@ -10,7 +10,7 @@
 package org.apache.ignite.spi.authentication;
 
 import org.gridgain.grid.*;
-import org.gridgain.grid.security.*;
+import org.apache.ignite.plugin.security.*;
 
 import java.util.*;
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/8c20bddb/modules/core/src/main/java/org/apache/ignite/spi/authentication/AuthenticationContext.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/authentication/AuthenticationContext.java b/modules/core/src/main/java/org/apache/ignite/spi/authentication/AuthenticationContext.java
index eb9dc8b..4e37ef9 100644
--- a/modules/core/src/main/java/org/apache/ignite/spi/authentication/AuthenticationContext.java
+++ b/modules/core/src/main/java/org/apache/ignite/spi/authentication/AuthenticationContext.java
@@ -9,7 +9,7 @@
 
 package org.apache.ignite.spi.authentication;
 
-import org.gridgain.grid.security.*;
+import org.apache.ignite.plugin.security.*;
 
 import java.net.*;
 import java.util.*;

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/8c20bddb/modules/core/src/main/java/org/apache/ignite/spi/authentication/AuthenticationContextAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/authentication/AuthenticationContextAdapter.java b/modules/core/src/main/java/org/apache/ignite/spi/authentication/AuthenticationContextAdapter.java
index e5bffb7..e53f2c6 100644
--- a/modules/core/src/main/java/org/apache/ignite/spi/authentication/AuthenticationContextAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/spi/authentication/AuthenticationContextAdapter.java
@@ -9,7 +9,7 @@
 
 package org.apache.ignite.spi.authentication;
 
-import org.gridgain.grid.security.*;
+import org.apache.ignite.plugin.security.*;
 
 import java.net.*;
 import java.util.*;

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/8c20bddb/modules/core/src/main/java/org/apache/ignite/spi/authentication/AuthenticationSpi.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/authentication/AuthenticationSpi.java b/modules/core/src/main/java/org/apache/ignite/spi/authentication/AuthenticationSpi.java
index 7698508..6945ac2 100644
--- a/modules/core/src/main/java/org/apache/ignite/spi/authentication/AuthenticationSpi.java
+++ b/modules/core/src/main/java/org/apache/ignite/spi/authentication/AuthenticationSpi.java
@@ -10,7 +10,7 @@
 package org.apache.ignite.spi.authentication;
 
 import org.apache.ignite.spi.*;
-import org.gridgain.grid.security.*;
+import org.apache.ignite.plugin.security.*;
 
 /**
  * Authentication SPI used for authenticating grid nodes and remote clients. This SPI

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/8c20bddb/modules/core/src/main/java/org/apache/ignite/spi/authentication/noop/NoopAuthenticationSpi.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/authentication/noop/NoopAuthenticationSpi.java b/modules/core/src/main/java/org/apache/ignite/spi/authentication/noop/NoopAuthenticationSpi.java
index 3ef7ced..3fb9404 100644
--- a/modules/core/src/main/java/org/apache/ignite/spi/authentication/noop/NoopAuthenticationSpi.java
+++ b/modules/core/src/main/java/org/apache/ignite/spi/authentication/noop/NoopAuthenticationSpi.java
@@ -14,7 +14,7 @@ import org.apache.ignite.resources.*;
 import org.apache.ignite.spi.*;
 import org.apache.ignite.spi.authentication.*;
 import org.gridgain.grid.kernal.managers.security.*;
-import org.gridgain.grid.security.*;
+import org.apache.ignite.plugin.security.*;
 import org.gridgain.grid.util.tostring.*;
 import org.gridgain.grid.util.typedef.internal.*;
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/8c20bddb/modules/core/src/main/java/org/apache/ignite/spi/discovery/DiscoverySpiNodeAuthenticator.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/DiscoverySpiNodeAuthenticator.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/DiscoverySpiNodeAuthenticator.java
index dd9fc3a..456d6a3 100644
--- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/DiscoverySpiNodeAuthenticator.java
+++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/DiscoverySpiNodeAuthenticator.java
@@ -12,7 +12,7 @@ package org.apache.ignite.spi.discovery;
 import org.apache.ignite.cluster.*;
 import org.gridgain.grid.*;
 import org.gridgain.grid.kernal.managers.security.*;
-import org.gridgain.grid.security.*;
+import org.apache.ignite.plugin.security.*;
 
 /**
  * Node authenticator.

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/8c20bddb/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySpi.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySpi.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySpi.java
index 32d3f54..7f6d5c1 100644
--- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySpi.java
+++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySpi.java
@@ -19,7 +19,7 @@ import org.apache.ignite.spi.*;
 import org.gridgain.grid.*;
 import org.gridgain.grid.kernal.*;
 import org.gridgain.grid.kernal.managers.security.*;
-import org.gridgain.grid.security.*;
+import org.apache.ignite.plugin.security.*;
 import org.apache.ignite.spi.discovery.*;
 import org.apache.ignite.spi.discovery.tcp.internal.*;
 import org.apache.ignite.spi.discovery.tcp.ipfinder.multicast.*;

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/8c20bddb/modules/core/src/main/java/org/apache/ignite/spi/package.html
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/package.html b/modules/core/src/main/java/org/apache/ignite/spi/package.html
new file mode 100644
index 0000000..72117f8
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/spi/package.html
@@ -0,0 +1,15 @@
+<!DOCTYPE html PUBLIC "-//W3C//DTD HTML 4.01 Transitional//EN" "http://www.w3.org/TR/html4/loose.dtd">
+<!--
+    @html.file.header
+    _________        _____ __________________        _____
+    __  ____/___________(_)______  /__  ____/______ ____(_)_______
+    _  / __  __  ___/__  / _  __  / _  / __  _  __ `/__  / __  __ \
+    / /_/ /  _  /    _  /  / /_/ /  / /_/ /  / /_/ / _  /  _  / / /
+    \____/   /_/     /_/   \_,__/   \____/   \__,_/  /_/   /_/ /_/
+-->
+<html>
+<body>
+    <!-- Package description. -->
+    Contains common classes and interfaces for SPI implementations.
+</body>
+</html>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/8c20bddb/modules/core/src/main/java/org/apache/ignite/spi/securesession/SecureSessionSpi.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/securesession/SecureSessionSpi.java b/modules/core/src/main/java/org/apache/ignite/spi/securesession/SecureSessionSpi.java
index 4fbff22..85ee7dd 100644
--- a/modules/core/src/main/java/org/apache/ignite/spi/securesession/SecureSessionSpi.java
+++ b/modules/core/src/main/java/org/apache/ignite/spi/securesession/SecureSessionSpi.java
@@ -10,7 +10,7 @@
 package org.apache.ignite.spi.securesession;
 
 import org.apache.ignite.spi.*;
-import org.gridgain.grid.security.*;
+import org.apache.ignite.plugin.security.*;
 import org.jetbrains.annotations.*;
 
 import java.util.*;

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/8c20bddb/modules/core/src/main/java/org/apache/ignite/spi/securesession/noop/NoopSecureSessionSpi.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/securesession/noop/NoopSecureSessionSpi.java b/modules/core/src/main/java/org/apache/ignite/spi/securesession/noop/NoopSecureSessionSpi.java
index e281f02..35cd089 100644
--- a/modules/core/src/main/java/org/apache/ignite/spi/securesession/noop/NoopSecureSessionSpi.java
+++ b/modules/core/src/main/java/org/apache/ignite/spi/securesession/noop/NoopSecureSessionSpi.java
@@ -12,7 +12,7 @@ package org.apache.ignite.spi.securesession.noop;
 import org.apache.ignite.*;
 import org.apache.ignite.resources.*;
 import org.apache.ignite.spi.*;
-import org.gridgain.grid.security.*;
+import org.apache.ignite.plugin.security.*;
 import org.apache.ignite.spi.securesession.*;
 import org.gridgain.grid.util.typedef.internal.*;
 import org.jetbrains.annotations.*;

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/8c20bddb/modules/core/src/main/java/org/gridgain/client/GridClientConfiguration.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/client/GridClientConfiguration.java b/modules/core/src/main/java/org/gridgain/client/GridClientConfiguration.java
index 12ae587..3555f05 100644
--- a/modules/core/src/main/java/org/gridgain/client/GridClientConfiguration.java
+++ b/modules/core/src/main/java/org/gridgain/client/GridClientConfiguration.java
@@ -15,7 +15,7 @@ import org.gridgain.client.marshaller.*;
 import org.gridgain.client.marshaller.jdk.*;
 import org.gridgain.client.marshaller.optimized.*;
 import org.gridgain.client.ssl.*;
-import org.gridgain.grid.security.*;
+import org.apache.ignite.plugin.security.*;
 import org.gridgain.grid.util.typedef.*;
 import org.gridgain.grid.util.typedef.internal.*;
 import org.jetbrains.annotations.*;

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/8c20bddb/modules/core/src/main/java/org/gridgain/client/impl/connection/GridClientConnectionManagerAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/client/impl/connection/GridClientConnectionManagerAdapter.java b/modules/core/src/main/java/org/gridgain/client/impl/connection/GridClientConnectionManagerAdapter.java
index c181141..273bbfa 100644
--- a/modules/core/src/main/java/org/gridgain/client/impl/connection/GridClientConnectionManagerAdapter.java
+++ b/modules/core/src/main/java/org/gridgain/client/impl/connection/GridClientConnectionManagerAdapter.java
@@ -16,7 +16,7 @@ import org.gridgain.client.impl.*;
 import org.gridgain.client.util.*;
 import org.gridgain.grid.*;
 import org.gridgain.grid.kernal.processors.rest.client.message.*;
-import org.gridgain.grid.security.*;
+import org.apache.ignite.plugin.security.*;
 import org.gridgain.grid.util.direct.*;
 import org.gridgain.grid.util.nio.*;
 import org.gridgain.grid.util.nio.ssl.*;

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/8c20bddb/modules/core/src/main/java/org/gridgain/client/router/GridTcpRouterConfiguration.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/client/router/GridTcpRouterConfiguration.java b/modules/core/src/main/java/org/gridgain/client/router/GridTcpRouterConfiguration.java
index b2ebb52..ee89c84 100644
--- a/modules/core/src/main/java/org/gridgain/client/router/GridTcpRouterConfiguration.java
+++ b/modules/core/src/main/java/org/gridgain/client/router/GridTcpRouterConfiguration.java
@@ -3,7 +3,7 @@ package org.gridgain.client.router;
 import org.apache.ignite.*;
 import org.apache.ignite.configuration.*;
 import org.gridgain.client.ssl.*;
-import org.gridgain.grid.security.*;
+import org.apache.ignite.plugin.security.*;
 import org.gridgain.grid.util.typedef.internal.*;
 import org.jetbrains.annotations.*;
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/8c20bddb/modules/core/src/main/java/org/gridgain/grid/kernal/GridGainEx.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/kernal/GridGainEx.java b/modules/core/src/main/java/org/gridgain/grid/kernal/GridGainEx.java
index 1401e06..9d72e65 100644
--- a/modules/core/src/main/java/org/gridgain/grid/kernal/GridGainEx.java
+++ b/modules/core/src/main/java/org/gridgain/grid/kernal/GridGainEx.java
@@ -32,7 +32,7 @@ import org.gridgain.grid.cache.affinity.rendezvous.*;
 import org.gridgain.grid.kernal.processors.interop.*;
 import org.gridgain.grid.kernal.processors.resource.*;
 import org.gridgain.grid.kernal.processors.spring.*;
-import org.gridgain.grid.segmentation.*;
+import org.apache.ignite.plugin.segmentation.*;
 import org.apache.ignite.spi.checkpoint.*;
 import org.apache.ignite.spi.checkpoint.noop.*;
 import org.apache.ignite.spi.collision.*;
@@ -81,7 +81,7 @@ import static org.gridgain.grid.cache.GridCacheMode.*;
 import static org.gridgain.grid.cache.GridCachePreloadMode.*;
 import static org.gridgain.grid.cache.GridCacheWriteSynchronizationMode.*;
 import static org.gridgain.grid.kernal.GridComponentType.*;
-import static org.gridgain.grid.segmentation.GridSegmentationPolicy.*;
+import static org.apache.ignite.plugin.segmentation.GridSegmentationPolicy.*;
 
 /**
  * This class defines a factory for the main GridGain API. It controls Grid life cycle

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/8c20bddb/modules/core/src/main/java/org/gridgain/grid/kernal/GridKernal.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/kernal/GridKernal.java b/modules/core/src/main/java/org/gridgain/grid/kernal/GridKernal.java
index 1f64654..d913605 100644
--- a/modules/core/src/main/java/org/gridgain/grid/kernal/GridKernal.java
+++ b/modules/core/src/main/java/org/gridgain/grid/kernal/GridKernal.java
@@ -62,7 +62,7 @@ import org.gridgain.grid.kernal.processors.session.*;
 import org.gridgain.grid.kernal.processors.streamer.*;
 import org.gridgain.grid.kernal.processors.task.*;
 import org.gridgain.grid.kernal.processors.timeout.*;
-import org.gridgain.grid.security.*;
+import org.apache.ignite.plugin.security.*;
 import org.apache.ignite.spi.securesession.noop.*;
 import org.gridgain.grid.util.*;
 import org.gridgain.grid.util.future.*;

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/8c20bddb/modules/core/src/main/java/org/gridgain/grid/kernal/GridTaskFutureImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/kernal/GridTaskFutureImpl.java b/modules/core/src/main/java/org/gridgain/grid/kernal/GridTaskFutureImpl.java
index e82761a..0117f9c 100644
--- a/modules/core/src/main/java/org/gridgain/grid/kernal/GridTaskFutureImpl.java
+++ b/modules/core/src/main/java/org/gridgain/grid/kernal/GridTaskFutureImpl.java
@@ -12,7 +12,7 @@ package org.gridgain.grid.kernal;
 import org.apache.ignite.cluster.*;
 import org.apache.ignite.compute.*;
 import org.gridgain.grid.*;
-import org.gridgain.grid.security.*;
+import org.apache.ignite.plugin.security.*;
 import org.gridgain.grid.util.future.*;
 import org.gridgain.grid.util.typedef.internal.*;
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/8c20bddb/modules/core/src/main/java/org/gridgain/grid/kernal/managers/GridManagerAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/kernal/managers/GridManagerAdapter.java b/modules/core/src/main/java/org/gridgain/grid/kernal/managers/GridManagerAdapter.java
index 152952b..4ceed98 100644
--- a/modules/core/src/main/java/org/gridgain/grid/kernal/managers/GridManagerAdapter.java
+++ b/modules/core/src/main/java/org/gridgain/grid/kernal/managers/GridManagerAdapter.java
@@ -21,7 +21,7 @@ import org.gridgain.grid.kernal.*;
 import org.gridgain.grid.kernal.managers.communication.*;
 import org.gridgain.grid.kernal.managers.eventstorage.*;
 import org.gridgain.grid.kernal.processors.cache.*;
-import org.gridgain.grid.security.*;
+import org.apache.ignite.plugin.security.*;
 import org.apache.ignite.spi.swapspace.*;
 import org.gridgain.grid.util.direct.*;
 import org.gridgain.grid.util.tostring.*;

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/8c20bddb/modules/core/src/main/java/org/gridgain/grid/kernal/managers/discovery/GridDiscoveryManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/kernal/managers/discovery/GridDiscoveryManager.java b/modules/core/src/main/java/org/gridgain/grid/kernal/managers/discovery/GridDiscoveryManager.java
index 1acd9d5..48fee0d 100644
--- a/modules/core/src/main/java/org/gridgain/grid/kernal/managers/discovery/GridDiscoveryManager.java
+++ b/modules/core/src/main/java/org/gridgain/grid/kernal/managers/discovery/GridDiscoveryManager.java
@@ -25,8 +25,8 @@ import org.gridgain.grid.kernal.managers.security.*;
 import org.gridgain.grid.kernal.processors.cache.*;
 import org.gridgain.grid.kernal.processors.jobmetrics.*;
 import org.gridgain.grid.kernal.processors.service.*;
-import org.gridgain.grid.security.*;
-import org.gridgain.grid.segmentation.*;
+import org.apache.ignite.plugin.security.*;
+import org.apache.ignite.plugin.segmentation.*;
 import org.apache.ignite.spi.discovery.*;
 import org.gridgain.grid.util.*;
 import org.gridgain.grid.util.future.*;
@@ -47,7 +47,7 @@ import java.util.zip.*;
 import static java.util.concurrent.TimeUnit.*;
 import static org.apache.ignite.events.IgniteEventType.*;
 import static org.gridgain.grid.kernal.GridNodeAttributes.*;
-import static org.gridgain.grid.segmentation.GridSegmentationPolicy.*;
+import static org.apache.ignite.plugin.segmentation.GridSegmentationPolicy.*;
 
 /**
  * Discovery SPI manager.

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/8c20bddb/modules/core/src/main/java/org/gridgain/grid/kernal/managers/eventstorage/GridEventStorageManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/kernal/managers/eventstorage/GridEventStorageManager.java b/modules/core/src/main/java/org/gridgain/grid/kernal/managers/eventstorage/GridEventStorageManager.java
index db12d53..ade9c52 100644
--- a/modules/core/src/main/java/org/gridgain/grid/kernal/managers/eventstorage/GridEventStorageManager.java
+++ b/modules/core/src/main/java/org/gridgain/grid/kernal/managers/eventstorage/GridEventStorageManager.java
@@ -19,7 +19,7 @@ import org.gridgain.grid.kernal.*;
 import org.gridgain.grid.kernal.managers.*;
 import org.gridgain.grid.kernal.managers.communication.*;
 import org.gridgain.grid.kernal.managers.deployment.*;
-import org.gridgain.grid.security.*;
+import org.apache.ignite.plugin.security.*;
 import org.apache.ignite.spi.eventstorage.*;
 import org.gridgain.grid.util.*;
 import org.gridgain.grid.util.future.*;

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/8c20bddb/modules/core/src/main/java/org/gridgain/grid/kernal/managers/securesession/GridSecureSessionManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/kernal/managers/securesession/GridSecureSessionManager.java b/modules/core/src/main/java/org/gridgain/grid/kernal/managers/securesession/GridSecureSessionManager.java
index d9e9dd5..703a74b 100644
--- a/modules/core/src/main/java/org/gridgain/grid/kernal/managers/securesession/GridSecureSessionManager.java
+++ b/modules/core/src/main/java/org/gridgain/grid/kernal/managers/securesession/GridSecureSessionManager.java
@@ -12,7 +12,7 @@ package org.gridgain.grid.kernal.managers.securesession;
 import org.gridgain.grid.*;
 import org.gridgain.grid.kernal.managers.*;
 import org.gridgain.grid.kernal.managers.security.*;
-import org.gridgain.grid.security.*;
+import org.apache.ignite.plugin.security.*;
 import org.jetbrains.annotations.*;
 
 import java.util.*;

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/8c20bddb/modules/core/src/main/java/org/gridgain/grid/kernal/managers/securesession/os/GridOsSecureSessionManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/kernal/managers/securesession/os/GridOsSecureSessionManager.java b/modules/core/src/main/java/org/gridgain/grid/kernal/managers/securesession/os/GridOsSecureSessionManager.java
index b0350ec..a3fe176 100644
--- a/modules/core/src/main/java/org/gridgain/grid/kernal/managers/securesession/os/GridOsSecureSessionManager.java
+++ b/modules/core/src/main/java/org/gridgain/grid/kernal/managers/securesession/os/GridOsSecureSessionManager.java
@@ -14,7 +14,7 @@ import org.gridgain.grid.kernal.*;
 import org.gridgain.grid.kernal.managers.*;
 import org.gridgain.grid.kernal.managers.securesession.*;
 import org.gridgain.grid.kernal.managers.security.*;
-import org.gridgain.grid.security.*;
+import org.apache.ignite.plugin.security.*;
 import org.jetbrains.annotations.*;
 
 import java.util.*;

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/8c20bddb/modules/core/src/main/java/org/gridgain/grid/kernal/managers/security/GridAllowAllPermissionSet.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/kernal/managers/security/GridAllowAllPermissionSet.java b/modules/core/src/main/java/org/gridgain/grid/kernal/managers/security/GridAllowAllPermissionSet.java
index 80d7af1..23dd83a 100644
--- a/modules/core/src/main/java/org/gridgain/grid/kernal/managers/security/GridAllowAllPermissionSet.java
+++ b/modules/core/src/main/java/org/gridgain/grid/kernal/managers/security/GridAllowAllPermissionSet.java
@@ -9,7 +9,7 @@
 
 package org.gridgain.grid.kernal.managers.security;
 
-import org.gridgain.grid.security.*;
+import org.apache.ignite.plugin.security.*;
 
 import java.util.*;
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/8c20bddb/modules/core/src/main/java/org/gridgain/grid/kernal/managers/security/GridSecurityContext.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/kernal/managers/security/GridSecurityContext.java b/modules/core/src/main/java/org/gridgain/grid/kernal/managers/security/GridSecurityContext.java
index 678cf24..b8e93d6 100644
--- a/modules/core/src/main/java/org/gridgain/grid/kernal/managers/security/GridSecurityContext.java
+++ b/modules/core/src/main/java/org/gridgain/grid/kernal/managers/security/GridSecurityContext.java
@@ -9,7 +9,7 @@
 
 package org.gridgain.grid.kernal.managers.security;
 
-import org.gridgain.grid.security.*;
+import org.apache.ignite.plugin.security.*;
 import org.gridgain.grid.util.typedef.internal.*;
 
 import java.io.*;

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/8c20bddb/modules/core/src/main/java/org/gridgain/grid/kernal/managers/security/GridSecurityImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/kernal/managers/security/GridSecurityImpl.java b/modules/core/src/main/java/org/gridgain/grid/kernal/managers/security/GridSecurityImpl.java
index 789df1e..efb4df1 100644
--- a/modules/core/src/main/java/org/gridgain/grid/kernal/managers/security/GridSecurityImpl.java
+++ b/modules/core/src/main/java/org/gridgain/grid/kernal/managers/security/GridSecurityImpl.java
@@ -11,7 +11,7 @@ package org.gridgain.grid.kernal.managers.security;
 
 import org.gridgain.grid.*;
 import org.gridgain.grid.kernal.*;
-import org.gridgain.grid.security.*;
+import org.apache.ignite.plugin.security.*;
 
 import java.io.*;
 import java.util.*;

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/8c20bddb/modules/core/src/main/java/org/gridgain/grid/kernal/managers/security/GridSecurityManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/kernal/managers/security/GridSecurityManager.java b/modules/core/src/main/java/org/gridgain/grid/kernal/managers/security/GridSecurityManager.java
index e4d79dd..28f3337 100644
--- a/modules/core/src/main/java/org/gridgain/grid/kernal/managers/security/GridSecurityManager.java
+++ b/modules/core/src/main/java/org/gridgain/grid/kernal/managers/security/GridSecurityManager.java
@@ -13,7 +13,7 @@ import org.apache.ignite.cluster.*;
 import org.apache.ignite.spi.authentication.*;
 import org.gridgain.grid.*;
 import org.gridgain.grid.kernal.managers.*;
-import org.gridgain.grid.security.*;
+import org.apache.ignite.plugin.security.*;
 import org.jetbrains.annotations.*;
 
 import java.util.*;

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/8c20bddb/modules/core/src/main/java/org/gridgain/grid/kernal/managers/security/GridSecuritySubjectAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/kernal/managers/security/GridSecuritySubjectAdapter.java b/modules/core/src/main/java/org/gridgain/grid/kernal/managers/security/GridSecuritySubjectAdapter.java
index 5453ac2..8b62e08 100644
--- a/modules/core/src/main/java/org/gridgain/grid/kernal/managers/security/GridSecuritySubjectAdapter.java
+++ b/modules/core/src/main/java/org/gridgain/grid/kernal/managers/security/GridSecuritySubjectAdapter.java
@@ -9,7 +9,7 @@
 
 package org.gridgain.grid.kernal.managers.security;
 
-import org.gridgain.grid.security.*;
+import org.apache.ignite.plugin.security.*;
 import org.gridgain.grid.util.tostring.*;
 import org.gridgain.grid.util.typedef.internal.*;
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/8c20bddb/modules/core/src/main/java/org/gridgain/grid/kernal/managers/security/os/GridOsSecurityManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/kernal/managers/security/os/GridOsSecurityManager.java b/modules/core/src/main/java/org/gridgain/grid/kernal/managers/security/os/GridOsSecurityManager.java
index ad12b79..3831148 100644
--- a/modules/core/src/main/java/org/gridgain/grid/kernal/managers/security/os/GridOsSecurityManager.java
+++ b/modules/core/src/main/java/org/gridgain/grid/kernal/managers/security/os/GridOsSecurityManager.java
@@ -15,7 +15,7 @@ import org.gridgain.grid.*;
 import org.gridgain.grid.kernal.*;
 import org.gridgain.grid.kernal.managers.*;
 import org.gridgain.grid.kernal.managers.security.*;
-import org.gridgain.grid.security.*;
+import org.apache.ignite.plugin.security.*;
 import org.gridgain.grid.util.typedef.*;
 import org.jetbrains.annotations.*;
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/8c20bddb/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/GridCacheAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/GridCacheAdapter.java b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/GridCacheAdapter.java
index a7e01da..35bec68 100644
--- a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/GridCacheAdapter.java
+++ b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/GridCacheAdapter.java
@@ -31,7 +31,7 @@ import org.gridgain.grid.kernal.processors.cache.dr.*;
 import org.gridgain.grid.kernal.processors.cache.query.*;
 import org.gridgain.grid.kernal.processors.dr.*;
 import org.gridgain.grid.kernal.processors.task.*;
-import org.gridgain.grid.security.*;
+import org.apache.ignite.plugin.security.*;
 import org.gridgain.grid.util.*;
 import org.gridgain.grid.util.future.*;
 import org.gridgain.grid.util.lang.*;

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/8c20bddb/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/GridCacheContext.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/GridCacheContext.java b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/GridCacheContext.java
index e62b86d..f7e4471 100644
--- a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/GridCacheContext.java
+++ b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/GridCacheContext.java
@@ -37,7 +37,7 @@ import org.gridgain.grid.kernal.processors.closure.*;
 import org.gridgain.grid.kernal.processors.offheap.*;
 import org.gridgain.grid.kernal.processors.portable.*;
 import org.gridgain.grid.kernal.processors.timeout.*;
-import org.gridgain.grid.security.*;
+import org.apache.ignite.plugin.security.*;
 import org.gridgain.grid.util.typedef.*;
 import org.gridgain.grid.util.typedef.internal.*;
 import org.gridgain.grid.util.*;

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/8c20bddb/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/GridCacheTxLocalAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/GridCacheTxLocalAdapter.java b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/GridCacheTxLocalAdapter.java
index 26b85cb..93ed43f 100644
--- a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/GridCacheTxLocalAdapter.java
+++ b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/GridCacheTxLocalAdapter.java
@@ -16,7 +16,7 @@ import org.gridgain.grid.cache.*;
 import org.gridgain.grid.kernal.processors.cache.distributed.near.*;
 import org.gridgain.grid.kernal.processors.cache.dr.*;
 import org.gridgain.grid.kernal.processors.dr.*;
-import org.gridgain.grid.security.*;
+import org.apache.ignite.plugin.security.*;
 import org.gridgain.grid.util.*;
 import org.gridgain.grid.util.future.*;
 import org.gridgain.grid.util.lang.*;

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/8c20bddb/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/distributed/dht/atomic/GridDhtAtomicCache.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/distributed/dht/atomic/GridDhtAtomicCache.java b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/distributed/dht/atomic/GridDhtAtomicCache.java
index b78eda0..a6bc796 100644
--- a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/distributed/dht/atomic/GridDhtAtomicCache.java
+++ b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/distributed/dht/atomic/GridDhtAtomicCache.java
@@ -21,7 +21,7 @@ import org.gridgain.grid.kernal.processors.cache.distributed.dht.preloader.*;
 import org.gridgain.grid.kernal.processors.cache.distributed.near.*;
 import org.gridgain.grid.kernal.processors.cache.dr.*;
 import org.gridgain.grid.kernal.processors.timeout.*;
-import org.gridgain.grid.security.*;
+import org.apache.ignite.plugin.security.*;
 import org.gridgain.grid.util.*;
 import org.gridgain.grid.util.future.*;
 import org.gridgain.grid.util.lang.*;

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/8c20bddb/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/distributed/dht/colocated/GridDhtColocatedCache.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/distributed/dht/colocated/GridDhtColocatedCache.java b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/distributed/dht/colocated/GridDhtColocatedCache.java
index cd737a8..96d837c 100644
--- a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/distributed/dht/colocated/GridDhtColocatedCache.java
+++ b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/distributed/dht/colocated/GridDhtColocatedCache.java
@@ -18,7 +18,7 @@ import org.gridgain.grid.kernal.processors.cache.*;
 import org.gridgain.grid.kernal.processors.cache.distributed.*;
 import org.gridgain.grid.kernal.processors.cache.distributed.dht.*;
 import org.gridgain.grid.kernal.processors.cache.distributed.near.*;
-import org.gridgain.grid.security.*;
+import org.apache.ignite.plugin.security.*;
 import org.gridgain.grid.util.typedef.*;
 import org.gridgain.grid.util.typedef.internal.*;
 import org.gridgain.grid.util.future.*;

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/8c20bddb/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/distributed/near/GridNearAtomicCache.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/distributed/near/GridNearAtomicCache.java b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/distributed/near/GridNearAtomicCache.java
index 7815538..7abba50 100644
--- a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/distributed/near/GridNearAtomicCache.java
+++ b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/distributed/near/GridNearAtomicCache.java
@@ -16,7 +16,7 @@ import org.gridgain.grid.kernal.processors.cache.*;
 import org.gridgain.grid.kernal.processors.cache.distributed.dht.*;
 import org.gridgain.grid.kernal.processors.cache.distributed.dht.atomic.*;
 import org.gridgain.grid.kernal.processors.cache.dr.*;
-import org.gridgain.grid.security.*;
+import org.apache.ignite.plugin.security.*;
 import org.gridgain.grid.util.*;
 import org.gridgain.grid.util.future.*;
 import org.gridgain.grid.util.typedef.*;

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/8c20bddb/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/distributed/near/GridNearTransactionalCache.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/distributed/near/GridNearTransactionalCache.java b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/distributed/near/GridNearTransactionalCache.java
index 576a547..918eec1 100644
--- a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/distributed/near/GridNearTransactionalCache.java
+++ b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/distributed/near/GridNearTransactionalCache.java
@@ -16,7 +16,7 @@ import org.gridgain.grid.cache.*;
 import org.gridgain.grid.kernal.processors.cache.*;
 import org.gridgain.grid.kernal.processors.cache.distributed.*;
 import org.gridgain.grid.kernal.processors.cache.distributed.dht.*;
-import org.gridgain.grid.security.*;
+import org.apache.ignite.plugin.security.*;
 import org.gridgain.grid.util.typedef.*;
 import org.gridgain.grid.util.typedef.internal.*;
 import org.gridgain.grid.util.future.*;

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/8c20bddb/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/local/atomic/GridLocalAtomicCache.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/local/atomic/GridLocalAtomicCache.java b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/local/atomic/GridLocalAtomicCache.java
index 802e772..46cc579 100644
--- a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/local/atomic/GridLocalAtomicCache.java
+++ b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/local/atomic/GridLocalAtomicCache.java
@@ -14,7 +14,7 @@ import org.gridgain.grid.*;
 import org.gridgain.grid.cache.*;
 import org.gridgain.grid.kernal.processors.cache.*;
 import org.gridgain.grid.kernal.processors.cache.local.*;
-import org.gridgain.grid.security.*;
+import org.apache.ignite.plugin.security.*;
 import org.gridgain.grid.util.*;
 import org.gridgain.grid.util.future.*;
 import org.gridgain.grid.util.typedef.*;

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/8c20bddb/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/query/GridCacheQueryAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/query/GridCacheQueryAdapter.java b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/query/GridCacheQueryAdapter.java
index 4931e04..470ed24 100644
--- a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/query/GridCacheQueryAdapter.java
+++ b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/query/GridCacheQueryAdapter.java
@@ -16,7 +16,7 @@ import org.gridgain.grid.*;
 import org.gridgain.grid.cache.*;
 import org.gridgain.grid.cache.query.*;
 import org.gridgain.grid.kernal.processors.cache.*;
-import org.gridgain.grid.security.*;
+import org.apache.ignite.plugin.security.*;
 import org.gridgain.grid.util.typedef.*;
 import org.gridgain.grid.util.typedef.internal.*;
 import org.jetbrains.annotations.*;

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/8c20bddb/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/query/continuous/GridCacheContinuousQueryAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/query/continuous/GridCacheContinuousQueryAdapter.java b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/query/continuous/GridCacheContinuousQueryAdapter.java
index 6384cd8..910f563 100644
--- a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/query/continuous/GridCacheContinuousQueryAdapter.java
+++ b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/query/continuous/GridCacheContinuousQueryAdapter.java
@@ -18,7 +18,7 @@ import org.gridgain.grid.cache.query.*;
 import org.gridgain.grid.cache.query.GridCacheContinuousQueryEntry;
 import org.gridgain.grid.kernal.processors.cache.*;
 import org.gridgain.grid.kernal.processors.continuous.*;
-import org.gridgain.grid.security.*;
+import org.apache.ignite.plugin.security.*;
 import org.gridgain.grid.util.typedef.*;
 import org.gridgain.grid.util.typedef.internal.*;
 import org.gridgain.grid.util.*;

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/8c20bddb/modules/core/src/main/java/org/gridgain/grid/kernal/processors/rest/GridRestProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/rest/GridRestProcessor.java b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/rest/GridRestProcessor.java
index 360af76..1f696ba 100644
--- a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/rest/GridRestProcessor.java
+++ b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/rest/GridRestProcessor.java
@@ -27,7 +27,7 @@ import org.gridgain.grid.kernal.processors.rest.handlers.top.*;
 import org.gridgain.grid.kernal.processors.rest.handlers.version.*;
 import org.gridgain.grid.kernal.processors.rest.protocols.tcp.*;
 import org.gridgain.grid.kernal.processors.rest.request.*;
-import org.gridgain.grid.security.*;
+import org.apache.ignite.plugin.security.*;
 import org.gridgain.grid.util.*;
 import org.gridgain.grid.util.future.*;
 import org.gridgain.grid.util.typedef.*;
@@ -40,7 +40,7 @@ import java.util.*;
 import java.util.concurrent.*;
 
 import static org.gridgain.grid.kernal.processors.rest.GridRestResponse.*;
-import static org.gridgain.grid.security.GridSecuritySubjectType.*;
+import static org.apache.ignite.plugin.security.GridSecuritySubjectType.*;
 
 /**
  * Rest processor implementation.

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/8c20bddb/modules/core/src/main/java/org/gridgain/grid/kernal/processors/task/GridTaskProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/task/GridTaskProcessor.java b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/task/GridTaskProcessor.java
index d58717e..0874db1 100644
--- a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/task/GridTaskProcessor.java
+++ b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/task/GridTaskProcessor.java
@@ -21,7 +21,7 @@ import org.gridgain.grid.kernal.managers.communication.*;
 import org.gridgain.grid.kernal.managers.deployment.*;
 import org.gridgain.grid.kernal.managers.eventstorage.*;
 import org.gridgain.grid.kernal.processors.*;
-import org.gridgain.grid.security.*;
+import org.apache.ignite.plugin.security.*;
 import org.gridgain.grid.util.*;
 import org.gridgain.grid.util.lang.*;
 import org.gridgain.grid.util.typedef.*;


[09/20] incubator-ignite git commit: # Renaming

Posted by sb...@apache.org.
# Renaming


Project: http://git-wip-us.apache.org/repos/asf/incubator-ignite/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-ignite/commit/f2f4a325
Tree: http://git-wip-us.apache.org/repos/asf/incubator-ignite/tree/f2f4a325
Diff: http://git-wip-us.apache.org/repos/asf/incubator-ignite/diff/f2f4a325

Branch: refs/heads/master
Commit: f2f4a3259af0e0a94b1d5ff8bc10835cdbb313f2
Parents: b69a23c
Author: sboikov <sb...@gridgain.com>
Authored: Fri Dec 5 17:11:02 2014 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Fri Dec 5 17:11:02 2014 +0300

----------------------------------------------------------------------
 .../src/test/resources/spring-server-node.xml   |   2 +-
 .../test/resources/spring-server-ssl-node.xml   |   2 +-
 .../configuration/IgniteConfiguration.java      |  15 ++-
 .../org/apache/ignite/spi/IgniteSpiAdapter.java |   4 +-
 .../spi/securesession/SecureSessionSpi.java     |  84 ++++++++++++++
 .../noop/NoopSecureSessionSpi.java              | 115 +++++++++++++++++++
 .../noop/NoopSecureSessionSpiMBean.java         |  21 ++++
 .../ignite/spi/securesession/noop/package.html  |  15 +++
 .../ignite/spi/securesession/package.html       |  15 +++
 .../org/gridgain/grid/kernal/GridGainEx.java    |   8 +-
 .../org/gridgain/grid/kernal/GridKernal.java    |   4 +-
 .../grid/spi/securesession/noop/package.html    |  15 ---
 .../grid/spi/securesession/package.html         |  15 ---
 13 files changed, 267 insertions(+), 48 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/f2f4a325/modules/clients/src/test/resources/spring-server-node.xml
----------------------------------------------------------------------
diff --git a/modules/clients/src/test/resources/spring-server-node.xml b/modules/clients/src/test/resources/spring-server-node.xml
index 4712d9e..c77c119 100644
--- a/modules/clients/src/test/resources/spring-server-node.xml
+++ b/modules/clients/src/test/resources/spring-server-node.xml
@@ -272,7 +272,7 @@
             Use remember me secure session SPI.
         -->
         <property name="secureSessionSpi">
-            <bean class="org.gridgain.grid.spi.securesession.rememberme.GridRememberMeSecureSessionSpi">
+            <bean class="org.apache.ignite.spi.securesession.rememberme.GridRememberMeSecureSessionSpi">
                 <!-- Session TTL = 1 hour -->
                 <property name="ttl" value="3600000"/>
             </bean>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/f2f4a325/modules/clients/src/test/resources/spring-server-ssl-node.xml
----------------------------------------------------------------------
diff --git a/modules/clients/src/test/resources/spring-server-ssl-node.xml b/modules/clients/src/test/resources/spring-server-ssl-node.xml
index a40d4b5..08c4ddb 100644
--- a/modules/clients/src/test/resources/spring-server-ssl-node.xml
+++ b/modules/clients/src/test/resources/spring-server-ssl-node.xml
@@ -239,7 +239,7 @@
             Use remember me secure session SPI.
         -->
         <property name="secureSessionSpi">
-            <bean class="org.gridgain.grid.spi.securesession.rememberme.GridRememberMeSecureSessionSpi">
+            <bean class="org.apache.ignite.spi.securesession.rememberme.GridRememberMeSecureSessionSpi">
                 <!-- Session TTL = 1 hour -->
                 <property name="ttl" value="3600000"/>
             </bean>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/f2f4a325/modules/core/src/main/java/org/apache/ignite/configuration/IgniteConfiguration.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/configuration/IgniteConfiguration.java b/modules/core/src/main/java/org/apache/ignite/configuration/IgniteConfiguration.java
index f8b2547..e3dd068 100644
--- a/modules/core/src/main/java/org/apache/ignite/configuration/IgniteConfiguration.java
+++ b/modules/core/src/main/java/org/apache/ignite/configuration/IgniteConfiguration.java
@@ -36,8 +36,7 @@ import org.apache.ignite.spi.discovery.*;
 import org.apache.ignite.spi.eventstorage.*;
 import org.apache.ignite.spi.failover.*;
 import org.apache.ignite.spi.loadbalancing.*;
-import org.gridgain.grid.spi.securesession.*;
-import org.gridgain.grid.spi.securesession.noop.*;
+import org.apache.ignite.spi.securesession.*;
 import org.gridgain.grid.spi.swapspace.*;
 import org.gridgain.grid.spi.swapspace.file.*;
 import org.gridgain.grid.util.typedef.internal.*;
@@ -354,7 +353,7 @@ public class IgniteConfiguration {
     private AuthenticationSpi authSpi;
 
     /** Secure session SPI. */
-    private GridSecureSessionSpi sesSpi;
+    private SecureSessionSpi sesSpi;
 
     /** Deployment SPI. */
     private DeploymentSpi deploySpi;
@@ -1991,22 +1990,22 @@ public class IgniteConfiguration {
 
     /**
      * Should return fully configured secure session SPI implementation. If not provided,
-     * {@link GridNoopSecureSessionSpi} will be used.
+     * {@link org.apache.ignite.spi.securesession.noop.NoopSecureSessionSpi} will be used.
      *
      * @return Grid secure session SPI implementation or {@code null} to use default implementation.
      */
-    public GridSecureSessionSpi getSecureSessionSpi() {
+    public SecureSessionSpi getSecureSessionSpi() {
         return sesSpi;
     }
 
     /**
-     * Sets fully configured instance of {@link GridSecureSessionSpi}.
+     * Sets fully configured instance of {@link org.apache.ignite.spi.securesession.SecureSessionSpi}.
      *
-     * @param sesSpi Fully configured instance of {@link GridSecureSessionSpi} or
+     * @param sesSpi Fully configured instance of {@link org.apache.ignite.spi.securesession.SecureSessionSpi} or
      * {@code null} if no SPI provided.
      * @see IgniteConfiguration#getSecureSessionSpi()
      */
-    public void setSecureSessionSpi(GridSecureSessionSpi sesSpi) {
+    public void setSecureSessionSpi(SecureSessionSpi sesSpi) {
         this.sesSpi = sesSpi;
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/f2f4a325/modules/core/src/main/java/org/apache/ignite/spi/IgniteSpiAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/IgniteSpiAdapter.java b/modules/core/src/main/java/org/apache/ignite/spi/IgniteSpiAdapter.java
index 2dca36a..9969650 100644
--- a/modules/core/src/main/java/org/apache/ignite/spi/IgniteSpiAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/spi/IgniteSpiAdapter.java
@@ -19,7 +19,7 @@ import org.gridgain.grid.kernal.*;
 import org.gridgain.grid.kernal.managers.communication.*;
 import org.gridgain.grid.kernal.managers.eventstorage.*;
 import org.gridgain.grid.security.*;
-import org.gridgain.grid.spi.securesession.*;
+import org.apache.ignite.spi.securesession.*;
 import org.gridgain.grid.spi.swapspace.*;
 import org.gridgain.grid.util.direct.*;
 import org.gridgain.grid.util.typedef.*;
@@ -139,7 +139,7 @@ public abstract class IgniteSpiAdapter implements IgniteSpi, IgniteSpiManagement
 
         // Always run consistency check for security SPIs.
         final boolean secSpi = AuthenticationSpi.class.isAssignableFrom(getClass()) ||
-            GridSecureSessionSpi.class.isAssignableFrom(getClass());
+            SecureSessionSpi.class.isAssignableFrom(getClass());
 
         final boolean check = secSpi || !Boolean.getBoolean(GG_SKIP_CONFIGURATION_CONSISTENCY_CHECK);
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/f2f4a325/modules/core/src/main/java/org/apache/ignite/spi/securesession/SecureSessionSpi.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/securesession/SecureSessionSpi.java b/modules/core/src/main/java/org/apache/ignite/spi/securesession/SecureSessionSpi.java
new file mode 100644
index 0000000..4fbff22
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/spi/securesession/SecureSessionSpi.java
@@ -0,0 +1,84 @@
+/* @java.file.header */
+
+/*  _________        _____ __________________        _____
+ *  __  ____/___________(_)______  /__  ____/______ ____(_)_______
+ *  _  / __  __  ___/__  / _  __  / _  / __  _  __ `/__  / __  __ \
+ *  / /_/ /  _  /    _  /  / /_/ /  / /_/ /  / /_/ / _  /  _  / / /
+ *  \____/   /_/     /_/   \_,__/   \____/   \__,_/  /_/   /_/ /_/
+ */
+
+package org.apache.ignite.spi.securesession;
+
+import org.apache.ignite.spi.*;
+import org.gridgain.grid.security.*;
+import org.jetbrains.annotations.*;
+
+import java.util.*;
+
+/**
+ * Secure session SPI allows for session creation and validation, typically after authentication
+ * has successfully happened. The main purpose of this SPI is to ensure that remote clients are
+ * authenticated only once and upon successful authentication get issued a secure session token
+ * to reuse for consequent requests (very much the same way like HTTP sessions work).
+ * <p>
+ * The default secure session SPI is {@link org.apache.ignite.spi.securesession.noop.NoopSecureSessionSpi}
+ * which permits any request.
+ * <p>
+ * Gridgain provides the following {@code GridSecureSessionSpi} implementations:
+ * <ul>
+ * <li>
+ *     {@link org.apache.ignite.spi.securesession.noop.NoopSecureSessionSpi} - permits any request.
+ * </li>
+ * <li>
+ *     {@code GridRememberMeSecureSessionSpi} -
+ *     validates client session with remember-me session token.
+ * </li>
+ * </ul>
+ * <p>
+ * <b>NOTE:</b> that multiple secure session SPIs may be started on the same grid node. In this case
+ * GridGain will differentiate between them based on {@link #supported(GridSecuritySubjectType)}
+ * value. The first SPI which returns {@code true} for a given subject type will be used for
+ * session validation.
+ * <p>
+ * <b>NOTE:</b> this SPI (i.e. methods in this interface) should never be used directly. SPIs provide
+ * internal view on the subsystem and is used internally by GridGain kernal. In rare use cases when
+ * access to a specific implementation of this SPI is required - an instance of this SPI can be obtained
+ * via {@link org.apache.ignite.Ignite#configuration()} method to check its configuration properties or call other non-SPI
+ * methods. Note again that calling methods from this interface on the obtained instance can lead
+ * to undefined behavior and explicitly not supported.
+ */
+public interface SecureSessionSpi extends IgniteSpi {
+    /**
+     * Checks if given subject is supported by this SPI. If not, then next secure session SPI
+     * in the list will be checked.
+     *
+     * @param subjType Subject type.
+     * @return {@code True} if subject type is supported, {@code false} otherwise.
+     */
+    public boolean supported(GridSecuritySubjectType subjType);
+
+    /**
+     * Validates given session token.
+     *
+     * @param subjType Subject type.
+     * @param subjId Unique subject ID such as local or remote node ID, client ID, etc.
+     * @param tok Token to validate.
+     * @param params Additional implementation-specific parameters.
+     * @return {@code True} if session token is valid, {@code false} otherwise.
+     * @throws org.apache.ignite.spi.IgniteSpiException If validation resulted in system error. Note that
+     *      bad credentials should not cause this exception.
+     */
+    public boolean validate(GridSecuritySubjectType subjType, UUID subjId, byte[] tok,
+        @Nullable Object params) throws IgniteSpiException;
+
+    /**
+     * Generates new session token.
+     *
+     * @param subjType Subject type.
+     * @param subjId Unique subject ID such as local or remote node ID, client ID, etc.
+     * @param params Additional implementation-specific parameters.
+     * @return Session token that should be used for further validation.
+     */
+    public byte[] generateSessionToken(GridSecuritySubjectType subjType, UUID subjId, @Nullable Object params)
+        throws IgniteSpiException;
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/f2f4a325/modules/core/src/main/java/org/apache/ignite/spi/securesession/noop/NoopSecureSessionSpi.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/securesession/noop/NoopSecureSessionSpi.java b/modules/core/src/main/java/org/apache/ignite/spi/securesession/noop/NoopSecureSessionSpi.java
new file mode 100644
index 0000000..e281f02
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/spi/securesession/noop/NoopSecureSessionSpi.java
@@ -0,0 +1,115 @@
+/* @java.file.header */
+
+/*  _________        _____ __________________        _____
+ *  __  ____/___________(_)______  /__  ____/______ ____(_)_______
+ *  _  / __  __  ___/__  / _  __  / _  / __  _  __ `/__  / __  __ \
+ *  / /_/ /  _  /    _  /  / /_/ /  / /_/ /  / /_/ / _  /  _  / / /
+ *  \____/   /_/     /_/   \_,__/   \____/   \__,_/  /_/   /_/ /_/
+ */
+
+package org.apache.ignite.spi.securesession.noop;
+
+import org.apache.ignite.*;
+import org.apache.ignite.resources.*;
+import org.apache.ignite.spi.*;
+import org.gridgain.grid.security.*;
+import org.apache.ignite.spi.securesession.*;
+import org.gridgain.grid.util.typedef.internal.*;
+import org.jetbrains.annotations.*;
+
+import java.util.*;
+
+/**
+ * Default no-op implementation of the secure session SPI which supports all subject types and denies any token.
+ * <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 no optional configuration parameters.
+ * <h2 class="header">Java Example</h2>
+ * GridNoopSecureSessionSpi is used by default and has no parameters to be explicitly configured.
+ * <pre name="code" class="java">
+ * GridNoopSecureSessionSpi spi = new GridNoopSecureSessionSpi();
+ *
+ * GridConfiguration cfg = new GridConfiguration();
+ *
+ * // Override default SecureSession SPI.
+ * cfg.setSecureSessionSpi(spi);
+ *
+ * // Start grid.
+ * GridGain.start(cfg);
+ * </pre>
+ * <h2 class="header">Spring Example</h2>
+ * GridNoopSecureSessionSpi can be configured from Spring XML configuration file:
+ * <pre name="code" class="xml">
+ * &lt;bean id="grid.custom.cfg" class="org.gridgain.grid.GridConfiguration" singleton="true"&gt;
+ *         ...
+ *         &lt;property name="secureSessionSpi"&gt;
+ *             &lt;bean class="org.gridgain.grid.spi.SecureSession.noop.GridNoopSecureSessionSpi"/&gt;
+ *         &lt;/property&gt;
+ *         ...
+ * &lt;/bean&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.apache.ignite.spi.securesession.SecureSessionSpi
+ */
+@IgniteSpiNoop
+@IgniteSpiMultipleInstancesSupport(true)
+public class NoopSecureSessionSpi extends IgniteSpiAdapter
+    implements SecureSessionSpi, NoopSecureSessionSpiMBean {
+    /** Empty bytes array. */
+    private static final byte[] EMPTY_BYTE_ARRAY = new byte[0];
+
+    /** Injected grid logger. */
+    @IgniteLoggerResource
+    private IgniteLogger log;
+
+    /** {@inheritDoc} */
+    @Override public boolean supported(GridSecuritySubjectType subjType) {
+        // If this SPI is included, then session management is disabled.
+        return true;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean validate(GridSecuritySubjectType subjType, UUID subjId, @Nullable byte[] tok,
+        @Nullable Object params) throws IgniteSpiException {
+        // Never validate any token - all tokens are invalid.
+        return false;
+    }
+
+    /** {@inheritDoc} */
+    @Override public byte[] generateSessionToken(GridSecuritySubjectType subjType, UUID subjId,
+        @Nullable Object params) {
+        return EMPTY_BYTE_ARRAY;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void spiStart(String gridName) throws IgniteSpiException {
+        // Start SPI start stopwatch.
+        startStopwatch();
+
+        registerMBean(gridName, this, NoopSecureSessionSpiMBean.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 String toString() {
+        return S.toString(NoopSecureSessionSpi.class, this);
+    }
+}

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

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/f2f4a325/modules/core/src/main/java/org/apache/ignite/spi/securesession/noop/package.html
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/securesession/noop/package.html b/modules/core/src/main/java/org/apache/ignite/spi/securesession/noop/package.html
new file mode 100644
index 0000000..0590db2
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/spi/securesession/noop/package.html
@@ -0,0 +1,15 @@
+<!DOCTYPE html PUBLIC "-//W3C//DTD HTML 4.01 Transitional//EN" "http://www.w3.org/TR/html4/loose.dtd">
+<!--
+    @html.file.header
+    _________        _____ __________________        _____
+    __  ____/___________(_)______  /__  ____/______ ____(_)_______
+    _  / __  __  ___/__  / _  __  / _  / __  _  __ `/__  / __  __ \
+    / /_/ /  _  /    _  /  / /_/ /  / /_/ /  / /_/ / _  /  _  / / /
+    \____/   /_/     /_/   \_,__/   \____/   \__,_/  /_/   /_/ /_/
+-->
+<html>
+<body>
+    <!-- Package description. -->
+    Contains <b>default</b> no-op implementation for secure session SPI.
+</body>
+</html>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/f2f4a325/modules/core/src/main/java/org/apache/ignite/spi/securesession/package.html
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/securesession/package.html b/modules/core/src/main/java/org/apache/ignite/spi/securesession/package.html
new file mode 100644
index 0000000..4328ca2
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/spi/securesession/package.html
@@ -0,0 +1,15 @@
+<!DOCTYPE html PUBLIC "-//W3C//DTD HTML 4.01 Transitional//EN" "http://www.w3.org/TR/html4/loose.dtd">
+<!--
+    @html.file.header
+    _________        _____ __________________        _____
+    __  ____/___________(_)______  /__  ____/______ ____(_)_______
+    _  / __  __  ___/__  / _  __  / _  / __  _  __ `/__  / __  __ \
+    / /_/ /  _  /    _  /  / /_/ /  / /_/ /  / /_/ / _  /  _  / / /
+    \____/   /_/     /_/   \_,__/   \____/   \__,_/  /_/   /_/ /_/
+-->
+<html>
+<body>
+    <!-- Package description. -->
+    Contains APIs for secure session SPI.
+</body>
+</html>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/f2f4a325/modules/core/src/main/java/org/gridgain/grid/kernal/GridGainEx.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/kernal/GridGainEx.java b/modules/core/src/main/java/org/gridgain/grid/kernal/GridGainEx.java
index 454b9c7..2adda16 100644
--- a/modules/core/src/main/java/org/gridgain/grid/kernal/GridGainEx.java
+++ b/modules/core/src/main/java/org/gridgain/grid/kernal/GridGainEx.java
@@ -50,8 +50,8 @@ import org.apache.ignite.spi.failover.*;
 import org.apache.ignite.spi.failover.always.*;
 import org.apache.ignite.spi.loadbalancing.*;
 import org.apache.ignite.spi.loadbalancing.roundrobin.*;
-import org.gridgain.grid.spi.securesession.*;
-import org.gridgain.grid.spi.securesession.noop.*;
+import org.apache.ignite.spi.securesession.*;
+import org.apache.ignite.spi.securesession.noop.*;
 import org.gridgain.grid.spi.swapspace.*;
 import org.gridgain.grid.spi.swapspace.file.*;
 import org.gridgain.grid.spi.swapspace.noop.*;
@@ -1468,7 +1468,7 @@ public class GridGainEx {
             EventStorageSpi evtSpi = cfg.getEventStorageSpi();
             CollisionSpi colSpi = cfg.getCollisionSpi();
             AuthenticationSpi authSpi = cfg.getAuthenticationSpi();
-            GridSecureSessionSpi sesSpi = cfg.getSecureSessionSpi();
+            SecureSessionSpi sesSpi = cfg.getSecureSessionSpi();
             DeploymentSpi deploySpi = cfg.getDeploymentSpi();
             CheckpointSpi[] cpSpi = cfg.getCheckpointSpi();
             FailoverSpi[] failSpi = cfg.getFailoverSpi();
@@ -1689,7 +1689,7 @@ public class GridGainEx {
                 authSpi = new NoopAuthenticationSpi();
 
             if (sesSpi == null)
-                sesSpi = new GridNoopSecureSessionSpi();
+                sesSpi = new NoopSecureSessionSpi();
 
             if (deploySpi == null)
                 deploySpi = new LocalDeploymentSpi();

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/f2f4a325/modules/core/src/main/java/org/gridgain/grid/kernal/GridKernal.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/kernal/GridKernal.java b/modules/core/src/main/java/org/gridgain/grid/kernal/GridKernal.java
index cc9fd05..1f64654 100644
--- a/modules/core/src/main/java/org/gridgain/grid/kernal/GridKernal.java
+++ b/modules/core/src/main/java/org/gridgain/grid/kernal/GridKernal.java
@@ -63,7 +63,7 @@ import org.gridgain.grid.kernal.processors.streamer.*;
 import org.gridgain.grid.kernal.processors.task.*;
 import org.gridgain.grid.kernal.processors.timeout.*;
 import org.gridgain.grid.security.*;
-import org.gridgain.grid.spi.securesession.noop.*;
+import org.apache.ignite.spi.securesession.noop.*;
 import org.gridgain.grid.util.*;
 import org.gridgain.grid.util.future.*;
 import org.gridgain.grid.util.lang.*;
@@ -1194,7 +1194,7 @@ public class GridKernal extends ClusterGroupAdapter implements GridEx, IgniteMBe
         if (!F.isEmpty(cfg.getSegmentationResolvers()))
             msgs.add("Network segmentation detection.");
 
-        if (cfg.getSecureSessionSpi() != null && !(cfg.getSecureSessionSpi() instanceof GridNoopSecureSessionSpi))
+        if (cfg.getSecureSessionSpi() != null && !(cfg.getSecureSessionSpi() instanceof NoopSecureSessionSpi))
             msgs.add("Secure session SPI.");
 
         if (cfg.getAuthenticationSpi() != null && !(cfg.getAuthenticationSpi() instanceof NoopAuthenticationSpi))

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/f2f4a325/modules/core/src/main/java/org/gridgain/grid/spi/securesession/noop/package.html
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/spi/securesession/noop/package.html b/modules/core/src/main/java/org/gridgain/grid/spi/securesession/noop/package.html
deleted file mode 100644
index 0590db2..0000000
--- a/modules/core/src/main/java/org/gridgain/grid/spi/securesession/noop/package.html
+++ /dev/null
@@ -1,15 +0,0 @@
-<!DOCTYPE html PUBLIC "-//W3C//DTD HTML 4.01 Transitional//EN" "http://www.w3.org/TR/html4/loose.dtd">
-<!--
-    @html.file.header
-    _________        _____ __________________        _____
-    __  ____/___________(_)______  /__  ____/______ ____(_)_______
-    _  / __  __  ___/__  / _  __  / _  / __  _  __ `/__  / __  __ \
-    / /_/ /  _  /    _  /  / /_/ /  / /_/ /  / /_/ / _  /  _  / / /
-    \____/   /_/     /_/   \_,__/   \____/   \__,_/  /_/   /_/ /_/
--->
-<html>
-<body>
-    <!-- Package description. -->
-    Contains <b>default</b> no-op implementation for secure session SPI.
-</body>
-</html>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/f2f4a325/modules/core/src/main/java/org/gridgain/grid/spi/securesession/package.html
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/spi/securesession/package.html b/modules/core/src/main/java/org/gridgain/grid/spi/securesession/package.html
deleted file mode 100644
index 4328ca2..0000000
--- a/modules/core/src/main/java/org/gridgain/grid/spi/securesession/package.html
+++ /dev/null
@@ -1,15 +0,0 @@
-<!DOCTYPE html PUBLIC "-//W3C//DTD HTML 4.01 Transitional//EN" "http://www.w3.org/TR/html4/loose.dtd">
-<!--
-    @html.file.header
-    _________        _____ __________________        _____
-    __  ____/___________(_)______  /__  ____/______ ____(_)_______
-    _  / __  __  ___/__  / _  __  / _  / __  _  __ `/__  / __  __ \
-    / /_/ /  _  /    _  /  / /_/ /  / /_/ /  / /_/ / _  /  _  / / /
-    \____/   /_/     /_/   \_,__/   \____/   \__,_/  /_/   /_/ /_/
--->
-<html>
-<body>
-    <!-- Package description. -->
-    Contains APIs for secure session SPI.
-</body>
-</html>


[07/20] incubator-ignite git commit: # Renaming

Posted by sb...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/b69a23cd/modules/core/src/main/java/org/apache/ignite/spi/loadbalancing/roundrobin/RoundRobinLoadBalancingSpi.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/loadbalancing/roundrobin/RoundRobinLoadBalancingSpi.java b/modules/core/src/main/java/org/apache/ignite/spi/loadbalancing/roundrobin/RoundRobinLoadBalancingSpi.java
new file mode 100644
index 0000000..bb49cc0
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/spi/loadbalancing/roundrobin/RoundRobinLoadBalancingSpi.java
@@ -0,0 +1,319 @@
+/* @java.file.header */
+
+/*  _________        _____ __________________        _____
+ *  __  ____/___________(_)______  /__  ____/______ ____(_)_______
+ *  _  / __  __  ___/__  / _  __  / _  / __  _  __ `/__  / __  __ \
+ *  / /_/ /  _  /    _  /  / /_/ /  / /_/ /  / /_/ / _  /  _  / / /
+ *  \____/   /_/     /_/   \_,__/   \____/   \__,_/  /_/   /_/ /_/
+ */
+
+package org.apache.ignite.spi.loadbalancing.roundrobin;
+
+import org.apache.ignite.*;
+import org.apache.ignite.cluster.*;
+import org.apache.ignite.compute.*;
+import org.apache.ignite.events.*;
+import org.apache.ignite.lang.*;
+import org.apache.ignite.resources.*;
+import org.apache.ignite.spi.*;
+import org.gridgain.grid.*;
+import org.gridgain.grid.kernal.managers.eventstorage.*;
+import org.apache.ignite.spi.loadbalancing.*;
+import org.gridgain.grid.util.typedef.internal.*;
+import org.jdk8.backport.*;
+import org.jetbrains.annotations.*;
+
+import java.util.*;
+
+import static org.apache.ignite.events.IgniteEventType.*;
+
+/**
+ * This SPI iterates through nodes in round-robin fashion and pick the next
+ * sequential node. Two modes of operation are supported: per-task and global
+ * (see {@link #setPerTask(boolean)} configuration).
+ * <p>
+ * When configured in per-task mode, implementation will pick a random starting
+ * node at the beginning of every task execution and then sequentially iterate through all
+ * nodes in topology starting from the picked node. This is the default configuration
+ * and should fit most of the use cases as it provides a fairly well-distributed
+ * split and also ensures that jobs within a single task are spread out across
+ * nodes to the maximum. For cases when split size is equal to the number of nodes,
+ * this mode guarantees that all nodes will participate in the split.
+ * <p>
+ * When configured in global mode, a single sequential queue of nodes is maintained for
+ * all tasks and the next node in the queue is picked every time. In this mode (unlike in
+ * {@code per-task} mode) it is possible that even if split size may be equal to the
+ * number of nodes, some jobs within the same task will be assigned to the same node if
+ * multiple tasks are executing concurrently.
+ * <h1 class="header">Coding Example</h1>
+ * If you are using {@link org.apache.ignite.compute.ComputeTaskSplitAdapter} then load balancing logic
+ * is transparent to your code and is handled automatically by the adapter.
+ * Here is an example of how your task will look:
+ * <pre name="code" class="java">
+ * public class MyFooBarTask extends GridComputeTaskSplitAdapter&lt;Object, Object&gt; {
+ *    &#64;Override
+ *    protected Collection&lt;? extends GridComputeJob&gt; split(int gridSize, Object arg) throws GridException {
+ *        List&lt;MyFooBarJob&gt; jobs = new ArrayList&lt;MyFooBarJob&gt;(gridSize);
+ *
+ *        for (int i = 0; i &lt; gridSize; i++) {
+ *            jobs.add(new MyFooBarJob(arg));
+ *        }
+ *
+ *        // Node assignment via load balancer
+ *        // happens automatically.
+ *        return jobs;
+ *    }
+ *    ...
+ * }
+ * </pre>
+ * If you need more fine-grained control over how some jobs within task get mapped to a node
+ * and use affinity load balancing for some other jobs within task, then you should use
+ * {@link org.apache.ignite.compute.ComputeTaskAdapter}. Here is an example of how your task will look. Note that in this
+ * case we manually inject load balancer and use it to pick the best node. Doing it in
+ * such way would allow user to map some jobs manually and for others use load balancer.
+ * <pre name="code" class="java">
+ * public class MyFooBarTask extends GridComputeTaskAdapter&lt;String, String&gt; {
+ *    // Inject load balancer.
+ *    &#64;GridLoadBalancerResource
+ *    GridComputeLoadBalancer balancer;
+ *
+ *    // Map jobs to grid nodes.
+ *    public Map&lt;? extends GridComputeJob, GridNode&gt; map(List&lt;GridNode&gt; subgrid, String arg) throws GridException {
+ *        Map&lt;MyFooBarJob, GridNode&gt; jobs = new HashMap&lt;MyFooBarJob, GridNode&gt;(subgrid.size());
+ *
+ *        // In more complex cases, you can actually do
+ *        // more complicated assignments of jobs to nodes.
+ *        for (int i = 0; i &lt; subgrid.size(); i++) {
+ *            // Pick the next best balanced node for the job.
+ *            jobs.put(new MyFooBarJob(arg), balancer.getBalancedNode())
+ *        }
+ *
+ *        return jobs;
+ *    }
+ *
+ *    // Aggregate results into one compound result.
+ *    public String reduce(List&lt;GridComputeJobResult&gt; results) throws GridException {
+ *        // For the purpose of this example we simply
+ *        // concatenate string representation of every
+ *        // job result
+ *        StringBuilder buf = new StringBuilder();
+ *
+ *        for (GridComputeJobResult res : results) {
+ *            // Append string representation of result
+ *            // returned by every job.
+ *            buf.append(res.getData().string());
+ *        }
+ *
+ *        return buf.string();
+ *    }
+ * }
+ * </pre>
+ * <p>
+ * <h1 class="header">Configuration</h1>
+ * In order to use this load balancer, you should configure your grid instance
+ * to use {@code GridRoundRobinLoadBalancingSpi} either from Spring XML file or
+ * directly. The following configuration parameters are supported:
+ * <h2 class="header">Mandatory</h2>
+ * This SPI has no mandatory configuration parameters.
+ * <h2 class="header">Optional</h2>
+ * The following configuration parameters are optional:
+ * <ul>
+ * <li>
+ *      Flag that indicates whether to use {@code per-task} or global
+ *      round-robin modes described above (see {@link #setPerTask(boolean)}).
+ * </li>
+ * </ul>
+ * Below is Java configuration example:
+ * <pre name="code" class="java">
+ * GridRandomLoadBalancingSpi = new GridRandomLoadBalancingSpi();
+ *
+ * // Configure SPI to use global round-robin mode.
+ * spi.setPerTask(false);
+ *
+ * GridConfiguration cfg = new GridConfiguration();
+ *
+ * // Override default load balancing SPI.
+ * cfg.setLoadBalancingSpi(spi);
+ *
+ * // Starts grid.
+ * G.start(cfg);
+ * </pre>
+ * Here is how you can configure {@code GridRandomLoadBalancingSpi} using Spring XML configuration:
+ * <pre name="code" class="xml">
+ * &lt;property name="loadBalancingSpi"&gt;
+ *     &lt;bean class="org.gridgain.grid.spi.loadBalancing.roundrobin.GridRoundRobinLoadBalancingSpi"&gt;
+ *         &lt;!-- Set to global round-robin mode. --&gt;
+ *         &lt;property name="perTask" value="false"/&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>
+ */
+@IgniteSpiMultipleInstancesSupport(true)
+public class RoundRobinLoadBalancingSpi extends IgniteSpiAdapter implements LoadBalancingSpi,
+    RoundRobinLoadBalancingSpiMBean {
+    /** Grid logger. */
+    @IgniteLoggerResource
+    private IgniteLogger log;
+
+    /** */
+    private RoundRobinGlobalLoadBalancer balancer;
+
+    /** */
+    private boolean isPerTask;
+
+    /** */
+    private final Map<IgniteUuid, RoundRobinPerTaskLoadBalancer> perTaskBalancers =
+        new ConcurrentHashMap8<>();
+
+    /** Event listener. */
+    private final GridLocalEventListener lsnr = new GridLocalEventListener() {
+        @Override public void onEvent(IgniteEvent evt) {
+            if (evt.type() == EVT_TASK_FAILED ||
+                evt.type() == EVT_TASK_FINISHED)
+                perTaskBalancers.remove(((IgniteTaskEvent)evt).taskSessionId());
+            else if (evt.type() == EVT_JOB_MAPPED) {
+                RoundRobinPerTaskLoadBalancer balancer =
+                    perTaskBalancers.get(((IgniteJobEvent)evt).taskSessionId());
+
+                if (balancer != null)
+                    balancer.onMapped();
+            }
+        }
+    };
+
+    /** {@inheritDoc} */
+    @Override public boolean isPerTask() {
+        return isPerTask;
+    }
+
+    /**
+     * Configuration parameter indicating whether a new round robin order should be
+     * created for every task. If {@code true} then load balancer is guaranteed
+     * to iterate through nodes sequentially for every task - so as long as number
+     * of jobs is less than or equal to the number of nodes, jobs are guaranteed to
+     * be assigned to unique nodes. If {@code false} then one round-robin order
+     * will be maintained for all tasks, so when tasks execute concurrently, it
+     * is possible for more than one job within task to be assigned to the same
+     * node.
+     * <p>
+     * Default is {@code false}.
+     *
+     * @param isPerTask Configuration parameter indicating whether a new round robin order should
+     *      be created for every task. Default is {@code false}.
+     */
+    @IgniteSpiConfiguration(optional = true)
+    public void setPerTask(boolean isPerTask) {
+        this.isPerTask = isPerTask;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void spiStart(@Nullable String gridName) throws IgniteSpiException {
+        startStopwatch();
+
+        if (log.isDebugEnabled())
+            log.debug(configInfo("isPerTask", isPerTask));
+
+        registerMBean(gridName, this, RoundRobinLoadBalancingSpiMBean.class);
+
+        balancer = new RoundRobinGlobalLoadBalancer(log);
+
+        // Ack ok start.
+        if (log.isDebugEnabled())
+            log.debug(startInfo());
+    }
+
+    /** {@inheritDoc} */
+    @Override public void spiStop() throws IgniteSpiException {
+        balancer = null;
+
+        perTaskBalancers.clear();
+
+        unregisterMBean();
+
+        // Ack ok stop.
+        if (log.isDebugEnabled())
+            log.debug(stopInfo());
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void onContextInitialized0(IgniteSpiContext spiCtx) throws IgniteSpiException {
+        if (!isPerTask)
+            balancer.onContextInitialized(spiCtx);
+        else {
+            if (!getSpiContext().isEventRecordable(EVT_TASK_FAILED, EVT_TASK_FINISHED, EVT_JOB_MAPPED))
+                throw new IgniteSpiException("Required event types are disabled: " +
+                    U.gridEventName(EVT_TASK_FAILED) + ", " +
+                    U.gridEventName(EVT_TASK_FINISHED) + ", " +
+                    U.gridEventName(EVT_JOB_MAPPED));
+
+            getSpiContext().addLocalEventListener(lsnr, EVT_TASK_FAILED, EVT_TASK_FINISHED, EVT_JOB_MAPPED);
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void onContextDestroyed0() {
+        if (!isPerTask) {
+            if (balancer != null)
+                balancer.onContextDestroyed();
+        }
+        else {
+            IgniteSpiContext spiCtx = getSpiContext();
+
+            if (spiCtx != null)
+                spiCtx.removeLocalEventListener(lsnr);
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public ClusterNode getBalancedNode(ComputeTaskSession ses, List<ClusterNode> top, ComputeJob job)
+        throws GridException {
+        A.notNull(ses, "ses", top, "top");
+
+        if (isPerTask) {
+            // Note that every session operates from single thread which
+            // allows us to use concurrent map and avoid synchronization.
+            RoundRobinPerTaskLoadBalancer taskBalancer = perTaskBalancers.get(ses.getId());
+
+            if (taskBalancer == null)
+                perTaskBalancers.put(ses.getId(), taskBalancer = new RoundRobinPerTaskLoadBalancer());
+
+            return taskBalancer.getBalancedNode(top);
+        }
+
+        return balancer.getBalancedNode(top);
+    }
+
+    /**
+     * THIS METHOD IS USED ONLY FOR TESTING.
+     *
+     * @param ses Task session.
+     * @return Internal list of nodes.
+     */
+    List<UUID> getNodeIds(ComputeTaskSession ses) {
+        if (isPerTask) {
+            RoundRobinPerTaskLoadBalancer balancer = perTaskBalancers.get(ses.getId());
+
+            if (balancer == null)
+                return Collections.emptyList();
+
+            List<UUID> ids = new ArrayList<>();
+
+            for (ClusterNode node : balancer.getNodes()) {
+                ids.add(node.id());
+            }
+
+            return ids;
+        }
+
+        return balancer.getNodeIds();
+    }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return S.toString(RoundRobinLoadBalancingSpi.class, this);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/b69a23cd/modules/core/src/main/java/org/apache/ignite/spi/loadbalancing/roundrobin/RoundRobinLoadBalancingSpiMBean.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/loadbalancing/roundrobin/RoundRobinLoadBalancingSpiMBean.java b/modules/core/src/main/java/org/apache/ignite/spi/loadbalancing/roundrobin/RoundRobinLoadBalancingSpiMBean.java
new file mode 100644
index 0000000..987f0b4
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/spi/loadbalancing/roundrobin/RoundRobinLoadBalancingSpiMBean.java
@@ -0,0 +1,37 @@
+/* @java.file.header */
+
+/*  _________        _____ __________________        _____
+ *  __  ____/___________(_)______  /__  ____/______ ____(_)_______
+ *  _  / __  __  ___/__  / _  __  / _  / __  _  __ `/__  / __  __ \
+ *  / /_/ /  _  /    _  /  / /_/ /  / /_/ /  / /_/ / _  /  _  / / /
+ *  \____/   /_/     /_/   \_,__/   \____/   \__,_/  /_/   /_/ /_/
+ */
+
+package org.apache.ignite.spi.loadbalancing.roundrobin;
+
+import org.apache.ignite.mbean.*;
+import org.apache.ignite.spi.*;
+
+/**
+ * Management bean for {@link RoundRobinLoadBalancingSpi} SPI.
+ */
+@IgniteMBeanDescription("MBean that provides access to round robin load balancing SPI configuration.")
+public interface RoundRobinLoadBalancingSpiMBean extends IgniteSpiManagementMBean {
+    /**
+     * Configuration parameter indicating whether a new round robin order should be
+     * created for every task. If {@code true} then load balancer is guaranteed
+     * to iterate through nodes sequentially for every task - so as long as number
+     * of jobs is less than or equal to the number of nodes, jobs are guaranteed to
+     * be assigned to unique nodes. If {@code false} then one round-robin order
+     * will be maintained for all tasks, so when tasks execute concurrently, it
+     * is possible for more than one job within task to be assigned to the same
+     * node.
+     * <p>
+     * Default is {@code true}.
+     *
+     * @return Configuration parameter indicating whether a new round robin order should
+     *      be created for every task. Default is {@code true}.
+     */
+    @IgniteMBeanDescription("Configuration parameter indicating whether a new round robin order should be created for every task.")
+    public boolean isPerTask();
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/b69a23cd/modules/core/src/main/java/org/apache/ignite/spi/loadbalancing/roundrobin/RoundRobinPerTaskLoadBalancer.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/loadbalancing/roundrobin/RoundRobinPerTaskLoadBalancer.java b/modules/core/src/main/java/org/apache/ignite/spi/loadbalancing/roundrobin/RoundRobinPerTaskLoadBalancer.java
new file mode 100644
index 0000000..308256e
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/spi/loadbalancing/roundrobin/RoundRobinPerTaskLoadBalancer.java
@@ -0,0 +1,96 @@
+/* @java.file.header */
+
+/*  _________        _____ __________________        _____
+ *  __  ____/___________(_)______  /__  ____/______ ____(_)_______
+ *  _  / __  __  ___/__  / _  __  / _  / __  _  __ `/__  / __  __ \
+ *  / /_/ /  _  /    _  /  / /_/ /  / /_/ /  / /_/ / _  /  _  / / /
+ *  \____/   /_/     /_/   \_,__/   \____/   \__,_/  /_/   /_/ /_/
+ */
+
+package org.apache.ignite.spi.loadbalancing.roundrobin;
+
+import org.apache.ignite.cluster.*;
+
+import java.util.*;
+
+/**
+ * Load balancer for per-task configuration.
+ */
+class RoundRobinPerTaskLoadBalancer {
+    /** Balancing nodes. */
+    private ArrayDeque<ClusterNode> nodeQueue;
+
+    /** Jobs mapped flag. */
+    private volatile boolean isMapped;
+
+    /** Mutex. */
+    private final Object mux = new Object();
+
+    /**
+     * Call back for job mapped event.
+     */
+    void onMapped() {
+        isMapped = true;
+    }
+
+    /**
+     * Gets balanced node for given topology. This implementation
+     * is to be used only from {@link org.apache.ignite.compute.ComputeTask#map(List, Object)} method
+     * and, therefore, does not need to be thread-safe.
+     *
+     * @param top Topology to pick from.
+     * @return Best balanced node.
+     */
+    ClusterNode getBalancedNode(List<ClusterNode> top) {
+        assert top != null;
+        assert !top.isEmpty();
+
+        boolean readjust = isMapped;
+
+        synchronized (mux) {
+            // Populate first time.
+            if (nodeQueue == null)
+                nodeQueue = new ArrayDeque<>(top);
+
+            // If job has been mapped, then it means
+            // that it is most likely being failed over.
+            // In this case topology might have changed
+            // and we need to readjust with every apply.
+            if (readjust)
+                // Add missing nodes.
+                for (ClusterNode node : top)
+                    if (!nodeQueue.contains(node))
+                        nodeQueue.offer(node);
+
+            ClusterNode next = nodeQueue.poll();
+
+            // If jobs have been mapped, we need to make sure
+            // that queued node is still in topology.
+            if (readjust && next != null) {
+                while (!top.contains(next) && !nodeQueue.isEmpty())
+                    next = nodeQueue.poll();
+
+                // No nodes found and queue is empty.
+                if (next != null && !top.contains(next))
+                    return null;
+            }
+
+            if (next != null)
+                // Add to the end.
+                nodeQueue.offer(next);
+
+            return next;
+        }
+    }
+
+    /**
+     * THIS METHOD IS USED ONLY FOR TESTING.
+     *
+     * @return Internal list of nodes.
+     */
+    List<ClusterNode> getNodes() {
+        synchronized (mux) {
+            return Collections.unmodifiableList(new ArrayList<>(nodeQueue));
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/b69a23cd/modules/core/src/main/java/org/apache/ignite/spi/loadbalancing/roundrobin/package.html
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/loadbalancing/roundrobin/package.html b/modules/core/src/main/java/org/apache/ignite/spi/loadbalancing/roundrobin/package.html
new file mode 100644
index 0000000..9909144
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/spi/loadbalancing/roundrobin/package.html
@@ -0,0 +1,15 @@
+<!DOCTYPE html PUBLIC "-//W3C//DTD HTML 4.01 Transitional//EN" "http://www.w3.org/TR/html4/loose.dtd">
+<!--
+    @html.file.header
+    _________        _____ __________________        _____
+    __  ____/___________(_)______  /__  ____/______ ____(_)_______
+    _  / __  __  ___/__  / _  __  / _  / __  _  __ `/__  / __  __ \
+    / /_/ /  _  /    _  /  / /_/ /  / /_/ /  / /_/ / _  /  _  / / /
+    \____/   /_/     /_/   \_,__/   \____/   \__,_/  /_/   /_/ /_/
+-->
+<html>
+<body>
+    <!-- Package description. -->
+    Contains <b>default</b> round-robin implementation for load balancing SPI.
+</body>
+</html>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/b69a23cd/modules/core/src/main/java/org/apache/ignite/spi/loadbalancing/weightedrandom/WeightedRandomLoadBalancingSpi.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/loadbalancing/weightedrandom/WeightedRandomLoadBalancingSpi.java b/modules/core/src/main/java/org/apache/ignite/spi/loadbalancing/weightedrandom/WeightedRandomLoadBalancingSpi.java
new file mode 100644
index 0000000..6c4de36
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/spi/loadbalancing/weightedrandom/WeightedRandomLoadBalancingSpi.java
@@ -0,0 +1,394 @@
+/* @java.file.header */
+
+/*  _________        _____ __________________        _____
+ *  __  ____/___________(_)______  /__  ____/______ ____(_)_______
+ *  _  / __  __  ___/__  / _  __  / _  / __  _  __ `/__  / __  __ \
+ *  / /_/ /  _  /    _  /  / /_/ /  / /_/ /  / /_/ / _  /  _  / / /
+ *  \____/   /_/     /_/   \_,__/   \____/   \__,_/  /_/   /_/ /_/
+ */
+
+package org.apache.ignite.spi.loadbalancing.weightedrandom;
+
+import org.apache.ignite.*;
+import org.apache.ignite.cluster.*;
+import org.apache.ignite.compute.*;
+import org.apache.ignite.events.*;
+import org.apache.ignite.lang.*;
+import org.apache.ignite.resources.*;
+import org.apache.ignite.spi.*;
+import org.gridgain.grid.kernal.managers.eventstorage.*;
+import org.apache.ignite.spi.loadbalancing.*;
+import org.gridgain.grid.util.typedef.*;
+import org.gridgain.grid.util.typedef.internal.*;
+import org.jdk8.backport.*;
+import org.jetbrains.annotations.*;
+
+import java.util.*;
+import java.util.concurrent.*;
+
+import static org.apache.ignite.events.IgniteEventType.*;
+
+/**
+ * Load balancing SPI that picks a random node for job execution. Note that you can
+ * optionally assign weights to nodes, so nodes with larger weights will end up getting
+ * proportionally more jobs routed to them (see {@link #setNodeWeight(int)}
+ * configuration property). By default all nodes get equal weight defined by
+ * {@link #DFLT_NODE_WEIGHT} (value is {@code 10}).
+ * <h1 class="header">Coding Example</h1>
+ * If you are using {@link org.apache.ignite.compute.ComputeTaskSplitAdapter} then load balancing logic
+ * is transparent to your code and is handled automatically by the adapter.
+ * Here is an example of how your task could look:
+ * <pre name="code" class="java">
+ * public class MyFooBarTask extends GridComputeTaskSplitAdapter&lt;Object, Object&gt; {
+ *    &#64;Override
+ *    protected Collection&lt;? extends GridComputeJob&gt; split(int gridSize, Object arg) throws GridException {
+ *        List&lt;MyFooBarJob&gt; jobs = new ArrayList&lt;MyFooBarJob&gt;(gridSize);
+ *
+ *        for (int i = 0; i &lt; gridSize; i++) {
+ *            jobs.add(new MyFooBarJob(arg));
+ *        }
+ *
+ *        // Node assignment via load balancer
+ *        // happens automatically.
+ *        return jobs;
+ *    }
+ *    ...
+ * }
+ * </pre>
+ * If you need more fine-grained control over how some jobs within task get mapped to a node
+ * and use affinity load balancing for some other jobs within task, then you should use
+ * {@link org.apache.ignite.compute.ComputeTaskAdapter}. Here is an example of how your task will look. Note that in this
+ * case we manually inject load balancer and use it to pick the best node. Doing it in
+ * such way would allow user to map some jobs manually and for others use load balancer.
+ * <pre name="code" class="java">
+ * public class MyFooBarTask extends GridComputeTaskAdapter&lt;String, String&gt; {
+ *    // Inject load balancer.
+ *    &#64;GridLoadBalancerResource
+ *    GridComputeLoadBalancer balancer;
+ *
+ *    // Map jobs to grid nodes.
+ *    public Map&lt;? extends GridComputeJob, GridNode&gt; map(List&lt;GridNode&gt; subgrid, String arg) throws GridException {
+ *        Map&lt;MyFooBarJob, GridNode&gt; jobs = new HashMap&lt;MyFooBarJob, GridNode&gt;(subgrid.size());
+ *
+ *        // In more complex cases, you can actually do
+ *        // more complicated assignments of jobs to nodes.
+ *        for (int i = 0; i &lt; subgrid.size(); i++) {
+ *            // Pick the next best balanced node for the job.
+ *            jobs.put(new MyFooBarJob(arg), balancer.getBalancedNode())
+ *        }
+ *
+ *        return jobs;
+ *    }
+ *
+ *    // Aggregate results into one compound result.
+ *    public String reduce(List&lt;GridComputeJobResult&gt; results) throws GridException {
+ *        // For the purpose of this example we simply
+ *        // concatenate string representation of every
+ *        // job result
+ *        StringBuilder buf = new StringBuilder();
+ *
+ *        for (GridComputeJobResult res : results) {
+ *            // Append string representation of result
+ *            // returned by every job.
+ *            buf.append(res.getData().string());
+ *        }
+ *
+ *        return buf.string();
+ *    }
+ * }
+ * </pre>
+ * <p>
+ * <h1 class="header">Configuration</h1>
+ * In order to use this load balancer, you should configure your grid instance
+ * to use {@code GridRandomLoadBalancingSpi} either from Spring XML file or
+ * directly. The following configuration parameters are supported:
+ * <h2 class="header">Mandatory</h2>
+ * This SPI has no mandatory configuration parameters.
+ * <h2 class="header">Optional</h2>
+ * The following configuration parameters are optional:
+ * <ul>
+ * <li>
+ *      Flag that indicates whether to use weight policy or simple random policy
+ *      (see {@link #setUseWeights(boolean)})
+ * </li>
+ * <li>
+ *      Weight of this node (see {@link #setNodeWeight(int)}). This parameter is ignored
+ *      if {@link #setUseWeights(boolean)} is set to {@code false}.
+ * </li>
+ * </ul>
+ * Below is Java configuration example:
+ * <pre name="code" class="java">
+ * GridWeightedRandomLoadBalancingSpi = new GridWeightedLoadBalancingSpi();
+ *
+ * // Configure SPI to used weighted
+ * // random load balancing.
+ * spi.setUseWeights(true);
+ *
+ * // Set weight for the local node.
+ * spi.setWeight( *);
+ *
+ * GridConfiguration cfg = new GridConfiguration();
+ *
+ * // Override default load balancing SPI.
+ * cfg.setLoadBalancingSpi(spi);
+ *
+ * // Starts grid.
+ * G.start(cfg);
+ * </pre>
+ * Here is how you can configure {@code GridRandomLoadBalancingSpi} using Spring XML configuration:
+ * <pre name="code" class="xml">
+ * &lt;property name="loadBalancingSpi"&gt;
+ *     &lt;bean class="org.gridgain.grid.spi.loadBalancing.weightedrandom.GridWeightedRandomLoadBalancingSpi"&gt;
+ *         &lt;property name="useWeights" value="true"/&gt;
+ *         &lt;property name="nodeWeight" value="10"/&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>
+ */
+@IgniteSpiMultipleInstancesSupport(true)
+@IgniteSpiConsistencyChecked(optional = true)
+public class WeightedRandomLoadBalancingSpi extends IgniteSpiAdapter implements LoadBalancingSpi,
+    WeightedRandomLoadBalancingSpiMBean {
+    /** Random number generator. */
+    private static final Random RAND = new Random();
+
+    /**
+     * Name of node attribute used to indicate load weight of a node
+     * (value is {@code "gridgain.node.weight.attr.name"}).
+     *
+     * @see org.apache.ignite.cluster.ClusterNode#attributes()
+     */
+    public static final String NODE_WEIGHT_ATTR_NAME = "gridgain.node.weight.attr.name";
+
+    /** Default weight assigned to every node if explicit one is not provided (value is {@code 10}). */
+    public static final int DFLT_NODE_WEIGHT = 10;
+
+    /** Grid logger. */
+    @IgniteLoggerResource
+    private IgniteLogger log;
+
+    /** */
+    private boolean isUseWeights;
+
+    /** Local event listener to listen to task completion events. */
+    private GridLocalEventListener evtLsnr;
+
+    /** Weight of this node. */
+    private int nodeWeight = DFLT_NODE_WEIGHT;
+
+    /** Task topologies. First pair value indicates whether or not jobs have been mapped. */
+    private ConcurrentMap<IgniteUuid, IgniteBiTuple<Boolean, WeightedTopology>> taskTops =
+        new ConcurrentHashMap8<>();
+
+    /**
+     * Sets a flag to indicate whether node weights should be checked when
+     * doing random load balancing. Default value is {@code false} which
+     * means that node weights are disregarded for load balancing logic.
+     *
+     * @param isUseWeights If {@code true} then random load is distributed according
+     *      to node weights.
+     */
+    @IgniteSpiConfiguration(optional = true)
+    public void setUseWeights(boolean isUseWeights) {
+        this.isUseWeights = isUseWeights;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean isUseWeights() {
+        return isUseWeights;
+    }
+
+    /**
+     * Sets weight of this node. Nodes with more processing capacity
+     * should be assigned proportionally larger weight. Default value
+     * is {@link #DFLT_NODE_WEIGHT} and is equal for all nodes.
+     *
+     * @param nodeWeight Weight of this node.
+     */
+    @IgniteSpiConfiguration(optional = true)
+    public void setNodeWeight(int nodeWeight) {
+        this.nodeWeight = nodeWeight;
+    }
+
+    /** {@inheritDoc} */
+    @Override public int getNodeWeight() {
+        return nodeWeight;
+    }
+
+    /** {@inheritDoc} */
+    @Override public Map<String, Object> getNodeAttributes() throws IgniteSpiException {
+        return F.<String, Object>asMap(createSpiAttributeName(NODE_WEIGHT_ATTR_NAME), nodeWeight);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void spiStart(@Nullable String gridName) throws IgniteSpiException {
+        startStopwatch();
+
+        assertParameter(nodeWeight > 0, "nodeWeight > 0");
+
+        if (log.isDebugEnabled()) {
+            log.debug(configInfo("isUseWeights", isUseWeights));
+            log.debug(configInfo("nodeWeight", nodeWeight));
+        }
+
+        registerMBean(gridName, this, WeightedRandomLoadBalancingSpiMBean.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 protected void onContextInitialized0(IgniteSpiContext spiCtx) throws IgniteSpiException {
+        getSpiContext().addLocalEventListener(evtLsnr = new GridLocalEventListener() {
+            @Override public void onEvent(IgniteEvent evt) {
+                assert evt instanceof IgniteTaskEvent || evt instanceof IgniteJobEvent;
+
+                if (evt.type() == EVT_TASK_FINISHED ||
+                    evt.type() == EVT_TASK_FAILED) {
+                    IgniteUuid sesId = ((IgniteTaskEvent)evt).taskSessionId();
+
+                    taskTops.remove(sesId);
+
+                    if (log.isDebugEnabled())
+                        log.debug("Removed task topology from topology cache for session: " + sesId);
+                }
+                // We should keep topology and use cache in GridComputeTask#map() method to
+                // avoid O(n*n/2) complexity, after that we can drop caches.
+                // Here we set mapped property and later cache will be ignored
+                else if (evt.type() == EVT_JOB_MAPPED) {
+                    IgniteUuid sesId = ((IgniteJobEvent)evt).taskSessionId();
+
+                    IgniteBiTuple<Boolean, WeightedTopology> weightedTop = taskTops.get(sesId);
+
+                    if (weightedTop != null)
+                        weightedTop.set1(true);
+
+                    if (log.isDebugEnabled())
+                        log.debug("Job has been mapped. Ignore cache for session: " + sesId);
+                }
+            }
+        },
+            EVT_TASK_FAILED,
+            EVT_TASK_FINISHED,
+            EVT_JOB_MAPPED
+        );
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void onContextDestroyed0() {
+        if (evtLsnr != null) {
+            IgniteSpiContext ctx = getSpiContext();
+
+            if (ctx != null)
+                ctx.removeLocalEventListener(evtLsnr);
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public ClusterNode getBalancedNode(ComputeTaskSession ses, List<ClusterNode> top, ComputeJob job) {
+        A.notNull(ses, "ses");
+        A.notNull(top, "top");
+        A.notNull(job, "job");
+
+        // Optimization for non-weighted randomization.
+        if (!isUseWeights)
+            return top.get(RAND.nextInt(top.size()));
+
+        IgniteBiTuple<Boolean, WeightedTopology> weightedTop = taskTops.get(ses.getId());
+
+        // Create new cached topology if there is no one. Do not
+        // use cached topology after task has been mapped.
+        if (weightedTop == null) {
+            // Called from GridComputeTask#map(). Put new topology and false as not mapped yet.
+            taskTops.put(ses.getId(), weightedTop = F.t(false, new WeightedTopology(top)));
+        }
+        // We have topology - check if task has been mapped.
+        else if (weightedTop.get1()) {
+            // Do not use cache after GridComputeTask#map().
+            return new WeightedTopology(top).pickWeightedNode();
+        }
+
+        return weightedTop.get2().pickWeightedNode();
+    }
+
+    /**
+     * @param node Node to get weight for.
+     * @return Node weight
+     */
+    private int getWeight(ClusterNode node) {
+        Integer weight = (Integer)node.attribute(createSpiAttributeName(NODE_WEIGHT_ATTR_NAME));
+
+        if (weight != null && weight == 0)
+            throw new IllegalStateException("Node weight cannot be zero: " + node);
+
+        return weight == null ? DFLT_NODE_WEIGHT : weight;
+    }
+
+    /**
+     * Holder for weighted topology.
+     */
+    private class WeightedTopology {
+        /** Total topology weight. */
+        private final int totalWeight;
+
+        /** Topology sorted by weight. */
+        private final SortedMap<Integer, ClusterNode> circle = new TreeMap<>();
+
+        /**
+         * @param top Topology.
+         */
+        WeightedTopology(Collection<ClusterNode> top) {
+            assert !F.isEmpty(top);
+
+            int totalWeight = 0;
+
+            for (ClusterNode node : top) {
+                totalWeight += getWeight(node);
+
+                // Complexity of this put is O(logN).
+                circle.put(totalWeight, node);
+            }
+
+            this.totalWeight = totalWeight;
+        }
+
+        /**
+         * Gets weighted node in random fashion.
+         *
+         * @return Weighted node.
+         */
+        ClusterNode pickWeightedNode() {
+            int weight = RAND.nextInt(totalWeight) + 1;
+
+            SortedMap<Integer, ClusterNode> pick = circle.tailMap(weight);
+
+            assert !pick.isEmpty();
+
+            return pick.get(pick.firstKey());
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override protected List<String> getConsistentAttributeNames() {
+        return Collections.singletonList(createSpiAttributeName(NODE_WEIGHT_ATTR_NAME));
+    }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return S.toString(WeightedRandomLoadBalancingSpi.class, this);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/b69a23cd/modules/core/src/main/java/org/apache/ignite/spi/loadbalancing/weightedrandom/WeightedRandomLoadBalancingSpiMBean.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/loadbalancing/weightedrandom/WeightedRandomLoadBalancingSpiMBean.java b/modules/core/src/main/java/org/apache/ignite/spi/loadbalancing/weightedrandom/WeightedRandomLoadBalancingSpiMBean.java
new file mode 100644
index 0000000..28ca25a
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/spi/loadbalancing/weightedrandom/WeightedRandomLoadBalancingSpiMBean.java
@@ -0,0 +1,37 @@
+/* @java.file.header */
+
+/*  _________        _____ __________________        _____
+ *  __  ____/___________(_)______  /__  ____/______ ____(_)_______
+ *  _  / __  __  ___/__  / _  __  / _  / __  _  __ `/__  / __  __ \
+ *  / /_/ /  _  /    _  /  / /_/ /  / /_/ /  / /_/ / _  /  _  / / /
+ *  \____/   /_/     /_/   \_,__/   \____/   \__,_/  /_/   /_/ /_/
+ */
+
+package org.apache.ignite.spi.loadbalancing.weightedrandom;
+
+import org.apache.ignite.mbean.*;
+import org.apache.ignite.spi.*;
+
+/**
+ * Management MBean for {@link WeightedRandomLoadBalancingSpi} SPI.
+ */
+@IgniteMBeanDescription("MBean that provides access to weighted random load balancing SPI configuration.")
+public interface WeightedRandomLoadBalancingSpiMBean extends IgniteSpiManagementMBean {
+    /**
+     * Checks whether node weights are considered when doing
+     * random load balancing.
+     *
+     * @return If {@code true} then random load is distributed according
+     *      to node weights.
+     */
+    @IgniteMBeanDescription("Whether node weights are considered when doing random load balancing.")
+    public boolean isUseWeights();
+
+    /**
+     * Gets weight of this node.
+     *
+     * @return Weight of this node.
+     */
+    @IgniteMBeanDescription("Weight of this node.")
+    public int getNodeWeight();
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/b69a23cd/modules/core/src/main/java/org/apache/ignite/spi/loadbalancing/weightedrandom/package.html
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/loadbalancing/weightedrandom/package.html b/modules/core/src/main/java/org/apache/ignite/spi/loadbalancing/weightedrandom/package.html
new file mode 100644
index 0000000..2da3d3a
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/spi/loadbalancing/weightedrandom/package.html
@@ -0,0 +1,15 @@
+<!DOCTYPE html PUBLIC "-//W3C//DTD HTML 4.01 Transitional//EN" "http://www.w3.org/TR/html4/loose.dtd">
+<!--
+    @html.file.header
+    _________        _____ __________________        _____
+    __  ____/___________(_)______  /__  ____/______ ____(_)_______
+    _  / __  __  ___/__  / _  __  / _  / __  _  __ `/__  / __  __ \
+    / /_/ /  _  /    _  /  / /_/ /  / /_/ /  / /_/ / _  /  _  / / /
+    \____/   /_/     /_/   \_,__/   \____/   \__,_/  /_/   /_/ /_/
+-->
+<html>
+<body>
+    <!-- Package description. -->
+    Contains weighted random-base implementation for load balancing SPI.
+</body>
+</html>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/b69a23cd/modules/core/src/main/java/org/gridgain/grid/cache/affinity/GridCacheAffinityKeyMapped.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/cache/affinity/GridCacheAffinityKeyMapped.java b/modules/core/src/main/java/org/gridgain/grid/cache/affinity/GridCacheAffinityKeyMapped.java
index 245d39d..505a8f0 100644
--- a/modules/core/src/main/java/org/gridgain/grid/cache/affinity/GridCacheAffinityKeyMapped.java
+++ b/modules/core/src/main/java/org/gridgain/grid/cache/affinity/GridCacheAffinityKeyMapped.java
@@ -83,7 +83,7 @@ import java.util.concurrent.*;
  * {@link org.apache.ignite.compute.ComputeJob} or any other grid computation, such as {@link Runnable}, {@link Callable}, or
  * {@link org.apache.ignite.lang.IgniteClosure}. It should be attached to a method or field that provides affinity key
  * for the computation. Only one annotation per class is allowed. Whenever such annotation is detected,
- * then {@link org.gridgain.grid.spi.loadbalancing.LoadBalancingSpi} will be bypassed, and computation will be routed to the grid node
+ * then {@link org.apache.ignite.spi.loadbalancing.LoadBalancingSpi} will be bypassed, and computation will be routed to the grid node
  * where the specified affinity key is cached. You can also use optional {@link GridCacheName @GridCacheName}
  * annotation whenever non-default cache name needs to be specified.
  * <p>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/b69a23cd/modules/core/src/main/java/org/gridgain/grid/kernal/GridGainEx.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/kernal/GridGainEx.java b/modules/core/src/main/java/org/gridgain/grid/kernal/GridGainEx.java
index fbfd071..454b9c7 100644
--- a/modules/core/src/main/java/org/gridgain/grid/kernal/GridGainEx.java
+++ b/modules/core/src/main/java/org/gridgain/grid/kernal/GridGainEx.java
@@ -48,8 +48,8 @@ import org.apache.ignite.spi.eventstorage.*;
 import org.apache.ignite.spi.eventstorage.memory.*;
 import org.apache.ignite.spi.failover.*;
 import org.apache.ignite.spi.failover.always.*;
-import org.gridgain.grid.spi.loadbalancing.*;
-import org.gridgain.grid.spi.loadbalancing.roundrobin.*;
+import org.apache.ignite.spi.loadbalancing.*;
+import org.apache.ignite.spi.loadbalancing.roundrobin.*;
 import org.gridgain.grid.spi.securesession.*;
 import org.gridgain.grid.spi.securesession.noop.*;
 import org.gridgain.grid.spi.swapspace.*;

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/b69a23cd/modules/core/src/main/java/org/gridgain/grid/kernal/managers/loadbalancer/GridLoadBalancerManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/kernal/managers/loadbalancer/GridLoadBalancerManager.java b/modules/core/src/main/java/org/gridgain/grid/kernal/managers/loadbalancer/GridLoadBalancerManager.java
index 3a89361..c4f5fed 100644
--- a/modules/core/src/main/java/org/gridgain/grid/kernal/managers/loadbalancer/GridLoadBalancerManager.java
+++ b/modules/core/src/main/java/org/gridgain/grid/kernal/managers/loadbalancer/GridLoadBalancerManager.java
@@ -17,7 +17,7 @@ import org.gridgain.grid.cache.affinity.*;
 import org.gridgain.grid.kernal.*;
 import org.gridgain.grid.kernal.managers.*;
 import org.gridgain.grid.kernal.managers.deployment.*;
-import org.gridgain.grid.spi.loadbalancing.*;
+import org.apache.ignite.spi.loadbalancing.*;
 import org.gridgain.grid.util.typedef.*;
 import org.gridgain.grid.util.typedef.internal.*;
 import org.jetbrains.annotations.*;

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/b69a23cd/modules/core/src/main/java/org/gridgain/grid/spi/loadbalancing/LoadBalancingSpi.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/spi/loadbalancing/LoadBalancingSpi.java b/modules/core/src/main/java/org/gridgain/grid/spi/loadbalancing/LoadBalancingSpi.java
deleted file mode 100644
index 91095fa..0000000
--- a/modules/core/src/main/java/org/gridgain/grid/spi/loadbalancing/LoadBalancingSpi.java
+++ /dev/null
@@ -1,114 +0,0 @@
-/* @java.file.header */
-
-/*  _________        _____ __________________        _____
- *  __  ____/___________(_)______  /__  ____/______ ____(_)_______
- *  _  / __  __  ___/__  / _  __  / _  / __  _  __ `/__  / __  __ \
- *  / /_/ /  _  /    _  /  / /_/ /  / /_/ /  / /_/ / _  /  _  / / /
- *  \____/   /_/     /_/   \_,__/   \____/   \__,_/  /_/   /_/ /_/
- */
-
-package org.gridgain.grid.spi.loadbalancing;
-
-import org.apache.ignite.cluster.*;
-import org.apache.ignite.compute.*;
-import org.apache.ignite.spi.*;
-import org.gridgain.grid.*;
-
-import java.util.*;
-
-/**
- * Load balancing SPI provides the next best balanced node for job
- * execution. This SPI is used either implicitly or explicitly whenever
- * a job gets mapped to a node during {@link org.apache.ignite.compute.ComputeTask#map(List, Object)}
- * invocation.
- * <h1 class="header">Coding Examples</h1>
- * If you are using {@link org.apache.ignite.compute.ComputeTaskSplitAdapter} then load balancing logic
- * is transparent to your code and is handled automatically by the adapter.
- * Here is an example of how your task could look:
- * <pre name="code" class="java">
- * public class MyFooBarTask extends GridComputeTaskSplitAdapter&lt;Object,Object&gt; {
- *    &#64;Override
- *    protected Collection&lt;? extends GridComputeJob&gt; split(int gridSize, Object arg) throws GridException {
- *        List&lt;MyFooBarJob&gt; jobs = new ArrayList&lt;MyFooBarJob&gt;(gridSize);
- *
- *        for (int i = 0; i &lt; gridSize; i++) {
- *            jobs.add(new MyFooBarJob(arg));
- *        }
- *
- *        // Node assignment via load balancer
- *        // happens automatically.
- *        return jobs;
- *    }
- *    ...
- * }
- * </pre>
- * If you need more fine-grained control over how some jobs within task get mapped to a node
- * <i>and</i> use, for example, affinity load balancing for some other jobs within task, then you should use
- * {@link org.apache.ignite.compute.ComputeTaskAdapter}. Here is an example of how your task could look. Note that in this
- * case we manually inject load balancer and use it to pick the best node. Doing it in
- * such way would allow user to map some jobs manually and for others use load balancer.
- * <pre name="code" class="java">
- * public class MyFooBarTask extends GridComputeTaskAdapter&lt;String,String&gt; {
- *    // Inject load balancer.
- *    &#64;GridLoadBalancerResource
- *    GridComputeLoadBalancer balancer;
- *
- *    // Map jobs to grid nodes.
- *    public Map&lt;? extends GridComputeJob, GridNode&gt; map(List&lt;GridNode&gt; subgrid, String arg) throws GridException {
- *        Map&lt;MyFooBarJob, GridNode&gt; jobs = new HashMap&lt;MyFooBarJob, GridNode&gt;(subgrid.size());
- *
- *        // In more complex cases, you can actually do
- *        // more complicated assignments of jobs to nodes.
- *        for (int i = 0; i &lt; subgrid.size(); i++) {
- *            // Pick the next best balanced node for the job.
- *            GridComputeJob myJob = new MyFooBarJob(arg);
- *
- *            jobs.put(myJob, balancer.getBalancedNode(myJob, null));
- *        }
- *
- *        return jobs;
- *    }
- *
- *    // Aggregate results into one compound result.
- *    public String reduce(List&lt;GridComputeJobResult&gt; results) throws GridException {
- *        // For the purpose of this example we simply
- *        // concatenate string representation of every
- *        // job result
- *        StringBuilder buf = new StringBuilder();
- *
- *        for (GridComputeJobResult res : results) {
- *            // Append string representation of result
- *            // returned by every job.
- *            buf.append(res.getData().toString());
- *        }
- *
- *        return buf.toString();
- *    }
- * }
- * </pre>
- * <p>
- * GridGain comes with the following load balancing SPI implementations out of the box:
- * <ul>
- * <li>{@link org.gridgain.grid.spi.loadbalancing.roundrobin.RoundRobinLoadBalancingSpi} - default</li>
- * <li>{@link org.gridgain.grid.spi.loadbalancing.adaptive.AdaptiveLoadBalancingSpi}</li>
- * <li>{@link org.gridgain.grid.spi.loadbalancing.weightedrandom.WeightedRandomLoadBalancingSpi}</li>
- * </ul>
- * <b>NOTE:</b> this SPI (i.e. methods in this interface) should never be used directly. SPIs provide
- * internal view on the subsystem and is used internally by GridGain kernal. In rare use cases when
- * access to a specific implementation of this SPI is required - an instance of this SPI can be obtained
- * via {@link org.apache.ignite.Ignite#configuration()} method to check its configuration properties or call other non-SPI
- * methods. Note again that calling methods from this interface on the obtained instance can lead
- * to undefined behavior and explicitly not supported.
- */
-public interface LoadBalancingSpi extends IgniteSpi {
-    /**
-     * Gets balanced node for specified job within given task session.
-     *
-     * @param ses Grid task session for currently executing task.
-     * @param top Topology of task nodes from which to pick the best balanced node for given job.
-     * @param job Job for which to pick the best balanced node.
-     * @throws GridException If failed to get next balanced node.
-     * @return Best balanced node for the given job within given task session.
-     */
-    public ClusterNode getBalancedNode(ComputeTaskSession ses, List<ClusterNode> top, ComputeJob job) throws GridException;
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/b69a23cd/modules/core/src/main/java/org/gridgain/grid/spi/loadbalancing/adaptive/AdaptiveCpuLoadProbe.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/spi/loadbalancing/adaptive/AdaptiveCpuLoadProbe.java b/modules/core/src/main/java/org/gridgain/grid/spi/loadbalancing/adaptive/AdaptiveCpuLoadProbe.java
deleted file mode 100644
index c438e9c..0000000
--- a/modules/core/src/main/java/org/gridgain/grid/spi/loadbalancing/adaptive/AdaptiveCpuLoadProbe.java
+++ /dev/null
@@ -1,229 +0,0 @@
-/* @java.file.header */
-
-/*  _________        _____ __________________        _____
- *  __  ____/___________(_)______  /__  ____/______ ____(_)_______
- *  _  / __  __  ___/__  / _  __  / _  / __  _  __ `/__  / __  __ \
- *  / /_/ /  _  /    _  /  / /_/ /  / /_/ /  / /_/ / _  /  _  / / /
- *  \____/   /_/     /_/   \_,__/   \____/   \__,_/  /_/   /_/ /_/
- */
-
-package org.gridgain.grid.spi.loadbalancing.adaptive;
-
-import org.apache.ignite.cluster.*;
-import org.gridgain.grid.util.typedef.internal.*;
-
-/**
- * Implementation of node load probing based on CPU load.
- * <p>
- * Based on {@link #setUseAverage(boolean)}
- * parameter, this implementation will either use average CPU load
- * values or current (default is to use averages).
- * <p>
- * Based on {@link #setUseProcessors(boolean)} parameter, this implementation
- * will either take number of processors on the node into account or not.
- * Since CPU load on multi-processor boxes shows medium load of multiple CPU's it
- * usually means that the remaining capacity is proportional to the number of
- * CPU's (or cores) on the node. This configuration parameter indicates
- * whether to divide each node's CPU load by the number of processors on that node
- * (default is {@code true}).
- * <p>
- * Also note that in some environments every processor may not be adding 100% of
- * processing power. For example, if you are using multi-core CPU's, then addition of
- * every core would probably result in about 75% of extra CPU power. To account
- * for that, you should set {@link #setProcessorCoefficient(double)} parameter to
- * {@code 0.75} .
- * <p>
- * Below is an example of how CPU load probe would be configured in GridGain
- * Spring configuration file:
- * <pre name="code" class="xml">
- * &lt;property name="loadBalancingSpi"&gt;
- *     &lt;bean class="org.gridgain.grid.spi.loadBalancing.adaptive.GridAdaptiveLoadBalancingSpi"&gt;
- *         &lt;property name="loadProbe"&gt;
- *             &lt;bean class="org.gridgain.grid.spi.loadBalancing.adaptive.GridAdaptiveCpuLoadProbe"&gt;
- *                 &lt;property name="useAverage" value="true"/&gt;
- *                 &lt;property name="useProcessors" value="true"/&gt;
- *                 &lt;property name="processorCoefficient" value="0.9"/&gt;
- *             &lt;/bean&gt;
- *         &lt;/property&gt;
- *     &lt;/bean&gt;
- * &lt;/property&gt;
- * </pre>
- * <p>
- * This implementation is used by default by {@link AdaptiveLoadBalancingSpi} SPI.
- */
-public class AdaptiveCpuLoadProbe implements AdaptiveLoadProbe {
-    /** Flag indicating whether to use average CPU load vs. current. */
-    private boolean useAvg = true;
-
-    /**
-     * Flag indicating whether to divide each node's CPU load
-     * by the number of processors on that node.
-     */
-    private boolean useProcs = true;
-
-    /**
-     * Coefficient of every CPU processor. By default it is {@code 1}, but
-     * in some environments every processor may not be adding 100% of processing
-     * power. For example, if you are using multi-core CPU's, then addition of
-     * every core would probably result in about 75% of extra CPU power, and hence
-     * you would set this coefficient to {@code 0.75} .
-     */
-    private double procCoefficient = 1;
-
-    /**
-     * Initializes CPU load probe to use CPU load average by default.
-     */
-    public AdaptiveCpuLoadProbe() {
-        // No-op.
-    }
-
-    /**
-     * Specifies whether to use average CPU load vs. current and whether or
-     * not to take number of processors into account.
-     * <p>
-     * Since CPU load on multi-processor boxes shows medium load of multiple CPU's it
-     * usually means that the remaining capacity is proportional to the number of
-     * CPU's (or cores) on the node.
-     *
-     * @param useAvg Flag indicating whether to use average CPU load vs. current
-     *      (default is {@code true}).
-     * @param useProcs Flag indicating whether to divide each node's CPU load
-     *      by the number of processors on that node (default is {@code true}).
-     */
-    public AdaptiveCpuLoadProbe(boolean useAvg, boolean useProcs) {
-        this.useAvg = useAvg;
-        this.useProcs = useProcs;
-    }
-
-    /**
-     * Specifies whether to use average CPU load vs. current and whether or
-     * not to take number of processors into account. It also allows to
-     * specify the coefficient of addition power every CPU adds.
-     * <p>
-     * Since CPU load on multi-processor boxes shows medium load of multiple CPU's it
-     * usually means that the remaining capacity is proportional to the number of
-     * CPU's (or cores) on the node.
-     * <p>
-     * Also, in some environments every processor may not be adding 100% of processing
-     * power. For example, if you are using multi-core CPU's, then addition of
-     * every core would probably result in about 75% of extra CPU power, and hence
-     * you would set this coefficient to {@code 0.75} .
-     *
-     * @param useAvg Flag indicating whether to use average CPU load vs. current
-     *      (default is {@code true}).
-     * @param useProcs Flag indicating whether to divide each node's CPU load
-     *      by the number of processors on that node (default is {@code true}).
-     * @param procCoefficient Coefficient of every CPU processor (default value is {@code 1}).
-     */
-    public AdaptiveCpuLoadProbe(boolean useAvg, boolean useProcs, double procCoefficient) {
-        this.useAvg = useAvg;
-        this.useProcs = useProcs;
-        this.procCoefficient = procCoefficient;
-    }
-
-    /**
-     * Gets flag indicating whether to use average CPU load vs. current.
-     *
-     * @return Flag indicating whether to use average CPU load vs. current.
-     */
-    public boolean isUseAverage() {
-        return useAvg;
-    }
-
-    /**
-     * Sets flag indicating whether to use average CPU load vs. current.
-     * If not explicitly set, then default value is {@code true}.
-     *
-     * @param useAvg Flag indicating whether to use average CPU load vs. current.
-     */
-    public void setUseAverage(boolean useAvg) {
-        this.useAvg = useAvg;
-    }
-
-    /**
-     * Gets flag indicating whether to use average CPU load vs. current
-     * (default is {@code true}).
-     * <p>
-     * Since CPU load on multi-processor boxes shows medium load of multiple CPU's it
-     * usually means that the remaining capacity is proportional to the number of
-     * CPU's (or cores) on the node.
-     *
-     * @return Flag indicating whether to divide each node's CPU load
-     *      by the number of processors on that node (default is {@code true}).
-     */
-    public boolean isUseProcessors() {
-        return useProcs;
-    }
-
-    /**
-     * Sets flag indicating whether to use average CPU load vs. current
-     * (default is {@code true}).
-     * <p>
-     * Since CPU load on multi-processor boxes shows medium load of multiple CPU's it
-     * usually means that the remaining capacity is proportional to the number of
-     * CPU's (or cores) on the node.
-     * <p>
-     * If not explicitly set, then default value is {@code true}.
-     *
-     * @param useProcs Flag indicating whether to divide each node's CPU load
-     *      by the number of processors on that node (default is {@code true}).
-     */
-    public void setUseProcessors(boolean useProcs) {
-        this.useProcs = useProcs;
-    }
-
-    /**
-     * Gets coefficient of every CPU processor. By default it is {@code 1}, but
-     * in some environments every processor may not be adding 100% of processing
-     * power. For example, if you are using multi-core CPU's, then addition of
-     * every core would probably result in about 75% of extra CPU power, and hence
-     * you would set this coefficient to {@code 0.75} .
-     * <p>
-     * This value is ignored if {@link #isUseProcessors()} is set to {@code false}.
-     *
-     * @return Coefficient of every CPU processor.
-     */
-    public double getProcessorCoefficient() {
-        return procCoefficient;
-    }
-
-    /**
-     * Sets coefficient of every CPU processor. By default it is {@code 1}, but
-     * in some environments every processor may not be adding 100% of processing
-     * power. For example, if you are using multi-core CPU's, then addition of
-     * every core would probably result in about 75% of extra CPU power, and hence
-     * you would set this coefficient to {@code 0.75} .
-     * <p>
-     * This value is ignored if {@link #isUseProcessors()} is set to {@code false}.
-     *
-     * @param procCoefficient Coefficient of every CPU processor.
-     */
-    public void setProcessorCoefficient(double procCoefficient) {
-        A.ensure(procCoefficient > 0, "procCoefficient > 0");
-
-        this.procCoefficient = procCoefficient;
-    }
-
-    /** {@inheritDoc} */
-    @Override public double getLoad(ClusterNode node, int jobsSentSinceLastUpdate) {
-        ClusterNodeMetrics metrics = node.metrics();
-
-        double k = 1.0d;
-
-        if (useProcs) {
-            int procs = metrics.getTotalCpus();
-
-            if (procs > 1)
-                k = procs * procCoefficient;
-        }
-
-        double load = (useAvg ? metrics.getAverageCpuLoad() : metrics.getCurrentCpuLoad()) / k;
-
-        return load < 0 ? 0 : load;
-    }
-
-    /** {@inheritDoc} */
-    @Override public String toString() {
-        return S.toString(AdaptiveCpuLoadProbe.class, this);
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/b69a23cd/modules/core/src/main/java/org/gridgain/grid/spi/loadbalancing/adaptive/AdaptiveJobCountLoadProbe.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/spi/loadbalancing/adaptive/AdaptiveJobCountLoadProbe.java b/modules/core/src/main/java/org/gridgain/grid/spi/loadbalancing/adaptive/AdaptiveJobCountLoadProbe.java
deleted file mode 100644
index 95ef91d..0000000
--- a/modules/core/src/main/java/org/gridgain/grid/spi/loadbalancing/adaptive/AdaptiveJobCountLoadProbe.java
+++ /dev/null
@@ -1,96 +0,0 @@
-/* @java.file.header */
-
-/*  _________        _____ __________________        _____
- *  __  ____/___________(_)______  /__  ____/______ ____(_)_______
- *  _  / __  __  ___/__  / _  __  / _  / __  _  __ `/__  / __  __ \
- *  / /_/ /  _  /    _  /  / /_/ /  / /_/ /  / /_/ / _  /  _  / / /
- *  \____/   /_/     /_/   \_,__/   \____/   \__,_/  /_/   /_/ /_/
- */
-
-package org.gridgain.grid.spi.loadbalancing.adaptive;
-
-import org.apache.ignite.cluster.*;
-import org.gridgain.grid.util.typedef.internal.*;
-
-/**
- * Implementation of node load probing based on active and waiting job count.
- * Based on {@link #setUseAverage(boolean)} parameter, this implementation will
- * either use average job count values or current (default is to use averages).
- * <p>
- * The load of a node is simply calculated by adding active and waiting job counts.
- * <p>
- * Below is an example of how CPU load probe would be configured in GridGain
- * Spring configuration file:
- * <pre name="code" class="xml">
- * &lt;property name="loadBalancingSpi"&gt;
- *     &lt;bean class="org.gridgain.grid.spi.loadBalancing.adaptive.GridAdaptiveLoadBalancingSpi"&gt;
- *         &lt;property name="loadProbe"&gt;
- *             &lt;bean class="org.gridgain.grid.spi.loadBalancing.adaptive.GridAdaptiveJobCountLoadProbe"&gt;
- *                 &lt;property name="useAverage" value="true"/&gt;
- *             &lt;/bean&gt;
- *         &lt;/property&gt;
- *     &lt;/bean&gt;
- * &lt;/property&gt;
- * </pre>
- */
-public class AdaptiveJobCountLoadProbe implements AdaptiveLoadProbe {
-    /** Flag indicating whether to use average CPU load vs. current. */
-    private boolean useAvg = true;
-
-    /**
-     * Initializes active job probe.
-     */
-    public AdaptiveJobCountLoadProbe() {
-        // No-op.
-    }
-
-    /**
-     * Creates new active job prove specifying whether to use average
-     * job counts vs. current.
-     *
-     * @param useAvg Flag indicating whether to use average job counts vs. current.
-     */
-    public AdaptiveJobCountLoadProbe(boolean useAvg) {
-        this.useAvg = useAvg;
-    }
-
-    /**
-     * Gets flag indicating whether to use average job counts vs. current.
-     *
-     * @return Flag indicating whether to use average job counts vs. current.
-     */
-    public boolean isUseAverage() {
-        return useAvg;
-    }
-
-    /**
-     * Sets flag indicating whether to use average job counts vs. current.
-     *
-     * @param useAvg Flag indicating whether to use average job counts vs. current.
-     */
-    public void setUseAverage(boolean useAvg) {
-        this.useAvg = useAvg;
-    }
-
-
-    /** {@inheritDoc} */
-    @Override public double getLoad(ClusterNode node, int jobsSentSinceLastUpdate) {
-        ClusterNodeMetrics metrics = node.metrics();
-
-        if (useAvg) {
-            double load = metrics.getAverageActiveJobs() + metrics.getAverageWaitingJobs();
-
-            if (load > 0)
-                return load;
-        }
-
-        double load = metrics.getCurrentActiveJobs() + metrics.getCurrentWaitingJobs();
-
-        return load < 0 ? 0 : load;
-    }
-
-    /** {@inheritDoc} */
-    @Override public String toString() {
-        return S.toString(AdaptiveJobCountLoadProbe.class, this);
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/b69a23cd/modules/core/src/main/java/org/gridgain/grid/spi/loadbalancing/adaptive/AdaptiveLoadBalancingSpi.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/spi/loadbalancing/adaptive/AdaptiveLoadBalancingSpi.java b/modules/core/src/main/java/org/gridgain/grid/spi/loadbalancing/adaptive/AdaptiveLoadBalancingSpi.java
deleted file mode 100644
index 069d269..0000000
--- a/modules/core/src/main/java/org/gridgain/grid/spi/loadbalancing/adaptive/AdaptiveLoadBalancingSpi.java
+++ /dev/null
@@ -1,581 +0,0 @@
-/* @java.file.header */
-
-/*  _________        _____ __________________        _____
- *  __  ____/___________(_)______  /__  ____/______ ____(_)_______
- *  _  / __  __  ___/__  / _  __  / _  / __  _  __ `/__  / __  __ \
- *  / /_/ /  _  /    _  /  / /_/ /  / /_/ /  / /_/ / _  /  _  / / /
- *  \____/   /_/     /_/   \_,__/   \____/   \__,_/  /_/   /_/ /_/
- */
-
-package org.gridgain.grid.spi.loadbalancing.adaptive;
-
-import org.apache.ignite.*;
-import org.apache.ignite.cluster.*;
-import org.apache.ignite.compute.*;
-import org.apache.ignite.events.*;
-import org.apache.ignite.lang.*;
-import org.apache.ignite.resources.*;
-import org.apache.ignite.spi.*;
-import org.gridgain.grid.*;
-import org.gridgain.grid.kernal.managers.eventstorage.*;
-import org.gridgain.grid.spi.loadbalancing.*;
-import org.gridgain.grid.util.typedef.*;
-import org.gridgain.grid.util.typedef.internal.*;
-import org.jdk8.backport.*;
-import org.jetbrains.annotations.*;
-
-import java.util.*;
-import java.util.concurrent.*;
-import java.util.concurrent.atomic.*;
-import java.util.concurrent.locks.*;
-
-import static org.apache.ignite.events.IgniteEventType.*;
-
-/**
- * Load balancing SPI that adapts to overall node performance. It
- * proportionally distributes more jobs to more performant nodes based
- * on a pluggable and dynamic node load probing.
- * <p>
- * <h1 class="header">Adaptive Node Probe</h1>
- * This SPI comes with pluggable algorithm to calculate a node load
- * at any given point of time. The algorithm is defined by
- * {@link AdaptiveLoadProbe} interface and user is
- * free to provide custom implementations. By default
- * {@link AdaptiveCpuLoadProbe} implementation is used
- * which distributes jobs to nodes based on average CPU load
- * on every node.
- * <p>
- * The following load probes are available with the product:
- * <ul>
- * <li>{@link AdaptiveCpuLoadProbe} - default</li>
- * <li>{@link AdaptiveProcessingTimeLoadProbe}</li>
- * <li>{@link AdaptiveJobCountLoadProbe}</li>
- * </ul>
- * Note that if {@link AdaptiveLoadProbe#getLoad(org.apache.ignite.cluster.ClusterNode, int)} returns a value of {@code 0},
- * then implementation will assume that load value is simply not available and
- * will try to calculate an average of load values for other nodes. If such
- * average cannot be obtained (all node load values are {@code 0}), then a value
- * of {@code 1} will be used.
- * <p>
- * When working with node metrics, take into account that all averages are
- * calculated over metrics history size defined by {@link org.apache.ignite.configuration.IgniteConfiguration#getMetricsExpireTime()}
- * and {@link org.apache.ignite.configuration.IgniteConfiguration#getMetricsHistorySize()} grid configuration parameters.
- * Generally the larger these configuration parameter values are, the more precise the metrics are.
- * You should tune these values based on the level of accuracy needed vs. the additional memory
- * that would be required for storing metrics.
- * <p>
- * You should also keep in mind that metrics for remote nodes are delayed (usually by the
- * heartbeat frequency). So if it is acceptable in your environment, set the heartbeat frequency
- * to be more inline with job execution time. Generally, the more often heartbeats between nodes
- * are exchanged, the more precise the metrics are. However, you should keep in mind that if
- * heartbeats are exchanged too often then it may create unnecessary traffic in the network.
- * Heartbeats (or metrics update frequency) can be configured via underlying
- * {@link org.apache.ignite.spi.discovery.DiscoverySpi} used in your grid.
- * <p>
- * Here is an example of how probing can be implemented to use
- * number of active and waiting jobs as probing mechanism:
- * <pre name="code" class="java">
- * public class FooBarLoadProbe implements GridAdaptiveLoadProbe {
- *     // Flag indicating whether to use average value or current.
- *     private int useAvg = true;
- *
- *     public FooBarLoadProbe(boolean useAvg) {
- *         this.useAvg = useAvg;
- *     }
- *
- *     // Calculate load based on number of active and waiting jobs.
- *     public double getLoad(GridNode node, int jobsSentSinceLastUpdate) {
- *         GridNodeMetrics metrics = node.getMetrics();
- *
- *         if (useAvg) {
- *             double load = metrics.getAverageActiveJobs() + metrics.getAverageWaitingJobs();
- *
- *             if (load > 0) {
- *                 return load;
- *             }
- *         }
- *
- *         return metrics.getCurrentActiveJobs() + metrics.getCurrentWaitingJobs();
- *     }
- * }
- * </pre>
- * <h1 class="header">Which Node Probe To Use</h1>
- * There is no correct answer here. Every single node probe will work better or worse in
- * different environments. CPU load probe (default option) is the safest approach to start
- * with as it simply attempts to utilize every CPU on the grid to the maximum. However, you should
- * experiment with other probes by executing load tests in your environment and observing
- * which probe gives you best performance and load balancing.
- * <p>
- * <h1 class="header">Task Coding Example</h1>
- * If you are using {@link org.apache.ignite.compute.ComputeTaskSplitAdapter} then load balancing logic
- * is transparent to your code and is handled automatically by the adapter.
- * Here is an example of how your task will look:
- * <pre name="code" class="java">
- * public class MyFooBarTask extends GridComputeTaskSplitAdapter&lt;Object, Object&gt; {
- *    &#64;Override
- *    protected Collection&lt;? extends GridComputeJob&gt; split(int gridSize, Object arg) throws GridException {
- *        List&lt;MyFooBarJob&gt; jobs = new ArrayList&lt;MyFooBarJob&gt;(gridSize);
- *
- *        for (int i = 0; i &lt; gridSize; i++) {
- *            jobs.add(new MyFooBarJob(arg));
- *        }
- *
- *        // Node assignment via load balancer
- *        // happens automatically.
- *        return jobs;
- *    }
- *    ...
- * }
- * </pre>
- * If you need more fine-grained control over how some jobs within task get mapped to a node
- * and use affinity load balancing for some other jobs within task, then you should use
- * {@link org.apache.ignite.compute.ComputeTaskAdapter}. Here is an example of how your task will look. Note that in this
- * case we manually inject load balancer and use it to pick the best node. Doing it in
- * such way would allow user to map some jobs manually and for others use load balancer.
- * <pre name="code" class="java">
- * public class MyFooBarTask extends GridComputeTaskAdapter&lt;String, String&gt; {
- *    // Inject load balancer.
- *    &#64;GridLoadBalancerResource
- *    GridComputeLoadBalancer balancer;
- *
- *    // Map jobs to grid nodes.
- *    public Map&lt;? extends GridComputeJob, GridNode&gt; map(List&lt;GridNode&gt; subgrid, String arg) throws GridException {
- *        Map&lt;MyFooBarJob, GridNode&gt; jobs = new HashMap&lt;MyFooBarJob, GridNode&gt;(subgrid.size());
- *
- *        // In more complex cases, you can actually do
- *        // more complicated assignments of jobs to nodes.
- *        for (int i = 0; i &lt; subgrid.size(); i++) {
- *            // Pick the next best balanced node for the job.
- *            jobs.put(new MyFooBarJob(arg), balancer.getBalancedNode())
- *        }
- *
- *        return jobs;
- *    }
- *
- *    // Aggregate results into one compound result.
- *    public String reduce(List&lt;GridComputeJobResult&gt; results) throws GridException {
- *        // For the purpose of this example we simply
- *        // concatenate string representation of every
- *        // job result
- *        StringBuilder buf = new StringBuilder();
- *
- *        for (GridComputeJobResult res : results) {
- *            // Append string representation of result
- *            // returned by every job.
- *            buf.append(res.getData().string());
- *        }
- *
- *        return buf.string();
- *    }
- * }
- * </pre>
- * <p>
- * <h1 class="header">Configuration</h1>
- * In order to use this load balancer, you should configure your grid instance
- * to use {@code GridJobsLoadBalancingSpi} either from Spring XML file or
- * directly. The following configuration parameters are supported:
- * <h2 class="header">Mandatory</h2>
- * This SPI has no mandatory configuration parameters.
- * <h2 class="header">Optional</h2>
- * This SPI has the following optional configuration parameters:
- * <ul>
- * <li>
- *      Adaptive node load probing implementation (see {@link #setLoadProbe(AdaptiveLoadProbe)}).
- *      This configuration parameter supplies a custom algorithm for probing a node's load.
- *      By default, {@link AdaptiveCpuLoadProbe} implementation is used which
- *      takes every node's CPU load and tries to send proportionally more jobs to less loaded nodes.
- * </li>
- * </ul>
- * <p>
- * Below is Java configuration example:
- * <pre name="code" class="java">
- * GridAdaptiveLoadBalancingSpi spi = new GridAdaptiveLoadBalancingSpi();
- *
- * // Configure probe to use latest job execution time vs. average.
- * GridAdaptiveProcessingTimeLoadProbe probe = new GridAdaptiveProcessingTimeLoadProbe(false);
- *
- * spi.setLoadProbe(probe);
- *
- * GridConfiguration cfg = new GridConfiguration();
- *
- * // Override default load balancing SPI.
- * cfg.setLoadBalancingSpi(spi);
- *
- * // Starts grid.
- * G.start(cfg);
- * </pre>
- * Here is how you can configure {@code GridJobsLoadBalancingSpi} using Spring XML configuration:
- * <pre name="code" class="xml">
- * &lt;property name="loadBalancingSpi"&gt;
- *     &lt;bean class="org.gridgain.grid.spi.loadBalancing.adaptive.GridAdaptiveLoadBalancingSpi"&gt;
- *         &lt;property name="loadProbe"&gt;
- *             &lt;bean class="org.gridgain.grid.spi.loadBalancing.adaptive.GridAdaptiveProcessingTimeLoadProbe"&gt;
- *                 &lt;constructor-arg value="false"/&gt;
- *             &lt;/bean&gt;
- *         &lt;/property&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>
- */
-@IgniteSpiMultipleInstancesSupport(true)
-public class AdaptiveLoadBalancingSpi extends IgniteSpiAdapter implements LoadBalancingSpi,
-    AdaptiveLoadBalancingSpiMBean {
-    /** Random number generator. */
-    private static final Random RAND = new Random();
-
-    /** Grid logger. */
-    @IgniteLoggerResource
-    private IgniteLogger log;
-
-    /** */
-    private AdaptiveLoadProbe probe = new AdaptiveCpuLoadProbe();
-
-    /** Local event listener to listen to task completion events. */
-    private GridLocalEventListener evtLsnr;
-
-    /** Task topologies. First pair value indicates whether or not jobs have been mapped. */
-    private ConcurrentMap<IgniteUuid, IgniteBiTuple<Boolean, WeightedTopology>> taskTops =
-        new ConcurrentHashMap8<>();
-
-    /** */
-    private final Map<UUID, AtomicInteger> nodeJobs = new HashMap<>();
-
-    /** */
-    private final ReadWriteLock rwLock = new ReentrantReadWriteLock();
-
-    /** {@inheritDoc} */
-    @Override public String getLoadProbeFormatted() {
-        return probe.toString();
-    }
-
-    /**
-     * Sets implementation of node load probe. By default {@link AdaptiveProcessingTimeLoadProbe}
-     * is used which proportionally distributes load based on the average job execution
-     * time on every node.
-     *
-     * @param probe Implementation of node load probe
-     */
-    @IgniteSpiConfiguration(optional = true)
-    public void setLoadProbe(AdaptiveLoadProbe probe) {
-        A.ensure(probe != null, "probe != null");
-
-        this.probe = probe;
-    }
-
-    /** {@inheritDoc} */
-    @Override public void spiStart(@Nullable String gridName) throws IgniteSpiException {
-        startStopwatch();
-
-        assertParameter(probe != null, "loadProbe != null");
-
-        if (log.isDebugEnabled())
-            log.debug(configInfo("loadProbe", probe));
-
-        registerMBean(gridName, this, AdaptiveLoadBalancingSpiMBean.class);
-
-        // Ack ok start.
-        if (log.isDebugEnabled())
-            log.debug(startInfo());
-    }
-
-    /** {@inheritDoc} */
-    @Override public void spiStop() throws IgniteSpiException {
-        rwLock.writeLock().lock();
-
-        try {
-            nodeJobs.clear();
-        }
-        finally {
-            rwLock.writeLock().unlock();
-        }
-
-        unregisterMBean();
-
-        // Ack ok stop.
-        if (log.isDebugEnabled())
-            log.debug(stopInfo());
-    }
-
-    /** {@inheritDoc} */
-    @Override protected void onContextInitialized0(IgniteSpiContext spiCtx) throws IgniteSpiException {
-        getSpiContext().addLocalEventListener(evtLsnr = new GridLocalEventListener() {
-            @Override public void onEvent(IgniteEvent evt) {
-                switch (evt.type()) {
-                    case EVT_TASK_FINISHED:
-                    case EVT_TASK_FAILED: {
-                        IgniteTaskEvent taskEvt = (IgniteTaskEvent)evt;
-
-                        taskTops.remove(taskEvt.taskSessionId());
-
-                        if (log.isDebugEnabled())
-                            log.debug("Removed task topology from topology cache for session: " +
-                                taskEvt.taskSessionId());
-
-                        break;
-                    }
-
-                    case EVT_JOB_MAPPED: {
-                        // We should keep topology and use cache in GridComputeTask#map() method to
-                        // avoid O(n*n/2) complexity, after that we can drop caches.
-                        // Here we set mapped property and later cache will be ignored
-                        IgniteJobEvent jobEvt = (IgniteJobEvent)evt;
-
-                        IgniteBiTuple<Boolean, WeightedTopology> weightedTop = taskTops.get(jobEvt.taskSessionId());
-
-                        if (weightedTop != null)
-                            weightedTop.set1(true);
-
-                        if (log.isDebugEnabled())
-                            log.debug("Job has been mapped. Ignore cache for session: " + jobEvt.taskSessionId());
-
-                        break;
-                    }
-
-                    case EVT_NODE_METRICS_UPDATED:
-                    case EVT_NODE_FAILED:
-                    case EVT_NODE_JOINED:
-                    case EVT_NODE_LEFT: {
-                        IgniteDiscoveryEvent discoEvt = (IgniteDiscoveryEvent)evt;
-
-                        rwLock.writeLock().lock();
-
-                        try {
-                            switch (evt.type()) {
-                                case EVT_NODE_JOINED: {
-                                    nodeJobs.put(discoEvt.eventNode().id(), new AtomicInteger(0));
-
-                                    break;
-                                }
-
-                                case EVT_NODE_LEFT:
-                                case EVT_NODE_FAILED: {
-                                    nodeJobs.remove(discoEvt.eventNode().id());
-
-                                    break;
-                                }
-
-                                case EVT_NODE_METRICS_UPDATED: {
-                                    // Reset counter.
-                                    nodeJobs.put(discoEvt.eventNode().id(), new AtomicInteger(0));
-
-                                    break;
-                                }
-                            }
-                        }
-                        finally {
-                            rwLock.writeLock().unlock();
-                        }
-                    }
-
-                }
-            }
-        },
-            EVT_NODE_METRICS_UPDATED,
-            EVT_NODE_FAILED,
-            EVT_NODE_JOINED,
-            EVT_NODE_LEFT,
-            EVT_TASK_FINISHED,
-            EVT_TASK_FAILED,
-            EVT_JOB_MAPPED
-        );
-
-        // Put all known nodes.
-        rwLock.writeLock().lock();
-
-        try {
-            for (ClusterNode node : getSpiContext().nodes())
-                nodeJobs.put(node.id(), new AtomicInteger(0));
-        }
-        finally {
-            rwLock.writeLock().unlock();
-        }
-    }
-
-    /** {@inheritDoc} */
-    @Override protected void onContextDestroyed0() {
-        if (evtLsnr != null) {
-            IgniteSpiContext ctx = getSpiContext();
-
-            if (ctx != null)
-                ctx.removeLocalEventListener(evtLsnr);
-        }
-    }
-
-    /** {@inheritDoc} */
-    @Override public ClusterNode getBalancedNode(ComputeTaskSession ses, List<ClusterNode> top, ComputeJob job)
-    throws GridException {
-        A.notNull(ses, "ses");
-        A.notNull(top, "top");
-        A.notNull(job, "job");
-
-        IgniteBiTuple<Boolean, WeightedTopology> weightedTop = taskTops.get(ses.getId());
-
-        // Create new cached topology if there is no one. Do not
-        // use cached topology after task has been mapped.
-        if (weightedTop == null)
-            // Called from GridComputeTask#map(). Put new topology and false as not mapped yet.
-            taskTops.put(ses.getId(), weightedTop = F.t(false, new WeightedTopology(top)));
-        // We have topology - check if task has been mapped.
-        else if (weightedTop.get1())
-            // Do not use cache after GridComputeTask#map().
-            return new WeightedTopology(top).pickWeightedNode();
-
-        return weightedTop.get2().pickWeightedNode();
-    }
-
-    /**
-     * Calculates node load based on set probe.
-     *
-     * @param top List of all nodes.
-     * @param node Node to get load for.
-     * @return Node load.
-     * @throws GridException If returned load is negative.
-     */
-    @SuppressWarnings({"TooBroadScope"})
-    private double getLoad(Collection<ClusterNode> top, ClusterNode node) throws GridException {
-        assert !F.isEmpty(top);
-
-        int jobsSentSinceLastUpdate = 0;
-
-        rwLock.readLock().lock();
-
-        try {
-            AtomicInteger cnt = nodeJobs.get(node.id());
-
-            jobsSentSinceLastUpdate = cnt == null ? 0 : cnt.get();
-        }
-        finally {
-            rwLock.readLock().unlock();
-        }
-
-        double load = probe.getLoad(node, jobsSentSinceLastUpdate);
-
-        if (load < 0)
-            throw new GridException("Failed to obtain non-negative load from adaptive load probe: " + load);
-
-        return load;
-    }
-
-    /**
-     * Holder for weighted topology.
-     */
-    private class WeightedTopology {
-        /** Topology sorted by weight. */
-        private final SortedMap<Double, ClusterNode> circle = new TreeMap<>();
-
-        /**
-         * @param top Task topology.
-         * @throws GridException If any load was negative.
-         */
-        WeightedTopology(List<ClusterNode> top) throws GridException {
-            assert !F.isEmpty(top);
-
-            double totalLoad = 0;
-
-            // We need to cache loads here to avoid calls later as load might be
-            // changed between the calls.
-            double[] nums = new double[top.size()];
-
-            int zeroCnt = 0;
-
-            // Compute loads.
-            for (int i = 0; i < top.size(); i++) {
-                double load = getLoad(top, top.get(i));
-
-                nums[i] = load;
-
-                if (load == 0)
-                    zeroCnt++;
-
-                totalLoad += load;
-            }
-
-            // Take care of zero loads.
-            if (zeroCnt > 0) {
-                double newTotal = totalLoad;
-
-                int nonZeroCnt = top.size() - zeroCnt;
-
-                for (int i = 0; i < nums.length; i++) {
-                    double load = nums[i];
-
-                    if (load == 0) {
-                        if (nonZeroCnt > 0)
-                            load = totalLoad / nonZeroCnt;
-
-                        if (load == 0)
-                            load = 1;
-
-                        nums[i] = load;
-
-                        newTotal += load;
-                    }
-                }
-
-                totalLoad = newTotal;
-            }
-
-            double totalWeight = 0;
-
-            // Calculate weights and total weight.
-            for (int i = 0; i < nums.length; i++) {
-                assert nums[i] > 0 : "Invalid load: " + nums[i];
-
-                double weight = totalLoad / nums[i];
-
-                // Convert to weight.
-                nums[i] = weight;
-
-                totalWeight += weight;
-            }
-
-            double weight = 0;
-
-            // Enforce range from 0 to 1.
-            for (int i = 0; i < nums.length; i++) {
-                weight = i == nums.length - 1 ? 1.0d : weight + nums[i] / totalWeight;
-
-                assert weight < 2 : "Invalid weight: " + weight;
-
-                // Complexity of this put is O(logN).
-                circle.put(weight, top.get(i));
-            }
-        }
-
-        /**
-         * Gets weighted node in random fashion.
-         *
-         * @return Weighted node.
-         */
-        ClusterNode pickWeightedNode() {
-            double weight = RAND.nextDouble();
-
-            SortedMap<Double, ClusterNode> pick = circle.tailMap(weight);
-
-            ClusterNode node = pick.get(pick.firstKey());
-
-            rwLock.readLock().lock();
-
-            try {
-                AtomicInteger cnt = nodeJobs.get(node.id());
-
-                if (cnt != null)
-                    cnt.incrementAndGet();
-            }
-            finally {
-                rwLock.readLock().unlock();
-            }
-
-            return node;
-        }
-    }
-
-    /** {@inheritDoc} */
-    @Override public String toString() {
-        return S.toString(AdaptiveLoadBalancingSpi.class, this);
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/b69a23cd/modules/core/src/main/java/org/gridgain/grid/spi/loadbalancing/adaptive/AdaptiveLoadBalancingSpiMBean.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/spi/loadbalancing/adaptive/AdaptiveLoadBalancingSpiMBean.java b/modules/core/src/main/java/org/gridgain/grid/spi/loadbalancing/adaptive/AdaptiveLoadBalancingSpiMBean.java
deleted file mode 100644
index 5553635..0000000
--- a/modules/core/src/main/java/org/gridgain/grid/spi/loadbalancing/adaptive/AdaptiveLoadBalancingSpiMBean.java
+++ /dev/null
@@ -1,27 +0,0 @@
-/* @java.file.header */
-
-/*  _________        _____ __________________        _____
- *  __  ____/___________(_)______  /__  ____/______ ____(_)_______
- *  _  / __  __  ___/__  / _  __  / _  / __  _  __ `/__  / __  __ \
- *  / /_/ /  _  /    _  /  / /_/ /  / /_/ /  / /_/ / _  /  _  / / /
- *  \____/   /_/     /_/   \_,__/   \____/   \__,_/  /_/   /_/ /_/
- */
-
-package org.gridgain.grid.spi.loadbalancing.adaptive;
-
-import org.apache.ignite.mbean.*;
-import org.apache.ignite.spi.*;
-
-/**
- * Management MBean for {@link AdaptiveLoadBalancingSpi} SPI.
- */
-@IgniteMBeanDescription("MBean that provides access to adaptive load balancing SPI configuration.")
-public interface AdaptiveLoadBalancingSpiMBean extends IgniteSpiManagementMBean {
-    /**
-     * Gets text description of current load probing implementation used.
-     *
-     * @return Text description of current load probing implementation used.
-     */
-    @IgniteMBeanDescription("Text description of current load probing implementation used.")
-    public String getLoadProbeFormatted();
-}


[10/20] incubator-ignite git commit: # Renaming

Posted by sb...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c31cec7c/modules/core/src/test/java/org/gridgain/grid/spi/swapspace/GridSwapSpaceSpiAbstractSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/gridgain/grid/spi/swapspace/GridSwapSpaceSpiAbstractSelfTest.java b/modules/core/src/test/java/org/gridgain/grid/spi/swapspace/GridSwapSpaceSpiAbstractSelfTest.java
index 3a60def..0a5f28a 100644
--- a/modules/core/src/test/java/org/gridgain/grid/spi/swapspace/GridSwapSpaceSpiAbstractSelfTest.java
+++ b/modules/core/src/test/java/org/gridgain/grid/spi/swapspace/GridSwapSpaceSpiAbstractSelfTest.java
@@ -25,7 +25,7 @@ import static org.apache.ignite.events.IgniteEventType.*;
 import static org.junit.Assert.*;
 
 /**
- * Test for various {@link GridSwapSpaceSpi} implementations.
+ * Test for various {@link SwapSpaceSpi} implementations.
  */
 public abstract class GridSwapSpaceSpiAbstractSelfTest extends GridCommonAbstractTest {
     /** Default swap space name. */
@@ -38,12 +38,12 @@ public abstract class GridSwapSpaceSpiAbstractSelfTest extends GridCommonAbstrac
     protected static final String SPACE2 = "space2";
 
     /** SPI to test. */
-    protected GridSwapSpaceSpi spi;
+    protected SwapSpaceSpi spi;
 
     /**
-     * @return New {@link GridSwapSpaceSpi} instance.
+     * @return New {@link SwapSpaceSpi} instance.
      */
-    protected abstract GridSwapSpaceSpi spi();
+    protected abstract SwapSpaceSpi spi();
 
     /** {@inheritDoc} */
     @Override protected void beforeTest() throws Exception {
@@ -66,7 +66,7 @@ public abstract class GridSwapSpaceSpiAbstractSelfTest extends GridCommonAbstrac
     /**
      * @return Swap context.
      */
-    protected GridSwapContext context() {
+    protected SwapContext context() {
         return context(null);
     }
 
@@ -74,8 +74,8 @@ public abstract class GridSwapSpaceSpiAbstractSelfTest extends GridCommonAbstrac
      * @param clsLdr Class loader.
      * @return Swap context.
      */
-    private GridSwapContext context(@Nullable ClassLoader clsLdr) {
-        GridSwapContext ctx = new GridSwapContext();
+    private SwapContext context(@Nullable ClassLoader clsLdr) {
+        SwapContext ctx = new SwapContext();
 
         ctx.classLoader(clsLdr != null ? clsLdr : getClass().getClassLoader());
 
@@ -102,21 +102,21 @@ public abstract class GridSwapSpaceSpiAbstractSelfTest extends GridCommonAbstrac
 
         byte[] val1 = Long.toString(key1).getBytes();
 
-        spi.store(DFLT_SPACE_NAME, new GridSwapKey(key1), val1, context());
+        spi.store(DFLT_SPACE_NAME, new SwapKey(key1), val1, context());
 
         assertEquals(1, spi.count(DFLT_SPACE_NAME));
 
-        assertArrayEquals(spi.read(DFLT_SPACE_NAME, new GridSwapKey(key1), context()), val1);
+        assertArrayEquals(spi.read(DFLT_SPACE_NAME, new SwapKey(key1), context()), val1);
 
         final byte[] val2 = "newValue".getBytes();
 
-        spi.store(DFLT_SPACE_NAME, new GridSwapKey(key1), val2, context());
+        spi.store(DFLT_SPACE_NAME, new SwapKey(key1), val2, context());
 
         assertEquals(1, spi.count(DFLT_SPACE_NAME));
 
-        assertArrayEquals(spi.read(DFLT_SPACE_NAME, new GridSwapKey(key1), context()), val2);
+        assertArrayEquals(spi.read(DFLT_SPACE_NAME, new SwapKey(key1), context()), val2);
 
-        spi.remove(DFLT_SPACE_NAME, new GridSwapKey(key1), new IgniteInClosure<byte[]>() {
+        spi.remove(DFLT_SPACE_NAME, new SwapKey(key1), new IgniteInClosure<byte[]>() {
             @Override public void apply(byte[] old) {
                 assertArrayEquals(val2, old);
             }
@@ -148,49 +148,49 @@ public abstract class GridSwapSpaceSpiAbstractSelfTest extends GridCommonAbstrac
 
         final byte[] val1 = Long.toString(key1).getBytes();
 
-        spi.store(space1, new GridSwapKey(key1), val1, context());
+        spi.store(space1, new SwapKey(key1), val1, context());
 
         assertEquals(1, spi.count(space1));
 
         assertEquals(0, spi.count(space2));
 
-        spi.store(space2, new GridSwapKey(key1), val1, context());
+        spi.store(space2, new SwapKey(key1), val1, context());
 
         assertEquals(1, spi.count(space1));
 
         assertEquals(1, spi.count(space2));
 
-        assertArrayEquals(spi.read(space1, new GridSwapKey(key1), context()), val1);
+        assertArrayEquals(spi.read(space1, new SwapKey(key1), context()), val1);
 
-        assertArrayEquals(spi.read(space2, new GridSwapKey(key1), context()), val1);
+        assertArrayEquals(spi.read(space2, new SwapKey(key1), context()), val1);
 
         long key2 = 2;
 
         byte[] val2 = Long.toString(key2).getBytes();
 
-        spi.store(space1, new GridSwapKey(key2), val2, context());
+        spi.store(space1, new SwapKey(key2), val2, context());
 
         assertEquals(2, spi.count(space1));
 
         assertEquals(1, spi.count(space2));
 
-        assertArrayEquals(spi.read(space1, new GridSwapKey(key2), context()), val2);
+        assertArrayEquals(spi.read(space1, new SwapKey(key2), context()), val2);
 
-        assertNull(spi.read(space2, new GridSwapKey(key2), context()));
+        assertNull(spi.read(space2, new SwapKey(key2), context()));
 
         final byte[] val12 = "newValue".getBytes();
 
-        spi.store(space1, new GridSwapKey(key1), val12, context());
+        spi.store(space1, new SwapKey(key1), val12, context());
 
         assertEquals(2, spi.count(space1));
 
         assertEquals(1, spi.count(space2));
 
-        assertArrayEquals(spi.read(space1, new GridSwapKey(key1), context()), val12);
+        assertArrayEquals(spi.read(space1, new SwapKey(key1), context()), val12);
 
-        assertArrayEquals(spi.read(space2, new GridSwapKey(key1), context()), val1);
+        assertArrayEquals(spi.read(space2, new SwapKey(key1), context()), val1);
 
-        spi.remove(space1, new GridSwapKey(key1), new IgniteInClosure<byte[]>() {
+        spi.remove(space1, new SwapKey(key1), new IgniteInClosure<byte[]>() {
             @Override public void apply(byte[] old) {
                 assertArrayEquals(val12, old);
             }
@@ -200,7 +200,7 @@ public abstract class GridSwapSpaceSpiAbstractSelfTest extends GridCommonAbstrac
 
         assertEquals(1, spi.count(space2));
 
-        spi.remove(space2, new GridSwapKey(key1), new IgniteInClosure<byte[]>() {
+        spi.remove(space2, new SwapKey(key1), new IgniteInClosure<byte[]>() {
             @Override public void apply(byte[] old) {
                 assertArrayEquals(val1, old);
             }
@@ -219,30 +219,30 @@ public abstract class GridSwapSpaceSpiAbstractSelfTest extends GridCommonAbstrac
     public void testBatchCrud() throws Exception {
         assertEquals(0, spi.count(DFLT_SPACE_NAME));
 
-        final Map<GridSwapKey, byte[]> batch = new HashMap<>();
+        final Map<SwapKey, byte[]> batch = new HashMap<>();
 
         int batchSize = 10;
 
         // Generate initial values.
         for (int i = 0; i < batchSize; i++)
-            batch.put(new GridSwapKey(i), Integer.toString(i).getBytes());
+            batch.put(new SwapKey(i), Integer.toString(i).getBytes());
 
         spi.storeAll(DFLT_SPACE_NAME, batch, context());
 
         assertEquals(batchSize, spi.count(DFLT_SPACE_NAME));
 
-        Map<GridSwapKey, byte[]> read = spi.readAll(DFLT_SPACE_NAME, batch.keySet(), context());
+        Map<SwapKey, byte[]> read = spi.readAll(DFLT_SPACE_NAME, batch.keySet(), context());
 
         // Check all entries are as expected.
-        assertTrue(F.forAll(read, new P1<Map.Entry<GridSwapKey, byte[]>>() {
-            @Override public boolean apply(Map.Entry<GridSwapKey, byte[]> e) {
+        assertTrue(F.forAll(read, new P1<Map.Entry<SwapKey, byte[]>>() {
+            @Override public boolean apply(Map.Entry<SwapKey, byte[]> e) {
                 return Arrays.equals(batch.get(e.getKey()), e.getValue());
             }
         }));
 
         // Generate new values.
         for (int i = 0; i < batchSize; i++)
-            batch.put(new GridSwapKey(i), Integer.toString(i + 1).getBytes());
+            batch.put(new SwapKey(i), Integer.toString(i + 1).getBytes());
 
         spi.storeAll(DFLT_SPACE_NAME, batch, context());
 
@@ -251,8 +251,8 @@ public abstract class GridSwapSpaceSpiAbstractSelfTest extends GridCommonAbstrac
         read = spi.readAll(DFLT_SPACE_NAME, batch.keySet(), context());
 
         // Check all entries are as expected.
-        assertTrue(F.forAll(read, new P1<Map.Entry<GridSwapKey, byte[]>>() {
-            @Override public boolean apply(Map.Entry<GridSwapKey, byte[]> e) {
+        assertTrue(F.forAll(read, new P1<Map.Entry<SwapKey, byte[]>>() {
+            @Override public boolean apply(Map.Entry<SwapKey, byte[]> e) {
                 return Arrays.equals(batch.get(e.getKey()), e.getValue());
             }
         }));
@@ -266,9 +266,9 @@ public abstract class GridSwapSpaceSpiAbstractSelfTest extends GridCommonAbstrac
      * @throws Exception If failed.
      */
     public void testDeleteIfNotPersist() throws Exception {
-        spi.store(SPACE1, new GridSwapKey("key1"), "value1".getBytes(), context());
+        spi.store(SPACE1, new SwapKey("key1"), "value1".getBytes(), context());
 
-        assertArrayEquals("value1".getBytes(), spi.read(SPACE1, new GridSwapKey("key1"), context()));
+        assertArrayEquals("value1".getBytes(), spi.read(SPACE1, new SwapKey("key1"), context()));
     }
 
     /**
@@ -281,7 +281,7 @@ public abstract class GridSwapSpaceSpiAbstractSelfTest extends GridCommonAbstrac
         final CountDownLatch readLatch = new CountDownLatch(cnt);
         final CountDownLatch rmvLatch = new CountDownLatch(cnt);
 
-        spi.setListener(new GridSwapSpaceSpiListener() {
+        spi.setListener(new SwapSpaceSpiListener() {
             @Override public void onSwapEvent(int evtType, @Nullable String spaceName, @Nullable byte[] keyBytes) {
                 info("Received event: " + evtType);
 
@@ -300,22 +300,22 @@ public abstract class GridSwapSpaceSpiAbstractSelfTest extends GridCommonAbstrac
         });
 
         for (int i = 0; i < cnt; i++)
-            assertNull(spi.read(SPACE1, new GridSwapKey("key" + i), context()));
+            assertNull(spi.read(SPACE1, new SwapKey("key" + i), context()));
 
         for (int i = 0; i < cnt; i++)
-            spi.store(SPACE1, new GridSwapKey("key" + i), str2ByteArray("value" + i), context());
+            spi.store(SPACE1, new SwapKey("key" + i), str2ByteArray("value" + i), context());
 
         assert storeLatch.await(5000, MILLISECONDS);
 
         for (int i = 0; i < cnt; i++)
-            assertArrayEquals(str2ByteArray("value" + i), spi.read(SPACE1, new GridSwapKey("key" + i), context()));
+            assertArrayEquals(str2ByteArray("value" + i), spi.read(SPACE1, new SwapKey("key" + i), context()));
 
         assert readLatch.await(5000, MILLISECONDS);
 
         for (int i = 0; i < cnt; i++) {
             final int tmp = i;
 
-            spi.remove(SPACE1, new GridSwapKey("key" + i), new CI1<byte[]>() {
+            spi.remove(SPACE1, new SwapKey("key" + i), new CI1<byte[]>() {
                 @Override public void apply(byte[] arr) {
                     assertArrayEquals(str2ByteArray("value" + tmp), arr);
 
@@ -327,7 +327,7 @@ public abstract class GridSwapSpaceSpiAbstractSelfTest extends GridCommonAbstrac
         assert rmvLatch.await(10000, MILLISECONDS);
 
         for (int i = 0; i < cnt; i++)
-            assertNull(spi.read(SPACE1, new GridSwapKey("key" + i), context()));
+            assertNull(spi.read(SPACE1, new SwapKey("key" + i), context()));
     }
 
 
@@ -341,7 +341,7 @@ public abstract class GridSwapSpaceSpiAbstractSelfTest extends GridCommonAbstrac
         final CountDownLatch readLatch = new CountDownLatch(cnt);
         final CountDownLatch rmvLatch = new CountDownLatch(cnt);
 
-        spi.setListener(new GridSwapSpaceSpiListener() {
+        spi.setListener(new SwapSpaceSpiListener() {
             @Override public void onSwapEvent(int evtType, @Nullable String spaceName, @Nullable byte[] keyBytes) {
                 info("Received event: " + evtType);
 
@@ -360,22 +360,22 @@ public abstract class GridSwapSpaceSpiAbstractSelfTest extends GridCommonAbstrac
         });
 
         for (int i = 0; i < cnt; i++)
-            assertNull(spi.read(SPACE1, new GridSwapKey("key" + i), context()));
+            assertNull(spi.read(SPACE1, new SwapKey("key" + i), context()));
 
         for (int i = 0; i < cnt; i++)
-            spi.store(SPACE1, new GridSwapKey("key" + i), null, context());
+            spi.store(SPACE1, new SwapKey("key" + i), null, context());
 
         assert storeLatch.await(5000, MILLISECONDS);
 
         for (int i = 0; i < cnt; i++)
-            assertNull(spi.read(SPACE1, new GridSwapKey("key" + i), context()));
+            assertNull(spi.read(SPACE1, new SwapKey("key" + i), context()));
 
         assert readLatch.await(5000, MILLISECONDS);
 
         for (int i = 0; i < cnt; i++) {
             final int tmp = i;
 
-            spi.remove(SPACE1, new GridSwapKey("key" + i), new CI1<byte[]>() {
+            spi.remove(SPACE1, new SwapKey("key" + i), new CI1<byte[]>() {
                 @Override public void apply(byte[] arr) {
                     assertNull(arr);
 
@@ -387,7 +387,7 @@ public abstract class GridSwapSpaceSpiAbstractSelfTest extends GridCommonAbstrac
         assert rmvLatch.await(10000, MILLISECONDS);
 
         for (int i = 0; i < cnt; i++)
-            assertNull(spi.read(SPACE1, new GridSwapKey("key" + i), context()));
+            assertNull(spi.read(SPACE1, new SwapKey("key" + i), context()));
     }
 
     /**
@@ -400,7 +400,7 @@ public abstract class GridSwapSpaceSpiAbstractSelfTest extends GridCommonAbstrac
         final CountDownLatch readLatch = new CountDownLatch(cnt);
         final CountDownLatch rmvLatch = new CountDownLatch(cnt);
 
-        spi.setListener(new GridSwapSpaceSpiListener() {
+        spi.setListener(new SwapSpaceSpiListener() {
             @Override public void onSwapEvent(int evtType, @Nullable String spaceName, @Nullable byte[] keyBytes) {
                 info("Received event: " + evtType);
 
@@ -425,7 +425,7 @@ public abstract class GridSwapSpaceSpiAbstractSelfTest extends GridCommonAbstrac
         for (int i = 0; i < cnt; i++) {
             String val = "value" + i;
 
-            spi.store(SPACE1, new GridSwapKey(new Key(i)), str2ByteArray(val), context());
+            spi.store(SPACE1, new SwapKey(new Key(i)), str2ByteArray(val), context());
 
             keys.add(i);
 
@@ -436,14 +436,14 @@ public abstract class GridSwapSpaceSpiAbstractSelfTest extends GridCommonAbstrac
 
         for (int i = 0; i < cnt; i++)
             assertArrayEquals(entries.get(i).getBytes(),
-                spi.read(SPACE1, new GridSwapKey(new Key(i)), context()));
+                spi.read(SPACE1, new SwapKey(new Key(i)), context()));
 
         assert readLatch.await(5000, MILLISECONDS) : "Count: " + readLatch.getCount();
 
         Collections.shuffle(keys);
 
         for (final Integer key : keys) {
-            spi.remove(SPACE1, new GridSwapKey(new Key(key)), new CI1<byte[]>() {
+            spi.remove(SPACE1, new SwapKey(new Key(key)), new CI1<byte[]>() {
                 @Override public void apply(byte[] arr) {
                     assertArrayEquals(entries.get(key).getBytes(), arr);
 
@@ -455,7 +455,7 @@ public abstract class GridSwapSpaceSpiAbstractSelfTest extends GridCommonAbstrac
         assert rmvLatch.await(5000, MILLISECONDS) : "Count: " + rmvLatch.getCount();
 
         for (final Integer key : keys)
-            assertNull(spi.read(SPACE1, new GridSwapKey(new Key(key)), context()));
+            assertNull(spi.read(SPACE1, new SwapKey(new Key(key)), context()));
     }
 
 
@@ -468,11 +468,11 @@ public abstract class GridSwapSpaceSpiAbstractSelfTest extends GridCommonAbstrac
         int cnt = 10;
 
         for (int i = 0; i < cnt; i++)
-            spi.store(SPACE1, new GridSwapKey("key" + i, i), str2ByteArray("value" + i), context());
+            spi.store(SPACE1, new SwapKey("key" + i, i), str2ByteArray("value" + i), context());
 
         for (int i = 0; i < cnt; i++)
             assertArrayEquals(str2ByteArray("value" + i),
-                spi.read(SPACE1, new GridSwapKey("key" + i, i), context()));
+                spi.read(SPACE1, new SwapKey("key" + i, i), context()));
 
         try (IgniteSpiCloseableIterator<Map.Entry<byte[], byte[]>> iter = spi.rawIterator(SPACE1)) {
             assertNotNull(iter);
@@ -501,18 +501,18 @@ public abstract class GridSwapSpaceSpiAbstractSelfTest extends GridCommonAbstrac
      * @throws Exception If failed.
      */
     public void testIterationOverPartition() throws Exception {
-        spi.store(SPACE1, new GridSwapKey("key", 0), str2ByteArray("value"), context());
+        spi.store(SPACE1, new SwapKey("key", 0), str2ByteArray("value"), context());
 
         spi.clear(SPACE1);
 
         int cnt = 10;
 
         for (int i = 0; i < cnt; i++)
-            spi.store(SPACE1, new GridSwapKey("key" + i, i), str2ByteArray("value" + i), context());
+            spi.store(SPACE1, new SwapKey("key" + i, i), str2ByteArray("value" + i), context());
 
         for (int i = 0; i < cnt; i++)
             assertArrayEquals(str2ByteArray("value" + i),
-                spi.read(SPACE1, new GridSwapKey("key" + i, i), context()));
+                spi.read(SPACE1, new SwapKey("key" + i, i), context()));
 
         try (IgniteSpiCloseableIterator<Map.Entry<byte[], byte[]>> iter = spi.rawIterator(SPACE1, 5)) {
             assertNotNull(iter);
@@ -530,7 +530,7 @@ public abstract class GridSwapSpaceSpiAbstractSelfTest extends GridCommonAbstrac
 
                 iter.remove();
 
-                assertNull(spi.read(SPACE1, new GridSwapKey(key, 5), context()));
+                assertNull(spi.read(SPACE1, new SwapKey(key, 5), context()));
 
                 i++;
             }
@@ -543,14 +543,14 @@ public abstract class GridSwapSpaceSpiAbstractSelfTest extends GridCommonAbstrac
      * @throws Exception If failed.
      */
     public void testSwapIterator() throws Exception {
-        spi.store(SPACE1, new GridSwapKey("key", 0), str2ByteArray("value"), context());
+        spi.store(SPACE1, new SwapKey("key", 0), str2ByteArray("value"), context());
 
         spi.clear(SPACE1);
 
         int cnt = 10;
 
         for (int i = 0; i < cnt; i++)
-            spi.store(SPACE1, new GridSwapKey("key" + i, i), str2ByteArray("value" + i), context());
+            spi.store(SPACE1, new SwapKey("key" + i, i), str2ByteArray("value" + i), context());
 
         IgniteSpiCloseableIterator<Map.Entry<byte[], byte[]>> iter = spi.rawIterator(SPACE1);
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c31cec7c/modules/core/src/test/java/org/gridgain/grid/spi/swapspace/file/GridFileSwapCompactionSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/gridgain/grid/spi/swapspace/file/GridFileSwapCompactionSelfTest.java b/modules/core/src/test/java/org/gridgain/grid/spi/swapspace/file/GridFileSwapCompactionSelfTest.java
index 84dcb75..73b2491 100644
--- a/modules/core/src/test/java/org/gridgain/grid/spi/swapspace/file/GridFileSwapCompactionSelfTest.java
+++ b/modules/core/src/test/java/org/gridgain/grid/spi/swapspace/file/GridFileSwapCompactionSelfTest.java
@@ -17,7 +17,7 @@ import java.nio.ByteBuffer;
 import java.util.*;
 
 /**
- * Test for {@link GridFileSwapSpaceSpi}.
+ * Test for {@link FileSwapSpaceSpi}.
  */
 public class GridFileSwapCompactionSelfTest extends GridCommonAbstractTest {
     /**
@@ -28,16 +28,16 @@ public class GridFileSwapCompactionSelfTest extends GridCommonAbstractTest {
 
         X.println("file: " + file.getPath());
 
-        GridFileSwapSpaceSpi.SwapFile f = new GridFileSwapSpaceSpi.SwapFile(file, 8);
+        FileSwapSpaceSpi.SwapFile f = new FileSwapSpaceSpi.SwapFile(file, 8);
 
         Random rnd = new Random();
 
-        ArrayList<GridFileSwapSpaceSpi.SwapValue> arr = new ArrayList<>();
+        ArrayList<FileSwapSpaceSpi.SwapValue> arr = new ArrayList<>();
 
         int size = 0;
 
         for (int a = 0; a < 100; a++) {
-            GridFileSwapSpaceSpi.SwapValue[] vals = new GridFileSwapSpaceSpi.SwapValue[1 + rnd.nextInt(10)];
+            FileSwapSpaceSpi.SwapValue[] vals = new FileSwapSpaceSpi.SwapValue[1 + rnd.nextInt(10)];
 
             int size0 = 0;
 
@@ -48,12 +48,12 @@ public class GridFileSwapCompactionSelfTest extends GridCommonAbstractTest {
 
                 size0 += bytes.length;
 
-                vals[i] = new GridFileSwapSpaceSpi.SwapValue(bytes);
+                vals[i] = new FileSwapSpaceSpi.SwapValue(bytes);
 
                 arr.add(vals[i]);
             }
 
-            f.write(new GridFileSwapSpaceSpi.SwapValues(vals, size0), 1);
+            f.write(new FileSwapSpaceSpi.SwapValues(vals, size0), 1);
 
             size += size0;
 
@@ -63,27 +63,27 @@ public class GridFileSwapCompactionSelfTest extends GridCommonAbstractTest {
 
         int i = 0;
 
-        for (GridFileSwapSpaceSpi.SwapValue val : arr)
+        for (FileSwapSpaceSpi.SwapValue val : arr)
             assertEquals(val.idx(), ++i);
 
         i = 0;
 
         for (int cnt = arr.size() / 2; i < cnt; i++) {
 
-            GridFileSwapSpaceSpi.SwapValue v = arr.remove(rnd.nextInt(arr.size()));
+            FileSwapSpaceSpi.SwapValue v = arr.remove(rnd.nextInt(arr.size()));
 
             assertTrue(f.tryRemove(v.idx(), v));
         }
 
         int hash0 = 0;
 
-        for (GridFileSwapSpaceSpi.SwapValue val : arr)
+        for (FileSwapSpaceSpi.SwapValue val : arr)
             hash0 += Arrays.hashCode(val.readValue(f.readCh));
 
-        ArrayList<T2<ByteBuffer, ArrayDeque<GridFileSwapSpaceSpi.SwapValue>>> bufs = new ArrayList();
+        ArrayList<T2<ByteBuffer, ArrayDeque<FileSwapSpaceSpi.SwapValue>>> bufs = new ArrayList();
 
         for (;;) {
-            ArrayDeque<GridFileSwapSpaceSpi.SwapValue> que = new ArrayDeque<>();
+            ArrayDeque<FileSwapSpaceSpi.SwapValue> que = new ArrayDeque<>();
 
             ByteBuffer buf = f.compact(que, 1024);
 
@@ -97,21 +97,21 @@ public class GridFileSwapCompactionSelfTest extends GridCommonAbstractTest {
 
         int hash1 = 0;
 
-        for (GridFileSwapSpaceSpi.SwapValue val : arr)
+        for (FileSwapSpaceSpi.SwapValue val : arr)
             hash1 += Arrays.hashCode(val.value(null));
 
         assertEquals(hash0, hash1);
 
         File file0 = new File(UUID.randomUUID().toString());
 
-        GridFileSwapSpaceSpi.SwapFile f0 = new GridFileSwapSpaceSpi.SwapFile(file0, 8);
+        FileSwapSpaceSpi.SwapFile f0 = new FileSwapSpaceSpi.SwapFile(file0, 8);
 
-        for (T2<ByteBuffer, ArrayDeque<GridFileSwapSpaceSpi.SwapValue>> t : bufs)
+        for (T2<ByteBuffer, ArrayDeque<FileSwapSpaceSpi.SwapValue>> t : bufs)
             f0.write(t.get2(), t.get1(), 1);
 
         int hash2 = 0;
 
-        for (GridFileSwapSpaceSpi.SwapValue val : arr)
+        for (FileSwapSpaceSpi.SwapValue val : arr)
             hash2 += Arrays.hashCode(val.readValue(f0.readCh));
 
         assertEquals(hash2, hash1);

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c31cec7c/modules/core/src/test/java/org/gridgain/grid/spi/swapspace/file/GridFileSwapSpaceSpiSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/gridgain/grid/spi/swapspace/file/GridFileSwapSpaceSpiSelfTest.java b/modules/core/src/test/java/org/gridgain/grid/spi/swapspace/file/GridFileSwapSpaceSpiSelfTest.java
index b25006a..a7b7f85 100644
--- a/modules/core/src/test/java/org/gridgain/grid/spi/swapspace/file/GridFileSwapSpaceSpiSelfTest.java
+++ b/modules/core/src/test/java/org/gridgain/grid/spi/swapspace/file/GridFileSwapSpaceSpiSelfTest.java
@@ -22,12 +22,12 @@ import java.util.concurrent.*;
 import java.util.concurrent.atomic.*;
 
 /**
- * Test for {@link GridFileSwapSpaceSpi}.
+ * Test for {@link FileSwapSpaceSpi}.
  */
 public class GridFileSwapSpaceSpiSelfTest extends GridSwapSpaceSpiAbstractSelfTest {
     /** {@inheritDoc} */
-    @Override protected GridSwapSpaceSpi spi() {
-        GridFileSwapSpaceSpi s = new GridFileSwapSpaceSpi();
+    @Override protected SwapSpaceSpi spi() {
+        FileSwapSpaceSpi s = new FileSwapSpaceSpi();
 
         s.setMaximumSparsity(0.05f);
         s.setWriteBufferSize(8 * 1024);
@@ -43,7 +43,7 @@ public class GridFileSwapSpaceSpiSelfTest extends GridSwapSpaceSpiAbstractSelfTe
     public void testMultithreadedWrite() throws Exception {
         final AtomicLong valCntr = new AtomicLong();
 
-        final GridSwapKey key = new GridSwapKey("key");
+        final SwapKey key = new SwapKey("key");
 
         final CountDownLatch wLatch = new CountDownLatch(1);
 
@@ -93,15 +93,15 @@ public class GridFileSwapSpaceSpiSelfTest extends GridSwapSpaceSpiAbstractSelfTe
      * @param i Integer.
      * @return Swap key.
      */
-    private GridSwapKey key(int i) {
-        return new GridSwapKey(i, i % 11, U.intToBytes(i));
+    private SwapKey key(int i) {
+        return new SwapKey(i, i % 11, U.intToBytes(i));
     }
 
     /**
      * @throws Exception If failed.
      */
     public void testMultithreadedOperations() throws Exception {
-        final ConcurrentHashMap8<GridSwapKey, byte[]> map = new ConcurrentHashMap8<>();
+        final ConcurrentHashMap8<SwapKey, byte[]> map = new ConcurrentHashMap8<>();
 
         Random rnd = new Random();
 
@@ -136,7 +136,7 @@ public class GridFileSwapSpaceSpiSelfTest extends GridSwapSpaceSpiAbstractSelfTe
                 Random rnd = new Random();
 
                 while (!fin.get()) {
-                    final GridSwapKey key = key(rnd.nextInt(keys));
+                    final SwapKey key = key(rnd.nextInt(keys));
 
                     switch(rnd.nextInt(13)) {
                         case 0: // store
@@ -175,12 +175,12 @@ public class GridFileSwapSpaceSpiSelfTest extends GridSwapSpaceSpiAbstractSelfTe
                         case 3: // storeAll
                         case 4:
                         case 9:
-                            Map<GridSwapKey, byte[]> m = new HashMap<>();
+                            Map<SwapKey, byte[]> m = new HashMap<>();
 
                             int cnt = 1 + rnd.nextInt(25);
 
                             for (int i = 0; i < cnt; i++) {
-                                GridSwapKey k = key(rnd.nextInt(keys));
+                                SwapKey k = key(rnd.nextInt(keys));
 
                                 val = map.remove(k);
 
@@ -196,12 +196,12 @@ public class GridFileSwapSpaceSpiSelfTest extends GridSwapSpaceSpiAbstractSelfTe
                             break;
 
                         case 5: // readAll
-                            HashSet<GridSwapKey> s = new HashSet<>();
+                            HashSet<SwapKey> s = new HashSet<>();
 
                             cnt = 1 + rnd.nextInt(25);
 
                             for (int i = 0; i < cnt; i++) {
-                                GridSwapKey k = key(rnd.nextInt(keys));
+                                SwapKey k = key(rnd.nextInt(keys));
 
                                 val = map.get(k);
 
@@ -214,10 +214,10 @@ public class GridFileSwapSpaceSpiSelfTest extends GridSwapSpaceSpiAbstractSelfTe
 
                                 s.removeAll(m.keySet());
 
-                                Iterator<GridSwapKey> iter = s.iterator();
+                                Iterator<SwapKey> iter = s.iterator();
 
                                 while (iter.hasNext()) {
-                                    GridSwapKey k = iter.next();
+                                    SwapKey k = iter.next();
 
                                     if (map.containsKey(k))
                                         iter.remove();
@@ -285,7 +285,7 @@ public class GridFileSwapSpaceSpiSelfTest extends GridSwapSpaceSpiAbstractSelfTe
                             cnt = 1 + rnd.nextInt(25);
 
                             for (int i = 0; i < cnt; i++) {
-                                GridSwapKey k = key(rnd.nextInt(keys));
+                                SwapKey k = key(rnd.nextInt(keys));
 
                                 val = map.get(k);
 
@@ -296,8 +296,8 @@ public class GridFileSwapSpaceSpiSelfTest extends GridSwapSpaceSpiAbstractSelfTe
                             if (s.isEmpty())
                                 break;
 
-                            spi.removeAll(space, s, new IgniteBiInClosure<GridSwapKey, byte[]>() {
-                                @Override public void apply(GridSwapKey k, byte[] bytes) {
+                            spi.removeAll(space, s, new IgniteBiInClosure<SwapKey, byte[]>() {
+                                @Override public void apply(SwapKey k, byte[] bytes) {
                                     if (bytes != null)
                                         assertNull(map.putIfAbsent(k, bytes));
                                 }
@@ -337,7 +337,7 @@ public class GridFileSwapSpaceSpiSelfTest extends GridSwapSpaceSpiAbstractSelfTe
 
         assertEquals(cnt, spi.count(space));
 
-        for (Map.Entry<GridSwapKey, byte[]> entry : map.entrySet())
+        for (Map.Entry<SwapKey, byte[]> entry : map.entrySet())
             hash1 += (Integer)entry.getKey().key() * Arrays.hashCode(entry.getValue());
 
         assertEquals(hash0, hash1);

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c31cec7c/modules/core/src/test/java/org/gridgain/grid/spi/swapspace/inmemory/GridTestSwapSpaceSpi.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/gridgain/grid/spi/swapspace/inmemory/GridTestSwapSpaceSpi.java b/modules/core/src/test/java/org/gridgain/grid/spi/swapspace/inmemory/GridTestSwapSpaceSpi.java
index 2f44ab3..7e96007 100644
--- a/modules/core/src/test/java/org/gridgain/grid/spi/swapspace/inmemory/GridTestSwapSpaceSpi.java
+++ b/modules/core/src/test/java/org/gridgain/grid/spi/swapspace/inmemory/GridTestSwapSpaceSpi.java
@@ -25,9 +25,9 @@ import static org.apache.ignite.events.IgniteEventType.*;
  * Test swap space SPI that stores values in map.
  */
 @IgniteSpiMultipleInstancesSupport(true)
-public class GridTestSwapSpaceSpi extends IgniteSpiAdapter implements GridSwapSpaceSpi {
+public class GridTestSwapSpaceSpi extends IgniteSpiAdapter implements SwapSpaceSpi {
     /** Listener. */
-    private GridSwapSpaceSpiListener lsnr;
+    private SwapSpaceSpiListener lsnr;
 
     /** Spaces map. */
     private ConcurrentMap<String, Space> spaces = new ConcurrentHashMap8<>();
@@ -65,7 +65,7 @@ public class GridTestSwapSpaceSpi extends IgniteSpiAdapter implements GridSwapSp
     }
 
     /** {@inheritDoc} */
-    @Override public byte[] read(@Nullable String spaceName, GridSwapKey key, GridSwapContext ctx)
+    @Override public byte[] read(@Nullable String spaceName, SwapKey key, SwapContext ctx)
         throws IgniteSpiException {
         Space space = space(spaceName);
 
@@ -73,16 +73,16 @@ public class GridTestSwapSpaceSpi extends IgniteSpiAdapter implements GridSwapSp
     }
 
     /** {@inheritDoc} */
-    @Override public Map<GridSwapKey, byte[]> readAll(@Nullable String spaceName, Iterable<GridSwapKey> keys,
-        GridSwapContext ctx) throws IgniteSpiException {
+    @Override public Map<SwapKey, byte[]> readAll(@Nullable String spaceName, Iterable<SwapKey> keys,
+        SwapContext ctx) throws IgniteSpiException {
         Space space = space(spaceName);
 
-        return space != null ? space.readAll(keys) : Collections.<GridSwapKey, byte[]>emptyMap();
+        return space != null ? space.readAll(keys) : Collections.<SwapKey, byte[]>emptyMap();
     }
 
     /** {@inheritDoc} */
-    @Override public void remove(@Nullable String spaceName, GridSwapKey key, @Nullable IgniteInClosure<byte[]> c,
-        GridSwapContext ctx) throws IgniteSpiException {
+    @Override public void remove(@Nullable String spaceName, SwapKey key, @Nullable IgniteInClosure<byte[]> c,
+        SwapContext ctx) throws IgniteSpiException {
         Space space = space(spaceName);
 
         if (space != null)
@@ -90,8 +90,8 @@ public class GridTestSwapSpaceSpi extends IgniteSpiAdapter implements GridSwapSp
     }
 
     /** {@inheritDoc} */
-    @Override public void removeAll(@Nullable String spaceName, Collection<GridSwapKey> keys,
-        @Nullable IgniteBiInClosure<GridSwapKey, byte[]> c, GridSwapContext ctx) throws IgniteSpiException {
+    @Override public void removeAll(@Nullable String spaceName, Collection<SwapKey> keys,
+        @Nullable IgniteBiInClosure<SwapKey, byte[]> c, SwapContext ctx) throws IgniteSpiException {
         Space space = space(spaceName);
 
         if (space != null)
@@ -99,19 +99,19 @@ public class GridTestSwapSpaceSpi extends IgniteSpiAdapter implements GridSwapSp
     }
 
     /** {@inheritDoc} */
-    @Override public void store(@Nullable String spaceName, GridSwapKey key, @Nullable byte[] val, GridSwapContext ctx)
+    @Override public void store(@Nullable String spaceName, SwapKey key, @Nullable byte[] val, SwapContext ctx)
         throws IgniteSpiException {
         ensureSpace(spaceName).store(key, val);
     }
 
     /** {@inheritDoc} */
-    @Override public void storeAll(@Nullable String spaceName, Map<GridSwapKey, byte[]> pairs, GridSwapContext ctx)
+    @Override public void storeAll(@Nullable String spaceName, Map<SwapKey, byte[]> pairs, SwapContext ctx)
         throws IgniteSpiException {
         ensureSpace(spaceName).storeAll(pairs);
     }
 
     /** {@inheritDoc} */
-    @Override public void setListener(@Nullable GridSwapSpaceSpiListener evictLsnr) {
+    @Override public void setListener(@Nullable SwapSpaceSpiListener evictLsnr) {
         lsnr = evictLsnr;
     }
 
@@ -123,7 +123,7 @@ public class GridTestSwapSpaceSpi extends IgniteSpiAdapter implements GridSwapSp
     }
 
     /** {@inheritDoc} */
-    @Override public <K> IgniteSpiCloseableIterator<K> keyIterator(@Nullable String spaceName, GridSwapContext ctx)
+    @Override public <K> IgniteSpiCloseableIterator<K> keyIterator(@Nullable String spaceName, SwapContext ctx)
         throws IgniteSpiException {
         return ensureSpace(spaceName).keyIterator();
     }
@@ -164,7 +164,7 @@ public class GridTestSwapSpaceSpi extends IgniteSpiAdapter implements GridSwapSp
     }
 
     private void fireEvent(int evtType, String spaceName, @Nullable byte[] key) {
-        GridSwapSpaceSpiListener lsnr0 = lsnr;
+        SwapSpaceSpiListener lsnr0 = lsnr;
 
         if (lsnr0 != null)
             lsnr0.onSwapEvent(evtType, spaceName, key);
@@ -172,7 +172,7 @@ public class GridTestSwapSpaceSpi extends IgniteSpiAdapter implements GridSwapSp
 
     private class Space {
         /** Data storage. */
-        private ConcurrentMap<GridSwapKey, byte[]> data = new ConcurrentHashMap8<>();
+        private ConcurrentMap<SwapKey, byte[]> data = new ConcurrentHashMap8<>();
 
         private final String name;
 
@@ -210,7 +210,7 @@ public class GridTestSwapSpaceSpi extends IgniteSpiAdapter implements GridSwapSp
          * @param key Key to read.
          * @return Read bytes.
          */
-        public byte[] read(GridSwapKey key) {
+        public byte[] read(SwapKey key) {
             byte[] bytes = data.get(key);
 
             fireEvent(EVT_SWAP_SPACE_DATA_READ, name, key.keyBytes());
@@ -222,10 +222,10 @@ public class GridTestSwapSpaceSpi extends IgniteSpiAdapter implements GridSwapSp
          * @param keys Keys to read.
          * @return Read keys.
          */
-        public Map<GridSwapKey, byte[]> readAll(Iterable<GridSwapKey> keys) {
-            Map<GridSwapKey, byte[]> res = new HashMap<>();
+        public Map<SwapKey, byte[]> readAll(Iterable<SwapKey> keys) {
+            Map<SwapKey, byte[]> res = new HashMap<>();
 
-            for (GridSwapKey key : keys) {
+            for (SwapKey key : keys) {
                 byte[] val = data.get(key);
 
                 if (val != null) {
@@ -242,7 +242,7 @@ public class GridTestSwapSpaceSpi extends IgniteSpiAdapter implements GridSwapSp
          * @param key Key to remove.
          * @param c Closure.
          */
-        public void remove(GridSwapKey key, IgniteInClosure<byte[]> c) {
+        public void remove(SwapKey key, IgniteInClosure<byte[]> c) {
             byte[] val = data.remove(key);
 
             if (val != null) {
@@ -256,8 +256,8 @@ public class GridTestSwapSpaceSpi extends IgniteSpiAdapter implements GridSwapSp
          * @param keys Keys to remove.
          * @param c Closure to apply for removed values.
          */
-        public void removeAll(Iterable<GridSwapKey> keys, IgniteBiInClosure<GridSwapKey, byte[]> c) {
-            for (GridSwapKey key : keys) {
+        public void removeAll(Iterable<SwapKey> keys, IgniteBiInClosure<SwapKey, byte[]> c) {
+            for (SwapKey key : keys) {
                 byte[] val = data.remove(key);
 
                 if (val != null) {
@@ -272,7 +272,7 @@ public class GridTestSwapSpaceSpi extends IgniteSpiAdapter implements GridSwapSp
          * @param key Key to store.
          * @param val Value to store.
          */
-        public void store(GridSwapKey key, byte[] val) {
+        public void store(SwapKey key, byte[] val) {
             if (val != null) {
                 data.put(key, val);
 
@@ -289,9 +289,9 @@ public class GridTestSwapSpaceSpi extends IgniteSpiAdapter implements GridSwapSp
         /**
          * @param pairs Values to store.
          */
-        public void storeAll(Map<GridSwapKey, byte[]> pairs) {
-            for (Map.Entry<GridSwapKey, byte[]> entry : pairs.entrySet()) {
-                GridSwapKey key = entry.getKey();
+        public void storeAll(Map<SwapKey, byte[]> pairs) {
+            for (Map.Entry<SwapKey, byte[]> entry : pairs.entrySet()) {
+                SwapKey key = entry.getKey();
                 byte[] val = entry.getValue();
 
                 store(key, val);
@@ -304,14 +304,14 @@ public class GridTestSwapSpaceSpi extends IgniteSpiAdapter implements GridSwapSp
         public Collection<Integer> partitions() {
             Collection<Integer> parts = new HashSet<>();
 
-            for (GridSwapKey key : data.keySet())
+            for (SwapKey key : data.keySet())
                 parts.add(key.partition());
 
             return parts;
         }
 
         public <K> IgniteSpiCloseableIterator<K> keyIterator() {
-            final Iterator<GridSwapKey> it = data.keySet().iterator();
+            final Iterator<SwapKey> it = data.keySet().iterator();
 
             return new IgniteSpiCloseableIterator<K>() {
                 @Override public void close() {
@@ -323,7 +323,7 @@ public class GridTestSwapSpaceSpi extends IgniteSpiAdapter implements GridSwapSp
                 }
 
                 @Override public K next() {
-                    GridSwapKey next = it.next();
+                    SwapKey next = it.next();
 
                     return (K)next.key();
                 }
@@ -335,7 +335,7 @@ public class GridTestSwapSpaceSpi extends IgniteSpiAdapter implements GridSwapSp
         }
 
         public IgniteSpiCloseableIterator<Map.Entry<byte[], byte[]>> rawIterator() {
-            final Iterator<Map.Entry<GridSwapKey, byte[]>> it = data.entrySet().iterator();
+            final Iterator<Map.Entry<SwapKey, byte[]>> it = data.entrySet().iterator();
 
             return new IgniteSpiCloseableIterator<Map.Entry<byte[], byte[]>>() {
                 @Override public void close() {
@@ -347,7 +347,7 @@ public class GridTestSwapSpaceSpi extends IgniteSpiAdapter implements GridSwapSp
                 }
 
                 @Override public Map.Entry<byte[], byte[]> next() {
-                    final Map.Entry<GridSwapKey, byte[]> next = it.next();
+                    final Map.Entry<SwapKey, byte[]> next = it.next();
 
                     return new Map.Entry<byte[], byte[]>() {
                         @Override public byte[] getKey() {
@@ -371,13 +371,13 @@ public class GridTestSwapSpaceSpi extends IgniteSpiAdapter implements GridSwapSp
         }
 
         public IgniteSpiCloseableIterator<Map.Entry<byte[], byte[]>> rawIterator(final int part) {
-            final Iterator<Map.Entry<GridSwapKey, byte[]>> it = data.entrySet().iterator();
+            final Iterator<Map.Entry<SwapKey, byte[]>> it = data.entrySet().iterator();
 
             return new IgniteSpiCloseableIterator<Map.Entry<byte[], byte[]>>() {
                 /** Next entry in this iterator. */
-                private Map.Entry<GridSwapKey, byte[]> next;
+                private Map.Entry<SwapKey, byte[]> next;
 
-                private Map.Entry<GridSwapKey, byte[]> cur;
+                private Map.Entry<SwapKey, byte[]> cur;
 
                 {
                     advance();
@@ -395,7 +395,7 @@ public class GridTestSwapSpaceSpi extends IgniteSpiAdapter implements GridSwapSp
                     if (next == null)
                         throw new NoSuchElementException();
 
-                    final Map.Entry<GridSwapKey, byte[]> ret = next;
+                    final Map.Entry<SwapKey, byte[]> ret = next;
 
                     cur = ret;
 
@@ -425,7 +425,7 @@ public class GridTestSwapSpaceSpi extends IgniteSpiAdapter implements GridSwapSp
 
                 private void advance() {
                     while (it.hasNext()) {
-                        Map.Entry<GridSwapKey, byte[]> entry = it.next();
+                        Map.Entry<SwapKey, byte[]> entry = it.next();
 
                         if(entry.getKey().partition() == part) {
                             cur = next;

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c31cec7c/modules/core/src/test/java/org/gridgain/grid/spi/swapspace/noop/GridNoopSwapSpaceSpiSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/gridgain/grid/spi/swapspace/noop/GridNoopSwapSpaceSpiSelfTest.java b/modules/core/src/test/java/org/gridgain/grid/spi/swapspace/noop/GridNoopSwapSpaceSpiSelfTest.java
index 9a1cbc7..d7b90de 100644
--- a/modules/core/src/test/java/org/gridgain/grid/spi/swapspace/noop/GridNoopSwapSpaceSpiSelfTest.java
+++ b/modules/core/src/test/java/org/gridgain/grid/spi/swapspace/noop/GridNoopSwapSpaceSpiSelfTest.java
@@ -17,7 +17,7 @@ import org.gridgain.grid.spi.swapspace.*;
 import org.gridgain.testframework.junits.common.*;
 
 /**
- * Tests for "noop" realization of {@link GridSwapSpaceSpi}.
+ * Tests for "noop" realization of {@link org.gridgain.grid.spi.swapspace.SwapSpaceSpi}.
  */
 public class GridNoopSwapSpaceSpiSelfTest extends GridCommonAbstractTest {
     /** {@inheritDoc} */
@@ -40,11 +40,11 @@ public class GridNoopSwapSpaceSpiSelfTest extends GridCommonAbstractTest {
         try {
             Ignite ignite = startGrid(1);
 
-            GridSwapSpaceSpi spi = ignite.configuration().getSwapSpaceSpi();
+            SwapSpaceSpi spi = ignite.configuration().getSwapSpaceSpi();
 
             assertNotNull(spi);
 
-            assertTrue(spi instanceof GridNoopSwapSpaceSpi);
+            assertTrue(spi instanceof NoopSwapSpaceSpi);
         }
         finally {
             stopAllGrids();

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c31cec7c/modules/core/src/test/java/org/gridgain/loadtests/swap/GridSwapEvictAllBenchmark.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/gridgain/loadtests/swap/GridSwapEvictAllBenchmark.java b/modules/core/src/test/java/org/gridgain/loadtests/swap/GridSwapEvictAllBenchmark.java
index fad2628..0f5a559 100644
--- a/modules/core/src/test/java/org/gridgain/loadtests/swap/GridSwapEvictAllBenchmark.java
+++ b/modules/core/src/test/java/org/gridgain/loadtests/swap/GridSwapEvictAllBenchmark.java
@@ -260,7 +260,7 @@ public class GridSwapEvictAllBenchmark {
         ccfg.setEvictionPolicy(new GridCacheFifoEvictionPolicy(EVICT_PLC_SIZE));
         ccfg.setStore(store);
 
-        GridFileSwapSpaceSpi swap = new GridFileSwapSpaceSpi();
+        FileSwapSpaceSpi swap = new FileSwapSpaceSpi();
 
 //        swap.setConcurrencyLevel(16);
 //        swap.setWriterThreadsCount(16);

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c31cec7c/modules/core/src/test/java/org/gridgain/testframework/GridSpiTestContext.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/gridgain/testframework/GridSpiTestContext.java b/modules/core/src/test/java/org/gridgain/testframework/GridSpiTestContext.java
index dfc1c7f..bc5384d 100644
--- a/modules/core/src/test/java/org/gridgain/testframework/GridSpiTestContext.java
+++ b/modules/core/src/test/java/org/gridgain/testframework/GridSpiTestContext.java
@@ -443,7 +443,7 @@ public class GridSpiTestContext implements IgniteSpiContext {
     }
 
     /** {@inheritDoc} */
-    @Override public <T> T readFromSwap(String spaceName, GridSwapKey key, @Nullable ClassLoader ldr)
+    @Override public <T> T readFromSwap(String spaceName, SwapKey key, @Nullable ClassLoader ldr)
         throws GridException {
         return null;
     }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c31cec7c/modules/indexing/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheAbstractQuerySelfTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheAbstractQuerySelfTest.java b/modules/indexing/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheAbstractQuerySelfTest.java
index ede7b64..4efc310 100644
--- a/modules/indexing/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheAbstractQuerySelfTest.java
+++ b/modules/indexing/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheAbstractQuerySelfTest.java
@@ -105,7 +105,7 @@ public abstract class GridCacheAbstractQuerySelfTest extends GridCommonAbstractT
         c.setIndexingSpi(indexing);
 
         // Otherwise noop swap space will be chosen on Windows.
-        c.setSwapSpaceSpi(new GridFileSwapSpaceSpi());
+        c.setSwapSpaceSpi(new FileSwapSpaceSpi());
 
         c.setMarshaller(new IgniteOptimizedMarshaller(false));
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c31cec7c/modules/indexing/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheOffHeapAndSwapSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheOffHeapAndSwapSelfTest.java b/modules/indexing/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheOffHeapAndSwapSelfTest.java
index 7e9cc9b..e5a72f2 100644
--- a/modules/indexing/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheOffHeapAndSwapSelfTest.java
+++ b/modules/indexing/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheOffHeapAndSwapSelfTest.java
@@ -111,7 +111,7 @@ public class GridCacheOffHeapAndSwapSelfTest extends GridCommonAbstractTest {
 
         cfg.setNetworkTimeout(2000);
 
-        cfg.setSwapSpaceSpi(new GridFileSwapSpaceSpi());
+        cfg.setSwapSpaceSpi(new FileSwapSpaceSpi());
 
         GridH2IndexingSpi indexingSpi = new GridH2IndexingSpi();
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c31cec7c/modules/indexing/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheQueryMultiThreadedSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheQueryMultiThreadedSelfTest.java b/modules/indexing/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheQueryMultiThreadedSelfTest.java
index a512a2f..8a7522f 100644
--- a/modules/indexing/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheQueryMultiThreadedSelfTest.java
+++ b/modules/indexing/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheQueryMultiThreadedSelfTest.java
@@ -74,7 +74,7 @@ public class GridCacheQueryMultiThreadedSelfTest extends GridCommonAbstractTest
 
         cfg.setDiscoverySpi(disco);
 
-        cfg.setSwapSpaceSpi(new GridFileSwapSpaceSpi());
+        cfg.setSwapSpaceSpi(new FileSwapSpaceSpi());
         cfg.setMarshaller(new IgniteOptimizedMarshaller(false));
 
         GridCacheConfiguration cacheCfg = defaultCacheConfiguration();

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c31cec7c/modules/indexing/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheSwapSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheSwapSelfTest.java b/modules/indexing/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheSwapSelfTest.java
index 14bf8ec..dfa298a 100644
--- a/modules/indexing/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheSwapSelfTest.java
+++ b/modules/indexing/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheSwapSelfTest.java
@@ -102,11 +102,11 @@ public class GridCacheSwapSelfTest extends GridCommonAbstractTest {
         try {
             Ignite ignite = startGrids(1);
 
-            GridSwapSpaceSpi swapSpi = ignite.configuration().getSwapSpaceSpi();
+            SwapSpaceSpi swapSpi = ignite.configuration().getSwapSpaceSpi();
 
             assertNotNull(swapSpi);
 
-            assertTrue(swapSpi instanceof GridNoopSwapSpaceSpi);
+            assertTrue(swapSpi instanceof NoopSwapSpaceSpi);
         }
         finally {
             stopAllGrids();
@@ -120,11 +120,11 @@ public class GridCacheSwapSelfTest extends GridCommonAbstractTest {
         try {
             Ignite ignite = startGrids(1);
 
-            GridSwapSpaceSpi swapSpi = ignite.configuration().getSwapSpaceSpi();
+            SwapSpaceSpi swapSpi = ignite.configuration().getSwapSpaceSpi();
 
             assertNotNull(swapSpi);
 
-            assertFalse(swapSpi instanceof GridNoopSwapSpaceSpi);
+            assertFalse(swapSpi instanceof NoopSwapSpaceSpi);
         }
         finally {
             stopAllGrids();

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c31cec7c/modules/indexing/src/test/java/org/gridgain/grid/kernal/processors/cache/GridIndexingWithNoopSwapSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/gridgain/grid/kernal/processors/cache/GridIndexingWithNoopSwapSelfTest.java b/modules/indexing/src/test/java/org/gridgain/grid/kernal/processors/cache/GridIndexingWithNoopSwapSelfTest.java
index 1b915bb..02355e5 100644
--- a/modules/indexing/src/test/java/org/gridgain/grid/kernal/processors/cache/GridIndexingWithNoopSwapSelfTest.java
+++ b/modules/indexing/src/test/java/org/gridgain/grid/kernal/processors/cache/GridIndexingWithNoopSwapSelfTest.java
@@ -54,7 +54,7 @@ public class GridIndexingWithNoopSwapSelfTest extends GridCommonAbstractTest {
 
         c.setIndexingSpi(indexing);
 
-        c.setSwapSpaceSpi(new GridNoopSwapSpaceSpi());
+        c.setSwapSpaceSpi(new NoopSwapSpaceSpi());
 
         GridCacheConfiguration cc = defaultCacheConfiguration();
 


[13/20] incubator-ignite git commit: # Renaming

Posted by sb...@apache.org.
# Renaming


Project: http://git-wip-us.apache.org/repos/asf/incubator-ignite/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-ignite/commit/c31cec7c
Tree: http://git-wip-us.apache.org/repos/asf/incubator-ignite/tree/c31cec7c
Diff: http://git-wip-us.apache.org/repos/asf/incubator-ignite/diff/c31cec7c

Branch: refs/heads/master
Commit: c31cec7cb3ec99b2c8818bae9bd413bd70a62095
Parents: f2f4a32
Author: sboikov <sb...@gridgain.com>
Authored: Fri Dec 5 17:13:27 2014 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Fri Dec 5 17:13:27 2014 +0300

----------------------------------------------------------------------
 .../configuration/IgniteConfiguration.java      |   13 +-
 .../org/apache/ignite/spi/IgniteSpiAdapter.java |    2 +-
 .../org/apache/ignite/spi/IgniteSpiContext.java |    2 +-
 .../org/gridgain/grid/kernal/GridGainEx.java    |    4 +-
 .../kernal/managers/GridManagerAdapter.java     |    2 +-
 .../swapspace/GridSwapSpaceManager.java         |   26 +-
 .../processors/cache/GridCacheSwapManager.java  |   24 +-
 .../spi/securesession/GridSecureSessionSpi.java |   85 -
 .../noop/GridNoopSecureSessionSpi.java          |  115 --
 .../noop/GridNoopSecureSessionSpiMBean.java     |   21 -
 .../grid/spi/swapspace/GridSwapContext.java     |   39 -
 .../grid/spi/swapspace/GridSwapKey.java         |  116 --
 .../grid/spi/swapspace/GridSwapSpaceSpi.java    |  204 --
 .../spi/swapspace/GridSwapSpaceSpiListener.java |   26 -
 .../grid/spi/swapspace/SwapContext.java         |   39 +
 .../gridgain/grid/spi/swapspace/SwapKey.java    |  116 ++
 .../grid/spi/swapspace/SwapSpaceSpi.java        |  202 ++
 .../spi/swapspace/SwapSpaceSpiListener.java     |   26 +
 .../grid/spi/swapspace/file/FileSwapArray.java  |  181 ++
 .../spi/swapspace/file/FileSwapSpaceSpi.java    | 1843 ++++++++++++++++++
 .../swapspace/file/FileSwapSpaceSpiMBean.java   |   59 +
 .../spi/swapspace/file/GridFileSwapArray.java   |  181 --
 .../swapspace/file/GridFileSwapSpaceSpi.java    | 1843 ------------------
 .../file/GridFileSwapSpaceSpiMBean.java         |   59 -
 .../swapspace/noop/GridNoopSwapSpaceSpi.java    |  126 --
 .../spi/swapspace/noop/NoopSwapSpaceSpi.java    |  126 ++
 .../core/src/test/config/spring-cache-swap.xml  |    2 +-
 .../managers/GridManagerStopSelfTest.java       |    2 +-
 .../swapspace/GridSwapSpaceManagerSelfTest.java |    6 +-
 .../cache/GridCacheMemoryModeSelfTest.java      |    2 +-
 .../cache/GridCacheOffHeapSelfTest.java         |    2 +-
 .../processors/cache/GridCacheOffHeapTest.java  |    2 +-
 .../cache/GridCacheP2PUndeploySelfTest.java     |    2 +-
 .../cache/GridCacheSwapReloadSelfTest.java      |    4 +-
 ...tractDistributedByteArrayValuesSelfTest.java |    2 +-
 ...tPartitionedOnlyByteArrayValuesSelfTest.java |    2 +-
 ...eColocatedOptimisticTransactionSelfTest.java |    2 +-
 ...idCacheReplicatedUnswapAdvancedSelfTest.java |    2 +-
 .../GridCacheLocalByteArrayValuesSelfTest.java  |    2 +-
 .../GridCacheSwapScanQueryAbstractSelfTest.java |    2 +-
 ...idFileSwapSpaceSpiMultithreadedLoadTest.java |   18 +-
 .../GridSwapSpaceSpiAbstractSelfTest.java       |  118 +-
 .../file/GridFileSwapCompactionSelfTest.java    |   30 +-
 .../file/GridFileSwapSpaceSpiSelfTest.java      |   36 +-
 .../inmemory/GridTestSwapSpaceSpi.java          |   74 +-
 .../noop/GridNoopSwapSpaceSpiSelfTest.java      |    6 +-
 .../swap/GridSwapEvictAllBenchmark.java         |    2 +-
 .../testframework/GridSpiTestContext.java       |    2 +-
 .../cache/GridCacheAbstractQuerySelfTest.java   |    2 +-
 .../cache/GridCacheOffHeapAndSwapSelfTest.java  |    2 +-
 .../GridCacheQueryMultiThreadedSelfTest.java    |    2 +-
 .../processors/cache/GridCacheSwapSelfTest.java |    8 +-
 .../cache/GridIndexingWithNoopSwapSelfTest.java |    2 +-
 53 files changed, 2796 insertions(+), 3020 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c31cec7c/modules/core/src/main/java/org/apache/ignite/configuration/IgniteConfiguration.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/configuration/IgniteConfiguration.java b/modules/core/src/main/java/org/apache/ignite/configuration/IgniteConfiguration.java
index e3dd068..dc2102d 100644
--- a/modules/core/src/main/java/org/apache/ignite/configuration/IgniteConfiguration.java
+++ b/modules/core/src/main/java/org/apache/ignite/configuration/IgniteConfiguration.java
@@ -38,7 +38,6 @@ import org.apache.ignite.spi.failover.*;
 import org.apache.ignite.spi.loadbalancing.*;
 import org.apache.ignite.spi.securesession.*;
 import org.gridgain.grid.spi.swapspace.*;
-import org.gridgain.grid.spi.swapspace.file.*;
 import org.gridgain.grid.util.typedef.internal.*;
 import org.jetbrains.annotations.*;
 
@@ -368,7 +367,7 @@ public class IgniteConfiguration {
     private LoadBalancingSpi[] loadBalancingSpi;
 
     /** Checkpoint SPI. */
-    private GridSwapSpaceSpi swapSpaceSpi;
+    private SwapSpaceSpi swapSpaceSpi;
 
     /** Indexing SPI. */
     private IndexingSpi[] indexingSpi;
@@ -2124,26 +2123,26 @@ public class IgniteConfiguration {
     }
 
     /**
-     * Sets fully configured instances of {@link GridSwapSpaceSpi}.
+     * Sets fully configured instances of {@link org.gridgain.grid.spi.swapspace.SwapSpaceSpi}.
      *
-     * @param swapSpaceSpi Fully configured instances of {@link GridSwapSpaceSpi} or
+     * @param swapSpaceSpi Fully configured instances of {@link org.gridgain.grid.spi.swapspace.SwapSpaceSpi} or
      *      <tt>null</tt> if no SPI provided.
      * @see IgniteConfiguration#getSwapSpaceSpi()
      */
-    public void setSwapSpaceSpi(GridSwapSpaceSpi swapSpaceSpi) {
+    public void setSwapSpaceSpi(SwapSpaceSpi swapSpaceSpi) {
         this.swapSpaceSpi = swapSpaceSpi;
     }
 
     /**
      * Should return fully configured swap space SPI implementation. If not provided,
-     * {@link GridFileSwapSpaceSpi} will be used.
+     * {@link org.gridgain.grid.spi.swapspace.file.FileSwapSpaceSpi} will be used.
      * <p>
      * Note that user can provide one or multiple instances of this SPI (and select later which one
      * is used in a particular context).
      *
      * @return Grid swap space SPI implementation or <tt>null</tt> to use default implementation.
      */
-    public GridSwapSpaceSpi getSwapSpaceSpi() {
+    public SwapSpaceSpi getSwapSpaceSpi() {
         return swapSpaceSpi;
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c31cec7c/modules/core/src/main/java/org/apache/ignite/spi/IgniteSpiAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/IgniteSpiAdapter.java b/modules/core/src/main/java/org/apache/ignite/spi/IgniteSpiAdapter.java
index 9969650..328ac82 100644
--- a/modules/core/src/main/java/org/apache/ignite/spi/IgniteSpiAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/spi/IgniteSpiAdapter.java
@@ -607,7 +607,7 @@ public abstract class IgniteSpiAdapter implements IgniteSpi, IgniteSpiManagement
         }
 
         /** {@inheritDoc} */
-        @Override public <T> T readFromSwap(String spaceName, GridSwapKey key, @Nullable ClassLoader ldr)
+        @Override public <T> T readFromSwap(String spaceName, SwapKey key, @Nullable ClassLoader ldr)
             throws GridException {
             return null;
         }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c31cec7c/modules/core/src/main/java/org/apache/ignite/spi/IgniteSpiContext.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/IgniteSpiContext.java b/modules/core/src/main/java/org/apache/ignite/spi/IgniteSpiContext.java
index 8c20c98..a4b2c8d 100644
--- a/modules/core/src/main/java/org/apache/ignite/spi/IgniteSpiContext.java
+++ b/modules/core/src/main/java/org/apache/ignite/spi/IgniteSpiContext.java
@@ -265,7 +265,7 @@ public interface IgniteSpiContext {
      * @return Swapped value.
      * @throws GridException If any exception occurs.
      */
-    @Nullable public <T> T readFromSwap(String spaceName, GridSwapKey key, @Nullable ClassLoader ldr)
+    @Nullable public <T> T readFromSwap(String spaceName, SwapKey key, @Nullable ClassLoader ldr)
         throws GridException;
 
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c31cec7c/modules/core/src/main/java/org/gridgain/grid/kernal/GridGainEx.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/kernal/GridGainEx.java b/modules/core/src/main/java/org/gridgain/grid/kernal/GridGainEx.java
index 2adda16..a9efc83 100644
--- a/modules/core/src/main/java/org/gridgain/grid/kernal/GridGainEx.java
+++ b/modules/core/src/main/java/org/gridgain/grid/kernal/GridGainEx.java
@@ -1473,7 +1473,7 @@ public class GridGainEx {
             CheckpointSpi[] cpSpi = cfg.getCheckpointSpi();
             FailoverSpi[] failSpi = cfg.getFailoverSpi();
             LoadBalancingSpi[] loadBalancingSpi = cfg.getLoadBalancingSpi();
-            GridSwapSpaceSpi swapspaceSpi = cfg.getSwapSpaceSpi();
+            SwapSpaceSpi swapspaceSpi = cfg.getSwapSpaceSpi();
             IndexingSpi[] indexingSpi = cfg.getIndexingSpi();
 
             execSvc = cfg.getExecutorService();
@@ -1718,7 +1718,7 @@ public class GridGainEx {
                     }
                 }
 
-                swapspaceSpi = needSwap ? new GridFileSwapSpaceSpi() : new GridNoopSwapSpaceSpi();
+                swapspaceSpi = needSwap ? new FileSwapSpaceSpi() : new NoopSwapSpaceSpi();
             }
 
             if (indexingSpi == null)

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c31cec7c/modules/core/src/main/java/org/gridgain/grid/kernal/managers/GridManagerAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/kernal/managers/GridManagerAdapter.java b/modules/core/src/main/java/org/gridgain/grid/kernal/managers/GridManagerAdapter.java
index 3c15e54..fd489ad 100644
--- a/modules/core/src/main/java/org/gridgain/grid/kernal/managers/GridManagerAdapter.java
+++ b/modules/core/src/main/java/org/gridgain/grid/kernal/managers/GridManagerAdapter.java
@@ -433,7 +433,7 @@ public abstract class GridManagerAdapter<T extends IgniteSpi> implements GridMan
                     }
 
                     @SuppressWarnings({"unchecked"})
-                    @Nullable @Override public <T> T readFromSwap(String spaceName, GridSwapKey key,
+                    @Nullable @Override public <T> T readFromSwap(String spaceName, SwapKey key,
                         @Nullable ClassLoader ldr) throws GridException {
                         assert ctx.swap().enabled();
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c31cec7c/modules/core/src/main/java/org/gridgain/grid/kernal/managers/swapspace/GridSwapSpaceManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/kernal/managers/swapspace/GridSwapSpaceManager.java b/modules/core/src/main/java/org/gridgain/grid/kernal/managers/swapspace/GridSwapSpaceManager.java
index a219fe2..e08935e 100644
--- a/modules/core/src/main/java/org/gridgain/grid/kernal/managers/swapspace/GridSwapSpaceManager.java
+++ b/modules/core/src/main/java/org/gridgain/grid/kernal/managers/swapspace/GridSwapSpaceManager.java
@@ -29,7 +29,7 @@ import static org.apache.ignite.events.IgniteEventType.*;
 /**
  *
  */
-public class GridSwapSpaceManager extends GridManagerAdapter<GridSwapSpaceSpi> {
+public class GridSwapSpaceManager extends GridManagerAdapter<SwapSpaceSpi> {
     /** */
     private IgniteMarshaller marsh;
 
@@ -45,7 +45,7 @@ public class GridSwapSpaceManager extends GridManagerAdapter<GridSwapSpaceSpi> {
         if (ctx.config().isDaemon())
             return;
 
-        getSpi().setListener(new GridSwapSpaceSpiListener() {
+        getSpi().setListener(new SwapSpaceSpiListener() {
             @Override public void onSwapEvent(int evtType, @Nullable String spaceName, @Nullable byte[] keyBytes) {
                 if (ctx.event().isRecordable(evtType)) {
                     String msg = null;
@@ -135,7 +135,7 @@ public class GridSwapSpaceManager extends GridManagerAdapter<GridSwapSpaceSpi> {
      * @return Value.
      * @throws GridException If failed.
      */
-    @Nullable public byte[] read(@Nullable String spaceName, GridSwapKey key, @Nullable ClassLoader ldr)
+    @Nullable public byte[] read(@Nullable String spaceName, SwapKey key, @Nullable ClassLoader ldr)
         throws GridException {
         assert key != null;
 
@@ -157,7 +157,7 @@ public class GridSwapSpaceManager extends GridManagerAdapter<GridSwapSpaceSpi> {
      * @throws GridException If failed.
      */
     @SuppressWarnings({"unchecked"})
-    @Nullable public <T> T readValue(@Nullable String spaceName, GridSwapKey key, @Nullable ClassLoader ldr)
+    @Nullable public <T> T readValue(@Nullable String spaceName, SwapKey key, @Nullable ClassLoader ldr)
         throws GridException {
         assert key != null;
 
@@ -173,7 +173,7 @@ public class GridSwapSpaceManager extends GridManagerAdapter<GridSwapSpaceSpi> {
      * @param ldr Class loader (optional).
      * @throws GridException If failed.
      */
-    public void write(@Nullable String spaceName, GridSwapKey key, byte[] val, @Nullable ClassLoader ldr)
+    public void write(@Nullable String spaceName, SwapKey key, byte[] val, @Nullable ClassLoader ldr)
         throws GridException {
         assert key != null;
         assert val != null;
@@ -195,7 +195,7 @@ public class GridSwapSpaceManager extends GridManagerAdapter<GridSwapSpaceSpi> {
      * @param ldr Class loader (optional).
      * @throws GridException If failed.
      */
-    public <K, V> void writeAll(String spaceName, Map<GridSwapKey, byte[]> batch,
+    public <K, V> void writeAll(String spaceName, Map<SwapKey, byte[]> batch,
         @Nullable ClassLoader ldr) throws GridException {
         getSpi().storeAll(spaceName, batch, context(ldr));
     }
@@ -213,7 +213,7 @@ public class GridSwapSpaceManager extends GridManagerAdapter<GridSwapSpaceSpi> {
         throws GridException {
         assert key != null;
 
-        write(spaceName, new GridSwapKey(key), marshal(val), ldr);
+        write(spaceName, new SwapKey(key), marshal(val), ldr);
     }
 
     /**
@@ -227,7 +227,7 @@ public class GridSwapSpaceManager extends GridManagerAdapter<GridSwapSpaceSpi> {
      * @param ldr Class loader (optional).
      * @throws GridException If failed.
      */
-    public void remove(@Nullable String spaceName, GridSwapKey key, @Nullable IgniteInClosure<byte[]> c,
+    public void remove(@Nullable String spaceName, SwapKey key, @Nullable IgniteInClosure<byte[]> c,
         @Nullable ClassLoader ldr) throws GridException {
         assert key != null;
 
@@ -250,8 +250,8 @@ public class GridSwapSpaceManager extends GridManagerAdapter<GridSwapSpaceSpi> {
      * @param ldr Class loader (optional).
      * @throws GridException If failed.
      */
-    public void removeAll(@Nullable String spaceName, Collection<GridSwapKey> keys,
-        IgniteBiInClosure<GridSwapKey, byte[]> c, @Nullable ClassLoader ldr) throws GridException {
+    public void removeAll(@Nullable String spaceName, Collection<SwapKey> keys,
+        IgniteBiInClosure<SwapKey, byte[]> c, @Nullable ClassLoader ldr) throws GridException {
         assert keys != null;
 
         try {
@@ -278,7 +278,7 @@ public class GridSwapSpaceManager extends GridManagerAdapter<GridSwapSpaceSpi> {
         @Nullable ClassLoader ldr) throws GridException {
         assert key != null;
 
-        remove(spaceName, new GridSwapKey(key), c, ldr);
+        remove(spaceName, new SwapKey(key), c, ldr);
     }
 
     /**
@@ -408,8 +408,8 @@ public class GridSwapSpaceManager extends GridManagerAdapter<GridSwapSpaceSpi> {
      * @param clsLdr Class loader.
      * @return Swap context.
      */
-    private GridSwapContext context(@Nullable ClassLoader clsLdr) {
-        GridSwapContext ctx = new GridSwapContext();
+    private SwapContext context(@Nullable ClassLoader clsLdr) {
+        SwapContext ctx = new SwapContext();
 
         ctx.classLoader(clsLdr != null ? clsLdr : U.gridClassLoader());
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c31cec7c/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/GridCacheSwapManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/GridCacheSwapManager.java b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/GridCacheSwapManager.java
index 6096f7e..ff7f0e6 100644
--- a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/GridCacheSwapManager.java
+++ b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/GridCacheSwapManager.java
@@ -403,7 +403,7 @@ public class GridCacheSwapManager<K, V> extends GridCacheManagerAdapter<K, V> {
         if (swapEnabled) {
             assert key != null;
 
-            byte[] valBytes = swapMgr.read(spaceName, new GridSwapKey(key, part, keyBytes),
+            byte[] valBytes = swapMgr.read(spaceName, new SwapKey(key, part, keyBytes),
                 cctx.deploy().globalLoader());
 
             return valBytes != null;
@@ -450,7 +450,7 @@ public class GridCacheSwapManager<K, V> extends GridCacheManagerAdapter<K, V> {
 
             assert key != null;
 
-            byte[] bytes = swapMgr.read(spaceName, new GridSwapKey(key, part, keyBytes), cctx.deploy().globalLoader());
+            byte[] bytes = swapMgr.read(spaceName, new SwapKey(key, part, keyBytes), cctx.deploy().globalLoader());
 
             if (bytes == null && lsnr != null)
                 return lsnr.entry;
@@ -534,7 +534,7 @@ public class GridCacheSwapManager<K, V> extends GridCacheManagerAdapter<K, V> {
         final GridTuple<GridCacheSwapEntry<V>> t = F.t1();
         final GridTuple<GridException> err = F.t1();
 
-        swapMgr.remove(spaceName, new GridSwapKey(key, part, keyBytes), new CI1<byte[]>() {
+        swapMgr.remove(spaceName, new SwapKey(key, part, keyBytes), new CI1<byte[]>() {
             @Override public void apply(byte[] rmv) {
                 if (rmv != null) {
                     try {
@@ -708,10 +708,10 @@ public class GridCacheSwapManager<K, V> extends GridCacheManagerAdapter<K, V> {
         // Swap is enabled.
         final GridTuple<GridException> err = F.t1();
 
-        Collection<GridSwapKey> converted = new ArrayList<>(F.viewReadOnly(keysList, new C1<K, GridSwapKey>() {
-            @Override public GridSwapKey apply(K key) {
+        Collection<SwapKey> converted = new ArrayList<>(F.viewReadOnly(keysList, new C1<K, SwapKey>() {
+            @Override public SwapKey apply(K key) {
                 try {
-                    return new GridSwapKey(key, cctx.affinity().partition(key), CU.marshal(cctx.shared(), key));
+                    return new SwapKey(key, cctx.affinity().partition(key), CU.marshal(cctx.shared(), key));
                 }
                 catch (GridException e) {
                     throw new GridRuntimeException(e);
@@ -721,8 +721,8 @@ public class GridCacheSwapManager<K, V> extends GridCacheManagerAdapter<K, V> {
 
         swapMgr.removeAll(spaceName,
             converted,
-            new IgniteBiInClosure<GridSwapKey, byte[]>() {
-                @Override public void apply(GridSwapKey swapKey, byte[] rmv) {
+            new IgniteBiInClosure<SwapKey, byte[]>() {
+                @Override public void apply(SwapKey swapKey, byte[] rmv) {
                     if (rmv != null) {
                         try {
                             GridCacheSwapEntry<V> entry = swapEntry(unmarshalSwapEntry(rmv));
@@ -873,7 +873,7 @@ public class GridCacheSwapManager<K, V> extends GridCacheManagerAdapter<K, V> {
         }
 
         if (swapEnabled)
-            swapMgr.remove(spaceName, new GridSwapKey(key, part, keyBytes), c,
+            swapMgr.remove(spaceName, new SwapKey(key, part, keyBytes), c,
                 cctx.deploy().globalLoader());
     }
 
@@ -957,10 +957,10 @@ public class GridCacheSwapManager<K, V> extends GridCacheManagerAdapter<K, V> {
             }
         }
         else {
-            Map<GridSwapKey, byte[]> batch = new LinkedHashMap<>();
+            Map<SwapKey, byte[]> batch = new LinkedHashMap<>();
 
             for (GridCacheBatchSwapEntry entry : swapped)
-                batch.put(new GridSwapKey(entry.key(), entry.partition(), entry.keyBytes()), entry.marshal());
+                batch.put(new SwapKey(entry.key(), entry.partition(), entry.keyBytes()), entry.marshal());
 
             swapMgr.writeAll(spaceName, batch, cctx.deploy().globalLoader());
 
@@ -991,7 +991,7 @@ public class GridCacheSwapManager<K, V> extends GridCacheManagerAdapter<K, V> {
         if (key == null)
             key = unmarshalKey(keyBytes, cctx.deploy().globalLoader());
 
-        swapMgr.write(spaceName, new GridSwapKey(key, part, keyBytes), entry, cctx.deploy().globalLoader());
+        swapMgr.write(spaceName, new SwapKey(key, part, keyBytes), entry, cctx.deploy().globalLoader());
 
         if (cctx.events().isRecordable(EVT_CACHE_OBJECT_SWAPPED))
             cctx.events().addEvent(part, key, cctx.nodeId(), (IgniteUuid)null, null,

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c31cec7c/modules/core/src/main/java/org/gridgain/grid/spi/securesession/GridSecureSessionSpi.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/spi/securesession/GridSecureSessionSpi.java b/modules/core/src/main/java/org/gridgain/grid/spi/securesession/GridSecureSessionSpi.java
deleted file mode 100644
index a87eaac..0000000
--- a/modules/core/src/main/java/org/gridgain/grid/spi/securesession/GridSecureSessionSpi.java
+++ /dev/null
@@ -1,85 +0,0 @@
-/* @java.file.header */
-
-/*  _________        _____ __________________        _____
- *  __  ____/___________(_)______  /__  ____/______ ____(_)_______
- *  _  / __  __  ___/__  / _  __  / _  / __  _  __ `/__  / __  __ \
- *  / /_/ /  _  /    _  /  / /_/ /  / /_/ /  / /_/ / _  /  _  / / /
- *  \____/   /_/     /_/   \_,__/   \____/   \__,_/  /_/   /_/ /_/
- */
-
-package org.gridgain.grid.spi.securesession;
-
-import org.apache.ignite.spi.*;
-import org.gridgain.grid.security.*;
-import org.gridgain.grid.spi.securesession.noop.*;
-import org.jetbrains.annotations.*;
-
-import java.util.*;
-
-/**
- * Secure session SPI allows for session creation and validation, typically after authentication
- * has successfully happened. The main purpose of this SPI is to ensure that remote clients are
- * authenticated only once and upon successful authentication get issued a secure session token
- * to reuse for consequent requests (very much the same way like HTTP sessions work).
- * <p>
- * The default secure session SPI is {@link GridNoopSecureSessionSpi}
- * which permits any request.
- * <p>
- * Gridgain provides the following {@code GridSecureSessionSpi} implementations:
- * <ul>
- * <li>
- *     {@link GridNoopSecureSessionSpi} - permits any request.
- * </li>
- * <li>
- *     {@code GridRememberMeSecureSessionSpi} -
- *     validates client session with remember-me session token.
- * </li>
- * </ul>
- * <p>
- * <b>NOTE:</b> that multiple secure session SPIs may be started on the same grid node. In this case
- * GridGain will differentiate between them based on {@link #supported(GridSecuritySubjectType)}
- * value. The first SPI which returns {@code true} for a given subject type will be used for
- * session validation.
- * <p>
- * <b>NOTE:</b> this SPI (i.e. methods in this interface) should never be used directly. SPIs provide
- * internal view on the subsystem and is used internally by GridGain kernal. In rare use cases when
- * access to a specific implementation of this SPI is required - an instance of this SPI can be obtained
- * via {@link org.apache.ignite.Ignite#configuration()} method to check its configuration properties or call other non-SPI
- * methods. Note again that calling methods from this interface on the obtained instance can lead
- * to undefined behavior and explicitly not supported.
- */
-public interface GridSecureSessionSpi extends IgniteSpi {
-    /**
-     * Checks if given subject is supported by this SPI. If not, then next secure session SPI
-     * in the list will be checked.
-     *
-     * @param subjType Subject type.
-     * @return {@code True} if subject type is supported, {@code false} otherwise.
-     */
-    public boolean supported(GridSecuritySubjectType subjType);
-
-    /**
-     * Validates given session token.
-     *
-     * @param subjType Subject type.
-     * @param subjId Unique subject ID such as local or remote node ID, client ID, etc.
-     * @param tok Token to validate.
-     * @param params Additional implementation-specific parameters.
-     * @return {@code True} if session token is valid, {@code false} otherwise.
-     * @throws org.apache.ignite.spi.IgniteSpiException If validation resulted in system error. Note that
-     *      bad credentials should not cause this exception.
-     */
-    public boolean validate(GridSecuritySubjectType subjType, UUID subjId, byte[] tok,
-        @Nullable Object params) throws IgniteSpiException;
-
-    /**
-     * Generates new session token.
-     *
-     * @param subjType Subject type.
-     * @param subjId Unique subject ID such as local or remote node ID, client ID, etc.
-     * @param params Additional implementation-specific parameters.
-     * @return Session token that should be used for further validation.
-     */
-    public byte[] generateSessionToken(GridSecuritySubjectType subjType, UUID subjId, @Nullable Object params)
-        throws IgniteSpiException;
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c31cec7c/modules/core/src/main/java/org/gridgain/grid/spi/securesession/noop/GridNoopSecureSessionSpi.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/spi/securesession/noop/GridNoopSecureSessionSpi.java b/modules/core/src/main/java/org/gridgain/grid/spi/securesession/noop/GridNoopSecureSessionSpi.java
deleted file mode 100644
index c52a011..0000000
--- a/modules/core/src/main/java/org/gridgain/grid/spi/securesession/noop/GridNoopSecureSessionSpi.java
+++ /dev/null
@@ -1,115 +0,0 @@
-/* @java.file.header */
-
-/*  _________        _____ __________________        _____
- *  __  ____/___________(_)______  /__  ____/______ ____(_)_______
- *  _  / __  __  ___/__  / _  __  / _  / __  _  __ `/__  / __  __ \
- *  / /_/ /  _  /    _  /  / /_/ /  / /_/ /  / /_/ / _  /  _  / / /
- *  \____/   /_/     /_/   \_,__/   \____/   \__,_/  /_/   /_/ /_/
- */
-
-package org.gridgain.grid.spi.securesession.noop;
-
-import org.apache.ignite.*;
-import org.apache.ignite.resources.*;
-import org.apache.ignite.spi.*;
-import org.gridgain.grid.security.*;
-import org.gridgain.grid.spi.securesession.*;
-import org.gridgain.grid.util.typedef.internal.*;
-import org.jetbrains.annotations.*;
-
-import java.util.*;
-
-/**
- * Default no-op implementation of the secure session SPI which supports all subject types and denies any token.
- * <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 no optional configuration parameters.
- * <h2 class="header">Java Example</h2>
- * GridNoopSecureSessionSpi is used by default and has no parameters to be explicitly configured.
- * <pre name="code" class="java">
- * GridNoopSecureSessionSpi spi = new GridNoopSecureSessionSpi();
- *
- * GridConfiguration cfg = new GridConfiguration();
- *
- * // Override default SecureSession SPI.
- * cfg.setSecureSessionSpi(spi);
- *
- * // Start grid.
- * GridGain.start(cfg);
- * </pre>
- * <h2 class="header">Spring Example</h2>
- * GridNoopSecureSessionSpi can be configured from Spring XML configuration file:
- * <pre name="code" class="xml">
- * &lt;bean id="grid.custom.cfg" class="org.gridgain.grid.GridConfiguration" singleton="true"&gt;
- *         ...
- *         &lt;property name="secureSessionSpi"&gt;
- *             &lt;bean class="org.gridgain.grid.spi.SecureSession.noop.GridNoopSecureSessionSpi"/&gt;
- *         &lt;/property&gt;
- *         ...
- * &lt;/bean&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 GridSecureSessionSpi
- */
-@IgniteSpiNoop
-@IgniteSpiMultipleInstancesSupport(true)
-public class GridNoopSecureSessionSpi extends IgniteSpiAdapter
-    implements GridSecureSessionSpi, GridNoopSecureSessionSpiMBean {
-    /** Empty bytes array. */
-    private static final byte[] EMPTY_BYTE_ARRAY = new byte[0];
-
-    /** Injected grid logger. */
-    @IgniteLoggerResource
-    private IgniteLogger log;
-
-    /** {@inheritDoc} */
-    @Override public boolean supported(GridSecuritySubjectType subjType) {
-        // If this SPI is included, then session management is disabled.
-        return true;
-    }
-
-    /** {@inheritDoc} */
-    @Override public boolean validate(GridSecuritySubjectType subjType, UUID subjId, @Nullable byte[] tok,
-        @Nullable Object params) throws IgniteSpiException {
-        // Never validate any token - all tokens are invalid.
-        return false;
-    }
-
-    /** {@inheritDoc} */
-    @Override public byte[] generateSessionToken(GridSecuritySubjectType subjType, UUID subjId,
-        @Nullable Object params) {
-        return EMPTY_BYTE_ARRAY;
-    }
-
-    /** {@inheritDoc} */
-    @Override public void spiStart(String gridName) throws IgniteSpiException {
-        // Start SPI start stopwatch.
-        startStopwatch();
-
-        registerMBean(gridName, this, GridNoopSecureSessionSpiMBean.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 String toString() {
-        return S.toString(GridNoopSecureSessionSpi.class, this);
-    }
-}

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

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c31cec7c/modules/core/src/main/java/org/gridgain/grid/spi/swapspace/GridSwapContext.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/spi/swapspace/GridSwapContext.java b/modules/core/src/main/java/org/gridgain/grid/spi/swapspace/GridSwapContext.java
deleted file mode 100644
index 7f24ea4..0000000
--- a/modules/core/src/main/java/org/gridgain/grid/spi/swapspace/GridSwapContext.java
+++ /dev/null
@@ -1,39 +0,0 @@
-/* @java.file.header */
-
-/*  _________        _____ __________________        _____
- *  __  ____/___________(_)______  /__  ____/______ ____(_)_______
- *  _  / __  __  ___/__  / _  __  / _  / __  _  __ `/__  / __  __ \
- *  / /_/ /  _  /    _  /  / /_/ /  / /_/ /  / /_/ / _  /  _  / / /
- *  \____/   /_/     /_/   \_,__/   \____/   \__,_/  /_/   /_/ /_/
- */
-
-package org.gridgain.grid.spi.swapspace;
-
-import org.gridgain.grid.util.typedef.internal.*;
-
-/**
- * Context for swap operations.
- */
-public class GridSwapContext {
-    /** */
-    private ClassLoader clsLdr;
-
-    /**
-     * @return Class loader.
-     */
-    public ClassLoader classLoader() {
-        return clsLdr;
-    }
-
-    /**
-     * @param clsLdr Class loader.
-     */
-    public void classLoader(ClassLoader clsLdr) {
-        this.clsLdr = clsLdr;
-    }
-
-    /** {@inheritDoc} */
-    @Override public String toString() {
-        return S.toString(GridSwapContext.class, this);
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c31cec7c/modules/core/src/main/java/org/gridgain/grid/spi/swapspace/GridSwapKey.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/spi/swapspace/GridSwapKey.java b/modules/core/src/main/java/org/gridgain/grid/spi/swapspace/GridSwapKey.java
deleted file mode 100644
index a7b8624..0000000
--- a/modules/core/src/main/java/org/gridgain/grid/spi/swapspace/GridSwapKey.java
+++ /dev/null
@@ -1,116 +0,0 @@
-/* @java.file.header */
-
-/*  _________        _____ __________________        _____
- *  __  ____/___________(_)______  /__  ____/______ ____(_)_______
- *  _  / __  __  ___/__  / _  __  / _  / __  _  __ `/__  / __  __ \
- *  / /_/ /  _  /    _  /  / /_/ /  / /_/ /  / /_/ / _  /  _  / / /
- *  \____/   /_/     /_/   \_,__/   \____/   \__,_/  /_/   /_/ /_/
- */
-
-package org.gridgain.grid.spi.swapspace;
-
-import org.gridgain.grid.util.typedef.internal.*;
-import org.gridgain.grid.util.tostring.*;
-import org.jetbrains.annotations.*;
-
-/**
- * Utility wrapper class that represents swap key.
- * <p>
- * This class also holds information about partition this key belongs to
- * (if needed for caches).
- */
-public class GridSwapKey {
-    /** */
-    @GridToStringInclude
-    private final Object key;
-
-    /** */
-    private final int part;
-
-    /** Serialized key. */
-    @GridToStringExclude
-    private byte[] keyBytes;
-
-    /**
-     * @param key Key.
-     */
-    public GridSwapKey(Object key) {
-        this(key, Integer.MAX_VALUE, null);
-    }
-
-    /**
-     * @param key Key.
-     * @param part Partition.
-     */
-    public GridSwapKey(Object key, int part) {
-        this(key, part, null);
-    }
-
-    /**
-     * @param key Key.
-     * @param part Part.
-     * @param keyBytes Key bytes.
-     */
-    public GridSwapKey(Object key, int part, @Nullable byte[] keyBytes) {
-        assert key != null;
-        assert part >= 0;
-
-        this.key = key;
-        this.part = part;
-        this.keyBytes = keyBytes;
-    }
-
-    /**
-     * @return Key.
-     */
-    public Object key() {
-        return key;
-    }
-
-    /**
-     * @return Partition this key belongs to.
-     */
-    public int partition() {
-        return part;
-    }
-
-    /**
-     * @return Serialized key.
-     */
-    @Nullable public byte[] keyBytes() {
-        return keyBytes;
-    }
-
-    /**
-     * @param keyBytes Serialized key.
-     */
-    public void keyBytes(byte[] keyBytes) {
-        assert keyBytes != null;
-
-        this.keyBytes = keyBytes;
-    }
-
-    /** {@inheritDoc} */
-    @Override public boolean equals(Object obj) {
-        if (obj == this)
-            return true;
-
-        if (obj instanceof GridSwapKey) {
-            GridSwapKey other = (GridSwapKey)obj;
-
-            return part == other.part && key.equals(other.key);
-        }
-
-        return false;
-    }
-
-    /** {@inheritDoc} */
-    @Override public int hashCode() {
-        return key.hashCode();
-    }
-
-    /** {@inheritDoc} */
-    @Override public String toString() {
-        return S.toString(GridSwapKey.class, this);
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c31cec7c/modules/core/src/main/java/org/gridgain/grid/spi/swapspace/GridSwapSpaceSpi.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/spi/swapspace/GridSwapSpaceSpi.java b/modules/core/src/main/java/org/gridgain/grid/spi/swapspace/GridSwapSpaceSpi.java
deleted file mode 100644
index a802097..0000000
--- a/modules/core/src/main/java/org/gridgain/grid/spi/swapspace/GridSwapSpaceSpi.java
+++ /dev/null
@@ -1,204 +0,0 @@
-/* @java.file.header */
-
-/*  _________        _____ __________________        _____
- *  __  ____/___________(_)______  /__  ____/______ ____(_)_______
- *  _  / __  __  ___/__  / _  __  / _  / __  _  __ `/__  / __  __ \
- *  / /_/ /  _  /    _  /  / /_/ /  / /_/ /  / /_/ / _  /  _  / / /
- *  \____/   /_/     /_/   \_,__/   \____/   \__,_/  /_/   /_/ /_/
- */
-
-package org.gridgain.grid.spi.swapspace;
-
-import org.apache.ignite.lang.*;
-import org.apache.ignite.spi.*;
-import org.gridgain.grid.spi.swapspace.file.*;
-import org.gridgain.grid.spi.swapspace.noop.*;
-import org.jetbrains.annotations.*;
-
-import java.util.*;
-
-/**
- * Provides a mechanism in grid for storing data on disk. GridGain cache uses swap space to overflow
- * data to disk if it cannot fit in memory. It's also possible to use swap space directly
- * by calling {@link org.apache.ignite.Ignite} API swap-related methods. Logically storage is organized into
- * independent 'spaces' in which data is stored.
- * <p>
- * All swap space implementations can be configured to prevent infinite growth and evict oldest entries.
- * <p>
- * The default swap space SPI is {@link GridFileSwapSpaceSpi} which stores values on disk in files and keeps keys in
- * memory.
- * <p>
- * Gridgain provides the following {@code GridSwapSpaceSpi} implementations:
- * <ul>
- * <li>
- *     {@link GridFileSwapSpaceSpi} - pure Java implementation with in-memory keys. This SPI is used by default.
- * </li>
- * <li>
- *     {@link GridNoopSwapSpaceSpi} - no-op SPI mainly for testing.
- * </li>
- * </ul>
- * <p>
- * <p>
- * <b>NOTE:</b> this SPI (i.e. methods in this interface) should never be used directly. SPIs provide
- * internal view on the subsystem and is used internally by GridGain kernal. In rare use cases when
- * access to a specific implementation of this SPI is required - an instance of this SPI can be obtained
- * via {@link org.apache.ignite.Ignite#configuration()} method to check its configuration properties or call other non-SPI
- * methods. Note again that calling methods from this interface on the obtained instance can lead
- * to undefined behavior and explicitly not supported.
- */
-public interface GridSwapSpaceSpi extends IgniteSpi {
-    /**
-     * Entirely clears data space with given name, if any.
-     *
-     * @param spaceName Space name to clear.
-     * @throws org.apache.ignite.spi.IgniteSpiException In case of any errors.
-     */
-    public void clear(@Nullable String spaceName) throws IgniteSpiException;
-
-    /**
-     * Gets size in bytes for data space with given name. If specified space does
-     * not exist this method returns {@code 0}.
-     *
-     * @param spaceName Space name to get size for.
-     * @return Size in bytes.
-     * @throws org.apache.ignite.spi.IgniteSpiException In case of any errors.
-     */
-    public long size(@Nullable String spaceName) throws IgniteSpiException;
-
-    /**
-     * Gets number of stored entries (keys) in data space with given name. If specified
-     * space does not exist this method returns {@code 0}.
-     *
-     * @param spaceName Space name to get number of entries for.
-     * @return Number of stored entries in specified space.
-     * @throws org.apache.ignite.spi.IgniteSpiException In case of any errors.
-     */
-    public long count(@Nullable String spaceName) throws IgniteSpiException;
-
-    /**
-     * Reads stored value as array of bytes by key from data space with given name.
-     * If specified space does not exist this method returns {@code null}.
-     *
-     * @param spaceName Name of the data space to read from.
-     * @param key Key used to read value from data space.
-     * @param ctx Swap context.
-     * @return Value as array of bytes stored in specified data space that matches
-     *      to given key.
-     * @throws org.apache.ignite.spi.IgniteSpiException In case of any errors.
-     */
-    @Nullable public byte[] read(@Nullable String spaceName, GridSwapKey key, GridSwapContext ctx)
-        throws IgniteSpiException;
-
-    /**
-     * Reads stored values as array of bytes by all passed keys from data space with
-     * given name. If specified space does not exist this method returns empty map.
-     *
-     * @param spaceName Name of the data space to read from.
-     * @param keys Keys used to read values from data space.
-     * @param ctx Swap context.
-     * @return Map in which keys are the ones passed into method and values are
-     *      corresponding values read from swap storage.
-     * @throws org.apache.ignite.spi.IgniteSpiException In case of any errors.
-     */
-    public Map<GridSwapKey, byte[]> readAll(@Nullable String spaceName,
-        Iterable<GridSwapKey> keys, GridSwapContext ctx) throws IgniteSpiException;
-
-    /**
-     * Removes value stored in data space with given name corresponding to specified key.
-     *
-     * @param spaceName Space name to remove value from.
-     * @param key Key to remove value in the specified space for.
-     * @param c Optional closure that takes removed value and executes after actual
-     *      removing. If there was no value in storage the closure is not executed.
-     * @param ctx Swap context.
-     * @throws org.apache.ignite.spi.IgniteSpiException In case of any errors.
-     */
-    public void remove(@Nullable String spaceName, GridSwapKey key,
-        @Nullable IgniteInClosure<byte[]> c, GridSwapContext ctx) throws IgniteSpiException;
-
-    /**
-     * Removes values stored in data space with given name corresponding to specified keys.
-     *
-     * @param spaceName Space name to remove values from.
-     * @param keys Keys to remove value in the specified space for.
-     * @param c Optional closure that takes removed value and executes after actual
-     *      removing. If there was no value in storage the closure is not executed.
-     * @param ctx Swap context.
-     * @throws org.apache.ignite.spi.IgniteSpiException In case of any errors.
-     */
-    public void removeAll(@Nullable String spaceName, Collection<GridSwapKey> keys,
-        @Nullable IgniteBiInClosure<GridSwapKey, byte[]> c, GridSwapContext ctx) throws IgniteSpiException;
-
-    /**
-     * Stores value as array of bytes with given key into data space with given name.
-     *
-     * @param spaceName Space name to store key-value pair into.
-     * @param key Key to store given value for. This key can be used further to
-     *      read or remove stored value.
-     * @param val Some value as array of bytes to store into specified data space.
-     * @param ctx Swap context.
-     * @throws org.apache.ignite.spi.IgniteSpiException In case of any errors.
-     */
-    public void store(@Nullable String spaceName, GridSwapKey key, @Nullable byte[] val, GridSwapContext ctx)
-        throws IgniteSpiException;
-
-    /**
-     * Stores key-value pairs (both keys and values are arrays of bytes) into data
-     * space with given name.
-     *
-     * @param spaceName Space name to store key-value pairs into.
-     * @param pairs Map of stored key-value pairs where each one is an array of bytes.
-     * @param ctx Swap context.
-     * @throws org.apache.ignite.spi.IgniteSpiException In case of any errors.
-     */
-    public void storeAll(@Nullable String spaceName, Map<GridSwapKey, byte[]> pairs, GridSwapContext ctx)
-        throws IgniteSpiException;
-
-    /**
-     * Sets eviction listener to receive notifications on evicted swap entries.
-     *
-     * @param evictLsnr Eviction listener ({@code null} to stop receiving notifications).
-     */
-    public void setListener(@Nullable GridSwapSpaceSpiListener evictLsnr);
-
-    /**
-     * Gets partitions IDs that are stored in the passed in space.
-     *
-     * @param spaceName Space name.
-     * @return Partitions IDs or {@code null} if space is unknown.
-     * @throws org.apache.ignite.spi.IgniteSpiException If failed.
-     */
-    @Nullable public Collection<Integer> partitions(@Nullable String spaceName) throws IgniteSpiException;
-
-    /**
-     * Gets iterator over space keys.
-     *
-     * @param spaceName Space name.
-     * @param ctx Swap context.
-     * @return Iterator over space entries or {@code null} if space is unknown.
-     * @throws org.apache.ignite.spi.IgniteSpiException If failed.
-     */
-    @Nullable <K> IgniteSpiCloseableIterator<K> keyIterator(@Nullable String spaceName, GridSwapContext ctx)
-        throws IgniteSpiException;
-
-    /**
-     * Gets raw iterator over space entries.
-     *
-     * @param spaceName Space name.
-     * @return Iterator over space entries or {@code null} if space is unknown.
-     * @throws org.apache.ignite.spi.IgniteSpiException If failed.
-     */
-    @Nullable public IgniteSpiCloseableIterator<Map.Entry<byte[], byte[]>> rawIterator(@Nullable String spaceName)
-        throws IgniteSpiException;
-
-    /**
-     * Gets raw iterator over space entries.
-     *
-     * @param spaceName Space name.
-     * @param part Partition.
-     * @return Iterator over space entries or {@code null} if space is unknown.
-     * @throws org.apache.ignite.spi.IgniteSpiException If failed.
-     */
-    @Nullable public IgniteSpiCloseableIterator<Map.Entry<byte[], byte[]>> rawIterator(@Nullable String spaceName,
-        int part) throws IgniteSpiException;
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c31cec7c/modules/core/src/main/java/org/gridgain/grid/spi/swapspace/GridSwapSpaceSpiListener.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/spi/swapspace/GridSwapSpaceSpiListener.java b/modules/core/src/main/java/org/gridgain/grid/spi/swapspace/GridSwapSpaceSpiListener.java
deleted file mode 100644
index 073d995..0000000
--- a/modules/core/src/main/java/org/gridgain/grid/spi/swapspace/GridSwapSpaceSpiListener.java
+++ /dev/null
@@ -1,26 +0,0 @@
-/* @java.file.header */
-
-/*  _________        _____ __________________        _____
- *  __  ____/___________(_)______  /__  ____/______ ____(_)_______
- *  _  / __  __  ___/__  / _  __  / _  / __  _  __ `/__  / __  __ \
- *  / /_/ /  _  /    _  /  / /_/ /  / /_/ /  / /_/ / _  /  _  / / /
- *  \____/   /_/     /_/   \_,__/   \____/   \__,_/  /_/   /_/ /_/
- */
-
-package org.gridgain.grid.spi.swapspace;
-
-import org.jetbrains.annotations.*;
-
-/**
- * Swap space SPI eviction listener.
- */
-public interface GridSwapSpaceSpiListener {
-    /**
-     * Notification for swap space events.
-     *
-     * @param evtType Event type. See {@link org.apache.ignite.events.IgniteSwapSpaceEvent}
-     * @param spaceName Space name for this event or {@code null} for default space.
-     * @param keyBytes Key bytes of affected entry. Not {@code null} only for evict notifications.
-     */
-    public void onSwapEvent(int evtType, @Nullable String spaceName, @Nullable byte[] keyBytes);
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c31cec7c/modules/core/src/main/java/org/gridgain/grid/spi/swapspace/SwapContext.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/spi/swapspace/SwapContext.java b/modules/core/src/main/java/org/gridgain/grid/spi/swapspace/SwapContext.java
new file mode 100644
index 0000000..c757651
--- /dev/null
+++ b/modules/core/src/main/java/org/gridgain/grid/spi/swapspace/SwapContext.java
@@ -0,0 +1,39 @@
+/* @java.file.header */
+
+/*  _________        _____ __________________        _____
+ *  __  ____/___________(_)______  /__  ____/______ ____(_)_______
+ *  _  / __  __  ___/__  / _  __  / _  / __  _  __ `/__  / __  __ \
+ *  / /_/ /  _  /    _  /  / /_/ /  / /_/ /  / /_/ / _  /  _  / / /
+ *  \____/   /_/     /_/   \_,__/   \____/   \__,_/  /_/   /_/ /_/
+ */
+
+package org.gridgain.grid.spi.swapspace;
+
+import org.gridgain.grid.util.typedef.internal.*;
+
+/**
+ * Context for swap operations.
+ */
+public class SwapContext {
+    /** */
+    private ClassLoader clsLdr;
+
+    /**
+     * @return Class loader.
+     */
+    public ClassLoader classLoader() {
+        return clsLdr;
+    }
+
+    /**
+     * @param clsLdr Class loader.
+     */
+    public void classLoader(ClassLoader clsLdr) {
+        this.clsLdr = clsLdr;
+    }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return S.toString(SwapContext.class, this);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c31cec7c/modules/core/src/main/java/org/gridgain/grid/spi/swapspace/SwapKey.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/spi/swapspace/SwapKey.java b/modules/core/src/main/java/org/gridgain/grid/spi/swapspace/SwapKey.java
new file mode 100644
index 0000000..1bce5f7
--- /dev/null
+++ b/modules/core/src/main/java/org/gridgain/grid/spi/swapspace/SwapKey.java
@@ -0,0 +1,116 @@
+/* @java.file.header */
+
+/*  _________        _____ __________________        _____
+ *  __  ____/___________(_)______  /__  ____/______ ____(_)_______
+ *  _  / __  __  ___/__  / _  __  / _  / __  _  __ `/__  / __  __ \
+ *  / /_/ /  _  /    _  /  / /_/ /  / /_/ /  / /_/ / _  /  _  / / /
+ *  \____/   /_/     /_/   \_,__/   \____/   \__,_/  /_/   /_/ /_/
+ */
+
+package org.gridgain.grid.spi.swapspace;
+
+import org.gridgain.grid.util.typedef.internal.*;
+import org.gridgain.grid.util.tostring.*;
+import org.jetbrains.annotations.*;
+
+/**
+ * Utility wrapper class that represents swap key.
+ * <p>
+ * This class also holds information about partition this key belongs to
+ * (if needed for caches).
+ */
+public class SwapKey {
+    /** */
+    @GridToStringInclude
+    private final Object key;
+
+    /** */
+    private final int part;
+
+    /** Serialized key. */
+    @GridToStringExclude
+    private byte[] keyBytes;
+
+    /**
+     * @param key Key.
+     */
+    public SwapKey(Object key) {
+        this(key, Integer.MAX_VALUE, null);
+    }
+
+    /**
+     * @param key Key.
+     * @param part Partition.
+     */
+    public SwapKey(Object key, int part) {
+        this(key, part, null);
+    }
+
+    /**
+     * @param key Key.
+     * @param part Part.
+     * @param keyBytes Key bytes.
+     */
+    public SwapKey(Object key, int part, @Nullable byte[] keyBytes) {
+        assert key != null;
+        assert part >= 0;
+
+        this.key = key;
+        this.part = part;
+        this.keyBytes = keyBytes;
+    }
+
+    /**
+     * @return Key.
+     */
+    public Object key() {
+        return key;
+    }
+
+    /**
+     * @return Partition this key belongs to.
+     */
+    public int partition() {
+        return part;
+    }
+
+    /**
+     * @return Serialized key.
+     */
+    @Nullable public byte[] keyBytes() {
+        return keyBytes;
+    }
+
+    /**
+     * @param keyBytes Serialized key.
+     */
+    public void keyBytes(byte[] keyBytes) {
+        assert keyBytes != null;
+
+        this.keyBytes = keyBytes;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean equals(Object obj) {
+        if (obj == this)
+            return true;
+
+        if (obj instanceof SwapKey) {
+            SwapKey other = (SwapKey)obj;
+
+            return part == other.part && key.equals(other.key);
+        }
+
+        return false;
+    }
+
+    /** {@inheritDoc} */
+    @Override public int hashCode() {
+        return key.hashCode();
+    }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return S.toString(SwapKey.class, this);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c31cec7c/modules/core/src/main/java/org/gridgain/grid/spi/swapspace/SwapSpaceSpi.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/spi/swapspace/SwapSpaceSpi.java b/modules/core/src/main/java/org/gridgain/grid/spi/swapspace/SwapSpaceSpi.java
new file mode 100644
index 0000000..1a07535
--- /dev/null
+++ b/modules/core/src/main/java/org/gridgain/grid/spi/swapspace/SwapSpaceSpi.java
@@ -0,0 +1,202 @@
+/* @java.file.header */
+
+/*  _________        _____ __________________        _____
+ *  __  ____/___________(_)______  /__  ____/______ ____(_)_______
+ *  _  / __  __  ___/__  / _  __  / _  / __  _  __ `/__  / __  __ \
+ *  / /_/ /  _  /    _  /  / /_/ /  / /_/ /  / /_/ / _  /  _  / / /
+ *  \____/   /_/     /_/   \_,__/   \____/   \__,_/  /_/   /_/ /_/
+ */
+
+package org.gridgain.grid.spi.swapspace;
+
+import org.apache.ignite.lang.*;
+import org.apache.ignite.spi.*;
+import org.jetbrains.annotations.*;
+
+import java.util.*;
+
+/**
+ * Provides a mechanism in grid for storing data on disk. GridGain cache uses swap space to overflow
+ * data to disk if it cannot fit in memory. It's also possible to use swap space directly
+ * by calling {@link org.apache.ignite.Ignite} API swap-related methods. Logically storage is organized into
+ * independent 'spaces' in which data is stored.
+ * <p>
+ * All swap space implementations can be configured to prevent infinite growth and evict oldest entries.
+ * <p>
+ * The default swap space SPI is {@link org.gridgain.grid.spi.swapspace.file.FileSwapSpaceSpi} which stores values on disk in files and keeps keys in
+ * memory.
+ * <p>
+ * Gridgain provides the following {@code GridSwapSpaceSpi} implementations:
+ * <ul>
+ * <li>
+ *     {@link org.gridgain.grid.spi.swapspace.file.FileSwapSpaceSpi} - pure Java implementation with in-memory keys. This SPI is used by default.
+ * </li>
+ * <li>
+ *     {@link org.gridgain.grid.spi.swapspace.noop.NoopSwapSpaceSpi} - no-op SPI mainly for testing.
+ * </li>
+ * </ul>
+ * <p>
+ * <p>
+ * <b>NOTE:</b> this SPI (i.e. methods in this interface) should never be used directly. SPIs provide
+ * internal view on the subsystem and is used internally by GridGain kernal. In rare use cases when
+ * access to a specific implementation of this SPI is required - an instance of this SPI can be obtained
+ * via {@link org.apache.ignite.Ignite#configuration()} method to check its configuration properties or call other non-SPI
+ * methods. Note again that calling methods from this interface on the obtained instance can lead
+ * to undefined behavior and explicitly not supported.
+ */
+public interface SwapSpaceSpi extends IgniteSpi {
+    /**
+     * Entirely clears data space with given name, if any.
+     *
+     * @param spaceName Space name to clear.
+     * @throws org.apache.ignite.spi.IgniteSpiException In case of any errors.
+     */
+    public void clear(@Nullable String spaceName) throws IgniteSpiException;
+
+    /**
+     * Gets size in bytes for data space with given name. If specified space does
+     * not exist this method returns {@code 0}.
+     *
+     * @param spaceName Space name to get size for.
+     * @return Size in bytes.
+     * @throws org.apache.ignite.spi.IgniteSpiException In case of any errors.
+     */
+    public long size(@Nullable String spaceName) throws IgniteSpiException;
+
+    /**
+     * Gets number of stored entries (keys) in data space with given name. If specified
+     * space does not exist this method returns {@code 0}.
+     *
+     * @param spaceName Space name to get number of entries for.
+     * @return Number of stored entries in specified space.
+     * @throws org.apache.ignite.spi.IgniteSpiException In case of any errors.
+     */
+    public long count(@Nullable String spaceName) throws IgniteSpiException;
+
+    /**
+     * Reads stored value as array of bytes by key from data space with given name.
+     * If specified space does not exist this method returns {@code null}.
+     *
+     * @param spaceName Name of the data space to read from.
+     * @param key Key used to read value from data space.
+     * @param ctx Swap context.
+     * @return Value as array of bytes stored in specified data space that matches
+     *      to given key.
+     * @throws org.apache.ignite.spi.IgniteSpiException In case of any errors.
+     */
+    @Nullable public byte[] read(@Nullable String spaceName, SwapKey key, SwapContext ctx)
+        throws IgniteSpiException;
+
+    /**
+     * Reads stored values as array of bytes by all passed keys from data space with
+     * given name. If specified space does not exist this method returns empty map.
+     *
+     * @param spaceName Name of the data space to read from.
+     * @param keys Keys used to read values from data space.
+     * @param ctx Swap context.
+     * @return Map in which keys are the ones passed into method and values are
+     *      corresponding values read from swap storage.
+     * @throws org.apache.ignite.spi.IgniteSpiException In case of any errors.
+     */
+    public Map<SwapKey, byte[]> readAll(@Nullable String spaceName,
+        Iterable<SwapKey> keys, SwapContext ctx) throws IgniteSpiException;
+
+    /**
+     * Removes value stored in data space with given name corresponding to specified key.
+     *
+     * @param spaceName Space name to remove value from.
+     * @param key Key to remove value in the specified space for.
+     * @param c Optional closure that takes removed value and executes after actual
+     *      removing. If there was no value in storage the closure is not executed.
+     * @param ctx Swap context.
+     * @throws org.apache.ignite.spi.IgniteSpiException In case of any errors.
+     */
+    public void remove(@Nullable String spaceName, SwapKey key,
+        @Nullable IgniteInClosure<byte[]> c, SwapContext ctx) throws IgniteSpiException;
+
+    /**
+     * Removes values stored in data space with given name corresponding to specified keys.
+     *
+     * @param spaceName Space name to remove values from.
+     * @param keys Keys to remove value in the specified space for.
+     * @param c Optional closure that takes removed value and executes after actual
+     *      removing. If there was no value in storage the closure is not executed.
+     * @param ctx Swap context.
+     * @throws org.apache.ignite.spi.IgniteSpiException In case of any errors.
+     */
+    public void removeAll(@Nullable String spaceName, Collection<SwapKey> keys,
+        @Nullable IgniteBiInClosure<SwapKey, byte[]> c, SwapContext ctx) throws IgniteSpiException;
+
+    /**
+     * Stores value as array of bytes with given key into data space with given name.
+     *
+     * @param spaceName Space name to store key-value pair into.
+     * @param key Key to store given value for. This key can be used further to
+     *      read or remove stored value.
+     * @param val Some value as array of bytes to store into specified data space.
+     * @param ctx Swap context.
+     * @throws org.apache.ignite.spi.IgniteSpiException In case of any errors.
+     */
+    public void store(@Nullable String spaceName, SwapKey key, @Nullable byte[] val, SwapContext ctx)
+        throws IgniteSpiException;
+
+    /**
+     * Stores key-value pairs (both keys and values are arrays of bytes) into data
+     * space with given name.
+     *
+     * @param spaceName Space name to store key-value pairs into.
+     * @param pairs Map of stored key-value pairs where each one is an array of bytes.
+     * @param ctx Swap context.
+     * @throws org.apache.ignite.spi.IgniteSpiException In case of any errors.
+     */
+    public void storeAll(@Nullable String spaceName, Map<SwapKey, byte[]> pairs, SwapContext ctx)
+        throws IgniteSpiException;
+
+    /**
+     * Sets eviction listener to receive notifications on evicted swap entries.
+     *
+     * @param evictLsnr Eviction listener ({@code null} to stop receiving notifications).
+     */
+    public void setListener(@Nullable SwapSpaceSpiListener evictLsnr);
+
+    /**
+     * Gets partitions IDs that are stored in the passed in space.
+     *
+     * @param spaceName Space name.
+     * @return Partitions IDs or {@code null} if space is unknown.
+     * @throws org.apache.ignite.spi.IgniteSpiException If failed.
+     */
+    @Nullable public Collection<Integer> partitions(@Nullable String spaceName) throws IgniteSpiException;
+
+    /**
+     * Gets iterator over space keys.
+     *
+     * @param spaceName Space name.
+     * @param ctx Swap context.
+     * @return Iterator over space entries or {@code null} if space is unknown.
+     * @throws org.apache.ignite.spi.IgniteSpiException If failed.
+     */
+    @Nullable <K> IgniteSpiCloseableIterator<K> keyIterator(@Nullable String spaceName, SwapContext ctx)
+        throws IgniteSpiException;
+
+    /**
+     * Gets raw iterator over space entries.
+     *
+     * @param spaceName Space name.
+     * @return Iterator over space entries or {@code null} if space is unknown.
+     * @throws org.apache.ignite.spi.IgniteSpiException If failed.
+     */
+    @Nullable public IgniteSpiCloseableIterator<Map.Entry<byte[], byte[]>> rawIterator(@Nullable String spaceName)
+        throws IgniteSpiException;
+
+    /**
+     * Gets raw iterator over space entries.
+     *
+     * @param spaceName Space name.
+     * @param part Partition.
+     * @return Iterator over space entries or {@code null} if space is unknown.
+     * @throws org.apache.ignite.spi.IgniteSpiException If failed.
+     */
+    @Nullable public IgniteSpiCloseableIterator<Map.Entry<byte[], byte[]>> rawIterator(@Nullable String spaceName,
+        int part) throws IgniteSpiException;
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c31cec7c/modules/core/src/main/java/org/gridgain/grid/spi/swapspace/SwapSpaceSpiListener.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/spi/swapspace/SwapSpaceSpiListener.java b/modules/core/src/main/java/org/gridgain/grid/spi/swapspace/SwapSpaceSpiListener.java
new file mode 100644
index 0000000..da8639d
--- /dev/null
+++ b/modules/core/src/main/java/org/gridgain/grid/spi/swapspace/SwapSpaceSpiListener.java
@@ -0,0 +1,26 @@
+/* @java.file.header */
+
+/*  _________        _____ __________________        _____
+ *  __  ____/___________(_)______  /__  ____/______ ____(_)_______
+ *  _  / __  __  ___/__  / _  __  / _  / __  _  __ `/__  / __  __ \
+ *  / /_/ /  _  /    _  /  / /_/ /  / /_/ /  / /_/ / _  /  _  / / /
+ *  \____/   /_/     /_/   \_,__/   \____/   \__,_/  /_/   /_/ /_/
+ */
+
+package org.gridgain.grid.spi.swapspace;
+
+import org.jetbrains.annotations.*;
+
+/**
+ * Swap space SPI eviction listener.
+ */
+public interface SwapSpaceSpiListener {
+    /**
+     * Notification for swap space events.
+     *
+     * @param evtType Event type. See {@link org.apache.ignite.events.IgniteSwapSpaceEvent}
+     * @param spaceName Space name for this event or {@code null} for default space.
+     * @param keyBytes Key bytes of affected entry. Not {@code null} only for evict notifications.
+     */
+    public void onSwapEvent(int evtType, @Nullable String spaceName, @Nullable byte[] keyBytes);
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c31cec7c/modules/core/src/main/java/org/gridgain/grid/spi/swapspace/file/FileSwapArray.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/spi/swapspace/file/FileSwapArray.java b/modules/core/src/main/java/org/gridgain/grid/spi/swapspace/file/FileSwapArray.java
new file mode 100644
index 0000000..e4a4f28
--- /dev/null
+++ b/modules/core/src/main/java/org/gridgain/grid/spi/swapspace/file/FileSwapArray.java
@@ -0,0 +1,181 @@
+/* @java.file.header */
+
+/*  _________        _____ __________________        _____
+ *  __  ____/___________(_)______  /__  ____/______ ____(_)_______
+ *  _  / __  __  ___/__  / _  __  / _  / __  _  __ `/__  / __  __ \
+ *  / /_/ /  _  /    _  /  / /_/ /  / /_/ /  / /_/ / _  /  _  / / /
+ *  \____/   /_/     /_/   \_,__/   \____/   \__,_/  /_/   /_/ /_/
+ */
+
+package org.gridgain.grid.spi.swapspace.file;
+
+import org.jetbrains.annotations.*;
+
+import java.util.concurrent.atomic.*;
+
+/**
+ * Growing array.
+ */
+class FileSwapArray<X> {
+    /** First partition size must be power of two. */
+    private static final int FIRST_ARRAY_SIZE = 4096;
+
+    /** */
+    private static final int LADDER_SIZE = Integer.numberOfLeadingZeros(FIRST_ARRAY_SIZE) + 1;
+
+    /** */
+    @SuppressWarnings("unchecked")
+    private final AtomicReferenceArray<X>[] ladder = new AtomicReferenceArray[LADDER_SIZE];
+
+    /** */
+    private int idx = 1;
+
+    /**
+     *
+     */
+    FileSwapArray() {
+        synchronized (ladder) {
+            ladder[0] = new AtomicReferenceArray<>(FIRST_ARRAY_SIZE);
+        }
+    }
+
+    /**
+     * @return Size.
+     */
+    public int size() {
+        return idx;
+    }
+
+    /**
+     * Adds value to the end.
+     *
+     * @param x Value.
+     * @return Index where it was added.
+     */
+    int add(X x) {
+        int i = idx++;
+
+        assert i >= 0 && i != Integer.MAX_VALUE : "Integer overflow";
+
+        Slot<X> s = slot(i);
+
+        assert s != null; // We should add always in one thread.
+
+        s.set(x);
+
+        int len = s.arr.length();
+
+        if (s.idx + 1 == len) {
+            synchronized (ladder) {
+                ladder[s.arrIdx + 1] = new AtomicReferenceArray<>(s.arrIdx == 0 ? len : len << 1);
+            }
+        }
+
+        return i;
+    }
+
+    /**
+     * @param size New size.
+     */
+    void truncate(int size) {
+        assert size > 0;
+
+        idx = size;
+
+        int arrIdx = arrayIndex(idx) + 1;
+
+        if (arrIdx < ladder.length && ladder[arrIdx] != null) {
+            synchronized (ladder) {
+                do {
+                    ladder[arrIdx++] = null;
+                }
+                while (arrIdx < ladder.length && ladder[arrIdx] != null);
+            }
+        }
+    }
+
+    /**
+     * @param idx Absolute slot index.
+     * @return Array index in {@link #ladder}.
+     */
+    static int arrayIndex(int idx) {
+        if (idx < FIRST_ARRAY_SIZE)
+            return 0;
+
+        return LADDER_SIZE - Integer.numberOfLeadingZeros(idx);
+    }
+
+    /**
+     * Slot for given absolute index.
+     *
+     * @param idx Absolute index.
+     * @return Slot.
+     */
+    @Nullable Slot<X> slot(int idx) {
+        assert idx > 0 : idx;
+
+        int arrIdx = arrayIndex(idx);
+
+        AtomicReferenceArray<X> arr = ladder[arrIdx];
+
+        if (arr == null) {
+            synchronized (ladder) { // Ensure visibility.
+                arr = ladder[arrIdx];
+            }
+
+            if (arr == null)
+                return null;
+        }
+
+        return new Slot<>(arrIdx, arr, arrIdx == 0 ? idx : idx - arr.length());
+    }
+
+    /**
+     * Slot in array.
+     */
+    @SuppressWarnings("PublicInnerClass")
+    static final class Slot<X> {
+        /** */
+        private final int arrIdx;
+
+        /** */
+        private final AtomicReferenceArray<X> arr;
+
+        /** */
+        private final int idx;
+
+        /**
+         * @param arrIdx Index of array.
+         * @param arr Array.
+         * @param idx Index within the array.
+         */
+        private Slot(int arrIdx, AtomicReferenceArray<X> arr, int idx) {
+            this.arrIdx = arrIdx;
+            this.arr = arr;
+            this.idx = idx;
+        }
+
+        /**
+         * @return Value.
+         */
+        public X get() {
+            return arr.get(idx);
+        }
+
+        /**
+         * @param exp Expected.
+         * @param x New value.
+         * @return {@code true} If succeeded.
+         */
+        public boolean cas(@Nullable X exp, @Nullable X x) {
+            return exp == x || arr.compareAndSet(idx, exp, x);
+        }
+
+        /**
+         * @param x value.
+         */
+        private void set(X x) {
+            arr.lazySet(idx, x);
+        }
+    }
+}


[17/20] incubator-ignite git commit: # Renaming

Posted by sb...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/ef258ece/modules/core/src/main/java/org/apache/ignite/spi/swapspace/file/package.html
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/swapspace/file/package.html b/modules/core/src/main/java/org/apache/ignite/spi/swapspace/file/package.html
new file mode 100644
index 0000000..cc92a66
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/spi/swapspace/file/package.html
@@ -0,0 +1,15 @@
+<!DOCTYPE html PUBLIC "-//W3C//DTD HTML 4.01 Transitional//EN" "http://www.w3.org/TR/html4/loose.dtd">
+<!--
+    @html.file.header
+    _________        _____ __________________        _____
+    __  ____/___________(_)______  /__  ____/______ ____(_)_______
+    _  / __  __  ___/__  / _  __  / _  / __  _  __ `/__  / __  __ \
+    / /_/ /  _  /    _  /  / /_/ /  / /_/ /  / /_/ / _  /  _  / / /
+    \____/   /_/     /_/   \_,__/   \____/   \__,_/  /_/   /_/ /_/
+-->
+<html>
+<body>
+    <!-- Package description. -->
+    Contains file-based swap space SPI.
+</body>
+</html>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/ef258ece/modules/core/src/main/java/org/apache/ignite/spi/swapspace/noop/NoopSwapSpaceSpi.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/swapspace/noop/NoopSwapSpaceSpi.java b/modules/core/src/main/java/org/apache/ignite/spi/swapspace/noop/NoopSwapSpaceSpi.java
new file mode 100644
index 0000000..b8a93c3
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/spi/swapspace/noop/NoopSwapSpaceSpi.java
@@ -0,0 +1,126 @@
+/* @java.file.header */
+
+/*  _________        _____ __________________        _____
+ *  __  ____/___________(_)______  /__  ____/______ ____(_)_______
+ *  _  / __  __  ___/__  / _  __  / _  / __  _  __ `/__  / __  __ \
+ *  / /_/ /  _  /    _  /  / /_/ /  / /_/ /  / /_/ / _  /  _  / / /
+ *  \____/   /_/     /_/   \_,__/   \____/   \__,_/  /_/   /_/ /_/
+ */
+
+package org.apache.ignite.spi.swapspace.noop;
+
+import org.apache.ignite.*;
+import org.apache.ignite.lang.*;
+import org.apache.ignite.resources.*;
+import org.apache.ignite.spi.*;
+import org.apache.ignite.spi.swapspace.*;
+import org.gridgain.grid.util.*;
+import org.gridgain.grid.util.typedef.internal.*;
+import org.jetbrains.annotations.*;
+
+import java.util.*;
+
+/**
+ * No-op implementation of {@link org.apache.ignite.spi.swapspace.SwapSpaceSpi}. Exists for testing and benchmarking purposes.
+ */
+@IgniteSpiNoop
+@IgniteSpiMultipleInstancesSupport(true)
+public class NoopSwapSpaceSpi extends IgniteSpiAdapter implements SwapSpaceSpi {
+    /** Logger. */
+    @IgniteLoggerResource
+    private IgniteLogger log;
+
+    /** {@inheritDoc} */
+    @Override public void spiStart(@Nullable String gridName) throws IgniteSpiException {
+        U.warn(log, "Swap space is disabled. To enable use GridFileSwapSpaceSpi.");
+    }
+
+    /** {@inheritDoc} */
+    @Override public void spiStop() throws IgniteSpiException {
+        // No-op.
+    }
+
+    /** {@inheritDoc} */
+    @Override public void clear(@Nullable String space) throws IgniteSpiException {
+        // No-op.
+    }
+
+    /** {@inheritDoc} */
+    @Override public long size(@Nullable String space) throws IgniteSpiException {
+        return 0;
+    }
+
+    /** {@inheritDoc} */
+    @Override public long count(@Nullable String space) throws IgniteSpiException {
+        return 0;
+    }
+
+    /** {@inheritDoc} */
+    @Override @Nullable public byte[] read(@Nullable String spaceName, SwapKey key, SwapContext ctx)
+        throws IgniteSpiException {
+        return null;
+    }
+
+    /** {@inheritDoc} */
+    @Override public Map<SwapKey, byte[]> readAll(@Nullable String spaceName, Iterable<SwapKey> keys,
+        SwapContext ctx) throws IgniteSpiException {
+        return Collections.emptyMap();
+    }
+
+    /** {@inheritDoc} */
+    @Override public void remove(@Nullable String spaceName, SwapKey key, @Nullable IgniteInClosure<byte[]> c,
+        SwapContext ctx) throws IgniteSpiException {
+        // No-op.
+    }
+
+    /** {@inheritDoc} */
+    @Override public void removeAll(@Nullable String spaceName, Collection<SwapKey> keys,
+        @Nullable IgniteBiInClosure<SwapKey, byte[]> c, SwapContext ctx) throws IgniteSpiException {
+        // No-op.
+    }
+
+    /** {@inheritDoc} */
+    @Override public void store(@Nullable String spaceName, SwapKey key, @Nullable byte[] val,
+        SwapContext ctx) throws IgniteSpiException {
+        // No-op.
+    }
+
+    /** {@inheritDoc} */
+    @Override public void storeAll(@Nullable String spaceName, Map<SwapKey, byte[]> pairs,
+        SwapContext ctx) throws IgniteSpiException {
+        // No-op.
+    }
+
+    /** {@inheritDoc} */
+    @Override public void setListener(@Nullable SwapSpaceSpiListener evictLsnr) {
+        // No-op.
+    }
+
+    /** {@inheritDoc} */
+    @Override public Collection<Integer> partitions(@Nullable String spaceName) throws IgniteSpiException {
+        return Collections.emptyList();
+    }
+
+    /** {@inheritDoc} */
+    @Override public <K> IgniteSpiCloseableIterator<K> keyIterator(@Nullable String spaceName,
+        SwapContext ctx) throws IgniteSpiException {
+        return new GridEmptyCloseableIterator<>();
+    }
+
+    /** {@inheritDoc} */
+    @Override public IgniteSpiCloseableIterator<Map.Entry<byte[], byte[]>> rawIterator(
+        @Nullable String spaceName) throws IgniteSpiException {
+        return new GridEmptyCloseableIterator<>();
+    }
+
+    /** {@inheritDoc} */
+    @Override public IgniteSpiCloseableIterator<Map.Entry<byte[], byte[]>> rawIterator(@Nullable String spaceName,
+        int part) throws IgniteSpiException {
+        return new GridEmptyCloseableIterator<>();
+    }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return S.toString(NoopSwapSpaceSpi.class, this);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/ef258ece/modules/core/src/main/java/org/apache/ignite/spi/swapspace/noop/package.html
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/swapspace/noop/package.html b/modules/core/src/main/java/org/apache/ignite/spi/swapspace/noop/package.html
new file mode 100644
index 0000000..d003e95
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/spi/swapspace/noop/package.html
@@ -0,0 +1,15 @@
+<!DOCTYPE html PUBLIC "-//W3C//DTD HTML 4.01 Transitional//EN" "http://www.w3.org/TR/html4/loose.dtd">
+<!--
+    @html.file.header
+    _________        _____ __________________        _____
+    __  ____/___________(_)______  /__  ____/______ ____(_)_______
+    _  / __  __  ___/__  / _  __  / _  / __  _  __ `/__  / __  __ \
+    / /_/ /  _  /    _  /  / /_/ /  / /_/ /  / /_/ / _  /  _  / / /
+    \____/   /_/     /_/   \_,__/   \____/   \__,_/  /_/   /_/ /_/
+-->
+<html>
+<body>
+    <!-- Package description. -->
+    Contains <b>default</b> no-op swap space SPI implementation.
+</body>
+</html>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/ef258ece/modules/core/src/main/java/org/apache/ignite/spi/swapspace/package.html
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/swapspace/package.html b/modules/core/src/main/java/org/apache/ignite/spi/swapspace/package.html
new file mode 100644
index 0000000..2106182
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/spi/swapspace/package.html
@@ -0,0 +1,15 @@
+<!DOCTYPE html PUBLIC "-//W3C//DTD HTML 4.01 Transitional//EN" "http://www.w3.org/TR/html4/loose.dtd">
+<!--
+    @html.file.header
+    _________        _____ __________________        _____
+    __  ____/___________(_)______  /__  ____/______ ____(_)_______
+    _  / __  __  ___/__  / _  __  / _  / __  _  __ `/__  / __  __ \
+    / /_/ /  _  /    _  /  / /_/ /  / /_/ /  / /_/ / _  /  _  / / /
+    \____/   /_/     /_/   \_,__/   \____/   \__,_/  /_/   /_/ /_/
+-->
+<html>
+<body>
+    <!-- Package description. -->
+    Contains APIs for swap space SPI.
+</body>
+</html>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/ef258ece/modules/core/src/main/java/org/gridgain/grid/kernal/GridGainEx.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/kernal/GridGainEx.java b/modules/core/src/main/java/org/gridgain/grid/kernal/GridGainEx.java
index a9efc83..1401e06 100644
--- a/modules/core/src/main/java/org/gridgain/grid/kernal/GridGainEx.java
+++ b/modules/core/src/main/java/org/gridgain/grid/kernal/GridGainEx.java
@@ -52,9 +52,9 @@ import org.apache.ignite.spi.loadbalancing.*;
 import org.apache.ignite.spi.loadbalancing.roundrobin.*;
 import org.apache.ignite.spi.securesession.*;
 import org.apache.ignite.spi.securesession.noop.*;
-import org.gridgain.grid.spi.swapspace.*;
-import org.gridgain.grid.spi.swapspace.file.*;
-import org.gridgain.grid.spi.swapspace.noop.*;
+import org.apache.ignite.spi.swapspace.*;
+import org.apache.ignite.spi.swapspace.file.*;
+import org.apache.ignite.spi.swapspace.noop.*;
 import org.gridgain.grid.util.*;
 import org.gridgain.grid.util.typedef.*;
 import org.gridgain.grid.util.typedef.internal.*;

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/ef258ece/modules/core/src/main/java/org/gridgain/grid/kernal/managers/GridManagerAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/kernal/managers/GridManagerAdapter.java b/modules/core/src/main/java/org/gridgain/grid/kernal/managers/GridManagerAdapter.java
index fd489ad..152952b 100644
--- a/modules/core/src/main/java/org/gridgain/grid/kernal/managers/GridManagerAdapter.java
+++ b/modules/core/src/main/java/org/gridgain/grid/kernal/managers/GridManagerAdapter.java
@@ -22,7 +22,7 @@ import org.gridgain.grid.kernal.managers.communication.*;
 import org.gridgain.grid.kernal.managers.eventstorage.*;
 import org.gridgain.grid.kernal.processors.cache.*;
 import org.gridgain.grid.security.*;
-import org.gridgain.grid.spi.swapspace.*;
+import org.apache.ignite.spi.swapspace.*;
 import org.gridgain.grid.util.direct.*;
 import org.gridgain.grid.util.tostring.*;
 import org.gridgain.grid.util.typedef.*;

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/ef258ece/modules/core/src/main/java/org/gridgain/grid/kernal/managers/swapspace/GridSwapSpaceManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/kernal/managers/swapspace/GridSwapSpaceManager.java b/modules/core/src/main/java/org/gridgain/grid/kernal/managers/swapspace/GridSwapSpaceManager.java
index e08935e..93523b9 100644
--- a/modules/core/src/main/java/org/gridgain/grid/kernal/managers/swapspace/GridSwapSpaceManager.java
+++ b/modules/core/src/main/java/org/gridgain/grid/kernal/managers/swapspace/GridSwapSpaceManager.java
@@ -16,7 +16,7 @@ import org.apache.ignite.spi.*;
 import org.gridgain.grid.*;
 import org.gridgain.grid.kernal.*;
 import org.gridgain.grid.kernal.managers.*;
-import org.gridgain.grid.spi.swapspace.*;
+import org.apache.ignite.spi.swapspace.*;
 import org.gridgain.grid.util.*;
 import org.gridgain.grid.util.typedef.internal.*;
 import org.gridgain.grid.util.lang.*;

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/ef258ece/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/GridCacheSwapManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/GridCacheSwapManager.java b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/GridCacheSwapManager.java
index ff7f0e6..4369213 100644
--- a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/GridCacheSwapManager.java
+++ b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/GridCacheSwapManager.java
@@ -15,7 +15,7 @@ import org.gridgain.grid.kernal.managers.swapspace.*;
 import org.gridgain.grid.kernal.processors.cache.query.*;
 import org.gridgain.grid.kernal.processors.license.*;
 import org.gridgain.grid.kernal.processors.offheap.*;
-import org.gridgain.grid.spi.swapspace.*;
+import org.apache.ignite.spi.swapspace.*;
 import org.gridgain.grid.util.*;
 import org.gridgain.grid.util.lang.*;
 import org.gridgain.grid.util.offheap.*;

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/ef258ece/modules/core/src/main/java/org/gridgain/grid/spi/swapspace/SwapContext.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/spi/swapspace/SwapContext.java b/modules/core/src/main/java/org/gridgain/grid/spi/swapspace/SwapContext.java
deleted file mode 100644
index c757651..0000000
--- a/modules/core/src/main/java/org/gridgain/grid/spi/swapspace/SwapContext.java
+++ /dev/null
@@ -1,39 +0,0 @@
-/* @java.file.header */
-
-/*  _________        _____ __________________        _____
- *  __  ____/___________(_)______  /__  ____/______ ____(_)_______
- *  _  / __  __  ___/__  / _  __  / _  / __  _  __ `/__  / __  __ \
- *  / /_/ /  _  /    _  /  / /_/ /  / /_/ /  / /_/ / _  /  _  / / /
- *  \____/   /_/     /_/   \_,__/   \____/   \__,_/  /_/   /_/ /_/
- */
-
-package org.gridgain.grid.spi.swapspace;
-
-import org.gridgain.grid.util.typedef.internal.*;
-
-/**
- * Context for swap operations.
- */
-public class SwapContext {
-    /** */
-    private ClassLoader clsLdr;
-
-    /**
-     * @return Class loader.
-     */
-    public ClassLoader classLoader() {
-        return clsLdr;
-    }
-
-    /**
-     * @param clsLdr Class loader.
-     */
-    public void classLoader(ClassLoader clsLdr) {
-        this.clsLdr = clsLdr;
-    }
-
-    /** {@inheritDoc} */
-    @Override public String toString() {
-        return S.toString(SwapContext.class, this);
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/ef258ece/modules/core/src/main/java/org/gridgain/grid/spi/swapspace/SwapKey.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/spi/swapspace/SwapKey.java b/modules/core/src/main/java/org/gridgain/grid/spi/swapspace/SwapKey.java
deleted file mode 100644
index 1bce5f7..0000000
--- a/modules/core/src/main/java/org/gridgain/grid/spi/swapspace/SwapKey.java
+++ /dev/null
@@ -1,116 +0,0 @@
-/* @java.file.header */
-
-/*  _________        _____ __________________        _____
- *  __  ____/___________(_)______  /__  ____/______ ____(_)_______
- *  _  / __  __  ___/__  / _  __  / _  / __  _  __ `/__  / __  __ \
- *  / /_/ /  _  /    _  /  / /_/ /  / /_/ /  / /_/ / _  /  _  / / /
- *  \____/   /_/     /_/   \_,__/   \____/   \__,_/  /_/   /_/ /_/
- */
-
-package org.gridgain.grid.spi.swapspace;
-
-import org.gridgain.grid.util.typedef.internal.*;
-import org.gridgain.grid.util.tostring.*;
-import org.jetbrains.annotations.*;
-
-/**
- * Utility wrapper class that represents swap key.
- * <p>
- * This class also holds information about partition this key belongs to
- * (if needed for caches).
- */
-public class SwapKey {
-    /** */
-    @GridToStringInclude
-    private final Object key;
-
-    /** */
-    private final int part;
-
-    /** Serialized key. */
-    @GridToStringExclude
-    private byte[] keyBytes;
-
-    /**
-     * @param key Key.
-     */
-    public SwapKey(Object key) {
-        this(key, Integer.MAX_VALUE, null);
-    }
-
-    /**
-     * @param key Key.
-     * @param part Partition.
-     */
-    public SwapKey(Object key, int part) {
-        this(key, part, null);
-    }
-
-    /**
-     * @param key Key.
-     * @param part Part.
-     * @param keyBytes Key bytes.
-     */
-    public SwapKey(Object key, int part, @Nullable byte[] keyBytes) {
-        assert key != null;
-        assert part >= 0;
-
-        this.key = key;
-        this.part = part;
-        this.keyBytes = keyBytes;
-    }
-
-    /**
-     * @return Key.
-     */
-    public Object key() {
-        return key;
-    }
-
-    /**
-     * @return Partition this key belongs to.
-     */
-    public int partition() {
-        return part;
-    }
-
-    /**
-     * @return Serialized key.
-     */
-    @Nullable public byte[] keyBytes() {
-        return keyBytes;
-    }
-
-    /**
-     * @param keyBytes Serialized key.
-     */
-    public void keyBytes(byte[] keyBytes) {
-        assert keyBytes != null;
-
-        this.keyBytes = keyBytes;
-    }
-
-    /** {@inheritDoc} */
-    @Override public boolean equals(Object obj) {
-        if (obj == this)
-            return true;
-
-        if (obj instanceof SwapKey) {
-            SwapKey other = (SwapKey)obj;
-
-            return part == other.part && key.equals(other.key);
-        }
-
-        return false;
-    }
-
-    /** {@inheritDoc} */
-    @Override public int hashCode() {
-        return key.hashCode();
-    }
-
-    /** {@inheritDoc} */
-    @Override public String toString() {
-        return S.toString(SwapKey.class, this);
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/ef258ece/modules/core/src/main/java/org/gridgain/grid/spi/swapspace/SwapSpaceSpi.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/spi/swapspace/SwapSpaceSpi.java b/modules/core/src/main/java/org/gridgain/grid/spi/swapspace/SwapSpaceSpi.java
deleted file mode 100644
index 1a07535..0000000
--- a/modules/core/src/main/java/org/gridgain/grid/spi/swapspace/SwapSpaceSpi.java
+++ /dev/null
@@ -1,202 +0,0 @@
-/* @java.file.header */
-
-/*  _________        _____ __________________        _____
- *  __  ____/___________(_)______  /__  ____/______ ____(_)_______
- *  _  / __  __  ___/__  / _  __  / _  / __  _  __ `/__  / __  __ \
- *  / /_/ /  _  /    _  /  / /_/ /  / /_/ /  / /_/ / _  /  _  / / /
- *  \____/   /_/     /_/   \_,__/   \____/   \__,_/  /_/   /_/ /_/
- */
-
-package org.gridgain.grid.spi.swapspace;
-
-import org.apache.ignite.lang.*;
-import org.apache.ignite.spi.*;
-import org.jetbrains.annotations.*;
-
-import java.util.*;
-
-/**
- * Provides a mechanism in grid for storing data on disk. GridGain cache uses swap space to overflow
- * data to disk if it cannot fit in memory. It's also possible to use swap space directly
- * by calling {@link org.apache.ignite.Ignite} API swap-related methods. Logically storage is organized into
- * independent 'spaces' in which data is stored.
- * <p>
- * All swap space implementations can be configured to prevent infinite growth and evict oldest entries.
- * <p>
- * The default swap space SPI is {@link org.gridgain.grid.spi.swapspace.file.FileSwapSpaceSpi} which stores values on disk in files and keeps keys in
- * memory.
- * <p>
- * Gridgain provides the following {@code GridSwapSpaceSpi} implementations:
- * <ul>
- * <li>
- *     {@link org.gridgain.grid.spi.swapspace.file.FileSwapSpaceSpi} - pure Java implementation with in-memory keys. This SPI is used by default.
- * </li>
- * <li>
- *     {@link org.gridgain.grid.spi.swapspace.noop.NoopSwapSpaceSpi} - no-op SPI mainly for testing.
- * </li>
- * </ul>
- * <p>
- * <p>
- * <b>NOTE:</b> this SPI (i.e. methods in this interface) should never be used directly. SPIs provide
- * internal view on the subsystem and is used internally by GridGain kernal. In rare use cases when
- * access to a specific implementation of this SPI is required - an instance of this SPI can be obtained
- * via {@link org.apache.ignite.Ignite#configuration()} method to check its configuration properties or call other non-SPI
- * methods. Note again that calling methods from this interface on the obtained instance can lead
- * to undefined behavior and explicitly not supported.
- */
-public interface SwapSpaceSpi extends IgniteSpi {
-    /**
-     * Entirely clears data space with given name, if any.
-     *
-     * @param spaceName Space name to clear.
-     * @throws org.apache.ignite.spi.IgniteSpiException In case of any errors.
-     */
-    public void clear(@Nullable String spaceName) throws IgniteSpiException;
-
-    /**
-     * Gets size in bytes for data space with given name. If specified space does
-     * not exist this method returns {@code 0}.
-     *
-     * @param spaceName Space name to get size for.
-     * @return Size in bytes.
-     * @throws org.apache.ignite.spi.IgniteSpiException In case of any errors.
-     */
-    public long size(@Nullable String spaceName) throws IgniteSpiException;
-
-    /**
-     * Gets number of stored entries (keys) in data space with given name. If specified
-     * space does not exist this method returns {@code 0}.
-     *
-     * @param spaceName Space name to get number of entries for.
-     * @return Number of stored entries in specified space.
-     * @throws org.apache.ignite.spi.IgniteSpiException In case of any errors.
-     */
-    public long count(@Nullable String spaceName) throws IgniteSpiException;
-
-    /**
-     * Reads stored value as array of bytes by key from data space with given name.
-     * If specified space does not exist this method returns {@code null}.
-     *
-     * @param spaceName Name of the data space to read from.
-     * @param key Key used to read value from data space.
-     * @param ctx Swap context.
-     * @return Value as array of bytes stored in specified data space that matches
-     *      to given key.
-     * @throws org.apache.ignite.spi.IgniteSpiException In case of any errors.
-     */
-    @Nullable public byte[] read(@Nullable String spaceName, SwapKey key, SwapContext ctx)
-        throws IgniteSpiException;
-
-    /**
-     * Reads stored values as array of bytes by all passed keys from data space with
-     * given name. If specified space does not exist this method returns empty map.
-     *
-     * @param spaceName Name of the data space to read from.
-     * @param keys Keys used to read values from data space.
-     * @param ctx Swap context.
-     * @return Map in which keys are the ones passed into method and values are
-     *      corresponding values read from swap storage.
-     * @throws org.apache.ignite.spi.IgniteSpiException In case of any errors.
-     */
-    public Map<SwapKey, byte[]> readAll(@Nullable String spaceName,
-        Iterable<SwapKey> keys, SwapContext ctx) throws IgniteSpiException;
-
-    /**
-     * Removes value stored in data space with given name corresponding to specified key.
-     *
-     * @param spaceName Space name to remove value from.
-     * @param key Key to remove value in the specified space for.
-     * @param c Optional closure that takes removed value and executes after actual
-     *      removing. If there was no value in storage the closure is not executed.
-     * @param ctx Swap context.
-     * @throws org.apache.ignite.spi.IgniteSpiException In case of any errors.
-     */
-    public void remove(@Nullable String spaceName, SwapKey key,
-        @Nullable IgniteInClosure<byte[]> c, SwapContext ctx) throws IgniteSpiException;
-
-    /**
-     * Removes values stored in data space with given name corresponding to specified keys.
-     *
-     * @param spaceName Space name to remove values from.
-     * @param keys Keys to remove value in the specified space for.
-     * @param c Optional closure that takes removed value and executes after actual
-     *      removing. If there was no value in storage the closure is not executed.
-     * @param ctx Swap context.
-     * @throws org.apache.ignite.spi.IgniteSpiException In case of any errors.
-     */
-    public void removeAll(@Nullable String spaceName, Collection<SwapKey> keys,
-        @Nullable IgniteBiInClosure<SwapKey, byte[]> c, SwapContext ctx) throws IgniteSpiException;
-
-    /**
-     * Stores value as array of bytes with given key into data space with given name.
-     *
-     * @param spaceName Space name to store key-value pair into.
-     * @param key Key to store given value for. This key can be used further to
-     *      read or remove stored value.
-     * @param val Some value as array of bytes to store into specified data space.
-     * @param ctx Swap context.
-     * @throws org.apache.ignite.spi.IgniteSpiException In case of any errors.
-     */
-    public void store(@Nullable String spaceName, SwapKey key, @Nullable byte[] val, SwapContext ctx)
-        throws IgniteSpiException;
-
-    /**
-     * Stores key-value pairs (both keys and values are arrays of bytes) into data
-     * space with given name.
-     *
-     * @param spaceName Space name to store key-value pairs into.
-     * @param pairs Map of stored key-value pairs where each one is an array of bytes.
-     * @param ctx Swap context.
-     * @throws org.apache.ignite.spi.IgniteSpiException In case of any errors.
-     */
-    public void storeAll(@Nullable String spaceName, Map<SwapKey, byte[]> pairs, SwapContext ctx)
-        throws IgniteSpiException;
-
-    /**
-     * Sets eviction listener to receive notifications on evicted swap entries.
-     *
-     * @param evictLsnr Eviction listener ({@code null} to stop receiving notifications).
-     */
-    public void setListener(@Nullable SwapSpaceSpiListener evictLsnr);
-
-    /**
-     * Gets partitions IDs that are stored in the passed in space.
-     *
-     * @param spaceName Space name.
-     * @return Partitions IDs or {@code null} if space is unknown.
-     * @throws org.apache.ignite.spi.IgniteSpiException If failed.
-     */
-    @Nullable public Collection<Integer> partitions(@Nullable String spaceName) throws IgniteSpiException;
-
-    /**
-     * Gets iterator over space keys.
-     *
-     * @param spaceName Space name.
-     * @param ctx Swap context.
-     * @return Iterator over space entries or {@code null} if space is unknown.
-     * @throws org.apache.ignite.spi.IgniteSpiException If failed.
-     */
-    @Nullable <K> IgniteSpiCloseableIterator<K> keyIterator(@Nullable String spaceName, SwapContext ctx)
-        throws IgniteSpiException;
-
-    /**
-     * Gets raw iterator over space entries.
-     *
-     * @param spaceName Space name.
-     * @return Iterator over space entries or {@code null} if space is unknown.
-     * @throws org.apache.ignite.spi.IgniteSpiException If failed.
-     */
-    @Nullable public IgniteSpiCloseableIterator<Map.Entry<byte[], byte[]>> rawIterator(@Nullable String spaceName)
-        throws IgniteSpiException;
-
-    /**
-     * Gets raw iterator over space entries.
-     *
-     * @param spaceName Space name.
-     * @param part Partition.
-     * @return Iterator over space entries or {@code null} if space is unknown.
-     * @throws org.apache.ignite.spi.IgniteSpiException If failed.
-     */
-    @Nullable public IgniteSpiCloseableIterator<Map.Entry<byte[], byte[]>> rawIterator(@Nullable String spaceName,
-        int part) throws IgniteSpiException;
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/ef258ece/modules/core/src/main/java/org/gridgain/grid/spi/swapspace/SwapSpaceSpiListener.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/spi/swapspace/SwapSpaceSpiListener.java b/modules/core/src/main/java/org/gridgain/grid/spi/swapspace/SwapSpaceSpiListener.java
deleted file mode 100644
index da8639d..0000000
--- a/modules/core/src/main/java/org/gridgain/grid/spi/swapspace/SwapSpaceSpiListener.java
+++ /dev/null
@@ -1,26 +0,0 @@
-/* @java.file.header */
-
-/*  _________        _____ __________________        _____
- *  __  ____/___________(_)______  /__  ____/______ ____(_)_______
- *  _  / __  __  ___/__  / _  __  / _  / __  _  __ `/__  / __  __ \
- *  / /_/ /  _  /    _  /  / /_/ /  / /_/ /  / /_/ / _  /  _  / / /
- *  \____/   /_/     /_/   \_,__/   \____/   \__,_/  /_/   /_/ /_/
- */
-
-package org.gridgain.grid.spi.swapspace;
-
-import org.jetbrains.annotations.*;
-
-/**
- * Swap space SPI eviction listener.
- */
-public interface SwapSpaceSpiListener {
-    /**
-     * Notification for swap space events.
-     *
-     * @param evtType Event type. See {@link org.apache.ignite.events.IgniteSwapSpaceEvent}
-     * @param spaceName Space name for this event or {@code null} for default space.
-     * @param keyBytes Key bytes of affected entry. Not {@code null} only for evict notifications.
-     */
-    public void onSwapEvent(int evtType, @Nullable String spaceName, @Nullable byte[] keyBytes);
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/ef258ece/modules/core/src/main/java/org/gridgain/grid/spi/swapspace/file/FileSwapArray.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/spi/swapspace/file/FileSwapArray.java b/modules/core/src/main/java/org/gridgain/grid/spi/swapspace/file/FileSwapArray.java
deleted file mode 100644
index e4a4f28..0000000
--- a/modules/core/src/main/java/org/gridgain/grid/spi/swapspace/file/FileSwapArray.java
+++ /dev/null
@@ -1,181 +0,0 @@
-/* @java.file.header */
-
-/*  _________        _____ __________________        _____
- *  __  ____/___________(_)______  /__  ____/______ ____(_)_______
- *  _  / __  __  ___/__  / _  __  / _  / __  _  __ `/__  / __  __ \
- *  / /_/ /  _  /    _  /  / /_/ /  / /_/ /  / /_/ / _  /  _  / / /
- *  \____/   /_/     /_/   \_,__/   \____/   \__,_/  /_/   /_/ /_/
- */
-
-package org.gridgain.grid.spi.swapspace.file;
-
-import org.jetbrains.annotations.*;
-
-import java.util.concurrent.atomic.*;
-
-/**
- * Growing array.
- */
-class FileSwapArray<X> {
-    /** First partition size must be power of two. */
-    private static final int FIRST_ARRAY_SIZE = 4096;
-
-    /** */
-    private static final int LADDER_SIZE = Integer.numberOfLeadingZeros(FIRST_ARRAY_SIZE) + 1;
-
-    /** */
-    @SuppressWarnings("unchecked")
-    private final AtomicReferenceArray<X>[] ladder = new AtomicReferenceArray[LADDER_SIZE];
-
-    /** */
-    private int idx = 1;
-
-    /**
-     *
-     */
-    FileSwapArray() {
-        synchronized (ladder) {
-            ladder[0] = new AtomicReferenceArray<>(FIRST_ARRAY_SIZE);
-        }
-    }
-
-    /**
-     * @return Size.
-     */
-    public int size() {
-        return idx;
-    }
-
-    /**
-     * Adds value to the end.
-     *
-     * @param x Value.
-     * @return Index where it was added.
-     */
-    int add(X x) {
-        int i = idx++;
-
-        assert i >= 0 && i != Integer.MAX_VALUE : "Integer overflow";
-
-        Slot<X> s = slot(i);
-
-        assert s != null; // We should add always in one thread.
-
-        s.set(x);
-
-        int len = s.arr.length();
-
-        if (s.idx + 1 == len) {
-            synchronized (ladder) {
-                ladder[s.arrIdx + 1] = new AtomicReferenceArray<>(s.arrIdx == 0 ? len : len << 1);
-            }
-        }
-
-        return i;
-    }
-
-    /**
-     * @param size New size.
-     */
-    void truncate(int size) {
-        assert size > 0;
-
-        idx = size;
-
-        int arrIdx = arrayIndex(idx) + 1;
-
-        if (arrIdx < ladder.length && ladder[arrIdx] != null) {
-            synchronized (ladder) {
-                do {
-                    ladder[arrIdx++] = null;
-                }
-                while (arrIdx < ladder.length && ladder[arrIdx] != null);
-            }
-        }
-    }
-
-    /**
-     * @param idx Absolute slot index.
-     * @return Array index in {@link #ladder}.
-     */
-    static int arrayIndex(int idx) {
-        if (idx < FIRST_ARRAY_SIZE)
-            return 0;
-
-        return LADDER_SIZE - Integer.numberOfLeadingZeros(idx);
-    }
-
-    /**
-     * Slot for given absolute index.
-     *
-     * @param idx Absolute index.
-     * @return Slot.
-     */
-    @Nullable Slot<X> slot(int idx) {
-        assert idx > 0 : idx;
-
-        int arrIdx = arrayIndex(idx);
-
-        AtomicReferenceArray<X> arr = ladder[arrIdx];
-
-        if (arr == null) {
-            synchronized (ladder) { // Ensure visibility.
-                arr = ladder[arrIdx];
-            }
-
-            if (arr == null)
-                return null;
-        }
-
-        return new Slot<>(arrIdx, arr, arrIdx == 0 ? idx : idx - arr.length());
-    }
-
-    /**
-     * Slot in array.
-     */
-    @SuppressWarnings("PublicInnerClass")
-    static final class Slot<X> {
-        /** */
-        private final int arrIdx;
-
-        /** */
-        private final AtomicReferenceArray<X> arr;
-
-        /** */
-        private final int idx;
-
-        /**
-         * @param arrIdx Index of array.
-         * @param arr Array.
-         * @param idx Index within the array.
-         */
-        private Slot(int arrIdx, AtomicReferenceArray<X> arr, int idx) {
-            this.arrIdx = arrIdx;
-            this.arr = arr;
-            this.idx = idx;
-        }
-
-        /**
-         * @return Value.
-         */
-        public X get() {
-            return arr.get(idx);
-        }
-
-        /**
-         * @param exp Expected.
-         * @param x New value.
-         * @return {@code true} If succeeded.
-         */
-        public boolean cas(@Nullable X exp, @Nullable X x) {
-            return exp == x || arr.compareAndSet(idx, exp, x);
-        }
-
-        /**
-         * @param x value.
-         */
-        private void set(X x) {
-            arr.lazySet(idx, x);
-        }
-    }
-}