You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by ag...@apache.org on 2017/04/10 14:18:07 UTC

[17/17] ignite git commit: IGNITE-4535 - Add option to store deserialized values on-heap

IGNITE-4535 - Add option to store deserialized values on-heap


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

Branch: refs/heads/ignite-3477-master
Commit: c56c4b8c6a648c5901515399071bba7844cedb7a
Parents: 7102d53
Author: Ilya Lantukh <il...@gridgain.com>
Authored: Mon Apr 10 14:48:54 2017 +0300
Committer: Alexey Goncharuk <al...@gmail.com>
Committed: Mon Apr 10 17:17:59 2017 +0300

----------------------------------------------------------------------
 .../apache/ignite/cache/CacheMemoryMode.java    |   63 -
 .../org/apache/ignite/cache/CachePeekMode.java  |    5 +-
 .../configuration/CacheConfiguration.java       |  309 +--
 .../configuration/CollectionConfiguration.java  |   16 -
 .../configuration/NearCacheConfiguration.java   |    1 -
 .../communication/GridIoMessageFactory.java     |   12 -
 .../processors/cache/CacheEvictionManager.java  |   15 -
 .../processors/cache/CacheMetricsImpl.java      |   11 +-
 .../cache/CacheOffheapEvictionManager.java      |   15 -
 .../processors/cache/GridCacheAdapter.java      |   97 +-
 .../processors/cache/GridCacheAttributes.java   |   14 -
 .../cache/GridCacheConcurrentMap.java           |    2 +
 .../cache/GridCacheConcurrentMapImpl.java       |    3 +-
 .../cache/GridCacheEvictionManager.java         | 1755 +-----------------
 .../cache/GridCacheEvictionRequest.java         |  248 ---
 .../cache/GridCacheEvictionResponse.java        |  225 ---
 .../processors/cache/GridCacheIoManager.java    |   14 -
 .../processors/cache/GridCacheProcessor.java    |   21 +-
 .../processors/cache/GridCacheUtils.java        |    8 -
 .../processors/cache/GridNoStorageCacheMap.java |    1 +
 .../cache/IgniteCacheOffheapManagerImpl.java    |    6 +-
 .../GridDistributedCacheAdapter.java            |    6 +
 .../distributed/dht/GridDhtCacheAdapter.java    |   29 +-
 .../distributed/dht/GridDhtLocalPartition.java  |  103 +-
 .../dht/GridDhtPartitionTopologyImpl.java       |    2 +-
 .../distributed/near/GridNearCacheAdapter.java  |   18 -
 .../datastructures/DataStructuresProcessor.java |    1 -
 .../utils/PlatformConfigurationUtils.java       |   17 +-
 .../visor/cache/VisorCacheConfiguration.java    |    1 -
 .../cache/VisorCacheEvictionConfiguration.java  |   55 -
 .../cache/VisorCacheQueryConfiguration.java     |    1 -
 .../internal/ClusterNodeMetricsSelfTest.java    |   46 +-
 .../BinaryObjectOffHeapUnswapTemporaryTest.java |  365 ----
 .../cache/CacheConfigurationLeakTest.java       |    1 +
 .../cache/CacheDeferredDeleteQueueTest.java     |    2 +-
 ...cheDhtLocalPartitionAfterRemoveSelfTest.java |    2 +-
 .../cache/CacheEnumOperationsAbstractTest.java  |   52 +-
 .../cache/CacheGetEntryAbstractTest.java        |    2 +
 ...erceptorPartitionCounterLocalSanityTest.java |   25 +-
 ...torPartitionCounterRandomOperationsTest.java |  111 --
 .../cache/CacheNearReaderUpdateTest.java        |   12 -
 .../cache/CacheOffheapMapEntrySelfTest.java     |   42 +-
 .../CachePutEventListenerErrorSelfTest.java     |   41 +-
 .../CacheSerializableTransactionsTest.java      |   54 +-
 .../cache/CacheSwapUnswapGetTest.java           |  310 ----
 ...idAbstractCacheInterceptorRebalanceTest.java |    6 -
 ...ridCacheAbstractByteArrayValuesSelfTest.java |    6 -
 .../cache/GridCacheAbstractFullApiSelfTest.java |  183 +-
 .../GridCacheAbstractLocalStoreSelfTest.java    |   13 -
 .../GridCacheAbstractRemoveFailureTest.java     |   26 +-
 .../cache/GridCacheAbstractSelfTest.java        |   21 +-
 .../cache/GridCacheClearSelfTest.java           |  113 +-
 .../GridCacheConcurrentTxMultiNodeTest.java     |    2 +-
 ...idCacheConfigurationConsistencySelfTest.java |   41 +-
 .../GridCacheDeploymentOffHeapSelfTest.java     |   41 -
 ...ridCacheDeploymentOffHeapValuesSelfTest.java |   41 -
 .../GridCacheEvictionEventAbstractTest.java     |    5 +-
 ...heInterceptorAtomicOffheapRebalanceTest.java |   30 -
 ...GridCacheInterceptorAtomicRebalanceTest.java |    6 -
 ...ceptorTransactionalOffheapRebalanceTest.java |   35 -
 ...heInterceptorTransactionalRebalanceTest.java |    6 -
 .../cache/GridCacheLifecycleAwareSelfTest.java  |    1 +
 .../cache/GridCacheMemoryModeSelfTest.java      |  285 ---
 .../cache/GridCacheObjectToStringSelfTest.java  |    1 +
 ...HeapMultiThreadedUpdateAbstractSelfTest.java |    3 -
 .../GridCacheOffHeapTieredAbstractSelfTest.java |  679 -------
 .../GridCacheOffHeapTieredAtomicSelfTest.java   |   32 -
 ...heOffHeapTieredEvictionAbstractSelfTest.java |  364 ----
 ...acheOffHeapTieredEvictionAtomicSelfTest.java |   32 -
 .../GridCacheOffHeapTieredEvictionSelfTest.java |   33 -
 .../cache/GridCacheOffHeapTieredSelfTest.java   |   33 -
 .../cache/GridCacheOffheapUpdateSelfTest.java   |    3 -
 .../cache/GridCachePartitionedGetSelfTest.java  |    1 -
 ...chePartitionedOffHeapLocalStoreSelfTest.java |   44 -
 .../GridCachePreloadingEvictionsSelfTest.java   |    7 +-
 .../cache/GridCachePutAllFailoverSelfTest.java  |   58 -
 .../cache/GridCacheReloadSelfTest.java          |    4 +-
 .../cache/GridCacheSwapPreloadSelfTest.java     |    1 -
 .../GridCacheTtlManagerEvictionSelfTest.java    |   26 +-
 .../GridCacheValueBytesPreloadingSelfTest.java  |   44 -
 .../cache/IgniteCacheAbstractTest.java          |    6 +-
 ...IgniteCacheAtomicPutAllFailoverSelfTest.java |    5 -
 .../IgniteCacheConfigVariationsFullApiTest.java |  189 +-
 .../IgniteCacheEntryListenerAbstractTest.java   |   11 -
 ...cheEntryListenerAtomicOffheapTieredTest.java |   32 -
 ...cheEntryListenerAtomicOffheapValuesTest.java |   32 -
 ...niteCacheEntryListenerExpiredEventsTest.java |   26 +-
 ...teCacheEntryListenerTxOffheapTieredTest.java |   32 -
 ...teCacheEntryListenerTxOffheapValuesTest.java |   32 -
 ...niteCacheExpireAndUpdateConsistencyTest.java |   35 +-
 .../IgniteCacheInterceptorSelfTestSuite.java    |    2 -
 ...gniteCacheInvokeReadThroughAbstractTest.java |    4 -
 ...iteCacheInvokeReadThroughSingleNodeTest.java |   32 +-
 .../cache/IgniteCacheInvokeReadThroughTest.java |   92 +-
 ...gniteCacheLoadRebalanceEvictionSelfTest.java |    3 +-
 ...gniteCacheP2pUnmarshallingNearErrorTest.java |    4 +-
 .../cache/IgniteCachePeekModesAbstractTest.java |  117 +-
 .../IgniteCacheReadThroughEvictionSelfTest.java |   10 +-
 .../IgniteCacheStoreValueAbstractTest.java      |    5 -
 .../cache/IgniteCacheTxNearPeekModesTest.java   |    5 +-
 .../cache/IgniteCacheTxPeekModesTest.java       |    5 +-
 .../cache/OffHeapTieredTransactionSelfTest.java |  134 --
 ...heapCacheMetricsForClusterGroupSelfTest.java |    3 -
 .../GridCacheBinaryObjectsAbstractSelfTest.java |   26 +-
 ...AtomicNearDisabledOffheapTieredSelfTest.java |   29 -
 ...ObjectsAtomicNearDisabledOnheapSelfTest.java |    8 +
 ...inaryObjectsAtomicOffheapTieredSelfTest.java |   29 -
 ...dCacheBinaryObjectsAtomicOnheapSelfTest.java |    8 +
 ...tionedNearDisabledOffheapTieredSelfTest.java |   30 -
 ...tsPartitionedNearDisabledOnheapSelfTest.java |    8 +
 ...ObjectsPartitionedOffheapTieredSelfTest.java |   30 -
 ...eBinaryObjectsPartitionedOnheapSelfTest.java |    8 +
 .../dht/GridCacheMemoryModeBinarySelfTest.java  |   36 -
 ...dCacheOffHeapTieredAtomicBinarySelfTest.java |   48 -
 .../GridCacheOffHeapTieredBinarySelfTest.java   |   48 -
 ...fHeapTieredEvictionAtomicBinarySelfTest.java |   96 -
 ...acheOffHeapTieredEvictionBinarySelfTest.java |   96 -
 ...BinaryObjectsLocalOffheapTieredSelfTest.java |   29 -
 ...idCacheBinaryObjectsLocalOnheapSelfTest.java |   26 +
 .../GridCacheQueueCleanupSelfTest.java          |    7 -
 ...dCacheQueueMultiNodeConsistencySelfTest.java |    7 -
 .../IgniteCollectionAbstractTest.java           |    7 -
 .../IgniteDataStructureUniqueNameTest.java      |    7 -
 .../GridCacheLocalAtomicOffheapSetSelfTest.java |   32 -
 .../GridCacheLocalAtomicQueueApiSelfTest.java   |    6 -
 .../local/GridCacheLocalAtomicSetSelfTest.java  |    7 -
 .../GridCacheLocalOffheapQueueApiSelfTest.java  |   31 -
 .../local/GridCacheLocalQueueApiSelfTest.java   |    7 -
 .../local/GridCacheLocalSetSelfTest.java        |    7 -
 ...artitionedAtomicOffheapQueueApiSelfTest.java |   32 -
 ...omicOffheapQueueCreateMultiNodeSelfTest.java |   32 -
 ...onedAtomicOffheapQueueMultiNodeSelfTest.java |   32 -
 ...dCachePartitionedAtomicQueueApiSelfTest.java |    7 -
 ...ionedAtomicQueueCreateMultiNodeSelfTest.java |    7 -
 ...micQueueFailoverDataConsistencySelfTest.java |    7 -
 ...PartitionedAtomicQueueMultiNodeSelfTest.java |    7 -
 ...itionedAtomicQueueRotativeMultiNodeTest.java |    7 -
 ...chePartitionedAtomicSetFailoverSelfTest.java |    7 -
 ...rtitionedDataStructuresFailoverSelfTest.java |    7 -
 ...artitionedOffHeapValuesQueueApiSelfTest.java |   35 -
 ...achePartitionedOffHeapValuesSetSelfTest.java |   32 -
 ...edOffheapDataStructuresFailoverSelfTest.java |   33 -
 ...hePartitionedOffheapSetFailoverSelfTest.java |   32 -
 .../GridCachePartitionedQueueApiSelfTest.java   |    7 -
 ...PartitionedQueueCreateMultiNodeSelfTest.java |    7 -
 ...dCachePartitionedQueueEntryMoveSelfTest.java |    7 -
 ...nedQueueFailoverDataConsistencySelfTest.java |    7 -
 ...CachePartitionedQueueJoinedNodeSelfTest.java |    7 -
 ...dCachePartitionedQueueMultiNodeSelfTest.java |    7 -
 ...hePartitionedQueueRotativeMultiNodeTest.java |    7 -
 ...GridCachePartitionedSetFailoverSelfTest.java |    7 -
 .../GridCachePartitionedSetSelfTest.java        |    7 -
 .../IgnitePartitionedQueueNoBackupsTest.java    |    7 -
 ...eplicatedDataStructuresFailoverSelfTest.java |    7 -
 .../GridCacheReplicatedQueueApiSelfTest.java    |    7 -
 ...idCacheReplicatedQueueMultiNodeSelfTest.java |    7 -
 ...cheReplicatedQueueRotativeMultiNodeTest.java |    7 -
 .../GridCacheReplicatedSetSelfTest.java         |    7 -
 ...tractDistributedByteArrayValuesSelfTest.java |  120 +-
 .../GridCacheAbstractNodeRestartSelfTest.java   |  116 +-
 ...tractPartitionedByteArrayValuesSelfTest.java |   33 -
 .../IgniteCacheAtomicNodeRestartTest.java       |    5 -
 .../IgniteCacheNearOffheapGetSelfTest.java      |  136 --
 .../IgniteCacheTxIteratorSelfTest.java          |   34 +-
 ...tPartitionedOnlyByteArrayValuesSelfTest.java |   55 +-
 ...eColocatedOptimisticTransactionSelfTest.java |    1 -
 ...GridCacheDhtEvictionNearReadersSelfTest.java |    6 +-
 .../dht/GridCacheDhtEvictionSelfTest.java       |  356 ----
 .../dht/GridCacheDhtPreloadOffHeapSelfTest.java |   38 -
 .../dht/GridCacheDhtPreloadOnheapSelfTest.java  |   26 +
 .../dht/GridCacheDhtPreloadSelfTest.java        |   38 +-
 ...icOffHeapTieredMultiNodeFullApiSelfTest.java |   43 -
 ...NearDisabledAtomicOnheapFullApiSelfTest.java |   38 +
 ...ledAtomicOnheapMultiNodeFullApiSelfTest.java |   28 +
 ...ionedNearDisabledOffHeapFullApiSelfTest.java |   33 -
 ...DisabledOffHeapMultiNodeFullApiSelfTest.java |   33 -
 ...abledOffHeapTieredAtomicFullApiSelfTest.java |   57 -
 ...earDisabledOffHeapTieredFullApiSelfTest.java |   33 -
 ...edOffHeapTieredMultiNodeFullApiSelfTest.java |   33 -
 ...tionedNearDisabledOnheapFullApiSelfTest.java |   27 +
 ...rDisabledOnheapMultiNodeFullApiSelfTest.java |   27 +
 .../dht/IgniteCacheConcurrentPutGetRemove.java  |   27 +-
 .../IgniteCacheCrossCacheTxFailoverTest.java    |   46 +-
 .../dht/IgniteCacheMultiTxLockSelfTest.java     |    1 +
 .../IgniteCachePutRetryAbstractSelfTest.java    |   53 +-
 .../dht/IgniteCachePutRetryAtomicSelfTest.java  |    2 +-
 ...gniteCachePutRetryTransactionalSelfTest.java |   28 +-
 ...eAtomicInvalidPartitionHandlingSelfTest.java |   65 +-
 ...tomicClientOnlyMultiNodeFullApiSelfTest.java |  113 +-
 ...eAtomicNearOnlyMultiNodeFullApiSelfTest.java |   52 -
 .../GridCacheAtomicOffHeapFullApiSelfTest.java  |   67 -
 ...heAtomicOffHeapMultiNodeFullApiSelfTest.java |   68 -
 ...CacheAtomicOffHeapTieredFullApiSelfTest.java |   32 -
 ...icOffHeapTieredMultiNodeFullApiSelfTest.java |   33 -
 .../GridCacheAtomicOnheapFullApiSelfTest.java   |   28 +
 ...cheAtomicOnheapMultiNodeFullApiSelfTest.java |   26 +
 ...PrimaryWriteOrderOffHeapFullApiSelfTest.java |   32 -
 ...yWriteOrderOffHeapTieredFullApiSelfTest.java |   33 -
 ...cPrimaryWriteOrderOnheapFullApiSelfTest.java |   28 +
 ...riteOrderOnheapMultiNodeFullApiSelfTest.java |   29 +
 ...ityOrderOffHeapMultiNodeFullApiSelfTest.java |   33 -
 ...erOffHeapTieredMultiNodeFullApiSelfTest.java |   33 -
 .../near/GridCacheNearReadersSelfTest.java      |    1 -
 ...ePartitionedAtomicOnheapFullApiSelfTest.java |   38 +
 ...nedAtomicOnheapMultiNodeFullApiSelfTest.java |   28 +
 .../GridCachePartitionedEvictionSelfTest.java   |    1 +
 .../GridCachePartitionedFullApiSelfTest.java    |    2 -
 ...achePartitionedMultiNodeFullApiSelfTest.java |   43 -
 ...ePartitionedMultiThreadedPutGetSelfTest.java |    2 +-
 ...dCachePartitionedOffHeapFullApiSelfTest.java |   32 -
 ...titionedOffHeapMultiNodeFullApiSelfTest.java |   32 -
 ...PartitionedOffHeapTieredFullApiSelfTest.java |   32 -
 ...edOffHeapTieredMultiNodeFullApiSelfTest.java |   74 -
 ...idCachePartitionedOnheapFullApiSelfTest.java |   26 +
 ...rtitionedOnheapMultiNodeFullApiSelfTest.java |   26 +
 .../GridNearOffheapCacheStoreUpdateTest.java    |   35 -
 ...stractReplicatedByteArrayValuesSelfTest.java |   29 -
 .../GridCacheReplicatedEvictionSelfTest.java    |  135 --
 ...idCacheReplicatedOffHeapFullApiSelfTest.java |   32 -
 ...plicatedOffHeapMultiNodeFullApiSelfTest.java |   32 -
 ...eReplicatedOffHeapTieredFullApiSelfTest.java |   33 -
 ...edOffHeapTieredMultiNodeFullApiSelfTest.java |   33 -
 ...ridCacheReplicatedOnheapFullApiSelfTest.java |   26 +
 ...eplicatedOnheapMultiNodeFullApiSelfTest.java |   26 +
 ...idCacheReplicatedPreloadOffHeapSelfTest.java |   37 -
 .../cache/eviction/EvictionAbstractTest.java    |   37 +-
 .../GridCacheBatchEvictUnswapSelfTest.java      |  199 --
 ...heConcurrentEvictionConsistencySelfTest.java |    6 +-
 .../GridCacheConcurrentEvictionsSelfTest.java   |    1 +
 .../GridCacheDistributedEvictionsSelfTest.java  |  235 ---
 .../GridCacheEmptyEntriesAbstractSelfTest.java  |    4 +-
 .../GridCacheEvictableEntryEqualsSelfTest.java  |    1 +
 .../GridCacheEvictionFilterSelfTest.java        |    2 +-
 .../GridCacheEvictionLockUnlockSelfTest.java    |    1 +
 .../GridCacheEvictionTouchSelfTest.java         |   12 +-
 ...cheSynchronousEvictionsFailoverSelfTest.java |  166 --
 .../lru/LruNearEvictionPolicySelfTest.java      |   28 -
 .../LruNearOnlyNearEvictionPolicySelfTest.java  |   55 -
 .../SortedEvictionPolicyPerformanceTest.java    |    2 +-
 ...CacheAtomicLocalOffheapExpiryPolicyTest.java |   30 -
 ...eCacheAtomicLocalOnheapExpiryPolicyTest.java |   26 +
 ...gniteCacheAtomicOffheapExpiryPolicyTest.java |   30 -
 ...IgniteCacheAtomicOnheapExpiryPolicyTest.java |   26 +
 ...rimaryWriteOrderOffheapExpiryPolicyTest.java |   31 -
 ...teOrderWithStoreOffheapExpiryPolicyTest.java |   31 -
 ...AtomicReplicatedOffheapExpiryPolicyTest.java |   30 -
 ...eAtomicWithStoreOffheapExpiryPolicyTest.java |   30 -
 .../IgniteCacheExpiryPolicyAbstractTest.java    |   13 -
 .../IgniteCacheExpiryPolicyTestSuite.java       |   13 -
 ...niteCacheTxLocalOffheapExpiryPolicyTest.java |   30 -
 .../IgniteCacheTxOffheapExpiryPolicyTest.java   |   30 -
 ...acheTxReplicatedOffheapExpiryPolicyTest.java |   30 -
 ...CacheTxWithStoreOffheapExpiryPolicyTest.java |   30 -
 ...dCacheLocalAtomicOffHeapFullApiSelfTest.java |   42 -
 ...LocalAtomicOffHeapTieredFullApiSelfTest.java |   32 -
 .../GridCacheLocalByteArrayValuesSelfTest.java  |   64 +-
 .../GridCacheLocalOffHeapFullApiSelfTest.java   |   30 -
 ...dCacheLocalOffHeapTieredFullApiSelfTest.java |   32 -
 ...LocalCacheOffHeapAndSwapMetricsSelfTest.java |  463 -----
 ...cheAtomicOffHeapMultiJvmFullApiSelfTest.java |   31 -
 ...micOffHeapTieredMultiJvmFullApiSelfTest.java |   36 -
 ...acheAtomicOnheapMultiJvmFullApiSelfTest.java |   28 +
 ...WriteOrderOnheapMultiJvmFullApiSelfTest.java |   29 +
 ...rityOrderOffHeapMultiJvmFullApiSelfTest.java |   31 -
 ...derOffHeapTieredMultiJvmFullApiSelfTest.java |   36 -
 ...micOffHeapTieredMultiJvmFullApiSelfTest.java |   36 -
 ...bledAtomicOnheapMultiJvmFullApiSelfTest.java |   28 +
 ...rDisabledOffHeapMultiJvmFullApiSelfTest.java |   31 -
 ...ledOffHeapTieredMultiJvmFullApiSelfTest.java |   36 -
 ...arDisabledOnheapMultiJvmFullApiSelfTest.java |   28 +
 ...rtitionedOffHeapMultiJvmFullApiSelfTest.java |   31 -
 ...nedOffHeapTieredMultiJvmFullApiSelfTest.java |   36 -
 ...artitionedOnheapMultiJvmFullApiSelfTest.java |   28 +
 ...eplicatedOffHeapMultiJvmFullApiSelfTest.java |   31 -
 ...tedOffHeapTieredMultiJvmFullApiSelfTest.java |   36 -
 ...ReplicatedOnheapMultiJvmFullApiSelfTest.java |   28 +
 .../GridCacheSwapScanQueryAbstractSelfTest.java |  450 -----
 .../query/GridCacheSwapScanQuerySelfTest.java   |   25 -
 .../IgniteCacheQueryCacheDestroySelfTest.java   |    2 -
 .../continuous/CacheContinuousBatchAckTest.java |   73 +-
 ...eContinuousQueryAsyncFilterListenerTest.java |  170 +-
 ...acheContinuousQueryExecuteInPrimaryTest.java |    2 -
 ...usQueryFactoryFilterRandomOperationTest.java |    2 -
 ...ContinuousQueryFailoverAbstractSelfTest.java |   10 -
 ...tomicPrimaryWriteOrderOffheapTieredTest.java |   33 -
 ...tinuousQueryFailoverTxOffheapTieredTest.java |   32 -
 .../CacheContinuousQueryOperationP2PTest.java   |   31 +-
 .../CacheContinuousQueryOrderingEventTest.java  |  127 +-
 ...acheContinuousQueryRandomOperationsTest.java |  246 ---
 ...CacheKeepBinaryIterationNearEnabledTest.java |    6 +-
 ...acheKeepBinaryIterationStoreEnabledTest.java |    6 +-
 .../CacheKeepBinaryIterationTest.java           |  105 +-
 ...ridCacheContinuousQueryAbstractSelfTest.java |   10 -
 ...eContinuousQueryAtomicOffheapTieredTest.java |   32 -
 ...eContinuousQueryAtomicOffheapValuesTest.java |   32 -
 ...CacheContinuousQueryTxOffheapTieredTest.java |   32 -
 ...CacheContinuousQueryTxOffheapValuesTest.java |   32 -
 ...edEntryPartitionedAtomicOffHeapSelfTest.java |   35 -
 ...PartitionedTransactionalOffHeapSelfTest.java |   36 -
 ...nedEntryReplicatedAtomicOffHeapSelfTest.java |   35 -
 ...yReplicatedTransactionalOffHeapSelfTest.java |   36 -
 .../DataStreamProcessorSelfTest.java            |    2 -
 .../igfs/IgfsAbstractBaseSelfTest.java          |   18 -
 .../processors/igfs/IgfsAbstractSelfTest.java   |   11 -
 .../IgfsAtomicPrimaryOffheapTieredSelfTest.java |   39 -
 .../IgfsAtomicPrimaryOffheapValuesSelfTest.java |   39 -
 ...sCachePerBlockLruEvictionPolicySelfTest.java |    1 +
 .../processors/igfs/IgfsMaxSizeSelfTest.java    |    1 -
 .../igfs/IgfsPrimaryOffheapTieredSelfTest.java  |   33 -
 .../igfs/IgfsPrimaryOffheapValuesSelfTest.java  |   33 -
 .../loadtests/GridCacheMultiNodeLoadTest.java   |    1 +
 .../GridCachePartitionedAtomicLongLoadTest.java |    2 +-
 ...gniteClientReconnectMassiveShutdownTest.java |    3 -
 .../ignite/testframework/GridTestUtils.java     |  113 --
 .../configvariations/ConfigVariations.java      |   26 +-
 ...IgniteCacheConfigVariationsAbstractTest.java |   38 +-
 .../junits/common/GridCommonAbstractTest.java   |   10 +-
 .../ConfigVariationsTestSuiteBuilderTest.java   |    4 +-
 .../ignite/testsuites/IgniteBasicTestSuite.java |    2 -
 .../testsuites/IgniteBinaryCacheTestSuite.java  |   19 -
 .../IgniteBinaryObjectsTestSuite.java           |   13 -
 .../IgniteCacheDataStructuresSelfTestSuite.java |   18 -
 .../IgniteCacheEvictionSelfTestSuite.java       |    8 -
 ...IgniteCacheFullApiMultiJvmSelfTestSuite.java |   37 +-
 .../IgniteCacheFullApiSelfTestSuite.java        |   84 +-
 .../IgniteCacheMetricsSelfTestSuite.java        |    2 -
 .../ignite/testsuites/IgniteCacheTestSuite.java |   27 +-
 .../testsuites/IgniteCacheTestSuite2.java       |   24 +-
 .../testsuites/IgniteCacheTestSuite3.java       |   14 +-
 .../testsuites/IgniteCacheTestSuite4.java       |    9 -
 .../ignite/testsuites/IgniteIgfsTestSuite.java  |    9 -
 .../impl/HadoopAbstractMapReduceTest.java       |    1 -
 .../igfs/HadoopFIleSystemFactorySelfTest.java   |    1 -
 .../igfs/HadoopIgfsDualAbstractSelfTest.java    |    1 -
 .../cache/CacheIndexStreamerTest.java           |    3 -
 .../cache/CacheIndexingOffheapCleanupTest.java  |  157 --
 .../CacheOffheapBatchIndexingBaseTest.java      |    6 +-
 .../CacheOffheapBatchIndexingMultiTypeTest.java |   17 +-
 ...CacheOffheapBatchIndexingSingleTypeTest.java |   20 +-
 .../CacheOperationsWithExpirationTest.java      |   68 +-
 .../cache/CacheQueryEvictDataLostTest.java      |  120 ++
 .../CacheQueryOffheapEvictDataLostTest.java     |  136 --
 .../CacheRandomOperationsMultithreadedTest.java |   64 +-
 .../ClientReconnectAfterClusterRestartTest.java |   16 +-
 .../cache/GridCacheOffHeapSelfTest.java         |    1 -
 .../GridCacheOffheapIndexEntryEvictTest.java    |    4 -
 .../cache/GridCacheOffheapIndexGetSelfTest.java |    8 -
 .../cache/GridCacheQuerySimpleBenchmark.java    |    3 -
 .../cache/GridIndexingWithNoopSwapSelfTest.java |    3 +-
 ...aryObjectQueryArgumentsOffheapLocalTest.java |   28 -
 ...teBinaryObjectQueryArgumentsOffheapTest.java |   30 -
 .../IgniteBinaryObjectQueryArgumentsTest.java   |   11 -
 ...CacheDistributedJoinQueryConditionsTest.java |   16 -
 .../cache/IgniteCacheOffheapEvictQueryTest.java |    6 -
 .../cache/IgniteCacheOffheapIndexScanTest.java  |    3 -
 ...CacheOffheapTieredMultithreadedSelfTest.java |  301 ---
 .../cache/IgniteCacheQueriesLoadTest1.java      |    2 -
 ...QueryMultiThreadedOffHeapTieredSelfTest.java |   38 -
 .../IgniteCacheQueryMultiThreadedSelfTest.java  |   24 +-
 ...QueryOffheapEvictsMultiThreadedSelfTest.java |   28 -
 ...eCacheQueryOffheapMultiThreadedSelfTest.java |   28 -
 .../cache/IgniteCrossCachesJoinsQueryTest.java  |   16 -
 ...QueryNodeRestartDistributedJoinSelfTest.java |   50 +-
 .../cache/ttl/CacheTtlAbstractSelfTest.java     |    9 +-
 .../ttl/CacheTtlAtomicAbstractSelfTest.java     |   29 +
 .../cache/ttl/CacheTtlAtomicLocalSelfTest.java  |   34 +
 .../ttl/CacheTtlAtomicPartitionedSelfTest.java  |   34 +
 .../ttl/CacheTtlOffheapAbstractSelfTest.java    |   29 -
 .../CacheTtlOffheapAtomicAbstractSelfTest.java  |   29 -
 .../ttl/CacheTtlOffheapAtomicLocalSelfTest.java |   34 -
 ...acheTtlOffheapAtomicPartitionedSelfTest.java |   34 -
 ...TtlOffheapTransactionalAbstractSelfTest.java |   29 -
 ...cheTtlOffheapTransactionalLocalSelfTest.java |   34 -
 ...OffheapTransactionalPartitionedSelfTest.java |   34 -
 .../ttl/CacheTtlOnheapAbstractSelfTest.java     |   29 -
 .../CacheTtlOnheapAtomicAbstractSelfTest.java   |   29 -
 .../ttl/CacheTtlOnheapAtomicLocalSelfTest.java  |   34 -
 ...CacheTtlOnheapAtomicPartitionedSelfTest.java |   34 -
 ...eTtlOnheapTransactionalAbstractSelfTest.java |   29 -
 ...acheTtlOnheapTransactionalLocalSelfTest.java |   34 -
 ...lOnheapTransactionalPartitionedSelfTest.java |   34 -
 .../CacheTtlTransactionalAbstractSelfTest.java  |   29 +
 .../ttl/CacheTtlTransactionalLocalSelfTest.java |   34 +
 ...acheTtlTransactionalPartitionedSelfTest.java |   34 +
 .../query/IgniteSqlSegmentedIndexSelfTest.java  |    3 +-
 .../IgniteCacheQuerySelfTestSuite.java          |   11 +-
 .../IgniteCacheQuerySelfTestSuite2.java         |   13 +-
 .../IgniteCacheQuerySelfTestSuite3.java         |    8 -
 .../IgniteCacheQuerySelfTestSuite4.java         |    4 -
 .../IgniteCacheWithIndexingTestSuite.java       |   26 +-
 .../Cache/CacheConfigurationTest.cs             |    9 -
 .../IgniteConfigurationSerializerTest.cs        |    2 -
 .../Cache/Configuration/CacheConfiguration.cs   |   23 -
 .../commands/cache/VisorCacheCommand.scala      |    5 -
 modules/yardstick/README.txt                    |    2 -
 .../config/benchmark-atomic-win.properties      |    8 +-
 .../config/benchmark-atomic.properties          |    6 -
 .../config/benchmark-client-mode.properties     |    9 -
 .../config/benchmark-multicast.properties       |    9 -
 .../config/benchmark-query-win.properties       |    5 +-
 .../yardstick/config/benchmark-query.properties |    3 -
 .../config/benchmark-tx-win.properties          |    6 +-
 .../yardstick/config/benchmark-tx.properties    |    4 -
 modules/yardstick/config/benchmark.properties   |   11 +-
 modules/yardstick/config/ignite-base-config.xml |   63 -
 .../config/ignite-base-load-config.xml          |   63 -
 .../config/ignite-cache-load-config.xml         |    5 -
 .../config/ignite-failover-base-config.xml      |   37 -
 .../ignite-int-max-values-offheap-config.xml    |   88 -
 .../ignite-int-max-values-swap-config.xml       |   93 -
 .../test-max-int-values-offheap.properties      |   65 -
 .../config/test-max-int-values-swap.properties  |   64 -
 .../yardstick/IgniteBenchmarkArguments.java     |   29 -
 .../org/apache/ignite/yardstick/IgniteNode.java |   10 -
 .../cache/IgniteGetOffHeapBenchmark.java        |   32 -
 .../cache/IgniteGetOffHeapValuesBenchmark.java  |   32 -
 .../cache/IgnitePutGetOffHeapBenchmark.java     |   32 -
 .../IgnitePutGetOffHeapValuesBenchmark.java     |   32 -
 .../cache/IgnitePutGetTxOffHeapBenchmark.java   |   32 -
 .../IgnitePutGetTxOffHeapValuesBenchmark.java   |   32 -
 .../cache/IgnitePutOffHeapBenchmark.java        |   32 -
 .../cache/IgnitePutOffHeapValuesBenchmark.java  |   32 -
 .../IgnitePutRandomValueSizeBenchmark.java      |    2 +-
 .../cache/IgnitePutTxOffHeapBenchmark.java      |   32 -
 .../IgnitePutTxOffHeapValuesBenchmark.java      |   32 -
 .../IgniteSqlQueryJoinOffHeapBenchmark.java     |   32 -
 .../cache/IgniteSqlQueryOffHeapBenchmark.java   |   32 -
 .../IgniteSqlQueryPutOffHeapBenchmark.java      |   32 -
 ...IgniteAtomicOffHeapInvokeRetryBenchmark.java |   31 -
 .../IgniteAtomicOffHeapRetriesBenchmark.java    |   31 -
 ...ransactionalOffHeapInvokeRetryBenchmark.java |   33 -
 ...ransactionalOffHeapWriteInvokeBenchmark.java |   37 -
 ...eTransactionalOffHeapWriteReadBenchmark.java |   32 -
 .../IgniteCacheRandomOperationBenchmark.java    |    7 -
 434 files changed, 1764 insertions(+), 17076 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/c56c4b8c/modules/core/src/main/java/org/apache/ignite/cache/CacheMemoryMode.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/cache/CacheMemoryMode.java b/modules/core/src/main/java/org/apache/ignite/cache/CacheMemoryMode.java
deleted file mode 100644
index 0133327..0000000
--- a/modules/core/src/main/java/org/apache/ignite/cache/CacheMemoryMode.java
+++ /dev/null
@@ -1,63 +0,0 @@
-/*
- * 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.cache;
-
-import org.apache.ignite.cache.eviction.EvictionPolicy;
-
-/**
- * Defines set of memory modes. Memory modes help control whether cache entries are
- * stored on heap memory, offheap memory, or in swap space.
- */
-public enum CacheMemoryMode {
-    /**
-     * Entries will be stored on-heap first. The onheap tiered storage works as follows:
-     * <nl>
-     * <li>Entries are cached on heap memory first.</li>
-     * <li>
-     *     If offheap memory is enabled and eviction policy evicts an entry from heap memory, entry will
-     *     be moved to offheap memory. If offheap memory is disabled, then entry is simply discarded.
-     * </li>
-     * <li>
-     *     If swap space is enabled and offheap memory fills up, then entry will be evicted into swap space.
-     *     If swap space is disabled, then entry will be discarded. If swap is enabled and offheap memory
-     *     is disabled, then entry will be evicted directly from heap memory into swap.
-     * </li>
-     * </nl>
-     * <p>
-     * <b>Note</b> that heap memory evictions are handled by configured {@link EvictionPolicy}
-     * implementation. By default, no eviction policy is enabled, so entries never leave heap
-     * memory space unless explicitly removed.
-     */
-    ONHEAP_TIERED,
-
-    /**
-     * Works the same as {@link #ONHEAP_TIERED}, except that entries never end up in heap memory and get
-     * stored in offheap memory right away. Entries get cached in offheap memory first and then
-     * get evicted to swap, if one is configured.
-     */
-    OFFHEAP_TIERED,
-
-    /**
-     * Entry keys will be stored on heap memory, and values will be stored in offheap memory. Note
-     * that in this mode entries can be evicted only to swap. The evictions will happen according
-     * to configured {@link EvictionPolicy}.
-     * <p/>
-     * Size returned by {@link CachePeekMode#OFFHEAP} is always zero, for this mode.
-     */
-    OFFHEAP_VALUES,
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/c56c4b8c/modules/core/src/main/java/org/apache/ignite/cache/CachePeekMode.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/cache/CachePeekMode.java b/modules/core/src/main/java/org/apache/ignite/cache/CachePeekMode.java
index 2bfee84..19928e8 100644
--- a/modules/core/src/main/java/org/apache/ignite/cache/CachePeekMode.java
+++ b/modules/core/src/main/java/org/apache/ignite/cache/CachePeekMode.java
@@ -63,10 +63,7 @@ public enum CachePeekMode {
     ONHEAP,
 
     /** Peeks value from the off-heap storage only, without loading off-heap value into cache. */
-    OFFHEAP,
-
-    /** Peeks value from the swap storage only, without loading swapped value into cache. */
-    SWAP;
+    OFFHEAP;
 
     /** Enumerated values. */
     private static final CachePeekMode[] VALS = values();

http://git-wip-us.apache.org/repos/asf/ignite/blob/c56c4b8c/modules/core/src/main/java/org/apache/ignite/configuration/CacheConfiguration.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/configuration/CacheConfiguration.java b/modules/core/src/main/java/org/apache/ignite/configuration/CacheConfiguration.java
index a9cb38e..cfdfed8 100644
--- a/modules/core/src/main/java/org/apache/ignite/configuration/CacheConfiguration.java
+++ b/modules/core/src/main/java/org/apache/ignite/configuration/CacheConfiguration.java
@@ -48,7 +48,6 @@ import org.apache.ignite.cache.CacheAtomicWriteOrderMode;
 import org.apache.ignite.cache.CacheAtomicityMode;
 import org.apache.ignite.cache.CacheEntryProcessor;
 import org.apache.ignite.cache.CacheInterceptor;
-import org.apache.ignite.cache.CacheMemoryMode;
 import org.apache.ignite.cache.CacheMode;
 import org.apache.ignite.cache.CacheRebalanceMode;
 import org.apache.ignite.cache.CacheWriteSynchronizationMode;
@@ -148,12 +147,6 @@ public class CacheConfiguration<K, V> extends MutableConfiguration<K, V> {
     /** Default maximum eviction queue ratio. */
     public static final float DFLT_MAX_EVICTION_OVERFLOW_RATIO = 10;
 
-    /** Default eviction synchronized flag. */
-    public static final boolean DFLT_EVICT_SYNCHRONIZED = false;
-
-    /** Default eviction key buffer size for batching synchronized evicts. */
-    public static final int DFLT_EVICT_KEY_BUFFER_SIZE = 1024;
-
     /** Default synchronous eviction timeout in milliseconds. */
     public static final long DFLT_EVICT_SYNCHRONIZED_TIMEOUT = 10000;
 
@@ -163,9 +156,6 @@ public class CacheConfiguration<K, V> extends MutableConfiguration<K, V> {
     /** Default value for eager ttl flag. */
     public static final boolean DFLT_EAGER_TTL = true;
 
-    /** Default off-heap storage size is {@code -1} which means that off-heap storage is disabled. */
-    public static final long DFLT_OFFHEAP_MEMORY = -1;
-
     /** Default value for 'maxConcurrentAsyncOps'. */
     public static final int DFLT_MAX_CONCURRENT_ASYNC_OPS = 500;
 
@@ -205,9 +195,6 @@ public class CacheConfiguration<K, V> extends MutableConfiguration<K, V> {
     /** Default number of queries detail metrics to collect. */
     public static final int DFLT_QRY_DETAIL_METRICS_SIZE = 0;
 
-    /** Default size for onheap SQL row cache size. */
-    public static final int DFLT_SQL_ONHEAP_ROW_CACHE_SIZE = 10 * 1024;
-
     /** Default value for keep binary in store behavior . */
     @SuppressWarnings({"UnnecessaryBoxing", "BooleanConstructorCall"})
     public static final Boolean DFLT_STORE_KEEP_BINARY = new Boolean(false);
@@ -240,24 +227,12 @@ public class CacheConfiguration<K, V> extends MutableConfiguration<K, V> {
     /** Cache expiration policy. */
     private EvictionPolicy evictPlc;
 
-    /** Flag indicating whether eviction is synchronized. */
-    private boolean evictSync = DFLT_EVICT_SYNCHRONIZED;
-
-    /** Eviction key buffer size. */
-    private int evictKeyBufSize = DFLT_EVICT_KEY_BUFFER_SIZE;
-
-    /** Synchronous eviction concurrency level. */
-    private int evictSyncConcurrencyLvl = DFLT_EVICT_SYNCHRONIZED_CONCURRENCY_LEVEL;
-
-    /** Synchronous eviction timeout. */
-    private long evictSyncTimeout = DFLT_EVICT_SYNCHRONIZED_TIMEOUT;
+    /** */
+    private boolean onheapCache;
 
     /** Eviction filter. */
     private EvictionFilter<?, ?> evictFilter;
 
-    /** Maximum eviction overflow ratio. */
-    private float evictMaxOverflowRatio = DFLT_MAX_EVICTION_OVERFLOW_RATIO;
-
     /** Eager ttl flag. */
     private boolean eagerTtl = DFLT_EAGER_TTL;
 
@@ -318,9 +293,6 @@ public class CacheConfiguration<K, V> extends MutableConfiguration<K, V> {
     /** Rebalance batches prefetch count. */
     private long rebalanceBatchesPrefetchCount = DFLT_REBALANCE_BATCHES_PREFETCH_COUNT;
 
-    /** Off-heap memory size. */
-    private long offHeapMaxMem = DFLT_OFFHEAP_MEMORY;
-
     /** Maximum number of concurrent asynchronous operations. */
     private int maxConcurrentAsyncOps = DFLT_MAX_CONCURRENT_ASYNC_OPS;
 
@@ -382,9 +354,6 @@ public class CacheConfiguration<K, V> extends MutableConfiguration<K, V> {
     private boolean sqlEscapeAll;
 
     /** */
-    private int sqlOnheapRowCacheSize = DFLT_SQL_ONHEAP_ROW_CACHE_SIZE;
-
-    /** */
     private transient Class<?>[] indexedTypes;
 
     /** Copy on read flag. */
@@ -449,12 +418,7 @@ public class CacheConfiguration<K, V> extends MutableConfiguration<K, V> {
         dfltLockTimeout = cc.getDefaultLockTimeout();
         eagerTtl = cc.isEagerTtl();
         evictFilter = cc.getEvictionFilter();
-        evictKeyBufSize = cc.getEvictSynchronizedKeyBufferSize();
-        evictMaxOverflowRatio = cc.getEvictMaxOverflowRatio();
         evictPlc = cc.getEvictionPolicy();
-        evictSync = cc.isEvictSynchronized();
-        evictSyncConcurrencyLvl = cc.getEvictSynchronizedConcurrencyLevel();
-        evictSyncTimeout = cc.getEvictSynchronizedTimeout();
         expiryPolicyFactory = cc.getExpiryPolicyFactory();
         indexedTypes = cc.getIndexedTypes();
         interceptor = cc.getInterceptor();
@@ -466,13 +430,13 @@ public class CacheConfiguration<K, V> extends MutableConfiguration<K, V> {
         listenerConfigurations = cc.listenerConfigurations;
         loadPrevVal = cc.isLoadPreviousValue();
         longQryWarnTimeout = cc.getLongQueryWarningTimeout();
-        offHeapMaxMem = cc.getOffHeapMaxMemory();
         maxConcurrentAsyncOps = cc.getMaxConcurrentAsyncOperations();
         memPlcName = cc.getMemoryPolicyName();
         sqlIndexMaxInlineSize = cc.getSqlIndexMaxInlineSize();
         name = cc.getName();
         nearCfg = cc.getNearConfiguration();
         nodeFilter = cc.getNodeFilter();
+        onheapCache = cc.isOnheapCacheEnabled();
         partitionLossPolicy = cc.getPartitionLossPolicy();
         pluginCfgs = cc.getPluginConfigurations();
         qryEntities = cc.getQueryEntities() == Collections.<QueryEntity>emptyList() ? null : cc.getQueryEntities();
@@ -489,7 +453,6 @@ public class CacheConfiguration<K, V> extends MutableConfiguration<K, V> {
         sqlSchema = cc.getSqlSchema();
         sqlEscapeAll = cc.isSqlEscapeAll();
         sqlFuncCls = cc.getSqlFunctionClasses();
-        sqlOnheapRowCacheSize = cc.getSqlOnheapRowCacheSize();
         startSize = cc.getStartSize();
         storeFactory = cc.getCacheStoreFactory();
         storeSesLsnrs = cc.getCacheStoreSessionListenerFactories();
@@ -573,6 +536,25 @@ public class CacheConfiguration<K, V> extends MutableConfiguration<K, V> {
     }
 
     /**
+     * @return On-heap cache enabled flag.
+     */
+    public boolean isOnheapCacheEnabled() {
+        return onheapCache;
+    }
+
+    /**
+     * Configures on-heap cache.
+     *
+     * @param onheapCache {@code True} if on-heap cache should be enabled.
+     * @return {@code this} for chaining.
+     */
+    public CacheConfiguration<K, V> setOnheapCacheEnabled(boolean onheapCache) {
+        this.onheapCache = onheapCache;
+
+        return this;
+    }
+
+    /**
      * @return Near enabled flag.
      */
     public NearCacheConfiguration<K, V> getNearConfiguration() {
@@ -637,153 +619,6 @@ public class CacheConfiguration<K, V> extends MutableConfiguration<K, V> {
     }
 
     /**
-     * Gets flag indicating whether eviction is synchronized between primary, backup and near nodes.
-     * If this parameter is {@code true} and swap is disabled then {@link IgniteCache#localEvict(Collection)}
-     * will involve all nodes where an entry is kept.  If this property is set to {@code false} then
-     * eviction is done independently on different cache nodes.
-     * <p>
-     * Default value is defined by {@link #DFLT_EVICT_SYNCHRONIZED}.
-     * <p>
-     * Note that it's not recommended to set this value to {@code true} if cache
-     * store is configured since it will allow to significantly improve cache
-     * performance.
-     *
-     * @return {@code true} If eviction is synchronized with backup nodes (or the
-     *      rest of the nodes in case of replicated cache), {@code false} if not.
-     */
-    public boolean isEvictSynchronized() {
-        return evictSync;
-    }
-
-    /**
-     * Sets flag indicating whether eviction is synchronized with backup nodes or near caches
-     * (or the rest of the nodes for replicated cache).
-     *
-     * @param evictSync {@code true} if synchronized, {@code false} if not.
-     * @return {@code this} for chaining.
-     */
-    public CacheConfiguration<K, V> setEvictSynchronized(boolean evictSync) {
-        this.evictSync = evictSync;
-
-        return this;
-    }
-
-    /**
-     * Gets size of the key buffer for synchronized evictions.
-     * <p>
-     * Default value is defined by {@link #DFLT_EVICT_KEY_BUFFER_SIZE}.
-     *
-     * @return Eviction key buffer size.
-     */
-    public int getEvictSynchronizedKeyBufferSize() {
-        return evictKeyBufSize;
-    }
-
-    /**
-     * Sets eviction key buffer size.
-     *
-     * @param evictKeyBufSize Eviction key buffer size.
-     * @return {@code this} for chaining.
-     */
-    public CacheConfiguration<K, V> setEvictSynchronizedKeyBufferSize(int evictKeyBufSize) {
-        this.evictKeyBufSize = evictKeyBufSize;
-
-        return this;
-    }
-
-    /**
-     * Gets concurrency level for synchronized evictions. This flag only makes sense
-     * with {@link #isEvictSynchronized()} set
-     * to {@code true}. When synchronized evictions are enabled, it is possible that
-     * local eviction policy will try to evict entries faster than evictions can be
-     * synchronized with backup or near nodes. This value specifies how many concurrent
-     * synchronous eviction sessions should be allowed before the system is forced to
-     * wait and let synchronous evictions catch up with the eviction policy.
-     * <p>
-     * Note that if synchronous evictions start lagging, it is possible that you have either
-     * too big or too small eviction key buffer size or small eviction timeout. In that case
-     * you will need to adjust {@link #getEvictSynchronizedKeyBufferSize} or
-     * {@link #getEvictSynchronizedTimeout()} values as well.
-     * <p>
-     * Default value is defined by {@link #DFLT_EVICT_SYNCHRONIZED_CONCURRENCY_LEVEL}.
-     *
-     * @return Synchronous eviction concurrency level.
-     */
-    public int getEvictSynchronizedConcurrencyLevel() {
-        return evictSyncConcurrencyLvl;
-    }
-
-    /**
-     * Sets concurrency level for synchronized evictions.
-     *
-     * @param evictSyncConcurrencyLvl Concurrency level for synchronized evictions.
-     * @return {@code this} for chaining.
-     */
-    public CacheConfiguration<K, V> setEvictSynchronizedConcurrencyLevel(int evictSyncConcurrencyLvl) {
-        this.evictSyncConcurrencyLvl = evictSyncConcurrencyLvl;
-
-        return this;
-    }
-
-    /**
-     * Gets timeout for synchronized evictions.
-     * <p>
-     * Node that initiates eviction waits for responses
-     * from remote nodes within this timeout.
-     * <p>
-     * Default value is defined by {@link #DFLT_EVICT_SYNCHRONIZED_TIMEOUT}.
-     *
-     * @return Synchronous eviction timeout.
-     */
-    public long getEvictSynchronizedTimeout() {
-        return evictSyncTimeout;
-    }
-
-    /**
-     * Sets timeout for synchronized evictions.
-     *
-     * @param evictSyncTimeout Timeout for synchronized evictions.
-     * @return {@code this} for chaining.
-     */
-    public CacheConfiguration<K, V> setEvictSynchronizedTimeout(long evictSyncTimeout) {
-        this.evictSyncTimeout = evictSyncTimeout;
-
-        return this;
-    }
-
-    /**
-     * This value denotes the maximum size of eviction queue in percents of cache
-     * size in case of distributed cache (replicated and partitioned) and using
-     * synchronized eviction (that is if {@link #isEvictSynchronized()} returns
-     * {@code true}).
-     * <p>
-     * That queue is used internally as a buffer to decrease network costs for
-     * synchronized eviction. Once queue size reaches specified value all required
-     * requests for all entries in the queue are sent to remote nodes and the queue
-     * is cleared.
-     * <p>
-     * Default value is defined by {@link #DFLT_MAX_EVICTION_OVERFLOW_RATIO} and
-     * equals to {@code 10%}.
-     *
-     * @return Maximum size of eviction queue in percents of cache size.
-     */
-    public float getEvictMaxOverflowRatio() {
-        return evictMaxOverflowRatio;
-    }
-
-    /**
-     * Sets maximum eviction overflow ratio.
-     *
-     * @param evictMaxOverflowRatio Maximum eviction overflow ratio.
-     * @return {@code this} for chaining.
-     */
-    public CacheConfiguration<K, V> setEvictMaxOverflowRatio(float evictMaxOverflowRatio) {
-        this.evictMaxOverflowRatio = evictMaxOverflowRatio;
-
-        return this;
-    }
-
-    /**
      * Gets eviction filter to specify which entries should not be evicted
      * (except explicit evict by calling {@link IgniteCache#localEvict(Collection)}).
      * If {@link EvictionFilter#evictAllowed(Cache.Entry)} method
@@ -1624,53 +1459,6 @@ public class CacheConfiguration<K, V> extends MutableConfiguration<K, V> {
     }
 
     /**
-     * Gets maximum amount of memory available to off-heap storage. Possible values are
-     * <ul>
-     * <li>{@code -1} - Means that off-heap storage is disabled.</li>
-     * <li>
-     *     {@code 0} - Ignite will not limit off-heap storage (it's up to user to properly
-     *     add and remove entries from cache to ensure that off-heap storage does not grow
-     *     indefinitely.
-     * </li>
-     * <li>Any positive value specifies the limit of off-heap storage in bytes.</li>
-     * </ul>
-     * Default value is {@code -1}, specified by {@link #DFLT_OFFHEAP_MEMORY} constant
-     * which means that off-heap storage is disabled by default.
-     * <p>
-     * Use off-heap storage to load gigabytes of data in memory without slowing down
-     * Garbage Collection. Essentially in this case you should allocate very small amount
-     * of memory to JVM and Ignite will cache most of the data in off-heap space
-     * without affecting JVM performance at all.
-     * <p>
-     * Note that Ignite will throw an exception if max memory is set to {@code -1} and
-     * {@code offHeapValuesOnly} flag is set to {@code true}.
-     *
-     * @return Maximum memory in bytes available to off-heap memory space.
-     */
-    public long getOffHeapMaxMemory() {
-        return offHeapMaxMem;
-    }
-
-    /**
-     * Sets maximum amount of memory available to off-heap storage. Possible values are <ul> <li>{@code -1} - Means that
-     * off-heap storage is disabled.</li> <li> {@code 0} - Ignite will not limit off-heap storage (it's up to user to
-     * properly add and remove entries from cache to ensure that off-heap storage does not grow infinitely. </li>
-     * <li>Any positive value specifies the limit of off-heap storage in bytes.</li> </ul> Default value is {@code -1},
-     * specified by {@link #DFLT_OFFHEAP_MEMORY} constant which means that off-heap storage is disabled by default. <p>
-     * Use off-heap storage to load gigabytes of data in memory without slowing down Garbage Collection. Essentially in
-     * this case you should allocate very small amount of memory to JVM and Ignite will cache most of the data in
-     * off-heap space without affecting JVM performance at all.
-     *
-     * @param offHeapMaxMem Maximum memory in bytes available to off-heap memory space.
-     * @return {@code this} for chaining.
-     */
-    public CacheConfiguration<K, V> setOffHeapMaxMemory(long offHeapMaxMem) {
-        this.offHeapMaxMem = offHeapMaxMem;
-
-        return this;
-    }
-
-    /**
      * Gets maximum number of query iterators that can be stored. Iterators are stored to
      * support query pagination when each page of data is sent to user's node only on demand.
      * Increase this property if you are running and processing lots of queries in parallel.
@@ -1696,28 +1484,6 @@ public class CacheConfiguration<K, V> extends MutableConfiguration<K, V> {
     }
 
     /**
-     * Gets memory mode for cache. Memory mode helps control whether value is stored in on-heap memory,
-     * off-heap memory, or swap space. Refer to {@link CacheMemoryMode} for more info.
-     *
-     * @return Memory mode.
-     */
-    public CacheMemoryMode getMemoryMode() {
-        return null;
-    }
-
-    /**
-     * Sets memory mode for cache.
-     *
-     * @param memMode Memory mode.
-     * @return {@code this} for chaining.
-     */
-    public CacheConfiguration<K, V> setMemoryMode(CacheMemoryMode memMode) {
-        //this.memMode = memMode;
-
-        return this;
-    }
-
-    /**
      * Gets cache interceptor.
      *
      * @return Cache interceptor.
@@ -1998,31 +1764,6 @@ public class CacheConfiguration<K, V> extends MutableConfiguration<K, V> {
     }
 
     /**
-     * Number of SQL rows which will be cached onheap to avoid deserialization on each SQL index access.
-     * This setting only makes sense when offheap is enabled for this cache.
-     *
-     * @return Cache size.
-     * @see #setOffHeapMaxMemory(long)
-     */
-    public int getSqlOnheapRowCacheSize() {
-        return sqlOnheapRowCacheSize;
-    }
-
-    /**
-     * Number of SQL rows which will be cached onheap to avoid deserialization on each SQL index access.
-     * This setting only makes sense when offheap is enabled for this cache.
-     *
-     * @param size Cache size.
-     * @see #setOffHeapMaxMemory(long)
-     * @return {@code this} for chaining.
-     */
-    public CacheConfiguration<K, V> setSqlOnheapRowCacheSize(int size) {
-        this.sqlOnheapRowCacheSize = size;
-
-        return this;
-    }
-
-    /**
      * Gets array of cache plugin configurations.
      *
      * @return Cache plugin configurations.
@@ -2132,7 +1873,7 @@ public class CacheConfiguration<K, V> extends MutableConfiguration<K, V> {
      * @see #getQueryParallelism()
      * @return {@code this} for chaining.
      */
-    public CacheConfiguration<K,V> setQueryParallelism(int qryParallelism) {
+    public CacheConfiguration<K, V> setQueryParallelism(int qryParallelism) {
         this.qryParallelism = qryParallelism;
 
         return this;
@@ -2542,7 +2283,7 @@ public class CacheConfiguration<K, V> extends MutableConfiguration<K, V> {
     /**
      *  Filter that accepts all nodes.
      */
-    public static class IgniteAllNodesPredicate  implements IgnitePredicate<ClusterNode> {
+    public static class IgniteAllNodesPredicate implements IgnitePredicate<ClusterNode> {
         /** */
         private static final long serialVersionUID = 0L;
 
@@ -2799,7 +2540,7 @@ public class CacheConfiguration<K, V> extends MutableConfiguration<K, V> {
 
             if (descending) {
                 if (descendings == null)
-                    descendings  = new HashSet<>();
+                    descendings = new HashSet<>();
 
                 descendings.add(field);
             }

http://git-wip-us.apache.org/repos/asf/ignite/blob/c56c4b8c/modules/core/src/main/java/org/apache/ignite/configuration/CollectionConfiguration.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/configuration/CollectionConfiguration.java b/modules/core/src/main/java/org/apache/ignite/configuration/CollectionConfiguration.java
index db1eadf..a890f51 100644
--- a/modules/core/src/main/java/org/apache/ignite/configuration/CollectionConfiguration.java
+++ b/modules/core/src/main/java/org/apache/ignite/configuration/CollectionConfiguration.java
@@ -19,7 +19,6 @@ package org.apache.ignite.configuration;
 
 import java.io.Serializable;
 import org.apache.ignite.cache.CacheAtomicityMode;
-import org.apache.ignite.cache.CacheMemoryMode;
 import org.apache.ignite.cache.CacheMode;
 import org.apache.ignite.cluster.ClusterNode;
 import org.apache.ignite.internal.util.typedef.internal.S;
@@ -108,21 +107,6 @@ public class CollectionConfiguration implements Serializable {
     }
 
     /**
-     * @return Cache memory mode.
-     */
-    public CacheMemoryMode getMemoryMode() {
-        return null;
-    }
-
-    /**
-     * @param memoryMode Memory mode.
-     * @return {@code this} for chaining.
-     */
-    public CollectionConfiguration setMemoryMode(CacheMemoryMode memoryMode) {
-        return this;
-    }
-
-    /**
      * @return Predicate specifying on which nodes the cache should be started.
      */
     public IgnitePredicate<ClusterNode> getNodeFilter() {

http://git-wip-us.apache.org/repos/asf/ignite/blob/c56c4b8c/modules/core/src/main/java/org/apache/ignite/configuration/NearCacheConfiguration.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/configuration/NearCacheConfiguration.java b/modules/core/src/main/java/org/apache/ignite/configuration/NearCacheConfiguration.java
index 7c948dc..0fc94655 100644
--- a/modules/core/src/main/java/org/apache/ignite/configuration/NearCacheConfiguration.java
+++ b/modules/core/src/main/java/org/apache/ignite/configuration/NearCacheConfiguration.java
@@ -71,7 +71,6 @@ public class NearCacheConfiguration<K, V> extends MutableConfiguration<K, V> {
      *
      * @return Near eviction policy.
      * @see CacheConfiguration#getEvictionPolicy()
-     * @see CacheConfiguration#isEvictSynchronized()
      */
     public EvictionPolicy<K, V> getNearEvictionPolicy() {
         return nearEvictPlc;

http://git-wip-us.apache.org/repos/asf/ignite/blob/c56c4b8c/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoMessageFactory.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoMessageFactory.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoMessageFactory.java
index ac3858c..8488d70 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoMessageFactory.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoMessageFactory.java
@@ -45,8 +45,6 @@ import org.apache.ignite.internal.processors.cache.CacheInvokeDirectResult;
 import org.apache.ignite.internal.processors.cache.CacheObjectByteArrayImpl;
 import org.apache.ignite.internal.processors.cache.CacheObjectImpl;
 import org.apache.ignite.internal.processors.cache.GridCacheEntryInfo;
-import org.apache.ignite.internal.processors.cache.GridCacheEvictionRequest;
-import org.apache.ignite.internal.processors.cache.GridCacheEvictionResponse;
 import org.apache.ignite.internal.processors.cache.GridCacheReturn;
 import org.apache.ignite.internal.processors.cache.KeyCacheObjectImpl;
 import org.apache.ignite.internal.processors.cache.binary.MetadataRequestMessage;
@@ -371,16 +369,6 @@ public class GridIoMessageFactory implements MessageFactory {
 
                 break;
 
-            case 14:
-                msg = new GridCacheEvictionRequest();
-
-                break;
-
-            case 15:
-                msg = new GridCacheEvictionResponse();
-
-                break;
-
             case 16:
                 msg = new GridCacheTxRecoveryRequest();
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/c56c4b8c/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheEvictionManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheEvictionManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheEvictionManager.java
index d536a98..b614728 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheEvictionManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheEvictionManager.java
@@ -41,11 +41,6 @@ public interface CacheEvictionManager extends GridCacheManager {
     public void touch(GridCacheEntryEx e, AffinityTopologyVersion topVer);
 
     /**
-     * Notifications.
-     */
-    public void unwind();
-
-    /**
      * @param entry Entry to attempt to evict.
      * @param obsoleteVer Obsolete version.
      * @param filter Optional entry filter.
@@ -60,19 +55,9 @@ public interface CacheEvictionManager extends GridCacheManager {
         @Nullable CacheEntryPredicate[] filter) throws IgniteCheckedException;
 
     /**
-     * @return Current size of evict queue.
-     */
-    public int evictQueueSize();
-
-    /**
      * @param keys Keys to evict.
      * @param obsoleteVer Obsolete version.
      * @throws IgniteCheckedException In case of error.
      */
     public void batchEvict(Collection<?> keys, @Nullable GridCacheVersion obsoleteVer) throws IgniteCheckedException;
-
-    /**
-     * @return {@code True} if either evicts or near evicts are synchronized, {@code false} otherwise.
-     */
-    public boolean evictSyncOrNearSync();
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/c56c4b8c/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheMetricsImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheMetricsImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheMetricsImpl.java
index eb63a60..aceef97 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheMetricsImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheMetricsImpl.java
@@ -236,7 +236,7 @@ public class CacheMetricsImpl implements CacheMetrics {
 
     /** {@inheritDoc} */
     @Override public long getOffHeapMaxSize() {
-        return cctx.config().getOffHeapMaxMemory();
+        return 0;
     }
 
     /** {@inheritDoc} */
@@ -260,14 +260,7 @@ public class CacheMetricsImpl implements CacheMetrics {
 
     /** {@inheritDoc} */
     @Override public int getDhtEvictQueueCurrentSize() {
-        GridCacheContext<?, ?> ctx = cctx.isNear() ? dhtCtx : cctx;
-
-        if (ctx == null)
-            return -1;
-
-        CacheEvictionManager evictMgr = ctx.evicts();
-
-        return evictMgr != null ? evictMgr.evictQueueSize() : -1;
+        return -1;
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/c56c4b8c/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheOffheapEvictionManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheOffheapEvictionManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheOffheapEvictionManager.java
index e6a9ee7..99df39d 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheOffheapEvictionManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheOffheapEvictionManager.java
@@ -62,11 +62,6 @@ public class CacheOffheapEvictionManager extends GridCacheManagerAdapter impleme
     }
 
     /** {@inheritDoc} */
-    @Override public void unwind() {
-        // No-op.
-    }
-
-    /** {@inheritDoc} */
     @Override public boolean evict(@Nullable GridCacheEntryEx entry,
         @Nullable GridCacheVersion obsoleteVer,
         boolean explicit,
@@ -75,17 +70,7 @@ public class CacheOffheapEvictionManager extends GridCacheManagerAdapter impleme
     }
 
     /** {@inheritDoc} */
-    @Override public int evictQueueSize() {
-        return 0;
-    }
-
-    /** {@inheritDoc} */
     @Override public void batchEvict(Collection<?> keys, @Nullable GridCacheVersion obsoleteVer) {
         // No-op.
     }
-
-    /** {@inheritDoc} */
-    @Override public boolean evictSyncOrNearSync() {
-        return false;
-    }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/c56c4b8c/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java
----------------------------------------------------------------------
diff --git 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
index 92d7423..93daeda 100644
--- 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
@@ -693,7 +693,8 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
     }
 
     /** {@inheritDoc} */
-    @Override public final Iterable<Cache.Entry<K, V>> localEntries(CachePeekMode[] peekModes) throws IgniteCheckedException {
+    @Override public final Iterable<Cache.Entry<K, V>> localEntries(
+        CachePeekMode[] peekModes) throws IgniteCheckedException {
         assert peekModes != null;
 
         ctx.checkSecurity(SecurityPermission.CACHE_READ);
@@ -707,9 +708,19 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
         if (ctx.isLocal()) {
             modes.primary = true;
             modes.backup = true;
+        }
+
+        if (modes.offheap) {
+            if (modes.heap && modes.near && ctx.isNear())
+                its.add(ctx.near().nearEntries().iterator());
+
+            if (modes.primary || modes.backup) {
+                AffinityTopologyVersion topVer = ctx.affinity().affinityTopologyVersion();
 
-            if (modes.heap)
-                its.add(iterator(map.entries().iterator(), !keepBinary));
+                IgniteCacheOffheapManager offheapMgr = ctx.isNear() ? ctx.near().dht().context().offheap() : ctx.offheap();
+
+                its.add(offheapMgr.<K, V>entriesIterator(modes.primary, modes.backup, topVer, ctx.keepBinary()));
+            }
         }
         else if (modes.heap) {
             if (modes.near && ctx.isNear())
@@ -722,16 +733,6 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
             }
         }
 
-        // Swap and offheap are disabled for near cache.
-        if (modes.primary || modes.backup) {
-            AffinityTopologyVersion topVer = ctx.affinity().affinityTopologyVersion();
-
-            IgniteCacheOffheapManager offheapMgr = ctx.isNear() ? ctx.near().dht().context().offheap() : ctx.offheap();
-
-            if (modes.offheap)
-                its.add(offheapMgr.<K, V>entriesIterator(modes.primary, modes.backup, topVer, ctx.keepBinary()));
-        }
-
         final Iterator<Cache.Entry<K, V>> it = F.flatIterators(its);
 
         return new Iterable<Cache.Entry<K, V>>() {
@@ -928,7 +929,7 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
      * @return Entry (never {@code null}).
      */
     public final GridCacheEntryEx entryEx(Object key) {
-        return entryEx(ctx.toCacheKeyObject(key), false);
+        return entryEx(ctx.toCacheKeyObject(key));
     }
 
     /**
@@ -936,20 +937,7 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
      * @return Entry (never {@code null}).
      */
     public final GridCacheEntryEx entryEx(KeyCacheObject key) {
-        return entryEx(key, false);
-    }
-
-    /**
-     * @param key Entry key.
-     * @param touch Whether created entry should be touched.
-     * @return Entry (never {@code null}).
-     */
-    public GridCacheEntryEx entryEx(KeyCacheObject key, boolean touch) {
-        GridCacheEntryEx e = entry0(key, ctx.affinity().affinityTopologyVersion(), true, touch);
-
-        assert e != null;
-
-        return e;
+        return entryEx(key, ctx.affinity().affinityTopologyVersion());
     }
 
     /**
@@ -958,7 +946,7 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
      * @return Entry (never {@code null}).
      */
     public GridCacheEntryEx entryEx(KeyCacheObject key, AffinityTopologyVersion topVer) {
-        GridCacheEntryEx e = entry0(key, topVer, true, false);
+        GridCacheEntryEx e = map.putEntryIfObsoleteOrAbsent(topVer, key, null, true, false);
 
         assert e != null;
 
@@ -1007,11 +995,11 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
     }
 
     /**
-         * Gets entry set containing internal entries.
-         *
-         * @param filter Filter.
-         * @return Entry set.
-         */
+     * Gets entry set containing internal entries.
+     *
+     * @param filter Filter.
+     * @return Entry set.
+     */
     @Override public final Set<Cache.Entry<K, V>> entrySetx(final CacheEntryPredicate... filter) {
         boolean keepBinary = ctx.keepBinary();
 
@@ -2316,8 +2304,7 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
      */
     public IgniteInternalFuture<V> getAndPutAsync0(final K key,
         final V val,
-        @Nullable final CacheEntryPredicate filter)
-    {
+        @Nullable final CacheEntryPredicate filter) {
         return asyncOp(new AsyncOp<V>() {
             @Override public IgniteInternalFuture<V> op(GridNearTxLocal tx, AffinityTopologyVersion readyTopVer) {
                 return tx.putAsync(ctx, readyTopVer, key, val, true, filter)
@@ -2873,7 +2860,7 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
 
         return syncOp(new SyncOp<V>(true) {
             @Override public V op(GridNearTxLocal tx) throws IgniteCheckedException {
-                K key0 = keepBinary ? (K) ctx.toCacheKeyObject(key) : key;
+                K key0 = keepBinary ? (K)ctx.toCacheKeyObject(key) : key;
 
                 V ret = tx.removeAllAsync(ctx,
                     null,
@@ -2883,9 +2870,9 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
                     /*singleRmv*/false).get().value();
 
                 if (ctx.config().getInterceptor() != null) {
-                    K key = keepBinary ? (K) ctx.unwrapBinaryIfNeeded(key0, true, false) : key0;
+                    K key = keepBinary ? (K)ctx.unwrapBinaryIfNeeded(key0, true, false) : key0;
 
-                    return (V) ctx.config().getInterceptor().onBeforeRemove(new CacheEntryImpl(key, ret)).get2();
+                    return (V)ctx.config().getInterceptor().onBeforeRemove(new CacheEntryImpl(key, ret)).get2();
                 }
 
                 return ret;
@@ -2949,7 +2936,7 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
 
         do {
             for (Iterator<CacheDataRow> it = ctx.offheap().iterator(true, true, null);
-                it.hasNext() && keys.size() < REMOVE_ALL_KEYS_BATCH;)
+                it.hasNext() && keys.size() < REMOVE_ALL_KEYS_BATCH; )
                 keys.add((K)it.next().key());
 
             removeAll(keys);
@@ -3484,7 +3471,7 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
 
         CacheObject cacheVal = ctx.toCacheObject(val);
 
-        GridCacheEntryEx entry = entryEx(key, false);
+        GridCacheEntryEx entry = entryEx(key);
 
         try {
             entry.initialValue(cacheVal,
@@ -3818,7 +3805,7 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
         ctx.kernalContext().task().setThreadContext(TC_SUBGRID, nodes);
 
         return ctx.kernalContext().task().execute(
-                new PartitionSizeLongTask(ctx.name(), ctx.affinity().affinityTopologyVersion(), peekModes, part), null);
+            new PartitionSizeLongTask(ctx.name(), ctx.affinity().affinityTopologyVersion(), peekModes, part), null);
     }
 
     /** {@inheritDoc} */
@@ -3899,7 +3886,8 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
     }
 
     /** {@inheritDoc} */
-    @Override public Iterator<Cache.Entry<K, V>> scanIterator(boolean keepBinary, @Nullable IgniteBiPredicate<Object, Object> p)
+    @Override public Iterator<Cache.Entry<K, V>> scanIterator(boolean keepBinary,
+        @Nullable IgniteBiPredicate<Object, Object> p)
         throws IgniteCheckedException {
         return igniteIterator(keepBinary, p);
     }
@@ -3927,7 +3915,8 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
      * @return Distributed ignite cache iterator.
      * @throws IgniteCheckedException If failed.
      */
-    private Iterator<Cache.Entry<K, V>> igniteIterator(boolean keepBinary, @Nullable IgniteBiPredicate<Object, Object> p)
+    private Iterator<Cache.Entry<K, V>> igniteIterator(boolean keepBinary,
+        @Nullable IgniteBiPredicate<Object, Object> p)
         throws IgniteCheckedException {
         GridCacheContext ctx0 = ctx.isNear() ? ctx.near().dht().context() : ctx;
 
@@ -4471,17 +4460,11 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
 
         GridCacheVersion obsoleteVer = ctx.versions().next();
 
-        if (!ctx.evicts().evictSyncOrNearSync()) {
-            try {
-                ctx.evicts().batchEvict(keys, obsoleteVer);
-            }
-            catch (IgniteCheckedException e) {
-                U.error(log, "Failed to perform batch evict for keys: " + keys, e);
-            }
+        try {
+            ctx.evicts().batchEvict(keys, obsoleteVer);
         }
-        else {
-            for (K k : keys)
-                evictx(k, obsoleteVer, CU.empty0());
+        catch (IgniteCheckedException e) {
+            U.error(log, "Failed to perform batch evict for keys: " + keys, e);
         }
     }
 
@@ -4500,7 +4483,8 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
      * @return Cached value.
      * @throws IgniteCheckedException If failed.
      */
-    @Nullable public final V get(K key, boolean deserializeBinary, final boolean needVer) throws IgniteCheckedException {
+    @Nullable public final V get(K key, boolean deserializeBinary,
+        final boolean needVer) throws IgniteCheckedException {
         String taskName = ctx.kernalContext().job().currentTaskName();
 
         return get0(key, taskName, deserializeBinary, needVer);
@@ -5330,7 +5314,8 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
          * @param peekModes Cache peek modes.
          * @param partition partition.
          */
-        private PartitionSizeLongJob(String cacheName, AffinityTopologyVersion topVer, CachePeekMode[] peekModes, int partition) {
+        private PartitionSizeLongJob(String cacheName, AffinityTopologyVersion topVer, CachePeekMode[] peekModes,
+            int partition) {
             super(cacheName, topVer);
 
             this.peekModes = peekModes;

http://git-wip-us.apache.org/repos/asf/ignite/blob/c56c4b8c/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAttributes.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAttributes.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAttributes.java
index 42a9cfe..292ea4a 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAttributes.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAttributes.java
@@ -205,20 +205,6 @@ public class GridCacheAttributes implements Serializable {
     }
 
     /**
-     * @return Flag indicating whether eviction is synchronized.
-     */
-    public boolean evictSynchronized() {
-        return ccfg.isEvictSynchronized();
-    }
-
-    /**
-     * @return Maximum eviction overflow ratio.
-     */
-    public float evictMaxOverflowRatio() {
-        return ccfg.getEvictMaxOverflowRatio();
-    }
-
-    /**
      * @return Default lock timeout.
      */
     public long defaultLockTimeout() {

http://git-wip-us.apache.org/repos/asf/ignite/blob/c56c4b8c/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheConcurrentMap.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheConcurrentMap.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheConcurrentMap.java
index 7d29d6c..debc65b 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheConcurrentMap.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheConcurrentMap.java
@@ -60,6 +60,7 @@ public interface GridCacheConcurrentMap {
 
     /**
      * Returns the number of key-value mappings in this map.
+     * It does not include entries from underlying data store.
      *
      * @return the number of key-value mappings in this map.
      */
@@ -68,6 +69,7 @@ public interface GridCacheConcurrentMap {
     /**
      * Returns the number of publicly available key-value mappings in this map.
      * It excludes entries that are marked as deleted.
+     * It also does not include entries from underlying data store.
      *
      * @return the number of publicly available key-value mappings in this map.
      */

http://git-wip-us.apache.org/repos/asf/ignite/blob/c56c4b8c/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheConcurrentMapImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheConcurrentMapImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheConcurrentMapImpl.java
index 61624ff..10f5ca3 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheConcurrentMapImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheConcurrentMapImpl.java
@@ -113,8 +113,7 @@ public class GridCacheConcurrentMapImpl implements GridCacheConcurrentMap {
         KeyCacheObject key,
         @Nullable final CacheObject val,
         final boolean create,
-        final boolean touch)
-    {
+        final boolean touch) {
         GridCacheMapEntry cur = null;
         GridCacheMapEntry created = null;
         GridCacheMapEntry created0 = null;