Merge branch sprint-2 into ignite-45

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

Branch: refs/heads/ignite-45
Commit: f0a805ff5715dc251cc17d4d66e091c69efa1e58
Parents: 3eb79df 7972da9
Author: Alexey Goncharuk <agoncha...@gridgain.com>
Authored: Tue Mar 17 19:37:27 2015 -0700
Committer: Alexey Goncharuk <agoncha...@gridgain.com>
Committed: Tue Mar 17 19:37:27 2015 -0700

----------------------------------------------------------------------
 .../apache/ignite/gridify/AbstractAopTest.java  |   6 +-
 .../rest/AbstractRestProcessorSelfTest.java     |   2 +-
 .../ignite/jdbc/JdbcEmptyCacheSelfTest.java     |   3 +
 modules/core/pom.xml                            | 105 ++++++++
 .../src/main/java/org/apache/ignite/Ignite.java |   1 +
 .../java/org/apache/ignite/IgniteCache.java     |  38 +--
 .../ignite/cache/CacheEntryProcessor.java       |  28 ++
 .../java/org/apache/ignite/cache/GridCache.java | 226 ----------------
 .../ignite/cache/IgniteEntryProcessor.java      |  28 --
 .../apache/ignite/cache/IgniteImmutable.java    |  45 ----
 .../cache/affinity/CacheAffinityKeyMapped.java  |  12 +-
 .../ignite/cache/query/SqlFieldsQuery.java      |   7 +
 .../apache/ignite/cache/query/TextQuery.java    |   7 +
 .../cache/store/CacheLoadOnlyStoreAdapter.java  |   6 +-
 .../configuration/CacheConfiguration.java       |   2 +-
 .../ignite/events/CacheQueryExecutedEvent.java  |   2 -
 .../ignite/internal/GridKernalContextImpl.java  |   7 +-
 .../org/apache/ignite/internal/IgniteEx.java    |  10 +-
 .../apache/ignite/internal/IgniteKernal.java    |   4 +-
 .../ignite/internal/MarshallerContextImpl.java  |  30 ++-
 .../internal/managers/GridManagerAdapter.java   |   6 -
 .../processors/cache/CacheObjectImpl.java       |   6 +
 .../internal/processors/cache/GridCache.java    | 227 ++++++++++++++++
 .../processors/cache/GridCacheAdapter.java      |   6 +-
 .../cache/GridCacheConcurrentMap.java           |   2 +-
 .../processors/cache/GridCacheStoreManager.java |  30 ++-
 .../cache/GridCacheWriteBehindStore.java        |  69 +++--
 .../processors/cache/IgniteCacheProxy.java      |  22 +-
 .../processors/cache/QueryCursorImpl.java       |  33 ++-
 .../cache/query/GridCacheQueryManager.java      |   1 -
 .../processors/cache/query/QueryCursorEx.java   |  46 ++++
 .../continuous/CacheContinuousQueryHandler.java |   3 +
 .../continuous/CacheContinuousQueryManager.java |  11 +-
 .../jdbc/GridCacheQueryJdbcMetadataTask.java    |   2 +-
 .../query/jdbc/GridCacheQueryJdbcTask.java      | 242 ++++++++++-------
 .../cacheobject/IgniteCacheObjectProcessor.java |   2 +-
 .../IgniteCacheObjectProcessorImpl.java         |  85 +++---
 .../datastreamer/DataStreamerCacheUpdaters.java |  12 +-
 .../datastructures/DataStructuresProcessor.java |   1 -
 .../datastructures/GridCacheSetImpl.java        |   1 -
 .../processors/query/GridQueryProcessor.java    |  14 +-
 .../processors/task/GridTaskProcessor.java      |   2 +-
 .../apache/ignite/internal/util/GridTimer.java  |   2 +-
 .../internal/util/IgniteExceptionRegistry.java  | 164 +++++------
 .../internal/util/io/GridFilenameUtils.java     |   1 -
 .../apache/ignite/internal/visor/VisorJob.java  |   3 +
 .../internal/visor/VisorMultiNodeTask.java      |   1 +
 .../visor/cache/VisorCacheMetadataTask.java     |   1 +
 .../internal/visor/cache/VisorCacheMetrics.java |   1 +
 .../cache/VisorCacheMetricsCollectorTask.java   |   1 +
 .../visor/cache/VisorCacheRebalanceTask.java    |   1 +
 .../visor/cache/VisorCacheResetMetricsTask.java |   1 +
 .../visor/cache/VisorCacheSwapBackupsTask.java  |   1 +
 .../compute/VisorComputeMonitoringHolder.java   |  10 +-
 .../visor/node/VisorNodeDataCollectorJob.java   |   4 +-
 .../node/VisorNodeDataCollectorJobResult.java   |  17 ++
 .../visor/node/VisorNodeDataCollectorTask.java  |   2 +
 .../node/VisorNodeDataCollectorTaskResult.java  |  10 +
 .../node/VisorNodeSuppressedErrorsTask.java     |  99 +++++++
 .../internal/visor/query/VisorQueryTask.java    |   4 +-
 .../org/apache/ignite/lang/IgniteImmutable.java |  49 ++++
 .../org/apache/ignite/lang/IgnitePredicate.java |   4 +-
 .../optimized/OptimizedMarshallerUtils.java     |  16 +-
 .../org/apache/ignite/spi/IgniteSpiAdapter.java |   7 +-
 .../org/apache/ignite/spi/IgniteSpiContext.java |   8 -
 .../TcpDiscoveryMulticastIpFinder.java          |   4 +-
 .../ignite/startup/BasicWarmupClosure.java      |   1 +
 .../StreamerCacheAffinityEventRouter.java       |   2 +-
 .../TransactionSynchronization.java             |   2 +-
 .../resources/META-INF/classnames.properties    |   5 +-
 modules/core/src/test/java/ExcludeList          |   4 +
 .../GridCacheAffinityBackupsSelfTest.java       |   2 +
 .../ignite/igfs/IgfsFragmentizerSelfTest.java   |   1 +
 .../cache/GridCacheAbstractFullApiSelfTest.java |   8 +-
 .../cache/GridCacheClearLocallySelfTest.java    |   1 -
 ...CacheFullTextQueryMultithreadedSelfTest.java |   1 -
 .../GridCacheReturnValueTransferSelfTest.java   |   1 +
 ...idCacheWriteBehindStoreAbstractSelfTest.java |   2 +-
 .../GridCacheAbstractJobExecutionTest.java      |   9 +-
 .../GridCacheAtomicTimeoutSelfTest.java         |   5 +
 .../GridCacheNodeFailureAbstractTest.java       |   1 +
 ...GridCachePreloadRestartAbstractSelfTest.java |   1 +
 .../IgniteTxTimeoutAbstractTest.java            |   2 +-
 .../dht/GridCacheDhtPreloadSelfTest.java        |   1 +
 .../near/GridCacheNearEvictionSelfTest.java     |   1 +
 .../near/GridCacheNearMultiNodeSelfTest.java    |   1 +
 ...GridCachePartitionedFilteredPutSelfTest.java |   1 +
 ...achePartitionedPreloadLifecycleSelfTest.java |   1 +
 .../GridCacheReplicatedInvalidateSelfTest.java  |   2 +-
 ...CacheReplicatedPreloadLifecycleSelfTest.java |   1 +
 ...eCacheAtomicStoreSessionWriteBehindTest.java |  38 +++
 ...acheStoreSessionWriteBehindAbstractTest.java | 269 +++++++++++++++++++
 ...gniteCacheTxStoreSessionWriteBehindTest.java |  32 +++
 .../GridCacheSwapScanQueryAbstractSelfTest.java |   1 +
 ...ridCacheContinuousQueryAbstractSelfTest.java |  66 ++++-
 .../processors/igfs/IgfsAbstractSelfTest.java   |   1 +
 .../igfs/IgfsDualAbstractSelfTest.java          |   1 +
 .../processors/igfs/IgfsProcessorSelfTest.java  |   1 +
 .../processors/igfs/IgfsStreamsSelfTest.java    |   1 +
 .../util/IgniteExceptionRegistrySelfTest.java   |  51 ++--
 .../OptimizedMarshallerNodeFailoverTest.java    | 200 ++++++++++++++
 .../testframework/GridSpiTestContext.java       |   5 -
 .../junits/GridTestKernalContext.java           |   2 -
 .../ignite/testsuites/IgniteCacheTestSuite.java |   2 +
 .../IgniteMarshallerSelfTestSuite.java          |   1 +
 .../query/h2/GridH2IndexingGeoSelfTest.java     |   1 -
 .../igfs/HadoopIgfsDualAbstractSelfTest.java    |   1 +
 .../HadoopDefaultMapReducePlannerSelfTest.java  |   6 +-
 .../HibernateAccessStrategyAdapter.java         |   1 -
 .../hibernate/HibernateCollectionRegion.java    |   2 +-
 .../cache/hibernate/HibernateEntityRegion.java  |   2 +-
 .../hibernate/HibernateGeneralDataRegion.java   |   2 +-
 .../hibernate/HibernateNaturalIdRegion.java     |   2 +-
 .../HibernateNonStrictAccessStrategy.java       |   2 +-
 .../hibernate/HibernateQueryResultsRegion.java  |   2 +-
 .../HibernateReadOnlyAccessStrategy.java        |   2 +-
 .../HibernateReadWriteAccessStrategy.java       |  34 +--
 .../ignite/cache/hibernate/HibernateRegion.java |   2 +-
 .../cache/hibernate/HibernateRegionFactory.java |   6 +-
 .../hibernate/HibernateTimestampsRegion.java    |   2 +-
 .../HibernateTransactionalAccessStrategy.java   |   4 +-
 .../HibernateTransactionalDataRegion.java       |   2 +-
 .../hibernate/HibernateL2CacheSelfTest.java     |   2 +
 .../processors/query/h2/IgniteH2Indexing.java   | 143 ++++------
 .../query/h2/sql/GridSqlQueryParser.java        |  33 ++-
 .../query/h2/sql/GridSqlQuerySplitter.java      |  19 +-
 .../h2/twostep/GridReduceQueryExecutor.java     |  47 ++--
 .../cache/GridCacheQuerySimpleBenchmark.java    | 196 ++++++++++++++
 ...hePartitionedQueryMultiThreadedSelfTest.java |   2 +
 ...dCacheAbstractReduceFieldsQuerySelfTest.java |   1 -
 .../query/h2/sql/GridQueryParsingTest.java      |   1 +
 .../scalar/src/test/resources/spring-cache.xml  |  70 +++--
 .../tests/ScalarAffinityRoutingSpec.scala       |   6 +-
 .../ignite/scalar/tests/ScalarCacheSpec.scala   |   4 +-
 .../schema-import/src/main/java/media/style.css |   2 +-
 .../schema/generator/SnippetGenerator.java      |  10 +-
 .../ignite/cache/spring/SpringCacheManager.java |   2 +-
 .../commands/cache/VisorCacheCommand.scala      |   8 +-
 .../commands/cache/VisorCacheCommandSpec.scala  |  11 +-
 139 files changed, 2232 insertions(+), 928 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/f0a805ff/modules/core/src/main/java/org/apache/ignite/Ignite.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/f0a805ff/modules/core/src/main/java/org/apache/ignite/configuration/CacheConfiguration.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/f0a805ff/modules/core/src/main/java/org/apache/ignite/internal/GridKernalContextImpl.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/f0a805ff/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/f0a805ff/modules/core/src/main/java/org/apache/ignite/internal/managers/GridManagerAdapter.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/f0a805ff/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCache.java
----------------------------------------------------------------------
diff --cc 
modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCache.java
index 0000000,7dbc05a..816a817
mode 000000,100644..100644
--- 
a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCache.java
+++ 
b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCache.java
@@@ -1,0 -1,222 +1,227 @@@
+ /*
+  * Licensed to the Apache Software Foundation (ASF) under one or more
+  * contributor license agreements.  See the NOTICE file distributed with
+  * this work for additional information regarding copyright ownership.
+  * The ASF licenses this file to You under the Apache License, Version 2.0
+  * (the "License"); you may not use this file except in compliance with
+  * the License.  You may obtain a copy of the License at
+  *
+  *      http://www.apache.org/licenses/LICENSE-2.0
+  *
+  * Unless required by applicable law or agreed to in writing, software
+  * distributed under the License is distributed on an "AS IS" BASIS,
+  * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  * See the License for the specific language governing permissions and
+  * limitations under the License.
+  */
+ 
+ package org.apache.ignite.internal.processors.cache;
+ 
+ import org.apache.ignite.*;
+ import org.apache.ignite.cache.*;
+ import org.apache.ignite.cache.affinity.*;
+ import org.apache.ignite.configuration.*;
+ import org.apache.ignite.internal.*;
+ import org.apache.ignite.internal.processors.cache.*;
+ import org.apache.ignite.mxbean.*;
+ import org.apache.ignite.transactions.*;
+ import org.jetbrains.annotations.*;
+ 
+ import javax.cache.*;
+ import java.util.*;
+ 
+ /**
+  * Main entry point for all <b>Data Grid APIs.</b> You can get a named cache 
by calling {@link org.apache.ignite.Ignite#cache(String)}
+  * method.
+  * <h1 class="header">Functionality</h1>
+  * This API extends {@link CacheProjection} API which contains vast majority 
of cache functionality
+  * and documentation. In addition to {@link CacheProjection} functionality 
this API provides:
+  * <ul>
+  * <li>
+  *  Various {@code 'loadCache(..)'} methods to load cache either 
synchronously or asynchronously.
+  *  These methods don't specify any keys to load, and leave it to the 
underlying storage to load cache
+  *  data based on the optionally passed in arguments.
+  * </li>
+  * <li>
+  *     Method {@link #affinity()} provides {@link 
org.apache.ignite.cache.affinity.CacheAffinityFunction} service for information 
on
+  *     data partitioning and mapping keys to grid nodes responsible for 
caching those keys.
+  * </li>
+  * <li>
+  *  Methods like {@code 'tx{Un}Synchronize(..)'} witch allow to get 
notifications for transaction state changes.
+  *  This feature is very useful when integrating cache transactions with some 
other in-house transactions.
+  * </li>
+  * <li>Method {@link #metrics()} to provide metrics for the whole cache.</li>
+  * <li>Method {@link #configuration()} to provide cache configuration 
bean.</li>
+  * </ul>
+  *
+  * @param <K> Cache key type.
+  * @param <V> Cache value type.
+  */
+ public interface GridCache<K, V> extends CacheProjection<K, V> {
+     /**
+      * Gets configuration bean for this cache.
+      *
+      * @return Configuration bean for this cache.
+      */
+     public CacheConfiguration configuration();
+ 
+     /**
+      * Registers transactions synchronizations for all transactions started 
by this cache.
+      * Use it whenever you need to get notifications on transaction lifecycle 
and possibly change
+      * its course. It is also particularly useful when integrating cache 
transactions
+      * with some other in-house transactions.
+      *
+      * @param syncs Transaction synchronizations to register.
+      */
+     public void txSynchronize(@Nullable TransactionSynchronization syncs);
+ 
+     /**
+      * Removes transaction synchronizations.
+      *
+      * @param syncs Transactions synchronizations to remove.
+      * @see #txSynchronize(TransactionSynchronization)
+      */
+     public void txUnsynchronize(@Nullable TransactionSynchronization syncs);
+ 
+     /**
+      * Gets registered transaction synchronizations.
+      *
+      * @return Registered transaction synchronizations.
+      * @see #txSynchronize(TransactionSynchronization)
+      */
+     public Collection<TransactionSynchronization> txSynchronizations();
+ 
+     /**
+      * Gets affinity service to provide information about data partitioning
+      * and distribution.
+      *
+      * @return Cache data affinity service.
+      */
+     public CacheAffinity<K> affinity();
+ 
+     /**
+      * Gets metrics (statistics) for this cache.
+      *
+      * @return Cache metrics.
+      */
+     public CacheMetrics metrics();
+ 
+     /**
+      * Gets metrics (statistics) for this cache.
+      *
+      * @return Cache metrics.
+      */
+     public CacheMetricsMXBean mxBean();
+ 
+     /**
+      * Gets size (in bytes) of all entries swapped to disk.
+      *
+      * @return Size (in bytes) of all entries swapped to disk.
+      * @throws IgniteCheckedException In case of error.
+      */
+     public long overflowSize() throws IgniteCheckedException;
+ 
+     /**
+      * Gets number of cache entries stored in off-heap memory.
+      *
+      * @return Number of cache entries stored in off-heap memory.
+      */
+     public long offHeapEntriesCount();
+ 
+     /**
+      * Gets memory size allocated in off-heap.
+      *
+      * @return Allocated memory size.
+      */
+     public long offHeapAllocatedSize();
+ 
+     /**
+      * Gets size in bytes for swap space.
+      *
+      * @return Size in bytes.
+      * @throws IgniteCheckedException If failed.
+      */
+     public long swapSize() throws IgniteCheckedException;
+ 
+     /**
+      * Gets number of swap entries (keys).
+      *
+      * @return Number of entries stored in swap.
+      * @throws IgniteCheckedException If failed.
+      */
+     public long swapKeys() throws IgniteCheckedException;
+ 
+     /**
+      * Gets iterator over keys and values belonging to this cache swap space 
on local node. This
+      * iterator is thread-safe, which means that cache (and therefore its 
swap space)
+      * may be modified concurrently with iteration over swap.
+      * <p>
+      * Returned iterator supports {@code remove} operation which delegates to
+      * {@link #removex(Object, org.apache.ignite.lang.IgnitePredicate[])} 
method.
+      * <h2 class="header">Cache Flags</h2>
+      * This method is not available if any of the following flags are set on 
projection:
+      * {@link 
org.apache.ignite.internal.processors.cache.CacheFlag#SKIP_SWAP}.
+      *
+      * @return Iterator over keys.
+      * @throws IgniteCheckedException If failed.
+      * @see #promote(Object)
+      */
+     public Iterator<Map.Entry<K, V>> swapIterator() throws 
IgniteCheckedException;
+ 
+     /**
+      * Gets iterator over keys and values belonging to this cache off-heap 
memory on local node. This
+      * iterator is thread-safe, which means that cache (and therefore its 
off-heap memory)
+      * may be modified concurrently with iteration over off-heap. To achieve 
better performance
+      * the keys and values deserialized on demand, whenever accessed.
+      * <p>
+      * Returned iterator supports {@code remove} operation which delegates to
+      * {@link #removex(Object, org.apache.ignite.lang.IgnitePredicate[])} 
method.
+      *
+      * @return Iterator over keys.
+      * @throws IgniteCheckedException If failed.
+      */
+     public Iterator<Map.Entry<K, V>> offHeapIterator() throws 
IgniteCheckedException;
+ 
+     /**
+      * Gets a random entry out of cache. In the worst cache scenario this 
method
+      * has complexity of <pre>O(S * N/64)</pre> where {@code N} is the size 
of internal hash
+      * table and {@code S} is the number of hash table buckets to sample, 
which is {@code 5}
+      * by default. However, if the table is pretty dense, with density factor 
of {@code N/64},
+      * which is true for near fully populated caches, this method will 
generally perform significantly
+      * faster with complexity of O(S) where {@code S = 5}.
+      * <p>
+      * Note that this method is not available on {@link CacheProjection} API 
since it is
+      * impossible (or very hard) to deterministically return a number value 
when pre-filtering
+      * and post-filtering is involved (e.g. projection level predicate 
filters).
+      *
+      * @return Random entry, or {@code null} if cache is empty.
+      */
+     @Nullable public Cache.Entry<K, V> randomEntry();
+ 
+     /**
+      * Forces this cache node to re-balance its partitions. This method is 
usually used when
+      * {@link CacheConfiguration#getRebalanceDelay()} configuration parameter 
has non-zero value.
+      * When many nodes are started or stopped almost concurrently, it is more 
efficient to delay
+      * rebalancing until the node topology is stable to make sure that no 
redundant re-partitioning
+      * happens.
+      * <p>
+      * In case of{@link CacheMode#PARTITIONED} caches, for better efficiency 
user should
+      * usually make sure that new nodes get placed on the same place of 
consistent hash ring as
+      * the left nodes, and that nodes are restarted before
+      * {@link CacheConfiguration#getRebalanceDelay() rebalanceDelay} expires. 
To place nodes
+      * on the same place in consistent hash ring, use
+      * {@link 
org.apache.ignite.cache.affinity.rendezvous.CacheRendezvousAffinityFunction#setHashIdResolver(CacheAffinityNodeHashResolver)}
 to make sure that
+      * a node maps to the same hash ID if re-started.
+      * <p>
+      * See {@link 
org.apache.ignite.configuration.CacheConfiguration#getRebalanceDelay()} for 
more information on how to configure
+      * rebalance re-partition delay.
+      * <p>
+      * @return Future that will be completed when rebalancing is finished.
+      */
+     public IgniteInternalFuture<?> forceRepartition();
++
++    /**
++     * @return {@code True} if local node is affinity node.
++     */
++    public boolean affinityNode();
+ }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/f0a805ff/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java
----------------------------------------------------------------------
diff --cc 
modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java
index c124dc2,02501c2..1e8a580
--- 
a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java
+++ 
b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java
@@@ -4217,9 -4217,11 +4217,11 @@@ public abstract class GridCacheAdapter<
  
                  try {
                      removex(item.getKey());
-                 } catch (IgniteCheckedException e) {
 -                }
++                } 
+                 catch (IgniteCheckedException e) {
 -                    throw new CacheException(e);
 -                }
 +                    throw CU.convertToCacheException(e);
-                 } finally {
++                } 
+                 finally {
                      ctx.gate().leave();
                  }
              }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/f0a805ff/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheConcurrentMap.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/f0a805ff/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheProxy.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/f0a805ff/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryManager.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/f0a805ff/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryHandler.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/f0a805ff/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryManager.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/f0a805ff/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/jdbc/GridCacheQueryJdbcMetadataTask.java
----------------------------------------------------------------------
diff --cc 
modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/jdbc/GridCacheQueryJdbcMetadataTask.java
index 48a993a,014192b..2013757
--- 
a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/jdbc/GridCacheQueryJdbcMetadataTask.java
+++ 
b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/jdbc/GridCacheQueryJdbcMetadataTask.java
@@@ -21,8 -21,7 +21,8 @@@ import org.apache.ignite.*
  import org.apache.ignite.cluster.*;
  import org.apache.ignite.compute.*;
  import org.apache.ignite.internal.*;
 +import org.apache.ignite.internal.managers.discovery.*;
+ import org.apache.ignite.internal.processors.cache.*;
  import org.apache.ignite.internal.processors.cache.query.*;
  import org.apache.ignite.internal.util.typedef.*;
  import org.apache.ignite.internal.util.typedef.internal.*;

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/f0a805ff/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/jdbc/GridCacheQueryJdbcTask.java
----------------------------------------------------------------------
diff --cc 
modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/jdbc/GridCacheQueryJdbcTask.java
index 3a35453,332c649..9b52052
--- 
a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/jdbc/GridCacheQueryJdbcTask.java
+++ 
b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/jdbc/GridCacheQueryJdbcTask.java
@@@ -18,14 -18,11 +18,13 @@@
  package org.apache.ignite.internal.processors.cache.query.jdbc;
  
  import org.apache.ignite.*;
- import org.apache.ignite.cache.*;
+ import org.apache.ignite.cache.query.*;
  import org.apache.ignite.cluster.*;
  import org.apache.ignite.compute.*;
 +import org.apache.ignite.internal.*;
 +import org.apache.ignite.internal.managers.discovery.*;
- import org.apache.ignite.internal.processors.cache.query.*;
+ import org.apache.ignite.internal.processors.cache.*;
  import org.apache.ignite.internal.processors.query.*;
- import org.apache.ignite.internal.util.lang.*;
  import org.apache.ignite.internal.util.typedef.*;
  import org.apache.ignite.internal.util.typedef.internal.*;
  import org.apache.ignite.marshaller.*;

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/f0a805ff/modules/core/src/main/java/org/apache/ignite/internal/processors/cacheobject/IgniteCacheObjectProcessor.java
----------------------------------------------------------------------
diff --cc 
modules/core/src/main/java/org/apache/ignite/internal/processors/cacheobject/IgniteCacheObjectProcessor.java
index d5eafbe,a8eda0e..717cf48
--- 
a/modules/core/src/main/java/org/apache/ignite/internal/processors/cacheobject/IgniteCacheObjectProcessor.java
+++ 
b/modules/core/src/main/java/org/apache/ignite/internal/processors/cacheobject/IgniteCacheObjectProcessor.java
@@@ -18,11 -18,10 +18,11 @@@
  package org.apache.ignite.internal.processors.cacheobject;
  
  import org.apache.ignite.*;
- import org.apache.ignite.cache.*;
  import org.apache.ignite.cluster.*;
 +import org.apache.ignite.configuration.*;
  import org.apache.ignite.internal.processors.*;
  import org.apache.ignite.internal.processors.cache.*;
+ import org.apache.ignite.lang.*;
  import org.jetbrains.annotations.*;
  
  /**

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/f0a805ff/modules/core/src/main/java/org/apache/ignite/internal/processors/cacheobject/IgniteCacheObjectProcessorImpl.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/f0a805ff/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/DataStructuresProcessor.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/f0a805ff/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheSetImpl.java
----------------------------------------------------------------------
diff --cc 
modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheSetImpl.java
index f70fd6e,0e59985..1ebd5d0
--- 
a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheSetImpl.java
+++ 
b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheSetImpl.java
@@@ -18,10 -18,8 +18,9 @@@
  package org.apache.ignite.internal.processors.datastructures;
  
  import org.apache.ignite.*;
- import org.apache.ignite.cache.*;
  import org.apache.ignite.cache.affinity.*;
  import org.apache.ignite.cluster.*;
 +import org.apache.ignite.internal.processors.affinity.*;
  import org.apache.ignite.internal.processors.cache.*;
  import org.apache.ignite.internal.processors.cache.query.*;
  import org.apache.ignite.internal.util.*;

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/f0a805ff/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/f0a805ff/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheRebalanceTask.java
----------------------------------------------------------------------
diff --cc 
modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheRebalanceTask.java
index 01e9c9a,0000000..dec598f
mode 100644,000000..100644
--- 
a/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheRebalanceTask.java
+++ 
b/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheRebalanceTask.java
@@@ -1,82 -1,0 +1,83 @@@
 +/*
 + * Licensed to the Apache Software Foundation (ASF) under one or more
 + * contributor license agreements.  See the NOTICE file distributed with
 + * this work for additional information regarding copyright ownership.
 + * The ASF licenses this file to You under the Apache License, Version 2.0
 + * (the "License"); you may not use this file except in compliance with
 + * the License.  You may obtain a copy of the License at
 + *
 + *      http://www.apache.org/licenses/LICENSE-2.0
 + *
 + * Unless required by applicable law or agreed to in writing, software
 + * distributed under the License is distributed on an "AS IS" BASIS,
 + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
 + * See the License for the specific language governing permissions and
 + * limitations under the License.
 + */
 +
 +package org.apache.ignite.internal.visor.cache;
 +
 +import org.apache.ignite.*;
 +import org.apache.ignite.cache.*;
 +import org.apache.ignite.internal.*;
++import org.apache.ignite.internal.processors.cache.*;
 +import org.apache.ignite.internal.processors.task.*;
 +import org.apache.ignite.internal.util.typedef.internal.*;
 +import org.apache.ignite.internal.visor.*;
 +
 +import java.util.*;
 +
 +/**
 + * Pre-loads caches. Made callable just to conform common pattern.
 + */
 +@GridInternal
 +public class VisorCacheRebalanceTask extends VisorOneNodeTask<Set<String>, 
Void> {
 +    /** */
 +    private static final long serialVersionUID = 0L;
 +
 +    /** {@inheritDoc} */
 +    @Override protected VisorCachesRebalanceJob job(Set<String> arg) {
 +        return new VisorCachesRebalanceJob(arg, debug);
 +    }
 +
 +    /**
 +     * Job that rebalance caches.
 +     */
 +    private static class VisorCachesRebalanceJob extends 
VisorJob<Set<String>, Void> {
 +        /** */
 +        private static final long serialVersionUID = 0L;
 +
 +        /**
 +         * @param arg Caches names.
 +         * @param debug Debug flag.
 +         */
 +        private VisorCachesRebalanceJob(Set<String> arg, boolean debug) {
 +            super(arg, debug);
 +        }
 +
 +        /** {@inheritDoc} */
 +        @Override protected Void run(Set<String> cacheNames) {
 +            try {
 +                Collection<IgniteInternalFuture<?>> futs = new ArrayList<>();
 +
 +                for (GridCache c : ignite.cachesx()) {
 +                    if (cacheNames.contains(c.name()))
 +                        futs.add(c.forceRepartition());
 +                }
 +
 +                for (IgniteInternalFuture f : futs)
 +                    f.get();
 +
 +                return null;
 +            }
 +            catch (IgniteCheckedException e) {
 +                throw U.convertException(e);
 +            }
 +        }
 +
 +        /** {@inheritDoc} */
 +        @Override public String toString() {
 +            return S.toString(VisorCachesRebalanceJob.class, this);
 +        }
 +    }
 +}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/f0a805ff/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorNodeDataCollectorJob.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/f0a805ff/modules/core/src/main/java/org/apache/ignite/startup/BasicWarmupClosure.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/f0a805ff/modules/core/src/main/resources/META-INF/classnames.properties
----------------------------------------------------------------------
diff --cc modules/core/src/main/resources/META-INF/classnames.properties
index 05ed24a,8e2f633..e8d9949
--- a/modules/core/src/main/resources/META-INF/classnames.properties
+++ b/modules/core/src/main/resources/META-INF/classnames.properties
@@@ -1495,7 -1477,10 +1496,9 @@@ org.apache.ignite.internal.visor.node.V
  
org.apache.ignite.internal.visor.node.VisorNodeRestartTask$VisorNodesRestartJob
  org.apache.ignite.internal.visor.node.VisorNodeStopTask
  org.apache.ignite.internal.visor.node.VisorNodeStopTask$VisorNodesStopJob
+ org.apache.ignite.internal.visor.node.VisorNodeSuppressedErrorsTask
+ 
org.apache.ignite.internal.visor.node.VisorNodeSuppressedErrorsTask$VisorNodeSuppressedErrorsJob
  org.apache.ignite.internal.visor.node.VisorPeerToPeerConfiguration
 -org.apache.ignite.internal.visor.node.VisorQueryConfiguration
  org.apache.ignite.internal.visor.node.VisorRestConfiguration
  org.apache.ignite.internal.visor.node.VisorSegmentationConfiguration
  org.apache.ignite.internal.visor.node.VisorSpisConfiguration

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/f0a805ff/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractFullApiSelfTest.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/f0a805ff/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheClearLocallySelfTest.java
----------------------------------------------------------------------
diff --cc 
modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheClearLocallySelfTest.java
index 49f3f9f,0000000..9c3596d
mode 100644,000000..100644
--- 
a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheClearLocallySelfTest.java
+++ 
b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheClearLocallySelfTest.java
@@@ -1,383 -1,0 +1,382 @@@
 +/*
 + * Licensed to the Apache Software Foundation (ASF) under one or more
 + * contributor license agreements.  See the NOTICE file distributed with
 + * this work for additional information regarding copyright ownership.
 + * The ASF licenses this file to You under the Apache License, Version 2.0
 + * (the "License"); you may not use this file except in compliance with
 + * the License.  You may obtain a copy of the License at
 + *
 + *      http://www.apache.org/licenses/LICENSE-2.0
 + *
 + * Unless required by applicable law or agreed to in writing, software
 + * distributed under the License is distributed on an "AS IS" BASIS,
 + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
 + * See the License for the specific language governing permissions and
 + * limitations under the License.
 + */
 +
 +package org.apache.ignite.internal.processors.cache;
 +
 +import org.apache.ignite.*;
- import org.apache.ignite.cache.*;
 +import org.apache.ignite.cluster.*;
 +import org.apache.ignite.configuration.*;
 +import org.apache.ignite.internal.*;
 +import org.apache.ignite.internal.util.typedef.*;
 +import org.apache.ignite.lang.*;
 +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.apache.ignite.testframework.junits.common.*;
 +import org.apache.ignite.transactions.*;
 +
 +import java.lang.reflect.*;
 +
 +import static org.apache.ignite.cache.CacheAtomicityMode.*;
 +import static org.apache.ignite.cache.CacheMode.*;
 +import static org.apache.ignite.cache.CacheWriteSynchronizationMode.*;
 +import static org.apache.ignite.internal.processors.cache.GridCacheAdapter.*;
 +
 +/**
 + * Test {@link GridCache#clearLocally()} operations in multinode environment 
with nodes having caches with different names.
 + */
 +public class GridCacheClearLocallySelfTest extends GridCommonAbstractTest {
 +    /** Local cache. */
 +    private static final String CACHE_LOCAL = "cache_local";
 +
 +    /** Partitioned cache. */
 +    private static final String CACHE_PARTITIONED = "cache_partitioned";
 +
 +    /** Co-located cache. */
 +    private static final String CACHE_COLOCATED = "cache_colocated";
 +
 +    /** Replicated cache. */
 +    private static final String CACHE_REPLICATED = "cache_replicated";
 +
 +    /** Grid nodes count. */
 +    private static final int GRID_CNT = 3;
 +
 +    /** VM IP finder for TCP discovery SPI. */
 +    private static final TcpDiscoveryIpFinder IP_FINDER = new 
TcpDiscoveryVmIpFinder(true);
 +
 +    /** Local caches. */
 +    private GridCache<Integer, Integer>[] cachesLoc;
 +
 +    /** Partitioned caches. */
 +    private GridCache<Integer, Integer>[] cachesPartitioned;
 +
 +    /** Colocated caches. */
 +    private GridCache<Integer, Integer>[] cachesColocated;
 +
 +    /** Replicated caches. */
 +    private GridCache<Integer, Integer>[] cachesReplicated;
 +
 +    /** {@inheritDoc} */
 +    @Override protected IgniteConfiguration getConfiguration(String gridName) 
throws Exception {
 +        IgniteConfiguration cfg = super.getConfiguration(gridName);
 +
 +        CacheConfiguration ccfgLoc = new CacheConfiguration();
 +
 +        ccfgLoc.setName(CACHE_LOCAL);
 +        ccfgLoc.setCacheMode(LOCAL);
 +        ccfgLoc.setWriteSynchronizationMode(FULL_SYNC);
 +        ccfgLoc.setAtomicityMode(TRANSACTIONAL);
 +
 +        CacheConfiguration ccfgPartitioned = new CacheConfiguration();
 +
 +        ccfgPartitioned.setName(CACHE_PARTITIONED);
 +        ccfgPartitioned.setCacheMode(PARTITIONED);
 +        ccfgPartitioned.setBackups(1);
 +        ccfgPartitioned.setWriteSynchronizationMode(FULL_SYNC);
 +        NearCacheConfiguration nearCfg = new NearCacheConfiguration();
 +
 +        nearCfg.setName(CACHE_PARTITIONED);
 +
 +        ccfgPartitioned.setNearConfiguration(nearCfg);
 +
 +        ccfgPartitioned.setNodeFilter(new 
AttributeFilter(getTestGridName(0)));
 +
 +        ccfgPartitioned.setAtomicityMode(TRANSACTIONAL);
 +
 +        CacheConfiguration ccfgColocated = new CacheConfiguration();
 +
 +        ccfgColocated.setName(CACHE_COLOCATED);
 +        ccfgColocated.setCacheMode(PARTITIONED);
 +        ccfgColocated.setBackups(1);
 +        ccfgColocated.setWriteSynchronizationMode(FULL_SYNC);
 +        ccfgColocated.setAtomicityMode(TRANSACTIONAL);
 +
 +        CacheConfiguration ccfgReplicated = new CacheConfiguration();
 +
 +        ccfgReplicated.setName(CACHE_REPLICATED);
 +        ccfgReplicated.setCacheMode(REPLICATED);
 +        ccfgReplicated.setWriteSynchronizationMode(FULL_SYNC);
 +        ccfgReplicated.setAtomicityMode(TRANSACTIONAL);
 +
 +        cfg.setCacheConfiguration(ccfgLoc, ccfgPartitioned, ccfgColocated, 
ccfgReplicated);
 +
 +        TcpDiscoverySpi discoSpi = new TcpDiscoverySpi();
 +
 +        discoSpi.setIpFinder(IP_FINDER);
 +
 +        cfg.setDiscoverySpi(discoSpi);
 +
 +        return cfg;
 +    }
 +
 +    /** {@inheritDoc} */
 +    @Override protected void afterTest() throws Exception {
 +        stopAllGrids();
 +
 +        cachesLoc = null;
 +        cachesPartitioned = null;
 +        cachesColocated = null;
 +        cachesReplicated = null;
 +    }
 +
 +    /**
 +     * Startup routine.
 +     *
 +     * @throws Exception If failed.
 +     */
 +    private void startUp() throws Exception {
 +        cachesLoc = (GridCache<Integer, 
Integer>[])Array.newInstance(GridCache.class, GRID_CNT);
 +        cachesPartitioned = (GridCache<Integer, 
Integer>[])Array.newInstance(GridCache.class, GRID_CNT);
 +        cachesColocated = (GridCache<Integer, 
Integer>[])Array.newInstance(GridCache.class, GRID_CNT);
 +        cachesReplicated = (GridCache<Integer, 
Integer>[])Array.newInstance(GridCache.class, GRID_CNT);
 +
 +        for (int i = 0; i < GRID_CNT; i++) {
 +            Ignite ignite = startGrid(i);
 +
 +            if (i == 1) {
 +                NearCacheConfiguration nearCfg = new NearCacheConfiguration();
 +
 +                nearCfg.setName(CACHE_PARTITIONED);
 +
 +                ignite.createCache(nearCfg);
 +            }
 +
 +            if (i == 2)
 +                ignite.jcache(CACHE_PARTITIONED);
 +
 +            cachesLoc[i] = ((IgniteKernal)ignite).cache(CACHE_LOCAL);
 +            cachesPartitioned[i] = 
((IgniteKernal)ignite).cache(CACHE_PARTITIONED);
 +            cachesColocated[i] = 
((IgniteKernal)ignite).cache(CACHE_COLOCATED);
 +            cachesReplicated[i] = 
((IgniteKernal)ignite).cache(CACHE_REPLICATED);
 +        }
 +    }
 +
 +    /**
 +     * Test {@link GridCache#clearLocally()} on LOCAL cache with no split.
 +     *
 +     * @throws Exception If failed.
 +     */
 +    public void testLocalNoSplit() throws Exception {
 +        test(Mode.TEST_LOCAL, CLEAR_ALL_SPLIT_THRESHOLD / 2);
 +    }
 +
 +    /**
 +     * Test {@link GridCache#clearLocally()} on LOCAL cache with split.
 +     *
 +     * @throws Exception If failed.
 +     */
 +    public void testLocalSplit() throws Exception {
 +        test(Mode.TEST_LOCAL, CLEAR_ALL_SPLIT_THRESHOLD + 1);
 +    }
 +
 +    /**
 +     * Test {@link GridCache#clearLocally()} on PARTITIONED cache with no 
split.
 +     *
 +     * @throws Exception If failed.
 +     */
 +    public void testPartitionedNoSplit() throws Exception {
 +        test(Mode.TEST_PARTITIONED, CLEAR_ALL_SPLIT_THRESHOLD / 2);
 +    }
 +
 +    /**
 +     * Test {@link GridCache#clearLocally()} on PARTITIONED cache with split.
 +     *
 +     * @throws Exception If failed.
 +     */
 +    public void testPartitionedSplit() throws Exception {
 +        test(Mode.TEST_PARTITIONED, CLEAR_ALL_SPLIT_THRESHOLD + 1);
 +    }
 +
 +    /**
 +     * Test {@link GridCache#clearLocally()} on co-located cache with no 
split.
 +     *
 +     * @throws Exception If failed.
 +     */
 +    public void testColocatedNoSplit() throws Exception {
 +        test(Mode.TEST_COLOCATED, CLEAR_ALL_SPLIT_THRESHOLD / 2);
 +    }
 +
 +    /**
 +     * Test {@link GridCache#clearLocally()} on co-located cache with split.
 +     *
 +     * @throws Exception If failed.
 +     */
 +    public void testColocatedSplit() throws Exception {
 +        test(Mode.TEST_COLOCATED, CLEAR_ALL_SPLIT_THRESHOLD + 1);
 +    }
 +
 +    /**
 +     * Test {@link GridCache#clearLocally()} on REPLICATED cache with no 
split.
 +     *
 +     * @throws Exception If failed.
 +     */
 +    public void testReplicatedNoSplit() throws Exception {
 +        test(Mode.TEST_REPLICATED, CLEAR_ALL_SPLIT_THRESHOLD / 2);
 +    }
 +
 +    /**
 +     * Test {@link GridCache#clearLocally()} on REPLICATED cache with split.
 +     *
 +     * @throws Exception If failed.
 +     */
 +    public void testReplicatedSplit() throws Exception {
 +        test(Mode.TEST_REPLICATED, CLEAR_ALL_SPLIT_THRESHOLD + 1);
 +    }
 +
 +    /**
 +     * Internal method for all tests.
 +     *
 +     * @param mode Test mode
 +     * @param keysCnt Keys count.
 +     * @throws Exception In case of exception.
 +     */
 +    private void test(Mode mode, int keysCnt) throws Exception {
 +        startUp();
 +
 +        switch (mode) {
 +            case TEST_LOCAL: {
 +                // Check on only one node.
 +                GridCache<Integer, Integer> cache = cachesLoc[0];
 +
 +                fillCache(cache, keysCnt);
 +
 +                cache.clearLocally();
 +
 +                assert cache.isEmpty();
 +
 +                break;
 +            }
 +
 +            case TEST_PARTITIONED: {
 +                // Take in count special case for near-only cache as well.
 +                fillCache(cachesPartitioned[0], keysCnt);
 +
 +                // Ensure correct no-op clean of CLIENT_ONLY cache.
 +                warmCache(cachesPartitioned[2], keysCnt);
 +                assert cachesPartitioned[2].isEmpty() : 
cachesPartitioned[2].values();
 +                cachesPartitioned[2].clearLocally();
 +                assert cachesPartitioned[2].isEmpty();
 +
 +                stopGrid(2); // Shutdown Grid in order to remove reader in 
NEAR_PARTITIONED cache.
 +
 +                // Ensure correct clearLocally of NEA_ONLY cache.
 +                warmCache(cachesPartitioned[1], keysCnt);
 +                assert !cachesPartitioned[1].isEmpty();
 +                cachesPartitioned[1].clearLocally();
 +                assert cachesPartitioned[1].isEmpty();
 +                fillCache(cachesPartitioned[1], keysCnt);
 +
 +                stopGrid(1); // Shutdown Grid in order to remove reader in 
NEAR_PARTITIONED cache.
 +
 +                // Ensure correct clearLocally of NEAR_PARTITIONED cache.
 +                assert !cachesPartitioned[0].isEmpty();
 +                cachesPartitioned[0].clearLocally();
 +                assert cachesPartitioned[0].isEmpty();
 +
 +                break;
 +            }
 +
 +            default: {
 +                assert mode == Mode.TEST_COLOCATED || mode == 
Mode.TEST_REPLICATED;
 +
 +                GridCache<Integer, Integer>[] caches = mode == 
Mode.TEST_COLOCATED ? cachesColocated : cachesReplicated;
 +
 +                fillCache(caches[0], keysCnt);
 +
 +                for (GridCache<Integer, Integer> cache : caches) {
 +                    assert !cache.isEmpty();
 +
 +                    cache.clearLocally();
 +
 +                    assert cache.isEmpty();
 +                }
 +            }
 +        }
 +    }
 +
 +    /**
 +     * Fill cache with values.
 +     *
 +     * @param cache Cache.
 +     * @param keysCnt Amount of keys to put.
 +     * @throws Exception If failed.
 +     */
 +    private void fillCache(GridCache<Integer, Integer> cache, int keysCnt) 
throws Exception {
 +        try (Transaction tx = cache.txStart()) {
 +            for (int i = 0; i < keysCnt; i++)
 +                cache.put(i, i);
 +
 +            tx.commit();
 +        }
 +    }
 +
 +    /**
 +     * Warm cache up.
 +     *
 +     * @param cache Cache.
 +     * @param keysCnt Amount of keys to get.
 +     * @throws Exception If failed.
 +     */
 +    private void warmCache(GridCache<Integer, Integer> cache, int keysCnt) 
throws Exception {
 +        for (int i = 0; i < keysCnt; i++)
 +            cache.get(i);
 +    }
 +
 +    /**
 +     * Test mode.
 +     */
 +    private enum Mode {
 +        /** Local cache. */
 +        TEST_LOCAL,
 +
 +        /** Partitioned cache. */
 +        TEST_PARTITIONED,
 +
 +        /** Co-located cache. */
 +        TEST_COLOCATED,
 +
 +        /** Replicated cache. */
 +        TEST_REPLICATED
 +    }
 +
 +    /**
 +     *
 +     */
 +    private static class AttributeFilter implements 
IgnitePredicate<ClusterNode> {
 +        /** */
 +        private String[] attrs;
 +
 +        /**
 +         * @param attrs Attribute values.
 +         */
 +        private AttributeFilter(String... attrs) {
 +            this.attrs = attrs;
 +        }
 +
 +        /** {@inheritDoc} */
 +        @Override public boolean apply(ClusterNode node) {
 +            String gridName = 
node.attribute(IgniteNodeAttributes.ATTR_GRID_NAME);
 +
 +            for (String attr : attrs) {
 +                if (F.eq(attr, gridName))
 +                    return true;
 +            }
 +
 +            return false;
 +        }
 +    }
 +}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/f0a805ff/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheReturnValueTransferSelfTest.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/f0a805ff/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/GridCacheAtomicTimeoutSelfTest.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/f0a805ff/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/GridCachePreloadRestartAbstractSelfTest.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/f0a805ff/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheNearEvictionSelfTest.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/f0a805ff/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheNearMultiNodeSelfTest.java
----------------------------------------------------------------------
diff --cc 
modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheNearMultiNodeSelfTest.java
index be80e81,83983f2..937e9f7
--- 
a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheNearMultiNodeSelfTest.java
+++ 
b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheNearMultiNodeSelfTest.java
@@@ -24,7 -24,7 +24,8 @@@ import org.apache.ignite.cache.store.*
  import org.apache.ignite.cluster.*;
  import org.apache.ignite.configuration.*;
  import org.apache.ignite.internal.*;
 +import org.apache.ignite.internal.processors.affinity.*;
+ import org.apache.ignite.internal.processors.cache.*;
  import org.apache.ignite.internal.processors.cache.distributed.*;
  import org.apache.ignite.internal.processors.cache.distributed.dht.*;
  import org.apache.ignite.internal.util.typedef.*;

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/f0a805ff/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedFilteredPutSelfTest.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/f0a805ff/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedPreloadLifecycleSelfTest.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/f0a805ff/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/GridCacheContinuousQueryAbstractSelfTest.java
----------------------------------------------------------------------
diff --cc 
modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/GridCacheContinuousQueryAbstractSelfTest.java
index 5de1d99,b19a6ca..2bac1df
--- 
a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/GridCacheContinuousQueryAbstractSelfTest.java
+++ 
b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/GridCacheContinuousQueryAbstractSelfTest.java
@@@ -73,19 -77,21 +76,21 @@@ public abstract class GridCacheContinuo
  
          cfg.setPeerClassLoadingEnabled(peerClassLoadingEnabled());
  
-         CacheConfiguration cacheCfg = defaultCacheConfiguration();
+         if (!gridName.equals(NO_CACHE_GRID_NAME)) {
+             CacheConfiguration cacheCfg = defaultCacheConfiguration();
  
 -            cacheCfg.setCacheMode(cacheMode());
 -            cacheCfg.setAtomicityMode(atomicityMode());
 -            cacheCfg.setDistributionMode(distributionMode());
 -            cacheCfg.setRebalanceMode(ASYNC);
 -            cacheCfg.setWriteSynchronizationMode(FULL_SYNC);
 -            cacheCfg.setCacheStoreFactory(new 
FactoryBuilder.SingletonFactory(new TestStore()));
 -            cacheCfg.setReadThrough(true);
 -            cacheCfg.setWriteThrough(true);
 -            cacheCfg.setLoadPreviousValue(true);
 +        cacheCfg.setCacheMode(cacheMode());
 +        cacheCfg.setAtomicityMode(atomicityMode());
 +        cacheCfg.setNearConfiguration(nearConfiguration());
 +        cacheCfg.setRebalanceMode(ASYNC);
 +        cacheCfg.setWriteSynchronizationMode(FULL_SYNC);
 +        cacheCfg.setCacheStoreFactory(new StoreFactory());
 +        cacheCfg.setReadThrough(true);
 +        cacheCfg.setWriteThrough(true);
 +        cacheCfg.setLoadPreviousValue(true);
  
-         cfg.setCacheConfiguration(cacheCfg);
+             cfg.setCacheConfiguration(cacheCfg);
+         }
  
          TcpDiscoverySpi disco = new TcpDiscoverySpi();
  

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/f0a805ff/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsAbstractSelfTest.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/f0a805ff/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsProcessorSelfTest.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/f0a805ff/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsStreamsSelfTest.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/f0a805ff/modules/hadoop/src/test/java/org/apache/ignite/igfs/HadoopIgfsDualAbstractSelfTest.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/f0a805ff/modules/hibernate/src/test/java/org/apache/ignite/cache/hibernate/HibernateL2CacheSelfTest.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/f0a805ff/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/IgniteH2Indexing.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/f0a805ff/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCachePartitionedQueryMultiThreadedSelfTest.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/f0a805ff/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/reducefields/GridCacheAbstractReduceFieldsQuerySelfTest.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/f0a805ff/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/sql/GridQueryParsingTest.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/f0a805ff/modules/spring/src/main/java/org/apache/ignite/cache/spring/SpringCacheManager.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/f0a805ff/modules/visor-console/src/main/scala/org/apache/ignite/visor/commands/cache/VisorCacheCommand.scala
----------------------------------------------------------------------

Reply via email to