# ignite-42

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

Branch: refs/heads/ignite-63
Commit: 550d603363f59df34ce82916e009f16a4dc5ed39
Parents: 7b1a738
Author: sboikov <sboi...@gridgain.com>
Authored: Thu Jan 22 18:11:21 2015 +0300
Committer: sboikov <sboi...@gridgain.com>
Committed: Thu Jan 22 18:11:32 2015 +0300

----------------------------------------------------------------------
 .../examples/datagrid/CacheQueryExample.java    |   1 -
 .../examples/ScalarCacheQueryExample.scala      |   1 -
 .../spi/checkpoint/s3/GridS3CheckpointSpi.java  |   3 +-
 .../ipfinder/s3/GridTcpDiscoveryS3IpFinder.java |   5 +-
 .../s3/GridS3CheckpointSpiSelfTest.java         |   4 +-
 .../GridClientPartitionAffinitySelfTest.java    |   2 +
 .../src/main/java/org/apache/ignite/Ignite.java |   1 -
 .../ignite/IgniteAuthenticationException.java   |  35 ++
 .../apache/ignite/IgniteBasicWarmupClosure.java | 560 ++++++++++++++++++
 .../java/org/apache/ignite/IgniteCache.java     |   1 -
 .../java/org/apache/ignite/IgniteCluster.java   |   1 -
 .../org/apache/ignite/IgniteDataLoader.java     |  30 +-
 .../ignite/IgniteDeploymentException.java       |  57 ++
 .../ignite/IgniteIllegalStateException.java     |  84 +++
 .../ignite/IgniteInterruptedException.java      |  56 ++
 .../org/apache/ignite/IgniteMetadataAware.java  | 187 ++++++
 .../apache/ignite/IgniteNotPeerDeployable.java  |  41 ++
 .../java/org/apache/ignite/IgnitePortables.java |   1 -
 .../org/apache/ignite/IgniteTransactions.java   |   1 -
 .../main/java/org/apache/ignite/Ignition.java   |  13 +-
 .../apache/ignite/cache/CacheConfiguration.java |   2 -
 .../cache/CachePartialUpdateException.java      |   3 -
 .../java/org/apache/ignite/cache/GridCache.java |   2 -
 .../org/apache/ignite/cache/GridCacheEntry.java |   3 +-
 .../org/apache/ignite/cache/GridCacheMode.java  |   2 -
 .../org/apache/ignite/cache/GridCacheName.java  |   2 -
 .../ignite/cache/GridCachePreloadMode.java      |   2 -
 .../ignite/cache/GridCacheProjection.java       |   2 -
 .../affinity/GridCacheAffinityFunction.java     |   1 -
 .../affinity/GridCacheAffinityKeyMapped.java    |   2 -
 ...GridCacheConsistentHashAffinityFunction.java |   4 +-
 .../fair/GridCachePartitionFairAffinity.java    |   1 -
 .../GridCacheRendezvousAffinityFunction.java    |   1 -
 .../ignite/cache/cloner/GridCacheCloner.java    |   1 -
 .../datastructures/GridCacheCountDownLatch.java |  13 +-
 .../datastructures/GridCacheDataStructures.java |   1 -
 .../cache/eviction/GridCacheEvictionFilter.java |   1 -
 .../cache/eviction/GridCacheEvictionPolicy.java |   1 -
 .../fifo/GridCacheFifoEvictionPolicy.java       |   2 -
 .../ggfs/GridCacheGgfsEvictionFilter.java       |   1 -
 .../GridCacheGgfsPerBlockLruEvictionPolicy.java |   1 -
 .../lru/GridCacheLruEvictionPolicy.java         |   2 -
 .../random/GridCacheRandomEvictionPolicy.java   |   2 -
 .../cache/query/GridCacheContinuousQuery.java   |   1 -
 .../ignite/cache/query/GridCacheQuery.java      |   2 -
 .../cache/store/CacheLoadOnlyStoreAdapter.java  |   1 -
 .../apache/ignite/cache/store/CacheStore.java   |   4 +-
 .../ignite/cache/store/CacheStoreAdapter.java   |   1 -
 .../cache/store/jdbc/CacheJdbcBlobStore.java    |   3 +-
 .../impl/GridClientAbstractProjection.java      |   5 +-
 .../org/apache/ignite/cluster/ClusterNode.java  |   5 +-
 .../ignite/cluster/ClusterNodeLocalMap.java     |   4 +-
 .../ignite/compute/ComputeJobSibling.java       |   3 +-
 .../ignite/compute/ComputeLoadBalancer.java     |   3 +-
 .../apache/ignite/hadoop/GridHadoopTask.java    |   3 +-
 .../processors/cache/CacheLockImpl.java         |   7 +-
 .../processors/cache/IgniteCacheProxy.java      |   1 -
 .../org/apache/ignite/jdbc/GridJdbcDriver.java  |   1 -
 .../org/apache/ignite/lang/IgniteFuture.java    |   7 +-
 .../org/apache/ignite/lang/IgnitePredicate.java |   2 -
 .../IgniteOptimizedObjectStreamRegistry.java    |  15 +-
 .../main/java/org/apache/ignite/package.html    |  23 +
 .../ignite/scheduler/SchedulerFuture.java       |   7 +-
 .../communication/tcp/TcpCommunicationSpi.java  |   3 +-
 .../discovery/tcp/TcpClientDiscoverySpi.java    |  11 +-
 .../spi/discovery/tcp/TcpDiscoverySpi.java      |   9 +-
 .../discovery/tcp/TcpDiscoverySpiAdapter.java   |   5 +-
 .../ipfinder/jdbc/TcpDiscoveryJdbcIpFinder.java |   3 +-
 .../TcpDiscoveryMulticastIpFinder.java          |   3 +-
 .../sharedfs/TcpDiscoverySharedFsIpFinder.java  |   3 +-
 .../spi/swapspace/file/FileSwapSpaceSpi.java    |   7 +-
 .../StreamerCacheAffinityEventRouter.java       |   1 -
 .../window/StreamerBoundedSizeBatchWindow.java  |  11 +-
 .../window/StreamerBoundedTimeBatchWindow.java  |  11 +-
 .../apache/ignite/transactions/IgniteTx.java    |   1 -
 .../transactions/IgniteTxSynchronization.java   |   1 -
 .../grid/GridAuthenticationException.java       |  37 --
 .../gridgain/grid/GridBasicWarmupClosure.java   | 563 -------------------
 .../gridgain/grid/GridDeploymentException.java  |  58 --
 .../grid/GridIllegalStateException.java         |  84 ---
 .../gridgain/grid/GridInterruptedException.java |  58 --
 .../org/gridgain/grid/GridMetadataAware.java    | 187 ------
 .../gridgain/grid/GridNotPeerDeployable.java    |  41 --
 .../grid/kernal/ClusterNodeLocalMapImpl.java    |   3 +-
 .../grid/kernal/GridEventConsumeHandler.java    |   5 +-
 .../java/org/gridgain/grid/kernal/GridEx.java   |   1 -
 .../org/gridgain/grid/kernal/GridGainEx.java    |  17 +-
 .../grid/kernal/GridJobContextImpl.java         |   2 -
 .../org/gridgain/grid/kernal/GridKernal.java    |   4 +-
 .../gridgain/grid/kernal/GridKernalContext.java |   3 +-
 .../grid/kernal/GridMessageListenHandler.java   |   5 +-
 .../gridgain/grid/kernal/IgniteComputeImpl.java |   3 +-
 .../grid/kernal/IgniteTransactionsEx.java       |   1 -
 .../grid/kernal/ggfs/common/GridGgfsLogger.java |   6 +-
 .../kernal/managers/GridManagerAdapter.java     |   1 -
 .../managers/communication/GridIoManager.java   |   5 +-
 .../deployment/GridDeploymentCommunication.java |   3 +-
 .../deployment/GridDeploymentLocalStore.java    |   3 +-
 .../discovery/GridDiscoveryManager.java         |   3 +-
 .../eventstorage/GridEventStorageManager.java   |   5 +-
 .../loadbalancer/GridLoadBalancerManager.java   |   2 -
 .../affinity/GridAffinityAssignmentCache.java   |   1 -
 .../affinity/GridAffinityProcessor.java         |   2 -
 .../processors/affinity/GridAffinityUtils.java  |   6 +-
 .../GridCacheAffinityFunctionContextImpl.java   |   1 -
 .../processors/cache/GridCacheAdapter.java      |  10 +-
 .../processors/cache/GridCacheAttributes.java   |   2 -
 .../cache/GridCacheConcurrentMap.java           |   1 -
 .../processors/cache/GridCacheContext.java      |   1 -
 .../GridCacheDefaultAffinityKeyMapper.java      |   1 -
 .../cache/GridCacheDeploymentManager.java       |   1 -
 .../processors/cache/GridCacheEntryEx.java      |   4 +-
 .../processors/cache/GridCacheEntryImpl.java    |   4 +-
 .../processors/cache/GridCacheEntrySet.java     |   1 -
 .../cache/GridCacheEvictionEntry.java           |   4 +-
 .../cache/GridCacheEvictionManager.java         |   4 +-
 .../cache/GridCacheFilterEvaluationEntry.java   |   4 +-
 .../processors/cache/GridCacheIterator.java     |   1 -
 .../processors/cache/GridCacheKeySet.java       |   1 -
 .../processors/cache/GridCacheMBeanAdapter.java |   1 -
 .../processors/cache/GridCacheMapAdapter.java   |   1 -
 .../processors/cache/GridCacheMapEntry.java     |   4 +-
 .../processors/cache/GridCacheMessage.java      |   1 -
 .../cache/GridCacheMetricsAdapter.java          |   1 -
 .../processors/cache/GridCacheMvccManager.java  |   1 -
 .../GridCachePartitionExchangeManager.java      |   7 +-
 .../cache/GridCachePreloaderAdapter.java        |   1 -
 .../processors/cache/GridCacheProcessor.java    |   3 -
 .../processors/cache/GridCacheProjectionEx.java |   1 -
 .../cache/GridCacheProjectionImpl.java          |   1 -
 .../kernal/processors/cache/GridCacheProxy.java |   1 -
 .../processors/cache/GridCacheProxyImpl.java    |   2 -
 .../cache/GridCacheSharedContext.java           |   1 -
 .../processors/cache/GridCacheStoreManager.java |   7 +-
 .../processors/cache/GridCacheTtlManager.java   |   3 +-
 .../kernal/processors/cache/GridCacheUtils.java |   1 -
 .../cache/GridCacheValueCollection.java         |   1 -
 .../cache/GridCacheWriteBehindStore.java        |  16 +-
 .../cache/affinity/GridCacheAffinityImpl.java   |   1 -
 .../cache/affinity/GridCacheAffinityProxy.java  |   1 -
 .../datastructures/GridCacheAtomicLongImpl.java |   1 -
 .../GridCacheAtomicReferenceImpl.java           |   1 -
 .../GridCacheAtomicSequenceImpl.java            |   1 -
 .../GridCacheAtomicStampedImpl.java             |   1 -
 .../GridCacheCountDownLatchImpl.java            |   1 -
 .../GridCacheDataStructuresManager.java         |   1 -
 .../datastructures/GridCacheQueueAdapter.java   |   1 -
 .../cache/datastructures/GridCacheSetImpl.java  |   2 -
 .../GridDistributedCacheAdapter.java            |   1 -
 .../GridDistributedTxRemoteAdapter.java         |   1 -
 .../GridPartitionedCacheEntryImpl.java          |   4 +-
 .../distributed/dht/GridDhtCacheAdapter.java    |   1 -
 .../distributed/dht/GridDhtCacheEntry.java      |   1 -
 .../distributed/dht/GridDhtCacheEntryImpl.java  |   1 -
 .../cache/distributed/dht/GridDhtGetFuture.java |   1 -
 .../distributed/dht/GridDhtLocalPartition.java  |   3 +-
 .../distributed/dht/GridDhtLockFuture.java      |   1 -
 .../dht/GridDhtTransactionalCacheAdapter.java   |   1 -
 .../distributed/dht/GridDhtTxLocalAdapter.java  |   1 -
 .../distributed/dht/GridDhtTxPrepareFuture.java |   3 +-
 .../dht/GridPartitionedGetFuture.java           |   1 -
 .../dht/atomic/GridDhtAtomicCache.java          |   1 -
 .../dht/atomic/GridDhtAtomicCacheEntry.java     |   1 -
 .../dht/atomic/GridDhtAtomicUpdateFuture.java   |   1 -
 .../dht/atomic/GridDhtAtomicUpdateRequest.java  |   1 -
 .../dht/atomic/GridNearAtomicUpdateFuture.java  |   1 -
 .../dht/atomic/GridNearAtomicUpdateRequest.java |   1 -
 .../dht/colocated/GridDhtColocatedCache.java    |   1 -
 .../colocated/GridDhtColocatedCacheEntry.java   |   1 -
 .../colocated/GridDhtColocatedLockFuture.java   |   1 -
 .../dht/preloader/GridDhtForceKeysFuture.java   |   3 +-
 .../preloader/GridDhtPartitionDemandPool.java   |  14 +-
 .../preloader/GridDhtPartitionSupplyPool.java   |   3 +-
 .../GridDhtPartitionsExchangeFuture.java        |   8 +-
 .../distributed/near/GridNearAtomicCache.java   |   1 -
 .../distributed/near/GridNearCacheAdapter.java  |   1 -
 .../distributed/near/GridNearCacheEntry.java    |   1 -
 .../distributed/near/GridNearGetFuture.java     |   1 -
 .../distributed/near/GridNearGetRequest.java    |   1 -
 .../distributed/near/GridNearLockFuture.java    |   1 -
 .../distributed/near/GridNearLockRequest.java   |   1 -
 .../near/GridNearTransactionalCache.java        |   1 -
 .../processors/cache/local/GridLocalCache.java  |   1 -
 .../cache/local/GridLocalLockFuture.java        |   1 -
 .../local/atomic/GridLocalAtomicCache.java      |   1 -
 .../query/GridCacheDistributedQueryFuture.java  |   3 +-
 .../query/GridCacheDistributedQueryManager.java |   4 +-
 .../cache/query/GridCacheLocalQueryFuture.java  |   1 -
 .../cache/query/GridCacheQueriesImpl.java       |   1 -
 .../cache/query/GridCacheQueriesProxy.java      |   1 -
 .../cache/query/GridCacheQueryAdapter.java      |   1 -
 .../query/GridCacheQueryFutureAdapter.java      |   5 +-
 .../cache/query/GridCacheQueryInfo.java         |   1 -
 .../cache/query/GridCacheQueryManager.java      |   4 +-
 .../cache/query/GridCacheQueryRequest.java      |   1 -
 .../GridCacheContinuousQueryAdapter.java        |  21 +-
 .../GridCacheContinuousQueryEntry.java          |   4 +-
 .../GridCacheContinuousQueryHandler.java        |   6 +-
 .../GridCacheContinuousQueryManager.java        |   1 -
 .../jdbc/GridCacheQueryJdbcMetadataTask.java    |   1 -
 .../query/jdbc/GridCacheQueryJdbcTask.java      |   1 -
 .../cache/transactions/IgniteTxEntry.java       |   1 -
 .../cache/transactions/IgniteTxEx.java          |   4 +-
 .../transactions/IgniteTxLocalAdapter.java      |   1 -
 .../cache/transactions/IgniteTxLocalEx.java     |   1 -
 .../clock/GridClockDeltaSnapshot.java           |   7 +-
 .../processors/clock/GridClockServer.java       |   3 +-
 .../clock/GridClockSyncProcessor.java           |   3 +-
 .../closure/GridClosureProcessor.java           |   2 -
 .../continuous/GridContinuousProcessor.java     |   9 +-
 .../dataload/GridDataLoadCacheUpdaters.java     |   2 -
 .../dataload/GridDataLoaderProcessor.java       |   5 +-
 .../dataload/IgniteDataLoaderImpl.java          |  22 +-
 .../processors/ggfs/GridGgfsDataManager.java    |   9 +-
 .../processors/ggfs/GridGgfsDeleteWorker.java   |   3 +-
 .../ggfs/GridGgfsFileWorkerBatch.java           |   3 +-
 .../ggfs/GridGgfsFragmentizerManager.java       |   7 +-
 .../processors/ggfs/GridGgfsMetaManager.java    |   4 +-
 .../processors/ggfs/GridGgfsProcessor.java      |   2 -
 .../kernal/processors/ggfs/GridGgfsServer.java  |   5 +-
 .../kernal/processors/job/GridJobProcessor.java |   3 +-
 .../kernal/processors/job/GridJobWorker.java    |   5 +-
 .../processors/query/GridQueryProcessor.java    |   5 +-
 .../handlers/cache/GridCacheCommandHandler.java |   1 -
 .../cache/GridCacheQueryCommandHandler.java     |   1 -
 .../service/GridServiceProcessor.java           |   6 +-
 .../processors/streamer/IgniteStreamerImpl.java |  13 +-
 .../processors/task/GridTaskProcessor.java      |  12 +-
 .../grid/kernal/visor/cache/VisorCache.java     |   1 -
 .../cache/VisorCacheAffinityConfiguration.java  |   1 -
 .../kernal/visor/cache/VisorCacheClearTask.java |   1 -
 .../visor/cache/VisorCacheCompactTask.java      |   1 -
 .../visor/cache/VisorCacheConfiguration.java    |   1 -
 .../kernal/visor/cache/VisorCacheLoadTask.java  |   1 -
 .../visor/cache/VisorCacheMetadataTask.java     |   1 -
 .../kernal/visor/cache/VisorCacheMetrics.java   |   1 -
 .../cache/VisorCacheMetricsCollectorTask.java   |   1 -
 .../cache/VisorCachePreloadConfiguration.java   |   1 -
 .../visor/cache/VisorCachePreloadTask.java      |   1 -
 .../visor/cache/VisorCacheResetMetricsTask.java |   1 -
 .../visor/cache/VisorCacheSwapBackupsTask.java  |   1 -
 .../visor/node/VisorNodeDataCollectorJob.java   |   1 -
 .../grid/kernal/visor/query/VisorQueryTask.java |   1 -
 .../main/java/org/gridgain/grid/package.html    |  23 -
 .../grid/util/GridAtomicInitializer.java        |   5 +-
 .../java/org/gridgain/grid/util/GridUtils.java  |  66 ++-
 .../grid/util/future/GridEmbeddedFuture.java    |   5 +-
 .../grid/util/future/GridFutureAdapter.java     |   5 +-
 .../grid/util/future/GridFutureAdapterEx.java   |   5 +-
 .../GridIpcSharedMemoryServerEndpoint.java      |   7 +-
 .../grid/util/lang/GridComputeJobWrapper.java   |   9 +-
 .../org/gridgain/grid/util/lang/GridFunc.java   |  46 +-
 .../util/lang/GridMetadataAwareAdapter.java     |   7 +-
 .../gridgain/grid/util/nio/GridNioFuture.java   |   7 +-
 .../grid/util/nio/GridNioFutureImpl.java        |   5 +-
 .../gridgain/grid/util/nio/GridNioServer.java   |  11 +-
 .../org/gridgain/grid/util/typedef/PCE.java     |   1 -
 .../gridgain/grid/util/worker/GridWorker.java   |   7 +-
 ...eCacheEntryListenerEagerTtlDisabledTest.java |   1 -
 ...cpCommunicationSpiMultithreadedSelfTest.java |   3 +-
 .../index/GridStreamerIndexSelfTest.java        |   3 +-
 .../kernal/GridJobMasterLeaveAwareSelfTest.java |   9 +-
 .../GridCheckpointManagerAbstractSelfTest.java  |   5 +-
 .../cache/GridCacheGenericTestStore.java        |   3 +-
 .../GridCachePartitionedAffinitySpreadTest.java |   2 +
 .../GridCachePreloadingEvictionsSelfTest.java   |   5 +-
 .../processors/cache/GridCacheTestStore.java    |   3 +-
 ...BehindStorePartitionedMultiNodeSelfTest.java |   5 +-
 ...ridCacheQueueJoinedNodeSelfAbstractTest.java |  11 +-
 .../dht/GridCacheDhtPreloadDelayedSelfTest.java |   4 +-
 ...eAtomicInvalidPartitionHandlingSelfTest.java |   3 +-
 .../GridCachePartitionedAffinitySelfTest.java   |   1 +
 .../GridCacheEmptyEntriesAbstractSelfTest.java  |   6 +-
 ...ridCacheContinuousQueryAbstractSelfTest.java | 114 ++--
 ...heGgfsPerBlockLruEvictionPolicySelfTest.java |   4 +-
 .../ggfs/GridGgfsAbstractSelfTest.java          |   5 +-
 .../streamer/GridStreamerSelfTest.java          |   3 +-
 .../gridgain/grid/lang/GridTupleSelfTest.java   |   5 +-
 .../communication/GridIoManagerBenchmark.java   |   5 +-
 .../GridContinuousOperationsLoadTest.java       |  11 +-
 ...JobExecutionSingleNodeSemaphoreLoadTest.java |   3 +-
 .../streamer/GridStreamerIndexLoadTest.java     |   3 +-
 .../loadtests/streamer/QueryClosure.java        |   5 +-
 .../GridTestSafeThreadFactory.java              |   3 +-
 .../gridgain/testframework/GridTestUtils.java   |  13 +-
 .../bamboo/GridDataGridTestSuite.java           |   2 +-
 .../tests/p2p/GridCacheDeploymentTestTask1.java |   1 -
 .../tests/p2p/GridExternalAffinityFunction.java |   1 -
 .../p2p/GridExternalAffinityKeyMapper.java      |   1 -
 .../GridUriDeploymentAbstractTestTask.java      |   1 -
 .../kernal/ggfs/hadoop/GridGgfsHadoopIpcIo.java |   3 +-
 .../hadoop/jobtracker/GridHadoopJobTracker.java |   1 -
 .../hadoop/shuffle/GridHadoopShuffleJob.java    |   5 +-
 .../taskexecutor/GridHadoopExecutorService.java |   3 +-
 .../hadoop/v2/GridHadoopV2CleanupTask.java      |   3 +-
 .../hadoop/v2/GridHadoopV2MapTask.java          |   3 +-
 .../hadoop/v2/GridHadoopV2ReduceTask.java       |   3 +-
 .../hadoop/v2/GridHadoopV2SetupTask.java        |   3 +-
 .../hadoop/v2/GridHadoopV2Splitter.java         |   3 +-
 .../hibernate/GridCacheHibernateBlobStore.java  |   3 +-
 .../processors/query/h2/GridH2Indexing.java     |   1 -
 .../cache/GridCacheAbstractQuerySelfTest.java   |   2 +
 .../query/h2/GridH2IndexRebuildTest.java        |   3 +-
 .../grid/cache/jta/GridCacheTmLookup.java       |   2 -
 .../cache/jta/jndi/GridCacheJndiTmLookup.java   |   6 +-
 .../reflect/GridCacheReflectionTmLookup.java    |   2 +-
 .../GridTmLookupLifecycleAwareSelfTest.java     |   2 +-
 .../org/gridgain/scalar/ScalarConversions.scala |   1 -
 .../gridgain/scalar/pimps/ScalarCachePimp.scala |   1 -
 .../pimps/ScalarCacheProjectionPimp.scala       |   1 -
 .../scalar/pimps/ScalarProjectionPimp.scala     |   4 +-
 .../main/scala/org/gridgain/scalar/scalar.scala |   4 -
 .../processors/schedule/ScheduleFutureImpl.java |   5 +-
 .../util/nodestart/GridNodeCallableImpl.java    |   7 +-
 .../deployment/uri/GridUriDeploymentSpi.java    |   3 +-
 .../main/scala/org/gridgain/visor/visor.scala   |   2 +-
 .../GridServletContextListenerStartup.java      |   3 +-
 317 files changed, 1496 insertions(+), 1831 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/550d6033/examples/src/main/java/org/gridgain/examples/datagrid/CacheQueryExample.java
----------------------------------------------------------------------
diff --git 
a/examples/src/main/java/org/gridgain/examples/datagrid/CacheQueryExample.java 
b/examples/src/main/java/org/gridgain/examples/datagrid/CacheQueryExample.java
index 3de6932..a8f906f 100644
--- 
a/examples/src/main/java/org/gridgain/examples/datagrid/CacheQueryExample.java
+++ 
b/examples/src/main/java/org/gridgain/examples/datagrid/CacheQueryExample.java
@@ -22,7 +22,6 @@ import org.apache.ignite.cache.*;
 import org.apache.ignite.cache.affinity.*;
 import org.apache.ignite.cache.query.*;
 import org.apache.ignite.lang.*;
-import org.gridgain.grid.cache.affinity.*;
 
 import java.io.*;
 import java.util.*;

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/550d6033/examples/src/main/scala/org/gridgain/scalar/examples/ScalarCacheQueryExample.scala
----------------------------------------------------------------------
diff --git 
a/examples/src/main/scala/org/gridgain/scalar/examples/ScalarCacheQueryExample.scala
 
b/examples/src/main/scala/org/gridgain/scalar/examples/ScalarCacheQueryExample.scala
index a938d8e..759d850 100644
--- 
a/examples/src/main/scala/org/gridgain/scalar/examples/ScalarCacheQueryExample.scala
+++ 
b/examples/src/main/scala/org/gridgain/scalar/examples/ScalarCacheQueryExample.scala
@@ -24,7 +24,6 @@ import scalar._
 import org.apache.ignite._
 import org.gridgain.grid.cache._
 import GridCacheMode._
-import affinity._
 import java.util._
 
 /**

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/550d6033/modules/aws/src/main/java/org/apache/ignite/spi/checkpoint/s3/GridS3CheckpointSpi.java
----------------------------------------------------------------------
diff --git 
a/modules/aws/src/main/java/org/apache/ignite/spi/checkpoint/s3/GridS3CheckpointSpi.java
 
b/modules/aws/src/main/java/org/apache/ignite/spi/checkpoint/s3/GridS3CheckpointSpi.java
index be4f950..7d0c8bd 100644
--- 
a/modules/aws/src/main/java/org/apache/ignite/spi/checkpoint/s3/GridS3CheckpointSpi.java
+++ 
b/modules/aws/src/main/java/org/apache/ignite/spi/checkpoint/s3/GridS3CheckpointSpi.java
@@ -25,7 +25,6 @@ import org.apache.ignite.*;
 import org.apache.ignite.marshaller.*;
 import org.apache.ignite.resources.*;
 import org.apache.ignite.spi.*;
-import org.gridgain.grid.*;
 import org.apache.ignite.spi.checkpoint.*;
 import org.gridgain.grid.util.typedef.*;
 import org.gridgain.grid.util.typedef.internal.*;
@@ -282,7 +281,7 @@ public class GridS3CheckpointSpi extends IgniteSpiAdapter 
implements CheckpointS
                     try {
                         U.sleep(200);
                     }
-                    catch (GridInterruptedException e) {
+                    catch (IgniteInterruptedException e) {
                         throw new IgniteSpiException("Thread has been 
interrupted.", e);
                     }
             }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/550d6033/modules/aws/src/main/java/org/apache/ignite/spi/discovery/tcp/ipfinder/s3/GridTcpDiscoveryS3IpFinder.java
----------------------------------------------------------------------
diff --git 
a/modules/aws/src/main/java/org/apache/ignite/spi/discovery/tcp/ipfinder/s3/GridTcpDiscoveryS3IpFinder.java
 
b/modules/aws/src/main/java/org/apache/ignite/spi/discovery/tcp/ipfinder/s3/GridTcpDiscoveryS3IpFinder.java
index 1b1cefe..3975ebb 100644
--- 
a/modules/aws/src/main/java/org/apache/ignite/spi/discovery/tcp/ipfinder/s3/GridTcpDiscoveryS3IpFinder.java
+++ 
b/modules/aws/src/main/java/org/apache/ignite/spi/discovery/tcp/ipfinder/s3/GridTcpDiscoveryS3IpFinder.java
@@ -24,7 +24,6 @@ import com.amazonaws.services.s3.model.*;
 import org.apache.ignite.*;
 import org.apache.ignite.resources.*;
 import org.apache.ignite.spi.*;
-import org.gridgain.grid.*;
 import org.apache.ignite.spi.discovery.tcp.ipfinder.*;
 import org.gridgain.grid.util.typedef.*;
 import org.gridgain.grid.util.typedef.internal.*;
@@ -250,7 +249,7 @@ public class GridTcpDiscoveryS3IpFinder extends 
TcpDiscoveryIpFinderAdapter {
                             try {
                                 U.sleep(200);
                             }
-                            catch (GridInterruptedException e) {
+                            catch (IgniteInterruptedException e) {
                                 throw new IgniteSpiException("Thread has been 
interrupted.", e);
                             }
                     }
@@ -270,7 +269,7 @@ public class GridTcpDiscoveryS3IpFinder extends 
TcpDiscoveryIpFinderAdapter {
             try {
                 U.await(initLatch);
             }
-            catch (GridInterruptedException e) {
+            catch (IgniteInterruptedException e) {
                 throw new IgniteSpiException("Thread has been interrupted.", 
e);
             }
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/550d6033/modules/aws/src/test/java/org/apache/ignite/spi/checkpoint/s3/GridS3CheckpointSpiSelfTest.java
----------------------------------------------------------------------
diff --git 
a/modules/aws/src/test/java/org/apache/ignite/spi/checkpoint/s3/GridS3CheckpointSpiSelfTest.java
 
b/modules/aws/src/test/java/org/apache/ignite/spi/checkpoint/s3/GridS3CheckpointSpiSelfTest.java
index ebf96d0..ac1f3e8 100644
--- 
a/modules/aws/src/test/java/org/apache/ignite/spi/checkpoint/s3/GridS3CheckpointSpiSelfTest.java
+++ 
b/modules/aws/src/test/java/org/apache/ignite/spi/checkpoint/s3/GridS3CheckpointSpiSelfTest.java
@@ -207,9 +207,9 @@ public class GridS3CheckpointSpiSelfTest extends 
GridSpiAbstractTest<GridS3Check
      * Wrapper around {@link 
GridTestUtils#retryAssert(org.apache.ignite.IgniteLogger, int, long, 
GridAbsClosure)}.
      * Provides s3-specific timeouts.
      * @param assertion Closure with assertion inside.
-     * @throws GridInterruptedException If was interrupted.
+     * @throws org.apache.ignite.IgniteInterruptedException If was interrupted.
      */
-    private void assertWithRetries(GridAbsClosureX assertion) throws 
GridInterruptedException {
+    private void assertWithRetries(GridAbsClosureX assertion) throws 
IgniteInterruptedException {
         GridTestUtils.retryAssert(log, 6, 5000, assertion);
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/550d6033/modules/clients/src/test/java/org/apache/ignite/client/impl/GridClientPartitionAffinitySelfTest.java
----------------------------------------------------------------------
diff --git 
a/modules/clients/src/test/java/org/apache/ignite/client/impl/GridClientPartitionAffinitySelfTest.java
 
b/modules/clients/src/test/java/org/apache/ignite/client/impl/GridClientPartitionAffinitySelfTest.java
index 6827eff..af644ac 100644
--- 
a/modules/clients/src/test/java/org/apache/ignite/client/impl/GridClientPartitionAffinitySelfTest.java
+++ 
b/modules/clients/src/test/java/org/apache/ignite/client/impl/GridClientPartitionAffinitySelfTest.java
@@ -28,6 +28,8 @@ import org.gridgain.testframework.junits.common.*;
 
 import java.util.*;
 
+import static 
org.apache.ignite.cache.affinity.consistenthash.GridCacheConsistentHashAffinityFunction.*;
+
 /**
  * Client's partitioned affinity tests.
  */

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/550d6033/modules/core/src/main/java/org/apache/ignite/Ignite.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/Ignite.java 
b/modules/core/src/main/java/org/apache/ignite/Ignite.java
index e023c15..ad6848b 100644
--- a/modules/core/src/main/java/org/apache/ignite/Ignite.java
+++ b/modules/core/src/main/java/org/apache/ignite/Ignite.java
@@ -23,7 +23,6 @@ import org.apache.ignite.configuration.*;
 import org.apache.ignite.fs.IgniteFsConfiguration;
 import org.apache.ignite.plugin.*;
 import org.apache.ignite.product.*;
-import org.gridgain.grid.cache.*;
 import org.apache.ignite.hadoop.*;
 import org.apache.ignite.plugin.security.*;
 import org.gridgain.grid.util.typedef.*;

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/550d6033/modules/core/src/main/java/org/apache/ignite/IgniteAuthenticationException.java
----------------------------------------------------------------------
diff --git 
a/modules/core/src/main/java/org/apache/ignite/IgniteAuthenticationException.java
 
b/modules/core/src/main/java/org/apache/ignite/IgniteAuthenticationException.java
new file mode 100644
index 0000000..e3ebd60
--- /dev/null
+++ 
b/modules/core/src/main/java/org/apache/ignite/IgniteAuthenticationException.java
@@ -0,0 +1,35 @@
+/*
+ * 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;
+
+/**
+ * Exception that represents authentication failure.
+ */
+public class IgniteAuthenticationException extends IgniteCheckedException {
+    /** */
+    private static final long serialVersionUID = 0L;
+
+    /**
+     * Creates authentication exception with given error message.
+     *
+     * @param msg Error message.
+     */
+    public IgniteAuthenticationException(String msg) {
+        super(msg);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/550d6033/modules/core/src/main/java/org/apache/ignite/IgniteBasicWarmupClosure.java
----------------------------------------------------------------------
diff --git 
a/modules/core/src/main/java/org/apache/ignite/IgniteBasicWarmupClosure.java 
b/modules/core/src/main/java/org/apache/ignite/IgniteBasicWarmupClosure.java
new file mode 100644
index 0000000..10632c6
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/IgniteBasicWarmupClosure.java
@@ -0,0 +1,560 @@
+/*
+ * 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;
+
+import org.apache.ignite.cache.*;
+import org.apache.ignite.configuration.*;
+import org.apache.ignite.lang.*;
+import org.apache.ignite.logger.*;
+import org.apache.ignite.spi.discovery.tcp.*;
+import org.apache.ignite.spi.discovery.tcp.ipfinder.*;
+import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.*;
+import org.gridgain.grid.util.typedef.*;
+import org.gridgain.grid.util.typedef.internal.*;
+
+import java.text.*;
+import java.util.*;
+import java.util.concurrent.*;
+
+/**
+ * Basic warm-up closure which warm-ups cache operations.
+ */
+public class IgniteBasicWarmupClosure implements 
IgniteInClosure<IgniteConfiguration> {
+    /** */
+    private static final long serialVersionUID = 9175346848249957458L;
+
+    /** Default grid count to warm up. */
+    public static final int DFLT_GRID_CNT = 2;
+
+    /** Default iteration count per thread. */
+    public static final int DFLT_ITERATION_CNT = 30_000;
+
+    /** Default key range. */
+    public static final int DFLT_KEY_RANGE = 10_000;
+
+    /** Grid count. */
+    private int gridCnt = DFLT_GRID_CNT;
+
+    /** Warmup date format. */
+    private static final SimpleDateFormat WARMUP_DATE_FMT = new 
SimpleDateFormat("HH:mm:ss");
+
+    /** Warmup thread count. */
+    private int threadCnt = Runtime.getRuntime().availableProcessors() * 2;
+
+    /** Per thread iteration count. */
+    private int iterCnt = DFLT_ITERATION_CNT;
+
+    /** Key range. */
+    private int keyRange = DFLT_KEY_RANGE;
+
+    /** Warmup discovery port. */
+    private int discoveryPort = 27000;
+
+    /** Methods to warmup. */
+    private String[] warmupMethods = {"put", "putx", "get", "remove", 
"removex", "putIfAbsent", "replace"};
+
+    /**
+     * Gets number of grids to start and run warmup.
+     *
+     * @return Number of grids.
+     */
+    public int getGridCount() {
+        return gridCnt;
+    }
+
+    /**
+     * Sets number of grids to start and run the warmup.
+     *
+     * @param gridCnt Number of grids.
+     */
+    public void setGridCount(int gridCnt) {
+        this.gridCnt = gridCnt;
+    }
+
+    /**
+     * Gets warmup methods to use for cache warmup.
+     *
+     * @return Warmup methods.
+     */
+    public String[] getWarmupMethods() {
+        return warmupMethods;
+    }
+
+    /**
+     * Sets warmup methods to use for cache warmup.
+     *
+     * @param warmupMethods Array of warmup methods.
+     */
+    public void setWarmupMethods(String... warmupMethods) {
+        this.warmupMethods = warmupMethods;
+    }
+
+    /**
+     * Gets thread count for warmup.
+     *
+     * @return Thread count.
+     */
+    public int getThreadCount() {
+        return threadCnt;
+    }
+
+    /**
+     * Sets thread count for warmup.
+     *
+     * @param threadCnt Thread count.
+     */
+    public void setThreadCount(int threadCnt) {
+        this.threadCnt = threadCnt;
+    }
+
+    /**
+     * Gets iteration count for warmup.
+     *
+     * @return Iteration count.
+     */
+    public int getIterationCount() {
+        return iterCnt;
+    }
+
+    /**
+     * Sets iteration count for warmup.
+     *
+     * @param iterCnt Iteration count for warmup.
+     */
+    public void setIterationCount(int iterCnt) {
+        this.iterCnt = iterCnt;
+    }
+
+    /**
+     * Gets key range.
+     *
+     * @return Key range.
+     */
+    public int getKeyRange() {
+        return keyRange;
+    }
+
+    /**
+     * Sets key range.
+     *
+     * @param keyRange Key range.
+     */
+    public void setKeyRange(int keyRange) {
+        this.keyRange = keyRange;
+    }
+
+    /**
+     * Gets discovery port for warmup.
+     *
+     * @return Discovery port.
+     */
+    public int getDiscoveryPort() {
+        return discoveryPort;
+    }
+
+    /**
+     * Sets discovery port for warmup.
+     *
+     * @param discoveryPort Discovery port.
+     */
+    public void setDiscoveryPort(int discoveryPort) {
+        this.discoveryPort = discoveryPort;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void apply(IgniteConfiguration gridCfg) {
+        // Remove cache duplicates, clean up the rest, etc.
+        IgniteConfiguration cfg = prepareConfiguration(gridCfg);
+
+        // Do nothing if no caches found.
+        if (cfg == null)
+            return;
+
+        out("Starting grids to warmup caches [gridCnt=" + gridCnt +
+            ", caches=" + cfg.getCacheConfiguration().length + ']');
+
+        Collection<Ignite> ignites = new LinkedList<>();
+
+        String old = 
System.getProperty(IgniteSystemProperties.GG_UPDATE_NOTIFIER);
+
+        try {
+            System.setProperty(IgniteSystemProperties.GG_UPDATE_NOTIFIER, 
"false");
+
+            TcpDiscoveryIpFinder ipFinder = new TcpDiscoveryVmIpFinder(true);
+
+            for (int i = 0; i < gridCnt; i++) {
+                IgniteConfiguration cfg0 = new IgniteConfiguration(cfg);
+
+                TcpDiscoverySpi discoSpi = new TcpDiscoverySpi();
+
+                discoSpi.setIpFinder(ipFinder);
+
+                discoSpi.setLocalPort(discoveryPort);
+
+                cfg0.setDiscoverySpi(discoSpi);
+
+                cfg0.setGridLogger(new IgniteNullLogger());
+
+                cfg0.setGridName("gridgain-warmup-grid-" + i);
+
+                ignites.add(Ignition.start(cfg0));
+            }
+
+            doWarmup(ignites);
+        }
+        catch (Exception e) {
+            throw new IgniteException(e);
+        }
+        finally {
+            for (Ignite ignite : ignites)
+                Ignition.stop(ignite.name(), false);
+
+            out("Stopped warmup grids.");
+
+            if (old == null)
+                old = "false";
+
+            System.setProperty(IgniteSystemProperties.GG_UPDATE_NOTIFIER, old);
+        }
+    }
+
+    /**
+     * @param grids Grids to warmup.
+     */
+    private void doWarmup(Iterable<Ignite> grids) throws Exception {
+        Ignite first = F.first(grids);
+
+        ExecutorService svc = Executors.newFixedThreadPool(threadCnt);
+
+        try {
+            for (GridCache<?, ?> cache : first.caches()) {
+                GridCache<Object, Object> cache0 = first.cache(cache.name());
+
+                for (String warmupMethod : warmupMethods) {
+                    Collection<Future> futs = new ArrayList<>(threadCnt);
+
+                    for (int i = 0; i < threadCnt; i++) {
+                        Callable call;
+
+                        switch (warmupMethod) {
+                            case "get": {
+                                call = new GetCallable(cache0);
+
+                                break;
+                            }
+
+                            case "put": {
+                                call = new PutCallable(cache0);
+
+                                break;
+                            }
+
+                            case "putx": {
+                                call = new PutxCallable(cache0);
+
+                                break;
+                            }
+
+                            case "remove": {
+                                call = new RemoveCallable(cache0);
+
+                                break;
+                            }
+
+                            case "removex": {
+                                call = new RemovexCallable(cache0);
+
+                                break;
+                            }
+
+                            case "putIfAbsent": {
+                                call = new PutIfAbsentCallable(cache0);
+
+                                break;
+                            }
+
+                            case "replace": {
+                                call = new ReplaceCallable(cache0);
+
+                                break;
+                            }
+
+                            default:
+                                throw new IgniteCheckedException("Unsupported 
warmup method: " + warmupMethod);
+                        }
+
+                        futs.add(svc.submit(call));
+                    }
+
+                    out("Running warmup [cacheName=" + cache.name() + ", 
method=" + warmupMethod + ']');
+
+                    for (Future fut : futs)
+                        fut.get();
+
+                    for (int key = 0; key < keyRange; key++)
+                        cache0.remove(key);
+                }
+            }
+        }
+        finally {
+            svc.shutdownNow();
+        }
+    }
+
+    /**
+     * Output for warmup messages.
+     *
+     * @param msg Format message.
+     */
+    private static void out(String msg) {
+        System.out.println('[' + WARMUP_DATE_FMT.format(new 
Date(System.currentTimeMillis())) + "][WARMUP][" +
+            Thread.currentThread().getName() + ']' + ' ' + msg);
+    }
+
+    /**
+     * Prepares configuration for warmup.
+     *
+     * @param gridCfg Original grid configuration.
+     * @return Prepared configuration or {@code null} if no caches found.
+     */
+    private IgniteConfiguration prepareConfiguration(IgniteConfiguration 
gridCfg) {
+        if (F.isEmpty(gridCfg.getCacheConfiguration()))
+            return null;
+
+        IgniteConfiguration cp = new IgniteConfiguration();
+
+        cp.setClientConnectionConfiguration(null);
+
+        Collection<CacheConfiguration> reduced = new ArrayList<>();
+
+        for (CacheConfiguration ccfg : gridCfg.getCacheConfiguration()) {
+            if (CU.isSystemCache(ccfg.getName()))
+                continue;
+
+            if (!matches(reduced, ccfg)) {
+                CacheConfiguration ccfgCp = new CacheConfiguration(ccfg);
+
+                if (ccfgCp.getDistributionMode() == 
GridCacheDistributionMode.CLIENT_ONLY)
+                    
ccfgCp.setDistributionMode(GridCacheDistributionMode.PARTITIONED_ONLY);
+                else if (ccfgCp.getDistributionMode() == 
GridCacheDistributionMode.NEAR_ONLY)
+                    
ccfgCp.setDistributionMode(GridCacheDistributionMode.NEAR_PARTITIONED);
+
+                ccfgCp.setCacheStoreFactory(null);
+                ccfgCp.setWriteBehindEnabled(false);
+
+                reduced.add(ccfgCp);
+            }
+        }
+
+        if (F.isEmpty(reduced))
+            return null;
+
+        CacheConfiguration[] res = new CacheConfiguration[reduced.size()];
+
+        reduced.toArray(res);
+
+        cp.setCacheConfiguration(res);
+
+        return cp;
+    }
+
+    /**
+     * Checks if passed configuration matches one of the configurations in the 
list.
+     *
+     * @param reduced Reduced configurations.
+     * @param ccfg Cache configuration to match.
+     * @return {@code True} if matching configuration is found, {@code false} 
otherwise.
+     */
+    private boolean matches(Iterable<CacheConfiguration> reduced, 
CacheConfiguration ccfg) {
+        for (CacheConfiguration ccfg0 : reduced) {
+            if (matches(ccfg0, ccfg))
+                return true;
+        }
+
+        return false;
+    }
+
+    /**
+     * Checks if cache configurations are alike for warmup.
+     *
+     * @param ccfg0 First configuration.
+     * @param ccfg1 Second configuration.
+     * @return {@code True} if configurations match.
+     */
+    private boolean matches(CacheConfiguration ccfg0, CacheConfiguration 
ccfg1) {
+        return
+            F.eq(ccfg0.getCacheMode(), ccfg1.getCacheMode()) &&
+            F.eq(ccfg0.getBackups(), ccfg1.getBackups()) &&
+            F.eq(ccfg0.getAtomicityMode(), ccfg1.getAtomicityMode()) &&
+            F.eq(ccfg0.getAtomicWriteOrderMode(), 
ccfg1.getAtomicWriteOrderMode()) &&
+            F.eq(ccfg0.getMemoryMode(), ccfg1.getMemoryMode()) &&
+            F.eq(ccfg0.getDistributionMode(), ccfg1.getDistributionMode());
+    }
+
+    /**
+     * Base class for all warmup callables.
+     */
+    private abstract class BaseWarmupCallable implements Callable<Object> {
+        /** Cache. */
+        protected final GridCache<Object, Object> cache;
+
+        /**
+         * @param cache Cache.
+         */
+        protected BaseWarmupCallable(GridCache<Object, Object> cache) {
+            this.cache = cache;
+        }
+
+        /** {@inheritDoc} */
+        @Override public Object call() throws Exception {
+            ThreadLocalRandom rnd = ThreadLocalRandom.current();
+
+            for (int i = 0; i < iterCnt; i++)
+                operation(rnd.nextInt(keyRange));
+
+            return null;
+        }
+
+        /**
+         * Runs operation.
+         *
+         * @param key Key.
+         * @throws Exception If failed.
+         */
+        protected abstract void operation(int key) throws Exception;
+    }
+
+    /**
+     *
+     */
+    private class GetCallable extends BaseWarmupCallable {
+        /**
+         * @param cache Cache.
+         */
+        private GetCallable(GridCache<Object, Object> cache) {
+            super(cache);
+        }
+
+        /** {@inheritDoc} */
+        @Override protected void operation(int key) throws Exception {
+            cache.get(key);
+        }
+    }
+
+    /**
+     *
+     */
+    private class PutCallable extends BaseWarmupCallable {
+        /**
+         * @param cache Cache.
+         */
+        private PutCallable(GridCache<Object, Object> cache) {
+            super(cache);
+        }
+
+        /** {@inheritDoc} */
+        @Override protected void operation(int key) throws Exception {
+            cache.put(key, key);
+        }
+    }
+
+    /**
+     *
+     */
+    private class PutxCallable extends BaseWarmupCallable {
+        /**
+         * @param cache Cache.
+         */
+        private PutxCallable(GridCache<Object, Object> cache) {
+            super(cache);
+        }
+
+        /** {@inheritDoc} */
+        @Override protected void operation(int key) throws Exception {
+            cache.putx(key, key);
+        }
+    }
+
+    /**
+     *
+     */
+    private class RemoveCallable extends BaseWarmupCallable {
+        /**
+         * @param cache Cache.
+         */
+        private RemoveCallable(GridCache<Object, Object> cache) {
+            super(cache);
+        }
+
+        /** {@inheritDoc} */
+        @Override protected void operation(int key) throws Exception {
+            cache.remove(key);
+        }
+    }
+
+    /**
+     *
+     */
+    private class RemovexCallable extends BaseWarmupCallable {
+        /**
+         * @param cache Cache.
+         */
+        private RemovexCallable(GridCache<Object, Object> cache) {
+            super(cache);
+        }
+
+        /** {@inheritDoc} */
+        @Override protected void operation(int key) throws Exception {
+            cache.removex(key);
+        }
+    }
+
+    /**
+     *
+     */
+    private class PutIfAbsentCallable extends BaseWarmupCallable {
+        /**
+         * @param cache Cache.
+         */
+        private PutIfAbsentCallable(GridCache<Object, Object> cache) {
+            super(cache);
+        }
+
+        /** {@inheritDoc} */
+        @Override protected void operation(int key) throws Exception {
+            cache.putIfAbsent(key, key);
+        }
+    }
+
+    /**
+     *
+     */
+    private class ReplaceCallable extends BaseWarmupCallable {
+        /**
+         * @param cache Cache.
+         */
+        private ReplaceCallable(GridCache<Object, Object> cache) {
+            super(cache);
+        }
+
+        /** {@inheritDoc} */
+        @Override protected void operation(int key) throws Exception {
+            cache.replace(key, key, key);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/550d6033/modules/core/src/main/java/org/apache/ignite/IgniteCache.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/IgniteCache.java 
b/modules/core/src/main/java/org/apache/ignite/IgniteCache.java
index c7c125d..b16f9df 100644
--- a/modules/core/src/main/java/org/apache/ignite/IgniteCache.java
+++ b/modules/core/src/main/java/org/apache/ignite/IgniteCache.java
@@ -21,7 +21,6 @@ import org.apache.ignite.cache.*;
 import org.apache.ignite.cache.query.*;
 import org.apache.ignite.cache.store.*;
 import org.apache.ignite.lang.*;
-import org.gridgain.grid.cache.*;
 import org.jetbrains.annotations.*;
 
 import javax.cache.*;

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/550d6033/modules/core/src/main/java/org/apache/ignite/IgniteCluster.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/IgniteCluster.java 
b/modules/core/src/main/java/org/apache/ignite/IgniteCluster.java
index 042897c..bde9f9c 100644
--- a/modules/core/src/main/java/org/apache/ignite/IgniteCluster.java
+++ b/modules/core/src/main/java/org/apache/ignite/IgniteCluster.java
@@ -19,7 +19,6 @@ package org.apache.ignite;
 
 import org.apache.ignite.cluster.*;
 import org.apache.ignite.lang.*;
-import org.gridgain.grid.cache.affinity.*;
 import org.gridgain.grid.util.lang.*;
 import org.jetbrains.annotations.*;
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/550d6033/modules/core/src/main/java/org/apache/ignite/IgniteDataLoader.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/IgniteDataLoader.java 
b/modules/core/src/main/java/org/apache/ignite/IgniteDataLoader.java
index 127f86c..a4b41dd 100644
--- a/modules/core/src/main/java/org/apache/ignite/IgniteDataLoader.java
+++ b/modules/core/src/main/java/org/apache/ignite/IgniteDataLoader.java
@@ -19,8 +19,6 @@ package org.apache.ignite;
 
 import org.apache.ignite.dataload.*;
 import org.apache.ignite.lang.*;
-import org.gridgain.grid.*;
-import org.gridgain.grid.cache.*;
 import org.jetbrains.annotations.*;
 
 import java.util.*;
@@ -238,11 +236,11 @@ public interface IgniteDataLoader<K, V> extends 
AutoCloseable {
      * @param key Key.
      * @return Future fo this operation.
      * @throws IgniteCheckedException If failed to map key to node.
-     * @throws GridInterruptedException If thread has been interrupted.
+     * @throws IgniteInterruptedException If thread has been interrupted.
      * @throws IllegalStateException If grid has been concurrently stopped or
      *      {@link #close(boolean)} has already been called on loader.
      */
-    public IgniteFuture<?> removeData(K key)  throws IgniteCheckedException, 
GridInterruptedException, IllegalStateException;
+    public IgniteFuture<?> removeData(K key)  throws IgniteCheckedException, 
IgniteInterruptedException, IllegalStateException;
 
     /**
      * Adds data for loading on remote node. This method can be called from 
multiple
@@ -256,11 +254,11 @@ public interface IgniteDataLoader<K, V> extends 
AutoCloseable {
      * @param val Value or {@code null} if respective entry must be removed 
from cache.
      * @return Future fo this operation.
      * @throws IgniteCheckedException If failed to map key to node.
-     * @throws GridInterruptedException If thread has been interrupted.
+     * @throws IgniteInterruptedException If thread has been interrupted.
      * @throws IllegalStateException If grid has been concurrently stopped or
      *      {@link #close(boolean)} has already been called on loader.
      */
-    public IgniteFuture<?> addData(K key, @Nullable V val) throws 
IgniteCheckedException, GridInterruptedException,
+    public IgniteFuture<?> addData(K key, @Nullable V val) throws 
IgniteCheckedException, IgniteInterruptedException,
         IllegalStateException;
 
     /**
@@ -274,11 +272,11 @@ public interface IgniteDataLoader<K, V> extends 
AutoCloseable {
      * @param entry Entry.
      * @return Future fo this operation.
      * @throws IgniteCheckedException If failed to map key to node.
-     * @throws GridInterruptedException If thread has been interrupted.
+     * @throws IgniteInterruptedException If thread has been interrupted.
      * @throws IllegalStateException If grid has been concurrently stopped or
      *      {@link #close(boolean)} has already been called on loader.
      */
-    public IgniteFuture<?> addData(Map.Entry<K, V> entry) throws 
IgniteCheckedException, GridInterruptedException,
+    public IgniteFuture<?> addData(Map.Entry<K, V> entry) throws 
IgniteCheckedException, IgniteInterruptedException,
         IllegalStateException;
 
     /**
@@ -321,33 +319,33 @@ public interface IgniteDataLoader<K, V> extends 
AutoCloseable {
      * use {@link #tryFlush()} method.
      *
      * @throws IgniteCheckedException If failed to map key to node.
-     * @throws GridInterruptedException If thread has been interrupted.
+     * @throws IgniteInterruptedException If thread has been interrupted.
      * @throws IllegalStateException If grid has been concurrently stopped or
      *      {@link #close(boolean)} has already been called on loader.
      * @see #tryFlush()
      */
-    public void flush() throws IgniteCheckedException, 
GridInterruptedException, IllegalStateException;
+    public void flush() throws IgniteCheckedException, 
IgniteInterruptedException, IllegalStateException;
 
     /**
      * Makes an attempt to load remaining data. This method is mostly similar 
to {@link #flush},
      * with the difference that it won't wait and will exit immediately.
      *
      * @throws IgniteCheckedException If failed to map key to node.
-     * @throws GridInterruptedException If thread has been interrupted.
+     * @throws IgniteInterruptedException If thread has been interrupted.
      * @throws IllegalStateException If grid has been concurrently stopped or
      *      {@link #close(boolean)} has already been called on loader.
      * @see #flush()
      */
-    public void tryFlush() throws IgniteCheckedException, 
GridInterruptedException, IllegalStateException;
+    public void tryFlush() throws IgniteCheckedException, 
IgniteInterruptedException, IllegalStateException;
 
     /**
      * Loads any remaining data and closes this loader.
      *
      * @param cancel {@code True} to cancel ongoing loading operations.
      * @throws IgniteCheckedException If failed to map key to node.
-     * @throws GridInterruptedException If thread has been interrupted.
+     * @throws IgniteInterruptedException If thread has been interrupted.
      */
-    public void close(boolean cancel) throws IgniteCheckedException, 
GridInterruptedException;
+    public void close(boolean cancel) throws IgniteCheckedException, 
IgniteInterruptedException;
 
     /**
      * Closes data loader. This method is identical to calling {@link 
#close(boolean) close(false)} method.
@@ -356,7 +354,7 @@ public interface IgniteDataLoader<K, V> extends 
AutoCloseable {
      * {@code try-with-resources} statement.
      *
      * @throws IgniteCheckedException If failed to close data loader.
-     * @throws GridInterruptedException If thread has been interrupted.
+     * @throws IgniteInterruptedException If thread has been interrupted.
      */
-    @Override public void close() throws IgniteCheckedException, 
GridInterruptedException;
+    @Override public void close() throws IgniteCheckedException, 
IgniteInterruptedException;
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/550d6033/modules/core/src/main/java/org/apache/ignite/IgniteDeploymentException.java
----------------------------------------------------------------------
diff --git 
a/modules/core/src/main/java/org/apache/ignite/IgniteDeploymentException.java 
b/modules/core/src/main/java/org/apache/ignite/IgniteDeploymentException.java
new file mode 100644
index 0000000..e5fd803
--- /dev/null
+++ 
b/modules/core/src/main/java/org/apache/ignite/IgniteDeploymentException.java
@@ -0,0 +1,57 @@
+/*
+ * 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;
+
+import org.jetbrains.annotations.*;
+
+/**
+ * Deployment or re-deployment failed.
+ */
+public class IgniteDeploymentException extends IgniteCheckedException {
+    /** */
+    private static final long serialVersionUID = 0L;
+
+    /**
+     * Creates new exception with given error message.
+     *
+     * @param msg Error message.
+     */
+    public IgniteDeploymentException(String msg) {
+        super(msg);
+    }
+
+    /**
+     * Creates new exception with given throwable as a nested cause and
+     * source of error message.
+     *
+     * @param cause Non-null throwable cause.
+     */
+    public IgniteDeploymentException(Throwable cause) {
+        this(cause.getMessage(), cause);
+    }
+
+    /**
+     * Creates a new exception with given error message and optional nested 
cause exception.
+     *
+     * @param msg Error message.
+     * @param cause Optional nested exception (can be {@code null}).
+     */
+    public IgniteDeploymentException(String msg, @Nullable Throwable cause) {
+        super(msg, cause);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/550d6033/modules/core/src/main/java/org/apache/ignite/IgniteIllegalStateException.java
----------------------------------------------------------------------
diff --git 
a/modules/core/src/main/java/org/apache/ignite/IgniteIllegalStateException.java 
b/modules/core/src/main/java/org/apache/ignite/IgniteIllegalStateException.java
new file mode 100644
index 0000000..6dc44bb
--- /dev/null
+++ 
b/modules/core/src/main/java/org/apache/ignite/IgniteIllegalStateException.java
@@ -0,0 +1,84 @@
+/*
+ * 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;
+
+import org.gridgain.grid.util.typedef.*;
+import org.jetbrains.annotations.*;
+
+/**
+ * This exception indicates the ignite access in invalid state.
+ */
+public class IgniteIllegalStateException extends IllegalStateException {
+    /** */
+    private static final long serialVersionUID = 0L;
+
+    /**
+     * Constructs exception with given message and cause.
+     *
+     * @param msg Exception message.
+     * @param cause Exception cause.
+     */
+    public IgniteIllegalStateException(String msg, @Nullable Throwable cause) {
+        super(msg, cause);
+    }
+
+    /**
+     * Creates exception given throwable as a cause and
+     * source of error message.
+     *
+     * @param cause Non-null throwable cause.
+     */
+    public IgniteIllegalStateException(Throwable cause) {
+        this(cause.getMessage(), cause);
+    }
+
+    /**
+     * Constructs exception with given message.
+     *
+     * @param msg Exception message.
+     */
+    public IgniteIllegalStateException(String msg) {
+        super(msg);
+    }
+
+    /**
+     * Checks if this exception has given class in {@code 'cause'} hierarchy.
+     *
+     * @param cls Cause class to check (if {@code null}, {@code false} is 
returned)..
+     * @return {@code True} if one of the causing exception is an instance of 
passed in class,
+     *      {@code false} otherwise.
+     */
+    public boolean hasCause(@Nullable Class<? extends Throwable>... cls) {
+        return X.hasCause(this, cls);
+    }
+
+    /**
+     * Gets first exception of given class from {@code 'cause'} hierarchy if 
any.
+     *
+     * @param cls Cause class to get cause (if {@code null}, {@code null} is 
returned).
+     * @return First causing exception of passed in class, {@code null} 
otherwise.
+     */
+    @Nullable public <T extends Throwable> T getCause(@Nullable Class<T> cls) {
+        return X.cause(this, cls);
+    }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return getClass() + ": " + getMessage();
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/550d6033/modules/core/src/main/java/org/apache/ignite/IgniteInterruptedException.java
----------------------------------------------------------------------
diff --git 
a/modules/core/src/main/java/org/apache/ignite/IgniteInterruptedException.java 
b/modules/core/src/main/java/org/apache/ignite/IgniteInterruptedException.java
new file mode 100644
index 0000000..1e7f267
--- /dev/null
+++ 
b/modules/core/src/main/java/org/apache/ignite/IgniteInterruptedException.java
@@ -0,0 +1,56 @@
+/*
+ * 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;
+
+/**
+ * This exception is used to wrap standard {@link InterruptedException} into 
{@link IgniteCheckedException}.
+ */
+@SuppressWarnings({"TypeMayBeWeakened"})
+public class IgniteInterruptedException extends IgniteCheckedException {
+    /** */
+    private static final long serialVersionUID = 0L;
+
+    /**
+     * Creates new exception with given throwable as a nested cause and
+     * source of error message.
+     *
+     * @param cause Non-null throwable cause.
+     */
+    public IgniteInterruptedException(InterruptedException cause) {
+        this(cause.getMessage(), cause);
+    }
+
+    /**
+     * Creates a new exception with given error message and optional nested 
cause exception.
+     *
+     * @param msg Error message.
+     */
+    public IgniteInterruptedException(String msg) {
+        super(msg);
+    }
+
+    /**
+     * Creates a new exception with given error message and optional nested 
cause exception.
+     *
+     * @param msg Error message.
+     * @param cause Optional nested exception (can be {@code null}).
+     */
+    public IgniteInterruptedException(String msg, InterruptedException cause) {
+        super(msg, cause);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/550d6033/modules/core/src/main/java/org/apache/ignite/IgniteMetadataAware.java
----------------------------------------------------------------------
diff --git 
a/modules/core/src/main/java/org/apache/ignite/IgniteMetadataAware.java 
b/modules/core/src/main/java/org/apache/ignite/IgniteMetadataAware.java
new file mode 100644
index 0000000..a49520e
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/IgniteMetadataAware.java
@@ -0,0 +1,187 @@
+/*
+ * 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;
+
+import org.jetbrains.annotations.*;
+import java.io.*;
+import java.util.*;
+import java.util.concurrent.*;
+
+/**
+ * Allows to attach metadata attributes to any entity that implements
+ * this interface. This promotes <i>meta-programming technique</i> throughout 
the
+ * GridGain APIs by allowing to attach and query the <b>metadata 
information</b> to top-level
+ * entities in GridGain APIs.
+ * <p>
+ * Note that unlike other types of attributes the metadata is explicitly
+ * local to VM. Local metadata does not have any distribution semantic and is 
not
+ * specifically distributed in any way. Note, however, that if object that 
implements this
+ * interface gets serialized and deserialized on the remote node the 
attachments may or may not
+ * be carried over (depending on the implementation). All classes that come 
with GridGain
+ * and implement this interface support proper serialization and 
deserialization.
+ * <p>
+ * For example, this may become useful for cache entries or
+ * cache transactions. Cache entry attachment can be used whenever entry
+ * needs to carry additional context and it is too expensive to keep
+ * looking that context up from a separate map by a key. For example,
+ * an expiration policy used by caches may add some expiration
+ * context to cache entries to properly expire them.
+ */
+public interface IgniteMetadataAware extends Serializable {
+    /**
+     * Copies all metadata from another instance.
+     *
+     * @param from Metadata aware instance to copy metadata from.
+     */
+    public void copyMeta(IgniteMetadataAware from);
+
+    /**
+     * Copies all metadata from given map.
+     *
+     * @param data Map to copy metadata from.
+     */
+    public void copyMeta(Map<String, ?> data);
+
+    /**
+     * Adds a new metadata.
+     *
+     * @param name Metadata name.
+     * @param val Metadata value.
+     * @param <V> Type of the value.
+     * @return Metadata previously associated with given name, or
+     *      {@code null} if there was none.
+     */
+    @Nullable public <V> V addMeta(String name, V val);
+
+    /**
+     * Adds given metadata value only if it was absent.
+     *
+     * @param name Metadata name.
+     * @param val Value to add if it's not attached already.
+     * @param <V> Type of the value.
+     * @return {@code null} if new value was put, or current value if put 
didn't happen.
+     */
+    @Nullable public <V> V putMetaIfAbsent(String name, V val);
+
+    /**
+     * Adds given metadata value only if it was absent.
+     *
+     * @param name Metadata name.
+     * @param c Factory closure to produce value to add if it's not attached 
already.
+     *      Not that unlike {@link #addMeta(String, Object)} method the 
factory closure will
+     *      not be called unless the value is required and therefore value 
will only be created
+     *      when it is actually needed.
+     * @param <V> Type of the value.
+     * @return {@code null} if new value was put, or current value if put 
didn't happen.
+     */
+    @Nullable public <V> V putMetaIfAbsent(String name, Callable<V> c);
+
+    /**
+     * Adds given metadata value only if it was absent. Unlike
+     * {@link #putMetaIfAbsent(String, Callable)}, this method always returns
+     * the latest value and never previous one.
+     *
+     * @param name Metadata name.
+     * @param val Value to add if it's not attached already.
+     * @param <V> Type of the value.
+     * @return The value of the metadata after execution of this method.
+     */
+    public <V> V addMetaIfAbsent(String name, V val);
+
+    /**
+     * Adds given metadata value only if it was absent.
+     *
+     * @param name Metadata name.
+     * @param c Factory closure to produce value to add if it's not attached 
already.
+     *      Not that unlike {@link #addMeta(String, Object)} method the 
factory closure will
+     *      not be called unless the value is required and therefore value 
will only be created
+     *      when it is actually needed. If {@code null} and metadata value is 
missing - {@code null}
+     *      will be returned from this method.
+     * @param <V> Type of the value.
+     * @return The value of the metadata after execution of this method.
+     */
+    @Nullable public <V> V addMetaIfAbsent(String name, @Nullable Callable<V> 
c);
+
+    /**
+     * Gets metadata by name.
+     *
+     * @param name Metadata name.
+     * @param <V> Type of the value.
+     * @return Metadata value or {@code null}.
+     */
+    public <V> V meta(String name);
+
+    /**
+     * Removes metadata by name.
+     *
+     * @param name Name of the metadata to remove.
+     * @param <V> Type of the value.
+     * @return Value of removed metadata or {@code null}.
+     */
+    public <V> V removeMeta(String name);
+
+    /**
+     * Removes metadata only if its current value is equal to {@code val} 
passed in.
+     *
+     * @param name Name of metadata attribute.
+     * @param val Value to compare.
+     * @param <V> Value type.
+     * @return {@code True} if value was removed, {@code false} otherwise.
+     */
+    public <V> boolean removeMeta(String name, V val);
+
+    /**
+     * Gets all metadata in this entry.
+     *
+     * @param <V> Type of the value.
+     * @return All metadata in this entry.
+     */
+    public <V> Map<String, V> allMeta();
+
+    /**
+     * Tests whether or not given metadata is set.
+     *
+     * @param name Name of the metadata to test.
+     * @return Whether or not given metadata is set.
+     */
+    public boolean hasMeta(String name);
+
+    /**
+     * Tests whether or not metadata with given name and value is set. Returns 
{@code true} if
+     * attachment with given name it set and its value is equals to provided 
value. Otherwise returns
+     * {@code false}.
+     *
+     * @param name Name of the metadata.
+     * @param val Value to compare. Cannot be {@code null}.
+     * @param <V> Type of the value.
+     * @return {@code true} if metadata with given name is set and its value 
is equals to provided value -
+     *      otherwise returns {@code false}.
+     */
+    public <V> boolean hasMeta(String name, V val);
+
+    /**
+     * Replaces given metadata with new {@code newVal} value only if its 
current value
+     * is equal to {@code curVal}. Otherwise, it is no-op.
+     *
+     * @param name Name of the metadata.
+     * @param curVal Current value to check.
+     * @param newVal New value.
+     * @return {@code true} if replacement occurred, {@code false} otherwise.
+     */
+    public <V> boolean replaceMeta(String name, V curVal, V newVal);
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/550d6033/modules/core/src/main/java/org/apache/ignite/IgniteNotPeerDeployable.java
----------------------------------------------------------------------
diff --git 
a/modules/core/src/main/java/org/apache/ignite/IgniteNotPeerDeployable.java 
b/modules/core/src/main/java/org/apache/ignite/IgniteNotPeerDeployable.java
new file mode 100644
index 0000000..05cf0c8
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/IgniteNotPeerDeployable.java
@@ -0,0 +1,41 @@
+/*
+ * 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;
+
+import java.lang.annotation.*;
+
+/**
+ * This annotations should be used to mark any type that should not be
+ * peer deployable. Peer deployment will fail for this object as if
+ * class could not be found.
+ * <p>
+ * This annotation is used as <b>non-distribution assertion</b> and should be
+ * applied to classes and interfaces that should never be distributed via
+ * peer-to-peer deployment.
+ * <p>
+ * Note, however, that if class is already available on the remote node it
+ * will not be peer-loaded but will simply be locally class loaded. It may 
appear
+ * as if it was successfully peer-loaded when in fact it was simply already
+ * available on the remote node.
+ */
+@Documented
+@Retention(RetentionPolicy.RUNTIME)
+@Target({ElementType.TYPE})
+public @interface IgniteNotPeerDeployable {
+    // No-op.
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/550d6033/modules/core/src/main/java/org/apache/ignite/IgnitePortables.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/IgnitePortables.java 
b/modules/core/src/main/java/org/apache/ignite/IgnitePortables.java
index a204386..9f8542a 100644
--- a/modules/core/src/main/java/org/apache/ignite/IgnitePortables.java
+++ b/modules/core/src/main/java/org/apache/ignite/IgnitePortables.java
@@ -19,7 +19,6 @@ package org.apache.ignite;
 
 import org.apache.ignite.cache.*;
 import org.apache.ignite.portables.*;
-import org.gridgain.grid.cache.*;
 import org.jetbrains.annotations.*;
 
 import java.sql.*;

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/550d6033/modules/core/src/main/java/org/apache/ignite/IgniteTransactions.java
----------------------------------------------------------------------
diff --git 
a/modules/core/src/main/java/org/apache/ignite/IgniteTransactions.java 
b/modules/core/src/main/java/org/apache/ignite/IgniteTransactions.java
index bc85829..05277a1 100644
--- a/modules/core/src/main/java/org/apache/ignite/IgniteTransactions.java
+++ b/modules/core/src/main/java/org/apache/ignite/IgniteTransactions.java
@@ -20,7 +20,6 @@ package org.apache.ignite;
 import org.apache.ignite.cache.*;
 import org.apache.ignite.configuration.*;
 import org.apache.ignite.transactions.*;
-import org.gridgain.grid.cache.*;
 import org.jetbrains.annotations.*;
 
 /**

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/550d6033/modules/core/src/main/java/org/apache/ignite/Ignition.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/Ignition.java 
b/modules/core/src/main/java/org/apache/ignite/Ignition.java
index 1df3e0a..7abc5ac 100644
--- a/modules/core/src/main/java/org/apache/ignite/Ignition.java
+++ b/modules/core/src/main/java/org/apache/ignite/Ignition.java
@@ -19,7 +19,6 @@ package org.apache.ignite;
 
 import org.apache.ignite.configuration.*;
 import org.apache.ignite.lifecycle.*;
-import org.gridgain.grid.*;
 import org.gridgain.grid.kernal.*;
 import org.jetbrains.annotations.*;
 
@@ -338,10 +337,10 @@ public class Ignition {
      *
      * @return An instance of default no-name grid. This method never returns
      *      {@code null}.
-     * @throws GridIllegalStateException Thrown if default grid was not 
properly
+     * @throws IgniteIllegalStateException Thrown if default grid was not 
properly
      *      initialized or grid instance was stopped or was not started.
      */
-    public static Ignite ignite() throws GridIllegalStateException {
+    public static Ignite ignite() throws IgniteIllegalStateException {
         return GridGainEx.grid();
     }
 
@@ -363,10 +362,10 @@ public class Ignition {
      * @param locNodeId ID of local node the requested grid instance is 
managing.
      * @return An instance of named grid. This method never returns
      *      {@code null}.
-     * @throws GridIllegalStateException Thrown if grid was not properly
+     * @throws IgniteIllegalStateException Thrown if grid was not properly
      *      initialized or grid instance was stopped or was not started.
      */
-    public static Ignite ignite(UUID locNodeId) throws 
GridIllegalStateException {
+    public static Ignite ignite(UUID locNodeId) throws 
IgniteIllegalStateException {
         return GridGainEx.grid(locNodeId);
     }
 
@@ -383,10 +382,10 @@ public class Ignition {
      *      then grid instance belonging to a default no-name grid will be 
returned.
      * @return An instance of named grid. This method never returns
      *      {@code null}.
-     * @throws GridIllegalStateException Thrown if default grid was not 
properly
+     * @throws IgniteIllegalStateException Thrown if default grid was not 
properly
      *      initialized or grid instance was stopped or was not started.
      */
-    public static Ignite ignite(@Nullable String name) throws 
GridIllegalStateException {
+    public static Ignite ignite(@Nullable String name) throws 
IgniteIllegalStateException {
         return GridGainEx.grid(name);
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/550d6033/modules/core/src/main/java/org/apache/ignite/cache/CacheConfiguration.java
----------------------------------------------------------------------
diff --git 
a/modules/core/src/main/java/org/apache/ignite/cache/CacheConfiguration.java 
b/modules/core/src/main/java/org/apache/ignite/cache/CacheConfiguration.java
index 0f6d59b..68675a0 100644
--- a/modules/core/src/main/java/org/apache/ignite/cache/CacheConfiguration.java
+++ b/modules/core/src/main/java/org/apache/ignite/cache/CacheConfiguration.java
@@ -27,8 +27,6 @@ import org.apache.ignite.cache.store.*;
 import org.apache.ignite.configuration.*;
 import org.apache.ignite.portables.PortableObject;
 import org.apache.ignite.spi.indexing.*;
-import org.gridgain.grid.cache.*;
-import org.gridgain.grid.cache.affinity.*;
 import org.gridgain.grid.util.typedef.internal.*;
 import org.jetbrains.annotations.*;
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/550d6033/modules/core/src/main/java/org/apache/ignite/cache/CachePartialUpdateException.java
----------------------------------------------------------------------
diff --git 
a/modules/core/src/main/java/org/apache/ignite/cache/CachePartialUpdateException.java
 
b/modules/core/src/main/java/org/apache/ignite/cache/CachePartialUpdateException.java
index bd4499a..cdaeb05 100644
--- 
a/modules/core/src/main/java/org/apache/ignite/cache/CachePartialUpdateException.java
+++ 
b/modules/core/src/main/java/org/apache/ignite/cache/CachePartialUpdateException.java
@@ -17,9 +17,6 @@
 
 package org.apache.ignite.cache;
 
-import org.apache.ignite.cache.*;
-import org.gridgain.grid.cache.*;
-
 import javax.cache.*;
 import java.util.*;
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/550d6033/modules/core/src/main/java/org/apache/ignite/cache/GridCache.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/cache/GridCache.java 
b/modules/core/src/main/java/org/apache/ignite/cache/GridCache.java
index 3a613a4..3c099ff 100644
--- a/modules/core/src/main/java/org/apache/ignite/cache/GridCache.java
+++ b/modules/core/src/main/java/org/apache/ignite/cache/GridCache.java
@@ -18,13 +18,11 @@
 package org.apache.ignite.cache;
 
 import org.apache.ignite.*;
-import org.apache.ignite.cache.*;
 import org.apache.ignite.cache.affinity.*;
 import org.apache.ignite.cache.datastructures.*;
 import org.apache.ignite.cache.store.CacheStore;
 import org.apache.ignite.lang.*;
 import org.apache.ignite.transactions.*;
-import org.gridgain.grid.cache.affinity.*;
 import org.jetbrains.annotations.*;
 
 import java.util.*;

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/550d6033/modules/core/src/main/java/org/apache/ignite/cache/GridCacheEntry.java
----------------------------------------------------------------------
diff --git 
a/modules/core/src/main/java/org/apache/ignite/cache/GridCacheEntry.java 
b/modules/core/src/main/java/org/apache/ignite/cache/GridCacheEntry.java
index 120d59e..02bdddb 100644
--- a/modules/core/src/main/java/org/apache/ignite/cache/GridCacheEntry.java
+++ b/modules/core/src/main/java/org/apache/ignite/cache/GridCacheEntry.java
@@ -20,7 +20,6 @@ package org.apache.ignite.cache;
 import org.apache.ignite.*;
 import org.apache.ignite.lang.*;
 import org.apache.ignite.transactions.*;
-import org.gridgain.grid.*;
 import org.jetbrains.annotations.*;
 
 import javax.cache.*;
@@ -88,7 +87,7 @@ import java.util.Map.*;
  * @param <K> Key type.
  * @param <V> Value type.
  */
-public interface GridCacheEntry<K, V> extends Map.Entry<K, V>, 
GridMetadataAware, Cache.Entry<K, V> {
+public interface GridCacheEntry<K, V> extends Map.Entry<K, V>, 
IgniteMetadataAware, Cache.Entry<K, V> {
     /**
      * Cache projection to which this entry belongs. Note that entry and its
      * parent projections have same flags and filters.

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/550d6033/modules/core/src/main/java/org/apache/ignite/cache/GridCacheMode.java
----------------------------------------------------------------------
diff --git 
a/modules/core/src/main/java/org/apache/ignite/cache/GridCacheMode.java 
b/modules/core/src/main/java/org/apache/ignite/cache/GridCacheMode.java
index 128d0e8..27fc454 100644
--- a/modules/core/src/main/java/org/apache/ignite/cache/GridCacheMode.java
+++ b/modules/core/src/main/java/org/apache/ignite/cache/GridCacheMode.java
@@ -17,8 +17,6 @@
 
 package org.apache.ignite.cache;
 
-import org.apache.ignite.cache.*;
-import org.gridgain.grid.cache.affinity.*;
 import org.jetbrains.annotations.*;
 
 /**

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/550d6033/modules/core/src/main/java/org/apache/ignite/cache/GridCacheName.java
----------------------------------------------------------------------
diff --git 
a/modules/core/src/main/java/org/apache/ignite/cache/GridCacheName.java 
b/modules/core/src/main/java/org/apache/ignite/cache/GridCacheName.java
index 24068ee..367f4b9 100644
--- a/modules/core/src/main/java/org/apache/ignite/cache/GridCacheName.java
+++ b/modules/core/src/main/java/org/apache/ignite/cache/GridCacheName.java
@@ -17,8 +17,6 @@
 
 package org.apache.ignite.cache;
 
-import org.gridgain.grid.cache.affinity.*;
-
 import java.lang.annotation.*;
 import java.util.concurrent.*;
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/550d6033/modules/core/src/main/java/org/apache/ignite/cache/GridCachePreloadMode.java
----------------------------------------------------------------------
diff --git 
a/modules/core/src/main/java/org/apache/ignite/cache/GridCachePreloadMode.java 
b/modules/core/src/main/java/org/apache/ignite/cache/GridCachePreloadMode.java
index ea596e6..4ee32d9 100644
--- 
a/modules/core/src/main/java/org/apache/ignite/cache/GridCachePreloadMode.java
+++ 
b/modules/core/src/main/java/org/apache/ignite/cache/GridCachePreloadMode.java
@@ -17,8 +17,6 @@
 
 package org.apache.ignite.cache;
 
-import org.apache.ignite.cache.*;
-import org.gridgain.grid.cache.affinity.*;
 import org.jetbrains.annotations.*;
 
 /**

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/550d6033/modules/core/src/main/java/org/apache/ignite/cache/GridCacheProjection.java
----------------------------------------------------------------------
diff --git 
a/modules/core/src/main/java/org/apache/ignite/cache/GridCacheProjection.java 
b/modules/core/src/main/java/org/apache/ignite/cache/GridCacheProjection.java
index 3f681b1..15977f3 100644
--- 
a/modules/core/src/main/java/org/apache/ignite/cache/GridCacheProjection.java
+++ 
b/modules/core/src/main/java/org/apache/ignite/cache/GridCacheProjection.java
@@ -18,13 +18,11 @@
 package org.apache.ignite.cache;
 
 import org.apache.ignite.*;
-import org.apache.ignite.cache.*;
 import org.apache.ignite.cache.query.*;
 import org.apache.ignite.cache.store.*;
 import org.apache.ignite.cluster.*;
 import org.apache.ignite.lang.*;
 import org.apache.ignite.transactions.*;
-import org.gridgain.grid.cache.affinity.*;
 import org.jetbrains.annotations.*;
 
 import java.sql.*;

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/550d6033/modules/core/src/main/java/org/apache/ignite/cache/affinity/GridCacheAffinityFunction.java
----------------------------------------------------------------------
diff --git 
a/modules/core/src/main/java/org/apache/ignite/cache/affinity/GridCacheAffinityFunction.java
 
b/modules/core/src/main/java/org/apache/ignite/cache/affinity/GridCacheAffinityFunction.java
index 439c231..c33bcad 100644
--- 
a/modules/core/src/main/java/org/apache/ignite/cache/affinity/GridCacheAffinityFunction.java
+++ 
b/modules/core/src/main/java/org/apache/ignite/cache/affinity/GridCacheAffinityFunction.java
@@ -19,7 +19,6 @@ package org.apache.ignite.cache.affinity;
 
 import org.apache.ignite.cache.*;
 import org.apache.ignite.cluster.*;
-import org.gridgain.grid.cache.*;
 
 import java.io.*;
 import java.util.*;

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/550d6033/modules/core/src/main/java/org/apache/ignite/cache/affinity/GridCacheAffinityKeyMapped.java
----------------------------------------------------------------------
diff --git 
a/modules/core/src/main/java/org/apache/ignite/cache/affinity/GridCacheAffinityKeyMapped.java
 
b/modules/core/src/main/java/org/apache/ignite/cache/affinity/GridCacheAffinityKeyMapped.java
index 4b38e01..3cbac33 100644
--- 
a/modules/core/src/main/java/org/apache/ignite/cache/affinity/GridCacheAffinityKeyMapped.java
+++ 
b/modules/core/src/main/java/org/apache/ignite/cache/affinity/GridCacheAffinityKeyMapped.java
@@ -17,8 +17,6 @@
 
 package org.apache.ignite.cache.affinity;
 
-import org.gridgain.grid.cache.*;
-
 import java.lang.annotation.*;
 import java.util.concurrent.*;
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/550d6033/modules/core/src/main/java/org/apache/ignite/cache/affinity/consistenthash/GridCacheConsistentHashAffinityFunction.java
----------------------------------------------------------------------
diff --git 
a/modules/core/src/main/java/org/apache/ignite/cache/affinity/consistenthash/GridCacheConsistentHashAffinityFunction.java
 
b/modules/core/src/main/java/org/apache/ignite/cache/affinity/consistenthash/GridCacheConsistentHashAffinityFunction.java
index 628e852..01b470a 100644
--- 
a/modules/core/src/main/java/org/apache/ignite/cache/affinity/consistenthash/GridCacheConsistentHashAffinityFunction.java
+++ 
b/modules/core/src/main/java/org/apache/ignite/cache/affinity/consistenthash/GridCacheConsistentHashAffinityFunction.java
@@ -23,8 +23,6 @@ import org.apache.ignite.cache.affinity.*;
 import org.apache.ignite.cluster.*;
 import org.apache.ignite.lang.*;
 import org.apache.ignite.resources.*;
-import org.gridgain.grid.*;
-import org.gridgain.grid.cache.affinity.*;
 import org.gridgain.grid.util.*;
 import org.gridgain.grid.util.tostring.*;
 import org.gridgain.grid.util.typedef.*;
@@ -591,7 +589,7 @@ public class GridCacheConsistentHashAffinityFunction 
implements GridCacheAffinit
                 try {
                     U.await(initLatch);
                 }
-                catch (GridInterruptedException ignored) {
+                catch (IgniteInterruptedException ignored) {
                     // Recover interrupted state flag.
                     Thread.currentThread().interrupt();
                 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/550d6033/modules/core/src/main/java/org/apache/ignite/cache/affinity/fair/GridCachePartitionFairAffinity.java
----------------------------------------------------------------------
diff --git 
a/modules/core/src/main/java/org/apache/ignite/cache/affinity/fair/GridCachePartitionFairAffinity.java
 
b/modules/core/src/main/java/org/apache/ignite/cache/affinity/fair/GridCachePartitionFairAffinity.java
index cfcf16b..d3d5c28 100644
--- 
a/modules/core/src/main/java/org/apache/ignite/cache/affinity/fair/GridCachePartitionFairAffinity.java
+++ 
b/modules/core/src/main/java/org/apache/ignite/cache/affinity/fair/GridCachePartitionFairAffinity.java
@@ -22,7 +22,6 @@ import org.apache.ignite.cache.affinity.*;
 import org.apache.ignite.cluster.*;
 import org.apache.ignite.events.*;
 import org.apache.ignite.lang.*;
-import org.gridgain.grid.cache.affinity.*;
 import org.gridgain.grid.util.typedef.*;
 import org.gridgain.grid.util.typedef.internal.*;
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/550d6033/modules/core/src/main/java/org/apache/ignite/cache/affinity/rendezvous/GridCacheRendezvousAffinityFunction.java
----------------------------------------------------------------------
diff --git 
a/modules/core/src/main/java/org/apache/ignite/cache/affinity/rendezvous/GridCacheRendezvousAffinityFunction.java
 
b/modules/core/src/main/java/org/apache/ignite/cache/affinity/rendezvous/GridCacheRendezvousAffinityFunction.java
index 088c389..b6e41a6 100644
--- 
a/modules/core/src/main/java/org/apache/ignite/cache/affinity/rendezvous/GridCacheRendezvousAffinityFunction.java
+++ 
b/modules/core/src/main/java/org/apache/ignite/cache/affinity/rendezvous/GridCacheRendezvousAffinityFunction.java
@@ -24,7 +24,6 @@ import org.apache.ignite.cluster.*;
 import org.apache.ignite.lang.*;
 import org.apache.ignite.marshaller.*;
 import org.apache.ignite.marshaller.optimized.*;
-import org.gridgain.grid.cache.affinity.*;
 import org.gridgain.grid.kernal.*;
 import org.gridgain.grid.util.typedef.*;
 import org.gridgain.grid.util.typedef.internal.*;

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/550d6033/modules/core/src/main/java/org/apache/ignite/cache/cloner/GridCacheCloner.java
----------------------------------------------------------------------
diff --git 
a/modules/core/src/main/java/org/apache/ignite/cache/cloner/GridCacheCloner.java
 
b/modules/core/src/main/java/org/apache/ignite/cache/cloner/GridCacheCloner.java
index e873d4a..892ee71 100644
--- 
a/modules/core/src/main/java/org/apache/ignite/cache/cloner/GridCacheCloner.java
+++ 
b/modules/core/src/main/java/org/apache/ignite/cache/cloner/GridCacheCloner.java
@@ -19,7 +19,6 @@ package org.apache.ignite.cache.cloner;
 
 import org.apache.ignite.*;
 import org.apache.ignite.cache.*;
-import org.gridgain.grid.cache.*;
 import org.jetbrains.annotations.*;
 
 /**

Reply via email to