You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by ag...@apache.org on 2014/12/16 03:48:12 UTC

[1/3] incubator-ignite git commit: GG-9141 - Cleanup, fixed delayed preloading.

Repository: incubator-ignite
Updated Branches:
  refs/heads/ignite-1 2ad4e6d1a -> 03c17b8f3


GG-9141 - Cleanup, fixed delayed preloading.


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

Branch: refs/heads/ignite-1
Commit: fa69fe8827b189d12f34378f9533d208ce062e46
Parents: 2ad4e6d
Author: Alexey Goncharuk <ag...@gridgain.com>
Authored: Mon Dec 15 16:18:24 2014 -0800
Committer: Alexey Goncharuk <ag...@gridgain.com>
Committed: Mon Dec 15 16:18:24 2014 -0800

----------------------------------------------------------------------
 .../java/org/apache/ignite/IgniteCompute.java   |  75 ++++++------
 .../grid/cache/GridCacheConfiguration.java      | 115 ++-----------------
 .../processors/cache/GridCacheAttributes.java   |  40 -------
 .../GridCachePartitionExchangeManager.java      |  16 +--
 .../processors/cache/GridCacheProcessor.java    |  16 ---
 .../preloader/GridDhtPartitionDemandPool.java   |   2 +-
 .../visor/cache/VisorCacheDgcConfiguration.java |   4 -
 .../tcp/GridCacheDhtLockBackupSelfTest.java     |   7 --
 .../GridCacheAbstractFailoverSelfTest.java      |   2 -
 .../GridCacheAbstractRemoveFailureTest.java     |   9 --
 .../GridCacheGroupLockFailoverSelfTest.java     |   3 -
 .../cache/GridCachePutAllFailoverSelfTest.java  |   3 -
 .../GridCacheReferenceCleanupSelfTest.java      |   1 -
 ...dCacheQueueMultiNodeConsistencySelfTest.java |   1 -
 ...dCachePartitionedQueueMultiNodeSelfTest.java |   1 -
 ...dCacheMultithreadedFailoverAbstractTest.java |   5 +-
 ...cOriginatingNodeFailureAbstractSelfTest.java |   1 -
 .../dht/GridCacheDhtPreloadDelayedSelfTest.java |   2 +-
 ...titionedExplicitLockNodeFailureSelfTest.java |   1 -
 ...idCachePartitionedMultiNodeLockSelfTest.java |   1 -
 ...GridCachePartitionedNodeFailureSelfTest.java |   1 -
 .../GridCachePartitionedNodeRestartTest.java    |   1 -
 ...ePartitionedOptimisticTxNodeRestartTest.java |   1 -
 .../GridCachePartitionedTxSalvageSelfTest.java  |   2 -
 .../near/GridCachePutArrayValueSelfTest.java    |   2 -
 25 files changed, 59 insertions(+), 253 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/fa69fe88/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 613a521..16da89f 100644
--- a/modules/core/src/main/java/org/apache/ignite/IgniteCompute.java
+++ b/modules/core/src/main/java/org/apache/ignite/IgniteCompute.java
@@ -12,7 +12,10 @@ package org.apache.ignite;
 import org.apache.ignite.cluster.*;
 import org.apache.ignite.compute.*;
 import org.apache.ignite.lang.*;
-import org.gridgain.grid.*;
+import org.apache.ignite.marshaller.optimized.IgniteOptimizedMarshaller;
+import org.apache.ignite.resources.*;
+import org.apache.ignite.spi.failover.FailoverSpi;
+import org.apache.ignite.spi.loadbalancing.LoadBalancingSpi;
 import org.jetbrains.annotations.*;
 
 import java.io.*;
@@ -21,66 +24,66 @@ import java.util.concurrent.*;
 
 /**
  * Defines compute grid functionality for executing tasks and closures over nodes
- * in the {@link org.apache.ignite.cluster.ClusterGroup}. Instance of {@code GridCompute} is obtained from grid projection
+ * in the {@link ClusterGroup}. Instance of {@code GridCompute} is obtained from grid projection
  * as follows:
  * <pre name="code" class="java">
  * GridCompute c = GridGain.grid().compute();
  * </pre>
  * The methods are grouped as follows:
  * <ul>
- * <li>{@code apply(...)} methods execute {@link org.apache.ignite.lang.IgniteClosure} jobs over nodes in the projection.</li>
+ * <li>{@code apply(...)} methods execute {@link IgniteClosure} jobs over nodes in the projection.</li>
  * <li>
  *     {@code call(...)} methods execute {@link Callable} jobs over nodes in the projection.
- *     Use {@link org.apache.ignite.lang.IgniteCallable} for better performance as it implements {@link Serializable}.
+ *     Use {@link IgniteCallable} for better performance as it implements {@link Serializable}.
  * </li>
  * <li>
  *     {@code run(...)} methods execute {@link Runnable} jobs over nodes in the projection.
- *     Use {@link org.apache.ignite.lang.IgniteRunnable} for better performance as it implements {@link Serializable}.
+ *     Use {@link IgniteRunnable} for better performance as it implements {@link Serializable}.
  * </li>
  * <li>{@code broadcast(...)} methods broadcast jobs to all nodes in the projection.</li>
  * <li>{@code affinity(...)} methods colocate jobs with nodes on which a specified key is cached.</li>
  * </ul>
  * Note that if attempt is made to execute a computation over an empty projection (i.e. projection that does
- * not have any alive nodes), then {@link org.apache.ignite.cluster.ClusterGroupEmptyException} will be thrown out of result future.
+ * not have any alive nodes), then {@link ClusterGroupEmptyException} will be thrown out of result future.
  * <h1 class="header">Serializable</h1>
  * Also note that {@link Runnable} and {@link Callable} implementations must support serialization as required
- * by the configured marshaller. For example, {@link org.apache.ignite.marshaller.optimized.IgniteOptimizedMarshaller} requires {@link Serializable}
+ * by the configured marshaller. For example, {@link IgniteOptimizedMarshaller} requires {@link Serializable}
  * objects by default, but can be configured not to. Generally speaking objects that implement {@link Serializable}
  * or {@link Externalizable} will perform better. For {@link Runnable} and {@link Callable} interfaces
- * GridGain provides analogous {@link org.apache.ignite.lang.IgniteRunnable} and {@link org.apache.ignite.lang.IgniteCallable} classes which are
+ * GridGain provides analogous {@link IgniteRunnable} and {@link IgniteCallable} classes which are
  * {@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.apache.ignite.spi.loadbalancing.LoadBalancingSpi},
+ * to be executed. The node will be selected based on the underlying {@link 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
- * load balancing behavior, consider implementing {@link org.apache.ignite.compute.ComputeTask} directly.
+ * load balancing behavior, consider implementing {@link ComputeTask} directly.
  * <h1 class="header">Fault Tolerance</h1>
  * GridGain guarantees that as long as there is at least one grid node standing, every job will be
  * executed. Jobs will automatically failover to another node if a remote node crashed
  * or has rejected execution due to lack of resources. By default, in case of failover, next
  * load balanced node will be picked for job execution. Also jobs will never be re-routed to the
  * nodes they have failed on. This behavior can be changed by configuring any of the existing or a custom
- * {@link org.apache.ignite.spi.failover.FailoverSpi} in grid configuration.
+ * {@link FailoverSpi} in grid configuration.
  * <h1 class="header">Resource Injection</h1>
  * All compute jobs, including closures, runnables, callables, and tasks can be injected with
  * grid resources. Both, field and method based injections are supported. The following grid
  * resources can be injected:
  * <ul>
- * <li>{@link org.apache.ignite.resources.IgniteTaskSessionResource}</li>
- * <li>{@link org.apache.ignite.resources.IgniteInstanceResource}</li>
- * <li>{@link org.apache.ignite.resources.IgniteLoggerResource}</li>
- * <li>{@link org.apache.ignite.resources.IgniteHomeResource}</li>
- * <li>{@link org.apache.ignite.resources.IgniteExecutorServiceResource}</li>
- * <li>{@link org.apache.ignite.resources.IgniteLocalNodeIdResource}</li>
- * <li>{@link org.apache.ignite.resources.IgniteMBeanServerResource}</li>
- * <li>{@link org.apache.ignite.resources.IgniteMarshallerResource}</li>
- * <li>{@link org.apache.ignite.resources.IgniteSpringApplicationContextResource}</li>
- * <li>{@link org.apache.ignite.resources.IgniteSpringResource}</li>
+ * <li>{@link IgniteTaskSessionResource}</li>
+ * <li>{@link IgniteInstanceResource}</li>
+ * <li>{@link IgniteLoggerResource}</li>
+ * <li>{@link IgniteHomeResource}</li>
+ * <li>{@link IgniteExecutorServiceResource}</li>
+ * <li>{@link IgniteLocalNodeIdResource}</li>
+ * <li>{@link IgniteMBeanServerResource}</li>
+ * <li>{@link IgniteMarshallerResource}</li>
+ * <li>{@link IgniteSpringApplicationContextResource}</li>
+ * <li>{@link IgniteSpringResource}</li>
  * </ul>
  * Refer to corresponding resource documentation for more information.
- * Here is an example of how to inject instance of {@link org.apache.ignite.Ignite} into a computation:
+ * Here is an example of how to inject instance of {@link Ignite} into a computation:
  * <pre name="code" class="java">
  * public class MyGridJob extends GridRunnable {
  *      ...
@@ -93,7 +96,7 @@ import java.util.concurrent.*;
  * Note that regardless of which method is used for executing computations, all relevant SPI implementations
  * configured for this grid instance will be used (i.e. failover, load balancing, collision resolution,
  * checkpoints, etc.). If you need to override configured defaults, you should use compute task together with
- * {@link org.apache.ignite.compute.ComputeTaskSpis} annotation. Refer to {@link org.apache.ignite.compute.ComputeTask} documentation for more information.
+ * {@link ComputeTaskSpis} annotation. Refer to {@link ComputeTask} documentation for more information.
  */
 public interface IgniteCompute extends IgniteAsyncSupport {
     /**
@@ -112,8 +115,8 @@ public interface IgniteCompute extends IgniteAsyncSupport {
      * @param cacheName Name of the cache to use for affinity co-location.
      * @param affKey Affinity key.
      * @param job Job which will be co-located on the node with given affinity key.
-     * @see org.apache.ignite.compute.ComputeJobContext#cacheName()
-     * @see org.apache.ignite.compute.ComputeJobContext#affinityKey()
+     * @see ComputeJobContext#cacheName()
+     * @see ComputeJobContext#affinityKey()
      * @throws IgniteCheckedException If job failed.
      */
     public void affinityRun(@Nullable String cacheName, Object affKey, Runnable job) throws IgniteCheckedException;
@@ -129,18 +132,18 @@ public interface IgniteCompute extends IgniteAsyncSupport {
      * @param job Job which will be co-located on the node with given affinity key.
      * @return Job result.
      * @throws IgniteCheckedException If job failed.
-     * @see org.apache.ignite.compute.ComputeJobContext#cacheName()
-     * @see org.apache.ignite.compute.ComputeJobContext#affinityKey()
+     * @see ComputeJobContext#cacheName()
+     * @see ComputeJobContext#affinityKey()
      */
     public <R> R affinityCall(@Nullable String cacheName, Object affKey, Callable<R> job) throws IgniteCheckedException;
 
     /**
      * Executes given task on the grid projection. For step-by-step explanation of task execution process
-     * refer to {@link org.apache.ignite.compute.ComputeTask} documentation.
+     * refer to {@link ComputeTask} documentation.
      * <p>
      * Supports asynchronous execution (see {@link IgniteAsyncSupport}).
      *
-     * @param taskCls Class of the task to execute. If class has {@link org.apache.ignite.compute.ComputeTaskName} annotation,
+     * @param taskCls Class of the task to execute. If class has {@link ComputeTaskName} annotation,
      *      then task is deployed under a name specified within annotation. Otherwise, full
      *      class name is used as task name.
      * @param arg Optional argument of task execution, can be {@code null}.
@@ -151,11 +154,11 @@ public interface IgniteCompute extends IgniteAsyncSupport {
 
     /**
      * Executes given task on this grid projection. For step-by-step explanation of task execution process
-     * refer to {@link org.apache.ignite.compute.ComputeTask} documentation.
+     * refer to {@link ComputeTask} documentation.
      * <p>
      * Supports asynchronous execution (see {@link IgniteAsyncSupport}).
      *
-     * @param task Instance of task to execute. If task class has {@link org.apache.ignite.compute.ComputeTaskName} annotation,
+     * @param task Instance of task to execute. If task class has {@link ComputeTaskName} annotation,
      *      then task is deployed under a name specified within annotation. Otherwise, full
      *      class name is used as task name.
      * @param arg Optional argument of task execution, can be {@code null}.
@@ -166,7 +169,7 @@ public interface IgniteCompute extends IgniteAsyncSupport {
 
     /**
      * Executes given task on this grid projection. For step-by-step explanation of task execution process
-     * refer to {@link org.apache.ignite.compute.ComputeTask} documentation.
+     * refer to {@link ComputeTask} documentation.
      * <p>
      * If task for given name has not been deployed yet, then {@code taskName} will be
      * used as task class name to auto-deploy the task (see {@link #localDeployTask(Class, ClassLoader)} method).
@@ -177,7 +180,7 @@ public interface IgniteCompute extends IgniteAsyncSupport {
      * @param arg Optional argument of task execution, can be {@code null}.
      * @return Task result.
      * @throws IgniteCheckedException If task failed.
-     * @see org.apache.ignite.compute.ComputeTask for information about task execution.
+     * @see ComputeTask for information about task execution.
      */
     public <T, R> R execute(String taskName, @Nullable T arg) throws IgniteCheckedException;
 
@@ -330,7 +333,7 @@ public interface IgniteCompute extends IgniteAsyncSupport {
      * Sets task name for the next executed task on this projection in the <b>current thread</b>.
      * When task starts execution, the name is reset, so one name is used only once. You may use
      * this method to set task name when executing jobs directly, without explicitly
-     * defining {@link org.apache.ignite.compute.ComputeTask}.
+     * defining {@link ComputeTask}.
      * <p>
      * Here is an example.
      * <pre name="code" class="java">
@@ -346,7 +349,7 @@ public interface IgniteCompute extends IgniteAsyncSupport {
      * Sets task timeout for the next executed task on this projection in the <b>current thread</b>.
      * When task starts execution, the timeout is reset, so one timeout is used only once. You may use
      * this method to set task name when executing jobs directly, without explicitly
-     * defining {@link org.apache.ignite.compute.ComputeTask}.
+     * defining {@link ComputeTask}.
      * <p>
      * Here is an example.
      * <pre name="code" class="java">
@@ -391,7 +394,7 @@ public interface IgniteCompute extends IgniteAsyncSupport {
      * <p>
      * This method has no effect if the class passed in was already deployed.
      *
-     * @param taskCls Task class to deploy. If task class has {@link org.apache.ignite.compute.ComputeTaskName} annotation,
+     * @param taskCls Task class to deploy. If task class has {@link ComputeTaskName} annotation,
      *      then task will be deployed under the name specified within annotation. Otherwise, full
      *      class name will be used as task's name.
      * @param clsLdr Task class loader. This class loader is in charge

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/fa69fe88/modules/core/src/main/java/org/gridgain/grid/cache/GridCacheConfiguration.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/cache/GridCacheConfiguration.java b/modules/core/src/main/java/org/gridgain/grid/cache/GridCacheConfiguration.java
index cb082fb..dff9bdc 100644
--- a/modules/core/src/main/java/org/gridgain/grid/cache/GridCacheConfiguration.java
+++ b/modules/core/src/main/java/org/gridgain/grid/cache/GridCacheConfiguration.java
@@ -9,6 +9,10 @@
 
 package org.gridgain.grid.cache;
 
+import org.apache.ignite.Ignite;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.portables.PortableObject;
+import org.apache.ignite.spi.indexing.*;
 import org.gridgain.grid.cache.affinity.*;
 import org.gridgain.grid.cache.affinity.consistenthash.*;
 import org.gridgain.grid.cache.cloner.*;
@@ -24,21 +28,17 @@ import java.util.*;
 
 /**
  * This class defines grid cache configuration. This configuration is passed to
- * grid via {@link org.apache.ignite.configuration.IgniteConfiguration#getCacheConfiguration()} method. It defines all configuration
+ * grid via {@link IgniteConfiguration#getCacheConfiguration()} method. It defines all configuration
  * parameters required to start a cache within grid instance. You can have multiple caches
  * configured with different names within one grid.
  * <p>
  * Cache configuration is set on {@link
- * org.apache.ignite.configuration.IgniteConfiguration#setCacheConfiguration(GridCacheConfiguration...)} method. This adapter is a simple bean and
+ * IgniteConfiguration#setCacheConfiguration(GridCacheConfiguration...)} method. This adapter is a simple bean and
  * can be configured from Spring XML files (or other DI frameworks). <p> Note that absolutely all configuration
  * properties are optional, so users should only change what they need.
  */
 @SuppressWarnings("RedundantFieldInitialization")
 public class GridCacheConfiguration extends MutableConfiguration {
-    /** DGC tracing logger name. */
-    public static final String DGC_TRACE_LOGGER_NAME =
-        "org.gridgain.grid.kernal.processors.cache.GridCacheDgcManager.trace";
-
     /** Default atomic sequence reservation size. */
     public static final int DFLT_ATOMIC_SEQUENCE_RESERVE_SIZE = 1000;
 
@@ -96,15 +96,6 @@ public class GridCacheConfiguration extends MutableConfiguration {
     /** Default preload batch size in bytes. */
     public static final int DFLT_PRELOAD_BATCH_SIZE = 512 * 1024; // 512K
 
-    /** Default distributed garbage collection frequency. */
-    public static final long DFLT_DGC_FREQUENCY = 10000;
-
-    /** Default timeout for lock not to be considered as suspicious. */
-    public static final long DFLT_DGC_SUSPECT_LOCK_TIMEOUT = 10000;
-
-    /** Default value for whether DGC should remove long running locks, or only report them. */
-    public static final boolean DFLT_DGC_REMOVE_LOCKS = true;
-
     /** Default maximum eviction queue ratio. */
     public static final float DFLT_MAX_EVICTION_OVERFLOW_RATIO = 10;
 
@@ -274,15 +265,6 @@ public class GridCacheConfiguration extends MutableConfiguration {
     /** Preload batch size. */
     private int preloadBatchSize = DFLT_PRELOAD_BATCH_SIZE;
 
-    /** Distributed garbage collection frequency. */
-    private long dgcFreq = DFLT_DGC_FREQUENCY;
-
-    /** Timeout after which DGC will consider remote locks as suspects. */
-    private long dgcSuspectLockTimeout = DFLT_DGC_SUSPECT_LOCK_TIMEOUT;
-
-    /** Flag indicating whether DGC should remove locks. */
-    private boolean dgcRmvLocks = DFLT_DGC_REMOVE_LOCKS;
-
     /** Off-heap memory size. */
     private long offHeapMaxMem = DFLT_OFFHEAP_MEMORY;
 
@@ -367,9 +349,6 @@ public class GridCacheConfiguration extends MutableConfiguration {
         cloner = cc.getCloner();
         dfltLockTimeout = cc.getDefaultLockTimeout();
         dfltQryTimeout = cc.getDefaultQueryTimeout();
-        dgcFreq = cc.getDgcFrequency();
-        dgcRmvLocks = cc.isDgcRemoveLocks();
-        dgcSuspectLockTimeout = cc.getDgcSuspectLockTimeout();
         distro = cc.getDistributionMode();
         eagerTtl = cc.isEagerTtl();
         evictFilter = cc.getEvictionFilter();
@@ -419,8 +398,8 @@ public class GridCacheConfiguration extends MutableConfiguration {
 
     /**
      * Cache name. If not provided or {@code null}, then this will be considered a default
-     * cache which can be accessed via {@link org.apache.ignite.Ignite#cache(String) Grid.cache(null)} method. Otherwise, if name
-     * is provided, the cache will be accessed via {@link org.apache.ignite.Ignite#cache(String)} method.
+     * cache which can be accessed via {@link Ignite#cache(String) Grid.cache(null)} method. Otherwise, if name
+     * is provided, the cache will be accessed via {@link Ignite#cache(String)} method.
      *
      * @return Cache name.
      */
@@ -1105,78 +1084,6 @@ public class GridCacheConfiguration extends MutableConfiguration {
     }
 
     /**
-     * Gets frequency at which distributed garbage collector will
-     * check other nodes if there are any zombie locks left over.
-     * <p>
-     * If not provided, default value is {@link GridCacheConfiguration#DFLT_DGC_FREQUENCY}.
-     *
-     * @return Frequency of distributed GC in milliseconds ({@code 0} to disable GC).
-     */
-    public long getDgcFrequency() {
-        return dgcFreq;
-    }
-
-    /**
-     * Sets frequency in milliseconds for internal distributed garbage collector. Pass {@code 0} to disable distributed
-     * garbage collection. <p> If not provided, default value is {@link GridCacheConfiguration#DFLT_DGC_FREQUENCY}.
-     *
-     * @param dgcFreq Frequency of distributed GC in milliseconds ({@code 0} to disable GC).
-     */
-    public void setDgcFrequency(long dgcFreq) {
-        this.dgcFreq = dgcFreq;
-    }
-
-    /**
-     * Gets timeout after which locks are considered to be suspicious.
-     * <p>
-     * If not provided, default value is {@link GridCacheConfiguration#DFLT_DGC_SUSPECT_LOCK_TIMEOUT}.
-     *
-     * @return Distributed GC suspect lock timeout.
-     */
-    public long getDgcSuspectLockTimeout() {
-        return dgcSuspectLockTimeout;
-    }
-
-    /**
-     * Sets suspect lock timeout in milliseconds for internal distributed garbage collector. If lock's lifetime is
-     * greater than the timeout, then lock is considered to be suspicious. <p> If not provided, default value is {@link
-     * GridCacheConfiguration#DFLT_DGC_SUSPECT_LOCK_TIMEOUT}.
-     *
-     * @param dgcSuspectLockTimeout Timeout in milliseconds.
-     */
-    public void setDgcSuspectLockTimeout(long dgcSuspectLockTimeout) {
-        this.dgcSuspectLockTimeout = dgcSuspectLockTimeout;
-    }
-
-    /**
-     * Gets system-wide flag indicating whether DGC manager should remove locks in question or only
-     * report them. Note, that this behavior could be overridden by specifically calling
-     * {@link GridCache#dgc(long, boolean, boolean)} method.
-     * <p>
-     * If {@code false} DGC manager will not release the locks that are not owned by any other node.
-     * This may be useful for debugging purposes. You may also enable DGC tracing by enabling DEBUG
-     * on {@link #DGC_TRACE_LOGGER_NAME} category.
-     * <p>
-     * If not provided, default value is {@link GridCacheConfiguration#DFLT_DGC_REMOVE_LOCKS}.
-     *
-     * @return {@code True} if DGC should remove locks.
-     * @see #DGC_TRACE_LOGGER_NAME
-     */
-    public boolean isDgcRemoveLocks() {
-        return dgcRmvLocks;
-    }
-
-    /**
-     * Sets DGC remove locks flag.
-     *
-     * @param dgcRmvLocks {@code True} to remove locks.
-     * @see #isDgcRemoveLocks()
-     */
-    public void setDgcRemoveLocks(boolean dgcRmvLocks) {
-        this.dgcRmvLocks = dgcRmvLocks;
-    }
-
-    /**
      * Flag indicating whether GridGain should use swap storage by default. By default
      * swap is disabled which is defined via {@link #DFLT_SWAP_ENABLED} constant.
      * <p>
@@ -1562,7 +1469,7 @@ public class GridCacheConfiguration extends MutableConfiguration {
      * SPI is configured. In majority of the cases default value should be used.
      *
      * @return Name of SPI to use for indexing.
-     * @see org.apache.ignite.spi.indexing.IndexingSpi
+     * @see GridIndexingSpi
      */
     public String getIndexingSpiName() {
         return indexingSpiName;
@@ -1576,7 +1483,7 @@ public class GridCacheConfiguration extends MutableConfiguration {
      * SPI is configured. In majority of the cases default value should be used.
      *
      * @param indexingSpiName Name.
-     * @see org.apache.ignite.spi.indexing.IndexingSpi
+     * @see GridIndexingSpi
      */
     public void setIndexingSpiName(String indexingSpiName) {
         this.indexingSpiName = indexingSpiName;
@@ -1744,7 +1651,7 @@ public class GridCacheConfiguration extends MutableConfiguration {
 
     /**
      * Flag indicating whether GridGain should store portable keys and values
-     * as instances of {@link org.apache.ignite.portables.PortableObject}.
+     * as instances of {@link PortableObject}.
      *
      * @return Portable enabled flag.
      */

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/fa69fe88/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/GridCacheAttributes.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/GridCacheAttributes.java b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/GridCacheAttributes.java
index 0d659f6..531f816 100644
--- a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/GridCacheAttributes.java
+++ b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/GridCacheAttributes.java
@@ -12,7 +12,6 @@ package org.gridgain.grid.kernal.processors.cache;
 import org.gridgain.grid.cache.*;
 import org.gridgain.grid.cache.affinity.*;
 import org.gridgain.grid.cache.affinity.consistenthash.*;
-import org.gridgain.grid.kernal.processors.cache.dr.*;
 import org.gridgain.grid.util.typedef.internal.*;
 import org.jetbrains.annotations.*;
 
@@ -70,15 +69,6 @@ public class GridCacheAttributes implements Externalizable {
     /** Preload batch size. */
     private int preloadBatchSize;
 
-    /** Distributed garbage collection frequency. */
-    private long dgcFreq;
-
-    /** Timeout after which DGC will consider remote locks as suspects. */
-    private long dgcSuspectLockTimeout;
-
-    /** Flag indicating whether DGC should remove locks. */
-    private boolean dgcRmvLocks;
-
     /** Synchronization mode. */
     private GridCacheWriteSynchronizationMode writeSyncMode;
 
@@ -162,9 +152,6 @@ public class GridCacheAttributes implements Externalizable {
         cacheMode = cfg.getCacheMode();
         dfltLockTimeout = cfg.getDefaultLockTimeout();
         dfltQryTimeout = cfg.getDefaultQueryTimeout();
-        dgcFreq = cfg.getDgcFrequency();
-        dgcRmvLocks = cfg.isDgcRemoveLocks();
-        dgcSuspectLockTimeout  = cfg.getDgcSuspectLockTimeout();
         evictMaxOverflowRatio = cfg.getEvictMaxOverflowRatio();
         evictNearSync = cfg.isEvictNearSynchronized();
         evictSync = cfg.isEvictSynchronized();
@@ -439,27 +426,6 @@ public class GridCacheAttributes implements Externalizable {
     }
 
     /**
-     * @return Distributed garbage collection frequency.
-     */
-    public long dgcFrequency() {
-        return dgcFreq;
-    }
-
-    /**
-     * @return Timeout after which DGC will consider remote locks as suspects.
-     */
-    public long dgcSuspectLockTimeout() {
-        return dgcSuspectLockTimeout;
-    }
-
-    /**
-     * @return Flag indicating whether DGC should remove locks.
-     */
-    public boolean dgcRemoveLocks() {
-        return dgcRmvLocks;
-    }
-
-    /**
      * @return Synchronization mode.
      */
     public GridCacheWriteSynchronizationMode writeSynchronization() {
@@ -535,9 +501,6 @@ public class GridCacheAttributes implements Externalizable {
         U.writeEnum0(out, cacheMode);
         out.writeLong(dfltLockTimeout);
         out.writeLong(dfltQryTimeout);
-        out.writeLong(dgcFreq);
-        out.writeBoolean(dgcRmvLocks);
-        out.writeLong(dgcSuspectLockTimeout);
         out.writeFloat(evictMaxOverflowRatio);
         out.writeBoolean(evictNearSync);
         out.writeBoolean(evictSync);
@@ -582,9 +545,6 @@ public class GridCacheAttributes implements Externalizable {
         cacheMode = GridCacheMode.fromOrdinal(U.readEnumOrdinal0(in));
         dfltLockTimeout = in.readLong();
         dfltQryTimeout = in.readLong();
-        dgcFreq = in.readLong();
-        dgcRmvLocks = in.readBoolean();
-        dgcSuspectLockTimeout = in.readLong();
         evictMaxOverflowRatio = in.readFloat();
         evictNearSync = in.readBoolean();
         evictSync  = in.readBoolean();

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/fa69fe88/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/GridCachePartitionExchangeManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/GridCachePartitionExchangeManager.java b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/GridCachePartitionExchangeManager.java
index a553dd5..b0c319d 100644
--- a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/GridCachePartitionExchangeManager.java
+++ b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/GridCachePartitionExchangeManager.java
@@ -756,7 +756,7 @@ public class GridCachePartitionExchangeManager<K, V> extends GridCacheSharedMana
 
                     busy = true;
 
-                    Map<Integer, GridDhtPreloaderAssignments<K, V>> assignsMap = null;
+                    Map<Integer, GridDhtPreloaderAssignments<K, V>> assignsMap = new HashMap<>();
 
                     boolean dummyReassign = exchFut.dummyReassign();
                     boolean forcePreload = exchFut.forcePreload();
@@ -820,17 +820,13 @@ public class GridCachePartitionExchangeManager<K, V> extends GridCacheSharedMana
                         for (GridCacheContext<K, V> cacheCtx : cctx.cacheContexts()) {
                             long delay = cacheCtx.config().getPreloadPartitionedDelay();
 
-                            // Don't delay for dummy reassigns to avoid infinite recursion.
-                            if (delay == 0 || forcePreload) {
-                                GridDhtPreloaderAssignments<K, V> assigns = cacheCtx.preloader().assign(exchFut);
+                            GridDhtPreloaderAssignments<K, V> assigns = null;
 
-                                if (assigns != null) {
-                                    if (assignsMap == null)
-                                        assignsMap = new HashMap<>();
+                            // Don't delay for dummy reassigns to avoid infinite recursion.
+                            if (delay == 0 || forcePreload)
+                                assigns = cacheCtx.preloader().assign(exchFut);
 
-                                    assignsMap.put(cacheCtx.cacheId(), assigns);
-                                }
-                            }
+                            assignsMap.put(cacheCtx.cacheId(), assigns);
                         }
                     }
                     finally {

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/fa69fe88/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/GridCacheProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/GridCacheProcessor.java b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/GridCacheProcessor.java
index a083805..1c08267 100644
--- a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/GridCacheProcessor.java
+++ b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/GridCacheProcessor.java
@@ -1071,25 +1071,9 @@ public class GridCacheProcessor extends GridProcessorAdapter {
                             "Default query timeout", locAttr.defaultQueryTimeout(), rmtAttr.defaultQueryTimeout(),
                             false);
 
-                        CU.checkAttributeMismatch(log, rmtAttr.cacheName(), rmt, "dgcFrequency",
-                            "Distributed garbage collector frequency", locAttr.dgcFrequency(), rmtAttr.dgcFrequency(),
-                            false);
-
                         CU.checkAttributeMismatch(log, rmtAttr.cacheName(), rmt, "defaultTimeToLive",
                             "Default time to live", locAttr.defaultTimeToLive(), rmtAttr.defaultTimeToLive(), false);
 
-                        CU.checkAttributeMismatch(log, rmtAttr.cacheName(), rmt, "dgcFrequency",
-                            "Distributed garbage collector frequency", locAttr.dgcFrequency(), rmtAttr.dgcFrequency(),
-                            false);
-
-                        CU.checkAttributeMismatch(log, rmtAttr.cacheName(), rmt, "dgcRemoveLocks",
-                            "Distributed garbage collector remove locks", locAttr.dgcRemoveLocks(),
-                            rmtAttr.dgcRemoveLocks(), false);
-
-                        CU.checkAttributeMismatch(log, rmtAttr.cacheName(), rmt, "dgcSuspectLockTimeout",
-                            "Distributed garbage collector suspect lock timeout", locAttr.dgcSuspectLockTimeout(),
-                            rmtAttr.dgcSuspectLockTimeout(), false);
-
                         CU.checkAttributeMismatch(log, rmtAttr.cacheName(), rmt, "preloadBatchSize",
                             "Preload batch size", locAttr.preloadBatchSize(), rmtAttr.preloadBatchSize(), false);
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/fa69fe88/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/distributed/dht/preloader/GridDhtPartitionDemandPool.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/distributed/dht/preloader/GridDhtPartitionDemandPool.java b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/distributed/dht/preloader/GridDhtPartitionDemandPool.java
index ac0b7e6..17291aa 100644
--- a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/distributed/dht/preloader/GridDhtPartitionDemandPool.java
+++ b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/distributed/dht/preloader/GridDhtPartitionDemandPool.java
@@ -568,7 +568,7 @@ public class GridDhtPartitionDemandPool<K, V> {
          * @param exchFut Exchange future.
          * @return Missed partitions.
          * @throws InterruptedException If interrupted.
-         * @throws org.apache.ignite.cluster.ClusterTopologyException If node left.
+         * @throws ClusterTopologyException If node left.
          * @throws IgniteCheckedException If failed to send message.
          */
         private Set<Integer> demandFromNode(ClusterNode node, final long topVer, GridDhtPartitionDemandMessage<K, V> d,

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/fa69fe88/modules/core/src/main/java/org/gridgain/grid/kernal/visor/cache/VisorCacheDgcConfiguration.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/kernal/visor/cache/VisorCacheDgcConfiguration.java b/modules/core/src/main/java/org/gridgain/grid/kernal/visor/cache/VisorCacheDgcConfiguration.java
index e030dc4..be170c6 100644
--- a/modules/core/src/main/java/org/gridgain/grid/kernal/visor/cache/VisorCacheDgcConfiguration.java
+++ b/modules/core/src/main/java/org/gridgain/grid/kernal/visor/cache/VisorCacheDgcConfiguration.java
@@ -37,10 +37,6 @@ public class VisorCacheDgcConfiguration implements Serializable {
     public static VisorCacheDgcConfiguration from(GridCacheConfiguration ccfg) {
         VisorCacheDgcConfiguration cfg = new VisorCacheDgcConfiguration();
 
-        cfg.frequency(ccfg.getDgcFrequency());
-        cfg.removedLocks(ccfg.isDgcRemoveLocks());
-        cfg.suspectLockTimeout(ccfg.getDgcSuspectLockTimeout());
-
         return cfg;
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/fa69fe88/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridCacheDhtLockBackupSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridCacheDhtLockBackupSelfTest.java b/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridCacheDhtLockBackupSelfTest.java
index 5031b2f..4445aaf 100644
--- a/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridCacheDhtLockBackupSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridCacheDhtLockBackupSelfTest.java
@@ -14,8 +14,6 @@ import org.apache.ignite.configuration.*;
 import org.apache.ignite.lang.*;
 import org.apache.ignite.marshaller.*;
 import org.apache.ignite.marshaller.jdk.*;
-import org.apache.ignite.resources.*;
-import org.gridgain.grid.*;
 import org.gridgain.grid.cache.*;
 import org.gridgain.grid.kernal.managers.communication.*;
 import org.gridgain.grid.kernal.processors.cache.distributed.near.*;
@@ -85,7 +83,6 @@ public class GridCacheDhtLockBackupSelfTest extends GridCommonAbstractTest {
         cacheCfg.setCacheMode(GridCacheMode.PARTITIONED);
         cacheCfg.setWriteSynchronizationMode(FULL_ASYNC);
         cacheCfg.setPreloadMode(SYNC);
-        cacheCfg.setDgcFrequency(0);
 
         return cacheCfg;
     }
@@ -232,10 +229,6 @@ public class GridCacheDhtLockBackupSelfTest extends GridCommonAbstractTest {
         /** */
         private int delayTime;
 
-        /** */
-        @IgniteMarshallerResource
-        private IgniteMarshaller marsh;
-
         /**
          * Creates test communication spi.
          *

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/fa69fe88/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheAbstractFailoverSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheAbstractFailoverSelfTest.java b/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheAbstractFailoverSelfTest.java
index aa0b031..21851f7 100644
--- a/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheAbstractFailoverSelfTest.java
+++ b/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheAbstractFailoverSelfTest.java
@@ -14,7 +14,6 @@ import org.apache.ignite.cluster.*;
 import org.apache.ignite.configuration.*;
 import org.apache.ignite.lang.*;
 import org.apache.ignite.resources.*;
-import org.gridgain.grid.*;
 import org.gridgain.grid.cache.*;
 import org.gridgain.grid.util.typedef.*;
 import org.gridgain.grid.util.typedef.internal.*;
@@ -70,7 +69,6 @@ public abstract class GridCacheAbstractFailoverSelfTest extends GridCacheAbstrac
         GridCacheConfiguration cfg = super.cacheConfiguration(gridName);
 
         cfg.setPreloadMode(SYNC);
-        cfg.setDgcFrequency(0);
 
         return cfg;
     }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/fa69fe88/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheAbstractRemoveFailureTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheAbstractRemoveFailureTest.java b/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheAbstractRemoveFailureTest.java
index 6686911..938a4a8 100644
--- a/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheAbstractRemoveFailureTest.java
+++ b/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheAbstractRemoveFailureTest.java
@@ -11,7 +11,6 @@ package org.gridgain.grid.kernal.processors.cache;
 
 import org.apache.ignite.*;
 import org.apache.ignite.lang.*;
-import org.gridgain.grid.*;
 import org.gridgain.grid.cache.*;
 import org.gridgain.grid.util.typedef.*;
 import org.gridgain.grid.util.typedef.internal.*;
@@ -98,14 +97,6 @@ public abstract class GridCacheAbstractRemoveFailureTest extends GridCacheAbstra
         return DUR + 60_000;
     }
 
-    @Override protected GridCacheConfiguration cacheConfiguration(String gridName) throws Exception {
-        GridCacheConfiguration cfg = super.cacheConfiguration(gridName);
-
-        cfg.setDgcFrequency(0);
-
-        return cfg;
-    }
-
     /**
      * @throws Exception If failed.
      */

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/fa69fe88/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheGroupLockFailoverSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheGroupLockFailoverSelfTest.java b/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheGroupLockFailoverSelfTest.java
index c812755..241de60 100644
--- a/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheGroupLockFailoverSelfTest.java
+++ b/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheGroupLockFailoverSelfTest.java
@@ -15,7 +15,6 @@ import org.apache.ignite.cluster.*;
 import org.apache.ignite.compute.*;
 import org.apache.ignite.configuration.*;
 import org.apache.ignite.lang.*;
-import org.gridgain.grid.*;
 import org.gridgain.grid.cache.*;
 import org.apache.ignite.spi.discovery.tcp.*;
 import org.apache.ignite.spi.discovery.tcp.ipfinder.*;
@@ -436,8 +435,6 @@ public class GridCacheGroupLockFailoverSelfTest extends GridCommonAbstractTest {
             cacheCfg.setCacheMode(GridCacheMode.PARTITIONED);
             cacheCfg.setStartSize(4500000);
             cacheCfg.setBackups(backups);
-            cacheCfg.setDgcSuspectLockTimeout(600000);
-            cacheCfg.setDgcFrequency(0);
             cacheCfg.setStoreValueBytes(true);
             cacheCfg.setDistributionMode(nearEnabled ? NEAR_PARTITIONED : PARTITIONED_ONLY);
             cacheCfg.setQueryIndexEnabled(false);

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/fa69fe88/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCachePutAllFailoverSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCachePutAllFailoverSelfTest.java b/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCachePutAllFailoverSelfTest.java
index 7998316..683cbd1 100644
--- a/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCachePutAllFailoverSelfTest.java
+++ b/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCachePutAllFailoverSelfTest.java
@@ -8,7 +8,6 @@ import org.apache.ignite.configuration.*;
 import org.apache.ignite.lang.*;
 import org.apache.ignite.resources.*;
 import org.apache.ignite.spi.*;
-import org.gridgain.grid.*;
 import org.gridgain.grid.cache.*;
 import org.apache.ignite.spi.discovery.tcp.*;
 import org.apache.ignite.spi.discovery.tcp.ipfinder.*;
@@ -606,8 +605,6 @@ public class GridCachePutAllFailoverSelfTest extends GridCommonAbstractTest {
 
             cacheCfg.setBackups(backups);
 
-            cacheCfg.setDgcSuspectLockTimeout(600000);
-            cacheCfg.setDgcFrequency(0);
             cacheCfg.setStoreValueBytes(true);
             cacheCfg.setDistributionMode(nearEnabled ? NEAR_PARTITIONED : PARTITIONED_ONLY);
             cacheCfg.setQueryIndexEnabled(false);

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/fa69fe88/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheReferenceCleanupSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheReferenceCleanupSelfTest.java b/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheReferenceCleanupSelfTest.java
index 4602445..c01bb78 100644
--- a/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheReferenceCleanupSelfTest.java
+++ b/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheReferenceCleanupSelfTest.java
@@ -56,7 +56,6 @@ public class GridCacheReferenceCleanupSelfTest extends GridCommonAbstractTest {
 
         cacheCfg.setCacheMode(mode);
         cacheCfg.setWriteSynchronizationMode(GridCacheWriteSynchronizationMode.FULL_SYNC);
-        cacheCfg.setDgcRemoveLocks(false);
         cacheCfg.setAtomicityMode(TRANSACTIONAL);
         cacheCfg.setDistributionMode(NEAR_PARTITIONED);
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/fa69fe88/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/datastructures/GridCacheQueueMultiNodeConsistencySelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/datastructures/GridCacheQueueMultiNodeConsistencySelfTest.java b/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/datastructures/GridCacheQueueMultiNodeConsistencySelfTest.java
index 54f70cf..8a12639 100644
--- a/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/datastructures/GridCacheQueueMultiNodeConsistencySelfTest.java
+++ b/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/datastructures/GridCacheQueueMultiNodeConsistencySelfTest.java
@@ -67,7 +67,6 @@ public class GridCacheQueueMultiNodeConsistencySelfTest extends GridCommonAbstra
         c.setDiscoverySpi(spi);
 
         cc.setCacheMode(PARTITIONED);
-        cc.setDgcFrequency(0);
         cc.setQueryIndexEnabled(true);
         cc.setSwapEnabled(false);
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/fa69fe88/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/datastructures/partitioned/GridCachePartitionedQueueMultiNodeSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/datastructures/partitioned/GridCachePartitionedQueueMultiNodeSelfTest.java b/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/datastructures/partitioned/GridCachePartitionedQueueMultiNodeSelfTest.java
index e42d53b..1068e41 100644
--- a/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/datastructures/partitioned/GridCachePartitionedQueueMultiNodeSelfTest.java
+++ b/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/datastructures/partitioned/GridCachePartitionedQueueMultiNodeSelfTest.java
@@ -32,7 +32,6 @@ public class GridCachePartitionedQueueMultiNodeSelfTest extends GridCacheQueueMu
         cc.setCacheMode(PARTITIONED);
         cc.setBackups(1);
         cc.setWriteSynchronizationMode(FULL_SYNC);
-        cc.setDgcFrequency(0);
         cc.setAtomicityMode(TRANSACTIONAL);
         cc.setDistributionMode(PARTITIONED_ONLY);
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/fa69fe88/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/distributed/GridCacheMultithreadedFailoverAbstractTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/distributed/GridCacheMultithreadedFailoverAbstractTest.java b/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/distributed/GridCacheMultithreadedFailoverAbstractTest.java
index cab17f0..c7bab9b 100644
--- a/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/distributed/GridCacheMultithreadedFailoverAbstractTest.java
+++ b/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/distributed/GridCacheMultithreadedFailoverAbstractTest.java
@@ -12,7 +12,6 @@ package org.gridgain.grid.kernal.processors.cache.distributed;
 import org.apache.ignite.*;
 import org.apache.ignite.configuration.*;
 import org.apache.ignite.lang.*;
-import org.gridgain.grid.*;
 import org.gridgain.grid.cache.*;
 import org.gridgain.grid.kernal.processors.cache.*;
 import org.gridgain.grid.kernal.processors.cache.distributed.dht.*;
@@ -198,8 +197,6 @@ public class GridCacheMultithreadedFailoverAbstractTest extends GridCommonAbstra
         ccfg.setEvictionPolicy(null);
         ccfg.setNearEvictionPolicy(null);
 
-        ccfg.setDgcFrequency(0);
-
         if (cacheMode() == PARTITIONED)
             ccfg.setBackups(backups());
 
@@ -243,7 +240,7 @@ public class GridCacheMultithreadedFailoverAbstractTest extends GridCommonAbstra
 
         final CyclicBarrier startBarrier = new CyclicBarrier(putThreads());
 
-        final ConcurrentHashMap<Integer, Integer> expVals = new ConcurrentHashMap<>();
+        final Map<Integer, Integer> expVals = new ConcurrentHashMap<>();
 
         final int keysPerThread = keyRange() / putThreads();
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/fa69fe88/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/distributed/GridCacheTxPessimisticOriginatingNodeFailureAbstractSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/distributed/GridCacheTxPessimisticOriginatingNodeFailureAbstractSelfTest.java b/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/distributed/GridCacheTxPessimisticOriginatingNodeFailureAbstractSelfTest.java
index ec20f23..8204e50 100644
--- a/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/distributed/GridCacheTxPessimisticOriginatingNodeFailureAbstractSelfTest.java
+++ b/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/distributed/GridCacheTxPessimisticOriginatingNodeFailureAbstractSelfTest.java
@@ -429,7 +429,6 @@ public abstract class GridCacheTxPessimisticOriginatingNodeFailureAbstractSelfTe
         GridCacheConfiguration cfg = super.cacheConfiguration(gridName);
 
         cfg.setStore(null);
-        cfg.setDgcFrequency(0);
 
         return cfg;
     }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/fa69fe88/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/distributed/dht/GridCacheDhtPreloadDelayedSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/distributed/dht/GridCacheDhtPreloadDelayedSelfTest.java b/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/distributed/dht/GridCacheDhtPreloadDelayedSelfTest.java
index ba1d3c3..9a21a5c 100644
--- a/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/distributed/dht/GridCacheDhtPreloadDelayedSelfTest.java
+++ b/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/distributed/dht/GridCacheDhtPreloadDelayedSelfTest.java
@@ -170,7 +170,7 @@ public class GridCacheDhtPreloadDelayedSelfTest extends GridCommonAbstractTest {
     }
 
     /** @throws Exception If failed. */
-    public void _testDelayedPreload() throws Exception { // TODO GG-9141
+    public void testDelayedPreload() throws Exception {
         delay = PRELOAD_DELAY;
 
         Ignite g0 = startGrid(0);

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/fa69fe88/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/distributed/near/GridCachePartitionedExplicitLockNodeFailureSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/distributed/near/GridCachePartitionedExplicitLockNodeFailureSelfTest.java b/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/distributed/near/GridCachePartitionedExplicitLockNodeFailureSelfTest.java
index 64f13d1..fd9caf8 100644
--- a/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/distributed/near/GridCachePartitionedExplicitLockNodeFailureSelfTest.java
+++ b/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/distributed/near/GridCachePartitionedExplicitLockNodeFailureSelfTest.java
@@ -53,7 +53,6 @@ public class GridCachePartitionedExplicitLockNodeFailureSelfTest extends GridCom
         GridCacheConfiguration cc = defaultCacheConfiguration();
 
         cc.setCacheMode(PARTITIONED);
-        cc.setDgcFrequency(0);
         cc.setWriteSynchronizationMode(GridCacheWriteSynchronizationMode.FULL_SYNC);
         cc.setBackups(GRID_CNT - 1);
         cc.setAtomicityMode(TRANSACTIONAL);

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/fa69fe88/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/distributed/near/GridCachePartitionedMultiNodeLockSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/distributed/near/GridCachePartitionedMultiNodeLockSelfTest.java b/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/distributed/near/GridCachePartitionedMultiNodeLockSelfTest.java
index ba4e427..f04e2db 100644
--- a/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/distributed/near/GridCachePartitionedMultiNodeLockSelfTest.java
+++ b/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/distributed/near/GridCachePartitionedMultiNodeLockSelfTest.java
@@ -30,7 +30,6 @@ public class GridCachePartitionedMultiNodeLockSelfTest extends GridCacheMultiNod
 
         cc.setCacheMode(PARTITIONED);
         cc.setBackups(2); // 2 backups, so all nodes are involved.
-        cc.setDgcFrequency(0);
         cc.setAtomicityMode(TRANSACTIONAL);
         cc.setDistributionMode(NEAR_PARTITIONED);
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/fa69fe88/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/distributed/near/GridCachePartitionedNodeFailureSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/distributed/near/GridCachePartitionedNodeFailureSelfTest.java b/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/distributed/near/GridCachePartitionedNodeFailureSelfTest.java
index 4ea7431..da1ee9b 100644
--- a/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/distributed/near/GridCachePartitionedNodeFailureSelfTest.java
+++ b/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/distributed/near/GridCachePartitionedNodeFailureSelfTest.java
@@ -30,7 +30,6 @@ public class GridCachePartitionedNodeFailureSelfTest extends GridCacheNodeFailur
         GridCacheConfiguration cc = defaultCacheConfiguration();
 
         cc.setCacheMode(PARTITIONED);
-        cc.setDgcFrequency(0);
         cc.setWriteSynchronizationMode(GridCacheWriteSynchronizationMode.FULL_SYNC);
         cc.setAtomicityMode(TRANSACTIONAL);
         cc.setDistributionMode(NEAR_PARTITIONED);

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/fa69fe88/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/distributed/near/GridCachePartitionedNodeRestartTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/distributed/near/GridCachePartitionedNodeRestartTest.java b/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/distributed/near/GridCachePartitionedNodeRestartTest.java
index e8d0b41..aed997d 100644
--- a/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/distributed/near/GridCachePartitionedNodeRestartTest.java
+++ b/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/distributed/near/GridCachePartitionedNodeRestartTest.java
@@ -40,7 +40,6 @@ public class GridCachePartitionedNodeRestartTest extends GridCacheAbstractNodeRe
         cc.setPreloadBatchSize(preloadBatchSize);
         cc.setAffinity(new GridCacheConsistentHashAffinityFunction(false, partitions));
         cc.setBackups(backups);
-        cc.setDgcFrequency(0);
 
         c.setCacheConfiguration(cc);
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/fa69fe88/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/distributed/near/GridCachePartitionedOptimisticTxNodeRestartTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/distributed/near/GridCachePartitionedOptimisticTxNodeRestartTest.java b/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/distributed/near/GridCachePartitionedOptimisticTxNodeRestartTest.java
index 90baf34..cf551a3 100644
--- a/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/distributed/near/GridCachePartitionedOptimisticTxNodeRestartTest.java
+++ b/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/distributed/near/GridCachePartitionedOptimisticTxNodeRestartTest.java
@@ -38,7 +38,6 @@ public class GridCachePartitionedOptimisticTxNodeRestartTest extends GridCacheAb
         cc.setPreloadBatchSize(preloadBatchSize);
         cc.setAffinity(new GridCacheConsistentHashAffinityFunction(false, partitions));
         cc.setBackups(backups);
-        cc.setDgcFrequency(0);
 
         c.setCacheConfiguration(cc);
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/fa69fe88/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/distributed/near/GridCachePartitionedTxSalvageSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/distributed/near/GridCachePartitionedTxSalvageSelfTest.java b/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/distributed/near/GridCachePartitionedTxSalvageSelfTest.java
index 3ba8a10..7dee9bf 100644
--- a/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/distributed/near/GridCachePartitionedTxSalvageSelfTest.java
+++ b/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/distributed/near/GridCachePartitionedTxSalvageSelfTest.java
@@ -12,7 +12,6 @@ package org.gridgain.grid.kernal.processors.cache.distributed.near;
 import org.apache.ignite.*;
 import org.apache.ignite.configuration.*;
 import org.apache.ignite.lang.*;
-import org.gridgain.grid.*;
 import org.gridgain.grid.cache.*;
 import org.gridgain.grid.cache.affinity.consistenthash.*;
 import org.gridgain.grid.kernal.*;
@@ -71,7 +70,6 @@ public class GridCachePartitionedTxSalvageSelfTest extends GridCommonAbstractTes
         cc.setAffinity(new GridCacheConsistentHashAffinityFunction(false, 18));
         cc.setBackups(1);
         cc.setPreloadMode(GridCachePreloadMode.SYNC);
-        cc.setDgcFrequency(0);
 
         c.setCacheConfiguration(cc);
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/fa69fe88/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/distributed/near/GridCachePutArrayValueSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/distributed/near/GridCachePutArrayValueSelfTest.java b/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/distributed/near/GridCachePutArrayValueSelfTest.java
index 1785cb5..e1ce455 100644
--- a/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/distributed/near/GridCachePutArrayValueSelfTest.java
+++ b/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/distributed/near/GridCachePutArrayValueSelfTest.java
@@ -10,7 +10,6 @@
 package org.gridgain.grid.kernal.processors.cache.distributed.near;
 
 import org.gridgain.grid.cache.*;
-import org.gridgain.grid.cache.affinity.consistenthash.*;
 import org.gridgain.grid.kernal.processors.cache.*;
 import org.gridgain.grid.util.typedef.internal.*;
 
@@ -33,7 +32,6 @@ public class GridCachePutArrayValueSelfTest extends GridCacheAbstractSelfTest {
 
         cacheCfg.setCacheMode(PARTITIONED);
         cacheCfg.setBackups(1);
-        cacheCfg.setDgcFrequency(0);
 
         return cacheCfg;
     }


[3/3] incubator-ignite git commit: GG-9141 - Fixing failover tests.

Posted by ag...@apache.org.
GG-9141 - Fixing failover tests.


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

Branch: refs/heads/ignite-1
Commit: 03c17b8f3caaa45fd3554a5a306d6e431d74e0ca
Parents: cbd30af
Author: Alexey Goncharuk <ag...@gridgain.com>
Authored: Mon Dec 15 18:48:04 2014 -0800
Committer: Alexey Goncharuk <ag...@gridgain.com>
Committed: Mon Dec 15 18:48:04 2014 -0800

----------------------------------------------------------------------
 .../processors/cache/GridCachePartitionExchangeManager.java   | 7 ++++++-
 .../grid/kernal/processors/cache/GridCacheTxLocalAdapter.java | 2 +-
 .../testsuites/bamboo/GridCacheFailoverTestSuite.java         | 4 ----
 3 files changed, 7 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/03c17b8f/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/GridCachePartitionExchangeManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/GridCachePartitionExchangeManager.java b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/GridCachePartitionExchangeManager.java
index b0c319d..201b960 100644
--- a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/GridCachePartitionExchangeManager.java
+++ b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/GridCachePartitionExchangeManager.java
@@ -470,6 +470,9 @@ public class GridCachePartitionExchangeManager<K, V> extends GridCacheSharedMana
                 m.addFullPartitionsMap(cacheCtx.cacheId(), cacheCtx.topology().partitionMap(true));
         }
 
+        for (GridClientPartitionTopology<K, V> top : cctx.exchange().clientTopologies())
+            m.addFullPartitionsMap(top.cacheId(), top.partitionMap(true));
+
         if (log.isDebugEnabled())
             log.debug("Sending all partitions [nodeIds=" + U.nodeIds(nodes) + ", msg=" + m + ']');
 
@@ -587,7 +590,9 @@ public class GridCachePartitionExchangeManager<K, V> extends GridCacheSharedMana
                     if (!cacheCtx.isLocal()) {
                         GridDhtPartitionTopology<K, V> top = cacheCtx.topology();
 
-                        updated |= top.update(null, msg.partitions().get(cacheCtx.cacheId())) != null;
+                        GridDhtPartitionFullMap partMap = msg.partitions().get(cacheCtx.cacheId());
+
+                        updated |= top.update(null, partMap) != null;
                     }
                 }
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/03c17b8f/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 b7fa4b2..abb9fef 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
@@ -2093,7 +2093,7 @@ public abstract class GridCacheTxLocalAdapter<K, V> extends GridCacheTxAdapter<K
                     if (log.isDebugEnabled())
                         log.debug("Got removed entry in putAllAsync method (will retry): " + cached);
 
-                    txEntry.cached(entryEx(cacheCtx, txEntry.txKey()), txEntry.keyBytes());
+                    txEntry.cached(entryEx(cached.context(), txEntry.txKey()), txEntry.keyBytes());
                 }
             }
         }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/03c17b8f/modules/core/src/test/java/org/gridgain/testsuites/bamboo/GridCacheFailoverTestSuite.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/gridgain/testsuites/bamboo/GridCacheFailoverTestSuite.java b/modules/core/src/test/java/org/gridgain/testsuites/bamboo/GridCacheFailoverTestSuite.java
index 65c6d07..7584e00 100644
--- a/modules/core/src/test/java/org/gridgain/testsuites/bamboo/GridCacheFailoverTestSuite.java
+++ b/modules/core/src/test/java/org/gridgain/testsuites/bamboo/GridCacheFailoverTestSuite.java
@@ -26,10 +26,6 @@ public class GridCacheFailoverTestSuite extends TestSuite {
     public static TestSuite suite() throws Exception {
         TestSuite suite = new TestSuite("Gridgain Cache Failover Test Suite");
 
-        // TODO GG-9141
-        if (true)
-            return suite;
-
         suite.addTestSuite(GridCacheAtomicInvalidPartitionHandlingSelfTest.class);
 
         // Group lock failover.


[2/3] incubator-ignite git commit: GG-9141 - Fixed exchange future.

Posted by ag...@apache.org.
GG-9141 - Fixed exchange future.


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

Branch: refs/heads/ignite-1
Commit: cbd30af3032f123b458437526ef4e5cef1512542
Parents: fa69fe8
Author: Alexey Goncharuk <ag...@gridgain.com>
Authored: Mon Dec 15 16:27:32 2014 -0800
Committer: Alexey Goncharuk <ag...@gridgain.com>
Committed: Mon Dec 15 16:27:32 2014 -0800

----------------------------------------------------------------------
 .../processors/cache/GridCacheStoreManager.java    | 11 +++--------
 .../preloader/GridDhtPartitionsExchangeFuture.java |  6 ++++--
 .../GridCacheDhtPreloadMultiThreadedSelfTest.java  | 17 +++++------------
 3 files changed, 12 insertions(+), 22 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/cbd30af3/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/GridCacheStoreManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/GridCacheStoreManager.java b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/GridCacheStoreManager.java
index f8b7623..e80f8b7 100644
--- a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/GridCacheStoreManager.java
+++ b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/GridCacheStoreManager.java
@@ -167,8 +167,7 @@ public class GridCacheStoreManager<K, V> extends GridCacheManagerAdapter<K, V> {
      * @return Whether DHT transaction can write to store from DHT.
      */
     public boolean writeToStoreFromDht() {
-        // TODO GG-9141
-        return false;
+        return cctx.config().isWriteBehindEnabled() || locStore;
     }
 
     /**
@@ -193,9 +192,7 @@ public class GridCacheStoreManager<K, V> extends GridCacheManagerAdapter<K, V> {
                     return true;
                 }
 
-                Collection<? extends K> keys0;
-
-                keys0 = convertPortable ?
+                Collection<? extends K> keys0 = convertPortable ?
                     F.viewReadOnly(keys, new C1<K, K>() {
                         @Override public K apply(K k) {
                             return (K)cctx.unwrapPortableIfNeeded(k, false);
@@ -443,9 +440,7 @@ public class GridCacheStoreManager<K, V> extends GridCacheManagerAdapter<K, V> {
         }
 
         if (store != null) {
-            Collection<? extends K> keys0;
-
-            keys0 = convertPortable ?
+            Collection<? extends K> keys0 = convertPortable ?
                 F.viewReadOnly(keys, new C1<K, K>() {
                     @Override public K apply(K k) {
                         return (K)cctx.unwrapPortableIfNeeded(k, false);

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/cbd30af3/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/distributed/dht/preloader/GridDhtPartitionsExchangeFuture.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/distributed/dht/preloader/GridDhtPartitionsExchangeFuture.java b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/distributed/dht/preloader/GridDhtPartitionsExchangeFuture.java
index 0a95411..65b4522 100644
--- a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/distributed/dht/preloader/GridDhtPartitionsExchangeFuture.java
+++ b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/distributed/dht/preloader/GridDhtPartitionsExchangeFuture.java
@@ -912,8 +912,10 @@ public class GridDhtPartitionsExchangeFuture<K, V> extends GridFutureAdapter<Lon
                                         // If local node is just joining.
                                         if (exchId.nodeId().equals(cctx.localNodeId())) {
                                             try {
-                                                for (GridCacheContext<K, V> cacheCtx : cctx.cacheContexts())
-                                                    cacheCtx.topology().beforeExchange(exchId);
+                                                for (GridCacheContext<K, V> cacheCtx : cctx.cacheContexts()) {
+                                                    if (!cacheCtx.isLocal())
+                                                        cacheCtx.topology().beforeExchange(exchId);
+                                                }
                                             }
                                             catch (IgniteCheckedException e) {
                                                 onDone(e);

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/cbd30af3/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/distributed/dht/GridCacheDhtPreloadMultiThreadedSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/distributed/dht/GridCacheDhtPreloadMultiThreadedSelfTest.java b/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/distributed/dht/GridCacheDhtPreloadMultiThreadedSelfTest.java
index 5342bc8..3277c68 100644
--- a/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/distributed/dht/GridCacheDhtPreloadMultiThreadedSelfTest.java
+++ b/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/distributed/dht/GridCacheDhtPreloadMultiThreadedSelfTest.java
@@ -32,9 +32,6 @@ public class GridCacheDhtPreloadMultiThreadedSelfTest extends GridCommonAbstract
     /** IP finder. */
     private static final TcpDiscoveryIpFinder IP_FINDER = new TcpDiscoveryVmIpFinder(true);
 
-    /** */
-    private boolean cacheEnabled = true;
-
     /**
      * Creates new test.
      */
@@ -126,7 +123,7 @@ public class GridCacheDhtPreloadMultiThreadedSelfTest extends GridCommonAbstract
     /**
      * @throws Exception If failed.
      */
-    public void _testConcurrentNodesStartStop() throws Exception { // TODO GG-9141
+    public void testConcurrentNodesStartStop() throws Exception {
         try {
             multithreadedAsync(
                 new Callable<Object>() {
@@ -156,16 +153,12 @@ public class GridCacheDhtPreloadMultiThreadedSelfTest extends GridCommonAbstract
 
         cfg.setGridName(gridName);
 
-        if (cacheEnabled) {
-            for (GridCacheConfiguration cCfg : cfg.getCacheConfiguration()) {
-                if (cCfg.getCacheMode() == GridCacheMode.PARTITIONED) {
-                    cCfg.setAffinity(new GridCacheConsistentHashAffinityFunction(2048, null));
-                    cCfg.setBackups(1);
-                }
+        for (GridCacheConfiguration cCfg : cfg.getCacheConfiguration()) {
+            if (cCfg.getCacheMode() == GridCacheMode.PARTITIONED) {
+                cCfg.setAffinity(new GridCacheConsistentHashAffinityFunction(2048, null));
+                cCfg.setBackups(1);
             }
         }
-        else
-            cfg.setCacheConfiguration();
 
         ((TcpDiscoverySpi)cfg.getDiscoverySpi()).setIpFinder(IP_FINDER);